You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@samza.apache.org by bo...@apache.org on 2018/09/26 00:23:00 UTC

[01/29] samza git commit: Cleaning the docs, these create confusion for other reviewers

Repository: samza
Updated Branches:
  refs/heads/NewKafkaSystemConsumer 361596317 -> d1675dde0


Cleaning the docs, these create confusion for other reviewers

bharathkk for review

Author: Sanil Jain <sn...@linkedin.com>

Reviewers: Bharath Kumarasubramanian <bk...@linkedin.com>

Closes #641 from Sanil15/SAMZA-1886


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

Branch: refs/heads/NewKafkaSystemConsumer
Commit: b7219e976c74a52fffc4864548249ef7101cd584
Parents: 40f7430
Author: Sanil Jain <sn...@linkedin.com>
Authored: Wed Sep 19 10:08:20 2018 -0700
Committer: Prateek Maheshwari <pm...@apache.org>
Committed: Wed Sep 19 10:08:20 2018 -0700

----------------------------------------------------------------------
 .../java/org/apache/samza/system/inmemory/InMemoryManager.java | 6 +-----
 .../org/apache/samza/system/inmemory/InMemorySystemAdmin.java  | 4 ----
 .../apache/samza/system/inmemory/InMemorySystemConsumer.java   | 4 ----
 .../apache/samza/system/inmemory/InMemorySystemFactory.java    | 4 ----
 .../apache/samza/system/inmemory/InMemorySystemProducer.java   | 4 ----
 5 files changed, 1 insertion(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/samza/blob/b7219e97/samza-core/src/main/java/org/apache/samza/system/inmemory/InMemoryManager.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/system/inmemory/InMemoryManager.java b/samza-core/src/main/java/org/apache/samza/system/inmemory/InMemoryManager.java
index a704bc4..8463e56 100644
--- a/samza-core/src/main/java/org/apache/samza/system/inmemory/InMemoryManager.java
+++ b/samza-core/src/main/java/org/apache/samza/system/inmemory/InMemoryManager.java
@@ -37,10 +37,6 @@ import org.apache.samza.system.SystemStreamPartition;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-
-/**
- * Initial draft of in-memory manager. It is test only and not meant for production use right now.
- */
 class InMemoryManager {
   private static final Logger LOG = LoggerFactory.getLogger(InMemoryManager.class);
   private static final int DEFAULT_PARTITION_COUNT = 1;
@@ -125,7 +121,7 @@ class InMemoryManager {
     Map<String, Map<SystemStreamPartition, List<IncomingMessageEnvelope>>> result =
         bufferedMessages.entrySet()
             .stream()
-            .filter(entry -> systemName.equals(entry.getKey().getSystem()) 
+            .filter(entry -> systemName.equals(entry.getKey().getSystem())
                 && streamNames.contains(entry.getKey().getStream()))
             .collect(Collectors.groupingBy(entry -> entry.getKey().getStream(),
                 Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)));

http://git-wip-us.apache.org/repos/asf/samza/blob/b7219e97/samza-core/src/main/java/org/apache/samza/system/inmemory/InMemorySystemAdmin.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/system/inmemory/InMemorySystemAdmin.java b/samza-core/src/main/java/org/apache/samza/system/inmemory/InMemorySystemAdmin.java
index 65f45a0..8e5f732 100644
--- a/samza-core/src/main/java/org/apache/samza/system/inmemory/InMemorySystemAdmin.java
+++ b/samza-core/src/main/java/org/apache/samza/system/inmemory/InMemorySystemAdmin.java
@@ -28,10 +28,6 @@ import org.apache.samza.system.SystemAdmin;
 import org.apache.samza.system.SystemStreamMetadata;
 import org.apache.samza.system.SystemStreamPartition;
 
-
-/**
- * Initial draft of in-memory {@link SystemAdmin}. It is test only and not meant for production use right now.
- */
 public class InMemorySystemAdmin implements SystemAdmin {
   private final InMemoryManager inMemoryManager;
   private final String systemName;

http://git-wip-us.apache.org/repos/asf/samza/blob/b7219e97/samza-core/src/main/java/org/apache/samza/system/inmemory/InMemorySystemConsumer.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/system/inmemory/InMemorySystemConsumer.java b/samza-core/src/main/java/org/apache/samza/system/inmemory/InMemorySystemConsumer.java
index 08540ae..dcab001 100644
--- a/samza-core/src/main/java/org/apache/samza/system/inmemory/InMemorySystemConsumer.java
+++ b/samza-core/src/main/java/org/apache/samza/system/inmemory/InMemorySystemConsumer.java
@@ -30,10 +30,6 @@ import org.apache.samza.system.SystemStreamPartition;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-
-/**
- * Initial draft of in-memory {@link SystemConsumer}. It is test only and not meant for production use right now.
- */
 public class InMemorySystemConsumer implements SystemConsumer {
   private static final Logger LOG = LoggerFactory.getLogger(InMemorySystemConsumer.class);
 

http://git-wip-us.apache.org/repos/asf/samza/blob/b7219e97/samza-core/src/main/java/org/apache/samza/system/inmemory/InMemorySystemFactory.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/system/inmemory/InMemorySystemFactory.java b/samza-core/src/main/java/org/apache/samza/system/inmemory/InMemorySystemFactory.java
index bb41a4e..6782c95 100644
--- a/samza-core/src/main/java/org/apache/samza/system/inmemory/InMemorySystemFactory.java
+++ b/samza-core/src/main/java/org/apache/samza/system/inmemory/InMemorySystemFactory.java
@@ -28,10 +28,6 @@ import org.apache.samza.system.SystemConsumer;
 import org.apache.samza.system.SystemFactory;
 import org.apache.samza.system.SystemProducer;
 
-
-/**
- * Initial draft of in-memory {@link SystemFactory}. It is test only and not meant for production use right now.
- */
 public class InMemorySystemFactory implements SystemFactory {
   private static final ConcurrentHashMap<String, InMemoryManager> IN_MEMORY_MANAGERS = new ConcurrentHashMap<>();
 

http://git-wip-us.apache.org/repos/asf/samza/blob/b7219e97/samza-core/src/main/java/org/apache/samza/system/inmemory/InMemorySystemProducer.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/system/inmemory/InMemorySystemProducer.java b/samza-core/src/main/java/org/apache/samza/system/inmemory/InMemorySystemProducer.java
index 052e72b..cd5e649 100644
--- a/samza-core/src/main/java/org/apache/samza/system/inmemory/InMemorySystemProducer.java
+++ b/samza-core/src/main/java/org/apache/samza/system/inmemory/InMemorySystemProducer.java
@@ -27,10 +27,6 @@ import org.apache.samza.system.SystemStreamPartition;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-
-/**
- * Initial draft of in-memory {@link SystemProducer}. It is test only and not meant for production use right now.
- */
 public class InMemorySystemProducer implements SystemProducer {
   private static final Logger LOG = LoggerFactory.getLogger(InMemorySystemProducer.class);
   private final InMemoryManager memoryManager;


[02/29] samza git commit: SAMZA-1874: Refactor SamzaContainer and TaskInstance unit tests to make shared context changes easier

Posted by bo...@apache.org.
SAMZA-1874: Refactor SamzaContainer and TaskInstance unit tests to make shared context changes easier

This replaces https://github.com/apache/samza/pull/638, I accidentally messed up that branch.
The difference between this PR and the last review by prateekm is https://github.com/apache/samza/pull/646/commits/5d552996ac50d2a0b1dd5034a624d9417e74dc57

Author: Cameron Lee <ca...@linkedin.com>

Reviewers: Prateek Maheshwari <pm...@apache.org>

Closes #646 from cameronlee314/refactor_unit_tests_for_shared_context_new


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

Branch: refs/heads/NewKafkaSystemConsumer
Commit: 19c6f4f6131ce86aa492073720f7168197a1f103
Parents: b7219e9
Author: Cameron Lee <ca...@linkedin.com>
Authored: Wed Sep 19 10:21:57 2018 -0700
Committer: Prateek Maheshwari <pm...@apache.org>
Committed: Wed Sep 19 10:21:57 2018 -0700

----------------------------------------------------------------------
 .../apache/samza/job/model/TestJobModel.java    |  50 ++
 .../samza/container/TestSamzaContainer.scala    | 729 ++++---------------
 .../samza/container/TestTaskInstance.scala      | 526 ++++---------
 .../TestTaskInstanceExceptionHandler.scala      | 144 ++++
 .../samza/system/chooser/MockSystemAdmin.scala  |  30 +
 .../chooser/TestBootstrappingChooser.scala      |   3 +-
 .../system/chooser/TestDefaultChooser.scala     |   1 -
 7 files changed, 528 insertions(+), 955 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/samza/blob/19c6f4f6/samza-core/src/test/java/org/apache/samza/job/model/TestJobModel.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/job/model/TestJobModel.java b/samza-core/src/test/java/org/apache/samza/job/model/TestJobModel.java
new file mode 100644
index 0000000..6c7c282
--- /dev/null
+++ b/samza-core/src/test/java/org/apache/samza/job/model/TestJobModel.java
@@ -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.job.model;
+
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableSet;
+import java.util.Map;
+import org.apache.samza.Partition;
+import org.apache.samza.config.Config;
+import org.apache.samza.config.MapConfig;
+import org.apache.samza.container.TaskName;
+import org.junit.Test;
+
+import static org.junit.Assert.*;
+
+
+public class TestJobModel {
+  @Test
+  public void testMaxChangeLogStreamPartitions() {
+    Config config = new MapConfig(ImmutableMap.of("a", "b"));
+    Map<TaskName, TaskModel> tasksForContainer1 = ImmutableMap.of(
+        new TaskName("t1"), new TaskModel(new TaskName("t1"), ImmutableSet.of(), new Partition(0)),
+        new TaskName("t2"), new TaskModel(new TaskName("t2"), ImmutableSet.of(), new Partition(1)));
+    Map<TaskName, TaskModel> tasksForContainer2 = ImmutableMap.of(
+        new TaskName("t3"), new TaskModel(new TaskName("t3"), ImmutableSet.of(), new Partition(2)),
+        new TaskName("t4"), new TaskModel(new TaskName("t4"), ImmutableSet.of(), new Partition(3)),
+        new TaskName("t5"), new TaskModel(new TaskName("t5"), ImmutableSet.of(), new Partition(4)));
+    ContainerModel containerModel1 = new ContainerModel("0", 0, tasksForContainer1);
+    ContainerModel containerModel2 = new ContainerModel("1", 1, tasksForContainer2);
+    Map<String, ContainerModel> containers = ImmutableMap.of("0", containerModel1, "1", containerModel2);
+    JobModel jobModel = new JobModel(config, containers);
+    assertEquals(jobModel.maxChangeLogStreamPartitions, 5);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/samza/blob/19c6f4f6/samza-core/src/test/scala/org/apache/samza/container/TestSamzaContainer.scala
----------------------------------------------------------------------
diff --git a/samza-core/src/test/scala/org/apache/samza/container/TestSamzaContainer.scala b/samza-core/src/test/scala/org/apache/samza/container/TestSamzaContainer.scala
index ff57047..30ca8c1 100644
--- a/samza-core/src/test/scala/org/apache/samza/container/TestSamzaContainer.scala
+++ b/samza-core/src/test/scala/org/apache/samza/container/TestSamzaContainer.scala
@@ -20,27 +20,22 @@
 package org.apache.samza.container
 
 import java.util
-import java.util.concurrent.TimeUnit
 import java.util.concurrent.atomic.AtomicReference
 
-import org.apache.samza.checkpoint.{Checkpoint, CheckpointManager}
-import org.apache.samza.config.{Config, MapConfig}
+import org.apache.samza.config.MapConfig
 import org.apache.samza.coordinator.JobModelManager
 import org.apache.samza.coordinator.server.{HttpServer, JobServlet}
 import org.apache.samza.job.model.{ContainerModel, JobModel, TaskModel}
-import org.apache.samza.metrics.MetricsRegistryMap
-import org.apache.samza.serializers.SerdeManager
-import org.apache.samza.storage.TaskStorageManager
+import org.apache.samza.metrics.{Gauge, Timer}
 import org.apache.samza.system._
-import org.apache.samza.system.chooser.RoundRobinChooser
-import org.apache.samza.task._
-import org.apache.samza.util.SinglePartitionWithoutOffsetsSystemAdmin
 import org.apache.samza.{Partition, SamzaContainerStatus}
 import org.junit.Assert._
-import org.junit.Test
-import org.mockito.Mockito.when
+import org.junit.{Before, Test}
+import org.mockito.Matchers.{any, notNull}
+import org.mockito.Mockito._
 import org.mockito.invocation.InvocationOnMock
 import org.mockito.stubbing.Answer
+import org.mockito.{ArgumentCaptor, Mock, MockitoAnnotations}
 import org.scalatest.junit.AssertionsForJUnit
 import org.scalatest.mockito.MockitoSugar
 
@@ -48,8 +43,137 @@ import scala.collection.JavaConversions._
 import scala.collection.JavaConverters._
 
 class TestSamzaContainer extends AssertionsForJUnit with MockitoSugar {
+  private val TASK_NAME = new TaskName("taskName")
+
+  @Mock
+  private var containerContext: SamzaContainerContext = null
+  @Mock
+  private var taskInstance: TaskInstance = null
+  @Mock
+  private var runLoop: Runnable = null
+  @Mock
+  private var systemAdmins: SystemAdmins = null
+  @Mock
+  private var consumerMultiplexer: SystemConsumers = null
+  @Mock
+  private var producerMultiplexer: SystemProducers = null
+  @Mock
+  private var metrics: SamzaContainerMetrics = null
+  @Mock
+  private var samzaContainerListener: SamzaContainerListener = null
+
+  private var samzaContainer: SamzaContainer = null
+
+  @Before
+  def setup(): Unit = {
+    MockitoAnnotations.initMocks(this)
+    this.samzaContainer = new SamzaContainer(
+      this.containerContext,
+      Map(TASK_NAME -> this.taskInstance),
+      this.runLoop,
+      this.systemAdmins,
+      this.consumerMultiplexer,
+      this.producerMultiplexer,
+      metrics)
+    this.samzaContainer.setContainerListener(this.samzaContainerListener)
+    when(this.metrics.containerStartupTime).thenReturn(mock[Timer])
+  }
+
+  @Test
+  def testExceptionInTaskInitShutsDownTask() {
+    when(this.taskInstance.initTask).thenThrow(new RuntimeException("Trigger a shutdown, please."))
+
+    this.samzaContainer.run
+
+    verify(this.taskInstance).shutdownTask
+    assertEquals(SamzaContainerStatus.FAILED, this.samzaContainer.getStatus())
+    verify(this.samzaContainerListener).beforeStart()
+    verify(this.samzaContainerListener, never()).afterStart()
+    verify(this.samzaContainerListener, never()).afterStop()
+    verify(this.samzaContainerListener).afterFailure(notNull(classOf[Exception]))
+    verifyZeroInteractions(this.runLoop)
+  }
+
+  @Test
+  def testErrorInTaskInitShutsDownTask(): Unit = {
+    when(this.taskInstance.initTask).thenThrow(new NoSuchMethodError("Trigger a shutdown, please."))
+
+    this.samzaContainer.run
+
+    verify(this.taskInstance).shutdownTask
+    assertEquals(SamzaContainerStatus.FAILED, this.samzaContainer.getStatus())
+    verify(this.samzaContainerListener).beforeStart()
+    verify(this.samzaContainerListener, never()).afterStart()
+    verify(this.samzaContainerListener, never()).afterStop()
+    verify(this.samzaContainerListener).afterFailure(notNull(classOf[Exception]))
+    verifyZeroInteractions(this.runLoop)
+  }
+
+  @Test
+  def testExceptionInTaskProcessRunLoop() {
+    when(this.runLoop.run()).thenThrow(new RuntimeException("Trigger a shutdown, please."))
+
+    this.samzaContainer.run
+
+    verify(this.taskInstance).shutdownTask
+    assertEquals(SamzaContainerStatus.FAILED, this.samzaContainer.getStatus())
+    verify(this.samzaContainerListener).beforeStart()
+    verify(this.samzaContainerListener).afterStart()
+    verify(this.samzaContainerListener, never()).afterStop()
+    verify(this.samzaContainerListener).afterFailure(notNull(classOf[Exception]))
+    verify(this.runLoop).run()
+  }
+
+  @Test
+  def testCleanRun(): Unit = {
+    doNothing().when(this.runLoop).run() // run loop completes successfully
+
+    this.samzaContainer.run
+
+    verify(this.taskInstance).shutdownTask
+    assertEquals(SamzaContainerStatus.STOPPED, this.samzaContainer.getStatus())
+    verify(this.samzaContainerListener).beforeStart()
+    verify(this.samzaContainerListener).afterStart()
+    verify(this.samzaContainerListener).afterStop()
+    verify(this.samzaContainerListener, never()).afterFailure(any())
+    verify(this.runLoop).run()
+  }
+
   @Test
-  def testReadJobModel {
+  def testFailureDuringShutdown(): Unit = {
+    doNothing().when(this.runLoop).run() // run loop completes successfully
+    when(this.taskInstance.shutdownTask).thenThrow(new RuntimeException("Trigger a shutdown, please."))
+
+    this.samzaContainer.run
+
+    verify(this.taskInstance).shutdownTask
+    assertEquals(SamzaContainerStatus.FAILED, this.samzaContainer.getStatus())
+    verify(this.samzaContainerListener).beforeStart()
+    verify(this.samzaContainerListener).afterStart()
+    verify(this.samzaContainerListener, never()).afterStop()
+    verify(this.samzaContainerListener).afterFailure(notNull(classOf[Exception]))
+    verify(this.runLoop).run()
+  }
+
+  @Test
+  def testStartStoresIncrementsCounter() {
+    when(this.taskInstance.taskName).thenReturn(TASK_NAME)
+    val restoreGauge = mock[Gauge[Long]]
+    when(this.metrics.taskStoreRestorationMetrics).thenReturn(Map(TASK_NAME -> restoreGauge))
+    when(this.taskInstance.startStores).thenAnswer(new Answer[Void] {
+      override def answer(invocation: InvocationOnMock): Void = {
+        Thread.sleep(1)
+        null
+      }
+    })
+    this.samzaContainer.startStores
+    val restoreGaugeValueCaptor = ArgumentCaptor.forClass(classOf[Long])
+    verify(restoreGauge).set(restoreGaugeValueCaptor.capture())
+    assertTrue(restoreGaugeValueCaptor.getValue >= 1)
+  }
+
+  @Test
+  def testReadJobModel() {
     val config = new MapConfig(Map("a" -> "b").asJava)
     val offsets = new util.HashMap[SystemStreamPartition, String]()
     offsets.put(new SystemStreamPartition("system","stream", new Partition(0)), "1")
@@ -74,7 +198,7 @@ class TestSamzaContainer extends AssertionsForJUnit with MockitoSugar {
   }
 
   @Test
-  def testReadJobModelWithTimeouts {
+  def testReadJobModelWithTimeouts() {
     val config = new MapConfig(Map("a" -> "b").asJava)
     val offsets = new util.HashMap[SystemStreamPartition, String]()
     offsets.put(new SystemStreamPartition("system","stream", new Partition(0)), "1")
@@ -101,551 +225,7 @@ class TestSamzaContainer extends AssertionsForJUnit with MockitoSugar {
   }
 
   @Test
-  def testChangelogPartitions {
-    val config = new MapConfig(Map("a" -> "b").asJava)
-    val offsets = new util.HashMap[SystemStreamPartition, String]()
-    offsets.put(new SystemStreamPartition("system", "stream", new Partition(0)), "1")
-    val tasksForContainer1 = Map(
-      new TaskName("t1") -> new TaskModel(new TaskName("t1"), offsets.keySet(), new Partition(0)),
-      new TaskName("t2") -> new TaskModel(new TaskName("t2"), offsets.keySet(), new Partition(1)))
-    val tasksForContainer2 = Map(
-      new TaskName("t3") -> new TaskModel(new TaskName("t3"), offsets.keySet(), new Partition(2)),
-      new TaskName("t4") -> new TaskModel(new TaskName("t4"), offsets.keySet(), new Partition(3)),
-      new TaskName("t5") -> new TaskModel(new TaskName("t6"), offsets.keySet(), new Partition(4)))
-    val containerModel1 = new ContainerModel("0", 0, tasksForContainer1)
-    val containerModel2 = new ContainerModel("1", 1, tasksForContainer2)
-    val containers = Map(
-      "0" -> containerModel1,
-      "1" -> containerModel2)
-    val jobModel = new JobModel(config, containers)
-    assertEquals(jobModel.maxChangeLogStreamPartitions, 5)
-  }
-
-  @Test
-  def testGetInputStreamMetadata {
-    val inputStreams = Set(
-      new SystemStreamPartition("test", "stream1", new Partition(0)),
-      new SystemStreamPartition("test", "stream1", new Partition(1)),
-      new SystemStreamPartition("test", "stream2", new Partition(0)),
-      new SystemStreamPartition("test", "stream2", new Partition(1)))
-    val systemAdmins = mock[SystemAdmins]
-    when(systemAdmins.getSystemAdmin("test")).thenReturn(new SinglePartitionWithoutOffsetsSystemAdmin)
-    val metadata = new StreamMetadataCache(systemAdmins).getStreamMetadata(inputStreams.map(_.getSystemStream))
-    assertNotNull(metadata)
-    assertEquals(2, metadata.size)
-    val stream1Metadata = metadata(new SystemStream("test", "stream1"))
-    val stream2Metadata = metadata(new SystemStream("test", "stream2"))
-    assertNotNull(stream1Metadata)
-    assertNotNull(stream2Metadata)
-    assertEquals("stream1", stream1Metadata.getStreamName)
-    assertEquals("stream2", stream2Metadata.getStreamName)
-  }
-
-  @Test
-  def testExceptionInTaskInitShutsDownTask {
-    val task = new StreamTask with InitableTask with ClosableTask {
-      var wasShutdown = false
-
-      def init(config: Config, context: TaskContext) {
-        throw new Exception("Trigger a shutdown, please.")
-      }
-
-      def process(envelope: IncomingMessageEnvelope, collector: MessageCollector, coordinator: TaskCoordinator) {
-      }
-
-      def close {
-        wasShutdown = true
-      }
-    }
-    val config = new MapConfig
-    val taskName = new TaskName("taskName")
-    val systemAdmins = new SystemAdmins(config)
-    val consumerMultiplexer = new SystemConsumers(
-      new RoundRobinChooser,
-      Map[String, SystemConsumer]())
-    val producerMultiplexer = new SystemProducers(
-      Map[String, SystemProducer](),
-      new SerdeManager)
-    val collector = new TaskInstanceCollector(producerMultiplexer)
-    val containerContext = new SamzaContainerContext("0", config, Set[TaskName](taskName), new MetricsRegistryMap)
-    val taskInstance: TaskInstance = new TaskInstance(
-      task,
-      taskName,
-      config,
-      new TaskInstanceMetrics,
-      null,
-      consumerMultiplexer,
-      collector,
-      containerContext
-    )
-    val runLoop = new RunLoop(
-      taskInstances = Map(taskName -> taskInstance),
-      consumerMultiplexer = consumerMultiplexer,
-      metrics = new SamzaContainerMetrics,
-      maxThrottlingDelayMs = TimeUnit.SECONDS.toMillis(1))
-    @volatile var onContainerFailedCalled = false
-    @volatile var onContainerStopCalled = false
-    @volatile var onContainerStartCalled = false
-    @volatile var onContainerFailedThrowable: Throwable = null
-    @volatile var onContainerBeforeStartCalled = false
-
-    val container = new SamzaContainer(
-      containerContext = containerContext,
-      taskInstances = Map(taskName -> taskInstance),
-      runLoop = runLoop,
-      systemAdmins = systemAdmins,
-      consumerMultiplexer = consumerMultiplexer,
-      producerMultiplexer = producerMultiplexer,
-      metrics = new SamzaContainerMetrics)
-
-    val containerListener = new SamzaContainerListener {
-      override def afterFailure(t: Throwable): Unit = {
-        onContainerFailedCalled = true
-        onContainerFailedThrowable = t
-      }
-
-      override def afterStop(): Unit = {
-        onContainerStopCalled = true
-      }
-
-      override def afterStart(): Unit = {
-        onContainerStartCalled = true
-      }
-
-      override def beforeStart(): Unit = {
-        onContainerBeforeStartCalled = true
-      }
-
-    }
-    container.setContainerListener(containerListener)
-
-    container.run
-    assertTrue(task.wasShutdown)
-    assertTrue(onContainerBeforeStartCalled)
-    assertFalse(onContainerStartCalled)
-    assertFalse(onContainerStopCalled)
-
-    assertTrue(onContainerFailedCalled)
-    assertNotNull(onContainerFailedThrowable)
-  }
-
-  // Exception in Runloop should cause SamzaContainer to transition to FAILED status, shutdown the components and then,
-  // invoke the callback
-  @Test
-  def testExceptionInTaskProcessRunLoop() {
-    val task = new StreamTask with InitableTask with ClosableTask {
-      var wasShutdown = false
-
-      def init(config: Config, context: TaskContext) {
-      }
-
-      def process(envelope: IncomingMessageEnvelope, collector: MessageCollector, coordinator: TaskCoordinator) {
-        throw new Exception("Trigger a shutdown, please.")
-      }
-
-      def close {
-        wasShutdown = true
-      }
-    }
-    val config = new MapConfig
-    val taskName = new TaskName("taskName")
-    val systemAdmins = new SystemAdmins(config)
-    val consumerMultiplexer = new SystemConsumers(
-      new RoundRobinChooser,
-      Map[String, SystemConsumer]())
-    val producerMultiplexer = new SystemProducers(
-      Map[String, SystemProducer](),
-      new SerdeManager)
-    val collector = new TaskInstanceCollector(producerMultiplexer)
-    val containerContext = new SamzaContainerContext("0", config, Set[TaskName](taskName), new MetricsRegistryMap)
-    val taskInstance: TaskInstance = new TaskInstance(
-      task,
-      taskName,
-      config,
-      new TaskInstanceMetrics,
-      null,
-      consumerMultiplexer,
-      collector,
-      containerContext
-    )
-
-    @volatile var onContainerFailedCalled = false
-    @volatile var onContainerStopCalled = false
-    @volatile var onContainerStartCalled = false
-    @volatile var onContainerFailedThrowable: Throwable = null
-    @volatile var onContainerBeforeStartCalled = false
-
-    val mockRunLoop = mock[RunLoop]
-    when(mockRunLoop.run).thenThrow(new RuntimeException("Trigger a shutdown, please."))
-
-    val container = new SamzaContainer(
-      containerContext = containerContext,
-      taskInstances = Map(taskName -> taskInstance),
-      runLoop = mockRunLoop,
-      systemAdmins = systemAdmins,
-      consumerMultiplexer = consumerMultiplexer,
-      producerMultiplexer = producerMultiplexer,
-      metrics = new SamzaContainerMetrics)
-    val containerListener = new SamzaContainerListener {
-      override def afterFailure(t: Throwable): Unit = {
-        onContainerFailedCalled = true
-        onContainerFailedThrowable = t
-      }
-
-      override def afterStop(): Unit = {
-        onContainerStopCalled = true
-      }
-
-      override def afterStart(): Unit = {
-        onContainerStartCalled = true
-      }
-
-      /**
-        * Method invoked before the {@link org.apache.samza.container.SamzaContainer} is started
-        */
-      override def beforeStart(): Unit = {
-        onContainerBeforeStartCalled = true
-      }
-    }
-    container.setContainerListener(containerListener)
-
-    container.run
-    assertTrue(task.wasShutdown)
-    assertTrue(onContainerBeforeStartCalled)
-    assertTrue(onContainerStartCalled)
-
-    assertFalse(onContainerStopCalled)
-
-    assertTrue(onContainerFailedCalled)
-    assertNotNull(onContainerFailedThrowable)
-
-    assertEquals(SamzaContainerStatus.FAILED, container.getStatus())
-  }
-
-  @Test
-  def testErrorInTaskInitShutsDownTask() {
-    val task = new StreamTask with InitableTask with ClosableTask {
-      var wasShutdown = false
-
-      def init(config: Config, context: TaskContext) {
-        throw new NoSuchMethodError("Trigger a shutdown, please.")
-      }
-
-      def process(envelope: IncomingMessageEnvelope, collector: MessageCollector, coordinator: TaskCoordinator) {
-      }
-
-      def close {
-        wasShutdown = true
-      }
-    }
-    val config = new MapConfig
-    val taskName = new TaskName("taskName")
-    val systemAdmins = new SystemAdmins(config)
-    val consumerMultiplexer = new SystemConsumers(
-      new RoundRobinChooser,
-      Map[String, SystemConsumer]())
-    val producerMultiplexer = new SystemProducers(
-      Map[String, SystemProducer](),
-      new SerdeManager)
-    val collector = new TaskInstanceCollector(producerMultiplexer)
-    val containerContext = new SamzaContainerContext("0", config, Set[TaskName](taskName), new MetricsRegistryMap)
-    val taskInstance: TaskInstance = new TaskInstance(
-      task,
-      taskName,
-      config,
-      new TaskInstanceMetrics,
-      null,
-      consumerMultiplexer,
-      collector,
-      containerContext
-    )
-    val runLoop = new RunLoop(
-      taskInstances = Map(taskName -> taskInstance),
-      consumerMultiplexer = consumerMultiplexer,
-      metrics = new SamzaContainerMetrics,
-      maxThrottlingDelayMs = TimeUnit.SECONDS.toMillis(1))
-    @volatile var onContainerFailedCalled = false
-    @volatile var onContainerStopCalled = false
-    @volatile var onContainerStartCalled = false
-    @volatile var onContainerFailedThrowable: Throwable = null
-    @volatile var onContainerBeforeStartCalled = false
-
-    val container = new SamzaContainer(
-      containerContext = containerContext,
-      taskInstances = Map(taskName -> taskInstance),
-      runLoop = runLoop,
-      systemAdmins = systemAdmins,
-      consumerMultiplexer = consumerMultiplexer,
-      producerMultiplexer = producerMultiplexer,
-      metrics = new SamzaContainerMetrics)
-    val containerListener = new SamzaContainerListener {
-      override def afterFailure(t: Throwable): Unit = {
-        onContainerFailedCalled = true
-        onContainerFailedThrowable = t
-      }
-
-      override def afterStop(): Unit = {
-        onContainerStopCalled = true
-      }
-
-      override def afterStart(): Unit = {
-        onContainerStartCalled = true
-      }
-
-      /**
-        * Method invoked before the {@link org.apache.samza.container.SamzaContainer} is started
-        */
-      override def beforeStart(): Unit = {
-        onContainerBeforeStartCalled = true
-      }
-    }
-    container.setContainerListener(containerListener)
-
-    container.run
-
-    assertTrue(task.wasShutdown)
-    assertTrue(onContainerBeforeStartCalled)
-    assertFalse(onContainerStopCalled)
-    assertFalse(onContainerStartCalled)
-
-    assertTrue(onContainerFailedCalled)
-    assertNotNull(onContainerFailedThrowable)
-  }
-
-  @Test
-  def testRunloopShutdownIsClean(): Unit = {
-    val task = new StreamTask with InitableTask with ClosableTask {
-      var wasShutdown = false
-
-      def init(config: Config, context: TaskContext) {
-      }
-
-      def process(envelope: IncomingMessageEnvelope, collector: MessageCollector, coordinator: TaskCoordinator) {
-      }
-
-      def close {
-        wasShutdown = true
-      }
-    }
-    val config = new MapConfig
-    val taskName = new TaskName("taskName")
-    val systemAdmins = new SystemAdmins(config)
-    val consumerMultiplexer = new SystemConsumers(
-      new RoundRobinChooser,
-      Map[String, SystemConsumer]())
-    val producerMultiplexer = new SystemProducers(
-      Map[String, SystemProducer](),
-      new SerdeManager)
-    val collector = new TaskInstanceCollector(producerMultiplexer)
-    val containerContext = new SamzaContainerContext("0", config, Set[TaskName](taskName), new MetricsRegistryMap)
-    val taskInstance: TaskInstance = new TaskInstance(
-      task,
-      taskName,
-      config,
-      new TaskInstanceMetrics,
-      null,
-      consumerMultiplexer,
-      collector,
-      containerContext
-    )
-
-    @volatile var onContainerFailedCalled = false
-    @volatile var onContainerStopCalled = false
-    @volatile var onContainerStartCalled = false
-    @volatile var onContainerFailedThrowable: Throwable = null
-    @volatile var onContainerBeforeStartCalled = false
-
-    val mockRunLoop = mock[RunLoop]
-    when(mockRunLoop.run).thenAnswer(new Answer[Unit] {
-      override def answer(invocation: InvocationOnMock): Unit = {
-        Thread.sleep(100)
-      }
-    })
-
-    val container = new SamzaContainer(
-      containerContext = containerContext,
-      taskInstances = Map(taskName -> taskInstance),
-      runLoop = mockRunLoop,
-      systemAdmins = systemAdmins,
-      consumerMultiplexer = consumerMultiplexer,
-      producerMultiplexer = producerMultiplexer,
-      metrics = new SamzaContainerMetrics)
-      val containerListener = new SamzaContainerListener {
-        override def afterFailure(t: Throwable): Unit = {
-          onContainerFailedCalled = true
-          onContainerFailedThrowable = t
-        }
-
-        override def afterStop(): Unit = {
-          onContainerStopCalled = true
-        }
-
-        override def afterStart(): Unit = {
-          onContainerStartCalled = true
-        }
-
-        /**
-          * Method invoked before the {@link org.apache.samza.container.SamzaContainer} is started
-          */
-        override def beforeStart(): Unit = {
-          onContainerBeforeStartCalled = true
-        }
-      }
-    container.setContainerListener(containerListener)
-
-    container.run
-    assertTrue(onContainerBeforeStartCalled)
-    assertFalse(onContainerFailedCalled)
-    assertTrue(onContainerStartCalled)
-    assertTrue(onContainerStopCalled)
-  }
-
-  @Test
-  def testFailureDuringShutdown: Unit = {
-    val task = new StreamTask with InitableTask with ClosableTask {
-      def init(config: Config, context: TaskContext) {
-      }
-
-      def process(envelope: IncomingMessageEnvelope, collector: MessageCollector, coordinator: TaskCoordinator) {
-
-      }
-
-      def close {
-        throw new Exception("Exception during shutdown, please.")
-      }
-    }
-    val config = new MapConfig
-    val taskName = new TaskName("taskName")
-    val systemAdmins = new SystemAdmins(config)
-    val consumerMultiplexer = new SystemConsumers(
-      new RoundRobinChooser,
-      Map[String, SystemConsumer]())
-    val producerMultiplexer = new SystemProducers(
-      Map[String, SystemProducer](),
-      new SerdeManager)
-    val collector = new TaskInstanceCollector(producerMultiplexer)
-    val containerContext = new SamzaContainerContext("0", config, Set[TaskName](taskName), new MetricsRegistryMap)
-    val taskInstance: TaskInstance = new TaskInstance(
-      task,
-      taskName,
-      config,
-      new TaskInstanceMetrics,
-      null,
-      consumerMultiplexer,
-      collector,
-      containerContext
-    )
-
-    @volatile var onContainerFailedCalled = false
-    @volatile var onContainerStopCalled = false
-    @volatile var onContainerStartCalled = false
-    @volatile var onContainerFailedThrowable: Throwable = null
-    @volatile var onContainerBeforeStartCalled = false
-
-    val mockRunLoop = mock[RunLoop]
-    when(mockRunLoop.run).thenAnswer(new Answer[Unit] {
-      override def answer(invocation: InvocationOnMock): Unit = {
-        Thread.sleep(100)
-      }
-    })
-
-    val container = new SamzaContainer(
-      containerContext = containerContext,
-      taskInstances = Map(taskName -> taskInstance),
-      runLoop = mockRunLoop,
-      systemAdmins = systemAdmins,
-      consumerMultiplexer = consumerMultiplexer,
-      producerMultiplexer = producerMultiplexer,
-      metrics = new SamzaContainerMetrics)
-
-    val containerListener = new SamzaContainerListener {
-        override def afterFailure(t: Throwable): Unit = {
-          onContainerFailedCalled = true
-          onContainerFailedThrowable = t
-        }
-
-        override def afterStop(): Unit = {
-          onContainerStopCalled = true
-        }
-
-        override def afterStart(): Unit = {
-          onContainerStartCalled = true
-        }
-
-      /**
-        * Method invoked before the {@link org.apache.samza.container.SamzaContainer} is started
-        */
-      override def beforeStart(): Unit = {
-        onContainerBeforeStartCalled = true
-      }
-    }
-    container.setContainerListener(containerListener)
-
-    container.run
-
-    assertTrue(onContainerBeforeStartCalled)
-    assertTrue(onContainerStartCalled)
-    assertTrue(onContainerFailedCalled)
-    assertFalse(onContainerStopCalled)
-  }
-
-  @Test
-  def testStartStoresIncrementsCounter {
-    val task = new StreamTask {
-      def process(envelope: IncomingMessageEnvelope, collector: MessageCollector, coordinator: TaskCoordinator) {
-      }
-    }
-    val config = new MapConfig
-    val taskName = new TaskName("taskName")
-    val systemAdmins = new SystemAdmins(config)
-    val consumerMultiplexer = new SystemConsumers(
-      new RoundRobinChooser,
-      Map[String, SystemConsumer]())
-    val producerMultiplexer = new SystemProducers(
-      Map[String, SystemProducer](),
-      new SerdeManager)
-    val collector = new TaskInstanceCollector(producerMultiplexer)
-    val containerContext = new SamzaContainerContext("0", config, Set[TaskName](taskName), new MetricsRegistryMap)
-    val mockTaskStorageManager = mock[TaskStorageManager]
-
-    when(mockTaskStorageManager.init).thenAnswer(new Answer[String] {
-      override def answer(invocation: InvocationOnMock): String = {
-        Thread.sleep(1)
-        ""
-      }
-    })
-
-    val taskInstance: TaskInstance = new TaskInstance(
-      task,
-      taskName,
-      config,
-      new TaskInstanceMetrics,
-      null,
-      consumerMultiplexer,
-      collector,
-      containerContext,
-      storageManager = mockTaskStorageManager
-    )
-    val containerMetrics = new SamzaContainerMetrics()
-    containerMetrics.addStoreRestorationGauge(taskName, "store")
-    val container = new SamzaContainer(
-      containerContext = containerContext,
-      taskInstances = Map(taskName -> taskInstance),
-      runLoop = null,
-      systemAdmins = systemAdmins,
-      consumerMultiplexer = consumerMultiplexer,
-      producerMultiplexer = producerMultiplexer,
-      metrics = containerMetrics)
-
-    container.startStores
-    assertNotNull(containerMetrics.taskStoreRestorationMetrics)
-    assertNotNull(containerMetrics.taskStoreRestorationMetrics.get(taskName))
-    assertTrue(containerMetrics.taskStoreRestorationMetrics.get(taskName).getValue >= 1)
-
-  }
-
-  @Test
-  def testGetChangelogSSPsForContainer() = {
+  def testGetChangelogSSPsForContainer() {
     val taskName0 = new TaskName("task0")
     val taskName1 = new TaskName("task1")
     val taskModel0 = new TaskModel(taskName0,
@@ -665,7 +245,7 @@ class TestSamzaContainer extends AssertionsForJUnit with MockitoSugar {
   }
 
   @Test
-  def testGetChangelogSSPsForContainerNoChangelogs() = {
+  def testGetChangelogSSPsForContainerNoChangelogs() {
     val taskName0 = new TaskName("task0")
     val taskName1 = new TaskName("task1")
     val taskModel0 = new TaskModel(taskName0,
@@ -677,29 +257,18 @@ class TestSamzaContainer extends AssertionsForJUnit with MockitoSugar {
     val containerModel = new ContainerModel("processorId", 0, Map(taskName0 -> taskModel0, taskName1 -> taskModel1))
     assertEquals(Set(), SamzaContainer.getChangelogSSPsForContainer(containerModel, Map()))
   }
-}
-
-class MockCheckpointManager extends CheckpointManager {
-  override def start() = {}
-  override def stop() = {}
 
-  override def register(taskName: TaskName): Unit = {}
+  class MockJobServlet(exceptionLimit: Int, jobModelRef: AtomicReference[JobModel]) extends JobServlet(jobModelRef) {
+    var exceptionCount = 0
 
-  override def readLastCheckpoint(taskName: TaskName): Checkpoint = { new Checkpoint(Map[SystemStreamPartition, String]().asJava) }
-
-  override def writeCheckpoint(taskName: TaskName, checkpoint: Checkpoint): Unit = { }
-}
-
-class MockJobServlet(exceptionLimit: Int, jobModelRef: AtomicReference[JobModel]) extends JobServlet(jobModelRef) {
-  var exceptionCount = 0
-
-  override protected def getObjectToWrite() = {
-    if (exceptionCount < exceptionLimit) {
-      exceptionCount += 1
-      throw new java.io.IOException("Throwing exception")
-    } else {
-      val jobModel = jobModelRef.get()
-      jobModel
+    override protected def getObjectToWrite(): JobModel = {
+      if (exceptionCount < exceptionLimit) {
+        exceptionCount += 1
+        throw new java.io.IOException("Throwing exception")
+      } else {
+        val jobModel = jobModelRef.get()
+        jobModel
+      }
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/19c6f4f6/samza-core/src/test/scala/org/apache/samza/container/TestTaskInstance.scala
----------------------------------------------------------------------
diff --git a/samza-core/src/test/scala/org/apache/samza/container/TestTaskInstance.scala b/samza-core/src/test/scala/org/apache/samza/container/TestTaskInstance.scala
index 1672191..b196131 100644
--- a/samza-core/src/test/scala/org/apache/samza/container/TestTaskInstance.scala
+++ b/samza-core/src/test/scala/org/apache/samza/container/TestTaskInstance.scala
@@ -20,429 +20,211 @@
 package org.apache.samza.container
 
 
-import java.util.concurrent.ConcurrentHashMap
-
 import org.apache.samza.Partition
 import org.apache.samza.checkpoint.{Checkpoint, OffsetManager}
-import org.apache.samza.config.{Config, MapConfig}
-import org.apache.samza.metrics.{Counter, Metric, MetricsRegistryMap}
-import org.apache.samza.serializers.SerdeManager
-import org.apache.samza.system.IncomingMessageEnvelope
-import org.apache.samza.system.SystemAdmin
-import org.apache.samza.system.SystemConsumer
-import org.apache.samza.system.SystemConsumers
-import org.apache.samza.system.SystemProducer
-import org.apache.samza.system.SystemProducers
-import org.apache.samza.system.SystemStream
-import org.apache.samza.system.SystemStreamMetadata
+import org.apache.samza.config.Config
+import org.apache.samza.metrics.Counter
 import org.apache.samza.storage.TaskStorageManager
-import org.apache.samza.system.SystemStreamMetadata.SystemStreamPartitionMetadata
-import org.apache.samza.system._
-import org.apache.samza.system.chooser.RoundRobinChooser
+import org.apache.samza.system.{IncomingMessageEnvelope, SystemAdmin, SystemConsumers, SystemStream, _}
 import org.apache.samza.task._
 import org.junit.Assert._
-import org.junit.Test
+import org.junit.{Before, Test}
 import org.mockito.Matchers._
-import org.mockito.Mockito
 import org.mockito.Mockito._
-import org.scalatest.Assertions.intercept
+import org.mockito.invocation.InvocationOnMock
+import org.mockito.stubbing.Answer
+import org.mockito.{Matchers, Mock, MockitoAnnotations}
+import org.scalatest.mockito.MockitoSugar
 
-import scala.collection.mutable.ListBuffer
 import scala.collection.JavaConverters._
 
-class TestTaskInstance {
-  @Test
-  def testOffsetsAreUpdatedOnProcess {
-    val task = new StreamTask {
-      def process(envelope: IncomingMessageEnvelope, collector: MessageCollector, coordinator: TaskCoordinator) {
-      }
-    }
-    val config = new MapConfig
-    val partition = new Partition(0)
-    val consumerMultiplexer = new SystemConsumers(
-      new RoundRobinChooser,
-      Map[String, SystemConsumer]())
-    val producerMultiplexer = new SystemProducers(
-      Map[String, SystemProducer](),
-      new SerdeManager)
-    val systemStream = new SystemStream("test-system", "test-stream")
-    val systemStreamPartition = new SystemStreamPartition(systemStream, partition)
-    val systemStreamPartitions = Set(systemStreamPartition)
-    // Pretend our last checkpointed (next) offset was 2.
-    val testSystemStreamMetadata = new SystemStreamMetadata(systemStream.getStream, Map(partition -> new SystemStreamPartitionMetadata("0", "1", "2")).asJava)
-    val offsetManager = OffsetManager(Map(systemStream -> testSystemStreamMetadata), config)
-    val taskName = new TaskName("taskName")
-    val collector = new TaskInstanceCollector(producerMultiplexer)
-    val containerContext = new SamzaContainerContext("0", config, Set(taskName).asJava, new MetricsRegistryMap)
-    val taskInstance: TaskInstance = new TaskInstance(
-      task,
-      taskName,
-      config,
-      new TaskInstanceMetrics,
-      null,
-      consumerMultiplexer,
-      collector,
-      containerContext,
-      offsetManager,
-      systemStreamPartitions = systemStreamPartitions)
-    // Pretend we got a message with offset 2 and next offset 3.
-    val coordinator = new ReadableCoordinator(taskName)
-    taskInstance.process(new IncomingMessageEnvelope(systemStreamPartition, "2", null, null), coordinator)
-    // Check to see if the offset manager has been properly updated with offset 3.
-    val lastProcessedOffset = offsetManager.getLastProcessedOffset(taskName, systemStreamPartition)
-    assertTrue(lastProcessedOffset.isDefined)
-    assertEquals("2", lastProcessedOffset.get)
-  }
-
-  /**
-   * Mock exception used to test exception counts metrics.
-   */
-  class TroublesomeException extends RuntimeException {
-  }
-
-  /**
-   * Mock exception used to test exception counts metrics.
-   */
-  class NonFatalException extends RuntimeException {
-  }
-
-  /**
-   * Mock exception used to test exception counts metrics.
-   */
-  class FatalException extends RuntimeException {
-  }
-
-  /**
-   * Task used to test exception counts metrics.
-   */
-  class TroublesomeTask extends StreamTask with WindowableTask {
-    def process(
-                 envelope: IncomingMessageEnvelope,
-                 collector: MessageCollector,
-                 coordinator: TaskCoordinator) {
-
-      envelope.getOffset().toInt match {
-        case offset if offset % 2 == 0 => throw new TroublesomeException
-        case _ => throw new NonFatalException
-      }
-    }
-
-    def window(collector: MessageCollector, coordinator: TaskCoordinator) {
-      throw new FatalException
-    }
+class TestTaskInstance extends MockitoSugar {
+  private val SYSTEM_NAME = "test-system"
+  private val TASK_NAME = new TaskName("taskName")
+  private val SYSTEM_STREAM_PARTITION =
+    new SystemStreamPartition(new SystemStream(SYSTEM_NAME, "test-stream"), new Partition(0))
+  private val SYSTEM_STREAM_PARTITIONS = Set(SYSTEM_STREAM_PARTITION)
+
+  @Mock
+  private var task: AllTask = null
+  @Mock
+  private var config: Config = null
+  @Mock
+  private var metrics: TaskInstanceMetrics = null
+  @Mock
+  private var systemAdmins: SystemAdmins = null
+  @Mock
+  private var systemAdmin: SystemAdmin = null
+  @Mock
+  private var consumerMultiplexer: SystemConsumers = null
+  @Mock
+  private var collector: TaskInstanceCollector = null
+  @Mock
+  private var containerContext: SamzaContainerContext = null
+  @Mock
+  private var offsetManager: OffsetManager = null
+  @Mock
+  private var taskStorageManager: TaskStorageManager = null
+  // not a mock; using MockTaskInstanceExceptionHandler
+  private var taskInstanceExceptionHandler: MockTaskInstanceExceptionHandler = null
+
+  private var taskInstance: TaskInstance = null
+
+  @Before
+  def setup(): Unit = {
+    MockitoAnnotations.initMocks(this)
+    // not using Mockito mock since Mockito doesn't work well with the call-by-name argument in maybeHandle
+    this.taskInstanceExceptionHandler = new MockTaskInstanceExceptionHandler
+    this.taskInstance = new TaskInstance(this.task,
+      TASK_NAME,
+      this.config,
+      this.metrics,
+      this.systemAdmins,
+      this.consumerMultiplexer,
+      this.collector,
+      this.containerContext,
+      this.offsetManager,
+      storageManager = this.taskStorageManager,
+      systemStreamPartitions = SYSTEM_STREAM_PARTITIONS,
+      exceptionHandler = this.taskInstanceExceptionHandler)
+    when(this.systemAdmins.getSystemAdmin(SYSTEM_NAME)).thenReturn(this.systemAdmin)
   }
 
-  /*
-   * Helper method used to retrieve the value of a counter from a group.
-   */
-  private def getCount(
-                        group: ConcurrentHashMap[String, Metric],
-                        name: String): Long = {
-    group.get("exception-ignored-" + name.toLowerCase).asInstanceOf[Counter].getCount
+  @Test
+  def testProcess() {
+    val processesCounter = mock[Counter]
+    when(this.metrics.processes).thenReturn(processesCounter)
+    val messagesActuallyProcessedCounter = mock[Counter]
+    when(this.metrics.messagesActuallyProcessed).thenReturn(messagesActuallyProcessedCounter)
+    when(this.offsetManager.getStartingOffset(TASK_NAME, SYSTEM_STREAM_PARTITION)).thenReturn(Some("0"))
+    val envelope = new IncomingMessageEnvelope(SYSTEM_STREAM_PARTITION, "0", null, null)
+    val coordinator = mock[ReadableCoordinator]
+    this.taskInstance.process(envelope, coordinator)
+    assertEquals(1, this.taskInstanceExceptionHandler.numTimesCalled)
+    verify(this.task).process(envelope, this.collector, coordinator)
+    verify(processesCounter).inc()
+    verify(messagesActuallyProcessedCounter).inc()
   }
 
-  /**
-   * Test task instance exception metrics with two ignored exceptions and one
-   * exception not ignored.
-   */
   @Test
-  def testExceptionCounts {
-    val task = new TroublesomeTask
-    val ignoredExceptions = classOf[TroublesomeException].getName + "," +
-      classOf[NonFatalException].getName
-    val config = new MapConfig(Map[String, String](
-      "task.ignored.exceptions" -> ignoredExceptions).asJava)
-
-    val partition = new Partition(0)
-    val consumerMultiplexer = new SystemConsumers(
-      new RoundRobinChooser,
-      Map[String, SystemConsumer]())
-    val producerMultiplexer = new SystemProducers(
-      Map[String, SystemProducer](),
-      new SerdeManager)
-    val systemStream = new SystemStream("test-system", "test-stream")
-    val systemStreamPartition = new SystemStreamPartition(systemStream, partition)
-    val systemStreamPartitions = Set(systemStreamPartition)
-    // Pretend our last checkpointed (next) offset was 2.
-    val testSystemStreamMetadata = new SystemStreamMetadata(systemStream.getStream, Map(partition -> new SystemStreamPartitionMetadata("0", "1", "2")).asJava)
-    val offsetManager = OffsetManager(Map(systemStream -> testSystemStreamMetadata), config)
-    val taskName = new TaskName("taskName")
-    val collector = new TaskInstanceCollector(producerMultiplexer)
-    val containerContext = new SamzaContainerContext("0", config, Set(taskName).asJava, new MetricsRegistryMap)
-
-    val registry = new MetricsRegistryMap
-    val taskMetrics = new TaskInstanceMetrics(registry = registry)
-    val taskInstance = new TaskInstance(
-      task,
-      taskName,
-      config,
-      taskMetrics,
-      null,
-      consumerMultiplexer,
-      collector,
-      containerContext,
-      offsetManager,
-      systemStreamPartitions = systemStreamPartitions,
-      exceptionHandler = TaskInstanceExceptionHandler(taskMetrics, config))
-
-    val coordinator = new ReadableCoordinator(taskName)
-    taskInstance.process(new IncomingMessageEnvelope(systemStreamPartition, "1", null, null), coordinator)
-    taskInstance.process(new IncomingMessageEnvelope(systemStreamPartition, "2", null, null), coordinator)
-    taskInstance.process(new IncomingMessageEnvelope(systemStreamPartition, "3", null, null), coordinator)
-
-    val group = registry.getGroup(taskMetrics.group)
-    assertEquals(1L, getCount(group, classOf[TroublesomeException].getName))
-    assertEquals(2L, getCount(group, classOf[NonFatalException].getName))
-
-    intercept[FatalException] {
-      taskInstance.window(coordinator)
-    }
-    assertFalse(group.contains(classOf[FatalException].getName.toLowerCase))
+  def testWindow() {
+    val windowsCounter = mock[Counter]
+    when(this.metrics.windows).thenReturn(windowsCounter)
+    val coordinator = mock[ReadableCoordinator]
+    this.taskInstance.window(coordinator)
+    assertEquals(1, this.taskInstanceExceptionHandler.numTimesCalled)
+    verify(this.task).window(this.collector, coordinator)
+    verify(windowsCounter).inc()
   }
 
-  /**
-   * Test task instance exception metrics with all exception ignored using a
-   * wildcard.
-   */
   @Test
-  def testIgnoreAllExceptions {
-    val task = new TroublesomeTask
-    val config = new MapConfig(Map[String, String](
-      "task.ignored.exceptions" -> "*").asJava)
-
-    val partition = new Partition(0)
-    val consumerMultiplexer = new SystemConsumers(
-      new RoundRobinChooser,
-      Map[String, SystemConsumer]())
-    val producerMultiplexer = new SystemProducers(
-      Map[String, SystemProducer](),
-      new SerdeManager)
-    val systemStream = new SystemStream("test-system", "test-stream")
-    val systemStreamPartition = new SystemStreamPartition(systemStream, partition)
-    val systemStreamPartitions = Set(systemStreamPartition)
-    // Pretend our last checkpointed (next) offset was 2.
-    val testSystemStreamMetadata = new SystemStreamMetadata(systemStream.getStream, Map(partition -> new SystemStreamPartitionMetadata("0", "1", "2")).asJava)
-    val offsetManager = OffsetManager(Map(systemStream -> testSystemStreamMetadata), config)
-    val taskName = new TaskName("taskName")
-    val collector = new TaskInstanceCollector(producerMultiplexer)
-    val containerContext = new SamzaContainerContext("0", config, Set(taskName).asJava, new MetricsRegistryMap)
-
-    val registry = new MetricsRegistryMap
-    val taskMetrics = new TaskInstanceMetrics(registry = registry)
-    val taskInstance = new TaskInstance(
-      task,
-      taskName,
-      config,
-      taskMetrics,
-      null,
-      consumerMultiplexer,
-      collector,
-      containerContext,
-      offsetManager,
-      systemStreamPartitions = systemStreamPartitions,
-      exceptionHandler = TaskInstanceExceptionHandler(taskMetrics, config))
-
-    val coordinator = new ReadableCoordinator(taskName)
-    taskInstance.process(new IncomingMessageEnvelope(systemStreamPartition, "1", null, null), coordinator)
-    taskInstance.process(new IncomingMessageEnvelope(systemStreamPartition, "2", null, null), coordinator)
-    taskInstance.process(new IncomingMessageEnvelope(systemStreamPartition, "3", null, null), coordinator)
-    taskInstance.window(coordinator)
-
-    val group = registry.getGroup(taskMetrics.group)
-    assertEquals(1L, getCount(group, classOf[TroublesomeException].getName))
-    assertEquals(2L, getCount(group, classOf[NonFatalException].getName))
-    assertEquals(1L, getCount(group, classOf[FatalException].getName))
+  def testOffsetsAreUpdatedOnProcess() {
+    when(this.metrics.processes).thenReturn(mock[Counter])
+    when(this.metrics.messagesActuallyProcessed).thenReturn(mock[Counter])
+    when(this.offsetManager.getStartingOffset(TASK_NAME, SYSTEM_STREAM_PARTITION)).thenReturn(Some("2"))
+    this.taskInstance.process(new IncomingMessageEnvelope(SYSTEM_STREAM_PARTITION, "4", null, null),
+      mock[ReadableCoordinator])
+    verify(this.offsetManager).update(TASK_NAME, SYSTEM_STREAM_PARTITION, "4")
   }
 
   /**
-   * Tests that the init() method of task can override the existing offset
-   * assignment.
+   * Tests that the init() method of task can override the existing offset assignment.
+   * This helps verify wiring for the task context (i.e. offset manager).
    */
   @Test
-  def testManualOffsetReset {
-
-    val partition0 = new SystemStreamPartition("system", "stream", new Partition(0))
-    val partition1 = new SystemStreamPartition("system", "stream", new Partition(1))
-
-    val task = new StreamTask with InitableTask {
-
-      override def init(config: Config, context: TaskContext): Unit = {
-
-        assertTrue("Can only update offsets for assigned partition",
-          context.getSystemStreamPartitions.contains(partition1))
-
-        context.setStartingOffset(partition1, "10")
+  def testManualOffsetReset() {
+    when(this.task.init(any(), any())).thenAnswer(new Answer[Void] {
+      override def answer(invocation: InvocationOnMock): Void = {
+        val taskContext = invocation.getArgumentAt(1, classOf[TaskContext])
+        taskContext.setStartingOffset(SYSTEM_STREAM_PARTITION, "10")
+        null
       }
-
-      override def process(envelope: IncomingMessageEnvelope, collector: MessageCollector, coordinator: TaskCoordinator): Unit = {}
-    }
-
-    val config = new MapConfig()
-    val chooser = new RoundRobinChooser()
-    val consumers = new SystemConsumers(chooser, consumers = Map.empty)
-    val producers = new SystemProducers(Map.empty, new SerdeManager())
-    val metrics = new TaskInstanceMetrics()
-    val taskName = new TaskName("Offset Reset Task 0")
-    val collector = new TaskInstanceCollector(producers)
-    val containerContext = new SamzaContainerContext("0", config, Set(taskName).asJava, new MetricsRegistryMap)
-
-    val offsetManager = new OffsetManager()
-
-    offsetManager.startingOffsets += taskName -> Map(partition0 -> "0", partition1 -> "0")
-
-    val taskInstance = new TaskInstance(
-      task,
-      taskName,
-      config,
-      metrics,
-      null,
-      consumers,
-      collector,
-      containerContext,
-      offsetManager,
-      systemStreamPartitions = Set(partition0, partition1))
-
+    })
     taskInstance.initTask
 
-    assertEquals(Some("0"), offsetManager.getStartingOffset(taskName, partition0))
-    assertEquals(Some("10"), offsetManager.getStartingOffset(taskName, partition1))
+    verify(this.offsetManager).setStartingOffset(TASK_NAME, SYSTEM_STREAM_PARTITION, "10")
+    verifyNoMoreInteractions(this.offsetManager)
   }
 
   @Test
-  def testIgnoreMessagesOlderThanStartingOffsets {
-    val partition0 = new SystemStreamPartition("system", "stream", new Partition(0))
-    val partition1 = new SystemStreamPartition("system", "stream", new Partition(1))
-    val config = new MapConfig()
-    val chooser = new RoundRobinChooser()
-    val consumers = new SystemConsumers(chooser, consumers = Map.empty)
-    val producers = new SystemProducers(Map.empty, new SerdeManager())
-    val metrics = new TaskInstanceMetrics()
-    val taskName = new TaskName("testing")
-    val collector = new TaskInstanceCollector(producers)
-    val containerContext = new SamzaContainerContext("0", config, Set(taskName).asJava, new MetricsRegistryMap)
-    val offsetManager = new OffsetManager()
-    offsetManager.startingOffsets += taskName -> Map(partition0 -> "0", partition1 -> "100")
-    val systemAdmins = Mockito.mock(classOf[SystemAdmins])
-    when(systemAdmins.getSystemAdmin("system")).thenReturn(new MockSystemAdmin)
-    var result = new ListBuffer[IncomingMessageEnvelope]
-
-    val task = new StreamTask {
-      def process(envelope: IncomingMessageEnvelope, collector: MessageCollector, coordinator: TaskCoordinator) {
-        result += envelope
+  def testIgnoreMessagesOlderThanStartingOffsets() {
+    val processesCounter = mock[Counter]
+    when(this.metrics.processes).thenReturn(processesCounter)
+    val messagesActuallyProcessedCounter = mock[Counter]
+    when(this.metrics.messagesActuallyProcessed).thenReturn(messagesActuallyProcessedCounter)
+    when(this.offsetManager.getStartingOffset(TASK_NAME, SYSTEM_STREAM_PARTITION)).thenReturn(Some("5"))
+    when(this.systemAdmin.offsetComparator(any(), any())).thenAnswer(new Answer[Integer] {
+      override def answer(invocation: InvocationOnMock): Integer = {
+        val offset1 = invocation.getArgumentAt(0, classOf[String])
+        val offset2 = invocation.getArgumentAt(1, classOf[String])
+        offset1.toLong.compareTo(offset2.toLong)
       }
-    }
-
-    val taskInstance = new TaskInstance(
-      task,
-      taskName,
-      config,
-      metrics,
-      systemAdmins,
-      consumers,
-      collector,
-      containerContext,
-      offsetManager,
-      systemStreamPartitions = Set(partition0, partition1))
-
-    val coordinator = new ReadableCoordinator(taskName)
-    val envelope1 = new IncomingMessageEnvelope(partition0, "1", null, null)
-    val envelope2 = new IncomingMessageEnvelope(partition0, "2", null, null)
-    val envelope3 = new IncomingMessageEnvelope(partition1, "1", null, null)
-    val envelope4 = new IncomingMessageEnvelope(partition1, "102", null, null)
-
-    taskInstance.process(envelope1, coordinator)
-    taskInstance.process(envelope2, coordinator)
-    taskInstance.process(envelope3, coordinator)
-    taskInstance.process(envelope4, coordinator)
-
-    val expected = List(envelope1, envelope2, envelope4)
-    assertEquals(expected, result.toList)
+    })
+    val oldEnvelope = new IncomingMessageEnvelope(SYSTEM_STREAM_PARTITION, "0", null, null)
+    val newEnvelope0 = new IncomingMessageEnvelope(SYSTEM_STREAM_PARTITION, "5", null, null)
+    val newEnvelope1 = new IncomingMessageEnvelope(SYSTEM_STREAM_PARTITION, "7", null, null)
+
+    this.taskInstance.process(oldEnvelope, mock[ReadableCoordinator])
+    this.taskInstance.process(newEnvelope0, mock[ReadableCoordinator])
+    this.taskInstance.process(newEnvelope1, mock[ReadableCoordinator])
+    verify(this.task).process(Matchers.eq(newEnvelope0), Matchers.eq(this.collector), any())
+    verify(this.task).process(Matchers.eq(newEnvelope1), Matchers.eq(this.collector), any())
+    verify(this.task, never()).process(Matchers.eq(oldEnvelope), any(), any())
+    verify(processesCounter, times(3)).inc()
+    verify(messagesActuallyProcessedCounter, times(2)).inc()
   }
 
   @Test
-  def testCommitOrder {
-    // Simple objects
-    val partition = new Partition(0)
-    val taskName = new TaskName("taskName")
-    val systemStream = new SystemStream("test-system", "test-stream")
-    val systemStreamPartition = new SystemStreamPartition(systemStream, partition)
-    val checkpoint = new Checkpoint(Map(systemStreamPartition -> "4").asJava)
-
-    // Mocks
-    val collector = Mockito.mock(classOf[TaskInstanceCollector])
-    val storageManager = Mockito.mock(classOf[TaskStorageManager])
-    val offsetManager = Mockito.mock(classOf[OffsetManager])
-    when(offsetManager.buildCheckpoint(any())).thenReturn(checkpoint)
-    val mockOrder = inOrder(offsetManager, collector, storageManager)
-
-    val taskInstance: TaskInstance = new TaskInstance(
-      Mockito.mock(classOf[StreamTask]),
-      taskName,
-      new MapConfig,
-      new TaskInstanceMetrics,
-      null,
-      Mockito.mock(classOf[SystemConsumers]),
-      collector,
-      Mockito.mock(classOf[SamzaContainerContext]),
-      offsetManager,
-      storageManager,
-      systemStreamPartitions = Set(systemStreamPartition))
+  def testCommitOrder() {
+    val commitsCounter = mock[Counter]
+    when(this.metrics.commits).thenReturn(commitsCounter)
+    val checkpoint = new Checkpoint(Map(SYSTEM_STREAM_PARTITION -> "4").asJava)
+    when(this.offsetManager.buildCheckpoint(TASK_NAME)).thenReturn(checkpoint)
 
     taskInstance.commit
 
+    val mockOrder = inOrder(this.offsetManager, this.collector, this.taskStorageManager)
+
     // We must first get a snapshot of the checkpoint so it doesn't change while we flush. SAMZA-1384
-    mockOrder.verify(offsetManager).buildCheckpoint(taskName)
+    mockOrder.verify(this.offsetManager).buildCheckpoint(TASK_NAME)
     // Producers must be flushed next and ideally the output would be flushed before the changelog
     // s.t. the changelog and checkpoints (state and inputs) are captured last
-    mockOrder.verify(collector).flush
+    mockOrder.verify(this.collector).flush
     // Local state is next, to ensure that the state (particularly the offset file) never points to a newer changelog
     // offset than what is reflected in the on disk state.
-    mockOrder.verify(storageManager).flush()
+    mockOrder.verify(this.taskStorageManager).flush()
     // Finally, checkpoint the inputs with the snapshotted checkpoint captured at the beginning of commit
-    mockOrder.verify(offsetManager).writeCheckpoint(taskName, checkpoint)
+    mockOrder.verify(offsetManager).writeCheckpoint(TASK_NAME, checkpoint)
+    verify(commitsCounter).inc()
   }
 
   @Test(expected = classOf[SystemProducerException])
-  def testProducerExceptionsIsPropagated {
-    // Simple objects
-    val partition = new Partition(0)
-    val taskName = new TaskName("taskName")
-    val systemStream = new SystemStream("test-system", "test-stream")
-    val systemStreamPartition = new SystemStreamPartition(systemStream, partition)
-
-    // Mocks
-    val collector = Mockito.mock(classOf[TaskInstanceCollector])
-    when(collector.flush).thenThrow(new SystemProducerException("Test"))
-    val storageManager = Mockito.mock(classOf[TaskStorageManager])
-    val offsetManager = Mockito.mock(classOf[OffsetManager])
-
-    val taskInstance: TaskInstance = new TaskInstance(
-      Mockito.mock(classOf[StreamTask]),
-      taskName,
-      new MapConfig,
-      new TaskInstanceMetrics,
-      null,
-      Mockito.mock(classOf[SystemConsumers]),
-      collector,
-      Mockito.mock(classOf[SamzaContainerContext]),
-      offsetManager,
-      storageManager,
-      systemStreamPartitions = Set(systemStreamPartition))
+  def testProducerExceptionsIsPropagated() {
+    when(this.metrics.commits).thenReturn(mock[Counter])
+    when(this.collector.flush).thenThrow(new SystemProducerException("systemProducerException"))
 
     try {
       taskInstance.commit // Should not swallow the SystemProducerException
     } finally {
-      Mockito.verify(offsetManager, times(0)).writeCheckpoint(any(classOf[TaskName]), any(classOf[Checkpoint]))
+      verify(offsetManager, never()).writeCheckpoint(any(), any())
     }
   }
 
-}
-
-class MockSystemAdmin extends SystemAdmin {
-  override def getOffsetsAfter(offsets: java.util.Map[SystemStreamPartition, String]) = { offsets }
-  override def getSystemStreamMetadata(streamNames: java.util.Set[String]) = null
+  /**
+    * Task type which has all task traits, which can be mocked.
+    */
+  trait AllTask extends StreamTask with InitableTask with WindowableTask {}
 
-  override def offsetComparator(offset1: String, offset2: String) = {
-    offset1.toLong compare offset2.toLong
+  /**
+    * Mock version of [TaskInstanceExceptionHandler] which just does a passthrough execution and keeps track of the
+    * number of times it is called. This is used to verify that the handler does get used to wrap the actual processing.
+    */
+  class MockTaskInstanceExceptionHandler extends TaskInstanceExceptionHandler {
+    var numTimesCalled = 0
+
+    override def maybeHandle(tryCodeBlock: => Unit): Unit = {
+      numTimesCalled += 1
+      tryCodeBlock
+    }
   }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/samza/blob/19c6f4f6/samza-core/src/test/scala/org/apache/samza/container/TestTaskInstanceExceptionHandler.scala
----------------------------------------------------------------------
diff --git a/samza-core/src/test/scala/org/apache/samza/container/TestTaskInstanceExceptionHandler.scala b/samza-core/src/test/scala/org/apache/samza/container/TestTaskInstanceExceptionHandler.scala
new file mode 100644
index 0000000..ca06b2a
--- /dev/null
+++ b/samza-core/src/test/scala/org/apache/samza/container/TestTaskInstanceExceptionHandler.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.container
+
+import com.google.common.collect.ImmutableMap
+import org.apache.samza.config.{Config, MapConfig, TaskConfig}
+import org.apache.samza.metrics.{Counter, MetricsHelper}
+import org.junit.{Before, Test}
+import org.mockito.Mockito._
+import org.mockito.{Mock, MockitoAnnotations}
+import org.scalatest.junit.AssertionsForJUnit
+import org.scalatest.mockito.MockitoSugar
+
+class TestTaskInstanceExceptionHandler extends AssertionsForJUnit with MockitoSugar {
+  @Mock
+  private var metrics: MetricsHelper = null
+  @Mock
+  private var troublesomeExceptionCounter: Counter = null
+  @Mock
+  private var nonFatalExceptionCounter: Counter = null
+  @Mock
+  private var fatalExceptionCounter: Counter = null
+
+  @Before
+  def setup() {
+    MockitoAnnotations.initMocks(this)
+    when(this.metrics.newCounter("exception-ignored-" + classOf[TroublesomeException].getName)).thenReturn(
+        this.troublesomeExceptionCounter)
+    when(this.metrics.newCounter("exception-ignored-" + classOf[NonFatalException].getName)).thenReturn(
+        this.nonFatalExceptionCounter)
+    when(this.metrics.newCounter("exception-ignored-" + classOf[FatalException].getName)).thenReturn(
+        this.fatalExceptionCounter)
+  }
+
+  /**
+   * Given that no exceptions are ignored, any exception should get propogated up.
+   */
+  @Test
+  def testHandleIgnoreNone() {
+    val handler = build(new MapConfig())
+    intercept[TroublesomeException] {
+      handler.maybeHandle(() -> {
+        throw new TroublesomeException()
+      })
+    }
+    verifyZeroInteractions(this.metrics, this.troublesomeExceptionCounter, this.nonFatalExceptionCounter,
+        this.fatalExceptionCounter)
+  }
+
+  /**
+   * Given that some exceptions are ignored, the ignored exceptions should not be thrown and should increment the proper
+   * metrics, and any other exception should get propagated up.
+   */
+  @Test
+  def testHandleIgnoreSome() {
+    val config = new MapConfig(ImmutableMap.of(TaskConfig.IGNORED_EXCEPTIONS,
+        String.join(",", classOf[TroublesomeException].getName, classOf[NonFatalException].getName)))
+    val handler = build(config)
+    handler.maybeHandle(() -> {
+      throw new TroublesomeException()
+    })
+    handler.maybeHandle(() -> {
+      throw new NonFatalException()
+    })
+    intercept[FatalException] {
+      handler.maybeHandle(() -> {
+        throw new FatalException()
+      })
+    }
+    handler.maybeHandle(() -> {
+      throw new TroublesomeException()
+    })
+    verify(this.troublesomeExceptionCounter, times(2)).inc()
+    // double check that the counter gets cached for multiple occurrences of the same exception type
+    verify(this.metrics).newCounter("exception-ignored-" + classOf[TroublesomeException].getName)
+    verify(this.nonFatalExceptionCounter).inc()
+    verifyZeroInteractions(this.fatalExceptionCounter)
+  }
+
+  /**
+   * Given that all exceptions are ignored, no exceptions should be thrown and the proper metrics should be incremented.
+   */
+  @Test
+  def testHandleIgnoreAll() {
+    val config = new MapConfig(ImmutableMap.of(TaskConfig.IGNORED_EXCEPTIONS, "*"))
+    val handler = build(config)
+    handler.maybeHandle(() -> {
+      throw new TroublesomeException()
+    })
+    handler.maybeHandle(() -> {
+      throw new TroublesomeException()
+    })
+    handler.maybeHandle(() -> {
+      throw new NonFatalException()
+    })
+    handler.maybeHandle(() -> {
+      throw new FatalException()
+    })
+
+    verify(this.troublesomeExceptionCounter, times(2)).inc()
+    // double check that the counter gets cached for multiple occurrences of the same exception type
+    verify(this.metrics).newCounter("exception-ignored-" + classOf[TroublesomeException].getName)
+    verify(this.nonFatalExceptionCounter).inc()
+    verify(this.fatalExceptionCounter).inc()
+  }
+
+  private def build(config: Config): TaskInstanceExceptionHandler = {
+    TaskInstanceExceptionHandler.apply(this.metrics, config)
+  }
+
+  /**
+   * Mock exception used to test exception counts metrics.
+   */
+  private class TroublesomeException extends RuntimeException {
+  }
+
+  /**
+   * Mock exception used to test exception counts metrics.
+   */
+  private class NonFatalException extends RuntimeException {
+  }
+
+  /**
+   * Mock exception used to test exception counts metrics.
+   */
+  private class FatalException extends RuntimeException {
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/19c6f4f6/samza-core/src/test/scala/org/apache/samza/system/chooser/MockSystemAdmin.scala
----------------------------------------------------------------------
diff --git a/samza-core/src/test/scala/org/apache/samza/system/chooser/MockSystemAdmin.scala b/samza-core/src/test/scala/org/apache/samza/system/chooser/MockSystemAdmin.scala
new file mode 100644
index 0000000..288dd25
--- /dev/null
+++ b/samza-core/src/test/scala/org/apache/samza/system/chooser/MockSystemAdmin.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.chooser
+
+import org.apache.samza.system.{SystemAdmin, SystemStreamPartition}
+
+class MockSystemAdmin extends SystemAdmin {
+  override def getOffsetsAfter(offsets: java.util.Map[SystemStreamPartition, String]) = { offsets }
+  override def getSystemStreamMetadata(streamNames: java.util.Set[String]) = null
+
+  override def offsetComparator(offset1: String, offset2: String) = {
+    offset1.toLong compare offset2.toLong
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/19c6f4f6/samza-core/src/test/scala/org/apache/samza/system/chooser/TestBootstrappingChooser.scala
----------------------------------------------------------------------
diff --git a/samza-core/src/test/scala/org/apache/samza/system/chooser/TestBootstrappingChooser.scala b/samza-core/src/test/scala/org/apache/samza/system/chooser/TestBootstrappingChooser.scala
index 1a99355..5116a51 100644
--- a/samza-core/src/test/scala/org/apache/samza/system/chooser/TestBootstrappingChooser.scala
+++ b/samza-core/src/test/scala/org/apache/samza/system/chooser/TestBootstrappingChooser.scala
@@ -23,7 +23,6 @@ import java.util.Arrays
 
 import org.apache.samza.system._
 import org.apache.samza.Partition
-import org.apache.samza.container.MockSystemAdmin
 import org.apache.samza.metrics.MetricsRegistryMap
 import org.apache.samza.system.SystemStreamMetadata.SystemStreamPartitionMetadata
 import org.junit.Assert._
@@ -301,4 +300,4 @@ object TestBootstrappingChooser {
       Array((wrapped: MessageChooser, bootstrapStreamMetadata: Map[SystemStream, SystemStreamMetadata]) =>
         new DefaultChooser(wrapped, bootstrapStreamMetadata = bootstrapStreamMetadata, registry = new MetricsRegistryMap(), systemAdmins = systemAdmins)))
   }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/samza/blob/19c6f4f6/samza-core/src/test/scala/org/apache/samza/system/chooser/TestDefaultChooser.scala
----------------------------------------------------------------------
diff --git a/samza-core/src/test/scala/org/apache/samza/system/chooser/TestDefaultChooser.scala b/samza-core/src/test/scala/org/apache/samza/system/chooser/TestDefaultChooser.scala
index c4c702d..a4917d4 100644
--- a/samza-core/src/test/scala/org/apache/samza/system/chooser/TestDefaultChooser.scala
+++ b/samza-core/src/test/scala/org/apache/samza/system/chooser/TestDefaultChooser.scala
@@ -21,7 +21,6 @@ package org.apache.samza.system.chooser
 
 import org.apache.samza.Partition
 import org.apache.samza.config.{DefaultChooserConfig, MapConfig}
-import org.apache.samza.container.MockSystemAdmin
 import org.apache.samza.metrics.MetricsRegistryMap
 import org.apache.samza.system.SystemStreamMetadata.SystemStreamPartitionMetadata
 import org.apache.samza.system._


[07/29] samza git commit: Merge branch 'master' of https://github.com/apache/samza

Posted by bo...@apache.org.
Merge branch 'master' of https://github.com/apache/samza


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

Branch: refs/heads/NewKafkaSystemConsumer
Commit: 927adff0b1d65baec79d8b1ca8b8e4cce63be032
Parents: 952dbbe c48bcd2
Author: Boris S <bo...@apache.org>
Authored: Thu Sep 20 09:38:51 2018 -0700
Committer: Boris S <bo...@apache.org>
Committed: Thu Sep 20 09:38:51 2018 -0700

----------------------------------------------------------------------
 .travis.yml                                     |   4 +-
 build.gradle                                    |   1 +
 gradle/dependency-versions.gradle               |   1 +
 .../samza/metadatastore/MetadataStore.java      |   6 +-
 .../samza/config/ClusterManagerConfig.java      |   1 -
 .../apache/samza/container/LocalityManager.java |   2 +-
 .../grouper/task/GroupByContainerCount.java     |   6 +-
 .../grouper/task/GroupByContainerIds.java       |   4 +-
 .../task/SingleContainerGrouperFactory.java     |   2 +-
 .../grouper/task/TaskAssignmentManager.java     |   2 +-
 .../metadatastore/CoordinatorStreamStore.java   |   2 +-
 .../samza/execution/ExecutionPlanner.java       | 178 +++--
 .../org/apache/samza/execution/JobGraph.java    |  82 +-
 .../samza/execution/JobGraphJsonGenerator.java  |   4 +-
 .../org/apache/samza/execution/JobNode.java     |   5 +-
 .../apache/samza/job/model/ContainerModel.java  |  31 +-
 .../org/apache/samza/job/model/JobModel.java    |   2 -
 .../apache/samza/processor/StreamProcessor.java |   2 +-
 .../model/JsonContainerModelMixIn.java          |  38 +-
 .../serializers/model/JsonJobModelMixIn.java    |   2 +
 .../serializers/model/JsonTaskModelMixIn.java   |   2 +
 .../serializers/model/SamzaObjectMapper.java    |  41 +-
 .../apache/samza/storage/StorageRecovery.java   |   2 +-
 .../samza/system/inmemory/InMemoryManager.java  |   6 +-
 .../system/inmemory/InMemorySystemAdmin.java    |   4 -
 .../system/inmemory/InMemorySystemConsumer.java |   4 -
 .../system/inmemory/InMemorySystemFactory.java  |   4 -
 .../system/inmemory/InMemorySystemProducer.java |   4 -
 .../org/apache/samza/table/TableManager.java    |  34 +-
 .../table/remote/RemoteReadWriteTable.java      |   2 +-
 .../samza/table/remote/RemoteReadableTable.java |   2 +-
 .../table/remote/RemoteTableDescriptor.java     |  46 +-
 .../samza/table/remote/RemoteTableProvider.java |  42 +-
 .../samza/table/remote/TableReadFunction.java   |   7 +
 .../samza/table/remote/TableWriteFunction.java  |   7 +
 .../samza/table/retry/FailsafeAdapter.java      | 103 +++
 .../table/retry/RetriableReadFunction.java      | 102 +++
 .../table/retry/RetriableWriteFunction.java     | 120 +++
 .../apache/samza/table/retry/RetryMetrics.java  |  59 ++
 .../samza/table/retry/TableRetryPolicy.java     | 257 +++++++
 .../java/org/apache/samza/zk/ProcessorData.java |  19 +-
 .../samza/zk/ZkJobCoordinatorFactory.java       |  17 +-
 .../java/org/apache/samza/zk/ZkKeyBuilder.java  |  30 +-
 .../org/apache/samza/zk/ZkMetadataStore.java    | 132 ++++
 .../apache/samza/zk/ZkMetadataStoreFactory.java |  36 +
 .../org/apache/samza/config/JobConfig.scala     |   1 +
 .../apache/samza/container/SamzaContainer.scala |  26 +-
 .../samza/coordinator/JobModelManager.scala     |   2 +-
 .../grouper/task/TestGroupByContainerCount.java |  89 +--
 .../grouper/task/TestGroupByContainerIds.java   |  37 +-
 .../samza/container/mock/ContainerMocks.java    |   6 +-
 .../coordinator/JobModelManagerTestUtil.java    |  13 +-
 .../TestCoordinatorStreamStore.java             |   2 +-
 .../samza/execution/TestExecutionPlanner.java   | 100 +--
 .../apache/samza/execution/TestJobGraph.java    |  38 +-
 .../apache/samza/job/model/TestJobModel.java    |  50 ++
 .../operators/impl/TestOperatorImplGraph.java   |   8 +-
 .../model/TestSamzaObjectMapper.java            | 205 +++--
 .../apache/samza/table/TestTableManager.java    |  11 +-
 .../samza/table/remote/TestRemoteTable.java     | 116 ++-
 .../table/remote/TestRemoteTableDescriptor.java |  10 +-
 .../retry/TestRetriableTableFunctions.java      | 316 ++++++++
 .../samza/table/retry/TestTableRetryPolicy.java |  82 ++
 .../org/apache/samza/zk/TestZkKeyBuilder.java   |   2 +-
 .../apache/samza/zk/TestZkMetadataStore.java    | 121 +++
 .../samza/container/TestSamzaContainer.scala    | 741 ++++---------------
 .../samza/container/TestTaskInstance.scala      | 526 ++++---------
 .../TestTaskInstanceExceptionHandler.scala      | 144 ++++
 .../samza/coordinator/TestJobCoordinator.scala  |  12 +-
 .../samza/system/chooser/MockSystemAdmin.scala  |  30 +
 .../chooser/TestBootstrappingChooser.scala      |   3 +-
 .../system/chooser/TestDefaultChooser.scala     |   1 -
 .../samza/system/hdfs/HdfsSystemAdmin.java      |  12 +
 .../samza/sql/translator/JoinTranslator.java    |   4 +-
 .../samza/example/PageViewCounterExample.java   |   1 -
 .../samza/test/framework/StreamAssert.java      |  73 +-
 .../apache/samza/test/framework/TestRunner.java | 289 ++++----
 .../test/framework/stream/CollectionStream.java | 204 -----
 .../system/CollectionStreamSystemSpec.java      |  90 ---
 .../system/InMemoryInputDescriptor.java         |  42 ++
 .../system/InMemoryOutputDescriptor.java        |  46 ++
 .../system/InMemorySystemDescriptor.java        | 118 +++
 .../AsyncStreamTaskIntegrationTest.java         | 108 +--
 .../StreamApplicationIntegrationTest.java       |  45 +-
 .../framework/StreamTaskIntegrationTest.java    | 112 ++-
 .../processor/TestZkLocalApplicationRunner.java |   6 +-
 .../table/TestLocalTableWithSideInputs.java     |  38 +-
 .../samza/test/table/TestRemoteTable.java       |  27 +-
 .../table/TestTableDescriptorsProvider.java     |  18 +-
 .../webapp/TestApplicationMasterRestClient.java |   2 +-
 90 files changed, 3299 insertions(+), 2085 deletions(-)
----------------------------------------------------------------------



[06/29] samza git commit: SAMZA-1890: Fix the creation of MapFunction in TestExecutionPlanner.

Posted by bo...@apache.org.
SAMZA-1890: Fix the creation of MapFunction in TestExecutionPlanner.

dnishimura Kindly take a look.

Author: Pawas Chhokra <pc...@pchhokra-mn3.linkedin.biz>

Reviewers: Sanil Jain <sn...@linkedin.com>, Daniel Nishimura <dn...@linkedin.com>

Closes #648 from PawasChhokra/TestExecutionPlanner


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

Branch: refs/heads/NewKafkaSystemConsumer
Commit: c48bcd2e1d6892b236b232d8dc569d2388751a4c
Parents: 1755268
Author: Pawas Chhokra <pc...@linkedin.com>
Authored: Wed Sep 19 16:34:40 2018 -0700
Committer: Prateek Maheshwari <pm...@apache.org>
Committed: Wed Sep 19 16:34:40 2018 -0700

----------------------------------------------------------------------
 .../org/apache/samza/execution/TestExecutionPlanner.java     | 8 ++++----
 1 file changed, 4 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/samza/blob/c48bcd2e/samza-core/src/test/java/org/apache/samza/execution/TestExecutionPlanner.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/execution/TestExecutionPlanner.java b/samza-core/src/test/java/org/apache/samza/execution/TestExecutionPlanner.java
index c089225..ad6b386 100644
--- a/samza-core/src/test/java/org/apache/samza/execution/TestExecutionPlanner.java
+++ b/samza-core/src/test/java/org/apache/samza/execution/TestExecutionPlanner.java
@@ -178,12 +178,12 @@ public class TestExecutionPlanner {
         OutputStream<KV<Object, Object>> output2 = appDesc.getOutputStream(output2Descriptor);
 
         messageStream1.map(m -> m)
-          .filter(m -> true)
-          .window(Windows.keyedTumblingWindow(m -> m, Duration.ofMillis(8), mock(Serde.class), mock(Serde.class)), "w1");
+            .filter(m -> true)
+            .window(Windows.keyedTumblingWindow(m -> m, Duration.ofMillis(8), (Serde<KV<Object, Object>>) mock(Serde.class), (Serde<KV<Object, Object>>) mock(Serde.class)), "w1");
 
         messageStream2.map(m -> m)
-          .filter(m -> true)
-          .window(Windows.keyedTumblingWindow(m -> m, Duration.ofMillis(16), mock(Serde.class), mock(Serde.class)), "w2");
+            .filter(m -> true)
+            .window(Windows.keyedTumblingWindow(m -> m, Duration.ofMillis(16), (Serde<KV<Object, Object>>) mock(Serde.class), (Serde<KV<Object, Object>>) mock(Serde.class)), "w2");
 
         messageStream1.join(messageStream2, (JoinFunction<Object, KV<Object, Object>, KV<Object, Object>, KV<Object, Object>>) mock(JoinFunction.class),
           mock(Serde.class), mock(Serde.class), mock(Serde.class), Duration.ofMillis(1600), "j1").sendTo(output1);


[29/29] samza git commit: Merge branch 'NewConsumer2' of https://github.com/sborya/samza into NewKafkaSystemConsumer

Posted by bo...@apache.org.
Merge branch 'NewConsumer2' of https://github.com/sborya/samza into NewKafkaSystemConsumer


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

Branch: refs/heads/NewKafkaSystemConsumer
Commit: d1675dde0250cc9c290da21ad1e1baa4819d58f1
Parents: 3615963 da39af7
Author: Boris S <bs...@linkedin.com>
Authored: Tue Sep 25 17:19:54 2018 -0700
Committer: Boris S <bs...@linkedin.com>
Committed: Tue Sep 25 17:19:54 2018 -0700

----------------------------------------------------------------------
 .gitignore                                      |    3 +-
 build.gradle                                    |    2 +
 docs/_case-studies/TEMPLATE.md                  |   32 +
 docs/_case-studies/digitalsmiths.md             |   31 +
 docs/_case-studies/fortscale.md                 |   30 +
 docs/_case-studies/index.md                     |   49 +
 docs/_case-studies/intuit.md                    |   31 +
 docs/_case-studies/linkedin.md                  |   30 +
 docs/_case-studies/netflix.md                   |   30 +
 docs/_case-studies/optimizely.md                |   58 +
 docs/_case-studies/redfin.md                    |   30 +
 docs/_case-studies/state.md                     |   31 +
 docs/_case-studies/uber.md                      |   30 +
 docs/_committers/TEMPLATE.md                    |   30 +
 docs/_committers/angela-murrell.md              |   28 +
 docs/_committers/boris-shkolnik.md              |   28 +
 docs/_committers/chinmay-soman.md               |   28 +
 docs/_committers/chris-riccomini.md             |   28 +
 docs/_committers/garry-turkington.md            |   28 +
 docs/_committers/jagadish-venkatraman.md        |   28 +
 docs/_committers/jake-maes.md                   |   28 +
 docs/_committers/jakob-homan.md                 |   29 +
 docs/_committers/jay-kreps.md                   |   28 +
 docs/_committers/martin-kleppmann.md            |   28 +
 docs/_committers/navina-ramesh.md               |   28 +
 docs/_committers/prateek-maheshwari.md          |   28 +
 docs/_committers/sriram-subramanian.md          |   28 +
 docs/_committers/wei-song.md                    |   28 +
 docs/_committers/xinyu-liu.md                   |   28 +
 docs/_committers/yan-fang.md                    |   28 +
 docs/_committers/yi-pan.md                      |   28 +
 docs/_committers/zhijie-shen.md                 |   28 +
 docs/_config.yml                                |   17 +
 docs/_includes/footer.html                      |   92 +
 docs/_includes/main-navigation.html             |   39 +
 docs/_layouts/case-study.html                   |  150 ++
 docs/_layouts/default.html                      |  309 +--
 docs/_layouts/news.html                         |   81 +
 docs/_layouts/page.html                         |   96 +-
 docs/_layouts/talks-and-meetups.html            |   42 +
 docs/_meetups/dec-2018.md                       |   46 +
 docs/_meetups/july-2018.md                      |   56 +
 docs/_menu/index.html                           |  160 ++
 ...ncing-the-release-of-apache-samza--0.14.0.md |   76 +
 docs/_news/2018-04-28-something-else.md         |   36 +
 ...ncing-the-release-of-apache-samza--0.14.1.md |   83 +
 docs/_news/TEMPLATE.md                          |   38 +
 docs/_news/index.md                             |   54 +
 docs/_powered-by/TEMPLATE.md                    |   23 +
 docs/_powered-by/intuit.md                      |   22 +
 docs/_powered-by/linkedin.md                    |   22 +
 docs/_powered-by/mobileaware.md                 |   22 +
 docs/_releases/0.10.md                          |   21 +
 docs/_releases/0.11.md                          |   21 +
 docs/_releases/0.12.md                          |   21 +
 docs/_releases/0.13.md                          |   21 +
 docs/_releases/0.14.md                          |   21 +
 docs/_releases/0.7.0.md                         |   21 +
 docs/_releases/0.8.md                           |   21 +
 docs/_releases/0.9.md                           |   21 +
 docs/_releases/TEMPLATE.md                      |   21 +
 docs/_talks/TEMPLATE.md                         |   34 +
 docs/_talks/june-2018--bangalore-kafka-group.md |   30 +
 docs/_talks/november-2017-big-data-span-2017.md |   30 +
 .../november-2018--totally-awesome-summit.md    |   30 +
 ...ber-21-2017--dataworks-summit-sydney-2017.md |   33 +
 docs/community/committers-old.md                |  103 +
 docs/community/committers.html                  |   97 +
 docs/community/committers.md                    |  103 -
 docs/css/bootstrap.css.map                      |    1 -
 docs/css/bootstrap.min.css                      |    7 -
 docs/css/font-awesome.min.css                   |    4 -
 docs/css/google-fonts.css                       |   72 +
 docs/css/ionicons.min.css                       |   11 +
 docs/css/main.new.css                           | 2262 ++++++++++++++++++
 docs/css/ropa-sans.css                          |   25 -
 docs/fonts/ionicons.eot                         |  Bin 0 -> 112650 bytes
 docs/fonts/ionicons.svg                         |  713 ++++++
 docs/fonts/ionicons.ttf                         |  Bin 0 -> 112472 bytes
 docs/fonts/ionicons.woff                        |  Bin 0 -> 65912 bytes
 docs/fonts/ionicons.woff2                       |  Bin 0 -> 50632 bytes
 docs/img/favicon/apple-touch-icon-114x114.png   |  Bin 0 -> 14109 bytes
 docs/img/favicon/apple-touch-icon-120x120.png   |  Bin 0 -> 12879 bytes
 docs/img/favicon/apple-touch-icon-144x144.png   |  Bin 0 -> 18520 bytes
 docs/img/favicon/apple-touch-icon-152x152.png   |  Bin 0 -> 20884 bytes
 docs/img/favicon/apple-touch-icon-57x57.png     |  Bin 0 -> 5351 bytes
 docs/img/favicon/apple-touch-icon-60x60.png     |  Bin 0 -> 5270 bytes
 docs/img/favicon/apple-touch-icon-72x72.png     |  Bin 0 -> 7156 bytes
 docs/img/favicon/apple-touch-icon-76x76.png     |  Bin 0 -> 7707 bytes
 docs/img/favicon/favicon-128.png                |  Bin 0 -> 13302 bytes
 docs/img/favicon/favicon-16x16.png              |  Bin 0 -> 824 bytes
 docs/img/favicon/favicon-196x196.png            |  Bin 0 -> 30623 bytes
 docs/img/favicon/favicon-32x32.png              |  Bin 0 -> 2288 bytes
 docs/img/favicon/favicon-96x96.png              |  Bin 0 -> 10467 bytes
 docs/img/favicon/favicon.ico                    |  Bin 0 -> 34494 bytes
 docs/img/favicon/mstile-144x144.png             |  Bin 0 -> 18520 bytes
 docs/img/favicon/mstile-150x150.png             |  Bin 0 -> 43894 bytes
 docs/img/favicon/mstile-310x150.png             |  Bin 0 -> 102518 bytes
 docs/img/favicon/mstile-310x310.png             |  Bin 0 -> 192956 bytes
 docs/img/favicon/mstile-70x70.png               |  Bin 0 -> 13302 bytes
 docs/img/feather.gif                            |  Bin 0 -> 16647 bytes
 docs/img/samza-just-logo-transparent.png        |  Bin 0 -> 260503 bytes
 docs/img/samza-just-logo.svg                    |  128 +
 docs/img/samza-logo-no-text.png                 |  Bin 0 -> 93664 bytes
 docs/img/samza-logo.png                         |  Bin 0 -> 6412 bytes
 docs/img/samza-logo.svg                         |  174 ++
 docs/img/samza-logo@2x.png                      |  Bin 0 -> 16164 bytes
 docs/index.md                                   |   22 +-
 docs/index.md.bak                               |   39 +
 docs/js/bootstrap.min.js                        |    6 -
 docs/js/jquery-1.11.1.min.js                    |    4 -
 docs/js/jquery.tablesorter.min.js               |    4 -
 docs/js/main.new.js                             |  645 +++++
 docs/learn/tutorials/versioned/index.md         |    3 +-
 docs/meetups/index.html                         |  106 +
 docs/powered-by/index.html                      |   48 +
 docs/startup/preview/index.md                   |    2 +-
 .../startup/releases/versioned/release-notes.md |   17 +-
 docs/talks/index.html                           |   85 +
 .../samza/metadatastore/MetadataStore.java      |    6 +-
 .../apache/samza/operators/TableDescriptor.java |   13 +-
 .../samza/table/TableDescriptorsProvider.java   |    8 +-
 .../StreamApplicationDescriptorImpl.java        |    8 +
 .../apache/samza/container/LocalityManager.java |    2 +-
 .../grouper/task/GroupByContainerCount.java     |    6 +-
 .../grouper/task/GroupByContainerIds.java       |    4 +-
 .../task/SingleContainerGrouperFactory.java     |    2 +-
 .../grouper/task/TaskAssignmentManager.java     |    2 +-
 .../metadatastore/CoordinatorStreamStore.java   |    2 +-
 .../apache/samza/job/model/ContainerModel.java  |   31 +-
 .../org/apache/samza/job/model/JobModel.java    |    2 -
 .../samza/operators/BaseTableDescriptor.java    |   19 +-
 .../model/JsonContainerModelMixIn.java          |   38 +-
 .../serializers/model/JsonJobModelMixIn.java    |    2 +
 .../serializers/model/JsonTaskModelMixIn.java   |    2 +
 .../serializers/model/SamzaObjectMapper.java    |   41 +-
 .../apache/samza/storage/StorageRecovery.java   |    2 +-
 .../samza/system/inmemory/InMemoryManager.java  |    6 +-
 .../system/inmemory/InMemorySystemAdmin.java    |    4 -
 .../system/inmemory/InMemorySystemConsumer.java |    4 -
 .../system/inmemory/InMemorySystemFactory.java  |    4 -
 .../system/inmemory/InMemorySystemProducer.java |    4 -
 .../table/caching/CachingTableDescriptor.java   |   37 +-
 .../table/hybrid/BaseHybridTableDescriptor.java |   50 +
 .../table/remote/RemoteTableDescriptor.java     |   13 +-
 .../java/org/apache/samza/zk/ProcessorData.java |   19 +-
 .../samza/zk/ZkJobCoordinatorFactory.java       |   17 +-
 .../java/org/apache/samza/zk/ZkKeyBuilder.java  |   30 +-
 .../org/apache/samza/zk/ZkMetadataStore.java    |  132 +
 .../apache/samza/zk/ZkMetadataStoreFactory.java |   36 +
 .../apache/samza/container/SamzaContainer.scala |   28 +-
 .../samza/coordinator/JobModelManager.scala     |    2 +-
 .../grouper/task/TestGroupByContainerCount.java |   89 +-
 .../grouper/task/TestGroupByContainerIds.java   |   37 +-
 .../samza/container/mock/ContainerMocks.java    |    6 +-
 .../coordinator/JobModelManagerTestUtil.java    |   13 +-
 .../TestCoordinatorStreamStore.java             |    2 +-
 .../samza/execution/TestExecutionPlanner.java   |    8 +-
 .../apache/samza/job/model/TestJobModel.java    |   50 +
 .../operators/impl/TestOperatorImplGraph.java   |    8 +-
 .../model/TestSamzaObjectMapper.java            |  205 +-
 .../samza/table/caching/TestCachingTable.java   |   22 +-
 .../org/apache/samza/zk/TestZkKeyBuilder.java   |    2 +-
 .../apache/samza/zk/TestZkMetadataStore.java    |  121 +
 .../samza/container/TestSamzaContainer.scala    |  741 ++----
 .../samza/container/TestTaskInstance.scala      |  526 ++--
 .../TestTaskInstanceExceptionHandler.scala      |  144 ++
 .../samza/coordinator/TestJobCoordinator.scala  |   12 +-
 .../samza/system/chooser/MockSystemAdmin.scala  |   30 +
 .../chooser/TestBootstrappingChooser.scala      |    3 +-
 .../system/chooser/TestDefaultChooser.scala     |    1 -
 .../clients/consumer/KafkaConsumerConfig.java   |  194 --
 .../samza/config/KafkaConsumerConfig.java       |  210 ++
 .../samza/system/kafka/KafkaConsumerProxy.java  |  347 ++-
 .../samza/system/kafka/KafkaSystemConsumer.java |  202 +-
 .../kafka/KafkaSystemConsumerMetrics.scala      |    4 +-
 .../samza/system/kafka/KafkaSystemFactory.scala |    3 +-
 .../consumer/TestKafkaConsumerConfig.java       |  137 --
 .../samza/config/TestKafkaConsumerConfig.java   |  150 ++
 .../system/kafka/TestKafkaSystemAdminJava.java  |   18 +-
 .../system/kafka/TestKafkaSystemConsumer.java   |   52 +-
 .../kv/inmemory/InMemoryTableDescriptor.java    |   13 +-
 .../inmemory/TestInMemoryTableDescriptor.java   |    4 +-
 .../storage/kv/RocksDbTableDescriptor.java      |   13 +-
 .../storage/kv/TestRocksDbTableDescriptor.java  |   11 +-
 .../kv/BaseLocalStoreBackedTableDescriptor.java |   13 +-
 .../kv/LocalStoreBackedReadWriteTable.java      |    1 +
 .../samza/sql/data/RexToJavaCompiler.java       |    5 +-
 .../samza/sql/dsl/SamzaSqlDslConverter.java     |   96 +
 .../sql/dsl/SamzaSqlDslConverterFactory.java    |   33 +
 .../sql/impl/ConfigBasedIOResolverFactory.java  |    7 +-
 .../samza/sql/interfaces/DslConverter.java      |   37 +
 .../sql/interfaces/DslConverterFactory.java     |   36 +
 .../samza/sql/interfaces/SamzaSqlDriver.java    |   56 +
 .../interfaces/SamzaSqlJavaTypeFactoryImpl.java |   72 +
 .../samza/sql/runner/SamzaSqlApplication.java   |   30 +-
 .../sql/runner/SamzaSqlApplicationConfig.java   |  117 +-
 .../sql/runner/SamzaSqlApplicationRunner.java   |   41 +-
 .../samza/sql/testutil/SamzaSqlQueryParser.java |   21 +-
 .../samza/sql/translator/JoinTranslator.java    |    1 +
 .../samza/sql/translator/ModifyTranslator.java  |  117 +
 .../samza/sql/translator/QueryTranslator.java   |   90 +-
 .../samza/sql/translator/ScanTranslator.java    |   10 +-
 .../apache/samza/sql/e2e/TestSamzaSqlTable.java |    4 +-
 .../runner/TestSamzaSqlApplicationConfig.java   |   49 +-
 .../runner/TestSamzaSqlApplicationRunner.java   |    2 +-
 .../samza/sql/system/TestAvroSystemFactory.java |    3 +-
 .../samza/sql/testutil/SamzaSqlTestConfig.java  |    3 +
 .../sql/testutil/TestIOResolverFactory.java     |    7 +-
 .../sql/testutil/TestSamzaSqlFileParser.java    |    1 +
 .../sql/translator/TestQueryTranslator.java     |  345 ++-
 .../samza/example/PageViewCounterExample.java   |    1 -
 .../samza/test/framework/StreamAssert.java      |   73 +-
 .../apache/samza/test/framework/TestRunner.java |  286 ++-
 .../test/framework/stream/CollectionStream.java |  204 --
 .../system/CollectionStreamSystemSpec.java      |   90 -
 .../system/InMemoryInputDescriptor.java         |   42 +
 .../system/InMemoryOutputDescriptor.java        |   46 +
 .../system/InMemorySystemDescriptor.java        |  118 +
 .../AsyncStreamTaskIntegrationTest.java         |  108 +-
 .../StreamApplicationIntegrationTest.java       |   45 +-
 .../framework/StreamTaskIntegrationTest.java    |  112 +-
 .../test/samzasql/TestSamzaSqlEndToEnd.java     |   64 +-
 .../apache/samza/test/table/TestLocalTable.java |    9 +-
 .../table/TestLocalTableWithSideInputs.java     |   38 +-
 .../samza/test/table/TestRemoteTable.java       |   33 +-
 .../table/TestTableDescriptorsProvider.java     |   10 +-
 .../webapp/TestApplicationMasterRestClient.java |    2 +-
 228 files changed, 10629 insertions(+), 3216 deletions(-)
----------------------------------------------------------------------



[25/29] samza git commit: Merge branch 'master' of https://github.com/apache/samza

Posted by bo...@apache.org.
Merge branch 'master' of https://github.com/apache/samza


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

Branch: refs/heads/NewKafkaSystemConsumer
Commit: ceebdc3be011d2e5e503c9fcbd9a9038feadafd2
Parents: 927adff b8058af
Author: Boris S <bo...@apache.org>
Authored: Mon Sep 24 17:07:30 2018 -0700
Committer: Boris S <bo...@apache.org>
Committed: Mon Sep 24 17:07:30 2018 -0700

----------------------------------------------------------------------
 .gitignore                                      |    3 +-
 build.gradle                                    |    2 +
 docs/_case-studies/TEMPLATE.md                  |   32 +
 docs/_case-studies/digitalsmiths.md             |   31 +
 docs/_case-studies/fortscale.md                 |   30 +
 docs/_case-studies/index.md                     |   49 +
 docs/_case-studies/intuit.md                    |   31 +
 docs/_case-studies/linkedin.md                  |   30 +
 docs/_case-studies/netflix.md                   |   30 +
 docs/_case-studies/optimizely.md                |   58 +
 docs/_case-studies/redfin.md                    |   30 +
 docs/_case-studies/state.md                     |   31 +
 docs/_case-studies/uber.md                      |   30 +
 docs/_committers/TEMPLATE.md                    |   30 +
 docs/_committers/angela-murrell.md              |   28 +
 docs/_committers/boris-shkolnik.md              |   28 +
 docs/_committers/chinmay-soman.md               |   28 +
 docs/_committers/chris-riccomini.md             |   28 +
 docs/_committers/garry-turkington.md            |   28 +
 docs/_committers/jagadish-venkatraman.md        |   28 +
 docs/_committers/jake-maes.md                   |   28 +
 docs/_committers/jakob-homan.md                 |   29 +
 docs/_committers/jay-kreps.md                   |   28 +
 docs/_committers/martin-kleppmann.md            |   28 +
 docs/_committers/navina-ramesh.md               |   28 +
 docs/_committers/prateek-maheshwari.md          |   28 +
 docs/_committers/sriram-subramanian.md          |   28 +
 docs/_committers/wei-song.md                    |   28 +
 docs/_committers/xinyu-liu.md                   |   28 +
 docs/_committers/yan-fang.md                    |   28 +
 docs/_committers/yi-pan.md                      |   28 +
 docs/_committers/zhijie-shen.md                 |   28 +
 docs/_config.yml                                |   17 +
 docs/_includes/footer.html                      |   92 +
 docs/_includes/main-navigation.html             |   39 +
 docs/_layouts/case-study.html                   |  150 ++
 docs/_layouts/default.html                      |  309 +--
 docs/_layouts/news.html                         |   81 +
 docs/_layouts/page.html                         |   96 +-
 docs/_layouts/talks-and-meetups.html            |   42 +
 docs/_meetups/dec-2018.md                       |   46 +
 docs/_meetups/july-2018.md                      |   56 +
 docs/_menu/index.html                           |  160 ++
 ...ncing-the-release-of-apache-samza--0.14.0.md |   76 +
 docs/_news/2018-04-28-something-else.md         |   36 +
 ...ncing-the-release-of-apache-samza--0.14.1.md |   83 +
 docs/_news/TEMPLATE.md                          |   38 +
 docs/_news/index.md                             |   54 +
 docs/_powered-by/TEMPLATE.md                    |   23 +
 docs/_powered-by/intuit.md                      |   22 +
 docs/_powered-by/linkedin.md                    |   22 +
 docs/_powered-by/mobileaware.md                 |   22 +
 docs/_releases/0.10.md                          |   21 +
 docs/_releases/0.11.md                          |   21 +
 docs/_releases/0.12.md                          |   21 +
 docs/_releases/0.13.md                          |   21 +
 docs/_releases/0.14.md                          |   21 +
 docs/_releases/0.7.0.md                         |   21 +
 docs/_releases/0.8.md                           |   21 +
 docs/_releases/0.9.md                           |   21 +
 docs/_releases/TEMPLATE.md                      |   21 +
 docs/_talks/TEMPLATE.md                         |   34 +
 docs/_talks/june-2018--bangalore-kafka-group.md |   30 +
 docs/_talks/november-2017-big-data-span-2017.md |   30 +
 .../november-2018--totally-awesome-summit.md    |   30 +
 ...ber-21-2017--dataworks-summit-sydney-2017.md |   33 +
 docs/community/committers-old.md                |  103 +
 docs/community/committers.html                  |   97 +
 docs/community/committers.md                    |  103 -
 docs/css/bootstrap.css.map                      |    1 -
 docs/css/bootstrap.min.css                      |    7 -
 docs/css/font-awesome.min.css                   |    4 -
 docs/css/google-fonts.css                       |   72 +
 docs/css/ionicons.min.css                       |   11 +
 docs/css/main.new.css                           | 2262 ++++++++++++++++++
 docs/css/ropa-sans.css                          |   25 -
 docs/fonts/ionicons.eot                         |  Bin 0 -> 112650 bytes
 docs/fonts/ionicons.svg                         |  713 ++++++
 docs/fonts/ionicons.ttf                         |  Bin 0 -> 112472 bytes
 docs/fonts/ionicons.woff                        |  Bin 0 -> 65912 bytes
 docs/fonts/ionicons.woff2                       |  Bin 0 -> 50632 bytes
 docs/img/favicon/apple-touch-icon-114x114.png   |  Bin 0 -> 14109 bytes
 docs/img/favicon/apple-touch-icon-120x120.png   |  Bin 0 -> 12879 bytes
 docs/img/favicon/apple-touch-icon-144x144.png   |  Bin 0 -> 18520 bytes
 docs/img/favicon/apple-touch-icon-152x152.png   |  Bin 0 -> 20884 bytes
 docs/img/favicon/apple-touch-icon-57x57.png     |  Bin 0 -> 5351 bytes
 docs/img/favicon/apple-touch-icon-60x60.png     |  Bin 0 -> 5270 bytes
 docs/img/favicon/apple-touch-icon-72x72.png     |  Bin 0 -> 7156 bytes
 docs/img/favicon/apple-touch-icon-76x76.png     |  Bin 0 -> 7707 bytes
 docs/img/favicon/favicon-128.png                |  Bin 0 -> 13302 bytes
 docs/img/favicon/favicon-16x16.png              |  Bin 0 -> 824 bytes
 docs/img/favicon/favicon-196x196.png            |  Bin 0 -> 30623 bytes
 docs/img/favicon/favicon-32x32.png              |  Bin 0 -> 2288 bytes
 docs/img/favicon/favicon-96x96.png              |  Bin 0 -> 10467 bytes
 docs/img/favicon/favicon.ico                    |  Bin 0 -> 34494 bytes
 docs/img/favicon/mstile-144x144.png             |  Bin 0 -> 18520 bytes
 docs/img/favicon/mstile-150x150.png             |  Bin 0 -> 43894 bytes
 docs/img/favicon/mstile-310x150.png             |  Bin 0 -> 102518 bytes
 docs/img/favicon/mstile-310x310.png             |  Bin 0 -> 192956 bytes
 docs/img/favicon/mstile-70x70.png               |  Bin 0 -> 13302 bytes
 docs/img/feather.gif                            |  Bin 0 -> 16647 bytes
 docs/img/samza-just-logo-transparent.png        |  Bin 0 -> 260503 bytes
 docs/img/samza-just-logo.svg                    |  128 +
 docs/img/samza-logo-no-text.png                 |  Bin 0 -> 93664 bytes
 docs/img/samza-logo.png                         |  Bin 0 -> 6412 bytes
 docs/img/samza-logo.svg                         |  174 ++
 docs/img/samza-logo@2x.png                      |  Bin 0 -> 16164 bytes
 docs/index.md                                   |   22 +-
 docs/index.md.bak                               |   39 +
 docs/js/bootstrap.min.js                        |    6 -
 docs/js/jquery-1.11.1.min.js                    |    4 -
 docs/js/jquery.tablesorter.min.js               |    4 -
 docs/js/main.new.js                             |  645 +++++
 docs/learn/tutorials/versioned/index.md         |    3 +-
 docs/meetups/index.html                         |  106 +
 docs/powered-by/index.html                      |   48 +
 docs/startup/preview/index.md                   |    2 +-
 .../startup/releases/versioned/release-notes.md |   17 +-
 docs/talks/index.html                           |   85 +
 .../apache/samza/operators/TableDescriptor.java |   13 +-
 .../samza/table/TableDescriptorsProvider.java   |    8 +-
 .../StreamApplicationDescriptorImpl.java        |    8 +
 .../samza/operators/BaseTableDescriptor.java    |   19 +-
 .../table/caching/CachingTableDescriptor.java   |   37 +-
 .../table/hybrid/BaseHybridTableDescriptor.java |   50 +
 .../table/remote/RemoteTableDescriptor.java     |   13 +-
 .../samza/table/caching/TestCachingTable.java   |   22 +-
 .../kv/inmemory/InMemoryTableDescriptor.java    |   13 +-
 .../inmemory/TestInMemoryTableDescriptor.java   |    4 +-
 .../storage/kv/RocksDbTableDescriptor.java      |   13 +-
 .../storage/kv/TestRocksDbTableDescriptor.java  |   11 +-
 .../kv/BaseLocalStoreBackedTableDescriptor.java |   13 +-
 .../kv/LocalStoreBackedReadWriteTable.java      |    1 +
 .../samza/sql/data/RexToJavaCompiler.java       |    5 +-
 .../samza/sql/dsl/SamzaSqlDslConverter.java     |   96 +
 .../sql/dsl/SamzaSqlDslConverterFactory.java    |   33 +
 .../sql/impl/ConfigBasedIOResolverFactory.java  |    7 +-
 .../samza/sql/interfaces/DslConverter.java      |   37 +
 .../sql/interfaces/DslConverterFactory.java     |   36 +
 .../samza/sql/interfaces/SamzaSqlDriver.java    |   56 +
 .../interfaces/SamzaSqlJavaTypeFactoryImpl.java |   72 +
 .../samza/sql/runner/SamzaSqlApplication.java   |   30 +-
 .../sql/runner/SamzaSqlApplicationConfig.java   |  117 +-
 .../sql/runner/SamzaSqlApplicationRunner.java   |   41 +-
 .../samza/sql/testutil/SamzaSqlQueryParser.java |   21 +-
 .../samza/sql/translator/JoinTranslator.java    |    1 +
 .../samza/sql/translator/ModifyTranslator.java  |  117 +
 .../samza/sql/translator/QueryTranslator.java   |   90 +-
 .../samza/sql/translator/ScanTranslator.java    |   10 +-
 .../apache/samza/sql/e2e/TestSamzaSqlTable.java |    4 +-
 .../runner/TestSamzaSqlApplicationConfig.java   |   49 +-
 .../runner/TestSamzaSqlApplicationRunner.java   |    2 +-
 .../samza/sql/system/TestAvroSystemFactory.java |    3 +-
 .../samza/sql/testutil/SamzaSqlTestConfig.java  |    3 +
 .../sql/testutil/TestIOResolverFactory.java     |    7 +-
 .../sql/testutil/TestSamzaSqlFileParser.java    |    1 +
 .../sql/translator/TestQueryTranslator.java     |  345 ++-
 .../test/samzasql/TestSamzaSqlEndToEnd.java     |   64 +-
 .../apache/samza/test/table/TestLocalTable.java |    9 +-
 .../table/TestLocalTableWithSideInputs.java     |    6 +-
 .../samza/test/table/TestRemoteTable.java       |   33 +-
 .../table/TestTableDescriptorsProvider.java     |   10 +-
 162 files changed, 8230 insertions(+), 713 deletions(-)
----------------------------------------------------------------------



[12/29] samza git commit: Samza SQL: Code re-org to accomodate Samza SQL engine to take Calcite IR as input

Posted by bo...@apache.org.
Samza SQL: Code re-org to accomodate Samza SQL engine to take Calcite IR as input

This PR has the following changes:
- Let QueryTranslator take Calcite IR as input
- Include 'INSERT INTO' sql statement for Calcite plan
- Basic DSLConverter Framework with SamzaSQL dialect as an example
- Some fixes to stream-table join wrt Serde

Author: Aditya Toomula <at...@linkedin.com>

Reviewers: Srinivasulu <sp...@linkedin.com>, Weiqing <wi...@linkedin.com>

Closes #630 from atoomula/dsl3 and squashes the following commits:

93c66cee [Aditya Toomula] Samza SQL: Code re-org to accomodate Samza SQL engine to take Calcite IR as input.
21c0175b [Aditya Toomula] Samza SQL: Code re-org to accomodate Samza SQL engine to take Calcite IR as input.
15a1e9fb [Aditya Toomula] Samza SQL: Code re-org to accomodate Samza SQL engine to take Calcite IR as input.
5bf0c7e1 [Aditya Toomula] Samza SQL: Code re-org to accomodate Samza SQL engine to take Calcite IR as input.
98cd9777 [Aditya Toomula] Samza SQL: Code re-org to accomodate Samza SQL engine to take Calcite IR as input.
63a66fb1 [Aditya Toomula] Samza SQL: Code re-org to accomodate Samza SQL engine to take Calcite IR as input.
6794b512 [Aditya Toomula] Samza SQL: Code re-org to accomodate Samza SQL engine to take Calcite IR as input.
c9d434a9 [Aditya Toomula] Samza SQL: Code re-org to accomodate Samza SQL engine to take Calcite IR as input.
94e53b64 [Aditya Toomula] Samza SQL: Code re-org to accomodate Samza SQL engine to take Calcite IR as input.
30c76ed9 [Aditya Toomula] Samza SQL: Code re-org to accomodate Samza SQL engine to take Calcite IR as input.


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

Branch: refs/heads/NewKafkaSystemConsumer
Commit: dec16392de2f5d323b6b1b3acf8de1689038f44d
Parents: db6996e
Author: Aditya Toomula <at...@linkedin.com>
Authored: Thu Sep 20 14:22:38 2018 -0700
Committer: Srinivasulu Punuru <sp...@linkedin.com>
Committed: Thu Sep 20 14:22:38 2018 -0700

----------------------------------------------------------------------
 .../samza/sql/data/RexToJavaCompiler.java       |   5 +-
 .../samza/sql/dsl/SamzaSqlDslConverter.java     |  96 ++++++
 .../sql/dsl/SamzaSqlDslConverterFactory.java    |  33 ++
 .../samza/sql/interfaces/DslConverter.java      |  37 ++
 .../sql/interfaces/DslConverterFactory.java     |  36 ++
 .../samza/sql/interfaces/SamzaSqlDriver.java    |  56 +++
 .../interfaces/SamzaSqlJavaTypeFactoryImpl.java |  72 ++++
 .../samza/sql/runner/SamzaSqlApplication.java   |  30 +-
 .../sql/runner/SamzaSqlApplicationConfig.java   | 117 ++++---
 .../sql/runner/SamzaSqlApplicationRunner.java   |  41 ++-
 .../samza/sql/testutil/SamzaSqlQueryParser.java |  21 +-
 .../samza/sql/translator/JoinTranslator.java    |   1 +
 .../samza/sql/translator/ModifyTranslator.java  | 117 +++++++
 .../samza/sql/translator/QueryTranslator.java   |  90 ++---
 .../samza/sql/translator/ScanTranslator.java    |  10 +-
 .../apache/samza/sql/e2e/TestSamzaSqlTable.java |   4 +-
 .../runner/TestSamzaSqlApplicationConfig.java   |  49 ++-
 .../runner/TestSamzaSqlApplicationRunner.java   |   2 +-
 .../samza/sql/system/TestAvroSystemFactory.java |   3 +-
 .../samza/sql/testutil/SamzaSqlTestConfig.java  |   3 +
 .../sql/testutil/TestSamzaSqlFileParser.java    |   1 +
 .../sql/translator/TestQueryTranslator.java     | 345 +++++++++++++------
 .../test/samzasql/TestSamzaSqlEndToEnd.java     |  64 +++-
 23 files changed, 948 insertions(+), 285 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/samza/blob/dec16392/samza-sql/src/main/java/org/apache/samza/sql/data/RexToJavaCompiler.java
----------------------------------------------------------------------
diff --git a/samza-sql/src/main/java/org/apache/samza/sql/data/RexToJavaCompiler.java b/samza-sql/src/main/java/org/apache/samza/sql/data/RexToJavaCompiler.java
index 21c81a8..1cfa95f 100644
--- a/samza-sql/src/main/java/org/apache/samza/sql/data/RexToJavaCompiler.java
+++ b/samza-sql/src/main/java/org/apache/samza/sql/data/RexToJavaCompiler.java
@@ -49,6 +49,7 @@ import org.apache.calcite.rex.RexProgram;
 import org.apache.calcite.rex.RexProgramBuilder;
 import org.apache.calcite.util.Pair;
 import org.apache.samza.SamzaException;
+import org.apache.samza.sql.interfaces.SamzaSqlJavaTypeFactoryImpl;
 import org.codehaus.commons.compiler.CompileException;
 import org.codehaus.commons.compiler.CompilerFactoryFactory;
 import org.codehaus.commons.compiler.IClassBodyEvaluator;
@@ -114,11 +115,11 @@ public class RexToJavaCompiler {
     final ParameterExpression root = DataContext.ROOT;
     final ParameterExpression inputValues = Expressions.parameter(Object[].class, "inputValues");
     final ParameterExpression outputValues = Expressions.parameter(Object[].class, "outputValues");
-    final JavaTypeFactoryImpl javaTypeFactory = new JavaTypeFactoryImpl(rexBuilder.getTypeFactory().getTypeSystem());
+    final JavaTypeFactoryImpl javaTypeFactory = new SamzaSqlJavaTypeFactoryImpl(rexBuilder.getTypeFactory().getTypeSystem());
 
     // public void execute(Object[] inputValues, Object[] outputValues)
     final RexToLixTranslator.InputGetter inputGetter = new RexToLixTranslator.InputGetterImpl(ImmutableList.of(
-        Pair.<org.apache.calcite.linq4j.tree.Expression, PhysType>of(
+        Pair.of(
             Expressions.variable(Object[].class, "inputValues"),
             PhysTypeImpl.of(javaTypeFactory, inputRowType, JavaRowFormat.ARRAY, false))));
 

http://git-wip-us.apache.org/repos/asf/samza/blob/dec16392/samza-sql/src/main/java/org/apache/samza/sql/dsl/SamzaSqlDslConverter.java
----------------------------------------------------------------------
diff --git a/samza-sql/src/main/java/org/apache/samza/sql/dsl/SamzaSqlDslConverter.java b/samza-sql/src/main/java/org/apache/samza/sql/dsl/SamzaSqlDslConverter.java
new file mode 100644
index 0000000..4ec6f4a
--- /dev/null
+++ b/samza-sql/src/main/java/org/apache/samza/sql/dsl/SamzaSqlDslConverter.java
@@ -0,0 +1,96 @@
+/*
+* 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.sql.dsl;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.calcite.rel.RelRoot;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.samza.SamzaException;
+import org.apache.samza.config.Config;
+import org.apache.samza.sql.interfaces.DslConverter;
+import org.apache.samza.sql.planner.QueryPlanner;
+import org.apache.samza.sql.runner.SamzaSqlApplicationConfig;
+import org.apache.samza.sql.testutil.SamzaSqlQueryParser;
+import org.apache.samza.sql.testutil.SqlFileParser;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.samza.sql.runner.SamzaSqlApplicationConfig.*;
+
+
+public class SamzaSqlDslConverter implements DslConverter {
+
+  private static final Logger LOG = LoggerFactory.getLogger(SamzaSqlDslConverter.class);
+
+  private final Config config;
+
+  SamzaSqlDslConverter(Config config) {
+    this.config = config;
+  }
+
+  @Override
+  public Collection<RelRoot> convertDsl(String dsl) {
+    // TODO: Introduce an API to parse a dsl string and return one or more sql statements
+    List<String> sqlStmts = fetchSqlFromConfig(config);
+    List<SamzaSqlQueryParser.QueryInfo> queryInfo = fetchQueryInfo(sqlStmts);
+    SamzaSqlApplicationConfig sqlConfig = new SamzaSqlApplicationConfig(config,
+        queryInfo.stream().map(SamzaSqlQueryParser.QueryInfo::getSources).flatMap(Collection::stream)
+            .collect(Collectors.toSet()),
+        queryInfo.stream().map(SamzaSqlQueryParser.QueryInfo::getSink).collect(Collectors.toSet()));
+
+    QueryPlanner planner =
+        new QueryPlanner(sqlConfig.getRelSchemaProviders(), sqlConfig.getSystemStreamConfigsBySource(),
+            sqlConfig.getUdfMetadata());
+
+    List<RelRoot> relRoots = new LinkedList<>();
+    for (String sql: sqlStmts) {
+      relRoots.add(planner.plan(sql));
+    }
+    return relRoots;
+  }
+
+  public static List<SamzaSqlQueryParser.QueryInfo> fetchQueryInfo(List<String> sqlStmts) {
+    return sqlStmts.stream().map(SamzaSqlQueryParser::parseQuery).collect(Collectors.toList());
+  }
+
+  public static List<String> fetchSqlFromConfig(Map<String, String> config) {
+    List<String> sql;
+    if (config.containsKey(CFG_SQL_STMT) && StringUtils.isNotBlank(config.get(CFG_SQL_STMT))) {
+      String sqlValue = config.get(CFG_SQL_STMT);
+      sql = Collections.singletonList(sqlValue);
+    } else if (config.containsKey(CFG_SQL_STMTS_JSON) && StringUtils.isNotBlank(config.get(CFG_SQL_STMTS_JSON))) {
+      sql = deserializeSqlStmts(config.get(CFG_SQL_STMTS_JSON));
+    } else if (config.containsKey(CFG_SQL_FILE)) {
+      String sqlFile = config.get(CFG_SQL_FILE);
+      sql = SqlFileParser.parseSqlFile(sqlFile);
+    } else {
+      String msg = "Config doesn't contain the SQL that needs to be executed.";
+      LOG.error(msg);
+      throw new SamzaException(msg);
+    }
+
+    return sql;
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/dec16392/samza-sql/src/main/java/org/apache/samza/sql/dsl/SamzaSqlDslConverterFactory.java
----------------------------------------------------------------------
diff --git a/samza-sql/src/main/java/org/apache/samza/sql/dsl/SamzaSqlDslConverterFactory.java b/samza-sql/src/main/java/org/apache/samza/sql/dsl/SamzaSqlDslConverterFactory.java
new file mode 100644
index 0000000..5176453
--- /dev/null
+++ b/samza-sql/src/main/java/org/apache/samza/sql/dsl/SamzaSqlDslConverterFactory.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.sql.dsl;
+
+import org.apache.samza.config.Config;
+import org.apache.samza.sql.interfaces.DslConverter;
+import org.apache.samza.sql.interfaces.DslConverterFactory;
+
+
+public class SamzaSqlDslConverterFactory implements DslConverterFactory {
+
+  @Override
+  public DslConverter create(Config config) {
+    return new SamzaSqlDslConverter(config);
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/dec16392/samza-sql/src/main/java/org/apache/samza/sql/interfaces/DslConverter.java
----------------------------------------------------------------------
diff --git a/samza-sql/src/main/java/org/apache/samza/sql/interfaces/DslConverter.java b/samza-sql/src/main/java/org/apache/samza/sql/interfaces/DslConverter.java
new file mode 100644
index 0000000..fc2ca8e
--- /dev/null
+++ b/samza-sql/src/main/java/org/apache/samza/sql/interfaces/DslConverter.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.sql.interfaces;
+
+import java.util.Collection;
+import org.apache.calcite.rel.RelRoot;
+
+
+/**
+ * Samza SQL Application uses {@link DslConverter} to convert the input dsl to Calcite logical plan.
+ */
+public interface DslConverter {
+
+  /**
+   * Convert the dsl into the Calcite logical plan.
+   * @return List of Root nodes of the Calcite logical plan.
+   * If DSL represents multiple SQL statements. You might return root nodes one for each SQL statement.
+   */
+  Collection<RelRoot> convertDsl(String dsl);
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/dec16392/samza-sql/src/main/java/org/apache/samza/sql/interfaces/DslConverterFactory.java
----------------------------------------------------------------------
diff --git a/samza-sql/src/main/java/org/apache/samza/sql/interfaces/DslConverterFactory.java b/samza-sql/src/main/java/org/apache/samza/sql/interfaces/DslConverterFactory.java
new file mode 100644
index 0000000..d42a96f
--- /dev/null
+++ b/samza-sql/src/main/java/org/apache/samza/sql/interfaces/DslConverterFactory.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.sql.interfaces;
+
+import org.apache.samza.config.Config;
+
+
+/**
+ * Factory that is used to create {@link DslConverter}
+ */
+public interface DslConverterFactory {
+
+  /**
+   * Create a {@link DslConverter} given the config
+   * @param config config needed to create the {@link DslConverter}
+   * @return {@link DslConverter} object created.
+   */
+  DslConverter create(Config config);
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/dec16392/samza-sql/src/main/java/org/apache/samza/sql/interfaces/SamzaSqlDriver.java
----------------------------------------------------------------------
diff --git a/samza-sql/src/main/java/org/apache/samza/sql/interfaces/SamzaSqlDriver.java b/samza-sql/src/main/java/org/apache/samza/sql/interfaces/SamzaSqlDriver.java
new file mode 100644
index 0000000..5c86df9
--- /dev/null
+++ b/samza-sql/src/main/java/org/apache/samza/sql/interfaces/SamzaSqlDriver.java
@@ -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.sql.interfaces;
+
+import java.sql.Connection;
+import java.sql.SQLException;
+import java.util.List;
+import java.util.Properties;
+import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.avatica.AvaticaConnection;
+import org.apache.calcite.avatica.ConnectStringParser;
+import org.apache.calcite.avatica.Meta;
+import org.apache.calcite.jdbc.CalciteFactory;
+import org.apache.calcite.jdbc.Driver;
+
+
+/**
+ * Calcite JDBC driver for SamzaSQL which takes in a {@link JavaTypeFactory}
+ */
+public class SamzaSqlDriver extends Driver {
+
+  private JavaTypeFactory typeFactory;
+
+  public SamzaSqlDriver(JavaTypeFactory typeFactory) {
+    this.typeFactory = typeFactory;
+  }
+
+  @Override
+  public Connection connect(String url, Properties info) throws SQLException {
+    if (!acceptsURL(url)) {
+      return null;
+    }
+    final String prefix = getConnectStringPrefix();
+    assert url.startsWith(prefix);
+    final String urlSuffix = url.substring(prefix.length());
+    final Properties info2 = ConnectStringParser.parse(urlSuffix, info);
+    final AvaticaConnection connection =
+        ((CalciteFactory) factory).newConnection(this, factory, url, info2, null, typeFactory);
+    handler.onConnectionInit(connection);
+    return connection;
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/dec16392/samza-sql/src/main/java/org/apache/samza/sql/interfaces/SamzaSqlJavaTypeFactoryImpl.java
----------------------------------------------------------------------
diff --git a/samza-sql/src/main/java/org/apache/samza/sql/interfaces/SamzaSqlJavaTypeFactoryImpl.java b/samza-sql/src/main/java/org/apache/samza/sql/interfaces/SamzaSqlJavaTypeFactoryImpl.java
new file mode 100644
index 0000000..50001c6
--- /dev/null
+++ b/samza-sql/src/main/java/org/apache/samza/sql/interfaces/SamzaSqlJavaTypeFactoryImpl.java
@@ -0,0 +1,72 @@
+/*
+ * 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.sql.interfaces;
+
+import com.google.common.collect.Lists;
+import org.apache.calcite.jdbc.JavaTypeFactoryImpl;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rel.type.RelDataTypeSystem;
+import org.apache.calcite.rel.type.RelRecordType;
+import org.apache.calcite.sql.type.JavaToSqlTypeConversionRules;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+
+/**
+ * Calcite does validation of projected field types in select statement with the output schema types. If one of the
+ * projected fields is an UDF with return type of {@link Object} or any other java type not defined in
+ * {@link JavaToSqlTypeConversionRules}, using the default {@link JavaTypeFactoryImpl} results in validation failure.
+ * Hence, extending {@link JavaTypeFactoryImpl} to make Calcite validation work with all output types of Samza SQL UDFs.
+ */
+public class SamzaSqlJavaTypeFactoryImpl
+    extends JavaTypeFactoryImpl {
+
+  public SamzaSqlJavaTypeFactoryImpl() {
+    this(RelDataTypeSystem.DEFAULT);
+  }
+
+  public SamzaSqlJavaTypeFactoryImpl(RelDataTypeSystem typeSystem) {
+    super(typeSystem);
+  }
+
+  @Override
+  public RelDataType toSql(RelDataType type) {
+    return toSql(this, type);
+  }
+
+  /** Converts a type in Java format to a SQL-oriented type. */
+  public static RelDataType toSql(final RelDataTypeFactory typeFactory,
+      RelDataType type) {
+    if (type instanceof RelRecordType) {
+      return typeFactory.createStructType(
+          Lists.transform(type.getFieldList(), a0 -> toSql(typeFactory, a0.getType())),
+          type.getFieldNames());
+    }
+    if (type instanceof JavaType) {
+      SqlTypeName typeName = JavaToSqlTypeConversionRules.instance().lookup(((JavaType) type).getJavaClass());
+      // For unknown sql type names, return ANY sql type to make Calcite validation not fail.
+      if (typeName == null) {
+        typeName = SqlTypeName.ANY;
+      }
+      return typeFactory.createTypeWithNullability(
+          typeFactory.createSqlType(typeName),
+          type.isNullable());
+    } else {
+      return JavaTypeFactoryImpl.toSql(typeFactory, type);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/dec16392/samza-sql/src/main/java/org/apache/samza/sql/runner/SamzaSqlApplication.java
----------------------------------------------------------------------
diff --git a/samza-sql/src/main/java/org/apache/samza/sql/runner/SamzaSqlApplication.java b/samza-sql/src/main/java/org/apache/samza/sql/runner/SamzaSqlApplication.java
index 9a871d7..fd1a2a8 100644
--- a/samza-sql/src/main/java/org/apache/samza/sql/runner/SamzaSqlApplication.java
+++ b/samza-sql/src/main/java/org/apache/samza/sql/runner/SamzaSqlApplication.java
@@ -19,10 +19,14 @@
 
 package org.apache.samza.sql.runner;
 
+import java.util.Collection;
+import java.util.HashSet;
 import java.util.List;
+import java.util.Set;
+import org.apache.calcite.rel.RelRoot;
 import org.apache.samza.application.StreamApplication;
 import org.apache.samza.application.StreamApplicationDescriptor;
-import org.apache.samza.sql.testutil.SamzaSqlQueryParser;
+import org.apache.samza.sql.dsl.SamzaSqlDslConverter;
 import org.apache.samza.sql.translator.QueryTranslator;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -38,12 +42,26 @@ public class SamzaSqlApplication implements StreamApplication {
   @Override
   public void describe(StreamApplicationDescriptor appDesc) {
     try {
-      SamzaSqlApplicationConfig sqlConfig = new SamzaSqlApplicationConfig(appDesc.getConfig());
+      // TODO: Introduce an API to return a dsl string containing one or more sql statements.
+      List<String> dslStmts = SamzaSqlDslConverter.fetchSqlFromConfig(appDesc.getConfig());
+
+      // 1. Get Calcite plan
+      Set<String> inputSystemStreams = new HashSet<>();
+      Set<String> outputSystemStreams = new HashSet<>();
+
+      Collection<RelRoot> relRoots =
+          SamzaSqlApplicationConfig.populateSystemStreamsAndGetRelRoots(dslStmts, appDesc.getConfig(),
+              inputSystemStreams, outputSystemStreams);
+
+      // 2. Populate configs
+      SamzaSqlApplicationConfig sqlConfig =
+          new SamzaSqlApplicationConfig(appDesc.getConfig(), inputSystemStreams, outputSystemStreams);
+
+      // 3. Translate Calcite plan to Samza stream operators
       QueryTranslator queryTranslator = new QueryTranslator(sqlConfig);
-      List<SamzaSqlQueryParser.QueryInfo> queries = sqlConfig.getQueryInfo();
-      for (SamzaSqlQueryParser.QueryInfo query : queries) {
-        LOG.info("Translating the query {} to samza stream graph", query.getSelectQuery());
-        queryTranslator.translate(query, appDesc);
+      for (RelRoot relRoot : relRoots) {
+        LOG.info("Translating relRoot {} to samza stream graph", relRoot);
+        queryTranslator.translate(relRoot, appDesc);
       }
     } catch (RuntimeException e) {
       LOG.error("SamzaSqlApplication threw exception.", e);

http://git-wip-us.apache.org/repos/asf/samza/blob/dec16392/samza-sql/src/main/java/org/apache/samza/sql/runner/SamzaSqlApplicationConfig.java
----------------------------------------------------------------------
diff --git a/samza-sql/src/main/java/org/apache/samza/sql/runner/SamzaSqlApplicationConfig.java b/samza-sql/src/main/java/org/apache/samza/sql/runner/SamzaSqlApplicationConfig.java
index 997312f..415cfdd 100644
--- a/samza-sql/src/main/java/org/apache/samza/sql/runner/SamzaSqlApplicationConfig.java
+++ b/samza-sql/src/main/java/org/apache/samza/sql/runner/SamzaSqlApplicationConfig.java
@@ -20,7 +20,6 @@
 package org.apache.samza.sql.runner;
 
 import java.util.Collection;
-import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
@@ -30,12 +29,18 @@ import java.util.Set;
 import java.util.function.BiFunction;
 import java.util.function.Function;
 import java.util.stream.Collectors;
+import org.apache.calcite.rel.BiRel;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelRoot;
+import org.apache.calcite.rel.core.TableModify;
 import org.apache.commons.lang.StringUtils;
 import org.apache.commons.lang.Validate;
-import org.apache.samza.SamzaException;
 import org.apache.samza.config.Config;
 import org.apache.samza.config.MapConfig;
+import org.apache.samza.sql.dsl.SamzaSqlDslConverterFactory;
 import org.apache.samza.sql.impl.ConfigBasedUdfResolver;
+import org.apache.samza.sql.interfaces.DslConverter;
+import org.apache.samza.sql.interfaces.DslConverterFactory;
 import org.apache.samza.sql.interfaces.RelSchemaProvider;
 import org.apache.samza.sql.interfaces.RelSchemaProviderFactory;
 import org.apache.samza.sql.interfaces.SamzaRelConverter;
@@ -47,9 +52,6 @@ import org.apache.samza.sql.interfaces.UdfMetadata;
 import org.apache.samza.sql.interfaces.UdfResolver;
 import org.apache.samza.sql.testutil.JsonUtil;
 import org.apache.samza.sql.testutil.ReflectionUtils;
-import org.apache.samza.sql.testutil.SamzaSqlQueryParser;
-import org.apache.samza.sql.testutil.SamzaSqlQueryParser.QueryInfo;
-import org.apache.samza.sql.testutil.SqlFileParser;
 import org.codehaus.jackson.type.TypeReference;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -92,37 +94,25 @@ public class SamzaSqlApplicationConfig {
 
   private final Map<String, SqlIOConfig> inputSystemStreamConfigBySource;
   private final Map<String, SqlIOConfig> outputSystemStreamConfigsBySource;
-
-  private final List<String> sql;
-
-  private final List<QueryInfo> queryInfo;
+  private final Map<String, SqlIOConfig> systemStreamConfigsBySource;
 
   private final long windowDurationMs;
 
-  public SamzaSqlApplicationConfig(Config staticConfig) {
-
-    sql = fetchSqlFromConfig(staticConfig);
-
-    queryInfo = fetchQueryInfo(sql);
+  public SamzaSqlApplicationConfig(Config staticConfig, Set<String> inputSystemStreams,
+      Set<String> outputSystemStreams) {
 
     ioResolver = createIOResolver(staticConfig);
 
-    udfResolver = createUdfResolver(staticConfig);
-    udfMetadata = udfResolver.getUdfs();
+    inputSystemStreamConfigBySource = inputSystemStreams.stream()
+         .collect(Collectors.toMap(Function.identity(), src -> ioResolver.fetchSourceInfo(src)));
 
-    inputSystemStreamConfigBySource = queryInfo.stream()
-        .map(QueryInfo::getSources)
-        .flatMap(Collection::stream)
-        .distinct()
-        .collect(Collectors.toMap(Function.identity(), ioResolver::fetchSourceInfo));
+    outputSystemStreamConfigsBySource = outputSystemStreams.stream()
+         .collect(Collectors.toMap(Function.identity(), x -> ioResolver.fetchSinkInfo(x)));
 
-    Set<SqlIOConfig> systemStreamConfigs = new HashSet<>(inputSystemStreamConfigBySource.values());
+    systemStreamConfigsBySource = new HashMap<>(inputSystemStreamConfigBySource);
+    systemStreamConfigsBySource.putAll(outputSystemStreamConfigsBySource);
 
-    outputSystemStreamConfigsBySource = queryInfo.stream()
-        .map(QueryInfo::getSink)
-        .distinct()
-        .collect(Collectors.toMap(Function.identity(), ioResolver::fetchSinkInfo));
-    systemStreamConfigs.addAll(outputSystemStreamConfigsBySource.values());
+    Set<SqlIOConfig> systemStreamConfigs = new HashSet<>(systemStreamConfigsBySource.values());
 
     relSchemaProvidersBySource = systemStreamConfigs.stream()
         .collect(Collectors.toMap(SqlIOConfig::getSource,
@@ -136,6 +126,9 @@ public class SamzaSqlApplicationConfig {
                 CFG_FMT_SAMZA_REL_CONVERTER_DOMAIN, (o, c) -> ((SamzaRelConverterFactory) o).create(x.getSystemStream(),
                     relSchemaProvidersBySource.get(x.getSource()), c))));
 
+    udfResolver = createUdfResolver(staticConfig);
+    udfMetadata = udfResolver.getUdfs();
+
     windowDurationMs = staticConfig.getLong(CFG_GROUPBY_WINDOW_DURATION_MS, DEFAULT_GROUPBY_WINDOW_DURATION_MS);
   }
 
@@ -151,30 +144,7 @@ public class SamzaSqlApplicationConfig {
     return factoryInvoker.apply(factory, pluginConfig);
   }
 
-  public static List<QueryInfo> fetchQueryInfo(List<String> sqlStmts) {
-    return sqlStmts.stream().map(SamzaSqlQueryParser::parseQuery).collect(Collectors.toList());
-  }
-
-  public static List<String> fetchSqlFromConfig(Map<String, String> config) {
-    List<String> sql;
-    if (config.containsKey(CFG_SQL_STMT) && StringUtils.isNotBlank(config.get(CFG_SQL_STMT))) {
-      String sqlValue = config.get(CFG_SQL_STMT);
-      sql = Collections.singletonList(sqlValue);
-    } else if (config.containsKey(CFG_SQL_STMTS_JSON) && StringUtils.isNotBlank(config.get(CFG_SQL_STMTS_JSON))) {
-      sql = deserializeSqlStmts(config.get(CFG_SQL_STMTS_JSON));
-    } else if (config.containsKey(CFG_SQL_FILE)) {
-      String sqlFile = config.get(CFG_SQL_FILE);
-      sql = SqlFileParser.parseSqlFile(sqlFile);
-    } else {
-      String msg = "Config doesn't contain the SQL that needs to be executed.";
-      LOG.error(msg);
-      throw new SamzaException(msg);
-    }
-
-    return sql;
-  }
-
-  private static List<String> deserializeSqlStmts(String value) {
+  public static List<String> deserializeSqlStmts(String value) {
     Validate.notEmpty(value, "json Value is not set or empty");
     return JsonUtil.fromJson(value, new TypeReference<List<String>>() {
     });
@@ -224,12 +194,45 @@ public class SamzaSqlApplicationConfig {
     return ret;
   }
 
-  public List<String> getSql() {
-    return sql;
+  public static Collection<RelRoot> populateSystemStreamsAndGetRelRoots(List<String> dslStmts, Config config,
+      Set<String> inputSystemStreams, Set<String> outputSystemStreams) {
+    // TODO: Get the converter factory based on the file type. Create abstraction around this.
+    DslConverterFactory dslConverterFactory = new SamzaSqlDslConverterFactory();
+    DslConverter dslConverter = dslConverterFactory.create(config);
+
+    Collection<RelRoot> relRoots = dslConverter.convertDsl(String.join("\n", dslStmts));
+
+    for (RelRoot relRoot : relRoots) {
+      SamzaSqlApplicationConfig.populateSystemStreams(relRoot.project(), inputSystemStreams, outputSystemStreams);
+    }
+
+    return relRoots;
+  }
+
+  private static void populateSystemStreams(RelNode relNode, Set<String> inputSystemStreams,
+      Set<String> outputSystemStreams) {
+    if (relNode instanceof TableModify) {
+      outputSystemStreams.add(getSystemStreamName(relNode));
+    } else {
+      if (relNode instanceof BiRel) {
+        BiRel biRelNode = (BiRel) relNode;
+        populateSystemStreams(biRelNode.getLeft(), inputSystemStreams, outputSystemStreams);
+        populateSystemStreams(biRelNode.getRight(), inputSystemStreams, outputSystemStreams);
+      } else {
+        if (relNode.getTable() != null) {
+          inputSystemStreams.add(getSystemStreamName(relNode));
+        }
+      }
+    }
+     List<RelNode> relNodes = relNode.getInputs();
+    if (relNodes == null || relNodes.isEmpty()) {
+      return;
+    }
+    relNodes.forEach(node -> populateSystemStreams(node, inputSystemStreams, outputSystemStreams));
   }
 
-  public List<QueryInfo> getQueryInfo() {
-    return queryInfo;
+  private static String getSystemStreamName(RelNode relNode) {
+    return relNode.getTable().getQualifiedName().stream().map(Object::toString).collect(Collectors.joining("."));
   }
 
   public Collection<UdfMetadata> getUdfMetadata() {
@@ -244,6 +247,10 @@ public class SamzaSqlApplicationConfig {
     return outputSystemStreamConfigsBySource;
   }
 
+  public Map<String, SqlIOConfig> getSystemStreamConfigsBySource() {
+    return systemStreamConfigsBySource;
+  }
+
   public Map<String, SamzaRelConverter> getSamzaRelConverters() {
     return samzaRelConvertersBySource;
   }

http://git-wip-us.apache.org/repos/asf/samza/blob/dec16392/samza-sql/src/main/java/org/apache/samza/sql/runner/SamzaSqlApplicationRunner.java
----------------------------------------------------------------------
diff --git a/samza-sql/src/main/java/org/apache/samza/sql/runner/SamzaSqlApplicationRunner.java b/samza-sql/src/main/java/org/apache/samza/sql/runner/SamzaSqlApplicationRunner.java
index 027fd23..cad032f 100644
--- a/samza-sql/src/main/java/org/apache/samza/sql/runner/SamzaSqlApplicationRunner.java
+++ b/samza-sql/src/main/java/org/apache/samza/sql/runner/SamzaSqlApplicationRunner.java
@@ -21,20 +21,21 @@ package org.apache.samza.sql.runner;
 
 import java.time.Duration;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 import org.apache.commons.lang3.Validate;
 import org.apache.samza.config.Config;
 import org.apache.samza.config.MapConfig;
 import org.apache.samza.job.ApplicationStatus;
-import org.apache.samza.metrics.MetricsReporter;
 import org.apache.samza.runtime.ApplicationRunner;
 import org.apache.samza.runtime.ApplicationRunners;
 import org.apache.samza.runtime.LocalApplicationRunner;
 import org.apache.samza.runtime.RemoteApplicationRunner;
+import org.apache.samza.sql.dsl.SamzaSqlDslConverter;
 import org.apache.samza.sql.interfaces.SqlIOConfig;
 import org.apache.samza.sql.interfaces.SqlIOResolver;
-import org.apache.samza.sql.testutil.SamzaSqlQueryParser;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -63,25 +64,31 @@ public class SamzaSqlApplicationRunner implements ApplicationRunner {
   public static Config computeSamzaConfigs(Boolean localRunner, Config config) {
     Map<String, String> newConfig = new HashMap<>();
 
-    SqlIOResolver ioResolver = SamzaSqlApplicationConfig.createIOResolver(config);
-    // Parse the sql and find the input stream streams
-    List<String> sqlStmts = SamzaSqlApplicationConfig.fetchSqlFromConfig(config);
+    // TODO: Introduce an API to return a dsl string containing one or more sql statements
+    List<String> dslStmts = SamzaSqlDslConverter.fetchSqlFromConfig(config);
 
     // This is needed because the SQL file may not be available in all the node managers.
-    String sqlJson = SamzaSqlApplicationConfig.serializeSqlStmts(sqlStmts);
+    String sqlJson = SamzaSqlApplicationConfig.serializeSqlStmts(dslStmts);
     newConfig.put(SamzaSqlApplicationConfig.CFG_SQL_STMTS_JSON, sqlJson);
 
-    List<SamzaSqlQueryParser.QueryInfo> queryInfo = SamzaSqlApplicationConfig.fetchQueryInfo(sqlStmts);
-    for (SamzaSqlQueryParser.QueryInfo query : queryInfo) {
-      // Populate stream to system mapping config for input and output system streams
-      for (String inputSource : query.getSources()) {
-        SqlIOConfig inputSystemStreamConfig = ioResolver.fetchSourceInfo(inputSource);
-        newConfig.put(String.format(CFG_FMT_SAMZA_STREAM_SYSTEM, inputSystemStreamConfig.getStreamName()),
-            inputSystemStreamConfig.getSystemName());
-        newConfig.putAll(inputSystemStreamConfig.getConfig());
-      }
-
-      SqlIOConfig outputSystemStreamConfig = ioResolver.fetchSinkInfo(query.getSink());
+    Set<String> inputSystemStreams = new HashSet<>();
+    Set<String> outputSystemStreams = new HashSet<>();
+
+    SamzaSqlApplicationConfig.populateSystemStreamsAndGetRelRoots(dslStmts, config,
+        inputSystemStreams, outputSystemStreams);
+
+    SqlIOResolver ioResolver = SamzaSqlApplicationConfig.createIOResolver(config);
+
+    // Populate stream to system mapping config for input and output system streams
+    for (String source : inputSystemStreams) {
+      SqlIOConfig inputSystemStreamConfig = ioResolver.fetchSourceInfo(source);
+      newConfig.put(String.format(CFG_FMT_SAMZA_STREAM_SYSTEM, inputSystemStreamConfig.getStreamName()),
+          inputSystemStreamConfig.getSystemName());
+      newConfig.putAll(inputSystemStreamConfig.getConfig());
+    }
+
+    for (String sink : outputSystemStreams) {
+      SqlIOConfig outputSystemStreamConfig = ioResolver.fetchSinkInfo(sink);
       newConfig.put(String.format(CFG_FMT_SAMZA_STREAM_SYSTEM, outputSystemStreamConfig.getStreamName()),
           outputSystemStreamConfig.getSystemName());
       newConfig.putAll(outputSystemStreamConfig.getConfig());

http://git-wip-us.apache.org/repos/asf/samza/blob/dec16392/samza-sql/src/main/java/org/apache/samza/sql/testutil/SamzaSqlQueryParser.java
----------------------------------------------------------------------
diff --git a/samza-sql/src/main/java/org/apache/samza/sql/testutil/SamzaSqlQueryParser.java b/samza-sql/src/main/java/org/apache/samza/sql/testutil/SamzaSqlQueryParser.java
index 39ea092..643c82f 100644
--- a/samza-sql/src/main/java/org/apache/samza/sql/testutil/SamzaSqlQueryParser.java
+++ b/samza-sql/src/main/java/org/apache/samza/sql/testutil/SamzaSqlQueryParser.java
@@ -24,9 +24,11 @@ import java.sql.DriverManager;
 import java.sql.SQLException;
 import java.util.ArrayList;
 import java.util.List;
+import java.util.Properties;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
+import org.apache.calcite.adapter.java.JavaTypeFactory;
 import org.apache.calcite.config.Lex;
 import org.apache.calcite.jdbc.CalciteConnection;
 import org.apache.calcite.plan.Contexts;
@@ -49,6 +51,8 @@ import org.apache.calcite.tools.FrameworkConfig;
 import org.apache.calcite.tools.Frameworks;
 import org.apache.calcite.tools.Planner;
 import org.apache.samza.SamzaException;
+import org.apache.samza.sql.interfaces.SamzaSqlDriver;
+import org.apache.samza.sql.interfaces.SamzaSqlJavaTypeFactoryImpl;
 
 
 /**
@@ -63,11 +67,13 @@ public class SamzaSqlQueryParser {
     private final List<String> sources;
     private String selectQuery;
     private String sink;
+    private String sql;
 
-    public QueryInfo(String selectQuery, List<String> sources, String sink) {
+    public QueryInfo(String selectQuery, List<String> sources, String sink, String sql) {
       this.selectQuery = selectQuery;
       this.sink = sink;
       this.sources = sources;
+      this.sql = sql;
     }
 
     public List<String> getSources() {
@@ -81,6 +87,10 @@ public class SamzaSqlQueryParser {
     public String getSink() {
       return sink;
     }
+
+    public String getSql() {
+      return sql;
+    }
   }
 
   public static QueryInfo parseQuery(String sql) {
@@ -116,14 +126,18 @@ public class SamzaSqlQueryParser {
       throw new SamzaException("Sql query is not of the expected format");
     }
 
-    return new QueryInfo(selectQuery, sources, sink);
+    return new QueryInfo(selectQuery, sources, sink, sql);
   }
 
   private static Planner createPlanner() {
     Connection connection;
     SchemaPlus rootSchema;
     try {
-      connection = DriverManager.getConnection("jdbc:calcite:");
+      JavaTypeFactory typeFactory = new SamzaSqlJavaTypeFactoryImpl();
+      SamzaSqlDriver driver = new SamzaSqlDriver(typeFactory);
+      DriverManager.deregisterDriver(DriverManager.getDriver("jdbc:calcite:"));
+      DriverManager.registerDriver(driver);
+      connection = driver.connect("jdbc:calcite:", new Properties());
       CalciteConnection calciteConnection = connection.unwrap(CalciteConnection.class);
       rootSchema = calciteConnection.getRootSchema();
     } catch (SQLException e) {
@@ -174,7 +188,6 @@ public class SamzaSqlQueryParser {
         getSource(basicCall.operand(0), sourceList);
       } else if (basicCall.getOperator() instanceof SqlUnnestOperator && basicCall.operand(0) instanceof SqlSelect) {
         sourceList.addAll(getSourcesFromSelectQuery(basicCall.operand(0)));
-        return;
       }
     } else if (node instanceof SqlSelect) {
       getSource(((SqlSelect) node).getFrom(), sourceList);

http://git-wip-us.apache.org/repos/asf/samza/blob/dec16392/samza-sql/src/main/java/org/apache/samza/sql/translator/JoinTranslator.java
----------------------------------------------------------------------
diff --git a/samza-sql/src/main/java/org/apache/samza/sql/translator/JoinTranslator.java b/samza-sql/src/main/java/org/apache/samza/sql/translator/JoinTranslator.java
index 7071b39..ac2c64d 100644
--- a/samza-sql/src/main/java/org/apache/samza/sql/translator/JoinTranslator.java
+++ b/samza-sql/src/main/java/org/apache/samza/sql/translator/JoinTranslator.java
@@ -127,6 +127,7 @@ class JoinTranslator {
                 "stream_" + joinId)
             .map(KV::getValue)
             .join(table, joinFn);
+    // MessageStream<SamzaSqlRelMessage> outputStream = inputStream.join(table, joinFn);
 
     context.registerMessageStream(join.getId(), outputStream);
   }

http://git-wip-us.apache.org/repos/asf/samza/blob/dec16392/samza-sql/src/main/java/org/apache/samza/sql/translator/ModifyTranslator.java
----------------------------------------------------------------------
diff --git a/samza-sql/src/main/java/org/apache/samza/sql/translator/ModifyTranslator.java b/samza-sql/src/main/java/org/apache/samza/sql/translator/ModifyTranslator.java
new file mode 100644
index 0000000..965338f
--- /dev/null
+++ b/samza-sql/src/main/java/org/apache/samza/sql/translator/ModifyTranslator.java
@@ -0,0 +1,117 @@
+/*
+* 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.sql.translator;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import org.apache.calcite.rel.core.TableModify;
+import org.apache.commons.lang.Validate;
+import org.apache.samza.SamzaException;
+import org.apache.samza.application.StreamApplicationDescriptor;
+import org.apache.samza.config.Config;
+import org.apache.samza.operators.KV;
+import org.apache.samza.operators.MessageStream;
+import org.apache.samza.operators.MessageStreamImpl;
+import org.apache.samza.operators.TableDescriptor;
+import org.apache.samza.operators.descriptors.DelegatingSystemDescriptor;
+import org.apache.samza.operators.descriptors.GenericOutputDescriptor;
+import org.apache.samza.operators.functions.MapFunction;
+import org.apache.samza.serializers.KVSerde;
+import org.apache.samza.serializers.NoOpSerde;
+import org.apache.samza.sql.data.SamzaSqlRelMessage;
+import org.apache.samza.sql.interfaces.SamzaRelConverter;
+import org.apache.samza.sql.interfaces.SqlIOConfig;
+import org.apache.samza.table.Table;
+import org.apache.samza.task.TaskContext;
+
+
+/**
+ * Translator to translate the TableModify in relational graph to the corresponding output streams in the StreamGraph
+ * implementation
+ */
+class ModifyTranslator {
+
+  private final Map<String, SamzaRelConverter> relMsgConverters;
+  private final Map<String, SqlIOConfig> systemStreamConfig;
+
+  ModifyTranslator(Map<String, SamzaRelConverter> converters, Map<String, SqlIOConfig> ssc) {
+    relMsgConverters = converters;
+    this.systemStreamConfig = ssc;
+  }
+
+  // OutputMapFunction converts SamzaSqlRelMessage to SamzaMessage in KV format
+  private static class OutputMapFunction implements MapFunction<SamzaSqlRelMessage, KV<Object, Object>> {
+    // All the user-supplied functions are expected to be serializable in order to enable full serialization of user
+    // DAG. We do not want to serialize samzaMsgConverter as it can be fully constructed during stream operator
+    // initialization.
+    private transient SamzaRelConverter samzaMsgConverter;
+    private final String outputTopic;
+
+    OutputMapFunction(String outputTopic) {
+      this.outputTopic = outputTopic;
+    }
+
+    @Override
+    public void init(Config config, TaskContext taskContext) {
+      TranslatorContext context = (TranslatorContext) taskContext.getUserContext();
+      this.samzaMsgConverter = context.getMsgConverter(outputTopic);
+    }
+
+    @Override
+    public KV<Object, Object> apply(SamzaSqlRelMessage message) {
+      return this.samzaMsgConverter.convertToSamzaMessage(message);
+    }
+  }
+
+  void translate(final TableModify tableModify, final TranslatorContext context) {
+    StreamApplicationDescriptor streamAppDesc = context.getStreamAppDescriptor();
+    List<String> tableNameParts = tableModify.getTable().getQualifiedName();
+    String targetName = SqlIOConfig.getSourceFromSourceParts(tableNameParts);
+
+    Validate.isTrue(relMsgConverters.containsKey(targetName), String.format("Unknown source %s", targetName));
+
+    SqlIOConfig sinkConfig = systemStreamConfig.get(targetName);
+
+    final String systemName = sinkConfig.getSystemName();
+    final String streamName = sinkConfig.getStreamName();
+
+    KVSerde<Object, Object> noOpKVSerde = KVSerde.of(new NoOpSerde<>(), new NoOpSerde<>());
+    DelegatingSystemDescriptor
+        sd = context.getSystemDescriptors().computeIfAbsent(systemName, DelegatingSystemDescriptor::new);
+    GenericOutputDescriptor<KV<Object, Object>> osd = sd.getOutputDescriptor(streamName, noOpKVSerde);
+
+    MessageStreamImpl<SamzaSqlRelMessage> stream =
+        (MessageStreamImpl<SamzaSqlRelMessage>) context.getMessageStream(tableModify.getInput().getId());
+    MessageStream<KV<Object, Object>> outputStream = stream.map(new OutputMapFunction(targetName));
+
+    Optional<TableDescriptor> tableDescriptor = sinkConfig.getTableDescriptor();
+    if (!tableDescriptor.isPresent()) {
+      outputStream.sendTo(streamAppDesc.getOutputStream(osd));
+    } else {
+      Table outputTable = streamAppDesc.getTable(tableDescriptor.get());
+      if (outputTable == null) {
+        String msg = "Failed to obtain table descriptor of " + sinkConfig.getSource();
+        throw new SamzaException(msg);
+      }
+      outputStream.sendTo(outputTable);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/dec16392/samza-sql/src/main/java/org/apache/samza/sql/translator/QueryTranslator.java
----------------------------------------------------------------------
diff --git a/samza-sql/src/main/java/org/apache/samza/sql/translator/QueryTranslator.java b/samza-sql/src/main/java/org/apache/samza/sql/translator/QueryTranslator.java
index fe4d8da..3a35b97 100644
--- a/samza-sql/src/main/java/org/apache/samza/sql/translator/QueryTranslator.java
+++ b/samza-sql/src/main/java/org/apache/samza/sql/translator/QueryTranslator.java
@@ -20,10 +20,10 @@
 package org.apache.samza.sql.translator;
 
 import java.util.Map;
-import java.util.Optional;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.RelRoot;
 import org.apache.calcite.rel.RelShuttleImpl;
+import org.apache.calcite.rel.core.TableModify;
 import org.apache.calcite.rel.core.TableScan;
 import org.apache.calcite.rel.logical.LogicalAggregate;
 import org.apache.calcite.rel.logical.LogicalFilter;
@@ -33,27 +33,13 @@ import org.apache.samza.SamzaException;
 import org.apache.samza.application.StreamApplicationDescriptor;
 import org.apache.samza.config.Config;
 import org.apache.samza.operators.ContextManager;
-import org.apache.samza.operators.KV;
-import org.apache.samza.operators.MessageStream;
-import org.apache.samza.operators.MessageStreamImpl;
-import org.apache.samza.operators.TableDescriptor;
-import org.apache.samza.operators.functions.MapFunction;
-import org.apache.samza.operators.descriptors.GenericOutputDescriptor;;
-import org.apache.samza.operators.descriptors.DelegatingSystemDescriptor;
-import org.apache.samza.serializers.KVSerde;
-import org.apache.samza.serializers.NoOpSerde;
 import org.apache.samza.sql.data.SamzaSqlExecutionContext;
-import org.apache.samza.sql.data.SamzaSqlRelMessage;
 import org.apache.samza.sql.interfaces.SamzaRelConverter;
-import org.apache.samza.sql.interfaces.SqlIOConfig;
 import org.apache.samza.sql.interfaces.SqlIOResolver;
 import org.apache.samza.sql.planner.QueryPlanner;
 import org.apache.samza.sql.runner.SamzaSqlApplicationConfig;
 import org.apache.samza.sql.testutil.SamzaSqlQueryParser;
-import org.apache.samza.table.Table;
 import org.apache.samza.task.TaskContext;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 
 /**
@@ -62,54 +48,56 @@ import org.slf4j.LoggerFactory;
  * It then walks the relational graph and then populates the Samza's {@link StreamApplicationDescriptor} accordingly.
  */
 public class QueryTranslator {
-  private static final Logger LOG = LoggerFactory.getLogger(QueryTranslator.class);
-
   private final ScanTranslator scanTranslator;
+  private final ModifyTranslator modifyTranslator;
   private final SamzaSqlApplicationConfig sqlConfig;
   private final Map<String, SamzaRelConverter> converters;
 
-  private static class OutputMapFunction implements MapFunction<SamzaSqlRelMessage, KV<Object, Object>> {
-    private transient SamzaRelConverter samzaMsgConverter;
-    private final String outputTopic;
-
-    OutputMapFunction(String outputTopic) {
-      this.outputTopic = outputTopic;
-    }
-
-    @Override
-    public void init(Config config, TaskContext taskContext) {
-      TranslatorContext context = (TranslatorContext) taskContext.getUserContext();
-      this.samzaMsgConverter = context.getMsgConverter(outputTopic);
-    }
-
-    @Override
-    public KV<Object, Object> apply(SamzaSqlRelMessage message) {
-      return this.samzaMsgConverter.convertToSamzaMessage(message);
-    }
-  }
-
   public QueryTranslator(SamzaSqlApplicationConfig sqlConfig) {
     this.sqlConfig = sqlConfig;
     scanTranslator =
         new ScanTranslator(sqlConfig.getSamzaRelConverters(), sqlConfig.getInputSystemStreamConfigBySource());
+    modifyTranslator =
+        new ModifyTranslator(sqlConfig.getSamzaRelConverters(), sqlConfig.getOutputSystemStreamConfigsBySource());
     this.converters = sqlConfig.getSamzaRelConverters();
   }
 
   public void translate(SamzaSqlQueryParser.QueryInfo queryInfo, StreamApplicationDescriptor appDesc) {
     QueryPlanner planner =
-        new QueryPlanner(sqlConfig.getRelSchemaProviders(), sqlConfig.getInputSystemStreamConfigBySource(),
+        new QueryPlanner(sqlConfig.getRelSchemaProviders(), sqlConfig.getSystemStreamConfigsBySource(),
             sqlConfig.getUdfMetadata());
+    final RelRoot relRoot = planner.plan(queryInfo.getSql());
+    translate(relRoot, appDesc);
+  }
+
+  public void translate(RelRoot relRoot, StreamApplicationDescriptor appDesc) {
     final SamzaSqlExecutionContext executionContext = new SamzaSqlExecutionContext(this.sqlConfig);
-    final RelRoot relRoot = planner.plan(queryInfo.getSelectQuery());
     final TranslatorContext context = new TranslatorContext(appDesc, relRoot, executionContext, this.converters);
-    final RelNode node = relRoot.project();
     final SqlIOResolver ioResolver = context.getExecutionContext().getSamzaSqlApplicationConfig().getIoResolver();
+    final RelNode node = relRoot.project();
 
     node.accept(new RelShuttleImpl() {
       int windowId = 0;
       int joinId = 0;
 
       @Override
+      public RelNode visit(RelNode relNode) {
+        if (relNode instanceof TableModify) {
+          return visit((TableModify) relNode);
+        }
+        return super.visit(relNode);
+      }
+
+      private RelNode visit(TableModify modify) {
+        if (!modify.isInsert()) {
+          throw new SamzaException("Not a supported operation: " + modify.toString());
+        }
+        RelNode node = super.visit(modify);
+        modifyTranslator.translate(modify, context);
+        return node;
+      }
+
+      @Override
       public RelNode visit(TableScan scan) {
         RelNode node = super.visit(scan);
         scanTranslator.translate(scan, context);
@@ -147,28 +135,6 @@ public class QueryTranslator {
       }
     });
 
-    String sink = queryInfo.getSink();
-    SqlIOConfig sinkConfig = sqlConfig.getOutputSystemStreamConfigsBySource().get(sink);
-    MessageStreamImpl<SamzaSqlRelMessage> stream = (MessageStreamImpl<SamzaSqlRelMessage>) context.getMessageStream(node.getId());
-    MessageStream<KV<Object, Object>> outputStream = stream.map(new OutputMapFunction(sink));
-
-    Optional<TableDescriptor> tableDescriptor = sinkConfig.getTableDescriptor();
-    if (!tableDescriptor.isPresent()) {
-      KVSerde<Object, Object> noOpKVSerde = KVSerde.of(new NoOpSerde<>(), new NoOpSerde<>());
-      String systemName = sinkConfig.getSystemName();
-      DelegatingSystemDescriptor sd = context.getSystemDescriptors().computeIfAbsent(systemName, DelegatingSystemDescriptor::new);
-      GenericOutputDescriptor<KV<Object, Object>> osd = sd.getOutputDescriptor(sinkConfig.getStreamName(), noOpKVSerde);
-      outputStream.sendTo(appDesc.getOutputStream(osd));
-    } else {
-      Table outputTable = appDesc.getTable(tableDescriptor.get());
-      if (outputTable == null) {
-        String msg = "Failed to obtain table descriptor of " + sinkConfig.getSource();
-        LOG.error(msg);
-        throw new SamzaException(msg);
-      }
-      outputStream.sendTo(outputTable);
-    }
-
     appDesc.withContextManager(new ContextManager() {
       @Override
       public void init(Config config, TaskContext taskContext) {

http://git-wip-us.apache.org/repos/asf/samza/blob/dec16392/samza-sql/src/main/java/org/apache/samza/sql/translator/ScanTranslator.java
----------------------------------------------------------------------
diff --git a/samza-sql/src/main/java/org/apache/samza/sql/translator/ScanTranslator.java b/samza-sql/src/main/java/org/apache/samza/sql/translator/ScanTranslator.java
index 2dc28be..771a5d5 100644
--- a/samza-sql/src/main/java/org/apache/samza/sql/translator/ScanTranslator.java
+++ b/samza-sql/src/main/java/org/apache/samza/sql/translator/ScanTranslator.java
@@ -27,15 +27,15 @@ import org.apache.samza.application.StreamApplicationDescriptor;
 import org.apache.samza.config.Config;
 import org.apache.samza.operators.KV;
 import org.apache.samza.operators.MessageStream;
-import org.apache.samza.operators.descriptors.GenericInputDescriptor;
 import org.apache.samza.operators.descriptors.DelegatingSystemDescriptor;
+import org.apache.samza.operators.descriptors.GenericInputDescriptor;
 import org.apache.samza.operators.functions.MapFunction;
 import org.apache.samza.serializers.KVSerde;
 import org.apache.samza.serializers.NoOpSerde;
 import org.apache.samza.sql.data.SamzaSqlRelMessage;
 import org.apache.samza.sql.interfaces.SamzaRelConverter;
-import org.apache.samza.sql.interfaces.SqlIOConfig;
 import org.apache.samza.task.TaskContext;
+import org.apache.samza.sql.interfaces.SqlIOConfig;
 
 
 /**
@@ -53,6 +53,9 @@ class ScanTranslator {
   }
 
   private static class ScanMapFunction implements MapFunction<KV<Object, Object>, SamzaSqlRelMessage> {
+    // All the user-supplied functions are expected to be serializable in order to enable full serialization of user
+    // DAG. We do not want to serialize samzaMsgConverter as it can be fully constructed during stream operator
+    // initialization.
     private transient SamzaRelConverter msgConverter;
     private final String streamName;
 
@@ -83,7 +86,8 @@ class ScanTranslator {
     final String streamName = sqlIOConfig.getStreamName();
 
     KVSerde<Object, Object> noOpKVSerde = KVSerde.of(new NoOpSerde<>(), new NoOpSerde<>());
-    DelegatingSystemDescriptor sd = context.getSystemDescriptors().computeIfAbsent(systemName, DelegatingSystemDescriptor::new);
+    DelegatingSystemDescriptor
+        sd = context.getSystemDescriptors().computeIfAbsent(systemName, DelegatingSystemDescriptor::new);
     GenericInputDescriptor<KV<Object, Object>> isd = sd.getInputDescriptor(streamName, noOpKVSerde);
     MessageStream<KV<Object, Object>> inputStream = streamAppDesc.getInputStream(isd);
     MessageStream<SamzaSqlRelMessage> samzaSqlRelMessageStream = inputStream.map(new ScanMapFunction(sourceName));

http://git-wip-us.apache.org/repos/asf/samza/blob/dec16392/samza-sql/src/test/java/org/apache/samza/sql/e2e/TestSamzaSqlTable.java
----------------------------------------------------------------------
diff --git a/samza-sql/src/test/java/org/apache/samza/sql/e2e/TestSamzaSqlTable.java b/samza-sql/src/test/java/org/apache/samza/sql/e2e/TestSamzaSqlTable.java
index cc339f1..2005c21 100644
--- a/samza-sql/src/test/java/org/apache/samza/sql/e2e/TestSamzaSqlTable.java
+++ b/samza-sql/src/test/java/org/apache/samza/sql/e2e/TestSamzaSqlTable.java
@@ -42,7 +42,7 @@ public class TestSamzaSqlTable {
 
     Map<String, String> staticConfigs = SamzaSqlTestConfig.fetchStaticConfigsWithFactories(numMessages);
 
-    String sql1 = "Insert into testDb.testTable.`$table` select id, name from testavro.SIMPLE1";
+    String sql1 = "Insert into testDb.testTable.`$table`(id,name) select id, name from testavro.SIMPLE1";
     List<String> sqlStmts = Arrays.asList(sql1);
     staticConfigs.put(SamzaSqlApplicationConfig.CFG_SQL_STMTS_JSON, JsonUtil.toJson(sqlStmts));
     SamzaSqlApplicationRunner appRunnable = new SamzaSqlApplicationRunner(true, new MapConfig(staticConfigs));
@@ -58,7 +58,7 @@ public class TestSamzaSqlTable {
     TestIOResolverFactory.TestTable.records.clear();
     Map<String, String> staticConfigs = SamzaSqlTestConfig.fetchStaticConfigsWithFactories(numMessages);
 
-    String sql1 = "Insert into testDb.testTable.`$table` select id __key__, name from testavro.SIMPLE1";
+    String sql1 = "Insert into testDb.testTable.`$table`(id,name) select id __key__, name from testavro.SIMPLE1";
     List<String> sqlStmts = Arrays.asList(sql1);
     staticConfigs.put(SamzaSqlApplicationConfig.CFG_SQL_STMTS_JSON, JsonUtil.toJson(sqlStmts));
     SamzaSqlApplicationRunner appRunnable = new SamzaSqlApplicationRunner(true, new MapConfig(staticConfigs));

http://git-wip-us.apache.org/repos/asf/samza/blob/dec16392/samza-sql/src/test/java/org/apache/samza/sql/runner/TestSamzaSqlApplicationConfig.java
----------------------------------------------------------------------
diff --git a/samza-sql/src/test/java/org/apache/samza/sql/runner/TestSamzaSqlApplicationConfig.java b/samza-sql/src/test/java/org/apache/samza/sql/runner/TestSamzaSqlApplicationConfig.java
index dda0e14..46c0651 100644
--- a/samza-sql/src/test/java/org/apache/samza/sql/runner/TestSamzaSqlApplicationConfig.java
+++ b/samza-sql/src/test/java/org/apache/samza/sql/runner/TestSamzaSqlApplicationConfig.java
@@ -20,19 +20,25 @@
 package org.apache.samza.sql.runner;
 
 import java.util.Arrays;
+import java.util.Collection;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
+import java.util.stream.Collectors;
 import org.apache.samza.SamzaException;
 import org.apache.samza.config.MapConfig;
 import org.apache.samza.sql.impl.ConfigBasedUdfResolver;
 import org.apache.samza.sql.interfaces.SqlIOConfig;
 import org.apache.samza.sql.testutil.JsonUtil;
+import org.apache.samza.sql.testutil.SamzaSqlQueryParser;
 import org.apache.samza.sql.testutil.SamzaSqlTestConfig;
 import org.junit.Assert;
 import org.junit.Test;
 
+import static org.apache.samza.sql.dsl.SamzaSqlDslConverter.*;
+import static org.apache.samza.sql.runner.SamzaSqlApplicationConfig.*;
+
 
 public class TestSamzaSqlApplicationConfig {
 
@@ -42,8 +48,14 @@ public class TestSamzaSqlApplicationConfig {
     config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, "Insert into testavro.COMPLEX1 select * from testavro.SIMPLE1");
     String configUdfResolverDomain = String.format(SamzaSqlApplicationConfig.CFG_FMT_UDF_RESOLVER_DOMAIN, "config");
     int numUdfs = config.get(configUdfResolverDomain + ConfigBasedUdfResolver.CFG_UDF_CLASSES).split(",").length;
-    SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config));
-    Assert.assertEquals(1, samzaSqlApplicationConfig.getQueryInfo().size());
+
+    List<String> sqlStmts = fetchSqlFromConfig(config);
+    List<SamzaSqlQueryParser.QueryInfo> queryInfo = fetchQueryInfo(sqlStmts);
+    SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config),
+        queryInfo.stream().map(SamzaSqlQueryParser.QueryInfo::getSources).flatMap(Collection::stream)
+            .collect(Collectors.toSet()),
+        queryInfo.stream().map(SamzaSqlQueryParser.QueryInfo::getSink).collect(Collectors.toSet()));
+
     Assert.assertEquals(numUdfs, samzaSqlApplicationConfig.getUdfMetadata().size());
     Assert.assertEquals(1, samzaSqlApplicationConfig.getInputSystemStreamConfigBySource().size());
     Assert.assertEquals(1, samzaSqlApplicationConfig.getOutputSystemStreamConfigsBySource().size());
@@ -54,17 +66,23 @@ public class TestSamzaSqlApplicationConfig {
 
     Map<String, String> config = SamzaSqlTestConfig.fetchStaticConfigsWithFactories(10);
 
-
     try {
       // Fail because no SQL config
-      new SamzaSqlApplicationConfig(new MapConfig(config));
+      fetchSqlFromConfig(config);
       Assert.fail();
     } catch (SamzaException e) {
     }
 
     // Pass
     config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, "Insert into testavro.COMPLEX1 select * from testavro.SIMPLE1");
-    new SamzaSqlApplicationConfig(new MapConfig(config));
+
+    List<String> sqlStmts = fetchSqlFromConfig(config);
+    List<SamzaSqlQueryParser.QueryInfo> queryInfo = fetchQueryInfo(sqlStmts);
+    new SamzaSqlApplicationConfig(new MapConfig(config),
+        queryInfo.stream().map(SamzaSqlQueryParser.QueryInfo::getSources).flatMap(Collection::stream)
+            .collect(Collectors.toSet()),
+        queryInfo.stream().map(SamzaSqlQueryParser.QueryInfo::getSink).collect(Collectors.toSet()));
+
     testWithoutConfigShouldFail(config, SamzaSqlApplicationConfig.CFG_IO_RESOLVER);
     testWithoutConfigShouldFail(config, SamzaSqlApplicationConfig.CFG_UDF_RESOLVER);
 
@@ -85,7 +103,12 @@ public class TestSamzaSqlApplicationConfig {
         "insert into testavro.Profile select * from testavro.SIMPLE1");
     Map<String, String> config = SamzaSqlTestConfig.fetchStaticConfigsWithFactories(10);
     config.put(SamzaSqlApplicationConfig.CFG_SQL_STMTS_JSON, JsonUtil.toJson(sqlStmts));
-    SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config));
+
+    List<SamzaSqlQueryParser.QueryInfo> queryInfo = fetchQueryInfo(sqlStmts);
+    SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config),
+        queryInfo.stream().map(SamzaSqlQueryParser.QueryInfo::getSources).flatMap(Collection::stream)
+            .collect(Collectors.toSet()),
+        queryInfo.stream().map(SamzaSqlQueryParser.QueryInfo::getSink).collect(Collectors.toSet()));
 
     Set<String> inputKeys = samzaSqlApplicationConfig.getInputSystemStreamConfigBySource().keySet();
     Set<String> outputKeys = samzaSqlApplicationConfig.getOutputSystemStreamConfigsBySource().keySet();
@@ -99,14 +122,24 @@ public class TestSamzaSqlApplicationConfig {
   private void testWithoutConfigShouldPass(Map<String, String> config, String configKey) {
     Map<String, String> badConfigs = new HashMap<>(config);
     badConfigs.remove(configKey);
-    new SamzaSqlApplicationConfig(new MapConfig(badConfigs));
+    List<String> sqlStmts = fetchSqlFromConfig(badConfigs);
+    List<SamzaSqlQueryParser.QueryInfo> queryInfo = fetchQueryInfo(sqlStmts);
+    new SamzaSqlApplicationConfig(new MapConfig(badConfigs),
+        queryInfo.stream().map(SamzaSqlQueryParser.QueryInfo::getSources).flatMap(Collection::stream)
+            .collect(Collectors.toSet()),
+        queryInfo.stream().map(SamzaSqlQueryParser.QueryInfo::getSink).collect(Collectors.toSet()));
   }
 
   private void testWithoutConfigShouldFail(Map<String, String> config, String configKey) {
     Map<String, String> badConfigs = new HashMap<>(config);
     badConfigs.remove(configKey);
     try {
-      new SamzaSqlApplicationConfig(new MapConfig(badConfigs));
+      List<String> sqlStmts = fetchSqlFromConfig(badConfigs);
+      List<SamzaSqlQueryParser.QueryInfo> queryInfo = fetchQueryInfo(sqlStmts);
+      new SamzaSqlApplicationConfig(new MapConfig(badConfigs),
+          queryInfo.stream().map(SamzaSqlQueryParser.QueryInfo::getSources).flatMap(Collection::stream)
+              .collect(Collectors.toSet()),
+          queryInfo.stream().map(SamzaSqlQueryParser.QueryInfo::getSink).collect(Collectors.toSet()));
       Assert.fail();
     } catch (IllegalArgumentException e) {
       // swallow

http://git-wip-us.apache.org/repos/asf/samza/blob/dec16392/samza-sql/src/test/java/org/apache/samza/sql/runner/TestSamzaSqlApplicationRunner.java
----------------------------------------------------------------------
diff --git a/samza-sql/src/test/java/org/apache/samza/sql/runner/TestSamzaSqlApplicationRunner.java b/samza-sql/src/test/java/org/apache/samza/sql/runner/TestSamzaSqlApplicationRunner.java
index 9fab5d5..1ac804e 100644
--- a/samza-sql/src/test/java/org/apache/samza/sql/runner/TestSamzaSqlApplicationRunner.java
+++ b/samza-sql/src/test/java/org/apache/samza/sql/runner/TestSamzaSqlApplicationRunner.java
@@ -36,7 +36,7 @@ public class TestSamzaSqlApplicationRunner {
   @Test
   public void testComputeSamzaConfigs() {
     Map<String, String> configs = SamzaSqlTestConfig.fetchStaticConfigsWithFactories(10);
-    String sql1 = "Insert into testavro.outputTopic select id, MyTest(id) as long_value from testavro.SIMPLE1";
+    String sql1 = "Insert into testavro.outputTopic(id,long_value) select id, MyTest(id) as long_value from testavro.SIMPLE1";
     configs.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, sql1);
     configs.put(SamzaSqlApplicationRunner.RUNNER_CONFIG, SamzaSqlApplicationRunner.class.getName());
     MapConfig samzaConfig = new MapConfig(configs);

http://git-wip-us.apache.org/repos/asf/samza/blob/dec16392/samza-sql/src/test/java/org/apache/samza/sql/system/TestAvroSystemFactory.java
----------------------------------------------------------------------
diff --git a/samza-sql/src/test/java/org/apache/samza/sql/system/TestAvroSystemFactory.java b/samza-sql/src/test/java/org/apache/samza/sql/system/TestAvroSystemFactory.java
index 676781c..458196f 100644
--- a/samza-sql/src/test/java/org/apache/samza/sql/system/TestAvroSystemFactory.java
+++ b/samza-sql/src/test/java/org/apache/samza/sql/system/TestAvroSystemFactory.java
@@ -31,7 +31,6 @@ import java.util.stream.IntStream;
 
 import org.apache.avro.generic.GenericData;
 import org.apache.avro.generic.GenericRecord;
-import org.apache.calcite.avatica.util.ByteString;
 import org.apache.samza.config.Config;
 import org.apache.samza.metrics.MetricsRegistry;
 import org.apache.samza.sql.avro.schemas.AddressRecord;
@@ -50,7 +49,6 @@ import org.apache.samza.system.SystemAdmin;
 import org.apache.samza.system.SystemConsumer;
 import org.apache.samza.system.SystemFactory;
 import org.apache.samza.system.SystemProducer;
-import org.apache.samza.system.SystemStream;
 import org.apache.samza.system.SystemStreamPartition;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -73,6 +71,7 @@ public class TestAvroSystemFactory implements SystemFactory {
   public static final byte[] DEFAULT_TRACKING_ID_BYTES =
       {76, 75, -24, 10, 33, -117, 24, -52, -110, -39, -5, 102, 65, 57, -62, -1};
 
+
   public static List<OutgoingMessageEnvelope> messages = new ArrayList<>();
 
   public static List<String> getPageKeyProfileNameJoin(int numMessages) {

http://git-wip-us.apache.org/repos/asf/samza/blob/dec16392/samza-sql/src/test/java/org/apache/samza/sql/testutil/SamzaSqlTestConfig.java
----------------------------------------------------------------------
diff --git a/samza-sql/src/test/java/org/apache/samza/sql/testutil/SamzaSqlTestConfig.java b/samza-sql/src/test/java/org/apache/samza/sql/testutil/SamzaSqlTestConfig.java
index 14e2243..a96fd08 100644
--- a/samza-sql/src/test/java/org/apache/samza/sql/testutil/SamzaSqlTestConfig.java
+++ b/samza-sql/src/test/java/org/apache/samza/sql/testutil/SamzaSqlTestConfig.java
@@ -128,6 +128,9 @@ public class SamzaSqlTestConfig {
         "testavro", "SIMPLE1"), SimpleRecord.SCHEMA$.toString());
 
     staticConfigs.put(configAvroRelSchemaProviderDomain + String.format(ConfigBasedAvroRelSchemaProviderFactory.CFG_SOURCE_SCHEMA,
+        "testavro", "simpleOutputTopic"), SimpleRecord.SCHEMA$.toString());
+
+    staticConfigs.put(configAvroRelSchemaProviderDomain + String.format(ConfigBasedAvroRelSchemaProviderFactory.CFG_SOURCE_SCHEMA,
         "testavro", "outputTopic"), ComplexRecord.SCHEMA$.toString());
 
     staticConfigs.put(configAvroRelSchemaProviderDomain + String.format(ConfigBasedAvroRelSchemaProviderFactory.CFG_SOURCE_SCHEMA,

http://git-wip-us.apache.org/repos/asf/samza/blob/dec16392/samza-sql/src/test/java/org/apache/samza/sql/testutil/TestSamzaSqlFileParser.java
----------------------------------------------------------------------
diff --git a/samza-sql/src/test/java/org/apache/samza/sql/testutil/TestSamzaSqlFileParser.java b/samza-sql/src/test/java/org/apache/samza/sql/testutil/TestSamzaSqlFileParser.java
index 1723e0e..a84f347 100644
--- a/samza-sql/src/test/java/org/apache/samza/sql/testutil/TestSamzaSqlFileParser.java
+++ b/samza-sql/src/test/java/org/apache/samza/sql/testutil/TestSamzaSqlFileParser.java
@@ -24,6 +24,7 @@ import java.io.IOException;
 import java.io.PrintWriter;
 import java.util.List;
 
+import org.apache.samza.sql.testutil.SqlFileParser;
 import org.junit.Assert;
 import org.junit.Test;
 


[09/29] samza git commit: SAMZA-1849: Table Descriptors should take Serde at construction time so that descriptors can be typed

Posted by bo...@apache.org.
SAMZA-1849: Table Descriptors should take Serde at construction time so that descriptors can be typed

Changed table descriptor to take serde in constructor, and removed withSerde() from all table descriptors.

Author: Wei Song <ws...@linkedin.com>

Reviewers: Prateek Maheshwari <pm...@linkedin.com>

Closes #649 from weisong44/SAMZA-1849 and squashes the following commits:

a3ba2f70 [Wei Song] Merge branch 'master' into SAMZA-1849
41299b5b [Wei Song] Merge remote-tracking branch 'upstream/master'
e7a716c0 [Wei Song] Updated based on review comments
0601566f [Wei Song] SAMZA-1849: Table Descriptors should take Serde at construction time so that descriptors can be typed
239a0950 [Wei Song] Merge remote-tracking branch 'upstream/master'
eca00204 [Wei Song] Merge remote-tracking branch 'upstream/master'
51562391 [Wei Song] Merge remote-tracking branch 'upstream/master'
de708f5e [Wei Song] Merge remote-tracking branch 'upstream/master'
df2f8d7b [Wei Song] Merge remote-tracking branch 'upstream/master'
f28b491d [Wei Song] Merge remote-tracking branch 'upstream/master'
4782c61d [Wei Song] Merge remote-tracking branch 'upstream/master'
0440f75f [Wei Song] Merge remote-tracking branch 'upstream/master'
aae0f380 [Wei Song] Merge remote-tracking branch 'upstream/master'
a15a7c9a [Wei Song] Merge remote-tracking branch 'upstream/master'
5cbf9af9 [Wei Song] Merge remote-tracking branch 'upstream/master'
3f7ed71f [Wei Song] Added self to committer list


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

Branch: refs/heads/NewKafkaSystemConsumer
Commit: d89391231144b1700895673448a13d83b1c92a3a
Parents: c48bcd2
Author: Wei Song <ws...@linkedin.com>
Authored: Thu Sep 20 10:17:49 2018 -0700
Committer: Wei Song <ws...@linkedin.com>
Committed: Thu Sep 20 10:17:49 2018 -0700

----------------------------------------------------------------------
 .../apache/samza/operators/TableDescriptor.java  | 13 ++-----------
 .../samza/table/TableDescriptorsProvider.java    |  8 +++-----
 .../samza/operators/BaseTableDescriptor.java     | 19 ++++++++++---------
 .../table/remote/RemoteTableDescriptor.java      | 13 ++++++++++++-
 .../kv/inmemory/InMemoryTableDescriptor.java     | 13 ++++++++++++-
 .../kv/inmemory/TestInMemoryTableDescriptor.java |  4 ++--
 .../samza/storage/kv/RocksDbTableDescriptor.java | 13 ++++++++++++-
 .../storage/kv/TestRocksDbTableDescriptor.java   | 11 +++++------
 .../kv/BaseLocalStoreBackedTableDescriptor.java  | 13 +++++++++++--
 .../sql/impl/ConfigBasedIOResolverFactory.java   |  7 +++----
 .../sql/testutil/TestIOResolverFactory.java      |  7 +++----
 .../apache/samza/test/table/TestLocalTable.java  |  9 ++++-----
 .../test/table/TestLocalTableWithSideInputs.java |  6 ++----
 .../test/table/TestTableDescriptorsProvider.java | 10 ++++------
 14 files changed, 85 insertions(+), 61 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/samza/blob/d8939123/samza-api/src/main/java/org/apache/samza/operators/TableDescriptor.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/operators/TableDescriptor.java b/samza-api/src/main/java/org/apache/samza/operators/TableDescriptor.java
index a60b6a9..dbcd65e 100644
--- a/samza-api/src/main/java/org/apache/samza/operators/TableDescriptor.java
+++ b/samza-api/src/main/java/org/apache/samza/operators/TableDescriptor.java
@@ -19,7 +19,6 @@
 package org.apache.samza.operators;
 
 import org.apache.samza.annotation.InterfaceStability;
-import org.apache.samza.serializers.KVSerde;
 
 /**
  * User facing class to collect metadata that fully describes a
@@ -30,8 +29,8 @@ import org.apache.samza.serializers.KVSerde;
  *
  * <pre>
  * {@code
- * TableDescriptor<Integer, String, ?> tableDesc = new RocksDbTableDescriptor("tbl")
- *     .withSerde(KVSerde.of(new IntegerSerde(), new StringSerde("UTF-8")))
+ * TableDescriptor<Integer, String, ?> tableDesc = new RocksDbTableDescriptor("tbl",
+ *         KVSerde.of(new IntegerSerde(), new StringSerde("UTF-8")))
  *     .withBlockSize(1024)
  *     .withConfig("some-key", "some-value");
  * }
@@ -55,14 +54,6 @@ public interface TableDescriptor<K, V, D extends TableDescriptor<K, V, D>> {
   String getTableId();
 
   /**
-   * Set the Serde for this table
-   * @param serde the serde
-   * @return this table descriptor instance
-   * @throws IllegalArgumentException if null is provided
-   */
-  D withSerde(KVSerde<K, V> serde);
-
-  /**
    * Add a configuration entry for the table
    * @param key the key
    * @param value the value

http://git-wip-us.apache.org/repos/asf/samza/blob/d8939123/samza-api/src/main/java/org/apache/samza/table/TableDescriptorsProvider.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/table/TableDescriptorsProvider.java b/samza-api/src/main/java/org/apache/samza/table/TableDescriptorsProvider.java
index 766a4b4..5f8d766 100644
--- a/samza-api/src/main/java/org/apache/samza/table/TableDescriptorsProvider.java
+++ b/samza-api/src/main/java/org/apache/samza/table/TableDescriptorsProvider.java
@@ -44,13 +44,11 @@ import org.apache.samza.operators.TableDescriptor;
  *   public List<TableDescriptor> getTableDescriptors() {
  *     List<TableDescriptor> tableDescriptors = new ArrayList<>();
  *     final TableReadFunction readRemoteTableFn = new MyStoreReadFunction();
- *     tableDescriptors.add(new RemoteTableDescriptor<>("remote-table-1")
- *       .withReadFunction(readRemoteTableFn)
- *       .withSerde(KVSerde.of(new StringSerde(), new StringSerde())));
+ *     tableDescriptors.add(new RemoteTableDescriptor<>("remote-table-1", KVSerde.of(new StringSerde(), new StringSerde()))
+ *       .withReadFunction(readRemoteTableFn);
  *
- *     tableDescriptors.add(new RocksDbTableDescriptor("local-table-1")
+ *     tableDescriptors.add(new RocksDbTableDescriptor("local-table-1", KVSerde.of(new LongSerde(), new StringSerde<>()))
  *       .withBlockSize(4096)
- *       .withSerde(KVSerde.of(new LongSerde(), new StringSerde<>())));
  *       .withConfig("some-key", "some-value");
  *     return tableDescriptors;
  *   }

http://git-wip-us.apache.org/repos/asf/samza/blob/d8939123/samza-core/src/main/java/org/apache/samza/operators/BaseTableDescriptor.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/operators/BaseTableDescriptor.java b/samza-core/src/main/java/org/apache/samza/operators/BaseTableDescriptor.java
index f81f3b8..1e4194a 100644
--- a/samza-core/src/main/java/org/apache/samza/operators/BaseTableDescriptor.java
+++ b/samza-core/src/main/java/org/apache/samza/operators/BaseTableDescriptor.java
@@ -51,18 +51,19 @@ abstract public class BaseTableDescriptor<K, V, D extends BaseTableDescriptor<K,
     this.tableId = tableId;
   }
 
-  @Override
-  public D withConfig(String key, String value) {
-    config.put(key, value);
-    return (D) this;
+  /**
+   * Constructs a table descriptor instance
+   * @param tableId Id of the table, it must confirm to pattern { @literal [\\d\\w-_]+ }
+   * @param serde the serde for key and value
+   */
+  protected BaseTableDescriptor(String tableId, KVSerde<K, V> serde) {
+    this.tableId = tableId;
+    this.serde = serde;
   }
 
   @Override
-  public D withSerde(KVSerde<K, V> serde) {
-    if (serde == null) {
-      throw new IllegalArgumentException("Serde cannot be null");
-    }
-    this.serde = serde;
+  public D withConfig(String key, String value) {
+    config.put(key, value);
     return (D) this;
   }
 

http://git-wip-us.apache.org/repos/asf/samza/blob/d8939123/samza-core/src/main/java/org/apache/samza/table/remote/RemoteTableDescriptor.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/table/remote/RemoteTableDescriptor.java b/samza-core/src/main/java/org/apache/samza/table/remote/RemoteTableDescriptor.java
index 537ff87..c31348f 100644
--- a/samza-core/src/main/java/org/apache/samza/table/remote/RemoteTableDescriptor.java
+++ b/samza-core/src/main/java/org/apache/samza/table/remote/RemoteTableDescriptor.java
@@ -23,6 +23,7 @@ import java.util.HashMap;
 import java.util.Map;
 
 import org.apache.samza.operators.BaseTableDescriptor;
+import org.apache.samza.serializers.KVSerde;
 import org.apache.samza.table.TableSpec;
 import org.apache.samza.table.retry.TableRetryPolicy;
 import org.apache.samza.table.utils.SerdeUtils;
@@ -79,12 +80,22 @@ public class RemoteTableDescriptor<K, V> extends BaseTableDescriptor<K, V, Remot
   private int asyncCallbackPoolSize = -1;
 
   /**
-   * {@inheritDoc}
+   * Constructs a table descriptor instance
+   * @param tableId Id of the table, it must confirm to pattern { @literal [\\d\\w-_]+ }
    */
   public RemoteTableDescriptor(String tableId) {
     super(tableId);
   }
 
+  /**
+   * Constructs a table descriptor instance
+   * @param tableId Id of the table, it must confirm to pattern { @literal [\\d\\w-_]+ }
+   * @param serde the serde for key and value
+   */
+  public RemoteTableDescriptor(String tableId, KVSerde<K, V> serde) {
+    super(tableId, serde);
+  }
+
   @Override
   public TableSpec getTableSpec() {
     validate();

http://git-wip-us.apache.org/repos/asf/samza/blob/d8939123/samza-kv-inmemory/src/main/java/org/apache/samza/storage/kv/inmemory/InMemoryTableDescriptor.java
----------------------------------------------------------------------
diff --git a/samza-kv-inmemory/src/main/java/org/apache/samza/storage/kv/inmemory/InMemoryTableDescriptor.java b/samza-kv-inmemory/src/main/java/org/apache/samza/storage/kv/inmemory/InMemoryTableDescriptor.java
index 8328417..d364234 100644
--- a/samza-kv-inmemory/src/main/java/org/apache/samza/storage/kv/inmemory/InMemoryTableDescriptor.java
+++ b/samza-kv-inmemory/src/main/java/org/apache/samza/storage/kv/inmemory/InMemoryTableDescriptor.java
@@ -21,6 +21,7 @@ package org.apache.samza.storage.kv.inmemory;
 import java.util.HashMap;
 import java.util.Map;
 
+import org.apache.samza.serializers.KVSerde;
 import org.apache.samza.storage.kv.BaseLocalStoreBackedTableDescriptor;
 import org.apache.samza.table.TableSpec;
 
@@ -34,12 +35,22 @@ import org.apache.samza.table.TableSpec;
 public class InMemoryTableDescriptor<K, V> extends BaseLocalStoreBackedTableDescriptor<K, V, InMemoryTableDescriptor<K, V>> {
 
   /**
-   * {@inheritDoc}
+   * Constructs a table descriptor instance
+   * @param tableId Id of the table, it must confirm to pattern { @literal [\\d\\w-_]+ }
    */
   public InMemoryTableDescriptor(String tableId) {
     super(tableId);
   }
 
+  /**
+   * Constructs a table descriptor instance
+   * @param tableId Id of the table, it must confirm to pattern { @literal [\\d\\w-_]+ }
+   * @param serde the serde for key and value
+   */
+  public InMemoryTableDescriptor(String tableId, KVSerde<K, V> serde) {
+    super(tableId, serde);
+  }
+
   @Override
   protected void generateTableSpecConfig(Map<String, String> tableSpecConfig) {
     super.generateTableSpecConfig(tableSpecConfig);

http://git-wip-us.apache.org/repos/asf/samza/blob/d8939123/samza-kv-inmemory/src/test/java/org/apache/samza/storage/kv/inmemory/TestInMemoryTableDescriptor.java
----------------------------------------------------------------------
diff --git a/samza-kv-inmemory/src/test/java/org/apache/samza/storage/kv/inmemory/TestInMemoryTableDescriptor.java b/samza-kv-inmemory/src/test/java/org/apache/samza/storage/kv/inmemory/TestInMemoryTableDescriptor.java
index 840fb70..89bd058 100644
--- a/samza-kv-inmemory/src/test/java/org/apache/samza/storage/kv/inmemory/TestInMemoryTableDescriptor.java
+++ b/samza-kv-inmemory/src/test/java/org/apache/samza/storage/kv/inmemory/TestInMemoryTableDescriptor.java
@@ -31,8 +31,8 @@ public class TestInMemoryTableDescriptor {
   @Test
   public void testTableSpec() {
 
-    TableSpec tableSpec = new InMemoryTableDescriptor<Integer, String>("1")
-        .withSerde(KVSerde.of(new IntegerSerde(), new StringSerde()))
+    TableSpec tableSpec = new InMemoryTableDescriptor("1",
+            KVSerde.of(new IntegerSerde(), new StringSerde()))
         .withConfig("inmemory.abc", "xyz")
         .getTableSpec();
 

http://git-wip-us.apache.org/repos/asf/samza/blob/d8939123/samza-kv-rocksdb/src/main/java/org/apache/samza/storage/kv/RocksDbTableDescriptor.java
----------------------------------------------------------------------
diff --git a/samza-kv-rocksdb/src/main/java/org/apache/samza/storage/kv/RocksDbTableDescriptor.java b/samza-kv-rocksdb/src/main/java/org/apache/samza/storage/kv/RocksDbTableDescriptor.java
index 9b81605..325d023 100644
--- a/samza-kv-rocksdb/src/main/java/org/apache/samza/storage/kv/RocksDbTableDescriptor.java
+++ b/samza-kv-rocksdb/src/main/java/org/apache/samza/storage/kv/RocksDbTableDescriptor.java
@@ -21,6 +21,7 @@ package org.apache.samza.storage.kv;
 import java.util.HashMap;
 import java.util.Map;
 
+import org.apache.samza.serializers.KVSerde;
 import org.apache.samza.table.TableSpec;
 
 
@@ -57,13 +58,23 @@ public class RocksDbTableDescriptor<K, V> extends BaseLocalStoreBackedTableDescr
   private String compactionStyle;
 
   /**
-   * {@inheritDoc}
+   * Constructs a table descriptor instance
+   * @param tableId Id of the table, it must confirm to pattern { @literal [\\d\\w-_]+ }
    */
   public RocksDbTableDescriptor(String tableId) {
     super(tableId);
   }
 
   /**
+   * Constructs a table descriptor instance
+   * @param tableId Id of the table, it must confirm to pattern { @literal [\\d\\w-_]+ }
+   * @param serde the serde for key and value
+   */
+  public RocksDbTableDescriptor(String tableId, KVSerde<K, V> serde) {
+    super(tableId, serde);
+  }
+
+  /**
    * Refer to <code>stores.store-name.write.batch.size</code> in Samza configuration guide
    * @param writeBatchSize write batch size
    * @return this table descriptor instance

http://git-wip-us.apache.org/repos/asf/samza/blob/d8939123/samza-kv-rocksdb/src/test/java/org/apache/samza/storage/kv/TestRocksDbTableDescriptor.java
----------------------------------------------------------------------
diff --git a/samza-kv-rocksdb/src/test/java/org/apache/samza/storage/kv/TestRocksDbTableDescriptor.java b/samza-kv-rocksdb/src/test/java/org/apache/samza/storage/kv/TestRocksDbTableDescriptor.java
index 50f0920..35a66e8 100644
--- a/samza-kv-rocksdb/src/test/java/org/apache/samza/storage/kv/TestRocksDbTableDescriptor.java
+++ b/samza-kv-rocksdb/src/test/java/org/apache/samza/storage/kv/TestRocksDbTableDescriptor.java
@@ -37,8 +37,8 @@ public class TestRocksDbTableDescriptor {
 
   @Test
   public void testSerde() {
-    TableSpec tableSpec = new RocksDbTableDescriptor<Integer, String>("1")
-        .withSerde(KVSerde.of(new IntegerSerde(), new StringSerde()))
+    TableSpec tableSpec = new RocksDbTableDescriptor("1",
+            KVSerde.of(new IntegerSerde(), new StringSerde()))
         .getTableSpec();
     Assert.assertNotNull(tableSpec.getSerde());
     Assert.assertEquals(tableSpec.getSerde().getKeySerde().getClass(), IntegerSerde.class);
@@ -48,8 +48,8 @@ public class TestRocksDbTableDescriptor {
   @Test
   public void testTableSpec() {
 
-    TableSpec tableSpec = new RocksDbTableDescriptor<Integer, String>("1")
-        .withSerde(KVSerde.of(new IntegerSerde(), new StringSerde()))
+    TableSpec tableSpec = new RocksDbTableDescriptor<Integer, String>("1",
+            KVSerde.of(new IntegerSerde(), new StringSerde()))
         .withBlockSize(1)
         .withCacheSize(2)
         .withCompactionStyle("fifo")
@@ -85,8 +85,7 @@ public class TestRocksDbTableDescriptor {
   @Test
   public void testTableSpecWithChangelogEnabled() {
 
-    TableSpec tableSpec = new RocksDbTableDescriptor<Integer, String>("1")
-        .withSerde(KVSerde.of(new IntegerSerde(), new StringSerde()))
+    TableSpec tableSpec = new RocksDbTableDescriptor("1", KVSerde.of(new IntegerSerde(), new StringSerde()))
         .withChangelogStream("changelog-$tream")
         .withChangelogReplicationFactor(10)
         .getTableSpec();

http://git-wip-us.apache.org/repos/asf/samza/blob/d8939123/samza-kv/src/main/java/org/apache/samza/storage/kv/BaseLocalStoreBackedTableDescriptor.java
----------------------------------------------------------------------
diff --git a/samza-kv/src/main/java/org/apache/samza/storage/kv/BaseLocalStoreBackedTableDescriptor.java b/samza-kv/src/main/java/org/apache/samza/storage/kv/BaseLocalStoreBackedTableDescriptor.java
index c46f9e1..96057d6 100644
--- a/samza-kv/src/main/java/org/apache/samza/storage/kv/BaseLocalStoreBackedTableDescriptor.java
+++ b/samza-kv/src/main/java/org/apache/samza/storage/kv/BaseLocalStoreBackedTableDescriptor.java
@@ -24,6 +24,7 @@ import java.util.List;
 import java.util.Map;
 
 import org.apache.samza.operators.BaseTableDescriptor;
+import org.apache.samza.serializers.KVSerde;
 import org.apache.samza.storage.SideInputsProcessor;
 
 
@@ -49,13 +50,21 @@ abstract public class BaseLocalStoreBackedTableDescriptor<K, V, D extends BaseLo
 
   /**
    * Constructs a table descriptor instance
-   *
-   * @param tableId Id of the table
+   * @param tableId Id of the table, it must confirm to pattern { @literal [\\d\\w-_]+ }
    */
   public BaseLocalStoreBackedTableDescriptor(String tableId) {
     super(tableId);
   }
 
+  /**
+   * Constructs a table descriptor instance
+   * @param tableId Id of the table, it must confirm to pattern { @literal [\\d\\w-_]+ }
+   * @param serde the serde for key and value
+   */
+  public BaseLocalStoreBackedTableDescriptor(String tableId, KVSerde<K, V> serde) {
+    super(tableId, serde);
+  }
+
   public D withSideInputs(List<String> sideInputs) {
     this.sideInputs = sideInputs;
     // Disable changelog

http://git-wip-us.apache.org/repos/asf/samza/blob/d8939123/samza-sql/src/main/java/org/apache/samza/sql/impl/ConfigBasedIOResolverFactory.java
----------------------------------------------------------------------
diff --git a/samza-sql/src/main/java/org/apache/samza/sql/impl/ConfigBasedIOResolverFactory.java b/samza-sql/src/main/java/org/apache/samza/sql/impl/ConfigBasedIOResolverFactory.java
index 1ada813..a1c1bdd 100644
--- a/samza-sql/src/main/java/org/apache/samza/sql/impl/ConfigBasedIOResolverFactory.java
+++ b/samza-sql/src/main/java/org/apache/samza/sql/impl/ConfigBasedIOResolverFactory.java
@@ -100,10 +100,9 @@ public class ConfigBasedIOResolverFactory implements SqlIOResolverFactory {
 
       TableDescriptor tableDescriptor = null;
       if (isTable) {
-        tableDescriptor = new RocksDbTableDescriptor("InputTable-" + name)
-            .withSerde(KVSerde.of(
-                new JsonSerdeV2<>(SamzaSqlCompositeKey.class),
-                new JsonSerdeV2<>(SamzaSqlRelMessage.class)));
+        tableDescriptor = new RocksDbTableDescriptor("InputTable-" + name, KVSerde.of(
+            new JsonSerdeV2<>(SamzaSqlCompositeKey.class),
+            new JsonSerdeV2<>(SamzaSqlRelMessage.class)));
       }
 
       return new SqlIOConfig(systemName, streamName, fetchSystemConfigs(systemName), tableDescriptor);

http://git-wip-us.apache.org/repos/asf/samza/blob/d8939123/samza-sql/src/test/java/org/apache/samza/sql/testutil/TestIOResolverFactory.java
----------------------------------------------------------------------
diff --git a/samza-sql/src/test/java/org/apache/samza/sql/testutil/TestIOResolverFactory.java b/samza-sql/src/test/java/org/apache/samza/sql/testutil/TestIOResolverFactory.java
index bd61afd..8318e8a 100644
--- a/samza-sql/src/test/java/org/apache/samza/sql/testutil/TestIOResolverFactory.java
+++ b/samza-sql/src/test/java/org/apache/samza/sql/testutil/TestIOResolverFactory.java
@@ -202,10 +202,9 @@ public class TestIOResolverFactory implements SqlIOResolverFactory {
             tableDescriptor = new TestTableDescriptor(TEST_TABLE_ID + tableDescMap.size());
           } else {
             String tableId = "InputTable-" + ioName.replace(".", "-").replace("$", "-");
-            tableDescriptor = new RocksDbTableDescriptor(tableId)
-                .withSerde(KVSerde.of(
-                    new JsonSerdeV2<>(SamzaSqlCompositeKey.class),
-                    new JsonSerdeV2<>(SamzaSqlRelMessage.class)));
+            tableDescriptor = new RocksDbTableDescriptor(tableId, KVSerde.of(
+                new JsonSerdeV2<>(SamzaSqlCompositeKey.class),
+                new JsonSerdeV2<>(SamzaSqlRelMessage.class)));
           }
           tableDescMap.put(ioName, tableDescriptor);
         }

http://git-wip-us.apache.org/repos/asf/samza/blob/d8939123/samza-test/src/test/java/org/apache/samza/test/table/TestLocalTable.java
----------------------------------------------------------------------
diff --git a/samza-test/src/test/java/org/apache/samza/test/table/TestLocalTable.java b/samza-test/src/test/java/org/apache/samza/test/table/TestLocalTable.java
index fbf0539..e1386c8 100644
--- a/samza-test/src/test/java/org/apache/samza/test/table/TestLocalTable.java
+++ b/samza-test/src/test/java/org/apache/samza/test/table/TestLocalTable.java
@@ -96,8 +96,8 @@ public class TestLocalTable extends AbstractIntegrationTestHarness {
 
     final StreamApplication app = appDesc -> {
 
-      Table<KV<Integer, Profile>> table = appDesc.getTable(new InMemoryTableDescriptor("t1")
-          .withSerde(KVSerde.of(new IntegerSerde(), new ProfileJsonSerde())));
+      Table<KV<Integer, Profile>> table = appDesc.getTable(new InMemoryTableDescriptor("t1",
+          KVSerde.of(new IntegerSerde(), new ProfileJsonSerde())));
       DelegatingSystemDescriptor ksd = new DelegatingSystemDescriptor("test");
       GenericInputDescriptor<Profile> isd = ksd.getInputDescriptor("Profile", new NoOpSerde<>());
 
@@ -134,7 +134,7 @@ public class TestLocalTable extends AbstractIntegrationTestHarness {
       final StreamApplication app = appDesc -> {
 
         Table<KV<Integer, Profile>> table = appDesc.getTable(
-            new InMemoryTableDescriptor("t1").withSerde(KVSerde.of(new IntegerSerde(), new ProfileJsonSerde())));
+            new InMemoryTableDescriptor("t1", KVSerde.of(new IntegerSerde(), new ProfileJsonSerde())));
         DelegatingSystemDescriptor ksd = new DelegatingSystemDescriptor("test");
         GenericInputDescriptor<Profile> profileISD = ksd.getInputDescriptor("Profile", new NoOpSerde<>());
         appDesc.getInputStream(profileISD)
@@ -209,8 +209,7 @@ public class TestLocalTable extends AbstractIntegrationTestHarness {
 
       final StreamApplication app = appDesc -> {
 
-        Table<KV<Integer, Profile>> profileTable = appDesc.getTable(new InMemoryTableDescriptor("t1")
-            .withSerde(profileKVSerde));
+        Table<KV<Integer, Profile>> profileTable = appDesc.getTable(new InMemoryTableDescriptor("t1", profileKVSerde));
 
         DelegatingSystemDescriptor ksd = new DelegatingSystemDescriptor("test");
         GenericInputDescriptor<Profile> profileISD1 = ksd.getInputDescriptor("Profile1", new NoOpSerde<>());

http://git-wip-us.apache.org/repos/asf/samza/blob/d8939123/samza-test/src/test/java/org/apache/samza/test/table/TestLocalTableWithSideInputs.java
----------------------------------------------------------------------
diff --git a/samza-test/src/test/java/org/apache/samza/test/table/TestLocalTableWithSideInputs.java b/samza-test/src/test/java/org/apache/samza/test/table/TestLocalTableWithSideInputs.java
index adcea48..3c22818 100644
--- a/samza-test/src/test/java/org/apache/samza/test/table/TestLocalTableWithSideInputs.java
+++ b/samza-test/src/test/java/org/apache/samza/test/table/TestLocalTableWithSideInputs.java
@@ -143,8 +143,7 @@ public class TestLocalTableWithSideInputs extends AbstractIntegrationTestHarness
     }
 
     protected TableDescriptor<Integer, Profile, ?> getTableDescriptor() {
-      return new InMemoryTableDescriptor<Integer, Profile>(PROFILE_TABLE)
-          .withSerde(KVSerde.of(new IntegerSerde(), new ProfileJsonSerde()))
+      return new InMemoryTableDescriptor(PROFILE_TABLE, KVSerde.of(new IntegerSerde(), new ProfileJsonSerde()))
           .withSideInputs(ImmutableList.of(PROFILE_STREAM))
           .withSideInputsProcessor((msg, store) -> {
               Profile profile = (Profile) msg.getMessage();
@@ -158,8 +157,7 @@ public class TestLocalTableWithSideInputs extends AbstractIntegrationTestHarness
   static class DurablePageViewProfileJoin extends PageViewProfileJoin {
     @Override
     protected TableDescriptor<Integer, Profile, ?> getTableDescriptor() {
-      return new RocksDbTableDescriptor<Integer, Profile>(PROFILE_TABLE)
-          .withSerde(KVSerde.of(new IntegerSerde(), new ProfileJsonSerde()))
+      return new RocksDbTableDescriptor(PROFILE_TABLE, KVSerde.of(new IntegerSerde(), new ProfileJsonSerde()))
           .withSideInputs(ImmutableList.of(PROFILE_STREAM))
           .withSideInputsProcessor((msg, store) -> {
               TestTableData.Profile profile = (TestTableData.Profile) msg.getMessage();

http://git-wip-us.apache.org/repos/asf/samza/blob/d8939123/samza-test/src/test/java/org/apache/samza/test/table/TestTableDescriptorsProvider.java
----------------------------------------------------------------------
diff --git a/samza-test/src/test/java/org/apache/samza/test/table/TestTableDescriptorsProvider.java b/samza-test/src/test/java/org/apache/samza/test/table/TestTableDescriptorsProvider.java
index 34ffbd4..d123cee 100644
--- a/samza-test/src/test/java/org/apache/samza/test/table/TestTableDescriptorsProvider.java
+++ b/samza-test/src/test/java/org/apache/samza/test/table/TestTableDescriptorsProvider.java
@@ -137,13 +137,11 @@ public class TestTableDescriptorsProvider {
       final RateLimiter readRateLimiter = mock(RateLimiter.class);
       final MyReadFunction readFn = new MyReadFunction();
 
-      tableDescriptors.add(new RemoteTableDescriptor<>("remote-table-1")
+      tableDescriptors.add(new RemoteTableDescriptor<>("remote-table-1", KVSerde.of(new StringSerde(), new LongSerde()))
           .withReadFunction(readFn)
-          .withRateLimiter(readRateLimiter, null, null)
-          .withSerde(KVSerde.of(new StringSerde(), new LongSerde())));
-      tableDescriptors.add(new RocksDbTableDescriptor("local-table-1")
-          .withBlockSize(4096)
-          .withSerde(KVSerde.of(new StringSerde(), new StringSerde())));
+          .withRateLimiter(readRateLimiter, null, null));
+      tableDescriptors.add(new RocksDbTableDescriptor("local-table-1", KVSerde.of(new StringSerde(), new StringSerde()))
+          .withBlockSize(4096));
       return tableDescriptors;
     }
   }


[17/29] samza git commit: SAMZA-1898: New UI layout for the Samza website

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/samza/blob/b8058af0/docs/css/main.new.css
----------------------------------------------------------------------
diff --git a/docs/css/main.new.css b/docs/css/main.new.css
new file mode 100644
index 0000000..7b35809
--- /dev/null
+++ b/docs/css/main.new.css
@@ -0,0 +1,2262 @@
+/*
+ * 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.
+ */
+
+/********************************************************************
+ *
+ * REFERENCE 
+ *
+ *******************************************************************/
+ 
+ /* 
+ 
+ font-family: 'Barlow', sans-serif;
+ 
+ logo color red: #eb1c23
+ logo color grey: #4f4f4f
+ apache logo feather purple: #7a2c68
+ 
+ */
+
+/********************************************************************
+ *
+ * GLOBAL 
+ *
+ *******************************************************************/
+
+body {
+  font-family: 'Barlow', sans-serif;
+  font-size: 22px;
+  color: #444;
+  padding: 0px;
+  margin: 0px;
+}
+
+a {
+  color: inherit;
+  text-decoration: none;
+  position: relative;
+  color: #eb1c23;
+}
+
+a:hover {
+  text-decoration: none;
+}
+
+hr {
+  border: none;
+  border-top: 3px solid #eee;
+  margin: 60px auto;
+}
+
+hr::before {
+  height: 0px;
+  border-top: 3px solid #F1F1F1;
+  content: '';
+  display: block;
+  margin-top: 2px;
+}
+
+.content a::after,
+.section a::after,
+a.side-navigation__group-title::after {
+  display: block;
+  content: "";
+  position: absolute;
+  left: 0;
+  right: 0;
+  bottom: -5px;
+  border-bottom: solid 2px #693b6a;
+  transform: scaleX(0);
+  transition: transform 250ms ease-in-out;
+}
+
+.content a:hover::after,
+.section a:hover::after,
+.content a.active::after,
+.section a.active::after,
+a.side-navigation__group-title:hover::after {
+  transform: scaleX(1);
+}
+
+.content {
+  max-width: 1200px;
+  margin: auto;
+  padding: 20px;
+  line-height: 25px;
+}
+
+.content p {
+  font-size: 18px;
+  line-height: 25px;
+}
+
+.side-by-side {
+  display: flex;
+}
+
+.side-by-side > * {
+  width: 50%;
+  box-sizing: border-box;
+}
+
+.button {
+  background: #eb1c23;
+  padding: 20px 40px;
+  color: #fff;
+  font-weight: bold;
+  text-transform: uppercase;
+  font-size: 18px;
+  margin: 20px auto;
+  display: inline-block;
+  transition: background .3s ease 0s;
+}
+
+.button::after, .section .button::after {
+  border: none;
+}
+
+.button:hover {
+  background: #7a2c68;
+}
+
+/********************************************************************
+ *
+ * NAVIGATION - MAIN 
+ *
+ *******************************************************************/
+
+.main-navigation {
+  background: #fff;
+  box-shadow: 0px -30px 49px 0px rgba(0,0,0,0.4);
+  z-index: 40;
+  padding: 0 15px;
+}
+
+.home .main-navigation {
+  position: fixed;
+  top: 0;
+  left: 0;
+  right: 0;
+}
+
+.main-navigation__toggle {
+  position: absolute;
+  right: 10px;
+  top: 0;
+  font-size: 30px;
+  transform: translate(-50%, 25%);
+  transition: color .3s ease 0s;
+  cursor: pointer;
+  color: #eb1c23;
+}
+
+.main-navigation__toggle--opened {
+  display: none;
+}
+
+.main-navigation__toggle:hover {
+  color: #7a2c68;
+}
+
+@media only screen and (min-width: 768px) {
+  .main-navigation__toggle {
+    display: none;
+  }
+}
+
+.main-navigation__inner {
+  max-width: 1200px;
+  margin: auto;
+  display: flex;
+  justify-content: space-between;
+  align-items: left;
+  flex-direction: column;
+}
+
+@media only screen and (min-width: 768px) {
+  .main-navigation__inner {
+    flex-direction: row;
+    align-items: center;
+  }
+}
+
+.main-navigation__items {
+  display: flex;
+  justify-content: flex-end;
+  align-items: left;
+  flex-direction: column;
+  padding-bottom: 30px;
+  display: none;
+}
+
+@media only screen and (min-width: 768px) {
+  .main-navigation__items {
+    align-items: center;
+    flex-direction: row;
+    padding-bottom: 0px;
+    display: flex;
+    flex-grow: 1;
+    flex-basis: 0;
+  }
+}
+
+.main-navigation__item {
+  color: #693b6a;
+  display: block;
+  padding: 5px;
+  text-transform: uppercase;
+  font-size: 15px;
+  margin: 15px 20px 15px;
+  position: relative;
+  cursor: pointer;
+}
+
+.main-navigation__item::after {
+  display: block;
+  content: "";
+  position: absolute;
+  left: 0;
+  right: 0;
+  bottom: -2px;
+  border-bottom: solid 2px #eb1c23;
+  transform: scaleX(0);
+  transition: transform 250ms ease-in-out;
+}
+
+.main-navigation__item:hover::after {
+  transform: scaleX(1);
+}
+
+.main-navigation__logo-img {
+  height: 25px;
+  width: auto;
+  padding: 20px 0 15px;
+}
+
+/********************************************************************
+ *
+ * QUICK LINKS
+ *
+ *******************************************************************/
+
+.quick-links {
+  max-width: 1200px;
+  margin: auto;
+  display: flex;
+  justify-content: left;
+  margin-bottom: 20px;
+}
+
+.quick-link {
+  width: 40px;
+  cursor: pointer;
+  color: rgba(0,0,0,.4);
+  transition: color .3s ease 0s;
+}
+
+.quick-link:hover {
+  color: #693b6a;
+}
+.quick-link i {
+  font-size: 30px;
+}
+
+/********************************************************************
+ *
+ * FOOTER
+ *
+ *******************************************************************/
+
+footer {
+  padding: 70px 20px 100px;
+  color: #777;
+}
+
+footer p {
+  font-size: 15px;
+}
+
+.footer-inner {
+  max-width: 1200px;
+  margin: auto;
+  padding: 0 20px;
+}
+
+.footer__heading {
+  text-transform: uppercase;
+  padding-bottom: 5px;
+  position: relative;
+  margin-bottom: 30px;
+}
+
+.footer__heading::after {
+  content: '';
+  position: absolute;
+  left: 0;
+  width: 150px;
+  bottom: -5px;
+  border-bottom: 2px solid #999;
+}
+
+.footer__items {
+  
+}
+
+.footer__item {
+  display: block;
+  font-size: 15px;
+  margin-left: 30px;
+}
+
+.footer__item:hover {
+  color: #7a2c68;
+}
+
+.footer__item::before {
+  content: '»';
+  position: absolute;
+  left: -30px;
+}
+
+footer .side-by-side { 
+  flex-direction: column;
+}
+
+@media only screen and (min-width: 768px) {
+  footer .side-by-side {
+    flex-direction: row;
+  }
+}
+
+footer .side-by-side > * {
+  width: 100%;
+  box-sizing: border-box;
+  margin-bottom: 20px;
+}
+
+@media only screen and (min-width: 768px) {
+  footer .side-by-side > * {
+    width: 30%;
+    margin: 0px;
+  }
+}
+
+/********************************************************************
+ *
+ * SECTION - HOME PAGE 
+ *
+ *******************************************************************/
+
+.section {
+  background: #efefef;
+  align-items: center;
+  font-size: 30px;
+  padding: 20px 20px;
+  position: relative;
+  background-image: linear-gradient(to bottom, #ffffff, #fbfbfb, #f6f6f6, #f2f2f2, #eeeeee);
+}
+
+.section--highlight {
+  color: #fff;
+  background: #111;
+}
+
+.section--bottom-flare::after {
+  content: '';
+  position: absolute;
+  bottom: 0px;
+  left: 50%;
+  transform: translate(-50%, 50%);
+  border-left: 30px solid transparent;
+  border-right: 30px solid transparent;
+  border-top: 40px solid #eee;
+  width: 1px;
+  height: 1px;
+  z-index: 20;
+}
+
+.section--highlight.section--bottom-flare::after { 
+  border-top-color: #111;
+}
+
+.section--what-is-samza {
+  padding: 0px 20px 100px;
+}
+
+.section--hero {
+  padding: 100px 20px;
+  z-index: 10;
+}
+
+.section--highlight.section--features {
+  padding-bottom: 100px;
+}
+
+.section__title {
+  padding: 10px;
+  padding-top: 100px;
+  text-align: left;
+  font-size: 40px;
+  margin: auto;
+  max-width: 1200px;
+  font-weight: normal;
+}
+
+.section__title--right {
+  text-align: right;
+}
+
+.section__title--center {
+  text-align: center;
+}
+
+.section__title--sub {
+  padding-top: 10px;
+  font-size: 25px;
+  margin-bottom: 30px;
+}
+
+.section__items {
+  color: #999;
+  display: flex;
+  justify-content: center;
+  max-width: 1200px;
+  margin: auto;
+  margin-top: 50px;
+  flex-wrap: wrap;
+}
+
+@media only screen and (min-width: 768px) {
+  .section__items {
+    justify-content: space-between;
+  }
+}
+
+.section__item {
+  padding: 20px;
+  /* width: 100; */
+  margin: 10px;
+  margin-bottom: 40px;
+  box-sizing: border-box;
+  text-align: center;
+}
+
+@media only screen and (min-width: 768px) {
+  .section__item {
+    width: 30%;
+  }
+}
+
+.section--highlight .section__item {
+  color: #fff;
+}
+
+.section--highlight .section__item-title {
+  color: #fff;
+}
+
+.section__item-title {
+  text-transform: uppercase;
+  font-weight: bold;
+  margin-bottom: 20px;
+  font-size: 20px;
+  color: #444;
+}
+
+.section__item-icon {
+  color: #fff;
+  font-size: 55px;
+  border-radius: 100%;
+  width: 55px;
+  height: 55px;
+  padding: 10px;
+  display: flex;
+  justify-content: center;
+  align-items: center;
+  margin: auto;
+  margin-bottom: 20px;
+  background: linear-gradient(45deg, #ff4747 0%,#eb1c23 50%,#7a2c68 51%,#333333 100%); /* W3C, IE10+, FF16+, Chrome26+, Opera12+, Safari7+ */
+}
+
+.section__item-features {
+  text-align: left;
+}
+
+.section__item-feature {
+  font-size: 18px;
+  position: relative;
+  margin-left: 30px;
+}
+
+.section__item-feature::before {
+  content: '»';
+  position: absolute;
+  left: -30px;
+}
+
+.section__item-logo {
+  display: inline-block;
+}
+
+.section__item-logo-bg {
+  width: 128px;
+  height: 128px;
+  transition: all .3s ease 0s;
+  opacity: .8;
+  background-color: #fff;
+  background-size: contain;
+  background-repeat: no-repeat;
+  background-position: center;
+  background-blend-mode: luminosity;
+}
+
+.section__item-logo-bg:hover {
+  opacity: 1;
+  background-blend-mode: unset;
+}
+
+/********************************************************************
+ *
+ * SECTION - HOME PAGE - BRANDS
+ *
+ *******************************************************************/
+.section--brands .section__items {
+  justify-content: center;
+  margin-bottom: 100px;
+  font-size: 0;
+}
+
+.section--brands .section__item {
+  width: 128px;
+  margin: 0;
+  padding: 0;
+}
+
+.section.section--brands .section__item-logo::after {
+  border: none;
+}
+
+/********************************************************************
+ *
+ * WEIRD MESH NODE CSS THING
+ *
+ *******************************************************************/
+
+#canvas {
+	position: absolute;
+	display: block;
+	top: 0;
+	left: 0;
+  right: 0;
+  bottom: 0;
+  pointer-events: none;
+  z-index: -1;
+}
+
+
+/********************************************************************
+ *
+ * PAGE & SIDE NAVIGATION
+ *
+ *******************************************************************/
+
+.page .container {
+  max-width: 1200px;
+  margin: auto;
+  margin-top: 30px;
+  display: flex;
+  position: relative;
+}
+
+.page .container .container__toggle {
+  position: absolute;
+  left: 300px;
+  top: 100px;
+  height: 40px;
+  width: 40px;
+  display: flex;
+  justify-content: center;
+  align-items: center;
+  color: #fff;
+  border-radius: 100%;
+  background: #eb1c23;
+  z-index: 20;
+  transform: translateX(-50%);
+  transition: all .3s ease 0s;
+}
+
+@media only screen and (min-width: 768px) {
+  .page .container .container__toggle {
+    display: none;
+  }
+}
+
+.page .container.container--opened .container__toggle {
+  width: 70px;
+  height: 70px;
+  left: 5px;
+  font-size: 30px;
+}
+
+.page .container .container__toggle-icon--opened {
+  display: none;
+}
+
+.page .container.container--opened .container__toggle-icon {
+  display: none;
+}
+
+.page .container.container--opened .container__toggle-icon--opened {
+  display: block;
+}
+
+.page .side-navigation {
+  box-sizing: border-box;
+  width: 300px;
+  padding: 20px;
+  padding-right: 30px;
+  border-right: 3px solid #eee;
+  transition: width, transform .3s ease 0s;
+}
+
+
+
+.page .section {
+  box-sizing: border-box;
+  padding: 20px;
+  padding-top: 0px;
+  background: #fff;
+  width: calc(100% - 300px);
+  transition: width, transform .3s ease 0s;
+  overflow-x: hidden;
+  font-size: 22px;
+}
+
+@media only screen and (min-width:768px) {
+  .page .section {
+    font-size: 30px;
+  }
+}
+
+@media only screen and (max-width: 768px) {
+
+  .page .container.container--opened .container__toggle {
+    width: 70px;
+    height: 70px;
+    left: 5px;
+    font-size: 30px;
+  }
+
+  .page .container .container__toggle-icon--opened {
+    display: none;
+  }
+
+  .page .container.container--opened .container__toggle-icon {
+    display: none;
+  }
+
+  .page .container.container--opened .container__toggle-icon--opened {
+    display: block;
+  }
+
+  .page .container.container--opened .side-navigation {
+    transform: translateX(-330px);
+    width: 0;
+  }
+
+  .page .container.container--opened .section {
+    width: 100%;
+    transform: translateX(-20px);
+  }
+}
+
+
+.page .section .content {
+  max-width: unset;
+  margin: 0;
+  padding-top: 0px;
+  min-width: 500px;
+}
+
+.page .container.container--opened .content {
+  min-width: unset;
+}
+
+.page .section .content > h2:first-child {
+  margin-top: 15px;
+}
+
+.side-navigation hr {
+  margin: 5px auto;
+  margin-bottom: 5px;
+  border-top-width: 1px;
+  margin-bottom: 10px;
+}
+
+.side-navigation hr::before {
+  border: none;
+}
+
+.side-navigation__group {
+  margin-bottom: 20px;
+}
+
+.side-navigation__group--has-nested {
+  cursor: pointer;
+}
+
+.side-navigation__group--has-nested > .side-navigation__group-title {
+  display: flex;
+  flex-direction: row-reverse;
+  justify-content: space-between;
+}
+
+.side-navigation__group--has-nested > .side-navigation__group-title .side-navigation__group-title-icon {
+}
+
+.side-navigation__group--has-nested > .side-navigation__group-items {
+  display: none;
+}
+
+.side-navigation__group--has-nested .side-navigation__group-items {
+  border-left: 2px solid #eee;
+  padding-left: 10px;
+}
+
+.side-navigation__group-has-submenus {
+  /* background: #f4f4f4; */
+  padding: 15px;
+  padding-top: 0px;
+  margin-left: 20px;
+  border-radius: 2px;
+  margin-top: 30px;
+}
+
+.side-navigation__group--has-nested-visible > .side-navigation__group-title {
+  font-weight: bold;
+  position: relative;
+}
+.side-navigation__group--has-nested-visible > .side-navigation__group-items {
+  display: block;
+  margin-left: 10px;
+}
+
+.side-navigation__group-title {
+  margin-bottom: 15px;
+  text-transform: uppercase;
+  display: block;
+  font-size: 18px;
+}
+
+a.side-navigation__group-title::after {
+  bottom: -6px;
+  border-bottom-color: #999;
+  right: unset;
+  width: 100px;
+}
+
+.side-navigation__group-item {
+  font-size: 15px;
+  display: block;
+  margin-bottom: 10px;
+  transition: color .3s ease 0s;
+}
+
+.side-navigation__group-items h4 {
+  font-size: 13px;
+  text-transform: uppercase;
+  font-weight: bold;
+  margin-top: 10px;
+}
+
+.side-navigation__group-item.active {
+  color: #7a2c68;
+  /* text-decoration: underline; */
+  font-weight: bold;
+}
+
+.side-navigation__group-item:hover {
+  color: #7a2c68;
+}
+
+.page footer {
+  margin-top: 40px;
+  background: #eee;
+}
+
+/****
+ *
+ * Markdown stuff
+ * 
+ ****/
+
+.page > .container {
+  min-height: calc(100vh - 437px);
+}
+
+.page h1, .page h2, .page h3, .page h4, .page h5, .page h6 {
+  font-weight: normal;
+  line-height: 1em;
+}
+
+.page ul, .page ol {
+  font-size: 15px;
+}
+
+.page li {
+  padding-left: 10px;
+}
+
+table {
+  font-size: 15px;
+}
+
+pre {
+  padding: 20px;
+  font-size: 15px;
+}
+
+p code {
+  background: #e5e5e5;
+  background: #ffffc9;
+  padding: 0px 2px;
+  margin: 0 5px;
+  font-size: .9em;
+}
+
+figure, .page .content div.highlight {
+  margin: 0;
+  display: block;
+  box-sizing: border-box;
+  overflow: auto;
+  /* max-width: 100%; */
+}
+
+.page .content img {
+  max-width: 100%;
+}
+
+.table {
+  border-collapse: collapse;
+}
+
+.table td, .table th {
+  padding: 20px;
+}
+
+.table.table-condensed td,
+.table.table-condensed th {
+  padding: 10px;
+}
+
+.table.table-striped tbody tr:nth-child(odd) {
+  background-color: #eee;
+}
+
+.table.table-bordered tbody td {
+  border: none;
+  border-top: 1px solid #888;
+  border-right: 1px solid #888;
+}
+.table.table-bordered tbody td:first-child {
+  border-left: 1px solid #888;
+}
+
+.table.table-bordered tbody tr:last-child {
+  border-bottom: 1px solid #888;
+}
+
+.table.table-bordered tbody tr:first-child {
+  border-top: 1px solid #888;
+}
+
+
+/****
+ *
+ * Releases List on Docs
+ * 
+ ****/
+
+.releases-list-divider {
+  margin: 10px auto 40px;
+}
+
+.releases-list-container {
+  display: flex;
+  flex-direction: column;
+}
+
+@media only screen and (min-width: 768px) {
+  .releases-list-container {
+    flex-direction: row;
+  }
+}
+
+.releases-list-container span {
+  font-size: 15px;
+  text-transform: uppercase;
+  font-weight: bold;
+  color: #777;
+  padding-bottom: 10px;
+}
+
+@media only screen and (min-width: 768px) {
+  .releases-list-container span {
+    padding-right: 10px;
+    padding-bottom: 0px;
+    border-right: 2px solid #eee;
+  }
+}
+
+.releases-list {
+  margin: 0;
+  padding: 0;
+  list-style-type: none;
+}
+
+@media only screen and (min-width: 768px) {
+  .releases-list {
+    padding-left: 10px;
+  }
+}
+
+.releases-list li {
+  display: inline-block;
+  margin-right: 10px;
+  text-transform: uppercase;
+  padding-left: 0px;
+}
+
+.releases-list li.hide {
+  display: none;
+}
+
+.releases-list li a {
+  color: #999;
+}
+
+/****
+ *
+ * Breadcrumbs List, same as releases
+ * 
+ ****/
+
+.breadcrumbs-list-divider {
+  margin: 10px auto 40px;
+}
+
+.breadcrumbs-list-container {
+  display: flex;
+  flex-direction: column;
+}
+
+@media only screen and (min-width: 768px) {
+  .breadcrumbs-list-container {
+    flex-direction: row;
+  }
+}
+
+.breadcrumbs-list-container span {
+  font-size: 15px;
+  text-transform: uppercase;
+  font-weight: bold;
+  color: #777;
+  padding-bottom: 10px;
+}
+
+.breadcrumbs-list-container span a {
+  color: inherit;
+}
+
+@media only screen and (min-width: 768px) {
+  .breadcrumbs-list-container span {
+    padding-right: 10px;
+    padding-bottom: 0px;
+    border-right: 2px solid #eee;
+  }
+}
+
+.breadcrumbs-list {
+  margin: 0;
+  padding: 0;
+  list-style-type: none;
+  flex-grow: 1;
+}
+
+@media only screen and (min-width: 768px) {
+  .breadcrumbs-list {
+    padding-left: 10px;
+  }
+}
+
+.breadcrumbs-list li {
+  display: inline-block;
+  margin-right: 10px;
+  text-transform: uppercase;
+  padding-left: 0px;
+}
+
+.breadcrumbs-list li.divider {
+  width: 0px;
+  font-size: 0px;
+  height: 20px;
+  border-left: 2px solid #eee;
+  vertical-align: middle;
+}
+
+.breadcrumbs-list li.hide {
+  display: none;
+}
+
+.breadcrumbs-list li a {
+  color: #999;
+}
+
+/****
+ *
+ * CASE STUDIES
+ * 
+ ****/
+
+ul.case-studies {
+  padding: 0;
+  margin: 0;
+  margin-top: 30px;
+}
+
+.case-studies li {
+  list-style-type: none;
+  display: flex;
+  padding-left: 0;
+  margin-bottom: 50px;
+}
+
+.case-studies li > a {
+  padding: 20px;
+  padding-left: 0px;
+}
+
+.case-studies li > a:after {
+  border: none;
+}
+
+.case-studies li .study-description {
+  margin: 40px 0 10px;
+  margin-left: -90px;
+}
+
+@media only screen and (min-width:768px) {
+  .case-studies li .study-description {
+    margin: 20px 0;
+  }
+}
+
+.case-studies li .study-detail {
+  padding: 20px;
+  width: 100%;
+  margin-right: -20px;
+}
+
+.case-studies li:nth-child(odd) .study-detail {
+}
+
+.case-studies li .study-detail > a {
+  text-transform: uppercase;
+  font-size: 17px;
+  color: #999;
+  transition: color .3s ease 0s;
+}
+
+@media only screen and (min-width:768px) {
+  .case-studies li .study-detail > a {
+    font-size: 22px;
+  }
+}
+
+.case-studies li .study-detail > a:after {
+  border: none;
+}
+.case-studies li .study-detail > a:hover {
+  color: #7a2c68;
+}
+
+.case-studies li a span {
+  display: block;
+  width: 50px;
+  height: 50px;
+  transition: all .3s ease 0s;
+  opacity: .8;
+  background-color: #fff;
+  background-size: contain;
+  background-repeat: no-repeat;
+  background-position: center;
+  background-blend-mode: luminosity;
+}
+
+@media only screen and (min-width: 768px) {
+  .case-studies li a span {
+    width: 128px;
+    height: 128px;
+  }
+}
+
+.case-studies li a:hover span {
+  opacity: 1;
+  background-blend-mode: unset;
+}
+
+.case-studies li p {
+
+}
+
+.case-studies li div a.btn {
+  padding: 15px 40px;
+  background: #eb1c23;
+  color: #fff;
+  font-weight: bold;
+  font-size: 15px;
+  display: block;
+  text-align: center;
+  margin: 20px 0;
+  margin-left: -90px;
+  transition: background .3s ease 0s;
+  width: 100%;
+}
+
+@media only screen and (min-width:768px) {
+  .case-studies li div a.btn {
+    margin: 20px 0;
+    text-align: left;
+    display: inline-block;
+    width: auto;
+  }
+}
+
+.case-studies li div a.btn:hover {
+  background: #7a2c68;
+  color: #fff;
+}
+
+.case-studies li div a.btn::after {
+  border: none;
+}
+
+/****
+ *
+ * CASE STUDY
+ *
+ ****/
+
+.case-study-hero {
+  display: flex;
+  align-items: center;
+  min-height: 300px;
+  background-size: 110px;
+  flex-direction: column;
+  justify-content: center;
+  margin: 0px;
+  position: relative;
+  padding: 50px 10px;
+}
+
+.case-study-hero > * {
+  padding: 20px;
+  /* box-shadow: 0 0 50px -21px #333; */
+  margin: 0;
+  max-width: 80%;
+  margin-bottom: 10px;
+  /* box-shadow: 5px 5px #9C4E8A, 10px 10px #7a2c68, 15px 15px #999, 20px 20px #eee */
+}
+
+.case-study-hero img {
+  width: 50vw;
+  height: 50vw;
+  max-width: 256px;
+  max-height: 256px;
+  border-radius:100%;
+  border: 3px solid #fff;
+  box-sizing: border-box;
+}
+
+.page.page--case-study .section--hero {
+  width: unset;
+  overflow: unset;
+  background-image: linear-gradient(to bottom, #ffffff, #fbfbfb, #f6f6f6, #f2f2f2, #eeeeee);
+}
+
+.page.page--case-study h3 {
+  text-transform: uppercase;
+}
+
+.case-studies-list {
+  display: flex;
+  flex-wrap: wrap;
+  margin-top: 40px;
+  justify-content: center;
+}
+
+@media only screen and (min-width: 768px) {
+  .case-studies-list {
+    justify-content: start;
+  }
+}
+
+
+.case-studies-list_item a {
+
+}
+
+.case-studies-list__item-logo-bg {
+  width: 30vw;
+  height: 30vw;
+  max-width: 128px;
+  max-height: 128px;
+  transition: all .3s ease 0s;
+  opacity: .8;
+  background-color: #fff;
+  background-size: contain;
+  background-repeat: no-repeat;
+  background-position: center;
+  background-blend-mode: luminosity;
+}
+
+.case-studies-list__item-logo-bg:hover {
+  opacity: 1;
+  background-blend-mode: unset;
+}
+
+/****
+ *
+ * POWERED BY
+ * 
+ ****/
+
+ ul.powered-by {
+  padding: 0;
+  margin: 0;
+  margin-top: 30px;
+}
+
+.powered-by li {
+  list-style-type: none;
+  display: flex;
+  padding-left: 0;
+}
+
+.powered-by li > span {
+  padding: 20px;
+  padding-left: 0px;
+}
+
+.powered-by li div {
+  padding: 20px;
+}
+
+.powered-by li:nth-child(odd) div {
+}
+
+.powered-by li div > a {
+  text-transform: uppercase;
+  font-size: 22px;
+  color: #999;
+  transition: color .3s ease 0s;
+}
+
+.powered-by li div > a:after {
+  border: none;
+}
+.powered-by li div > a:hover {
+  color: #7a2c68;
+}
+
+.powered-by li span span {
+  display: block;
+  width: 50px;
+  height: 50px;
+  transition: all .3s ease 0s;
+  opacity: .8;
+  background-color: #fff;
+  background-size: contain;
+  background-repeat: no-repeat;
+  background-position: center;
+  background-blend-mode: luminosity;
+}
+
+@media only screen and (min-width: 768px) {
+  .powered-by li span span {
+    width: 128px;
+    height: 128px;
+  }
+}
+
+.powered-by li span:hover span {
+  opacity: 1;
+  background-blend-mode: unset;
+}
+
+.powered-by li p {
+
+}
+
+.powered-by li div a.btn {
+  padding: 15px 40px;
+  background: #eb1c23;
+  color: #fff;
+  font-weight: bold;
+  font-size: 15px;
+  display: inline-block;
+  margin: 20px 0;
+  transition: background .3s ease 0s;
+}
+
+.powered-by li div a.btn:hover {
+  background: #7a2c68;
+}
+
+.powered-by li div a.btn::after {
+  border: none;
+}
+
+/**
+ *
+ * News
+ *
+ **/
+
+.news__cards {
+  display: flex;  
+}
+
+.news__card {
+  display: block;
+  flex-grow: 1;
+  font-weight: normal;
+  padding: 40px;
+  box-sizing: 20px;
+  margin: 50px 0px;
+  position: relative;
+  padding-left: 90px;
+}
+
+@media only screen and (min-width: 768px) {
+  .news_card {
+    padding-left: 120px;
+  }
+}
+
+.news__card:hover {
+  background: #eee;
+}
+
+.news_cards .news__card::after {
+  border: none;
+}
+
+.news__card-title {
+  margin-top: 50px;
+  margin-left: -67px;
+  margin-right: -10px;
+}
+
+@media only screen and (min-width: 768px) {
+  .news__card-title {
+    margin: 10px 0;
+  }
+}
+
+.news__card-date {
+  font-size: 20px;
+  margin-top: 10px;
+  margin-bottom: 20px;
+  padding-bottom: 10px;
+  border-bottom: 1px solid #ddd;
+  text-transform: uppercase;
+  color: #444;
+}
+
+.news__card-description {
+  font-size: 18px;
+  color: #555;
+  margin: 10px 0;
+}
+
+.news__card-description {
+  margin-left: -67px;
+  margin-right: -10px;
+}
+
+@media only screen and (min-width: 768px) {
+  .news__card-description {
+    margin: 10px 0;
+  }
+}
+
+.news__card-button {
+  padding: 15px 40px;
+  background: #eb1c23;
+  color: #fff;
+  font-weight: bold;
+  font-size: 15px;
+  display: block;
+  width: 100%;
+  transition: background .3s ease 0s;
+  margin-left: -67px;
+  margin-right: -10px;
+  text-align: center;
+}
+
+@media only screen and (min-width: 768px) {
+  .news__card-button {
+    display: inline-block;
+    width: auto;
+    margin: 20px 0;
+    text-align: left;
+  }
+}
+
+
+.news__card-button:hover {
+  background: #7a2c68;
+}
+
+.news__card-button::after {
+  border: none;
+}
+
+
+.news__card-icon {
+  position: absolute;
+  left: 20px;
+  top: 50px;
+  color: #fff;
+  font-size: 20px;
+  border-radius: 100%;
+  width: 35px;
+  height: 35px;
+  padding: 10px;
+  display: flex;
+  justify-content: center;
+  align-items: center;
+  margin: auto;
+  margin-bottom: 20px;
+  background: linear-gradient(45deg, #ff4747 0%,#eb1c23 50%,#7a2c68 51%,#333333 100%); /* W3C, IE10+, FF16+, Chrome26+, Opera12+, Safari7+ */
+}
+
+@media only screen and (min-width: 768px) {
+  .news_card-icon {
+    width: 55px;
+    height: 55px;
+    font-size: 30px;
+  }
+}
+
+.news-end {
+  display: flex;
+}
+
+.news-end-action {
+  background: #7a2c68;
+  padding: 20px;
+  color: #fff;
+  font-size: 15px;
+  flex-grow: 1;
+  display: block;
+  transition: background .3s ease 0s;
+  box-sizing: border-box;
+}
+
+@media only screen and (min-width:768px) {
+  .news-end-action {
+    padding: 40px;
+  }
+}
+
+.news-previous {
+  text-align: right;
+}
+
+.news-next {
+  background: #9C4E8A;
+}
+
+.news-end-action:hover {
+  background: #eb1c23;
+}
+
+.news-end .news-end-action:after {
+  border: none;
+}
+
+.news-end-action-title {
+  display: flex;
+  justify-content: space-between;
+  align-items: center;
+  flex-direction: column;
+  box-sizing: border-box;
+  text-align: right;
+}
+
+.news-next .news-end-action-title {
+  flex-direction: column-reverse;
+  text-align: left;
+  /* align-items: flex-end; */
+}
+
+@media only screen and (min-width:768px) {
+  .news-end .news-end-action-title {
+    flex-direction: row;
+    align-items: center;
+    font-size: 20px;
+  }
+}
+
+.news-end-action-title i {
+  margin: 10px;
+  font-size: 40px;
+}
+
+
+/**
+ *
+ * Talks
+ *
+ **/
+.talks {
+  margin-top: 50px;
+}
+
+.talk {
+  margin-bottom: 30px;
+  position: relative;
+  padding: 20px;
+  padding-left: 55px;
+  transition: background .3s ease 0s;
+  filter: opacity(.8) grayscale(1);
+}
+
+@media only screen and (min-width: 768px) {
+  .talk {
+    padding-left: 90px;
+  }
+}
+
+.talk.hide {
+  display: none;
+}
+
+.talk--upcoming {
+  filter: none;
+}
+
+.talk--upcoming + .talk:not(.talk--upcoming) {
+  margin-top: 100px;
+}
+
+.talk--upcoming + .talk:not(.talk--upcoming)::before {
+  content: 'Past Events';
+  border-bottom: 4px solid #eee;
+  width: 100%;
+  display: block;
+  position: absolute;
+  left: 0px;
+  top: -70px;
+  padding-bottom: 20px;
+}
+
+.talk-upcoming-tag {
+  color: #eb1c23;
+  padding: 5px 21px;
+  text-transform: uppercase;
+  display: flex;
+  justify-content: space-between;
+  align-items: center;
+  font-weight: bold;
+  border: 2px solid #eee;
+  font-size: 12px;
+  margin-bottom: 20px;
+  margin-left: 25px;
+}
+
+@media only screen and (min-width: 900px) {
+  .talk-upcoming-tag {
+    border: none;
+    font-size: 15px;
+  }
+}
+
+
+.talk-upcoming-tag i { 
+  margin-right: 20px;
+  font-size: 20px;
+}
+
+.talk-upcoming-section {
+  display: none;
+  
+}
+
+@media only screen and (min-width: 900px) {
+  .talk-upcoming-section {
+    position: absolute;
+    top: 15px;
+    right: 15px;
+  }
+}
+
+.talk--upcoming .talk-upcoming-section {
+  display: block;
+}
+
+
+.talk:hover {
+  background: #efefef;
+  filter: none;
+}
+
+.talk * {
+  box-sizing: border-box;
+}
+
+.talk-icon {
+  position: absolute;
+  left: 20px;
+  top: 30px;
+  color: #fff;
+  font-size: 20px;
+  border-radius: 100%;
+  width: 35px;
+  height: 35px;
+  padding: 10px;
+  display: flex;
+  justify-content: center;
+  align-items: center;
+  margin: auto;
+  margin-bottom: 20px;
+  background: linear-gradient(45deg, #ff4747 0%,#eb1c23 50%,#7a2c68 51%,#333333 100%); /* W3C, IE10+, FF16+, Chrome26+, Opera12+, Safari7+ */
+}
+
+
+@media only screen and (min-width: 768px) {
+  .talk-icon {
+    width: 75px;
+    height: 75px;
+    font-size: 30px;
+  }
+}
+
+.talk-info {
+}
+
+.talk-date {
+  padding: 10px;
+  padding-left: 0;
+  margin-left: 25px;
+  text-transform: uppercase;
+  margin-bottom: 20px;
+  font-size: 20px;
+  border-bottom: 1px solid #ddd;
+}
+
+.talk-event {
+  padding: 10px;
+  margin-left: -50px;
+  margin-right: -30px;
+}
+
+@media only screen and (min-width: 768px) {
+  .talk-event {
+    margin-left: 20px;
+    margin-right: 0px;
+  }
+}
+
+.talk-event-link {
+
+}
+
+.talk-event-name {
+  margin-bottom: 10px;
+}
+
+.talk-event-image {
+  height: 300px;
+  width: 100%;
+  background-color: #ddd;
+  background-size: cover;
+  background-position: center;
+  background-repeat: no-repeat;
+}
+
+.talk-event-host {
+  font-size: 15px;
+  color: #999;
+  text-transform: uppercase;
+  margin-bottom: 10px;
+  margin-top: 40px;
+}
+
+.talk-event-by {
+  font-size: 15px;
+  color: #999;
+  text-transform: uppercase;
+  margin-bottom: 10px;
+}
+
+.talk-video {
+  display: block;
+  height: 300px;
+  width: 100%;
+  background-color: #ddd;
+  background-size: cover;
+  background-position: center;
+  background-repeat: no-repeat;
+  position: relative;
+}
+
+.talk-video-icon {
+  position: absolute;
+  top: 50%;
+  left: 50%;
+  transform: translate(-50%, -50%);
+  color: rgba(0,0,0,.4);
+  font-size: 100px;
+  transition: color .3s ease 0s;
+}
+
+.talk-video:hover .talk-video-icon {
+  color: #eb1c23;
+}
+
+.talks .talk-video::after {
+  border: none;
+}
+
+.talk--upcoming .talk-action {
+  background: #eb1c23;
+}
+
+.talk-action {
+  padding: 15px 40px;
+  background: #eb1c23;
+  color: #fff;
+  font-weight: bold;
+  font-size: 15px;
+  display: block;
+  text-align: center;
+  margin: 20px 0;
+  width: 100%;
+  transition: background .3s ease 0s;
+}
+
+@media only screen and (min-width: 768px) {
+  .talk-action {
+    width: auto;
+    text-align: left;
+    display: inline-block;
+  }
+}
+
+.talk-action:hover {
+  background: #7a2c68;
+}
+
+.talk-actions .talk-action::after {
+  border: none;
+}
+
+.talk-actions {
+  margin-left: -35px;
+}
+
+@media only screen and (min-width: 768px) {
+  .talk-actions {
+    margin-left: 30px;
+    margin-right: 0px;
+  }
+}
+
+.talk-description {
+  margin-left: -35px;
+}
+
+@media only screen and (min-width: 768px) {
+  .talk-description {
+    margin-left: 30px;
+    margin-right: 0px;
+  }
+}
+
+/**
+ *
+ * Meetups
+ *
+ **/
+
+.meetups {
+  margin-top: 50px;
+}
+
+.meetup {
+  margin-bottom: 30px;
+  position: relative;
+  padding: 20px;
+  padding-left: 55px;
+  transition: background .3s ease 0s;
+  filter: opacity(.8) grayscale(1);
+}
+
+@media only screen and (min-width: 768px) {
+  .meetup {
+    padding-left: 90px;
+  }
+}
+
+.meet.hide {
+  display: none;
+}
+
+.meetup.hide {
+  display: none;
+}
+
+.meet--upcoming, .meetup--upcoming {
+  filter: none;
+}
+
+.meetup--upcoming + .meetup:not(.meetup--upcoming) {
+  margin-top: 100px;
+}
+
+.meetup--upcoming + .meetup:not(.meetup--upcoming)::before {
+  content: 'Past Events';
+  border-bottom: 4px solid #eee;
+  width: 100%;
+  display: block;
+  position: absolute;
+  left: 0px;
+  top: -70px;
+  padding-bottom: 20px;
+}
+
+.meet-upcoming-tag {
+  color: #eb1c23;
+  padding: 5px 21px;
+  text-transform: uppercase;
+  display: flex;
+  justify-content: space-between;
+  align-items: center;
+  font-weight: bold;
+  border: 2px solid #eee;
+  font-size: 12px;
+  margin-bottom: 20px;
+}
+
+
+.meet-upcoming-tag i { 
+  margin-right: 20px;
+  font-size: 20px;
+}
+
+.meet-upcoming-section {
+  display: none;
+  
+}
+
+.meet--upcoming .meet-upcoming-section {
+  display: block;
+}
+
+.meetup:hover {
+  filter: none;
+}
+
+.meet:hover {
+  filter: none;
+  background-image: linear-gradient(to bottom, #ffffff, #fbfbfb, #f6f6f6, #f2f2f2, #ddd);
+}
+
+.meetup * {
+  box-sizing: border-box;
+}
+
+.meetup-icon {
+  position: absolute;
+  left: 20px;
+  top: 30px;
+  color: #fff;
+  font-size: 20px;
+  border-radius: 100%;
+  width: 35px;
+  height: 35px;
+  padding: 10px;
+  display: flex;
+  justify-content: center;
+  align-items: center;
+  margin: auto;
+  margin-bottom: 20px;
+  background: linear-gradient(45deg, #ff4747 0%,#eb1c23 50%,#7a2c68 51%,#333333 100%); /* W3C, IE10+, FF16+, Chrome26+, Opera12+, Safari7+ */
+}
+
+
+@media only screen and (min-width: 768px) {
+  .meetup-icon {
+    width: 75px;
+    height: 75px;
+    font-size: 30px;
+  }
+}
+
+.meetup-info {
+}
+
+.meetup-date-group {
+  padding: 10px;
+  padding-left: 0;
+  margin-left: 25px;
+  text-transform: uppercase;
+  margin-bottom: 20px;
+  font-size: 20px;
+}
+
+@media only screen and (min-width: 768px) {
+  .meetup-date-group {
+    font-size: 25px;
+    padding-top: 30px;
+  }
+}
+
+.meet {
+  padding: 10px;
+  margin-left: 20px;
+  margin-bottom: 40px;
+  padding: 10px 20px;
+  margin-left: -80px;
+  margin-right: -40px;
+  background-image: linear-gradient(to bottom, #ffffff, #fbfbfb, #f6f6f6, #f2f2f2, #eeeeee);
+}
+
+@media only screen and (min-width: 768px) {
+  .meet {
+    padding: 20px 40px;
+    margin-left: 30px;
+    margin-right: 0px;
+  }
+}
+
+.meet-link {
+
+}
+
+.meet-name {
+  margin-bottom: 10px;
+}
+
+.meet-image {
+  height: 300px;
+  width: 100%;
+  background-color: #ddd;
+  background-size: cover;
+  background-position: center;
+  background-repeat: no-repeat;
+}
+
+.meet-host {
+  font-size: 15px;
+  color: #999;
+  text-transform: uppercase;
+  margin-bottom: 10px;
+}
+
+.meet-date {
+  font-size: 15px;
+  color: #999;
+  text-transform: uppercase;
+  margin-bottom: 10px;
+  margin-top: 40px;
+}
+
+.meet-presenter {
+  font-size: 15px;
+  color: #999;
+  text-transform: uppercase;
+  margin-bottom: 10px;
+}
+
+.meet-presenter span > span::before {
+  content: '& '
+}
+
+.meet-presenter span > span:first-child::before {
+  content: ''
+}
+
+.meet-video {
+  display: block;
+  height: 300px;
+  width: 100%;
+  background-color: #ddd;
+  background-size: cover;
+  background-position: center;
+  background-repeat: no-repeat;
+  position: relative;
+}
+
+.meet-video-icon {
+  position: absolute;
+  top: 50%;
+  left: 50%;
+  transform: translate(-50%, -50%);
+  color: rgba(0,0,0,.4);
+  font-size: 100px;
+  transition: color .3s ease 0s;
+}
+
+.meet-video:hover .meet-video-icon {
+  color: #eb1c23;
+}
+
+.meetups .meet-video::after {
+  border: none;
+}
+
+.meet--upcoming .meet-action {
+  background: #eb1c23;
+}
+
+.meet-action {
+  padding: 15px 40px;
+  background: #eb1c23;
+  color: #fff;
+  font-weight: bold;
+  font-size: 15px;
+  display: block;
+  text-align: center;
+  margin: 20px 0;
+  width: 100%;
+  transition: background .3s ease 0s;
+}
+
+@media only screen and (min-width: 768px) {
+  .meet-action {
+    width: auto;
+    text-align: left;
+    display: inline-block;
+  }
+}
+
+.meet-action:hover {
+  background: #7a2c68;
+}
+
+.meet-actions .meet-action::after {
+  border: none;
+}
+
+.meet-actions {
+}
+
+.meet-description {
+  margin-top: 30px;
+  font-size: 20px;
+}
+
+
+
+/**
+ *
+ * Pagination
+ *
+ **/
+
+.pagination {
+  
+}
+
+.pagination.hide {
+  display: none;
+}
+
+.pagination-inner {
+  display: flex;
+}
+
+.paginate-action {
+  cursor: pointer;
+  flex-grow: 1;
+  padding: 40px;
+  background: #7a2c68;
+  color: #fff;
+  display: flex;
+  justify-content: space-between;
+  align-items: center;
+  transition: background .3s ease 0;
+  font-size: 25px;
+}
+
+.paginate-action.hide {
+  display: none;
+}
+
+.paginate-action i {
+  font-size: 50px;
+}
+
+.paginate-action:hover {
+  background: #eb1c23;
+}
+
+.paginate-previous {
+
+}
+
+.paginate-next {
+  background: #9C4E8A;
+}
+
+/****
+ *
+ * Committers
+ * 
+ ****/
+
+.committers {
+  padding: 0;
+  margin: 0;
+  list-style-type: none;
+  display: flex;
+  flex-direction: column;
+}
+
+@media only screen and (min-width: 768px) {
+  .committers {
+    flex-wrap: wrap;
+    justify-content: space-between;
+    flex-direction: row;
+  }
+}
+
+.committers-hr {
+  margin-bottom: 20px;
+}
+
+.committer {
+  /* background: #eee; */
+  padding: 20px;
+  margin: 20px 0;
+  display: flex;
+  border-radius: 4px;
+}
+
+@media only screen and (min-width: 768px) {
+  .committer {
+    width: 40%;
+  }
+}
+
+.committer * {
+  box-sizing: border-box;
+}
+
+.committer:hover {
+  background: #eee;
+}
+
+.committer:hover .committer-image span {
+  background-blend-mode: unset;
+}
+
+.committer-name {
+  font-size: 25px;
+  margin-bottom: 10px;
+}
+
+@media only screen and (min-width: 768px) {
+  .committer-name {
+    font-size: 30px;
+  }
+}
+
+.committer-icon {
+  font-size: 30px;
+  margin-right: 10px;
+}
+
+.committer-link {
+  color: #999;
+  transition: color .3s ease 0s;
+}
+
+.committer-link:hover {
+  color: #7a2c68;
+}
+
+.committer .committer-link:after {
+  border: none;
+}
+
+.committer-roles {
+  display: flex;
+  flex-direction: column;
+  margin-bottom: 10px;
+}
+
+.committer-roles span::before {
+  content: '» '
+}
+
+.committer-image {
+  margin-right: 30px;
+  box-sizing: content-box;
+  display: block;
+}
+
+.committer-image span {
+  width: 32px;
+  height: 32px;
+  background-color: #fff;
+  background-size: cover;
+  background-repeat: no-repeat;
+  background-position: center;
+  background-blend-mode: luminosity;
+  display: block;
+  transition: background .3s ease 0s;
+  border-radius: 100%;
+}
+
+@media only screen and (min-width: 768px) {
+  .committer-image span {
+    width: 64px;
+    height: 64px;
+  }
+}
+
+.committer-image a {
+  border: 3px solid #eee;
+  transition: border .3s ease 0s;
+  display: block;
+  border-radius: 100%;
+}
+
+.committer-image a::after {
+  border: none;
+}
+
+.committer-image a:hover {
+  border: 3px solid #fff;
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/b8058af0/docs/css/ropa-sans.css
----------------------------------------------------------------------
diff --git a/docs/css/ropa-sans.css b/docs/css/ropa-sans.css
deleted file mode 100644
index 4c93c19..0000000
--- a/docs/css/ropa-sans.css
+++ /dev/null
@@ -1,25 +0,0 @@
-/*
- * 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.
- */
-
-@font-face {
-  font-family: 'Ropa Sans';
-  font-style: normal;
-  font-weight: 400;
-  src: local('Ropa Sans'), local('RopaSans-Regular'), url('../fonts/RopaSans-Regular-webfont.woff') format('woff');
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/b8058af0/docs/fonts/ionicons.eot
----------------------------------------------------------------------
diff --git a/docs/fonts/ionicons.eot b/docs/fonts/ionicons.eot
new file mode 100644
index 0000000..4b1fd0f
Binary files /dev/null and b/docs/fonts/ionicons.eot differ


[16/29] samza git commit: SAMZA-1898: New UI layout for the Samza website

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/samza/blob/b8058af0/docs/fonts/ionicons.svg
----------------------------------------------------------------------
diff --git a/docs/fonts/ionicons.svg b/docs/fonts/ionicons.svg
new file mode 100644
index 0000000..3d1e759
--- /dev/null
+++ b/docs/fonts/ionicons.svg
@@ -0,0 +1,713 @@
+<?xml version="1.0" encoding="UTF-8" standalone="no"?>
+<!DOCTYPE svg PUBLIC "-//W3C//DTD SVG 1.1//EN" "http://www.w3.org/Graphics/SVG/1.1/DTD/svg11.dtd">
+<!--
+2018-6-14: Created with FontForge (http://fontforge.org)
+-->
+<svg xmlns="http://www.w3.org/2000/svg" xmlns:xlink="http://www.w3.org/1999/xlink" version="1.1">
+<metadata>
+Created by FontForge 20160407 at Thu Jun 14 08:50:34 2018
+ By Adam Bradley
+Copyright (c) 2018, Adam Bradley
+</metadata>
+<defs>
+<font id="Ionicons" horiz-adv-x="416">
+  <font-face font-family="Ionicons" font-weight="400" font-stretch="normal" units-per-em="512" panose-1="2 0 5 3 0 0 0 0 0 0" ascent="448" descent="-64" bbox="-0.75 -64 512.487 448" underline-thickness="25.6" underline-position="-51.2" unicode-range="U+F100-F4CD"/>
+    <missing-glyph/>
+    <glyph glyph-name="ion-ios-color-fill" unicode="" horiz-adv-x="384" d="M167 330l172 -170c4 -4 1 -11 -5 -11v0c-10 0 -23 -1 -29 -7c-8 -8 -88 -86 -126 -123c-11 -10 -28 -10 -39 0l-131 126c-11 11 -12 29 -1 40l85 84l-50 49c-19 19 -19 49 0 68v0c9 9 22 14 34 14s25 -5 34 -14l56 -56v0zM64 338l48 -49l29 28l-50 49c-4 4 -9 6 -14 6 c-6 0 -11 -3 -15 -8c-6 -8 -5 -19 2 -26zM339 108c11 -12 45 -53 45 -76c0 -26 -22 -48 -48 -48v0c-26 0 -48 21 -48 48c0 22 35 64 45 76c2 2 4 2 6 0z"/>
+    <glyph glyph-name="ion-md-backspace" unicode="" horiz-adv-x="426" d="M390 352c20 0 36 -16 36 -36v-248c0 -20 -16 -36 -36 -36h-266c-12 0 -22 7 -28 16l-96 144l96 144c6 9 16 16 28 16h266zM337 128l-63 64l63 64l-25 25l-64 -64l-63 64l-25 -25l64 -64l-64 -64l25 -25l63 64l64 -64z"/>
+    <glyph glyph-name="ion-md-paper-plane" unicode="" d="M0 177l416 223l-104 -416l-104 104l-74 -104l-15 148zM295 51l73 288l-291 -156l69 -25l181 138l-115 -161z"/>
+    <glyph glyph-name="ion-ios-cart" unicode="" d="M88 24c0 13 10 24 23 24s25 -11 25 -24s-11 -24 -24 -24s-24 11 -24 24zM336 0c-13 0 -24 12 -24 25s12 23 25 23s23 -12 23 -25s-11 -23 -24 -23zM416 316v-3l-16 -116c0 -3 -2 -5 -5 -5l-278 -47v0l4 -24c3 -16 7 -23 11 -26c3 -3 7 -3 9 -3v0h237c8 0 14 -6 14 -14 s-6 -14 -14 -14h-237c-13 0 -25 5 -33 16c-7 9 -12 22 -14 36l-40 212s-2 9 -2 10v0c-1 7 -5 15 -10 16s-17 2 -26 2s-16 5 -16 14s7 14 16 14c27 0 41 0 49 -9c6 -8 8 -12 12 -19c1 -3 5 -4 8 -4l325 -32c2 0 5 -2 6 -4z"/>
+    <glyph glyph-name="ion-ios-rose" unicode="" horiz-adv-x="320" d="M320 307c-77 -89 -16 -136 -16 -204s-56 -119 -144 -119s-144 47 -144 115c0 60 35 100 93 131c25 14 33 17 52 26c30 13 65 27 105 38c17 5 35 9 54 13zM39 206c5 38 3 91 -39 135c0 0 66 -15 115 -41c14 -8 28 -17 37 -27c-12 -6 -24 -11 -36 -17c-7 -4 -14 -7 -20 -11 c-25 -14 -44 -29 -57 -39zM256 370c0 0 26 -29 32 -51c-33 -7 -78 -20 -115 -36c-15 15 -43 32 -43 32c52 56 126 55 126 55zM144 351c-12 -8 -21 -16 -31 -27c-26 12 -41 18 -62 24c5 14 14 31 29 52c0 0 35 -11 64 -49z"/>
+    <glyph glyph-name="ion-ios-refresh" unicode="" horiz-adv-x="384" d="M369 159v0c9 0 16 -7 15 -16c-8 -98 -91 -175 -192 -175c-106 0 -192 85 -192 191c0 104 84 189 188 191c2 0 4 2 4 4v46c0 13 14 21 25 14l96 -67c10 -6 10 -21 0 -27l-96 -67c-11 -7 -25 0 -25 13v50c0 2 -2 4 -4 4c-88 -2 -158 -73 -158 -161c0 -89 72 -161 162 -161 c85 0 155 65 162 148c1 8 7 13 15 13z"/>
+    <glyph glyph-name="ion-md-musical-note" unicode="" horiz-adv-x="256" d="M128 384h128v-85h-85v-214c0 -47 -39 -85 -86 -85s-85 38 -85 85s38 86 85 86c16 0 30 -5 43 -12v225z"/>
+    <glyph glyph-name="ion-md-list-box" unicode="" horiz-adv-x="384" d="M344 384c22 0 40 -18 40 -40v-304c0 -22 -18 -40 -40 -40h-304c-22 0 -40 18 -40 40v304c0 22 18 40 40 40h304zM240 80v48h-160v-48h160zM304 168v48h-224v-48h224zM304 256v48h-224v-48h224z"/>
+    <glyph glyph-name="ion-ios-mail" unicode="" d="M413 301c1 1 3 0 3 -2v-219c0 -18 -14 -32 -32 -32h-352c-18 0 -32 14 -32 32v219c0 2 2 3 3 2l108 -110c1 -1 1 -2 0 -3l-75 -80c-5 -5 -5 -13 0 -18c2 -2 6 -4 9 -4s7 2 9 4l75 80c1 1 2 1 3 0l18 -18c16 -16 36 -25 58 -25s43 8 58 24l18 19c1 1 2 1 3 0l75 -80 c2 -2 6 -4 9 -4s7 2 9 4c5 5 5 13 0 18l-75 80c-1 1 -1 2 0 3zM208 153c-15 0 -29 6 -39 16l-157 160c5 4 12 7 20 7h352c8 0 14 -3 20 -7l-157 -160c-10 -11 -24 -16 -39 -16z"/>
+    <glyph glyph-name="ion-md-ice-cream" unicode="" horiz-adv-x="320" d="M32 192h256l-128 -224zM296 287c14 -4 24 -16 24 -31v0c0 -18 -14 -32 -32 -32h-256c-18 0 -32 14 -32 32v0c0 15 10 27 24 31c-2 6 -2 15 -2 21c0 42 34 76 76 76c8 0 10 -1 17 -3c-5 -12 -3 -22 -1 -31l3 -1c2 15 11 29 11 29c18 23 46 38 77 38c53 0 96 -43 96 -96 c0 -10 -2 -24 -5 -33z"/>
+    <glyph glyph-name="ion-md-at" unicode="" d="M218 236c2 0 4 -1 6 -1s5 -1 7 -1l-7 -66c-3 -8 -7 -13 -11 -16s-8 -5 -14 -5s-10 3 -13 9s-4 15 -3 27c2 18 5 30 11 39s14 14 24 14zM208 400c115 0 208 -93 208 -208s-93 -208 -208 -208s-208 93 -208 208s93 208 208 208zM336 198c2 40 -8 72 -29 95s-51 35 -91 35 s-73 -13 -97 -37s-37 -57 -39 -99s9 -76 31 -100s53 -36 94 -36c10 0 21 1 31 3s19 6 26 10l-6 24c-7 -3 -15 -5 -24 -7s-18 -2 -26 -2c-31 0 -55 10 -71 29s-24 46 -22 79c2 34 12 61 30 81s42 30 72 30c31 0 55 -10 70 -28s22 -44 20 -77c-1 -16 -4 -30 -11 -40 s-16 -15 -27 -15c-3 0 -5 2 -7 5s-2 10 -2 18l8 83h-1h1c-9 5 -18 9 -25 11s-17 4 -29 4c-17 0 -33 -7 -45 -22s-19 -35 -22 -59c-2 -19 1 -34 9 -46s19 -17 34 -17c8 0 17 2 23 6s11 10 15 18c2 -8 7 -14 13 -18s13 -7 21 -7c25 0 44 7 56 23s19 35 20 56z"/>
+    <glyph glyph-name="ion-ios-create" unicode="" horiz-adv-x="448" d="M372 362c3 3 9 3 12 0l26 -26c3 -3 3 -9 0 -12l-202 -202c-2 -2 -4 -2 -6 -2h-26c-4 0 -8 4 -8 8v26c0 2 0 4 2 6zM444 381c3 -3 4 -8 4 -12s-1 -8 -4 -11l-12 -12c-2 -2 -3 -2 -5 0l-22 22l-10 10c-2 2 -2 4 0 6l11 12c3 3 8 4 12 4s8 -1 11 -4v-1zM218 106l1 1l148 147 c2 2 6 2 6 -2v-209c0 -32 -26 -59 -58 -59h-256c-32 0 -59 27 -59 59v224c0 32 27 58 59 58h241c4 0 4 -5 2 -7l-147 -147l-1 -1c-3 -3 -5 -7 -5 -11v-1v-38v-3c0 -9 7 -16 16 -16h3h38h1c4 0 8 2 11 5z"/>
+    <glyph glyph-name="ion-logo-chrome" unicode="" horiz-adv-x="448" d="M157 192c0 37 30 67 67 67s67 -30 67 -67s-30 -67 -67 -67s-67 30 -67 67zM445 230c2 -12 3 -25 3 -38c0 -106 -74 -195 -174 -218c-10 -2 -20 -4 -30 -5c-7 -1 -13 -1 -20 -1c-8 0 -17 0 -25 1v0v0v0v1l102 177c8 13 13 28 13 45c0 27 -13 50 -32 67h156 c3 -9 5 -19 7 -29v0c-2 10 -4 20 -7 29v0c3 -9 5 -19 7 -29v0zM224 103c10 0 20 2 29 5l-78 -135v0c-42 9 -79 30 -108 59c-42 41 -67 97 -67 160c0 31 6 61 18 88c7 16 16 31 26 45v0l103 -178c16 -27 43 -44 77 -44zM59 343v0v1c24 26 55 47 90 59c23 8 49 13 75 13 c73 0 138 -36 179 -90h1c10 -14 18 -29 25 -44v0c-7 15 -15 30 -25 44h-1c10 -14 19 -29 26 -44h-205c-44 0 -79 -32 -87 -72z"/>
+    <glyph glyph-name="ion-md-volume-mute" unicode="" horiz-adv-x="192" d="M0 256h85l107 112v-352l-107 112h-85v128z"/>
+    <glyph glyph-name="ion-md-skip-forward" unicode="" horiz-adv-x="384" d="M0 384l277 -192l-277 -192v384zM320 0v384h64v-384h-64z"/>
+    <glyph glyph-name="ion-ios-infinite" unicode="" horiz-adv-x="464" d="M433 266c20 -20 31 -46 31 -74s-12 -54 -32 -74s-47 -30 -75 -30s-55 10 -75 30l-125 123c-14 13 -32 20 -51 20s-36 -7 -50 -20c-13 -13 -21 -31 -21 -49s8 -36 21 -49c14 -13 31 -20 50 -20s37 7 51 20l31 30c2 2 3 2 5 0l20 -19c2 -2 2 -4 0 -6l-31 -30 c-20 -20 -48 -30 -76 -30s-55 10 -75 30s-31 46 -31 74s11 54 31 74s47 30 75 30s56 -10 76 -30l125 -123c14 -13 32 -20 51 -20s36 7 50 20c13 13 20 31 20 49s-7 36 -20 49c-14 13 -31 20 -50 20s-37 -7 -51 -20l-31 -30c-2 -2 -3 -2 -5 0l-20 19c-2 2 -2 4 0 6l31 30 c20 20 48 30 76 30s55 -10 75 -30z"/>
+    <glyph glyph-name="ion-logo-windows" unicode="" horiz-adv-x="448" d="M448 183v0v-215l-248 36v179h248zM184 183v0v-177l-184 27v150h184zM448 416v0v-217h-248v182zM184 378v0v-179h-184v153z"/>
+    <glyph glyph-name="ion-md-microphone" unicode="" horiz-adv-x="256" d="M256 336h-56v-37h56v-43h-56v-37h56v-43h-56v-37h56v-42c0 -29 -25 -52 -55 -52h-30v-77h-86v77h-30c-30 0 -55 23 -55 52v42h152v37h-152v43h152v37h-152v43h152v37h-152v28c0 29 25 52 55 52h146c30 0 55 -23 55 -52v-28z"/>
+    <glyph glyph-name="ion-ios-share-alt" unicode="" horiz-adv-x="384" d="M381 218c2 -2 3 -7 3 -10s0 -6 -3 -8l-143 -134c-5 -5 -14 -2 -14 5v65c0 2 -2 5 -4 5c-98 -1 -161 -44 -212 -108c-3 -3 -9 1 -8 5c21 109 80 214 220 236c2 0 4 2 4 4v66c0 5 6 8 10 8c2 0 4 0 6 -2z"/>
+    <glyph glyph-name="ion-md-wine" unicode="" horiz-adv-x="384" d="M171 171l-171 170v43h384v-43l-171 -170v-131h107v-40h-256v40h107v131zM96 299h192l43 42h-278z"/>
+    <glyph glyph-name="ion-md-skip-backward" unicode="" horiz-adv-x="384" d="M107 192l277 192v-384zM0 0v384h64v-384h-64z"/>
+    <glyph glyph-name="ion-ios-bug" unicode="" horiz-adv-x="448" d="M343 321c-6 -16 -17 -31 -29 -43c-24 -24 -56 -37 -90 -37s-67 13 -91 37c-12 12 -22 27 -28 43c29 39 72 63 119 63s90 -24 119 -63zM90 298c22 -45 69 -80 122 -83v-215c-40 2 -73 23 -100 55l-5 -5c-5 -4 -6 -11 -4 -17c3 -8 1 -18 -5 -25c-9 -10 -24 -11 -34 -2 s-11 24 -2 34l3 3s2 1 2 1c4 13 13 26 27 37c-15 26 -26 56 -29 89c-2 0 -4 -1 -6 -1c-6 -1 -10 -4 -12 -9c-4 -10 -14 -17 -25 -16c-12 1 -21 10 -22 22c-1 14 10 26 24 26h3c4 2 8 5 12 6c7 2 14 4 25 4c1 23 6 45 13 65c-6 4 -11 7 -15 11c-5 4 -9 9 -12 14 c-2 3 -3 5 -6 7c-2 1 -4 3 -6 5c-7 7 -7 18 -3 27c6 12 21 17 33 10c11 -6 15 -20 10 -31c2 -4 4 -7 6 -9s6 -3 6 -3zM424 192c14 0 25 -12 24 -26c-1 -12 -10 -21 -22 -22c-11 -1 -20 6 -24 15c-2 6 -7 9 -13 10c-2 0 -4 1 -6 1c-3 -33 -14 -64 -29 -90 c14 -11 23 -23 27 -36c0 0 1 0 2 -1l3 -3c9 -10 8 -25 -2 -34s-25 -8 -34 2c-6 7 -8 16 -5 24c2 6 1 14 -4 18c-2 1 -3 2 -5 4c-27 -32 -60 -52 -100 -54v215c53 3 100 37 122 82c0 0 4 2 6 4c3 3 4 6 6 9c-5 11 -1 2
 5 10 31c12 6 27 3 33 -9s3 -27 -9 -33v0 c-2 -1 -5 -4 -6 -6c-3 -5 -7 -11 -12 -15c-4 -4 -9 -7 -15 -11c7 -20 12 -42 13 -65c11 0 18 -2 25 -4c4 -1 8 -4 12 -6h3z"/>
+    <glyph glyph-name="ion-ios-help-circle" unicode="" d="M208 400c115 0 208 -93 208 -208s-93 -208 -208 -208s-208 93 -208 208s93 208 208 208zM204 96c12 0 21 9 21 21s-9 20 -21 20s-22 -8 -22 -20s10 -21 22 -21zM244 193c18 11 28 24 28 44c0 30 -25 51 -63 51c-40 0 -64 -23 -65 -55h36c1 15 11 25 26 25s27 -10 27 -22 s-7 -19 -24 -29c-18 -11 -25 -23 -23 -44v-8h35v8c0 13 6 20 23 30z"/>
+    <glyph glyph-name="ion-md-bulb" unicode="" horiz-adv-x="288" d="M80 20v18h128v-18c0 -11 -9 -20 -20 -20h-88c-11 0 -20 9 -20 20zM144 384c80 0 144 -60 144 -134c0 -46 -25 -87 -62 -111v-43c0 -11 -9 -19 -20 -19h-124c-11 0 -21 8 -21 19v43c-37 24 -61 65 -61 111c0 74 64 134 144 134z"/>
+    <glyph glyph-name="ion-ios-document" unicode="" horiz-adv-x="304" d="M208 293c-10 0 -19 9 -19 19v72c0 4 3 8 7 8v0c8 0 15 -3 21 -8l76 -63c7 -6 11 -14 11 -23v0c0 -3 -2 -5 -5 -5h-91zM163 312c0 -25 20 -45 45 -45h96v-243c0 -18 -14 -32 -32 -32h-240c-18 0 -32 14 -32 32v336c0 18 14 32 32 32h131v-80z"/>
+    <glyph glyph-name="ion-ios-search" unicode="" horiz-adv-x="384" d="M380 28c6 -7 6 -18 -1 -24c-3 -3 -7 -4 -11 -4s-9 2 -12 5l-106 107c-26 -21 -60 -34 -97 -34c-85 0 -153 69 -153 153s68 153 153 153s153 -69 153 -153c0 -36 -12 -69 -33 -95zM153 111c32 0 62 12 85 35s35 53 35 85s-12 62 -35 85s-53 35 -85 35s-62 -12 -85 -35 s-35 -53 -35 -85s12 -62 35 -85s53 -35 85 -35z"/>
+    <glyph glyph-name="ion-logo-youtube" unicode="" horiz-adv-x="512" d="M509 299c2 -36 3 -71 3 -107s-1 -71 -3 -107c0 -45 -33 -81 -74 -81c-58 -3 -118 -4 -179 -4s-121 1 -179 4c-41 0 -74 36 -74 81c-2 36 -3 71 -3 107s2 71 4 107c0 45 32 81 73 81c55 3 112 4 170 4h9h9c58 0 115 -1 170 -4c41 0 74 -36 74 -81zM207 94l145 98l-145 99 v-197z"/>
+    <glyph glyph-name="ion-md-aperture" unicode="" horiz-adv-x="448" d="M224 416c124 0 224 -100 224 -224s-100 -224 -224 -224s-224 100 -224 224s100 224 224 224zM360 56c36 36 56 85 56 136s-20 100 -56 136s-85 56 -136 56s-100 -20 -136 -56s-56 -85 -56 -136s20 -100 56 -136s85 -56 136 -56s100 20 136 56zM168 342l56 -86h-147 c17 40 50 71 91 86zM381 160h-102l72 129c21 -27 33 -60 33 -97c0 -11 -1 -22 -3 -32zM328 314l-49 -90l-78 126c8 1 15 2 23 2c40 0 76 -14 104 -38zM67 224h102l-72 -129c-21 27 -33 60 -33 97c0 11 1 22 3 32zM280 42l-56 86h147c-17 -40 -50 -71 -91 -86zM120 70l49 90 l78 -126c-8 -1 -15 -2 -23 -2c-40 0 -76 14 -104 38z"/>
+    <glyph glyph-name="ion-md-list" unicode="" horiz-adv-x="352" d="M0 120v48h256v-48h-256zM0 216v48h320v-48h-320zM0 312v48h352v-48h-352zM0 24v48h288v-48h-288z"/>
+    <glyph glyph-name="ion-ios-power" unicode="" horiz-adv-x="384" d="M316 336c43 -36 68 -89 68 -145c0 -105 -86 -191 -192 -191s-192 86 -192 191c0 56 25 109 68 145c3 3 8 5 12 5c5 0 9 -3 13 -7c3 -4 5 -8 5 -13s-3 -10 -7 -13c-35 -29 -55 -72 -55 -117c0 -85 70 -155 156 -155s156 70 156 155c0 45 -20 87 -55 117c-4 3 -7 8 -7 13 s2 9 5 13s9 7 14 7c4 0 8 -2 11 -5zM193 174c-10 0 -19 8 -19 18v174c0 10 9 18 19 18s18 -8 18 -18v-174c0 -10 -8 -18 -18 -18z"/>
+    <glyph glyph-name="ion-md-grid" unicode="" horiz-adv-x="384" d="M346 384c21 0 38 -17 38 -38v-308c0 -21 -17 -38 -38 -38h-308c-21 0 -38 17 -38 38v308c0 21 17 38 38 38h308zM115 38v77h-77v-77h77zM115 154v76h-77v-76h77zM115 269v77h-77v-77h77zM230 38v77h-76v-77h76zM230 154v76h-76v-76h76zM230 269v77h-76v-77h76zM346 38v77 h-77v-77h77zM346 154v76h-77v-76h77zM346 269v77h-77v-77h77z"/>
+    <glyph glyph-name="ion-ios-glasses" unicode="" horiz-adv-x="446" d="M432 207c8 0 14 -7 14 -15s-6 -15 -14 -15h-7c-3 -21 -13 -40 -28 -55c-17 -17 -40 -26 -64 -26c-51 0 -93 43 -93 96v0c0 6 -7 15 -17 15s-17 -9 -17 -15v0c0 -53 -42 -96 -93 -96c-24 0 -47 9 -64 26c-15 15 -25 34 -28 55h-7c-8 0 -14 7 -14 15s6 15 14 15h7 c3 21 13 40 28 55c17 16 40 26 64 26c20 0 40 -6 56 -19c14 -11 24 -25 30 -42c7 5 16 7 24 7s17 -2 24 -7c6 17 16 31 30 42c16 13 36 19 56 19c24 0 47 -9 64 -26c15 -15 25 -34 28 -55h7z"/>
+    <glyph glyph-name="ion-md-hand" unicode="" horiz-adv-x="414" d="M401 174c15 -15 18 -36 3 -51c0 0 -96 -103 -126 -121v0c-28 -21 -65 -34 -100 -34c-55 0 -101 36 -117 85v0v1c0 1 -1 2 -1 3l-58 189c-5 15 2 32 17 37s30 -3 35 -18l34 -88c1 -4 2 -2 2 1l-22 161c-3 15 7 30 22 33s30 -8 33 -23l25 -138c0 -2 2 -2 2 0v177 c0 15 13 28 28 28s28 -13 28 -28l8 -175c0 -5 3 -4 4 -1l23 135c2 15 16 27 31 25s27 -17 25 -32l-21 -172c-1 -12 -3 -30 2 -36c8 -9 20 -9 31 2l42 42c15 15 35 13 50 -2z"/>
+    <glyph glyph-name="ion-ios-bluetooth" unicode="" horiz-adv-x="256" d="M250 115c7 -6 7 -16 1 -23l-116 -120c-2 -2 -6 -4 -9 -4v0c-6 0 -11 5 -11 11v170l-85 -74c-7 -6 -19 -5 -25 2v0c-6 7 -5 18 2 24l108 91l-109 92c-7 6 -8 17 -2 24v0c6 7 18 7 25 1l86 -73v169c0 6 5 11 11 11v0c3 0 7 -2 9 -4l116 -121c6 -7 6 -17 -1 -23l-88 -76z M151 347c-1 1 -3 1 -3 -1v-119c0 -2 2 -3 3 -2l63 53c1 1 1 2 0 3zM148 38c0 -2 2 -2 3 -1l63 65c1 1 1 2 0 3l-63 53c-1 1 -3 0 -3 -2v-118z"/>
+    <glyph glyph-name="ion-ios-time" unicode="" d="M208 400c115 0 208 -93 208 -208s-93 -208 -208 -208s-208 93 -208 208s93 208 208 208zM222 174v146c0 8 -6 14 -14 14s-14 -6 -14 -14v-132h-82c-8 0 -14 -6 -14 -14s6 -14 14 -14h96c8 0 14 6 14 14z"/>
+    <glyph glyph-name="ion-ios-trending-up" unicode="" horiz-adv-x="448" d="M433 320c8 0 15 -7 15 -15v-118c0 -8 -7 -15 -15 -15s-15 7 -15 15v82l-150 -152c-3 -3 -6 -4 -10 -4s-8 1 -11 4l-86 84l-135 -133c-3 -3 -7 -4 -11 -4s-8 1 -11 4s-4 6 -4 10c0 6 2 9 4 11l146 143c3 3 7 5 11 5s8 -2 11 -5l86 -84l138 142h-84c-8 0 -15 7 -15 15 s7 15 15 15h121z"/>
+    <glyph glyph-name="ion-md-contact" unicode="" d="M208 400c114 0 208 -94 208 -208s-94 -208 -208 -208s-208 94 -208 208s94 208 208 208zM208 338c-34 0 -62 -29 -62 -63s28 -62 62 -62s62 28 62 62s-28 63 -62 63zM208 37c52 0 98 27 125 67c-1 42 -83 64 -125 64s-124 -22 -125 -64c27 -40 73 -67 125 -67z"/>
+    <glyph glyph-name="ion-md-close-circle" unicode="" d="M208 400c115 0 208 -93 208 -208s-93 -208 -208 -208s-208 93 -208 208s93 208 208 208zM312 117l-75 75l75 75l-29 29l-75 -75l-75 75l-29 -29l75 -75l-75 -75l29 -29l75 75l75 -75z"/>
+    <glyph glyph-name="ion-logo-sass" unicode="" horiz-adv-x="512" d="M512 119v-8c-1 -9 -7 -17 -15 -22s-12 -4 -13 -3s1 3 4 5c13 8 17 20 10 33c-5 10 -15 16 -26 20c-24 8 -48 7 -72 0c4 -12 7 -23 -2 -34c-10 -12 -23 -19 -39 -22c-7 -2 -14 2 -15 9c-3 19 17 36 30 47c-7 11 -11 24 -14 37c-15 -17 -31 -40 -26 -61 c3 -14 -1 -27 -14 -36s-30 -14 -46 -11c-4 1 -6 5 -5 8c2 14 25 32 35 43c2 2 3 4 1 7c-6 12 -10 23 -14 36c-13 -29 -30 -80 -57 -98c-9 -6 -17 -4 -21 5c-3 8 -2 20 -1 29c-6 -13 -11 -27 -20 -38c-7 -8 -24 -7 -30 1c-16 21 -13 50 -5 73l-29 -15c7 -16 8 -32 4 -49 c-5 -19 -17 -35 -35 -46c-19 -12 -64 -24 -77 4c-7 16 -4 30 6 44c17 24 48 37 74 50c-32 23 -80 42 -96 80c-15 35 17 68 43 89c59 48 166 95 244 59c15 -7 36 -24 31 -52c-4 -24 -13 -39 -28 -54c-32 -32 -152 -77 -176 -15c-1 1 -1 4 1 4c2 -1 20 -15 43 -16 c17 -1 36 3 52 8c33 11 71 33 80 67c4 13 -2 28 -15 34c-49 24 -116 -4 -159 -26c-33 -17 -84 -45 -82 -87c1 -37 56 -60 81 -82c15 7 42 15 52 27c11 13 26 25 44 26c8 0 14 -3 16 -10c2 -6 1 -11 0 -19c5 4 11 4
  15 -1c12 -14 -22 -50 -10 -66c14 14 21 42 30 59 c4 8 17 51 26 53c7 2 18 4 25 0c2 -1 3 -3 2 -5c-7 -19 -7 -33 3 -52c14 20 29 43 34 67c1 2 3 3 5 4c7 2 18 3 25 0c3 -1 3 -2 2 -5c-7 -22 -6 -38 6 -58c31 11 69 15 97 -5c-3 2 -7 5 0 0c5 -4 3 -2 0 0c12 -8 19 -18 21 -32zM113 94v0c1 7 -1 13 -3 20 c-24 -8 -66 -35 -64 -62c1 -10 8 -13 17 -12s19 7 27 13c14 11 22 24 23 41zM216 166c1 3 -2 5 -5 4c-29 -8 -43 -57 -33 -81c1 -3 4 -3 6 -1c17 16 28 56 32 78zM286 84c8 -3 27 17 26 27c-8 -9 -18 -18 -26 -27zM367 102c11 6 16 19 12 30c-8 -6 -21 -19 -20 -31 c0 -2 6 0 8 1z"/>
+    <glyph glyph-name="ion-ios-apps" unicode="" horiz-adv-x="384" d="M371 310c18 -8 18 -21 0 -29l-148 -67c-17 -8 -45 -8 -62 0l-148 67c-18 8 -18 21 0 29l150 68c16 7 43 7 59 0zM161 73c17 -8 45 -8 62 0l100 45c4 2 9 2 13 0l35 -16c18 -8 18 -21 0 -29l-148 -67c-17 -8 -45 -8 -62 0l-148 67c-18 8 -18 21 0 29l35 16c4 2 10 2 14 0z M371 206c18 -8 18 -21 0 -29l-148 -67c-17 -8 -45 -8 -62 0l-148 67c-18 8 -18 21 0 29l32 15c4 2 9 2 13 0l108 -49c15 -5 37 -5 52 0l108 49c4 2 9 2 13 0z"/>
+    <glyph glyph-name="ion-ios-outlet" unicode="" d="M311 370c3 0 6 0 8 -2c58 -37 97 -102 97 -176s-39 -138 -97 -176c-2 -2 -5 -2 -8 -2h-206c-3 0 -6 0 -8 2c-58 38 -97 102 -97 176s39 139 97 176c2 2 5 2 8 2h206zM145 177c2 0 4 2 4 4v111c0 2 -2 4 -4 4h-22c-2 0 -4 -2 -4 -4v-111c0 -2 2 -4 4 -4h22zM234 58 c2 0 4 2 4 4v41c0 16 -14 30 -30 30s-30 -15 -30 -31v-40c0 -2 2 -4 4 -4h52zM293 192c2 0 4 2 4 4v81c0 2 -2 4 -4 4h-22c-2 0 -4 -2 -4 -4v-81c0 -2 2 -4 4 -4h22z"/>
+    <glyph glyph-name="ion-ios-tablet-landscape" unicode="" horiz-adv-x="448" d="M7 345v0c4 4 11 7 17 7h400c6 0 13 -3 17 -7v0c4 -4 7 -11 7 -17v-272c0 -6 -3 -13 -7 -17v0c-4 -4 -11 -7 -17 -7h-400c-6 0 -13 3 -17 7v0c-4 4 -7 11 -7 17v272c0 6 3 13 7 17zM17 192c0 -4 3 -7 7 -7s7 3 7 7s-3 7 -7 7s-7 -3 -7 -7zM438 192c0 8 -6 14 -14 14 s-14 -6 -14 -14s6 -14 14 -14s14 6 14 14zM400 336h-352v-288h352v288z"/>
+    <glyph glyph-name="ion-md-arrow-dropdown-circle" unicode="" d="M208 400c115 0 208 -93 208 -208s-93 -208 -208 -208s-208 93 -208 208s93 208 208 208zM208 128l96 96h-192z"/>
+    <glyph glyph-name="ion-logo-flickr" unicode="" horiz-adv-x="384" d="M288 384c53 0 96 -43 96 -96v-192c0 -53 -43 -96 -96 -96h-192c-53 0 -96 43 -96 96v192c0 53 43 96 96 96h192zM120 144c26 0 48 22 48 48s-22 48 -48 48s-48 -22 -48 -48s22 -48 48 -48zM264 144c26 0 48 22 48 48s-22 48 -48 48s-48 -22 -48 -48s22 -48 48 -48z"/>
+    <glyph glyph-name="ion-md-tv" unicode="" horiz-adv-x="448" d="M407 372c22 0 41 -18 41 -40v-240c0 -22 -19 -40 -41 -40h-102v-40h-163v40h-101c-22 0 -41 18 -41 40v240c0 22 19 40 41 40h366zM407 92v0v240h-366v-240h366z"/>
+    <glyph glyph-name="ion-ios-bowtie" unicode="" horiz-adv-x="448" d="M259 234c10 -17 19 -53 17 -85c-1 -17 -5 -22 -5 -22s-12 -7 -36 -7h-11c-35 3 -40 15 -40 15s4 12 8 43s0 58 0 58s7 5 19 7c8 2 17 3 23 3v0c11 0 20 -4 25 -12zM174 239c0 0 3 -16 2 -32v-10c0 -3 -1 -6 -1 -10c0 -5 0 -10 -1 -15l-2 -10c-4 -21 -10 -34 -10 -34 s-77 -80 -110 -80c-28 0 -52 68 -52 144s26 144 52 144c35 0 100 -57 122 -97zM396 336c26 0 52 -68 52 -144s-24 -144 -52 -144c-37 0 -98 69 -106 79c0 0 4 5 5 22v9v8c0 11 -2 23 -4 33l-3 11c-3 10 -7 18 -10 24c0 1 -2 4 -2 4l1 1c22 40 83 97 119 97z"/>
+    <glyph glyph-name="ion-ios-barcode" unicode="" d="M74 60c4 0 7 -2 10 -5s4 -6 4 -10c0 -8 -7 -13 -15 -13h-56c-10 0 -17 6 -17 16v288c0 10 8 16 18 16h54c4 0 9 -1 12 -4s4 -6 4 -10c0 -8 -7 -14 -15 -14h-40c-2 0 -4 -2 -4 -4v-256c0 -2 2 -4 4 -4h41zM399 352c10 0 17 -6 17 -16v-288c0 -10 -7 -16 -17 -16h-57 c-4 0 -7 1 -10 4s-4 6 -4 10c0 8 7 14 15 14h40c2 0 4 2 4 4v256c0 2 -2 4 -4 4h-41c-5 0 -9 3 -12 8c0 1 -1 1 -1 2c-3 10 4 18 14 18h56zM74 272c8 0 14 -6 14 -14v-132c0 -8 -6 -14 -14 -14s-14 6 -14 14v132c0 8 6 14 14 14zM342 112c-8 0 -14 6 -14 14v132 c0 8 6 14 14 14s14 -6 14 -14v-132c0 -8 -6 -14 -14 -14zM278 304c8 0 14 -6 14 -14v-196c0 -8 -6 -14 -14 -14s-14 6 -14 14v196c0 8 6 14 14 14zM138 304c8 0 14 -6 14 -14v-196c0 -8 -6 -14 -14 -14s-14 6 -14 14v196c0 8 6 14 14 14zM208 288c8 0 14 -6 14 -14v-164 c0 -8 -6 -14 -14 -14s-14 6 -14 14v164c0 8 6 14 14 14z"/>
+    <glyph glyph-name="ion-ios-shirt" unicode="" horiz-adv-x="413" d="M413 266l-62 -64c-2 -3 -7 -1 -7 3v45c0 5 -5 10 -11 9c-4 -1 -7 -5 -7 -9v-61v-181c0 -4 -4 -8 -8 -8h-222c-4 0 -8 4 -8 8v181v61c0 5 -5 10 -11 9c-4 -1 -7 -5 -7 -9v-45c0 -4 -5 -6 -7 -3l-62 64c-1 2 -1 4 0 6c12 12 61 63 89 83c20 15 41 29 63 29s22 -8 54 -8 s32 8 54 8s42 -14 62 -29c28 -20 77 -71 89 -83c1 -2 2 -4 1 -6zM207 305c30 0 54 13 54 36c0 30 -24 11 -54 11s-54 19 -54 -11c0 -23 24 -36 54 -36z"/>
+    <glyph glyph-name="ion-md-home" unicode="" d="M160 0h-98v192h-62l208 192l208 -192h-62v-192h-98v128h-96v-128z"/>
+    <glyph glyph-name="ion-ios-cafe" unicode="" d="M322 44c8 0 14 -7 14 -15s-7 -13 -15 -13h-259c-8 0 -14 7 -14 15s6 13 14 13h260zM351 330v0c39 -14 65 -49 65 -88c0 -25 -11 -49 -30 -67s-45 -27 -72 -27h-6v0c-16 -28 -36 -50 -58 -64c-19 -12 -38 -20 -44 -20h-60c-9 0 -46 12 -78 49c-18 21 -33 47 -44 79 c-14 38 -22 84 -24 136c0 11 4 22 12 30c7 7 17 10 27 10h274c10 0 19 -3 26 -10c8 -8 12 -17 12 -28zM322 176v0c37 3 66 32 66 66c0 13 -5 26 -13 37c-7 9 -16 16 -26 21v0c-4 -47 -13 -90 -27 -124z"/>
+    <glyph glyph-name="ion-md-navigate" unicode="" horiz-adv-x="320" d="M160 384l160 -369l-15 -15l-145 64l-145 -64l-15 15z"/>
+    <glyph glyph-name="ion-md-arrow-round-down" unicode="" horiz-adv-x="332" d="M9 163c-12 12 -12 31 0 43s34 12 46 0l79 -83v214c0 17 14 31 32 31s32 -14 32 -31v-214l79 83c12 12 34 12 46 0s12 -31 0 -43l-134 -138c-6 -6 -14 -9 -23 -9v0c-9 0 -17 3 -23 9z"/>
+    <glyph glyph-name="ion-md-color-fill" unicode="" horiz-adv-x="448" d="M104 370l30 30l257 -256l-54 -11l-153 -149l-184 178l142 141zM288 162l-102 103l-103 -103h205zM391 144c0 0 57 -61 57 -92s-26 -56 -57 -56s-56 25 -56 56s56 92 56 92z"/>
+    <glyph glyph-name="ion-ios-star-half" unicode="" horiz-adv-x="448" d="M431 256c9 0 17 -7 17 -16c0 -4 -2 -8 -5 -11l-3 -2l-120 -86l46 -135c2 -7 1 -14 -5 -18c-3 -2 -6 -4 -9 -4s-7 2 -10 4l-118 84l-118 -84c-3 -2 -7 -4 -10 -4s-6 2 -9 4c-6 4 -7 12 -5 18l46 135l-121 85c-5 4 -7 7 -7 11v3c0 9 7 16 16 16v0h148l45 133 c2 6 8 11 15 11s13 -5 15 -11l45 -133h147zM384 221c3 2 1 7 -3 7h-97v0c-12 0 -23 8 -27 19l-29 88v0c-1 2 -4 2 -4 0v-210c0 -12 6 -23 16 -30v0l78 -56c3 -2 7 1 6 5l-30 88c-4 12 0 25 10 32z"/>
+    <glyph glyph-name="ion-ios-person" unicode="" horiz-adv-x="384" d="M384 10c1 -5 -3 -10 -8 -10h-368c-5 0 -9 5 -8 10c7 32 49 47 63 52c16 6 38 7 52 10c8 2 20 6 24 11s1 49 1 49s-7 12 -11 22s-8 38 -8 38s-8 0 -11 14c-3 16 -9 22 -8 34s7 11 7 11s-6 16 -7 51c-1 41 31 82 90 82c58 0 91 -41 90 -82c-1 -35 -7 -51 -7 -51s6 1 7 -11 s-5 -18 -8 -34c-3 -14 -11 -14 -11 -14s-4 -28 -8 -38s-12 -22 -12 -22s-2 -44 2 -49s16 -9 24 -11c14 -3 36 -4 52 -10c14 -5 56 -20 63 -52z"/>
+    <glyph glyph-name="ion-md-settings" unicode="" d="M366 171l47 -34c3 -3 4 -10 2 -14l-43 -71c-2 -4 -7 -7 -12 -5l-54 21c-12 -8 -23 -16 -36 -21l-8 -55c-1 -4 -6 -8 -11 -8h-85c-5 0 -10 3 -11 8l-8 55c-13 5 -25 13 -36 21l-54 -21c-4 -2 -10 1 -12 5l-43 71c-3 5 -2 11 2 14l45 34c0 7 -1 14 -1 21s1 14 1 21l-46 34 c-3 3 -4 10 -2 14l43 71c2 4 7 7 12 5l54 -21c12 8 23 16 36 21l8 55c1 4 6 8 11 8h85c5 0 10 -4 11 -8l7 -55c13 -5 25 -13 36 -21l53 21c4 2 11 -1 13 -5l43 -71c3 -5 2 -11 -2 -14l-45 -34c0 -7 1 -14 1 -21s0 -15 -1 -21zM207 119c41 0 75 32 75 73s-34 73 -75 73 s-74 -32 -74 -73s33 -73 74 -73z"/>
+    <glyph glyph-name="ion-ios-lock" unicode="" horiz-adv-x="320" d="M280 256c22 0 40 -18 40 -40v-192c0 -22 -18 -40 -40 -40h-240c-22 0 -40 18 -40 40v192c0 22 18 40 40 40h24v48c0 53 44 97 97 96s95 -44 95 -97v-47h24zM174 131c11 5 18 16 18 29c0 18 -15 33 -34 32c-16 -1 -29 -13 -30 -29c-1 -14 6 -26 18 -32v-69 c0 -8 7 -14 15 -14s13 6 13 14v69zM228 256v48c0 18 -7 35 -20 48s-30 20 -48 20s-35 -7 -48 -20s-20 -30 -20 -48v-48h136z"/>
+    <glyph glyph-name="ion-ios-egg" unicode="" horiz-adv-x="320" d="M160 416v0c64 0 160 -133 160 -257s-64 -191 -160 -191s-160 67 -160 191s96 257 160 257z"/>
+    <glyph glyph-name="ion-ios-repeat" unicode="" horiz-adv-x="384" d="M273 290c4 -4 6 -9 6 -15s-2 -11 -6 -15l-36 -36c-3 -3 -6 -4 -10 -4s-7 1 -10 4s-4 7 -4 11c0 3 2 7 4 9l16 16h-155c-13 0 -25 -5 -35 -15s-15 -22 -15 -35v-16c0 -8 -6 -14 -14 -14v0c-8 0 -14 6 -14 14v16c0 21 8 40 23 55s34 23 55 23h157l-16 16c-2 2 -4 6 -4 9 c0 5 2 10 7 13c1 1 2 2 3 2c5 1 10 0 14 -4zM370 204c8 0 14 -6 14 -14v-16c0 -43 -35 -78 -78 -78h-157l16 -16c2 -2 4 -6 4 -9c0 -5 -2 -10 -7 -13c-1 -1 -2 -1 -3 -1c-5 -1 -10 -1 -14 3l-34 34c-4 4 -6 9 -6 15s2 12 6 16l36 35c3 3 6 4 10 4s7 -1 10 -4s4 -7 4 -11 c0 -3 -2 -7 -4 -9l-16 -16h155c13 0 25 5 35 15s15 22 15 35v16c0 8 6 14 14 14v0z"/>
+    <glyph glyph-name="ion-ios-color-palette" unicode="" horiz-adv-x="384" d="M366 100c12 -11 14 -35 0 -48c-39 -34 -94 -52 -149 -52h-1c-55 0 -111 18 -153 55c-84 74 -84 195 0 269c45 40 102 60 158 60c48 0 94 -15 128 -45c46 -41 46 -108 0 -149l-30 -27c-14 -13 -14 -31 0 -44c7 -6 11 -8 23 -10c8 -1 17 -3 24 -9zM56 232 c0 -18 14 -32 32 -32s32 14 32 32s-14 32 -32 32s-32 -14 -32 -32zM96 106c18 0 32 14 32 32s-14 32 -32 32s-32 -14 -32 -32s14 -32 32 -32zM160 267c18 0 32 14 32 32s-14 32 -32 32s-32 -14 -32 -32s14 -32 32 -32zM232 48c26 0 48 22 48 48s-22 48 -48 48 s-48 -22 -48 -48s22 -48 48 -48zM256 256c18 0 32 14 32 32s-14 32 -32 32s-32 -14 -32 -32s14 -32 32 -32z"/>
+    <glyph glyph-name="ion-ios-alert" unicode="" d="M208 400c115 0 208 -93 208 -208s-93 -208 -208 -208s-208 93 -208 208s93 208 208 208zM225 290c0 10 -7 18 -17 18v0c-10 0 -17 -8 -17 -18l3 -115c0 -8 6 -15 14 -15v0c8 0 14 7 14 15zM208 94c11 0 19 8 19 18s-8 19 -19 19s-19 -9 -19 -19s8 -18 19 -18z"/>
+    <glyph glyph-name="ion-ios-bus" unicode="" horiz-adv-x="320" d="M14 31c11 -12 24 -9 58 -9c0 -10 1 -22 -3 -22h-55c-4 0 -5 7 -6 29c0 3 4 4 6 2zM306 31c2 2 6 1 6 -2c-1 -22 -2 -29 -6 -29h-55s-3 12 -3 22c34 0 47 -3 58 9zM288 384c18 0 32 -14 32 -32v0v-276c0 -22 -18 -40 -40 -40h-240c-22 0 -40 18 -40 40v276v0 c0 18 14 32 32 32h8h240h8zM66 72c12 1 21 10 22 22c1 15 -11 27 -26 26c-12 -1 -21 -10 -22 -22c-1 -15 11 -27 26 -26zM142 150c2 0 4 2 4 4v140c0 2 -2 4 -4 4h-98c-9 0 -16 -7 -16 -16v-116c0 -9 7 -16 16 -16h98zM258 72c12 1 21 10 22 22c1 15 -11 27 -26 26 c-12 -1 -21 -10 -22 -22c-1 -15 11 -27 26 -26zM292 166v116c0 9 -7 16 -16 16h-98c-2 0 -4 -2 -4 -4v-140c0 -2 2 -4 4 -4h98c9 0 16 7 16 16zM277 328c9 0 15 6 15 14s-6 14 -15 14h-234c-9 0 -15 -6 -15 -14s6 -14 15 -14h234z"/>
+    <glyph glyph-name="ion-md-beer" unicode="" horiz-adv-x="448" d="M384 308c47 0 64 -33 64 -68v-96c0 -35 -14 -68 -64 -68h-16v-33v-11c0 -35 -29 -64 -64 -64h-176c-35 0 -64 29 -64 64v173v2c-26 0 -48 22 -48 48v55c-10 11 -16 26 -16 42c0 35 29 64 64 64c18 0 34 -8 46 -20c10 12 26 20 43 20c14 0 27 -6 37 -14c12 9 26 14 42 14 c17 0 34 -6 46 -16c11 10 26 16 42 16c35 0 64 -29 64 -64c0 -15 -7 -29 -16 -38v-6h16zM320 256v49v15h-20c-3 -8 -7 -16 -13 -23c-13 -16 -33 -25 -55 -25c-26 0 -48 13 -61 33c-6 -2 -12 -3 -18 -3c-12 0 -23 4 -32 10c-2 1 -3 3 -5 4c-1 -2 -2 -3 -3 -5l-1 -1v-54h208z M64 320c9 0 18 4 24 11c2 2 3 4 4 6c0 0 6 10 12 11s20 6 30 -5c4 -5 11 -9 19 -9c3 0 5 1 7 2c3 1 7 2 9 4v0c3 2 6 4 10 4c7 0 12 -4 14 -10c0 0 1 -1 1 -2c1 -3 1 -5 3 -8c7 -12 20 -20 35 -20c13 0 26 6 33 16c5 7 7 15 7 24s9 8 9 8h55c6 0 13 -3 15 -9c1 3 1 6 1 9 c0 18 -14 32 -32 32c-9 0 -17 -4 -23 -10c-2 -2 -6 -5 -16 -6h-8c-6 1 -11 3 -13 5c-7 7 -17 11 -28 11c-9 0 -18 -4 -25 -9c-2 -2 -4 -5 -6 -7c0 -1 -1 -1 -2 -2c-3 -3 -7 -5 -11 -
 5c-5 0 -8 3 -11 6c-1 2 -2 3 -3 5s-3 4 -5 6c-4 4 -10 6 -16 6c-10 0 -18 -9 -25 -16 c-7 -8 -27 -8 -34 0s-18 16 -30 16c-18 0 -32 -14 -32 -32c0 -2 1 -5 1 -7c2 -9 7 -17 15 -21v-63c0 -9 7 -16 16 -16v75zM400 144v96c0 18 -6 24 -24 24h-8v-144h8c18 0 24 6 24 24z"/>
+    <glyph glyph-name="ion-md-snow" unicode="" horiz-adv-x="412" d="M411 150l-57 -16l58 -33l-24 -42l-57 33l16 -57l-31 -8l-24 88l-62 36v-71l64 -64l-22 -23l-42 42v-67h-48v66l-42 -42l-22 22l64 65v71l-61 -35l-23 -88l-31 8l15 57l-58 -33l-24 41l57 33l-57 15l8 31l88 -23l63 36l-63 36l-88 -24l-8 31l57 15l-57 33l24 42l58 -34 l-15 57l31 9l23 -88l61 -35v71l-64 65l22 22l42 -42v66h48v-67l42 42l22 -23l-64 -64v-71l62 36l24 88l31 -9l-16 -57l57 33l24 -42l-58 -33l57 -15l-8 -31l-88 24l-60 -35l60 -35l88 23z"/>
+    <glyph glyph-name="ion-md-download" unicode="" horiz-adv-x="320" d="M320 248l-160 -158l-160 158h91v136h138v-136h91zM0 45h320v-45h-320v45z"/>
+    <glyph glyph-name="ion-md-star-outline" unicode="" horiz-adv-x="404" d="M404 238l-110 -96l33 -142l-125 75l-125 -75l33 142l-110 96l145 12l57 134l57 -134zM219 103l60 -36l-16 68l-5 19l15 12l53 46l-70 6l-19 2l-8 18l-27 64l-27 -64l-8 -18l-19 -2l-70 -6l53 -46l15 -12l-5 -19l-16 -68l60 36l17 10z"/>
+    <glyph glyph-name="ion-ios-microphone" unicode="" horiz-adv-x="288" d="M203 244c-7 0 -12 -5 -12 -12s5 -12 12 -12h73c7 0 12 -5 12 -12s-5 -12 -12 -12h-73c-7 0 -12 -5 -12 -12s5 -12 12 -12h77c4 0 8 -4 8 -8c0 -14 -3 -27 -8 -39c-1 -3 -4 -5 -7 -5h-258c-3 0 -6 2 -7 5c-5 12 -8 25 -8 39c0 4 4 8 8 8h77c7 0 12 5 12 12s-5 12 -12 12 h-73c-7 0 -12 5 -12 12s5 12 12 12h73c7 0 12 5 12 12s-5 12 -12 12h-73c-7 0 -12 5 -12 12s5 12 12 12h73c7 0 12 5 12 12s-5 12 -12 12h-77c-4 0 -8 4 -8 8c2 47 31 88 73 106c5 2 11 -1 11 -7v-36c0 -7 5 -12 12 -12s12 5 12 12v41c0 7 5 12 12 12v0c7 0 12 -5 12 -12 v-57c0 -7 5 -12 12 -12s12 5 12 12v57c0 7 5 12 12 12v0c7 0 12 -5 12 -12v-41c0 -7 5 -12 12 -12s12 5 12 12v36c0 6 6 9 11 7c42 -18 71 -58 73 -106c0 -4 -4 -8 -8 -8h-77c-7 0 -12 -5 -12 -12s5 -12 12 -12h73c7 0 12 -5 12 -12s-5 -12 -12 -12h-73zM30 89c-2 3 0 7 3 7 h222c3 0 6 -4 4 -7c-19 -22 -46 -36 -76 -40c-4 0 -7 -4 -7 -8v-41c0 -18 -14 -32 -32 -32v0c-18 0 -32 14 -32 32v41c0 4 -3 8 -7 8c-30 4 -56 18 -75 40z"/>
+    <glyph glyph-name="ion-md-git-network" unicode="" horiz-adv-x="384" d="M384 352c0 -24 -13 -44 -32 -55v-93l-128 -64v-53c19 -11 32 -31 32 -55c0 -35 -29 -64 -64 -64s-64 29 -64 64c0 24 13 44 32 55v53l-128 64v93c-19 11 -32 31 -32 55c0 35 29 64 64 64s64 -29 64 -64c0 -24 -13 -44 -32 -55v-53l96 -48l96 48v53c-19 11 -32 31 -32 55 c0 35 29 64 64 64s64 -29 64 -64zM64 392c-22 0 -40 -18 -40 -40s18 -40 40 -40s40 18 40 40s-18 40 -40 40zM192 -8c22 0 40 18 40 40s-18 40 -40 40s-40 -18 -40 -40s18 -40 40 -40zM320 312c22 0 40 18 40 40s-18 40 -40 40s-40 -18 -40 -40s18 -40 40 -40z"/>
+    <glyph glyph-name="ion-md-code-working" unicode="" horiz-adv-x="448" d="M158 94l-30 -30l-128 128l128 128l30 -30l-98 -98zM290 94l98 98l-98 98l30 30l128 -128l-128 -128zM124 172v40h40v-40h-40zM324 212v-40h-40v40h40zM204 172v40h40v-40h-40z"/>
+    <glyph glyph-name="ion-md-phone-portrait" unicode="" horiz-adv-x="288" d="M247 416c23 0 41 -19 41 -41v-366c0 -22 -18 -41 -41 -41h-206c-23 0 -41 19 -41 41v366c0 22 18 41 41 41h206zM252 48v288h-216v-288h216z"/>
+    <glyph glyph-name="ion-ios-videocam" unicode="" horiz-adv-x="448" d="M419 294h20c5 0 9 -3 9 -8v-188c0 -5 -4 -9 -9 -9h-20c-3 0 -7 1 -10 3l-86 55c-2 2 -4 4 -4 7v76c0 3 2 5 4 7l86 55c3 2 7 2 10 2zM242 64h-190c-28 0 -52 23 -52 51v154c0 28 23 51 51 51h190c28 0 52 -23 52 -51v-153c0 -28 -23 -52 -51 -52z"/>
+    <glyph glyph-name="ion-logo-no-smoking" unicode="" horiz-adv-x="448" d="M328 144v48h16v-48h-16zM80 144v48h82l48 -48h-130zM332 388c69 -38 116 -112 116 -196c0 -124 -100 -224 -224 -224c-24 0 -47 4 -68 11c-2 1 -3 1 -5 2c-12 4 -24 9 -35 15h-1c-69 38 -115 112 -115 196c0 124 100 224 224 224c24 0 47 -4 68 -11c2 -1 3 -1 5 -2 c12 -4 24 -9 35 -15v0zM224 22c37 0 72 12 100 32l-238 238c-20 -28 -32 -63 -32 -100c0 -65 36 -121 90 -150c6 -3 13 -7 19 -9c2 -1 3 -1 5 -2c9 -3 19 -5 29 -7c9 -2 18 -2 27 -2zM362 93c20 28 32 62 32 99c0 66 -37 123 -91 151c-6 3 -12 6 -18 8c-2 1 -3 1 -5 2 c-9 3 -19 5 -29 7c-9 2 -18 2 -27 2c-37 0 -71 -12 -99 -32zM320 192v-34l-34 34h34zM352 144v48h16v-48h-16zM328 235c16 -7 16 -23 16 -30v-1h-16v1c0 8 -1 13 -7 16c-4 2 -13 4 -39 4h-3c-13 0 -24 0 -33 13c-5 8 -5 19 -2 29c-4 1 -8 1 -12 3c-17 7 -26 20 -26 38 c0 33 27 44 39 44v-16c-1 0 -23 -2 -23 -28c0 -11 5 -19 16 -23c9 -4 18 -3 18 -3c3 0 6 -2 7 -5s2 -6 0 -8c-5 -7 -7 -18 -4 -22c4 -6 7 -6 20 -6h3c24 0 37 -2 46 -6zM368 204v0h-16c0 
 28 -4 37 -7 42c-6 10 -14 14 -25 14h-30c-3 0 -5 2 -6 4s-1 6 0 8c0 0 8 18 6 32 c-1 8 -6 17 -27 17v16c24 0 40 -11 43 -30c2 -11 0 -23 -3 -31h17c16 0 30 -8 39 -22c6 -10 9 -24 9 -50z"/>
+    <glyph glyph-name="ion-md-add" unicode="" horiz-adv-x="320" d="M320 171h-139v-139h-42v139h-139v42h139v139h42v-139h139v-42z"/>
+    <glyph glyph-name="ion-md-arrow-dropup" unicode="" horiz-adv-x="256" d="M0 128l128 128l128 -128h-256z"/>
+    <glyph glyph-name="ion-ios-checkmark-circle" unicode="" d="M208 400c115 0 208 -93 208 -208s-93 -208 -208 -208s-208 93 -208 208s93 208 208 208zM314 250c2 2 2 3 0 5l-17 18c-1 1 -2 2 -3 2v0c-1 0 -2 -1 -3 -2l-122 -123l-44 45c-1 1 -2 1 -3 1s-2 0 -3 -1l-18 -18c-2 -2 -2 -4 0 -6l56 -56c4 -4 8 -5 12 -5c5 0 10 3 12 5v0 z"/>
+    <glyph glyph-name="ion-ios-link" unicode="" d="M232 107c15 1 23 -17 13 -27l-66 -65c-20 -20 -46 -31 -74 -31s-54 11 -74 31s-31 46 -31 74s10 55 30 75l69 68c8 8 18 15 28 20c6 3 12 5 19 7c1 0 3 1 4 1c5 1 11 3 16 3h7c4 0 7 -1 11 -1h6c22 -4 42 -14 58 -30c12 -12 20 -25 24 -41c2 -8 -2 -17 -10 -19v0h-1 c-8 -2 -16 2 -19 10c-4 10 -9 19 -17 27c-12 12 -29 20 -47 21h-4h-11c-5 -1 -10 -2 -15 -4s-9 -4 -14 -7c-4 -3 -8 -6 -12 -10l-68 -68c-14 -14 -22 -32 -22 -52s8 -37 22 -51s31 -22 51 -22s38 8 52 22l64 64c3 2 6 5 10 5h1zM385 369c20 -20 31 -46 31 -74 s-10 -55 -30 -75l-69 -68c-8 -8 -18 -15 -28 -20c-6 -3 -12 -5 -19 -7c-6 -2 -14 -3 -20 -3c-3 0 -4 -1 -7 -1c-4 0 -7 1 -11 1c-24 2 -47 13 -64 30v0v0c-12 12 -20 24 -24 38c-3 9 3 18 12 20c8 2 16 -2 19 -10c4 -10 9 -18 16 -25c12 -12 29 -20 47 -21h4h11 c5 1 10 2 15 4s9 5 14 8c4 3 8 5 12 9l68 68c14 14 22 32 22 52s-8 37 -22 51s-31 22 -51 22s-37 -8 -51 -22l-64 -63c-3 -3 -8 -5 -13 -5c-14 1 -20 18 -10 28l64 63c20 20 46 31 74 31s54 -11 74 -31z"/>
+    <glyph glyph-name="ion-md-checkbox-outline" unicode="" horiz-adv-x="384" d="M105 233l66 -66l183 183l30 -30l-213 -213l-96 96zM341 43v170h43v-170c0 -23 -20 -43 -43 -43h-298c-23 0 -43 20 -43 43v298c0 23 20 43 43 43h213v-43h-213v-298h298z"/>
+    <glyph glyph-name="ion-md-cloud-upload" unicode="" horiz-adv-x="480" d="M387 231c52 -3 93 -46 93 -99c0 -55 -45 -100 -100 -100h-260c-66 0 -120 54 -120 120c0 62 47 113 107 119c25 48 75 81 133 81c73 0 133 -52 147 -121zM272 172h68l-100 100l-100 -100h68v-76h64v76z"/>
+    <glyph glyph-name="ion-logo-instagram" unicode="" horiz-adv-x="384" d="M272 352h-160c-21 0 -41 -9 -56 -24s-24 -35 -24 -56v-160c0 -21 9 -41 24 -56s35 -24 56 -24h160c21 0 41 9 56 24s24 35 24 56v160c0 21 -9 41 -24 56s-35 24 -56 24zM272 384v0c62 0 112 -50 112 -112v-160c0 -62 -50 -112 -112 -112h-160c-62 0 -112 50 -112 112v160 c0 62 50 112 112 112h160zM296 272c-13 0 -24 11 -24 24s11 24 24 24s24 -11 24 -24s-11 -24 -24 -24zM192 256c-35 0 -64 -29 -64 -64s29 -64 64 -64s64 29 64 64s-29 64 -64 64zM192 288v0c53 0 96 -43 96 -96s-43 -96 -96 -96s-96 43 -96 96s43 96 96 96z"/>
+    <glyph glyph-name="ion-md-volume-high" unicode="" horiz-adv-x="384" d="M0 256h85l107 112v-352l-107 112h-85v128zM288 192c0 -38 -21 -73 -53 -88v177c32 -16 53 -51 53 -89zM235 384c85 -20 149 -99 149 -192s-64 -172 -149 -192v45c62 19 106 78 106 147s-44 128 -106 147v45z"/>
+    <glyph glyph-name="ion-md-help-buoy" unicode="" d="M208 400c115 0 208 -93 208 -208s-93 -208 -208 -208s-208 93 -208 208s93 208 208 208zM269 44c19 8 37 20 52 35s27 33 35 52l-40 12c-6 -13 -14 -25 -24 -35s-22 -18 -35 -24zM147 340c-19 -8 -37 -20 -52 -35s-27 -33 -35 -52l40 -12c6 13 14 25 24 35s22 18 35 24z M208 256c-35 0 -64 -29 -64 -64s29 -64 64 -64s64 29 64 64s-29 64 -64 64zM321 305c-15 15 -33 27 -52 35l-12 -40c13 -6 25 -14 35 -24s18 -22 24 -35l40 12c-8 19 -20 37 -35 52zM95 79c15 -15 33 -27 52 -35l12 40c-13 6 -25 14 -35 24s-18 22 -24 35l-40 -12 c8 -19 20 -37 35 -52z"/>
+    <glyph glyph-name="ion-md-restaurant" unicode="" horiz-adv-x="415" d="M116 168l-91 89c-34 33 -34 86 0 119l153 -148zM264 206l-33 -31l150 -145l-30 -30l-150 146l-150 -146l-30 30s163 159 211 207c-15 32 -4 77 31 111c41 40 101 49 132 17c33 -31 24 -89 -17 -129c-34 -34 -80 -45 -114 -30z"/>
+    <glyph glyph-name="ion-md-body" unicode="" horiz-adv-x="384" d="M192 400c22 0 40 -18 40 -40s-18 -40 -40 -40s-40 18 -40 40s18 40 40 40zM384 256h-128v-272h-43v144h-42v-144h-43v272h-128v43h384v-43z"/>
+    <glyph glyph-name="ion-md-crop" unicode="" d="M304 135v153h-153v38h153c21 0 38 -17 38 -38v-153h-38zM112 102h304v-38h-74v-80h-38v80h-192c-21 0 -38 17 -38 38v186h-74v38h74v74h38v-298z"/>
+    <glyph glyph-name="ion-md-chatbubbles" unicode="" d="M85 80h219v-7c0 -17 -13 -31 -30 -31h-216l-58 -58v274c0 17 13 30 30 30h10v-163c0 -20 25 -45 45 -45zM381 400c19 0 35 -16 35 -35v-314l-76 53h-238c-19 0 -35 16 -35 35v226c0 19 16 35 35 35h279z"/>
+    <glyph glyph-name="ion-logo-rss" unicode="" horiz-adv-x="384" d="M56 112c31 0 56 -25 56 -56s-25 -56 -56 -56s-56 25 -56 56s25 56 56 56zM0 256c140 0 256 -116 256 -256h-80c0 48 -14 94 -48 128s-80 48 -128 48v80zM0 384c212 0 384 -172 384 -384h-80c0 171 -133 304 -304 304v80z"/>
+    <glyph glyph-name="ion-md-outlet" unicode="" horiz-adv-x="448" d="M338 416c61 0 110 -51 110 -113v-222c0 -62 -49 -113 -110 -113h-228c-61 0 -110 51 -110 113v222c0 62 49 113 110 113h228zM143 197v102c0 11 -9 21 -20 21h-23c-11 0 -19 -9 -20 -20v-2v-101v-1c0 -11 9 -20 20 -20h23c11 0 20 9 20 20v1zM265 50v31 c0 23 -18 42 -41 42s-41 -19 -41 -42v-29c0 -12 9 -20 20 -20h42c11 0 20 7 20 18zM368 197v102c0 11 -9 21 -20 21h-24c-11 0 -19 -9 -20 -20v-2v-101v-1c0 -11 9 -20 20 -20h24c11 0 20 9 20 20v1z"/>
+    <glyph glyph-name="ion-ios-information-circle-outline" unicode="" d="M187 276c0 11 10 20 21 20s21 -9 21 -20s-10 -20 -21 -20s-21 9 -21 20zM189 240h38v-144h-38v144zM208 372c-48 0 -93 -19 -127 -53s-53 -79 -53 -127s19 -93 53 -127s79 -53 127 -53s93 19 127 53s53 79 53 127s-19 93 -53 127s-79 53 -127 53zM208 400v0 c115 0 208 -93 208 -208s-93 -208 -208 -208s-208 93 -208 208s93 208 208 208z"/>
+    <glyph glyph-name="ion-md-flash-off" unicode="" horiz-adv-x="408" d="M408 12l-24 -24l-140 140l-72 -144v192h-64v88l-108 107l24 25zM300 240l-24 -48l-168 168v40h192l-64 -160h64z"/>
+    <glyph glyph-name="ion-md-videocam" unicode="" horiz-adv-x="448" d="M352 228l96 98v-268l-96 98v-86c0 -13 -11 -24 -25 -24h-302c-14 0 -25 11 -25 24v244c0 13 11 24 25 24h302c14 0 25 -11 25 -24v-86z"/>
+    <glyph glyph-name="ion-ios-square-outline" unicode="" horiz-adv-x="320" d="M292 352c15 0 28 -13 28 -28v-264c0 -15 -13 -28 -28 -28h-264c-15 0 -28 13 -28 28v264c0 15 13 28 28 28h264zM292 68v248c0 4 -4 8 -8 8h-248c-4 0 -8 -4 -8 -8v-248c0 -4 4 -8 8 -8h248c4 0 8 4 8 8z"/>
+    <glyph glyph-name="ion-ios-information-circle" unicode="" d="M208 400c115 0 208 -93 208 -208s-93 -208 -208 -208s-208 93 -208 208s93 208 208 208zM227 96v144h-38v-144h38zM208 256c11 0 21 9 21 20s-10 20 -21 20s-21 -9 -21 -20s10 -20 21 -20z"/>
+    <glyph glyph-name="ion-ios-heart-dislike" unicode="" horiz-adv-x="448" d="M392 21c6 -6 6 -14 0 -20c-2 -2 -5 -2 -7 -3c-5 -1 -9 -1 -13 3l-368 362c-6 6 -6 14 0 20s14 6 20 0zM32 279c0 10 1 20 4 30l270 -265c-36 -31 -66 -52 -66 -52s-101 72 -161 154c-32 43 -47 96 -47 133zM402 146c-16 -22 -33 -42 -52 -61l-282 277c20 18 47 30 76 30 v0v0h1c40 0 75 -21 95 -52c20 31 55 52 95 52h1v0v0c62 -1 112 -51 112 -113c0 -37 -14 -90 -46 -133z"/>
+    <glyph glyph-name="ion-md-search" unicode="" horiz-adv-x="384" d="M274 143l110 -110l-33 -33l-109 110v17l-7 6c-25 -21 -58 -34 -93 -34c-79 0 -142 63 -142 142s63 143 141 143c79 0 142 -64 142 -143c0 -36 -13 -68 -34 -93l7 -5h18zM142 143c55 0 99 43 99 98s-44 99 -99 99s-98 -44 -98 -99s43 -98 98 -98z"/>
+    <glyph glyph-name="ion-ios-download" unicode="" horiz-adv-x="320" d="M280 304c22 0 40 -18 40 -40v-240c0 -22 -18 -40 -40 -40h-240c-22 0 -40 18 -40 40v240c0 22 18 40 40 40h107v-180l-48 47c-5 5 -13 5 -18 0s-5 -13 0 -18l70 -69v0c1 -1 2 -2 4 -3s3 -1 5 -1c3 0 7 2 9 4l70 69c5 5 5 13 0 18s-13 5 -18 0l-48 -47v180h107zM173 387 v-83h-26v83c0 7 6 13 13 13s13 -6 13 -13z"/>
+    <glyph glyph-name="ion-ios-boat" unicode="" horiz-adv-x="400" d="M296 26c0 0 27 -24 63 -36c2 -1 1 -4 -1 -4c-23 2 -40 6 -62 12c-22 -10 -67 -14 -96 -14s-73 5 -96 14c-21 -6 -39 -9 -62 -12c-2 0 -3 3 -1 4c36 12 63 36 63 36c26 -9 66 -15 96 -15s70 6 96 15zM390 185c6 -3 10 -9 10 -16c0 -2 0 -5 -1 -7l-55 -136 c-28 0 -57 22 -57 22c-18 -9 -53 -17 -79 -19h-8h-8c-26 2 -61 10 -79 19c0 0 -29 -22 -57 -22l-55 136c-1 2 -1 5 -1 7c0 7 4 13 10 16v0l187 85c2 1 4 1 6 0l187 -85v0zM156 160c7 0 12 10 12 22s-5 22 -12 22s-12 -10 -12 -22s5 -22 12 -22zM244 160c7 0 12 10 12 22 s-5 22 -12 22s-12 -10 -12 -22s5 -22 12 -22zM336 321l18 -97c1 -3 -2 -5 -5 -4l-26 11c-1 0 -2 2 -2 3l-16 86c-3 11 -7 16 -16 16h-177c-9 0 -15 -5 -17 -16l-16 -86c0 -1 -1 -3 -2 -3l-25 -11c-3 -1 -7 1 -6 4l18 97c5 26 17 47 40 47h34l2 8c4 14 16 24 31 24h57 c15 0 27 -10 31 -24l2 -8h35c22 0 34 -20 40 -47v0z"/>
+    <glyph glyph-name="ion-ios-volume-off" unicode="" horiz-adv-x="384" d="M326 -16c-5 0 -9 3 -12 7l-228 388c-4 7 -2 15 5 19s15 2 19 -5l228 -388c4 -7 2 -15 -5 -19c-2 -1 -5 -2 -7 -2zM69 248l51 41l40 -69v-124c0 -6 -3 -11 -9 -14c-2 -1 -5 -2 -7 -2c-4 0 -7 2 -10 4l-65 52h-53c-9 0 -16 7 -16 16v80c0 9 7 16 16 16h53zM384 192v0 c0 -53 -19 -103 -53 -143l-13 22c27 34 42 77 42 121v0v0c0 53 -20 102 -58 139c-5 5 -5 12 0 17s11 5 16 0c42 -42 66 -97 66 -156v0zM321 192c0 -31 -8 -60 -23 -85l-15 24c9 19 14 39 14 61v0v0c0 37 -14 72 -40 98c-5 5 -5 12 0 17s12 5 17 0c30 -31 47 -71 47 -115v0v0 zM254 192c0 -3 -1 -7 -1 -10l-48 82c0 1 1 1 2 2c5 5 12 4 17 -1c19 -20 30 -45 30 -73v0v0z"/>
+    <glyph glyph-name="ion-logo-reddit" unicode="" horiz-adv-x="449" d="M259 160c0 18 14 31 32 31s32 -13 32 -31s-14 -32 -32 -32s-32 14 -32 32zM127 160c0 18 14 31 32 31s32 -13 32 -31s-14 -32 -32 -32s-32 14 -32 32zM449 197c0 -20 -11 -37 -27 -45c1 -5 1 -9 1 -14c0 -76 -88 -138 -198 -138s-199 61 -199 137c0 5 0 10 1 15 c-16 8 -27 25 -27 45c0 28 23 50 50 50c13 0 24 -5 33 -13c33 23 79 39 129 41h7l29 98l86 -17c7 16 22 28 41 28c25 0 44 -20 44 -45s-19 -45 -44 -45c-24 0 -45 19 -45 44l-69 14l-24 -77c50 -2 96 -18 129 -41c9 8 21 13 34 13c27 0 49 -22 49 -50zM375 366 c-15 0 -27 -12 -27 -27s12 -27 27 -27s27 12 27 27s-12 27 -27 27zM18 197c0 -11 6 -21 14 -27c7 19 20 38 37 53c-5 4 -12 6 -19 6c-18 0 -32 -14 -32 -32zM355 55c32 22 50 52 50 83v9c-1 6 -3 12 -5 18c-6 16 -15 30 -30 43c-4 4 -10 8 -15 12v0c-35 24 -80 37 -130 37 s-96 -13 -131 -37v0c-5 -4 -11 -8 -15 -12c-15 -13 -24 -27 -30 -43c-2 -6 -4 -12 -5 -18v-9c0 -31 18 -61 50 -83c35 -24 81 -37 131 -37s95 13 130 37zM417 170c8 6 14 16 14 27c0 18 -13 32 -31 
 32c-7 0 -15 -2 -20 -6c17 -15 30 -34 37 -53zM289 90l13 -12 c-1 -1 -29 -30 -77 -30s-77 29 -78 30l13 12s24 -24 65 -24c40 0 64 24 64 24z"/>
+    <glyph glyph-name="ion-ios-flashlight" unicode="" horiz-adv-x="170" d="M146 416c18 0 24 -14 24 -32h-170c0 18 6 32 24 32h122zM25 300c-16 18 -25 33 -25 68h170c0 -33 -10 -50 -25 -68c-8 -9 -12 -19 -12 -31v-266c0 -22 -18 -35 -40 -35h-16c-22 0 -40 13 -40 35v266c0 12 -4 22 -12 31zM57 210v-36c0 -16 13 -28 28 -28s28 12 28 28v36 c0 16 -13 28 -28 28s-28 -12 -28 -28zM65 175c0 11 9 20 20 20s20 -9 20 -20s-9 -20 -20 -20s-20 9 -20 20z"/>
+    <glyph glyph-name="ion-md-qr-scanner" unicode="" d="M48 324v-68h-48v68c0 34 27 60 61 60h67v-48h-68c-7 0 -12 -5 -12 -12zM356 384c33 0 60 -26 60 -60v-68h-48v68c0 7 -6 12 -13 12h-67v48h68zM368 61v67h48v-67c0 -34 -26 -61 -60 -61h-68v49h68c7 0 12 5 12 12zM60 49h68v-49h-67c-34 0 -61 27 -61 61v67h48v-67 c0 -7 5 -12 12 -12z"/>
+    <glyph glyph-name="ion-ios-navigate" unicode="" d="M208 400c115 0 208 -93 208 -208s-93 -208 -208 -208s-208 93 -208 208s93 208 208 208zM208 64l96 224l-224 -96h128v-128z"/>
+    <glyph glyph-name="ion-md-power" unicode="" d="M231 400v-231h-46v231h46zM343 350c45 -38 73 -95 73 -158c0 -115 -93 -208 -208 -208s-208 93 -208 208c0 63 28 120 73 158l33 -33c-36 -30 -60 -74 -60 -125c0 -89 72 -162 162 -162s162 73 162 162c0 51 -24 96 -60 125z"/>
+    <glyph glyph-name="ion-ios-stopwatch" unicode="" horiz-adv-x="384" d="M352 304l-26 26c-3 3 -3 8 0 11l12 11c3 3 8 3 11 0l25 -25c3 -3 3 -8 0 -11l-11 -12c-3 -3 -8 -3 -11 0zM21 304l-11 12c-3 3 -3 8 0 11l25 25c3 3 8 3 11 0l12 -11c3 -3 3 -8 0 -11l-26 -26c-3 -3 -8 -3 -11 0zM216 366c95 -12 168 -92 168 -190 c0 -106 -86 -192 -192 -192s-192 86 -192 192c0 98 73 178 168 190v18c0 9 7 16 16 16h16c9 0 16 -7 16 -16v-18zM206 147c11 5 18 16 18 29s-7 24 -18 29v99c0 8 -6 14 -14 14s-14 -6 -14 -14v-99c-11 -5 -18 -16 -18 -29s7 -24 18 -29v-19c0 -8 6 -14 14 -14s14 6 14 14 v19z"/>
+    <glyph glyph-name="ion-md-key" unicode="" d="M201 224h215v-64h-43v-64h-69v64h-103c-14 -40 -53 -72 -98 -72c-57 0 -103 47 -103 104s44 104 101 104c45 0 86 -32 100 -72zM104 158c19 0 34 15 34 34s-15 34 -34 34s-35 -15 -35 -34s16 -34 35 -34z"/>
+    <glyph glyph-name="ion-ios-arrow-down" unicode="" horiz-adv-x="336" d="M168 154l127 127c9 9 25 9 34 0s9 -25 0 -34l-144 -144c-9 -9 -24 -10 -33 -1l-145 145c-5 5 -7 11 -7 17s2 12 7 17c9 9 25 9 34 0z"/>
+    <glyph glyph-name="ion-logo-linkedin" unicode="" horiz-adv-x="384" d="M353 384c18 0 31 -13 31 -30v-321c0 -17 -13 -33 -31 -33h-320c-18 0 -33 16 -33 33v321c0 17 15 30 33 30h320zM119 64v171h-55v-171h55zM93 261c18 0 30 13 30 29c0 17 -11 30 -29 30s-30 -13 -30 -30c0 -16 11 -29 29 -29v0zM320 64v100c0 51 -27 75 -63 75 c-29 0 -42 -17 -50 -28v24h-55v-171h55v97c0 5 0 10 2 14c4 10 13 20 28 20c20 0 28 -15 28 -37v-94h55z"/>
+    <glyph glyph-name="ion-ios-arrow-dropdown" unicode="" d="M100 237c8 8 19 8 27 0l81 -81l81 79c8 8 19 8 27 0c4 -4 6 -8 6 -13s-2 -10 -6 -14l-94 -94c-8 -7 -20 -6 -27 1l-95 95c-8 8 -8 19 0 27zM0 192c0 115 93 208 208 208s208 -93 208 -208s-93 -208 -208 -208s-208 93 -208 208zM332 316c-33 33 -77 52 -124 52 s-91 -19 -124 -52s-52 -77 -52 -124s19 -91 52 -124s77 -52 124 -52s91 19 124 52s52 77 52 124s-19 91 -52 124z"/>
+    <glyph glyph-name="ion-ios-settings" unicode="" horiz-adv-x="384" d="M352 192c0 -21 14 -39 32 -46c-5 -20 -13 -40 -24 -57c-6 3 -13 4 -20 4c-13 0 -25 -5 -35 -15c-15 -15 -18 -36 -10 -54c-17 -11 -37 -19 -57 -24c-7 18 -25 32 -46 32s-39 -14 -46 -32c-20 5 -40 13 -57 24c8 18 5 39 -10 54c-10 10 -22 15 -35 15c-7 0 -14 -1 -20 -4 c-11 17 -19 37 -24 57c18 7 32 25 32 46s-14 39 -32 46c5 20 13 39 24 57c6 -3 13 -4 20 -4c13 0 25 4 35 14c15 15 18 37 10 55c17 11 37 19 57 24c7 -18 25 -32 46 -32s39 14 46 32c20 -5 40 -13 57 -24c-8 -18 -5 -40 10 -55c10 -10 22 -14 35 -14c7 0 14 1 20 4 c11 -17 19 -37 24 -57c-19 -7 -32 -25 -32 -46zM193 112c44 0 80 36 80 80s-36 80 -80 80s-80 -36 -80 -80s36 -80 80 -80z"/>
+    <glyph glyph-name="ion-md-sad" unicode="" d="M208 160c45 0 83 -26 102 -64h-204c19 38 57 64 102 64zM208 400c114 0 208 -94 208 -208s-93 -208 -208 -208s-208 94 -208 208s93 208 208 208zM208 26c92 0 166 74 166 166s-74 166 -166 166s-166 -74 -166 -166s74 -166 166 -166zM281 213c-18 0 -31 13 -31 31 s13 31 31 31s31 -13 31 -31s-13 -31 -31 -31zM135 213c-18 0 -31 13 -31 31s13 31 31 31s31 -13 31 -31s-13 -31 -31 -31z"/>
+    <glyph glyph-name="ion-ios-tennisball" unicode="" d="M202 -16c-110 3 -199 93 -202 203c8 1 15 1 23 1c48 0 94 -18 128 -52s53 -80 53 -128c0 -8 -1 -16 -2 -24zM182 374c0 -115 92 -207 207 -207c9 0 18 0 26 1c-11 -96 -89 -173 -186 -183c1 8 1 15 1 23c0 115 -92 207 -207 207c-7 0 -15 0 -22 -1c10 96 87 172 182 184 c-1 -8 -1 -16 -1 -24zM261 246c-34 34 -53 80 -53 128c0 9 1 18 2 26c112 -2 204 -93 206 -205c-9 -1 -18 -2 -27 -2c-48 0 -94 19 -128 53zM184 399v0v0v0z"/>
+    <glyph glyph-name="ion-ios-card" unicode="" horiz-adv-x="448" d="M448 320v-48h-448v48c0 18 14 32 32 32h384c18 0 32 -14 32 -32zM0 64v160h448v-160c0 -18 -14 -32 -32 -32h-384c-18 0 -32 14 -32 32zM142 112c-8 0 -14 -6 -14 -14v0c0 -8 6 -14 14 -14h164c8 0 14 6 14 14v0c0 8 -6 14 -14 14h-164zM70 112c-8 0 -14 -6 -14 -14v0 c0 -8 6 -14 14 -14h12c8 0 14 6 14 14v0c0 8 -6 14 -14 14h-12z"/>
+    <glyph glyph-name="ion-md-create" unicode="" horiz-adv-x="384" d="M0 80l236 236l80 -80l-236 -236h-80v80zM378 298l-40 -40l-80 80l40 40c9 9 20 9 29 0l51 -51c9 -9 9 -20 0 -29z"/>
+    <glyph glyph-name="ion-logo-vk" unicode="" horiz-adv-x="448" d="M395 149c41 -38 53 -58 53 -70c0 -6 -3 -9 -6 -11c-5 -4 -22 -4 -34 -4h-28c-9 0 -18 -1 -26 2c-6 2 -12 5 -18 9c-12 9 -21 21 -31 32c-5 6 -10 11 -17 15c-8 4 -14 2 -19 -4c-6 -6 -7 -18 -7 -26s1 -18 -6 -24c-2 -1 -4 -2 -6 -2c-4 -1 -17 -2 -33 -2s-68 8 -105 49 c-36 39 -62 79 -82 120c-11 23 -30 55 -30 66c0 5 5 9 10 9h66c9 0 15 -4 19 -11s10 -26 23 -52c13 -25 25 -45 41 -50c2 0 8 -1 10 3c4 7 5 18 6 25c4 29 3 46 -5 67c-2 5 -7 7 -11 9c-7 3 -13 2 -11 5c4 5 12 10 19 12s21 4 40 4s40 -1 48 -7c1 -1 4 -4 5 -6 c5 -11 2 -29 1 -55c-1 -46 3 -51 5 -54s5 -3 6 -3c2 0 4 0 6 1c8 4 26 26 40 52c13 25 18 36 21 44s8 16 14 16h74c16 0 20 -3 20 -14c0 -17 -18 -39 -32 -58c-34 -45 -40 -53 -40 -59c0 -3 0 -5 1 -8c3 -8 10 -12 19 -20z"/>
+    <glyph glyph-name="ion-ios-wifi" unicode="" horiz-adv-x="448" d="M224 352c82 0 163 -33 222 -88c3 -3 3 -9 0 -12l-27 -28c-3 -3 -8 -3 -11 0c-23 22 -51 39 -80 51c-33 14 -68 21 -104 21s-71 -7 -104 -21c-29 -12 -56 -29 -79 -51c-3 -3 -9 -3 -12 0l-27 28c-3 3 -2 9 1 12c58 55 139 88 221 88zM81 170c-3 3 -3 9 0 12 c38 34 88 55 143 55s104 -21 142 -55c4 -3 4 -9 1 -12l-29 -28c-3 -3 -8 -3 -11 0c-28 25 -65 39 -103 39s-75 -14 -103 -39c-3 -3 -8 -3 -11 0zM224 124c23 0 44 -10 59 -26c3 -3 3 -8 0 -11l-53 -53c-3 -3 -9 -3 -12 0l-53 53c-3 3 -3 8 0 11c15 16 36 26 59 26z"/>
+    <glyph glyph-name="ion-logo-vimeo" unicode="" horiz-adv-x="448" d="M445 334c7 -33 1 -66 -13 -97s-31 -59 -51 -87c-26 -36 -52 -71 -85 -101c-19 -18 -41 -35 -66 -44c-31 -11 -52 -5 -71 22c-14 19 -22 40 -28 63c-12 45 -25 90 -38 134c-4 12 -9 22 -15 33c-3 5 -8 10 -12 14c-5 4 -11 3 -17 0c-10 -6 -28 -18 -28 -18l-21 27 c26 24 82 71 82 71c11 9 33 25 48 27c21 3 38 -4 50 -22c11 -17 15 -37 18 -57c7 -41 11 -82 23 -122c3 -10 8 -20 13 -29c7 -11 15 -13 25 -5c4 3 7 6 10 10c20 24 37 52 48 81c3 9 3 17 3 27c0 13 -10 25 -25 26s-24 -1 -39 -7c6 24 24 62 54 84c29 21 71 24 94 16 s36 -23 41 -46z"/>
+    <glyph glyph-name="ion-ios-checkmark-circle-outline" unicode="" d="M315 255c2 -2 1 -4 0 -6l-134 -134v0c-2 -2 -7 -5 -12 -5c-4 0 -7 1 -11 5l-56 56c-2 2 -2 4 0 6l17 18c1 1 2 1 3 1s2 0 3 -1l44 -45l122 123c1 1 2 1 3 1v0c1 0 2 0 3 -1zM208 372c-48 0 -93 -19 -127 -53s-53 -79 -53 -127s19 -93 53 -127s79 -53 127 -53 s93 19 127 53s53 79 53 127s-19 93 -53 127s-79 53 -127 53zM208 400v0c115 0 208 -93 208 -208s-93 -208 -208 -208s-208 93 -208 208s93 208 208 208z"/>
+    <glyph glyph-name="ion-md-battery-full" unicode="" horiz-adv-x="224" d="M194 371c16 0 30 -14 30 -30v-343c0 -16 -13 -30 -30 -30h-164c-16 0 -30 13 -30 30v343c0 16 14 30 30 30h37v45h90v-45h37z"/>
+    <glyph glyph-name="ion-md-color-wand" unicode="" horiz-adv-x="384" d="M137 291l247 -254l-36 -37l-248 253zM117 316v68h37v-68h-37zM117 118v68h37v-68h-37zM206 235v37h69v-37h-69zM242 332l-48 -48l-25 26l47 48zM105 310l-26 -26l-47 48l26 26zM32 170l47 48l26 -26l-47 -48zM0 235v37h65v-37h-65z"/>
+    <glyph glyph-name="ion-md-analytics" unicode="" horiz-adv-x="448" d="M347 270c4 -1 9 -2 13 -2c11 0 22 4 30 10l58 -39v-212c0 -24 -19 -43 -43 -43h-362c-24 0 -43 19 -43 43v57l61 57c7 -4 15 -6 24 -6c13 0 24 5 33 13l58 -33c0 -3 -1 -6 -1 -9c0 -26 22 -48 48 -48s48 22 48 48c0 12 -4 22 -11 30zM85 231c-26 0 -48 -22 -48 -48 c0 -6 1 -12 3 -17l-40 -38v229c0 24 19 43 43 43h362c24 0 43 -19 43 -43v-80l-41 28c1 4 1 7 1 11c0 26 -22 48 -48 48s-48 -22 -48 -48c0 -10 3 -20 9 -28l-87 -135c-3 1 -7 1 -11 1c-12 0 -22 -4 -30 -11l-60 34v6c0 26 -22 48 -48 48z"/>
+    <glyph glyph-name="ion-ios-compass" unicode="" d="M232 168c1 -1 1 -3 0 -4l-98 -49c-2 -1 -4 1 -3 3l49 98c1 1 3 1 4 0zM208 400c115 0 208 -93 208 -208s-93 -208 -208 -208s-208 93 -208 208s93 208 208 208zM249 151v0l81 160c1 2 -1 4 -3 3l-159 -81h-1l-81 -160c-1 -2 1 -4 3 -3z"/>
+    <glyph glyph-name="ion-logo-dribbble" unicode="" horiz-adv-x="384" d="M192 384c106 0 192 -86 192 -192s-86 -192 -192 -192s-192 86 -192 192s86 192 192 192zM314 296c-17 -25 -47 -47 -85 -64c5 -10 9 -20 13 -31c33 3 75 4 110 2c-2 35 -16 68 -38 93zM192 352c-14 0 -26 -2 -39 -5c23 -23 43 -53 62 -87c34 14 62 33 79 55 c-28 23 -63 37 -102 37zM120 334c-41 -21 -71 -58 -83 -103h25c44 0 85 6 121 17c-19 34 -40 63 -63 86zM32 200v-8c0 -40 15 -78 40 -106c24 45 67 83 122 106c4 2 8 3 14 4c-3 8 -6 16 -10 24c-40 -13 -88 -20 -138 -21c-9 0 -19 1 -28 1zM192 32c20 0 39 3 57 10 c-3 25 -8 53 -15 79c-4 17 -10 33 -16 49c-7 -2 -13 -4 -17 -6c-48 -22 -86 -56 -107 -98c27 -21 61 -34 98 -34zM278 58c40 26 68 67 73 116c-30 2 -68 3 -99 0c4 -12 9 -25 12 -38c7 -27 11 -52 14 -78z"/>
+    <glyph glyph-name="ion-ios-fitness" unicode="" horiz-adv-x="448" d="M320 168h-192c-4 0 -8 4 -8 8v32c0 4 4 8 8 8h192c4 0 8 -4 8 -8v-32c0 -4 -4 -8 -8 -8zM346 86v212c0 12 10 22 22 22v0c12 0 22 -10 22 -22v-212c0 -12 -10 -22 -22 -22v0c-12 0 -22 10 -22 22zM428 256c11 0 20 -9 20 -20v-88c0 -11 -9 -20 -20 -20h-12 c-4 0 -8 4 -8 8v112c0 4 4 8 8 8h12zM102 86c0 -12 -10 -22 -22 -22v0c-12 0 -22 10 -22 22v212c0 12 10 22 22 22v0c12 0 22 -10 22 -22v-212zM32 256c4 0 8 -4 8 -8v-112c0 -4 -4 -8 -8 -8h-12c-11 0 -20 9 -20 20v88c0 11 9 20 20 20h12z"/>
+    <glyph glyph-name="ion-ios-aperture" unicode="" d="M202 77h179c-25 -37 -61 -66 -103 -81l-22 22l-55 55c-1 1 -1 4 1 4zM119 113c0 2 2 3 3 2l127 -127c-13 -3 -27 -4 -41 -4c-32 0 -62 7 -89 20v109zM208 400c32 0 63 -7 90 -20v-109c0 -2 -3 -3 -4 -2l-126 127c13 3 26 4 40 4zM20 281h110c2 0 2 -2 1 -3l-127 -127 c-3 13 -4 27 -4 41c0 32 7 62 20 89zM139 388l77 -77c1 -1 1 -4 -1 -4h-180c25 37 61 66 104 81zM119 227c0 1 0 2 1 3l50 50c1 1 2 1 3 1h71c1 0 1 0 2 -1l50 -50c1 -1 2 -2 2 -3v-70c0 -1 -1 -2 -2 -3l-50 -50c-1 -1 -1 -1 -2 -1h-71c-1 0 -2 0 -3 1l-50 50 c-1 1 -1 2 -1 3v70zM412 232c3 -13 4 -26 4 -40c0 -32 -7 -62 -20 -89h-109c-2 0 -2 2 -1 3zM324 365c37 -25 65 -61 80 -104l-77 -77c-1 -1 -3 0 -3 2v179zM12 122l78 78c1 1 3 0 3 -2v-179c-37 25 -66 60 -81 103z"/>
+    <glyph glyph-name="ion-md-document" unicode="" horiz-adv-x="320" d="M192 400l128 -128v-248c0 -22 -18 -40 -40 -40h-240c-22 0 -40 18 -40 40v336c0 22 18 40 40 40h152zM176 256h112l-112 112v-112z"/>
+    <glyph glyph-name="ion-ios-bicycle" unicode="" horiz-adv-x="448" d="M93 159c-17 0 -34 -8 -46 -20s-19 -29 -19 -46s7 -34 19 -46s29 -19 46 -19s34 7 46 19s20 29 20 46s-8 35 -20 47s-29 19 -46 19zM93 187v0c51 0 94 -43 94 -94s-43 -93 -94 -93s-93 42 -93 93s42 94 93 94zM288 320c-18 0 -32 14 -32 32s14 32 32 32s32 -14 32 -32 s-14 -32 -32 -32zM355 159c-17 0 -34 -8 -46 -20s-20 -29 -20 -46s8 -34 20 -46s29 -19 46 -19s34 7 46 19s19 29 19 46s-7 35 -19 47s-29 19 -46 19zM355 187v0c51 0 93 -43 93 -94s-42 -93 -93 -93s-94 42 -94 93s43 94 94 94zM336 256c9 0 16 -7 16 -16s-7 -16 -16 -16 h-57s-19 24 -38 52l-49 -52c18 -14 48 -25 48 -38v-90c0 -9 -7 -16 -16 -16s-16 7 -16 16v66s-16 16 -62 42c-6 4 -18 10 -18 27c0 8 3 17 9 23l72 69c6 6 14 9 22 9c11 0 21 -7 27 -16l30 -60h48z"/>
+    <glyph glyph-name="ion-ios-clipboard" unicode="" horiz-adv-x="352" d="M60 288c0 9 7 16 16 16h200c9 0 16 -7 16 -16v-240c0 -9 -7 -16 -16 -16h-200c-9 0 -16 7 -16 16v240zM242 367c8 0 14 -7 14 -15v-18c0 -1 -1 -2 -2 -2h-156c-1 0 -2 1 -2 2v16c0 10 8 17 18 17h14c0 6 1 11 3 16c2 6 6 11 10 16c9 10 21 17 35 17s27 -7 36 -17 c4 -5 8 -10 10 -16c2 -5 2 -10 2 -16h18zM179 352c7 1 12 6 13 13c1 10 -8 19 -18 18c-7 -1 -12 -7 -13 -14c-1 -10 8 -18 18 -17zM320 400c18 0 32 -14 32 -32v-368c0 -17 -15 -32 -32 -32h-288c-18 0 -32 14 -32 56v344c0 18 14 32 32 32h76c2 0 3 -2 2 -3 c-3 -4 -6 -9 -8 -16c0 -1 0 -1 -1 -1h-1c-22 0 -27 -18 -27 -37v-9c0 -1 -1 -2 -2 -2h-22c-9 0 -16 -7 -16 -16v-297c0 -9 7 -16 16 -16h253c9 0 16 7 16 16v297c0 9 -7 16 -16 16h-20c-1 0 -2 1 -2 2v14v1v0c-1 19 -8 34 -27 34h-1c-1 0 -2 0 -2 1c-2 5 -4 9 -7 13 c-1 1 0 3 2 3h75z"/>
+    <glyph glyph-name="ion-ios-arrow-dropright-circle" unicode="" d="M0 192c0 115 93 208 208 208s208 -93 208 -208s-93 -208 -208 -208s-208 93 -208 208zM244 192l-79 -81c-8 -8 -8 -19 0 -27c4 -4 8 -6 13 -6s10 2 14 6l94 94c7 8 6 20 -1 27l-95 95c-8 8 -19 8 -27 0s-8 -19 0 -27z"/>
+    <glyph glyph-name="ion-md-mail-unread" unicode="" horiz-adv-x="464" d="M336 320c0 35 29 64 64 64s64 -29 64 -64s-29 -64 -64 -64s-64 29 -64 64zM351 245c14 -9 31 -15 49 -15s34 5 48 14v-185c0 -24 -19 -43 -43 -43h-362c-24 0 -43 19 -43 43v266c0 24 19 43 43 43h281c-9 -14 -14 -30 -14 -48s5 -36 15 -50l-101 -67l-176 117v-43 l176 -117z"/>
+    <glyph glyph-name="ion-md-shirt" unicode="" horiz-adv-x="384" d="M256 384l128 -32v-96l-77 16l13 -272h-256l13 272l-77 -16v96l128 32c11 -19 40 -32 64 -32s53 13 64 32z"/>
+    <glyph glyph-name="ion-ios-finger-print" unicode="" d="M403 266c8 -24 13 -50 13 -73v-11c0 -7 -7 -13 -14 -13v0c-4 0 -8 1 -10 4s-4 6 -4 10c0 2 1 7 1 10c0 20 -5 42 -12 63s-17 38 -27 48c-5 5 -5 15 0 20c3 3 6 4 10 4v0c4 0 7 -1 10 -4c13 -13 24 -34 33 -58zM95 337c-4 0 -9 2 -11 6c-2 3 -3 6 -2 10s2 7 5 9 c35 25 76 38 121 38c47 0 92 -17 131 -49c6 -5 7 -13 2 -19c-3 -3 -7 -5 -11 -5c-3 0 -6 1 -8 3c-34 28 -73 43 -114 43c-39 0 -75 -12 -105 -33c-2 -2 -5 -3 -8 -3zM27 193c0 -30 6 -53 15 -82c1 -4 1 -8 -1 -11s-5 -5 -9 -6c-1 0 -3 -1 -4 -1c-6 0 -11 4 -13 10 c-10 31 -15 57 -15 90c0 50 18 99 51 137c3 3 7 4 11 4v0c3 0 6 -1 8 -3c3 -2 5 -5 5 -9s-1 -7 -3 -10c-29 -33 -45 -75 -45 -119zM308 318c26 -23 46 -55 57 -94c12 -43 16 -96 10 -155c-1 -7 -7 -12 -14 -12h-1c-8 1 -13 7 -12 15c5 55 2 106 -9 145c-10 34 -27 62 -49 81 s-50 29 -79 29c-40 0 -76 -14 -101 -41c-23 -25 -36 -59 -33 -95c2 -31 6 -48 9 -63c5 -23 9 -42 6 -92c0 -7 -7 -13 -14 -13c-1 0 -3 1 -4 1c-6 2 -10 7 -10 14c3 46 -1 64 -5 85c-3 16 -7 33 -9 66c-3 43 1
 1 84 40 115c30 32 73 50 121 50c36 0 70 -12 97 -36zM326 180 c5 -31 7 -56 7 -77c0 -42 -5 -76 -5 -77c-1 -7 -7 -12 -14 -12h-2c-4 1 -7 3 -9 6s-3 6 -2 10c0 0 4 33 4 73c0 20 -2 43 -7 73c-5 31 -16 57 -32 75c-16 17 -36 26 -58 26c-15 0 -29 -3 -42 -9c-2 -1 -4 -1 -6 -1c-4 0 -8 2 -11 5s-4 8 -3 12s4 7 8 9c16 8 35 12 54 12 c30 0 56 -12 77 -34s35 -53 41 -91zM137 244c-11 -16 -14 -35 -10 -59c9 -54 18 -116 10 -174c-1 -7 -7 -12 -14 -12h-1c-4 0 -7 2 -9 5s-3 7 -3 11c8 54 -1 114 -10 165c-5 31 0 58 15 80c3 4 7 6 11 6c3 0 7 -1 9 -3c5 -5 6 -13 2 -19zM270 60c7 0 13 -5 13 -13 c0 -1 1 -21 -3 -41c-1 -6 -6 -11 -13 -11h-5c-6 2 -10 9 -9 16c3 17 3 35 3 35c0 8 5 14 13 14h1zM247 242c11 -10 19 -26 24 -44c8 -28 11 -67 12 -114c0 -4 -2 -7 -4 -10c-3 -3 -5 -4 -9 -4v0c-8 0 -14 7 -14 14c-1 45 -5 82 -12 108c-3 12 -13 39 -38 39 c-11 0 -20 -4 -26 -11c-6 -8 -9 -21 -6 -37c13 -65 16 -127 9 -184c-1 -7 -7 -12 -14 -12h-1c-7 1 -13 7 -12 15c7 54 4 114 -8 176c-6 31 2 50 10 60c11 13 28 20 48 20c15 0 30 -6 41 -16zM217 202c8 -19 14 -51 16 -
 90c3 -39 2 -80 -2 -116c-1 -7 -6 -12 -13 -12h-2 c-4 0 -7 2 -9 5s-3 6 -3 10c4 33 4 73 2 110s-7 67 -14 83c-3 7 0 15 7 18c2 1 3 1 5 1v0c6 0 11 -4 13 -9z"/>
+    <glyph glyph-name="ion-logo-buffer" unicode="" horiz-adv-x="384" d="M7 284c-9 4 -9 11 0 15l169 82c4 2 10 3 16 3s12 -1 16 -3l169 -82c9 -4 9 -11 0 -15l-169 -82c-4 -2 -10 -3 -16 -3s-12 1 -16 3zM377 200c9 -4 9 -12 0 -16l-169 -81c-4 -2 -10 -4 -16 -4s-12 2 -16 4l-169 81c-9 4 -9 12 0 16c0 0 27 13 33 16c5 3 7 3 13 0 s123 -60 123 -60c4 -2 10 -3 16 -3s12 1 16 3c0 0 121 59 125 61s5 2 9 0s35 -17 35 -17zM377 100c9 -4 9 -11 0 -15l-169 -82c-4 -2 -10 -3 -16 -3s-12 1 -16 3l-169 82c-9 4 -9 11 0 15c0 0 27 14 33 17c5 3 7 2 13 -1s123 -59 123 -59c4 -2 10 -3 16 -3s12 1 16 3 c0 0 121 58 125 60s5 2 9 0s35 -17 35 -17z"/>
+    <glyph glyph-name="ion-ios-arrow-forward" unicode="" horiz-adv-x="192" d="M134 192l-127 127c-9 9 -9 25 0 34s25 9 34 0l144 -144c9 -9 10 -24 1 -33l-145 -145c-5 -5 -11 -7 -17 -7s-12 2 -17 7c-9 9 -9 25 0 34z"/>
+    <glyph glyph-name="ion-md-construct" unicode="" horiz-adv-x="480" d="M415 55c7 -5 7 -18 -2 -25l-41 -41c-7 -7 -17 -7 -24 0l-114 113l-96 -112c-8 -8 -19 -8 -27 0l-43 43c-10 8 -10 21 -2 27l105 105l-14 15c-41 -16 -89 -8 -123 26c-35 35 -44 88 -23 130l78 -75l53 53l-76 75c42 19 96 12 131 -23c34 -34 43 -80 27 -121l13 -13v0 l58 -58v0zM478 231c2 -2 2 -5 0 -7l-55 -56c-2 -2 -6 -2 -8 0l-35 34c-2 2 -2 6 0 8l10 10l-14 13c-6 4 -17 4 -30 0l-31 -37l-57 57l29 28s-1 47 -17 60s-45 29 -45 29v30c30 0 55 -7 91 -25s63 -45 70 -52s17 -18 23 -27s5 -18 5 -18l18 -16l4 4c2 2 6 2 8 0z"/>
+    <glyph glyph-name="ion-ios-restaurant" unicode="" horiz-adv-x="288" d="M272 384c5 0 16 0 16 -11v-357c0 -9 -7 -16 -16 -16s-16 7 -16 16v128s-32 23 -32 80c0 83 23 160 48 160zM176 384h8s24 -107 24 -128s-13 -38 -32 -45v-195c0 -9 -7 -16 -16 -16s-16 7 -16 16v195c-19 7 -32 24 -32 45s24 128 24 128h8l-10 -104c0 -4 4 -8 8 -8 s8 4 8 8l6 104h8l6 -104c0 -4 4 -8 8 -8s8 4 8 8zM48 384c26 0 48 -64 48 -128c0 -21 -13 -38 -32 -45v-195c0 -9 -7 -16 -16 -16s-16 7 -16 16v195c-19 7 -32 24 -32 45c0 64 22 128 48 128z"/>
+    <glyph glyph-name="ion-md-repeat" unicode="" horiz-adv-x="384" d="M85 296v0v-83h-42v125h256v62l85 -83l-85 -83v62h-214zM299 88v0v83h42v-125h-256v-62l-85 83l85 83v-62h214z"/>
+    <glyph glyph-name="ion-ios-reverse-camera" unicode="" horiz-adv-x="384" d="M354 288c18 0 30 -13 30 -31v-176c0 -18 -12 -33 -30 -33h-320c-18 0 -34 15 -34 33v176c0 18 16 31 34 31h7v8c0 4 4 8 8 8h26c4 0 8 -4 8 -8v-8h4c4 0 9 2 12 5c28 32 40 43 52 43h85c12 0 23 -11 51 -43c3 -3 7 -5 12 -5h55zM244 108c1 4 1 8 -2 11c-2 2 -5 4 -8 4 c-2 0 -5 -2 -7 -3c-11 -8 -23 -11 -35 -11c-15 0 -30 5 -42 16c-10 9 -16 21 -19 34h20c1 0 1 1 2 1c3 2 4 5 2 8l-32 42c-1 1 -3 2 -5 2s-3 -1 -4 -2l-31 -42s-1 -1 -1 -2c0 -4 3 -6 6 -6h20c3 -19 12 -37 27 -51c16 -15 36 -22 57 -22c17 0 34 4 48 14c2 2 3 5 4 7z M301 175c2 3 1 7 -2 9h-2h-21c-3 20 -13 40 -26 52c-16 14 -36 23 -58 23c-17 0 -34 -5 -48 -15c-2 -1 -3 -4 -4 -6c-1 -4 -1 -9 2 -12c2 -2 5 -3 8 -3c2 0 5 1 7 2c11 7 23 11 35 11c15 0 29 -5 41 -16c10 -10 17 -22 20 -36h-21h-2c-3 -2 -4 -6 -2 -9l33 -43v-1 c3 -2 7 -1 9 1z"/>
+    <glyph glyph-name="ion-md-move" unicode="" horiz-adv-x="448" d="M448 192l-96 -96v64h-96v-96h64l-96 -96l-96 96h64v96h-96v-64l-96 96l96 96v-64h96v96h-64l96 96l96 -96h-64v-96h96v64z"/>
+    <glyph glyph-name="ion-ios-book" unicode="" d="M98 384c54 0 100 -28 100 -82v-16v0v0v0v-284c0 -2 -3 -3 -4 -1v0c-10 34 -38 68 -82 71c-34 2 -67 -13 -93 -37c-2 -2 -5 -3 -7 -3h-4c-4 0 -8 3 -8 6v290c0 32 44 56 98 56zM318 384c54 0 98 -24 98 -56v-290c0 -3 -4 -6 -8 -6h-4c-2 0 -5 2 -7 3c-29 26 -60 39 -93 37 c-43 -3 -72 -36 -82 -70v-1c-1 -2 -4 -1 -4 1v284v0v0v0v16c0 54 46 82 100 82z"/>
+    <glyph glyph-name="ion-ios-cloud-outline" unicode="" horiz-adv-x="512" d="M288 324c-40 0 -77 -20 -100 -53l-7 -9c-4 -5 -10 -7 -16 -5l-11 4c-5 2 -10 4 -16 4c-11 0 -22 -5 -30 -14s-14 -20 -14 -32v-11c0 -6 -4 -10 -9 -12l-10 -4c-14 -4 -25 -13 -34 -25c-8 -12 -13 -27 -13 -41c0 -17 7 -33 19 -46s28 -20 44 -20h330c16 0 32 7 44 20 s19 29 19 46c0 15 -5 30 -15 42s-23 21 -37 23l-11 2c-6 1 -10 5 -11 11l-1 11c-3 30 -17 57 -39 77s-52 32 -82 32zM288 352v0c77 0 141 -59 149 -134c43 -8 75 -47 75 -92c0 -50 -41 -94 -91 -94h-330c-50 0 -91 44 -91 94c0 41 26 80 66 93c1 39 33 74 72 74 c10 0 19 -3 27 -6c27 39 72 65 123 65z"/>
+    <glyph glyph-name="ion-ios-arrow-dropdown-circle" unicode="" d="M0 192c0 115 93 208 208 208s208 -93 208 -208s-93 -208 -208 -208s-208 93 -208 208zM289 235l-81 -79l-81 81c-8 8 -19 8 -27 0s-8 -19 0 -27l95 -95c7 -7 19 -8 27 -1l94 94c4 4 6 9 6 14s-2 9 -6 13c-8 8 -19 8 -27 0z"/>
+    <glyph glyph-name="ion-md-pin" unicode="" horiz-adv-x="320" d="M160 416c88 0 160 -71 160 -157c0 -118 -160 -291 -160 -291s-160 173 -160 291c0 86 72 157 160 157zM160 203c32 0 57 25 57 56s-25 56 -57 56s-57 -25 -57 -56s25 -56 57 -56z"/>
+    <glyph glyph-name="ion-ios-list-box" unicode="" d="M0 365c0 19 16 35 35 35h346c19 0 35 -16 35 -35v-346c0 -19 -16 -35 -35 -35h-346c-19 0 -35 16 -35 35v346zM89 68c9 1 16 8 17 17c2 13 -9 25 -22 23c-9 -1 -16 -8 -17 -17c-2 -13 9 -25 22 -23zM89 172c9 1 16 8 17 17c2 13 -9 25 -22 23c-9 -1 -16 -8 -17 -17 c-2 -13 9 -25 22 -23zM89 276c9 1 16 8 17 17c2 13 -9 25 -22 23c-9 -1 -16 -8 -17 -17c-2 -13 9 -25 22 -23zM337 74c8 0 14 6 14 14v0c0 8 -6 14 -14 14h-180c-8 0 -14 -6 -14 -14v0c0 -8 6 -14 14 -14h180zM337 178c8 0 14 6 14 14v0c0 8 -6 14 -14 14h-180 c-8 0 -14 -6 -14 -14v0c0 -8 6 -14 14 -14h180zM337 282c8 0 14 6 14 14v0c0 8 -6 14 -14 14h-180c-8 0 -14 -6 -14 -14v0c0 -8 6 -14 14 -14h180z"/>
+    <glyph glyph-name="ion-ios-disc" unicode="" d="M208 282c50 0 90 -40 90 -90s-40 -90 -90 -90s-90 40 -90 90s40 90 90 90zM208 152c22 0 40 18 40 40s-18 40 -40 40s-40 -18 -40 -40s18 -40 40 -40zM208 400c115 0 208 -93 208 -208s-93 -208 -208 -208s-208 93 -208 208s93 208 208 208zM208 78c63 0 114 51 114 114 s-51 114 -114 114s-114 -51 -114 -114s51 -114 114 -114z"/>
+    <glyph glyph-name="ion-ios-cube" unicode="" d="M393 318c6 -3 6 -11 0 -14l-179 -94c-4 -2 -8 -2 -12 0l-179 94c-6 3 -6 11 0 14l179 81c4 2 8 2 12 0zM232 -15c-5 -3 -11 1 -11 7v187c0 3 1 6 4 7l180 93c5 3 11 -1 11 -7v-185c0 -3 -2 -6 -4 -7zM0 272c0 6 6 10 11 7l180 -93c3 -1 4 -4 4 -7v-187 c0 -6 -6 -10 -11 -7l-180 95c-2 1 -4 4 -4 7v185z"/>
+    <glyph glyph-name="ion-md-thunderstorm" unicode="" horiz-adv-x="448" d="M361 287c48 -3 87 -44 87 -93c0 -52 -41 -82 -93 -82h-94l18 36h-33l33 67h-99v-103h-68c-62 0 -112 38 -112 100c0 55 38 100 91 111c6 1 15 2 21 2c50 0 91 -31 106 -75h33c-6 24 -19 47 -37 65c-25 25 -57 37 -91 40c25 31 61 45 101 45c68 0 124 -48 137 -113z M180 80v32h81l-59 -128v96h-22z"/>
+    <glyph glyph-name="ion-md-male" unicode="" d="M368 400h48v-48v-128h-48v94l-107 -107c17 -24 27 -52 27 -83c0 -80 -64 -144 -144 -144s-144 64 -144 144s64 144 144 144c31 0 59 -10 83 -27l107 107h-94v48h128zM144 32c53 0 96 43 96 96s-43 96 -96 96s-96 -43 -96 -96s43 -96 96 -96z"/>
+    <glyph glyph-name="ion-ios-color-wand" unicode="" horiz-adv-x="384" d="M382 42c3 -3 3 -8 0 -11l-29 -29c-3 -3 -8 -3 -11 0l-245 246c-3 3 -3 8 0 11l29 29c3 3 8 3 11 0zM356 28c3 3 3 8 0 11l-190 191c-3 3 -8 3 -11 0v0c-3 -3 -3 -9 0 -12l190 -190c3 -3 8 -3 11 0v0zM129 328c-7 0 -12 5 -12 12v32c0 7 5 12 12 12s12 -5 12 -12v-32 c0 -7 -5 -12 -12 -12zM129 185c7 0 12 -5 12 -12v-32c0 -7 -5 -12 -12 -12s-12 5 -12 12v32c0 7 5 12 12 12zM207 258c0 7 5 12 12 12h32c7 0 12 -5 12 -12s-5 -12 -12 -12h-32c-7 0 -12 5 -12 12zM56 258c0 -7 -6 -12 -12 -12h-32c-7 0 -12 5 -12 12s5 12 12 12h32 c7 0 12 -5 12 -12zM61 307l-21 21c-5 5 -5 12 0 17c2 2 5 3 8 3s7 -1 9 -3l21 -21c5 -5 5 -12 0 -17c-2 -2 -6 -3 -9 -3s-6 1 -8 3zM69 208c3 0 7 -1 9 -3c5 -5 5 -12 0 -17l-21 -21c-2 -2 -6 -4 -9 -4s-6 2 -8 4s-4 5 -4 8s2 7 4 9l21 21c2 2 5 3 8 3v0zM188 304 c-3 0 -6 1 -8 3s-4 6 -4 9s2 6 4 8l21 21c2 2 5 3 8 3s7 -1 9 -3c5 -5 5 -12 0 -17l-21 -21c-2 -2 -6 -3 -9 -3z"/>
+    <glyph glyph-name="ion-ios-pie" unicode="" horiz-adv-x="448" d="M244 380c0 2 2 4 4 4c111 -4 200 -96 200 -208c0 -114 -92 -207 -207 -208c-88 -1 -164 54 -195 131c-1 2 1 5 3 6l189 48c4 1 6 4 6 8v219zM0 209c0 90 59 205 216 207c2 0 4 -2 4 -4v-229c0 -4 -2 -7 -6 -8l-194 -50c-2 0 -4 1 -5 3c-15 32 -15 81 -15 81z"/>
+    <glyph glyph-name="ion-md-person" unicode="" horiz-adv-x="384" d="M192 192c-53 0 -96 43 -96 96s43 96 96 96s96 -43 96 -96s-43 -96 -96 -96zM192 144c64 0 192 -32 192 -96v-48h-384v48c0 64 128 96 192 96z"/>
+    <glyph glyph-name="ion-md-alert" unicode="" d="M208 400c114 0 208 -94 208 -208s-94 -208 -208 -208s-208 94 -208 208s94 208 208 208zM232 88v40h-48v-40h48zM232 176v128h-48v-128h48z"/>
+    <glyph glyph-name="ion-md-tennisball" unicode="" d="M336 356c-42 -45 -64 -103 -64 -164s23 -119 65 -163c-36 -28 -81 -45 -129 -45s-93 17 -128 45c42 44 64 102 64 163s-22 119 -64 164c35 28 80 44 128 44s93 -16 128 -44zM305 193c0 27 5 53 15 78c9 23 23 44 40 62c35 -37 56 -87 56 -141s-21 -103 -55 -140 c-18 19 -31 39 -41 63c-10 25 -15 51 -15 78zM111 193c0 -27 -5 -53 -15 -78c-10 -24 -23 -45 -41 -64c-34 37 -55 87 -55 141c0 55 21 105 56 142c17 -18 31 -40 40 -63c10 -25 15 -51 15 -78z"/>
+    <glyph glyph-name="ion-logo-closed-captioning" unicode="" horiz-adv-x="448" d="M0 352h448v-320h-448v320zM406 192c0 23 0 38 -3 70s-20 50 -52 53s-86 3 -127 3s-95 0 -127 -3s-49 -21 -52 -53s-3 -47 -3 -70s1 -41 3 -70s17 -50 52 -53s84 -3 127 -3s92 0 127 3s50 24 52 53s3 47 3 70zM326 168c0 1 -1 3 -1 3h46c0 -24 -5 -42 -16 -54 s-29 -19 -54 -19c-12 0 -22 2 -31 5s-15 8 -21 15s-10 16 -13 28s-5 28 -5 46s2 33 6 45s8 22 15 29s15 12 24 15s19 5 30 5c22 0 38 -7 49 -19s17 -32 17 -54h-46v1c0 18 -11 29 -24 29c-14 0 -22 -9 -24 -26c0 0 -1 -10 -1 -26s1 -24 1 -24c1 -15 10 -26 24 -26 s24 10 24 26v1zM171 168c0 1 -1 3 -1 3h46c0 -24 -5 -42 -16 -54s-29 -19 -54 -19c-12 0 -21 2 -30 5s-16 8 -22 15s-10 16 -13 28s-5 28 -5 46s2 33 6 45s8 22 15 29s15 12 24 15s19 5 30 5c22 0 38 -7 49 -19s17 -32 17 -54h-45v1c0 18 -11 29 -24 29 c-14 0 -23 -9 -25 -26c0 0 -1 -10 -1 -26s1 -24 1 -24c1 -15 10 -26 24 -26s24 10 24 26v1z"/>
+    <glyph glyph-name="ion-ios-qr-scanner" unicode="" d="M126 384c8 0 14 -6 14 -14s-6 -14 -14 -14h-60c-9 0 -18 -4 -26 -12c-7 -8 -12 -17 -12 -26v-60c0 -8 -6 -14 -14 -14s-14 6 -14 14v59c0 37 30 67 67 67h59zM350 384c36 0 66 -30 66 -66v-60c0 -8 -6 -14 -14 -14s-14 6 -14 14v60c0 9 -5 18 -12 26c-8 8 -17 12 -26 12 h-60c-8 0 -14 6 -14 14s6 14 14 14h60zM126 28c8 0 14 -6 14 -14s-6 -14 -14 -14h-60c-36 0 -66 30 -66 66v60c0 8 6 14 14 14s14 -6 14 -14v-60c0 -9 5 -18 12 -26c8 -8 17 -12 26 -12h60zM402 140c8 0 14 -6 14 -14v-59c0 -37 -30 -67 -67 -67h-59c-8 0 -14 6 -14 14 s6 14 14 14h60c9 0 18 4 26 12c7 8 12 17 12 26v60c0 8 6 14 14 14z"/>
+    <glyph glyph-name="ion-md-flashlight" unicode="" horiz-adv-x="352" d="M48 150l1 122h255v-122l-64 -64v-118h-128v118zM155 416h42v-64h-42v64zM0 338l30 30l45 -46l-30 -30zM277 322l45 46l30 -30l-45 -46z"/>
+    <glyph glyph-name="ion-ios-code" unicode="" horiz-adv-x="384" d="M268 305l114 -109c1 -1 2 -2 2 -4s-1 -3 -2 -4l-114 -109c-1 -1 -2 -2 -4 -2s-3 1 -4 2l-14 13c-1 1 -2 2 -2 4s1 3 2 4l96 92l-96 92c-1 1 -2 2 -2 4s1 3 2 4l14 13c1 1 3 2 4 2s3 -1 4 -2zM140 288c0 -2 -1 -3 -2 -4l-96 -92l96 -92c1 -1 2 -2 2 -4s-1 -3 -2 -4 l-14 -13c-1 -1 -2 -2 -4 -2c-1 0 -3 1 -4 2l-114 109c-1 1 -2 2 -2 4s1 3 2 4l114 109c1 1 3 2 4 2s3 -1 4 -2l14 -13c1 -1 2 -2 2 -4z"/>
+    <glyph glyph-name="ion-md-bowtie" unicode="" horiz-adv-x="448" d="M244 144h-40c-15 0 -28 13 -28 28v40c0 15 13 28 28 28h40c15 0 28 -13 28 -28v-40c0 -15 -13 -28 -28 -28zM144 168c0 -18 8 -35 22 -45c-6 -27 -102 -91 -134 -91c-18 0 -32 14 -32 32v256c0 18 14 32 32 32c32 0 128 -64 134 -91c-14 -10 -22 -27 -22 -45v-48z M416 352c18 0 32 -14 32 -32v-256c0 -18 -14 -32 -32 -32c-32 0 -128 64 -134 91c14 10 22 27 22 45v48c0 18 -8 35 -22 45c6 27 102 91 134 91z"/>
+    <glyph glyph-name="ion-md-infinite" unicode="" d="M387 260c19 -18 29 -42 29 -68s-11 -50 -30 -68s-43 -28 -70 -28c-26 0 -51 10 -70 28l-38 36l-38 -36c-19 -18 -43 -28 -70 -28s-52 10 -71 28s-29 42 -29 68s10 50 29 68s44 28 71 28c26 0 51 -10 70 -28l108 -104c10 -10 24 -15 38 -15s28 5 38 15s15 22 15 36 s-5 26 -15 36s-24 15 -38 15s-28 -5 -38 -15v0l-17 -16l-33 32l18 16c19 18 43 28 70 28s52 -10 71 -28zM138 156l37 36l-37 36c-10 10 -24 15 -38 15s-28 -5 -38 -15s-15 -22 -15 -36s5 -26 15 -36s24 -15 38 -15s28 5 38 15z"/>
+    <glyph glyph-name="ion-md-volume-off" unicode="" horiz-adv-x="384" d="M342 192c0 69 -45 128 -107 147v45c85 -20 149 -99 149 -192c0 -35 -8 -68 -24 -96l-32 32c9 19 14 41 14 64zM192 368v-104l-51 51zM357 51v0l24 -24l-24 -24l-40 40c-23 -21 -51 -36 -82 -43v45c19 6 37 15 52 28l-95 95v-152l-107 112h-85v128h85l9 10l-91 91l24 24z M288 192c0 -7 0 -15 -2 -22l-51 51v60c32 -16 53 -51 53 -89z"/>
+    <glyph glyph-name="ion-ios-flag" unicode="" horiz-adv-x="320" d="M311 367c5 1 9 -3 9 -8v-191c0 -4 -3 -7 -7 -8c0 0 -37 -7 -72 -7s-57 6 -83 12s-51 15 -77 15c-21 0 -37 -2 -47 -4c-4 -1 -6 -4 -6 -8v-154c0 -8 -8 -15 -16 -14c-7 1 -12 7 -12 14v354l4 3c2 1 21 13 77 13c28 0 55 -5 81 -11c25 -5 50 -12 78 -12c18 0 48 3 61 5 c4 0 7 1 10 1z"/>
+    <glyph glyph-name="ion-md-cloud-download" unicode="" horiz-adv-x="480" d="M387 231c52 -3 93 -46 93 -99c0 -55 -45 -100 -100 -100h-260c-66 0 -120 54 -120 120c0 62 47 113 107 119c25 48 75 81 133 81c73 0 133 -52 147 -121zM208 180h-68l100 -100l100 100h-68v76h-64v-76z"/>
+    <glyph glyph-name="ion-md-git-compare" unicode="" horiz-adv-x="448" d="M160 66v62l96 -96l-96 -96v64h-22c-39 0 -64 11 -84 36c-18 23 -22 48 -22 94v167c-19 11 -32 31 -32 55c0 35 29 64 64 64s64 -29 64 -64c0 -24 -13 -44 -32 -55v-167c0 -35 2 -47 8 -54c5 -6 9 -10 34 -10h22zM64 392c-22 0 -40 -18 -40 -40s18 -40 40 -40s40 18 40 40 s-18 40 -40 40zM416 87c19 -11 32 -31 32 -55c0 -35 -29 -64 -64 -64s-64 29 -64 64c0 24 13 44 32 55v170c0 35 -2 47 -8 54c-5 6 -9 9 -34 9h-22v-64l-96 96l96 96v-62h22c39 0 64 -10 84 -35c18 -23 22 -48 22 -94v-170zM384 -8c22 0 40 18 40 40s-18 40 -40 40 s-40 -18 -40 -40s18 -40 40 -40z"/>
+    <glyph glyph-name="ion-ios-mic" unicode="" horiz-adv-x="220" d="M110 112c-35 0 -64 29 -64 64v160c0 35 29 64 64 64v0c35 0 64 -29 64 -64v-160c0 -35 -29 -64 -64 -64v0zM206 256c8 0 14 -6 14 -14v-69c0 -56 -42 -102 -96 -109v-52h36c8 0 14 -6 14 -14s-6 -14 -14 -14h-100c-8 0 -14 6 -14 14s6 14 14 14h36v52 c-54 7 -96 53 -96 109v69c0 8 6 14 14 14s14 -6 14 -14v-69c0 -45 37 -82 82 -82s82 37 82 82v69c0 8 6 14 14 14z"/>
+    <glyph glyph-name="ion-md-transgender" unicode="" horiz-adv-x="448" d="M320 416h128v-128h-35v69l-88 -88c7 -15 11 -31 11 -49c0 -55 -40 -100 -91 -110v-46h63v-40h-63v-56h-42v56h-63v40h63v46c-52 10 -91 55 -91 110c0 18 4 34 11 49l-19 19l-31 -31l-29 29l31 31l-40 40v-69h-35v128h128v-35h-63l37 -37l32 32l28 -29l-31 -31l15 -15 c20 19 48 31 78 31c24 0 46 -7 64 -20c0 0 6 -5 14 -12l81 81h-63v35zM224 158c35 0 64 29 64 64s-29 64 -64 64s-64 -29 -64 -64s29 -64 64 -64z"/>
+    <glyph glyph-name="ion-md-headset" unicode="" horiz-adv-x="384" d="M192 400c106 0 192 -88 192 -197v-153c0 -36 -29 -66 -64 -66h-64v176h85v43c0 85 -67 153 -149 153s-149 -68 -149 -153v-43h85v-176h-64c-35 0 -64 30 -64 66v153c0 109 86 197 192 197z"/>
+    <glyph glyph-name="ion-md-arrow-down" unicode="" horiz-adv-x="342" d="M192 363v-260l120 120l30 -31l-171 -171l-171 171l30 30l120 -119v260h42z"/>
+    <glyph glyph-name="ion-ios-partly-sunny" unicode="" horiz-adv-x="480" d="M152 384c9 0 16 -7 16 -16v-30c0 -9 -7 -16 -16 -16v0c-9 0 -16 7 -16 16v30c0 9 7 16 16 16v0zM63 232c0 -9 -7 -16 -16 -16h-31c-9 0 -16 7 -16 16v0c0 9 7 16 16 16h31c9 0 16 -7 16 -16v0zM64 299l-21 21c-6 6 -6 16 0 22c3 3 8 4 12 4v0c4 0 8 -1 11 -4l20 -21 c6 -6 6 -16 0 -22c-3 -3 -7 -5 -11 -5s-8 2 -11 5zM255 343c6 -6 6 -16 0 -22l-20 -21c-3 -3 -7 -4 -11 -4s-8 1 -11 4v0h-1c-6 6 -6 16 0 22l21 21c3 3 7 5 11 5v0c4 0 8 -2 11 -5zM77 175c4 0 8 -2 11 -5c6 -6 6 -16 0 -22l-20 -21c-3 -3 -7 -5 -11 -5s-9 2 -12 5 c-6 6 -6 17 0 23l21 20c3 3 7 5 11 5v0zM387 189c51 0 93 -43 93 -94s-42 -95 -93 -95h-226c-41 0 -74 36 -74 77c0 39 29 73 67 76v1v10v6c2 39 24 74 56 92c5 3 10 5 15 7c13 5 26 8 41 8c54 0 99 -39 110 -89c3 0 6 1 9 1h2zM135 164c-34 7 -59 36 -59 72 c0 40 33 74 73 74c23 0 45 -11 58 -28h-1c-5 -2 -9 -5 -14 -8c-33 -19 -56 -53 -58 -93v-6c0 -3 1 -7 1 -10v-1z"/>
+    <glyph glyph-name="ion-md-cloudy-night" unicode="" horiz-adv-x="448" d="M91 265h2h-2zM310 145c42 -3 74 -38 74 -81c0 -44 -36 -80 -80 -80h-208c-53 0 -96 44 -96 97c0 47 32 86 78 95c5 1 13 1 18 1c43 0 78 -26 91 -64h28c-6 21 -16 40 -32 56c-21 21 -48 32 -77 34c21 27 52 39 86 39c58 0 107 -41 118 -97zM80 223l-13 -17v0v5 c0 92 63 168 148 189c-9 -21 -14 -45 -14 -70c0 -20 3 -39 9 -57c-6 1 -12 1 -18 1c-22 0 -43 -4 -61 -12c-20 -9 -37 -22 -51 -39zM386 141c-3 3 -6 7 -10 10h3c24 0 48 5 69 14c-7 -27 -19 -53 -35 -74c-4 19 -14 36 -27 50z"/>
+    <glyph glyph-name="ion-md-nutrition" unicode="" d="M309 212c8 -8 13 -19 13 -31c0 -14 -6 -28 -17 -36l-233 -168c-9 -6 -19 -9 -28 -9c-25 0 -44 20 -44 44c0 8 2 15 5 21v1l158 255v0c8 10 21 16 35 16c10 0 20 -4 28 -10h1zM416 303l-109 -56l-42 41l68 128l49 -28l-32 -56l37 20z"/>
+    <glyph glyph-name="ion-ios-options" unicode="" d="M251 72c6 14 21 24 37 24s31 -10 37 -24h75c9 0 16 -7 16 -16v0c0 -9 -7 -16 -16 -16h-75c-6 -14 -21 -24 -37 -24s-31 10 -37 24h-235c-9 0 -16 7 -16 16v0c0 9 7 16 16 16h235zM91 208c6 14 21 24 37 24s31 -10 37 -24h235c9 0 16 -7 16 -16v0c0 -9 -7 -16 -16 -16 h-235c-6 -14 -21 -24 -37 -24s-31 10 -37 24h-75c-9 0 -16 7 -16 16v0c0 9 7 16 16 16h75zM251 344c6 14 21 24 37 24s31 -10 37 -24h75c9 0 16 -7 16 -16v0c0 -9 -7 -16 -16 -16h-75c-6 -14 -21 -24 -37 -24s-31 10 -37 24h-235c-9 0 -16 7 -16 16v0c0 9 7 16 16 16h235z "/>
+    <glyph glyph-name="ion-md-log-out" unicode="" d="M144 171v42h190l-45 45l31 30l96 -96l-96 -96l-30 30l44 45h-190zM208 27c44 0 85 17 116 48l30 -30c-8 -8 -17 -16 -27 -23c-35 -24 -76 -38 -119 -38c-114 0 -208 93 -208 208s94 208 208 208c43 0 84 -13 119 -37c10 -7 19 -16 27 -24l-30 -30c-31 31 -72 48 -116 48 s-86 -17 -117 -48s-48 -73 -48 -117s17 -86 48 -117s73 -48 117 -48zM401 192l-1 1l-1 -1l1 -1z"/>
+    <glyph glyph-name="ion-md-appstore" unicode="" d="M416 296c-19 -214 -13 -312 -13 -312h-390s6 96 -13 312h104c0 57 47 104 104 104s104 -47 104 -104h104zM208 374c-43 0 -78 -35 -78 -78h156c0 43 -35 78 -78 78zM156 50l143 85l-143 84v-169z"/>
+    <glyph glyph-name="ion-md-cellular" unicode="" horiz-adv-x="384" d="M0 0l384 384v-384h-384z"/>
+    <glyph glyph-name="ion-ios-beer" unicode="" horiz-adv-x="384" d="M48 286v-96h-26c-9 0 -17 8 -17 17c0 16 11 22 11 35v51c0 19 -16 12 -16 41v4c0 26 25 43 50 43c10 0 18 -1 23 -2c6 -1 12 1 16 6c9 10 22 17 36 17c9 0 19 -2 26 -7c6 -4 13 -2 17 3c10 11 23 18 44 18c24 0 47 -16 56 -39c2 -5 7 -9 13 -9h1c21 0 38 -17 38 -38v-22 c0 -2 -3 -4 -5 -2v0c-4 3 -8 4 -13 4h-230c-13 0 -24 -11 -24 -24zM322 -4c8 0 14 -6 14 -14s-6 -14 -14 -14h-276c-8 0 -14 6 -14 14s6 14 14 14h276zM336 254c26 0 48 -22 48 -48v-96c0 -26 -22 -48 -48 -48h-32c0 -12 2 -18 5 -30c1 -6 0 -12 -5 -16c-3 -3 -7 -4 -11 -4 h-214c-6 0 -12 3 -15 8c-3 4 -4 9 -3 14v0c2 12 3 18 3 30v214c0 9 7 16 16 16h208c9 0 16 -7 16 -16v-24h32zM356 110v96c0 11 -9 20 -20 20h-32v-136h32c11 0 20 9 20 20z"/>
+    <glyph glyph-name="ion-ios-snow" unicode="" horiz-adv-x="384" d="M377 102c7 -4 9 -13 5 -20c-3 -5 -8 -8 -13 -8c-3 0 -6 1 -8 2l-32 19c-4 -16 -2 -32 -2 -32c1 -8 -4 -15 -12 -17c-1 0 -2 -1 -3 -1c-7 0 -13 6 -14 13c0 1 -4 26 6 52l-96 56v-99c27 -5 46 -20 47 -21c3 -2 6 -6 6 -10s-1 -8 -4 -11s-7 -5 -11 -5c-3 0 -7 1 -10 3 c0 0 -12 10 -28 14v-38c0 -8 -8 -15 -16 -15s-15 7 -15 15v38c-16 -4 -29 -14 -29 -14c-3 -2 -7 -3 -10 -3c-4 0 -8 2 -11 5s-4 7 -4 11s3 8 6 10c1 1 21 16 48 21v99l-96 -56c10 -26 6 -51 6 -52c-1 -7 -7 -13 -14 -13c-1 0 -2 1 -3 1c-8 2 -13 9 -12 17c0 0 2 16 -3 32 l-32 -19c-2 -1 -5 -2 -8 -2c-5 0 -10 3 -13 8c-4 7 -2 16 5 20l33 19c-12 12 -26 19 -26 19c-8 3 -11 11 -9 19c2 6 7 9 13 9c2 0 4 0 6 -1c1 0 24 -10 42 -31l96 56l-96 56c-18 -21 -41 -31 -42 -31c-2 -1 -4 -1 -6 -1c-6 0 -11 4 -13 10c-2 8 1 16 9 19c0 0 14 6 26 18 l-33 19c-7 4 -9 13 -5 20c3 5 8 8 13 8c3 0 6 -1 8 -2l32 -19c5 16 3 32 3 32c-1 8 4 16 12 18h3c7 0 13 -6 14 -13c0 -1 4 -26 -6 -52l96 -56v100c-27 5 -47 19 -48 20c-3 2 -6 6 -6 10s
 1 8 4 11s7 5 11 5s7 -1 10 -3c0 0 13 -10 29 -14v38c0 8 7 15 15 15s16 -7 16 -15 v-38c16 4 28 14 28 14c3 2 7 3 10 3c4 0 8 -2 11 -5s4 -7 4 -11s-3 -8 -6 -10c-1 -1 -20 -16 -47 -21v-99l96 56c-10 26 -7 51 -7 52c1 7 8 13 15 13c1 0 2 -1 3 -1c8 -2 13 -9 12 -17c0 -1 -2 -16 2 -32l32 19c2 1 5 2 8 2c5 0 10 -3 13 -8c4 -7 2 -16 -5 -20l-33 -19 c12 -12 26 -19 26 -19c8 -3 11 -11 9 -19c-2 -6 -7 -9 -13 -9c-2 0 -4 0 -6 1c-1 0 -24 10 -42 31l-96 -56l96 -56c18 21 41 31 42 31c2 1 4 1 6 1c6 0 11 -4 13 -10c2 -8 -1 -16 -9 -19c0 0 -14 -6 -26 -18z"/>
+    <glyph glyph-name="ion-ios-happy" unicode="" d="M208 400c115 0 208 -93 208 -208s-93 -208 -208 -208s-208 93 -208 208s93 208 208 208zM280 248c-13 0 -24 -11 -24 -24s11 -24 24 -24s24 11 24 24s-11 24 -24 24zM136 248c-13 0 -24 -11 -24 -24s11 -24 24 -24s24 11 24 24s-11 24 -24 24zM208 79c45 0 82 30 94 71 c2 5 -3 10 -8 10h-172c-5 0 -10 -5 -8 -10c12 -41 49 -71 94 -71z"/>
+    <glyph glyph-name="ion-md-wallet" unicode="" d="M155 107c0 -24 18 -43 42 -43h197v-21c0 -24 -18 -43 -42 -43h-308c-24 0 -44 18 -44 42v300c0 24 20 42 44 42h308c23 0 42 -19 42 -43v-21h-197c-24 0 -42 -19 -42 -43v-170zM197 262c0 9 7 16 16 16h187c9 0 16 -7 16 -16v-140c0 -9 -7 -16 -16 -16h-187 c-9 0 -16 7 -16 16v140zM274 160c16 1 29 14 30 30c1 19 -15 35 -34 34c-16 -1 -29 -14 -30 -30c-1 -19 15 -35 34 -34z"/>
+    <glyph glyph-name="ion-md-funnel" unicode="" horiz-adv-x="448" d="M176 48v48h96v-48h-96zM0 336h448v-48h-448v48zM80 167v50h288v-50h-288z"/>
+    <glyph glyph-name="ion-ios-refresh-circle" unicode="" d="M208 400c115 0 208 -93 208 -208s-93 -208 -208 -208s-208 93 -208 208s93 208 208 208zM208 64c67 0 123 51 128 117c0 6 -4 11 -10 11v0c-5 0 -10 -4 -10 -9c-4 -55 -51 -98 -107 -99c-59 -1 -108 47 -109 106c-1 60 48 110 108 110v-42c0 -6 7 -10 12 -7l73 47 c5 3 5 10 0 13l-73 53c-5 4 -12 -1 -12 -7v-37c-71 0 -128 -57 -128 -128s57 -128 128 -128z"/>
+    <glyph glyph-name="ion-ios-watch" unicode="" d="M392 287c16 -25 24 -53 24 -82v-41c0 -21 -4 -40 -12 -58c-8 -17 -19 -33 -33 -46c-28 -25 -66 -39 -108 -39c-41 0 -54 8 -65 14c-7 4 -13 6 -27 7c-38 2 -69 16 -94 40c-3 2 -4 5 -4 9c0 3 2 7 4 9c2 3 5 4 9 4c3 0 8 -1 10 -3c20 -19 45 -30 77 -32 c20 -1 29 -6 37 -11c9 -5 19 -10 53 -10c35 0 67 12 90 32c12 10 21 22 27 36c7 15 10 31 10 48v41c0 24 -7 48 -20 68c-12 20 -30 35 -51 46c-3 2 -6 5 -7 8s-1 7 1 10c2 4 7 7 12 7c2 0 4 0 6 -1c25 -13 46 -32 61 -56zM236 65c-6 1 -11 6 -11 13v2c1 6 7 11 13 11h2 c7 -1 14 -2 21 -2c26 0 48 9 65 25c2 2 6 4 9 4c4 0 7 -2 9 -5c5 -5 5 -13 0 -18c-11 -10 -24 -19 -38 -24s-29 -8 -45 -8c-8 0 -16 1 -25 2zM238 86v0v0v0zM175 26c7 0 13 -6 13 -13s-6 -13 -13 -13c-46 0 -86 16 -121 47s-54 73 -54 114v44c0 29 8 57 24 82 c15 24 36 43 61 56c2 1 4 1 6 1c5 0 10 -3 12 -7c2 -3 2 -7 1 -10s-4 -6 -7 -8c-21 -11 -39 -26 -51 -46c-13 -20 -20 -44 -20 -68v-44c0 -34 17 -68 46 -94c30 -27 63 -41 103 -41zM300 333c-1 -14 11 -26 25 -26v0c2 0 3 -3 1 -4c-8
  -7 -18 -10 -29 -10h-178 c-11 0 -21 3 -29 9c-1 1 0 3 0 4v0c0 1 1 1 2 1c13 1 24 12 23 26c-1 12 -11 22 -23 23c-4 0 -7 -1 -10 -2s-5 2 -4 4c8 16 23 26 41 26h178c11 0 22 -4 30 -11c5 -4 8 -9 11 -15c1 -2 0 -5 -3 -4c-4 2 -8 2 -12 2c-12 -1 -22 -11 -23 -23zM279 343c-2 8 -9 15 -17 17 c-16 4 -31 -10 -27 -26c2 -8 9 -15 17 -17c16 -4 31 10 27 26z"/>
+    <glyph glyph-name="ion-ios-reorder" unicode="" horiz-adv-x="352" d="M0 128v16h352v-16h-352zM0 184v16h352v-16h-352zM0 240v16h352v-16h-352z"/>
+    <glyph glyph-name="ion-ios-volume-high" unicode="" horiz-adv-x="383" d="M151 302c6 -3 9 -8 9 -14v-192c0 -6 -3 -11 -9 -14c-2 -1 -5 -2 -7 -2c-4 0 -7 2 -10 4l-65 52h-53c-9 0 -16 7 -16 16v80c0 9 7 16 16 16h53l65 53c3 2 6 3 10 3c2 0 5 -1 7 -2zM383 192v0v0c0 -59 -23 -114 -65 -156c-2 -2 -6 -4 -9 -4s-6 2 -8 4c-5 5 -5 12 0 17 c38 37 58 86 58 139v0v0c0 53 -20 102 -58 139c-5 5 -5 12 0 17s12 5 17 0c42 -42 65 -97 65 -156zM320 192v0c0 -44 -16 -84 -46 -115c-2 -2 -6 -4 -9 -4s-6 2 -8 4c-5 5 -6 12 -1 17c26 26 40 61 40 98v0v0c0 37 -14 72 -40 98c-5 5 -4 12 1 17s12 5 17 0 c30 -31 46 -71 46 -115v0zM223 266c19 -20 30 -46 30 -74v0v0c0 -28 -11 -54 -30 -74c-2 -2 -6 -3 -9 -3s-6 1 -8 3c-5 5 -5 12 0 17c15 15 23 36 23 57v0v0c0 22 -8 42 -23 57c-5 5 -5 12 0 17s12 5 17 0z"/>
+    <glyph glyph-name="ion-ios-contrast" unicode="" d="M208 400c115 0 208 -93 208 -208s-93 -208 -208 -208s-208 93 -208 208s93 208 208 208zM335 65c34 34 53 79 53 127s-19 93 -53 127s-79 53 -127 53v-360c48 0 93 19 127 53z"/>
+    <glyph glyph-name="ion-ios-today" unicode="" horiz-adv-x="384" d="M324 416c33 0 60 -27 60 -60v-328c0 -33 -27 -60 -60 -60h-264c-33 0 -60 27 -60 60v328c0 33 27 60 60 60h264zM356 28v328c0 18 -14 32 -32 32h-264c-18 0 -32 -14 -32 -32v-328c0 -18 14 -32 32 -32h264c18 0 32 14 32 32zM296 272c13 0 24 -11 24 -24v-192 c0 -13 -11 -24 -24 -24h-208c-13 0 -24 11 -24 24v192c0 13 11 24 24 24h208zM78 324c-8 0 -14 6 -14 14s6 14 14 14h100c8 0 14 -6 14 -14s-6 -14 -14 -14h-100z"/>
+    <glyph glyph-name="ion-ios-mic-off" unicode="" horiz-adv-x="256" d="M242 -16c-5 0 -9 3 -12 7l-228 388c-4 7 -2 15 5 19s16 2 20 -5l227 -388c4 -7 2 -15 -5 -19c-2 -1 -5 -2 -7 -2zM192 176c0 -9 -2 -17 -5 -25l-120 205c8 26 33 44 61 44v0c35 0 64 -29 64 -64v-160zM64 176v65l75 -128c-4 -1 -7 -1 -11 -1v0c-35 0 -64 29 -64 64z M238 173c0 -26 -9 -49 -24 -68l-15 26c7 12 11 27 11 42v69c0 8 6 14 14 14s14 -6 14 -14v-69zM128 91c8 0 15 1 22 3l15 -25c-7 -3 -15 -4 -23 -5v-52h36c8 0 14 -6 14 -14s-6 -14 -14 -14h-100c-8 0 -14 6 -14 14s6 14 14 14h36v52c-54 7 -96 53 -96 109v69 c0 8 6 14 14 14s14 -6 14 -14v-69c0 -45 37 -82 82 -82z"/>
+    <glyph glyph-name="ion-ios-medkit" unicode="" d="M384 320c18 0 32 -14 32 -32v-256c0 -18 -14 -32 -32 -32h-352c-18 0 -32 14 -32 32v256c0 18 14 32 32 32h84v32c0 18 14 32 32 32h120c18 0 32 -14 32 -32v-32h84zM144 348v-28h128v28c0 4 -4 8 -8 8h-112c-4 0 -8 -4 -8 -8zM272 144c9 0 16 7 16 16s-7 16 -16 16h-32 h-16v16v32c0 9 -7 16 -16 16s-16 -7 -16 -16v-32v-16h-16h-32c-9 0 -16 -7 -16 -16s7 -16 16 -16h32h16v-16v-32c0 -9 7 -16 16 -16s16 7 16 16v32v16h16h32z"/>
+    <glyph glyph-name="ion-ios-arrow-round-down" unicode="" horiz-adv-x="192" d="M188 152c5 -5 5 -13 0 -18l-83 -82c-2 -2 -6 -4 -9 -4c-2 0 -3 0 -5 1s-3 2 -4 3v0l-83 82c-5 5 -5 13 0 18s13 6 18 1l61 -61v231c0 7 6 13 13 13s13 -6 13 -13v-231l61 60c5 5 13 5 18 0z"/>
+    <glyph glyph-name="ion-md-link" unicode="" horiz-adv-x="448" d="M43 192c0 -38 31 -69 69 -69h88v-43h-88c-62 0 -112 50 -112 112s50 112 112 112h88v-43h-88c-38 0 -69 -31 -69 -69zM128 170v44h192v-44h-192zM336 304c62 0 112 -50 112 -112s-50 -112 -112 -112h-88v43h88c38 0 69 31 69 69s-31 69 -69 69h-88v43h88z"/>
+    <glyph glyph-name="ion-ios-chatbubbles" unicode="" d="M383 127l16 -58s1 -2 1 -3c0 -4 -4 -8 -8 -8c-2 0 -4 1 -4 1l-57 21c-3 1 -6 2 -10 2c-3 0 -6 -1 -9 -2s-15 -5 -26 -8s-32 -8 -47 -8c-96 0 -171 77 -171 169c0 12 2 23 4 34c16 76 86 133 170 133c96 0 174 -75 174 -167c0 -33 -11 -64 -28 -90c-1 -1 -1 -2 -2 -3 c-2 -4 -3 -9 -3 -13zM270 56c4 1 9 2 13 3c-4 -5 -7 -9 -7 -9c-30 -35 -73 -56 -120 -56c-21 0 -42 5 -61 12c-4 2 -6 2 -6 2c-2 0 -4 0 -6 -1l-56 -22c-1 0 -2 -1 -3 -1c-2 0 -3 1 -5 2c-2 2 -4 5 -3 8c0 0 12 60 12 62s-1 4 -3 8c-16 24 -25 53 -25 82c0 42 19 82 52 111 l8 7c-1 -4 -3 -9 -4 -13s-1 -9 -2 -13c-1 -6 -2 -14 -2 -21c0 -92 75 -169 171 -169c10 0 24 2 34 4c5 1 9 3 13 4z"/>
+    <glyph glyph-name="ion-ios-beaker" unicode="" horiz-adv-x="384" d="M381 400c2 0 3 -1 3 -2s-2 -4 -3 -6s-12 -15 -14 -33v-310c0 -35 -28 -65 -63 -65h-208c-35 0 -64 29 -64 64v276c0 32 -2 32 -32 36c0 12 10 40 64 40h270h47zM48 371c12 -13 12 -29 12 -47v-36h279v0v71v3c0 3 1 6 2 10h-7h-270c-4 0 -10 0 -16 -1v0z"/>
+    <glyph glyph-name="ion-ios-git-branch" unicode="" horiz-adv-x="332" d="M262 352c39 0 70 -31 70 -70c0 -34 -24 -63 -56 -69c-1 -28 -14 -46 -38 -64c-25 -19 -56 -25 -78 -29c-27 -5 -42 -14 -51 -24c19 -13 31 -35 31 -58c0 -39 -31 -70 -70 -70s-70 31 -70 70c0 17 6 33 17 46c10 11 24 19 38 22v172c-15 3 -28 11 -38 22 c-11 13 -17 29 -17 46c0 39 31 70 70 70s70 -31 70 -70c0 -17 -6 -34 -18 -47c-10 -11 -24 -19 -39 -22v-169c10 13 32 33 73 40c19 3 45 7 65 23c17 13 26 24 27 43c-32 7 -56 35 -56 68c0 39 31 70 70 70zM28 346c0 -23 19 -42 42 -42s42 19 42 42s-19 42 -42 42 s-42 -19 -42 -42zM112 38c0 23 -19 42 -42 42s-42 -19 -42 -42s19 -42 42 -42s42 19 42 42zM262 240c23 0 42 19 42 42s-19 42 -42 42s-42 -19 -42 -42s19 -42 42 -42z"/>
+    <glyph glyph-name="ion-logo-google" unicode="" horiz-adv-x="408" d="M406 224c0 -1 2 -14 2 -37c0 -54 -18 -106 -54 -144s-87 -59 -141 -59c-59 0 -116 22 -156 64c-38 39 -57 92 -57 144s20 108 59 147c40 39 94 61 150 61v0c73 0 120 -38 137 -53l-58 -58c-10 9 -38 30 -77 30c-34 0 -68 -12 -90 -35c-23 -23 -39 -58 -39 -92 c0 -35 15 -69 39 -92c23 -22 58 -36 91 -36c46 0 102 30 114 87h-116v82h194z"/>
+    <glyph glyph-name="ion-ios-recording" unicode="" horiz-adv-x="480" d="M370 302c61 0 110 -49 110 -110s-49 -110 -110 -110h-260c-61 0 -110 49 -110 110s49 110 110 110v0v0c61 0 110 -49 110 -110c0 -30 -12 -58 -32 -78h105c-20 20 -33 48 -33 78c0 61 49 110 110 110v0v0zM448 192c0 43 -35 78 -78 78s-78 -35 -78 -78s35 -78 78 -78 s78 35 78 78zM110 114c43 0 78 35 78 78s-35 78 -78 78s-78 -35 -78 -78s35 -78 78 -78z"/>
+    <glyph glyph-name="ion-md-checkmark-circle" unicode="" d="M208 400c114 0 208 -94 208 -208s-94 -208 -208 -208s-208 94 -208 208s94 208 208 208zM165 81l192 192l-29 30l-163 -162l-76 77l-30 -30z"/>
+    <glyph glyph-name="ion-ios-checkbox-outline" unicode="" horiz-adv-x="384" d="M352 384c18 0 32 -14 32 -32v-320c0 -18 -14 -32 -32 -32h-320c-18 0 -32 14 -32 32v320c0 18 14 32 32 32h320zM356 36v312c0 4 -4 8 -8 8h-312c-4 0 -8 -4 -8 -8v-312c0 -4 4 -8 8 -8h312c4 0 8 4 8 8zM300 255c2 -2 1 -4 0 -6l-134 -134v0c-2 -2 -7 -5 -12 -5 c-4 0 -7 1 -11 5l-56 56c-2 2 -2 4 0 6l17 18c1 1 2 1 3 1s2 0 3 -1l44 -45l122 123c1 1 2 1 3 1v0c1 0 2 0 3 -1z"/>
+    <glyph glyph-name="ion-ios-arrow-round-up" unicode="" horiz-adv-x="192" d="M188 232c-5 -5 -13 -5 -18 0l-61 60v-231c0 -7 -6 -13 -13 -13s-13 6 -13 13v231l-61 -61c-5 -5 -13 -4 -18 1s-5 13 0 18l83 82v0c1 1 2 2 4 3s3 1 5 1c3 0 7 -2 9 -4l83 -82c5 -5 5 -13 0 -18z"/>
+    <glyph glyph-name="ion-ios-home" unicode="" d="M211 344l156 -125c1 -1 1 -2 1 -3v-230c0 -1 -1 -2 -2 -2h-108c-1 0 -2 1 -2 2v140c0 1 -1 2 -2 2h-92c-1 0 -2 -1 -2 -2v-140c0 -1 -1 -2 -2 -2h-108c-1 0 -2 1 -2 2v230c0 1 1 2 2 3l156 125c2 1 3 1 5 0zM411 244c3 -2 5 -6 5 -10s-1 -8 -4 -11s-6 -4 -10 -4 c-3 0 -7 1 -9 3l-184 147l-1 1l-1 -1l-184 -148c-2 -2 -6 -3 -9 -3c-4 0 -7 1 -10 4s-4 7 -4 11s2 8 5 10l43 35v92c0 1 1 2 2 2h60c1 0 2 -1 2 -2v-42l82 67c4 3 9 5 14 5s10 -2 14 -5z"/>
+    <glyph glyph-name="ion-ios-paper" unicode="" d="M76 368c0 18 12 30 30 30l278 2c18 0 32 -14 32 -32v-353c0 -17 -14 -31 -31 -31h-353c-18 0 -32 14 -32 32v288c0 18 14 32 32 32h16v-290c0 -8 6 -14 14 -14v0c8 0 14 6 14 14v322zM142 336c-8 0 -14 -6 -14 -14v0c0 -8 6 -14 14 -14h84c8 0 14 6 14 14v0 c0 8 -6 14 -14 14h-84zM142 176c-8 0 -14 -6 -14 -14v0c0 -8 6 -14 14 -14h148c8 0 14 6 14 14v0c0 8 -6 14 -14 14h-148zM338 68c8 0 14 6 14 14v0c0 8 -6 14 -14 14h-196c-8 0 -14 -6 -14 -14v0c0 -8 6 -14 14 -14h196zM338 228c8 0 14 6 14 14v0c0 8 -6 14 -14 14h-196 c-8 0 -14 -6 -14 -14v0c0 -8 6 -14 14 -14h196z"/>
+    <glyph glyph-name="ion-md-cafe" unicode="" d="M0 0v48h368v-48h-368zM376 384c22 0 40 -18 40 -40v-80c0 -22 -18 -40 -40 -40h-40v-64c0 -44 -36 -80 -80 -80h-144c-44 0 -80 36 -80 80v224h293h11h40zM376 272v64h-40v-64h40z"/>
+    <glyph glyph-name="ion-md-medical" unicode="" horiz-adv-x="408" d="M300 192l108 -62l-48 -84l-108 63v-125h-96v125l-108 -63l-48 84l108 62l-108 62l48 84l108 -63v125h96v-125l108 63l48 -84z"/>
+    <glyph glyph-name="ion-ios-log-in" unicode="" horiz-adv-x="414" d="M368 368c25 0 46 -21 46 -46v-260c0 -25 -21 -46 -46 -46h-280c-25 0 -46 21 -46 46c0 8 6 14 14 14s14 -6 14 -14c0 -10 8 -18 18 -18h

<TRUNCATED>

[21/29] samza git commit: SAMZA-1898: New UI layout for the Samza website

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/samza/blob/b8058af0/docs/_layouts/page.html
----------------------------------------------------------------------
diff --git a/docs/_layouts/page.html b/docs/_layouts/page.html
index d1bd58a..0b1bfa6 100644
--- a/docs/_layouts/page.html
+++ b/docs/_layouts/page.html
@@ -1,6 +1,4 @@
----
-layout: default
----
+<!DOCTYPE html>
 <!--
    Licensed to the Apache Software Foundation (ASF) under one or more
    contributor license agreements.  See the NOTICE file distributed with
@@ -17,7 +15,95 @@ layout: default
    See the License for the specific language governing permissions and
    limitations under the License.
 -->
+<html lang="en">
 
-<h2>{{ page.title }}</h2>
+<head>
+  <meta http-equiv="X-UA-Compatible" content="IE=edge">
+  <meta name="viewport" content="width=device-width, initial-scale=1, shrink-to-fit=no">
+  <meta charset="utf-8">
+  <title>Samza {% if page.title %}- {{page.title}}{% endif %}</title>
+  <link rel="apple-touch-icon-precomposed" sizes="57x57" href="/img/favicon/apple-touch-icon-57x57.png" />
+  <link rel="apple-touch-icon-precomposed" sizes="114x114" href="/img/favicon/apple-touch-icon-114x114.png" />
+  <link rel="apple-touch-icon-precomposed" sizes="72x72" href="/img/favicon/apple-touch-icon-72x72.png" />
+  <link rel="apple-touch-icon-precomposed" sizes="144x144" href="/img/favicon/apple-touch-icon-144x144.png" />
+  <link rel="apple-touch-icon-precomposed" sizes="60x60" href="/img/favicon/apple-touch-icon-60x60.png" />
+  <link rel="apple-touch-icon-precomposed" sizes="120x120" href="/img/favicon/apple-touch-icon-120x120.png" />
+  <link rel="apple-touch-icon-precomposed" sizes="76x76" href="/img/favicon/apple-touch-icon-76x76.png" />
+  <link rel="apple-touch-icon-precomposed" sizes="152x152" href="/img/favicon/apple-touch-icon-152x152.png" />
+  <link rel="icon" type="image/png" href="/img/favicon/favicon-196x196.png" sizes="196x196" />
+  <link rel="icon" type="image/png" href="/img/favicon/favicon-96x96.png" sizes="96x96" />
+  <link rel="icon" type="image/png" href="/img/favicon/favicon-32x32.png" sizes="32x32" />
+  <link rel="icon" type="image/png" href="/img/favicon/favicon-16x16.png" sizes="16x16" />
+  <link rel="icon" type="image/png" href="/img/favicon/favicon-128.png" sizes="128x128" />
+  <meta name="application-name" content="https://samza.apache.org" />
+  <meta name="msapplication-TileColor" content="#FFFFFF" />
+  <meta name="msapplication-TileImage" content="/img/favicon/mstile-144x144.png" />
+  <meta name="msapplication-square70x70logo" content="/img/favicon/mstile-70x70.png" />
+  <meta name="msapplication-square150x150logo" content="/img/favicon/mstile-150x150.png" />
+  <meta name="msapplication-wide310x150logo" content="/img/favicon/mstile-310x150.png" />
+  <meta name="msapplication-square310x310logo" content="/img/favicon/mstile-310x310.png" />
+  <link href="/css/ionicons.min.css" rel="stylesheet">
+  <link href="/css/google-fonts.css" rel="stylesheet">
+  <link href="/css/syntax.css" rel="stylesheet"/>
+  <link rel="stylesheet" href="/css/main.new.css" />
+</head>
 
-{{ content }}
+<body class="page">
+  {% include main-navigation.html %}
+
+  <div class="container">
+      <div class="container__toggle">
+        <i class="icon ion-md-arrow-dropleft-circle container__toggle-icon"></i>
+        <i class="icon ion-md-arrow-dropright-circle container__toggle-icon container__toggle-icon--opened"></i>
+      </div>
+      
+      <!-- There is only one menu, but made it as a no-output collection to grab data only -->
+      {% for menu in site.menu %}
+        {{ menu.content }}
+      {% endfor %}
+      
+      <div class="section">
+        <div class="content">
+          {% if page.hide_title != true %}
+          <h2>{{ page.title }}</h2>
+          {% endif %}
+
+          {% if page.url contains "/learn/documentation" or page.url contains "/releases/" %}
+
+          <div class="releases-list-container">
+
+            <span>Releases</span>
+
+            <ul class="releases-list" data-releases-list>
+              <li class="hide"><a href="{{ page.url | replace:"versioned","latest"}}">latest</a></li>
+              
+              {% assign releases_sorted = site.releases | sort: 'order' | reverse %}
+              {% for release in releases_sorted %}
+
+              <li class="hide"><a href="{{ page.url | replace:"versioned", release.version }}">{{ release.version }}</a></li>
+
+              {% endfor %}
+
+            </ul>
+
+          </div>
+
+          <hr class="releases-list-divider">
+
+          {% endif %}
+    
+          {{ content }}
+           
+        </div>
+      </div>
+
+  </div>
+  
+
+  <!-- footer starts here -->
+
+  {% include footer.html %}
+
+</body>
+
+</html>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/samza/blob/b8058af0/docs/_layouts/talks-and-meetups.html
----------------------------------------------------------------------
diff --git a/docs/_layouts/talks-and-meetups.html b/docs/_layouts/talks-and-meetups.html
new file mode 100644
index 0000000..2086f3c
--- /dev/null
+++ b/docs/_layouts/talks-and-meetups.html
@@ -0,0 +1,42 @@
+---
+layout: page
+---
+<!--
+   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.
+-->
+
+<div class="breadcrumbs-list-container">
+
+    <span>{{ page.date | date: "%B %-d, %Y"}}</span>
+
+    <ul class="breadcrumbs-list">
+        <li class=""><a href="/news/">News</a></li>
+        <li class="divider"></li>
+        <li class="">
+            <i class="icon ion-md-arrow-dropleft"></i>
+            <a href="/news/">Previous</a>
+        </li>
+        <li class="" style="float: right;">
+            <a href="/news/">Next</a>
+            <i class="icon ion-md-arrow-dropright"></i>
+        </li>
+    </ul>
+
+</div>
+
+<hr class="breadcrumbs-list-divider">
+
+{{ page.content | remove: page.excerpt }}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/samza/blob/b8058af0/docs/_meetups/dec-2018.md
----------------------------------------------------------------------
diff --git a/docs/_meetups/dec-2018.md b/docs/_meetups/dec-2018.md
new file mode 100644
index 0000000..3aa8f60
--- /dev/null
+++ b/docs/_meetups/dec-2018.md
@@ -0,0 +1,46 @@
+---
+date: December 2018
+display_date: December 2018
+meetups:
+    - name: 'How we did something awesome using samza'
+      date: December 15th, 2018
+      url: https://google.com
+      host: LinkedIn
+      image: 
+      presenters:
+        - name: Xinyu Liu
+          website: 
+          image:
+          affiliation: LinkedIn
+      abstract: We will discuss x y and z.
+    - name: Another amazing meetup
+      date: December 12th, 2018
+      url: https://google.com
+      host: LinkedIn
+      image: 
+      presenters:
+        - name: Ajith Muralidharan
+          website: 
+          image:
+          affiliation: LinkedIn
+        - name: Vivek Nelamangala
+          website:
+          image: 
+          affiliation: LinkedIn
+---
+<!--
+   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.
+-->
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/samza/blob/b8058af0/docs/_meetups/july-2018.md
----------------------------------------------------------------------
diff --git a/docs/_meetups/july-2018.md b/docs/_meetups/july-2018.md
new file mode 100644
index 0000000..00ac7ea
--- /dev/null
+++ b/docs/_meetups/july-2018.md
@@ -0,0 +1,56 @@
+---
+date: July 2018
+display_date: July 2018
+meetups:
+    - name: 'Beam me up Samza: How we built a Samza Runner for Apache Beam'
+      date: July 19, 2018
+      url: https://google.com
+      host: LinkedIn
+      image: 
+      presenters:
+        - name: Xinyu Liu
+          website: 
+          image:
+          affiliation: LinkedIn
+      abstract: Apache Beam is an open source unified programming model to define and execute data processing pipelines
+      video:
+          url: https://www.youtube.com/watch?v=o5GaifLoZho&list=PLZDyxA22zzGx34wdHESUux2_V1qfkQ8zx&index=44&t=0s
+          image: https://img.youtube.com/vi/o5GaifLoZho/maxresdefault.jpg
+      slides:
+          url: https://www.slideshare.net/AbhishekShivanna/streamprocessingatlinkedinwithapachesamza-105630048
+    - name: Concourse - Near real-time notifications platform at LinkedIn
+      date: July 19, 2018
+      url: https://google.com
+      host: LinkedIn
+      image: 
+      presenters:
+        - name: Ajith Muralidharan
+          website: 
+          image:
+          affiliation: LinkedIn
+        - name: Vivek Nelamangala
+          website:
+          image: 
+          affiliation: LinkedIn
+      video:
+          url: https://www.youtube.com/watch?v=Fszo6jThq0I&list=PLZDyxA22zzGx34wdHESUux2_V1qfkQ8zx&index=2&t=0s
+          image: https://img.youtube.com/vi/Fszo6jThq0I/maxresdefault.jpg
+      slides:
+          url: https://www.slideshare.net/AbhishekShivanna/streamprocessingatlinkedinwithapachesamza-105630048
+---
+<!--
+   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.
+-->
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/samza/blob/b8058af0/docs/_menu/index.html
----------------------------------------------------------------------
diff --git a/docs/_menu/index.html b/docs/_menu/index.html
new file mode 100644
index 0000000..bf31496
--- /dev/null
+++ b/docs/_menu/index.html
@@ -0,0 +1,160 @@
+---
+items:
+  - menu_title: Case Studies
+    get_items: case-studies
+    items:
+      - menu_title: View All
+        url: /case-studies/
+      - divider: true
+  - menu_title: Getting Started
+    items:
+      - menu_title: QuickStart
+        url: /startup/hello-samza/version/
+      - menu_title: Code Examples
+        url: /learn/tutorials/version/
+  - menu_title: Documentation
+    has_sub: true
+    has_sub_subs: true
+    items_attributes: 'data-documentation="/learn/documentation/version/"'
+    items: 
+      - menu_title: Release Notes
+        url: /startup/releases/version/release-notes.html
+      - divider: true
+  - menu_title: Talks
+    url: '/talks/'
+  - menu_title: Meetups
+    url: '/meetups/'
+  - menu_title: Releases
+    get_items: releases
+    map_title: version
+    map_url: link
+    sort: 'order'
+    reverse: true
+  - menu_title: News
+    url: '/news/'
+  - menu_title: Community
+    items:
+      - menu_title: Contact Us
+        url: '/community/mailing-lists.html'
+      - menu_title: Contributor's Corner 
+        url: '/contribute/contributors-corner.html'
+      - menu_title: PMC members and committers
+        url: '/community/committers.html'
+      - menu_title: Powered By
+        url: '/powered-by/'
+---
+<!--
+   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.
+-->
+
+
+<div class="side-navigation">
+
+  {% for item in page.items %}
+
+    {% if item.has_sub_subs %}
+      {% assign sub_class = 'side-navigation__group-has-submenus' %}
+    {% else %}
+      {% assign sub_class = '' %}
+    {% endif %}
+
+    {% if item.get_items or item.items or item.has_sub %}
+      {% assign has_sub = true  %}
+    {% else %}
+      {% assign has_sub = false %}
+    {% endif %}
+    
+    <!-- Start Group -->
+    {% if has_sub %}
+    <div class="side-navigation__group side-navigation__group--has-nested" data-plugin="sub-menu" data-sub-menu-show-class="side-navigation__group--has-nested-visible">
+    {% else %}
+    <div class="side-navigation__group">
+    {% endif %}
+
+    <!-- Make menu_title, and start items group if needed -->
+    {% if has_sub %}
+      <div class="side-navigation__group-title">
+        <i class="side-navigation__group-title-icon icon ion-md-arrow-dropdown"></i>
+        {{ item.menu_title }}
+      </div>
+      <div class="side-navigation__group-items {{ sub_class }}" data-sub-menu {{ item.items_attributes | replace: 'version', site.version }}>
+    {% else %}
+      <a class="side-navigation__group-title" href="{{ item.url | replace: 'version', site.version }}">
+        {{ item.menu_title }}
+      </a>
+    {% endif %}
+
+    <!-- Handle sub navigation items from data -->
+    {% if item.items %}
+      {% for subitem in item.items %}
+        {% if subitem.divider %}
+          <hr>
+          {% continue %}
+        {% endif %}
+      <a class="side-navigation__group-item" href="{{ subitem.url | replace: 'version', site.version }}">{{ subitem.menu_title }}</a>
+      {% endfor %}
+
+    {% endif %}
+
+    <!-- Handle sub nagivation from site collections -->
+    {% if item.get_items %}
+
+      {% assign subnav = site[item.get_items] %}
+
+      {% if item.sort %}
+
+      {% assign subnav = subnav | sort: item.sort %}
+
+      {% endif %}
+
+      {% if item.reverse %}
+
+      {% assign subnav = subnav | reverse %}
+
+      {% endif %}
+
+      {% for subitem in subnav %}
+        {% if subitem.exclude_from_loop %}
+          {% continue %}
+        {% endif %}
+
+        {% assign url = subitem.url %}
+        {% assign menu_title = subitem.menu_title %}
+
+        {% if item.map_title %}
+          {% assign menu_title = subitem[item.map_title] %}
+        {% endif %}
+
+        {% if item.map_url %}
+          {% assign url = subitem[item.map_url] %}
+        {% endif %}
+
+      <a class="side-navigation__group-item" href="{{ url }}">{{ menu_title }}</a>
+      {% endfor %}
+
+    {% endif %}
+
+    <!-- Close sub nav group -->
+    {% if has_sub %}
+      </div>
+    {% endif %}
+
+    <!-- Close menu group -->
+    </div>
+
+  {% endfor %}
+
+</div>

http://git-wip-us.apache.org/repos/asf/samza/blob/b8058af0/docs/_news/2018-01-31-announcing-the-release-of-apache-samza--0.14.0.md
----------------------------------------------------------------------
diff --git a/docs/_news/2018-01-31-announcing-the-release-of-apache-samza--0.14.0.md b/docs/_news/2018-01-31-announcing-the-release-of-apache-samza--0.14.0.md
new file mode 100644
index 0000000..78a2a8a
--- /dev/null
+++ b/docs/_news/2018-01-31-announcing-the-release-of-apache-samza--0.14.0.md
@@ -0,0 +1,76 @@
+---
+layout: news
+title: Announcing the release of Apache Samza 0.14.0
+icon: git-pull-request
+authors:
+    - name: Xinyu
+      website: 
+      image: 
+excerpt_separator: <!--more-->
+---
+<!--
+   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.
+-->
+
+
+Testing the excerpt
+
+<!--more-->
+
+
+Announcing the release of Apache Samza 0.14.0
+
+We are very excited to announce the release of **Apache Samza 0.14.0**  
+Samza has been powering real-time applications in production across several large companies (including LinkedIn, Netflix, Uber, Slack, Redfin, TripAdvisor, etc) for years now. Samza provides leading support for large-scale stateful stream processing with:
+
+-   First class support for local state (with RocksDB store). This allows a stateful application to scale up to 1.1 Million events/sec on a single machine with SSD.
+-   Support for incremental checkpointing of state instead of full snapshots. This enables Samza to scale to applications with very large state.
+-   A fully pluggable model for input sources (e.g. Kafka, Kinesis, DynamoDB streams etc.) and output systems (HDFS, Kafka, ElastiCache etc.).
+-   A fully asynchronous programming model that makes parallelizing remote calls efficient and effortless.
+-   High level API for expressing complex stream processing pipelines in a few lines of code.
+-   Flexible deployment model for running the the applications in any hosting environment and with cluster managers other than YARN.
+-   Features like canaries, upgrades and rollbacks that support extremely large deployments with minimal downtime.
+
+### New Features, Upgrades and Bug Fixes
+
+The 0.14.0 release contains the following highly anticipated features:
+
+-   Samza SQL
+-   Azure EventHubs producer, consumer and checkpoint provider
+-   AWS Kinesis consumer
+
+This release also includes improvements such as durable state in high-level API, Zookeeper-based deployment stability, and multi-stage batch processing, and bug fixes such as KafkaSystemProducer concurrent sends and flushes.  
+Overall, [65 JIRAs](https://issues.apache.org/jira/browse/SAMZA-1109?jql=project%20%3D%20SAMZA%20AND%20status%20%3D%20Resolved%20AND%20fixVersion%20%3D%200.14.0%20ORDER%20BY%20priority%20DESC%2C%20key%20ASC) were resolved in this release. For more details about this release, please check out the [release notes](http://samza.apache.org/startup/releases/0.14/release-notes.html).
+
+### Community Developments
+
+We’ve made great community progress since the last release (0.13.1). We presented the unified data processing with Samza at the 2017 Big Data conference held in Spain and the Dataworks Summit in Sydney, and held a demo at @scale conference in San Jose. Here are the details to these conferences.
+
+-   Nov 17, 2017 - [Unified Stream Processing at Scale with Apache Samza (BigDataSpain 2017)](https://www.bigdataspain.org/2017/talk/apache-samza-jake-maes) ([Slides](https://www.slideshare.net/secret/oQe3debYJoY5q3))
+-   Sept 21, 2017 - [Unified Batch & Stream Processing with Apache Samza (Dataworks Summit Sydney 2017)](https://dataworkssummit.com/sydney-2017/) ([Slides](https://www.slideshare.net/Hadoop_Summit/unified-batch-stream-processing-with-apache-samza))
+-   Aug 31, 2017 - Demo of Stream Processing@LinkedIn (@scale conference 2017) ([Slides](https://www.slideshare.net/XinyuLiu11/samza-demo-scale-2017))
+
+In Dec 4th, we held the meetup for [Stream Processing with Apache Kafka & Apache Samza](https://www.meetup.com/Stream-Processing-Meetup-LinkedIn/events/244889719/), which has the following presentations for Samza:
+
+-   [Samza SQL](https://youtu.be/YDGIDO29Dqk) ([slides](https://www.slideshare.net/SamarthShetty2/stream-processing-using-samza-sql))
+-   [Streaming data pipelines at Slack](https://youtu.be/wbS1P9ehgd0) ([slides](https://speakerdeck.com/vananth22/streaming-data-pipelines-at-slack))
+
+As future development, we are continuing working on improvements to the new High Level API, SQL, Stream-Table Join and flexible deployment features.
+
+### Contribute
+
+It’s a great time to get involved. You can start by reviewing the [tutorials](http://samza.apache.org/startup/preview/#try-it-out), signing up for the [mailing list](http://samza.apache.org/community/mailing-lists.html), and grabbing some [newbie JIRAs](https://issues.apache.org/jira/issues/?jql=project%20%3D%20SAMZA%20AND%20labels%20%3D%20newbie%20AND%20status%20%3D%20Open).  
+I’d like to close by thanking everyone who’s been involved in the project. It’s been a great experience to be involved in this community, and I look forward to its continued growth.

http://git-wip-us.apache.org/repos/asf/samza/blob/b8058af0/docs/_news/2018-04-28-something-else.md
----------------------------------------------------------------------
diff --git a/docs/_news/2018-04-28-something-else.md b/docs/_news/2018-04-28-something-else.md
new file mode 100644
index 0000000..443844b
--- /dev/null
+++ b/docs/_news/2018-04-28-something-else.md
@@ -0,0 +1,36 @@
+---
+layout: news
+title: Totally Something Else Besides Release Notes
+icon: analytics
+authors:
+    - name: Xinyu
+      website: 
+      image: 
+excerpt_separator: <!--more-->
+---
+<!--
+   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.
+-->
+
+
+Totally something else to read about other than release notes...
+
+<!--more-->
+
+
+Super excited to write something else here.
+
+

http://git-wip-us.apache.org/repos/asf/samza/blob/b8058af0/docs/_news/2018-05-31-announcing-the-release-of-apache-samza--0.14.1.md
----------------------------------------------------------------------
diff --git a/docs/_news/2018-05-31-announcing-the-release-of-apache-samza--0.14.1.md b/docs/_news/2018-05-31-announcing-the-release-of-apache-samza--0.14.1.md
new file mode 100644
index 0000000..244f42d
--- /dev/null
+++ b/docs/_news/2018-05-31-announcing-the-release-of-apache-samza--0.14.1.md
@@ -0,0 +1,83 @@
+---
+layout: news
+title: Announcing the release of Apache Samza 0.14.1
+icon: git-pull-request
+authors:
+    - name: Xinyu
+      website: 
+      image: 
+excerpt_separator: <!--more-->
+---
+<!--
+   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.
+-->
+
+
+A minor release which contains improvements over multiple areas: SQL, Standalone, Eventhub, and Host-affinity.
+
+<!--more-->
+
+
+Announcing the release of Apache Samza 0.14.1
+
+We are very excited to announce the release of **Apache Samza 0.14.1**  
+Samza has been powering real-time applications in production across several large companies (including LinkedIn, Netflix, Uber, Slack, Redfin, TripAdvisor, etc) for years now. Samza provides leading support for large-scale stateful stream processing with:
+
+-   First class support for local state (with RocksDB store). This allows a stateful application to scale up to 1.1 Million events/sec on a single machine with SSD.
+-   Support for incremental checkpointing of state instead of full snapshots. This enables Samza to scale to applications with very large state.
+-   A fully pluggable model for input sources (e.g. Kafka, Kinesis, DynamoDB streams etc.) and output systems (HDFS, Kafka, ElastiCache etc.).
+-   A fully asynchronous programming model that makes parallelizing remote calls efficient and effortless.
+-   High level API for expressing complex stream processing pipelines in a few lines of code.
+-   Flexible deployment model for running the the applications in any hosting environment and with cluster managers other than YARN.
+-   Features like canaries, upgrades and rollbacks that support extremely large deployments with minimal downtime.
+
+### Enhancements, Upgrades and Bug Fixes
+
+This is a minor release which contains improvements over multiple areas. In particular:  
+
+-   **SQL**
+-   [SAMZA-1681](https://issues.apache.org/jira/browse/SAMZA-1681) Add support for handling older record schema versions in AvroRelConverter
+-   [SAMZA-1671](https://issues.apache.org/jira/browse/SAMZA-1671) Add insert into table support
+-   [SAMZA-1651](https://issues.apache.org/jira/browse/SAMZA-1651) Implement GROUP BY SQL operator
+-   **Standalone**
+-   [SAMZA-1689](https://issues.apache.org/jira/browse/SAMZA-1689) Add validations before state transitions in ZkBarrierForVersionUpgrade
+-   [SAMZA-1686](https://issues.apache.org/jira/browse/SAMZA-1686) Set finite operation timeout when creating zkClient
+-   [SAMZA-1667](https://issues.apache.org/jira/browse/SAMZA-1667) Skip storing configuration as a part of JobModel in zookeeper data nodes
+-   [SAMZA-1647](https://issues.apache.org/jira/browse/SAMZA-1647) Fix NPE in JobModelExpired event handler
+-   **Eventhub**
+-   [SAMZA-1688](https://issues.apache.org/jira/browse/SAMZA-1688) Use per partition eventhubs client
+-   [SAMZA-1676](https://issues.apache.org/jira/browse/SAMZA-1676) Miscellaneous fix and improvement for eventhubs system
+-   [SAMZA-1656](https://issues.apache.org/jira/browse/SAMZA-1656) EventHubSystemAdmin does not fetch metadata for valid streams
+-   **Host-affinity**
+-   [SAMZA-1687](https://issues.apache.org/jira/browse/SAMZA-1687) Prioritize preferred host requests over ANY-HOST requests
+-   [SAMZA-1649](https://issues.apache.org/jira/browse/SAMZA-1649) Improve host-aware allocation to account for strict locality
+
+In addition, Samza is also upgraded to support Kafka version 0.11 in this release.  
+Overall, [51 JIRAs](https://issues.apache.org/jira/projects/SAMZA/versions/12343155) were resolved in this release. A source download of the 0.14.1 release is available [here](http://www.apache.org/dyn/closer.cgi/samza/0.14.1). The release JARs are also available in Apache’s Maven repository. See Samza’s [download](http://samza.apache.org/startup/download/) page for details and Samza’s [feature preview](https://samza.apache.org/startup/preview/) for new features. We requires JDK version newer than 1.8.0_111 when running 0.14.1 release for users who are using Scala 2.12.
+
+### Community Developments
+
+In March 21th, we held the meetup for [Stream Processing with Apache Kafka & Apache Samza](https://www.meetup.com/Stream-Processing-Meetup-LinkedIn/events/248309045/), which has the following presentations for Samza:
+
+-   [Conquering the Lambda architecture in LinkedIn metrics platform with Apache Calcite and Apache Samza](https://www.youtube.com/watch?v=ZPWInJ4USIU) ([Slides](https://www.slideshare.net/KhaiTran17/conquering-the-lambda-architecture-in-linkedin-metrics-platform-with-apache-calcite-and-apache-samza))
+-   [Building Venice with Apache Kafka & Samza](https://www.youtube.com/watch?v=Usz8E4S-hZE)
+
+In industry, Samza got new adopters, including [Ntent](http://www.ntent.com) and [Movico](https://movio.co/en/).
+
+### Contribute
+
+It’s a great time to get involved. You can start by reviewing the [tutorials](http://samza.apache.org/startup/preview/#try-it-out), signing up for the [mailing list](http://samza.apache.org/community/mailing-lists.html), and grabbing some [newbie JIRAs](https://issues.apache.org/jira/issues/?jql=project%20%3D%20SAMZA%20AND%20labels%20%3D%20newbie%20AND%20status%20%3D%20Open).  
+I’d like to close by thanking everyone who’s been involved in the project. It’s been a great experience to be involved in this community, and I look forward to its continued growth.

http://git-wip-us.apache.org/repos/asf/samza/blob/b8058af0/docs/_news/TEMPLATE.md
----------------------------------------------------------------------
diff --git a/docs/_news/TEMPLATE.md b/docs/_news/TEMPLATE.md
new file mode 100644
index 0000000..84faf18
--- /dev/null
+++ b/docs/_news/TEMPLATE.md
@@ -0,0 +1,38 @@
+---
+exclude_from_loop: true # useful for drafts 
+layout: news
+title: Your title with dashes for spaces and special chars # name the file with YYYY-DD-MM-your-title-with-dashes-for-spaces-and-special-chars.md
+icon: git-pull-request # iconic icons https://ionicons.com/
+authors: #optional
+    - name: 
+      website: #optional
+      image: #optional
+    - name: 
+      website: 
+      image: 
+excerpt_separator: <!--more-->
+---
+<!--
+   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.
+-->
+
+
+An excerpt which should be brief to provide the jist of what the article will be about.
+
+<!--more-->
+
+
+All the content of the news article, which can be both html and markdown.
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/samza/blob/b8058af0/docs/_news/index.md
----------------------------------------------------------------------
diff --git a/docs/_news/index.md b/docs/_news/index.md
new file mode 100644
index 0000000..231f63c
--- /dev/null
+++ b/docs/_news/index.md
@@ -0,0 +1,54 @@
+---
+layout: page
+title: Samza News
+exclude_from_loop: true
+---
+<!--
+   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.
+-->
+
+<div class="news_cards">
+
+  {% assign sorted = (site.news | sort: 'date') | reverse %}
+
+  {% for post in sorted %}
+
+    {% if post.exclude_from_loop %}
+      {% continue %}
+    {% endif %}
+
+    {% assign icon = "ion-md-paper" %}
+
+    {% if post.icon %}
+
+    {% assign icon = "ion-md-" | append: post.icon %}
+
+    {% endif %}
+
+  <a class="news__card" href="{{ post.url }}">
+    <i class="news__card-icon icon {{ icon }}"></i>
+    <div class="news__card-date">{{ post.date | date: "%B %-d, %Y" }}</div>
+    <div class="news__card-title">{{ post.title }}</div>
+    {% if post.excerpt %}
+    <div class="news__card-description">
+      {{ post.excerpt }}
+    </div>
+    {% endif %}
+    <span class="news__card-button">Read more</span>
+  </a>
+  {% endfor %}
+
+</div>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/samza/blob/b8058af0/docs/_powered-by/TEMPLATE.md
----------------------------------------------------------------------
diff --git a/docs/_powered-by/TEMPLATE.md b/docs/_powered-by/TEMPLATE.md
new file mode 100644
index 0000000..8225f7e
--- /dev/null
+++ b/docs/_powered-by/TEMPLATE.md
@@ -0,0 +1,23 @@
+---
+exclude_from_loop: true # wont be able to find this page, useful for draft
+name: Company # formatted name of company eg LinkedIn
+domain: company.com # just the domain, no protocol
+---
+<!--
+   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.
+-->
+
+The description to show which can be html or markdown, but remember when referencing the <a class="external-link" href="https://www.company.com/" rel="nofollow">Company</a>, we should be using a `rel="nofollow"` attribute in an anchor tag `a`.
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/samza/blob/b8058af0/docs/_powered-by/intuit.md
----------------------------------------------------------------------
diff --git a/docs/_powered-by/intuit.md b/docs/_powered-by/intuit.md
new file mode 100644
index 0000000..e77eddc
--- /dev/null
+++ b/docs/_powered-by/intuit.md
@@ -0,0 +1,22 @@
+---
+name: Intuit
+domain: intuit.com
+---
+<!--
+   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.
+-->
+
+At <a class="external-link" href="https://www.intuit.com/" rel="nofollow">Intuit</a>, we use Samza to enrich events with more contextual data from various sources (CMDB, Change Management, Incident Management, Problem Management). This gives us more meaningful events that an operations centre person can act on.
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/samza/blob/b8058af0/docs/_powered-by/linkedin.md
----------------------------------------------------------------------
diff --git a/docs/_powered-by/linkedin.md b/docs/_powered-by/linkedin.md
new file mode 100644
index 0000000..2ba4c40
--- /dev/null
+++ b/docs/_powered-by/linkedin.md
@@ -0,0 +1,22 @@
+---
+name: LinkedIn
+domain: linkedin.com
+---
+<!--
+   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.
+-->
+
+Samza was originally developed at <a class="external-link" href="https://linkedin.com/" rel="nofollow">LinkedIn</a>. It's currently used to process tracking data, service log data, and for data ingestion pipelines for realtime services.
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/samza/blob/b8058af0/docs/_powered-by/mobileaware.md
----------------------------------------------------------------------
diff --git a/docs/_powered-by/mobileaware.md b/docs/_powered-by/mobileaware.md
new file mode 100644
index 0000000..ea07155
--- /dev/null
+++ b/docs/_powered-by/mobileaware.md
@@ -0,0 +1,22 @@
+---
+name: MobileAware
+domain: mobileaware.com
+---
+<!--
+   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.
+-->
+
+At <a class="external-link" href="https://www.mobileaware.com/" rel="nofollow">MobileAware</a>, we use Samza to enrich events with more contextual data from various sources (CMDB, Change Management, Incident Management, Problem Management). This gives us more meaningful events that an operations centre person can act on.
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/samza/blob/b8058af0/docs/_releases/0.10.md
----------------------------------------------------------------------
diff --git a/docs/_releases/0.10.md b/docs/_releases/0.10.md
new file mode 100644
index 0000000..d4e8850
--- /dev/null
+++ b/docs/_releases/0.10.md
@@ -0,0 +1,21 @@
+---
+version: '0.10'
+order: 10
+link: /learn/documentation/0.10/introduction/background.html
+---
+<!--
+   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/samza/blob/b8058af0/docs/_releases/0.11.md
----------------------------------------------------------------------
diff --git a/docs/_releases/0.11.md b/docs/_releases/0.11.md
new file mode 100644
index 0000000..6f21b05
--- /dev/null
+++ b/docs/_releases/0.11.md
@@ -0,0 +1,21 @@
+---
+version: '0.11'
+order: 11
+link: /learn/documentation/0.11/introduction/background.html
+---
+<!--
+   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/samza/blob/b8058af0/docs/_releases/0.12.md
----------------------------------------------------------------------
diff --git a/docs/_releases/0.12.md b/docs/_releases/0.12.md
new file mode 100644
index 0000000..bd9eb3b
--- /dev/null
+++ b/docs/_releases/0.12.md
@@ -0,0 +1,21 @@
+---
+version: '0.12'
+order: 12
+link: /learn/documentation/0.12/introduction/background.html
+---
+<!--
+   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/samza/blob/b8058af0/docs/_releases/0.13.md
----------------------------------------------------------------------
diff --git a/docs/_releases/0.13.md b/docs/_releases/0.13.md
new file mode 100644
index 0000000..65e64f0
--- /dev/null
+++ b/docs/_releases/0.13.md
@@ -0,0 +1,21 @@
+---
+version: '0.13'
+order: 13
+link: /learn/documentation/0.13/introduction/background.html
+---
+<!--
+   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/samza/blob/b8058af0/docs/_releases/0.14.md
----------------------------------------------------------------------
diff --git a/docs/_releases/0.14.md b/docs/_releases/0.14.md
new file mode 100644
index 0000000..8a0ce10
--- /dev/null
+++ b/docs/_releases/0.14.md
@@ -0,0 +1,21 @@
+---
+version: '0.14'
+order: 14
+link: /learn/documentation/0.14/introduction/background.html
+---
+<!--
+   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/samza/blob/b8058af0/docs/_releases/0.7.0.md
----------------------------------------------------------------------
diff --git a/docs/_releases/0.7.0.md b/docs/_releases/0.7.0.md
new file mode 100644
index 0000000..a46019e
--- /dev/null
+++ b/docs/_releases/0.7.0.md
@@ -0,0 +1,21 @@
+---
+version: '0.7.0'
+order: 7
+link: /learn/documentation/0.7.0/introduction/background.html
+---
+<!--
+   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/samza/blob/b8058af0/docs/_releases/0.8.md
----------------------------------------------------------------------
diff --git a/docs/_releases/0.8.md b/docs/_releases/0.8.md
new file mode 100644
index 0000000..3438356
--- /dev/null
+++ b/docs/_releases/0.8.md
@@ -0,0 +1,21 @@
+---
+version: '0.8'
+order: 8
+link: /learn/documentation/0.8/introduction/background.html
+---
+<!--
+   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/samza/blob/b8058af0/docs/_releases/0.9.md
----------------------------------------------------------------------
diff --git a/docs/_releases/0.9.md b/docs/_releases/0.9.md
new file mode 100644
index 0000000..71f3659
--- /dev/null
+++ b/docs/_releases/0.9.md
@@ -0,0 +1,21 @@
+---
+version: '0.9'
+order: 9
+link: /learn/documentation/0.9/introduction/background.html
+---
+<!--
+   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/samza/blob/b8058af0/docs/_releases/TEMPLATE.md
----------------------------------------------------------------------
diff --git a/docs/_releases/TEMPLATE.md b/docs/_releases/TEMPLATE.md
new file mode 100644
index 0000000..0fdb4fe
--- /dev/null
+++ b/docs/_releases/TEMPLATE.md
@@ -0,0 +1,21 @@
+---
+exclude_from_loop: true # useful for drafts
+version: '0.7' # make sure it is a string
+order: 7 # this matters for sorting... since .7 needs to be thought of as an earlier version than .14... if you need to do 1.7 do 107, 1.14 as 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.
+-->

http://git-wip-us.apache.org/repos/asf/samza/blob/b8058af0/docs/_talks/TEMPLATE.md
----------------------------------------------------------------------
diff --git a/docs/_talks/TEMPLATE.md b/docs/_talks/TEMPLATE.md
new file mode 100644
index 0000000..5c1de4e
--- /dev/null
+++ b/docs/_talks/TEMPLATE.md
@@ -0,0 +1,34 @@
+---
+exclude_from_loop: true # if you dont want anyone to find this page, useful for a draft
+date: September 2018 # a jekyll date, used for sorting
+display_date: September 25th, 2018 # would use a date format but sometimes date is missing a DAY and it shows up with 1st as default
+event:
+  name: TALK TITLE
+  url: https://google.com # HTTP... to the event page
+  host: Some Cool Summit 2018 # Name of group/company hosting the event
+  by: Bob Sherman # Name of talk presenter (optional)
+  image: # HTTP... url to image of event (optional)
+video:
+  url: # HTTP... url to video of talk
+  image: # HTTP... url to IMAGE of video #https://img.youtube.com/vi/VIDEO_ID/maxresdefault.jpg
+slides:
+  url: # HTTP... url to slides
+---
+<!--
+   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.
+-->
+
+A possible description that is either in markdown or html, and is totally optional.
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/samza/blob/b8058af0/docs/_talks/june-2018--bangalore-kafka-group.md
----------------------------------------------------------------------
diff --git a/docs/_talks/june-2018--bangalore-kafka-group.md b/docs/_talks/june-2018--bangalore-kafka-group.md
new file mode 100644
index 0000000..f88a646
--- /dev/null
+++ b/docs/_talks/june-2018--bangalore-kafka-group.md
@@ -0,0 +1,30 @@
+---
+date: June 2018
+display_date: June 2018
+event:
+  name: Stream Processing at LinkedIn with Apache Samza
+  url: https://www.meetup.com/Bangalore-Apache-Kafka-Group/events/251707854/
+  host: Bangalore Kafka Group
+  image: 
+video:
+  url: 
+  image:
+slides:
+  url: https://www.slideshare.net/AbhishekShivanna/streamprocessingatlinkedinwithapachesamza-105630048
+---
+<!--
+   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.
+-->
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/samza/blob/b8058af0/docs/_talks/november-2017-big-data-span-2017.md
----------------------------------------------------------------------
diff --git a/docs/_talks/november-2017-big-data-span-2017.md b/docs/_talks/november-2017-big-data-span-2017.md
new file mode 100644
index 0000000..dc1b660
--- /dev/null
+++ b/docs/_talks/november-2017-big-data-span-2017.md
@@ -0,0 +1,30 @@
+---
+date: November 2017
+display_date: November 2017
+event:
+  name: Unified Stream Processing at Scale with Apache Samza
+  url: https://www.meetup.com/Bangalore-Apache-Kafka-Group/events/251707854/
+  host: BigDataSpain 2017
+  image:
+video:
+  url:
+  image:
+slides:
+  url: https://www.slideshare.net/AbhishekShivanna/streamprocessingatlinkedinwithapachesamza-105630048
+---
+<!--
+   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.
+-->
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/samza/blob/b8058af0/docs/_talks/november-2018--totally-awesome-summit.md
----------------------------------------------------------------------
diff --git a/docs/_talks/november-2018--totally-awesome-summit.md b/docs/_talks/november-2018--totally-awesome-summit.md
new file mode 100644
index 0000000..53676f8
--- /dev/null
+++ b/docs/_talks/november-2018--totally-awesome-summit.md
@@ -0,0 +1,30 @@
+---
+date: November 12th, 2018
+display_date: November 12th, 2018
+event:
+  name: Totally awesome event
+  url: https://google.com
+  host: Totally awesome summit
+  image:
+video:
+  url:
+  image:
+slides:
+  url:
+---
+<!--
+   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.
+-->
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/samza/blob/b8058af0/docs/_talks/september-21-2017--dataworks-summit-sydney-2017.md
----------------------------------------------------------------------
diff --git a/docs/_talks/september-21-2017--dataworks-summit-sydney-2017.md b/docs/_talks/september-21-2017--dataworks-summit-sydney-2017.md
new file mode 100644
index 0000000..eac68be
--- /dev/null
+++ b/docs/_talks/september-21-2017--dataworks-summit-sydney-2017.md
@@ -0,0 +1,33 @@
+---
+date: Sept 21, 2017
+display_date: Sept 21, 2017
+event:
+  name: Unified Batch & Stream Processing with Apache Samza
+  url: https://www.meetup.com/Bangalore-Apache-Kafka-Group/events/251707854/
+  host: Dataworks Summit Sydney 2017
+  by: Bob Sherman
+  image: 
+video:
+  url:
+  image:
+slides:
+  url: https://www.slideshare.net/AbhishekShivanna/streamprocessingatlinkedinwithapachesamza-105630048
+---
+<!--
+   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.
+-->
+
+A description of the talk, if desired which can be in markdown or html.
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/samza/blob/b8058af0/docs/community/committers-old.md
----------------------------------------------------------------------
diff --git a/docs/community/committers-old.md b/docs/community/committers-old.md
new file mode 100644
index 0000000..222a661
--- /dev/null
+++ b/docs/community/committers-old.md
@@ -0,0 +1,103 @@
+---
+layout: page
+title: Committers
+---
+<!--
+   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.
+-->
+
+**Yi Pan**<br/>
+Committer, PMC member and VP of Apache Samza<br/>
+<a href="https://www.linkedin.com/pub/yi-pan/9/85a/238" target="_blank"><i class="fa fa-linkedin committer-icon"></i></a>
+<a href="https://twitter.com/nickpan47" target="_blank"><i class="fa fa-twitter committer-icon"></i></a>
+
+**Navina Ramesh**<br/>
+Committer, and PMC member<br/>
+<a href="https://www.linkedin.com/in/rnavina" target="_blank"><i class="fa fa-linkedin committer-icon"></i></a>
+<a href="https://twitter.com/navina_r" target="_blank"><i class="fa fa-twitter committer-icon"></i></a>
+
+**Jagadish Venkatraman**<br/>
+Committer, and PMC member<br/>
+<a href="https://www.linkedin.com/in/jagadishvenkat" target="_blank"><i class="fa fa-linkedin committer-icon"></i></a>
+<a href="https://twitter.com/vjagadish1989" target="_blank"><i class="fa fa-twitter committer-icon"></i></a>
+
+**Jake Maes**<br/>
+Committer, and PMC member<br/>
+<a href="www.linkedin.com/in/jacobmaes" target="_blank"><i class="fa fa-linkedin committer-icon"></i></a>
+<a href="https://twitter.com/jakemaes" target="_blank"><i class="fa fa-twitter committer-icon"></i></a>
+
+**Xinyu Liu**<br/>
+Committer, and PMC member<br/>
+<a href="https://www.linkedin.com/in/xinyu-liu-b0b21648" target="_blank"><i class="fa fa-linkedin committer-icon"></i></a>
+
+**Chris Riccomini**<br/>
+Committer, and PMC member<br/>
+<a href="http://www.linkedin.com/in/riccomini" target="_blank"><i class="fa fa-linkedin committer-icon"></i></a>
+<a href="http://twitter.com/criccomini" target="_blank"><i class="fa fa-twitter committer-icon"></i></a>
+
+**Chinmay Soman**<br/>
+Committer, and PMC member<br/>
+<a href="https://www.linkedin.com/pub/chinmay-soman/5/610/35" target="_blank"><i class="fa fa-linkedin committer-icon"></i></a>
+<a href="https://twitter.com/ChinmaySoman" target="_blank"><i class="fa fa-twitter committer-icon"></i></a>
+
+**Yan Fang**<br/>
+Committer, and PMC member<br/>
+<a href="https://www.linkedin.com/in/yanfangus" target="_blank"><i class="fa fa-linkedin committer-icon"></i></a>
+<a href="https://twitter.com/yanfang724" target="_blank"><i class="fa fa-twitter committer-icon"></i></a>
+
+**Garry Turkington**<br/>
+Committer, and PMC member<br/>
+<a href="http://uk.linkedin.com/in/garryturkington" target="_blank"><i class="fa fa-linkedin committer-icon"></i></a>
+<a href="http://twitter.com/garryturk" target="_blank"><i class="fa fa-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="fa fa-linkedin committer-icon"></i></a>
+<a href="http://twitter.com/blueboxtraveler" target="_blank"><i class="fa fa-twitter committer-icon"></i></a>
+
+**Martin Kleppmann**<br/>
+Committer, and PMC member<br/>
+<a href="https://www.linkedin.com/in/martinkleppmann" target="_blank"><i class="fa fa-linkedin committer-icon"></i></a>
+<a href="https://twitter.com/martinkl" target="_blank"><i class="fa fa-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="fa fa-linkedin committer-icon"></i></a>
+<a href="http://twitter.com/jaykreps" target="_blank"><i class="fa fa-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="fa fa-linkedin committer-icon"></i></a>
+<a href="http://twitter.com/sriramsub1" target="_blank"><i class="fa fa-twitter committer-icon"></i></a>
+
+**Zhijie Shen**<br/>
+Committer, and PMC member<br/>
+<a href="https://www.linkedin.com/in/zjshen" target="_blank"><i class="fa fa-linkedin committer-icon"></i></a>
+<a href="https://twitter.com/zhijieshen" target="_blank"><i class="fa fa-twitter committer-icon"></i></a>
+
+**Boris Shkolnik**<br/>
+Committer<br/>
+<a href="https://www.linkedin.com/in/boryas" target="_blank"><i class="fa fa-linkedin committer-icon"></i></a>
+<a href="https://twitter.com/sborya" target="_blank"><i class="fa fa-twitter committer-icon"></i></a>
+
+**Prateek Maheshwari**<br/>
+Committer<br/>
+<a href="https://www.linkedin.com/in/mprateek" target="_blank"><i class="fa fa-linkedin committer-icon"></i></a>
+
+**Wei Song**<br/>
+Committer<br/>
+<a href="https://www.linkedin.com/in/weisong44" target="_blank"><i class="fa fa-linkedin committer-icon"></i></a>
+

http://git-wip-us.apache.org/repos/asf/samza/blob/b8058af0/docs/community/committers.html
----------------------------------------------------------------------
diff --git a/docs/community/committers.html b/docs/community/committers.html
new file mode 100644
index 0000000..d155fb9
--- /dev/null
+++ b/docs/community/committers.html
@@ -0,0 +1,97 @@
+---
+layout: page
+title: Committers
+exclude_from_loop: true
+---
+<!--
+   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.
+-->
+
+A list of people who have contributed to Samza
+
+<hr class="committers-hr"/>
+
+<ul class="committers">
+
+  {% assign committers = site.committers | sort: 'order' %}
+
+  {% for committer in committers %}
+    {% if committer.exclude_from_loop %}
+        {% continue %}
+    {% endif %}
+
+  <li class="committer">
+
+    <span class="committer-image" title="{{ committer.name }}">
+
+      {% if committer.image %}
+      <span style="background-image: url('{{ committer.image }}');"></span>
+      {% elsif committer.github %}
+      <a href="https://github.com/{{ committer.github }}" target="_blank" rel="nofollow"><span style="background-image: url('https://github.com/{{ committer.github}}.png?size=256');"></span></a>
+      {% else %}
+      <span style="background-image: url('https://api.adorable.io/avatars/256/{{ committer.name }}');"></span>
+      {% endif %}
+      
+    </span>
+    <div class="committer-info">
+      <div class="committer-name">
+          {{ committer.name }}
+      </div>
+      <div class="committer-roles">
+          {% if committer.samza_title %}
+          <span>{{ committer.samza_title }}</span>
+          {% endif %}
+
+          {% if committer.apache_member %}
+          <span>Apache Member</span>
+          {% endif %}
+
+          {% if committer.pmc_member %}
+          <span>PMC Member</span>
+          {% endif %}
+
+          <span>Committer</span>
+      </div>
+      <div class="committer-links">
+
+          {% if committer.website %}
+          <a href="{{ committer.website}}" class="committer-link" rel="nofollow" target="_blank">
+            <i class="committer-icon icon ion-md-globe"></i>
+          </a>
+          {% endif %}
+
+          {% if committer.linkedin %}
+          <a href="{{ committer.linkedin}}" class="committer-link" rel="nofollow" target="_blank">
+            <i class="committer-icon icon ion-logo-linkedin"></i>
+          </a>
+          {% endif %}
+
+          {% if committer.twitter %}
+          <a href="https://twitter.com/{{ committer.twitter}}" class="committer-link" rel="nofollow" target="_blank">
+            <i class="committer-icon icon ion-logo-twitter"></i>
+          </a>
+          {% endif %}
+
+      </div>
+      
+
+     
+    </div>
+  </li>
+
+  {% endfor %}
+
+</ul>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/samza/blob/b8058af0/docs/community/committers.md
----------------------------------------------------------------------
diff --git a/docs/community/committers.md b/docs/community/committers.md
deleted file mode 100644
index 222a661..0000000
--- a/docs/community/committers.md
+++ /dev/null
@@ -1,103 +0,0 @@
----
-layout: page
-title: Committers
----
-<!--
-   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.
--->
-
-**Yi Pan**<br/>
-Committer, PMC member and VP of Apache Samza<br/>
-<a href="https://www.linkedin.com/pub/yi-pan/9/85a/238" target="_blank"><i class="fa fa-linkedin committer-icon"></i></a>
-<a href="https://twitter.com/nickpan47" target="_blank"><i class="fa fa-twitter committer-icon"></i></a>
-
-**Navina Ramesh**<br/>
-Committer, and PMC member<br/>
-<a href="https://www.linkedin.com/in/rnavina" target="_blank"><i class="fa fa-linkedin committer-icon"></i></a>
-<a href="https://twitter.com/navina_r" target="_blank"><i class="fa fa-twitter committer-icon"></i></a>
-
-**Jagadish Venkatraman**<br/>
-Committer, and PMC member<br/>
-<a href="https://www.linkedin.com/in/jagadishvenkat" target="_blank"><i class="fa fa-linkedin committer-icon"></i></a>
-<a href="https://twitter.com/vjagadish1989" target="_blank"><i class="fa fa-twitter committer-icon"></i></a>
-
-**Jake Maes**<br/>
-Committer, and PMC member<br/>
-<a href="www.linkedin.com/in/jacobmaes" target="_blank"><i class="fa fa-linkedin committer-icon"></i></a>
-<a href="https://twitter.com/jakemaes" target="_blank"><i class="fa fa-twitter committer-icon"></i></a>
-
-**Xinyu Liu**<br/>
-Committer, and PMC member<br/>
-<a href="https://www.linkedin.com/in/xinyu-liu-b0b21648" target="_blank"><i class="fa fa-linkedin committer-icon"></i></a>
-
-**Chris Riccomini**<br/>
-Committer, and PMC member<br/>
-<a href="http://www.linkedin.com/in/riccomini" target="_blank"><i class="fa fa-linkedin committer-icon"></i></a>
-<a href="http://twitter.com/criccomini" target="_blank"><i class="fa fa-twitter committer-icon"></i></a>
-
-**Chinmay Soman**<br/>
-Committer, and PMC member<br/>
-<a href="https://www.linkedin.com/pub/chinmay-soman/5/610/35" target="_blank"><i class="fa fa-linkedin committer-icon"></i></a>
-<a href="https://twitter.com/ChinmaySoman" target="_blank"><i class="fa fa-twitter committer-icon"></i></a>
-
-**Yan Fang**<br/>
-Committer, and PMC member<br/>
-<a href="https://www.linkedin.com/in/yanfangus" target="_blank"><i class="fa fa-linkedin committer-icon"></i></a>
-<a href="https://twitter.com/yanfang724" target="_blank"><i class="fa fa-twitter committer-icon"></i></a>
-
-**Garry Turkington**<br/>
-Committer, and PMC member<br/>
-<a href="http://uk.linkedin.com/in/garryturkington" target="_blank"><i class="fa fa-linkedin committer-icon"></i></a>
-<a href="http://twitter.com/garryturk" target="_blank"><i class="fa fa-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="fa fa-linkedin committer-icon"></i></a>
-<a href="http://twitter.com/blueboxtraveler" target="_blank"><i class="fa fa-twitter committer-icon"></i></a>
-
-**Martin Kleppmann**<br/>
-Committer, and PMC member<br/>
-<a href="https://www.linkedin.com/in/martinkleppmann" target="_blank"><i class="fa fa-linkedin committer-icon"></i></a>
-<a href="https://twitter.com/martinkl" target="_blank"><i class="fa fa-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="fa fa-linkedin committer-icon"></i></a>
-<a href="http://twitter.com/jaykreps" target="_blank"><i class="fa fa-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="fa fa-linkedin committer-icon"></i></a>
-<a href="http://twitter.com/sriramsub1" target="_blank"><i class="fa fa-twitter committer-icon"></i></a>
-
-**Zhijie Shen**<br/>
-Committer, and PMC member<br/>
-<a href="https://www.linkedin.com/in/zjshen" target="_blank"><i class="fa fa-linkedin committer-icon"></i></a>
-<a href="https://twitter.com/zhijieshen" target="_blank"><i class="fa fa-twitter committer-icon"></i></a>
-
-**Boris Shkolnik**<br/>
-Committer<br/>
-<a href="https://www.linkedin.com/in/boryas" target="_blank"><i class="fa fa-linkedin committer-icon"></i></a>
-<a href="https://twitter.com/sborya" target="_blank"><i class="fa fa-twitter committer-icon"></i></a>
-
-**Prateek Maheshwari**<br/>
-Committer<br/>
-<a href="https://www.linkedin.com/in/mprateek" target="_blank"><i class="fa fa-linkedin committer-icon"></i></a>
-
-**Wei Song**<br/>
-Committer<br/>
-<a href="https://www.linkedin.com/in/weisong44" target="_blank"><i class="fa fa-linkedin committer-icon"></i></a>
-


[04/29] samza git commit: SAMZA-1879: Remove deprecated containerId from ContainerModel

Posted by bo...@apache.org.
SAMZA-1879: Remove deprecated containerId from ContainerModel

Author: Cameron Lee <ca...@linkedin.com>

Reviewers: Prateek Maheshwari <pm...@apache.org>, Shanthoosh Venkatraman <sv...@linkedin.com>

Closes #639 from cameronlee314/remove_container_id


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

Branch: refs/heads/NewKafkaSystemConsumer
Commit: 3bb24c8eec40099dbc177df7c77f2793d7d70653
Parents: 160927a
Author: Cameron Lee <ca...@linkedin.com>
Authored: Wed Sep 19 12:18:46 2018 -0700
Committer: Prateek Maheshwari <pm...@apache.org>
Committed: Wed Sep 19 12:18:46 2018 -0700

----------------------------------------------------------------------
 .../grouper/task/GroupByContainerCount.java     |   6 +-
 .../grouper/task/GroupByContainerIds.java       |   4 +-
 .../task/SingleContainerGrouperFactory.java     |   2 +-
 .../apache/samza/job/model/ContainerModel.java  |  31 +--
 .../org/apache/samza/job/model/JobModel.java    |   2 -
 .../model/JsonContainerModelMixIn.java          |  38 ++--
 .../serializers/model/JsonJobModelMixIn.java    |   2 +
 .../serializers/model/JsonTaskModelMixIn.java   |   2 +
 .../serializers/model/SamzaObjectMapper.java    |  41 ++--
 .../apache/samza/storage/StorageRecovery.java   |   2 +-
 .../samza/coordinator/JobModelManager.scala     |   2 +-
 .../grouper/task/TestGroupByContainerCount.java |  89 ++++----
 .../grouper/task/TestGroupByContainerIds.java   |  37 ++--
 .../samza/container/mock/ContainerMocks.java    |   6 +-
 .../coordinator/JobModelManagerTestUtil.java    |  13 +-
 .../apache/samza/job/model/TestJobModel.java    |   4 +-
 .../operators/impl/TestOperatorImplGraph.java   |   8 +-
 .../model/TestSamzaObjectMapper.java            | 205 +++++++++++++++----
 .../samza/container/TestSamzaContainer.scala    |  12 +-
 .../samza/coordinator/TestJobCoordinator.scala  |  12 +-
 .../webapp/TestApplicationMasterRestClient.java |   2 +-
 21 files changed, 310 insertions(+), 210 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/samza/blob/3bb24c8e/samza-core/src/main/java/org/apache/samza/container/grouper/task/GroupByContainerCount.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/container/grouper/task/GroupByContainerCount.java b/samza-core/src/main/java/org/apache/samza/container/grouper/task/GroupByContainerCount.java
index 74c69d6..b4d6c90 100644
--- a/samza-core/src/main/java/org/apache/samza/container/grouper/task/GroupByContainerCount.java
+++ b/samza-core/src/main/java/org/apache/samza/container/grouper/task/GroupByContainerCount.java
@@ -78,7 +78,7 @@ public class GroupByContainerCount implements BalancingTaskNameGrouper {
     // Convert to a Set of ContainerModel
     Set<ContainerModel> containerModels = new HashSet<>();
     for (int i = 0; i < containerCount; i++) {
-      containerModels.add(new ContainerModel(String.valueOf(i), i, taskGroups[i]));
+      containerModels.add(new ContainerModel(String.valueOf(i), taskGroups[i]));
     }
 
     return Collections.unmodifiableSet(containerModels);
@@ -194,7 +194,7 @@ public class GroupByContainerCount implements BalancingTaskNameGrouper {
   private void saveTaskAssignments(Set<ContainerModel> containers, TaskAssignmentManager taskAssignmentManager) {
     for (ContainerModel container : containers) {
       for (TaskName taskName : container.getTasks().keySet()) {
-        taskAssignmentManager.writeTaskContainerMapping(taskName.getTaskName(), container.getProcessorId());
+        taskAssignmentManager.writeTaskContainerMapping(taskName.getTaskName(), container.getId());
       }
     }
   }
@@ -301,7 +301,7 @@ public class GroupByContainerCount implements BalancingTaskNameGrouper {
         containerTaskModels.put(model.getTaskName(), model);
       }
       containerModels.add(
-          new ContainerModel(container.containerId, Integer.valueOf(container.containerId), containerTaskModels));
+          new ContainerModel(container.containerId, containerTaskModels));
     }
     return Collections.unmodifiableSet(containerModels);
   }

http://git-wip-us.apache.org/repos/asf/samza/blob/3bb24c8e/samza-core/src/main/java/org/apache/samza/container/grouper/task/GroupByContainerIds.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/container/grouper/task/GroupByContainerIds.java b/samza-core/src/main/java/org/apache/samza/container/grouper/task/GroupByContainerIds.java
index f5a5a86..9dab943 100644
--- a/samza-core/src/main/java/org/apache/samza/container/grouper/task/GroupByContainerIds.java
+++ b/samza-core/src/main/java/org/apache/samza/container/grouper/task/GroupByContainerIds.java
@@ -100,9 +100,7 @@ public class GroupByContainerIds implements TaskNameGrouper {
     // Convert to a Set of ContainerModel
     Set<ContainerModel> containerModels = new HashSet<>();
     for (int i = 0; i < containerCount; i++) {
-      // containerId in ContainerModel constructor is set to -1 because processorId can be any string and does
-      // not have an integer equivalent. So, we set it to -1. After 0.13, this parameter will be removed.
-      containerModels.add(new ContainerModel(containersIds.get(i), -1, taskGroups[i]));
+      containerModels.add(new ContainerModel(containersIds.get(i), taskGroups[i]));
     }
 
     return Collections.unmodifiableSet(containerModels);

http://git-wip-us.apache.org/repos/asf/samza/blob/3bb24c8e/samza-core/src/main/java/org/apache/samza/container/grouper/task/SingleContainerGrouperFactory.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/container/grouper/task/SingleContainerGrouperFactory.java b/samza-core/src/main/java/org/apache/samza/container/grouper/task/SingleContainerGrouperFactory.java
index 15cd224..ee962d5 100644
--- a/samza-core/src/main/java/org/apache/samza/container/grouper/task/SingleContainerGrouperFactory.java
+++ b/samza-core/src/main/java/org/apache/samza/container/grouper/task/SingleContainerGrouperFactory.java
@@ -50,7 +50,7 @@ class SingleContainerGrouper implements TaskNameGrouper {
     for (TaskModel taskModel: taskModels) {
       taskNameTaskModelMap.put(taskModel.getTaskName(), taskModel);
     }
-    ContainerModel containerModel = new ContainerModel(containerId, -1, taskNameTaskModelMap);
+    ContainerModel containerModel = new ContainerModel(containerId, taskNameTaskModelMap);
     return Collections.singleton(containerModel);
   }
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/3bb24c8e/samza-core/src/main/java/org/apache/samza/job/model/ContainerModel.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/job/model/ContainerModel.java b/samza-core/src/main/java/org/apache/samza/job/model/ContainerModel.java
index bd4fa94..980806b 100644
--- a/samza-core/src/main/java/org/apache/samza/job/model/ContainerModel.java
+++ b/samza-core/src/main/java/org/apache/samza/job/model/ContainerModel.java
@@ -19,10 +19,9 @@
 
 package org.apache.samza.job.model;
 
-import org.apache.samza.container.TaskName;
-
 import java.util.Collections;
 import java.util.Map;
+import org.apache.samza.container.TaskName;
 
 /**
  * <p>
@@ -41,28 +40,16 @@ import java.util.Map;
  * </p>
  */
 public class ContainerModel {
-  @Deprecated
-  private final int containerId;
-  private final String processorId;
+  private final String id;
   private final Map<TaskName, TaskModel> tasks;
 
-  public ContainerModel(String processorId, int containerId, Map<TaskName, TaskModel> tasks) {
-    this.containerId = containerId;
-    if (processorId == null) {
-      this.processorId = String.valueOf(containerId);
-    } else {
-      this.processorId = processorId;
-    }
+  public ContainerModel(String id, Map<TaskName, TaskModel> tasks) {
+    this.id = id;
     this.tasks = Collections.unmodifiableMap(tasks);
   }
 
-  @Deprecated
-  public int getContainerId() {
-    return containerId;
-  }
-
-  public String getProcessorId() {
-    return processorId;
+  public String getId() {
+    return id;
   }
 
   public Map<TaskName, TaskModel> getTasks() {
@@ -71,14 +58,14 @@ public class ContainerModel {
 
   @Override
   public String toString() {
-    return "ContainerModel [processorId=" + processorId + ", tasks=" + tasks + "]";
+    return "ContainerModel [id=" + id + ", tasks=" + tasks + "]";
   }
 
   @Override
   public int hashCode() {
     final int prime = 31;
     int result = 1;
-    result = prime * result + ((processorId == null) ? 0 : processorId.hashCode());
+    result = prime * result + ((id == null) ? 0 : id.hashCode());
     result = prime * result + ((tasks == null) ? 0 : tasks.hashCode());
     return result;
   }
@@ -92,7 +79,7 @@ public class ContainerModel {
     if (getClass() != obj.getClass())
       return false;
     ContainerModel other = (ContainerModel) obj;
-    if (!processorId.equals(other.processorId))
+    if (!id.equals(other.id))
       return false;
     if (tasks == null) {
       if (other.tasks != null)

http://git-wip-us.apache.org/repos/asf/samza/blob/3bb24c8e/samza-core/src/main/java/org/apache/samza/job/model/JobModel.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/job/model/JobModel.java b/samza-core/src/main/java/org/apache/samza/job/model/JobModel.java
index d2f8fda..be26f10 100644
--- a/samza-core/src/main/java/org/apache/samza/job/model/JobModel.java
+++ b/samza-core/src/main/java/org/apache/samza/job/model/JobModel.java
@@ -25,7 +25,6 @@ import java.util.Map;
 import org.apache.samza.config.Config;
 import org.apache.samza.container.LocalityManager;
 import org.apache.samza.coordinator.stream.messages.SetContainerHostMapping;
-import org.codehaus.jackson.annotate.JsonIgnoreProperties;
 
 /**
  * <p>
@@ -39,7 +38,6 @@ import org.codehaus.jackson.annotate.JsonIgnoreProperties;
  * an id, partition information, etc.
  * </p>
  */
-@JsonIgnoreProperties(ignoreUnknown = true)
 public class JobModel {
   private static final String EMPTY_STRING = "";
   private final Config config;

http://git-wip-us.apache.org/repos/asf/samza/blob/3bb24c8e/samza-core/src/main/java/org/apache/samza/serializers/model/JsonContainerModelMixIn.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/serializers/model/JsonContainerModelMixIn.java b/samza-core/src/main/java/org/apache/samza/serializers/model/JsonContainerModelMixIn.java
index e19afec..f18c42a 100644
--- a/samza-core/src/main/java/org/apache/samza/serializers/model/JsonContainerModelMixIn.java
+++ b/samza-core/src/main/java/org/apache/samza/serializers/model/JsonContainerModelMixIn.java
@@ -19,29 +19,35 @@
 
 package org.apache.samza.serializers.model;
 
+import java.util.Map;
 import org.apache.samza.container.TaskName;
 import org.apache.samza.job.model.TaskModel;
-import org.codehaus.jackson.annotate.JsonCreator;
 import org.codehaus.jackson.annotate.JsonProperty;
 
-import java.util.Map;
-
 /**
- * A mix-in Jackson class to convert Samza's ContainerModel to/from JSON.
+ * A mix-in Jackson class to convert {@link org.apache.samza.job.model.ContainerModel} to JSON.
+ * Notes:
+ * 1) Constructor is not needed because this mixin is not used for deserialization. See {@link SamzaObjectMapper}.
+ * 2) It is unnecessary to use {@link org.codehaus.jackson.annotate.JsonIgnoreProperties#ignoreUnknown()} here since
+ * {@link SamzaObjectMapper} already uses custom deserialization code for the
+ * {@link org.apache.samza.job.model.ContainerModel}.
+ * 3) See {@link SamzaObjectMapper} for more context about why the JSON keys are named in this specified way.
  */
 public abstract class JsonContainerModelMixIn {
-  @JsonCreator
-  public JsonContainerModelMixIn(@JsonProperty("processor-id") String processorId, @JsonProperty("tasks") Map<TaskName, TaskModel> tasks) {
-  }
-
-  @Deprecated
-  @JsonProperty("container-id")
-  abstract int getContainerId();
-
-  @JsonProperty("processor-id")
-  abstract String getProcessorId();
-
-  @JsonProperty("tasks")
+  /**
+   * This is intentionally not "id" for backwards compatibility reasons. See {@link SamzaObjectMapper} for more details.
+   */
+  static final String PROCESSOR_ID_KEY = "processor-id";
+  /**
+   * This is used for backwards compatibility. See {@link SamzaObjectMapper} for more details.
+   */
+  static final String CONTAINER_ID_KEY = "container-id";
+  static final String TASKS_KEY = "tasks";
+
+  @JsonProperty(PROCESSOR_ID_KEY)
+  abstract String getId();
+
+  @JsonProperty(TASKS_KEY)
   abstract Map<TaskName, TaskModel> getTasks();
 }
 

http://git-wip-us.apache.org/repos/asf/samza/blob/3bb24c8e/samza-core/src/main/java/org/apache/samza/serializers/model/JsonJobModelMixIn.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/serializers/model/JsonJobModelMixIn.java b/samza-core/src/main/java/org/apache/samza/serializers/model/JsonJobModelMixIn.java
index 4b0c404..c40173a 100644
--- a/samza-core/src/main/java/org/apache/samza/serializers/model/JsonJobModelMixIn.java
+++ b/samza-core/src/main/java/org/apache/samza/serializers/model/JsonJobModelMixIn.java
@@ -23,11 +23,13 @@ import java.util.Map;
 import org.apache.samza.config.Config;
 import org.apache.samza.job.model.ContainerModel;
 import org.codehaus.jackson.annotate.JsonCreator;
+import org.codehaus.jackson.annotate.JsonIgnoreProperties;
 import org.codehaus.jackson.annotate.JsonProperty;
 
 /**
  * A mix-in Jackson class to convert Samza's JobModel to/from JSON.
  */
+@JsonIgnoreProperties(ignoreUnknown = true)
 public abstract class JsonJobModelMixIn {
   @JsonCreator
   public JsonJobModelMixIn(@JsonProperty("config") Config config, @JsonProperty("containers") Map<String, ContainerModel> containers) {

http://git-wip-us.apache.org/repos/asf/samza/blob/3bb24c8e/samza-core/src/main/java/org/apache/samza/serializers/model/JsonTaskModelMixIn.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/serializers/model/JsonTaskModelMixIn.java b/samza-core/src/main/java/org/apache/samza/serializers/model/JsonTaskModelMixIn.java
index 3ebe391..13a7d59 100644
--- a/samza-core/src/main/java/org/apache/samza/serializers/model/JsonTaskModelMixIn.java
+++ b/samza-core/src/main/java/org/apache/samza/serializers/model/JsonTaskModelMixIn.java
@@ -25,11 +25,13 @@ import org.apache.samza.Partition;
 import org.apache.samza.container.TaskName;
 import org.apache.samza.system.SystemStreamPartition;
 import org.codehaus.jackson.annotate.JsonCreator;
+import org.codehaus.jackson.annotate.JsonIgnoreProperties;
 import org.codehaus.jackson.annotate.JsonProperty;
 
 /**
  * A mix-in Jackson class to convert Samza's TaskModel to/from JSON.
  */
+@JsonIgnoreProperties(ignoreUnknown = true)
 public abstract class JsonTaskModelMixIn {
   @JsonCreator
   public JsonTaskModelMixIn(@JsonProperty("task-name") TaskName taskName, @JsonProperty("system-stream-partitions") Set<SystemStreamPartition> systemStreamPartitions, @JsonProperty("changelog-partition") Partition changelogPartition) {

http://git-wip-us.apache.org/repos/asf/samza/blob/3bb24c8e/samza-core/src/main/java/org/apache/samza/serializers/model/SamzaObjectMapper.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/serializers/model/SamzaObjectMapper.java b/samza-core/src/main/java/org/apache/samza/serializers/model/SamzaObjectMapper.java
index 53b59b2..15206e1 100644
--- a/samza-core/src/main/java/org/apache/samza/serializers/model/SamzaObjectMapper.java
+++ b/samza-core/src/main/java/org/apache/samza/serializers/model/SamzaObjectMapper.java
@@ -19,6 +19,9 @@
 
 package org.apache.samza.serializers.model;
 
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
 import org.apache.samza.Partition;
 import org.apache.samza.SamzaException;
 import org.apache.samza.config.Config;
@@ -48,10 +51,6 @@ import org.codehaus.jackson.map.introspect.AnnotatedMethod;
 import org.codehaus.jackson.map.module.SimpleModule;
 import org.codehaus.jackson.type.TypeReference;
 
-import java.io.IOException;
-import java.util.HashMap;
-import java.util.Map;
-
 /**
  * <p>
  * A collection of utility classes and (de)serializers to make Samza's job model
@@ -59,7 +58,7 @@ import java.util.Map;
  * Jackson-specific code is isolated so that Samza's core data model does not
  * require a direct dependency on Jackson.
  * </p>
- * 
+ *
  * <p>
  * To use Samza's job data model, use the SamzaObjectMapper.getObjectMapper()
  * method.
@@ -99,19 +98,31 @@ public class SamzaObjectMapper {
       public ContainerModel deserialize(JsonParser jp, DeserializationContext ctxt) throws IOException, JsonProcessingException {
         ObjectCodec oc = jp.getCodec();
         JsonNode node = oc.readTree(jp);
-        int containerId = node.get("container-id").getIntValue();
-        if (node.get("container-id") == null) {
-          throw new SamzaException("JobModel did not contain a container-id. This can never happen. JobModel corrupt!");
-        }
-        String processorId;
-        if (node.get("processor-id") == null) {
-          processorId = String.valueOf(containerId);
+        /*
+         * Before Samza 0.13, "container-id" was used.
+         * In Samza 0.13, "processor-id" was added to be the id to use and "container-id" was deprecated. However,
+         * "container-id" still needed to be checked for backwards compatibility in case "processor-id" was missing
+         * (i.e. from a job model corresponding to a version of the job that was on a pre Samza 0.13 version).
+         * In Samza 1.0, "container-id" was further cleaned up from ContainerModel. This logic is still being left here
+         * as a fallback for backwards compatibility with pre Samza 0.13. ContainerModel.getProcessorId was changed to
+         * ContainerModel.getId in the Java API, but "processor-id" still needs to be used as the JSON key for backwards
+         * compatibility with Samza 0.13 and Samza 0.14.
+         */
+        String id;
+        if (node.get(JsonContainerModelMixIn.PROCESSOR_ID_KEY) == null) {
+          if (node.get(JsonContainerModelMixIn.CONTAINER_ID_KEY) == null) {
+            throw new SamzaException(
+                String.format("JobModel was missing %s and %s. This should never happen. JobModel corrupt!",
+                    JsonContainerModelMixIn.PROCESSOR_ID_KEY, JsonContainerModelMixIn.CONTAINER_ID_KEY));
+          }
+          id = String.valueOf(node.get(JsonContainerModelMixIn.CONTAINER_ID_KEY).getIntValue());
         } else {
-          processorId = node.get("processor-id").getTextValue();
+          id = node.get(JsonContainerModelMixIn.PROCESSOR_ID_KEY).getTextValue();
         }
         Map<TaskName, TaskModel> tasksMapping =
-            OBJECT_MAPPER.readValue(node.get("tasks"), new TypeReference<Map<TaskName, TaskModel>>() { });
-        return new ContainerModel(processorId, containerId, tasksMapping);
+            OBJECT_MAPPER.readValue(node.get(JsonContainerModelMixIn.TASKS_KEY),
+                new TypeReference<Map<TaskName, TaskModel>>() { });
+        return new ContainerModel(id, tasksMapping);
       }
     });
 

http://git-wip-us.apache.org/repos/asf/samza/blob/3bb24c8e/samza-core/src/main/java/org/apache/samza/storage/StorageRecovery.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/storage/StorageRecovery.java b/samza-core/src/main/java/org/apache/samza/storage/StorageRecovery.java
index f9c6c0c..bf46018 100644
--- a/samza-core/src/main/java/org/apache/samza/storage/StorageRecovery.java
+++ b/samza-core/src/main/java/org/apache/samza/storage/StorageRecovery.java
@@ -209,7 +209,7 @@ public class StorageRecovery extends CommandLine {
 
     for (ContainerModel containerModel : containers.values()) {
       HashMap<String, StorageEngine> taskStores = new HashMap<String, StorageEngine>();
-      SamzaContainerContext containerContext = new SamzaContainerContext(containerModel.getProcessorId(), jobConfig, containerModel.getTasks()
+      SamzaContainerContext containerContext = new SamzaContainerContext(containerModel.getId(), jobConfig, containerModel.getTasks()
           .keySet(), new MetricsRegistryMap());
 
       for (TaskModel taskModel : containerModel.getTasks().values()) {

http://git-wip-us.apache.org/repos/asf/samza/blob/3bb24c8e/samza-core/src/main/scala/org/apache/samza/coordinator/JobModelManager.scala
----------------------------------------------------------------------
diff --git a/samza-core/src/main/scala/org/apache/samza/coordinator/JobModelManager.scala b/samza-core/src/main/scala/org/apache/samza/coordinator/JobModelManager.scala
index f939736..f7698c0 100644
--- a/samza-core/src/main/scala/org/apache/samza/coordinator/JobModelManager.scala
+++ b/samza-core/src/main/scala/org/apache/samza/coordinator/JobModelManager.scala
@@ -208,7 +208,7 @@ object JobModelManager extends Logging {
         case _ => containerGrouper.group(taskModels.asJava, containerIds)
       }
     }
-    val containerMap = containerModels.asScala.map { case (containerModel) => containerModel.getProcessorId -> containerModel }.toMap
+    val containerMap = containerModels.asScala.map { case (containerModel) => containerModel.getId -> containerModel }.toMap
 
     if (isHostAffinityEnabled) {
       new JobModel(config, containerMap.asJava, localityManager)

http://git-wip-us.apache.org/repos/asf/samza/blob/3bb24c8e/samza-core/src/test/java/org/apache/samza/container/grouper/task/TestGroupByContainerCount.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/container/grouper/task/TestGroupByContainerCount.java b/samza-core/src/test/java/org/apache/samza/container/grouper/task/TestGroupByContainerCount.java
index e89d673..8d2d394 100644
--- a/samza-core/src/test/java/org/apache/samza/container/grouper/task/TestGroupByContainerCount.java
+++ b/samza-core/src/test/java/org/apache/samza/container/grouper/task/TestGroupByContainerCount.java
@@ -18,33 +18,22 @@
  */
 package org.apache.samza.container.grouper.task;
 
-import org.apache.samza.SamzaException;
-import org.apache.samza.container.LocalityManager;
-import org.apache.samza.job.model.ContainerModel;
-import org.apache.samza.job.model.TaskModel;
-import org.junit.Before;
-import org.junit.Test;
-
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Map;
 import java.util.Set;
 import java.util.UUID;
+import org.apache.samza.SamzaException;
+import org.apache.samza.container.LocalityManager;
+import org.apache.samza.job.model.ContainerModel;
+import org.apache.samza.job.model.TaskModel;
+import org.junit.Before;
+import org.junit.Test;
 
-import static org.apache.samza.container.mock.ContainerMocks.generateTaskContainerMapping;
-import static org.apache.samza.container.mock.ContainerMocks.generateTaskModels;
-import static org.apache.samza.container.mock.ContainerMocks.getTaskModel;
-import static org.apache.samza.container.mock.ContainerMocks.getTaskName;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
-import static org.mockito.Mockito.anyCollection;
-import static org.mockito.Mockito.anyString;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.never;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.when;
+import static org.apache.samza.container.mock.ContainerMocks.*;
+import static org.junit.Assert.*;
+import static org.mockito.Mockito.*;
 
 public class TestGroupByContainerCount {
   private TaskAssignmentManager taskAssignmentManager;
@@ -83,7 +72,7 @@ public class TestGroupByContainerCount {
 
     Map<String, ContainerModel> containersMap = new HashMap<>();
     for (ContainerModel container : containers) {
-      containersMap.put(container.getProcessorId(), container);
+      containersMap.put(container.getId(), container);
     }
 
     assertEquals(2, containers.size());
@@ -91,8 +80,8 @@ public class TestGroupByContainerCount {
     ContainerModel container1 = containersMap.get("1");
     assertNotNull(container0);
     assertNotNull(container1);
-    assertEquals("0", container0.getProcessorId());
-    assertEquals("1", container1.getProcessorId());
+    assertEquals("0", container0.getId());
+    assertEquals("1", container1.getId());
     assertEquals(3, container0.getTasks().size());
     assertEquals(2, container1.getTasks().size());
     assertTrue(container0.getTasks().containsKey(getTaskName(0)));
@@ -110,7 +99,7 @@ public class TestGroupByContainerCount {
 
     Map<String, ContainerModel> containersMap = new HashMap<>();
     for (ContainerModel container : containers) {
-      containersMap.put(container.getProcessorId(), container);
+      containersMap.put(container.getId(), container);
     }
 
     assertEquals(2, containers.size());
@@ -118,8 +107,8 @@ public class TestGroupByContainerCount {
     ContainerModel container1 = containersMap.get("1");
     assertNotNull(container0);
     assertNotNull(container1);
-    assertEquals("0", container0.getProcessorId());
-    assertEquals("1", container1.getProcessorId());
+    assertEquals("0", container0.getId());
+    assertEquals("1", container1.getId());
     assertEquals(11, container0.getTasks().size());
     assertEquals(10, container1.getTasks().size());
 
@@ -182,7 +171,7 @@ public class TestGroupByContainerCount {
 
     Map<String, ContainerModel> containersMap = new HashMap<>();
     for (ContainerModel container : containers) {
-      containersMap.put(container.getProcessorId(), container);
+      containersMap.put(container.getId(), container);
     }
 
     assertEquals(4, containers.size());
@@ -194,8 +183,8 @@ public class TestGroupByContainerCount {
     assertNotNull(container1);
     assertNotNull(container2);
     assertNotNull(container3);
-    assertEquals("0", container0.getProcessorId());
-    assertEquals("1", container1.getProcessorId());
+    assertEquals("0", container0.getId());
+    assertEquals("1", container1.getId());
     assertEquals(3, container0.getTasks().size());
     assertEquals(2, container1.getTasks().size());
     assertEquals(2, container2.getTasks().size());
@@ -264,7 +253,7 @@ public class TestGroupByContainerCount {
 
     Map<String, ContainerModel> containersMap = new HashMap<>();
     for (ContainerModel container : containers) {
-      containersMap.put(container.getProcessorId(), container);
+      containersMap.put(container.getId(), container);
     }
 
     assertEquals(2, containers.size());
@@ -272,8 +261,8 @@ public class TestGroupByContainerCount {
     ContainerModel container1 = containersMap.get("1");
     assertNotNull(container0);
     assertNotNull(container1);
-    assertEquals("0", container0.getProcessorId());
-    assertEquals("1", container1.getProcessorId());
+    assertEquals("0", container0.getId());
+    assertEquals("1", container1.getId());
     assertEquals(5, container0.getTasks().size());
     assertEquals(4, container1.getTasks().size());
 
@@ -343,7 +332,7 @@ public class TestGroupByContainerCount {
 
     Map<String, ContainerModel> containersMap = new HashMap<>();
     for (ContainerModel container : containers) {
-      containersMap.put(container.getProcessorId(), container);
+      containersMap.put(container.getId(), container);
     }
 
     assertEquals(2, containers.size());
@@ -351,8 +340,8 @@ public class TestGroupByContainerCount {
     ContainerModel container1 = containersMap.get("1");
     assertNotNull(container0);
     assertNotNull(container1);
-    assertEquals("0", container0.getProcessorId());
-    assertEquals("1", container1.getProcessorId());
+    assertEquals("0", container0.getId());
+    assertEquals("1", container1.getId());
     assertEquals(5, container0.getTasks().size());
     assertEquals(4, container1.getTasks().size());
 
@@ -397,7 +386,7 @@ public class TestGroupByContainerCount {
 
     containersMap = new HashMap<>();
     for (ContainerModel container : containers) {
-      containersMap.put(container.getProcessorId(), container);
+      containersMap.put(container.getId(), container);
     }
 
     assertEquals(3, containers.size());
@@ -407,9 +396,9 @@ public class TestGroupByContainerCount {
     assertNotNull(container0);
     assertNotNull(container1);
     assertNotNull(container2);
-    assertEquals("0", container0.getProcessorId());
-    assertEquals("1", container1.getProcessorId());
-    assertEquals("2", container2.getProcessorId());
+    assertEquals("0", container0.getId());
+    assertEquals("1", container1.getId());
+    assertEquals("2", container2.getId());
     assertEquals(3, container0.getTasks().size());
     assertEquals(3, container1.getTasks().size());
     assertEquals(3, container2.getTasks().size());
@@ -474,7 +463,7 @@ public class TestGroupByContainerCount {
 
     Map<String, ContainerModel> containersMap = new HashMap<>();
     for (ContainerModel container : containers) {
-      containersMap.put(container.getProcessorId(), container);
+      containersMap.put(container.getId(), container);
     }
 
     assertEquals(2, containers.size());
@@ -482,8 +471,8 @@ public class TestGroupByContainerCount {
     ContainerModel container1 = containersMap.get("1");
     assertNotNull(container0);
     assertNotNull(container1);
-    assertEquals("0", container0.getProcessorId());
-    assertEquals("1", container1.getProcessorId());
+    assertEquals("0", container0.getId());
+    assertEquals("1", container1.getId());
     assertEquals(5, container0.getTasks().size());
     assertEquals(4, container1.getTasks().size());
 
@@ -544,7 +533,7 @@ public class TestGroupByContainerCount {
 
     Map<String, ContainerModel> containersMap = new HashMap<>();
     for (ContainerModel container : containers) {
-      containersMap.put(container.getProcessorId(), container);
+      containersMap.put(container.getId(), container);
     }
 
     assertEquals(2, containers.size());
@@ -552,8 +541,8 @@ public class TestGroupByContainerCount {
     ContainerModel container1 = containersMap.get("1");
     assertNotNull(container0);
     assertNotNull(container1);
-    assertEquals("0", container0.getProcessorId());
-    assertEquals("1", container1.getProcessorId());
+    assertEquals("0", container0.getId());
+    assertEquals("1", container1.getId());
     assertEquals(6, container0.getTasks().size());
     assertEquals(3, container1.getTasks().size());
 
@@ -610,7 +599,7 @@ public class TestGroupByContainerCount {
 
     Map<String, ContainerModel> containersMap = new HashMap<>();
     for (ContainerModel container : containers) {
-      containersMap.put(container.getProcessorId(), container);
+      containersMap.put(container.getId(), container);
     }
 
     assertEquals(3, containers.size());
@@ -620,9 +609,9 @@ public class TestGroupByContainerCount {
     assertNotNull(container0);
     assertNotNull(container1);
     assertNotNull(container2);
-    assertEquals("0", container0.getProcessorId());
-    assertEquals("1", container1.getProcessorId());
-    assertEquals("2", container2.getProcessorId());
+    assertEquals("0", container0.getId());
+    assertEquals("1", container1.getId());
+    assertEquals("2", container2.getId());
     assertEquals(2, container0.getTasks().size());
     assertEquals(2, container1.getTasks().size());
     assertEquals(2, container1.getTasks().size());
@@ -782,7 +771,7 @@ public class TestGroupByContainerCount {
     Set<ContainerModel> prevContainers = new HashSet<>();
     taskModels.forEach(model -> {
         prevContainers.add(
-          new ContainerModel(UUID.randomUUID().toString(), -1, Collections.singletonMap(model.getTaskName(), model)));
+          new ContainerModel(UUID.randomUUID().toString(), Collections.singletonMap(model.getTaskName(), model)));
       });
     Map<String, String> prevTaskToContainerMapping = generateTaskContainerMapping(prevContainers);
     when(taskAssignmentManager.readTaskAssignment()).thenReturn(prevTaskToContainerMapping);

http://git-wip-us.apache.org/repos/asf/samza/blob/3bb24c8e/samza-core/src/test/java/org/apache/samza/container/grouper/task/TestGroupByContainerIds.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/container/grouper/task/TestGroupByContainerIds.java b/samza-core/src/test/java/org/apache/samza/container/grouper/task/TestGroupByContainerIds.java
index 13afeef..b9fe6fb 100644
--- a/samza-core/src/test/java/org/apache/samza/container/grouper/task/TestGroupByContainerIds.java
+++ b/samza-core/src/test/java/org/apache/samza/container/grouper/task/TestGroupByContainerIds.java
@@ -38,13 +38,9 @@ import org.apache.samza.job.model.TaskModel;
 import org.junit.Before;
 import org.junit.Test;
 
-import static org.apache.samza.container.mock.ContainerMocks.generateTaskModels;
-import static org.apache.samza.container.mock.ContainerMocks.getTaskName;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
+import static org.apache.samza.container.mock.ContainerMocks.*;
+import static org.junit.Assert.*;
+import static org.mockito.Mockito.*;
 
 
 public class TestGroupByContainerIds {
@@ -91,7 +87,7 @@ public class TestGroupByContainerIds {
 
     Map<String, ContainerModel> containersMap = new HashMap<>();
     for (ContainerModel container : containers) {
-      containersMap.put(container.getProcessorId(), container);
+      containersMap.put(container.getId(), container);
     }
 
     assertEquals(2, containers.size());
@@ -99,8 +95,8 @@ public class TestGroupByContainerIds {
     ContainerModel container1 = containersMap.get("1");
     assertNotNull(container0);
     assertNotNull(container1);
-    assertEquals("0", container0.getProcessorId());
-    assertEquals("1", container1.getProcessorId());
+    assertEquals("0", container0.getId());
+    assertEquals("1", container1.getId());
     assertEquals(3, container0.getTasks().size());
     assertEquals(2, container1.getTasks().size());
     assertTrue(container0.getTasks().containsKey(getTaskName(0)));
@@ -118,7 +114,7 @@ public class TestGroupByContainerIds {
 
     Map<String, ContainerModel> containersMap = new HashMap<>();
     for (ContainerModel container : containers) {
-      containersMap.put(container.getProcessorId(), container);
+      containersMap.put(container.getId(), container);
     }
 
     assertEquals(2, containers.size());
@@ -126,8 +122,8 @@ public class TestGroupByContainerIds {
     ContainerModel container1 = containersMap.get("1");
     assertNotNull(container0);
     assertNotNull(container1);
-    assertEquals("0", container0.getProcessorId());
-    assertEquals("1", container1.getProcessorId());
+    assertEquals("0", container0.getId());
+    assertEquals("1", container1.getId());
     assertEquals(3, container0.getTasks().size());
     assertEquals(2, container1.getTasks().size());
     assertTrue(container0.getTasks().containsKey(getTaskName(0)));
@@ -159,7 +155,7 @@ public class TestGroupByContainerIds {
 
     Map<String, ContainerModel> containersMap = new HashMap<>();
     for (ContainerModel container : containers) {
-      containersMap.put(container.getProcessorId(), container);
+      containersMap.put(container.getId(), container);
     }
 
     assertEquals(2, containers.size());
@@ -167,8 +163,8 @@ public class TestGroupByContainerIds {
     ContainerModel container1 = containersMap.get("2");
     assertNotNull(container0);
     assertNotNull(container1);
-    assertEquals("4", container0.getProcessorId());
-    assertEquals("2", container1.getProcessorId());
+    assertEquals("4", container0.getId());
+    assertEquals("2", container1.getId());
     assertEquals(3, container0.getTasks().size());
     assertEquals(2, container1.getTasks().size());
     assertTrue(container0.getTasks().containsKey(getTaskName(0)));
@@ -195,7 +191,7 @@ public class TestGroupByContainerIds {
 
     Map<String, ContainerModel> containersMap = new HashMap<>();
     for (ContainerModel container : containers) {
-      containersMap.put(container.getProcessorId(), container);
+      containersMap.put(container.getId(), container);
     }
 
     assertEquals(2, containers.size());
@@ -203,8 +199,8 @@ public class TestGroupByContainerIds {
     ContainerModel container1 = containersMap.get("2");
     assertNotNull(container0);
     assertNotNull(container1);
-    assertEquals("4", container0.getProcessorId());
-    assertEquals("2", container1.getProcessorId());
+    assertEquals("4", container0.getId());
+    assertEquals("2", container1.getId());
     assertEquals(11, container0.getTasks().size());
     assertEquals(10, container1.getTasks().size());
 
@@ -238,14 +234,13 @@ public class TestGroupByContainerIds {
   public void testFewerTasksThanContainers() {
     final String testContainerId1 = "1";
     final String testContainerId2 = "2";
-    final int testProcessorId = 1;
 
     Set<TaskModel> taskModels = generateTaskModels(1);
     List<String> containerIds = ImmutableList.of(testContainerId1, testContainerId2);
 
     Map<TaskName, TaskModel> expectedTasks = taskModels.stream()
                                                        .collect(Collectors.toMap(TaskModel::getTaskName, x -> x));
-    ContainerModel expectedContainerModel = new ContainerModel(testContainerId1, testProcessorId, expectedTasks);
+    ContainerModel expectedContainerModel = new ContainerModel(testContainerId1, expectedTasks);
 
     Set<ContainerModel> actualContainerModels = buildSimpleGrouper().group(taskModels, containerIds);
 

http://git-wip-us.apache.org/repos/asf/samza/blob/3bb24c8e/samza-core/src/test/java/org/apache/samza/container/mock/ContainerMocks.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/container/mock/ContainerMocks.java b/samza-core/src/test/java/org/apache/samza/container/mock/ContainerMocks.java
index 9369f4b..ca9def2 100644
--- a/samza-core/src/test/java/org/apache/samza/container/mock/ContainerMocks.java
+++ b/samza-core/src/test/java/org/apache/samza/container/mock/ContainerMocks.java
@@ -56,7 +56,7 @@ public class ContainerMocks {
     Set<ContainerModel> containers = generateContainerModels(numContainers, taskCount);
     for (ContainerModel container : containers) {
       for (TaskName taskName : container.getTasks().keySet()) {
-        mapping.put(taskName.getTaskName(), container.getProcessorId());
+        mapping.put(taskName.getTaskName(), container.getId());
       }
     }
     return mapping;
@@ -78,7 +78,7 @@ public class ContainerMocks {
     for (int partition : partitions) {
       tasks.put(getTaskName(partition), getTaskModel(partition));
     }
-    return new ContainerModel(containerId, -1, tasks);
+    return new ContainerModel(containerId, tasks);
   }
 
   public static Set<TaskModel> generateTaskModels(int[] partitions) {
@@ -121,7 +121,7 @@ public class ContainerMocks {
     Map<String, String> taskMapping = new HashMap<>();
     for (ContainerModel container : containers) {
       for (TaskName taskName : container.getTasks().keySet()) {
-        taskMapping.put(taskName.getTaskName(), container.getProcessorId());
+        taskMapping.put(taskName.getTaskName(), container.getId());
       }
     }
     return taskMapping;

http://git-wip-us.apache.org/repos/asf/samza/blob/3bb24c8e/samza-core/src/test/java/org/apache/samza/coordinator/JobModelManagerTestUtil.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/coordinator/JobModelManagerTestUtil.java b/samza-core/src/test/java/org/apache/samza/coordinator/JobModelManagerTestUtil.java
index b7514c4..ea25ec1 100644
--- a/samza-core/src/test/java/org/apache/samza/coordinator/JobModelManagerTestUtil.java
+++ b/samza-core/src/test/java/org/apache/samza/coordinator/JobModelManagerTestUtil.java
@@ -19,20 +19,17 @@
 
 package org.apache.samza.coordinator;
 
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
 import org.apache.samza.config.Config;
 import org.apache.samza.container.LocalityManager;
-import org.apache.samza.container.TaskName;
 import org.apache.samza.coordinator.server.HttpServer;
 import org.apache.samza.job.model.ContainerModel;
 import org.apache.samza.job.model.JobModel;
-import org.apache.samza.job.model.TaskModel;
 import org.apache.samza.system.StreamMetadataCache;
 
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
 /**
  * Utils to create instances of {@link JobModelManager} in unit tests
  */
@@ -45,7 +42,7 @@ public class JobModelManagerTestUtil {
   public static JobModelManager getJobModelManagerWithLocalityManager(Config config, int containerCount, LocalityManager localityManager, HttpServer server) {
     Map<String, ContainerModel> containers = new java.util.HashMap<>();
     for (int i = 0; i < containerCount; i++) {
-      ContainerModel container = new ContainerModel(String.valueOf(i), i, new HashMap<TaskName, TaskModel>());
+      ContainerModel container = new ContainerModel(String.valueOf(i), new HashMap<>());
       containers.put(String.valueOf(i), container);
     }
     JobModel jobModel = new JobModel(config, containers, localityManager);

http://git-wip-us.apache.org/repos/asf/samza/blob/3bb24c8e/samza-core/src/test/java/org/apache/samza/job/model/TestJobModel.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/job/model/TestJobModel.java b/samza-core/src/test/java/org/apache/samza/job/model/TestJobModel.java
index 6c7c282..77fe639 100644
--- a/samza-core/src/test/java/org/apache/samza/job/model/TestJobModel.java
+++ b/samza-core/src/test/java/org/apache/samza/job/model/TestJobModel.java
@@ -41,8 +41,8 @@ public class TestJobModel {
         new TaskName("t3"), new TaskModel(new TaskName("t3"), ImmutableSet.of(), new Partition(2)),
         new TaskName("t4"), new TaskModel(new TaskName("t4"), ImmutableSet.of(), new Partition(3)),
         new TaskName("t5"), new TaskModel(new TaskName("t5"), ImmutableSet.of(), new Partition(4)));
-    ContainerModel containerModel1 = new ContainerModel("0", 0, tasksForContainer1);
-    ContainerModel containerModel2 = new ContainerModel("1", 1, tasksForContainer2);
+    ContainerModel containerModel1 = new ContainerModel("0", tasksForContainer1);
+    ContainerModel containerModel2 = new ContainerModel("1", tasksForContainer2);
     Map<String, ContainerModel> containers = ImmutableMap.of("0", containerModel1, "1", containerModel2);
     JobModel jobModel = new JobModel(config, containers);
     assertEquals(jobModel.maxChangeLogStreamPartitions, 5);

http://git-wip-us.apache.org/repos/asf/samza/blob/3bb24c8e/samza-core/src/test/java/org/apache/samza/operators/impl/TestOperatorImplGraph.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/operators/impl/TestOperatorImplGraph.java b/samza-core/src/test/java/org/apache/samza/operators/impl/TestOperatorImplGraph.java
index 6f8a8bc..bd9b2d6 100644
--- a/samza-core/src/test/java/org/apache/samza/operators/impl/TestOperatorImplGraph.java
+++ b/samza-core/src/test/java/org/apache/samza/operators/impl/TestOperatorImplGraph.java
@@ -552,13 +552,13 @@ public class TestOperatorImplGraph {
     ssps.add(ssp0);
     ssps.add(ssp2);
     TaskModel tm0 = new TaskModel(task0, ssps, new Partition(0));
-    ContainerModel cm0 = new ContainerModel("c0", 0, Collections.singletonMap(task0, tm0));
+    ContainerModel cm0 = new ContainerModel("c0", Collections.singletonMap(task0, tm0));
     TaskModel tm1 = new TaskModel(task1, Collections.singleton(ssp1), new Partition(1));
-    ContainerModel cm1 = new ContainerModel("c1", 1, Collections.singletonMap(task1, tm1));
+    ContainerModel cm1 = new ContainerModel("c1", Collections.singletonMap(task1, tm1));
 
     Map<String, ContainerModel> cms = new HashMap<>();
-    cms.put(cm0.getProcessorId(), cm0);
-    cms.put(cm1.getProcessorId(), cm1);
+    cms.put(cm0.getId(), cm0);
+    cms.put(cm1.getId(), cm1);
 
     JobModel jobModel = new JobModel(config, cms, null);
     Multimap<SystemStream, String> streamToTasks = OperatorImplGraph.getStreamToConsumerTasks(jobModel);

http://git-wip-us.apache.org/repos/asf/samza/blob/3bb24c8e/samza-core/src/test/java/org/apache/samza/serializers/model/TestSamzaObjectMapper.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/serializers/model/TestSamzaObjectMapper.java b/samza-core/src/test/java/org/apache/samza/serializers/model/TestSamzaObjectMapper.java
index 02c3a9d..0f90dd5 100644
--- a/samza-core/src/test/java/org/apache/samza/serializers/model/TestSamzaObjectMapper.java
+++ b/samza-core/src/test/java/org/apache/samza/serializers/model/TestSamzaObjectMapper.java
@@ -19,8 +19,13 @@
 
 package org.apache.samza.serializers.model;
 
-import junit.framework.Assert;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableSet;
+import java.io.IOException;
+import java.util.Map;
+import java.util.Set;
 import org.apache.samza.Partition;
+import org.apache.samza.SamzaException;
 import org.apache.samza.config.Config;
 import org.apache.samza.config.MapConfig;
 import org.apache.samza.container.TaskName;
@@ -29,76 +34,186 @@ import org.apache.samza.job.model.JobModel;
 import org.apache.samza.job.model.TaskModel;
 import org.apache.samza.system.SystemStreamPartition;
 import org.codehaus.jackson.map.ObjectMapper;
+import org.codehaus.jackson.node.ArrayNode;
+import org.codehaus.jackson.node.ObjectNode;
 import org.junit.Before;
 import org.junit.Test;
 
-import java.io.IOException;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
+import static org.junit.Assert.*;
 
-import static org.junit.Assert.assertEquals;
 
 public class TestSamzaObjectMapper {
   private JobModel jobModel;
+  private ObjectMapper samzaObjectMapper;
 
   @Before
-  public void setup() throws IOException {
-    Map<String, String> configMap = new HashMap<String, String>();
-    Set<SystemStreamPartition> ssp = new HashSet<>();
-    configMap.put("a", "b");
-    Config config = new MapConfig(configMap);
+  public void setup() {
+    Config config = new MapConfig(ImmutableMap.of("a", "b"));
     TaskName taskName = new TaskName("test");
-    ssp.add(new SystemStreamPartition("foo", "bar", new Partition(1)));
-    TaskModel taskModel = new TaskModel(taskName, ssp, new Partition(2));
-    Map<TaskName, TaskModel> tasks = new HashMap<TaskName, TaskModel>();
-    tasks.put(taskName, taskModel);
-    ContainerModel containerModel = new ContainerModel("1", 1, tasks);
-    Map<String, ContainerModel> containerMap = new HashMap<String, ContainerModel>();
-    containerMap.put("1", containerModel);
-    jobModel = new JobModel(config, containerMap);
+    Set<SystemStreamPartition> ssps = ImmutableSet.of(new SystemStreamPartition("foo", "bar", new Partition(1)));
+    TaskModel taskModel = new TaskModel(taskName, ssps, new Partition(2));
+    Map<TaskName, TaskModel> tasks = ImmutableMap.of(taskName, taskModel);
+    ContainerModel containerModel = new ContainerModel("1", tasks);
+    Map<String, ContainerModel> containerMap = ImmutableMap.of("1", containerModel);
+    this.jobModel = new JobModel(config, containerMap);
+    this.samzaObjectMapper = SamzaObjectMapper.getObjectMapper();
   }
 
   @Test
-  public void testJsonTaskModel() throws Exception {
-    ObjectMapper mapper = SamzaObjectMapper.getObjectMapper();
+  public void testSerializeJobModel() throws IOException {
+    String serializedString = this.samzaObjectMapper.writeValueAsString(this.jobModel);
+    // use a plain ObjectMapper to read JSON to make comparison easier
+    ObjectNode serializedAsJson = (ObjectNode) new ObjectMapper().readTree(serializedString);
+    ObjectNode expectedJson = buildJobModelJson();
+
+    /*
+     * Jackson serializes all get* methods even if they aren't regular getters. We only care about certain fields now
+     * since those are the only ones that get deserialized.
+     */
+    assertEquals(expectedJson.get("config"), serializedAsJson.get("config"));
+    assertEquals(expectedJson.get("containers"), serializedAsJson.get("containers"));
+  }
+
+  @Test
+  public void testDeserializeJobModel() throws IOException {
+    ObjectNode asJson = buildJobModelJson();
+    assertEquals(this.jobModel, deserializeFromObjectNode(asJson));
+  }
 
-    String str = mapper.writeValueAsString(jobModel);
-    JobModel obj = mapper.readValue(str, JobModel.class);
-    assertEquals(jobModel, obj);
+  /**
+   * Deserialization should not fail if there are fields which are ignored.
+   */
+  @Test
+  public void testDeserializeWithIgnoredFields() throws IOException {
+    ObjectNode jobModelJson = buildJobModelJson();
+    // JobModel ignores all unknown fields
+    jobModelJson.put("unknown_job_model_key", "unknown_job_model_value");
+    ObjectNode taskPartitionMappings = new ObjectMapper().createObjectNode();
+    taskPartitionMappings.put("1", (Integer) null);
+    // old key that used to be serialized
+    jobModelJson.put("task-partition-mappings", taskPartitionMappings);
+    ObjectNode allContainerLocality = new ObjectMapper().createObjectNode();
+    allContainerLocality.put("1", (Integer) null);
+    // currently gets serialized since there is a getAllContainerLocality
+    jobModelJson.put("all-container-locality", allContainerLocality);
+    assertEquals(this.jobModel, deserializeFromObjectNode(jobModelJson));
   }
 
   /**
-   * Critical test to guarantee compatibility between samza 0.12 container models and 0.13+
-   *
-   * Samza 0.12 contains only "container-id" (integer) in the ContainerModel. "processor-id" (String) is added in 0.13.
-   * When serializing, we serialize both the fields in 0.13. Deserialization correctly handles the fields in 0.13.
+   * Given a {@link ContainerModel} JSON with a processor-id and a container-id, deserialization should properly ignore
+   * the container-id.
    */
   @Test
-  public void testContainerModelCompatible() throws Exception {
-    String newJobModelString = "{\"config\":{\"a\":\"b\"},\"containers\":{\"1\":{\"processor-id\":\"1\",\"container-id\":1,\"tasks\":{\"test\":{\"task-name\":\"test\",\"system-stream-partitions\":[{\"system\":\"foo\",\"partition\":1,\"stream\":\"bar\"}],\"changelog-partition\":2}}}},\"max-change-log-stream-partitions\":3,\"all-container-locality\":{\"1\":null}}";
-    ObjectMapper mapper = SamzaObjectMapper.getObjectMapper();
-    JobModel jobModel = mapper.readValue(newJobModelString, JobModel.class);
+  public void testDeserializeContainerIdAndProcessorId() throws IOException {
+    ObjectNode jobModelJson = buildJobModelJson();
+    ObjectNode containerModelJson = (ObjectNode) jobModelJson.get("containers").get("1");
+    containerModelJson.put("container-id", 123);
+    assertEquals(this.jobModel, deserializeFromObjectNode(jobModelJson));
+  }
 
-    String oldJobModelString = "{\"config\":{\"a\":\"b\"},\"containers\":{\"1\":{\"container-id\":1,\"tasks\":{\"test\":{\"task-name\":\"test\",\"system-stream-partitions\":[{\"system\":\"foo\",\"partition\":1,\"stream\":\"bar\"}],\"changelog-partition\":2}}}},\"max-change-log-stream-partitions\":3,\"all-container-locality\":{\"1\":null}}";
-    ObjectMapper mapper1 = SamzaObjectMapper.getObjectMapper();
-    JobModel jobModel1 = mapper1.readValue(oldJobModelString, JobModel.class);
+  /**
+   * Given a {@link ContainerModel} JSON with an unknown field, deserialization should properly ignore it.
+   */
+  @Test
+  public void testDeserializeUnknownContainerModelField() throws IOException {
+    ObjectNode jobModelJson = buildJobModelJson();
+    ObjectNode containerModelJson = (ObjectNode) jobModelJson.get("containers").get("1");
+    containerModelJson.put("unknown_container_model_key", "unknown_container_model_value");
+    assertEquals(this.jobModel, deserializeFromObjectNode(jobModelJson));
+  }
 
-    Assert.assertEquals(jobModel, jobModel1);
+  /**
+   * Given a {@link ContainerModel} JSON without a processor-id but with a container-id, deserialization should use the
+   * container-id to calculate the processor-id.
+   */
+  @Test
+  public void testDeserializeContainerModelOnlyContainerId() throws IOException {
+    ObjectNode jobModelJson = buildJobModelJson();
+    ObjectNode containerModelJson = (ObjectNode) jobModelJson.get("containers").get("1");
+    containerModelJson.remove("processor-id");
+    containerModelJson.put("container-id", 1);
+    assertEquals(this.jobModel, deserializeFromObjectNode(jobModelJson));
   }
 
+  /**
+   * Given a {@link ContainerModel} JSON with an unknown field, deserialization should properly ignore it.
+   */
   @Test
-  public void testUnknownFieldsInJobModelJsonDoesNotFailDeserialization() throws Exception {
-    String newJobModelString = "{\"config\":{\"a\":\"b\"},\"containers\":{\"1\":{\"processor-id\":\"1\",\"container-id\":1,\"tasks\":{\"test\":{\"task-name\":\"test\",\"system-stream-partitions\":[{\"system\":\"foo\",\"partition\":1,\"stream\":\"bar\"}],\"changelog-partition\":2}}}},\"max-change-log-stream-partitions\":3,\"all-container-locality\":{\"1\":null}, \"task-partition-mapping\":{\"1\":null}}";
-    ObjectMapper mapper = SamzaObjectMapper.getObjectMapper();
-    JobModel jobModel = mapper.readValue(newJobModelString, JobModel.class);
+  public void testDeserializeUnknownTaskModelField() throws IOException {
+    ObjectNode jobModelJson = buildJobModelJson();
+    ObjectNode taskModelJson = (ObjectNode) jobModelJson.get("containers").get("1").get("tasks").get("test");
+    taskModelJson.put("unknown_task_model_key", "unknown_task_model_value");
+    assertEquals(this.jobModel, deserializeFromObjectNode(jobModelJson));
+  }
 
-    String oldJobModelString = "{\"config\":{\"a\":\"b\"},\"containers\":{\"1\":{\"container-id\":1,\"tasks\":{\"test\":{\"task-name\":\"test\",\"system-stream-partitions\":[{\"system\":\"foo\",\"partition\":1,\"stream\":\"bar\"}],\"changelog-partition\":2}}}},\"max-change-log-stream-partitions\":3,\"all-container-locality\":{\"1\":null}}";
-    ObjectMapper mapper1 = SamzaObjectMapper.getObjectMapper();
-    JobModel jobModel1 = mapper1.readValue(oldJobModelString, JobModel.class);
+  /**
+   * Given a {@link ContainerModel} JSON with neither a processor-id nor a container-id, deserialization should fail.
+   */
+  @Test(expected = SamzaException.class)
+  public void testDeserializeContainerModelMissingProcessorIdAndContainerId() throws IOException {
+    ObjectNode jobModelJson = buildJobModelJson();
+    ObjectNode containerModelJson = (ObjectNode) jobModelJson.get("containers").get("1");
+    containerModelJson.remove("processor-id");
+    deserializeFromObjectNode(jobModelJson);
+  }
 
-    Assert.assertEquals(jobModel, jobModel1);
+  /**
+   * Given a {@link ContainerModel} JSON with only an "id" field, deserialization should fail.
+   * This verifies that even though {@link ContainerModel} has a getId method, the "id" field is not used, since
+   * "processor-id" is the field that is supposed to be used.
+   */
+  @Test(expected = SamzaException.class)
+  public void testDeserializeContainerModelIdFieldOnly() throws IOException {
+    ObjectNode jobModelJson = buildJobModelJson();
+    ObjectNode containerModelJson = (ObjectNode) jobModelJson.get("containers").get("1");
+    containerModelJson.remove("processor-id");
+    containerModelJson.put("id", 1);
+    deserializeFromObjectNode(jobModelJson);
   }
 
+  private JobModel deserializeFromObjectNode(ObjectNode jobModelJson) throws IOException {
+    // use plain ObjectMapper to get JSON string
+    String jsonString = new ObjectMapper().writeValueAsString(jobModelJson);
+    return this.samzaObjectMapper.readValue(jsonString, JobModel.class);
+  }
+
+  /**
+   * Builds {@link ObjectNode} which matches the {@link JobModel} built in setup.
+   */
+  private static ObjectNode buildJobModelJson() {
+    ObjectMapper objectMapper = new ObjectMapper();
+
+    ObjectNode configJson = objectMapper.createObjectNode();
+    configJson.put("a", "b");
+
+    ObjectNode containerModel1TaskTestSSPJson = objectMapper.createObjectNode();
+    containerModel1TaskTestSSPJson.put("system", "foo");
+    containerModel1TaskTestSSPJson.put("stream", "bar");
+    containerModel1TaskTestSSPJson.put("partition", 1);
+
+    ArrayNode containerModel1TaskTestSSPsJson = objectMapper.createArrayNode();
+    containerModel1TaskTestSSPsJson.add(containerModel1TaskTestSSPJson);
+
+    ObjectNode containerModel1TaskTestJson = objectMapper.createObjectNode();
+    containerModel1TaskTestJson.put("task-name", "test");
+    containerModel1TaskTestJson.put("system-stream-partitions", containerModel1TaskTestSSPsJson);
+    containerModel1TaskTestJson.put("changelog-partition", 2);
+
+    ObjectNode containerModel1TasksJson = objectMapper.createObjectNode();
+    containerModel1TasksJson.put("test", containerModel1TaskTestJson);
+
+    ObjectNode containerModel1Json = objectMapper.createObjectNode();
+    // important: needs to be "processor-id" for compatibility between Samza 0.14 and 1.0
+    containerModel1Json.put("processor-id", "1");
+    containerModel1Json.put("tasks", containerModel1TasksJson);
+
+    ObjectNode containersJson = objectMapper.createObjectNode();
+    containersJson.put("1", containerModel1Json);
+
+    ObjectNode jobModelJson = objectMapper.createObjectNode();
+    jobModelJson.put("config", configJson);
+    jobModelJson.put("containers", containersJson);
+
+    return jobModelJson;
+  }
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/3bb24c8e/samza-core/src/test/scala/org/apache/samza/container/TestSamzaContainer.scala
----------------------------------------------------------------------
diff --git a/samza-core/src/test/scala/org/apache/samza/container/TestSamzaContainer.scala b/samza-core/src/test/scala/org/apache/samza/container/TestSamzaContainer.scala
index 30ca8c1..57c0bf0 100644
--- a/samza-core/src/test/scala/org/apache/samza/container/TestSamzaContainer.scala
+++ b/samza-core/src/test/scala/org/apache/samza/container/TestSamzaContainer.scala
@@ -181,8 +181,8 @@ class TestSamzaContainer extends AssertionsForJUnit with MockitoSugar {
       new TaskName("t1") -> new TaskModel(new TaskName("t1"), offsets.keySet(), new Partition(0)),
       new TaskName("t2") -> new TaskModel(new TaskName("t2"), offsets.keySet(), new Partition(0)))
     val containers = Map(
-      "0" -> new ContainerModel("0", 0, tasks),
-      "1" -> new ContainerModel("1", 0, tasks))
+      "0" -> new ContainerModel("0", tasks),
+      "1" -> new ContainerModel("1", tasks))
     val jobModel = new JobModel(config, containers)
     def jobModelGenerator(): JobModel = jobModel
     val server = new HttpServer
@@ -206,8 +206,8 @@ class TestSamzaContainer extends AssertionsForJUnit with MockitoSugar {
       new TaskName("t1") -> new TaskModel(new TaskName("t1"), offsets.keySet(), new Partition(0)),
       new TaskName("t2") -> new TaskModel(new TaskName("t2"), offsets.keySet(), new Partition(0)))
     val containers = Map(
-      "0" -> new ContainerModel("0", 0, tasks),
-      "1" -> new ContainerModel("1", 1, tasks))
+      "0" -> new ContainerModel("0", tasks),
+      "1" -> new ContainerModel("1", tasks))
     val jobModel = new JobModel(config, containers)
     def jobModelGenerator(): JobModel = jobModel
     val server = new HttpServer
@@ -234,7 +234,7 @@ class TestSamzaContainer extends AssertionsForJUnit with MockitoSugar {
     val taskModel1 = new TaskModel(taskName1,
       Set(new SystemStreamPartition("input", "stream", new Partition(1))),
       new Partition(11))
-    val containerModel = new ContainerModel("processorId", 0, Map(taskName0 -> taskModel0, taskName1 -> taskModel1))
+    val containerModel = new ContainerModel("processorId", Map(taskName0 -> taskModel0, taskName1 -> taskModel1))
     val changeLogSystemStreams = Map("store0" -> new SystemStream("changelogSystem0", "store0-changelog"),
       "store1" -> new SystemStream("changelogSystem1", "store1-changelog"))
     val expected = Set(new SystemStreamPartition("changelogSystem0", "store0-changelog", new Partition(10)),
@@ -254,7 +254,7 @@ class TestSamzaContainer extends AssertionsForJUnit with MockitoSugar {
     val taskModel1 = new TaskModel(taskName1,
       Set(new SystemStreamPartition("input", "stream", new Partition(1))),
       new Partition(11))
-    val containerModel = new ContainerModel("processorId", 0, Map(taskName0 -> taskModel0, taskName1 -> taskModel1))
+    val containerModel = new ContainerModel("processorId", Map(taskName0 -> taskModel0, taskName1 -> taskModel1))
     assertEquals(Set(), SamzaContainer.getChangelogSSPsForContainer(containerModel, Map()))
   }
 

http://git-wip-us.apache.org/repos/asf/samza/blob/3bb24c8e/samza-core/src/test/scala/org/apache/samza/coordinator/TestJobCoordinator.scala
----------------------------------------------------------------------
diff --git a/samza-core/src/test/scala/org/apache/samza/coordinator/TestJobCoordinator.scala b/samza-core/src/test/scala/org/apache/samza/coordinator/TestJobCoordinator.scala
index 42610ae..2488355 100644
--- a/samza-core/src/test/scala/org/apache/samza/coordinator/TestJobCoordinator.scala
+++ b/samza-core/src/test/scala/org/apache/samza/coordinator/TestJobCoordinator.scala
@@ -69,8 +69,8 @@ class TestJobCoordinator extends FlatSpec with PrivateMethodTester {
     val container1Tasks = Map(
       task1Name -> new TaskModel(task1Name, checkpoint1.keySet.asJava, new Partition(3)))
     val containers = Map(
-      "0" -> new ContainerModel("0", 0, container0Tasks.asJava),
-      "1" -> new ContainerModel("1", 1, container1Tasks.asJava))
+      "0" -> new ContainerModel("0", container0Tasks.asJava),
+      "1" -> new ContainerModel("1", container1Tasks.asJava))
 
 
     // The test does not pass offsets for task2 (Partition 2) to the checkpointmanager, this will verify that we get an offset 0 for this partition
@@ -137,8 +137,8 @@ class TestJobCoordinator extends FlatSpec with PrivateMethodTester {
     val container1Tasks = Map(
       task1Name -> new TaskModel(task1Name, ssp1.asJava, new Partition(3)))
     val containers = Map(
-      Integer.valueOf(0) -> new ContainerModel("0", 0, container0Tasks.asJava),
-      Integer.valueOf(1) -> new ContainerModel("1", 1, container1Tasks.asJava))
+      Integer.valueOf(0) -> new ContainerModel("0", container0Tasks.asJava),
+      Integer.valueOf(1) -> new ContainerModel("1", container1Tasks.asJava))
     val changelogInfo0 = MockCoordinatorStreamWrappedConsumer.CHANGELOGPREFIX + "mock:" + task0Name.getTaskName() -> "4"
 
     // Configs which are processed by the MockCoordinatorStream as special configs which are interpreted as
@@ -196,7 +196,7 @@ class TestJobCoordinator extends FlatSpec with PrivateMethodTester {
     val container0Tasks = Map(
       task1Name -> new TaskModel(task1Name, Set(new SystemStreamPartition("test", "stream1", new Partition(1))).asJava, new Partition(0)))
     val containers = Map(
-      "0" -> new ContainerModel("0", 0, container0Tasks.asJava))
+      "0" -> new ContainerModel("0", container0Tasks.asJava))
     val jobModel = new JobModel(config, containers.asJava)
     assertEquals(config, coordinator.jobModel.getConfig)
     assertEquals(jobModel, coordinator.jobModel)
@@ -218,7 +218,7 @@ class TestJobCoordinator extends FlatSpec with PrivateMethodTester {
       task1Name -> new TaskModel(task1Name, Set(new SystemStreamPartition("test", "stream1", new Partition(1))).asJava, new Partition(0)))
 
     val containers = Map(
-      "0" -> new ContainerModel("0", 0, container0Tasks.asJava))
+      "0" -> new ContainerModel("0", container0Tasks.asJava))
     val jobModel = new JobModel(config, containers.asJava)
     assertEquals(config, coordinator.jobModel.getConfig)
     assertEquals(jobModel, coordinator.jobModel)

http://git-wip-us.apache.org/repos/asf/samza/blob/3bb24c8e/samza-yarn/src/test/java/org/apache/samza/webapp/TestApplicationMasterRestClient.java
----------------------------------------------------------------------
diff --git a/samza-yarn/src/test/java/org/apache/samza/webapp/TestApplicationMasterRestClient.java b/samza-yarn/src/test/java/org/apache/samza/webapp/TestApplicationMasterRestClient.java
index dbe534f..9c0dea7 100644
--- a/samza-yarn/src/test/java/org/apache/samza/webapp/TestApplicationMasterRestClient.java
+++ b/samza-yarn/src/test/java/org/apache/samza/webapp/TestApplicationMasterRestClient.java
@@ -279,7 +279,7 @@ public class TestApplicationMasterRestClient {
     Set<ContainerModel> containerModels = grouper.group(taskModels);
     HashMap<String, ContainerModel> containers = new HashMap<>();
     for (ContainerModel containerModel : containerModels) {
-      containers.put(containerModel.getProcessorId(), containerModel);
+      containers.put(containerModel.getId(), containerModel);
     }
     return containers;
   }


[05/29] samza git commit: SAMZA-1840: Refactor TestRunner Apis to use StreamDescriptor and SystemDescriptor

Posted by bo...@apache.org.
SAMZA-1840: Refactor TestRunner Apis to use StreamDescriptor and SystemDescriptor

CollectionStream -> InMemoryInputDescriptor & InMemoryOutputDescriptor
CollectionStreamSystemSpec -> InMemorySystemDescriptor

Author: Sanil Jain <sn...@linkedin.com>

Reviewers: Prateek Maheshwari <pm...@apache.org>, Cameron Lee <ca...@linkedin.com>

Closes #634 from Sanil15/SAMZA-1840


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

Branch: refs/heads/NewKafkaSystemConsumer
Commit: 1755268cff201663a41ca06e9dcc4602d41fc306
Parents: 3bb24c8
Author: Sanil Jain <sn...@linkedin.com>
Authored: Wed Sep 19 12:21:12 2018 -0700
Committer: Prateek Maheshwari <pm...@apache.org>
Committed: Wed Sep 19 12:21:12 2018 -0700

----------------------------------------------------------------------
 .../samza/example/PageViewCounterExample.java   |   1 -
 .../samza/test/framework/StreamAssert.java      |  73 ++---
 .../apache/samza/test/framework/TestRunner.java | 286 +++++++++----------
 .../test/framework/stream/CollectionStream.java | 204 -------------
 .../system/CollectionStreamSystemSpec.java      |  90 ------
 .../system/InMemoryInputDescriptor.java         |  42 +++
 .../system/InMemoryOutputDescriptor.java        |  46 +++
 .../system/InMemorySystemDescriptor.java        | 118 ++++++++
 .../AsyncStreamTaskIntegrationTest.java         | 108 ++++---
 .../StreamApplicationIntegrationTest.java       |  45 ++-
 .../framework/StreamTaskIntegrationTest.java    | 112 +++++---
 .../table/TestLocalTableWithSideInputs.java     |  32 ++-
 12 files changed, 573 insertions(+), 584 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/samza/blob/1755268c/samza-test/src/main/java/org/apache/samza/example/PageViewCounterExample.java
----------------------------------------------------------------------
diff --git a/samza-test/src/main/java/org/apache/samza/example/PageViewCounterExample.java b/samza-test/src/main/java/org/apache/samza/example/PageViewCounterExample.java
index b540585..e2ebc93 100644
--- a/samza-test/src/main/java/org/apache/samza/example/PageViewCounterExample.java
+++ b/samza-test/src/main/java/org/apache/samza/example/PageViewCounterExample.java
@@ -53,7 +53,6 @@ public class PageViewCounterExample implements StreamApplication {
     Config config = cmdLine.loadConfig(cmdLine.parser().parse(args));
     PageViewCounterExample app = new PageViewCounterExample();
     ApplicationRunner runner = ApplicationRunners.getApplicationRunner(app, config);
-
     runner.run();
     runner.waitForFinish();
   }

http://git-wip-us.apache.org/repos/asf/samza/blob/1755268c/samza-test/src/main/java/org/apache/samza/test/framework/StreamAssert.java
----------------------------------------------------------------------
diff --git a/samza-test/src/main/java/org/apache/samza/test/framework/StreamAssert.java b/samza-test/src/main/java/org/apache/samza/test/framework/StreamAssert.java
index 9972d7f..42379f3 100644
--- a/samza-test/src/main/java/org/apache/samza/test/framework/StreamAssert.java
+++ b/samza-test/src/main/java/org/apache/samza/test/framework/StreamAssert.java
@@ -22,9 +22,9 @@ package org.apache.samza.test.framework;
 import com.google.common.base.Preconditions;
 import java.time.Duration;
 import java.util.stream.Collectors;
-import org.apache.samza.test.framework.stream.CollectionStream;
 import java.util.List;
 import java.util.Map;
+import org.apache.samza.test.framework.system.InMemoryOutputDescriptor;
 import org.hamcrest.collection.IsIterableContainingInAnyOrder;
 import org.hamcrest.collection.IsIterableContainingInOrder;
 
@@ -32,22 +32,24 @@ import static org.junit.Assert.assertThat;
 
 
 /**
- * Assertion utils non the content of a {@link CollectionStream}.
+ * Assertion utils on the content of a stream described by
+ * {@link org.apache.samza.operators.descriptors.base.stream.StreamDescriptor}.
  */
 public class StreamAssert {
   /**
-   * Util to assert  presence of messages in a stream with single partition in any order
+   * Verifies that the {@code expected} messages are present in any order in the single partition stream
+   * represented by {@code outputDescriptor}
    *
-   * @param collectionStream represents the actual stream which will be consumed to compare against expected list
-   * @param expected represents the expected stream of messages
+   * @param expected expected stream of messages
+   * @param outputDescriptor describes the stream which will be consumed to compare against expected list
    * @param timeout maximum time to wait for consuming the stream
-   * @param <M> represents the type of Message in the stream
+   * @param <StreamMessageType> type of messages in the stream
    * @throws InterruptedException when {@code consumeStream} is interrupted by another thread during polling messages
    */
-  public static <M> void containsInAnyOrder(CollectionStream<M> collectionStream, final List<M> expected, Duration timeout)
-      throws InterruptedException {
-    Preconditions.checkNotNull(collectionStream, "This util is intended to use only on CollectionStream");
-    assertThat(TestRunner.consumeStream(collectionStream, timeout)
+  public static <StreamMessageType> void containsInAnyOrder(List<StreamMessageType> expected,
+      InMemoryOutputDescriptor<StreamMessageType> outputDescriptor, Duration timeout) throws InterruptedException {
+    Preconditions.checkNotNull(outputDescriptor);
+    assertThat(TestRunner.consumeStream(outputDescriptor, timeout)
         .entrySet()
         .stream()
         .flatMap(entry -> entry.getValue().stream())
@@ -55,19 +57,20 @@ public class StreamAssert {
   }
 
   /**
-   * Util to assert presence of messages in a stream with multiple partition in any order
+   * Verifies that the {@code expected} messages are present in any order in the multi partition stream
+   * represented by {@code outputDescriptor}
    *
-   * @param collectionStream represents the actual stream which will be consumed to compare against expected partition map
-   * @param expected represents a map of partitionId as key and list of messages in stream as value
+   * @param expected map of partitionId as key and list of messages in stream as value
+   * @param outputDescriptor describes the stream which will be consumed to compare against expected partition map
    * @param timeout maximum time to wait for consuming the stream
-   * @param <M> represents the type of Message in the stream
+   * @param <StreamMessageType>  type of messages in the stream
    * @throws InterruptedException when {@code consumeStream} is interrupted by another thread during polling messages
    *
    */
-  public static <M> void containsInAnyOrder(CollectionStream<M> collectionStream, final Map<Integer, List<M>> expected,
-      Duration timeout) throws InterruptedException {
-    Preconditions.checkNotNull(collectionStream, "This util is intended to use only on CollectionStream");
-    Map<Integer, List<M>> actual = TestRunner.consumeStream(collectionStream, timeout);
+  public static <StreamMessageType> void containsInAnyOrder(Map<Integer, List<StreamMessageType>> expected,
+      InMemoryOutputDescriptor<StreamMessageType> outputDescriptor, Duration timeout) throws InterruptedException {
+    Preconditions.checkNotNull(outputDescriptor);
+    Map<Integer, List<StreamMessageType>> actual = TestRunner.consumeStream(outputDescriptor, timeout);
     for (Integer paritionId : expected.keySet()) {
       assertThat(actual.get(paritionId),
           IsIterableContainingInAnyOrder.containsInAnyOrder(expected.get(paritionId).toArray()));
@@ -75,18 +78,19 @@ public class StreamAssert {
   }
 
   /**
-   * Util to assert ordering of messages in a stream with single partition
+   * Verifies that the {@code expected} messages are present in order in the single partition stream
+   * represented by {@code outputDescriptor}
    *
-   * @param collectionStream represents the actual stream which will be consumed to compare against expected list
-   * @param expected represents the expected stream of messages
+   * @param expected  expected stream of messages
+   * @param outputDescriptor describes the stream which will be consumed to compare against expected list
    * @param timeout maximum time to wait for consuming the stream
-   * @param <M> represents the type of Message in the stream
+   * @param <StreamMessageType> type of messages in the stream
    * @throws InterruptedException when {@code consumeStream} is interrupted by another thread during polling messages
    */
-  public static <M> void containsInOrder(CollectionStream<M> collectionStream, final List<M> expected, Duration timeout)
-      throws InterruptedException {
-    Preconditions.checkNotNull(collectionStream, "This util is intended to use only on CollectionStream");
-    assertThat(TestRunner.consumeStream(collectionStream, timeout)
+  public static <StreamMessageType> void containsInOrder(List<StreamMessageType> expected,
+      InMemoryOutputDescriptor<StreamMessageType> outputDescriptor, Duration timeout) throws InterruptedException {
+    Preconditions.checkNotNull(outputDescriptor);
+    assertThat(TestRunner.consumeStream(outputDescriptor, timeout)
         .entrySet()
         .stream()
         .flatMap(entry -> entry.getValue().stream())
@@ -94,18 +98,19 @@ public class StreamAssert {
   }
 
   /**
-   * Util to assert ordering of messages in a multi-partitioned stream
+   * Verifies that the {@code expected} messages are present in order in the multi partition stream
+   * represented by {@code outputDescriptor}
    *
-   * @param collectionStream represents the actual stream which will be consumed to compare against expected partition map
-   * @param expected represents a map of partitionId as key and list of messages as value
+   * @param expected map of partitionId as key and list of messages as value
+   * @param outputDescriptor describes the stream which will be consumed to compare against expected partition map
    * @param timeout maximum time to wait for consuming the stream
-   * @param <M> represents the type of Message in the stream
+   * @param <StreamMessageType> type of messages in the stream
    * @throws InterruptedException when {@code consumeStream} is interrupted by another thread during polling messages
    */
-  public static <M> void containsInOrder(CollectionStream<M> collectionStream, final Map<Integer, List<M>> expected,
-      Duration timeout) throws InterruptedException {
-    Preconditions.checkNotNull(collectionStream, "This util is intended to use only on CollectionStream");
-    Map<Integer, List<M>> actual = TestRunner.consumeStream(collectionStream, timeout);
+  public static <StreamMessageType> void containsInOrder(Map<Integer, List<StreamMessageType>> expected,
+      InMemoryOutputDescriptor<StreamMessageType> outputDescriptor, Duration timeout) throws InterruptedException {
+    Preconditions.checkNotNull(outputDescriptor);
+    Map<Integer, List<StreamMessageType>> actual = TestRunner.consumeStream(outputDescriptor, timeout);
     for (Integer paritionId : expected.keySet()) {
       assertThat(actual.get(paritionId), IsIterableContainingInOrder.contains(expected.get(paritionId).toArray()));
     }

http://git-wip-us.apache.org/repos/asf/samza/blob/1755268c/samza-test/src/main/java/org/apache/samza/test/framework/TestRunner.java
----------------------------------------------------------------------
diff --git a/samza-test/src/main/java/org/apache/samza/test/framework/TestRunner.java b/samza-test/src/main/java/org/apache/samza/test/framework/TestRunner.java
index 033bcdf..5c4ba3b 100644
--- a/samza-test/src/main/java/org/apache/samza/test/framework/TestRunner.java
+++ b/samza-test/src/main/java/org/apache/samza/test/framework/TestRunner.java
@@ -35,7 +35,6 @@ import org.apache.samza.application.SamzaApplication;
 import org.apache.samza.application.StreamApplication;
 import org.apache.samza.application.TaskApplication;
 import org.apache.samza.config.Config;
-import org.apache.samza.config.InMemorySystemConfig;
 import org.apache.samza.config.JobConfig;
 import org.apache.samza.config.JobCoordinatorConfig;
 import org.apache.samza.config.MapConfig;
@@ -62,58 +61,47 @@ import org.apache.samza.task.AsyncStreamTaskFactory;
 import org.apache.samza.task.StreamTask;
 import org.apache.samza.task.StreamTaskFactory;
 import org.apache.samza.task.TaskFactory;
-import org.apache.samza.test.framework.stream.CollectionStream;
-import org.apache.samza.test.framework.system.CollectionStreamSystemSpec;
+import org.apache.samza.test.framework.system.InMemoryInputDescriptor;
+import org.apache.samza.test.framework.system.InMemoryOutputDescriptor;
+import org.apache.samza.test.framework.system.InMemorySystemDescriptor;
 import org.junit.Assert;
 
 
 /**
- * TestRunner provides apis to quickly set up tests for Samza low level and high level apis. Default running mode
- * for test is Single container without any distributed coordination service. Test runner maintains global job config
- * {@code configs} that are used to run the Samza job
+ * TestRunner provides APIs to set up integration tests for a Samza application.
+ * Running mode for test is Single container mode
+ * Test sets following configuration for the application
  *
- * For single container mode following configs are set by default
+ * The following configs are set by default
  *  <ol>
  *    <li>"job.coordination.utils.factory" = {@link PassthroughCoordinationUtilsFactory}</li>
  *    <li>"job.coordination.factory" = {@link PassthroughJobCoordinatorFactory}</li>
  *    <li>"task.name.grouper.factory" = {@link SingleContainerGrouperFactory}</li>
  *    <li>"job.name" = "test-samza"</li>
  *    <li>"processor.id" = "1"</li>
- *    <li>"inmemory.scope = " Scope id generated to isolate the run for InMemorySystem</li>
  *  </ol>
  *
  */
 public class TestRunner {
-
-  private static final String JOB_NAME = "test-samza";
-  public enum Mode {
-    SINGLE_CONTAINER, MULTI_CONTAINER
-  }
+  public static final String JOB_NAME = "samza-test";
 
   private Map<String, String> configs;
-  private Map<String, CollectionStreamSystemSpec> systems;
   private Class taskClass;
   private StreamApplication app;
-  private String testId;
-  private SystemFactory factory;
-
-  /**
-   * Mode defines single or multi container running configuration, by default a single container configuration is assumed
+  /*
+   * inMemoryScope is a unique global key per TestRunner, this key when configured with {@link InMemorySystemDescriptor}
+   * provides an isolated state to run with in memory system
    */
-  private Mode mode;
+  private String inMemoryScope;
 
   private TestRunner() {
-    this.testId = RandomStringUtils.random(10, true, true);
-    this.systems = new HashMap<String, CollectionStreamSystemSpec>();
     this.configs = new HashMap<>();
-    this.mode = Mode.SINGLE_CONTAINER;
-    this.factory = new InMemorySystemFactory();
-    configs.put(InMemorySystemConfig.INMEMORY_SCOPE, testId);
+    this.inMemoryScope = RandomStringUtils.random(10, true, true);
     configs.put(JobConfig.JOB_NAME(), JOB_NAME);
-    configs.putIfAbsent(JobConfig.PROCESSOR_ID(), "1");
-    configs.putIfAbsent(JobCoordinatorConfig.JOB_COORDINATION_UTILS_FACTORY, PassthroughCoordinationUtilsFactory.class.getName());
-    configs.putIfAbsent(JobCoordinatorConfig.JOB_COORDINATOR_FACTORY, PassthroughJobCoordinatorFactory.class.getName());
-    configs.putIfAbsent(TaskConfig.GROUPER_FACTORY(), SingleContainerGrouperFactory.class.getName());
+    configs.put(JobConfig.PROCESSOR_ID(), "1");
+    configs.put(JobCoordinatorConfig.JOB_COORDINATION_UTILS_FACTORY, PassthroughCoordinationUtilsFactory.class.getName());
+    configs.put(JobCoordinatorConfig.JOB_COORDINATOR_FACTORY, PassthroughJobCoordinatorFactory.class.getName());
+    configs.put(TaskConfig.GROUPER_FACTORY(), SingleContainerGrouperFactory.class.getName());
   }
 
   /**
@@ -129,7 +117,7 @@ public class TestRunner {
 
   /**
    * Constructs a new {@link TestRunner} from following components
-   * @param app represent a class containing Samza job logic implementing {@link StreamApplication}
+   * @param app samza job implementing {@link StreamApplication}
    */
   private TestRunner(StreamApplication app) {
     this();
@@ -138,20 +126,9 @@ public class TestRunner {
   }
 
   /**
-   * Registers a system with TestRunner if not already registered and configures all the system configs to global
-   * job configs
-   */
-  private void registerSystem(String systemName) {
-    if (!systems.containsKey(systemName)) {
-      systems.put(systemName, CollectionStreamSystemSpec.create(systemName, JOB_NAME));
-      configs.putAll(systems.get(systemName).getSystemConfigs());
-    }
-  }
-
-  /**
    * Creates an instance of {@link TestRunner} for Low Level Samza Api
-   * @param taskClass represent a class extending either {@link StreamTask} or {@link AsyncStreamTask}
-   * @return a {@link TestRunner} for {@code taskClass}
+   * @param taskClass samza job extending either {@link StreamTask} or {@link AsyncStreamTask}
+   * @return this {@link TestRunner}
    */
   public static TestRunner of(Class taskClass) {
     Preconditions.checkNotNull(taskClass);
@@ -162,8 +139,8 @@ public class TestRunner {
 
   /**
    * Creates an instance of {@link TestRunner} for High Level/Fluent Samza Api
-   * @param app represent a class representing Samza job by implementing {@link StreamApplication}
-   * @return a {@link TestRunner} for {@code app}
+   * @param app samza job implementing {@link StreamApplication}
+   * @return this {@link TestRunner}
    */
   public static TestRunner of(StreamApplication app) {
     Preconditions.checkNotNull(app);
@@ -171,11 +148,11 @@ public class TestRunner {
   }
 
   /**
-   * Only adds a config from {@code config} to global {@code configs} if they dont exist in it.
-   * @param config represents the {@link Config} supposed to be added to global configs
-   * @return calling instance of {@link TestRunner} with added configs if they don't exist
+   * Only adds a config from {@code config} to samza job {@code configs} if they dont exist in it.
+   * @param config configs for the application
+   * @return this {@link TestRunner}
    */
-  public TestRunner addConfigs(Config config) {
+  public TestRunner addConfigs(Map<String, String> config) {
     Preconditions.checkNotNull(config);
     config.forEach(this.configs::putIfAbsent);
     return this;
@@ -186,7 +163,7 @@ public class TestRunner {
    * exisiting in {@code configs}
    * @param key key of the config
    * @param value value of the config
-   * @return calling instance of {@link TestRunner} with added config
+   * @return this {@link TestRunner}
    */
   public TestRunner addOverrideConfig(String key, String value) {
     Preconditions.checkNotNull(key);
@@ -197,94 +174,72 @@ public class TestRunner {
   }
 
   /**
-   * Configures {@code stream} with the TestRunner, adds all the stream specific configs to global job configs.
-   * <p>
-   * Every stream belongs to a System (here a {@link CollectionStreamSystemSpec}), this utility also registers the system with
-   * {@link TestRunner} if not registered already. Then it creates and initializes the stream partitions with messages for
-   * the registered System
-   * <p>
-   * @param stream represents the stream that is supposed to be configured with {@link TestRunner}
-   * @return calling instance of {@link TestRunner} with {@code stream} configured with it
+   * Adds the provided input stream with mock data to the test application.
+   *
+   * @param descriptor describes the stream that is supposed to be input to Samza application
+   * @param messages messages used to initialize the single partition stream
+   * @param <StreamMessageType> a message with null key or a KV {@link org.apache.samza.operators.KV}.
+   *                            key of KV represents key of {@link org.apache.samza.system.IncomingMessageEnvelope} or
+   *                           {@link org.apache.samza.system.OutgoingMessageEnvelope} and value is message
+   * @return this {@link TestRunner}
    */
-  public TestRunner addInputStream(CollectionStream stream) {
-    Preconditions.checkNotNull(stream);
-    registerSystem(stream.getSystemName());
-    initializeInput(stream);
-    stream.setTestId(testId);
-    if (configs.containsKey(TaskConfig.INPUT_STREAMS())) {
-      configs.put(TaskConfig.INPUT_STREAMS(),
-          configs.get(TaskConfig.INPUT_STREAMS()).concat("," + stream.getSystemName() + "." + stream.getPhysicalName()));
-    } else {
-      configs.put(TaskConfig.INPUT_STREAMS(), stream.getSystemName() + "." + stream.getPhysicalName());
-    }
-    stream.getStreamConfig().forEach((key, val) -> {
-        configs.putIfAbsent((String) key, (String) val);
-      });
-
+  public <StreamMessageType> TestRunner addInputStream(InMemoryInputDescriptor descriptor,
+      List<StreamMessageType> messages) {
+    Preconditions.checkNotNull(descriptor, messages);
+    Map<Integer, Iterable<StreamMessageType>> partitionData = new HashMap<Integer, Iterable<StreamMessageType>>();
+    partitionData.put(0, messages);
+    initializeInMemoryInputStream(descriptor, partitionData);
     return this;
   }
 
   /**
-   * Creates an in memory stream with {@link InMemorySystemFactory} and initializes the metadata for the stream.
-   * Initializes each partition of that stream with messages from {@code stream.getInitPartitions}
-   *
-   * @param stream represents the stream to initialize with the in memory system
-   * @param <T> can represent a message or a KV {@link org.apache.samza.operators.KV}, key of which represents key of a
-   *            {@link org.apache.samza.system.IncomingMessageEnvelope} or {@link org.apache.samza.system.OutgoingMessageEnvelope}
-   *            and value represents the message
+   * Adds the provided input stream with mock data to the test application.
+   * @param descriptor describes the stream that is supposed to be input to Samza application
+   * @param messages map whose key is partitionId and value is messages in the partition
+   * @param <StreamMessageType> message with null key or a KV {@link org.apache.samza.operators.KV}.
+   *                           A key of which represents key of {@link org.apache.samza.system.IncomingMessageEnvelope} or
+   *                           {@link org.apache.samza.system.OutgoingMessageEnvelope} and value is message
+   * @return this {@link TestRunner}
    */
-  private <T> void initializeInput(CollectionStream stream) {
-    Preconditions.checkNotNull(stream);
-    Preconditions.checkState(stream.getInitPartitions().size() >= 1);
-    String streamName = stream.getStreamName();
-    String systemName = stream.getSystemName();
-    Map<Integer, Iterable<T>> partitions = stream.getInitPartitions();
-    StreamSpec spec = new StreamSpec(streamName, stream.getPhysicalName(), systemName, partitions.size());
-    factory.getAdmin(systemName, new MapConfig(configs)).createStream(spec);
-    SystemProducer producer = factory.getProducer(systemName, new MapConfig(configs), null);
-    partitions.forEach((partitionId, partition) -> {
-        partition.forEach(e -> {
-            Object key = e instanceof KV ? ((KV) e).getKey() : null;
-            Object value = e instanceof KV ? ((KV) e).getValue() : e;
-            producer.send(systemName,
-                new OutgoingMessageEnvelope(new SystemStream(systemName, stream.getPhysicalName()), Integer.valueOf(partitionId), key,
-                    value));
-          });
-        producer.send(systemName,
-            new OutgoingMessageEnvelope(new SystemStream(systemName, stream.getPhysicalName()), Integer.valueOf(partitionId), null,
-                new EndOfStreamMessage(null)));
-      });
+  public <StreamMessageType> TestRunner addInputStream(InMemoryInputDescriptor descriptor,
+      Map<Integer, ? extends Iterable<StreamMessageType>> messages) {
+    Preconditions.checkNotNull(descriptor, messages);
+    Map<Integer, Iterable<StreamMessageType>> partitionData = new HashMap<Integer, Iterable<StreamMessageType>>();
+    partitionData.putAll(messages);
+    initializeInMemoryInputStream(descriptor, partitionData);
+    return this;
   }
 
   /**
-   * Configures {@code stream} with the TestRunner, adds all the stream specific configs to global job configs.
-   * <p>
-   * Every stream belongs to a System (here a {@link CollectionStreamSystemSpec}), this utility also registers the system with
-   * {@link TestRunner} if not registered already. Then it creates the stream partitions with the registered System
-   * <p>
-   * @param stream represents the stream that is supposed to be configured with {@link TestRunner}
-   * @return calling instance of {@link TestRunner} with {@code stream} configured with it
+   * Adds the provided output stream to the test application.
+   * @param streamDescriptor describes the stream that is supposed to be output for the Samza application
+   * @param partitionCount partition count of output stream
+   * @return this {@link TestRunner}
    */
-  public TestRunner addOutputStream(CollectionStream stream) {
-    Preconditions.checkNotNull(stream);
-    Preconditions.checkState(stream.getInitPartitions().size() >= 1);
-    registerSystem(stream.getSystemName());
-    stream.setTestId(testId);
-    StreamSpec spec = new StreamSpec(stream.getStreamName(), stream.getPhysicalName(), stream.getSystemName(), stream.getInitPartitions().size());
+  public TestRunner addOutputStream(InMemoryOutputDescriptor streamDescriptor, int partitionCount) {
+    Preconditions.checkNotNull(streamDescriptor);
+    Preconditions.checkState(partitionCount >= 1);
+    InMemorySystemDescriptor imsd = (InMemorySystemDescriptor) streamDescriptor.getSystemDescriptor();
+    imsd.withInMemoryScope(this.inMemoryScope);
+    Config config = new MapConfig(streamDescriptor.toConfig(), streamDescriptor.getSystemDescriptor().toConfig());
+    InMemorySystemFactory factory = new InMemorySystemFactory();
+    String physicalName = (String) streamDescriptor.getPhysicalName().orElse(streamDescriptor.getStreamId());
+    StreamSpec spec = new StreamSpec(streamDescriptor.getStreamId(), physicalName, streamDescriptor.getSystemName(),
+        partitionCount);
     factory
-        .getAdmin(stream.getSystemName(), new MapConfig(configs))
+        .getAdmin(streamDescriptor.getSystemName(), config)
         .createStream(spec);
-    configs.putAll(stream.getStreamConfig());
+    addConfigs(streamDescriptor.toConfig());
+    addConfigs(streamDescriptor.getSystemDescriptor().toConfig());
     return this;
   }
 
-
   /**
-   * Utility to run a test configured using TestRunner
+   * Run the application with the specified timeout
    *
-   * @param timeout time to wait for the high level application or low level task to finish. This timeout does not include
+   * @param timeout time to wait for the application to finish. This timeout does not include
    *                input stream initialization time or the assertion time over output streams. This timeout just accounts
-   *                for time that samza job takes run. Samza job won't be invoked with negative or zero timeout
+   *                for time that samza job takes run. Timeout must be greater than 0.
    * @throws SamzaException if Samza job fails with exception and returns UnsuccessfulFinish as the statuscode
    */
   public void run(Duration timeout) {
@@ -301,34 +256,33 @@ public class TestRunner {
       throw new SamzaException(ExceptionUtils.getStackTrace(status.getThrowable()));
     }
   }
+
   /**
-   * Utility to read the messages from a stream from the beginning, this is supposed to be used after executing the
-   * TestRunner in order to assert over the streams (ex output streams).
+   * Gets the contents of the output stream represented by {@code outputDescriptor} after {@link TestRunner#run(Duration)}
+   * has completed
    *
-   * @param stream represents {@link CollectionStream} whose current state of partitions is requested to be fetched
-   * @param timeout poll timeout in Ms
-   * @param <T> represents type of message
+   * @param outputDescriptor describes the stream to be consumed
+   * @param timeout timeout for consumption of stream in Ms
+   * @param <StreamMessageType> type of message
    *
-   * @return a map key of which represents the {@code partitionId} and value represents the current state of the partition
-   *         i.e messages in the partition
-   * @throws InterruptedException Thrown when a blocking poll has been interrupted by another thread.
+   * @return a map whose key is {@code partitionId} and value is messages in partition
+   * @throws SamzaException Thrown when a poll is incomplete
    */
-  public static <T> Map<Integer, List<T>> consumeStream(CollectionStream stream, Duration timeout) throws InterruptedException {
-    Preconditions.checkNotNull(stream);
-    Preconditions.checkNotNull(stream.getSystemName());
-    String streamName = stream.getStreamName();
-    String systemName = stream.getSystemName();
+  public static <StreamMessageType> Map<Integer, List<StreamMessageType>> consumeStream(
+      InMemoryOutputDescriptor outputDescriptor, Duration timeout) throws SamzaException {
+    Preconditions.checkNotNull(outputDescriptor);
+    String streamId = outputDescriptor.getStreamId();
+    String systemName = outputDescriptor.getSystemName();
     Set<SystemStreamPartition> ssps = new HashSet<>();
-    Set<String> streamNames = new HashSet<>();
-    streamNames.add(streamName);
+    Set<String> streamIds = new HashSet<>();
+    streamIds.add(streamId);
     SystemFactory factory = new InMemorySystemFactory();
-    HashMap<String, String> config = new HashMap<>();
-    config.put(InMemorySystemConfig.INMEMORY_SCOPE, stream.getTestId());
-    Map<String, SystemStreamMetadata> metadata =
-        factory.getAdmin(systemName, new MapConfig(config)).getSystemStreamMetadata(streamNames);
-    SystemConsumer consumer = factory.getConsumer(systemName, new MapConfig(config), null);
-    metadata.get(stream.getPhysicalName()).getSystemStreamPartitionMetadata().keySet().forEach(partition -> {
-        SystemStreamPartition temp = new SystemStreamPartition(systemName, streamName, partition);
+    Config config = new MapConfig(outputDescriptor.toConfig(), outputDescriptor.getSystemDescriptor().toConfig());
+    Map<String, SystemStreamMetadata> metadata = factory.getAdmin(systemName, config).getSystemStreamMetadata(streamIds);
+    SystemConsumer consumer = factory.getConsumer(systemName, config, null);
+    String name = (String) outputDescriptor.getPhysicalName().orElse(streamId);
+    metadata.get(name).getSystemStreamPartitionMetadata().keySet().forEach(partition -> {
+        SystemStreamPartition temp = new SystemStreamPartition(systemName, streamId, partition);
         ssps.add(temp);
         consumer.register(temp, "0");
       });
@@ -337,12 +291,17 @@ public class TestRunner {
     Map<SystemStreamPartition, List<IncomingMessageEnvelope>> output = new HashMap<>();
     HashSet<SystemStreamPartition> didNotReachEndOfStream = new HashSet<>(ssps);
     while (System.currentTimeMillis() < t + timeout.toMillis()) {
-      Map<SystemStreamPartition, List<IncomingMessageEnvelope>> currentState = consumer.poll(ssps, 10);
+      Map<SystemStreamPartition, List<IncomingMessageEnvelope>> currentState = null;
+      try {
+        currentState = consumer.poll(ssps, 10);
+      } catch (InterruptedException e) {
+        throw new SamzaException("Timed out while consuming stream \n" + e.getMessage());
+      }
       for (Map.Entry<SystemStreamPartition, List<IncomingMessageEnvelope>> entry : currentState.entrySet()) {
         SystemStreamPartition ssp = entry.getKey();
         output.computeIfAbsent(ssp, k -> new LinkedList<IncomingMessageEnvelope>());
         List<IncomingMessageEnvelope> currentBuffer = entry.getValue();
-        Integer totalMessagesToFetch = Integer.valueOf(metadata.get(stream.getStreamName())
+        Integer totalMessagesToFetch = Integer.valueOf(metadata.get(outputDescriptor.getStreamId())
             .getSystemStreamPartitionMetadata()
             .get(ssp.getPartition())
             .getNewestOffset());
@@ -364,7 +323,7 @@ public class TestRunner {
     return output.entrySet()
         .stream()
         .collect(Collectors.toMap(entry -> entry.getKey().getPartition().getPartitionId(),
-            entry -> entry.getValue().stream().map(e -> (T) e.getMessage()).collect(Collectors.toList())));
+            entry -> entry.getValue().stream().map(e -> (StreamMessageType) e.getMessage()).collect(Collectors.toList())));
   }
 
   private TaskFactory createTaskFactory() {
@@ -388,4 +347,41 @@ public class TestRunner {
     throw new SamzaException(String.format("Not supported task.class %s. task.class has to implement either StreamTask "
         + "or AsyncStreamTask", taskClass.getName()));
   }
+
+  /**
+   * Creates an in memory stream with {@link InMemorySystemFactory} and feeds its partition with stream of messages
+   * @param partitonData key of the map represents partitionId and value represents
+   *                 messages in the partition
+   * @param descriptor describes a stream to initialize with the in memory system
+   */
+  private <StreamMessageType> void initializeInMemoryInputStream(InMemoryInputDescriptor descriptor,
+      Map<Integer, Iterable<StreamMessageType>> partitonData) {
+    String systemName = descriptor.getSystemName();
+    String streamName = (String) descriptor.getPhysicalName().orElse(descriptor.getStreamId());
+    if (configs.containsKey(TaskConfig.INPUT_STREAMS())) {
+      configs.put(TaskConfig.INPUT_STREAMS(),
+          configs.get(TaskConfig.INPUT_STREAMS()).concat("," + systemName + "." + streamName));
+    } else {
+      configs.put(TaskConfig.INPUT_STREAMS(), systemName + "." + streamName);
+    }
+    InMemorySystemDescriptor imsd = (InMemorySystemDescriptor) descriptor.getSystemDescriptor();
+    imsd.withInMemoryScope(this.inMemoryScope);
+    addConfigs(descriptor.toConfig());
+    addConfigs(descriptor.getSystemDescriptor().toConfig());
+    StreamSpec spec = new StreamSpec(descriptor.getStreamId(), streamName, systemName, partitonData.size());
+    SystemFactory factory = new InMemorySystemFactory();
+    Config config = new MapConfig(descriptor.toConfig(), descriptor.getSystemDescriptor().toConfig());
+    factory.getAdmin(systemName, config).createStream(spec);
+    SystemProducer producer = factory.getProducer(systemName, config, null);
+    SystemStream sysStream = new SystemStream(systemName, streamName);
+    partitonData.forEach((partitionId, partition) -> {
+        partition.forEach(e -> {
+            Object key = e instanceof KV ? ((KV) e).getKey() : null;
+            Object value = e instanceof KV ? ((KV) e).getValue() : e;
+            producer.send(systemName, new OutgoingMessageEnvelope(sysStream, Integer.valueOf(partitionId), key, value));
+          });
+        producer.send(systemName, new OutgoingMessageEnvelope(sysStream, Integer.valueOf(partitionId), null,
+          new EndOfStreamMessage(null)));
+      });
+  }
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/1755268c/samza-test/src/main/java/org/apache/samza/test/framework/stream/CollectionStream.java
----------------------------------------------------------------------
diff --git a/samza-test/src/main/java/org/apache/samza/test/framework/stream/CollectionStream.java b/samza-test/src/main/java/org/apache/samza/test/framework/stream/CollectionStream.java
deleted file mode 100644
index 320a0ac..0000000
--- a/samza-test/src/main/java/org/apache/samza/test/framework/stream/CollectionStream.java
+++ /dev/null
@@ -1,204 +0,0 @@
-/*
- * 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.test.framework.stream;
-
-import com.google.common.base.Preconditions;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.Map;
-
-/**
- * A CollectionStream represents an in memory stream of messages that can either have single or multiple partitions.
- * Every CollectionStream is coupled with a {@link org.apache.samza.test.framework.system.CollectionStreamSystemSpec} that
- * contains all the specification for system
- *<p>
- * When sending messages using {@code CollectionStream<KV<K, V>>}, messages use K as key and V as message
- * When sending messages using {@code CollectionStream<T>}, messages use a nullkey.
- *</p>
- * @param <T>
- *        can represent a message with null key or a KV {@link org.apache.samza.operators.KV}, key of which represents key of a
- *        {@link org.apache.samza.system.IncomingMessageEnvelope} or {@link org.apache.samza.system.OutgoingMessageEnvelope}
- *        and value represents the message of the same
- */
-public class CollectionStream<T> {
-  private String testId;
-  private final String streamName;
-  private final String physicalName;
-  private final String systemName;
-  private Map<Integer, Iterable<T>> initPartitions;
-  private Map<String, String> streamConfig;
-  private static final String STREAM_TO_SYSTEM = "streams.%s.samza.system";
-  private static final String PHYSICAL_NAME = "streams.%s.samza.physical.name";
-
-  /**
-   * Constructs a new CollectionStream from specified components.
-   * @param systemName represents name of the system stream is associated with
-   * @param streamName represents name of the stream
-   */
-  private CollectionStream(String systemName, String streamName) {
-    Preconditions.checkNotNull(systemName);
-    Preconditions.checkNotNull(streamName);
-    this.systemName = systemName;
-    this.streamName = streamName;
-    this.streamConfig = new HashMap<>();
-    // TODO: Once SAMZA-1737 is resolved, generate a randomized physical name
-    this.physicalName = streamName;
-    streamConfig.put(String.format(STREAM_TO_SYSTEM, this.streamName), systemName);
-    streamConfig.put(String.format(PHYSICAL_NAME, this.streamName), physicalName);
-  }
-
-
-  /**
-   * Constructs a new CollectionStream with multiple empty partitions from specified components.
-   * @param systemName represents name of the system stream is associated with
-   * @param streamName represents name of the stream
-   * @param partitionCount represents number of partitions, each of these partitions will be empty
-   */
-  private CollectionStream(String systemName, String streamName, Integer partitionCount) {
-    this(systemName, streamName);
-    Preconditions.checkState(partitionCount > 0);
-    initPartitions = new HashMap<>();
-    for (int i = 0; i < partitionCount; i++) {
-      initPartitions.put(i, new ArrayList<>());
-    }
-  }
-
-  /**
-   * Constructs a new CollectionStream with single partition from specified components.
-   * @param systemName represents name of the system stream is associated with
-   * @param streamName represents name of the stream
-   * @param initPartition represents the messages that the stream will be intialized with, default partitionId for the
-   *                  this single partition stream is 0
-   */
-  private CollectionStream(String systemName, String streamName, Iterable<T> initPartition) {
-    this(systemName, streamName);
-    Preconditions.checkNotNull(initPartition);
-    initPartitions = new HashMap<>();
-    initPartitions.put(0, initPartition);
-  }
-
-  /**
-   * Constructs a new CollectionStream with multiple partitions from specified components.
-   * @param systemName represents name of the system stream is associated with
-   * @param streamName represents name of the stream
-   * @param initPartitions represents the partition state, key of the map represents partitionId and value represents
-   *                   the messages that partition will be initialized with
-   */
-  private CollectionStream(String systemName, String streamName, Map<Integer, ? extends Iterable<T>> initPartitions) {
-    this(systemName, streamName);
-    Preconditions.checkNotNull(initPartitions);
-    this.initPartitions = new HashMap<>(initPartitions);
-  }
-
-  /**
-   * @return The Map of partitions that input stream is supposed to be initialized with, this method is
-   * used internally and should not be used for asserting over streams.
-   * The true state of stream is determined by {@code consmeStream()} of {@link org.apache.samza.test.framework.TestRunner}
-   */
-  public Map<Integer, Iterable<T>> getInitPartitions() {
-    return initPartitions;
-  }
-
-  public String getStreamName() {
-    return streamName;
-  }
-
-  public String getSystemName() {
-    return systemName;
-  }
-
-  public Map<String, String> getStreamConfig() {
-    return streamConfig;
-  }
-
-  public String getTestId() {
-    return testId;
-  }
-
-  public void setTestId(String testId) {
-    this.testId = testId;
-  }
-
-  public String getPhysicalName() {
-    return physicalName;
-  }
-
-  /**
-   * Creates an in memory stream with the name {@code streamName} and initializes the stream to only one partition
-   *
-   * @param systemName represents name of the system stream is associated with
-   * @param streamName represents the name of the Stream
-   * @param <T> represents the type of each message in a stream
-   * @return an {@link CollectionStream} with only one partition that can contain messages of the type
-   */
-  public static <T> CollectionStream<T> empty(String systemName, String streamName) {
-    return new CollectionStream<>(systemName, streamName, 1);
-  }
-
-  /**
-   * Creates an in memory stream with the name {@code streamName} and initializes the stream to have as many partitions
-   * as specified by {@code partitionCount}. These partitions are empty and are supposed to be used by Samza job to produce
-   * messages to.
-   *
-   * @param systemName represents name of the system stream is associated with
-   * @param streamName represents the name of the Stream
-   * @param partitionCount represents the number of partitions the stream would have
-   * @param <T> represents the type of each message in a stream
-   * @return an empty {@link CollectionStream} with multiple partitions that can contain messages of the type {@code T}
-   */
-  public static <T> CollectionStream<T> empty(String systemName, String streamName, int partitionCount) {
-    return new CollectionStream<>(systemName, streamName, partitionCount);
-  }
-
-  /**
-   * Creates an in memory stream with the name {@code streamName}. Stream is created with single partition having
-   * {@code partitionId} is 0. This partition is intialzied with messages of type T
-   *
-   * @param systemName represents name of the system stream is associated with
-   * @param streamName represents the name of the Stream
-   * @param partition represents the messages that the {@link org.apache.samza.system.SystemStreamPartition} will be
-   *                  initialized with
-   * @param <T> represents the type of a message in the stream
-   * @return a {@link CollectionStream} with only one partition containing messages of the type {@code T}
-   *
-   */
-  public static <T> CollectionStream<T> of(String systemName, String streamName, Iterable<T> partition) {
-    return new CollectionStream<>(systemName, streamName, partition);
-  }
-
-  /**
-   * Creates an in memory stream with the name {@code streamName} and initializes the stream to have as many partitions
-   * as the size of {@code partitions} map. Key of the map {@code partitions} represents the {@code partitionId} of
-   * each {@link org.apache.samza.Partition} for a {@link org.apache.samza.system.SystemStreamPartition} and value is
-   * an Iterable of messages that the {@link org.apache.samza.system.SystemStreamPartition} should be initialized with.
-   *
-   * @param systemName represents name of the system stream is associated with
-   * @param streamName represents the name of the Stream
-   * @param partitions Key of an entry in partitions represents a {@code partitionId} of a {@link org.apache.samza.Partition}
-   *                   and value represents the stream of messages the {@link org.apache.samza.system.SystemStreamPartition}
-   *                   will be initialized with
-   * @param <T> represents the type of a message in the stream
-   * @return a {@link CollectionStream} with multiple partitions each containing messages of the type {@code T}
-   *
-   */
-  public static <T> CollectionStream<T> of(String systemName, String streamName, Map<Integer, ? extends Iterable<T>> partitions) {
-    return new CollectionStream<>(systemName, streamName, partitions);
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/1755268c/samza-test/src/main/java/org/apache/samza/test/framework/system/CollectionStreamSystemSpec.java
----------------------------------------------------------------------
diff --git a/samza-test/src/main/java/org/apache/samza/test/framework/system/CollectionStreamSystemSpec.java b/samza-test/src/main/java/org/apache/samza/test/framework/system/CollectionStreamSystemSpec.java
deleted file mode 100644
index 5658f61..0000000
--- a/samza-test/src/main/java/org/apache/samza/test/framework/system/CollectionStreamSystemSpec.java
+++ /dev/null
@@ -1,90 +0,0 @@
-/*
- * 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.test.framework.system;
-
-import com.google.common.base.Preconditions;
-import java.util.HashMap;
-import java.util.Map;
-import org.apache.commons.lang3.StringUtils;
-import org.apache.samza.system.inmemory.InMemorySystemFactory;
-
-
-/**
- * CollectionStreamSystem represents a system that interacts with an underlying {@link InMemorySystemFactory} to create
- * various input and output streams and initialize {@link org.apache.samza.system.SystemStreamPartition} with messages
- * <p>
- * Following system level configs are set by default
- * <ol>
- *   <li>"systems.%s.default.stream.samza.offset.default" = "oldest"</li>
- *   <li>"jobs.job-name.systems.%s.default.stream.samza.offset.default" = "oldest"</li>
- *   <li>"systems.%s.samza.factory" = {@link InMemorySystemFactory}</li>
- *   <li>"jobs.job-name.systems.%s.samza.factory" = {@link InMemorySystemFactory}</li>
- * </ol>
- * The "systems.*" configs are required since the planner uses the system to get metadata about streams during
- * planning. The "jobs.job-name.systems.*" configs are required since configs generated from user provided
- * system/stream descriptors override configs originally supplied to the planner. Configs in the "jobs.job-name.*"
- * scope have the highest precedence.
- */
-public class CollectionStreamSystemSpec {
-  private static final String CONFIG_OVERRIDE_PREFIX = "jobs.%s."; // prefix to override configs generated by the planner
-  private static final String SYSTEM_FACTORY = "systems.%s.samza.factory";
-  private static final String SYSTEM_OFFSET = "systems.%s.default.stream.samza.offset.default";
-
-  private String systemName;
-  private Map<String, String> systemConfigs;
-
-  /**
-   * Constructs a new CollectionStreamSystem from specified components.
-   * <p>
-   * Every {@link CollectionStreamSystemSpec} is assumed to consume from the oldest offset, since stream is in memory and
-   * is used for testing purpose. System uses {@link InMemorySystemFactory} to initialize in memory streams.
-   * <p>
-   * @param systemName represents unique name of the system
-   */
-  private CollectionStreamSystemSpec(String systemName, String jobName) {
-    this.systemName = systemName;
-    systemConfigs = new HashMap<String, String>();
-    systemConfigs.put(String.format(SYSTEM_FACTORY, systemName), InMemorySystemFactory.class.getName());
-    systemConfigs.put(String.format(CONFIG_OVERRIDE_PREFIX + SYSTEM_FACTORY, jobName, systemName), InMemorySystemFactory.class.getName());
-    systemConfigs.put(String.format(SYSTEM_OFFSET, systemName), "oldest");
-    systemConfigs.put(String.format(CONFIG_OVERRIDE_PREFIX + SYSTEM_OFFSET, jobName, systemName), "oldest");
-  }
-
-  public String getSystemName() {
-    return systemName;
-  }
-
-  public Map<String, String> getSystemConfigs() {
-    return systemConfigs;
-  }
-
-  /**
-   * Creates a {@link CollectionStreamSystemSpec} with name {@code systemName}
-   * @param systemName represents name of the {@link CollectionStreamSystemSpec}
-   * @param jobName name of the job
-   * @return an instance of {@link CollectionStreamSystemSpec}
-   */
-  public static CollectionStreamSystemSpec create(String systemName, String jobName) {
-    Preconditions.checkState(StringUtils.isNotBlank(systemName));
-    Preconditions.checkState(StringUtils.isNotBlank(jobName));
-    return new CollectionStreamSystemSpec(systemName, jobName);
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/samza/blob/1755268c/samza-test/src/main/java/org/apache/samza/test/framework/system/InMemoryInputDescriptor.java
----------------------------------------------------------------------
diff --git a/samza-test/src/main/java/org/apache/samza/test/framework/system/InMemoryInputDescriptor.java b/samza-test/src/main/java/org/apache/samza/test/framework/system/InMemoryInputDescriptor.java
new file mode 100644
index 0000000..6065bf0
--- /dev/null
+++ b/samza-test/src/main/java/org/apache/samza/test/framework/system/InMemoryInputDescriptor.java
@@ -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.test.framework.system;
+
+import org.apache.samza.operators.descriptors.base.stream.InputDescriptor;
+import org.apache.samza.serializers.NoOpSerde;
+
+/**
+ * A descriptor for an in memory stream of messages that can either have single or multiple partitions.
+ * <p>
+ *  An instance of this descriptor may be obtained from an appropriately configured {@link InMemorySystemDescriptor}.
+ * <p>
+ * @param <StreamMessageType> type of messages in input stream
+ */
+public class InMemoryInputDescriptor<StreamMessageType>
+    extends InputDescriptor<StreamMessageType, InMemoryInputDescriptor<StreamMessageType>> {
+  /**
+   * Constructs a new InMemoryInputDescriptor from specified components.
+   * @param systemDescriptor name of the system stream is associated with
+   * @param streamId name of the stream
+   */
+  InMemoryInputDescriptor(String streamId, InMemorySystemDescriptor systemDescriptor) {
+    super(streamId, new NoOpSerde<>(), systemDescriptor, null);
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/1755268c/samza-test/src/main/java/org/apache/samza/test/framework/system/InMemoryOutputDescriptor.java
----------------------------------------------------------------------
diff --git a/samza-test/src/main/java/org/apache/samza/test/framework/system/InMemoryOutputDescriptor.java b/samza-test/src/main/java/org/apache/samza/test/framework/system/InMemoryOutputDescriptor.java
new file mode 100644
index 0000000..75fe7ae
--- /dev/null
+++ b/samza-test/src/main/java/org/apache/samza/test/framework/system/InMemoryOutputDescriptor.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.test.framework.system;
+
+import org.apache.samza.operators.descriptors.base.stream.OutputDescriptor;
+import org.apache.samza.operators.descriptors.base.system.SystemDescriptor;
+import org.apache.samza.serializers.NoOpSerde;
+
+/**
+ * A descriptor for an in memory output stream.
+ * <p>
+ * An instance of this descriptor may be obtained from an appropriately configured {@link InMemorySystemDescriptor}.
+ * <p>
+ * Stream properties configured using a descriptor override corresponding properties provided in configuration.
+ *
+ * @param <StreamMessageType> type of messages in this stream.
+ */
+public class InMemoryOutputDescriptor<StreamMessageType>
+    extends OutputDescriptor<StreamMessageType, InMemoryOutputDescriptor<StreamMessageType>> {
+
+  /**
+   * Constructs an {@link OutputDescriptor} instance.
+   * @param streamId id of the stream
+   * @param systemDescriptor system descriptor this stream descriptor was obtained from
+   */
+  InMemoryOutputDescriptor(String streamId, SystemDescriptor systemDescriptor) {
+    super(streamId, new NoOpSerde<>(), systemDescriptor);
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/1755268c/samza-test/src/main/java/org/apache/samza/test/framework/system/InMemorySystemDescriptor.java
----------------------------------------------------------------------
diff --git a/samza-test/src/main/java/org/apache/samza/test/framework/system/InMemorySystemDescriptor.java b/samza-test/src/main/java/org/apache/samza/test/framework/system/InMemorySystemDescriptor.java
new file mode 100644
index 0000000..92b23ef
--- /dev/null
+++ b/samza-test/src/main/java/org/apache/samza/test/framework/system/InMemorySystemDescriptor.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.test.framework.system;
+
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.samza.config.InMemorySystemConfig;
+import org.apache.samza.operators.descriptors.base.system.OutputDescriptorProvider;
+import org.apache.samza.operators.descriptors.base.system.SimpleInputDescriptorProvider;
+import org.apache.samza.operators.descriptors.base.system.SystemDescriptor;
+import org.apache.samza.serializers.Serde;
+import org.apache.samza.system.SystemStreamMetadata;
+import org.apache.samza.system.inmemory.InMemorySystemFactory;
+import org.apache.samza.config.JavaSystemConfig;
+import org.apache.samza.test.framework.TestRunner;
+
+
+/**
+ * A descriptor for InMemorySystem.
+ * System properties configured using a descriptor override corresponding properties provided in configuration.
+ * <p>
+ * Following system level configs are set by default
+ * <ol>
+ *   <li>"systems.%s.default.stream.samza.offset.default" = "oldest"</li>
+ *   <li>"systems.%s.samza.factory" = {@link InMemorySystemFactory}</li>
+ *   <li>"inmemory.scope = "Scope id generated to isolate the system in memory</li>
+ * </ol>
+ */
+public class InMemorySystemDescriptor extends SystemDescriptor<InMemorySystemDescriptor>
+    implements SimpleInputDescriptorProvider, OutputDescriptorProvider {
+  private static final String FACTORY_CLASS_NAME = InMemorySystemFactory.class.getName();
+  /**
+   * <p>
+   * The "systems.*" configs are required since the planner uses the system to get metadata about streams during
+   * planning. The "jobs.job-name.systems.*" configs are required since configs generated from user provided
+   * system/stream descriptors override configs originally supplied to the planner. Configs in the "jobs.job-name.*"
+   * scope have the highest precedence.
+   *
+   * For this case, it generates following overridden configs
+   * <ol>
+   *      <li>"jobs.<job-name>.systems.%s.default.stream.samza.offset.default" = "oldest"</li>
+   *      <li>"jobs.<job-name>.systems.%s.samza.factory" = {@link InMemorySystemFactory}</li>
+   * </ol>
+   *
+   **/
+  private static final String CONFIG_OVERRIDE_PREFIX = "jobs.%s.";
+  private static final String DEFAULT_STREAM_OFFSET_DEFAULT_CONFIG_KEY = "systems.%s.default.stream.samza.offset.default";
+
+  private String inMemoryScope;
+
+  /**
+   * Constructs a new InMemorySystemDescriptor from specified components.
+   * <p>
+   * Every {@link InMemorySystemDescriptor} is configured to consume from the oldest offset, since stream is in memory and
+   * is used for testing purpose. System uses {@link InMemorySystemFactory} to initialize in memory streams.
+   * <p>
+   * @param systemName unique name of the system
+   */
+  public InMemorySystemDescriptor(String systemName) {
+    super(systemName, FACTORY_CLASS_NAME, null, null);
+    this.withDefaultStreamOffsetDefault(SystemStreamMetadata.OffsetType.OLDEST);
+  }
+
+  @Override
+  public <StreamMessageType> InMemoryInputDescriptor<StreamMessageType> getInputDescriptor(
+      String streamId, Serde<StreamMessageType> serde) {
+    return new InMemoryInputDescriptor<StreamMessageType>(streamId, this);
+  }
+
+  @Override
+  public <StreamMessageType> InMemoryOutputDescriptor<StreamMessageType> getOutputDescriptor(
+      String streamId, Serde<StreamMessageType> serde) {
+    return new InMemoryOutputDescriptor<StreamMessageType>(streamId, this);
+  }
+
+  /**
+   * {@code inMemoryScope} defines the unique instance of InMemorySystem, that this system uses
+   * This method is framework use only, users are not supposed to use it
+   *
+   * @param inMemoryScope acts as a unique global identifier for this instance of InMemorySystem
+   * @return this system descriptor
+   */
+  public InMemorySystemDescriptor withInMemoryScope(String inMemoryScope) {
+    this.inMemoryScope = inMemoryScope;
+    return this;
+  }
+
+  @Override
+  public Map<String, String> toConfig() {
+    HashMap<String, String> configs = new HashMap<>(super.toConfig());
+    configs.put(InMemorySystemConfig.INMEMORY_SCOPE, this.inMemoryScope);
+    configs.put(String.format(CONFIG_OVERRIDE_PREFIX + JavaSystemConfig.SYSTEM_FACTORY_FORMAT, TestRunner.JOB_NAME, getSystemName()),
+        FACTORY_CLASS_NAME);
+    configs.put(
+        String.format(CONFIG_OVERRIDE_PREFIX + DEFAULT_STREAM_OFFSET_DEFAULT_CONFIG_KEY, TestRunner.JOB_NAME,
+            getSystemName()), SystemStreamMetadata.OffsetType.OLDEST.toString());
+    return configs;
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/samza/blob/1755268c/samza-test/src/test/java/org/apache/samza/test/framework/AsyncStreamTaskIntegrationTest.java
----------------------------------------------------------------------
diff --git a/samza-test/src/test/java/org/apache/samza/test/framework/AsyncStreamTaskIntegrationTest.java b/samza-test/src/test/java/org/apache/samza/test/framework/AsyncStreamTaskIntegrationTest.java
index 3a1eba0..581b1c3 100644
--- a/samza-test/src/test/java/org/apache/samza/test/framework/AsyncStreamTaskIntegrationTest.java
+++ b/samza-test/src/test/java/org/apache/samza/test/framework/AsyncStreamTaskIntegrationTest.java
@@ -27,7 +27,10 @@ import java.util.List;
 import java.util.Map;
 import java.util.stream.Collectors;
 import org.apache.samza.operators.KV;
-import org.apache.samza.test.framework.stream.CollectionStream;
+import org.apache.samza.serializers.NoOpSerde;
+import org.apache.samza.test.framework.system.InMemoryInputDescriptor;
+import org.apache.samza.test.framework.system.InMemoryOutputDescriptor;
+import org.apache.samza.test.framework.system.InMemorySystemDescriptor;
 import org.hamcrest.collection.IsIterableContainingInOrder;
 import org.junit.Assert;
 import org.junit.Test;
@@ -40,16 +43,21 @@ public class AsyncStreamTaskIntegrationTest {
     List<Integer> inputList = Arrays.asList(1, 2, 3, 4, 5);
     List<Integer> outputList = Arrays.asList(10, 20, 30, 40, 50);
 
-    CollectionStream<Integer> input = CollectionStream.of("async-test", "ints", inputList);
-    CollectionStream output = CollectionStream.empty("async-test", "ints-out");
+    InMemorySystemDescriptor isd = new InMemorySystemDescriptor("async-test");
+
+    InMemoryInputDescriptor<Integer> imid = isd
+        .getInputDescriptor("ints", new NoOpSerde<Integer>());
+
+    InMemoryOutputDescriptor imod = isd
+        .getOutputDescriptor("ints-out", new NoOpSerde<>());
 
     TestRunner
         .of(MyAsyncStreamTask.class)
-        .addInputStream(input)
-        .addOutputStream(output)
+        .addInputStream(imid, inputList)
+        .addOutputStream(imod, 1)
         .run(Duration.ofSeconds(2));
 
-    Assert.assertThat(TestRunner.consumeStream(output, Duration.ofMillis(1000)).get(0),
+    Assert.assertThat(TestRunner.consumeStream(imod, Duration.ofMillis(1000)).get(0),
         IsIterableContainingInOrder.contains(outputList.toArray()));
   }
 
@@ -58,49 +66,70 @@ public class AsyncStreamTaskIntegrationTest {
     List<Integer> inputList = Arrays.asList(1, 2, 3, 4, 5);
     List<Integer> outputList = Arrays.asList(50, 10, 20, 30, 40);
 
-    CollectionStream<Integer> input = CollectionStream.of("async-test", "ints", inputList);
-    CollectionStream output = CollectionStream.empty("async-test", "ints-out");
+    InMemorySystemDescriptor isd = new InMemorySystemDescriptor("async-test");
+
+    InMemoryInputDescriptor<Integer> imid = isd
+        .getInputDescriptor("ints", new NoOpSerde<Integer>());
+
+    InMemoryOutputDescriptor imod = isd
+        .getOutputDescriptor("ints-out", new NoOpSerde<>());
 
     TestRunner
         .of(MyAsyncStreamTask.class)
-        .addInputStream(input)
-        .addOutputStream(output)
+        .addInputStream(imid, inputList)
+        .addOutputStream(imod, 1)
         .run(Duration.ofSeconds(2));
 
-    StreamAssert.containsInAnyOrder(output, outputList, Duration.ofMillis(1000));
+    StreamAssert.containsInAnyOrder(outputList, imod, Duration.ofMillis(1000));
   }
 
   @Test
   public void testAsyncTaskWithMultiplePartition() throws Exception {
     Map<Integer, List<KV>> inputPartitionData = new HashMap<>();
     Map<Integer, List<Integer>> expectedOutputPartitionData = new HashMap<>();
-    List<Integer> partition = Arrays.asList(1, 2, 3, 4, 5);
-    List<Integer> outputPartition = partition.stream().map(x -> x * 10).collect(Collectors.toList());
-    for (int i = 0; i < 5; i++) {
-      List<KV> keyedPartition = new ArrayList<>();
-      for (Integer val : partition) {
-        keyedPartition.add(KV.of(i, val));
-      }
-      inputPartitionData.put(i, keyedPartition);
-      expectedOutputPartitionData.put(i, new ArrayList<Integer>(outputPartition));
-    }
+    genData(inputPartitionData, expectedOutputPartitionData);
 
-    CollectionStream<KV> inputStream = CollectionStream.of("async-test", "ints", inputPartitionData);
-    CollectionStream outputStream = CollectionStream.empty("async-test", "ints-out", 5);
+    InMemorySystemDescriptor isd = new InMemorySystemDescriptor("async-test");
+
+    InMemoryInputDescriptor<KV> imid = isd
+        .getInputDescriptor("ints", new NoOpSerde<KV>());
+    InMemoryOutputDescriptor imod = isd
+        .getOutputDescriptor("ints-out", new NoOpSerde<>());
 
     TestRunner
         .of(MyAsyncStreamTask.class)
-        .addInputStream(inputStream)
-        .addOutputStream(outputStream)
+        .addInputStream(imid, inputPartitionData)
+        .addOutputStream(imod, 5)
         .run(Duration.ofSeconds(2));
 
-    StreamAssert.containsInOrder(outputStream, expectedOutputPartitionData, Duration.ofMillis(1000));
+    StreamAssert.containsInOrder(expectedOutputPartitionData, imod, Duration.ofMillis(1000));
   }
 
   @Test
   public void testAsyncTaskWithMultiplePartitionMultithreaded() throws Exception {
     Map<Integer, List<KV>> inputPartitionData = new HashMap<>();
     Map<Integer, List<Integer>> expectedOutputPartitionData = new HashMap<>();
+    genData(inputPartitionData, expectedOutputPartitionData);
+
+    InMemorySystemDescriptor isd = new InMemorySystemDescriptor("async-test");
+
+    InMemoryInputDescriptor<KV> imid = isd
+        .getInputDescriptor("ints", new NoOpSerde<>());
+
+    InMemoryOutputDescriptor imod = isd
+        .getOutputDescriptor("ints-out", new NoOpSerde<>());
+
+    TestRunner
+        .of(MyAsyncStreamTask.class)
+        .addInputStream(imid, inputPartitionData)
+        .addOutputStream(imod, 5)
+        .addOverrideConfig("task.max.concurrency", "4")
+        .run(Duration.ofSeconds(2));
+
+    StreamAssert.containsInAnyOrder(expectedOutputPartitionData, imod, Duration.ofMillis(1000));
+  }
+
+  public void genData(Map<Integer, List<KV>> inputPartitionData, Map<Integer, List<Integer>> expectedOutputPartitionData) {
     List<Integer> partition = Arrays.asList(1, 2, 3, 4, 5);
     List<Integer> outputPartition = partition.stream().map(x -> x * 10).collect(Collectors.toList());
     for (int i = 0; i < 5; i++) {
@@ -111,18 +140,6 @@ public class AsyncStreamTaskIntegrationTest {
       inputPartitionData.put(i, keyedPartition);
       expectedOutputPartitionData.put(i, new ArrayList<Integer>(outputPartition));
     }
-
-    CollectionStream<KV> inputStream = CollectionStream.of("async-test", "ints", inputPartitionData);
-    CollectionStream outputStream = CollectionStream.empty("async-test", "ints-out", 5);
-
-    TestRunner
-        .of(MyAsyncStreamTask.class)
-        .addInputStream(inputStream)
-        .addOutputStream(outputStream)
-        .addOverrideConfig("task.max.concurrency", "4")
-        .run(Duration.ofSeconds(2));
-
-    StreamAssert.containsInAnyOrder(outputStream, expectedOutputPartitionData, Duration.ofMillis(1000));
   }
 
   /**
@@ -130,15 +147,20 @@ public class AsyncStreamTaskIntegrationTest {
    */
   @Test(expected = AssertionError.class)
   public void testSamzaJobTimeoutFailureForAsyncTask() {
-    List<Integer> inputList = Arrays.asList(1, 2, 3, 4);
+    InMemorySystemDescriptor isd = new InMemorySystemDescriptor("async-test");
+
+    InMemoryInputDescriptor<Integer> imid = isd
+        .getInputDescriptor("ints", new NoOpSerde<>());
 
-    CollectionStream<Integer> input = CollectionStream.of("async-test", "ints", inputList);
-    CollectionStream output = CollectionStream.empty("async-test", "ints-out");
+    InMemoryOutputDescriptor imod = isd
+        .getOutputDescriptor("ints-out", new NoOpSerde<>());
 
     TestRunner
         .of(MyAsyncStreamTask.class)
-        .addInputStream(input)
-        .addOutputStream(output)
+        .addInputStream(imid, Arrays.asList(1, 2, 3, 4))
+        .addOutputStream(imod, 1)
         .run(Duration.ofMillis(1));
   }
+
+
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/1755268c/samza-test/src/test/java/org/apache/samza/test/framework/StreamApplicationIntegrationTest.java
----------------------------------------------------------------------
diff --git a/samza-test/src/test/java/org/apache/samza/test/framework/StreamApplicationIntegrationTest.java b/samza-test/src/test/java/org/apache/samza/test/framework/StreamApplicationIntegrationTest.java
index 1000f22..6dd9159 100644
--- a/samza-test/src/test/java/org/apache/samza/test/framework/StreamApplicationIntegrationTest.java
+++ b/samza-test/src/test/java/org/apache/samza/test/framework/StreamApplicationIntegrationTest.java
@@ -36,7 +36,9 @@ import org.apache.samza.system.SystemStream;
 import org.apache.samza.system.kafka.KafkaInputDescriptor;
 import org.apache.samza.system.kafka.KafkaSystemDescriptor;
 import org.apache.samza.test.controlmessages.TestData;
-import org.apache.samza.test.framework.stream.CollectionStream;
+import org.apache.samza.test.framework.system.InMemoryInputDescriptor;
+import org.apache.samza.test.framework.system.InMemoryOutputDescriptor;
+import org.apache.samza.test.framework.system.InMemorySystemDescriptor;
 import org.junit.Assert;
 import org.junit.Test;
 
@@ -82,17 +84,22 @@ public class StreamApplicationIntegrationTest {
       pageviews.add(pv);
     }
 
-    CollectionStream<PageView> input = CollectionStream.of("test", "PageView", pageviews);
-    CollectionStream output = CollectionStream.empty("test", "Output", 10);
+    InMemorySystemDescriptor isd = new InMemorySystemDescriptor("test");
+
+    InMemoryInputDescriptor<PageView> imid = isd
+        .getInputDescriptor("PageView", new NoOpSerde<PageView>());
+
+    InMemoryOutputDescriptor<PageView> imod = isd
+        .getOutputDescriptor("Output", new NoOpSerde<PageView>());
 
     TestRunner
         .of(pageViewRepartition)
-        .addInputStream(input)
-        .addOutputStream(output)
+        .addInputStream(imid, pageviews)
+        .addOutputStream(imod, 10)
         .addOverrideConfig("job.default.system", "test")
         .run(Duration.ofMillis(1500));
 
-    Assert.assertEquals(TestRunner.consumeStream(output, Duration.ofMillis(1000)).get(random.nextInt(count)).size(), 1);
+    Assert.assertEquals(TestRunner.consumeStream(imod, Duration.ofMillis(1000)).get(random.nextInt(count)).size(), 1);
   }
 
   public static final class Values {
@@ -107,13 +114,18 @@ public class StreamApplicationIntegrationTest {
   @Test(expected = SamzaException.class)
   public void testSamzaJobStartMissingConfigFailureForStreamApplication() {
 
-    CollectionStream<TestData.PageView> input = CollectionStream.of("test", "PageView", new ArrayList<>());
-    CollectionStream output = CollectionStream.empty("test", "Output", 10);
+    InMemorySystemDescriptor isd = new InMemorySystemDescriptor("test");
+
+    InMemoryInputDescriptor<PageView> imid = isd
+        .getInputDescriptor("PageView", new NoOpSerde<PageView>());
+
+    InMemoryOutputDescriptor<PageView> imod = isd
+        .getOutputDescriptor("Output", new NoOpSerde<PageView>());
 
     TestRunner
         .of(pageViewRepartition)
-        .addInputStream(input)
-        .addOutputStream(output)
+        .addInputStream(imid, new ArrayList<>())
+        .addOutputStream(imod, 10)
         .run(Duration.ofMillis(1000));
   }
 
@@ -131,12 +143,17 @@ public class StreamApplicationIntegrationTest {
       pageviews.add(new TestData.PageView(null, memberId));
     }
 
-    CollectionStream<TestData.PageView> input = CollectionStream.of("test", "PageView", pageviews);
-    CollectionStream output = CollectionStream.empty("test", "Output", 1);
+    InMemorySystemDescriptor isd = new InMemorySystemDescriptor("test");
+
+    InMemoryInputDescriptor<PageView> imid = isd
+        .getInputDescriptor("PageView", new NoOpSerde<PageView>());
+
+    InMemoryOutputDescriptor<PageView> imod = isd
+        .getOutputDescriptor("Output", new NoOpSerde<PageView>());
 
     TestRunner.of(pageViewFilter)
-        .addInputStream(input)
-        .addOutputStream(output)
+        .addInputStream(imid, pageviews)
+        .addOutputStream(imod, 10)
         .addOverrideConfig("job.default.system", "test")
         .run(Duration.ofMillis(1000));
   }

http://git-wip-us.apache.org/repos/asf/samza/blob/1755268c/samza-test/src/test/java/org/apache/samza/test/framework/StreamTaskIntegrationTest.java
----------------------------------------------------------------------
diff --git a/samza-test/src/test/java/org/apache/samza/test/framework/StreamTaskIntegrationTest.java b/samza-test/src/test/java/org/apache/samza/test/framework/StreamTaskIntegrationTest.java
index f888b4a..0580598 100644
--- a/samza-test/src/test/java/org/apache/samza/test/framework/StreamTaskIntegrationTest.java
+++ b/samza-test/src/test/java/org/apache/samza/test/framework/StreamTaskIntegrationTest.java
@@ -28,7 +28,10 @@ import java.util.Map;
 import java.util.stream.Collectors;
 import org.apache.samza.SamzaException;
 import org.apache.samza.operators.KV;
-import org.apache.samza.test.framework.stream.CollectionStream;
+import org.apache.samza.serializers.NoOpSerde;
+import org.apache.samza.test.framework.system.InMemoryInputDescriptor;
+import org.apache.samza.test.framework.system.InMemoryOutputDescriptor;
+import org.apache.samza.test.framework.system.InMemorySystemDescriptor;
 import org.hamcrest.collection.IsIterableContainingInOrder;
 import org.junit.Assert;
 import org.junit.Test;
@@ -41,13 +44,23 @@ public class StreamTaskIntegrationTest {
     List<Integer> inputList = Arrays.asList(1, 2, 3, 4, 5);
     List<Integer> outputList = Arrays.asList(10, 20, 30, 40, 50);
 
-    CollectionStream<Integer> input = CollectionStream.of("test", "input", inputList);
-    CollectionStream output = CollectionStream.empty("test", "output");
+    InMemorySystemDescriptor isd = new InMemorySystemDescriptor("test");
 
-    TestRunner.of(MyStreamTestTask.class).addInputStream(input).addOutputStream(output).run(Duration.ofSeconds(1));
+    InMemoryInputDescriptor<Integer> imid = isd
+        .getInputDescriptor("input", new NoOpSerde<Integer>());
 
-    Assert.assertThat(TestRunner.consumeStream(output, Duration.ofMillis(1000)).get(0),
+    InMemoryOutputDescriptor<Integer> imod = isd
+        .getOutputDescriptor("output", new NoOpSerde<Integer>());
+
+    TestRunner
+        .of(MyStreamTestTask.class)
+        .addInputStream(imid, inputList)
+        .addOutputStream(imod, 1)
+        .run(Duration.ofSeconds(1));
+
+    Assert.assertThat(TestRunner.consumeStream(imod, Duration.ofMillis(1000)).get(0),
         IsIterableContainingInOrder.contains(outputList.toArray()));
+
   }
 
   /**
@@ -57,10 +70,19 @@ public class StreamTaskIntegrationTest {
   public void testSamzaJobFailureForSyncTask() {
     List<Double> inputList = Arrays.asList(1.2, 2.3, 3.33, 4.5);
 
-    CollectionStream<Double> input = CollectionStream.of("test", "doubles", inputList);
-    CollectionStream output = CollectionStream.empty("test", "output");
+    InMemorySystemDescriptor isd = new InMemorySystemDescriptor("test");
 
-    TestRunner.of(MyStreamTestTask.class).addInputStream(input).addOutputStream(output).run(Duration.ofSeconds(1));
+    InMemoryInputDescriptor<Double> imid = isd
+        .getInputDescriptor("doubles", new NoOpSerde<Double>());
+
+    InMemoryOutputDescriptor imod = isd
+        .getOutputDescriptor("output", new NoOpSerde<>());
+
+    TestRunner
+        .of(MyStreamTestTask.class)
+        .addInputStream(imid, inputList)
+        .addOutputStream(imod, 1)
+        .run(Duration.ofSeconds(1));
   }
 
   @Test
@@ -68,50 +90,72 @@ public class StreamTaskIntegrationTest {
     List<Integer> inputList = Arrays.asList(1, 2, 3, 4, 5);
     List<Integer> outputList = Arrays.asList(10, 20, 30, 40, 50);
 
-    CollectionStream<Integer> input = CollectionStream.of("test", "input", inputList);
-    CollectionStream output = CollectionStream.empty("test", "output");
+    InMemorySystemDescriptor isd = new InMemorySystemDescriptor("test");
+
+    InMemoryInputDescriptor<Integer> imid = isd
+        .getInputDescriptor("input", new NoOpSerde<Integer>());
+
+    InMemoryOutputDescriptor<Integer> imod = isd
+        .getOutputDescriptor("output", new NoOpSerde<Integer>());
 
     TestRunner
         .of(MyStreamTestTask.class)
-        .addInputStream(input)
-        .addOutputStream(output)
+        .addInputStream(imid, inputList)
+        .addOutputStream(imod, 1)
         .addOverrideConfig("job.container.thread.pool.size", "4")
         .run(Duration.ofSeconds(1));
 
-    StreamAssert.containsInOrder(output, outputList, Duration.ofMillis(1000));
+    StreamAssert.containsInOrder(outputList, imod, Duration.ofMillis(1000));
   }
 
   @Test
   public void testSyncTaskWithMultiplePartition() throws Exception {
     Map<Integer, List<KV>> inputPartitionData = new HashMap<>();
     Map<Integer, List<Integer>> expectedOutputPartitionData = new HashMap<>();
-    List<Integer> partition = Arrays.asList(1, 2, 3, 4, 5);
-    List<Integer> outputPartition = partition.stream().map(x -> x * 10).collect(Collectors.toList());
-    for (int i = 0; i < 5; i++) {
-      List<KV> keyedPartition = new ArrayList<>();
-      for (Integer val : partition) {
-        keyedPartition.add(KV.of(i, val));
-      }
-      inputPartitionData.put(i, keyedPartition);
-      expectedOutputPartitionData.put(i, new ArrayList<Integer>(outputPartition));
-    }
+    genData(inputPartitionData, expectedOutputPartitionData);
+
+    InMemorySystemDescriptor isd = new InMemorySystemDescriptor("test");
+
+    InMemoryInputDescriptor<KV> imid = isd
+        .getInputDescriptor("input", new NoOpSerde<KV>());
 
-    CollectionStream<KV> inputStream = CollectionStream.of("test", "input", inputPartitionData);
-    CollectionStream outputStream = CollectionStream.empty("test", "output", 5);
+    InMemoryOutputDescriptor<Integer> imod = isd
+        .getOutputDescriptor("output", new NoOpSerde<Integer>());
 
     TestRunner
         .of(MyStreamTestTask.class)
-        .addInputStream(inputStream)
-        .addOutputStream(outputStream)
+        .addInputStream(imid, inputPartitionData)
+        .addOutputStream(imod, 5)
         .run(Duration.ofSeconds(2));
 
-    StreamAssert.containsInOrder(outputStream, expectedOutputPartitionData, Duration.ofMillis(1000));
+    StreamAssert.containsInOrder(expectedOutputPartitionData, imod, Duration.ofMillis(1000));
   }
 
   @Test
   public void testSyncTaskWithMultiplePartitionMultithreaded() throws Exception {
     Map<Integer, List<KV>> inputPartitionData = new HashMap<>();
     Map<Integer, List<Integer>> expectedOutputPartitionData = new HashMap<>();
+    genData(inputPartitionData, expectedOutputPartitionData);
+
+    InMemorySystemDescriptor isd = new InMemorySystemDescriptor("test");
+
+    InMemoryInputDescriptor<KV> imid = isd
+        .getInputDescriptor("input", new NoOpSerde<KV>());
+
+    InMemoryOutputDescriptor<Integer> imod = isd
+        .getOutputDescriptor("output", new NoOpSerde<Integer>());
+
+    TestRunner
+        .of(MyStreamTestTask.class)
+        .addInputStream(imid, inputPartitionData)
+        .addOutputStream(imod, 5)
+        .addOverrideConfig("job.container.thread.pool.size", "4")
+        .run(Duration.ofSeconds(2));
+
+    StreamAssert.containsInOrder(expectedOutputPartitionData, imod, Duration.ofMillis(1000));
+  }
+
+  public void genData(Map<Integer, List<KV>> inputPartitionData, Map<Integer, List<Integer>> expectedOutputPartitionData) {
     List<Integer> partition = Arrays.asList(1, 2, 3, 4, 5);
     List<Integer> outputPartition = partition.stream().map(x -> x * 10).collect(Collectors.toList());
     for (int i = 0; i < 5; i++) {
@@ -122,17 +166,5 @@ public class StreamTaskIntegrationTest {
       inputPartitionData.put(i, keyedPartition);
       expectedOutputPartitionData.put(i, new ArrayList<Integer>(outputPartition));
     }
-
-    CollectionStream<KV> inputStream = CollectionStream.of("test", "input", inputPartitionData);
-    CollectionStream outputStream = CollectionStream.empty("test", "output", 5);
-
-    TestRunner
-        .of(MyStreamTestTask.class)
-        .addInputStream(inputStream)
-        .addOutputStream(outputStream)
-        .addOverrideConfig("job.container.thread.pool.size", "4")
-        .run(Duration.ofSeconds(2));
-
-    StreamAssert.containsInOrder(outputStream, expectedOutputPartitionData, Duration.ofMillis(1000));
   }
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/1755268c/samza-test/src/test/java/org/apache/samza/test/table/TestLocalTableWithSideInputs.java
----------------------------------------------------------------------
diff --git a/samza-test/src/test/java/org/apache/samza/test/table/TestLocalTableWithSideInputs.java b/samza-test/src/test/java/org/apache/samza/test/table/TestLocalTableWithSideInputs.java
index 5c067ad..adcea48 100644
--- a/samza-test/src/test/java/org/apache/samza/test/table/TestLocalTableWithSideInputs.java
+++ b/samza-test/src/test/java/org/apache/samza/test/table/TestLocalTableWithSideInputs.java
@@ -27,6 +27,7 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.stream.Collectors;
+import org.apache.samza.SamzaException;
 import org.apache.samza.application.StreamApplicationDescriptor;
 import org.apache.samza.application.StreamApplication;
 import org.apache.samza.config.JobConfig;
@@ -44,7 +45,9 @@ import org.apache.samza.storage.kv.inmemory.InMemoryTableDescriptor;
 import org.apache.samza.system.kafka.KafkaSystemDescriptor;
 import org.apache.samza.table.Table;
 import org.apache.samza.test.framework.TestRunner;
-import org.apache.samza.test.framework.stream.CollectionStream;
+import org.apache.samza.test.framework.system.InMemoryInputDescriptor;
+import org.apache.samza.test.framework.system.InMemoryOutputDescriptor;
+import org.apache.samza.test.framework.system.InMemorySystemDescriptor;
 import org.apache.samza.test.harness.AbstractIntegrationTestHarness;
 import org.junit.Test;
 
@@ -84,25 +87,28 @@ public class TestLocalTableWithSideInputs extends AbstractIntegrationTestHarness
     configs.put(String.format(StreamConfig.SYSTEM_FOR_STREAM_ID(), ENRICHED_PAGEVIEW_STREAM), systemName);
     configs.put(JobConfig.JOB_DEFAULT_SYSTEM(), systemName);
 
-    CollectionStream<PageView> pageViewStream =
-        CollectionStream.of(systemName, PAGEVIEW_STREAM, pageViews);
-    CollectionStream<Profile> profileStream =
-        CollectionStream.of(systemName, PROFILE_STREAM, profiles);
+    InMemorySystemDescriptor isd = new InMemorySystemDescriptor(systemName);
 
-    CollectionStream<EnrichedPageView> outputStream =
-        CollectionStream.empty(systemName, ENRICHED_PAGEVIEW_STREAM);
+    InMemoryInputDescriptor<PageView> pageViewStreamDesc = isd
+        .getInputDescriptor(PAGEVIEW_STREAM, new NoOpSerde<PageView>());
+
+    InMemoryInputDescriptor<Profile> profileStreamDesc = isd
+        .getInputDescriptor(PROFILE_STREAM, new NoOpSerde<Profile>());
+
+    InMemoryOutputDescriptor<EnrichedPageView> outputStreamDesc = isd
+        .getOutputDescriptor(ENRICHED_PAGEVIEW_STREAM, new NoOpSerde<EnrichedPageView>());
 
     TestRunner
         .of(app)
-        .addInputStream(pageViewStream)
-        .addInputStream(profileStream)
-        .addOutputStream(outputStream)
+        .addInputStream(pageViewStreamDesc, pageViews)
+        .addInputStream(profileStreamDesc, profiles)
+        .addOutputStream(outputStreamDesc, 1)
         .addConfigs(new MapConfig(configs))
         .addOverrideConfig(ClusterManagerConfig.CLUSTER_MANAGER_HOST_AFFINITY_ENABLED, Boolean.FALSE.toString())
         .run(Duration.ofMillis(100000));
 
     try {
-      Map<Integer, List<EnrichedPageView>> result = TestRunner.consumeStream(outputStream, Duration.ofMillis(1000));
+      Map<Integer, List<EnrichedPageView>> result = TestRunner.consumeStream(outputStreamDesc, Duration.ofMillis(1000));
       List<EnrichedPageView> results = result.values().stream()
           .flatMap(List::stream)
           .collect(Collectors.toList());
@@ -117,7 +123,7 @@ public class TestLocalTableWithSideInputs extends AbstractIntegrationTestHarness
       assertEquals("Mismatch between the expected and actual join count", results.size(),
           expectedEnrichedPageviews.size());
       assertTrue("Pageview profile join did not succeed for all inputs", successfulJoin);
-    } catch (InterruptedException e) {
+    } catch (SamzaException e) {
       e.printStackTrace();
     }
   }
@@ -163,4 +169,4 @@ public class TestLocalTableWithSideInputs extends AbstractIntegrationTestHarness
             });
     }
   }
-}
+}
\ No newline at end of file


[13/29] samza git commit: SAMZA-1898: New UI layout for the Samza website

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/samza/blob/b8058af0/docs/js/jquery.tablesorter.min.js
----------------------------------------------------------------------
diff --git a/docs/js/jquery.tablesorter.min.js b/docs/js/jquery.tablesorter.min.js
deleted file mode 100644
index b8605df..0000000
--- a/docs/js/jquery.tablesorter.min.js
+++ /dev/null
@@ -1,4 +0,0 @@
-
-(function($){$.extend({tablesorter:new
-function(){var parsers=[],widgets=[];this.defaults={cssHeader:"header",cssAsc:"headerSortUp",cssDesc:"headerSortDown",cssChildRow:"expand-child",sortInitialOrder:"asc",sortMultiSortKey:"shiftKey",sortForce:null,sortAppend:null,sortLocaleCompare:true,textExtraction:"simple",parsers:{},widgets:[],widgetZebra:{css:["even","odd"]},headers:{},widthFixed:false,cancelSelection:true,sortList:[],headerList:[],dateFormat:"us",decimal:'/\.|\,/g',onRenderHeader:null,selectorHeaders:'thead th',debug:false};function benchmark(s,d){log(s+","+(new Date().getTime()-d.getTime())+"ms");}this.benchmark=benchmark;function log(s){if(typeof console!="undefined"&&typeof console.debug!="undefined"){console.log(s);}else{alert(s);}}function buildParserCache(table,$headers){if(table.config.debug){var parsersDebug="";}if(table.tBodies.length==0)return;var rows=table.tBodies[0].rows;if(rows[0]){var list=[],cells=rows[0].cells,l=cells.length;for(var i=0;i<l;i++){var p=false;if($.metadata&&($($headers[i]).metadata
 ()&&$($headers[i]).metadata().sorter)){p=getParserById($($headers[i]).metadata().sorter);}else if((table.config.headers[i]&&table.config.headers[i].sorter)){p=getParserById(table.config.headers[i].sorter);}if(!p){p=detectParserForColumn(table,rows,-1,i);}if(table.config.debug){parsersDebug+="column:"+i+" parser:"+p.id+"\n";}list.push(p);}}if(table.config.debug){log(parsersDebug);}return list;};function detectParserForColumn(table,rows,rowIndex,cellIndex){var l=parsers.length,node=false,nodeValue=false,keepLooking=true;while(nodeValue==''&&keepLooking){rowIndex++;if(rows[rowIndex]){node=getNodeFromRowAndCellIndex(rows,rowIndex,cellIndex);nodeValue=trimAndGetNodeText(table.config,node);if(table.config.debug){log('Checking if value was empty on row:'+rowIndex);}}else{keepLooking=false;}}for(var i=1;i<l;i++){if(parsers[i].is(nodeValue,table,node)){return parsers[i];}}return parsers[0];}function getNodeFromRowAndCellIndex(rows,rowIndex,cellIndex){return rows[rowIndex].cells[cellIndex];}f
 unction trimAndGetNodeText(config,node){return $.trim(getElementText(config,node));}function getParserById(name){var l=parsers.length;for(var i=0;i<l;i++){if(parsers[i].id.toLowerCase()==name.toLowerCase()){return parsers[i];}}return false;}function buildCache(table){if(table.config.debug){var cacheTime=new Date();}var totalRows=(table.tBodies[0]&&table.tBodies[0].rows.length)||0,totalCells=(table.tBodies[0].rows[0]&&table.tBodies[0].rows[0].cells.length)||0,parsers=table.config.parsers,cache={row:[],normalized:[]};for(var i=0;i<totalRows;++i){var c=$(table.tBodies[0].rows[i]),cols=[];if(c.hasClass(table.config.cssChildRow)){cache.row[cache.row.length-1]=cache.row[cache.row.length-1].add(c);continue;}cache.row.push(c);for(var j=0;j<totalCells;++j){cols.push(parsers[j].format(getElementText(table.config,c[0].cells[j]),table,c[0].cells[j]));}cols.push(cache.normalized.length);cache.normalized.push(cols);cols=null;};if(table.config.debug){benchmark("Building cache for "+totalRows+" row
 s:",cacheTime);}return cache;};function getElementText(config,node){var text="";if(!node)return"";if(!config.supportsTextContent)config.supportsTextContent=node.textContent||false;if(config.textExtraction=="simple"){if(config.supportsTextContent){text=node.textContent;}else{if(node.childNodes[0]&&node.childNodes[0].hasChildNodes()){text=node.childNodes[0].innerHTML;}else{text=node.innerHTML;}}}else{if(typeof(config.textExtraction)=="function"){text=config.textExtraction(node);}else{text=$(node).text();}}return text;}function appendToTable(table,cache){if(table.config.debug){var appendTime=new Date()}var c=cache,r=c.row,n=c.normalized,totalRows=n.length,checkCell=(n[0].length-1),tableBody=$(table.tBodies[0]),rows=[];for(var i=0;i<totalRows;i++){var pos=n[i][checkCell];rows.push(r[pos]);if(!table.config.appender){var l=r[pos].length;for(var j=0;j<l;j++){tableBody[0].appendChild(r[pos][j]);}}}if(table.config.appender){table.config.appender(table,rows);}rows=null;if(table.config.debug){
 benchmark("Rebuilt table:",appendTime);}applyWidget(table);setTimeout(function(){$(table).trigger("sortEnd");},0);};function buildHeaders(table){if(table.config.debug){var time=new Date();}var meta=($.metadata)?true:false;var header_index=computeTableHeaderCellIndexes(table);$tableHeaders=$(table.config.selectorHeaders,table).each(function(index){this.column=header_index[this.parentNode.rowIndex+"-"+this.cellIndex];this.order=formatSortingOrder(table.config.sortInitialOrder);this.count=this.order;if(checkHeaderMetadata(this)||checkHeaderOptions(table,index))this.sortDisabled=true;if(checkHeaderOptionsSortingLocked(table,index))this.order=this.lockedOrder=checkHeaderOptionsSortingLocked(table,index);if(!this.sortDisabled){var $th=$(this).addClass(table.config.cssHeader);if(table.config.onRenderHeader)table.config.onRenderHeader.apply($th);}table.config.headerList[index]=this;});if(table.config.debug){benchmark("Built headers:",time);log($tableHeaders);}return $tableHeaders;};function
  computeTableHeaderCellIndexes(t){var matrix=[];var lookup={};var thead=t.getElementsByTagName('THEAD')[0];var trs=thead.getElementsByTagName('TR');for(var i=0;i<trs.length;i++){var cells=trs[i].cells;for(var j=0;j<cells.length;j++){var c=cells[j];var rowIndex=c.parentNode.rowIndex;var cellId=rowIndex+"-"+c.cellIndex;var rowSpan=c.rowSpan||1;var colSpan=c.colSpan||1
-var firstAvailCol;if(typeof(matrix[rowIndex])=="undefined"){matrix[rowIndex]=[];}for(var k=0;k<matrix[rowIndex].length+1;k++){if(typeof(matrix[rowIndex][k])=="undefined"){firstAvailCol=k;break;}}lookup[cellId]=firstAvailCol;for(var k=rowIndex;k<rowIndex+rowSpan;k++){if(typeof(matrix[k])=="undefined"){matrix[k]=[];}var matrixrow=matrix[k];for(var l=firstAvailCol;l<firstAvailCol+colSpan;l++){matrixrow[l]="x";}}}}return lookup;}function checkCellColSpan(table,rows,row){var arr=[],r=table.tHead.rows,c=r[row].cells;for(var i=0;i<c.length;i++){var cell=c[i];if(cell.colSpan>1){arr=arr.concat(checkCellColSpan(table,headerArr,row++));}else{if(table.tHead.length==1||(cell.rowSpan>1||!r[row+1])){arr.push(cell);}}}return arr;};function checkHeaderMetadata(cell){if(($.metadata)&&($(cell).metadata().sorter===false)){return true;};return false;}function checkHeaderOptions(table,i){if((table.config.headers[i])&&(table.config.headers[i].sorter===false)){return true;};return false;}function checkHead
 erOptionsSortingLocked(table,i){if((table.config.headers[i])&&(table.config.headers[i].lockedOrder))return table.config.headers[i].lockedOrder;return false;}function applyWidget(table){var c=table.config.widgets;var l=c.length;for(var i=0;i<l;i++){getWidgetById(c[i]).format(table);}}function getWidgetById(name){var l=widgets.length;for(var i=0;i<l;i++){if(widgets[i].id.toLowerCase()==name.toLowerCase()){return widgets[i];}}};function formatSortingOrder(v){if(typeof(v)!="Number"){return(v.toLowerCase()=="desc")?1:0;}else{return(v==1)?1:0;}}function isValueInArray(v,a){var l=a.length;for(var i=0;i<l;i++){if(a[i][0]==v){return true;}}return false;}function setHeadersCss(table,$headers,list,css){$headers.removeClass(css[0]).removeClass(css[1]);var h=[];$headers.each(function(offset){if(!this.sortDisabled){h[this.column]=$(this);}});var l=list.length;for(var i=0;i<l;i++){h[list[i][0]].addClass(css[list[i][1]]);}}function fixColumnWidth(table,$headers){var c=table.config;if(c.widthFixed){
 var colgroup=$('<colgroup>');$("tr:first td",table.tBodies[0]).each(function(){colgroup.append($('<col>').css('width',$(this).width()));});$(table).prepend(colgroup);};}function updateHeaderSortCount(table,sortList){var c=table.config,l=sortList.length;for(var i=0;i<l;i++){var s=sortList[i],o=c.headerList[s[0]];o.count=s[1];o.count++;}}function multisort(table,sortList,cache){if(table.config.debug){var sortTime=new Date();}var dynamicExp="var sortWrapper = function(a,b) {",l=sortList.length;for(var i=0;i<l;i++){var c=sortList[i][0];var order=sortList[i][1];var s=(table.config.parsers[c].type=="text")?((order==0)?makeSortFunction("text","asc",c):makeSortFunction("text","desc",c)):((order==0)?makeSortFunction("numeric","asc",c):makeSortFunction("numeric","desc",c));var e="e"+i;dynamicExp+="var "+e+" = "+s;dynamicExp+="if("+e+") { return "+e+"; } ";dynamicExp+="else { ";}var orgOrderCol=cache.normalized[0].length-1;dynamicExp+="return a["+orgOrderCol+"]-b["+orgOrderCol+"];";for(var i=0
 ;i<l;i++){dynamicExp+="}; ";}dynamicExp+="return 0; ";dynamicExp+="}; ";if(table.config.debug){benchmark("Evaling expression:"+dynamicExp,new Date());}eval(dynamicExp);cache.normalized.sort(sortWrapper);if(table.config.debug){benchmark("Sorting on "+sortList.toString()+" and dir "+order+" time:",sortTime);}return cache;};function makeSortFunction(type,direction,index){var a="a["+index+"]",b="b["+index+"]";if(type=='text'&&direction=='asc'){return"("+a+" == "+b+" ? 0 : ("+a+" === null ? Number.POSITIVE_INFINITY : ("+b+" === null ? Number.NEGATIVE_INFINITY : ("+a+" < "+b+") ? -1 : 1 )));";}else if(type=='text'&&direction=='desc'){return"("+a+" == "+b+" ? 0 : ("+a+" === null ? Number.POSITIVE_INFINITY : ("+b+" === null ? Number.NEGATIVE_INFINITY : ("+b+" < "+a+") ? -1 : 1 )));";}else if(type=='numeric'&&direction=='asc'){return"("+a+" === null && "+b+" === null) ? 0 :("+a+" === null ? Number.POSITIVE_INFINITY : ("+b+" === null ? Number.NEGATIVE_INFINITY : "+a+" - "+b+"));";}else if(typ
 e=='numeric'&&direction=='desc'){return"("+a+" === null && "+b+" === null) ? 0 :("+a+" === null ? Number.POSITIVE_INFINITY : ("+b+" === null ? Number.NEGATIVE_INFINITY : "+b+" - "+a+"));";}};function makeSortText(i){return"((a["+i+"] < b["+i+"]) ? -1 : ((a["+i+"] > b["+i+"]) ? 1 : 0));";};function makeSortTextDesc(i){return"((b["+i+"] < a["+i+"]) ? -1 : ((b["+i+"] > a["+i+"]) ? 1 : 0));";};function makeSortNumeric(i){return"a["+i+"]-b["+i+"];";};function makeSortNumericDesc(i){return"b["+i+"]-a["+i+"];";};function sortText(a,b){if(table.config.sortLocaleCompare)return a.localeCompare(b);return((a<b)?-1:((a>b)?1:0));};function sortTextDesc(a,b){if(table.config.sortLocaleCompare)return b.localeCompare(a);return((b<a)?-1:((b>a)?1:0));};function sortNumeric(a,b){return a-b;};function sortNumericDesc(a,b){return b-a;};function getCachedSortType(parsers,i){return parsers[i].type;};this.construct=function(settings){return this.each(function(){if(!this.tHead||!this.tBodies)return;var $this,
 $document,$headers,cache,config,shiftDown=0,sortOrder;this.config={};config=$.extend(this.config,$.tablesorter.defaults,settings);$this=$(this);$.data(this,"tablesorter",config);$headers=buildHeaders(this);this.config.parsers=buildParserCache(this,$headers);cache=buildCache(this);var sortCSS=[config.cssDesc,config.cssAsc];fixColumnWidth(this);$headers.click(function(e){var totalRows=($this[0].tBodies[0]&&$this[0].tBodies[0].rows.length)||0;if(!this.sortDisabled&&totalRows>0){$this.trigger("sortStart");var $cell=$(this);var i=this.column;this.order=this.count++%2;if(this.lockedOrder)this.order=this.lockedOrder;if(!e[config.sortMultiSortKey]){config.sortList=[];if(config.sortForce!=null){var a=config.sortForce;for(var j=0;j<a.length;j++){if(a[j][0]!=i){config.sortList.push(a[j]);}}}config.sortList.push([i,this.order]);}else{if(isValueInArray(i,config.sortList)){for(var j=0;j<config.sortList.length;j++){var s=config.sortList[j],o=config.headerList[s[0]];if(s[0]==i){o.count=s[1];o.count
 ++;s[1]=o.count%2;}}}else{config.sortList.push([i,this.order]);}};setTimeout(function(){setHeadersCss($this[0],$headers,config.sortList,sortCSS);appendToTable($this[0],multisort($this[0],config.sortList,cache));},1);return false;}}).mousedown(function(){if(config.cancelSelection){this.onselectstart=function(){return false};return false;}});$this.bind("update",function(){var me=this;setTimeout(function(){me.config.parsers=buildParserCache(me,$headers);cache=buildCache(me);},1);}).bind("updateCell",function(e,cell){var config=this.config;var pos=[(cell.parentNode.rowIndex-1),cell.cellIndex];cache.normalized[pos[0]][pos[1]]=config.parsers[pos[1]].format(getElementText(config,cell),cell);}).bind("sorton",function(e,list){$(this).trigger("sortStart");config.sortList=list;var sortList=config.sortList;updateHeaderSortCount(this,sortList);setHeadersCss(this,$headers,sortList,sortCSS);appendToTable(this,multisort(this,sortList,cache));}).bind("appendCache",function(){appendToTable(this,cache
 );}).bind("applyWidgetId",function(e,id){getWidgetById(id).format(this);}).bind("applyWidgets",function(){applyWidget(this);});if($.metadata&&($(this).metadata()&&$(this).metadata().sortlist)){config.sortList=$(this).metadata().sortlist;}if(config.sortList.length>0){$this.trigger("sorton",[config.sortList]);}applyWidget(this);});};this.addParser=function(parser){var l=parsers.length,a=true;for(var i=0;i<l;i++){if(parsers[i].id.toLowerCase()==parser.id.toLowerCase()){a=false;}}if(a){parsers.push(parser);};};this.addWidget=function(widget){widgets.push(widget);};this.formatFloat=function(s){var i=parseFloat(s);return(isNaN(i))?0:i;};this.formatInt=function(s){var i=parseInt(s);return(isNaN(i))?0:i;};this.isDigit=function(s,config){return/^[-+]?\d*$/.test($.trim(s.replace(/[,.']/g,'')));};this.clearTableBody=function(table){if($.browser.msie){function empty(){while(this.firstChild)this.removeChild(this.firstChild);}empty.apply(table.tBodies[0]);}else{table.tBodies[0].innerHTML="";}};}}
 );$.fn.extend({tablesorter:$.tablesorter.construct});var ts=$.tablesorter;ts.addParser({id:"text",is:function(s){return true;},format:function(s){return $.trim(s.toLocaleLowerCase());},type:"text"});ts.addParser({id:"digit",is:function(s,table){var c=table.config;return $.tablesorter.isDigit(s,c);},format:function(s){return $.tablesorter.formatFloat(s);},type:"numeric"});ts.addParser({id:"currency",is:function(s){return/^[£$€?.]/.test(s);},format:function(s){return $.tablesorter.formatFloat(s.replace(new RegExp(/[£$€]/g),""));},type:"numeric"});ts.addParser({id:"ipAddress",is:function(s){return/^\d{2,3}[\.]\d{2,3}[\.]\d{2,3}[\.]\d{2,3}$/.test(s);},format:function(s){var a=s.split("."),r="",l=a.length;for(var i=0;i<l;i++){var item=a[i];if(item.length==2){r+="0"+item;}else{r+=item;}}return $.tablesorter.formatFloat(r);},type:"numeric"});ts.addParser({id:"url",is:function(s){return/^(https?|ftp|file):\/\/$/.test(s);},format:function(s){return jQuery.trim(s.replace(new RegExp(/(ht
 tps?|ftp|file):\/\//),''));},type:"text"});ts.addParser({id:"isoDate",is:function(s){return/^\d{4}[\/-]\d{1,2}[\/-]\d{1,2}$/.test(s);},format:function(s){return $.tablesorter.formatFloat((s!="")?new Date(s.replace(new RegExp(/-/g),"/")).getTime():"0");},type:"numeric"});ts.addParser({id:"percent",is:function(s){return/\%$/.test($.trim(s));},format:function(s){return $.tablesorter.formatFloat(s.replace(new RegExp(/%/g),""));},type:"numeric"});ts.addParser({id:"usLongDate",is:function(s){return s.match(new RegExp(/^[A-Za-z]{3,10}\.? [0-9]{1,2}, ([0-9]{4}|'?[0-9]{2}) (([0-2]?[0-9]:[0-5][0-9])|([0-1]?[0-9]:[0-5][0-9]\s(AM|PM)))$/));},format:function(s){return $.tablesorter.formatFloat(new Date(s).getTime());},type:"numeric"});ts.addParser({id:"shortDate",is:function(s){return/\d{1,2}[\/\-]\d{1,2}[\/\-]\d{2,4}/.test(s);},format:function(s,table){var c=table.config;s=s.replace(/\-/g,"/");if(c.dateFormat=="us"){s=s.replace(/(\d{1,2})[\/\-](\d{1,2})[\/\-](\d{4})/,"$3/$1/$2");}else if(c.date
 Format=="uk"){s=s.replace(/(\d{1,2})[\/\-](\d{1,2})[\/\-](\d{4})/,"$3/$2/$1");}else if(c.dateFormat=="dd/mm/yy"||c.dateFormat=="dd-mm-yy"){s=s.replace(/(\d{1,2})[\/\-](\d{1,2})[\/\-](\d{2})/,"$1/$2/$3");}return $.tablesorter.formatFloat(new Date(s).getTime());},type:"numeric"});ts.addParser({id:"time",is:function(s){return/^(([0-2]?[0-9]:[0-5][0-9])|([0-1]?[0-9]:[0-5][0-9]\s(am|pm)))$/.test(s);},format:function(s){return $.tablesorter.formatFloat(new Date("2000/01/01 "+s).getTime());},type:"numeric"});ts.addParser({id:"metadata",is:function(s){return false;},format:function(s,table,cell){var c=table.config,p=(!c.parserMetadataName)?'sortValue':c.parserMetadataName;return $(cell).metadata()[p];},type:"numeric"});ts.addWidget({id:"zebra",format:function(table){if(table.config.debug){var time=new Date();}var $tr,row=-1,odd;$("tr:visible",table.tBodies[0]).each(function(i){$tr=$(this);if(!$tr.hasClass(table.config.cssChildRow))row++;odd=(row%2==0);$tr.removeClass(table.config.widgetZebr
 a.css[odd?0:1]).addClass(table.config.widgetZebra.css[odd?1:0])});if(table.config.debug){$.tablesorter.benchmark("Applying Zebra widget",time);}}});})(jQuery);
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/samza/blob/b8058af0/docs/js/main.new.js
----------------------------------------------------------------------
diff --git a/docs/js/main.new.js b/docs/js/main.new.js
new file mode 100644
index 0000000..98791b9
--- /dev/null
+++ b/docs/js/main.new.js
@@ -0,0 +1,645 @@
+/*
+ * 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.
+ */
+
+/***************************************
+ * GLOBAL FUNCTIONS
+ **************************************/
+
+// credit: http://www.javascriptkit.com/javatutors/touchevents2.shtml
+function swipedetect(el, callback){
+  
+	var touchsurface = el,
+	swipedir,
+	startX,
+	startY,
+	distX,
+	distY,
+	threshold = 50, //required min distance traveled to be considered swipe
+	restraint = 100, // maximum distance allowed at the same time in perpendicular direction
+	allowedTime = 500, // maximum time allowed to travel that distance
+	elapsedTime,
+	startTime,
+	handleswipe = callback || function(swipedir, event){}
+
+	touchsurface.addEventListener('touchstart', function(e){
+			var touchobj = e.changedTouches[0]
+			swipedir = 'none'
+			dist = 0
+			startX = touchobj.pageX
+			startY = touchobj.pageY
+			startTime = new Date().getTime() // record time when finger first makes contact with surface
+			e.preventDefault()
+	}, false)
+
+	touchsurface.addEventListener('touchmove', function(e){
+			e.preventDefault() // prevent scrolling when inside DIV
+	}, false)
+
+	touchsurface.addEventListener('touchend', function(e){
+			var touchobj = e.changedTouches[0]
+			distX = touchobj.pageX - startX // get horizontal dist traveled by finger while in contact with surface
+			distY = touchobj.pageY - startY // get vertical dist traveled by finger while in contact with surface
+			elapsedTime = new Date().getTime() - startTime // get time elapsed
+			if (elapsedTime <= allowedTime){ // first condition for awipe met
+					if (Math.abs(distX) >= threshold && Math.abs(distY) <= restraint){ // 2nd condition for horizontal swipe met
+							swipedir = (distX < 0)? 'left' : 'right' // if dist traveled is negative, it indicates left swipe
+					}
+					else if (Math.abs(distY) >= threshold && Math.abs(distX) <= restraint){ // 2nd condition for vertical swipe met
+							swipedir = (distY < 0)? 'up' : 'down' // if dist traveled is negative, it indicates up swipe
+					}
+			}
+			handleswipe(swipedir, e)
+			e.preventDefault()
+	}, false)
+}
+
+/***************************************
+ * MAIN MENU - NAVIGATION TOGGLE MOBILE
+ **************************************/
+
+const menu = document.querySelectorAll('[data-plugin="menu"]');
+const menuOpened = document.querySelectorAll('[data-menu-opened]');
+const menuClosed = document.querySelectorAll('[data-menu-closed]');
+
+if (menu) {
+	menuClosed.forEach( (closedItem) => {
+		closedItem.addEventListener("click", function(e) {
+
+			if (window.innerWidth > 768) {
+				return;
+			}
+
+			menuClosed.forEach( (item) => {
+				item.style.display = 'none';
+			})
+			menuOpened.forEach( (item) => {
+				item.style.display = 'flex';
+			})
+		});
+	});
+
+	menuOpened.forEach( (openedItem) => {
+		openedItem.addEventListener("click", function(e) {
+
+			if (window.innerWidth > 768) {
+				return;
+			}
+
+			menuOpened.forEach( (item) => {
+				item.style.display = 'none';
+			})
+			menuClosed.forEach( (item) => {
+				item.style.display = 'flex';
+			})
+		});
+	});
+
+	window.addEventListener("resize", function(e){
+		if (window.innerWidth > 768) {
+			menuOpened.forEach( (item) => {
+				item.style = ''
+			})
+			menuClosed.forEach( (item) => {
+				item.style = ''
+			})
+		}
+	});
+}
+
+/***************************************
+ * SIDE MENU - & RETRIEVE DOCUMENTATION DYNAMICALLY
+ **************************************/
+
+///////////////////////////
+//
+// SIDE MENU FUNCTIONS
+//
+///////////////////////////
+ 
+var getDocumentationMenu = (url, cb, docMenu) => {
+	var myRequest = new Request(url);
+	
+	fetch(myRequest).then((response) => {
+		
+		return response.text().then((text) => {
+			cb(response.status, text, docMenu);
+		});
+
+	});
+};
+
+// This attaches all listeners
+var doMenu = () => {
+	const subMenuSelector = '[data-plugin="sub-menu"]';
+	const subMenus = document.querySelectorAll(subMenuSelector);
+	const curLoc = window.location.pathname;
+
+	subMenus.forEach( (subMenu) => {
+
+		var showClass = subMenu.getAttribute('data-sub-menu-show-class');
+
+		// figure out opening the submenus and making things active
+		var allSubSubMenus = subMenu.querySelectorAll('[data-sub-menu]');
+		var openSubMenu = false;
+
+
+		allSubSubMenus.forEach( (curSubSubMenu) => {
+
+			var curSubMenuItems = curSubSubMenu.children;
+
+			Array.from(curSubMenuItems).forEach( (child) => {
+				var childLoc = child.getAttribute('href');
+	
+				if (curLoc.includes(childLoc)) {
+					child.classList.add('active');
+					openSubMenu = true;
+					curSubSubMenu.classList.add(showClass);
+				}
+			});
+
+		});
+
+		if (openSubMenu) {
+			subMenu.classList.add(showClass);
+		}
+
+		// create the listener
+		subMenu.addEventListener('click', function(e) {
+			e.stopPropagation();
+
+			// must be a title to trigger things:
+			if (!e.target.classList.contains('side-navigation__group-title') && !e.target.classList.contains('side-navigation__group-title-icon')) {
+				return;
+			}
+
+			var closestSubMenu = e.target.closest(subMenuSelector);
+
+			closestSubMenu.classList.toggle(showClass);
+
+			var siblings = Array.from(closestSubMenu.parentNode.children);
+
+			siblings.forEach( sibling => {
+
+				// Turn off all siblings (not itself)
+				if (!sibling.isEqualNode(closestSubMenu)) {
+					sibling.classList.remove(showClass);
+				}
+
+			});
+			
+		}, true);
+	});
+};
+
+// This takes the response of the documentation and builds the menu in right format
+var buildDocMenu = (status, body, docMenu) => {
+	if (status == 404) {
+		doMenu();
+		return;
+	}
+
+	const docMenuLinkBase = docMenu.getAttribute('data-documentation');
+	var div = document.createElement('div');
+	div.innerHTML = body;
+
+	div.querySelectorAll('h4').forEach( h4 => {
+
+		var listItems = h4.nextElementSibling.children;
+
+		var group = document.createElement('div');
+		var itemsDiv = document.createElement('div');
+		var icon = document.createElement('i');
+		icon.classList.add('side-navigation__group-title-icon', 'icon', 'ion-md-arrow-dropdown');
+		group.classList.add('side-navigation__group', 'side-navigation__group--has-nested');
+		group.setAttribute('data-sub-menu-show-class', 'side-navigation__group--has-nested-visible');
+		group.setAttribute('data-plugin', 'sub-menu');
+		itemsDiv.classList.add('side-navigation__group-items');
+		itemsDiv.setAttribute('data-sub-menu', true);
+
+		h4.classList.add('side-navigation__group-title');
+		h4.prepend(icon);
+
+		Array.from(listItems).forEach( listItem => {
+			var link = listItem.querySelector('a');
+			linkDestination = docMenuLinkBase + '/' + link.getAttribute('href');
+
+			linkText = link.text.trim();
+
+			var newLink = document.createElement('a');
+			newLink.classList.add('side-navigation__group-item');
+			newLink.setAttribute('href', linkDestination);
+			newLink.text = linkText;
+
+			itemsDiv.appendChild(newLink);
+		})
+
+		group.appendChild(h4);
+		group.appendChild(itemsDiv);
+
+		docMenu.appendChild(group);
+	});
+
+	// this needs to be called here bc this is the callback passed to asynchonous function getDocumentation..
+	doMenu();
+};
+
+// This is the function to trigger to begin side menu setup.
+var doSideMenu = () => {
+	const docMenuSelector = '[data-documentation]';
+	const docMenu = document.querySelector(docMenuSelector);
+
+	// handle the mobile triggering of the side menu.
+	handleMobileSideNavigation();
+
+	if (docMenu) {
+		const docMenuLinkBase = docMenu.getAttribute('data-documentation');
+		const docMenuLink = window.location.protocol + '//' + window.location.host + docMenuLinkBase;
+		// get the documentation menu, then build it.
+		getDocumentationMenu(docMenuLink, buildDocMenu, docMenu);
+
+	}
+};
+
+var handleMobileSideNavigation = function() {
+	const containerToggle = document.querySelector('.container__toggle');
+	const container = document.querySelector('.container');
+
+	if (containerToggle) {
+		// Detect clicks
+		containerToggle.addEventListener("click", function(e){
+			container.classList.toggle('container--opened');
+
+			localStorage['container-opened'] = JSON.stringify(container.classList.contains('container--opened'));
+		});
+
+		if (localStorage['container-opened']) {
+
+			var opened = JSON.parse(localStorage['container-opened']);
+			
+			if (opened) {
+				container.classList.add('container--opened');
+			} else {
+				container.classList.remove('container--opened');
+			} 
+			
+		} else {
+			
+			container.classList.remove('container--opened');
+		}
+
+		// Detect swipes
+		swipedetect(containerToggle, function(swipedir) {
+				// swipedir contains either "none", "left", "right", "top", or "down"
+				if (!['left', 'right'].includes(swipedir)) {
+						container.classList.toggle('container--opened');
+						localStorage['container-opened'] = JSON.stringify(container.classList.contains('container--opened'));
+						return;
+				}
+
+				// close it
+				if (swipedir == 'left') {
+					container.classList.add('container--opened');
+				} else {
+					container.classList.remove('container--opened');
+				}
+
+				localStorage['container-opened'] = JSON.stringify(container.classList.contains('container--opened'));
+		});
+	}
+};
+
+// SETUP
+doSideMenu();
+
+
+/***************************************
+ * RELEASES LIST
+ **************************************/
+var doReleasesList = () => {
+	const releasesListSelector = '[data-releases-list]';
+	const releasesList = document.querySelector(releasesListSelector);
+	const curLoc = window.location.pathname;
+
+	var tryFile = function (url, cb) {
+		var myRequest = new Request(url);
+		fetch(myRequest).then((response) => {
+			cb(response.status != 404);
+		});
+	}
+
+	if (releasesList) {
+
+		// console.log(releasesList);
+		// console.log(curLoc);
+
+		var items = releasesList.children;
+
+		Array.from(items).forEach( item => {
+			var anchor = item.querySelector('a');
+			var link = anchor.getAttribute('href');
+
+			tryFile(link, (status) => {
+				console.log(link);
+				console.log(status);
+				console.log('---');
+
+				if (status) {
+					item.classList.remove('hide');
+				} else {
+					item.classList.add('hide');
+				}
+
+				if (curLoc.includes(link)) {
+					anchor.classList.add('active');
+				}
+			})
+		});
+
+	}
+
+};
+
+doReleasesList();
+
+/***************************************
+ * CANVAS - HOME PAGE HERO
+ **************************************/
+// CREDIT: https://codepen.io/dudleystorey/pen/NbNjjX
+var doCanvas = function() {
+	const hero = document.getElementById("hero");
+	const canvasBody = document.getElementById("canvas");
+
+	if (canvasBody) {
+
+		let resizeReset = function() {
+			w = canvasBody.width = hero.offsetWidth;
+			h = canvasBody.height = hero.offsetHeight;
+		}
+
+		const opts = { 
+			particleColor: "rgb(200,200,200)",
+			lineColor: "rgb(200,200,200)",
+			particleAmount: 30,
+			defaultSpeed: 1,
+			variantSpeed: 1,
+			defaultRadius: 2,
+			variantRadius: 2,
+			linkRadius: 200,
+		};
+
+		let deBouncer = function() {
+				clearTimeout(tid);
+				tid = setTimeout(function() {
+						resizeReset();
+				}, delay);
+		};
+
+		let checkDistance = function(x1, y1, x2, y2){ 
+			return Math.sqrt(Math.pow(x2 - x1, 2) + Math.pow(y2 - y1, 2));
+		};
+
+		let linkPoints = function(point1, hubs){ 
+			for (let i = 0; i < hubs.length; i++) {
+				let distance = checkDistance(point1.x, point1.y, hubs[i].x, hubs[i].y);
+				let opacity = 1 - distance / opts.linkRadius;
+				if (opacity > 0) { 
+					drawArea.lineWidth = 0.5;
+					drawArea.strokeStyle = `rgba(${rgb[0]}, ${rgb[1]}, ${rgb[2]}, ${opacity})`;
+					drawArea.beginPath();
+					drawArea.moveTo(point1.x, point1.y);
+					drawArea.lineTo(hubs[i].x, hubs[i].y);
+					drawArea.closePath();
+					drawArea.stroke();
+				}
+			}
+		}
+
+		Particle = function(xPos, yPos){ 
+			this.x = Math.random() * w; 
+			this.y = Math.random() * h;
+			this.speed = opts.defaultSpeed + Math.random() * opts.variantSpeed; 
+			this.directionAngle = Math.floor(Math.random() * 360); 
+			this.color = opts.particleColor;
+			this.radius = opts.defaultRadius + Math.random() * opts. variantRadius; 
+			this.vector = {
+				x: Math.cos(this.directionAngle) * this.speed,
+				y: Math.sin(this.directionAngle) * this.speed
+			};
+			this.update = function(){ 
+				this.border(); 
+				this.x += this.vector.x; 
+				this.y += this.vector.y; 
+			};
+			this.border = function(){ 
+				if (this.x >= w || this.x <= 0) { 
+					this.vector.x *= -1;
+				}
+				if (this.y >= h || this.y <= 0) {
+					this.vector.y *= -1;
+				}
+				if (this.x > w) this.x = w;
+				if (this.y > h) this.y = h;
+				if (this.x < 0) this.x = 0;
+				if (this.y < 0) this.y = 0;	
+			};
+			this.draw = function(){ 
+				drawArea.beginPath();
+				drawArea.arc(this.x, this.y, this.radius, 0, Math.PI*2);
+				drawArea.closePath();
+				drawArea.fillStyle = this.color;
+				drawArea.fill();
+			};
+		};
+
+		function setup(){ 
+			particles = [];
+			resizeReset();
+			for (let i = 0; i < opts.particleAmount; i++){
+				particles.push( new Particle() );
+			}
+			window.requestAnimationFrame(loop);
+		}
+
+		function loop(){ 
+			window.requestAnimationFrame(loop);
+			drawArea.clearRect(0,0,w,h);
+			for (let i = 0; i < particles.length; i++){
+				particles[i].update();
+				particles[i].draw();
+			}
+			for (let i = 0; i < particles.length; i++){
+				linkPoints(particles[i], particles);
+			}
+		}
+
+		window.addEventListener("resize", function(){
+			deBouncer();
+		});
+
+		drawArea = canvasBody.getContext("2d");
+		let delay = 200, tid,
+		rgb = opts.lineColor.match(/\d+/g);
+		resizeReset();
+		setup();
+	}
+}
+
+// Do the canvas
+doCanvas();
+
+
+/***************************************
+ * EVENTS
+ **************************************/
+
+var doEvents = () => {
+	const eventSelector = '[data-plugin="event"]';
+	const events = document.querySelectorAll(eventSelector);
+	
+	if (!events) {
+		return;
+	}
+
+	events.forEach( event => {
+
+		var date = event.getAttribute('data-date');
+		var upcomingClass = event.getAttribute('data-upcoming-class');
+
+		var d = new Date(date);
+		var timestamp = d.getTime();
+
+		var c = new Date();
+		var curstamp = c.getTime();
+
+		if (timestamp >= curstamp) {
+			event.classList.add(upcomingClass);
+		}
+	});
+
+};
+
+doEvents();
+
+
+/***************************************
+ * PAGINATE
+ **************************************/
+
+var doPagination = () => {
+	const paginateSelector = '[data-plugin="paginate"]';
+	const paginates = document.querySelectorAll(paginateSelector);
+	
+	if (!paginates) {
+		return;
+	}
+
+	paginates.forEach( paginate => {
+
+		var maxPosts = parseInt(paginate.getAttribute('data-max-posts'));
+
+		var posts = paginate.children;
+		var count = 0;
+		var nextBtn = document.querySelector(paginate.getAttribute('data-next'));
+		var prevBtn = document.querySelector(paginate.getAttribute('data-previous'));
+		var pagination = document.querySelector(paginate.getAttribute('data-pagination'));
+
+		// Initial page load, hide extra posts
+		Array.from(posts).forEach( post => {
+			if (count >= maxPosts) {
+				post.classList.add('hide');
+			}
+
+			count++
+		});
+
+		// Initial page load, determine if next shows
+		if (posts && posts.length && posts.length > maxPosts) {
+			nextBtn.classList.remove('hide');
+		} else {
+			pagination.classList.add('hide');
+		}
+
+		// handle next
+		nextBtn.addEventListener("click", function(e) {
+			
+			// Find index of last showing post
+			var visiblePosts = Array.prototype.filter.call(posts, (post) => {
+				return !Array.from(post.classList).includes('hide');
+			});
+
+			var lastPost = visiblePosts[visiblePosts.length - 1];
+			var index = Array.from(posts).indexOf(lastPost);
+
+			// Hide all + show up to max posts after index
+			Array.from(posts).forEach( (post, i) => {
+				post.classList.add('hide');
+
+				// show next posts and prev button
+				if (i > index && i <= (index + maxPosts)) {
+					post.classList.remove('hide');
+					prevBtn.classList.remove('hide');
+				}
+				
+			});
+
+			// determine if next button does not show up
+			if (posts.length <= index + maxPosts + 1) { // 1 bc indexes start at 0
+				nextBtn.classList.add('hide');
+			}
+
+			paginate.scrollIntoView();
+		});
+
+		// handle previous
+		prevBtn.addEventListener("click", function(e) {
+
+			// Find index of first showing post
+			var visiblePosts = Array.prototype.filter.call(posts, (post) => {
+				return !Array.from(post.classList).includes('hide');
+			});
+
+			var firstPost = visiblePosts[0];
+			var index = Array.from(posts).indexOf(firstPost);
+
+			// Hide all + show up to max posts before index
+			Array.from(posts).forEach( (post, i) => {
+				post.classList.add('hide');
+
+				// show next posts and next button
+				if (i < index && i >= index - maxPosts) {
+					post.classList.remove('hide');
+					nextBtn.classList.remove('hide');
+				}
+				
+			});
+
+			// determine if prev button does not show up
+			if (index - maxPosts == 0) {
+				prevBtn.classList.add('hide');
+			}
+
+			paginate.scrollIntoView();
+			
+		});
+
+
+	});
+};
+
+doPagination();
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/samza/blob/b8058af0/docs/learn/tutorials/versioned/index.md
----------------------------------------------------------------------
diff --git a/docs/learn/tutorials/versioned/index.md b/docs/learn/tutorials/versioned/index.md
index d155b82..97a9fd5 100644
--- a/docs/learn/tutorials/versioned/index.md
+++ b/docs/learn/tutorials/versioned/index.md
@@ -53,5 +53,4 @@ title: Tutorials
 <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/>
--->
+<a href="committing.html">Committing</a><br/> \-->

http://git-wip-us.apache.org/repos/asf/samza/blob/b8058af0/docs/meetups/index.html
----------------------------------------------------------------------
diff --git a/docs/meetups/index.html b/docs/meetups/index.html
new file mode 100644
index 0000000..404fd18
--- /dev/null
+++ b/docs/meetups/index.html
@@ -0,0 +1,106 @@
+---
+layout: page
+exclude_from_loop: true
+title: Samza Meetups
+---
+<!--
+   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.
+-->
+
+<div class="meetups" data-plugin="paginate" data-max-posts="2" data-previous=".paginate-previous--meetup" data-next=".paginate-next--meetup" data-pagination=".pagination--meetup">
+    {% assign sorted = (site.meetups | sort: 'date') | reverse %}
+    {% for meetup in sorted %}
+        {% if meetup.exclude_from_loop %}
+            {% continue %}
+        {% endif %}
+    <div class="meetup" data-plugin="event" data-date="{{ meetup.date | date: '%-d %B %Y' }}" data-upcoming-class="meetup--upcoming">
+        <i class="meetup-icon ion-md-calendar"></i>
+        <div class="meetup-info">
+            <div class="meetup-date-group">{{ meetup.display_date }}</div>
+            
+            {% for meet in meetup.meetups %}
+
+            <div class="meet" data-plugin="event" data-date="{{ meet.date | date: '%-d %B %Y' }}" data-upcoming-class="meet--upcoming">
+                <div class="meet-upcoming-section">
+                    <div class="meet-upcoming-tag">
+                        <i class="ion-md-alarm"></i>
+                        Upcoming Event!
+                    </div>
+                </div>
+                <div>
+                    <div class="meet-name">{{ meet.name}}</div>
+                    <div class="meet-date"><strong>&raquo; Date &HorizontalLine;</strong> {{ meet.date }}</div>
+                    <div class="meet-host"><strong>&raquo; Presented At &HorizontalLine;</strong> {{ meet.host }}</div>
+
+                    {% if meet.presenters %}
+                    <div class="meet-presenter">
+
+                        <strong>&raquo; Presented By &HorizontalLine;</strong>
+
+                        <span>
+                        {% for presenter in meet.presenters %}
+
+                            <span>{{ presenter.name }}</span>
+
+                        {% endfor %}
+                        </span>
+
+                    </div>
+                    {% endif %}
+
+                    {% if meet.image %}
+                    <div class="meet-image" style="background-image:url('{{ meet.image }}')"></div>
+                    {% endif %}
+                </div>
+                {% if meet.video and meet.video.image %}
+                <a class="meet-video" href="{{ meet.video.url }}" style="background-image:url('{{ meet.video.image}}')">
+                    <i class="meet-video-icon ion-md-arrow-dropright-circle"></i>
+                </a>
+                {% endif %}
+
+                {% if meet.abstract %}
+                <div class="meet-description">
+                    {{ meet.abstract }}
+                </div>
+                {% endif %}
+                <div class="meet-actions">    
+                    {% if meet.slides %}
+                    <a class="meet-action" href="{{ meet.slides.url }}">View Slides</a>
+                    {% endif %}
+                    <a class="meet-action" href="{{ meet.url }}">Meetup Details
+                    </a>
+                </div>
+            </div>
+
+            {% endfor %}
+        </div>
+    </div>
+    {% endfor %}
+</div>
+    
+<div class="pagination pagination--meetup">
+    <hr/>
+    <div class="pagination-inner">
+        <div class="paginate-previous paginate-previous--meetup paginate-action hide">
+            <i class="icon ion-md-arrow-dropleft-circle"></i>
+            Previous
+        </div>
+        <div class="paginate-next paginate-next--meetup paginate-action hide">
+            Next
+            <i class="icon ion-md-arrow-dropright-circle"></i>
+        </div>
+    </div>
+</div>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/samza/blob/b8058af0/docs/powered-by/index.html
----------------------------------------------------------------------
diff --git a/docs/powered-by/index.html b/docs/powered-by/index.html
new file mode 100644
index 0000000..3838165
--- /dev/null
+++ b/docs/powered-by/index.html
@@ -0,0 +1,48 @@
+---
+layout: page
+title: Powered By
+exclude_from_loop: true
+---
+<!--
+   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.
+-->
+
+A list of companies powered by Samza
+
+<ul class="powered-by">
+
+  {% for company in site.powered-by %}
+    {% if company.exclude_from_loop %}
+        {% continue %}
+    {% endif %}
+
+  <li>
+    <span title="{{ company.name }}">
+      <span style="background-image: url('https://logo.clearbit.com/{{ company.domain }}?size=256');"></span>
+    </span>
+    <div>
+      <a href="https://{{ company.domain }}" class="external-link" rel="nofollow">
+        <i class="icon ion-md-share-alt"></i> {{ company.name }}
+      </a>
+      <p>
+      {{ company.content }}
+      </p>
+    </div>
+  </li>
+
+  {% endfor %}
+
+</ul>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/samza/blob/b8058af0/docs/startup/preview/index.md
----------------------------------------------------------------------
diff --git a/docs/startup/preview/index.md b/docs/startup/preview/index.md
index 20e4aaf..f456503 100644
--- a/docs/startup/preview/index.md
+++ b/docs/startup/preview/index.md
@@ -26,7 +26,7 @@ title: Feature Preview
 
 ---
 
-# Overview
+## Overview
 Samza 0.13.0 introduces a new programming model and a new deployment model. They're being released as a preview because they represent major enhancements to how developers work with Samza, so it is beneficial for both early adopters and the Samza development community to experiment with the release and provide feedback. The following sections introduce the new features and link to tutorials which demonstrate how to use them. Please try them and send feedback to the [dev mailing list](mailto:dev@samza.apache.org)
 
 ---

http://git-wip-us.apache.org/repos/asf/samza/blob/b8058af0/docs/startup/releases/versioned/release-notes.md
----------------------------------------------------------------------
diff --git a/docs/startup/releases/versioned/release-notes.md b/docs/startup/releases/versioned/release-notes.md
index 18983f1..a356318 100644
--- a/docs/startup/releases/versioned/release-notes.md
+++ b/docs/startup/releases/versioned/release-notes.md
@@ -30,18 +30,21 @@ title: Release Notes
 {% endif %}
 
 {% if site.version != "latest" %}
-## Download
+### Download
 All Samza JARs are published through [Apache's Maven repository](https://repository.apache.org/content/groups/public/org/apache/samza/). See [here](../download/index.html) for more details.
 
-## Source Release
+### Source Release
 [samza-sources-{{site.version}}.tgz](http://www.apache.org/dyn/closer.lua/samza/{{site.version}}.*)
 
 
 {% endif %}
 
 
-
-## Release Notes
+{% if site.version != "latest" %}
+### Release Notes {{site.version}}
+{% else %}
+### Release Notes
+{% endif %}
 <!-- Add notes on new features, modified behavior of existing features, operational/performance improvements, new tools etc -->
 * [SAMZA-1510](https://issues.apache.org/jira/browse/SAMZA-1510) - Samza SQL
 * [SAMZA-1438](https://issues.apache.org/jira/browse/SAMZA-1438) - Producer and consumer for Azure EventHubs
@@ -52,18 +55,18 @@ All Samza JARs are published through [Apache's Maven repository](https://reposit
 * [SAMZA-1406](https://issues.apache.org/jira/browse/SAMZA-1406) - Enhancements to the ZooKeeper-based deployment model
 * [SAMZA-1321](https://issues.apache.org/jira/browse/SAMZA-1321) - Support for multi-stage batch processing
 
-## Upgrade Notes
+### Upgrade Notes
 <!-- Add detailed notes on how someone using an older version of samza (typically, currentVersion - 1) can upgrade to the latest -->
 <!-- Notes typically include config changes, public-api changes, new user guides/tutorials etc -->
 
-### Configuration Changes
+#### Configuration Changes
 
 <!-- PR 290 -->
 * Introduced a new **mandatory** configuration - `job.coordination.utils.factory`. Read more about it
 [here](/learn/documentation/{{site.version}}/jobs/configuration-table.html). <br />This config is applicable to all Samza
 applications deployed using the `LocalApplicationRunner` (that is, non-yarn deployments).
 
-### API Changes
+#### API Changes
 
 <!-- PR 292 -->
 * The following APIs in `SystemAdmin` have been deprecated in the previous versions and hence, replaced with newer APIs.

http://git-wip-us.apache.org/repos/asf/samza/blob/b8058af0/docs/talks/index.html
----------------------------------------------------------------------
diff --git a/docs/talks/index.html b/docs/talks/index.html
new file mode 100644
index 0000000..0ee8fa6
--- /dev/null
+++ b/docs/talks/index.html
@@ -0,0 +1,85 @@
+---
+layout: page
+exclude_from_loop: true
+title: Samza Talks
+---
+<!--
+   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.
+-->
+
+<div class="talks" data-plugin="paginate" data-max-posts="2" data-previous=".paginate-previous--talk" data-next=".paginate-next--talk" data-pagination=".pagination--talk">
+    {% assign sorted = (site.talks | sort: 'date') | reverse %}
+    {% for talk in sorted %}
+        {% if talk.exclude_from_loop %}
+            {% continue %}
+        {% endif %}
+    <div class="talk" data-plugin="event" data-date="{{ talk.date | date: '%-d %B %Y' }}" data-upcoming-class="talk--upcoming">
+        <div class="talk-upcoming-section">
+            <div class="talk-upcoming-tag">
+                <i class="ion-md-alarm"></i>
+                Upcoming Event!
+            </div>
+        </div>
+        <i class="talk-icon ion-md-quote"></i>
+        <div class="talk-info">
+            <div class="talk-date">{{ talk.display_date }}</div>
+            <div class="talk-event" href="{{ talk.event.url }}">
+                <a class="talk-event-link" href="{{ talk.event.url }}">
+                    <div class="talk-event-name">{{ talk.event.name}}</div>
+                    <div class="talk-event-host"><strong>&raquo; Presented At &HorizontalLine;</strong> {{ talk.event.host }}</div>
+                    {% if talk.event.by %}
+                    <div class="talk-event-by"><strong>&raquo; Presented By &HorizontalLine;</strong> {{ talk.event.by }}</div>
+                    {% endif %}
+                    {% if talk.event.image %}
+                    <div class="talk-event-image" style="background-image:url('{{ talk.event.image }}')"></div>
+                    {% endif %}
+                </a>
+                {% if talk.video and talk.video.image %}
+                <a class="talk-video" href="{{ talk.video.url }}" style="background-image:url('{{ talk.video.image}}')">
+                    <i class="talk-video-icon ion-md-arrow-dropright-circle"></i>
+                </a>
+                {% endif %}
+            </div>
+        </div>
+        {% if talk.content %}
+        <div class="talk-description">
+            {{ talk.content }}
+        </div>
+        {% endif %}
+        <div class="talk-actions">    
+            {% if talk.slides and talk.slides.url %}
+            <a class="talk-action" href="{{ talk.slides.url }}">View Slides</a>
+            {% endif %}
+            <a class="talk-action" href="{{ talk.event.url }}">Event Details
+            </a>
+        </div>
+    </div>
+    {% endfor %}
+</div>
+<div class="pagination pagination--talk">
+    <hr/>
+    <div class="pagination-inner">
+        <div class="paginate-previous paginate-previous--talk paginate-action hide">
+            <i class="icon ion-md-arrow-dropleft-circle"></i>
+            Previous
+        </div>
+        <div class="paginate-next paginate-next--talk paginate-action hide">
+            Next
+            <i class="icon ion-md-arrow-dropright-circle"></i>
+        </div>
+    </div>
+</div>
+    
\ No newline at end of file


[11/29] samza git commit: Samza SQL: Code re-org to accomodate Samza SQL engine to take Calcite IR as input

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/samza/blob/dec16392/samza-sql/src/test/java/org/apache/samza/sql/translator/TestQueryTranslator.java
----------------------------------------------------------------------
diff --git a/samza-sql/src/test/java/org/apache/samza/sql/translator/TestQueryTranslator.java b/samza-sql/src/test/java/org/apache/samza/sql/translator/TestQueryTranslator.java
index c9f59e6..e5d3659 100644
--- a/samza-sql/src/test/java/org/apache/samza/sql/translator/TestQueryTranslator.java
+++ b/samza-sql/src/test/java/org/apache/samza/sql/translator/TestQueryTranslator.java
@@ -22,7 +22,9 @@ package org.apache.samza.sql.translator;
 import java.util.Collection;
 import java.util.HashMap;
 import java.util.HashSet;
+import java.util.List;
 import java.util.Map;
+import java.util.stream.Collectors;
 import org.apache.samza.SamzaException;
 import org.apache.samza.application.StreamApplicationDescriptorImpl;
 import org.apache.samza.config.Config;
@@ -43,6 +45,9 @@ import org.junit.Before;
 import org.junit.Test;
 import org.mockito.internal.util.reflection.Whitebox;
 
+import static org.apache.samza.sql.dsl.SamzaSqlDslConverter.*;
+
+
 public class TestQueryTranslator {
 
   // Helper functions to validate the cloned copies of TranslatorContext and SamzaSqlExecutionContext
@@ -79,14 +84,20 @@ public class TestQueryTranslator {
   public void testTranslate() {
     Map<String, String> config = SamzaSqlTestConfig.fetchStaticConfigsWithFactories(10);
     config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT,
-        "Insert into testavro.outputTopic select MyTest(id) from testavro.level1.level2.SIMPLE1 as s where s.id = 10");
+        "Insert into testavro.outputTopic(id) select MyTest(id) from testavro.level1.level2.SIMPLE1 as s where s.id = 10");
     Config samzaConfig = SamzaSqlApplicationRunner.computeSamzaConfigs(true, new MapConfig(config));
-    SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config));
+
+    List<String> sqlStmts = fetchSqlFromConfig(config);
+    List<SamzaSqlQueryParser.QueryInfo> queryInfo = fetchQueryInfo(sqlStmts);
+    SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config),
+        queryInfo.stream().map(SamzaSqlQueryParser.QueryInfo::getSources).flatMap(Collection::stream)
+            .collect(Collectors.toSet()),
+        queryInfo.stream().map(SamzaSqlQueryParser.QueryInfo::getSink).collect(Collectors.toSet()));
+
     QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig);
-    SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0);
-    StreamApplicationDescriptorImpl appDesc = new StreamApplicationDescriptorImpl(descriptor -> { },samzaConfig);
+    StreamApplicationDescriptorImpl appDesc = new StreamApplicationDescriptorImpl(streamApp -> { },samzaConfig);
 
-    translator.translate(queryInfo, appDesc);
+    translator.translate(queryInfo.get(0), appDesc);
     OperatorSpecGraph specGraph = appDesc.getOperatorSpecGraph();
 
     StreamConfig streamConfig = new StreamConfig(samzaConfig);
@@ -128,17 +139,20 @@ public class TestQueryTranslator {
   public void testTranslateComplex() {
     Map<String, String> config = SamzaSqlTestConfig.fetchStaticConfigsWithFactories(10);
     config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT,
-        "Insert into testavro.outputTopic select Flatten(array_values) from testavro.COMPLEX1");
-//    config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT,
-//        "Insert into testavro.foo2 select string_value, SUM(id) from testavro.COMPLEX1 "
-//            + "GROUP BY TumbleWindow(CURRENT_TIME, INTERVAL '1' HOUR), string_value");
+        "Insert into testavro.outputTopic(string_value) select Flatten(array_values) from testavro.COMPLEX1");
     Config samzaConfig = SamzaSqlApplicationRunner.computeSamzaConfigs(true, new MapConfig(config));
-    SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config));
+
+    List<String> sqlStmts = fetchSqlFromConfig(config);
+    List<SamzaSqlQueryParser.QueryInfo> queryInfo = fetchQueryInfo(sqlStmts);
+    SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config),
+        queryInfo.stream().map(SamzaSqlQueryParser.QueryInfo::getSources).flatMap(Collection::stream)
+            .collect(Collectors.toSet()),
+        queryInfo.stream().map(SamzaSqlQueryParser.QueryInfo::getSink).collect(Collectors.toSet()));
+
     QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig);
-    SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0);
-    StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(appDesc -> { }, samzaConfig);
+    StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(streamApp -> { }, samzaConfig);
 
-    translator.translate(queryInfo, streamAppDesc);
+    translator.translate(queryInfo.get(0), streamAppDesc);
     OperatorSpecGraph specGraph = streamAppDesc.getOperatorSpecGraph();
 
     StreamConfig streamConfig = new StreamConfig(samzaConfig);
@@ -163,14 +177,21 @@ public class TestQueryTranslator {
   public void testTranslateSubQuery() {
     Map<String, String> config = SamzaSqlTestConfig.fetchStaticConfigsWithFactories(10);
     config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT,
-        "Insert into testavro.outputTopic select Flatten(a), id from (select id, array_values a, string_value s from testavro.COMPLEX1)");
+        "Insert into testavro.outputTopic(string_value, id) select Flatten(a), id "
+            + " from (select id, array_values a, string_value s from testavro.COMPLEX1)");
     Config samzaConfig = SamzaSqlApplicationRunner.computeSamzaConfigs(true, new MapConfig(config));
-    SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config));
+
+    List<String> sqlStmts = fetchSqlFromConfig(config);
+    List<SamzaSqlQueryParser.QueryInfo> queryInfo = fetchQueryInfo(sqlStmts);
+    SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config),
+        queryInfo.stream().map(SamzaSqlQueryParser.QueryInfo::getSources).flatMap(Collection::stream)
+            .collect(Collectors.toSet()),
+        queryInfo.stream().map(SamzaSqlQueryParser.QueryInfo::getSink).collect(Collectors.toSet()));
+
     QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig);
-    SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0);
-    StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(appDesc -> { }, samzaConfig);
+    StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(streamApp -> { }, samzaConfig);
 
-    translator.translate(queryInfo, streamAppDesc);
+    translator.translate(queryInfo.get(0), streamAppDesc);
     OperatorSpecGraph specGraph = streamAppDesc.getOperatorSpecGraph();
 
     StreamConfig streamConfig = new StreamConfig(samzaConfig);
@@ -195,115 +216,151 @@ public class TestQueryTranslator {
   public void testTranslateStreamTableJoinWithoutJoinOperator() {
     Map<String, String> config = SamzaSqlTestConfig.fetchStaticConfigsWithFactories(configs, 1);
     String sql =
-        "Insert into testavro.enrichedPageViewTopic"
+        "Insert into testavro.enrichedPageViewTopic(profileName, pageKey)"
             + " select p.name as profileName, pv.pageKey"
             + " from testavro.PAGEVIEW as pv, testavro.PROFILE.`$table` as p"
             + " where p.id = pv.profileId";
     config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, sql);
     Config samzaConfig = SamzaSqlApplicationRunner.computeSamzaConfigs(true, new MapConfig(config));
-    SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config));
+
+    List<String> sqlStmts = fetchSqlFromConfig(config);
+    List<SamzaSqlQueryParser.QueryInfo> queryInfo = fetchQueryInfo(sqlStmts);
+    SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config),
+        queryInfo.stream().map(SamzaSqlQueryParser.QueryInfo::getSources).flatMap(Collection::stream)
+            .collect(Collectors.toSet()),
+        queryInfo.stream().map(SamzaSqlQueryParser.QueryInfo::getSink).collect(Collectors.toSet()));
+
     QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig);
-    SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0);
-    StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(appDesc -> { }, samzaConfig);
+    StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(streamApp -> { }, samzaConfig);
 
-    translator.translate(queryInfo, streamAppDesc);
+    translator.translate(queryInfo.get(0), streamAppDesc);
   }
 
   @Test (expected = SamzaException.class)
   public void testTranslateStreamTableJoinWithFullJoinOperator() {
     Map<String, String> config = SamzaSqlTestConfig.fetchStaticConfigsWithFactories(configs, 1);
     String sql =
-        "Insert into testavro.enrichedPageViewTopic"
+        "Insert into testavro.enrichedPageViewTopic(profileName, pageKey)"
             + " select p.name as profileName, pv.pageKey"
             + " from testavro.PAGEVIEW as pv"
             + " full join testavro.PROFILE.`$table` as p"
             + " on p.id = pv.profileId";
     config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, sql);
     Config samzaConfig = SamzaSqlApplicationRunner.computeSamzaConfigs(true, new MapConfig(config));
-    SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config));
+
+    List<String> sqlStmts = fetchSqlFromConfig(config);
+    List<SamzaSqlQueryParser.QueryInfo> queryInfo = fetchQueryInfo(sqlStmts);
+    SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config),
+        queryInfo.stream().map(SamzaSqlQueryParser.QueryInfo::getSources).flatMap(Collection::stream)
+            .collect(Collectors.toSet()),
+        queryInfo.stream().map(SamzaSqlQueryParser.QueryInfo::getSink).collect(Collectors.toSet()));
+
     QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig);
-    SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0);
-    StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(appDesc -> { }, samzaConfig);
+    StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(streamApp -> { }, samzaConfig);
 
-    translator.translate(queryInfo, streamAppDesc);
+    translator.translate(queryInfo.get(0), streamAppDesc);
   }
 
   @Test (expected = IllegalStateException.class)
   public void testTranslateStreamTableJoinWithSelfJoinOperator() {
     Map<String, String> config = SamzaSqlTestConfig.fetchStaticConfigsWithFactories(configs, 1);
     String sql =
-        "Insert into testavro.enrichedPageViewTopic"
+        "Insert into testavro.enrichedPageViewTopic(profileName)"
             + " select p1.name as profileName"
             + " from testavro.PROFILE.`$table` as p1"
             + " join testavro.PROFILE.`$table` as p2"
             + " on p1.id = p2.id";
     config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, sql);
     Config samzaConfig = SamzaSqlApplicationRunner.computeSamzaConfigs(true, new MapConfig(config));
-    SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config));
+
+    List<String> sqlStmts = fetchSqlFromConfig(config);
+    List<SamzaSqlQueryParser.QueryInfo> queryInfo = fetchQueryInfo(sqlStmts);
+    SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config),
+        queryInfo.stream().map(SamzaSqlQueryParser.QueryInfo::getSources).flatMap(Collection::stream)
+            .collect(Collectors.toSet()),
+        queryInfo.stream().map(SamzaSqlQueryParser.QueryInfo::getSink).collect(Collectors.toSet()));
+
     QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig);
-    SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0);
-    StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(appDesc -> { }, samzaConfig);
+    StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(streamApp -> { }, samzaConfig);
 
-    translator.translate(queryInfo, streamAppDesc);
+    translator.translate(queryInfo.get(0), streamAppDesc);
   }
 
   @Test (expected = SamzaException.class)
   public void testTranslateStreamTableJoinWithThetaCondition() {
     Map<String, String> config = SamzaSqlTestConfig.fetchStaticConfigsWithFactories(configs, 1);
     String sql =
-        "Insert into testavro.enrichedPageViewTopic"
+        "Insert into testavro.enrichedPageViewTopic(profileName, pageKey)"
             + " select p.name as profileName, pv.pageKey"
             + " from testavro.PAGEVIEW as pv"
             + " join testavro.PROFILE.`$table` as p"
             + " on p.id <> pv.profileId";
     config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, sql);
     Config samzaConfig = SamzaSqlApplicationRunner.computeSamzaConfigs(true, new MapConfig(config));
-    SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config));
+
+    List<String> sqlStmts = fetchSqlFromConfig(config);
+    List<SamzaSqlQueryParser.QueryInfo> queryInfo = fetchQueryInfo(sqlStmts);
+    SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config),
+        queryInfo.stream().map(SamzaSqlQueryParser.QueryInfo::getSources).flatMap(Collection::stream)
+            .collect(Collectors.toSet()),
+        queryInfo.stream().map(SamzaSqlQueryParser.QueryInfo::getSink).collect(Collectors.toSet()));
+
     QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig);
-    SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0);
-    StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(appDesc -> { }, samzaConfig);
-    translator.translate(queryInfo, streamAppDesc);
+    StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(streamApp -> { }, samzaConfig);
+    translator.translate(queryInfo.get(0), streamAppDesc);
   }
 
   @Test (expected = SamzaException.class)
   public void testTranslateStreamTableCrossJoin() {
     Map<String, String> config = SamzaSqlTestConfig.fetchStaticConfigsWithFactories(configs, 1);
     String sql =
-        "Insert into testavro.enrichedPageViewTopic"
+        "Insert into testavro.enrichedPageViewTopic(profileName, pageKey)"
             + " select p.name as profileName, pv.pageKey"
             + " from testavro.PAGEVIEW as pv, testavro.PROFILE.`$table` as p";
     config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, sql);
     Config samzaConfig = SamzaSqlApplicationRunner.computeSamzaConfigs(true, new MapConfig(config));
-    SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config));
+
+    List<String> sqlStmts = fetchSqlFromConfig(config);
+    List<SamzaSqlQueryParser.QueryInfo> queryInfo = fetchQueryInfo(sqlStmts);
+    SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config),
+        queryInfo.stream().map(SamzaSqlQueryParser.QueryInfo::getSources).flatMap(Collection::stream)
+            .collect(Collectors.toSet()),
+        queryInfo.stream().map(SamzaSqlQueryParser.QueryInfo::getSink).collect(Collectors.toSet()));
+
     QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig);
-    SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0);
-    StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(appDesc -> { }, samzaConfig);
-    translator.translate(queryInfo, streamAppDesc);
+    StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(streamApp -> { }, samzaConfig);
+    translator.translate(queryInfo.get(0), streamAppDesc);
   }
 
   @Test (expected = SamzaException.class)
   public void testTranslateStreamTableJoinWithAndLiteralCondition() {
     Map<String, String> config = SamzaSqlTestConfig.fetchStaticConfigsWithFactories(configs, 1);
     String sql =
-        "Insert into testavro.enrichedPageViewTopic"
+        "Insert into testavro.enrichedPageViewTopic(profileName, pageKey)"
             + " select p.name as profileName, pv.pageKey"
             + " from testavro.PAGEVIEW as pv"
             + " join testavro.PROFILE.`$table` as p"
             + " on p.id = pv.profileId and p.name = 'John'";
     config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, sql);
     Config samzaConfig = SamzaSqlApplicationRunner.computeSamzaConfigs(true, new MapConfig(config));
-    SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config));
+
+    List<String> sqlStmts = fetchSqlFromConfig(config);
+    List<SamzaSqlQueryParser.QueryInfo> queryInfo = fetchQueryInfo(sqlStmts);
+    SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config),
+        queryInfo.stream().map(SamzaSqlQueryParser.QueryInfo::getSources).flatMap(Collection::stream)
+            .collect(Collectors.toSet()),
+        queryInfo.stream().map(SamzaSqlQueryParser.QueryInfo::getSink).collect(Collectors.toSet()));
+
     QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig);
-    SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0);
-    StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(appDesc -> { }, samzaConfig);
-    translator.translate(queryInfo, streamAppDesc);
+    StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(streamApp -> { }, samzaConfig);
+    translator.translate(queryInfo.get(0), streamAppDesc);
   }
 
   @Test (expected = SamzaException.class)
   public void testTranslateStreamTableJoinWithSubQuery() {
     Map<String, String> config = SamzaSqlTestConfig.fetchStaticConfigsWithFactories(configs, 1);
     String sql =
-        "Insert into testavro.enrichedPageViewTopic"
+        "Insert into testavro.enrichedPageViewTopic(profileName, pageKey)"
             + " select p.name as profileName, pv.pageKey"
             + " from testavro.PAGEVIEW as pv"
             + " where exists "
@@ -311,83 +368,113 @@ public class TestQueryTranslator {
             + " where p.id = pv.profileId)";
     config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, sql);
     Config samzaConfig = SamzaSqlApplicationRunner.computeSamzaConfigs(true, new MapConfig(config));
-    SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config));
+
+    List<String> sqlStmts = fetchSqlFromConfig(config);
+    List<SamzaSqlQueryParser.QueryInfo> queryInfo = fetchQueryInfo(sqlStmts);
+    SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config),
+        queryInfo.stream().map(SamzaSqlQueryParser.QueryInfo::getSources).flatMap(Collection::stream)
+            .collect(Collectors.toSet()),
+        queryInfo.stream().map(SamzaSqlQueryParser.QueryInfo::getSink).collect(Collectors.toSet()));
+
     QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig);
-    SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0);
-    StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(appDesc -> { }, samzaConfig);
-    translator.translate(queryInfo, streamAppDesc);
+    StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(streamApp -> { }, samzaConfig);
+    translator.translate(queryInfo.get(0), streamAppDesc);
   }
 
   @Test (expected = SamzaException.class)
   public void testTranslateTableTableJoin() {
     Map<String, String> config = SamzaSqlTestConfig.fetchStaticConfigsWithFactories(configs, 1);
     String sql =
-        "Insert into testavro.enrichedPageViewTopic"
+        "Insert into testavro.enrichedPageViewTopic(profileName, pageKey)"
             + " select p.name as profileName, pv.pageKey"
             + " from testavro.PAGEVIEW.`$table` as pv"
             + " join testavro.PROFILE.`$table` as p"
             + " on p.id = pv.profileId";
     config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, sql);
     Config samzaConfig = SamzaSqlApplicationRunner.computeSamzaConfigs(true, new MapConfig(config));
-    SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config));
+
+    List<String> sqlStmts = fetchSqlFromConfig(config);
+    List<SamzaSqlQueryParser.QueryInfo> queryInfo = fetchQueryInfo(sqlStmts);
+    SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config),
+        queryInfo.stream().map(SamzaSqlQueryParser.QueryInfo::getSources).flatMap(Collection::stream)
+            .collect(Collectors.toSet()),
+        queryInfo.stream().map(SamzaSqlQueryParser.QueryInfo::getSink).collect(Collectors.toSet()));
+
     QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig);
-    SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0);
-    StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(appDesc -> { }, samzaConfig);
-    translator.translate(queryInfo, streamAppDesc);
+    StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(streamApp -> { }, samzaConfig);
+    translator.translate(queryInfo.get(0), streamAppDesc);
   }
 
   @Test (expected = SamzaException.class)
   public void testTranslateStreamStreamJoin() {
     Map<String, String> config = SamzaSqlTestConfig.fetchStaticConfigsWithFactories(configs, 1);
     String sql =
-        "Insert into testavro.enrichedPageViewTopic"
+        "Insert into testavro.enrichedPageViewTopic(profileName, pageKey)"
             + " select p.name as profileName, pv.pageKey"
             + " from testavro.PAGEVIEW as pv"
             + " join testavro.PROFILE as p"
             + " on p.id = pv.profileId";
     config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, sql);
     Config samzaConfig = SamzaSqlApplicationRunner.computeSamzaConfigs(true, new MapConfig(config));
-    SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config));
+
+    List<String> sqlStmts = fetchSqlFromConfig(config);
+    List<SamzaSqlQueryParser.QueryInfo> queryInfo = fetchQueryInfo(sqlStmts);
+    SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config),
+        queryInfo.stream().map(SamzaSqlQueryParser.QueryInfo::getSources).flatMap(Collection::stream)
+            .collect(Collectors.toSet()),
+        queryInfo.stream().map(SamzaSqlQueryParser.QueryInfo::getSink).collect(Collectors.toSet()));
+
     QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig);
-    SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0);
-    StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(appDesc -> { }, samzaConfig);
-    translator.translate(queryInfo, streamAppDesc);
+    StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(streamApp -> { }, samzaConfig);
+    translator.translate(queryInfo.get(0), streamAppDesc);
   }
 
   @Test (expected = SamzaException.class)
   public void testTranslateJoinWithIncorrectLeftJoin() {
     Map<String, String> config = SamzaSqlTestConfig.fetchStaticConfigsWithFactories(configs, 1);
     String sql =
-        "Insert into testavro.enrichedPageViewTopic"
+        "Insert into testavro.enrichedPageViewTopic(profileName, pageKey)"
             + " select p.name as profileName, pv.pageKey"
             + " from testavro.PAGEVIEW.`$table` as pv"
             + " left join testavro.PROFILE as p"
             + " on p.id = pv.profileId";
     config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, sql);
     Config samzaConfig = SamzaSqlApplicationRunner.computeSamzaConfigs(true, new MapConfig(config));
-    SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config));
+
+    List<String> sqlStmts = fetchSqlFromConfig(config);
+    List<SamzaSqlQueryParser.QueryInfo> queryInfo = fetchQueryInfo(sqlStmts);
+    SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config),
+        queryInfo.stream().map(SamzaSqlQueryParser.QueryInfo::getSources).flatMap(Collection::stream)
+            .collect(Collectors.toSet()),
+        queryInfo.stream().map(SamzaSqlQueryParser.QueryInfo::getSink).collect(Collectors.toSet()));
+
     QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig);
-    SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0);
-    StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(appDesc -> { }, samzaConfig);
-    translator.translate(queryInfo, streamAppDesc);
+    StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(streamApp -> { }, samzaConfig);
+    translator.translate(queryInfo.get(0), streamAppDesc);
   }
 
   @Test (expected = SamzaException.class)
   public void testTranslateJoinWithIncorrectRightJoin() {
     Map<String, String> config = SamzaSqlTestConfig.fetchStaticConfigsWithFactories(configs, 1);
     String sql =
-        "Insert into testavro.enrichedPageViewTopic"
+        "Insert into testavro.enrichedPageViewTopic(profileName, pageKey)"
             + " select p.name as profileName, pv.pageKey"
             + " from testavro.PAGEVIEW as pv"
             + " right join testavro.PROFILE.`$table` as p"
             + " on p.id = pv.profileId";
     config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, sql);
     Config samzaConfig = SamzaSqlApplicationRunner.computeSamzaConfigs(true, new MapConfig(config));
-    SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config));
+
+    List<String> sqlStmts = fetchSqlFromConfig(config);
+    List<SamzaSqlQueryParser.QueryInfo> queryInfo = fetchQueryInfo(sqlStmts);
+    SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config),
+        queryInfo.stream().map(SamzaSqlQueryParser.QueryInfo::getSources).flatMap(Collection::stream)
+            .collect(Collectors.toSet()),
+        queryInfo.stream().map(SamzaSqlQueryParser.QueryInfo::getSink).collect(Collectors.toSet()));
+
     QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig);
-    SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0);
-    StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(appDesc -> { }, samzaConfig);
-    translator.translate(queryInfo, streamAppDesc);
+    StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(streamApp -> { }, samzaConfig);
+    translator.translate(queryInfo.get(0), streamAppDesc);
   }
 
   @Test (expected = SamzaException.class)
@@ -398,55 +485,73 @@ public class TestQueryTranslator {
     config.put(configIOResolverDomain + SamzaSqlApplicationConfig.CFG_FACTORY,
         ConfigBasedIOResolverFactory.class.getName());
     String sql =
-        "Insert into testavro.enrichedPageViewTopic"
+        "Insert into testavro.enrichedPageViewTopic(profileName, pageKey)"
             + " select p.name as profileName, pv.pageKey"
             + " from testavro.PAGEVIEW as pv"
             + " join testavro.`$table` as p"
             + " on p.id = pv.profileId";
     config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, sql);
     Config samzaConfig = SamzaSqlApplicationRunner.computeSamzaConfigs(true, new MapConfig(config));
-    SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config));
+
+    List<String> sqlStmts = fetchSqlFromConfig(config);
+    List<SamzaSqlQueryParser.QueryInfo> queryInfo = fetchQueryInfo(sqlStmts);
+    SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config),
+        queryInfo.stream().map(SamzaSqlQueryParser.QueryInfo::getSources).flatMap(Collection::stream)
+            .collect(Collectors.toSet()),
+        queryInfo.stream().map(SamzaSqlQueryParser.QueryInfo::getSink).collect(Collectors.toSet()));
+
     QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig);
-    SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0);
-    StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(appDesc -> { }, samzaConfig);
-    translator.translate(queryInfo, streamAppDesc);
+    StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(streamApp -> { }, samzaConfig);
+    translator.translate(queryInfo.get(0), streamAppDesc);
   }
 
   @Test (expected = SamzaException.class)
   public void testTranslateStreamTableInnerJoinWithUdf() {
     Map<String, String> config = SamzaSqlTestConfig.fetchStaticConfigsWithFactories(configs, 10);
     String sql =
-        "Insert into testavro.enrichedPageViewTopic"
+        "Insert into testavro.enrichedPageViewTopic(profileName, pageKey)"
             + " select p.name as profileName, pv.pageKey"
             + " from testavro.PAGEVIEW as pv"
             + " join testavro.PROFILE.`$table` as p"
             + " on MyTest(p.id) = MyTest(pv.profileId)";
     config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, sql);
     Config samzaConfig = SamzaSqlApplicationRunner.computeSamzaConfigs(true, new MapConfig(config));
-    SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config));
+
+    List<String> sqlStmts = fetchSqlFromConfig(config);
+    List<SamzaSqlQueryParser.QueryInfo> queryInfo = fetchQueryInfo(sqlStmts);
+    SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config),
+        queryInfo.stream().map(SamzaSqlQueryParser.QueryInfo::getSources).flatMap(Collection::stream)
+            .collect(Collectors.toSet()),
+        queryInfo.stream().map(SamzaSqlQueryParser.QueryInfo::getSink).collect(Collectors.toSet()));
+
     QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig);
-    SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0);
-    StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(appDesc -> { }, samzaConfig);
-    translator.translate(queryInfo, streamAppDesc);
+    StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(streamApp -> { }, samzaConfig);
+    translator.translate(queryInfo.get(0), streamAppDesc);
   }
 
   @Test
   public void testTranslateStreamTableInnerJoin() {
     Map<String, String> config = SamzaSqlTestConfig.fetchStaticConfigsWithFactories(configs, 10);
     String sql =
-        "Insert into testavro.enrichedPageViewTopic"
+        "Insert into testavro.enrichedPageViewTopic(profileName, pageKey)"
             + " select p.name as profileName, pv.pageKey"
             + " from testavro.PAGEVIEW as pv"
             + " join testavro.PROFILE.`$table` as p"
             + " on p.id = pv.profileId";
     config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, sql);
     Config samzaConfig = SamzaSqlApplicationRunner.computeSamzaConfigs(true, new MapConfig(config));
-    SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config));
+
+    List<String> sqlStmts = fetchSqlFromConfig(config);
+    List<SamzaSqlQueryParser.QueryInfo> queryInfo = fetchQueryInfo(sqlStmts);
+    SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config),
+        queryInfo.stream().map(SamzaSqlQueryParser.QueryInfo::getSources).flatMap(Collection::stream)
+            .collect(Collectors.toSet()),
+        queryInfo.stream().map(SamzaSqlQueryParser.QueryInfo::getSink).collect(Collectors.toSet()));
+
     QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig);
-    SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0);
-    StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(appDesc -> { }, samzaConfig);
+    StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(streamApp -> { }, samzaConfig);
 
-    translator.translate(queryInfo, streamAppDesc);
+    translator.translate(queryInfo.get(0), streamAppDesc);
     OperatorSpecGraph specGraph = streamAppDesc.getOperatorSpecGraph();
 
     StreamConfig streamConfig = new StreamConfig(samzaConfig);
@@ -487,19 +592,25 @@ public class TestQueryTranslator {
   public void testTranslateStreamTableLeftJoin() {
     Map<String, String> config = SamzaSqlTestConfig.fetchStaticConfigsWithFactories(configs, 10);
     String sql =
-        "Insert into testavro.enrichedPageViewTopic"
+        "Insert into testavro.enrichedPageViewTopic(profileName, pageKey)"
             + " select p.name as profileName, pv.pageKey"
             + " from testavro.PAGEVIEW as pv"
             + " left join testavro.PROFILE.`$table` as p"
             + " on p.id = pv.profileId";
     config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, sql);
     Config samzaConfig = SamzaSqlApplicationRunner.computeSamzaConfigs(true, new MapConfig(config));
-    SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config));
+
+    List<String> sqlStmts = fetchSqlFromConfig(config);
+    List<SamzaSqlQueryParser.QueryInfo> queryInfo = fetchQueryInfo(sqlStmts);
+    SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config),
+        queryInfo.stream().map(SamzaSqlQueryParser.QueryInfo::getSources).flatMap(Collection::stream)
+            .collect(Collectors.toSet()),
+        queryInfo.stream().map(SamzaSqlQueryParser.QueryInfo::getSink).collect(Collectors.toSet()));
+
     QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig);
-    SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0);
-    StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(appDesc -> { }, samzaConfig);
+    StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(streamApp -> { }, samzaConfig);
 
-    translator.translate(queryInfo, streamAppDesc);
+    translator.translate(queryInfo.get(0), streamAppDesc);
 
     OperatorSpecGraph specGraph = streamAppDesc.getOperatorSpecGraph();
 
@@ -541,18 +652,24 @@ public class TestQueryTranslator {
   public void testTranslateStreamTableRightJoin() {
     Map<String, String> config = SamzaSqlTestConfig.fetchStaticConfigsWithFactories(configs, 10);
     String sql =
-        "Insert into testavro.enrichedPageViewTopic"
+        "Insert into testavro.enrichedPageViewTopic(profileName, pageKey)"
             + " select p.name as profileName, pv.pageKey"
             + " from testavro.PROFILE.`$table` as p"
             + " right join testavro.PAGEVIEW as pv"
             + " on p.id = pv.profileId";
     config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, sql);
     Config samzaConfig = SamzaSqlApplicationRunner.computeSamzaConfigs(true, new MapConfig(config));
-    SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config));
+
+    List<String> sqlStmts = fetchSqlFromConfig(config);
+    List<SamzaSqlQueryParser.QueryInfo> queryInfo = fetchQueryInfo(sqlStmts);
+    SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config),
+        queryInfo.stream().map(SamzaSqlQueryParser.QueryInfo::getSources).flatMap(Collection::stream)
+            .collect(Collectors.toSet()),
+        queryInfo.stream().map(SamzaSqlQueryParser.QueryInfo::getSink).collect(Collectors.toSet()));
+
     QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig);
-    SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0);
-    StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(appDesc -> { }, samzaConfig);
-    translator.translate(queryInfo, streamAppDesc);
+    StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(streamApp -> { }, samzaConfig);
+    translator.translate(queryInfo.get(0), streamAppDesc);
 
     OperatorSpecGraph specGraph = streamAppDesc.getOperatorSpecGraph();
 
@@ -594,19 +711,25 @@ public class TestQueryTranslator {
   public void testTranslateGroupBy() {
     Map<String, String> config = SamzaSqlTestConfig.fetchStaticConfigsWithFactories(configs, 10);
     String sql =
-        "Insert into testavro.pageViewCountTopic"
+        "Insert into testavro.pageViewCountTopic(jobName, pageKey, `count`)"
             + " select 'SampleJob' as jobName, pv.pageKey, count(*) as `count`"
             + " from testavro.PAGEVIEW as pv"
             + " where pv.pageKey = 'job' or pv.pageKey = 'inbox'"
             + " group by (pv.pageKey)";
     config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, sql);
     Config samzaConfig = SamzaSqlApplicationRunner.computeSamzaConfigs(true, new MapConfig(config));
-    SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config));
+
+    List<String> sqlStmts = fetchSqlFromConfig(config);
+    List<SamzaSqlQueryParser.QueryInfo> queryInfo = fetchQueryInfo(sqlStmts);
+    SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config),
+        queryInfo.stream().map(SamzaSqlQueryParser.QueryInfo::getSources).flatMap(Collection::stream)
+            .collect(Collectors.toSet()),
+        queryInfo.stream().map(SamzaSqlQueryParser.QueryInfo::getSink).collect(Collectors.toSet()));
+
     QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig);
-    SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0);
-    StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(appDesc -> { }, samzaConfig);
+    StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(streamApp -> { }, samzaConfig);
 
-    translator.translate(queryInfo, streamAppDesc);
+    translator.translate(queryInfo.get(0), streamAppDesc);
     OperatorSpecGraph specGraph = streamAppDesc.getOperatorSpecGraph();
 
     Assert.assertEquals(1, specGraph.getInputOperators().size());
@@ -619,16 +742,22 @@ public class TestQueryTranslator {
   public void testTranslateGroupByWithSumAggregator() {
     Map<String, String> config = SamzaSqlTestConfig.fetchStaticConfigsWithFactories(configs, 10);
     String sql =
-        "Insert into testavro.pageViewCountTopic"
+        "Insert into testavro.pageViewCountTopic(jobName, pageKey, `sum`)"
             + " select 'SampleJob' as jobName, pv.pageKey, sum(pv.profileId) as `sum`"
             + " from testavro.PAGEVIEW as pv" + " where pv.pageKey = 'job' or pv.pageKey = 'inbox'"
             + " group by (pv.pageKey)";
     config.put(SamzaSqlApplicationConfig.CFG_SQL_STMT, sql);
     Config samzaConfig = SamzaSqlApplicationRunner.computeSamzaConfigs(true, new MapConfig(config));
-    SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config));
+
+    List<String> sqlStmts = fetchSqlFromConfig(config);
+    List<SamzaSqlQueryParser.QueryInfo> queryInfo = fetchQueryInfo(sqlStmts);
+    SamzaSqlApplicationConfig samzaSqlApplicationConfig = new SamzaSqlApplicationConfig(new MapConfig(config),
+        queryInfo.stream().map(SamzaSqlQueryParser.QueryInfo::getSources).flatMap(Collection::stream)
+            .collect(Collectors.toSet()),
+        queryInfo.stream().map(SamzaSqlQueryParser.QueryInfo::getSink).collect(Collectors.toSet()));
+
     QueryTranslator translator = new QueryTranslator(samzaSqlApplicationConfig);
-    SamzaSqlQueryParser.QueryInfo queryInfo = samzaSqlApplicationConfig.getQueryInfo().get(0);
-    StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(appDesc -> { }, samzaConfig);
-    translator.translate(queryInfo, streamAppDesc);
+    StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(streamApp -> { }, samzaConfig);
+    translator.translate(queryInfo.get(0), streamAppDesc);
   }
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/dec16392/samza-test/src/test/java/org/apache/samza/test/samzasql/TestSamzaSqlEndToEnd.java
----------------------------------------------------------------------
diff --git a/samza-test/src/test/java/org/apache/samza/test/samzasql/TestSamzaSqlEndToEnd.java b/samza-test/src/test/java/org/apache/samza/test/samzasql/TestSamzaSqlEndToEnd.java
index c211f03..919c91a 100644
--- a/samza-test/src/test/java/org/apache/samza/test/samzasql/TestSamzaSqlEndToEnd.java
+++ b/samza-test/src/test/java/org/apache/samza/test/samzasql/TestSamzaSqlEndToEnd.java
@@ -75,7 +75,28 @@ public class TestSamzaSqlEndToEnd extends AbstractIntegrationTestHarness {
 
     TestAvroSystemFactory.messages.clear();
     Map<String, String> staticConfigs = SamzaSqlTestConfig.fetchStaticConfigsWithFactories(configs, numMessages);
-    String sql1 = "Insert into testavro.outputTopic select id, TIMESTAMPDIFF(HOUR, CURRENT_TIMESTAMP, LOCALTIMESTAMP) + MONTH(CURRENT_DATE) as long_value from testavro.SIMPLE1";
+    String sql1 = "Insert into testavro.simpleOutputTopic select * from testavro.SIMPLE1";
+    List<String> sqlStmts = Arrays.asList(sql1);
+    staticConfigs.put(SamzaSqlApplicationConfig.CFG_SQL_STMTS_JSON, JsonUtil.toJson(sqlStmts));
+    SamzaSqlApplicationRunner runner = new SamzaSqlApplicationRunner(true, new MapConfig(staticConfigs));
+    runner.runAndWaitForFinish();
+
+    List<Integer> outMessages = TestAvroSystemFactory.messages.stream()
+        .map(x -> Integer.valueOf(((GenericRecord) x.getMessage()).get("id").toString()))
+        .sorted()
+        .collect(Collectors.toList());
+    Assert.assertEquals(numMessages, outMessages.size());
+    Assert.assertTrue(IntStream.range(0, numMessages).boxed().collect(Collectors.toList()).equals(outMessages));
+  }
+
+  @Test
+  public void testEndToEndWithProjection() throws Exception {
+    int numMessages = 20;
+
+    TestAvroSystemFactory.messages.clear();
+    Map<String, String> staticConfigs = SamzaSqlTestConfig.fetchStaticConfigsWithFactories(configs, numMessages);
+    String sql1 = "Insert into testavro.outputTopic(id, long_value) "
+        + " select id, TIMESTAMPDIFF(HOUR, CURRENT_TIMESTAMP, LOCALTIMESTAMP) + MONTH(CURRENT_DATE) as long_value from testavro.SIMPLE1";
     List<String> sqlStmts = Arrays.asList(sql1);
     staticConfigs.put(SamzaSqlApplicationConfig.CFG_SQL_STMTS_JSON, JsonUtil.toJson(sqlStmts));
     SamzaSqlApplicationRunner runner = new SamzaSqlApplicationRunner(true, new MapConfig(staticConfigs));
@@ -97,9 +118,9 @@ public class TestSamzaSqlEndToEnd extends AbstractIntegrationTestHarness {
 
     LOG.info(" Class Path : " + RelOptUtil.class.getProtectionDomain().getCodeSource().getLocation().toURI().getPath());
     String sql1 =
-        "Insert into testavro.outputTopic "
-            + "select Flatten(array_values) as string_value, id, bytes_value, fixed_value "
-            + "from testavro.COMPLEX1";
+        "Insert into testavro.outputTopic(string_value, id, bytes_value, fixed_value, float_value) "
+            + " select Flatten(array_values) as string_value, id, bytes_value, fixed_value, float_value "
+            + " from testavro.COMPLEX1";
     List<String> sqlStmts = Collections.singletonList(sql1);
     staticConfigs.put(SamzaSqlApplicationConfig.CFG_SQL_STMTS_JSON, JsonUtil.toJson(sqlStmts));
     SamzaSqlApplicationRunner runner = new SamzaSqlApplicationRunner(true, new MapConfig(staticConfigs));
@@ -121,7 +142,7 @@ public class TestSamzaSqlEndToEnd extends AbstractIntegrationTestHarness {
     TestAvroSystemFactory.messages.clear();
     Map<String, String> staticConfigs = SamzaSqlTestConfig.fetchStaticConfigsWithFactories(configs, numMessages);
     String sql1 =
-        "Insert into testavro.outputTopic select Flatten(a) as id from (select MyTestArray(id) a from testavro.SIMPLE1)";
+        "Insert into testavro.outputTopic(id) select Flatten(a) as id from (select MyTestArray(id) a from testavro.SIMPLE1)";
     List<String> sqlStmts = Collections.singletonList(sql1);
     staticConfigs.put(SamzaSqlApplicationConfig.CFG_SQL_STMTS_JSON, JsonUtil.toJson(sqlStmts));
     SamzaSqlApplicationRunner runner = new SamzaSqlApplicationRunner(true, new MapConfig(staticConfigs));
@@ -142,7 +163,8 @@ public class TestSamzaSqlEndToEnd extends AbstractIntegrationTestHarness {
     int numMessages = 20;
     TestAvroSystemFactory.messages.clear();
     Map<String, String> staticConfigs = SamzaSqlTestConfig.fetchStaticConfigsWithFactories(configs, numMessages);
-    String sql1 = "Insert into testavro.outputTopic select id, MyTest(id) as long_value from testavro.SIMPLE1";
+    String sql1 = "Insert into testavro.outputTopic(id, long_value) "
+        + "select id, MyTest(id) as long_value from testavro.SIMPLE1";
     List<String> sqlStmts = Collections.singletonList(sql1);
     staticConfigs.put(SamzaSqlApplicationConfig.CFG_SQL_STMTS_JSON, JsonUtil.toJson(sqlStmts));
     SamzaSqlApplicationRunner runner = new SamzaSqlApplicationRunner(true, new MapConfig(staticConfigs));
@@ -166,7 +188,11 @@ public class TestSamzaSqlEndToEnd extends AbstractIntegrationTestHarness {
     int numMessages = 20;
     TestAvroSystemFactory.messages.clear();
     Map<String, String> staticConfigs = SamzaSqlTestConfig.fetchStaticConfigsWithFactories(configs, numMessages);
-    String sql1 = "Insert into testavro.outputTopic select id from testavro.SIMPLE1 where RegexMatch('.*4', Name)";
+    String sql1 =
+        "Insert into testavro.outputTopic(id) "
+            + "select id "
+            + "from testavro.SIMPLE1 "
+            + "where RegexMatch('.*4', name)";
     List<String> sqlStmts = Collections.singletonList(sql1);
     staticConfigs.put(SamzaSqlApplicationConfig.CFG_SQL_STMTS_JSON, JsonUtil.toJson(sqlStmts));
     SamzaSqlApplicationRunner runner = new SamzaSqlApplicationRunner(true, new MapConfig(staticConfigs));
@@ -186,7 +212,8 @@ public class TestSamzaSqlEndToEnd extends AbstractIntegrationTestHarness {
     staticConfigs.putAll(configs);
     String sql =
         "Insert into testavro.enrichedPageViewTopic "
-            + "select pv.pageKey, p.name as profileName "
+            + "select pv.pageKey as __key__, pv.pageKey as pageKey, coalesce(null, 'N/A') as companyName,"
+            + "       p.name as profileName, p.address as profileAddress "
             + "from testavro.PROFILE.`$table` as p "
             + "join testavro.PAGEVIEW as pv "
             + " on p.id = pv.profileId";
@@ -215,7 +242,8 @@ public class TestSamzaSqlEndToEnd extends AbstractIntegrationTestHarness {
     staticConfigs.putAll(configs);
     String sql =
         "Insert into testavro.enrichedPageViewTopic "
-            + "select pv.pageKey, p.name as profileName, p.address as profileAddress "
+            + "select pv.pageKey as __key__, pv.pageKey as pageKey, p.name as companyName, p.name as profileName,"
+            + "       p.address as profileAddress "
             + "from testavro.PROFILE.`$table` as p "
             + "join testavro.PAGEVIEW as pv "
             + " on p.id = pv.profileId";
@@ -249,7 +277,8 @@ public class TestSamzaSqlEndToEnd extends AbstractIntegrationTestHarness {
     staticConfigs.putAll(configs);
     String sql =
         "Insert into testavro.enrichedPageViewTopic "
-            + "select pv.pageKey, p.name as profileName "
+            + "select pv.pageKey as __key__, pv.pageKey as pageKey, p.name as companyName, p.name as profileName,"
+            + "       p.address as profileAddress "
             + "from testavro.PROFILE.`$table` as p "
             + "join testavro.PAGEVIEW as pv "
             + " on p.id = pv.profileId "
@@ -282,7 +311,8 @@ public class TestSamzaSqlEndToEnd extends AbstractIntegrationTestHarness {
     Map<String, String> staticConfigs = SamzaSqlTestConfig.fetchStaticConfigsWithFactories(configs, numMessages, true);
     String sql =
         "Insert into testavro.enrichedPageViewTopic "
-            + "select pv.pageKey, p.name as profileName "
+            + "select pv.pageKey as __key__, pv.pageKey as pageKey, p.name as companyName, p.name as profileName,"
+            + "       p.address as profileAddress "
             + "from testavro.PAGEVIEW as pv "
             + "join testavro.PROFILE.`$table` as p "
             + " on pv.profileId = p.id";
@@ -311,7 +341,8 @@ public class TestSamzaSqlEndToEnd extends AbstractIntegrationTestHarness {
     Map<String, String> staticConfigs = SamzaSqlTestConfig.fetchStaticConfigsWithFactories(configs, numMessages, true);
     String sql =
         "Insert into testavro.enrichedPageViewTopic "
-            + "select pv.pageKey, p.name as profileName "
+            + "select pv.pageKey as __key__, pv.pageKey as pageKey, p.name as companyName, p.name as profileName,"
+            + "       p.address as profileAddress "
             + "from testavro.PAGEVIEW as pv "
             + "left join testavro.PROFILE.`$table` as p "
             + " on pv.profileId = p.id";
@@ -340,7 +371,8 @@ public class TestSamzaSqlEndToEnd extends AbstractIntegrationTestHarness {
     Map<String, String> staticConfigs = SamzaSqlTestConfig.fetchStaticConfigsWithFactories(configs, numMessages, true);
     String sql =
         "Insert into testavro.enrichedPageViewTopic "
-            + "select pv.pageKey, p.name as profileName "
+            + "select pv.pageKey as __key__, pv.pageKey as pageKey, p.name as companyName, p.name as profileName,"
+            + "       p.address as profileAddress "
             + "from testavro.PROFILE.`$table` as p "
             + "right join testavro.PAGEVIEW as pv "
             + " on p.id = pv.profileId";
@@ -369,7 +401,8 @@ public class TestSamzaSqlEndToEnd extends AbstractIntegrationTestHarness {
     Map<String, String> staticConfigs = SamzaSqlTestConfig.fetchStaticConfigsWithFactories(configs, numMessages);
     String sql =
         "Insert into testavro.enrichedPageViewTopic "
-            + "select pv.pageKey, p.name as profileName, c.name as companyName "
+            + "select pv.pageKey as __key__, pv.pageKey as pageKey, c.name as companyName, p.name as profileName,"
+            + "       p.address as profileAddress "
             + "from testavro.PAGEVIEW as pv "
             + "join testavro.PROFILE.`$table` as p "
             + " on p.id = pv.profileId "
@@ -399,7 +432,8 @@ public class TestSamzaSqlEndToEnd extends AbstractIntegrationTestHarness {
     Map<String, String> staticConfigs = SamzaSqlTestConfig.fetchStaticConfigsWithFactories(configs, numMessages);
     String sql =
         "Insert into testavro.enrichedPageViewTopic "
-            + "select pv.pageKey, p.name as profileName, c.name as companyName "
+            + "select pv.pageKey as __key__, pv.pageKey as pageKey, c.name as companyName, p.name as profileName,"
+            + "       p.address as profileAddress "
             + "from testavro.PAGEVIEW as pv "
             + "join testavro.PROFILE.`$table` as p "
             + " on p.id = pv.profileId "


[10/29] samza git commit: SAMZA-1854: Changed caching table descriptor to take table descriptor instead of run-time objects

Posted by bo...@apache.org.
SAMZA-1854: Changed caching table descriptor to take table descriptor instead of run-time objects

As per subject, changed caching table descriptor to take table descriptor instead of run-time objects
 - Added BaseHybridTableDescriptor, which models a hybrid table that may contain other tables
 - Modified StreamApplicationDescriptorImpl to also include tables contained within a hybrid table

Author: Wei Song <ws...@linkedin.com>

Reviewers: Jagadish Venkatraman <jv...@linkedin.com>

Closes #645 from weisong44/SAMZA-1854 and squashes the following commits:

2c0d1362 [Wei Song] Updated based on review comments
dd18bbee [Wei Song] Merge branch 'master' into SAMZA-1854
a6c94add [Wei Song] Merge remote-tracking branch 'upstream/master'
41299b5b [Wei Song] Merge remote-tracking branch 'upstream/master'
239a0950 [Wei Song] Merge remote-tracking branch 'upstream/master'
a87a9b04 [Wei Song] SAMZA-1854: Changed caching table descriptor to take table descriptor instead of run-time objects
eca00204 [Wei Song] Merge remote-tracking branch 'upstream/master'
51562391 [Wei Song] Merge remote-tracking branch 'upstream/master'
de708f5e [Wei Song] Merge remote-tracking branch 'upstream/master'
df2f8d7b [Wei Song] Merge remote-tracking branch 'upstream/master'
f28b491d [Wei Song] Merge remote-tracking branch 'upstream/master'
4782c61d [Wei Song] Merge remote-tracking branch 'upstream/master'
0440f75f [Wei Song] Merge remote-tracking branch 'upstream/master'
aae0f380 [Wei Song] Merge remote-tracking branch 'upstream/master'
a15a7c9a [Wei Song] Merge remote-tracking branch 'upstream/master'
5cbf9af9 [Wei Song] Merge remote-tracking branch 'upstream/master'
3f7ed71f [Wei Song] Added self to committer list


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

Branch: refs/heads/NewKafkaSystemConsumer
Commit: db6996ed99bb6a677f588f247b373345077580b0
Parents: d893912
Author: Wei Song <ws...@linkedin.com>
Authored: Thu Sep 20 11:19:06 2018 -0700
Committer: Wei Song <ws...@linkedin.com>
Committed: Thu Sep 20 11:19:06 2018 -0700

----------------------------------------------------------------------
 .../StreamApplicationDescriptorImpl.java        |  8 ++++
 .../table/caching/CachingTableDescriptor.java   | 37 +++++++++------
 .../table/hybrid/BaseHybridTableDescriptor.java | 50 ++++++++++++++++++++
 .../samza/table/caching/TestCachingTable.java   | 22 ++++++---
 .../kv/LocalStoreBackedReadWriteTable.java      |  1 +
 .../samza/test/table/TestRemoteTable.java       | 33 ++++++-------
 6 files changed, 112 insertions(+), 39 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/samza/blob/db6996ed/samza-core/src/main/java/org/apache/samza/application/StreamApplicationDescriptorImpl.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/application/StreamApplicationDescriptorImpl.java b/samza-core/src/main/java/org/apache/samza/application/StreamApplicationDescriptorImpl.java
index ae7a45d..d50b0d0 100644
--- a/samza-core/src/main/java/org/apache/samza/application/StreamApplicationDescriptorImpl.java
+++ b/samza-core/src/main/java/org/apache/samza/application/StreamApplicationDescriptorImpl.java
@@ -23,6 +23,7 @@ import com.google.common.base.Preconditions;
 import java.util.Collections;
 import java.util.HashSet;
 import java.util.LinkedHashMap;
+import java.util.List;
 import java.util.Map;
 import java.util.Optional;
 import java.util.Set;
@@ -54,6 +55,7 @@ import org.apache.samza.serializers.NoOpSerde;
 import org.apache.samza.serializers.Serde;
 import org.apache.samza.table.Table;
 import org.apache.samza.table.TableSpec;
+import org.apache.samza.table.hybrid.BaseHybridTableDescriptor;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -171,6 +173,12 @@ public class StreamApplicationDescriptorImpl extends ApplicationDescriptorImpl<S
 
   @Override
   public <K, V> Table<KV<K, V>> getTable(TableDescriptor<K, V, ?> tableDescriptor) {
+
+    if (tableDescriptor instanceof BaseHybridTableDescriptor) {
+      List<? extends TableDescriptor<K, V, ?>> tableDescs = ((BaseHybridTableDescriptor) tableDescriptor).getTableDescriptors();
+      tableDescs.forEach(td -> getTable(td));
+    }
+
     String tableId = tableDescriptor.getTableId();
     Preconditions.checkState(StringUtils.isNotBlank(tableId) && ID_PATTERN.matcher(tableId).matches(),
         String.format("tableId: %s must confirm to pattern: %s", tableId, ID_PATTERN.toString()));

http://git-wip-us.apache.org/repos/asf/samza/blob/db6996ed/samza-core/src/main/java/org/apache/samza/table/caching/CachingTableDescriptor.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/table/caching/CachingTableDescriptor.java b/samza-core/src/main/java/org/apache/samza/table/caching/CachingTableDescriptor.java
index a1accd8..4896e93 100644
--- a/samza-core/src/main/java/org/apache/samza/table/caching/CachingTableDescriptor.java
+++ b/samza-core/src/main/java/org/apache/samza/table/caching/CachingTableDescriptor.java
@@ -20,28 +20,30 @@
 package org.apache.samza.table.caching;
 
 import java.time.Duration;
+import java.util.Arrays;
 import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
 
 import org.apache.samza.operators.BaseTableDescriptor;
-import org.apache.samza.operators.KV;
-import org.apache.samza.operators.TableImpl;
-import org.apache.samza.table.Table;
+import org.apache.samza.operators.TableDescriptor;
 import org.apache.samza.table.TableSpec;
+import org.apache.samza.table.hybrid.BaseHybridTableDescriptor;
 
 import com.google.common.base.Preconditions;
 
+
 /**
  * Table descriptor for {@link CachingTable}.
  * @param <K> type of the key in the cache
  * @param <V> type of the value in the cache
  */
-public class CachingTableDescriptor<K, V> extends BaseTableDescriptor<K, V, CachingTableDescriptor<K, V>> {
+public class CachingTableDescriptor<K, V> extends BaseHybridTableDescriptor<K, V, CachingTableDescriptor<K, V>> {
   private Duration readTtl;
   private Duration writeTtl;
   private long cacheSize;
-  private Table<KV<K, V>> cache;
-  private Table<KV<K, V>> table;
+  private TableDescriptor<K, V, ?> cache;
+  private TableDescriptor<K, V, ?> table;
   private boolean isWriteAround;
 
   /**
@@ -52,6 +54,13 @@ public class CachingTableDescriptor<K, V> extends BaseTableDescriptor<K, V, Cach
   }
 
   @Override
+  public List<? extends TableDescriptor<K, V, ?>> getTableDescriptors() {
+    return cache != null
+        ? Arrays.asList(cache, table)
+        : Arrays.asList(table);
+  }
+
+  @Override
   public TableSpec getTableSpec() {
     validate();
 
@@ -59,7 +68,7 @@ public class CachingTableDescriptor<K, V> extends BaseTableDescriptor<K, V, Cach
     generateTableSpecConfig(tableSpecConfig);
 
     if (cache != null) {
-      tableSpecConfig.put(CachingTableProvider.CACHE_TABLE_ID, ((TableImpl) cache).getTableSpec().getId());
+      tableSpecConfig.put(CachingTableProvider.CACHE_TABLE_ID, ((BaseTableDescriptor) cache).getTableSpec().getId());
     } else {
       if (readTtl != null) {
         tableSpecConfig.put(CachingTableProvider.READ_TTL_MS, String.valueOf(readTtl.toMillis()));
@@ -72,31 +81,31 @@ public class CachingTableDescriptor<K, V> extends BaseTableDescriptor<K, V, Cach
       }
     }
 
-    tableSpecConfig.put(CachingTableProvider.REAL_TABLE_ID, ((TableImpl) table).getTableSpec().getId());
+    tableSpecConfig.put(CachingTableProvider.REAL_TABLE_ID, ((BaseTableDescriptor) table).getTableSpec().getId());
     tableSpecConfig.put(CachingTableProvider.WRITE_AROUND, String.valueOf(isWriteAround));
 
     return new TableSpec(tableId, serde, CachingTableProviderFactory.class.getName(), tableSpecConfig);
   }
 
   /**
-   * Specify a cache instance (as Table abstraction) to be used for caching.
+   * Specify a cache (as Table descriptor) to be used for caching.
    * Cache get is not synchronized with put for better parallelism in the read path
    * of {@link CachingTable}. As such, cache table implementation is expected to be
    * thread-safe for concurrent accesses.
-   * @param cache cache instance
+   * @param cache cache table descriptor
    * @return this descriptor
    */
-  public CachingTableDescriptor withCache(Table<KV<K, V>> cache) {
+  public CachingTableDescriptor withCache(TableDescriptor<K, V, ?> cache) {
     this.cache = cache;
     return this;
   }
 
   /**
-   * Specify the table instance for the actual table input/output.
-   * @param table table instance
+   * Specify the target table descriptor for the actual table input/output.
+   * @param table the target table descriptor
    * @return this descriptor
    */
-  public CachingTableDescriptor withTable(Table<KV<K, V>> table) {
+  public CachingTableDescriptor withTable(TableDescriptor<K, V, ?> table) {
     this.table = table;
     return this;
   }

http://git-wip-us.apache.org/repos/asf/samza/blob/db6996ed/samza-core/src/main/java/org/apache/samza/table/hybrid/BaseHybridTableDescriptor.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/table/hybrid/BaseHybridTableDescriptor.java b/samza-core/src/main/java/org/apache/samza/table/hybrid/BaseHybridTableDescriptor.java
new file mode 100644
index 0000000..48efd0c
--- /dev/null
+++ b/samza-core/src/main/java/org/apache/samza/table/hybrid/BaseHybridTableDescriptor.java
@@ -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.table.hybrid;
+
+import java.util.List;
+import org.apache.samza.operators.BaseTableDescriptor;
+import org.apache.samza.operators.TableDescriptor;
+
+/**
+ * Base class for hybrid table descriptors. A hybrid table consists of one or more
+ * table descriptors, and it orchestrates operations between them to achieve more advanced
+ * functionality.
+ *
+ * @param <K> the type of the key
+ * @param <V> the type of the value
+ * @param <D> the type of this table descriptor
+ */
+abstract public class BaseHybridTableDescriptor<K, V, D extends BaseHybridTableDescriptor<K, V, D>>
+    extends BaseTableDescriptor<K, V, D> {
+
+  /**
+   * {@inheritDoc}
+   */
+  public BaseHybridTableDescriptor(String tableId) {
+    super(tableId);
+  }
+
+  /**
+   * Get tables contained within this table.
+   * @return list of tables
+   */
+  abstract public List<? extends TableDescriptor<K, V, ?>> getTableDescriptors();
+
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/db6996ed/samza-core/src/test/java/org/apache/samza/table/caching/TestCachingTable.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/table/caching/TestCachingTable.java b/samza-core/src/test/java/org/apache/samza/table/caching/TestCachingTable.java
index 49c72dc..ec1c915 100644
--- a/samza-core/src/test/java/org/apache/samza/table/caching/TestCachingTable.java
+++ b/samza-core/src/test/java/org/apache/samza/table/caching/TestCachingTable.java
@@ -36,11 +36,11 @@ import org.apache.samza.metrics.Counter;
 import org.apache.samza.metrics.Gauge;
 import org.apache.samza.metrics.MetricsRegistry;
 import org.apache.samza.metrics.Timer;
-import org.apache.samza.operators.TableImpl;
+import org.apache.samza.operators.BaseTableDescriptor;
+import org.apache.samza.operators.TableDescriptor;
 import org.apache.samza.storage.kv.Entry;
 import org.apache.samza.table.ReadWriteTable;
 import org.apache.samza.table.ReadableTable;
-import org.apache.samza.table.Table;
 import org.apache.samza.table.TableSpec;
 import org.apache.samza.table.caching.guava.GuavaCacheTable;
 import org.apache.samza.table.caching.guava.GuavaCacheTableDescriptor;
@@ -79,12 +79,12 @@ public class TestCachingTable {
     guavaTableDesc.withCache(CacheBuilder.newBuilder().build());
     TableSpec spec = guavaTableDesc.getTableSpec();
     Assert.assertTrue(spec.getConfig().containsKey(GuavaCacheTableProvider.GUAVA_CACHE));
-    doTestSerialize(new TableImpl(guavaTableDesc.getTableSpec()));
+    doTestSerialize(guavaTableDesc);
   }
 
-  private void doTestSerialize(Table cache) {
+  private void doTestSerialize(TableDescriptor cache) {
     CachingTableDescriptor desc = new CachingTableDescriptor("1");
-    desc.withTable(new TableImpl(new TableSpec("2", null, null, new HashMap<>())));
+    desc.withTable(createDummyTableDescriptor("2"));
     if (cache == null) {
       desc.withReadTtl(Duration.ofMinutes(3));
       desc.withWriteTtl(Duration.ofMinutes(3));
@@ -153,8 +153,8 @@ public class TestCachingTable {
 
   private void doTestCacheOps(boolean isWriteAround) {
     CachingTableDescriptor desc = new CachingTableDescriptor("1");
-    desc.withTable(new TableImpl(new TableSpec("realTable", null, null, new HashMap<>())));
-    desc.withCache(new TableImpl(new TableSpec("cacheTable", null, null, new HashMap<>())));
+    desc.withTable(createDummyTableDescriptor("realTable"));
+    desc.withCache(createDummyTableDescriptor("cacheTable"));
     if (isWriteAround) {
       desc.withWriteAround();
     }
@@ -363,4 +363,12 @@ public class TestCachingTable {
     Assert.assertNull(guavaCache.getIfPresent("foo1"));
     Assert.assertNull(guavaCache.getIfPresent("foo3"));
   }
+
+  private TableDescriptor createDummyTableDescriptor(String tableId) {
+    BaseTableDescriptor tableDescriptor = mock(BaseTableDescriptor.class);
+    when(tableDescriptor.getTableId()).thenReturn(tableId);
+    when(tableDescriptor.getTableSpec()).thenReturn(
+        new TableSpec(tableId, null, null, new HashMap<>()));
+    return tableDescriptor;
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/samza/blob/db6996ed/samza-kv/src/main/java/org/apache/samza/storage/kv/LocalStoreBackedReadWriteTable.java
----------------------------------------------------------------------
diff --git a/samza-kv/src/main/java/org/apache/samza/storage/kv/LocalStoreBackedReadWriteTable.java b/samza-kv/src/main/java/org/apache/samza/storage/kv/LocalStoreBackedReadWriteTable.java
index 98c3e3c..9eeb55e 100644
--- a/samza-kv/src/main/java/org/apache/samza/storage/kv/LocalStoreBackedReadWriteTable.java
+++ b/samza-kv/src/main/java/org/apache/samza/storage/kv/LocalStoreBackedReadWriteTable.java
@@ -40,6 +40,7 @@ public class LocalStoreBackedReadWriteTable<K, V> extends LocalStoreBackedReadab
 
   /**
    * Constructs an instance of {@link LocalStoreBackedReadWriteTable}
+   * @param tableId the table Id
    * @param kvStore the backing store
    */
   public LocalStoreBackedReadWriteTable(String tableId, KeyValueStore kvStore) {

http://git-wip-us.apache.org/repos/asf/samza/blob/db6996ed/samza-test/src/test/java/org/apache/samza/test/table/TestRemoteTable.java
----------------------------------------------------------------------
diff --git a/samza-test/src/test/java/org/apache/samza/test/table/TestRemoteTable.java b/samza-test/src/test/java/org/apache/samza/test/table/TestRemoteTable.java
index 4cf99ff..e23cb58 100644
--- a/samza-test/src/test/java/org/apache/samza/test/table/TestRemoteTable.java
+++ b/samza-test/src/test/java/org/apache/samza/test/table/TestRemoteTable.java
@@ -42,6 +42,7 @@ import org.apache.samza.metrics.Counter;
 import org.apache.samza.metrics.MetricsRegistry;
 import org.apache.samza.metrics.Timer;
 import org.apache.samza.operators.KV;
+import org.apache.samza.operators.TableDescriptor;
 import org.apache.samza.operators.descriptors.GenericInputDescriptor;
 import org.apache.samza.operators.descriptors.DelegatingSystemDescriptor;
 import org.apache.samza.runtime.LocalApplicationRunner;
@@ -67,6 +68,7 @@ import static org.mockito.Matchers.any;
 import static org.mockito.Matchers.anyString;
 import static org.mockito.Mockito.doReturn;
 import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.withSettings;
 
 
 public class TestRemoteTable extends AbstractIntegrationTestHarness {
@@ -136,19 +138,18 @@ public class TestRemoteTable extends AbstractIntegrationTestHarness {
     }
   }
 
-  private <K, V> Table<KV<K, V>> getCachingTable(Table<KV<K, V>> actualTable, boolean defaultCache, String id, StreamApplicationDescriptor appDesc) {
+  private <K, V> Table<KV<K, V>> getCachingTable(TableDescriptor<K, V, ?> actualTableDesc, boolean defaultCache, String id, StreamApplicationDescriptor appDesc) {
     CachingTableDescriptor<K, V> cachingDesc = new CachingTableDescriptor<>("caching-table-" + id);
     if (defaultCache) {
       cachingDesc.withReadTtl(Duration.ofMinutes(5));
       cachingDesc.withWriteTtl(Duration.ofMinutes(5));
     } else {
-      GuavaCacheTableDescriptor<K, V> guavaDesc = new GuavaCacheTableDescriptor<>("guava-table-" + id);
-      guavaDesc.withCache(CacheBuilder.newBuilder().expireAfterAccess(5, TimeUnit.MINUTES).build());
-      Table<KV<K, V>> guavaTable = appDesc.getTable(guavaDesc);
-      cachingDesc.withCache(guavaTable);
+      GuavaCacheTableDescriptor<K, V> guavaTableDesc = new GuavaCacheTableDescriptor<>("guava-table-" + id);
+      guavaTableDesc.withCache(CacheBuilder.newBuilder().expireAfterAccess(5, TimeUnit.MINUTES).build());
+      cachingDesc.withCache(guavaTableDesc);
     }
 
-    cachingDesc.withTable(actualTable);
+    cachingDesc.withTable(actualTableDesc);
     return appDesc.getTable(cachingDesc);
   }
 
@@ -180,8 +181,8 @@ public class TestRemoteTable extends AbstractIntegrationTestHarness {
     configs.put("streams.PageView.source", Base64Serializer.serialize(pageViews));
     configs.put("streams.PageView.partitionCount", String.valueOf(partitionCount));
 
-    final RateLimiter readRateLimiter = mock(RateLimiter.class);
-    final RateLimiter writeRateLimiter = mock(RateLimiter.class);
+    final RateLimiter readRateLimiter = mock(RateLimiter.class, withSettings().serializable());
+    final RateLimiter writeRateLimiter = mock(RateLimiter.class, withSettings().serializable());
     final StreamApplication app = appDesc -> {
       RemoteTableDescriptor<Integer, TestTableData.Profile> inputTableDesc = new RemoteTableDescriptor<>("profile-table-1");
       inputTableDesc
@@ -197,17 +198,13 @@ public class TestRemoteTable extends AbstractIntegrationTestHarness {
           .withWriteFunction(writer)
           .withRateLimiter(writeRateLimiter, null, null);
 
-      Table<KV<Integer, EnrichedPageView>> outputTable = appDesc.getTable(outputTableDesc);
+      Table<KV<Integer, EnrichedPageView>> outputTable = withCache
+          ? getCachingTable(outputTableDesc, defaultCache, "output", appDesc)
+          : appDesc.getTable(outputTableDesc);
 
-      if (withCache) {
-        outputTable = getCachingTable(outputTable, defaultCache, "output", appDesc);
-      }
-
-      Table<KV<Integer, Profile>> inputTable = appDesc.getTable(inputTableDesc);
-
-      if (withCache) {
-        inputTable = getCachingTable(inputTable, defaultCache, "input", appDesc);
-      }
+      Table<KV<Integer, Profile>> inputTable = withCache
+          ? getCachingTable(inputTableDesc, defaultCache, "input", appDesc)
+          : appDesc.getTable(inputTableDesc);
 
       DelegatingSystemDescriptor ksd = new DelegatingSystemDescriptor("test");
       GenericInputDescriptor<TestTableData.PageView> isd = ksd.getInputDescriptor("PageView", new NoOpSerde<>());


[19/29] samza git commit: SAMZA-1898: New UI layout for the Samza website

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/samza/blob/b8058af0/docs/css/bootstrap.min.css
----------------------------------------------------------------------
diff --git a/docs/css/bootstrap.min.css b/docs/css/bootstrap.min.css
deleted file mode 100644
index 679272d..0000000
--- a/docs/css/bootstrap.min.css
+++ /dev/null
@@ -1,7 +0,0 @@
-/*!
- * Bootstrap v3.1.1 (http://getbootstrap.com)
- * Copyright 2011-2014 Twitter, Inc.
- * Licensed under MIT (https://github.com/twbs/bootstrap/blob/master/LICENSE)
- */
-
-/*! normalize.css v3.0.0 | MIT License | git.io/normalize */html{font-family:sans-serif;-ms-text-size-adjust:100%;-webkit-text-size-adjust:100%}body{margin:0}article,aside,details,figcaption,figure,footer,header,hgroup,main,nav,section,summary{display:block}audio,canvas,progress,video{display:inline-block;vertical-align:baseline}audio:not([controls]){display:none;height:0}[hidden],template{display:none}a{background:0 0}a:active,a:hover{outline:0}abbr[title]{border-bottom:1px dotted}b,strong{font-weight:700}dfn{font-style:italic}h1{font-size:2em;margin:.67em 0}mark{background:#ff0;color:#000}small{font-size:80%}sub,sup{font-size:75%;line-height:0;position:relative;vertical-align:baseline}sup{top:-.5em}sub{bottom:-.25em}img{border:0}svg:not(:root){overflow:hidden}figure{margin:1em 40px}hr{-moz-box-sizing:content-box;box-sizing:content-box;height:0}pre{overflow:auto}code,kbd,pre,samp{font-family:monospace,monospace;font-size:1em}button,input,optgroup,select,textarea{color:inherit;font:
 inherit;margin:0}button{overflow:visible}button,select{text-transform:none}button,html input[type=button],input[type=reset],input[type=submit]{-webkit-appearance:button;cursor:pointer}button[disabled],html input[disabled]{cursor:default}button::-moz-focus-inner,input::-moz-focus-inner{border:0;padding:0}input{line-height:normal}input[type=checkbox],input[type=radio]{box-sizing:border-box;padding:0}input[type=number]::-webkit-inner-spin-button,input[type=number]::-webkit-outer-spin-button{height:auto}input[type=search]{-webkit-appearance:textfield;-moz-box-sizing:content-box;-webkit-box-sizing:content-box;box-sizing:content-box}input[type=search]::-webkit-search-cancel-button,input[type=search]::-webkit-search-decoration{-webkit-appearance:none}fieldset{border:1px solid silver;margin:0 2px;padding:.35em .625em .75em}legend{border:0;padding:0}textarea{overflow:auto}optgroup{font-weight:700}table{border-collapse:collapse;border-spacing:0}td,th{padding:0}@media print{*{text-shadow:none!
 important;color:#000!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) ")"}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}p,h2,h3{orphans:3;widows:3}h2,h3{page-break-after:avoid}select{background:#fff!important}.navbar{display:none}.table td,.table th{background-color:#fff!important}.btn>.caret,.dropup>.btn>.caret{border-top-color:#000!important}.label{border:1px solid #000}.table{border-collapse:collapse!important}.table-bordered th,.table-bordered td{border:1px solid #ddd!important}}*{-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box}:before,:after{-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box}html{font-size:62.5%;-we
 bkit-tap-highlight-color:rgba(0,0,0,0)}body{font-family:"Helvetica Neue",Helvetica,Arial,sans-serif;font-size:14px;line-height:1.42857143;color:#333;background-color:#fff}input,button,select,textarea{font-family:inherit;font-size:inherit;line-height:inherit}a{color:#428bca;text-decoration:none}a:hover,a:focus{color:#2a6496;text-decoration:underline}a:focus{outline:thin dotted;outline:5px auto -webkit-focus-ring-color;outline-offset:-2px}figure{margin:0}img{vertical-align:middle}.img-responsive,.thumbnail>img,.thumbnail a>img,.carousel-inner>.item>img,.carousel-inner>.item>a>img{display:block;max-width:100%;height:auto}.img-rounded{border-radius:6px}.img-thumbnail{padding:4px;line-height:1.42857143;background-color:#fff;border:1px solid #ddd;border-radius:4px;-webkit-transition:all .2s ease-in-out;transition:all .2s ease-in-out;display:inline-block;max-width:100%;height:auto}.img-circle{border-radius:50%}hr{margin-top:20px;margin-bottom:20px;border:0;border-top:1px solid #eee}.sr-onl
 y{position:absolute;width:1px;height:1px;margin:-1px;padding:0;overflow:hidden;clip:rect(0,0,0,0);border:0}h1,h2,h3,h4,h5,h6,.h1,.h2,.h3,.h4,.h5,.h6{font-family:inherit;font-weight:500;line-height:1.1;color:inherit}h1 small,h2 small,h3 small,h4 small,h5 small,h6 small,.h1 small,.h2 small,.h3 small,.h4 small,.h5 small,.h6 small,h1 .small,h2 .small,h3 .small,h4 .small,h5 .small,h6 .small,.h1 .small,.h2 .small,.h3 .small,.h4 .small,.h5 .small,.h6 .small{font-weight:400;line-height:1;color:#999}h1,.h1,h2,.h2,h3,.h3{margin-top:20px;margin-bottom:10px}h1 small,.h1 small,h2 small,.h2 small,h3 small,.h3 small,h1 .small,.h1 .small,h2 .small,.h2 .small,h3 .small,.h3 .small{font-size:65%}h4,.h4,h5,.h5,h6,.h6{margin-top:10px;margin-bottom:10px}h4 small,.h4 small,h5 small,.h5 small,h6 small,.h6 small,h4 .small,.h4 .small,h5 .small,.h5 .small,h6 .small,.h6 .small{font-size:75%}h1,.h1{font-size:36px}h2,.h2{font-size:30px}h3,.h3{font-size:24px}h4,.h4{font-size:18px}h5,.h5{font-size:14px}h6,.h6{font
 -size:12px}p{margin:0 0 10px}.lead{margin-bottom:20px;font-size:16px;font-weight:200;line-height:1.4}@media (min-width:768px){.lead{font-size:21px}}small,.small{font-size:85%}cite{font-style:normal}.text-left{text-align:left}.text-right{text-align:right}.text-center{text-align:center}.text-justify{text-align:justify}.text-muted{color:#999}.text-primary{color:#428bca}a.text-primary:hover{color:#3071a9}.text-success{color:#3c763d}a.text-success:hover{color:#2b542c}.text-info{color:#31708f}a.text-info:hover{color:#245269}.text-warning{color:#8a6d3b}a.text-warning:hover{color:#66512c}.text-danger{color:#a94442}a.text-danger:hover{color:#843534}.bg-primary{color:#fff;background-color:#428bca}a.bg-primary:hover{background-color:#3071a9}.bg-success{background-color:#dff0d8}a.bg-success:hover{background-color:#c1e2b3}.bg-info{background-color:#d9edf7}a.bg-info:hover{background-color:#afd9ee}.bg-warning{background-color:#fcf8e3}a.bg-warning:hover{background-color:#f7ecb5}.bg-danger{backgroun
 d-color:#f2dede}a.bg-danger:hover{background-color:#e4b9b9}.page-header{padding-bottom:9px;margin:40px 0 20px;border-bottom:1px solid #eee}ul,ol{margin-top:0;margin-bottom:10px}ul ul,ol ul,ul ol,ol ol{margin-bottom:0}.list-unstyled{padding-left:0;list-style:none}.list-inline{padding-left:0;list-style:none;margin-left:-5px}.list-inline>li{display:inline-block;padding-left:5px;padding-right:5px}dl{margin-top:0;margin-bottom:20px}dt,dd{line-height:1.42857143}dt{font-weight:700}dd{margin-left:0}@media (min-width:768px){.dl-horizontal dt{float:left;width:160px;clear:left;text-align:right;overflow:hidden;text-overflow:ellipsis;white-space:nowrap}.dl-horizontal dd{margin-left:180px}}abbr[title],abbr[data-original-title]{cursor:help;border-bottom:1px dotted #999}.initialism{font-size:90%;text-transform:uppercase}blockquote{padding:10px 20px;margin:0 0 20px;font-size:17.5px;border-left:5px solid #eee}blockquote p:last-child,blockquote ul:last-child,blockquote ol:last-child{margin-bottom:0}bl
 ockquote footer,blockquote small,blockquote .small{display:block;font-size:80%;line-height:1.42857143;color:#999}blockquote footer:before,blockquote small:before,blockquote .small:before{content:'\2014 \00A0'}.blockquote-reverse,blockquote.pull-right{padding-right:15px;padding-left:0;border-right:5px solid #eee;border-left:0;text-align:right}.blockquote-reverse footer:before,blockquote.pull-right footer:before,.blockquote-reverse small:before,blockquote.pull-right small:before,.blockquote-reverse .small:before,blockquote.pull-right .small:before{content:''}.blockquote-reverse footer:after,blockquote.pull-right footer:after,.blockquote-reverse small:after,blockquote.pull-right small:after,.blockquote-reverse .small:after,blockquote.pull-right .small:after{content:'\00A0 \2014'}blockquote:before,blockquote:after{content:""}address{margin-bottom:20px;font-style:normal;line-height:1.42857143}code,kbd,pre,samp{font-family:Menlo,Monaco,Consolas,"Courier New",monospace}code{padding:2px 4px
 ;font-size:90%;color:#c7254e;background-color:#f9f2f4;white-space:nowrap;border-radius:4px}kbd{padding:2px 4px;font-size:90%;color:#fff;background-color:#333;border-radius:3px;box-shadow:inset 0 -1px 0 rgba(0,0,0,.25)}pre{display:block;padding:9.5px;margin:0 0 10px;font-size:13px;line-height:1.42857143;word-break:break-all;word-wrap:break-word;color:#333;background-color:#f5f5f5;border:1px solid #ccc;border-radius:4px}pre code{padding:0;font-size:inherit;color:inherit;white-space:pre-wrap;background-color:transparent;border-radius:0}.pre-scrollable{max-height:340px;overflow-y:scroll}.container{margin-right:auto;margin-left:auto;padding-left:15px;padding-right:15px}@media (min-width:768px){.container{width:750px}}@media (min-width:992px){.container{width:970px}}@media (min-width:1200px){.container{width:1170px}}.container-fluid{margin-right:auto;margin-left:auto;padding-left:15px;padding-right:15px}.row{margin-left:-15px;margin-right:-15px}.col-xs-1,.col-sm-1,.col-md-1,.col-lg-1,.col
 -xs-2,.col-sm-2,.col-md-2,.col-lg-2,.col-xs-3,.col-sm-3,.col-md-3,.col-lg-3,.col-xs-4,.col-sm-4,.col-md-4,.col-lg-4,.col-xs-5,.col-sm-5,.col-md-5,.col-lg-5,.col-xs-6,.col-sm-6,.col-md-6,.col-lg-6,.col-xs-7,.col-sm-7,.col-md-7,.col-lg-7,.col-xs-8,.col-sm-8,.col-md-8,.col-lg-8,.col-xs-9,.col-sm-9,.col-md-9,.col-lg-9,.col-xs-10,.col-sm-10,.col-md-10,.col-lg-10,.col-xs-11,.col-sm-11,.col-md-11,.col-lg-11,.col-xs-12,.col-sm-12,.col-md-12,.col-lg-12{position:relative;min-height:1px;padding-left:15px;padding-right:15px}.col-xs-1,.col-xs-2,.col-xs-3,.col-xs-4,.col-xs-5,.col-xs-6,.col-xs-7,.col-xs-8,.col-xs-9,.col-xs-10,.col-xs-11,.col-xs-12{float:left}.col-xs-12{width:100%}.col-xs-11{width:91.66666667%}.col-xs-10{width:83.33333333%}.col-xs-9{width:75%}.col-xs-8{width:66.66666667%}.col-xs-7{width:58.33333333%}.col-xs-6{width:50%}.col-xs-5{width:41.66666667%}.col-xs-4{width:33.33333333%}.col-xs-3{width:25%}.col-xs-2{width:16.66666667%}.col-xs-1{width:8.33333333%}.col-xs-pull-12{right:100%}.co
 l-xs-pull-11{right:91.66666667%}.col-xs-pull-10{right:83.33333333%}.col-xs-pull-9{right:75%}.col-xs-pull-8{right:66.66666667%}.col-xs-pull-7{right:58.33333333%}.col-xs-pull-6{right:50%}.col-xs-pull-5{right:41.66666667%}.col-xs-pull-4{right:33.33333333%}.col-xs-pull-3{right:25%}.col-xs-pull-2{right:16.66666667%}.col-xs-pull-1{right:8.33333333%}.col-xs-pull-0{right:0}.col-xs-push-12{left:100%}.col-xs-push-11{left:91.66666667%}.col-xs-push-10{left:83.33333333%}.col-xs-push-9{left:75%}.col-xs-push-8{left:66.66666667%}.col-xs-push-7{left:58.33333333%}.col-xs-push-6{left:50%}.col-xs-push-5{left:41.66666667%}.col-xs-push-4{left:33.33333333%}.col-xs-push-3{left:25%}.col-xs-push-2{left:16.66666667%}.col-xs-push-1{left:8.33333333%}.col-xs-push-0{left:0}.col-xs-offset-12{margin-left:100%}.col-xs-offset-11{margin-left:91.66666667%}.col-xs-offset-10{margin-left:83.33333333%}.col-xs-offset-9{margin-left:75%}.col-xs-offset-8{margin-left:66.66666667%}.col-xs-offset-7{margin-left:58.33333333%}.col-x
 s-offset-6{margin-left:50%}.col-xs-offset-5{margin-left:41.66666667%}.col-xs-offset-4{margin-left:33.33333333%}.col-xs-offset-3{margin-left:25%}.col-xs-offset-2{margin-left:16.66666667%}.col-xs-offset-1{margin-left:8.33333333%}.col-xs-offset-0{margin-left:0}@media (min-width:768px){.col-sm-1,.col-sm-2,.col-sm-3,.col-sm-4,.col-sm-5,.col-sm-6,.col-sm-7,.col-sm-8,.col-sm-9,.col-sm-10,.col-sm-11,.col-sm-12{float:left}.col-sm-12{width:100%}.col-sm-11{width:91.66666667%}.col-sm-10{width:83.33333333%}.col-sm-9{width:75%}.col-sm-8{width:66.66666667%}.col-sm-7{width:58.33333333%}.col-sm-6{width:50%}.col-sm-5{width:41.66666667%}.col-sm-4{width:33.33333333%}.col-sm-3{width:25%}.col-sm-2{width:16.66666667%}.col-sm-1{width:8.33333333%}.col-sm-pull-12{right:100%}.col-sm-pull-11{right:91.66666667%}.col-sm-pull-10{right:83.33333333%}.col-sm-pull-9{right:75%}.col-sm-pull-8{right:66.66666667%}.col-sm-pull-7{right:58.33333333%}.col-sm-pull-6{right:50%}.col-sm-pull-5{right:41.66666667%}.col-sm-pull-4{r
 ight:33.33333333%}.col-sm-pull-3{right:25%}.col-sm-pull-2{right:16.66666667%}.col-sm-pull-1{right:8.33333333%}.col-sm-pull-0{right:0}.col-sm-push-12{left:100%}.col-sm-push-11{left:91.66666667%}.col-sm-push-10{left:83.33333333%}.col-sm-push-9{left:75%}.col-sm-push-8{left:66.66666667%}.col-sm-push-7{left:58.33333333%}.col-sm-push-6{left:50%}.col-sm-push-5{left:41.66666667%}.col-sm-push-4{left:33.33333333%}.col-sm-push-3{left:25%}.col-sm-push-2{left:16.66666667%}.col-sm-push-1{left:8.33333333%}.col-sm-push-0{left:0}.col-sm-offset-12{margin-left:100%}.col-sm-offset-11{margin-left:91.66666667%}.col-sm-offset-10{margin-left:83.33333333%}.col-sm-offset-9{margin-left:75%}.col-sm-offset-8{margin-left:66.66666667%}.col-sm-offset-7{margin-left:58.33333333%}.col-sm-offset-6{margin-left:50%}.col-sm-offset-5{margin-left:41.66666667%}.col-sm-offset-4{margin-left:33.33333333%}.col-sm-offset-3{margin-left:25%}.col-sm-offset-2{margin-left:16.66666667%}.col-sm-offset-1{margin-left:8.33333333%}.col-sm-
 offset-0{margin-left:0}}@media (min-width:992px){.col-md-1,.col-md-2,.col-md-3,.col-md-4,.col-md-5,.col-md-6,.col-md-7,.col-md-8,.col-md-9,.col-md-10,.col-md-11,.col-md-12{float:left}.col-md-12{width:100%}.col-md-11{width:91.66666667%}.col-md-10{width:83.33333333%}.col-md-9{width:75%}.col-md-8{width:66.66666667%}.col-md-7{width:58.33333333%}.col-md-6{width:50%}.col-md-5{width:41.66666667%}.col-md-4{width:33.33333333%}.col-md-3{width:25%}.col-md-2{width:16.66666667%}.col-md-1{width:8.33333333%}.col-md-pull-12{right:100%}.col-md-pull-11{right:91.66666667%}.col-md-pull-10{right:83.33333333%}.col-md-pull-9{right:75%}.col-md-pull-8{right:66.66666667%}.col-md-pull-7{right:58.33333333%}.col-md-pull-6{right:50%}.col-md-pull-5{right:41.66666667%}.col-md-pull-4{right:33.33333333%}.col-md-pull-3{right:25%}.col-md-pull-2{right:16.66666667%}.col-md-pull-1{right:8.33333333%}.col-md-pull-0{right:0}.col-md-push-12{left:100%}.col-md-push-11{left:91.66666667%}.col-md-push-10{left:83.33333333%}.col-md
 -push-9{left:75%}.col-md-push-8{left:66.66666667%}.col-md-push-7{left:58.33333333%}.col-md-push-6{left:50%}.col-md-push-5{left:41.66666667%}.col-md-push-4{left:33.33333333%}.col-md-push-3{left:25%}.col-md-push-2{left:16.66666667%}.col-md-push-1{left:8.33333333%}.col-md-push-0{left:0}.col-md-offset-12{margin-left:100%}.col-md-offset-11{margin-left:91.66666667%}.col-md-offset-10{margin-left:83.33333333%}.col-md-offset-9{margin-left:75%}.col-md-offset-8{margin-left:66.66666667%}.col-md-offset-7{margin-left:58.33333333%}.col-md-offset-6{margin-left:50%}.col-md-offset-5{margin-left:41.66666667%}.col-md-offset-4{margin-left:33.33333333%}.col-md-offset-3{margin-left:25%}.col-md-offset-2{margin-left:16.66666667%}.col-md-offset-1{margin-left:8.33333333%}.col-md-offset-0{margin-left:0}}@media (min-width:1200px){.col-lg-1,.col-lg-2,.col-lg-3,.col-lg-4,.col-lg-5,.col-lg-6,.col-lg-7,.col-lg-8,.col-lg-9,.col-lg-10,.col-lg-11,.col-lg-12{float:left}.col-lg-12{width:100%}.col-lg-11{width:91.66666667
 %}.col-lg-10{width:83.33333333%}.col-lg-9{width:75%}.col-lg-8{width:66.66666667%}.col-lg-7{width:58.33333333%}.col-lg-6{width:50%}.col-lg-5{width:41.66666667%}.col-lg-4{width:33.33333333%}.col-lg-3{width:25%}.col-lg-2{width:16.66666667%}.col-lg-1{width:8.33333333%}.col-lg-pull-12{right:100%}.col-lg-pull-11{right:91.66666667%}.col-lg-pull-10{right:83.33333333%}.col-lg-pull-9{right:75%}.col-lg-pull-8{right:66.66666667%}.col-lg-pull-7{right:58.33333333%}.col-lg-pull-6{right:50%}.col-lg-pull-5{right:41.66666667%}.col-lg-pull-4{right:33.33333333%}.col-lg-pull-3{right:25%}.col-lg-pull-2{right:16.66666667%}.col-lg-pull-1{right:8.33333333%}.col-lg-pull-0{right:0}.col-lg-push-12{left:100%}.col-lg-push-11{left:91.66666667%}.col-lg-push-10{left:83.33333333%}.col-lg-push-9{left:75%}.col-lg-push-8{left:66.66666667%}.col-lg-push-7{left:58.33333333%}.col-lg-push-6{left:50%}.col-lg-push-5{left:41.66666667%}.col-lg-push-4{left:33.33333333%}.col-lg-push-3{left:25%}.col-lg-push-2{left:16.66666667%}.co
 l-lg-push-1{left:8.33333333%}.col-lg-push-0{left:0}.col-lg-offset-12{margin-left:100%}.col-lg-offset-11{margin-left:91.66666667%}.col-lg-offset-10{margin-left:83.33333333%}.col-lg-offset-9{margin-left:75%}.col-lg-offset-8{margin-left:66.66666667%}.col-lg-offset-7{margin-left:58.33333333%}.col-lg-offset-6{margin-left:50%}.col-lg-offset-5{margin-left:41.66666667%}.col-lg-offset-4{margin-left:33.33333333%}.col-lg-offset-3{margin-left:25%}.col-lg-offset-2{margin-left:16.66666667%}.col-lg-offset-1{margin-left:8.33333333%}.col-lg-offset-0{margin-left:0}}table{max-width:100%;background-color:transparent}th{text-align:left}.table{width:100%;margin-bottom:20px}.table>thead>tr>th,.table>tbody>tr>th,.table>tfoot>tr>th,.table>thead>tr>td,.table>tbody>tr>td,.table>tfoot>tr>td{padding:8px;line-height:1.42857143;vertical-align:top;border-top:1px solid #ddd}.table>thead>tr>th{vertical-align:bottom;border-bottom:2px solid #ddd}.table>caption+thead>tr:first-child>th,.table>colgroup+thead>tr:first-chi
 ld>th,.table>thead:first-child>tr:first-child>th,.table>caption+thead>tr:first-child>td,.table>colgroup+thead>tr:first-child>td,.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>thead>tr>th,.table-condensed>tbody>tr>th,.table-condensed>tfoot>tr>th,.table-condensed>thead>tr>td,.table-condensed>tbody>tr>td,.table-condensed>tfoot>tr>td{padding:5px}.table-bordered{border:1px solid #ddd}.table-bordered>thead>tr>th,.table-bordered>tbody>tr>th,.table-bordered>tfoot>tr>th,.table-bordered>thead>tr>td,.table-bordered>tbody>tr>td,.table-bordered>tfoot>tr>td{border:1px solid #ddd}.table-bordered>thead>tr>th,.table-bordered>thead>tr>td{border-bottom-width:2px}.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 col[class*=col-]{position:static;flo
 at:none;display:table-column}table td[class*=col-],table th[class*=col-]{position:static;float:none;display:table-cell}.table>thead>tr>td.active,.table>tbody>tr>td.active,.table>tfoot>tr>td.active,.table>thead>tr>th.active,.table>tbody>tr>th.active,.table>tfoot>tr>th.active,.table>thead>tr.active>td,.table>tbody>tr.active>td,.table>tfoot>tr.active>td,.table>thead>tr.active>th,.table>tbody>tr.active>th,.table>tfoot>tr.active>th{background-color:#f5f5f5}.table-hover>tbody>tr>td.active:hover,.table-hover>tbody>tr>th.active:hover,.table-hover>tbody>tr.active:hover>td,.table-hover>tbody>tr.active:hover>th{background-color:#e8e8e8}.table>thead>tr>td.success,.table>tbody>tr>td.success,.table>tfoot>tr>td.success,.table>thead>tr>th.success,.table>tbody>tr>th.success,.table>tfoot>tr>th.success,.table>thead>tr.success>td,.table>tbody>tr.success>td,.table>tfoot>tr.success>td,.table>thead>tr.success>th,.table>tbody>tr.success>th,.table>tfoot>tr.success>th{background-color:#dff0d8}.table-hover>tb
 ody>tr>td.success:hover,.table-hover>tbody>tr>th.success:hover,.table-hover>tbody>tr.success:hover>td,.table-hover>tbody>tr.success:hover>th{background-color:#d0e9c6}.table>thead>tr>td.info,.table>tbody>tr>td.info,.table>tfoot>tr>td.info,.table>thead>tr>th.info,.table>tbody>tr>th.info,.table>tfoot>tr>th.info,.table>thead>tr.info>td,.table>tbody>tr.info>td,.table>tfoot>tr.info>td,.table>thead>tr.info>th,.table>tbody>tr.info>th,.table>tfoot>tr.info>th{background-color:#d9edf7}.table-hover>tbody>tr>td.info:hover,.table-hover>tbody>tr>th.info:hover,.table-hover>tbody>tr.info:hover>td,.table-hover>tbody>tr.info:hover>th{background-color:#c4e3f3}.table>thead>tr>td.warning,.table>tbody>tr>td.warning,.table>tfoot>tr>td.warning,.table>thead>tr>th.warning,.table>tbody>tr>th.warning,.table>tfoot>tr>th.warning,.table>thead>tr.warning>td,.table>tbody>tr.warning>td,.table>tfoot>tr.warning>td,.table>thead>tr.warning>th,.table>tbody>tr.warning>th,.table>tfoot>tr.warning>th{background-color:#fcf8e3}
 .table-hover>tbody>tr>td.warning:hover,.table-hover>tbody>tr>th.warning:hover,.table-hover>tbody>tr.warning:hover>td,.table-hover>tbody>tr.warning:hover>th{background-color:#faf2cc}.table>thead>tr>td.danger,.table>tbody>tr>td.danger,.table>tfoot>tr>td.danger,.table>thead>tr>th.danger,.table>tbody>tr>th.danger,.table>tfoot>tr>th.danger,.table>thead>tr.danger>td,.table>tbody>tr.danger>td,.table>tfoot>tr.danger>td,.table>thead>tr.danger>th,.table>tbody>tr.danger>th,.table>tfoot>tr.danger>th{background-color:#f2dede}.table-hover>tbody>tr>td.danger:hover,.table-hover>tbody>tr>th.danger:hover,.table-hover>tbody>tr.danger:hover>td,.table-hover>tbody>tr.danger:hover>th{background-color:#ebcccc}@media (max-width:767px){.table-responsive{width:100%;margin-bottom:15px;overflow-y:hidden;overflow-x:scroll;-ms-overflow-style:-ms-autohiding-scrollbar;border:1px solid #ddd;-webkit-overflow-scrolling:touch}.table-responsive>.table{margin-bottom:0}.table-responsive>.table>thead>tr>th,.table-responsiv
 e>.table>tbody>tr>th,.table-responsive>.table>tfoot>tr>th,.table-responsive>.table>thead>tr>td,.table-responsive>.table>tbody>tr>td,.table-responsive>.table>tfoot>tr>td{white-space:nowrap}.table-responsive>.table-bordered{border:0}.table-responsive>.table-bordered>thead>tr>th:first-child,.table-responsive>.table-bordered>tbody>tr>th:first-child,.table-responsive>.table-bordered>tfoot>tr>th:first-child,.table-responsive>.table-bordered>thead>tr>td:first-child,.table-responsive>.table-bordered>tbody>tr>td:first-child,.table-responsive>.table-bordered>tfoot>tr>td:first-child{border-left:0}.table-responsive>.table-bordered>thead>tr>th:last-child,.table-responsive>.table-bordered>tbody>tr>th:last-child,.table-responsive>.table-bordered>tfoot>tr>th:last-child,.table-responsive>.table-bordered>thead>tr>td:last-child,.table-responsive>.table-bordered>tbody>tr>td:last-child,.table-responsive>.table-bordered>tfoot>tr>td:last-child{border-right:0}.table-responsive>.table-bordered>tbody>tr:last
 -child>th,.table-responsive>.table-bordered>tfoot>tr:last-child>th,.table-responsive>.table-bordered>tbody>tr:last-child>td,.table-responsive>.table-bordered>tfoot>tr:last-child>td{border-bottom:0}}fieldset{padding:0;margin:0;border:0;min-width:0}legend{display:block;width:100%;padding:0;margin-bottom:20px;font-size:21px;line-height:inherit;color:#333;border:0;border-bottom:1px solid #e5e5e5}label{display:inline-block;margin-bottom:5px;font-weight:700}input[type=search]{-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box}input[type=radio],input[type=checkbox]{margin:4px 0 0;margin-top:1px \9;line-height:normal}input[type=file]{display:block}input[type=range]{display:block;width:100%}select[multiple],select[size]{height:auto}input[type=file]:focus,input[type=radio]:focus,input[type=checkbox]:focus{outline:thin dotted;outline:5px auto -webkit-focus-ring-color;outline-offset:-2px}output{display:block;padding-top:7px;font-size:14px;line-height:1.42857143;color
 :#555}.form-control{display:block;width:100%;height:34px;padding:6px 12px;font-size:14px;line-height:1.42857143;color:#555;background-color:#fff;background-image:none;border:1px solid #ccc;border-radius:4px;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,.075);box-shadow:inset 0 1px 1px rgba(0,0,0,.075);-webkit-transition:border-color ease-in-out .15s,box-shadow ease-in-out .15s;transition:border-color ease-in-out .15s,box-shadow ease-in-out .15s}.form-control:focus{border-color:#66afe9;outline:0;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,.075),0 0 8px rgba(102,175,233,.6);box-shadow:inset 0 1px 1px rgba(0,0,0,.075),0 0 8px rgba(102,175,233,.6)}.form-control::-moz-placeholder{color:#999;opacity:1}.form-control:-ms-input-placeholder{color:#999}.form-control::-webkit-input-placeholder{color:#999}.form-control[disabled],.form-control[readonly],fieldset[disabled] .form-control{cursor:not-allowed;background-color:#eee;opacity:1}textarea.form-control{height:auto}input[type=search]{-webki
 t-appearance:none}input[type=date]{line-height:34px}.form-group{margin-bottom:15px}.radio,.checkbox{display:block;min-height:20px;margin-top:10px;margin-bottom:10px;padding-left:20px}.radio label,.checkbox label{display:inline;font-weight:400;cursor:pointer}.radio input[type=radio],.radio-inline input[type=radio],.checkbox input[type=checkbox],.checkbox-inline input[type=checkbox]{float:left;margin-left:-20px}.radio+.radio,.checkbox+.checkbox{margin-top:-5px}.radio-inline,.checkbox-inline{display:inline-block;padding-left:20px;margin-bottom:0;vertical-align:middle;font-weight:400;cursor:pointer}.radio-inline+.radio-inline,.checkbox-inline+.checkbox-inline{margin-top:0;margin-left:10px}input[type=radio][disabled],input[type=checkbox][disabled],.radio[disabled],.radio-inline[disabled],.checkbox[disabled],.checkbox-inline[disabled],fieldset[disabled] input[type=radio],fieldset[disabled] input[type=checkbox],fieldset[disabled] .radio,fieldset[disabled] .radio-inline,fieldset[disabled] .
 checkbox,fieldset[disabled] .checkbox-inline{cursor:not-allowed}.input-sm{height:30px;padding:5px 10px;font-size:12px;line-height:1.5;border-radius:3px}select.input-sm{height:30px;line-height:30px}textarea.input-sm,select[multiple].input-sm{height:auto}.input-lg{height:46px;padding:10px 16px;font-size:18px;line-height:1.33;border-radius:6px}select.input-lg{height:46px;line-height:46px}textarea.input-lg,select[multiple].input-lg{height:auto}.has-feedback{position:relative}.has-feedback .form-control{padding-right:42.5px}.has-feedback .form-control-feedback{position:absolute;top:25px;right:0;display:block;width:34px;height:34px;line-height:34px;text-align:center}.has-success .help-block,.has-success .control-label,.has-success .radio,.has-success .checkbox,.has-success .radio-inline,.has-success .checkbox-inline{color:#3c763d}.has-success .form-control{border-color:#3c763d;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,.075);box-shadow:inset 0 1px 1px rgba(0,0,0,.075)}.has-success .for
 m-control:focus{border-color:#2b542c;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,.075),0 0 6px #67b168;box-shadow:inset 0 1px 1px rgba(0,0,0,.075),0 0 6px #67b168}.has-success .input-group-addon{color:#3c763d;border-color:#3c763d;background-color:#dff0d8}.has-success .form-control-feedback{color:#3c763d}.has-warning .help-block,.has-warning .control-label,.has-warning .radio,.has-warning .checkbox,.has-warning .radio-inline,.has-warning .checkbox-inline{color:#8a6d3b}.has-warning .form-control{border-color:#8a6d3b;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,.075);box-shadow:inset 0 1px 1px rgba(0,0,0,.075)}.has-warning .form-control:focus{border-color:#66512c;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,.075),0 0 6px #c0a16b;box-shadow:inset 0 1px 1px rgba(0,0,0,.075),0 0 6px #c0a16b}.has-warning .input-group-addon{color:#8a6d3b;border-color:#8a6d3b;background-color:#fcf8e3}.has-warning .form-control-feedback{color:#8a6d3b}.has-error .help-block,.has-error .control-label,.has-e
 rror .radio,.has-error .checkbox,.has-error .radio-inline,.has-error .checkbox-inline{color:#a94442}.has-error .form-control{border-color:#a94442;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,.075);box-shadow:inset 0 1px 1px rgba(0,0,0,.075)}.has-error .form-control:focus{border-color:#843534;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,.075),0 0 6px #ce8483;box-shadow:inset 0 1px 1px rgba(0,0,0,.075),0 0 6px #ce8483}.has-error .input-group-addon{color:#a94442;border-color:#a94442;background-color:#f2dede}.has-error .form-control-feedback{color:#a94442}.form-control-static{margin-bottom:0}.help-block{display:block;margin-top:5px;margin-bottom:10px;color:#737373}@media (min-width:768px){.form-inline .form-group{display:inline-block;margin-bottom:0;vertical-align:middle}.form-inline .form-control{display:inline-block;width:auto;vertical-align:middle}.form-inline .input-group>.form-control{width:100%}.form-inline .control-label{margin-bottom:0;vertical-align:middle}.form-inline .radio
 ,.form-inline .checkbox{display:inline-block;margin-top:0;margin-bottom:0;padding-left:0;vertical-align:middle}.form-inline .radio input[type=radio],.form-inline .checkbox input[type=checkbox]{float:none;margin-left:0}.form-inline .has-feedback .form-control-feedback{top:0}}.form-horizontal .control-label,.form-horizontal .radio,.form-horizontal .checkbox,.form-horizontal .radio-inline,.form-horizontal .checkbox-inline{margin-top:0;margin-bottom:0;padding-top:7px}.form-horizontal .radio,.form-horizontal .checkbox{min-height:27px}.form-horizontal .form-group{margin-left:-15px;margin-right:-15px}.form-horizontal .form-control-static{padding-top:7px}@media (min-width:768px){.form-horizontal .control-label{text-align:right}}.form-horizontal .has-feedback .form-control-feedback{top:0;right:15px}.btn{display:inline-block;margin-bottom:0;font-weight:400;text-align:center;vertical-align:middle;cursor:pointer;background-image:none;border:1px solid transparent;white-space:nowrap;padding:6px 1
 2px;font-size:14px;line-height:1.42857143;border-radius:4px;-webkit-user-select:none;-moz-user-select:none;-ms-user-select:none;user-select:none}.btn:focus,.btn:active:focus,.btn.active:focus{outline:thin dotted;outline:5px auto -webkit-focus-ring-color;outline-offset:-2px}.btn:hover,.btn:focus{color:#333;text-decoration:none}.btn:active,.btn.active{outline:0;background-image:none;-webkit-box-shadow:inset 0 3px 5px rgba(0,0,0,.125);box-shadow:inset 0 3px 5px rgba(0,0,0,.125)}.btn.disabled,.btn[disabled],fieldset[disabled] .btn{cursor:not-allowed;pointer-events:none;opacity:.65;filter:alpha(opacity=65);-webkit-box-shadow:none;box-shadow:none}.btn-default{color:#333;background-color:#fff;border-color:#ccc}.btn-default:hover,.btn-default:focus,.btn-default:active,.btn-default.active,.open .dropdown-toggle.btn-default{color:#333;background-color:#ebebeb;border-color:#adadad}.btn-default:active,.btn-default.active,.open .dropdown-toggle.btn-default{background-image:none}.btn-default.disa
 bled,.btn-default[disabled],fieldset[disabled] .btn-default,.btn-default.disabled:hover,.btn-default[disabled]:hover,fieldset[disabled] .btn-default:hover,.btn-default.disabled:focus,.btn-default[disabled]:focus,fieldset[disabled] .btn-default:focus,.btn-default.disabled:active,.btn-default[disabled]:active,fieldset[disabled] .btn-default:active,.btn-default.disabled.active,.btn-default[disabled].active,fieldset[disabled] .btn-default.active{background-color:#fff;border-color:#ccc}.btn-default .badge{color:#fff;background-color:#333}.btn-primary{color:#fff;background-color:#428bca;border-color:#357ebd}.btn-primary:hover,.btn-primary:focus,.btn-primary:active,.btn-primary.active,.open .dropdown-toggle.btn-primary{color:#fff;background-color:#3276b1;border-color:#285e8e}.btn-primary:active,.btn-primary.active,.open .dropdown-toggle.btn-primary{background-image:none}.btn-primary.disabled,.btn-primary[disabled],fieldset[disabled] .btn-primary,.btn-primary.disabled:hover,.btn-primary[dis
 abled]:hover,fieldset[disabled] .btn-primary:hover,.btn-primary.disabled:focus,.btn-primary[disabled]:focus,fieldset[disabled] .btn-primary:focus,.btn-primary.disabled:active,.btn-primary[disabled]:active,fieldset[disabled] .btn-primary:active,.btn-primary.disabled.active,.btn-primary[disabled].active,fieldset[disabled] .btn-primary.active{background-color:#428bca;border-color:#357ebd}.btn-primary .badge{color:#428bca;background-color:#fff}.btn-success{color:#fff;background-color:#5cb85c;border-color:#4cae4c}.btn-success:hover,.btn-success:focus,.btn-success:active,.btn-success.active,.open .dropdown-toggle.btn-success{color:#fff;background-color:#47a447;border-color:#398439}.btn-success:active,.btn-success.active,.open .dropdown-toggle.btn-success{background-image:none}.btn-success.disabled,.btn-success[disabled],fieldset[disabled] .btn-success,.btn-success.disabled:hover,.btn-success[disabled]:hover,fieldset[disabled] .btn-success:hover,.btn-success.disabled:focus,.btn-success[dis
 abled]:focus,fieldset[disabled] .btn-success:focus,.btn-success.disabled:active,.btn-success[disabled]:active,fieldset[disabled] .btn-success:active,.btn-success.disabled.active,.btn-success[disabled].active,fieldset[disabled] .btn-success.active{background-color:#5cb85c;border-color:#4cae4c}.btn-success .badge{color:#5cb85c;background-color:#fff}.btn-info{color:#fff;background-color:#5bc0de;border-color:#46b8da}.btn-info:hover,.btn-info:focus,.btn-info:active,.btn-info.active,.open .dropdown-toggle.btn-info{color:#fff;background-color:#39b3d7;border-color:#269abc}.btn-info:active,.btn-info.active,.open .dropdown-toggle.btn-info{background-image:none}.btn-info.disabled,.btn-info[disabled],fieldset[disabled] .btn-info,.btn-info.disabled:hover,.btn-info[disabled]:hover,fieldset[disabled] .btn-info:hover,.btn-info.disabled:focus,.btn-info[disabled]:focus,fieldset[disabled] .btn-info:focus,.btn-info.disabled:active,.btn-info[disabled]:active,fieldset[disabled] .btn-info:active,.btn-info
 .disabled.active,.btn-info[disabled].active,fieldset[disabled] .btn-info.active{background-color:#5bc0de;border-color:#46b8da}.btn-info .badge{color:#5bc0de;background-color:#fff}.btn-warning{color:#fff;background-color:#f0ad4e;border-color:#eea236}.btn-warning:hover,.btn-warning:focus,.btn-warning:active,.btn-warning.active,.open .dropdown-toggle.btn-warning{color:#fff;background-color:#ed9c28;border-color:#d58512}.btn-warning:active,.btn-warning.active,.open .dropdown-toggle.btn-warning{background-image:none}.btn-warning.disabled,.btn-warning[disabled],fieldset[disabled] .btn-warning,.btn-warning.disabled:hover,.btn-warning[disabled]:hover,fieldset[disabled] .btn-warning:hover,.btn-warning.disabled:focus,.btn-warning[disabled]:focus,fieldset[disabled] .btn-warning:focus,.btn-warning.disabled:active,.btn-warning[disabled]:active,fieldset[disabled] .btn-warning:active,.btn-warning.disabled.active,.btn-warning[disabled].active,fieldset[disabled] .btn-warning.active{background-color:#
 f0ad4e;border-color:#eea236}.btn-warning .badge{color:#f0ad4e;background-color:#fff}.btn-danger{color:#fff;background-color:#d9534f;border-color:#d43f3a}.btn-danger:hover,.btn-danger:focus,.btn-danger:active,.btn-danger.active,.open .dropdown-toggle.btn-danger{color:#fff;background-color:#d2322d;border-color:#ac2925}.btn-danger:active,.btn-danger.active,.open .dropdown-toggle.btn-danger{background-image:none}.btn-danger.disabled,.btn-danger[disabled],fieldset[disabled] .btn-danger,.btn-danger.disabled:hover,.btn-danger[disabled]:hover,fieldset[disabled] .btn-danger:hover,.btn-danger.disabled:focus,.btn-danger[disabled]:focus,fieldset[disabled] .btn-danger:focus,.btn-danger.disabled:active,.btn-danger[disabled]:active,fieldset[disabled] .btn-danger:active,.btn-danger.disabled.active,.btn-danger[disabled].active,fieldset[disabled] .btn-danger.active{background-color:#d9534f;border-color:#d43f3a}.btn-danger .badge{color:#d9534f;background-color:#fff}.btn-link{color:#428bca;font-weight:
 400;cursor:pointer;border-radius:0}.btn-link,.btn-link:active,.btn-link[disabled],fieldset[disabled] .btn-link{background-color:transparent;-webkit-box-shadow:none;box-shadow:none}.btn-link,.btn-link:hover,.btn-link:focus,.btn-link:active{border-color:transparent}.btn-link:hover,.btn-link:focus{color:#2a6496;text-decoration:underline;background-color:transparent}.btn-link[disabled]:hover,fieldset[disabled] .btn-link:hover,.btn-link[disabled]:focus,fieldset[disabled] .btn-link:focus{color:#999;text-decoration:none}.btn-lg,.btn-group-lg>.btn{padding:10px 16px;font-size:18px;line-height:1.33;border-radius:6px}.btn-sm,.btn-group-sm>.btn{padding:5px 10px;font-size:12px;line-height:1.5;border-radius:3px}.btn-xs,.btn-group-xs>.btn{padding:1px 5px;font-size:12px;line-height:1.5;border-radius:3px}.btn-block{display:block;width:100%;padding-left:0;padding-right:0}.btn-block+.btn-block{margin-top:5px}input[type=submit].btn-block,input[type=reset].btn-block,input[type=button].btn-block{width:10
 0%}.fade{opacity:0;-webkit-transition:opacity .15s linear;transition:opacity .15s linear}.fade.in{opacity:1}.collapse{display:none}.collapse.in{display:block}.collapsing{position:relative;height:0;overflow:hidden;-webkit-transition:height .35s ease;transition:height .35s ease}@font-face{font-family:'Glyphicons Halflings';src:url(../fonts/glyphicons-halflings-regular.eot);src:url(../fonts/glyphicons-halflings-regular.eot?#iefix) format('embedded-opentype'),url(../fonts/glyphicons-halflings-regular.woff) format('woff'),url(../fonts/glyphicons-halflings-regular.ttf) format('truetype'),url(../fonts/glyphicons-halflings-regular.svg#glyphicons_halflingsregular) format('svg')}.glyphicon{position:relative;top:1px;display:inline-block;font-family:'Glyphicons Halflings';font-style:normal;font-weight:400;line-height:1;-webkit-font-smoothing:antialiased;-moz-osx-font-smoothing:grayscale}.glyphicon-asterisk:before{content:"\2a"}.glyphicon-plus:before{content:"\2b"}.glyphicon-euro:before{content:
 "\20ac"}.glyphicon-minus:before{content:"\2212"}.glyphicon-cloud:before{content:"\2601"}.glyphicon-envelope:before{content:"\2709"}.glyphicon-pencil:before{content:"\270f"}.glyphicon-glass:before{content:"\e001"}.glyphicon-music:before{content:"\e002"}.glyphicon-search:before{content:"\e003"}.glyphicon-heart:before{content:"\e005"}.glyphicon-star:before{content:"\e006"}.glyphicon-star-empty:before{content:"\e007"}.glyphicon-user:before{content:"\e008"}.glyphicon-film:before{content:"\e009"}.glyphicon-th-large:before{content:"\e010"}.glyphicon-th:before{content:"\e011"}.glyphicon-th-list:before{content:"\e012"}.glyphicon-ok:before{content:"\e013"}.glyphicon-remove:before{content:"\e014"}.glyphicon-zoom-in:before{content:"\e015"}.glyphicon-zoom-out:before{content:"\e016"}.glyphicon-off:before{content:"\e017"}.glyphicon-signal:before{content:"\e018"}.glyphicon-cog:before{content:"\e019"}.glyphicon-trash:before{content:"\e020"}.glyphicon-home:before{content:"\e021"}.glyphicon-file:befor
 e{content:"\e022"}.glyphicon-time:before{content:"\e023"}.glyphicon-road:before{content:"\e024"}.glyphicon-download-alt:before{content:"\e025"}.glyphicon-download:before{content:"\e026"}.glyphicon-upload:before{content:"\e027"}.glyphicon-inbox:before{content:"\e028"}.glyphicon-play-circle:before{content:"\e029"}.glyphicon-repeat:before{content:"\e030"}.glyphicon-refresh:before{content:"\e031"}.glyphicon-list-alt:before{content:"\e032"}.glyphicon-lock:before{content:"\e033"}.glyphicon-flag:before{content:"\e034"}.glyphicon-headphones:before{content:"\e035"}.glyphicon-volume-off:before{content:"\e036"}.glyphicon-volume-down:before{content:"\e037"}.glyphicon-volume-up:before{content:"\e038"}.glyphicon-qrcode:before{content:"\e039"}.glyphicon-barcode:before{content:"\e040"}.glyphicon-tag:before{content:"\e041"}.glyphicon-tags:before{content:"\e042"}.glyphicon-book:before{content:"\e043"}.glyphicon-bookmark:before{content:"\e044"}.glyphicon-print:before{content:"\e045"}.glyphicon-camera:
 before{content:"\e046"}.glyphicon-font:before{content:"\e047"}.glyphicon-bold:before{content:"\e048"}.glyphicon-italic:before{content:"\e049"}.glyphicon-text-height:before{content:"\e050"}.glyphicon-text-width:before{content:"\e051"}.glyphicon-align-left:before{content:"\e052"}.glyphicon-align-center:before{content:"\e053"}.glyphicon-align-right:before{content:"\e054"}.glyphicon-align-justify:before{content:"\e055"}.glyphicon-list:before{content:"\e056"}.glyphicon-indent-left:before{content:"\e057"}.glyphicon-indent-right:before{content:"\e058"}.glyphicon-facetime-video:before{content:"\e059"}.glyphicon-picture:before{content:"\e060"}.glyphicon-map-marker:before{content:"\e062"}.glyphicon-adjust:before{content:"\e063"}.glyphicon-tint:before{content:"\e064"}.glyphicon-edit:before{content:"\e065"}.glyphicon-share:before{content:"\e066"}.glyphicon-check:before{content:"\e067"}.glyphicon-move:before{content:"\e068"}.glyphicon-step-backward:before{content:"\e069"}.glyphicon-fast-backward
 :before{content:"\e070"}.glyphicon-backward:before{content:"\e071"}.glyphicon-play:before{content:"\e072"}.glyphicon-pause:before{content:"\e073"}.glyphicon-stop:before{content:"\e074"}.glyphicon-forward:before{content:"\e075"}.glyphicon-fast-forward:before{content:"\e076"}.glyphicon-step-forward:before{content:"\e077"}.glyphicon-eject:before{content:"\e078"}.glyphicon-chevron-left:before{content:"\e079"}.glyphicon-chevron-right:before{content:"\e080"}.glyphicon-plus-sign:before{content:"\e081"}.glyphicon-minus-sign:before{content:"\e082"}.glyphicon-remove-sign:before{content:"\e083"}.glyphicon-ok-sign:before{content:"\e084"}.glyphicon-question-sign:before{content:"\e085"}.glyphicon-info-sign:before{content:"\e086"}.glyphicon-screenshot:before{content:"\e087"}.glyphicon-remove-circle:before{content:"\e088"}.glyphicon-ok-circle:before{content:"\e089"}.glyphicon-ban-circle:before{content:"\e090"}.glyphicon-arrow-left:before{content:"\e091"}.glyphicon-arrow-right:before{content:"\e092"
 }.glyphicon-arrow-up:before{content:"\e093"}.glyphicon-arrow-down:before{content:"\e094"}.glyphicon-share-alt:before{content:"\e095"}.glyphicon-resize-full:before{content:"\e096"}.glyphicon-resize-small:before{content:"\e097"}.glyphicon-exclamation-sign:before{content:"\e101"}.glyphicon-gift:before{content:"\e102"}.glyphicon-leaf:before{content:"\e103"}.glyphicon-fire:before{content:"\e104"}.glyphicon-eye-open:before{content:"\e105"}.glyphicon-eye-close:before{content:"\e106"}.glyphicon-warning-sign:before{content:"\e107"}.glyphicon-plane:before{content:"\e108"}.glyphicon-calendar:before{content:"\e109"}.glyphicon-random:before{content:"\e110"}.glyphicon-comment:before{content:"\e111"}.glyphicon-magnet:before{content:"\e112"}.glyphicon-chevron-up:before{content:"\e113"}.glyphicon-chevron-down:before{content:"\e114"}.glyphicon-retweet:before{content:"\e115"}.glyphicon-shopping-cart:before{content:"\e116"}.glyphicon-folder-close:before{content:"\e117"}.glyphicon-folder-open:before{con
 tent:"\e118"}.glyphicon-resize-vertical:before{content:"\e119"}.glyphicon-resize-horizontal:before{content:"\e120"}.glyphicon-hdd:before{content:"\e121"}.glyphicon-bullhorn:before{content:"\e122"}.glyphicon-bell:before{content:"\e123"}.glyphicon-certificate:before{content:"\e124"}.glyphicon-thumbs-up:before{content:"\e125"}.glyphicon-thumbs-down:before{content:"\e126"}.glyphicon-hand-right:before{content:"\e127"}.glyphicon-hand-left:before{content:"\e128"}.glyphicon-hand-up:before{content:"\e129"}.glyphicon-hand-down:before{content:"\e130"}.glyphicon-circle-arrow-right:before{content:"\e131"}.glyphicon-circle-arrow-left:before{content:"\e132"}.glyphicon-circle-arrow-up:before{content:"\e133"}.glyphicon-circle-arrow-down:before{content:"\e134"}.glyphicon-globe:before{content:"\e135"}.glyphicon-wrench:before{content:"\e136"}.glyphicon-tasks:before{content:"\e137"}.glyphicon-filter:before{content:"\e138"}.glyphicon-briefcase:before{content:"\e139"}.glyphicon-fullscreen:before{content:"
 \e140"}.glyphicon-dashboard:before{content:"\e141"}.glyphicon-paperclip:before{content:"\e142"}.glyphicon-heart-empty:before{content:"\e143"}.glyphicon-link:before{content:"\e144"}.glyphicon-phone:before{content:"\e145"}.glyphicon-pushpin:before{content:"\e146"}.glyphicon-usd:before{content:"\e148"}.glyphicon-gbp:before{content:"\e149"}.glyphicon-sort:before{content:"\e150"}.glyphicon-sort-by-alphabet:before{content:"\e151"}.glyphicon-sort-by-alphabet-alt:before{content:"\e152"}.glyphicon-sort-by-order:before{content:"\e153"}.glyphicon-sort-by-order-alt:before{content:"\e154"}.glyphicon-sort-by-attributes:before{content:"\e155"}.glyphicon-sort-by-attributes-alt:before{content:"\e156"}.glyphicon-unchecked:before{content:"\e157"}.glyphicon-expand:before{content:"\e158"}.glyphicon-collapse-down:before{content:"\e159"}.glyphicon-collapse-up:before{content:"\e160"}.glyphicon-log-in:before{content:"\e161"}.glyphicon-flash:before{content:"\e162"}.glyphicon-log-out:before{content:"\e163"}.g
 lyphicon-new-window:before{content:"\e164"}.glyphicon-record:before{content:"\e165"}.glyphicon-save:before{content:"\e166"}.glyphicon-open:before{content:"\e167"}.glyphicon-saved:before{content:"\e168"}.glyphicon-import:before{content:"\e169"}.glyphicon-export:before{content:"\e170"}.glyphicon-send:before{content:"\e171"}.glyphicon-floppy-disk:before{content:"\e172"}.glyphicon-floppy-saved:before{content:"\e173"}.glyphicon-floppy-remove:before{content:"\e174"}.glyphicon-floppy-save:before{content:"\e175"}.glyphicon-floppy-open:before{content:"\e176"}.glyphicon-credit-card:before{content:"\e177"}.glyphicon-transfer:before{content:"\e178"}.glyphicon-cutlery:before{content:"\e179"}.glyphicon-header:before{content:"\e180"}.glyphicon-compressed:before{content:"\e181"}.glyphicon-earphone:before{content:"\e182"}.glyphicon-phone-alt:before{content:"\e183"}.glyphicon-tower:before{content:"\e184"}.glyphicon-stats:before{content:"\e185"}.glyphicon-sd-video:before{content:"\e186"}.glyphicon-hd-
 video:before{content:"\e187"}.glyphicon-subtitles:before{content:"\e188"}.glyphicon-sound-stereo:before{content:"\e189"}.glyphicon-sound-dolby:before{content:"\e190"}.glyphicon-sound-5-1:before{content:"\e191"}.glyphicon-sound-6-1:before{content:"\e192"}.glyphicon-sound-7-1:before{content:"\e193"}.glyphicon-copyright-mark:before{content:"\e194"}.glyphicon-registration-mark:before{content:"\e195"}.glyphicon-cloud-download:before{content:"\e197"}.glyphicon-cloud-upload:before{content:"\e198"}.glyphicon-tree-conifer:before{content:"\e199"}.glyphicon-tree-deciduous:before{content:"\e200"}.caret{display:inline-block;width:0;height:0;margin-left:2px;vertical-align:middle;border-top:4px solid;border-right:4px solid transparent;border-left:4px solid transparent}.dropdown{position:relative}.dropdown-toggle:focus{outline:0}.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;font-size:14px;background
 -color:#fff;border:1px solid #ccc;border:1px solid rgba(0,0,0,.15);border-radius:4px;-webkit-box-shadow:0 6px 12px rgba(0,0,0,.175);box-shadow:0 6px 12px rgba(0,0,0,.175);background-clip:padding-box}.dropdown-menu.pull-right{right:0;left:auto}.dropdown-menu .divider{height:1px;margin:9px 0;overflow:hidden;background-color:#e5e5e5}.dropdown-menu>li>a{display:block;padding:3px 20px;clear:both;font-weight:400;line-height:1.42857143;color:#333;white-space:nowrap}.dropdown-menu>li>a:hover,.dropdown-menu>li>a:focus{text-decoration:none;color:#262626;background-color:#f5f5f5}.dropdown-menu>.active>a,.dropdown-menu>.active>a:hover,.dropdown-menu>.active>a:focus{color:#fff;text-decoration:none;outline:0;background-color:#428bca}.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;background-color:transparent;background-image:none;filter:progid:DXImageTrans
 form.Microsoft.gradient(enabled=false);cursor:not-allowed}.open>.dropdown-menu{display:block}.open>a{outline:0}.dropdown-menu-right{left:auto;right:0}.dropdown-menu-left{left:0;right:auto}.dropdown-header{display:block;padding:3px 20px;font-size:12px;line-height:1.42857143;color:#999}.dropdown-backdrop{position:fixed;left:0;right:0;bottom:0;top:0;z-index:990}.pull-right>.dropdown-menu{right:0;left:auto}.dropup .caret,.navbar-fixed-bottom .dropdown .caret{border-top:0;border-bottom:4px solid;content:""}.dropup .dropdown-menu,.navbar-fixed-bottom .dropdown .dropdown-menu{top:auto;bottom:100%;margin-bottom:1px}@media (min-width:768px){.navbar-right .dropdown-menu{left:auto;right:0}.navbar-right .dropdown-menu-left{left:0;right:auto}}.btn-group,.btn-group-vertical{position:relative;display:inline-block;vertical-align:middle}.btn-group>.btn,.btn-group-vertical>.btn{position:relative;float:left}.btn-group>.btn:hover,.btn-group-vertical>.btn:hover,.btn-group>.btn:focus,.btn-group-vertical>
 .btn:focus,.btn-group>.btn:active,.btn-group-vertical>.btn:active,.btn-group>.btn.active,.btn-group-vertical>.btn.active{z-index:2}.btn-group>.btn:focus,.btn-group-vertical>.btn:focus{outline:0}.btn-group .btn+.btn,.btn-group .btn+.btn-group,.btn-group .btn-group+.btn,.btn-group .btn-group+.btn-group{margin-left:-1px}.btn-toolbar{margin-left:-5px}.btn-toolbar .btn-group,.btn-toolbar .input-group{float:left}.btn-toolbar>.btn,.btn-toolbar>.btn-group,.btn-toolbar>.input-group{margin-left:5px}.btn-group>.btn:not(:first-child):not(:last-child):not(.dropdown-toggle){border-radius:0}.btn-group>.btn:first-child{margin-left:0}.btn-group>.btn:first-child:not(:last-child):not(.dropdown-toggle){border-bottom-right-radius:0;border-top-right-radius:0}.btn-group>.btn:last-child:not(:first-child),.btn-group>.dropdown-toggle:not(:first-child){border-bottom-left-radius:0;border-top-left-radius:0}.btn-group>.btn-group{float:left}.btn-group>.btn-group:not(:first-child):not(:last-child)>.btn{border-radi
 us:0}.btn-group>.btn-group:first-child>.btn:last-child,.btn-group>.btn-group:first-child>.dropdown-toggle{border-bottom-right-radius:0;border-top-right-radius:0}.btn-group>.btn-group:last-child>.btn:first-child{border-bottom-left-radius:0;border-top-left-radius:0}.btn-group .dropdown-toggle:active,.btn-group.open .dropdown-toggle{outline:0}.btn-group>.btn+.dropdown-toggle{padding-left:8px;padding-right:8px}.btn-group>.btn-lg+.dropdown-toggle{padding-left:12px;padding-right:12px}.btn-group.open .dropdown-toggle{-webkit-box-shadow:inset 0 3px 5px rgba(0,0,0,.125);box-shadow:inset 0 3px 5px rgba(0,0,0,.125)}.btn-group.open .dropdown-toggle.btn-link{-webkit-box-shadow:none;box-shadow:none}.btn .caret{margin-left:0}.btn-lg .caret{border-width:5px 5px 0;border-bottom-width:0}.dropup .btn-lg .caret{border-width:0 5px 5px}.btn-group-vertical>.btn,.btn-group-vertical>.btn-group,.btn-group-vertical>.btn-group>.btn{display:block;float:none;width:100%;max-width:100%}.btn-group-vertical>.btn-gro
 up>.btn{float:none}.btn-group-vertical>.btn+.btn,.btn-group-vertical>.btn+.btn-group,.btn-group-vertical>.btn-group+.btn,.btn-group-vertical>.btn-group+.btn-group{margin-top:-1px;margin-left:0}.btn-group-vertical>.btn:not(:first-child):not(:last-child){border-radius:0}.btn-group-vertical>.btn:first-child:not(:last-child){border-top-right-radius:4px;border-bottom-right-radius:0;border-bottom-left-radius:0}.btn-group-vertical>.btn:last-child:not(:first-child){border-bottom-left-radius:4px;border-top-right-radius:0;border-top-left-radius:0}.btn-group-vertical>.btn-group:not(:first-child):not(:last-child)>.btn{border-radius:0}.btn-group-vertical>.btn-group:first-child:not(:last-child)>.btn:last-child,.btn-group-vertical>.btn-group:first-child:not(:last-child)>.dropdown-toggle{border-bottom-right-radius:0;border-bottom-left-radius:0}.btn-group-vertical>.btn-group:last-child:not(:first-child)>.btn:first-child{border-top-right-radius:0;border-top-left-radius:0}.btn-group-justified{display:
 table;width:100%;table-layout:fixed;border-collapse:separate}.btn-group-justified>.btn,.btn-group-justified>.btn-group{float:none;display:table-cell;width:1%}.btn-group-justified>.btn-group .btn{width:100%}[data-toggle=buttons]>.btn>input[type=radio],[data-toggle=buttons]>.btn>input[type=checkbox]{display:none}.input-group{position:relative;display:table;border-collapse:separate}.input-group[class*=col-]{float:none;padding-left:0;padding-right:0}.input-group .form-control{position:relative;z-index:2;float:left;width:100%;margin-bottom:0}.input-group-lg>.form-control,.input-group-lg>.input-group-addon,.input-group-lg>.input-group-btn>.btn{height:46px;padding:10px 16px;font-size:18px;line-height:1.33;border-radius:6px}select.input-group-lg>.form-control,select.input-group-lg>.input-group-addon,select.input-group-lg>.input-group-btn>.btn{height:46px;line-height:46px}textarea.input-group-lg>.form-control,textarea.input-group-lg>.input-group-addon,textarea.input-group-lg>.input-group-btn
 >.btn,select[multiple].input-group-lg>.form-control,select[multiple].input-group-lg>.input-group-addon,select[multiple].input-group-lg>.input-group-btn>.btn{height:auto}.input-group-sm>.form-control,.input-group-sm>.input-group-addon,.input-group-sm>.input-group-btn>.btn{height:30px;padding:5px 10px;font-size:12px;line-height:1.5;border-radius:3px}select.input-group-sm>.form-control,select.input-group-sm>.input-group-addon,select.input-group-sm>.input-group-btn>.btn{height:30px;line-height:30px}textarea.input-group-sm>.form-control,textarea.input-group-sm>.input-group-addon,textarea.input-group-sm>.input-group-btn>.btn,select[multiple].input-group-sm>.form-control,select[multiple].input-group-sm>.input-group-addon,select[multiple].input-group-sm>.input-group-btn>.btn{height:auto}.input-group-addon,.input-group-btn,.input-group .form-control{display:table-cell}.input-group-addon:not(:first-child):not(:last-child),.input-group-btn:not(:first-child):not(:last-child),.input-group .form-
 control:not(:first-child):not(:last-child){border-radius:0}.input-group-addon,.input-group-btn{width:1%;white-space:nowrap;vertical-align:middle}.input-group-addon{padding:6px 12px;font-size:14px;font-weight:400;line-height:1;color:#555;text-align:center;background-color:#eee;border:1px solid #ccc;border-radius:4px}.input-group-addon.input-sm{padding:5px 10px;font-size:12px;border-radius:3px}.input-group-addon.input-lg{padding:10px 16px;font-size:18px;border-radius:6px}.input-group-addon input[type=radio],.input-group-addon input[type=checkbox]{margin-top:0}.input-group .form-control:first-child,.input-group-addon:first-child,.input-group-btn:first-child>.btn,.input-group-btn:first-child>.btn-group>.btn,.input-group-btn:first-child>.dropdown-toggle,.input-group-btn:last-child>.btn:not(:last-child):not(.dropdown-toggle),.input-group-btn:last-child>.btn-group:not(:last-child)>.btn{border-bottom-right-radius:0;border-top-right-radius:0}.input-group-addon:first-child{border-right:0}.inp
 ut-group .form-control:last-child,.input-group-addon:last-child,.input-group-btn:last-child>.btn,.input-group-btn:last-child>.btn-group>.btn,.input-group-btn:last-child>.dropdown-toggle,.input-group-btn:first-child>.btn:not(:first-child),.input-group-btn:first-child>.btn-group:not(:first-child)>.btn{border-bottom-left-radius:0;border-top-left-radius:0}.input-group-addon:last-child{border-left:0}.input-group-btn{position:relative;font-size:0;white-space:nowrap}.input-group-btn>.btn{position:relative}.input-group-btn>.btn+.btn{margin-left:-1px}.input-group-btn>.btn:hover,.input-group-btn>.btn:focus,.input-group-btn>.btn:active{z-index:2}.input-group-btn:first-child>.btn,.input-group-btn:first-child>.btn-group{margin-right:-1px}.input-group-btn:last-child>.btn,.input-group-btn:last-child>.btn-group{margin-left:-1px}.nav{margin-bottom:0;padding-left:0;list-style:none}.nav>li{position:relative;display:block}.nav>li>a{position:relative;display:block;padding:10px 15px}.nav>li>a:hover,.nav>
 li>a:focus{text-decoration:none;background-color:#eee}.nav>li.disabled>a{color:#999}.nav>li.disabled>a:hover,.nav>li.disabled>a:focus{color:#999;text-decoration:none;background-color:transparent;cursor:not-allowed}.nav .open>a,.nav .open>a:hover,.nav .open>a:focus{background-color:#eee;border-color:#428bca}.nav .nav-divider{height:1px;margin:9px 0;overflow:hidden;background-color:#e5e5e5}.nav>li>a>img{max-width:none}.nav-tabs{border-bottom:1px solid #ddd}.nav-tabs>li{float:left;margin-bottom:-1px}.nav-tabs>li>a{margin-right:2px;line-height:1.42857143;border:1px solid transparent;border-radius:4px 4px 0 0}.nav-tabs>li>a:hover{border-color:#eee #eee #ddd}.nav-tabs>li.active>a,.nav-tabs>li.active>a:hover,.nav-tabs>li.active>a:focus{color:#555;background-color:#fff;border:1px solid #ddd;border-bottom-color:transparent;cursor:default}.nav-tabs.nav-justified{width:100%;border-bottom:0}.nav-tabs.nav-justified>li{float:none}.nav-tabs.nav-justified>li>a{text-align:center;margin-bottom:5px}.n
 av-tabs.nav-justified>.dropdown .dropdown-menu{top:auto;left:auto}@media (min-width:768px){.nav-tabs.nav-justified>li{display:table-cell;width:1%}.nav-tabs.nav-justified>li>a{margin-bottom:0}}.nav-tabs.nav-justified>li>a{margin-right:0;border-radius:4px}.nav-tabs.nav-justified>.active>a,.nav-tabs.nav-justified>.active>a:hover,.nav-tabs.nav-justified>.active>a:focus{border:1px solid #ddd}@media (min-width:768px){.nav-tabs.nav-justified>li>a{border-bottom:1px solid #ddd;border-radius:4px 4px 0 0}.nav-tabs.nav-justified>.active>a,.nav-tabs.nav-justified>.active>a:hover,.nav-tabs.nav-justified>.active>a:focus{border-bottom-color:#fff}}.nav-pills>li{float:left}.nav-pills>li>a{border-radius:4px}.nav-pills>li+li{margin-left:2px}.nav-pills>li.active>a,.nav-pills>li.active>a:hover,.nav-pills>li.active>a:focus{color:#fff;background-color:#428bca}.nav-stacked>li{float:none}.nav-stacked>li+li{margin-top:2px;margin-left:0}.nav-justified{width:100%}.nav-justified>li{float:none}.nav-justified>li>a
 {text-align:center;margin-bottom:5px}.nav-justified>.dropdown .dropdown-menu{top:auto;left:auto}@media (min-width:768px){.nav-justified>li{display:table-cell;width:1%}.nav-justified>li>a{margin-bottom:0}}.nav-tabs-justified{border-bottom:0}.nav-tabs-justified>li>a{margin-right:0;border-radius:4px}.nav-tabs-justified>.active>a,.nav-tabs-justified>.active>a:hover,.nav-tabs-justified>.active>a:focus{border:1px solid #ddd}@media (min-width:768px){.nav-tabs-justified>li>a{border-bottom:1px solid #ddd;border-radius:4px 4px 0 0}.nav-tabs-justified>.active>a,.nav-tabs-justified>.active>a:hover,.nav-tabs-justified>.active>a:focus{border-bottom-color:#fff}}.tab-content>.tab-pane{display:none}.tab-content>.active{display:block}.nav-tabs .dropdown-menu{margin-top:-1px;border-top-right-radius:0;border-top-left-radius:0}.navbar{position:relative;min-height:50px;margin-bottom:20px;border:1px solid transparent}@media (min-width:768px){.navbar{border-radius:4px}}@media (min-width:768px){.navbar-head
 er{float:left}}.navbar-collapse{max-height:340px;overflow-x:visible;padding-right:15px;padding-left:15px;border-top:1px solid transparent;box-shadow:inset 0 1px 0 rgba(255,255,255,.1);-webkit-overflow-scrolling:touch}.navbar-collapse.in{overflow-y:auto}@media (min-width:768px){.navbar-collapse{width:auto;border-top:0;box-shadow:none}.navbar-collapse.collapse{display:block!important;height:auto!important;padding-bottom:0;overflow:visible!important}.navbar-collapse.in{overflow-y:visible}.navbar-fixed-top .navbar-collapse,.navbar-static-top .navbar-collapse,.navbar-fixed-bottom .navbar-collapse{padding-left:0;padding-right:0}}.container>.navbar-header,.container-fluid>.navbar-header,.container>.navbar-collapse,.container-fluid>.navbar-collapse{margin-right:-15px;margin-left:-15px}@media (min-width:768px){.container>.navbar-header,.container-fluid>.navbar-header,.container>.navbar-collapse,.container-fluid>.navbar-collapse{margin-right:0;margin-left:0}}.navbar-static-top{z-index:1000;bo
 rder-width:0 0 1px}@media (min-width:768px){.navbar-static-top{border-radius:0}}.navbar-fixed-top,.navbar-fixed-bottom{position:fixed;right:0;left:0;z-index:1030}@media (min-width:768px){.navbar-fixed-top,.navbar-fixed-bottom{border-radius:0}}.navbar-fixed-top{top:0;border-width:0 0 1px}.navbar-fixed-bottom{bottom:0;margin-bottom:0;border-width:1px 0 0}.navbar-brand{float:left;padding:15px;font-size:18px;line-height:20px;height:50px}.navbar-brand:hover,.navbar-brand:focus{text-decoration:none}@media (min-width:768px){.navbar>.container .navbar-brand,.navbar>.container-fluid .navbar-brand{margin-left:-15px}}.navbar-toggle{position:relative;float:right;margin-right:15px;padding:9px 10px;margin-top:8px;margin-bottom:8px;background-color:transparent;background-image:none;border:1px solid transparent;border-radius:4px}.navbar-toggle:focus{outline:0}.navbar-toggle .icon-bar{display:block;width:22px;height:2px;border-radius:1px}.navbar-toggle .icon-bar+.icon-bar{margin-top:4px}@media (min-
 width:768px){.navbar-toggle{display:none}}.navbar-nav{margin:7.5px -15px}.navbar-nav>li>a{padding-top:10px;padding-bottom:10px;line-height:20px}@media (max-width:767px){.navbar-nav .open .dropdown-menu{position:static;float:none;width:auto;margin-top:0;background-color:transparent;border:0;box-shadow:none}.navbar-nav .open .dropdown-menu>li>a,.navbar-nav .open .dropdown-menu .dropdown-header{padding:5px 15px 5px 25px}.navbar-nav .open .dropdown-menu>li>a{line-height:20px}.navbar-nav .open .dropdown-menu>li>a:hover,.navbar-nav .open .dropdown-menu>li>a:focus{background-image:none}}@media (min-width:768px){.navbar-nav{float:left;margin:0}.navbar-nav>li{float:left}.navbar-nav>li>a{padding-top:15px;padding-bottom:15px}.navbar-nav.navbar-right:last-child{margin-right:-15px}}@media (min-width:768px){.navbar-left{float:left!important}.navbar-right{float:right!important}}.navbar-form{margin-left:-15px;margin-right:-15px;padding:10px 15px;border-top:1px solid transparent;border-bottom:1px so
 lid transparent;-webkit-box-shadow:inset 0 1px 0 rgba(255,255,255,.1),0 1px 0 rgba(255,255,255,.1);box-shadow:inset 0 1px 0 rgba(255,255,255,.1),0 1px 0 rgba(255,255,255,.1);margin-top:8px;margin-bottom:8px}@media (min-width:768px){.navbar-form .form-group{display:inline-block;margin-bottom:0;vertical-align:middle}.navbar-form .form-control{display:inline-block;width:auto;vertical-align:middle}.navbar-form .input-group>.form-control{width:100%}.navbar-form .control-label{margin-bottom:0;vertical-align:middle}.navbar-form .radio,.navbar-form .checkbox{display:inline-block;margin-top:0;margin-bottom:0;padding-left:0;vertical-align:middle}.navbar-form .radio input[type=radio],.navbar-form .checkbox input[type=checkbox]{float:none;margin-left:0}.navbar-form .has-feedback .form-control-feedback{top:0}}@media (max-width:767px){.navbar-form .form-group{margin-bottom:5px}}@media (min-width:768px){.navbar-form{width:auto;border:0;margin-left:0;margin-right:0;padding-top:0;padding-bottom:0;-w
 ebkit-box-shadow:none;box-shadow:none}.navbar-form.navbar-right:last-child{margin-right:-15px}}.navbar-nav>li>.dropdown-menu{margin-top:0;border-top-right-radius:0;border-top-left-radius:0}.navbar-fixed-bottom .navbar-nav>li>.dropdown-menu{border-bottom-right-radius:0;border-bottom-left-radius:0}.navbar-btn{margin-top:8px;margin-bottom:8px}.navbar-btn.btn-sm{margin-top:10px;margin-bottom:10px}.navbar-btn.btn-xs{margin-top:14px;margin-bottom:14px}.navbar-text{margin-top:15px;margin-bottom:15px}@media (min-width:768px){.navbar-text{float:left;margin-left:15px;margin-right:15px}.navbar-text.navbar-right:last-child{margin-right:0}}.navbar-default{background-color:#f8f8f8;border-color:#e7e7e7}.navbar-default .navbar-brand{color:#777}.navbar-default .navbar-brand:hover,.navbar-default .navbar-brand:focus{color:#5e5e5e;background-color:transparent}.navbar-default .navbar-text{color:#777}.navbar-default .navbar-nav>li>a{color:#777}.navbar-default .navbar-nav>li>a:hover,.navbar-default .navb
 ar-nav>li>a:focus{color:#333;background-color:transparent}.navbar-default .navbar-nav>.active>a,.navbar-default .navbar-nav>.active>a:hover,.navbar-default .navbar-nav>.active>a:focus{color:#555;background-color:#e7e7e7}.navbar-default .navbar-nav>.disabled>a,.navbar-default .navbar-nav>.disabled>a:hover,.navbar-default .navbar-nav>.disabled>a:focus{color:#ccc;background-color:transparent}.navbar-default .navbar-toggle{border-color:#ddd}.navbar-default .navbar-toggle:hover,.navbar-default .navbar-toggle:focus{background-color:#ddd}.navbar-default .navbar-toggle .icon-bar{background-color:#888}.navbar-default .navbar-collapse,.navbar-default .navbar-form{border-color:#e7e7e7}.navbar-default .navbar-nav>.open>a,.navbar-default .navbar-nav>.open>a:hover,.navbar-default .navbar-nav>.open>a:focus{background-color:#e7e7e7;color:#555}@media (max-width:767px){.navbar-default .navbar-nav .open .dropdown-menu>li>a{color:#777}.navbar-default .navbar-nav .open .dropdown-menu>li>a:hover,.navbar-
 default .navbar-nav .open .dropdown-menu>li>a:focus{color:#333;background-color:transparent}.navbar-default .navbar-nav .open .dropdown-menu>.active>a,.navbar-default .navbar-nav .open .dropdown-menu>.active>a:hover,.navbar-default .navbar-nav .open .dropdown-menu>.active>a:focus{color:#555;background-color:#e7e7e7}.navbar-default .navbar-nav .open .dropdown-menu>.disabled>a,.navbar-default .navbar-nav .open .dropdown-menu>.disabled>a:hover,.navbar-default .navbar-nav .open .dropdown-menu>.disabled>a:focus{color:#ccc;background-color:transparent}}.navbar-default .navbar-link{color:#777}.navbar-default .navbar-link:hover{color:#333}.navbar-inverse{background-color:#222;border-color:#080808}.navbar-inverse .navbar-brand{color:#999}.navbar-inverse .navbar-brand:hover,.navbar-inverse .navbar-brand:focus{color:#fff;background-color:transparent}.navbar-inverse .navbar-text{color:#999}.navbar-inverse .navbar-nav>li>a{color:#999}.navbar-inverse .navbar-nav>li>a:hover,.navbar-inverse .navbar
 -nav>li>a:focus{color:#fff;background-color:transparent}.navbar-inverse .navbar-nav>.active>a,.navbar-inverse .navbar-nav>.active>a:hover,.navbar-inverse .navbar-nav>.active>a:focus{color:#fff;background-color:#080808}.navbar-inverse .navbar-nav>.disabled>a,.navbar-inverse .navbar-nav>.disabled>a:hover,.navbar-inverse .navbar-nav>.disabled>a:focus{color:#444;background-color:transparent}.navbar-inverse .navbar-toggle{border-color:#333}.navbar-inverse .navbar-toggle:hover,.navbar-inverse .navbar-toggle:focus{background-color:#333}.navbar-inverse .navbar-toggle .icon-bar{background-color:#fff}.navbar-inverse .navbar-collapse,.navbar-inverse .navbar-form{border-color:#101010}.navbar-inverse .navbar-nav>.open>a,.navbar-inverse .navbar-nav>.open>a:hover,.navbar-inverse .navbar-nav>.open>a:focus{background-color:#080808;color:#fff}@media (max-width:767px){.navbar-inverse .navbar-nav .open .dropdown-menu>.dropdown-header{border-color:#080808}.navbar-inverse .navbar-nav .open .dropdown-menu
  .divider{background-color:#080808}.navbar-inverse .navbar-nav .open .dropdown-menu>li>a{color:#999}.navbar-inverse .navbar-nav .open .dropdown-menu>li>a:hover,.navbar-inverse .navbar-nav .open .dropdown-menu>li>a:focus{color:#fff;background-color:transparent}.navbar-inverse .navbar-nav .open .dropdown-menu>.active>a,.navbar-inverse .navbar-nav .open .dropdown-menu>.active>a:hover,.navbar-inverse .navbar-nav .open .dropdown-menu>.active>a:focus{color:#fff;background-color:#080808}.navbar-inverse .navbar-nav .open .dropdown-menu>.disabled>a,.navbar-inverse .navbar-nav .open .dropdown-menu>.disabled>a:hover,.navbar-inverse .navbar-nav .open .dropdown-menu>.disabled>a:focus{color:#444;background-color:transparent}}.navbar-inverse .navbar-link{color:#999}.navbar-inverse .navbar-link:hover{color:#fff}.breadcrumb{padding:8px 15px;margin-bottom:20px;list-style:none;background-color:#f5f5f5;border-radius:4px}.breadcrumb>li{display:inline-block}.breadcrumb>li+li:before{content:"/\00a0";paddi
 ng:0 5px;color:#ccc}.breadcrumb>.active{color:#999}.pagination{display:inline-block;padding-left:0;margin:20px 0;border-radius:4px}.pagination>li{display:inline}.pagination>li>a,.pagination>li>span{position:relative;float:left;padding:6px 12px;line-height:1.42857143;text-decoration:none;color:#428bca;background-color:#fff;border:1px solid #ddd;margin-left:-1px}.pagination>li:first-child>a,.pagination>li:first-child>span{margin-left:0;border-bottom-left-radius:4px;border-top-left-radius:4px}.pagination>li:last-child>a,.pagination>li:last-child>span{border-bottom-right-radius:4px;border-top-right-radius:4px}.pagination>li>a:hover,.pagination>li>span:hover,.pagination>li>a:focus,.pagination>li>span:focus{color:#2a6496;background-color:#eee;border-color:#ddd}.pagination>.active>a,.pagination>.active>span,.pagination>.active>a:hover,.pagination>.active>span:hover,.pagination>.active>a:focus,.pagination>.active>span:focus{z-index:2;color:#fff;background-color:#428bca;border-color:#428bca;
 cursor:default}.pagination>.disabled>span,.pagination>.disabled>span:hover,.pagination>.disabled>span:focus,.pagination>.disabled>a,.pagination>.disabled>a:hover,.pagination>.disabled>a:focus{color:#999;background-color:#fff;border-color:#ddd;cursor:not-allowed}.pagination-lg>li>a,.pagination-lg>li>span{padding:10px 16px;font-size:18px}.pagination-lg>li:first-child>a,.pagination-lg>li:first-child>span{border-bottom-left-radius:6px;border-top-left-radius:6px}.pagination-lg>li:last-child>a,.pagination-lg>li:last-child>span{border-bottom-right-radius:6px;border-top-right-radius:6px}.pagination-sm>li>a,.pagination-sm>li>span{padding:5px 10px;font-size:12px}.pagination-sm>li:first-child>a,.pagination-sm>li:first-child>span{border-bottom-left-radius:3px;border-top-left-radius:3px}.pagination-sm>li:last-child>a,.pagination-sm>li:last-child>span{border-bottom-right-radius:3px;border-top-right-radius:3px}.pager{padding-left:0;margin:20px 0;list-style:none;text-align:center}.pager li{display:
 inline}.pager li>a,.pager li>span{display:inline-block;padding:5px 14px;background-color:#fff;border:1px solid #ddd;border-radius:15px}.pager li>a:hover,.pager li>a:focus{text-decoration:none;background-color:#eee}.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;background-color:#fff;cursor:not-allowed}.label{display:inline;padding:.2em .6em .3em;font-size:75%;font-weight:700;line-height:1;color:#fff;text-align:center;white-space:nowrap;vertical-align:baseline;border-radius:.25em}.label[href]:hover,.label[href]:focus{color:#fff;text-decoration:none;cursor:pointer}.label:empty{display:none}.btn .label{position:relative;top:-1px}.label-default{background-color:#999}.label-default[href]:hover,.label-default[href]:focus{background-color:gray}.label-primary{background-color:#428bca}.label-primary[href]:hover,.label-primary[href]:focus{backg
 round-color:#3071a9}.label-success{background-color:#5cb85c}.label-success[href]:hover,.label-success[href]:focus{background-color:#449d44}.label-info{background-color:#5bc0de}.label-info[href]:hover,.label-info[href]:focus{background-color:#31b0d5}.label-warning{background-color:#f0ad4e}.label-warning[href]:hover,.label-warning[href]:focus{background-color:#ec971f}.label-danger{background-color:#d9534f}.label-danger[href]:hover,.label-danger[href]:focus{background-color:#c9302c}.badge{display:inline-block;min-width:10px;padding:3px 7px;font-size:12px;font-weight:700;color:#fff;line-height:1;vertical-align:baseline;white-space:nowrap;text-align:center;background-color:#999;border-radius:10px}.badge:empty{display:none}.btn .badge{position:relative;top:-1px}.btn-xs .badge{top:0;padding:1px 5px}a.badge:hover,a.badge:focus{color:#fff;text-decoration:none;cursor:pointer}a.list-group-item.active>.badge,.nav-pills>.active>a>.badge{color:#428bca;background-color:#fff}.nav-pills>li>a>.badge{
 margin-left:3px}.jumbotron{padding:30px;margin-bottom:30px;color:inherit;background-color:#eee}.jumbotron h1,.jumbotron .h1{color:inherit}.jumbotron p{margin-bottom:15px;font-size:21px;font-weight:200}.container .jumbotron{border-radius:6px}.jumbotron .container{max-width:100%}@media screen and (min-width:768px){.jumbotron{padding-top:48px;padding-bottom:48px}.container .jumbotron{padding-left:60px;padding-right:60px}.jumbotron h1,.jumbotron .h1{font-size:63px}}.thumbnail{display:block;padding:4px;margin-bottom:20px;line-height:1.42857143;background-color:#fff;border:1px solid #ddd;border-radius:4px;-webkit-transition:all .2s ease-in-out;transition:all .2s ease-in-out}.thumbnail>img,.thumbnail a>img{margin-left:auto;margin-right:auto}a.thumbnail:hover,a.thumbnail:focus,a.thumbnail.active{border-color:#428bca}.thumbnail .caption{padding:9px;color:#333}.alert{padding:15px;margin-bottom:20px;border:1px solid transparent;border-radius:4px}.alert h4{margin-top:0;color:inherit}.alert .ale
 rt-link{font-weight:700}.alert>p,.alert>ul{margin-bottom:0}.alert>p+p{margin-top:5px}.alert-dismissable{padding-right:35px}.alert-dismissable .close{position:relative;top:-2px;right:-21px;color:inherit}.alert-success{background-color:#dff0d8;border-color:#d6e9c6;color:#3c763d}.alert-success hr{border-top-color:#c9e2b3}.alert-success .alert-link{color:#2b542c}.alert-info{background-color:#d9edf7;border-color:#bce8f1;color:#31708f}.alert-info hr{border-top-color:#a6e1ec}.alert-info .alert-link{color:#245269}.alert-warning{background-color:#fcf8e3;border-color:#faebcc;color:#8a6d3b}.alert-warning hr{border-top-color:#f7e1b5}.alert-warning .alert-link{color:#66512c}.alert-danger{background-color:#f2dede;border-color:#ebccd1;color:#a94442}.alert-danger hr{border-top-color:#e4b9c0}.alert-danger .alert-link{color:#843534}@-webkit-keyframes progress-bar-stripes{from{background-position:40px 0}to{background-position:0 0}}@keyframes progress-bar-stripes{from{background-position:40px 0}to{back
 ground-position:0 0}}.progress{overflow:hidden;height:20px;margin-bottom:20px;background-color:#f5f5f5;border-radius:4px;-webkit-box-shadow:inset 0 1px 2px rgba(0,0,0,.1);box-shadow:inset 0 1px 2px rgba(0,0,0,.1)}.progress-bar{float:left;width:0;height:100%;font-size:12px;line-height:20px;color:#fff;text-align:center;background-color:#428bca;-webkit-box-shadow:inset 0 -1px 0 rgba(0,0,0,.15);box-shadow:inset 0 -1px 0 rgba(0,0,0,.15);-webkit-transition:width .6s ease;transition:width .6s ease}.progress-striped .progress-bar{background-image:-webkit-linear-gradient(45deg,rgba(255,255,255,.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,.15) 50%,rgba(255,255,255,.15) 75%,transparent 75%,transparent);background-image:linear-gradient(45deg,rgba(255,255,255,.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,.15) 50%,rgba(255,255,255,.15) 75%,transparent 75%,transparent);background-size:40px 40px}.progress.active .progress-bar{-webkit-animation:progress-bar-stripes 2s linear
  infinite;animation:progress-bar-stripes 2s linear infinite}.progress-bar-success{background-color:#5cb85c}.progress-striped .progress-bar-success{background-image:-webkit-linear-gradient(45deg,rgba(255,255,255,.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,.15) 50%,rgba(255,255,255,.15) 75%,transparent 75%,transparent);background-image:linear-gradient(45deg,rgba(255,255,255,.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,.15) 50%,rgba(255,255,255,.15) 75%,transparent 75%,transparent)}.progress-bar-info{background-color:#5bc0de}.progress-striped .progress-bar-info{background-image:-webkit-linear-gradient(45deg,rgba(255,255,255,.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,.15) 50%,rgba(255,255,255,.15) 75%,transparent 75%,transparent);background-image:linear-gradient(45deg,rgba(255,255,255,.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,.15) 50%,rgba(255,255,255,.15) 75%,transparent 75%,transparent)}.progress-bar-warning{background-color:#f
 0ad4e}.progress-striped .progress-bar-warning{background-image:-webkit-linear-gradient(45deg,rgba(255,255,255,.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,.15) 50%,rgba(255,255,255,.15) 75%,transparent 75%,transparent);background-image:linear-gradient(45deg,rgba(255,255,255,.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,.15) 50%,rgba(255,255,255,.15) 75%,transparent 75%,transparent)}.progress-bar-danger{background-color:#d9534f}.progress-striped .progress-bar-danger{background-image:-webkit-linear-gradient(45deg,rgba(255,255,255,.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,.15) 50%,rgba(255,255,255,.15) 75%,transparent 75%,transparent);background-image:linear-gradient(45deg,rgba(255,255,255,.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,.15) 50%,rgba(255,255,255,.15) 75%,transparent 75%,transparent)}.media,.media-body{overflow:hidden;zoom:1}.media,.media .media{margin-top:15px}.media:first-child{margin-top:0}.media-object{display:block
 }.media-heading{margin:0 0 5px}.media>.pull-left{margin-right:10px}.media>.pull-right{margin-left:10px}.media-list{padding-left:0;list-style:none}.list-group{margin-bottom:20px;padding-left:0}.list-group-item{position:relative;display:block;padding:10px 15px;margin-bottom:-1px;background-color:#fff;border:1px solid #ddd}.list-group-item:first-child{border-top-right-radius:4px;border-top-left-radius:4px}.list-group-item:last-child{margin-bottom:0;border-bottom-right-radius:4px;border-bottom-left-radius:4px}.list-group-item>.badge{float:right}.list-group-item>.badge+.badge{margin-right:5px}a.list-group-item{color:#555}a.list-group-item .list-group-item-heading{color:#333}a.list-group-item:hover,a.list-group-item:focus{text-decoration:none;background-color:#f5f5f5}a.list-group-item.active,a.list-group-item.active:hover,a.list-group-item.active:focus{z-index:2;color:#fff;background-color:#428bca;border-color:#428bca}a.list-group-item.active .list-group-item-heading,a.list-group-item.act
 ive:hover .list-group-item-heading,a.list-group-item.active:focus .list-group-item-heading{color:inherit}a.list-group-item.active .list-group-item-text,a.list-group-item.active:hover .list-group-item-text,a.list-group-item.active:focus .list-group-item-text{color:#e1edf7}.list-group-item-success{color:#3c763d;background-color:#dff0d8}a.list-group-item-success{color:#3c763d}a.list-group-item-success .list-group-item-heading{color:inherit}a.list-group-item-success:hover,a.list-group-item-success:focus{color:#3c763d;background-color:#d0e9c6}a.list-group-item-success.active,a.list-group-item-success.active:hover,a.list-group-item-success.active:focus{color:#fff;background-color:#3c763d;border-color:#3c763d}.list-group-item-info{color:#31708f;background-color:#d9edf7}a.list-group-item-info{color:#31708f}a.list-group-item-info .list-group-item-heading{color:inherit}a.list-group-item-info:hover,a.list-group-item-info:focus{color:#31708f;background-color:#c4e3f3}a.list-group-item-info.activ
 e,a.list-group-item-info.active:hover,a.list-group-item-info.active:focus{color:#fff;background-color:#31708f;border-color:#31708f}.list-group-item-warning{color:#8a6d3b;background-color:#fcf8e3}a.list-group-item-warning{color:#8a6d3b}a.list-group-item-warning .list-group-item-heading{color:inherit}a.list-group-item-warning:hover,a.list-group-item-warning:focus{color:#8a6d3b;background-color:#faf2cc}a.list-group-item-warning.active,a.list-group-item-warning.active:hover,a.list-group-item-warning.active:focus{color:#fff;background-color:#8a6d3b;border-color:#8a6d3b}.list-group-item-danger{color:#a94442;background-color:#f2dede}a.list-group-item-danger{color:#a94442}a.list-group-item-danger .list-group-item-heading{color:inherit}a.list-group-item-danger:hover,a.list-group-item-danger:focus{color:#a94442;background-color:#ebcccc}a.list-group-item-danger.active,a.list-group-item-danger.active:hover,a.list-group-item-danger.active:focus{color:#fff;background-color:#a94442;border-color:#a
 94442}.list-group-item-heading{margin-top:0;margin-bottom:5px}.list-group-item-text{margin-bottom:0;line-height:1.3}.panel{margin-bottom:20px;background-color:#fff;border:1px solid transparent;border-radius:4px;-webkit-box-shadow:0 1px 1px rgba(0,0,0,.05);box-shadow:0 1px 1px rgba(0,0,0,.05)}.panel-body{padding:15px}.panel-heading{padding:10px 15px;border-bottom:1px solid transparent;border-top-right-radius:3px;border-top-left-radius:3px}.panel-heading>.dropdown .dropdown-toggle{color:inherit}.panel-title{margin-top:0;margin-bottom:0;font-size:16px;color:inherit}.panel-title>a{color:inherit}.panel-footer{padding:10px 15px;background-color:#f5f5f5;border-top:1px solid #ddd;border-bottom-right-radius:3px;border-bottom-left-radius:3px}.panel>.list-group{margin-bottom:0}.panel>.list-group .list-group-item{border-width:1px 0;border-radius:0}.panel>.list-group:first-child .list-group-item:first-child{border-top:0;border-top-right-radius:3px;border-top-left-radius:3px}.panel>.list-group:la
 st-child .list-group-item:last-child{border-bottom:0;border-bottom-right-radius:3px;border-bottom-left-radius:3px}.panel-heading+.list-group .list-group-item:first-child{border-top-width:0}.panel>.table,.panel>.table-responsive>.table{margin-bottom:0}.panel>.table:first-child,.panel>.table-responsive:first-child>.table:first-child{border-top-right-radius:3px;border-top-left-radius:3px}.panel>.table:first-child>thead:first-child>tr:first-child td:first-child,.panel>.table-responsive:first-child>.table:first-child>thead:first-child>tr:first-child td:first-child,.panel>.table:first-child>tbody:first-child>tr:first-child td:first-child,.panel>.table-responsive:first-child>.table:first-child>tbody:first-child>tr:first-child td:first-child,.panel>.table:first-child>thead:first-child>tr:first-child th:first-child,.panel>.table-responsive:first-child>.table:first-child>thead:first-child>tr:first-child th:first-child,.panel>.table:first-child>tbody:first-child>tr:first-child th:first-child,.
 panel>.table-responsive:first-child>.table:first-child>tbody:first-child>tr:first-child th:first-child{border-top-left-radius:3px}.panel>.table:first-child>thead:first-child>tr:first-child td:last-child,.panel>.table-responsive:first-child>.table:first-child>thead:first-child>tr:first-child td:last-child,.panel>.table:first-child>tbody:first-child>tr:first-child td:last-child,.panel>.table-responsive:first-child>.table:first-child>tbody:first-child>tr:first-child td:last-child,.panel>.table:first-child>thead:first-child>tr:first-child th:last-child,.panel>.table-responsive:first-child>.table:first-child>thead:first-child>tr:first-child th:last-child,.panel>.table:first-child>tbody:first-child>tr:first-child th:last-child,.panel>.table-responsive:first-child>.table:first-child>tbody:first-child>tr:first-child th:last-child{border-top-right-radius:3px}.panel>.table:last-child,.panel>.table-responsive:last-child>.table:last-child{border-bottom-right-radius:3px;border-bottom-left-radius
 :3px}.panel>.table:last-child>tbody:last-child>tr:last-child td:first-child,.panel>.table-responsive:last-child>.table:last-child>tbody:last-child>tr:last-child td:first-child,.panel>.table:last-child>tfoot:last-child>tr:last-child td:first-child,.panel>.table-responsive:last-child>.table:last-child>tfoot:last-child>tr:last-child td:first-child,.panel>.table:last-child>tbody:last-child>tr:last-child th:first-child,.panel>.table-responsive:last-child>.table:last-child>tbody:last-child>tr:last-child th:first-child,.panel>.table:last-child>tfoot:last-child>tr:last-child th:first-child,.panel>.table-responsive:last-child>.table:last-child>tfoot:last-child>tr:last-child th:first-child{border-bottom-left-radius:3px}.panel>.table:last-child>tbody:last-child>tr:last-child td:last-child,.panel>.table-responsive:last-child>.table:last-child>tbody:last-child>tr:last-child td:last-child,.panel>.table:last-child>tfoot:last-child>tr:last-child td:last-child,.panel>.table-responsive:last-child>.ta
 ble:last-child>tfoot:last-child>tr:last-child td:last-child,.panel>.table:last-child>tbody:last-child>tr:last-child th:last-child,.panel>.table-responsive:last-child>.table:last-child>tbody:last-child>tr:last-child th:last-child,.panel>.table:last-child>tfoot:last-child>tr:last-child th:last-child,.panel>.table-responsive:last-child>.table:last-child>tfoot:last-child>tr:last-child th:last-child{border-bottom-right-radius:3px}.panel>.panel-body+.table,.panel>.panel-body+.table-responsive{border-top:1px solid #ddd}.panel>.table>tbody:first-child>tr:first-child th,.panel>.table>tbody:first-child>tr:first-child td{border-top:0}.panel>.table-bordered,.panel>.table-responsive>.table-bordered{border:0}.panel>.table-bordered>thead>tr>th:first-child,.panel>.table-responsive>.table-bordered>thead>tr>th:first-child,.panel>.table-bordered>tbody>tr>th:first-child,.panel>.table-responsive>.table-bordered>tbody>tr>th:first-child,.panel>.table-bordered>tfoot>tr>th:first-child,.panel>.table-responsi
 ve>.table-bordered>tfoot>tr>th:first-child,.panel>.table-bordered>thead>tr>td:first-child,.panel>.table-responsive>.table-bordered>thead>tr>td:first-child,.panel>.table-bordered>tbody>tr>td:first-child,.panel>.table-responsive>.table-bordered>tbody>tr>td:first-child,.panel>.table-bordered>tfoot>tr>td:first-child,.panel>.table-responsive>.table-bordered>tfoot>tr>td:first-child{border-left:0}.panel>.table-bordered>thead>tr>th:last-child,.panel>.table-responsive>.table-bordered>thead>tr>th:last-child,.panel>.table-bordered>tbody>tr>th:last-child,.panel>.table-responsive>.table-bordered>tbody>tr>th:last-child,.panel>.table-bordered>tfoot>tr>th:last-child,.panel>.table-responsive>.table-bordered>tfoot>tr>th:last-child,.panel>.table-bordered>thead>tr>td:last-child,.panel>.table-responsive>.table-bordered>thead>tr>td:last-child,.panel>.table-bordered>tbody>tr>td:last-child,.panel>.table-responsive>.table-bordered>tbody>tr>td:last-child,.panel>.table-bordered>tfoot>tr>td:last-child,.panel>.
 table-responsive>.table-bordered>tfoot>tr>td:last-child{border-right:0}.panel>.table-bordered>thead>tr:first-child>td,.panel>.table-responsive>.table-bordered>thead>tr:first-child>td,.panel>.table-bordered>tbody>tr:first-child>td,.panel>.table-responsive>.table-bordered>tbody>tr:first-child>td,.panel>.table-bordered>thead>tr:first-child>th,.panel>.table-responsive>.table-bordered>thead>tr:first-child>th,.panel>.table-bordered>tbody>tr:first-child>th,.panel>.table-responsive>.table-bordered>tbody>tr:first-child>th{border-bottom:0}.panel>.table-bordered>tbody>tr:last-child>td,.panel>.table-responsive>.table-bordered>tbody>tr:last-child>td,.panel>.table-bordered>tfoot>tr:last-child>td,.panel>.table-responsive>.table-bordered>tfoot>tr:last-child>td,.panel>.table-bordered>tbody>tr:last-child>th,.panel>.table-responsive>.table-bordered>tbody>tr:last-child>th,.panel>.table-bordered>tfoot>tr:last-child>th,.panel>.table-responsive>.table-bordered>tfoot>tr:last-child>th{border-bottom:0}.panel
 >.table-responsive{border:0;margin-bottom:0}.panel-group{margin-bottom:20px}.panel-group .panel{margin-bottom:0;border-radius:4px;overflow:hidden}.panel-group .panel+.panel{margin-top:5px}.panel-group .panel-heading{border-bottom:0}.panel-group .panel-heading+.panel-collapse .panel-body{border-top:1px solid #ddd}.panel-group .panel-footer{border-top:0}.panel-group .panel-footer+.panel-collapse .panel-body{border-bottom:1px solid #ddd}.panel-default{border-color:#ddd}.panel-default>.panel-heading{color:#333;background-color:#f5f5f5;border-color:#ddd}.panel-default>.panel-heading+.panel-collapse .panel-body{border-top-color:#ddd}.panel-default>.panel-footer+.panel-collapse .panel-body{border-bottom-color:#ddd}.panel-primary{border-color:#428bca}.panel-primary>.panel-heading{color:#fff;background-color:#428bca;border-color:#428bca}.panel-primary>.panel-heading+.panel-collapse .panel-body{border-top-color:#428bca}.panel-primary>.panel-footer+.panel-collapse .panel-body{border-bottom-col
 or:#428bca}.panel-success{border-color:#d6e9c6}.panel-success>.panel-heading{color:#3c763d;background-color:#dff0d8;border-color:#d6e9c6}.panel-success>.panel-heading+.panel-collapse .panel-body{border-top-color:#d6e9c6}.panel-success>.panel-footer+.panel-collapse .panel-body{border-bottom-color:#d6e9c6}.panel-info{border-color:#bce8f1}.panel-info>.panel-heading{color:#31708f;background-color:#d9edf7;border-color:#bce8f1}.panel-info>.panel-heading+.panel-collapse .panel-body{border-top-color:#bce8f1}.panel-info>.panel-footer+.panel-collapse .panel-body{border-bottom-color:#bce8f1}.panel-warning{border-color:#faebcc}.panel-warning>.panel-heading{color:#8a6d3b;background-color:#fcf8e3;border-color:#faebcc}.panel-warning>.panel-heading+.panel-collapse .panel-body{border-top-color:#faebcc}.panel-warning>.panel-footer+.panel-collapse .panel-body{border-bottom-color:#faebcc}.panel-danger{border-color:#ebccd1}.panel-danger>.panel-heading{color:#a94442;background-color:#f2dede;border-color:
 #ebccd1}.panel-danger>.panel-heading+.panel-collapse .panel-body{border-top-color:#ebccd1}.panel-danger>.panel-footer+.panel-collapse .panel-body{border-bottom-color:#ebccd1}.well{min-height:20px;padding:19px;margin-bottom:20px;background-color:#f5f5f5;border:1px solid #e3e3e3;border-radius:4px;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,.05);box-shadow:inset 0 1px 1px rgba(0,0,0,.05)}.well blockquote{border-color:#ddd;border-color:rgba(0,0,0,.15)}.well-lg{padding:24px;border-radius:6px}.well-sm{padding:9px;border-radius:3px}.close{float:right;font-size:21px;font-weight:700;line-height:1;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:.5;filter:alpha(opacity=50)}button.close{padding:0;cursor:pointer;background:0 0;border:0;-webkit-appearance:none}.modal-open{overflow:hidden}.modal{display:none;overflow:auto;overflow-y:scroll;position:fixed;top:0;right:0;bottom:0;left:0;z-index:
 1050;-webkit-overflow-scrolling:touch;outline:0}.modal.fade .modal-dialog{-webkit-transform:translate(0,-25%);-ms-transform:translate(0,-25%);transform:translate(0,-25%);-webkit-transition:-webkit-transform .3s ease-out;-moz-transition:-moz-

<TRUNCATED>

[23/29] samza git commit: review comments

Posted by bo...@apache.org.
review comments


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

Branch: refs/heads/NewKafkaSystemConsumer
Commit: 5397a34e2a6a5df0d7ae088ec2b309e65b53b4e7
Parents: 1d1fb89
Author: Boris S <bo...@apache.org>
Authored: Mon Sep 24 10:54:27 2018 -0700
Committer: Boris S <bo...@apache.org>
Committed: Mon Sep 24 10:54:27 2018 -0700

----------------------------------------------------------------------
 .../apache/samza/container/SamzaContainer.scala |   2 +-
 .../clients/consumer/KafkaConsumerConfig.java   | 194 ----------------
 .../samza/config/KafkaConsumerConfig.java       | 198 +++++++++++++++++
 .../samza/system/kafka/KafkaConsumerProxy.java  | 220 +++++++++----------
 .../samza/system/kafka/KafkaSystemConsumer.java | 187 ++++++++--------
 .../kafka/KafkaSystemConsumerMetrics.scala      |   2 +-
 .../samza/system/kafka/KafkaSystemFactory.scala |   4 +-
 .../consumer/TestKafkaConsumerConfig.java       | 137 ------------
 .../samza/config/TestKafkaConsumerConfig.java   | 152 +++++++++++++
 .../system/kafka/TestKafkaSystemConsumer.java   |  12 +-
 10 files changed, 552 insertions(+), 556 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/samza/blob/5397a34e/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
index e71fcb3..fba7329 100644
--- a/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala
+++ b/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala
@@ -822,7 +822,7 @@ class SamzaContainer(
     }
 
     try {
-      info("Shutting down SamzaContaier.")
+      info("Shutting down SamzaContainer.")
       removeShutdownHook
 
       jmxServer.stop

http://git-wip-us.apache.org/repos/asf/samza/blob/5397a34e/samza-kafka/src/main/scala/org/apache/kafka/clients/consumer/KafkaConsumerConfig.java
----------------------------------------------------------------------
diff --git a/samza-kafka/src/main/scala/org/apache/kafka/clients/consumer/KafkaConsumerConfig.java b/samza-kafka/src/main/scala/org/apache/kafka/clients/consumer/KafkaConsumerConfig.java
deleted file mode 100644
index 8ada1b4..0000000
--- a/samza-kafka/src/main/scala/org/apache/kafka/clients/consumer/KafkaConsumerConfig.java
+++ /dev/null
@@ -1,194 +0,0 @@
-/*
- *
- * 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.kafka.clients.consumer;
-
-import java.util.Map;
-import java.util.Properties;
-import org.apache.commons.lang3.StringUtils;
-import org.apache.kafka.common.serialization.ByteArrayDeserializer;
-import org.apache.samza.SamzaException;
-import org.apache.samza.config.Config;
-import org.apache.samza.config.ConfigException;
-import org.apache.samza.config.JobConfig;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import scala.Option;
-
-
-/**
- * The configuration class for KafkaConsumer
- */
-public class KafkaConsumerConfig extends ConsumerConfig {
-
-  public static final Logger LOG = LoggerFactory.getLogger(KafkaConsumerConfig.class);
-
-  private static final String PRODUCER_CLIENT_ID_PREFIX = "kafka-producer";
-  private static final String CONSUMER_CLIENT_ID_PREFIX = "kafka-consumer";
-  private static final String ADMIN_CLIENT_ID_PREFIX = "samza-admin";
-  private static final String SAMZA_OFFSET_LARGEST = "largest";
-  private static final String SAMZA_OFFSET_SMALLEST = "smallest";
-  private static final String KAFKA_OFFSET_LATEST = "latest";
-  private static final String KAFKA_OFFSET_EARLIEST = "earliest";
-  private static final String KAFKA_OFFSET_NONE = "none";
-
-  /*
-   * By default, KafkaConsumer will fetch ALL available messages for all the partitions.
-   * This may cause memory issues. That's why we will limit the number of messages per partition we get on EACH poll().
-   */
-  static final String DEFAULT_KAFKA_CONSUMER_MAX_POLL_RECORDS = "100";
-
-  private KafkaConsumerConfig(Properties props) {
-    super(props);
-  }
-
-  /**
-   * Create kafka consumer configs, based on the subset of global configs.
-   * @param config
-   * @param systemName
-   * @param clientId
-   * @param injectProps
-   * @return KafkaConsumerConfig
-   */
-  public static KafkaConsumerConfig getKafkaSystemConsumerConfig(Config config, String systemName, String clientId,
-      Map<String, String> injectProps) {
-
-    final Config subConf = config.subset(String.format("systems.%s.consumer.", systemName), true);
-
-    final String groupId = getConsumerGroupId(config);
-
-    final Properties consumerProps = new Properties();
-    consumerProps.putAll(subConf);
-
-    consumerProps.setProperty(ConsumerConfig.GROUP_ID_CONFIG, groupId);
-    consumerProps.setProperty(ConsumerConfig.CLIENT_ID_CONFIG, clientId);
-
-    //Kafka client configuration
-
-    // put overrides
-    consumerProps.putAll(injectProps);
-
-    // These are values we enforce in sazma, and they cannot be overwritten.
-
-    // Disable consumer auto-commit because Samza controls commits
-    consumerProps.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false");
-
-    // Translate samza config value to kafka config value
-    consumerProps.setProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG,
-        getAutoOffsetResetValue(consumerProps));
-
-    // make sure bootstrap configs are in ?? SHOULD WE FAIL IF THEY ARE NOT?
-    if (!subConf.containsKey(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG)) {
-      // get it from the producer config
-      String bootstrapServers =
-          config.get(String.format("systems.%s.producer.%s", systemName, ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG));
-      if (StringUtils.isEmpty(bootstrapServers)) {
-        throw new SamzaException("Missing " + ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG + " config  for " + systemName);
-      }
-      consumerProps.setProperty(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers);
-    }
-
-    // Always use default partition assignment strategy. Do not allow override.
-    consumerProps.setProperty(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG, RangeAssignor.class.getName());
-
-    // the consumer is fully typed, and deserialization can be too. But in case it is not provided we should
-    // default to byte[]
-    if (!consumerProps.containsKey(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG)) {
-      LOG.info("setting default key serialization for the consumer(for {}) to ByteArrayDeserializer", systemName);
-      consumerProps.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName());
-    }
-    if (!consumerProps.containsKey(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG)) {
-      LOG.info("setting default value serialization for the consumer(for {}) to ByteArrayDeserializer", systemName);
-      consumerProps.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName());
-    }
-
-    // Override default max poll config if there is no value
-    consumerProps.computeIfAbsent(ConsumerConfig.MAX_POLL_RECORDS_CONFIG,
-        (k) -> DEFAULT_KAFKA_CONSUMER_MAX_POLL_RECORDS);
-
-    return new KafkaConsumerConfig(consumerProps);
-  }
-
-  // group id should be unique per job
-  static String getConsumerGroupId(Config config) {
-    JobConfig jobConfig = new JobConfig(config);
-    Option<String> jobIdOption = jobConfig.getJobId();
-    Option<String> jobNameOption = jobConfig.getName();
-    return (jobNameOption.isDefined() ? jobNameOption.get() : "undefined_job_name") + "-" + (jobIdOption.isDefined()
-        ? jobIdOption.get() : "undefined_job_id");
-  }
-
-  // client id should be unique per job
-  public static String getConsumerClientId(Config config) {
-    return getConsumerClientId(CONSUMER_CLIENT_ID_PREFIX, config);
-  }
-  public static String getProducerClientId(Config config) {
-    return getConsumerClientId(PRODUCER_CLIENT_ID_PREFIX, config);
-  }
-  public static String getAdminClientId(Config config) {
-    return getConsumerClientId(ADMIN_CLIENT_ID_PREFIX, config);
-  }
-
-  private static String getConsumerClientId(String id, Config config) {
-    if (config.get(JobConfig.JOB_NAME()) == null) {
-      throw new ConfigException("Missing job name");
-    }
-    String jobName = config.get(JobConfig.JOB_NAME());
-    String jobId = (config.get(JobConfig.JOB_ID()) != null) ? config.get(JobConfig.JOB_ID()) : "1";
-
-    return String.format("%s-%s-%s", id.replaceAll("[^A-Za-z0-9]", "_"), jobName.replaceAll("[^A-Za-z0-9]", "_"),
-        jobId.replaceAll("[^A-Za-z0-9]", "_"));
-  }
-
-  /**
-   * If settings for auto.reset in samza are different from settings in Kafka (auto.offset.reset),
-   * then need to convert them (see kafka.apache.org/documentation):
-   * "largest" -> "latest"
-   * "smallest" -> "earliest"
-   *
-   * If no setting specified we return "latest" (same as Kafka).
-   * @param properties All consumer related {@link Properties} parsed from samza config
-   * @return String representing the config value for "auto.offset.reset" property
-   */
-  static String getAutoOffsetResetValue(Properties properties) {
-    String autoOffsetReset = properties.getProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, KAFKA_OFFSET_LATEST);
-
-    // accept kafka values directly
-    if (autoOffsetReset.equals(KAFKA_OFFSET_EARLIEST) || autoOffsetReset.equals(KAFKA_OFFSET_LATEST)
-        || autoOffsetReset.equals(KAFKA_OFFSET_NONE)) {
-      return autoOffsetReset;
-    }
-
-    String newAutoOffsetReset;
-    switch (autoOffsetReset) {
-      case SAMZA_OFFSET_LARGEST:
-        newAutoOffsetReset =  KAFKA_OFFSET_LATEST;
-        break;
-      case SAMZA_OFFSET_SMALLEST:
-        newAutoOffsetReset =  KAFKA_OFFSET_EARLIEST;
-        break;
-      default:
-        newAutoOffsetReset =  KAFKA_OFFSET_LATEST;
-    }
-    LOG.info("AutoOffsetReset value converted from {} to {}", autoOffsetReset,  newAutoOffsetReset);
-    return newAutoOffsetReset;
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/samza/blob/5397a34e/samza-kafka/src/main/scala/org/apache/samza/config/KafkaConsumerConfig.java
----------------------------------------------------------------------
diff --git a/samza-kafka/src/main/scala/org/apache/samza/config/KafkaConsumerConfig.java b/samza-kafka/src/main/scala/org/apache/samza/config/KafkaConsumerConfig.java
new file mode 100644
index 0000000..4bbe00f
--- /dev/null
+++ b/samza-kafka/src/main/scala/org/apache/samza/config/KafkaConsumerConfig.java
@@ -0,0 +1,198 @@
+/*
+ *
+ * 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.HashMap;
+import java.util.Map;
+import java.util.Properties;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.clients.consumer.RangeAssignor;
+import org.apache.kafka.common.serialization.ByteArrayDeserializer;
+import org.apache.samza.SamzaException;
+import org.apache.samza.config.JobConfig;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.Option;
+
+
+/**
+ * The configuration class for KafkaConsumer
+ */
+public class KafkaConsumerConfig extends HashMap<String, Object> {
+
+  public static final Logger LOG = LoggerFactory.getLogger(KafkaConsumerConfig.class);
+
+  private static final String PRODUCER_CLIENT_ID_PREFIX = "kafka-producer";
+  private static final String CONSUMER_CLIENT_ID_PREFIX = "kafka-consumer";
+  private static final String ADMIN_CLIENT_ID_PREFIX = "samza-admin";
+
+  /*
+   * By default, KafkaConsumer will fetch some big number of available messages for all the partitions.
+   * This may cause memory issues. That's why we will limit the number of messages per partition we get on EACH poll().
+   */
+  static final String DEFAULT_KAFKA_CONSUMER_MAX_POLL_RECORDS = "100";
+
+  private KafkaConsumerConfig(Map<String, Object> map) {
+    super(map);
+  }
+
+  /**
+   * This is a help method to create the configs for use in Kafka consumer.
+   * The values are based on the "consumer" subset of the configs provided by the app and Samza overrides.
+   *
+   * @param config - config provided by the app.
+   * @param systemName - system name for which the consumer is configured.
+   * @param clientId - client id to be used in the Kafka consumer.
+   * @return KafkaConsumerConfig
+   */
+  public static KafkaConsumerConfig getKafkaSystemConsumerConfig(Config config, String systemName, String clientId) {
+
+    Config subConf = config.subset(String.format("systems.%s.consumer.", systemName), true);
+
+    //Kafka client configuration
+    String groupId = getConsumerGroupId(config);
+
+    Map<String, Object> consumerProps = new HashMap<>();
+    consumerProps.putAll(subConf);
+
+    consumerProps.put(ConsumerConfig.GROUP_ID_CONFIG, groupId);
+    consumerProps.put(ConsumerConfig.CLIENT_ID_CONFIG, clientId);
+
+
+    // These are values we enforce in sazma, and they cannot be overwritten.
+
+    // Disable consumer auto-commit because Samza controls commits
+    consumerProps.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false");
+
+    // Translate samza config value to kafka config value
+    consumerProps.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG,
+        getAutoOffsetResetValue((String)consumerProps.get(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG)));
+
+    // make sure bootstrap configs are in, if not - get them from the producer
+    if (!subConf.containsKey(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG)) {
+      String bootstrapServers =
+          config.get(String.format("systems.%s.producer.%s", systemName, ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG));
+      if (StringUtils.isEmpty(bootstrapServers)) {
+        throw new SamzaException("Missing " + ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG + " config  for " + systemName);
+      }
+      consumerProps.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers);
+    }
+
+    // Always use default partition assignment strategy. Do not allow override.
+    consumerProps.put(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG, RangeAssignor.class.getName());
+
+    // the consumer is fully typed, and deserialization can be too. But in case it is not provided we should
+    // default to byte[]
+    if (!consumerProps.containsKey(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG)) {
+      LOG.info("setting key serialization for the consumer(for system {}) to ByteArrayDeserializer", systemName);
+      consumerProps.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName());
+    }
+    if (!consumerProps.containsKey(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG)) {
+      LOG.info("setting value serialization for the consumer(for system {}) to ByteArrayDeserializer", systemName);
+      consumerProps.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName());
+    }
+
+    // Override default max poll config if there is no value
+    consumerProps.computeIfAbsent(ConsumerConfig.MAX_POLL_RECORDS_CONFIG,
+        (k) -> DEFAULT_KAFKA_CONSUMER_MAX_POLL_RECORDS);
+
+    return new KafkaConsumerConfig(consumerProps);
+  }
+
+  // group id should be unique per job
+  static String getConsumerGroupId(Config config) {
+    JobConfig jobConfig = new JobConfig(config);
+    Option<String> jobIdOption = jobConfig.getJobId();
+    Option<String> jobNameOption = jobConfig.getName();
+    return (jobNameOption.isDefined() ? jobNameOption.get() : "undefined_job_name") + "-" + (jobIdOption.isDefined()
+        ? jobIdOption.get() : "undefined_job_id");
+  }
+
+  // client id should be unique per job
+  public static String getConsumerClientId(Config config) {
+    return getConsumerClientId(CONSUMER_CLIENT_ID_PREFIX, config);
+  }
+
+  public static String getProducerClientId(Config config) {
+    return getConsumerClientId(PRODUCER_CLIENT_ID_PREFIX, config);
+  }
+
+  public static String getAdminClientId(Config config) {
+    return getConsumerClientId(ADMIN_CLIENT_ID_PREFIX, config);
+  }
+
+  static String getConsumerClientId(String id, Config config) {
+    if (config.get(JobConfig.JOB_NAME()) == null) {
+      throw new ConfigException("Missing job name");
+    }
+    String jobName = config.get(JobConfig.JOB_NAME());
+    String jobId = (config.get(JobConfig.JOB_ID()) != null) ? config.get(JobConfig.JOB_ID()) : "1";
+
+    return String.format("%s-%s-%s", id.replaceAll(
+        "\\W", "_"),
+        jobName.replaceAll("\\W", "_"),
+        jobId.replaceAll("\\W", "_"));
+  }
+
+  /**
+   * If settings for auto.reset in samza are different from settings in Kafka (auto.offset.reset),
+   * then need to convert them (see kafka.apache.org/documentation):
+   * "largest" -> "latest"
+   * "smallest" -> "earliest"
+   *
+   * If no setting specified we return "latest" (same as Kafka).
+   * @param autoOffsetReset value from the app provided config
+   * @return String representing the config value for "auto.offset.reset" property
+   */
+  static String getAutoOffsetResetValue(final String autoOffsetReset) {
+    final String SAMZA_OFFSET_LARGEST = "largest";
+    final String SAMZA_OFFSET_SMALLEST = "smallest";
+    final String KAFKA_OFFSET_LATEST = "latest";
+    final String KAFKA_OFFSET_EARLIEST = "earliest";
+    final String KAFKA_OFFSET_NONE = "none";
+
+    if (autoOffsetReset == null) {
+     return KAFKA_OFFSET_LATEST; // return default
+    }
+
+    // accept kafka values directly
+    if (autoOffsetReset.equals(KAFKA_OFFSET_EARLIEST) || autoOffsetReset.equals(KAFKA_OFFSET_LATEST)
+        || autoOffsetReset.equals(KAFKA_OFFSET_NONE)) {
+      return autoOffsetReset;
+    }
+
+    String newAutoOffsetReset;
+    switch (autoOffsetReset) {
+      case SAMZA_OFFSET_LARGEST:
+        newAutoOffsetReset =  KAFKA_OFFSET_LATEST;
+        break;
+      case SAMZA_OFFSET_SMALLEST:
+        newAutoOffsetReset =  KAFKA_OFFSET_EARLIEST;
+        break;
+      default:
+        newAutoOffsetReset =  KAFKA_OFFSET_LATEST;
+    }
+    LOG.info("AutoOffsetReset value converted from {} to {}", autoOffsetReset,  newAutoOffsetReset);
+    return newAutoOffsetReset;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/samza/blob/5397a34e/samza-kafka/src/main/scala/org/apache/samza/system/kafka/KafkaConsumerProxy.java
----------------------------------------------------------------------
diff --git a/samza-kafka/src/main/scala/org/apache/samza/system/kafka/KafkaConsumerProxy.java b/samza-kafka/src/main/scala/org/apache/samza/system/kafka/KafkaConsumerProxy.java
index b67df0a..d2f7096 100644
--- a/samza-kafka/src/main/scala/org/apache/samza/system/kafka/KafkaConsumerProxy.java
+++ b/samza-kafka/src/main/scala/org/apache/samza/system/kafka/KafkaConsumerProxy.java
@@ -30,6 +30,7 @@ import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
+import javax.print.DocFlavor;
 import kafka.common.KafkaException;
 import kafka.common.TopicAndPartition;
 import org.apache.kafka.clients.consumer.Consumer;
@@ -47,7 +48,8 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 /**
- * Separate thread that reads messages from kafka and puts them into the BlockingEnvelopeMap.
+ * This class contains a separate thread that reads messages from kafka and puts them  into the BlockingEnvelopeMap
+ * through KafkaSystemConsumer.KafkaConsumerMessageSink object.
  * This class is not thread safe. There will be only one instance of this class per KafkaSystemConsumer object.
  * We still need some synchronization around kafkaConsumer. See pollConsumer() method for details.
  */
@@ -74,7 +76,8 @@ import org.slf4j.LoggerFactory;
   private volatile Throwable failureCause = null;
   private final CountDownLatch consumerPollThreadStartLatch = new CountDownLatch(1);
 
-  /* package private */KafkaConsumerProxy(Consumer<K, V> kafkaConsumer, String systemName, String clientId,
+  // package private constructor
+  KafkaConsumerProxy(Consumer<K, V> kafkaConsumer, String systemName, String clientId,
       KafkaSystemConsumer.KafkaConsumerMessageSink messageSink, KafkaSystemConsumerMetrics samzaConsumerMetrics,
       String metricName) {
 
@@ -93,6 +96,11 @@ import org.slf4j.LoggerFactory;
         "Samza KafkaConsumerProxy Poll " + consumerPollThread.getName() + " - " + systemName);
   }
 
+  @Override
+  public String toString() {
+    return String.format("consumerProxy-%s-%s", systemName, clientId);
+  }
+
   public void start() {
     if (!consumerPollThread.isAlive()) {
       LOG.info("Starting KafkaConsumerProxy polling thread for system " + systemName + " " + this.toString());
@@ -108,12 +116,43 @@ import org.slf4j.LoggerFactory;
         }
       }
     } else {
-      LOG.debug("Tried to start an already started KafkaConsumerProxy (%s). Ignoring.", this.toString());
+      LOG.warn("Tried to start an already started KafkaConsumerProxy (%s). Ignoring.", this.toString());
+    }
+
+    if (topicPartitions2SSP.size() == 0) {
+      String msg = String.format("Cannot start empty set of TopicPartitions for system %s, clientid %s",
+          systemName, clientId);
+      LOG.error(msg);
+      throw new SamzaException(msg);
     }
   }
 
-  // add new partition to the list of polled partitions
-  // this method is called only at the beginning, before the thread is started
+  /**
+   * Stop the thread and wait for it to stop.
+   * @param timeoutMs how long to wait in join
+   */
+  public void stop(long timeoutMs) {
+    LOG.info("Shutting down KafkaConsumerProxy poll thread:" + consumerPollThread.getName());
+
+    isRunning = false;
+    try {
+      consumerPollThread.join(timeoutMs);
+      // join returns event if the thread didn't finish
+      // in this case we should interrupt it and wait again
+      if (consumerPollThread.isAlive()) {
+        consumerPollThread.interrupt();
+        consumerPollThread.join(timeoutMs);
+      }
+    } catch (InterruptedException e) {
+      LOG.warn("Join in KafkaConsumerProxy has failed", e);
+      consumerPollThread.interrupt();
+    }
+  }
+
+  /**
+   * Add new partition to the list of polled partitions.
+   * This method should be called only at the beginning, before the thread is started.
+   */
   public void addTopicPartition(SystemStreamPartition ssp, long nextOffset) {
     LOG.info(String.format("Adding new topic and partition %s, offset = %s to queue for consumer %s", ssp, nextOffset,
         this));
@@ -124,67 +163,13 @@ import org.slf4j.LoggerFactory;
 
     nextOffsets.put(ssp, nextOffset);
 
-    // we reuse existing metrics. They assume host and port for the broker
-    // for now fake the port with the consumer name
     kafkaConsumerMetrics.setTopicPartitionValue(metricName, nextOffsets.size());
   }
 
-  /**
-   * creates a separate thread for pulling messages
-   */
-  private Runnable createProxyThreadRunnable() {
-    Runnable runnable=  () -> {
-      isRunning = true;
-
-      try {
-        consumerPollThreadStartLatch.countDown();
-        LOG.info("Starting runnable " + consumerPollThread.getName());
-        initializeLags();
-        while (isRunning) {
-          fetchMessages();
-        }
-      } catch (Throwable throwable) {
-        LOG.error(String.format("Error in KafkaConsumerProxy poll thread for system: %s.", systemName), throwable);
-        // SamzaKafkaSystemConsumer uses the failureCause to propagate the throwable to the container
-        failureCause = throwable;
-        isRunning = false;
-      }
-
-      if (!isRunning) {
-        LOG.info("Stopping the KafkaConsumerProxy poll thread for system: {}.", systemName);
-      }
-    };
-
-    return runnable;
-  }
-
-  private void initializeLags() {
-    // This is expensive, so only do it once at the beginning. After the first poll, we can rely on metrics for lag.
-    Map<TopicPartition, Long> endOffsets = kafkaConsumer.endOffsets(topicPartitions2SSP.keySet());
-    endOffsets.forEach((tp, offset) -> {
-      SystemStreamPartition ssp = topicPartitions2SSP.get(tp);
-      long startingOffset = nextOffsets.get(ssp);
-      // End offsets are the offset of the newest message + 1
-      // If the message we are about to consume is < end offset, we are starting with a lag.
-      long initialLag = endOffsets.get(tp) - startingOffset;
-
-      LOG.info("Initial lag for SSP {} is {} (end={}, startOffset={})", ssp, initialLag, endOffsets.get(tp), startingOffset);
-      latestLags.put(ssp, initialLag);
-      sink.setIsAtHighWatermark(ssp, initialLag == 0);
-    });
-
-    // initialize lag metrics
-    refreshLatencyMetrics();
-  }
-
   // the actual polling of the messages from kafka
-  public Map<SystemStreamPartition, List<IncomingMessageEnvelope>> pollConsumer(
+  private Map<SystemStreamPartition, List<IncomingMessageEnvelope>> pollConsumer(
       Set<SystemStreamPartition> systemStreamPartitions, long timeout) {
 
-    if (topicPartitions2SSP.size() == 0) {
-      throw new SamzaException("cannot poll empty set of TopicPartitions");
-    }
-
     // Since we need to poll only from some subset of TopicPartitions (passed as the argument),
     // we need to pause the rest.
     List<TopicPartition> topicPartitionsToPause = new ArrayList<>();
@@ -201,10 +186,9 @@ import org.slf4j.LoggerFactory;
     }
 
     ConsumerRecords<K, V> records;
-    // make a call on the client
+
     try {
-      // Currently, when doing checkpoint we are making a safeOffset request through this client, thus we need to synchronize
-      // them. In the future we may use this client for the actually checkpointing.
+      // Synchronize, in case the consumer is used in some other thread (metadata or something else)
       synchronized (kafkaConsumer) {
         // Since we are not polling from ALL the subscribed topics, so we need to "change" the subscription temporarily
         kafkaConsumer.pause(topicPartitionsToPause);
@@ -213,12 +197,7 @@ import org.slf4j.LoggerFactory;
         // resume original set of subscription - may be required for checkpointing
         kafkaConsumer.resume(topicPartitionsToPause);
       }
-    } catch (InvalidOffsetException e) {
-      // If the consumer has thrown this exception it means that auto reset is not set for this consumer.
-      // So we just rethrow.
-      LOG.error("Caught InvalidOffsetException in pollConsumer", e);
-      throw e;
-    } catch (KafkaException e) {
+    } catch (Exception e) {
       // we may get InvalidOffsetException | AuthorizationException | KafkaException exceptions,
       // but we still just rethrow, and log it up the stack.
       LOG.error("Caught a Kafka exception in pollConsumer", e);
@@ -230,11 +209,10 @@ import org.slf4j.LoggerFactory;
 
   private Map<SystemStreamPartition, List<IncomingMessageEnvelope>> processResults(ConsumerRecords<K, V> records) {
     if (records == null) {
-      throw new SamzaException("processResults is called with null object for records");
+      throw new SamzaException("ERROR:records is null, after pollConsumer call (in processResults)");
     }
 
-    int capacity = (int) (records.count() / 0.75 + 1); // to avoid rehash, allocate more then 75% of expected capacity.
-    Map<SystemStreamPartition, List<IncomingMessageEnvelope>> results = new HashMap<>(capacity);
+    Map<SystemStreamPartition, List<IncomingMessageEnvelope>> results = new HashMap<>(records.count());
     // Parse the returned records and convert them into the IncomingMessageEnvelope.
     // Note. They have been already de-serialized by the consumer.
     for (ConsumerRecord<K, V> record : records) {
@@ -268,6 +246,52 @@ import org.slf4j.LoggerFactory;
     return results;
   }
 
+   // creates a separate thread for getting the messages.
+  private Runnable createProxyThreadRunnable() {
+    Runnable runnable=  () -> {
+      isRunning = true;
+
+      try {
+        consumerPollThreadStartLatch.countDown();
+        LOG.info("Starting runnable " + consumerPollThread.getName());
+        initializeLags();
+        while (isRunning) {
+          fetchMessages();
+        }
+      } catch (Throwable throwable) {
+        LOG.error(String.format("Error in KafkaConsumerProxy poll thread for system: %s.", systemName), throwable);
+        // KafkaSystemConsumer uses the failureCause to propagate the throwable to the container
+        failureCause = throwable;
+        isRunning = false;
+      }
+
+      if (!isRunning) {
+        LOG.info("KafkaConsumerProxy for system {} has stopped.", systemName);
+      }
+    };
+
+    return runnable;
+  }
+
+  private void initializeLags() {
+    // This is expensive, so only do it once at the beginning. After the first poll, we can rely on metrics for lag.
+    Map<TopicPartition, Long> endOffsets = kafkaConsumer.endOffsets(topicPartitions2SSP.keySet());
+    endOffsets.forEach((tp, offset) -> {
+      SystemStreamPartition ssp = topicPartitions2SSP.get(tp);
+      long startingOffset = nextOffsets.get(ssp);
+      // End offsets are the offset of the newest message + 1
+      // If the message we are about to consume is < end offset, we are starting with a lag.
+      long initialLag = endOffsets.get(tp) - startingOffset;
+
+      LOG.info("Initial lag for SSP {} is {} (end={}, startOffset={})", ssp, initialLag, endOffsets.get(tp), startingOffset);
+      latestLags.put(ssp, initialLag);
+      sink.setIsAtHighWatermark(ssp, initialLag == 0);
+    });
+
+    // initialize lag metrics
+    refreshLatencyMetrics();
+  }
+
   private int getRecordSize(ConsumerRecord<K, V> r) {
     int keySize = (r.key() == null) ? 0 : r.serializedKeySize();
     return keySize + r.serializedValueSize();
@@ -291,9 +315,7 @@ import org.slf4j.LoggerFactory;
     kafkaConsumerMetrics.setHighWatermarkValue(tap, highWatermark);
   }
 
-  /*
-   This method put messages into blockingEnvelopeMap.
-   */
+
   private void moveMessagesToTheirQueue(SystemStreamPartition ssp, List<IncomingMessageEnvelope> envelopes) {
     long nextOffset = nextOffsets.get(ssp);
 
@@ -317,11 +339,9 @@ import org.slf4j.LoggerFactory;
     }
   }
 
-  /*
-    The only way to figure out lag for the KafkaConsumer is to look at the metrics after each poll() call.
-    One of the metrics (records-lag) shows how far behind the HighWatermark the consumer is.
-    This method populates the lag information for each SSP into latestLags member variable.
-   */
+  // The only way to figure out lag for the KafkaConsumer is to look at the metrics after each poll() call.
+  // One of the metrics (records-lag) shows how far behind the HighWatermark the consumer is.
+  // This method populates the lag information for each SSP into latestLags member variable.
   private void populateCurrentLags(Set<SystemStreamPartition> ssps) {
 
     Map<MetricName, ? extends Metric> consumerMetrics = kafkaConsumer.metrics();
@@ -339,12 +359,6 @@ import org.slf4j.LoggerFactory;
       // so the lag is now at least 0, which is the same as Samza's definition.
       // If the lag is not 0, then isAtHead is not true, and kafkaClient keeps polling.
       long currentLag = (currentLagM != null) ? (long) currentLagM.value() : -1L;
-      /*
-      Metric averageLagM = consumerMetrics.get(new MetricName(tp + ".records-lag-avg", "consumer-fetch-manager-metrics", "", tags));
-      double averageLag = (averageLagM != null) ? averageLagM.value() : -1.0;
-      Metric maxLagM = consumerMetrics.get(new MetricName(tp + ".records-lag-max", "consumer-fetch-manager-metrics", "", tags));
-      double maxLag = (maxLagM != null) ? maxLagM.value() : -1.0;
-      */
       latestLags.put(ssp, currentLag);
 
       // calls the setIsAtHead for the BlockingEnvelopeMap
@@ -352,10 +366,8 @@ import org.slf4j.LoggerFactory;
     }
   }
 
-  /*
-    Get the latest lag for a specific SSP.
-   */
-  public long getLatestLag(SystemStreamPartition ssp) {
+  // Get the latest lag for a specific SSP.
+  private long getLatestLag(SystemStreamPartition ssp) {
     Long lag = latestLags.get(ssp);
     if (lag == null) {
       throw new SamzaException("Unknown/unregistered ssp in latestLags request: " + ssp);
@@ -363,9 +375,7 @@ import org.slf4j.LoggerFactory;
     return lag;
   }
 
-  /*
-    Using the consumer to poll the messages from the stream.
-   */
+  // Using the consumer to poll the messages from the stream.
   private void fetchMessages() {
     Set<SystemStreamPartition> sspsToFetch = new HashSet<>();
     for (SystemStreamPartition ssp : nextOffsets.keySet()) {
@@ -380,7 +390,7 @@ import org.slf4j.LoggerFactory;
       Map<SystemStreamPartition, List<IncomingMessageEnvelope>> response;
       LOG.debug("pollConsumer from following SSPs: {}; total#={}", sspsToFetch, sspsToFetch.size());
 
-      response = pollConsumer(sspsToFetch, 500); // TODO should be default value from ConsumerConfig
+      response = pollConsumer(sspsToFetch, 500L);
 
       // move the responses into the queue
       for (Map.Entry<SystemStreamPartition, List<IncomingMessageEnvelope>> e : response.entrySet()) {
@@ -430,27 +440,5 @@ import org.slf4j.LoggerFactory;
   Throwable getFailureCause() {
     return failureCause;
   }
-
-  /**
-   * stop the thread and wait for it to stop
-   * @param timeoutMs how long to wait in join
-   */
-  public void stop(long timeoutMs) {
-    LOG.info("Shutting down KafkaConsumerProxy poll thread:" + consumerPollThread.getName());
-
-    isRunning = false;
-    try {
-      consumerPollThread.join(timeoutMs);
-      // join returns event if the thread didn't finish
-      // in this case we should interrupt it and wait again
-      if (consumerPollThread.isAlive()) {
-        consumerPollThread.interrupt();
-        consumerPollThread.join(timeoutMs);
-      }
-    } catch (InterruptedException e) {
-      LOG.warn("Join in KafkaConsumerProxy has failed", e);
-      consumerPollThread.interrupt();
-    }
-  }
 }
 

http://git-wip-us.apache.org/repos/asf/samza/blob/5397a34e/samza-kafka/src/main/scala/org/apache/samza/system/kafka/KafkaSystemConsumer.java
----------------------------------------------------------------------
diff --git a/samza-kafka/src/main/scala/org/apache/samza/system/kafka/KafkaSystemConsumer.java b/samza-kafka/src/main/scala/org/apache/samza/system/kafka/KafkaSystemConsumer.java
index 9101a89..e5ded8d 100644
--- a/samza-kafka/src/main/scala/org/apache/samza/system/kafka/KafkaSystemConsumer.java
+++ b/samza-kafka/src/main/scala/org/apache/samza/system/kafka/KafkaSystemConsumer.java
@@ -31,9 +31,8 @@ import java.util.concurrent.atomic.AtomicBoolean;
 import kafka.common.TopicAndPartition;
 import org.apache.kafka.clients.consumer.Consumer;
 import org.apache.kafka.clients.consumer.KafkaConsumer;
-import org.apache.kafka.clients.consumer.KafkaConsumerConfig;
+import org.apache.samza.config.KafkaConsumerConfig;
 import org.apache.kafka.common.TopicPartition;
-import org.apache.samza.Partition;
 import org.apache.samza.SamzaException;
 import org.apache.samza.config.Config;
 import org.apache.samza.config.KafkaConfig;
@@ -56,32 +55,33 @@ public class KafkaSystemConsumer<K, V> extends BlockingEnvelopeMap implements Sy
 
   private final Consumer<K, V> kafkaConsumer;
   private final String systemName;
-  private final KafkaSystemConsumerMetrics samzaConsumerMetrics;
   private final String clientId;
-  private final String metricName;
   private final AtomicBoolean stopped = new AtomicBoolean(false);
   private final AtomicBoolean started = new AtomicBoolean(false);
   private final Config config;
   private final boolean fetchThresholdBytesEnabled;
+  private final KafkaSystemConsumerMetrics metrics;
 
   // This sink is used to transfer the messages from the proxy/consumer to the BlockingEnvelopeMap.
-  /* package private */final KafkaConsumerMessageSink messageSink;
+  final KafkaConsumerMessageSink messageSink;
 
-  // proxy is doing the actual reading
+  // This proxy contains a separate thread, which reads kafka messages (with consumer.poll()) and populates
+  // BlockeingEnvelopMap's buffers.
   final private KafkaConsumerProxy proxy;
 
-  /* package private */final Map<TopicPartition, String> topicPartitions2Offset = new HashMap<>();
-  /* package private */final Map<TopicPartition, SystemStreamPartition> topicPartitions2SSP = new HashMap<>();
 
-  /* package private */ long perPartitionFetchThreshold;
-  /* package private */ long perPartitionFetchThresholdBytes;
+  // keep registration data until the start - mapping between registered SSPs and topicPartitions, and the offsets
+  final Map<TopicPartition, String> topicPartitionsToOffset = new HashMap<>();
+  final Map<TopicPartition, SystemStreamPartition> topicPartitionsToSSP = new HashMap<>();
+
+  long perPartitionFetchThreshold;
+  long perPartitionFetchThresholdBytes;
 
   /**
-   * Constructor
    * @param systemName system name for which we create the consumer
-   * @param config config
-   * @param metrics metrics
-   * @param clock - system clock
+   * @param config config passed into the the app
+   * @param metrics metrics collecting object
+   * @param clock - system clock, allows to override internal clock (System.currentTimeMillis())
    */
   public KafkaSystemConsumer(Consumer<K, V> kafkaConsumer, String systemName, Config config, String clientId,
       KafkaSystemConsumerMetrics metrics, Clock clock) {
@@ -89,54 +89,50 @@ public class KafkaSystemConsumer<K, V> extends BlockingEnvelopeMap implements Sy
     super(metrics.registry(), clock, metrics.getClass().getName());
 
     this.kafkaConsumer = kafkaConsumer;
-    this.samzaConsumerMetrics = metrics;
     this.clientId = clientId;
     this.systemName = systemName;
     this.config = config;
-    this.metricName = String.format("%s %s", systemName, clientId);
+    this.metrics = metrics;
 
-    this.fetchThresholdBytesEnabled = new KafkaConfig(config).isConsumerFetchThresholdBytesEnabled(systemName);
+    fetchThresholdBytesEnabled = new KafkaConfig(config).isConsumerFetchThresholdBytesEnabled(systemName);
 
     // create a sink for passing the messages between the proxy and the consumer
     messageSink = new KafkaConsumerMessageSink();
 
-    // Create the proxy to do the actual message reading. It is a separate thread that reads the messages from the stream
-    // and puts them into the sink.
-    proxy = new KafkaConsumerProxy(kafkaConsumer, systemName, clientId, messageSink, samzaConsumerMetrics, metricName);
-    LOG.info("Created consumer proxy: " + proxy);
+    // Create the proxy to do the actual message reading.
+    String metricName = String.format("%s %s", systemName, clientId);
+    proxy = new KafkaConsumerProxy(kafkaConsumer, systemName, clientId, messageSink, metrics, metricName);
+    LOG.info("{}: Created KafkaConsumerProxy {} ", this, proxy );
 
-    LOG.info("Created SamzaKafkaSystemConsumer for system={}, clientId={}, metricName={}, KafkaConsumer={}", systemName,
-        clientId, metricName, this.kafkaConsumer.toString());
+    LOG.info("{}: Created KafkaSystemConsumer {}", this, kafkaConsumer);
   }
 
   /**
-   * create kafka consumer
+   * Create internal kafka consumer object, which will be used in the Proxy.
    * @param systemName system name for which we create the consumer
    * @param clientId client id to use int the kafka client
    * @param config config
-   * @return kafka consumer
+   * @return kafka consumer object
    */
   public static KafkaConsumer<byte[], byte[]> getKafkaConsumerImpl(String systemName, String clientId, Config config) {
 
-    Map<String, String> injectProps = new HashMap<>();
-
     // extract kafka client configs
     KafkaConsumerConfig consumerConfig =
-        KafkaConsumerConfig.getKafkaSystemConsumerConfig(config, systemName, clientId, injectProps);
+        KafkaConsumerConfig.getKafkaSystemConsumerConfig(config, systemName, clientId);
 
-    LOG.info("KafkaClient properties for systemName {}: {}", systemName, consumerConfig.originals());
+    LOG.info("{}:{} KafkaClient properties {}", systemName, clientId, consumerConfig);
 
-    return new KafkaConsumer<>(consumerConfig.originals());
+    return new KafkaConsumer(consumerConfig);
   }
 
   @Override
   public void start() {
     if (!started.compareAndSet(false, true)) {
-      LOG.warn("attempting to start the consumer for the second (or more) time.");
+      LOG.warn("{}: Attempting to start the consumer for the second (or more) time.", this);
       return;
     }
     if (stopped.get()) {
-      LOG.warn("attempting to start a stopped consumer");
+      LOG.warn("{}: Attempting to start a stopped consumer", this);
       return;
     }
     // initialize the subscriptions for all the registered TopicPartitions
@@ -145,58 +141,59 @@ public class KafkaSystemConsumer<K, V> extends BlockingEnvelopeMap implements Sy
     setFetchThresholds();
 
     startConsumer();
-    LOG.info("consumer {} started", this);
+    LOG.info("{}: Consumer started", this);
   }
 
   private void startSubscription() {
     //subscribe to all the registered TopicPartitions
-    LOG.info("consumer {}, subscribes to {} ", this, topicPartitions2SSP.keySet());
+    LOG.info("{}: Consumer subscribes to {}", this, topicPartitionsToSSP.keySet());
     try {
       synchronized (kafkaConsumer) {
         // we are using assign (and not subscribe), so we need to specify both topic and partition
-        kafkaConsumer.assign(topicPartitions2SSP.keySet());
+        kafkaConsumer.assign(topicPartitionsToSSP.keySet());
       }
     } catch (Exception e) {
-      LOG.warn("startSubscription failed.", e);
+      LOG.warn("{}: Start subscription failed", this);
       throw new SamzaException(e);
     }
   }
 
-  /*
-   Set the offsets to start from.
-   Add the TopicPartitions to the proxy.
-   Start the proxy thread.
+  /**
+   * Set the offsets to start from.
+   * Register the TopicPartitions with the proxy.
+   * Start the proxy.
    */
   void startConsumer() {
-    //set the offset for each TopicPartition
-    if (topicPartitions2Offset.size() <= 0) {
-      LOG.warn("Consumer {} is not subscribed to any SSPs", this);
+    // set the offset for each TopicPartition
+    if (topicPartitionsToOffset.size() <= 0) {
+      LOG.warn("{}: Consumer is not subscribed to any SSPs", this);
     }
 
-    topicPartitions2Offset.forEach((tp, startingOffsetString) -> {
+    topicPartitionsToOffset.forEach((tp, startingOffsetString) -> {
       long startingOffset = Long.valueOf(startingOffsetString);
 
       try {
         synchronized (kafkaConsumer) {
-          // TODO in the future we may need to add special handling here for BEGIN/END_OFFSET
-          // this will call KafkaConsumer.seekToBegin/End()
           kafkaConsumer.seek(tp, startingOffset); // this value should already be the 'upcoming' value
         }
       } catch (Exception e) {
-        // all other exceptions - non recoverable
-        LOG.error("Got Exception while seeking to " + startingOffsetString + " for " + tp, e);
-        throw new SamzaException(e);
+        // all recoverable execptions are handled by the client.
+        // if we get here there is nothing left to do but bail out.
+        String msg = String.format("%s: Got Exception while seeking to %s for partition %s",
+            this, startingOffsetString, tp);
+        LOG.error(msg, e);
+        throw new SamzaException(msg, e);
       }
 
-      LOG.info("Changing consumer's starting offset for tp = " + tp + " to " + startingOffsetString);
+      LOG.info("{}: Changing consumer's starting offset for tp = %s to %s", this, tp, startingOffsetString);
 
       // add the partition to the proxy
-      proxy.addTopicPartition(topicPartitions2SSP.get(tp), startingOffset);
+      proxy.addTopicPartition(topicPartitionsToSSP.get(tp), startingOffset);
     });
 
     // start the proxy thread
     if (proxy != null && !proxy.isRunning()) {
-      LOG.info("Starting proxy: " + proxy);
+      LOG.info("{}: Starting proxy {}", this, proxy);
       proxy.start();
     }
   }
@@ -209,57 +206,59 @@ public class KafkaSystemConsumer<K, V> extends BlockingEnvelopeMap implements Sy
     long fetchThreshold = FETCH_THRESHOLD;
     if (fetchThresholdOption.isDefined()) {
       fetchThreshold = Long.valueOf(fetchThresholdOption.get());
-      LOG.info("fetchThresholdOption is configured. fetchThreshold=" + fetchThreshold);
+      LOG.info("{}: fetchThresholdOption is configured. fetchThreshold={}", this, fetchThreshold);
     }
 
     Option<String> fetchThresholdBytesOption = kafkaConfig.getConsumerFetchThresholdBytes(systemName);
     long fetchThresholdBytes = FETCH_THRESHOLD_BYTES;
     if (fetchThresholdBytesOption.isDefined()) {
       fetchThresholdBytes = Long.valueOf(fetchThresholdBytesOption.get());
-      LOG.info("fetchThresholdBytesOption is configured. fetchThresholdBytes=" + fetchThresholdBytes);
+      LOG.info("{}: fetchThresholdBytesOption is configured. fetchThresholdBytes={}", this, fetchThresholdBytes);
     }
 
-    int numTPs = topicPartitions2SSP.size();
-    assert (numTPs == topicPartitions2Offset.size());
+    int numTPs = topicPartitionsToSSP.size();
+    if (numTPs == topicPartitionsToOffset.size()) {
+      throw new SamzaException("topicPartitionsToSSP.size() doesn't match topicPartitionsToOffset.size()");
+    }
 
-    LOG.info("fetchThresholdBytes = " + fetchThresholdBytes + "; fetchThreshold=" + fetchThreshold);
-    LOG.info("number of topicPartitions " + numTPs);
+    LOG.info("{}: fetchThresholdBytes = {}; fetchThreshold={}; partitions num={}",
+        this, fetchThresholdBytes, fetchThreshold, numTPs);
 
     if (numTPs > 0) {
       perPartitionFetchThreshold = fetchThreshold / numTPs;
-      LOG.info("perPartitionFetchThreshold=" + perPartitionFetchThreshold);
+      LOG.info("{}: perPartitionFetchThreshold={}", this, perPartitionFetchThreshold);
       if (fetchThresholdBytesEnabled) {
         // currently this feature cannot be enabled, because we do not have the size of the messages available.
         // messages get double buffered, hence divide by 2
         perPartitionFetchThresholdBytes = (fetchThresholdBytes / 2) / numTPs;
-        LOG.info("perPartitionFetchThresholdBytes is enabled. perPartitionFetchThresholdBytes="
-            + perPartitionFetchThresholdBytes);
+        LOG.info("{} :perPartitionFetchThresholdBytes is enabled. perPartitionFetchThresholdBytes={}",
+            this, perPartitionFetchThresholdBytes);
       }
     }
   }
 
   @Override
   public void stop() {
-    LOG.info("Stopping Samza kafkaConsumer " + this);
-
     if (!stopped.compareAndSet(false, true)) {
-      LOG.warn("attempting to stop stopped consumer.");
+      LOG.warn("{}: Attempting to stop stopped consumer.", this);
       return;
     }
 
-    // stop the proxy (with 5 minutes timeout)
+    LOG.info("{}: Stopping Samza kafkaConsumer ", this);
+
+    // stop the proxy (with 1 minute timeout)
     if (proxy != null) {
-      LOG.info("Stopping proxy " + proxy);
+      LOG.info("{}: Stopping proxy {}", this, proxy);
       proxy.stop(TimeUnit.SECONDS.toMillis(60));
     }
 
     try {
       synchronized (kafkaConsumer) {
-        LOG.info("Closing kafka consumer " + kafkaConsumer);
+        LOG.info("{}: Closing kafkaSystemConsumer {}", this, kafkaConsumer);
         kafkaConsumer.close();
       }
     } catch (Exception e) {
-      LOG.warn("failed to stop SamzaRawKafkaConsumer + " + this, e);
+      LOG.warn("{}: Failed to stop KafkaSystemConsumer.", this, e);
     }
   }
 
@@ -270,45 +269,45 @@ public class KafkaSystemConsumer<K, V> extends BlockingEnvelopeMap implements Sy
   public void register(SystemStreamPartition systemStreamPartition, String offset) {
     if (started.get()) {
       String msg =
-          String.format("Trying to register partition after consumer has been started. sn=%s, ssp=%s", systemName,
-              systemStreamPartition);
-      LOG.error(msg);
+          String.format("%s: Trying to register partition after consumer has been started. ssp=%s",
+              this, systemStreamPartition);
       throw new SamzaException(msg);
     }
 
     if (!systemStreamPartition.getSystem().equals(systemName)) {
-      LOG.warn("ignoring SSP " + systemStreamPartition + ", because this consumer's system is " + systemName);
+      LOG.warn("{}: ignoring SSP {}, because this consumer's system doesn't match.", this, systemStreamPartition);
       return;
     }
+    LOG.info("{}: Registering ssp = {} with offset {}", this, systemStreamPartition, offset);
+
     super.register(systemStreamPartition, offset);
 
     TopicPartition tp = toTopicPartition(systemStreamPartition);
 
-    topicPartitions2SSP.put(tp, systemStreamPartition);
+    topicPartitionsToSSP.put(tp, systemStreamPartition);
 
-    LOG.info("Registering ssp = " + systemStreamPartition + " with offset " + offset);
 
-    String existingOffset = topicPartitions2Offset.get(tp);
+    String existingOffset = topicPartitionsToOffset.get(tp);
     // register the older (of the two) offset in the consumer, to guarantee we do not miss any messages.
     if (existingOffset == null || compareOffsets(existingOffset, offset) > 0) {
-      topicPartitions2Offset.put(tp, offset);
+      topicPartitionsToOffset.put(tp, offset);
     }
 
-    samzaConsumerMetrics.registerTopicAndPartition(toTopicAndPartition(tp));
+    metrics.registerTopicAndPartition(toTopicAndPartition(tp));
   }
 
   /**
    * Compare two String offsets.
-   * Note. There is a method in KafkaAdmin that does that, but that would require instantiation of systemadmin for each consumer.
+   * Note. There is a method in KafkaSystemAdmin that does that, but that would require instantiation of systemadmin for each consumer.
    * @return see {@link Long#compareTo(Long)}
    */
-  public static int compareOffsets(String offset1, String offset2) {
+  private static int compareOffsets(String offset1, String offset2) {
     return Long.valueOf(offset1).compareTo(Long.valueOf(offset2));
   }
 
   @Override
   public String toString() {
-    return systemName + "/" + clientId + "/" + super.toString();
+    return String.format("%s:%s", systemName, clientId);
   }
 
   @Override
@@ -318,17 +317,11 @@ public class KafkaSystemConsumer<K, V> extends BlockingEnvelopeMap implements Sy
     // check if the proxy is running
     if (!proxy.isRunning()) {
       stop();
-      if (proxy.getFailureCause() != null) {
-        String message = "KafkaConsumerProxy has stopped";
-        throw new SamzaException(message, proxy.getFailureCause());
-      } else {
-        LOG.warn("Failure cause is not populated for KafkaConsumerProxy");
-        throw new SamzaException("KafkaConsumerProxy has stopped");
-      }
+      String message = String.format("%s: KafkaConsumerProxy has stopped.", this);
+      throw new SamzaException(message, proxy.getFailureCause());
     }
 
-    Map<SystemStreamPartition, List<IncomingMessageEnvelope>> res = super.poll(systemStreamPartitions, timeout);
-    return res;
+    return super.poll(systemStreamPartitions, timeout);
   }
 
   /**
@@ -353,9 +346,6 @@ public class KafkaSystemConsumer<K, V> extends BlockingEnvelopeMap implements Sy
     return systemName;
   }
 
-  ////////////////////////////////////
-  // inner class for the message sink
-  ////////////////////////////////////
   public class KafkaConsumerMessageSink {
 
     public void setIsAtHighWatermark(SystemStreamPartition ssp, boolean isAtHighWatermark) {
@@ -363,8 +353,8 @@ public class KafkaSystemConsumer<K, V> extends BlockingEnvelopeMap implements Sy
     }
 
     boolean needsMoreMessages(SystemStreamPartition ssp) {
-        LOG.debug("needsMoreMessages from following SSP: {}. fetchLimitByBytes enabled={}; messagesSizeInQueue={};"
-                + "(limit={}); messagesNumInQueue={}(limit={};", ssp, fetchThresholdBytesEnabled,
+        LOG.debug("{}: needsMoreMessages from following SSP: {}. fetchLimitByBytes enabled={}; messagesSizeInQueue={};"
+                + "(limit={}); messagesNumInQueue={}(limit={};", this, ssp, fetchThresholdBytesEnabled,
             getMessagesSizeInQueue(ssp), perPartitionFetchThresholdBytes, getNumMessagesInQueue(ssp),
             perPartitionFetchThreshold);
 
@@ -376,16 +366,15 @@ public class KafkaSystemConsumer<K, V> extends BlockingEnvelopeMap implements Sy
     }
 
     void addMessage(SystemStreamPartition ssp, IncomingMessageEnvelope envelope) {
-      LOG.trace("Incoming message ssp = {}: envelope = {}.", ssp, envelope);
+      LOG.trace("{}: Incoming message ssp = {}: envelope = {}.", this, ssp, envelope);
 
       try {
         put(ssp, envelope);
       } catch (InterruptedException e) {
         throw new SamzaException(
-            String.format("Interrupted while trying to add message with offset %s for ssp %s", envelope.getOffset(),
-                ssp));
+            String.format("%s: Consumer was interrupted while trying to add message with offset %s for ssp %s",
+                this, envelope.getOffset(), ssp));
       }
     }
-  }  // end of KafkaMessageSink class
-  ///////////////////////////////////////////////////////////////////////////
+  }
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/5397a34e/samza-kafka/src/main/scala/org/apache/samza/system/kafka/KafkaSystemConsumerMetrics.scala
----------------------------------------------------------------------
diff --git a/samza-kafka/src/main/scala/org/apache/samza/system/kafka/KafkaSystemConsumerMetrics.scala b/samza-kafka/src/main/scala/org/apache/samza/system/kafka/KafkaSystemConsumerMetrics.scala
index 7dce261..c4552e6 100644
--- a/samza-kafka/src/main/scala/org/apache/samza/system/kafka/KafkaSystemConsumerMetrics.scala
+++ b/samza-kafka/src/main/scala/org/apache/samza/system/kafka/KafkaSystemConsumerMetrics.scala
@@ -50,7 +50,7 @@ class KafkaSystemConsumerMetrics(val systemName: String = "unknown", val registr
     clientBytesRead.put(clientName, newCounter("%s-bytes-read" format clientName))
     clientReads.put((clientName), newCounter("%s-messages-read" format clientName))
     clientSkippedFetchRequests.put((clientName), newCounter("%s-skipped-fetch-requests" format clientName))
-    topicPartitions.put(clientName, newGauge("%s-topic-partitions" format clientName, 0))
+    topicPartitions.put(clientName, newGauge("%s-registered-topic-partitions" format clientName, 0))
   }
 
   // java friendlier interfaces

http://git-wip-us.apache.org/repos/asf/samza/blob/5397a34e/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
index 5342b08..deaee56 100644
--- 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
@@ -22,7 +22,7 @@ package org.apache.samza.system.kafka
 import java.util.Properties
 
 import kafka.utils.ZkUtils
-import org.apache.kafka.clients.consumer.{KafkaConsumer, KafkaConsumerConfig}
+import org.apache.kafka.clients.consumer.KafkaConsumer
 import org.apache.kafka.clients.producer.KafkaProducer
 import org.apache.samza.SamzaException
 import org.apache.samza.config.ApplicationConfig.ApplicationMode
@@ -30,7 +30,7 @@ import org.apache.samza.config.KafkaConfig.Config2Kafka
 import org.apache.samza.config.StorageConfig._
 import org.apache.samza.config.SystemConfig.Config2System
 import org.apache.samza.config.TaskConfig.Config2Task
-import org.apache.samza.config.{ApplicationConfig, Config, KafkaConfig, StreamConfig}
+import org.apache.samza.config._
 import org.apache.samza.metrics.MetricsRegistry
 import org.apache.samza.system.{SystemAdmin, SystemConsumer, SystemFactory, SystemProducer}
 import org.apache.samza.util._

http://git-wip-us.apache.org/repos/asf/samza/blob/5397a34e/samza-kafka/src/test/java/org/apache/kafka/clients/consumer/TestKafkaConsumerConfig.java
----------------------------------------------------------------------
diff --git a/samza-kafka/src/test/java/org/apache/kafka/clients/consumer/TestKafkaConsumerConfig.java b/samza-kafka/src/test/java/org/apache/kafka/clients/consumer/TestKafkaConsumerConfig.java
deleted file mode 100644
index 264098b..0000000
--- a/samza-kafka/src/test/java/org/apache/kafka/clients/consumer/TestKafkaConsumerConfig.java
+++ /dev/null
@@ -1,137 +0,0 @@
-/*
- * 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.kafka.clients.consumer;
-
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Map;
-import org.apache.kafka.common.serialization.ByteArrayDeserializer;
-import org.apache.samza.SamzaException;
-import org.apache.samza.config.Config;
-import org.apache.samza.config.MapConfig;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-
-
-public class TestKafkaConsumerConfig {
-  private final Map<String, String> props = new HashMap<>();
-  public final static String SYSTEM_NAME = "testSystem";
-  public final static String KAFKA_PRODUCER_PROPERTY_PREFIX = "systems." + SYSTEM_NAME + ".producer.";
-  public final static String KAFKA_CONSUMER_PROPERTY_PREFIX = "systems." + SYSTEM_NAME + ".consumer.";
-  private final static String CLIENT_ID = "clientId";
-
-  @Before
-  public void setProps() {
-
-  }
-
-  @Test
-  public void testDefaultsAndOverrides() {
-
-    Map<String, String> overrides = new HashMap<>();
-    overrides.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "true"); // should be ignored
-    overrides.put(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG, "Ignore"); // should be ignored
-    overrides.put(ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG, "100"); // should NOT be ignored
-
-    // if KAFKA_CONSUMER_PROPERTY_PREFIX is set, then PRODUCER should be ignored
-    props.put(KAFKA_PRODUCER_PROPERTY_PREFIX + "bootstrap.servers", "ignroeThis:9092");
-    props.put(KAFKA_CONSUMER_PROPERTY_PREFIX + "bootstrap.servers", "useThis:9092");
-
-    // should be overridden
-    props.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "true"); //ignore
-    props.put(ConsumerConfig.MAX_POLL_RECORDS_CONFIG, "1000"); // ignore
-
-
-    // should be overridden
-    props.put(ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG, "200");
-
-    Config config = new MapConfig(props);
-    KafkaConsumerConfig kafkaConsumerConfig = KafkaConsumerConfig.getKafkaSystemConsumerConfig(
-        config, SYSTEM_NAME, CLIENT_ID, overrides);
-
-    Assert.assertEquals(kafkaConsumerConfig.getBoolean(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG), false);
-
-    Assert.assertEquals(
-        kafkaConsumerConfig.getInt(ConsumerConfig.MAX_POLL_RECORDS_CONFIG),
-        Integer.valueOf(KafkaConsumerConfig.DEFAULT_KAFKA_CONSUMER_MAX_POLL_RECORDS));
-
-    Assert.assertEquals(
-        kafkaConsumerConfig.getList(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG).get(0),
-        RangeAssignor.class.getName());
-
-    Assert.assertEquals(
-        kafkaConsumerConfig.getList(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG).get(0),
-        "useThis:9092");
-    Assert.assertEquals(
-        kafkaConsumerConfig.getInt(ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG).longValue(),
-        100);
-
-    Assert.assertEquals(
-        kafkaConsumerConfig.getClass(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG),
-        ByteArrayDeserializer.class);
-
-    Assert.assertEquals(
-        kafkaConsumerConfig.getClass(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG),
-        ByteArrayDeserializer.class);
-
-    Assert.assertEquals(
-        kafkaConsumerConfig.getString(ConsumerConfig.CLIENT_ID_CONFIG),
-        CLIENT_ID);
-
-    Assert.assertEquals(
-        kafkaConsumerConfig.getString(ConsumerConfig.GROUP_ID_CONFIG),
-        KafkaConsumerConfig.getConsumerGroupId(config));
-  }
-
-  @Test
-  // test stuff that should not be overridden
-  public void testNotOverride() {
-
-    // if KAFKA_CONSUMER_PROPERTY_PREFIX is not set, then PRODUCER should be used
-    props.put(KAFKA_PRODUCER_PROPERTY_PREFIX + ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "useThis:9092");
-    props.put(KAFKA_CONSUMER_PROPERTY_PREFIX + ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, TestKafkaConsumerConfig.class.getName());
-    props.put(KAFKA_CONSUMER_PROPERTY_PREFIX + ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, TestKafkaConsumerConfig.class.getName());
-
-
-    Config config = new MapConfig(props);
-    KafkaConsumerConfig kafkaConsumerConfig = KafkaConsumerConfig.getKafkaSystemConsumerConfig(
-        config, SYSTEM_NAME, CLIENT_ID, Collections.emptyMap());
-
-    Assert.assertEquals(
-        kafkaConsumerConfig.getList(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG).get(0),
-        "useThis:9092");
-
-    Assert.assertEquals(
-        kafkaConsumerConfig.getClass(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG),
-        TestKafkaConsumerConfig.class);
-
-    Assert.assertEquals(
-        kafkaConsumerConfig.getClass(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG),
-        TestKafkaConsumerConfig.class);
-  }
-
-
-
-  @Test(expected = SamzaException.class)
-  public void testNoBootstrapServers() {
-    KafkaConsumerConfig kafkaConsumerConfig = KafkaConsumerConfig.getKafkaSystemConsumerConfig(
-        new MapConfig(Collections.emptyMap()), SYSTEM_NAME, "clientId", Collections.emptyMap());
-
-    Assert.fail("didn't get exception for the missing config:" + ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG);
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/5397a34e/samza-kafka/src/test/java/org/apache/samza/config/TestKafkaConsumerConfig.java
----------------------------------------------------------------------
diff --git a/samza-kafka/src/test/java/org/apache/samza/config/TestKafkaConsumerConfig.java b/samza-kafka/src/test/java/org/apache/samza/config/TestKafkaConsumerConfig.java
new file mode 100644
index 0000000..719ea22
--- /dev/null
+++ b/samza-kafka/src/test/java/org/apache/samza/config/TestKafkaConsumerConfig.java
@@ -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.config;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.clients.consumer.RangeAssignor;
+import org.apache.kafka.common.serialization.ByteArrayDeserializer;
+import org.apache.samza.SamzaException;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+
+public class TestKafkaConsumerConfig {
+  private final Map<String, String> props = new HashMap<>();
+  public final static String SYSTEM_NAME = "testSystem";
+  public final static String KAFKA_PRODUCER_PROPERTY_PREFIX = "systems." + SYSTEM_NAME + ".producer.";
+  public final static String KAFKA_CONSUMER_PROPERTY_PREFIX = "systems." + SYSTEM_NAME + ".consumer.";
+  private final static String CLIENT_ID = "clientId";
+
+  @Before
+  public void setProps() {
+
+  }
+
+  @Test
+  public void testDefaults() {
+
+    props.put(KAFKA_CONSUMER_PROPERTY_PREFIX + ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "true"); // should be ignored
+    props.put(KAFKA_CONSUMER_PROPERTY_PREFIX + ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG, "Ignore"); // should be ignored
+    props.put(KAFKA_CONSUMER_PROPERTY_PREFIX + ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG, "100"); // should NOT be ignored
+
+    // if KAFKA_CONSUMER_PROPERTY_PREFIX is set, then PRODUCER should be ignored
+    props.put(KAFKA_PRODUCER_PROPERTY_PREFIX + "bootstrap.servers", "ignroeThis:9092");
+    props.put(KAFKA_CONSUMER_PROPERTY_PREFIX + "bootstrap.servers", "useThis:9092");
+
+    Config config = new MapConfig(props);
+    KafkaConsumerConfig kafkaConsumerConfig = KafkaConsumerConfig.getKafkaSystemConsumerConfig(
+        config, SYSTEM_NAME, CLIENT_ID);
+
+    Assert.assertEquals("false", kafkaConsumerConfig.get(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG));
+
+    Assert.assertEquals(
+        KafkaConsumerConfig.DEFAULT_KAFKA_CONSUMER_MAX_POLL_RECORDS,
+        kafkaConsumerConfig.get(ConsumerConfig.MAX_POLL_RECORDS_CONFIG));
+
+    Assert.assertEquals(
+        RangeAssignor.class.getName(),
+        kafkaConsumerConfig.get(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG));
+
+    Assert.assertEquals(
+        "useThis:9092",
+        kafkaConsumerConfig.get(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG));
+    Assert.assertEquals(
+        "100",
+        kafkaConsumerConfig.get(ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG));
+
+    Assert.assertEquals(
+        ByteArrayDeserializer.class.getName(),
+        kafkaConsumerConfig.get(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG));
+
+    Assert.assertEquals(
+        ByteArrayDeserializer.class.getName(),
+        kafkaConsumerConfig.get(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG) );
+
+    Assert.assertEquals(
+        CLIENT_ID,
+        kafkaConsumerConfig.get(ConsumerConfig.CLIENT_ID_CONFIG));
+
+    Assert.assertEquals(
+        KafkaConsumerConfig.getConsumerGroupId(config),
+        kafkaConsumerConfig.get(ConsumerConfig.GROUP_ID_CONFIG));
+  }
+
+  @Test
+  // test stuff that should not be overridden
+  public void testNotOverride() {
+
+    // if KAFKA_CONSUMER_PROPERTY_PREFIX is not set, then PRODUCER should be used
+    props.put(KAFKA_PRODUCER_PROPERTY_PREFIX + ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "useThis:9092");
+    props.put(KAFKA_CONSUMER_PROPERTY_PREFIX + ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, TestKafkaConsumerConfig.class.getName());
+    props.put(KAFKA_CONSUMER_PROPERTY_PREFIX + ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, TestKafkaConsumerConfig.class.getName());
+
+
+    Config config = new MapConfig(props);
+    KafkaConsumerConfig kafkaConsumerConfig = KafkaConsumerConfig.getKafkaSystemConsumerConfig(
+        config, SYSTEM_NAME, CLIENT_ID);
+
+    Assert.assertEquals(
+        "useThis:9092",
+        kafkaConsumerConfig.get(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG));
+
+    Assert.assertEquals(
+        TestKafkaConsumerConfig.class.getName(),
+        kafkaConsumerConfig.get(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG));
+
+    Assert.assertEquals(
+        TestKafkaConsumerConfig.class.getName(),
+        kafkaConsumerConfig.get(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG));
+  }
+
+  @Test
+  public void testGetConsumerClientId() {
+    Map<String, String> map = new HashMap<>();
+
+    map.put(JobConfig.JOB_NAME(), "jobName");
+    map.put(JobConfig.JOB_ID(), "jobId");
+    String result =  KafkaConsumerConfig.getConsumerClientId("consumer", new MapConfig(map));
+    Assert.assertEquals("consumer-jobName-jobId", result);
+
+    result =  KafkaConsumerConfig.getConsumerClientId("consumer-", new MapConfig(map));
+    Assert.assertEquals("consumer_-jobName-jobId", result);
+
+    result =  KafkaConsumerConfig.getConsumerClientId("super-duper-consumer", new MapConfig(map));
+    Assert.assertEquals("super_duper_consumer-jobName-jobId", result);
+
+    map.put(JobConfig.JOB_NAME(), " very important!job");
+    result =  KafkaConsumerConfig.getConsumerClientId("consumer", new MapConfig(map));
+    Assert.assertEquals("consumer-_very_important_job-jobId", result);
+
+    map.put(JobConfig.JOB_ID(), "number-#3");
+    result =  KafkaConsumerConfig.getConsumerClientId("consumer", new MapConfig(map));
+    Assert.assertEquals("consumer-_very_important_job-number__3", result);
+  }
+
+
+
+  @Test(expected = SamzaException.class)
+  public void testNoBootstrapServers() {
+    KafkaConsumerConfig kafkaConsumerConfig = KafkaConsumerConfig.getKafkaSystemConsumerConfig(
+        new MapConfig(Collections.emptyMap()), SYSTEM_NAME, "clientId");
+
+    Assert.fail("didn't get exception for the missing config:" + ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG);
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/5397a34e/samza-kafka/src/test/scala/org/apache/samza/system/kafka/TestKafkaSystemConsumer.java
----------------------------------------------------------------------
diff --git a/samza-kafka/src/test/scala/org/apache/samza/system/kafka/TestKafkaSystemConsumer.java b/samza-kafka/src/test/scala/org/apache/samza/system/kafka/TestKafkaSystemConsumer.java
index d90bc35..9e8ff44 100644
--- a/samza-kafka/src/test/scala/org/apache/samza/system/kafka/TestKafkaSystemConsumer.java
+++ b/samza-kafka/src/test/scala/org/apache/samza/system/kafka/TestKafkaSystemConsumer.java
@@ -27,7 +27,7 @@ import java.util.Map;
 import org.apache.kafka.clients.consumer.Consumer;
 import org.apache.kafka.clients.consumer.ConsumerConfig;
 import org.apache.kafka.clients.consumer.KafkaConsumer;
-import org.apache.kafka.clients.consumer.KafkaConsumerConfig;
+import org.apache.samza.config.KafkaConsumerConfig;
 import org.apache.kafka.common.serialization.ByteArraySerializer;
 import org.apache.samza.Partition;
 import org.apache.samza.config.Config;
@@ -67,8 +67,8 @@ public class TestKafkaSystemConsumer {
 
     Config config = new MapConfig(map);
     KafkaConsumerConfig consumerConfig =
-        KafkaConsumerConfig.getKafkaSystemConsumerConfig(config, TEST_SYSTEM, TEST_CLIENT_ID, Collections.emptyMap());
-    final KafkaConsumer<byte[], byte[]> kafkaConsumer = new MockKafkaConsumer(consumerConfig.originals());
+        KafkaConsumerConfig.getKafkaSystemConsumerConfig(config, TEST_SYSTEM, TEST_CLIENT_ID);
+    final KafkaConsumer<byte[], byte[]> kafkaConsumer = new MockKafkaConsumer(consumerConfig);
 
     MockKafkaSystmeCosumer newKafkaSystemConsumer =
         new MockKafkaSystmeCosumer(kafkaConsumer, TEST_SYSTEM, config, TEST_CLIENT_ID,
@@ -116,9 +116,9 @@ public class TestKafkaSystemConsumer {
     consumer.register(ssp1, "3");
     consumer.register(ssp2, "0");
 
-    assertEquals("0", consumer.topicPartitions2Offset.get(KafkaSystemConsumer.toTopicPartition(ssp0)));
-    assertEquals("2", consumer.topicPartitions2Offset.get(KafkaSystemConsumer.toTopicPartition(ssp1)));
-    assertEquals("0", consumer.topicPartitions2Offset.get(KafkaSystemConsumer.toTopicPartition(ssp2)));
+    assertEquals("0", consumer.topicPartitionsToOffset.get(KafkaSystemConsumer.toTopicPartition(ssp0)));
+    assertEquals("2", consumer.topicPartitionsToOffset.get(KafkaSystemConsumer.toTopicPartition(ssp1)));
+    assertEquals("0", consumer.topicPartitionsToOffset.get(KafkaSystemConsumer.toTopicPartition(ssp2)));
   }
 
   @Test


[26/29] samza git commit: Merge branch 'master' into NewConsumer2

Posted by bo...@apache.org.
Merge branch 'master' into NewConsumer2


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

Branch: refs/heads/NewKafkaSystemConsumer
Commit: 93ca950f72d774d625fa74d448a24184179e5e4d
Parents: 4ae563c ceebdc3
Author: Boris S <bo...@apache.org>
Authored: Mon Sep 24 17:08:09 2018 -0700
Committer: Boris S <bo...@apache.org>
Committed: Mon Sep 24 17:08:09 2018 -0700

----------------------------------------------------------------------
 .gitignore                                      |    3 +-
 build.gradle                                    |    2 +
 docs/_case-studies/TEMPLATE.md                  |   32 +
 docs/_case-studies/digitalsmiths.md             |   31 +
 docs/_case-studies/fortscale.md                 |   30 +
 docs/_case-studies/index.md                     |   49 +
 docs/_case-studies/intuit.md                    |   31 +
 docs/_case-studies/linkedin.md                  |   30 +
 docs/_case-studies/netflix.md                   |   30 +
 docs/_case-studies/optimizely.md                |   58 +
 docs/_case-studies/redfin.md                    |   30 +
 docs/_case-studies/state.md                     |   31 +
 docs/_case-studies/uber.md                      |   30 +
 docs/_committers/TEMPLATE.md                    |   30 +
 docs/_committers/angela-murrell.md              |   28 +
 docs/_committers/boris-shkolnik.md              |   28 +
 docs/_committers/chinmay-soman.md               |   28 +
 docs/_committers/chris-riccomini.md             |   28 +
 docs/_committers/garry-turkington.md            |   28 +
 docs/_committers/jagadish-venkatraman.md        |   28 +
 docs/_committers/jake-maes.md                   |   28 +
 docs/_committers/jakob-homan.md                 |   29 +
 docs/_committers/jay-kreps.md                   |   28 +
 docs/_committers/martin-kleppmann.md            |   28 +
 docs/_committers/navina-ramesh.md               |   28 +
 docs/_committers/prateek-maheshwari.md          |   28 +
 docs/_committers/sriram-subramanian.md          |   28 +
 docs/_committers/wei-song.md                    |   28 +
 docs/_committers/xinyu-liu.md                   |   28 +
 docs/_committers/yan-fang.md                    |   28 +
 docs/_committers/yi-pan.md                      |   28 +
 docs/_committers/zhijie-shen.md                 |   28 +
 docs/_config.yml                                |   17 +
 docs/_includes/footer.html                      |   92 +
 docs/_includes/main-navigation.html             |   39 +
 docs/_layouts/case-study.html                   |  150 ++
 docs/_layouts/default.html                      |  309 +--
 docs/_layouts/news.html                         |   81 +
 docs/_layouts/page.html                         |   96 +-
 docs/_layouts/talks-and-meetups.html            |   42 +
 docs/_meetups/dec-2018.md                       |   46 +
 docs/_meetups/july-2018.md                      |   56 +
 docs/_menu/index.html                           |  160 ++
 ...ncing-the-release-of-apache-samza--0.14.0.md |   76 +
 docs/_news/2018-04-28-something-else.md         |   36 +
 ...ncing-the-release-of-apache-samza--0.14.1.md |   83 +
 docs/_news/TEMPLATE.md                          |   38 +
 docs/_news/index.md                             |   54 +
 docs/_powered-by/TEMPLATE.md                    |   23 +
 docs/_powered-by/intuit.md                      |   22 +
 docs/_powered-by/linkedin.md                    |   22 +
 docs/_powered-by/mobileaware.md                 |   22 +
 docs/_releases/0.10.md                          |   21 +
 docs/_releases/0.11.md                          |   21 +
 docs/_releases/0.12.md                          |   21 +
 docs/_releases/0.13.md                          |   21 +
 docs/_releases/0.14.md                          |   21 +
 docs/_releases/0.7.0.md                         |   21 +
 docs/_releases/0.8.md                           |   21 +
 docs/_releases/0.9.md                           |   21 +
 docs/_releases/TEMPLATE.md                      |   21 +
 docs/_talks/TEMPLATE.md                         |   34 +
 docs/_talks/june-2018--bangalore-kafka-group.md |   30 +
 docs/_talks/november-2017-big-data-span-2017.md |   30 +
 .../november-2018--totally-awesome-summit.md    |   30 +
 ...ber-21-2017--dataworks-summit-sydney-2017.md |   33 +
 docs/community/committers-old.md                |  103 +
 docs/community/committers.html                  |   97 +
 docs/community/committers.md                    |  103 -
 docs/css/bootstrap.css.map                      |    1 -
 docs/css/bootstrap.min.css                      |    7 -
 docs/css/font-awesome.min.css                   |    4 -
 docs/css/google-fonts.css                       |   72 +
 docs/css/ionicons.min.css                       |   11 +
 docs/css/main.new.css                           | 2262 ++++++++++++++++++
 docs/css/ropa-sans.css                          |   25 -
 docs/fonts/ionicons.eot                         |  Bin 0 -> 112650 bytes
 docs/fonts/ionicons.svg                         |  713 ++++++
 docs/fonts/ionicons.ttf                         |  Bin 0 -> 112472 bytes
 docs/fonts/ionicons.woff                        |  Bin 0 -> 65912 bytes
 docs/fonts/ionicons.woff2                       |  Bin 0 -> 50632 bytes
 docs/img/favicon/apple-touch-icon-114x114.png   |  Bin 0 -> 14109 bytes
 docs/img/favicon/apple-touch-icon-120x120.png   |  Bin 0 -> 12879 bytes
 docs/img/favicon/apple-touch-icon-144x144.png   |  Bin 0 -> 18520 bytes
 docs/img/favicon/apple-touch-icon-152x152.png   |  Bin 0 -> 20884 bytes
 docs/img/favicon/apple-touch-icon-57x57.png     |  Bin 0 -> 5351 bytes
 docs/img/favicon/apple-touch-icon-60x60.png     |  Bin 0 -> 5270 bytes
 docs/img/favicon/apple-touch-icon-72x72.png     |  Bin 0 -> 7156 bytes
 docs/img/favicon/apple-touch-icon-76x76.png     |  Bin 0 -> 7707 bytes
 docs/img/favicon/favicon-128.png                |  Bin 0 -> 13302 bytes
 docs/img/favicon/favicon-16x16.png              |  Bin 0 -> 824 bytes
 docs/img/favicon/favicon-196x196.png            |  Bin 0 -> 30623 bytes
 docs/img/favicon/favicon-32x32.png              |  Bin 0 -> 2288 bytes
 docs/img/favicon/favicon-96x96.png              |  Bin 0 -> 10467 bytes
 docs/img/favicon/favicon.ico                    |  Bin 0 -> 34494 bytes
 docs/img/favicon/mstile-144x144.png             |  Bin 0 -> 18520 bytes
 docs/img/favicon/mstile-150x150.png             |  Bin 0 -> 43894 bytes
 docs/img/favicon/mstile-310x150.png             |  Bin 0 -> 102518 bytes
 docs/img/favicon/mstile-310x310.png             |  Bin 0 -> 192956 bytes
 docs/img/favicon/mstile-70x70.png               |  Bin 0 -> 13302 bytes
 docs/img/feather.gif                            |  Bin 0 -> 16647 bytes
 docs/img/samza-just-logo-transparent.png        |  Bin 0 -> 260503 bytes
 docs/img/samza-just-logo.svg                    |  128 +
 docs/img/samza-logo-no-text.png                 |  Bin 0 -> 93664 bytes
 docs/img/samza-logo.png                         |  Bin 0 -> 6412 bytes
 docs/img/samza-logo.svg                         |  174 ++
 docs/img/samza-logo@2x.png                      |  Bin 0 -> 16164 bytes
 docs/index.md                                   |   22 +-
 docs/index.md.bak                               |   39 +
 docs/js/bootstrap.min.js                        |    6 -
 docs/js/jquery-1.11.1.min.js                    |    4 -
 docs/js/jquery.tablesorter.min.js               |    4 -
 docs/js/main.new.js                             |  645 +++++
 docs/learn/tutorials/versioned/index.md         |    3 +-
 docs/meetups/index.html                         |  106 +
 docs/powered-by/index.html                      |   48 +
 docs/startup/preview/index.md                   |    2 +-
 .../startup/releases/versioned/release-notes.md |   17 +-
 docs/talks/index.html                           |   85 +
 .../apache/samza/operators/TableDescriptor.java |   13 +-
 .../samza/table/TableDescriptorsProvider.java   |    8 +-
 .../StreamApplicationDescriptorImpl.java        |    8 +
 .../samza/operators/BaseTableDescriptor.java    |   19 +-
 .../table/caching/CachingTableDescriptor.java   |   37 +-
 .../table/hybrid/BaseHybridTableDescriptor.java |   50 +
 .../table/remote/RemoteTableDescriptor.java     |   13 +-
 .../samza/table/caching/TestCachingTable.java   |   22 +-
 .../kv/inmemory/InMemoryTableDescriptor.java    |   13 +-
 .../inmemory/TestInMemoryTableDescriptor.java   |    4 +-
 .../storage/kv/RocksDbTableDescriptor.java      |   13 +-
 .../storage/kv/TestRocksDbTableDescriptor.java  |   11 +-
 .../kv/BaseLocalStoreBackedTableDescriptor.java |   13 +-
 .../kv/LocalStoreBackedReadWriteTable.java      |    1 +
 .../samza/sql/data/RexToJavaCompiler.java       |    5 +-
 .../samza/sql/dsl/SamzaSqlDslConverter.java     |   96 +
 .../sql/dsl/SamzaSqlDslConverterFactory.java    |   33 +
 .../sql/impl/ConfigBasedIOResolverFactory.java  |    7 +-
 .../samza/sql/interfaces/DslConverter.java      |   37 +
 .../sql/interfaces/DslConverterFactory.java     |   36 +
 .../samza/sql/interfaces/SamzaSqlDriver.java    |   56 +
 .../interfaces/SamzaSqlJavaTypeFactoryImpl.java |   72 +
 .../samza/sql/runner/SamzaSqlApplication.java   |   30 +-
 .../sql/runner/SamzaSqlApplicationConfig.java   |  117 +-
 .../sql/runner/SamzaSqlApplicationRunner.java   |   41 +-
 .../samza/sql/testutil/SamzaSqlQueryParser.java |   21 +-
 .../samza/sql/translator/JoinTranslator.java    |    1 +
 .../samza/sql/translator/ModifyTranslator.java  |  117 +
 .../samza/sql/translator/QueryTranslator.java   |   90 +-
 .../samza/sql/translator/ScanTranslator.java    |   10 +-
 .../apache/samza/sql/e2e/TestSamzaSqlTable.java |    4 +-
 .../runner/TestSamzaSqlApplicationConfig.java   |   49 +-
 .../runner/TestSamzaSqlApplicationRunner.java   |    2 +-
 .../samza/sql/system/TestAvroSystemFactory.java |    3 +-
 .../samza/sql/testutil/SamzaSqlTestConfig.java  |    3 +
 .../sql/testutil/TestIOResolverFactory.java     |    7 +-
 .../sql/testutil/TestSamzaSqlFileParser.java    |    1 +
 .../sql/translator/TestQueryTranslator.java     |  345 ++-
 .../test/samzasql/TestSamzaSqlEndToEnd.java     |   64 +-
 .../apache/samza/test/table/TestLocalTable.java |    9 +-
 .../table/TestLocalTableWithSideInputs.java     |    6 +-
 .../samza/test/table/TestRemoteTable.java       |   33 +-
 .../table/TestTableDescriptorsProvider.java     |   10 +-
 162 files changed, 8230 insertions(+), 713 deletions(-)
----------------------------------------------------------------------



[14/29] samza git commit: SAMZA-1898: New UI layout for the Samza website

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/samza/blob/b8058af0/docs/js/jquery-1.11.1.min.js
----------------------------------------------------------------------
diff --git a/docs/js/jquery-1.11.1.min.js b/docs/js/jquery-1.11.1.min.js
deleted file mode 100644
index ab28a24..0000000
--- a/docs/js/jquery-1.11.1.min.js
+++ /dev/null
@@ -1,4 +0,0 @@
-/*! jQuery v1.11.1 | (c) 2005, 2014 jQuery Foundation, Inc. | jquery.org/license */
-!function(a,b){"object"==typeof module&&"object"==typeof module.exports?module.exports=a.document?b(a,!0):function(a){if(!a.document)throw new Error("jQuery requires a window with a document");return b(a)}:b(a)}("undefined"!=typeof window?window:this,function(a,b){var c=[],d=c.slice,e=c.concat,f=c.push,g=c.indexOf,h={},i=h.toString,j=h.hasOwnProperty,k={},l="1.11.1",m=function(a,b){return new m.fn.init(a,b)},n=/^[\s\uFEFF\xA0]+|[\s\uFEFF\xA0]+$/g,o=/^-ms-/,p=/-([\da-z])/gi,q=function(a,b){return b.toUpperCase()};m.fn=m.prototype={jquery:l,constructor:m,selector:"",length:0,toArray:function(){return d.call(this)},get:function(a){return null!=a?0>a?this[a+this.length]:this[a]:d.call(this)},pushStack:function(a){var b=m.merge(this.constructor(),a);return b.prevObject=this,b.context=this.context,b},each:function(a,b){return m.each(this,a,b)},map:function(a){return this.pushStack(m.map(this,function(b,c){return a.call(b,c,b)}))},slice:function(){return this.pushStack(d.apply(this,argumen
 ts))},first:function(){return this.eq(0)},last:function(){return this.eq(-1)},eq:function(a){var b=this.length,c=+a+(0>a?b:0);return this.pushStack(c>=0&&b>c?[this[c]]:[])},end:function(){return this.prevObject||this.constructor(null)},push:f,sort:c.sort,splice:c.splice},m.extend=m.fn.extend=function(){var a,b,c,d,e,f,g=arguments[0]||{},h=1,i=arguments.length,j=!1;for("boolean"==typeof g&&(j=g,g=arguments[h]||{},h++),"object"==typeof g||m.isFunction(g)||(g={}),h===i&&(g=this,h--);i>h;h++)if(null!=(e=arguments[h]))for(d in e)a=g[d],c=e[d],g!==c&&(j&&c&&(m.isPlainObject(c)||(b=m.isArray(c)))?(b?(b=!1,f=a&&m.isArray(a)?a:[]):f=a&&m.isPlainObject(a)?a:{},g[d]=m.extend(j,f,c)):void 0!==c&&(g[d]=c));return g},m.extend({expando:"jQuery"+(l+Math.random()).replace(/\D/g,""),isReady:!0,error:function(a){throw new Error(a)},noop:function(){},isFunction:function(a){return"function"===m.type(a)},isArray:Array.isArray||function(a){return"array"===m.type(a)},isWindow:function(a){return null!=a&&a=
 =a.window},isNumeric:function(a){return!m.isArray(a)&&a-parseFloat(a)>=0},isEmptyObject:function(a){var b;for(b in a)return!1;return!0},isPlainObject:function(a){var b;if(!a||"object"!==m.type(a)||a.nodeType||m.isWindow(a))return!1;try{if(a.constructor&&!j.call(a,"constructor")&&!j.call(a.constructor.prototype,"isPrototypeOf"))return!1}catch(c){return!1}if(k.ownLast)for(b in a)return j.call(a,b);for(b in a);return void 0===b||j.call(a,b)},type:function(a){return null==a?a+"":"object"==typeof a||"function"==typeof a?h[i.call(a)]||"object":typeof a},globalEval:function(b){b&&m.trim(b)&&(a.execScript||function(b){a.eval.call(a,b)})(b)},camelCase:function(a){return a.replace(o,"ms-").replace(p,q)},nodeName:function(a,b){return a.nodeName&&a.nodeName.toLowerCase()===b.toLowerCase()},each:function(a,b,c){var d,e=0,f=a.length,g=r(a);if(c){if(g){for(;f>e;e++)if(d=b.apply(a[e],c),d===!1)break}else for(e in a)if(d=b.apply(a[e],c),d===!1)break}else if(g){for(;f>e;e++)if(d=b.call(a[e],e,a[e]),d
 ===!1)break}else for(e in a)if(d=b.call(a[e],e,a[e]),d===!1)break;return a},trim:function(a){return null==a?"":(a+"").replace(n,"")},makeArray:function(a,b){var c=b||[];return null!=a&&(r(Object(a))?m.merge(c,"string"==typeof a?[a]:a):f.call(c,a)),c},inArray:function(a,b,c){var d;if(b){if(g)return g.call(b,a,c);for(d=b.length,c=c?0>c?Math.max(0,d+c):c:0;d>c;c++)if(c in b&&b[c]===a)return c}return-1},merge:function(a,b){var c=+b.length,d=0,e=a.length;while(c>d)a[e++]=b[d++];if(c!==c)while(void 0!==b[d])a[e++]=b[d++];return a.length=e,a},grep:function(a,b,c){for(var d,e=[],f=0,g=a.length,h=!c;g>f;f++)d=!b(a[f],f),d!==h&&e.push(a[f]);return e},map:function(a,b,c){var d,f=0,g=a.length,h=r(a),i=[];if(h)for(;g>f;f++)d=b(a[f],f,c),null!=d&&i.push(d);else for(f in a)d=b(a[f],f,c),null!=d&&i.push(d);return e.apply([],i)},guid:1,proxy:function(a,b){var c,e,f;return"string"==typeof b&&(f=a[b],b=a,a=f),m.isFunction(a)?(c=d.call(arguments,2),e=function(){return a.apply(b||this,c.concat(d.call(ar
 guments)))},e.guid=a.guid=a.guid||m.guid++,e):void 0},now:function(){return+new Date},support:k}),m.each("Boolean Number String Function Array Date RegExp Object Error".split(" "),function(a,b){h["[object "+b+"]"]=b.toLowerCase()});function r(a){var b=a.length,c=m.type(a);return"function"===c||m.isWindow(a)?!1:1===a.nodeType&&b?!0:"array"===c||0===b||"number"==typeof b&&b>0&&b-1 in a}var s=function(a){var b,c,d,e,f,g,h,i,j,k,l,m,n,o,p,q,r,s,t,u="sizzle"+-new Date,v=a.document,w=0,x=0,y=gb(),z=gb(),A=gb(),B=function(a,b){return a===b&&(l=!0),0},C="undefined",D=1<<31,E={}.hasOwnProperty,F=[],G=F.pop,H=F.push,I=F.push,J=F.slice,K=F.indexOf||function(a){for(var b=0,c=this.length;c>b;b++)if(this[b]===a)return b;return-1},L="checked|selected|async|autofocus|autoplay|controls|defer|disabled|hidden|ismap|loop|multiple|open|readonly|required|scoped",M="[\\x20\\t\\r\\n\\f]",N="(?:\\\\.|[\\w-]|[^\\x00-\\xa0])+",O=N.replace("w","w#"),P="\\["+M+"*("+N+")(?:"+M+"*([*^$|!~]?=)"+M+"*(?:'((?:\\\\.|[
 ^\\\\'])*)'|\"((?:\\\\.|[^\\\\\"])*)\"|("+O+"))|)"+M+"*\\]",Q=":("+N+")(?:\\((('((?:\\\\.|[^\\\\'])*)'|\"((?:\\\\.|[^\\\\\"])*)\")|((?:\\\\.|[^\\\\()[\\]]|"+P+")*)|.*)\\)|)",R=new RegExp("^"+M+"+|((?:^|[^\\\\])(?:\\\\.)*)"+M+"+$","g"),S=new RegExp("^"+M+"*,"+M+"*"),T=new RegExp("^"+M+"*([>+~]|"+M+")"+M+"*"),U=new RegExp("="+M+"*([^\\]'\"]*?)"+M+"*\\]","g"),V=new RegExp(Q),W=new RegExp("^"+O+"$"),X={ID:new RegExp("^#("+N+")"),CLASS:new RegExp("^\\.("+N+")"),TAG:new RegExp("^("+N.replace("w","w*")+")"),ATTR:new RegExp("^"+P),PSEUDO:new RegExp("^"+Q),CHILD:new RegExp("^:(only|first|last|nth|nth-last)-(child|of-type)(?:\\("+M+"*(even|odd|(([+-]|)(\\d*)n|)"+M+"*(?:([+-]|)"+M+"*(\\d+)|))"+M+"*\\)|)","i"),bool:new RegExp("^(?:"+L+")$","i"),needsContext:new RegExp("^"+M+"*[>+~]|:(even|odd|eq|gt|lt|nth|first|last)(?:\\("+M+"*((?:-\\d)?\\d*)"+M+"*\\)|)(?=[^-]|$)","i")},Y=/^(?:input|select|textarea|button)$/i,Z=/^h\d$/i,$=/^[^{]+\{\s*\[native \w/,_=/^(?:#([\w-]+)|(\w+)|\.([\w-]+))$/,ab=/[+~]/,
 bb=/'|\\/g,cb=new RegExp("\\\\([\\da-f]{1,6}"+M+"?|("+M+")|.)","ig"),db=function(a,b,c){var d="0x"+b-65536;return d!==d||c?b:0>d?String.fromCharCode(d+65536):String.fromCharCode(d>>10|55296,1023&d|56320)};try{I.apply(F=J.call(v.childNodes),v.childNodes),F[v.childNodes.length].nodeType}catch(eb){I={apply:F.length?function(a,b){H.apply(a,J.call(b))}:function(a,b){var c=a.length,d=0;while(a[c++]=b[d++]);a.length=c-1}}}function fb(a,b,d,e){var f,h,j,k,l,o,r,s,w,x;if((b?b.ownerDocument||b:v)!==n&&m(b),b=b||n,d=d||[],!a||"string"!=typeof a)return d;if(1!==(k=b.nodeType)&&9!==k)return[];if(p&&!e){if(f=_.exec(a))if(j=f[1]){if(9===k){if(h=b.getElementById(j),!h||!h.parentNode)return d;if(h.id===j)return d.push(h),d}else if(b.ownerDocument&&(h=b.ownerDocument.getElementById(j))&&t(b,h)&&h.id===j)return d.push(h),d}else{if(f[2])return I.apply(d,b.getElementsByTagName(a)),d;if((j=f[3])&&c.getElementsByClassName&&b.getElementsByClassName)return I.apply(d,b.getElementsByClassName(j)),d}if(c.qsa&&
 (!q||!q.test(a))){if(s=r=u,w=b,x=9===k&&a,1===k&&"object"!==b.nodeName.toLowerCase()){o=g(a),(r=b.getAttribute("id"))?s=r.replace(bb,"\\$&"):b.setAttribute("id",s),s="[id='"+s+"'] ",l=o.length;while(l--)o[l]=s+qb(o[l]);w=ab.test(a)&&ob(b.parentNode)||b,x=o.join(",")}if(x)try{return I.apply(d,w.querySelectorAll(x)),d}catch(y){}finally{r||b.removeAttribute("id")}}}return i(a.replace(R,"$1"),b,d,e)}function gb(){var a=[];function b(c,e){return a.push(c+" ")>d.cacheLength&&delete b[a.shift()],b[c+" "]=e}return b}function hb(a){return a[u]=!0,a}function ib(a){var b=n.createElement("div");try{return!!a(b)}catch(c){return!1}finally{b.parentNode&&b.parentNode.removeChild(b),b=null}}function jb(a,b){var c=a.split("|"),e=a.length;while(e--)d.attrHandle[c[e]]=b}function kb(a,b){var c=b&&a,d=c&&1===a.nodeType&&1===b.nodeType&&(~b.sourceIndex||D)-(~a.sourceIndex||D);if(d)return d;if(c)while(c=c.nextSibling)if(c===b)return-1;return a?1:-1}function lb(a){return function(b){var c=b.nodeName.toLower
 Case();return"input"===c&&b.type===a}}function mb(a){return function(b){var c=b.nodeName.toLowerCase();return("input"===c||"button"===c)&&b.type===a}}function nb(a){return hb(function(b){return b=+b,hb(function(c,d){var e,f=a([],c.length,b),g=f.length;while(g--)c[e=f[g]]&&(c[e]=!(d[e]=c[e]))})})}function ob(a){return a&&typeof a.getElementsByTagName!==C&&a}c=fb.support={},f=fb.isXML=function(a){var b=a&&(a.ownerDocument||a).documentElement;return b?"HTML"!==b.nodeName:!1},m=fb.setDocument=function(a){var b,e=a?a.ownerDocument||a:v,g=e.defaultView;return e!==n&&9===e.nodeType&&e.documentElement?(n=e,o=e.documentElement,p=!f(e),g&&g!==g.top&&(g.addEventListener?g.addEventListener("unload",function(){m()},!1):g.attachEvent&&g.attachEvent("onunload",function(){m()})),c.attributes=ib(function(a){return a.className="i",!a.getAttribute("className")}),c.getElementsByTagName=ib(function(a){return a.appendChild(e.createComment("")),!a.getElementsByTagName("*").length}),c.getElementsByClassNam
 e=$.test(e.getElementsByClassName)&&ib(function(a){return a.innerHTML="<div class='a'></div><div class='a i'></div>",a.firstChild.className="i",2===a.getElementsByClassName("i").length}),c.getById=ib(function(a){return o.appendChild(a).id=u,!e.getElementsByName||!e.getElementsByName(u).length}),c.getById?(d.find.ID=function(a,b){if(typeof b.getElementById!==C&&p){var c=b.getElementById(a);return c&&c.parentNode?[c]:[]}},d.filter.ID=function(a){var b=a.replace(cb,db);return function(a){return a.getAttribute("id")===b}}):(delete d.find.ID,d.filter.ID=function(a){var b=a.replace(cb,db);return function(a){var c=typeof a.getAttributeNode!==C&&a.getAttributeNode("id");return c&&c.value===b}}),d.find.TAG=c.getElementsByTagName?function(a,b){return typeof b.getElementsByTagName!==C?b.getElementsByTagName(a):void 0}:function(a,b){var c,d=[],e=0,f=b.getElementsByTagName(a);if("*"===a){while(c=f[e++])1===c.nodeType&&d.push(c);return d}return f},d.find.CLASS=c.getElementsByClassName&&function(a
 ,b){return typeof b.getElementsByClassName!==C&&p?b.getElementsByClassName(a):void 0},r=[],q=[],(c.qsa=$.test(e.querySelectorAll))&&(ib(function(a){a.innerHTML="<select msallowclip=''><option selected=''></option></select>",a.querySelectorAll("[msallowclip^='']").length&&q.push("[*^$]="+M+"*(?:''|\"\")"),a.querySelectorAll("[selected]").length||q.push("\\["+M+"*(?:value|"+L+")"),a.querySelectorAll(":checked").length||q.push(":checked")}),ib(function(a){var b=e.createElement("input");b.setAttribute("type","hidden"),a.appendChild(b).setAttribute("name","D"),a.querySelectorAll("[name=d]").length&&q.push("name"+M+"*[*^$|!~]?="),a.querySelectorAll(":enabled").length||q.push(":enabled",":disabled"),a.querySelectorAll("*,:x"),q.push(",.*:")})),(c.matchesSelector=$.test(s=o.matches||o.webkitMatchesSelector||o.mozMatchesSelector||o.oMatchesSelector||o.msMatchesSelector))&&ib(function(a){c.disconnectedMatch=s.call(a,"div"),s.call(a,"[s!='']:x"),r.push("!=",Q)}),q=q.length&&new RegExp(q.join("
 |")),r=r.length&&new RegExp(r.join("|")),b=$.test(o.compareDocumentPosition),t=b||$.test(o.contains)?function(a,b){var c=9===a.nodeType?a.documentElement:a,d=b&&b.parentNode;return a===d||!(!d||1!==d.nodeType||!(c.contains?c.contains(d):a.compareDocumentPosition&&16&a.compareDocumentPosition(d)))}:function(a,b){if(b)while(b=b.parentNode)if(b===a)return!0;return!1},B=b?function(a,b){if(a===b)return l=!0,0;var d=!a.compareDocumentPosition-!b.compareDocumentPosition;return d?d:(d=(a.ownerDocument||a)===(b.ownerDocument||b)?a.compareDocumentPosition(b):1,1&d||!c.sortDetached&&b.compareDocumentPosition(a)===d?a===e||a.ownerDocument===v&&t(v,a)?-1:b===e||b.ownerDocument===v&&t(v,b)?1:k?K.call(k,a)-K.call(k,b):0:4&d?-1:1)}:function(a,b){if(a===b)return l=!0,0;var c,d=0,f=a.parentNode,g=b.parentNode,h=[a],i=[b];if(!f||!g)return a===e?-1:b===e?1:f?-1:g?1:k?K.call(k,a)-K.call(k,b):0;if(f===g)return kb(a,b);c=a;while(c=c.parentNode)h.unshift(c);c=b;while(c=c.parentNode)i.unshift(c);while(h[d]=
 ==i[d])d++;return d?kb(h[d],i[d]):h[d]===v?-1:i[d]===v?1:0},e):n},fb.matches=function(a,b){return fb(a,null,null,b)},fb.matchesSelector=function(a,b){if((a.ownerDocument||a)!==n&&m(a),b=b.replace(U,"='$1']"),!(!c.matchesSelector||!p||r&&r.test(b)||q&&q.test(b)))try{var d=s.call(a,b);if(d||c.disconnectedMatch||a.document&&11!==a.document.nodeType)return d}catch(e){}return fb(b,n,null,[a]).length>0},fb.contains=function(a,b){return(a.ownerDocument||a)!==n&&m(a),t(a,b)},fb.attr=function(a,b){(a.ownerDocument||a)!==n&&m(a);var e=d.attrHandle[b.toLowerCase()],f=e&&E.call(d.attrHandle,b.toLowerCase())?e(a,b,!p):void 0;return void 0!==f?f:c.attributes||!p?a.getAttribute(b):(f=a.getAttributeNode(b))&&f.specified?f.value:null},fb.error=function(a){throw new Error("Syntax error, unrecognized expression: "+a)},fb.uniqueSort=function(a){var b,d=[],e=0,f=0;if(l=!c.detectDuplicates,k=!c.sortStable&&a.slice(0),a.sort(B),l){while(b=a[f++])b===a[f]&&(e=d.push(f));while(e--)a.splice(d[e],1)}return k=
 null,a},e=fb.getText=function(a){var b,c="",d=0,f=a.nodeType;if(f){if(1===f||9===f||11===f){if("string"==typeof a.textContent)return a.textContent;for(a=a.firstChild;a;a=a.nextSibling)c+=e(a)}else if(3===f||4===f)return a.nodeValue}else while(b=a[d++])c+=e(b);return c},d=fb.selectors={cacheLength:50,createPseudo:hb,match:X,attrHandle:{},find:{},relative:{">":{dir:"parentNode",first:!0}," ":{dir:"parentNode"},"+":{dir:"previousSibling",first:!0},"~":{dir:"previousSibling"}},preFilter:{ATTR:function(a){return a[1]=a[1].replace(cb,db),a[3]=(a[3]||a[4]||a[5]||"").replace(cb,db),"~="===a[2]&&(a[3]=" "+a[3]+" "),a.slice(0,4)},CHILD:function(a){return a[1]=a[1].toLowerCase(),"nth"===a[1].slice(0,3)?(a[3]||fb.error(a[0]),a[4]=+(a[4]?a[5]+(a[6]||1):2*("even"===a[3]||"odd"===a[3])),a[5]=+(a[7]+a[8]||"odd"===a[3])):a[3]&&fb.error(a[0]),a},PSEUDO:function(a){var b,c=!a[6]&&a[2];return X.CHILD.test(a[0])?null:(a[3]?a[2]=a[4]||a[5]||"":c&&V.test(c)&&(b=g(c,!0))&&(b=c.indexOf(")",c.length-b)-c.len
 gth)&&(a[0]=a[0].slice(0,b),a[2]=c.slice(0,b)),a.slice(0,3))}},filter:{TAG:function(a){var b=a.replace(cb,db).toLowerCase();return"*"===a?function(){return!0}:function(a){return a.nodeName&&a.nodeName.toLowerCase()===b}},CLASS:function(a){var b=y[a+" "];return b||(b=new RegExp("(^|"+M+")"+a+"("+M+"|$)"))&&y(a,function(a){return b.test("string"==typeof a.className&&a.className||typeof a.getAttribute!==C&&a.getAttribute("class")||"")})},ATTR:function(a,b,c){return function(d){var e=fb.attr(d,a);return null==e?"!="===b:b?(e+="","="===b?e===c:"!="===b?e!==c:"^="===b?c&&0===e.indexOf(c):"*="===b?c&&e.indexOf(c)>-1:"$="===b?c&&e.slice(-c.length)===c:"~="===b?(" "+e+" ").indexOf(c)>-1:"|="===b?e===c||e.slice(0,c.length+1)===c+"-":!1):!0}},CHILD:function(a,b,c,d,e){var f="nth"!==a.slice(0,3),g="last"!==a.slice(-4),h="of-type"===b;return 1===d&&0===e?function(a){return!!a.parentNode}:function(b,c,i){var j,k,l,m,n,o,p=f!==g?"nextSibling":"previousSibling",q=b.parentNode,r=h&&b.nodeName.toLowe
 rCase(),s=!i&&!h;if(q){if(f){while(p){l=b;while(l=l[p])if(h?l.nodeName.toLowerCase()===r:1===l.nodeType)return!1;o=p="only"===a&&!o&&"nextSibling"}return!0}if(o=[g?q.firstChild:q.lastChild],g&&s){k=q[u]||(q[u]={}),j=k[a]||[],n=j[0]===w&&j[1],m=j[0]===w&&j[2],l=n&&q.childNodes[n];while(l=++n&&l&&l[p]||(m=n=0)||o.pop())if(1===l.nodeType&&++m&&l===b){k[a]=[w,n,m];break}}else if(s&&(j=(b[u]||(b[u]={}))[a])&&j[0]===w)m=j[1];else while(l=++n&&l&&l[p]||(m=n=0)||o.pop())if((h?l.nodeName.toLowerCase()===r:1===l.nodeType)&&++m&&(s&&((l[u]||(l[u]={}))[a]=[w,m]),l===b))break;return m-=e,m===d||m%d===0&&m/d>=0}}},PSEUDO:function(a,b){var c,e=d.pseudos[a]||d.setFilters[a.toLowerCase()]||fb.error("unsupported pseudo: "+a);return e[u]?e(b):e.length>1?(c=[a,a,"",b],d.setFilters.hasOwnProperty(a.toLowerCase())?hb(function(a,c){var d,f=e(a,b),g=f.length;while(g--)d=K.call(a,f[g]),a[d]=!(c[d]=f[g])}):function(a){return e(a,0,c)}):e}},pseudos:{not:hb(function(a){var b=[],c=[],d=h(a.replace(R,"$1"));retu
 rn d[u]?hb(function(a,b,c,e){var f,g=d(a,null,e,[]),h=a.length;while(h--)(f=g[h])&&(a[h]=!(b[h]=f))}):function(a,e,f){return b[0]=a,d(b,null,f,c),!c.pop()}}),has:hb(function(a){return function(b){return fb(a,b).length>0}}),contains:hb(function(a){return function(b){return(b.textContent||b.innerText||e(b)).indexOf(a)>-1}}),lang:hb(function(a){return W.test(a||"")||fb.error("unsupported lang: "+a),a=a.replace(cb,db).toLowerCase(),function(b){var c;do if(c=p?b.lang:b.getAttribute("xml:lang")||b.getAttribute("lang"))return c=c.toLowerCase(),c===a||0===c.indexOf(a+"-");while((b=b.parentNode)&&1===b.nodeType);return!1}}),target:function(b){var c=a.location&&a.location.hash;return c&&c.slice(1)===b.id},root:function(a){return a===o},focus:function(a){return a===n.activeElement&&(!n.hasFocus||n.hasFocus())&&!!(a.type||a.href||~a.tabIndex)},enabled:function(a){return a.disabled===!1},disabled:function(a){return a.disabled===!0},checked:function(a){var b=a.nodeName.toLowerCase();return"input"
 ===b&&!!a.checked||"option"===b&&!!a.selected},selected:function(a){return a.parentNode&&a.parentNode.selectedIndex,a.selected===!0},empty:function(a){for(a=a.firstChild;a;a=a.nextSibling)if(a.nodeType<6)return!1;return!0},parent:function(a){return!d.pseudos.empty(a)},header:function(a){return Z.test(a.nodeName)},input:function(a){return Y.test(a.nodeName)},button:function(a){var b=a.nodeName.toLowerCase();return"input"===b&&"button"===a.type||"button"===b},text:function(a){var b;return"input"===a.nodeName.toLowerCase()&&"text"===a.type&&(null==(b=a.getAttribute("type"))||"text"===b.toLowerCase())},first:nb(function(){return[0]}),last:nb(function(a,b){return[b-1]}),eq:nb(function(a,b,c){return[0>c?c+b:c]}),even:nb(function(a,b){for(var c=0;b>c;c+=2)a.push(c);return a}),odd:nb(function(a,b){for(var c=1;b>c;c+=2)a.push(c);return a}),lt:nb(function(a,b,c){for(var d=0>c?c+b:c;--d>=0;)a.push(d);return a}),gt:nb(function(a,b,c){for(var d=0>c?c+b:c;++d<b;)a.push(d);return a})}},d.pseudos.n
 th=d.pseudos.eq;for(b in{radio:!0,checkbox:!0,file:!0,password:!0,image:!0})d.pseudos[b]=lb(b);for(b in{submit:!0,reset:!0})d.pseudos[b]=mb(b);function pb(){}pb.prototype=d.filters=d.pseudos,d.setFilters=new pb,g=fb.tokenize=function(a,b){var c,e,f,g,h,i,j,k=z[a+" "];if(k)return b?0:k.slice(0);h=a,i=[],j=d.preFilter;while(h){(!c||(e=S.exec(h)))&&(e&&(h=h.slice(e[0].length)||h),i.push(f=[])),c=!1,(e=T.exec(h))&&(c=e.shift(),f.push({value:c,type:e[0].replace(R," ")}),h=h.slice(c.length));for(g in d.filter)!(e=X[g].exec(h))||j[g]&&!(e=j[g](e))||(c=e.shift(),f.push({value:c,type:g,matches:e}),h=h.slice(c.length));if(!c)break}return b?h.length:h?fb.error(a):z(a,i).slice(0)};function qb(a){for(var b=0,c=a.length,d="";c>b;b++)d+=a[b].value;return d}function rb(a,b,c){var d=b.dir,e=c&&"parentNode"===d,f=x++;return b.first?function(b,c,f){while(b=b[d])if(1===b.nodeType||e)return a(b,c,f)}:function(b,c,g){var h,i,j=[w,f];if(g){while(b=b[d])if((1===b.nodeType||e)&&a(b,c,g))return!0}else while(
 b=b[d])if(1===b.nodeType||e){if(i=b[u]||(b[u]={}),(h=i[d])&&h[0]===w&&h[1]===f)return j[2]=h[2];if(i[d]=j,j[2]=a(b,c,g))return!0}}}function sb(a){return a.length>1?function(b,c,d){var e=a.length;while(e--)if(!a[e](b,c,d))return!1;return!0}:a[0]}function tb(a,b,c){for(var d=0,e=b.length;e>d;d++)fb(a,b[d],c);return c}function ub(a,b,c,d,e){for(var f,g=[],h=0,i=a.length,j=null!=b;i>h;h++)(f=a[h])&&(!c||c(f,d,e))&&(g.push(f),j&&b.push(h));return g}function vb(a,b,c,d,e,f){return d&&!d[u]&&(d=vb(d)),e&&!e[u]&&(e=vb(e,f)),hb(function(f,g,h,i){var j,k,l,m=[],n=[],o=g.length,p=f||tb(b||"*",h.nodeType?[h]:h,[]),q=!a||!f&&b?p:ub(p,m,a,h,i),r=c?e||(f?a:o||d)?[]:g:q;if(c&&c(q,r,h,i),d){j=ub(r,n),d(j,[],h,i),k=j.length;while(k--)(l=j[k])&&(r[n[k]]=!(q[n[k]]=l))}if(f){if(e||a){if(e){j=[],k=r.length;while(k--)(l=r[k])&&j.push(q[k]=l);e(null,r=[],j,i)}k=r.length;while(k--)(l=r[k])&&(j=e?K.call(f,l):m[k])>-1&&(f[j]=!(g[j]=l))}}else r=ub(r===g?r.splice(o,r.length):r),e?e(null,g,r,i):I.apply(g,r)})}fu
 nction wb(a){for(var b,c,e,f=a.length,g=d.relative[a[0].type],h=g||d.relative[" "],i=g?1:0,k=rb(function(a){return a===b},h,!0),l=rb(function(a){return K.call(b,a)>-1},h,!0),m=[function(a,c,d){return!g&&(d||c!==j)||((b=c).nodeType?k(a,c,d):l(a,c,d))}];f>i;i++)if(c=d.relative[a[i].type])m=[rb(sb(m),c)];else{if(c=d.filter[a[i].type].apply(null,a[i].matches),c[u]){for(e=++i;f>e;e++)if(d.relative[a[e].type])break;return vb(i>1&&sb(m),i>1&&qb(a.slice(0,i-1).concat({value:" "===a[i-2].type?"*":""})).replace(R,"$1"),c,e>i&&wb(a.slice(i,e)),f>e&&wb(a=a.slice(e)),f>e&&qb(a))}m.push(c)}return sb(m)}function xb(a,b){var c=b.length>0,e=a.length>0,f=function(f,g,h,i,k){var l,m,o,p=0,q="0",r=f&&[],s=[],t=j,u=f||e&&d.find.TAG("*",k),v=w+=null==t?1:Math.random()||.1,x=u.length;for(k&&(j=g!==n&&g);q!==x&&null!=(l=u[q]);q++){if(e&&l){m=0;while(o=a[m++])if(o(l,g,h)){i.push(l);break}k&&(w=v)}c&&((l=!o&&l)&&p--,f&&r.push(l))}if(p+=q,c&&q!==p){m=0;while(o=b[m++])o(r,s,g,h);if(f){if(p>0)while(q--)r[q]||s[
 q]||(s[q]=G.call(i));s=ub(s)}I.apply(i,s),k&&!f&&s.length>0&&p+b.length>1&&fb.uniqueSort(i)}return k&&(w=v,j=t),r};return c?hb(f):f}return h=fb.compile=function(a,b){var c,d=[],e=[],f=A[a+" "];if(!f){b||(b=g(a)),c=b.length;while(c--)f=wb(b[c]),f[u]?d.push(f):e.push(f);f=A(a,xb(e,d)),f.selector=a}return f},i=fb.select=function(a,b,e,f){var i,j,k,l,m,n="function"==typeof a&&a,o=!f&&g(a=n.selector||a);if(e=e||[],1===o.length){if(j=o[0]=o[0].slice(0),j.length>2&&"ID"===(k=j[0]).type&&c.getById&&9===b.nodeType&&p&&d.relative[j[1].type]){if(b=(d.find.ID(k.matches[0].replace(cb,db),b)||[])[0],!b)return e;n&&(b=b.parentNode),a=a.slice(j.shift().value.length)}i=X.needsContext.test(a)?0:j.length;while(i--){if(k=j[i],d.relative[l=k.type])break;if((m=d.find[l])&&(f=m(k.matches[0].replace(cb,db),ab.test(j[0].type)&&ob(b.parentNode)||b))){if(j.splice(i,1),a=f.length&&qb(j),!a)return I.apply(e,f),e;break}}}return(n||h(a,o))(f,b,!p,e,ab.test(a)&&ob(b.parentNode)||b),e},c.sortStable=u.split("").sort
 (B).join("")===u,c.detectDuplicates=!!l,m(),c.sortDetached=ib(function(a){return 1&a.compareDocumentPosition(n.createElement("div"))}),ib(function(a){return a.innerHTML="<a href='#'></a>","#"===a.firstChild.getAttribute("href")})||jb("type|href|height|width",function(a,b,c){return c?void 0:a.getAttribute(b,"type"===b.toLowerCase()?1:2)}),c.attributes&&ib(function(a){return a.innerHTML="<input/>",a.firstChild.setAttribute("value",""),""===a.firstChild.getAttribute("value")})||jb("value",function(a,b,c){return c||"input"!==a.nodeName.toLowerCase()?void 0:a.defaultValue}),ib(function(a){return null==a.getAttribute("disabled")})||jb(L,function(a,b,c){var d;return c?void 0:a[b]===!0?b.toLowerCase():(d=a.getAttributeNode(b))&&d.specified?d.value:null}),fb}(a);m.find=s,m.expr=s.selectors,m.expr[":"]=m.expr.pseudos,m.unique=s.uniqueSort,m.text=s.getText,m.isXMLDoc=s.isXML,m.contains=s.contains;var t=m.expr.match.needsContext,u=/^<(\w+)\s*\/?>(?:<\/\1>|)$/,v=/^.[^:#\[\.,]*$/;function w(a,b,c
 ){if(m.isFunction(b))return m.grep(a,function(a,d){return!!b.call(a,d,a)!==c});if(b.nodeType)return m.grep(a,function(a){return a===b!==c});if("string"==typeof b){if(v.test(b))return m.filter(b,a,c);b=m.filter(b,a)}return m.grep(a,function(a){return m.inArray(a,b)>=0!==c})}m.filter=function(a,b,c){var d=b[0];return c&&(a=":not("+a+")"),1===b.length&&1===d.nodeType?m.find.matchesSelector(d,a)?[d]:[]:m.find.matches(a,m.grep(b,function(a){return 1===a.nodeType}))},m.fn.extend({find:function(a){var b,c=[],d=this,e=d.length;if("string"!=typeof a)return this.pushStack(m(a).filter(function(){for(b=0;e>b;b++)if(m.contains(d[b],this))return!0}));for(b=0;e>b;b++)m.find(a,d[b],c);return c=this.pushStack(e>1?m.unique(c):c),c.selector=this.selector?this.selector+" "+a:a,c},filter:function(a){return this.pushStack(w(this,a||[],!1))},not:function(a){return this.pushStack(w(this,a||[],!0))},is:function(a){return!!w(this,"string"==typeof a&&t.test(a)?m(a):a||[],!1).length}});var x,y=a.document,z=/^(
 ?:\s*(<[\w\W]+>)[^>]*|#([\w-]*))$/,A=m.fn.init=function(a,b){var c,d;if(!a)return this;if("string"==typeof a){if(c="<"===a.charAt(0)&&">"===a.charAt(a.length-1)&&a.length>=3?[null,a,null]:z.exec(a),!c||!c[1]&&b)return!b||b.jquery?(b||x).find(a):this.constructor(b).find(a);if(c[1]){if(b=b instanceof m?b[0]:b,m.merge(this,m.parseHTML(c[1],b&&b.nodeType?b.ownerDocument||b:y,!0)),u.test(c[1])&&m.isPlainObject(b))for(c in b)m.isFunction(this[c])?this[c](b[c]):this.attr(c,b[c]);return this}if(d=y.getElementById(c[2]),d&&d.parentNode){if(d.id!==c[2])return x.find(a);this.length=1,this[0]=d}return this.context=y,this.selector=a,this}return a.nodeType?(this.context=this[0]=a,this.length=1,this):m.isFunction(a)?"undefined"!=typeof x.ready?x.ready(a):a(m):(void 0!==a.selector&&(this.selector=a.selector,this.context=a.context),m.makeArray(a,this))};A.prototype=m.fn,x=m(y);var B=/^(?:parents|prev(?:Until|All))/,C={children:!0,contents:!0,next:!0,prev:!0};m.extend({dir:function(a,b,c){var d=[],e=
 a[b];while(e&&9!==e.nodeType&&(void 0===c||1!==e.nodeType||!m(e).is(c)))1===e.nodeType&&d.push(e),e=e[b];return d},sibling:function(a,b){for(var c=[];a;a=a.nextSibling)1===a.nodeType&&a!==b&&c.push(a);return c}}),m.fn.extend({has:function(a){var b,c=m(a,this),d=c.length;return this.filter(function(){for(b=0;d>b;b++)if(m.contains(this,c[b]))return!0})},closest:function(a,b){for(var c,d=0,e=this.length,f=[],g=t.test(a)||"string"!=typeof a?m(a,b||this.context):0;e>d;d++)for(c=this[d];c&&c!==b;c=c.parentNode)if(c.nodeType<11&&(g?g.index(c)>-1:1===c.nodeType&&m.find.matchesSelector(c,a))){f.push(c);break}return this.pushStack(f.length>1?m.unique(f):f)},index:function(a){return a?"string"==typeof a?m.inArray(this[0],m(a)):m.inArray(a.jquery?a[0]:a,this):this[0]&&this[0].parentNode?this.first().prevAll().length:-1},add:function(a,b){return this.pushStack(m.unique(m.merge(this.get(),m(a,b))))},addBack:function(a){return this.add(null==a?this.prevObject:this.prevObject.filter(a))}});function
  D(a,b){do a=a[b];while(a&&1!==a.nodeType);return a}m.each({parent:function(a){var b=a.parentNode;return b&&11!==b.nodeType?b:null},parents:function(a){return m.dir(a,"parentNode")},parentsUntil:function(a,b,c){return m.dir(a,"parentNode",c)},next:function(a){return D(a,"nextSibling")},prev:function(a){return D(a,"previousSibling")},nextAll:function(a){return m.dir(a,"nextSibling")},prevAll:function(a){return m.dir(a,"previousSibling")},nextUntil:function(a,b,c){return m.dir(a,"nextSibling",c)},prevUntil:function(a,b,c){return m.dir(a,"previousSibling",c)},siblings:function(a){return m.sibling((a.parentNode||{}).firstChild,a)},children:function(a){return m.sibling(a.firstChild)},contents:function(a){return m.nodeName(a,"iframe")?a.contentDocument||a.contentWindow.document:m.merge([],a.childNodes)}},function(a,b){m.fn[a]=function(c,d){var e=m.map(this,b,c);return"Until"!==a.slice(-5)&&(d=c),d&&"string"==typeof d&&(e=m.filter(d,e)),this.length>1&&(C[a]||(e=m.unique(e)),B.test(a)&&(e=e
 .reverse())),this.pushStack(e)}});var E=/\S+/g,F={};function G(a){var b=F[a]={};return m.each(a.match(E)||[],function(a,c){b[c]=!0}),b}m.Callbacks=function(a){a="string"==typeof a?F[a]||G(a):m.extend({},a);var b,c,d,e,f,g,h=[],i=!a.once&&[],j=function(l){for(c=a.memory&&l,d=!0,f=g||0,g=0,e=h.length,b=!0;h&&e>f;f++)if(h[f].apply(l[0],l[1])===!1&&a.stopOnFalse){c=!1;break}b=!1,h&&(i?i.length&&j(i.shift()):c?h=[]:k.disable())},k={add:function(){if(h){var d=h.length;!function f(b){m.each(b,function(b,c){var d=m.type(c);"function"===d?a.unique&&k.has(c)||h.push(c):c&&c.length&&"string"!==d&&f(c)})}(arguments),b?e=h.length:c&&(g=d,j(c))}return this},remove:function(){return h&&m.each(arguments,function(a,c){var d;while((d=m.inArray(c,h,d))>-1)h.splice(d,1),b&&(e>=d&&e--,f>=d&&f--)}),this},has:function(a){return a?m.inArray(a,h)>-1:!(!h||!h.length)},empty:function(){return h=[],e=0,this},disable:function(){return h=i=c=void 0,this},disabled:function(){return!h},lock:function(){return i=voi
 d 0,c||k.disable(),this},locked:function(){return!i},fireWith:function(a,c){return!h||d&&!i||(c=c||[],c=[a,c.slice?c.slice():c],b?i.push(c):j(c)),this},fire:function(){return k.fireWith(this,arguments),this},fired:function(){return!!d}};return k},m.extend({Deferred:function(a){var b=[["resolve","done",m.Callbacks("once memory"),"resolved"],["reject","fail",m.Callbacks("once memory"),"rejected"],["notify","progress",m.Callbacks("memory")]],c="pending",d={state:function(){return c},always:function(){return e.done(arguments).fail(arguments),this},then:function(){var a=arguments;return m.Deferred(function(c){m.each(b,function(b,f){var g=m.isFunction(a[b])&&a[b];e[f[1]](function(){var a=g&&g.apply(this,arguments);a&&m.isFunction(a.promise)?a.promise().done(c.resolve).fail(c.reject).progress(c.notify):c[f[0]+"With"](this===d?c.promise():this,g?[a]:arguments)})}),a=null}).promise()},promise:function(a){return null!=a?m.extend(a,d):d}},e={};return d.pipe=d.then,m.each(b,function(a,f){var g=
 f[2],h=f[3];d[f[1]]=g.add,h&&g.add(function(){c=h},b[1^a][2].disable,b[2][2].lock),e[f[0]]=function(){return e[f[0]+"With"](this===e?d:this,arguments),this},e[f[0]+"With"]=g.fireWith}),d.promise(e),a&&a.call(e,e),e},when:function(a){var b=0,c=d.call(arguments),e=c.length,f=1!==e||a&&m.isFunction(a.promise)?e:0,g=1===f?a:m.Deferred(),h=function(a,b,c){return function(e){b[a]=this,c[a]=arguments.length>1?d.call(arguments):e,c===i?g.notifyWith(b,c):--f||g.resolveWith(b,c)}},i,j,k;if(e>1)for(i=new Array(e),j=new Array(e),k=new Array(e);e>b;b++)c[b]&&m.isFunction(c[b].promise)?c[b].promise().done(h(b,k,c)).fail(g.reject).progress(h(b,j,i)):--f;return f||g.resolveWith(k,c),g.promise()}});var H;m.fn.ready=function(a){return m.ready.promise().done(a),this},m.extend({isReady:!1,readyWait:1,holdReady:function(a){a?m.readyWait++:m.ready(!0)},ready:function(a){if(a===!0?!--m.readyWait:!m.isReady){if(!y.body)return setTimeout(m.ready);m.isReady=!0,a!==!0&&--m.readyWait>0||(H.resolveWith(y,[m]),m
 .fn.triggerHandler&&(m(y).triggerHandler("ready"),m(y).off("ready")))}}});function I(){y.addEventListener?(y.removeEventListener("DOMContentLoaded",J,!1),a.removeEventListener("load",J,!1)):(y.detachEvent("onreadystatechange",J),a.detachEvent("onload",J))}function J(){(y.addEventListener||"load"===event.type||"complete"===y.readyState)&&(I(),m.ready())}m.ready.promise=function(b){if(!H)if(H=m.Deferred(),"complete"===y.readyState)setTimeout(m.ready);else if(y.addEventListener)y.addEventListener("DOMContentLoaded",J,!1),a.addEventListener("load",J,!1);else{y.attachEvent("onreadystatechange",J),a.attachEvent("onload",J);var c=!1;try{c=null==a.frameElement&&y.documentElement}catch(d){}c&&c.doScroll&&!function e(){if(!m.isReady){try{c.doScroll("left")}catch(a){return setTimeout(e,50)}I(),m.ready()}}()}return H.promise(b)};var K="undefined",L;for(L in m(k))break;k.ownLast="0"!==L,k.inlineBlockNeedsLayout=!1,m(function(){var a,b,c,d;c=y.getElementsByTagName("body")[0],c&&c.style&&(b=y.crea
 teElement("div"),d=y.createElement("div"),d.style.cssText="position:absolute;border:0;width:0;height:0;top:0;left:-9999px",c.appendChild(d).appendChild(b),typeof b.style.zoom!==K&&(b.style.cssText="display:inline;margin:0;border:0;padding:1px;width:1px;zoom:1",k.inlineBlockNeedsLayout=a=3===b.offsetWidth,a&&(c.style.zoom=1)),c.removeChild(d))}),function(){var a=y.createElement("div");if(null==k.deleteExpando){k.deleteExpando=!0;try{delete a.test}catch(b){k.deleteExpando=!1}}a=null}(),m.acceptData=function(a){var b=m.noData[(a.nodeName+" ").toLowerCase()],c=+a.nodeType||1;return 1!==c&&9!==c?!1:!b||b!==!0&&a.getAttribute("classid")===b};var M=/^(?:\{[\w\W]*\}|\[[\w\W]*\])$/,N=/([A-Z])/g;function O(a,b,c){if(void 0===c&&1===a.nodeType){var d="data-"+b.replace(N,"-$1").toLowerCase();if(c=a.getAttribute(d),"string"==typeof c){try{c="true"===c?!0:"false"===c?!1:"null"===c?null:+c+""===c?+c:M.test(c)?m.parseJSON(c):c}catch(e){}m.data(a,b,c)}else c=void 0}return c}function P(a){var b;for(b
  in a)if(("data"!==b||!m.isEmptyObject(a[b]))&&"toJSON"!==b)return!1;return!0}function Q(a,b,d,e){if(m.acceptData(a)){var f,g,h=m.expando,i=a.nodeType,j=i?m.cache:a,k=i?a[h]:a[h]&&h;
-if(k&&j[k]&&(e||j[k].data)||void 0!==d||"string"!=typeof b)return k||(k=i?a[h]=c.pop()||m.guid++:h),j[k]||(j[k]=i?{}:{toJSON:m.noop}),("object"==typeof b||"function"==typeof b)&&(e?j[k]=m.extend(j[k],b):j[k].data=m.extend(j[k].data,b)),g=j[k],e||(g.data||(g.data={}),g=g.data),void 0!==d&&(g[m.camelCase(b)]=d),"string"==typeof b?(f=g[b],null==f&&(f=g[m.camelCase(b)])):f=g,f}}function R(a,b,c){if(m.acceptData(a)){var d,e,f=a.nodeType,g=f?m.cache:a,h=f?a[m.expando]:m.expando;if(g[h]){if(b&&(d=c?g[h]:g[h].data)){m.isArray(b)?b=b.concat(m.map(b,m.camelCase)):b in d?b=[b]:(b=m.camelCase(b),b=b in d?[b]:b.split(" ")),e=b.length;while(e--)delete d[b[e]];if(c?!P(d):!m.isEmptyObject(d))return}(c||(delete g[h].data,P(g[h])))&&(f?m.cleanData([a],!0):k.deleteExpando||g!=g.window?delete g[h]:g[h]=null)}}}m.extend({cache:{},noData:{"applet ":!0,"embed ":!0,"object ":"clsid:D27CDB6E-AE6D-11cf-96B8-444553540000"},hasData:function(a){return a=a.nodeType?m.cache[a[m.expando]]:a[m.expando],!!a&&!P(a)},
 data:function(a,b,c){return Q(a,b,c)},removeData:function(a,b){return R(a,b)},_data:function(a,b,c){return Q(a,b,c,!0)},_removeData:function(a,b){return R(a,b,!0)}}),m.fn.extend({data:function(a,b){var c,d,e,f=this[0],g=f&&f.attributes;if(void 0===a){if(this.length&&(e=m.data(f),1===f.nodeType&&!m._data(f,"parsedAttrs"))){c=g.length;while(c--)g[c]&&(d=g[c].name,0===d.indexOf("data-")&&(d=m.camelCase(d.slice(5)),O(f,d,e[d])));m._data(f,"parsedAttrs",!0)}return e}return"object"==typeof a?this.each(function(){m.data(this,a)}):arguments.length>1?this.each(function(){m.data(this,a,b)}):f?O(f,a,m.data(f,a)):void 0},removeData:function(a){return this.each(function(){m.removeData(this,a)})}}),m.extend({queue:function(a,b,c){var d;return a?(b=(b||"fx")+"queue",d=m._data(a,b),c&&(!d||m.isArray(c)?d=m._data(a,b,m.makeArray(c)):d.push(c)),d||[]):void 0},dequeue:function(a,b){b=b||"fx";var c=m.queue(a,b),d=c.length,e=c.shift(),f=m._queueHooks(a,b),g=function(){m.dequeue(a,b)};"inprogress"===e&&(
 e=c.shift(),d--),e&&("fx"===b&&c.unshift("inprogress"),delete f.stop,e.call(a,g,f)),!d&&f&&f.empty.fire()},_queueHooks:function(a,b){var c=b+"queueHooks";return m._data(a,c)||m._data(a,c,{empty:m.Callbacks("once memory").add(function(){m._removeData(a,b+"queue"),m._removeData(a,c)})})}}),m.fn.extend({queue:function(a,b){var c=2;return"string"!=typeof a&&(b=a,a="fx",c--),arguments.length<c?m.queue(this[0],a):void 0===b?this:this.each(function(){var c=m.queue(this,a,b);m._queueHooks(this,a),"fx"===a&&"inprogress"!==c[0]&&m.dequeue(this,a)})},dequeue:function(a){return this.each(function(){m.dequeue(this,a)})},clearQueue:function(a){return this.queue(a||"fx",[])},promise:function(a,b){var c,d=1,e=m.Deferred(),f=this,g=this.length,h=function(){--d||e.resolveWith(f,[f])};"string"!=typeof a&&(b=a,a=void 0),a=a||"fx";while(g--)c=m._data(f[g],a+"queueHooks"),c&&c.empty&&(d++,c.empty.add(h));return h(),e.promise(b)}});var S=/[+-]?(?:\d*\.|)\d+(?:[eE][+-]?\d+|)/.source,T=["Top","Right","Botto
 m","Left"],U=function(a,b){return a=b||a,"none"===m.css(a,"display")||!m.contains(a.ownerDocument,a)},V=m.access=function(a,b,c,d,e,f,g){var h=0,i=a.length,j=null==c;if("object"===m.type(c)){e=!0;for(h in c)m.access(a,b,h,c[h],!0,f,g)}else if(void 0!==d&&(e=!0,m.isFunction(d)||(g=!0),j&&(g?(b.call(a,d),b=null):(j=b,b=function(a,b,c){return j.call(m(a),c)})),b))for(;i>h;h++)b(a[h],c,g?d:d.call(a[h],h,b(a[h],c)));return e?a:j?b.call(a):i?b(a[0],c):f},W=/^(?:checkbox|radio)$/i;!function(){var a=y.createElement("input"),b=y.createElement("div"),c=y.createDocumentFragment();if(b.innerHTML="  <link/><table></table><a href='/a'>a</a><input type='checkbox'/>",k.leadingWhitespace=3===b.firstChild.nodeType,k.tbody=!b.getElementsByTagName("tbody").length,k.htmlSerialize=!!b.getElementsByTagName("link").length,k.html5Clone="<:nav></:nav>"!==y.createElement("nav").cloneNode(!0).outerHTML,a.type="checkbox",a.checked=!0,c.appendChild(a),k.appendChecked=a.checked,b.innerHTML="<textarea>x</textarea>
 ",k.noCloneChecked=!!b.cloneNode(!0).lastChild.defaultValue,c.appendChild(b),b.innerHTML="<input type='radio' checked='checked' name='t'/>",k.checkClone=b.cloneNode(!0).cloneNode(!0).lastChild.checked,k.noCloneEvent=!0,b.attachEvent&&(b.attachEvent("onclick",function(){k.noCloneEvent=!1}),b.cloneNode(!0).click()),null==k.deleteExpando){k.deleteExpando=!0;try{delete b.test}catch(d){k.deleteExpando=!1}}}(),function(){var b,c,d=y.createElement("div");for(b in{submit:!0,change:!0,focusin:!0})c="on"+b,(k[b+"Bubbles"]=c in a)||(d.setAttribute(c,"t"),k[b+"Bubbles"]=d.attributes[c].expando===!1);d=null}();var X=/^(?:input|select|textarea)$/i,Y=/^key/,Z=/^(?:mouse|pointer|contextmenu)|click/,$=/^(?:focusinfocus|focusoutblur)$/,_=/^([^.]*)(?:\.(.+)|)$/;function ab(){return!0}function bb(){return!1}function cb(){try{return y.activeElement}catch(a){}}m.event={global:{},add:function(a,b,c,d,e){var f,g,h,i,j,k,l,n,o,p,q,r=m._data(a);if(r){c.handler&&(i=c,c=i.handler,e=i.selector),c.guid||(c.guid=
 m.guid++),(g=r.events)||(g=r.events={}),(k=r.handle)||(k=r.handle=function(a){return typeof m===K||a&&m.event.triggered===a.type?void 0:m.event.dispatch.apply(k.elem,arguments)},k.elem=a),b=(b||"").match(E)||[""],h=b.length;while(h--)f=_.exec(b[h])||[],o=q=f[1],p=(f[2]||"").split(".").sort(),o&&(j=m.event.special[o]||{},o=(e?j.delegateType:j.bindType)||o,j=m.event.special[o]||{},l=m.extend({type:o,origType:q,data:d,handler:c,guid:c.guid,selector:e,needsContext:e&&m.expr.match.needsContext.test(e),namespace:p.join(".")},i),(n=g[o])||(n=g[o]=[],n.delegateCount=0,j.setup&&j.setup.call(a,d,p,k)!==!1||(a.addEventListener?a.addEventListener(o,k,!1):a.attachEvent&&a.attachEvent("on"+o,k))),j.add&&(j.add.call(a,l),l.handler.guid||(l.handler.guid=c.guid)),e?n.splice(n.delegateCount++,0,l):n.push(l),m.event.global[o]=!0);a=null}},remove:function(a,b,c,d,e){var f,g,h,i,j,k,l,n,o,p,q,r=m.hasData(a)&&m._data(a);if(r&&(k=r.events)){b=(b||"").match(E)||[""],j=b.length;while(j--)if(h=_.exec(b[j])||
 [],o=q=h[1],p=(h[2]||"").split(".").sort(),o){l=m.event.special[o]||{},o=(d?l.delegateType:l.bindType)||o,n=k[o]||[],h=h[2]&&new RegExp("(^|\\.)"+p.join("\\.(?:.*\\.|)")+"(\\.|$)"),i=f=n.length;while(f--)g=n[f],!e&&q!==g.origType||c&&c.guid!==g.guid||h&&!h.test(g.namespace)||d&&d!==g.selector&&("**"!==d||!g.selector)||(n.splice(f,1),g.selector&&n.delegateCount--,l.remove&&l.remove.call(a,g));i&&!n.length&&(l.teardown&&l.teardown.call(a,p,r.handle)!==!1||m.removeEvent(a,o,r.handle),delete k[o])}else for(o in k)m.event.remove(a,o+b[j],c,d,!0);m.isEmptyObject(k)&&(delete r.handle,m._removeData(a,"events"))}},trigger:function(b,c,d,e){var f,g,h,i,k,l,n,o=[d||y],p=j.call(b,"type")?b.type:b,q=j.call(b,"namespace")?b.namespace.split("."):[];if(h=l=d=d||y,3!==d.nodeType&&8!==d.nodeType&&!$.test(p+m.event.triggered)&&(p.indexOf(".")>=0&&(q=p.split("."),p=q.shift(),q.sort()),g=p.indexOf(":")<0&&"on"+p,b=b[m.expando]?b:new m.Event(p,"object"==typeof b&&b),b.isTrigger=e?2:3,b.namespace=q.join("
 ."),b.namespace_re=b.namespace?new RegExp("(^|\\.)"+q.join("\\.(?:.*\\.|)")+"(\\.|$)"):null,b.result=void 0,b.target||(b.target=d),c=null==c?[b]:m.makeArray(c,[b]),k=m.event.special[p]||{},e||!k.trigger||k.trigger.apply(d,c)!==!1)){if(!e&&!k.noBubble&&!m.isWindow(d)){for(i=k.delegateType||p,$.test(i+p)||(h=h.parentNode);h;h=h.parentNode)o.push(h),l=h;l===(d.ownerDocument||y)&&o.push(l.defaultView||l.parentWindow||a)}n=0;while((h=o[n++])&&!b.isPropagationStopped())b.type=n>1?i:k.bindType||p,f=(m._data(h,"events")||{})[b.type]&&m._data(h,"handle"),f&&f.apply(h,c),f=g&&h[g],f&&f.apply&&m.acceptData(h)&&(b.result=f.apply(h,c),b.result===!1&&b.preventDefault());if(b.type=p,!e&&!b.isDefaultPrevented()&&(!k._default||k._default.apply(o.pop(),c)===!1)&&m.acceptData(d)&&g&&d[p]&&!m.isWindow(d)){l=d[g],l&&(d[g]=null),m.event.triggered=p;try{d[p]()}catch(r){}m.event.triggered=void 0,l&&(d[g]=l)}return b.result}},dispatch:function(a){a=m.event.fix(a);var b,c,e,f,g,h=[],i=d.call(arguments),j=(m.
 _data(this,"events")||{})[a.type]||[],k=m.event.special[a.type]||{};if(i[0]=a,a.delegateTarget=this,!k.preDispatch||k.preDispatch.call(this,a)!==!1){h=m.event.handlers.call(this,a,j),b=0;while((f=h[b++])&&!a.isPropagationStopped()){a.currentTarget=f.elem,g=0;while((e=f.handlers[g++])&&!a.isImmediatePropagationStopped())(!a.namespace_re||a.namespace_re.test(e.namespace))&&(a.handleObj=e,a.data=e.data,c=((m.event.special[e.origType]||{}).handle||e.handler).apply(f.elem,i),void 0!==c&&(a.result=c)===!1&&(a.preventDefault(),a.stopPropagation()))}return k.postDispatch&&k.postDispatch.call(this,a),a.result}},handlers:function(a,b){var c,d,e,f,g=[],h=b.delegateCount,i=a.target;if(h&&i.nodeType&&(!a.button||"click"!==a.type))for(;i!=this;i=i.parentNode||this)if(1===i.nodeType&&(i.disabled!==!0||"click"!==a.type)){for(e=[],f=0;h>f;f++)d=b[f],c=d.selector+" ",void 0===e[c]&&(e[c]=d.needsContext?m(c,this).index(i)>=0:m.find(c,this,null,[i]).length),e[c]&&e.push(d);e.length&&g.push({elem:i,hand
 lers:e})}return h<b.length&&g.push({elem:this,handlers:b.slice(h)}),g},fix:function(a){if(a[m.expando])return a;var b,c,d,e=a.type,f=a,g=this.fixHooks[e];g||(this.fixHooks[e]=g=Z.test(e)?this.mouseHooks:Y.test(e)?this.keyHooks:{}),d=g.props?this.props.concat(g.props):this.props,a=new m.Event(f),b=d.length;while(b--)c=d[b],a[c]=f[c];return a.target||(a.target=f.srcElement||y),3===a.target.nodeType&&(a.target=a.target.parentNode),a.metaKey=!!a.metaKey,g.filter?g.filter(a,f):a},props:"altKey bubbles cancelable ctrlKey currentTarget eventPhase metaKey relatedTarget shiftKey target timeStamp view which".split(" "),fixHooks:{},keyHooks:{props:"char charCode key keyCode".split(" "),filter:function(a,b){return null==a.which&&(a.which=null!=b.charCode?b.charCode:b.keyCode),a}},mouseHooks:{props:"button buttons clientX clientY fromElement offsetX offsetY pageX pageY screenX screenY toElement".split(" "),filter:function(a,b){var c,d,e,f=b.button,g=b.fromElement;return null==a.pageX&&null!=b.cl
 ientX&&(d=a.target.ownerDocument||y,e=d.documentElement,c=d.body,a.pageX=b.clientX+(e&&e.scrollLeft||c&&c.scrollLeft||0)-(e&&e.clientLeft||c&&c.clientLeft||0),a.pageY=b.clientY+(e&&e.scrollTop||c&&c.scrollTop||0)-(e&&e.clientTop||c&&c.clientTop||0)),!a.relatedTarget&&g&&(a.relatedTarget=g===a.target?b.toElement:g),a.which||void 0===f||(a.which=1&f?1:2&f?3:4&f?2:0),a}},special:{load:{noBubble:!0},focus:{trigger:function(){if(this!==cb()&&this.focus)try{return this.focus(),!1}catch(a){}},delegateType:"focusin"},blur:{trigger:function(){return this===cb()&&this.blur?(this.blur(),!1):void 0},delegateType:"focusout"},click:{trigger:function(){return m.nodeName(this,"input")&&"checkbox"===this.type&&this.click?(this.click(),!1):void 0},_default:function(a){return m.nodeName(a.target,"a")}},beforeunload:{postDispatch:function(a){void 0!==a.result&&a.originalEvent&&(a.originalEvent.returnValue=a.result)}}},simulate:function(a,b,c,d){var e=m.extend(new m.Event,c,{type:a,isSimulated:!0,origin
 alEvent:{}});d?m.event.trigger(e,null,b):m.event.dispatch.call(b,e),e.isDefaultPrevented()&&c.preventDefault()}},m.removeEvent=y.removeEventListener?function(a,b,c){a.removeEventListener&&a.removeEventListener(b,c,!1)}:function(a,b,c){var d="on"+b;a.detachEvent&&(typeof a[d]===K&&(a[d]=null),a.detachEvent(d,c))},m.Event=function(a,b){return this instanceof m.Event?(a&&a.type?(this.originalEvent=a,this.type=a.type,this.isDefaultPrevented=a.defaultPrevented||void 0===a.defaultPrevented&&a.returnValue===!1?ab:bb):this.type=a,b&&m.extend(this,b),this.timeStamp=a&&a.timeStamp||m.now(),void(this[m.expando]=!0)):new m.Event(a,b)},m.Event.prototype={isDefaultPrevented:bb,isPropagationStopped:bb,isImmediatePropagationStopped:bb,preventDefault:function(){var a=this.originalEvent;this.isDefaultPrevented=ab,a&&(a.preventDefault?a.preventDefault():a.returnValue=!1)},stopPropagation:function(){var a=this.originalEvent;this.isPropagationStopped=ab,a&&(a.stopPropagation&&a.stopPropagation(),a.cance
 lBubble=!0)},stopImmediatePropagation:function(){var a=this.originalEvent;this.isImmediatePropagationStopped=ab,a&&a.stopImmediatePropagation&&a.stopImmediatePropagation(),this.stopPropagation()}},m.each({mouseenter:"mouseover",mouseleave:"mouseout",pointerenter:"pointerover",pointerleave:"pointerout"},function(a,b){m.event.special[a]={delegateType:b,bindType:b,handle:function(a){var c,d=this,e=a.relatedTarget,f=a.handleObj;return(!e||e!==d&&!m.contains(d,e))&&(a.type=f.origType,c=f.handler.apply(this,arguments),a.type=b),c}}}),k.submitBubbles||(m.event.special.submit={setup:function(){return m.nodeName(this,"form")?!1:void m.event.add(this,"click._submit keypress._submit",function(a){var b=a.target,c=m.nodeName(b,"input")||m.nodeName(b,"button")?b.form:void 0;c&&!m._data(c,"submitBubbles")&&(m.event.add(c,"submit._submit",function(a){a._submit_bubble=!0}),m._data(c,"submitBubbles",!0))})},postDispatch:function(a){a._submit_bubble&&(delete a._submit_bubble,this.parentNode&&!a.isTrig
 ger&&m.event.simulate("submit",this.parentNode,a,!0))},teardown:function(){return m.nodeName(this,"form")?!1:void m.event.remove(this,"._submit")}}),k.changeBubbles||(m.event.special.change={setup:function(){return X.test(this.nodeName)?(("checkbox"===this.type||"radio"===this.type)&&(m.event.add(this,"propertychange._change",function(a){"checked"===a.originalEvent.propertyName&&(this._just_changed=!0)}),m.event.add(this,"click._change",function(a){this._just_changed&&!a.isTrigger&&(this._just_changed=!1),m.event.simulate("change",this,a,!0)})),!1):void m.event.add(this,"beforeactivate._change",function(a){var b=a.target;X.test(b.nodeName)&&!m._data(b,"changeBubbles")&&(m.event.add(b,"change._change",function(a){!this.parentNode||a.isSimulated||a.isTrigger||m.event.simulate("change",this.parentNode,a,!0)}),m._data(b,"changeBubbles",!0))})},handle:function(a){var b=a.target;return this!==b||a.isSimulated||a.isTrigger||"radio"!==b.type&&"checkbox"!==b.type?a.handleObj.handler.apply(th
 is,arguments):void 0},teardown:function(){return m.event.remove(this,"._change"),!X.test(this.nodeName)}}),k.focusinBubbles||m.each({focus:"focusin",blur:"focusout"},function(a,b){var c=function(a){m.event.simulate(b,a.target,m.event.fix(a),!0)};m.event.special[b]={setup:function(){var d=this.ownerDocument||this,e=m._data(d,b);e||d.addEventListener(a,c,!0),m._data(d,b,(e||0)+1)},teardown:function(){var d=this.ownerDocument||this,e=m._data(d,b)-1;e?m._data(d,b,e):(d.removeEventListener(a,c,!0),m._removeData(d,b))}}}),m.fn.extend({on:function(a,b,c,d,e){var f,g;if("object"==typeof a){"string"!=typeof b&&(c=c||b,b=void 0);for(f in a)this.on(f,b,c,a[f],e);return this}if(null==c&&null==d?(d=b,c=b=void 0):null==d&&("string"==typeof b?(d=c,c=void 0):(d=c,c=b,b=void 0)),d===!1)d=bb;else if(!d)return this;return 1===e&&(g=d,d=function(a){return m().off(a),g.apply(this,arguments)},d.guid=g.guid||(g.guid=m.guid++)),this.each(function(){m.event.add(this,a,d,c,b)})},one:function(a,b,c,d){return 
 this.on(a,b,c,d,1)},off:function(a,b,c){var d,e;if(a&&a.preventDefault&&a.handleObj)return d=a.handleObj,m(a.delegateTarget).off(d.namespace?d.origType+"."+d.namespace:d.origType,d.selector,d.handler),this;if("object"==typeof a){for(e in a)this.off(e,b,a[e]);return this}return(b===!1||"function"==typeof b)&&(c=b,b=void 0),c===!1&&(c=bb),this.each(function(){m.event.remove(this,a,c,b)})},trigger:function(a,b){return this.each(function(){m.event.trigger(a,b,this)})},triggerHandler:function(a,b){var c=this[0];return c?m.event.trigger(a,b,c,!0):void 0}});function db(a){var b=eb.split("|"),c=a.createDocumentFragment();if(c.createElement)while(b.length)c.createElement(b.pop());return c}var eb="abbr|article|aside|audio|bdi|canvas|data|datalist|details|figcaption|figure|footer|header|hgroup|mark|meter|nav|output|progress|section|summary|time|video",fb=/ jQuery\d+="(?:null|\d+)"/g,gb=new RegExp("<(?:"+eb+")[\\s/>]","i"),hb=/^\s+/,ib=/<(?!area|br|col|embed|hr|img|input|link|meta|param)(([\w:]
 +)[^>]*)\/>/gi,jb=/<([\w:]+)/,kb=/<tbody/i,lb=/<|&#?\w+;/,mb=/<(?:script|style|link)/i,nb=/checked\s*(?:[^=]|=\s*.checked.)/i,ob=/^$|\/(?:java|ecma)script/i,pb=/^true\/(.*)/,qb=/^\s*<!(?:\[CDATA\[|--)|(?:\]\]|--)>\s*$/g,rb={option:[1,"<select multiple='multiple'>","</select>"],legend:[1,"<fieldset>","</fieldset>"],area:[1,"<map>","</map>"],param:[1,"<object>","</object>"],thead:[1,"<table>","</table>"],tr:[2,"<table><tbody>","</tbody></table>"],col:[2,"<table><tbody></tbody><colgroup>","</colgroup></table>"],td:[3,"<table><tbody><tr>","</tr></tbody></table>"],_default:k.htmlSerialize?[0,"",""]:[1,"X<div>","</div>"]},sb=db(y),tb=sb.appendChild(y.createElement("div"));rb.optgroup=rb.option,rb.tbody=rb.tfoot=rb.colgroup=rb.caption=rb.thead,rb.th=rb.td;function ub(a,b){var c,d,e=0,f=typeof a.getElementsByTagName!==K?a.getElementsByTagName(b||"*"):typeof a.querySelectorAll!==K?a.querySelectorAll(b||"*"):void 0;if(!f)for(f=[],c=a.childNodes||a;null!=(d=c[e]);e++)!b||m.nodeName(d,b)?f.push
 (d):m.merge(f,ub(d,b));return void 0===b||b&&m.nodeName(a,b)?m.merge([a],f):f}function vb(a){W.test(a.type)&&(a.defaultChecked=a.checked)}function wb(a,b){return m.nodeName(a,"table")&&m.nodeName(11!==b.nodeType?b:b.firstChild,"tr")?a.getElementsByTagName("tbody")[0]||a.appendChild(a.ownerDocument.createElement("tbody")):a}function xb(a){return a.type=(null!==m.find.attr(a,"type"))+"/"+a.type,a}function yb(a){var b=pb.exec(a.type);return b?a.type=b[1]:a.removeAttribute("type"),a}function zb(a,b){for(var c,d=0;null!=(c=a[d]);d++)m._data(c,"globalEval",!b||m._data(b[d],"globalEval"))}function Ab(a,b){if(1===b.nodeType&&m.hasData(a)){var c,d,e,f=m._data(a),g=m._data(b,f),h=f.events;if(h){delete g.handle,g.events={};for(c in h)for(d=0,e=h[c].length;e>d;d++)m.event.add(b,c,h[c][d])}g.data&&(g.data=m.extend({},g.data))}}function Bb(a,b){var c,d,e;if(1===b.nodeType){if(c=b.nodeName.toLowerCase(),!k.noCloneEvent&&b[m.expando]){e=m._data(b);for(d in e.events)m.removeEvent(b,d,e.handle);b.rem
 oveAttribute(m.expando)}"script"===c&&b.text!==a.text?(xb(b).text=a.text,yb(b)):"object"===c?(b.parentNode&&(b.outerHTML=a.outerHTML),k.html5Clone&&a.innerHTML&&!m.trim(b.innerHTML)&&(b.innerHTML=a.innerHTML)):"input"===c&&W.test(a.type)?(b.defaultChecked=b.checked=a.checked,b.value!==a.value&&(b.value=a.value)):"option"===c?b.defaultSelected=b.selected=a.defaultSelected:("input"===c||"textarea"===c)&&(b.defaultValue=a.defaultValue)}}m.extend({clone:function(a,b,c){var d,e,f,g,h,i=m.contains(a.ownerDocument,a);if(k.html5Clone||m.isXMLDoc(a)||!gb.test("<"+a.nodeName+">")?f=a.cloneNode(!0):(tb.innerHTML=a.outerHTML,tb.removeChild(f=tb.firstChild)),!(k.noCloneEvent&&k.noCloneChecked||1!==a.nodeType&&11!==a.nodeType||m.isXMLDoc(a)))for(d=ub(f),h=ub(a),g=0;null!=(e=h[g]);++g)d[g]&&Bb(e,d[g]);if(b)if(c)for(h=h||ub(a),d=d||ub(f),g=0;null!=(e=h[g]);g++)Ab(e,d[g]);else Ab(a,f);return d=ub(f,"script"),d.length>0&&zb(d,!i&&ub(a,"script")),d=h=e=null,f},buildFragment:function(a,b,c,d){for(var e
 ,f,g,h,i,j,l,n=a.length,o=db(b),p=[],q=0;n>q;q++)if(f=a[q],f||0===f)if("object"===m.type(f))m.merge(p,f.nodeType?[f]:f);else if(lb.test(f)){h=h||o.appendChild(b.createElement("div")),i=(jb.exec(f)||["",""])[1].toLowerCase(),l=rb[i]||rb._default,h.innerHTML=l[1]+f.replace(ib,"<$1></$2>")+l[2],e=l[0];while(e--)h=h.lastChild;if(!k.leadingWhitespace&&hb.test(f)&&p.push(b.createTextNode(hb.exec(f)[0])),!k.tbody){f="table"!==i||kb.test(f)?"<table>"!==l[1]||kb.test(f)?0:h:h.firstChild,e=f&&f.childNodes.length;while(e--)m.nodeName(j=f.childNodes[e],"tbody")&&!j.childNodes.length&&f.removeChild(j)}m.merge(p,h.childNodes),h.textContent="";while(h.firstChild)h.removeChild(h.firstChild);h=o.lastChild}else p.push(b.createTextNode(f));h&&o.removeChild(h),k.appendChecked||m.grep(ub(p,"input"),vb),q=0;while(f=p[q++])if((!d||-1===m.inArray(f,d))&&(g=m.contains(f.ownerDocument,f),h=ub(o.appendChild(f),"script"),g&&zb(h),c)){e=0;while(f=h[e++])ob.test(f.type||"")&&c.push(f)}return h=null,o},cleanData:
 function(a,b){for(var d,e,f,g,h=0,i=m.expando,j=m.cache,l=k.deleteExpando,n=m.event.special;null!=(d=a[h]);h++)if((b||m.acceptData(d))&&(f=d[i],g=f&&j[f])){if(g.events)for(e in g.events)n[e]?m.event.remove(d,e):m.removeEvent(d,e,g.handle);j[f]&&(delete j[f],l?delete d[i]:typeof d.removeAttribute!==K?d.removeAttribute(i):d[i]=null,c.push(f))}}}),m.fn.extend({text:function(a){return V(this,function(a){return void 0===a?m.text(this):this.empty().append((this[0]&&this[0].ownerDocument||y).createTextNode(a))},null,a,arguments.length)},append:function(){return this.domManip(arguments,function(a){if(1===this.nodeType||11===this.nodeType||9===this.nodeType){var b=wb(this,a);b.appendChild(a)}})},prepend:function(){return this.domManip(arguments,function(a){if(1===this.nodeType||11===this.nodeType||9===this.nodeType){var b=wb(this,a);b.insertBefore(a,b.firstChild)}})},before:function(){return this.domManip(arguments,function(a){this.parentNode&&this.parentNode.insertBefore(a,this)})},after:fu
 nction(){return this.domManip(arguments,function(a){this.parentNode&&this.parentNode.insertBefore(a,this.nextSibling)})},remove:function(a,b){for(var c,d=a?m.filter(a,this):this,e=0;null!=(c=d[e]);e++)b||1!==c.nodeType||m.cleanData(ub(c)),c.parentNode&&(b&&m.contains(c.ownerDocument,c)&&zb(ub(c,"script")),c.parentNode.removeChild(c));return this},empty:function(){for(var a,b=0;null!=(a=this[b]);b++){1===a.nodeType&&m.cleanData(ub(a,!1));while(a.firstChild)a.removeChild(a.firstChild);a.options&&m.nodeName(a,"select")&&(a.options.length=0)}return this},clone:function(a,b){return a=null==a?!1:a,b=null==b?a:b,this.map(function(){return m.clone(this,a,b)})},html:function(a){return V(this,function(a){var b=this[0]||{},c=0,d=this.length;if(void 0===a)return 1===b.nodeType?b.innerHTML.replace(fb,""):void 0;if(!("string"!=typeof a||mb.test(a)||!k.htmlSerialize&&gb.test(a)||!k.leadingWhitespace&&hb.test(a)||rb[(jb.exec(a)||["",""])[1].toLowerCase()])){a=a.replace(ib,"<$1></$2>");try{for(;d>c;
 c++)b=this[c]||{},1===b.nodeType&&(m.cleanData(ub(b,!1)),b.innerHTML=a);b=0}catch(e){}}b&&this.empty().append(a)},null,a,arguments.length)},replaceWith:function(){var a=arguments[0];return this.domManip(arguments,function(b){a=this.parentNode,m.cleanData(ub(this)),a&&a.replaceChild(b,this)}),a&&(a.length||a.nodeType)?this:this.remove()},detach:function(a){return this.remove(a,!0)},domManip:function(a,b){a=e.apply([],a);var c,d,f,g,h,i,j=0,l=this.length,n=this,o=l-1,p=a[0],q=m.isFunction(p);if(q||l>1&&"string"==typeof p&&!k.checkClone&&nb.test(p))return this.each(function(c){var d=n.eq(c);q&&(a[0]=p.call(this,c,d.html())),d.domManip(a,b)});if(l&&(i=m.buildFragment(a,this[0].ownerDocument,!1,this),c=i.firstChild,1===i.childNodes.length&&(i=c),c)){for(g=m.map(ub(i,"script"),xb),f=g.length;l>j;j++)d=i,j!==o&&(d=m.clone(d,!0,!0),f&&m.merge(g,ub(d,"script"))),b.call(this[j],d,j);if(f)for(h=g[g.length-1].ownerDocument,m.map(g,yb),j=0;f>j;j++)d=g[j],ob.test(d.type||"")&&!m._data(d,"globalEv
 al")&&m.contains(h,d)&&(d.src?m._evalUrl&&m._evalUrl(d.src):m.globalEval((d.text||d.textContent||d.innerHTML||"").replace(qb,"")));i=c=null}return this}}),m.each({appendTo:"append",prependTo:"prepend",insertBefore:"before",insertAfter:"after",replaceAll:"replaceWith"},function(a,b){m.fn[a]=function(a){for(var c,d=0,e=[],g=m(a),h=g.length-1;h>=d;d++)c=d===h?this:this.clone(!0),m(g[d])[b](c),f.apply(e,c.get());return this.pushStack(e)}});var Cb,Db={};function Eb(b,c){var d,e=m(c.createElement(b)).appendTo(c.body),f=a.getDefaultComputedStyle&&(d=a.getDefaultComputedStyle(e[0]))?d.display:m.css(e[0],"display");return e.detach(),f}function Fb(a){var b=y,c=Db[a];return c||(c=Eb(a,b),"none"!==c&&c||(Cb=(Cb||m("<iframe frameborder='0' width='0' height='0'/>")).appendTo(b.documentElement),b=(Cb[0].contentWindow||Cb[0].contentDocument).document,b.write(),b.close(),c=Eb(a,b),Cb.detach()),Db[a]=c),c}!function(){var a;k.shrinkWrapBlocks=function(){if(null!=a)return a;a=!1;var b,c,d;return c=y.ge
 tElementsByTagName("body")[0],c&&c.style?(b=y.createElement("div"),d=y.createElement("div"),d.style.cssText="position:absolute;border:0;width:0;height:0;top:0;left:-9999px",c.appendChild(d).appendChild(b),typeof b.style.zoom!==K&&(b.style.cssText="-webkit-box-sizing:content-box;-moz-box-sizing:content-box;box-sizing:content-box;display:block;margin:0;border:0;padding:1px;width:1px;zoom:1",b.appendChild(y.createElement("div")).style.width="5px",a=3!==b.offsetWidth),c.removeChild(d),a):void 0}}();var Gb=/^margin/,Hb=new RegExp("^("+S+")(?!px)[a-z%]+$","i"),Ib,Jb,Kb=/^(top|right|bottom|left)$/;a.getComputedStyle?(Ib=function(a){return a.ownerDocument.defaultView.getComputedStyle(a,null)},Jb=function(a,b,c){var d,e,f,g,h=a.style;return c=c||Ib(a),g=c?c.getPropertyValue(b)||c[b]:void 0,c&&(""!==g||m.contains(a.ownerDocument,a)||(g=m.style(a,b)),Hb.test(g)&&Gb.test(b)&&(d=h.width,e=h.minWidth,f=h.maxWidth,h.minWidth=h.maxWidth=h.width=g,g=c.width,h.width=d,h.minWidth=e,h.maxWidth=f)),void
  0===g?g:g+""}):y.documentElement.currentStyle&&(Ib=function(a){return a.currentStyle},Jb=function(a,b,c){var d,e,f,g,h=a.style;return c=c||Ib(a),g=c?c[b]:void 0,null==g&&h&&h[b]&&(g=h[b]),Hb.test(g)&&!Kb.test(b)&&(d=h.left,e=a.runtimeStyle,f=e&&e.left,f&&(e.left=a.currentStyle.left),h.left="fontSize"===b?"1em":g,g=h.pixelLeft+"px",h.left=d,f&&(e.left=f)),void 0===g?g:g+""||"auto"});function Lb(a,b){return{get:function(){var c=a();if(null!=c)return c?void delete this.get:(this.get=b).apply(this,arguments)}}}!function(){var b,c,d,e,f,g,h;if(b=y.createElement("div"),b.innerHTML="  <link/><table></table><a href='/a'>a</a><input type='checkbox'/>",d=b.getElementsByTagName("a")[0],c=d&&d.style){c.cssText="float:left;opacity:.5",k.opacity="0.5"===c.opacity,k.cssFloat=!!c.cssFloat,b.style.backgroundClip="content-box",b.cloneNode(!0).style.backgroundClip="",k.clearCloneStyle="content-box"===b.style.backgroundClip,k.boxSizing=""===c.boxSizing||""===c.MozBoxSizing||""===c.WebkitBoxSizing,m.ex
 tend(k,{reliableHiddenOffsets:function(){return null==g&&i(),g},boxSizingReliable:function(){return null==f&&i(),f},pixelPosition:function(){return null==e&&i(),e},reliableMarginRight:function(){return null==h&&i(),h}});function i(){var b,c,d,i;c=y.getElementsByTagName("body")[0],c&&c.style&&(b=y.createElement("div"),d=y.createElement("div"),d.style.cssText="position:absolute;border:0;width:0;height:0;top:0;left:-9999px",c.appendChild(d).appendChild(b),b.style.cssText="-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box;display:block;margin-top:1%;top:1%;border:1px;padding:1px;width:4px;position:absolute",e=f=!1,h=!0,a.getComputedStyle&&(e="1%"!==(a.getComputedStyle(b,null)||{}).top,f="4px"===(a.getComputedStyle(b,null)||{width:"4px"}).width,i=b.appendChild(y.createElement("div")),i.style.cssText=b.style.cssText="-webkit-box-sizing:content-box;-moz-box-sizing:content-box;box-sizing:content-box;display:block;margin:0;border:0;padding:0",i.style.marginRight=
 i.style.width="0",b.style.width="1px",h=!parseFloat((a.getComputedStyle(i,null)||{}).marginRight)),b.innerHTML="<table><tr><td></td><td>t</td></tr></table>",i=b.getElementsByTagName("td"),i[0].style.cssText="margin:0;border:0;padding:0;display:none",g=0===i[0].offsetHeight,g&&(i[0].style.display="",i[1].style.display="none",g=0===i[0].offsetHeight),c.removeChild(d))}}}(),m.swap=function(a,b,c,d){var e,f,g={};for(f in b)g[f]=a.style[f],a.style[f]=b[f];e=c.apply(a,d||[]);for(f in b)a.style[f]=g[f];return e};var Mb=/alpha\([^)]*\)/i,Nb=/opacity\s*=\s*([^)]*)/,Ob=/^(none|table(?!-c[ea]).+)/,Pb=new RegExp("^("+S+")(.*)$","i"),Qb=new RegExp("^([+-])=("+S+")","i"),Rb={position:"absolute",visibility:"hidden",display:"block"},Sb={letterSpacing:"0",fontWeight:"400"},Tb=["Webkit","O","Moz","ms"];function Ub(a,b){if(b in a)return b;var c=b.charAt(0).toUpperCase()+b.slice(1),d=b,e=Tb.length;while(e--)if(b=Tb[e]+c,b in a)return b;return d}function Vb(a,b){for(var c,d,e,f=[],g=0,h=a.length;h>g;g++
 )d=a[g],d.style&&(f[g]=m._data(d,"olddisplay"),c=d.style.display,b?(f[g]||"none"!==c||(d.style.display=""),""===d.style.display&&U(d)&&(f[g]=m._data(d,"olddisplay",Fb(d.nodeName)))):(e=U(d),(c&&"none"!==c||!e)&&m._data(d,"olddisplay",e?c:m.css(d,"display"))));for(g=0;h>g;g++)d=a[g],d.style&&(b&&"none"!==d.style.display&&""!==d.style.display||(d.style.display=b?f[g]||"":"none"));return a}function Wb(a,b,c){var d=Pb.exec(b);return d?Math.max(0,d[1]-(c||0))+(d[2]||"px"):b}function Xb(a,b,c,d,e){for(var f=c===(d?"border":"content")?4:"width"===b?1:0,g=0;4>f;f+=2)"margin"===c&&(g+=m.css(a,c+T[f],!0,e)),d?("content"===c&&(g-=m.css(a,"padding"+T[f],!0,e)),"margin"!==c&&(g-=m.css(a,"border"+T[f]+"Width",!0,e))):(g+=m.css(a,"padding"+T[f],!0,e),"padding"!==c&&(g+=m.css(a,"border"+T[f]+"Width",!0,e)));return g}function Yb(a,b,c){var d=!0,e="width"===b?a.offsetWidth:a.offsetHeight,f=Ib(a),g=k.boxSizing&&"border-box"===m.css(a,"boxSizing",!1,f);if(0>=e||null==e){if(e=Jb(a,b,f),(0>e||null==e)&&(
 e=a.style[b]),Hb.test(e))return e;d=g&&(k.boxSizingReliable()||e===a.style[b]),e=parseFloat(e)||0}return e+Xb(a,b,c||(g?"border":"content"),d,f)+"px"}m.extend({cssHooks:{opacity:{get:function(a,b){if(b){var c=Jb(a,"opacity");return""===c?"1":c}}}},cssNumber:{columnCount:!0,fillOpacity:!0,flexGrow:!0,flexShrink:!0,fontWeight:!0,lineHeight:!0,opacity:!0,order:!0,orphans:!0,widows:!0,zIndex:!0,zoom:!0},cssProps:{"float":k.cssFloat?"cssFloat":"styleFloat"},style:function(a,b,c,d){if(a&&3!==a.nodeType&&8!==a.nodeType&&a.style){var e,f,g,h=m.camelCase(b),i=a.style;if(b=m.cssProps[h]||(m.cssProps[h]=Ub(i,h)),g=m.cssHooks[b]||m.cssHooks[h],void 0===c)return g&&"get"in g&&void 0!==(e=g.get(a,!1,d))?e:i[b];if(f=typeof c,"string"===f&&(e=Qb.exec(c))&&(c=(e[1]+1)*e[2]+parseFloat(m.css(a,b)),f="number"),null!=c&&c===c&&("number"!==f||m.cssNumber[h]||(c+="px"),k.clearCloneStyle||""!==c||0!==b.indexOf("background")||(i[b]="inherit"),!(g&&"set"in g&&void 0===(c=g.set(a,c,d)))))try{i[b]=c}catch(j){}
 }},css:function(a,b,c,d){var e,f,g,h=m.camelCase(b);return b=m.cssProps[h]||(m.cssProps[h]=Ub(a.style,h)),g=m.cssHooks[b]||m.cssHooks[h],g&&"get"in g&&(f=g.get(a,!0,c)),void 0===f&&(f=Jb(a,b,d)),"normal"===f&&b in Sb&&(f=Sb[b]),""===c||c?(e=parseFloat(f),c===!0||m.isNumeric(e)?e||0:f):f}}),m.each(["height","width"],function(a,b){m.cssHooks[b]={get:function(a,c,d){return c?Ob.test(m.css(a,"display"))&&0===a.offsetWidth?m.swap(a,Rb,function(){return Yb(a,b,d)}):Yb(a,b,d):void 0},set:function(a,c,d){var e=d&&Ib(a);return Wb(a,c,d?Xb(a,b,d,k.boxSizing&&"border-box"===m.css(a,"boxSizing",!1,e),e):0)}}}),k.opacity||(m.cssHooks.opacity={get:function(a,b){return Nb.test((b&&a.currentStyle?a.currentStyle.filter:a.style.filter)||"")?.01*parseFloat(RegExp.$1)+"":b?"1":""},set:function(a,b){var c=a.style,d=a.currentStyle,e=m.isNumeric(b)?"alpha(opacity="+100*b+")":"",f=d&&d.filter||c.filter||"";c.zoom=1,(b>=1||""===b)&&""===m.trim(f.replace(Mb,""))&&c.removeAttribute&&(c.removeAttribute("filter
 "),""===b||d&&!d.filter)||(c.filter=Mb.test(f)?f.replace(Mb,e):f+" "+e)}}),m.cssHooks.marginRight=Lb(k.reliableMarginRight,function(a,b){return b?m.swap(a,{display:"inline-block"},Jb,[a,"marginRight"]):void 0}),m.each({margin:"",padding:"",border:"Width"},function(a,b){m.cssHooks[a+b]={expand:function(c){for(var d=0,e={},f="string"==typeof c?c.split(" "):[c];4>d;d++)e[a+T[d]+b]=f[d]||f[d-2]||f[0];return e}},Gb.test(a)||(m.cssHooks[a+b].set=Wb)}),m.fn.extend({css:function(a,b){return V(this,function(a,b,c){var d,e,f={},g=0;if(m.isArray(b)){for(d=Ib(a),e=b.length;e>g;g++)f[b[g]]=m.css(a,b[g],!1,d);return f}return void 0!==c?m.style(a,b,c):m.css(a,b)},a,b,arguments.length>1)},show:function(){return Vb(this,!0)},hide:function(){return Vb(this)},toggle:function(a){return"boolean"==typeof a?a?this.show():this.hide():this.each(function(){U(this)?m(this).show():m(this).hide()})}});function Zb(a,b,c,d,e){return new Zb.prototype.init(a,b,c,d,e)}m.Tween=Zb,Zb.prototype={constructor:Zb,init:fun
 ction(a,b,c,d,e,f){this.elem=a,this.prop=c,this.easing=e||"swing",this.options=b,this.start=this.now=this.cur(),this.end=d,this.unit=f||(m.cssNumber[c]?"":"px")
-},cur:function(){var a=Zb.propHooks[this.prop];return a&&a.get?a.get(this):Zb.propHooks._default.get(this)},run:function(a){var b,c=Zb.propHooks[this.prop];return this.pos=b=this.options.duration?m.easing[this.easing](a,this.options.duration*a,0,1,this.options.duration):a,this.now=(this.end-this.start)*b+this.start,this.options.step&&this.options.step.call(this.elem,this.now,this),c&&c.set?c.set(this):Zb.propHooks._default.set(this),this}},Zb.prototype.init.prototype=Zb.prototype,Zb.propHooks={_default:{get:function(a){var b;return null==a.elem[a.prop]||a.elem.style&&null!=a.elem.style[a.prop]?(b=m.css(a.elem,a.prop,""),b&&"auto"!==b?b:0):a.elem[a.prop]},set:function(a){m.fx.step[a.prop]?m.fx.step[a.prop](a):a.elem.style&&(null!=a.elem.style[m.cssProps[a.prop]]||m.cssHooks[a.prop])?m.style(a.elem,a.prop,a.now+a.unit):a.elem[a.prop]=a.now}}},Zb.propHooks.scrollTop=Zb.propHooks.scrollLeft={set:function(a){a.elem.nodeType&&a.elem.parentNode&&(a.elem[a.prop]=a.now)}},m.easing={linear:fu
 nction(a){return a},swing:function(a){return.5-Math.cos(a*Math.PI)/2}},m.fx=Zb.prototype.init,m.fx.step={};var $b,_b,ac=/^(?:toggle|show|hide)$/,bc=new RegExp("^(?:([+-])=|)("+S+")([a-z%]*)$","i"),cc=/queueHooks$/,dc=[ic],ec={"*":[function(a,b){var c=this.createTween(a,b),d=c.cur(),e=bc.exec(b),f=e&&e[3]||(m.cssNumber[a]?"":"px"),g=(m.cssNumber[a]||"px"!==f&&+d)&&bc.exec(m.css(c.elem,a)),h=1,i=20;if(g&&g[3]!==f){f=f||g[3],e=e||[],g=+d||1;do h=h||".5",g/=h,m.style(c.elem,a,g+f);while(h!==(h=c.cur()/d)&&1!==h&&--i)}return e&&(g=c.start=+g||+d||0,c.unit=f,c.end=e[1]?g+(e[1]+1)*e[2]:+e[2]),c}]};function fc(){return setTimeout(function(){$b=void 0}),$b=m.now()}function gc(a,b){var c,d={height:a},e=0;for(b=b?1:0;4>e;e+=2-b)c=T[e],d["margin"+c]=d["padding"+c]=a;return b&&(d.opacity=d.width=a),d}function hc(a,b,c){for(var d,e=(ec[b]||[]).concat(ec["*"]),f=0,g=e.length;g>f;f++)if(d=e[f].call(c,b,a))return d}function ic(a,b,c){var d,e,f,g,h,i,j,l,n=this,o={},p=a.style,q=a.nodeType&&U(a),r=m._
 data(a,"fxshow");c.queue||(h=m._queueHooks(a,"fx"),null==h.unqueued&&(h.unqueued=0,i=h.empty.fire,h.empty.fire=function(){h.unqueued||i()}),h.unqueued++,n.always(function(){n.always(function(){h.unqueued--,m.queue(a,"fx").length||h.empty.fire()})})),1===a.nodeType&&("height"in b||"width"in b)&&(c.overflow=[p.overflow,p.overflowX,p.overflowY],j=m.css(a,"display"),l="none"===j?m._data(a,"olddisplay")||Fb(a.nodeName):j,"inline"===l&&"none"===m.css(a,"float")&&(k.inlineBlockNeedsLayout&&"inline"!==Fb(a.nodeName)?p.zoom=1:p.display="inline-block")),c.overflow&&(p.overflow="hidden",k.shrinkWrapBlocks()||n.always(function(){p.overflow=c.overflow[0],p.overflowX=c.overflow[1],p.overflowY=c.overflow[2]}));for(d in b)if(e=b[d],ac.exec(e)){if(delete b[d],f=f||"toggle"===e,e===(q?"hide":"show")){if("show"!==e||!r||void 0===r[d])continue;q=!0}o[d]=r&&r[d]||m.style(a,d)}else j=void 0;if(m.isEmptyObject(o))"inline"===("none"===j?Fb(a.nodeName):j)&&(p.display=j);else{r?"hidden"in r&&(q=r.hidden):r=m
 ._data(a,"fxshow",{}),f&&(r.hidden=!q),q?m(a).show():n.done(function(){m(a).hide()}),n.done(function(){var b;m._removeData(a,"fxshow");for(b in o)m.style(a,b,o[b])});for(d in o)g=hc(q?r[d]:0,d,n),d in r||(r[d]=g.start,q&&(g.end=g.start,g.start="width"===d||"height"===d?1:0))}}function jc(a,b){var c,d,e,f,g;for(c in a)if(d=m.camelCase(c),e=b[d],f=a[c],m.isArray(f)&&(e=f[1],f=a[c]=f[0]),c!==d&&(a[d]=f,delete a[c]),g=m.cssHooks[d],g&&"expand"in g){f=g.expand(f),delete a[d];for(c in f)c in a||(a[c]=f[c],b[c]=e)}else b[d]=e}function kc(a,b,c){var d,e,f=0,g=dc.length,h=m.Deferred().always(function(){delete i.elem}),i=function(){if(e)return!1;for(var b=$b||fc(),c=Math.max(0,j.startTime+j.duration-b),d=c/j.duration||0,f=1-d,g=0,i=j.tweens.length;i>g;g++)j.tweens[g].run(f);return h.notifyWith(a,[j,f,c]),1>f&&i?c:(h.resolveWith(a,[j]),!1)},j=h.promise({elem:a,props:m.extend({},b),opts:m.extend(!0,{specialEasing:{}},c),originalProperties:b,originalOptions:c,startTime:$b||fc(),duration:c.durati
 on,tweens:[],createTween:function(b,c){var d=m.Tween(a,j.opts,b,c,j.opts.specialEasing[b]||j.opts.easing);return j.tweens.push(d),d},stop:function(b){var c=0,d=b?j.tweens.length:0;if(e)return this;for(e=!0;d>c;c++)j.tweens[c].run(1);return b?h.resolveWith(a,[j,b]):h.rejectWith(a,[j,b]),this}}),k=j.props;for(jc(k,j.opts.specialEasing);g>f;f++)if(d=dc[f].call(j,a,k,j.opts))return d;return m.map(k,hc,j),m.isFunction(j.opts.start)&&j.opts.start.call(a,j),m.fx.timer(m.extend(i,{elem:a,anim:j,queue:j.opts.queue})),j.progress(j.opts.progress).done(j.opts.done,j.opts.complete).fail(j.opts.fail).always(j.opts.always)}m.Animation=m.extend(kc,{tweener:function(a,b){m.isFunction(a)?(b=a,a=["*"]):a=a.split(" ");for(var c,d=0,e=a.length;e>d;d++)c=a[d],ec[c]=ec[c]||[],ec[c].unshift(b)},prefilter:function(a,b){b?dc.unshift(a):dc.push(a)}}),m.speed=function(a,b,c){var d=a&&"object"==typeof a?m.extend({},a):{complete:c||!c&&b||m.isFunction(a)&&a,duration:a,easing:c&&b||b&&!m.isFunction(b)&&b};return 
 d.duration=m.fx.off?0:"number"==typeof d.duration?d.duration:d.duration in m.fx.speeds?m.fx.speeds[d.duration]:m.fx.speeds._default,(null==d.queue||d.queue===!0)&&(d.queue="fx"),d.old=d.complete,d.complete=function(){m.isFunction(d.old)&&d.old.call(this),d.queue&&m.dequeue(this,d.queue)},d},m.fn.extend({fadeTo:function(a,b,c,d){return this.filter(U).css("opacity",0).show().end().animate({opacity:b},a,c,d)},animate:function(a,b,c,d){var e=m.isEmptyObject(a),f=m.speed(b,c,d),g=function(){var b=kc(this,m.extend({},a),f);(e||m._data(this,"finish"))&&b.stop(!0)};return g.finish=g,e||f.queue===!1?this.each(g):this.queue(f.queue,g)},stop:function(a,b,c){var d=function(a){var b=a.stop;delete a.stop,b(c)};return"string"!=typeof a&&(c=b,b=a,a=void 0),b&&a!==!1&&this.queue(a||"fx",[]),this.each(function(){var b=!0,e=null!=a&&a+"queueHooks",f=m.timers,g=m._data(this);if(e)g[e]&&g[e].stop&&d(g[e]);else for(e in g)g[e]&&g[e].stop&&cc.test(e)&&d(g[e]);for(e=f.length;e--;)f[e].elem!==this||null!=a&
 &f[e].queue!==a||(f[e].anim.stop(c),b=!1,f.splice(e,1));(b||!c)&&m.dequeue(this,a)})},finish:function(a){return a!==!1&&(a=a||"fx"),this.each(function(){var b,c=m._data(this),d=c[a+"queue"],e=c[a+"queueHooks"],f=m.timers,g=d?d.length:0;for(c.finish=!0,m.queue(this,a,[]),e&&e.stop&&e.stop.call(this,!0),b=f.length;b--;)f[b].elem===this&&f[b].queue===a&&(f[b].anim.stop(!0),f.splice(b,1));for(b=0;g>b;b++)d[b]&&d[b].finish&&d[b].finish.call(this);delete c.finish})}}),m.each(["toggle","show","hide"],function(a,b){var c=m.fn[b];m.fn[b]=function(a,d,e){return null==a||"boolean"==typeof a?c.apply(this,arguments):this.animate(gc(b,!0),a,d,e)}}),m.each({slideDown:gc("show"),slideUp:gc("hide"),slideToggle:gc("toggle"),fadeIn:{opacity:"show"},fadeOut:{opacity:"hide"},fadeToggle:{opacity:"toggle"}},function(a,b){m.fn[a]=function(a,c,d){return this.animate(b,a,c,d)}}),m.timers=[],m.fx.tick=function(){var a,b=m.timers,c=0;for($b=m.now();c<b.length;c++)a=b[c],a()||b[c]!==a||b.splice(c--,1);b.length|
 |m.fx.stop(),$b=void 0},m.fx.timer=function(a){m.timers.push(a),a()?m.fx.start():m.timers.pop()},m.fx.interval=13,m.fx.start=function(){_b||(_b=setInterval(m.fx.tick,m.fx.interval))},m.fx.stop=function(){clearInterval(_b),_b=null},m.fx.speeds={slow:600,fast:200,_default:400},m.fn.delay=function(a,b){return a=m.fx?m.fx.speeds[a]||a:a,b=b||"fx",this.queue(b,function(b,c){var d=setTimeout(b,a);c.stop=function(){clearTimeout(d)}})},function(){var a,b,c,d,e;b=y.createElement("div"),b.setAttribute("className","t"),b.innerHTML="  <link/><table></table><a href='/a'>a</a><input type='checkbox'/>",d=b.getElementsByTagName("a")[0],c=y.createElement("select"),e=c.appendChild(y.createElement("option")),a=b.getElementsByTagName("input")[0],d.style.cssText="top:1px",k.getSetAttribute="t"!==b.className,k.style=/top/.test(d.getAttribute("style")),k.hrefNormalized="/a"===d.getAttribute("href"),k.checkOn=!!a.value,k.optSelected=e.selected,k.enctype=!!y.createElement("form").enctype,c.disabled=!0,k.opt
 Disabled=!e.disabled,a=y.createElement("input"),a.setAttribute("value",""),k.input=""===a.getAttribute("value"),a.value="t",a.setAttribute("type","radio"),k.radioValue="t"===a.value}();var lc=/\r/g;m.fn.extend({val:function(a){var b,c,d,e=this[0];{if(arguments.length)return d=m.isFunction(a),this.each(function(c){var e;1===this.nodeType&&(e=d?a.call(this,c,m(this).val()):a,null==e?e="":"number"==typeof e?e+="":m.isArray(e)&&(e=m.map(e,function(a){return null==a?"":a+""})),b=m.valHooks[this.type]||m.valHooks[this.nodeName.toLowerCase()],b&&"set"in b&&void 0!==b.set(this,e,"value")||(this.value=e))});if(e)return b=m.valHooks[e.type]||m.valHooks[e.nodeName.toLowerCase()],b&&"get"in b&&void 0!==(c=b.get(e,"value"))?c:(c=e.value,"string"==typeof c?c.replace(lc,""):null==c?"":c)}}}),m.extend({valHooks:{option:{get:function(a){var b=m.find.attr(a,"value");return null!=b?b:m.trim(m.text(a))}},select:{get:function(a){for(var b,c,d=a.options,e=a.selectedIndex,f="select-one"===a.type||0>e,g=f?
 null:[],h=f?e+1:d.length,i=0>e?h:f?e:0;h>i;i++)if(c=d[i],!(!c.selected&&i!==e||(k.optDisabled?c.disabled:null!==c.getAttribute("disabled"))||c.parentNode.disabled&&m.nodeName(c.parentNode,"optgroup"))){if(b=m(c).val(),f)return b;g.push(b)}return g},set:function(a,b){var c,d,e=a.options,f=m.makeArray(b),g=e.length;while(g--)if(d=e[g],m.inArray(m.valHooks.option.get(d),f)>=0)try{d.selected=c=!0}catch(h){d.scrollHeight}else d.selected=!1;return c||(a.selectedIndex=-1),e}}}}),m.each(["radio","checkbox"],function(){m.valHooks[this]={set:function(a,b){return m.isArray(b)?a.checked=m.inArray(m(a).val(),b)>=0:void 0}},k.checkOn||(m.valHooks[this].get=function(a){return null===a.getAttribute("value")?"on":a.value})});var mc,nc,oc=m.expr.attrHandle,pc=/^(?:checked|selected)$/i,qc=k.getSetAttribute,rc=k.input;m.fn.extend({attr:function(a,b){return V(this,m.attr,a,b,arguments.length>1)},removeAttr:function(a){return this.each(function(){m.removeAttr(this,a)})}}),m.extend({attr:function(a,b,c){v
 ar d,e,f=a.nodeType;if(a&&3!==f&&8!==f&&2!==f)return typeof a.getAttribute===K?m.prop(a,b,c):(1===f&&m.isXMLDoc(a)||(b=b.toLowerCase(),d=m.attrHooks[b]||(m.expr.match.bool.test(b)?nc:mc)),void 0===c?d&&"get"in d&&null!==(e=d.get(a,b))?e:(e=m.find.attr(a,b),null==e?void 0:e):null!==c?d&&"set"in d&&void 0!==(e=d.set(a,c,b))?e:(a.setAttribute(b,c+""),c):void m.removeAttr(a,b))},removeAttr:function(a,b){var c,d,e=0,f=b&&b.match(E);if(f&&1===a.nodeType)while(c=f[e++])d=m.propFix[c]||c,m.expr.match.bool.test(c)?rc&&qc||!pc.test(c)?a[d]=!1:a[m.camelCase("default-"+c)]=a[d]=!1:m.attr(a,c,""),a.removeAttribute(qc?c:d)},attrHooks:{type:{set:function(a,b){if(!k.radioValue&&"radio"===b&&m.nodeName(a,"input")){var c=a.value;return a.setAttribute("type",b),c&&(a.value=c),b}}}}}),nc={set:function(a,b,c){return b===!1?m.removeAttr(a,c):rc&&qc||!pc.test(c)?a.setAttribute(!qc&&m.propFix[c]||c,c):a[m.camelCase("default-"+c)]=a[c]=!0,c}},m.each(m.expr.match.bool.source.match(/\w+/g),function(a,b){var c
 =oc[b]||m.find.attr;oc[b]=rc&&qc||!pc.test(b)?function(a,b,d){var e,f;return d||(f=oc[b],oc[b]=e,e=null!=c(a,b,d)?b.toLowerCase():null,oc[b]=f),e}:function(a,b,c){return c?void 0:a[m.camelCase("default-"+b)]?b.toLowerCase():null}}),rc&&qc||(m.attrHooks.value={set:function(a,b,c){return m.nodeName(a,"input")?void(a.defaultValue=b):mc&&mc.set(a,b,c)}}),qc||(mc={set:function(a,b,c){var d=a.getAttributeNode(c);return d||a.setAttributeNode(d=a.ownerDocument.createAttribute(c)),d.value=b+="","value"===c||b===a.getAttribute(c)?b:void 0}},oc.id=oc.name=oc.coords=function(a,b,c){var d;return c?void 0:(d=a.getAttributeNode(b))&&""!==d.value?d.value:null},m.valHooks.button={get:function(a,b){var c=a.getAttributeNode(b);return c&&c.specified?c.value:void 0},set:mc.set},m.attrHooks.contenteditable={set:function(a,b,c){mc.set(a,""===b?!1:b,c)}},m.each(["width","height"],function(a,b){m.attrHooks[b]={set:function(a,c){return""===c?(a.setAttribute(b,"auto"),c):void 0}}})),k.style||(m.attrHooks.styl
 e={get:function(a){return a.style.cssText||void 0},set:function(a,b){return a.style.cssText=b+""}});var sc=/^(?:input|select|textarea|button|object)$/i,tc=/^(?:a|area)$/i;m.fn.extend({prop:function(a,b){return V(this,m.prop,a,b,arguments.length>1)},removeProp:function(a){return a=m.propFix[a]||a,this.each(function(){try{this[a]=void 0,delete this[a]}catch(b){}})}}),m.extend({propFix:{"for":"htmlFor","class":"className"},prop:function(a,b,c){var d,e,f,g=a.nodeType;if(a&&3!==g&&8!==g&&2!==g)return f=1!==g||!m.isXMLDoc(a),f&&(b=m.propFix[b]||b,e=m.propHooks[b]),void 0!==c?e&&"set"in e&&void 0!==(d=e.set(a,c,b))?d:a[b]=c:e&&"get"in e&&null!==(d=e.get(a,b))?d:a[b]},propHooks:{tabIndex:{get:function(a){var b=m.find.attr(a,"tabindex");return b?parseInt(b,10):sc.test(a.nodeName)||tc.test(a.nodeName)&&a.href?0:-1}}}}),k.hrefNormalized||m.each(["href","src"],function(a,b){m.propHooks[b]={get:function(a){return a.getAttribute(b,4)}}}),k.optSelected||(m.propHooks.selected={get:function(a){var b
 =a.parentNode;return b&&(b.selectedIndex,b.parentNode&&b.parentNode.selectedIndex),null}}),m.each(["tabIndex","readOnly","maxLength","cellSpacing","cellPadding","rowSpan","colSpan","useMap","frameBorder","contentEditable"],function(){m.propFix[this.toLowerCase()]=this}),k.enctype||(m.propFix.enctype="encoding");var uc=/[\t\r\n\f]/g;m.fn.extend({addClass:function(a){var b,c,d,e,f,g,h=0,i=this.length,j="string"==typeof a&&a;if(m.isFunction(a))return this.each(function(b){m(this).addClass(a.call(this,b,this.className))});if(j)for(b=(a||"").match(E)||[];i>h;h++)if(c=this[h],d=1===c.nodeType&&(c.className?(" "+c.className+" ").replace(uc," "):" ")){f=0;while(e=b[f++])d.indexOf(" "+e+" ")<0&&(d+=e+" ");g=m.trim(d),c.className!==g&&(c.className=g)}return this},removeClass:function(a){var b,c,d,e,f,g,h=0,i=this.length,j=0===arguments.length||"string"==typeof a&&a;if(m.isFunction(a))return this.each(function(b){m(this).removeClass(a.call(this,b,this.className))});if(j)for(b=(a||"").match(E)|
 |[];i>h;h++)if(c=this[h],d=1===c.nodeType&&(c.className?(" "+c.className+" ").replace(uc," "):"")){f=0;while(e=b[f++])while(d.indexOf(" "+e+" ")>=0)d=d.replace(" "+e+" "," ");g=a?m.trim(d):"",c.className!==g&&(c.className=g)}return this},toggleClass:function(a,b){var c=typeof a;return"boolean"==typeof b&&"string"===c?b?this.addClass(a):this.removeClass(a):this.each(m.isFunction(a)?function(c){m(this).toggleClass(a.call(this,c,this.className,b),b)}:function(){if("string"===c){var b,d=0,e=m(this),f=a.match(E)||[];while(b=f[d++])e.hasClass(b)?e.removeClass(b):e.addClass(b)}else(c===K||"boolean"===c)&&(this.className&&m._data(this,"__className__",this.className),this.className=this.className||a===!1?"":m._data(this,"__className__")||"")})},hasClass:function(a){for(var b=" "+a+" ",c=0,d=this.length;d>c;c++)if(1===this[c].nodeType&&(" "+this[c].className+" ").replace(uc," ").indexOf(b)>=0)return!0;return!1}}),m.each("blur focus focusin focusout load resize scroll unload click dblclick mou
 sedown mouseup mousemove mouseover mouseout mouseenter mouseleave change select submit keydown keypress keyup error contextmenu".split(" "),function(a,b){m.fn[b]=function(a,c){return arguments.length>0?this.on(b,null,a,c):this.trigger(b)}}),m.fn.extend({hover:function(a,b){return this.mouseenter(a).mouseleave(b||a)},bind:function(a,b,c){return this.on(a,null,b,c)},unbind:function(a,b){return this.off(a,null,b)},delegate:function(a,b,c,d){return this.on(b,a,c,d)},undelegate:function(a,b,c){return 1===arguments.length?this.off(a,"**"):this.off(b,a||"**",c)}});var vc=m.now(),wc=/\?/,xc=/(,)|(\[|{)|(}|])|"(?:[^"\\\r\n]|\\["\\\/bfnrt]|\\u[\da-fA-F]{4})*"\s*:?|true|false|null|-?(?!0\d)\d+(?:\.\d+|)(?:[eE][+-]?\d+|)/g;m.parseJSON=function(b){if(a.JSON&&a.JSON.parse)return a.JSON.parse(b+"");var c,d=null,e=m.trim(b+"");return e&&!m.trim(e.replace(xc,function(a,b,e,f){return c&&b&&(d=0),0===d?a:(c=e||b,d+=!f-!e,"")}))?Function("return "+e)():m.error("Invalid JSON: "+b)},m.parseXML=function(b
 ){var c,d;if(!b||"string"!=typeof b)return null;try{a.DOMParser?(d=new DOMParser,c=d.parseFromString(b,"text/xml")):(c=new ActiveXObject("Microsoft.XMLDOM"),c.async="false",c.loadXML(b))}catch(e){c=void 0}return c&&c.documentElement&&!c.getElementsByTagName("parsererror").length||m.error("Invalid XML: "+b),c};var yc,zc,Ac=/#.*$/,Bc=/([?&])_=[^&]*/,Cc=/^(.*?):[ \t]*([^\r\n]*)\r?$/gm,Dc=/^(?:about|app|app-storage|.+-extension|file|res|widget):$/,Ec=/^(?:GET|HEAD)$/,Fc=/^\/\//,Gc=/^([\w.+-]+:)(?:\/\/(?:[^\/?#]*@|)([^\/?#:]*)(?::(\d+)|)|)/,Hc={},Ic={},Jc="*/".concat("*");try{zc=location.href}catch(Kc){zc=y.createElement("a"),zc.href="",zc=zc.href}yc=Gc.exec(zc.toLowerCase())||[];function Lc(a){return function(b,c){"string"!=typeof b&&(c=b,b="*");var d,e=0,f=b.toLowerCase().match(E)||[];if(m.isFunction(c))while(d=f[e++])"+"===d.charAt(0)?(d=d.slice(1)||"*",(a[d]=a[d]||[]).unshift(c)):(a[d]=a[d]||[]).push(c)}}function Mc(a,b,c,d){var e={},f=a===Ic;function g(h){var i;return e[h]=!0,m.each
 (a[h]||[],function(a,h){var j=h(b,c,d);return"string"!=typeof j||f||e[j]?f?!(i=j):void 0:(b.dataTypes.unshift(j),g(j),!1)}),i}return g(b.dataTypes[0])||!e["*"]&&g("*")}function Nc(a,b){var c,d,e=m.ajaxSettings.flatOptions||{};for(d in b)void 0!==b[d]&&((e[d]?a:c||(c={}))[d]=b[d]);return c&&m.extend(!0,a,c),a}function Oc(a,b,c){var d,e,f,g,h=a.contents,i=a.dataTypes;while("*"===i[0])i.shift(),void 0===e&&(e=a.mimeType||b.getResponseHeader("Content-Type"));if(e)for(g in h)if(h[g]&&h[g].test(e)){i.unshift(g);break}if(i[0]in c)f=i[0];else{for(g in c){if(!i[0]||a.converters[g+" "+i[0]]){f=g;break}d||(d=g)}f=f||d}return f?(f!==i[0]&&i.unshift(f),c[f]):void 0}function Pc(a,b,c,d){var e,f,g,h,i,j={},k=a.dataTypes.slice();if(k[1])for(g in a.converters)j[g.toLowerCase()]=a.converters[g];f=k.shift();while(f)if(a.responseFields[f]&&(c[a.responseFields[f]]=b),!i&&d&&a.dataFilter&&(b=a.dataFilter(b,a.dataType)),i=f,f=k.shift())if("*"===f)f=i;else if("*"!==i&&i!==f){if(g=j[i+" "+f]||j["* "+f],!g)f
 or(e in j)if(h=e.split(" "),h[1]===f&&(g=j[i+" "+h[0]]||j["* "+h[0]])){g===!0?g=j[e]:j[e]!==!0&&(f=h[0],k.unshift(h[1]));break}if(g!==!0)if(g&&a["throws"])b=g(b);else try{b=g(b)}catch(l){return{state:"parsererror",error:g?l:"No conversion from "+i+" to "+f}}}return{state:"success",data:b}}m.extend({active:0,lastModified:{},etag:{},ajaxSettings:{url:zc,type:"GET",isLocal:Dc.test(yc[1]),global:!0,processData:!0,async:!0,contentType:"application/x-www-form-urlencoded; charset=UTF-8",accepts:{"*":Jc,text:"text/plain",html:"text/html",xml:"application/xml, text/xml",json:"application/json, text/javascript"},contents:{xml:/xml/,html:/html/,json:/json/},responseFields:{xml:"responseXML",text:"responseText",json:"responseJSON"},converters:{"* text":String,"text html":!0,"text json":m.parseJSON,"text xml":m.parseXML},flatOptions:{url:!0,context:!0}},ajaxSetup:function(a,b){return b?Nc(Nc(a,m.ajaxSettings),b):Nc(m.ajaxSettings,a)},ajaxPrefilter:Lc(Hc),ajaxTransport:Lc(Ic),ajax:function(a,b){"
 object"==typeof a&&(b=a,a=void 0),b=b||{};var c,d,e,f,g,h,i,j,k=m.ajaxSetup({},b),l=k.context||k,n=k.context&&(l.nodeType||l.jquery)?m(l):m.event,o=m.Deferred(),p=m.Callbacks("once memory"),q=k.statusCode||{},r={},s={},t=0,u="canceled",v={readyState:0,getResponseHeader:function(a){var b;if(2===t){if(!j){j={};while(b=Cc.exec(f))j[b[1].toLowerCase()]=b[2]}b=j[a.toLowerCase()]}return null==b?null:b},getAllResponseHeaders:function(){return 2===t?f:null},setRequestHeader:function(a,b){var c=a.toLowerCase();return t||(a=s[c]=s[c]||a,r[a]=b),this},overrideMimeType:function(a){return t||(k.mimeType=a),this},statusCode:function(a){var b;if(a)if(2>t)for(b in a)q[b]=[q[b],a[b]];else v.always(a[v.status]);return this},abort:function(a){var b=a||u;return i&&i.abort(b),x(0,b),this}};if(o.promise(v).complete=p.add,v.success=v.done,v.error=v.fail,k.url=((a||k.url||zc)+"").replace(Ac,"").replace(Fc,yc[1]+"//"),k.type=b.method||b.type||k.method||k.type,k.dataTypes=m.trim(k.dataType||"*").toLowerCase(
 ).match(E)||[""],null==k.crossDomain&&(c=Gc.exec(k.url.toLowerCase()),k.crossDomain=!(!c||c[1]===yc[1]&&c[2]===yc[2]&&(c[3]||("http:"===c[1]?"80":"443"))===(yc[3]||("http:"===yc[1]?"80":"443")))),k.data&&k.processData&&"string"!=typeof k.data&&(k.data=m.param(k.data,k.traditional)),Mc(Hc,k,b,v),2===t)return v;h=k.global,h&&0===m.active++&&m.event.trigger("ajaxStart"),k.type=k.type.toUpperCase(),k.hasContent=!Ec.test(k.type),e=k.url,k.hasContent||(k.data&&(e=k.url+=(wc.test(e)?"&":"?")+k.data,delete k.data),k.cache===!1&&(k.url=Bc.test(e)?e.replace(Bc,"$1_="+vc++):e+(wc.test(e)?"&":"?")+"_="+vc++)),k.ifModified&&(m.lastModified[e]&&v.setRequestHeader("If-Modified-Since",m.lastModified[e]),m.etag[e]&&v.setRequestHeader("If-None-Match",m.etag[e])),(k.data&&k.hasContent&&k.contentType!==!1||b.contentType)&&v.setRequestHeader("Content-Type",k.contentType),v.setRequestHeader("Accept",k.dataTypes[0]&&k.accepts[k.dataTypes[0]]?k.accepts[k.dataTypes[0]]+("*"!==k.dataTypes[0]?", "+Jc+"; q=0.0
 1":""):k.accepts["*"]);for(d in k.headers)v.setRequestHeader(d,k.headers[d]);if(k.beforeSend&&(k.beforeSend.call(l,v,k)===!1||2===t))return v.abort();u="abort";for(d in{success:1,error:1,complete:1})v[d](k[d]);if(i=Mc(Ic,k,b,v)){v.readyState=1,h&&n.trigger("ajaxSend",[v,k]),k.async&&k.timeout>0&&(g=setTimeout(function(){v.abort("timeout")},k.timeout));try{t=1,i.send(r,x)}catch(w){if(!(2>t))throw w;x(-1,w)}}else x(-1,"No Transport");function x(a,b,c,d){var j,r,s,u,w,x=b;2!==t&&(t=2,g&&clearTimeout(g),i=void 0,f=d||"",v.readyState=a>0?4:0,j=a>=200&&300>a||304===a,c&&(u=Oc(k,v,c)),u=Pc(k,u,v,j),j?(k.ifModified&&(w=v.getResponseHeader("Last-Modified"),w&&(m.lastModified[e]=w),w=v.getResponseHeader("etag"),w&&(m.etag[e]=w)),204===a||"HEAD"===k.type?x="nocontent":304===a?x="notmodified":(x=u.state,r=u.data,s=u.error,j=!s)):(s=x,(a||!x)&&(x="error",0>a&&(a=0))),v.status=a,v.statusText=(b||x)+"",j?o.resolveWith(l,[r,x,v]):o.rejectWith(l,[v,x,s]),v.statusCode(q),q=void 0,h&&n.trigger(j?"ajax
 Success":"ajaxError",[v,k,j?r:s]),p.fireWith(l,[v,x]),h&&(n.trigger("ajaxComplete",[v,k]),--m.active||m.event.trigger("ajaxStop")))}return v},getJSON:function(a,b,c){return m.get(a,b,c,"json")},getScript:function(a,b){return m.get(a,void 0,b,"script")}}),m.each(["get","post"],function(a,b){m[b]=function(a,c,d,e){return m.isFunction(c)&&(e=e||d,d=c,c=void 0),m.ajax({url:a,type:b,dataType:e,data:c,success:d})}}),m.each(["ajaxStart","ajaxStop","ajaxComplete","ajaxError","ajaxSuccess","ajaxSend"],function(a,b){m.fn[b]=function(a){return this.on(b,a)}}),m._evalUrl=function(a){return m.ajax({url:a,type:"GET",dataType:"script",async:!1,global:!1,"throws":!0})},m.fn.extend({wrapAll:function(a){if(m.isFunction(a))return this.each(function(b){m(this).wrapAll(a.call(this,b))});if(this[0]){var b=m(a,this[0].ownerDocument).eq(0).clone(!0);this[0].parentNode&&b.insertBefore(this[0]),b.map(function(){var a=this;while(a.firstChild&&1===a.firstChild.nodeType)a=a.firstChild;return a}).append(this)}re
 turn this},wrapInner:function(a){return this.each(m.isFunction(a)?function(b){m(this).wrapInner(a.call(this,b))}:function(){var b=m(this),c=b.contents();c.length?c.wrapAll(a):b.append(a)})},wrap:function(a){var b=m.isFunction(a);return this.each(function(c){m(this).wrapAll(b?a.call(this,c):a)})},unwrap:function(){return this.parent().each(function(){m.nodeName(this,"body")||m(this).replaceWith(this.childNodes)}).end()}}),m.expr.filters.hidden=function(a){return a.offsetWidth<=0&&a.offsetHeight<=0||!k.reliableHiddenOffsets()&&"none"===(a.style&&a.style.display||m.css(a,"display"))},m.expr.filters.visible=function(a){return!m.expr.filters.hidden(a)};var Qc=/%20/g,Rc=/\[\]$/,Sc=/\r?\n/g,Tc=/^(?:submit|button|image|reset|file)$/i,Uc=/^(?:input|select|textarea|keygen)/i;function Vc(a,b,c,d){var e;if(m.isArray(b))m.each(b,function(b,e){c||Rc.test(a)?d(a,e):Vc(a+"["+("object"==typeof e?b:"")+"]",e,c,d)});else if(c||"object"!==m.type(b))d(a,b);else for(e in b)Vc(a+"["+e+"]",b[e],c,d)}m.para
 m=function(a,b){var c,d=[],e=function(a,b){b=m.isFunction(b)?b():null==b?"":b,d[d.length]=encodeURIComponent(a)+"="+encodeURIComponent(b)};if(void 0===b&&(b=m.ajaxSettings&&m.ajaxSettings.traditional),m.isArray(a)||a.jquery&&!m.isPlainObject(a))m.each(a,function(){e(this.name,this.value)});else for(c in a)Vc(c,a[c],b,e);return d.join("&").replace(Qc,"+")},m.fn.extend({serialize:function(){return m.param(this.serializeArray())},serializeArray:function(){return this.map(function(){var a=m.prop(this,"elements");return a?m.makeArray(a):this}).filter(function(){var a=this.type;return this.name&&!m(this).is(":disabled")&&Uc.test(this.nodeName)&&!Tc.test(a)&&(this.checked||!W.test(a))}).map(function(a,b){var c=m(this).val();return null==c?null:m.isArray(c)?m.map(c,function(a){return{name:b.name,value:a.replace(Sc,"\r\n")}}):{name:b.name,value:c.replace(Sc,"\r\n")}}).get()}}),m.ajaxSettings.xhr=void 0!==a.ActiveXObject?function(){return!this.isLocal&&/^(get|post|head|p

<TRUNCATED>

[15/29] samza git commit: SAMZA-1898: New UI layout for the Samza website

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/samza/blob/b8058af0/docs/fonts/ionicons.ttf
----------------------------------------------------------------------
diff --git a/docs/fonts/ionicons.ttf b/docs/fonts/ionicons.ttf
new file mode 100644
index 0000000..67bd842
Binary files /dev/null and b/docs/fonts/ionicons.ttf differ

http://git-wip-us.apache.org/repos/asf/samza/blob/b8058af0/docs/fonts/ionicons.woff
----------------------------------------------------------------------
diff --git a/docs/fonts/ionicons.woff b/docs/fonts/ionicons.woff
new file mode 100644
index 0000000..ec1c1f8
Binary files /dev/null and b/docs/fonts/ionicons.woff differ

http://git-wip-us.apache.org/repos/asf/samza/blob/b8058af0/docs/fonts/ionicons.woff2
----------------------------------------------------------------------
diff --git a/docs/fonts/ionicons.woff2 b/docs/fonts/ionicons.woff2
new file mode 100644
index 0000000..4233951
Binary files /dev/null and b/docs/fonts/ionicons.woff2 differ

http://git-wip-us.apache.org/repos/asf/samza/blob/b8058af0/docs/img/favicon/apple-touch-icon-114x114.png
----------------------------------------------------------------------
diff --git a/docs/img/favicon/apple-touch-icon-114x114.png b/docs/img/favicon/apple-touch-icon-114x114.png
new file mode 100644
index 0000000..1ed1c8b
Binary files /dev/null and b/docs/img/favicon/apple-touch-icon-114x114.png differ

http://git-wip-us.apache.org/repos/asf/samza/blob/b8058af0/docs/img/favicon/apple-touch-icon-120x120.png
----------------------------------------------------------------------
diff --git a/docs/img/favicon/apple-touch-icon-120x120.png b/docs/img/favicon/apple-touch-icon-120x120.png
new file mode 100644
index 0000000..dd31736
Binary files /dev/null and b/docs/img/favicon/apple-touch-icon-120x120.png differ

http://git-wip-us.apache.org/repos/asf/samza/blob/b8058af0/docs/img/favicon/apple-touch-icon-144x144.png
----------------------------------------------------------------------
diff --git a/docs/img/favicon/apple-touch-icon-144x144.png b/docs/img/favicon/apple-touch-icon-144x144.png
new file mode 100644
index 0000000..3de46b4
Binary files /dev/null and b/docs/img/favicon/apple-touch-icon-144x144.png differ

http://git-wip-us.apache.org/repos/asf/samza/blob/b8058af0/docs/img/favicon/apple-touch-icon-152x152.png
----------------------------------------------------------------------
diff --git a/docs/img/favicon/apple-touch-icon-152x152.png b/docs/img/favicon/apple-touch-icon-152x152.png
new file mode 100644
index 0000000..7859ee8
Binary files /dev/null and b/docs/img/favicon/apple-touch-icon-152x152.png differ

http://git-wip-us.apache.org/repos/asf/samza/blob/b8058af0/docs/img/favicon/apple-touch-icon-57x57.png
----------------------------------------------------------------------
diff --git a/docs/img/favicon/apple-touch-icon-57x57.png b/docs/img/favicon/apple-touch-icon-57x57.png
new file mode 100644
index 0000000..39b9c06
Binary files /dev/null and b/docs/img/favicon/apple-touch-icon-57x57.png differ

http://git-wip-us.apache.org/repos/asf/samza/blob/b8058af0/docs/img/favicon/apple-touch-icon-60x60.png
----------------------------------------------------------------------
diff --git a/docs/img/favicon/apple-touch-icon-60x60.png b/docs/img/favicon/apple-touch-icon-60x60.png
new file mode 100644
index 0000000..ce429eb
Binary files /dev/null and b/docs/img/favicon/apple-touch-icon-60x60.png differ

http://git-wip-us.apache.org/repos/asf/samza/blob/b8058af0/docs/img/favicon/apple-touch-icon-72x72.png
----------------------------------------------------------------------
diff --git a/docs/img/favicon/apple-touch-icon-72x72.png b/docs/img/favicon/apple-touch-icon-72x72.png
new file mode 100644
index 0000000..5adc4b1
Binary files /dev/null and b/docs/img/favicon/apple-touch-icon-72x72.png differ

http://git-wip-us.apache.org/repos/asf/samza/blob/b8058af0/docs/img/favicon/apple-touch-icon-76x76.png
----------------------------------------------------------------------
diff --git a/docs/img/favicon/apple-touch-icon-76x76.png b/docs/img/favicon/apple-touch-icon-76x76.png
new file mode 100644
index 0000000..955996a
Binary files /dev/null and b/docs/img/favicon/apple-touch-icon-76x76.png differ

http://git-wip-us.apache.org/repos/asf/samza/blob/b8058af0/docs/img/favicon/favicon-128.png
----------------------------------------------------------------------
diff --git a/docs/img/favicon/favicon-128.png b/docs/img/favicon/favicon-128.png
new file mode 100644
index 0000000..f3c8921
Binary files /dev/null and b/docs/img/favicon/favicon-128.png differ

http://git-wip-us.apache.org/repos/asf/samza/blob/b8058af0/docs/img/favicon/favicon-16x16.png
----------------------------------------------------------------------
diff --git a/docs/img/favicon/favicon-16x16.png b/docs/img/favicon/favicon-16x16.png
new file mode 100644
index 0000000..931f10c
Binary files /dev/null and b/docs/img/favicon/favicon-16x16.png differ

http://git-wip-us.apache.org/repos/asf/samza/blob/b8058af0/docs/img/favicon/favicon-196x196.png
----------------------------------------------------------------------
diff --git a/docs/img/favicon/favicon-196x196.png b/docs/img/favicon/favicon-196x196.png
new file mode 100644
index 0000000..6927b70
Binary files /dev/null and b/docs/img/favicon/favicon-196x196.png differ

http://git-wip-us.apache.org/repos/asf/samza/blob/b8058af0/docs/img/favicon/favicon-32x32.png
----------------------------------------------------------------------
diff --git a/docs/img/favicon/favicon-32x32.png b/docs/img/favicon/favicon-32x32.png
new file mode 100644
index 0000000..0d3c34d
Binary files /dev/null and b/docs/img/favicon/favicon-32x32.png differ

http://git-wip-us.apache.org/repos/asf/samza/blob/b8058af0/docs/img/favicon/favicon-96x96.png
----------------------------------------------------------------------
diff --git a/docs/img/favicon/favicon-96x96.png b/docs/img/favicon/favicon-96x96.png
new file mode 100644
index 0000000..92ceb89
Binary files /dev/null and b/docs/img/favicon/favicon-96x96.png differ

http://git-wip-us.apache.org/repos/asf/samza/blob/b8058af0/docs/img/favicon/favicon.ico
----------------------------------------------------------------------
diff --git a/docs/img/favicon/favicon.ico b/docs/img/favicon/favicon.ico
new file mode 100644
index 0000000..1cd6d1a
Binary files /dev/null and b/docs/img/favicon/favicon.ico differ

http://git-wip-us.apache.org/repos/asf/samza/blob/b8058af0/docs/img/favicon/mstile-144x144.png
----------------------------------------------------------------------
diff --git a/docs/img/favicon/mstile-144x144.png b/docs/img/favicon/mstile-144x144.png
new file mode 100644
index 0000000..3de46b4
Binary files /dev/null and b/docs/img/favicon/mstile-144x144.png differ

http://git-wip-us.apache.org/repos/asf/samza/blob/b8058af0/docs/img/favicon/mstile-150x150.png
----------------------------------------------------------------------
diff --git a/docs/img/favicon/mstile-150x150.png b/docs/img/favicon/mstile-150x150.png
new file mode 100644
index 0000000..3cb26d7
Binary files /dev/null and b/docs/img/favicon/mstile-150x150.png differ

http://git-wip-us.apache.org/repos/asf/samza/blob/b8058af0/docs/img/favicon/mstile-310x150.png
----------------------------------------------------------------------
diff --git a/docs/img/favicon/mstile-310x150.png b/docs/img/favicon/mstile-310x150.png
new file mode 100644
index 0000000..5193749
Binary files /dev/null and b/docs/img/favicon/mstile-310x150.png differ

http://git-wip-us.apache.org/repos/asf/samza/blob/b8058af0/docs/img/favicon/mstile-310x310.png
----------------------------------------------------------------------
diff --git a/docs/img/favicon/mstile-310x310.png b/docs/img/favicon/mstile-310x310.png
new file mode 100644
index 0000000..0380569
Binary files /dev/null and b/docs/img/favicon/mstile-310x310.png differ

http://git-wip-us.apache.org/repos/asf/samza/blob/b8058af0/docs/img/favicon/mstile-70x70.png
----------------------------------------------------------------------
diff --git a/docs/img/favicon/mstile-70x70.png b/docs/img/favicon/mstile-70x70.png
new file mode 100644
index 0000000..f3c8921
Binary files /dev/null and b/docs/img/favicon/mstile-70x70.png differ

http://git-wip-us.apache.org/repos/asf/samza/blob/b8058af0/docs/img/feather.gif
----------------------------------------------------------------------
diff --git a/docs/img/feather.gif b/docs/img/feather.gif
new file mode 100644
index 0000000..0db293e
Binary files /dev/null and b/docs/img/feather.gif differ

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

http://git-wip-us.apache.org/repos/asf/samza/blob/b8058af0/docs/img/samza-just-logo.svg
----------------------------------------------------------------------
diff --git a/docs/img/samza-just-logo.svg b/docs/img/samza-just-logo.svg
new file mode 100644
index 0000000..f84368f
--- /dev/null
+++ b/docs/img/samza-just-logo.svg
@@ -0,0 +1,128 @@
+<?xml version="1.0" encoding="UTF-8" standalone="no"?>
+<!-- Created with Inkscape (http://www.inkscape.org/) -->
+
+<svg
+   xmlns:dc="http://purl.org/dc/elements/1.1/"
+   xmlns:cc="http://creativecommons.org/ns#"
+   xmlns:rdf="http://www.w3.org/1999/02/22-rdf-syntax-ns#"
+   xmlns:svg="http://www.w3.org/2000/svg"
+   xmlns="http://www.w3.org/2000/svg"
+   xmlns:sodipodi="http://sodipodi.sourceforge.net/DTD/sodipodi-0.dtd"
+   xmlns:inkscape="http://www.inkscape.org/namespaces/inkscape"
+   id="svg2"
+   version="1.1"
+   inkscape:version="0.91 r13725"
+   xml:space="preserve"
+   width="1250"
+   height="990"
+   viewBox="0 0 1250 990"
+   sodipodi:docname="Samza-just-logo.svg"
+   inkscape:export-filename="/home/nsolis/Documents/samza-logo/Samza-logo.png"
+   inkscape:export-xdpi="286.15891"
+   inkscape:export-ydpi="286.15891"><metadata
+     id="metadata8"><rdf:RDF><cc:Work
+         rdf:about=""><dc:format>image/svg+xml</dc:format><dc:type
+           rdf:resource="http://purl.org/dc/dcmitype/StillImage" /><dc:title></dc:title></cc:Work></rdf:RDF></metadata><defs
+     id="defs6"><clipPath
+       clipPathUnits="userSpaceOnUse"
+       id="clipPath16"><path
+         d="M 0,792 1000,792 1000,0 0,0 0,792 Z"
+         id="path18"
+         inkscape:connector-curvature="0" /></clipPath></defs><sodipodi:namedview
+     pagecolor="#ffffff"
+     bordercolor="#666666"
+     borderopacity="1"
+     objecttolerance="10"
+     gridtolerance="10"
+     guidetolerance="10"
+     inkscape:pageopacity="0"
+     inkscape:pageshadow="2"
+     inkscape:window-width="1920"
+     inkscape:window-height="1099"
+     id="namedview4"
+     showgrid="false"
+     inkscape:zoom="0.68"
+     inkscape:cx="402.1077"
+     inkscape:cy="492.05882"
+     inkscape:window-x="1920"
+     inkscape:window-y="28"
+     inkscape:window-maximized="1"
+     inkscape:current-layer="g14" /><g
+     id="g10"
+     inkscape:groupmode="layer"
+     inkscape:label="Samza"
+     transform="matrix(1.25,0,0,-1.25,0,990)"><g
+       id="g12"><g
+         id="g14"
+         clip-path="url(#clipPath16)"><g
+           id="g3370"
+           transform="matrix(0.28626938,0,0,1,48.272298,0)"
+           inkscape:export-xdpi="286.15891"
+           inkscape:export-ydpi="286.15891"
+           style="fill:#000000"><rect
+             transform="scale(1,-1)"
+             y="-491.76471"
+             x="54.44706"
+             height="257.64706"
+             width="908.23523"
+             id="rect3368"
+             style="fill:#000000;fill-opacity:1"
+             inkscape:export-xdpi="286.15891"
+             inkscape:export-ydpi="286.15891" /></g><g
+           id="g20"
+           transform="translate(150.27,319.0088)"><path
+             d="M 0,0 C 5.637,-4.369 -0.678,-12.957 -6.524,-8.922 -12.666,-4.668 -5.984,4.638 0,0 m 53.821,105.424 6.634,12.562 C -12.52,136.946 -63.206,49.347 -14.854,-4.955 c 0.542,-10.587 14.024,-14.858 20.54,-6.451 6.837,8.823 -1.729,21.207 -12.369,18.031 -35.374,44.359 3.478,110.341 60.504,98.799"
+             style="fill:#ee3538;fill-opacity:1;fill-rule:nonzero;stroke:none"
+             id="path22"
+             inkscape:connector-curvature="0" /></g><g
+           id="g24"
+           transform="translate(94.2788,344.2451)"><path
+             d="M 0,0 C -2.437,1.889 -2.825,5.382 -0.981,7.761 2.24,11.918 8.898,9.63 8.917,4.39 8.935,-0.187 3.651,-2.83 0,0 m 147.521,109.731 c 4.542,-11.997 -13.571,-18.745 -18.038,-6.807 -4.478,11.975 13.541,18.665 18.038,6.807 M 8.671,14.566 c -3.663,61.302 54.446,108.122 114.279,90.547 0.934,-12.123 14.692,-18.519 24.549,-11.58 7.565,-5.143 14.518,-11.595 20.531,-19.408 l 12.898,5.64 c -7.707,10.728 -16.829,19.345 -26.824,25.959 0.506,13.757 -15.753,21.383 -25.986,12.236 C 59.648,138.863 -10.435,85.858 -4.95,12.264 -15.906,0.651 2.716,-15.239 12.466,-2.662 16.835,2.977 15.072,11.242 8.671,14.566"
+             style="fill:#606163;fill-opacity:1;fill-rule:nonzero;stroke:none"
+             id="path26"
+             inkscape:connector-curvature="0" /></g><g
+           id="g28"
+           transform="translate(291.4043,375.8301)"
+           style="fill:#505050;fill-opacity:1"><path
+             d="m 0,0 c -3.25,-4.195 -9.984,-1.818 -9.897,3.482 0.072,4.526 5.303,7.078 8.914,4.279 C 1.446,5.877 1.854,2.392 0,0 m -148.224,-102.441 c -5.005,12.042 13.441,19.216 17.938,7.066 4.311,-11.65 -13.103,-18.692 -17.938,-7.066 M 2.66,12.463 C -9.658,22.01 -23.788,2.779 -10.849,-6.092 c 3.918,-60.597 -52.936,-107.987 -112.922,-91.371 -1.004,12.259 -15.045,18.545 -24.847,11.275 -7.968,5.251 -15.277,11.94 -21.562,20.104 l -12.9,-5.64 c 8.039,-11.192 17.63,-20.085 28.154,-26.8 -0.167,-13.64 16.001,-20.913 26.075,-11.778 67.955,-19.65 136.232,32.567 131.735,104.793 5.733,4.685 5.6,13.458 -0.224,17.972"
+             style="fill:#505050;fill-opacity:1;fill-rule:nonzero;stroke:none"
+             id="path30"
+             inkscape:connector-curvature="0" /></g><g
+           id="g32"
+           transform="translate(205.458,409.9463)"><path
+             d="M 0,0 C 4.769,1.514 9.832,-0.853 11.775,-5.354 13.964,-10.413 11.499,-16.249 6.353,-18.2 -5.929,-22.856 -12.458,-3.953 0,0 m -12.492,-11.618 c 1.735,-11.037 14.157,-16.747 23.676,-10.816 7.687,-8.096 11.574,-19.89 8.146,-32.491 -4.734,-17.39 -20.747,-26.301 -36.13,-25.014 l 0,-13.767 c 45.393,-3.116 68.625,53.433 35.29,83.281 C 19.631,4.126 1.91,12.145 -8.248,1.716 -19.569,3.093 -31.569,0.479 -42.124,-7.29 c -29.316,-21.583 -23.57,-63.799 4.07,-79.999 l 0,17.298 c -1.725,1.771 -3.313,3.771 -4.713,6.028 -14.544,23.445 3.289,53.32 30.275,52.345"
+             style="fill:#606163;fill-opacity:1;fill-rule:nonzero;stroke:none"
+             id="path34"
+             inkscape:connector-curvature="0" /></g><g
+           id="g36"
+           transform="translate(188.6582,302.4775)"><path
+             d="M 0,0 C 3.643,-0.222 7.187,-0.115 10.625,0.27 8.372,0.1 2.535,-0.224 -3.835,0.338 l 0,-0.001 C -2.544,0.182 -1.269,0.077 0,0"
+             style="fill:#ee3538;fill-opacity:1;fill-rule:nonzero;stroke:none"
+             id="path38"
+             inkscape:connector-curvature="0" /></g><g
+           id="g40"
+           transform="translate(234.3667,409.6387)"><path
+             d="M 0,0 C -5.785,4.482 1.002,13.219 6.775,8.744 12.56,4.261 5.777,-4.479 0,0 m 14.81,3.208 c 1.155,11.444 -13.45,17.29 -20.498,8.197 -6.443,-8.314 0.811,-20.163 11.089,-18.336 31.339,-38.922 5.149,-94.856 -40.484,-99.961 0.63,0.048 0.997,0.084 0.997,0.084 0,0 -0.367,-0.036 -0.997,-0.084 -3.438,-0.384 -6.983,-0.491 -10.625,-0.269 -1.269,0.077 -2.545,0.182 -3.835,0.337 l 0,0.001 c -4.654,0.411 -9.578,1.297 -13.585,3.067 l 0,-13.611 c 5.899,-2.435 12.844,-3.246 14.983,-3.447 0.815,-0.07 1.628,-0.13 2.437,-0.172 62.532,-3.258 104.522,72.613 60.518,124.194"
+             style="fill:#ee3538;fill-opacity:1;fill-rule:nonzero;stroke:none"
+             id="path42"
+             inkscape:connector-curvature="0" /></g><path
+           style="fill:#505050;fill-opacity:1"
+           d="m 237.03125,594.18448 c -0.42969,-0.42994 -0.78125,-4.2192 -0.78125,-8.42059 l 0,-7.63889 3.94781,0 c 2.1713,0 6.70652,-0.71855 10.07827,-1.59678 25.31744,-6.59436 38.89378,-32.50962 29.78495,-56.85515 -2.92176,-7.80912 -8.14704,-14.80575 -10.35148,-13.8606 -0.92879,0.39822 -3.38085,1.20763 -5.44901,1.79869 -7.9813,2.28098 -19.02821,-4.01963 -21.80627,-12.43722 -1.0398,-3.15062 -1.55377,-3.50727 -5.05432,-3.50727 -4.90537,0 -14.09268,2.66458 -18.85828,5.46944 -8.98466,5.28804 -16.33289,14.48717 -19.83032,24.82527 -4.26406,12.60415 -1.28178,29.7618 6.81782,39.22436 3.60732,4.21435 3.6375,4.33247 3.6375,14.24026 0,6.68605 -0.38632,9.99067 -1.16793,9.99067 -2.52517,0 -17.35701,-16.26163 -20.47639,-22.4503 -9.9887,-19.81702 -9.3421,-42.55887 1.68783,-59.36379 11.31768,-17.24332 29.2723,-27.45018 48.75393,-27.71568 7.29315,-0.0994 9.18213,-0.49766 11.91164,-2.51144 10.88052,-8.02739 29.08258,0.18698 29.08258,13.12463 0,2.44339 1.29526,4.72346 5.123,9.01809 6.20157,6.95799 1
 0.92932,15.56316 13.07368,23.79594 2.04347,7.8454 2.04486,22.27099 0.003,30.11057 -5.37759,20.64593 -23.39328,38.84149 -42.67877,43.10484 -7.72068,1.70678 -16.55833,2.54504 -17.44791,1.65495 l -2.1e-4,0 z m 28.07925,-93.63667 c 7.23351,-2.07487 10.04965,-13.09915 4.9073,-19.21049 -7.50536,-8.9196 -21.94001,-3.81432 -21.69183,7.67202 0.12094,5.59736 3.50802,10.61813 8.03709,11.91361 3.29536,0.94259 4.30435,0.89932 8.74744,-0.37514 z"
+           id="path4192"
+           inkscape:connector-curvature="0"
+           transform="matrix(0.8,0,0,-0.8,0,792)" /><path
+           style="fill:#505050;fill-opacity:1"
+           d="m 118.27138,567.23632 c -4.81493,-1.68764 -6.72274,-3.2206 -8.65256,-6.95248 -2.05659,-3.977 -1.86396,-9.14324 0.48453,-12.99456 1.60357,-2.62973 1.98802,-5.39988 2.06434,-14.87435 0.16051,-19.9286 3.90673,-35.94542 12.59237,-53.83826 17.72653,-36.5175 53.69611,-63.12367 94.34705,-69.78711 14.0362,-2.3008 38.10199,-1.53492 49.95539,1.5898 l 8.33333,2.19678 4.05243,-2.27021 c 7.42219,-4.15798 17.56402,-2.66937 23.2416,3.4114 2.62105,2.80716 5.51847,9.20909 5.51847,12.19317 0,1.24195 2.10156,3.52889 5.46013,5.94178 7.34407,5.27619 15.55833,13.08867 22.28814,21.19797 l 5.557,6.69608 -7.53805,3.29815 -7.53805,3.29816 -6.92832,-7.44441 c -3.81059,-4.09442 -9.54756,-9.46512 -12.74884,-11.93489 l -5.82051,-4.49048 -4.19933,1.68023 c -8.09239,3.23794 -15.49576,1.66048 -21.59258,-4.60079 -1.85511,-1.90514 -3.66195,-5.00458 -4.01521,-6.88765 -0.85759,-4.57128 -1.7265,-5.13079 -10.84104,-6.98075 -24.51588,-4.97596 -47.436,-2.28574 -69.51413,8.15912 -13.6512,6.4582 -24.01808,13.99
 605 -34.4535,25.05143 -18.64604,19.75381 -28.46612,42.93514 -30.10076,71.05608 l -0.62081,10.67971 3.70354,3.59682 c 10.62817,10.32195 0.71147,27.82731 -13.03463,23.00926 z m 8.37184,-7.39955 c 3.25521,-2.28004 3.25521,-9.06017 0,-11.34021 -5.10327,-3.57447 -11.57378,-0.4045 -11.57378,5.67011 0,6.0746 6.47051,9.24457 11.57378,5.6701 z M 298.90025,436.35135 c 6.25266,-4.45229 6.04252,-13.79313 -0.43288,-19.24182 -2.79791,-2.35429 -3.98935,-2.68105 -8.27435,-2.26931 -10.34815,0.99432 -14.43538,9.66147 -9.18369,19.47436 2.81646,5.2626 11.90723,6.29753 17.89092,2.03677 z"
+           id="path4194"
+           inkscape:connector-curvature="0"
+           transform="matrix(0.8,0,0,-0.8,0,792)" /><path
+           style="fill:#ed1c24;fill-opacity:1"
+           d="m 231.5625,628.52654 c -1.71875,-0.24803 -6.28906,-1.19347 -10.15625,-2.10097 l -7.03125,-1.64999 0,-8.33257 c 0,-6.81762 0.27223,-8.2281 1.49735,-7.75798 6.25262,2.39936 12.17881,3.16538 24.54432,3.17259 12.51579,0.007 14.97906,-0.29985 22.39583,-2.79273 20.63492,-6.93569 36.20948,-20.46462 45.8111,-39.79406 6.29736,-12.6775 8.23176,-21.1494 8.20563,-35.9375 -0.0193,-10.92248 -0.33863,-12.89983 -3.62016,-22.41635 -1.99566,-5.78744 -5.65243,-13.56044 -8.2083,-17.44791 -4.01033,-6.09972 -5.02051,-7.01124 -7.77378,-7.01463 -6.69868,-0.008 -14.08999,-7.43461 -14.0981,-14.16498 -0.005,-4.25293 3.8294,-9.84754 8.35021,-12.18306 4.09114,-2.11354 9.1445,-1.79095 13.15997,0.84009 3.17243,2.07865 6.60188,7.5836 6.60704,10.6056 0.002,1.25469 2.79532,6.41257 6.20708,11.46196 11.16726,16.52752 15.86614,31.58711 15.86311,50.84012 -0.003,19.5561 -5.09674,36.41057 -15.87236,52.52031 -13.47326,20.14273 -33.35242,34.39899 -55.2642,39.63247 -8.43402,2.0144 -24.79058,3.36045 -30.61724,2.
 51959 z M 301.95513,477.16346 C 303.62981,475.48878 305,473.40555 305,472.53405 c 0,-2.81683 -4.61833,-7.16947 -7.6071,-7.16947 -7.58445,0 -10.22341,10.35633 -3.47198,13.62543 3.52499,1.70683 4.78796,1.4197 8.03421,-1.82655 z"
+           id="path4200"
+           inkscape:connector-curvature="0"
+           transform="matrix(0.8,0,0,-0.8,0,792)" /><path
+           style="fill:#ed1c24;fill-opacity:1"
+           d="m 176.55722,608.47635 c -3.41397,-2.251 -5.19458,-4.6579 -6.67488,-9.02262 -0.58766,-1.73274 -3.35628,-6.18587 -6.15249,-9.89583 -6.23576,-8.27351 -12.11964,-20.66772 -15.07044,-31.7454 -3.1998,-12.01243 -3.1873,-30.94026 0.0289,-43.75 8.06246,-32.11203 31.30535,-58.18768 60.81216,-68.22371 10.80888,-3.67638 25.53732,-5.67302 35.01673,-4.74699 8.3026,0.81106 17.7745,2.6021 17.7745,3.36098 0,0.30528 -1.65982,3.7137 -3.68848,7.57426 l -3.68848,7.01919 -5.94694,-0.97407 c -7.19905,-1.17916 -21.55976,-0.3054 -29.38444,1.78786 -14.86874,3.97769 -31.36479,15.09191 -40.7105,27.42869 -17.73211,23.40728 -21.47871,53.61195 -9.80403,79.03905 1.8873,4.11048 4.88016,9.57923 6.65081,12.15278 3.00743,4.37116 3.52711,4.67918 7.89425,4.67918 5.68424,0 10.57091,3.13196 12.64983,8.10751 1.87504,4.4876 1.82272,7.10035 -0.22875,11.42351 -3.32576,7.0085 -13.18254,9.93634 -19.47772,5.78561 z m 10.16694,-4.95675 c 2.3422,-0.89878 4.85846,-6.12171 4.04763,-8.40153 -2.00828,-5.64674 -8.66903,-7
 .50147 -12.34646,-3.43796 -5.32506,5.88413 0.85227,14.697 8.29883,11.83949 z"
+           id="path4202"
+           inkscape:connector-curvature="0"
+           transform="matrix(0.8,0,0,-0.8,0,792)" /></g></g></g></svg>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/samza/blob/b8058af0/docs/img/samza-logo-no-text.png
----------------------------------------------------------------------
diff --git a/docs/img/samza-logo-no-text.png b/docs/img/samza-logo-no-text.png
new file mode 100644
index 0000000..6e8daa4
Binary files /dev/null and b/docs/img/samza-logo-no-text.png differ

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

http://git-wip-us.apache.org/repos/asf/samza/blob/b8058af0/docs/img/samza-logo.svg
----------------------------------------------------------------------
diff --git a/docs/img/samza-logo.svg b/docs/img/samza-logo.svg
new file mode 100644
index 0000000..ebb16ab
--- /dev/null
+++ b/docs/img/samza-logo.svg
@@ -0,0 +1,174 @@
+<?xml version="1.0" encoding="UTF-8" standalone="no"?>
+<!-- Created with Inkscape (http://www.inkscape.org/) -->
+
+<svg
+   xmlns:dc="http://purl.org/dc/elements/1.1/"
+   xmlns:cc="http://creativecommons.org/ns#"
+   xmlns:rdf="http://www.w3.org/1999/02/22-rdf-syntax-ns#"
+   xmlns:svg="http://www.w3.org/2000/svg"
+   xmlns="http://www.w3.org/2000/svg"
+   xmlns:sodipodi="http://sodipodi.sourceforge.net/DTD/sodipodi-0.dtd"
+   xmlns:inkscape="http://www.inkscape.org/namespaces/inkscape"
+   id="svg2"
+   version="1.1"
+   inkscape:version="0.91 r13725"
+   xml:space="preserve"
+   width="1250"
+   height="990"
+   viewBox="0 0 1250 990"
+   sodipodi:docname="Samza-logo.svg"><metadata
+     id="metadata8"><rdf:RDF><cc:Work
+         rdf:about=""><dc:format>image/svg+xml</dc:format><dc:type
+           rdf:resource="http://purl.org/dc/dcmitype/StillImage" /><dc:title /></cc:Work></rdf:RDF></metadata><defs
+     id="defs6"><clipPath
+       clipPathUnits="userSpaceOnUse"
+       id="clipPath16"><path
+         d="M 0,792 1000,792 1000,0 0,0 0,792 Z"
+         id="path18"
+         inkscape:connector-curvature="0" /></clipPath></defs><sodipodi:namedview
+     pagecolor="#ffffff"
+     bordercolor="#666666"
+     borderopacity="1"
+     objecttolerance="10"
+     gridtolerance="10"
+     guidetolerance="10"
+     inkscape:pageopacity="0"
+     inkscape:pageshadow="2"
+     inkscape:window-width="1690"
+     inkscape:window-height="821"
+     id="namedview4"
+     showgrid="false"
+     inkscape:zoom="0.68"
+     inkscape:cx="677.10769"
+     inkscape:cy="495"
+     inkscape:window-x="2025"
+     inkscape:window-y="183"
+     inkscape:window-maximized="0"
+     inkscape:current-layer="g3370" /><g
+     id="g10"
+     inkscape:groupmode="layer"
+     inkscape:label="Samza"
+     transform="matrix(1.25,0,0,-1.25,0,990)"><g
+       id="g12"><g
+         id="g14"
+         clip-path="url(#clipPath16)"><g
+           id="g3370"><rect
+             transform="scale(1,-1)"
+             y="-491.76471"
+             x="54.44706"
+             height="257.64706"
+             width="908.23523"
+             id="rect3368"
+             style="fill:#ffffff;fill-opacity:1" /></g><g
+           id="g20"
+           transform="translate(150.27,319.0088)"><path
+             d="M 0,0 C 5.637,-4.369 -0.678,-12.957 -6.524,-8.922 -12.666,-4.668 -5.984,4.638 0,0 m 53.821,105.424 6.634,12.562 C -12.52,136.946 -63.206,49.347 -14.854,-4.955 c 0.542,-10.587 14.024,-14.858 20.54,-6.451 6.837,8.823 -1.729,21.207 -12.369,18.031 -35.374,44.359 3.478,110.341 60.504,98.799"
+             style="fill:#ee3538;fill-opacity:1;fill-rule:nonzero;stroke:none"
+             id="path22"
+             inkscape:connector-curvature="0" /></g><g
+           id="g24"
+           transform="translate(94.2788,344.2451)"><path
+             d="M 0,0 C -2.437,1.889 -2.825,5.382 -0.981,7.761 2.24,11.918 8.898,9.63 8.917,4.39 8.935,-0.187 3.651,-2.83 0,0 m 147.521,109.731 c 4.542,-11.997 -13.571,-18.745 -18.038,-6.807 -4.478,11.975 13.541,18.665 18.038,6.807 M 8.671,14.566 c -3.663,61.302 54.446,108.122 114.279,90.547 0.934,-12.123 14.692,-18.519 24.549,-11.58 7.565,-5.143 14.518,-11.595 20.531,-19.408 l 12.898,5.64 c -7.707,10.728 -16.829,19.345 -26.824,25.959 0.506,13.757 -15.753,21.383 -25.986,12.236 C 59.648,138.863 -10.435,85.858 -4.95,12.264 -15.906,0.651 2.716,-15.239 12.466,-2.662 16.835,2.977 15.072,11.242 8.671,14.566"
+             style="fill:#606163;fill-opacity:1;fill-rule:nonzero;stroke:none"
+             id="path26"
+             inkscape:connector-curvature="0" /></g><g
+           id="g28"
+           transform="translate(291.4043,375.8301)"
+           style="fill:#505050;fill-opacity:1"><path
+             d="m 0,0 c -3.25,-4.195 -9.984,-1.818 -9.897,3.482 0.072,4.526 5.303,7.078 8.914,4.279 C 1.446,5.877 1.854,2.392 0,0 m -148.224,-102.441 c -5.005,12.042 13.441,19.216 17.938,7.066 4.311,-11.65 -13.103,-18.692 -17.938,-7.066 M 2.66,12.463 C -9.658,22.01 -23.788,2.779 -10.849,-6.092 c 3.918,-60.597 -52.936,-107.987 -112.922,-91.371 -1.004,12.259 -15.045,18.545 -24.847,11.275 -7.968,5.251 -15.277,11.94 -21.562,20.104 l -12.9,-5.64 c 8.039,-11.192 17.63,-20.085 28.154,-26.8 -0.167,-13.64 16.001,-20.913 26.075,-11.778 67.955,-19.65 136.232,32.567 131.735,104.793 5.733,4.685 5.6,13.458 -0.224,17.972"
+             style="fill:#505050;fill-opacity:1;fill-rule:nonzero;stroke:none"
+             id="path30"
+             inkscape:connector-curvature="0" /></g><g
+           id="g32"
+           transform="translate(205.458,409.9463)"><path
+             d="M 0,0 C 4.769,1.514 9.832,-0.853 11.775,-5.354 13.964,-10.413 11.499,-16.249 6.353,-18.2 -5.929,-22.856 -12.458,-3.953 0,0 m -12.492,-11.618 c 1.735,-11.037 14.157,-16.747 23.676,-10.816 7.687,-8.096 11.574,-19.89 8.146,-32.491 -4.734,-17.39 -20.747,-26.301 -36.13,-25.014 l 0,-13.767 c 45.393,-3.116 68.625,53.433 35.29,83.281 C 19.631,4.126 1.91,12.145 -8.248,1.716 -19.569,3.093 -31.569,0.479 -42.124,-7.29 c -29.316,-21.583 -23.57,-63.799 4.07,-79.999 l 0,17.298 c -1.725,1.771 -3.313,3.771 -4.713,6.028 -14.544,23.445 3.289,53.32 30.275,52.345"
+             style="fill:#606163;fill-opacity:1;fill-rule:nonzero;stroke:none"
+             id="path34"
+             inkscape:connector-curvature="0" /></g><g
+           id="g36"
+           transform="translate(188.6582,302.4775)"><path
+             d="M 0,0 C 3.643,-0.222 7.187,-0.115 10.625,0.27 8.372,0.1 2.535,-0.224 -3.835,0.338 l 0,-0.001 C -2.544,0.182 -1.269,0.077 0,0"
+             style="fill:#ee3538;fill-opacity:1;fill-rule:nonzero;stroke:none"
+             id="path38"
+             inkscape:connector-curvature="0" /></g><g
+           id="g40"
+           transform="translate(234.3667,409.6387)"><path
+             d="M 0,0 C -5.785,4.482 1.002,13.219 6.775,8.744 12.56,4.261 5.777,-4.479 0,0 m 14.81,3.208 c 1.155,11.444 -13.45,17.29 -20.498,8.197 -6.443,-8.314 0.811,-20.163 11.089,-18.336 31.339,-38.922 5.149,-94.856 -40.484,-99.961 0.63,0.048 0.997,0.084 0.997,0.084 0,0 -0.367,-0.036 -0.997,-0.084 -3.438,-0.384 -6.983,-0.491 -10.625,-0.269 -1.269,0.077 -2.545,0.182 -3.835,0.337 l 0,0.001 c -4.654,0.411 -9.578,1.297 -13.585,3.067 l 0,-13.611 c 5.899,-2.435 12.844,-3.246 14.983,-3.447 0.815,-0.07 1.628,-0.13 2.437,-0.172 62.532,-3.258 104.522,72.613 60.518,124.194"
+             style="fill:#ee3538;fill-opacity:1;fill-rule:nonzero;stroke:none"
+             id="path42"
+             inkscape:connector-curvature="0" /></g><g
+           id="g46"
+           transform="translate(376.3662,361.7266)"><path
+             d="m 0,0 c -7.059,2.665 -12.353,6.312 -15.882,10.941 -3.529,4.625 -5.295,9.919 -5.295,15.883 l 0,3.06 c 0,10.823 3.019,18.977 9.059,24.47 6.038,5.49 16.273,8.235 30.707,8.235 3.607,0 7.331,-0.198 11.177,-0.588 3.841,-0.393 8.353,-1.139 13.529,-2.236 L 42.825,42.824 C 34.509,44.707 26.744,45.647 19.53,45.647 11.214,45.647 5.648,44.508 2.824,42.236 0,39.961 -1.411,36.471 -1.411,31.766 l 0,-1.882 c 0,-3.14 0.783,-5.572 2.352,-7.295 1.567,-1.729 4.39,-3.452 8.472,-5.176 L 31.53,8.235 c 6.901,-2.823 12.155,-6.67 15.765,-11.529 3.607,-4.864 5.412,-10.199 5.412,-16.001 l 0,-5.646 c 0,-10.824 -3.099,-19.023 -9.294,-24.589 -6.198,-5.567 -16.746,-8.354 -31.648,-8.354 -8.941,0 -18.904,1.257 -29.882,3.765 l 0.471,17.647 c 10.036,-2.511 19.841,-3.763 29.411,-3.763 8.472,0 14.118,1.057 16.942,3.175 2.823,2.119 4.235,5.53 4.235,10.237 l 0,3.293 c 0,3.606 -0.786,6.589 -2.352,8.941 -1.571,2.353 -4.159,4.235 -7.765,5.648 L 0,0 Z"
+             style="fill:#ee3538;fill-opacity:1;fill-rule:nonzero;stroke:none"
+             id="path48"
+             inkscape:connector-curvature="0" /></g><g
+           id="g50"
+           transform="translate(483.4258,321.7256)"><path
+             d="m 0,0 c 4.077,0 8.39,1.802 12.941,5.411 4.548,3.607 8.706,8.861 12.471,15.767 l 0,18.118 c -3.14,0.624 -6.512,1.058 -10.118,1.293 -3.61,0.235 -6.669,0.353 -9.177,0.353 -6.588,0 -11.61,-1.764 -15.058,-5.294 -3.453,-3.529 -5.177,-8.353 -5.177,-14.47 l 0,-2.589 c 0,-5.963 1.331,-10.552 4,-13.765 C -7.453,1.607 -4.081,0 0,0 m 45.412,-15.059 -19.059,0 0,18.354 C 17.096,-10.824 6.824,-17.883 -4.471,-17.883 c -8.941,0 -16.158,3.096 -21.647,9.295 -5.493,6.194 -8.236,14.783 -8.236,25.765 l 0,6.589 c 0,4.706 0.823,9.094 2.471,13.175 1.647,4.079 4,7.567 7.059,10.472 3.058,2.901 6.782,5.213 11.176,6.941 4.39,1.724 9.331,2.588 14.824,2.588 3.449,0 7.449,-0.315 12.001,-0.941 4.548,-0.629 8.625,-1.334 12.235,-2.118 l 0,12.471 c 0,7.06 -1.529,12 -4.588,14.824 -3.058,2.824 -7.806,4.235 -14.236,4.235 -2.353,0 -4.942,-0.117 -7.764,-0.352 C -4,84.825 -6.824,84.509 -9.648,84.12 -12.471,83.725 -15.257,83.333 -18,82.943 c -2.747,-0.393 -5.141,-0.747 -7.177,-1.06 l 0,17.178 c 5.176,0.784 10
 .743,1.566 16.706,2.354 5.96,0.782 10.978,1.175 15.059,1.175 13.489,0 23.331,-2.981 29.53,-8.942 6.195,-5.963 9.294,-15.609 9.294,-28.941 l 0,-79.766 z"
+             style="fill:#ee3538;fill-opacity:1;fill-rule:nonzero;stroke:none"
+             id="path52"
+             inkscape:connector-curvature="0" /></g><g
+           id="g54"
+           transform="translate(564.1318,306.667)"><path
+             d="m 0,0 0,114.825 18.823,0 0,-17.176 c 4.077,6.431 8.743,11.371 14.001,14.823 5.254,3.449 10.861,5.176 16.824,5.176 14.43,0 23.842,-7.215 28.236,-21.647 4.235,7.06 9.058,12.431 14.47,16.119 5.414,3.684 11.254,5.528 17.531,5.528 9.882,0 17.447,-3.375 22.705,-10.116 5.255,-6.748 7.884,-16.471 7.884,-29.178 l 0,-78.354 -20,0 0,75.53 c 0,8.472 -1.336,14.508 -4.001,18.118 -2.668,3.607 -6.198,5.413 -10.588,5.413 -4.395,0 -8.906,-1.729 -13.531,-5.177 -4.628,-3.452 -8.668,-8.158 -12.117,-14.118 l 0,-79.766 -20.001,0 0,75.53 c 0,8.472 -1.335,14.508 -4,18.118 -2.669,3.607 -6.199,5.413 -10.589,5.413 -4.392,0 -8.904,-1.729 -13.53,-5.177 C 27.489,90.432 23.448,85.726 20,79.766 L 20,0 0,0 Z"
+             style="fill:#ee3538;fill-opacity:1;fill-rule:nonzero;stroke:none"
+             id="path56"
+             inkscape:connector-curvature="0" /></g><g
+           id="g58"
+           transform="translate(731.6621,306.667)"><path
+             d="m 0,0 0,15.294 52.708,81.884 -51.531,0 0,17.647 75.531,0 0,-15.295 -52.473,-81.884 52.473,0 L 76.708,0 0,0 Z"
+             style="fill:#ee3538;fill-opacity:1;fill-rule:nonzero;stroke:none"
+             id="path60"
+             inkscape:connector-curvature="0" /></g><g
+           id="g62"
+           transform="translate(868.6045,321.7256)"><path
+             d="m 0,0 c 4.078,0 8.392,1.802 12.942,5.411 4.548,3.607 8.705,8.861 12.471,15.767 l 0,18.118 c -3.14,0.624 -6.511,1.058 -10.118,1.293 -3.61,0.235 -6.669,0.353 -9.176,0.353 -6.589,0 -11.611,-1.764 -15.06,-5.294 -3.453,-3.529 -5.176,-8.353 -5.176,-14.47 l 0,-2.589 c 0,-5.963 1.331,-10.552 4.001,-13.765 C -7.451,1.607 -4.081,0 0,0 m 45.413,-15.059 -19.059,0 0,18.354 c -9.257,-14.119 -19.53,-21.178 -30.825,-21.178 -8.941,0 -16.157,3.096 -21.647,9.295 -5.492,6.194 -8.235,14.783 -8.235,25.765 l 0,6.589 c 0,4.706 0.824,9.094 2.471,13.175 1.647,4.079 3.999,7.567 7.059,10.472 3.059,2.901 6.784,5.213 11.177,6.941 4.389,1.724 9.332,2.588 14.823,2.588 3.449,0 7.449,-0.315 12.001,-0.941 4.548,-0.629 8.626,-1.334 12.235,-2.118 l 0,12.471 c 0,7.06 -1.529,12 -4.589,14.824 -3.057,2.824 -7.804,4.235 -14.235,4.235 -2.353,0 -4.941,-0.117 -7.765,-0.352 -2.823,-0.236 -5.647,-0.552 -8.47,-0.941 -2.825,-0.395 -5.611,-0.787 -8.354,-1.177 -2.745,-0.393 -5.139,-0.747 -7.177,-1.06 l 0,17.178 c 5.1
 78,0.784 10.743,1.566 16.707,2.354 5.959,0.782 10.978,1.175 15.059,1.175 13.489,0 23.332,-2.981 29.529,-8.942 6.196,-5.963 9.295,-15.609 9.295,-28.941 l 0,-79.766 z"
+             style="fill:#ee3538;fill-opacity:1;fill-rule:nonzero;stroke:none"
+             id="path64"
+             inkscape:connector-curvature="0" /></g><path
+           style="fill:#505050;fill-opacity:1"
+           d="m 237.03125,594.18448 c -0.42969,-0.42994 -0.78125,-4.2192 -0.78125,-8.42059 l 0,-7.63889 3.94781,0 c 2.1713,0 6.70652,-0.71855 10.07827,-1.59678 25.31744,-6.59436 38.89378,-32.50962 29.78495,-56.85515 -2.92176,-7.80912 -8.14704,-14.80575 -10.35148,-13.8606 -0.92879,0.39822 -3.38085,1.20763 -5.44901,1.79869 -7.9813,2.28098 -19.02821,-4.01963 -21.80627,-12.43722 -1.0398,-3.15062 -1.55377,-3.50727 -5.05432,-3.50727 -4.90537,0 -14.09268,2.66458 -18.85828,5.46944 -8.98466,5.28804 -16.33289,14.48717 -19.83032,24.82527 -4.26406,12.60415 -1.28178,29.7618 6.81782,39.22436 3.60732,4.21435 3.6375,4.33247 3.6375,14.24026 0,6.68605 -0.38632,9.99067 -1.16793,9.99067 -2.52517,0 -17.35701,-16.26163 -20.47639,-22.4503 -9.9887,-19.81702 -9.3421,-42.55887 1.68783,-59.36379 11.31768,-17.24332 29.2723,-27.45018 48.75393,-27.71568 7.29315,-0.0994 9.18213,-0.49766 11.91164,-2.51144 10.88052,-8.02739 29.08258,0.18698 29.08258,13.12463 0,2.44339 1.29526,4.72346 5.123,9.01809 6.20157,6.95799 1
 0.92932,15.56316 13.07368,23.79594 2.04347,7.8454 2.04486,22.27099 0.003,30.11057 -5.37759,20.64593 -23.39328,38.84149 -42.67877,43.10484 -7.72068,1.70678 -16.55833,2.54504 -17.44791,1.65495 l -2.1e-4,0 z m 28.07925,-93.63667 c 7.23351,-2.07487 10.04965,-13.09915 4.9073,-19.21049 -7.50536,-8.9196 -21.94001,-3.81432 -21.69183,7.67202 0.12094,5.59736 3.50802,10.61813 8.03709,11.91361 3.29536,0.94259 4.30435,0.89932 8.74744,-0.37514 z"
+           id="path4192"
+           inkscape:connector-curvature="0"
+           transform="matrix(0.8,0,0,-0.8,0,792)" /><path
+           style="fill:#505050;fill-opacity:1"
+           d="m 118.27138,567.23632 c -4.81493,-1.68764 -6.72274,-3.2206 -8.65256,-6.95248 -2.05659,-3.977 -1.86396,-9.14324 0.48453,-12.99456 1.60357,-2.62973 1.98802,-5.39988 2.06434,-14.87435 0.16051,-19.9286 3.90673,-35.94542 12.59237,-53.83826 17.72653,-36.5175 53.69611,-63.12367 94.34705,-69.78711 14.0362,-2.3008 38.10199,-1.53492 49.95539,1.5898 l 8.33333,2.19678 4.05243,-2.27021 c 7.42219,-4.15798 17.56402,-2.66937 23.2416,3.4114 2.62105,2.80716 5.51847,9.20909 5.51847,12.19317 0,1.24195 2.10156,3.52889 5.46013,5.94178 7.34407,5.27619 15.55833,13.08867 22.28814,21.19797 l 5.557,6.69608 -7.53805,3.29815 -7.53805,3.29816 -6.92832,-7.44441 c -3.81059,-4.09442 -9.54756,-9.46512 -12.74884,-11.93489 l -5.82051,-4.49048 -4.19933,1.68023 c -8.09239,3.23794 -15.49576,1.66048 -21.59258,-4.60079 -1.85511,-1.90514 -3.66195,-5.00458 -4.01521,-6.88765 -0.85759,-4.57128 -1.7265,-5.13079 -10.84104,-6.98075 -24.51588,-4.97596 -47.436,-2.28574 -69.51413,8.15912 -13.6512,6.4582 -24.01808,13.99
 605 -34.4535,25.05143 -18.64604,19.75381 -28.46612,42.93514 -30.10076,71.05608 l -0.62081,10.67971 3.70354,3.59682 c 10.62817,10.32195 0.71147,27.82731 -13.03463,23.00926 z m 8.37184,-7.39955 c 3.25521,-2.28004 3.25521,-9.06017 0,-11.34021 -5.10327,-3.57447 -11.57378,-0.4045 -11.57378,5.67011 0,6.0746 6.47051,9.24457 11.57378,5.6701 z M 298.90025,436.35135 c 6.25266,-4.45229 6.04252,-13.79313 -0.43288,-19.24182 -2.79791,-2.35429 -3.98935,-2.68105 -8.27435,-2.26931 -10.34815,0.99432 -14.43538,9.66147 -9.18369,19.47436 2.81646,5.2626 11.90723,6.29753 17.89092,2.03677 z"
+           id="path4194"
+           inkscape:connector-curvature="0"
+           transform="matrix(0.8,0,0,-0.8,0,792)" /><path
+           style="fill:#ed1c24;fill-opacity:1"
+           d="M 462.67662,607.91217 C 447.34524,605.42524 448.75,606.75069 448.75,594.77182 c 0,-5.71825 0.35156,-10.39674 0.78125,-10.39664 0.42969,10e-5 4.53125,0.76731 9.11458,1.70492 5.37625,1.09982 14.06268,1.74662 24.47917,1.82273 17.75351,0.12975 22.11106,-0.98844 26.31918,-6.75377 2.53356,-3.4711 3.47236,-13.54948 1.82832,-19.62781 -2.11105,-7.805 -5.75108,-10.32643 -27.10583,-18.77608 -21.14979,-8.36854 -28.53926,-12.59565 -33.27444,-19.0345 -5.20921,-7.08342 -6.64818,-13.08922 -6.02661,-25.15302 0.72604,-14.09128 2.74975,-19.45083 10.13647,-26.84527 8.14892,-8.15741 15.11485,-10.37813 34.64268,-11.04397 12.67242,-0.43208 28.01695,0.75986 32.64504,2.53582 1.10907,0.42559 1.46019,2.91311 1.46019,10.34472 0,5.38141 -0.38619,9.78438 -0.85819,9.78438 -0.47201,0 -5.74545,-0.74803 -11.71875,-1.6623 -13.39387,-2.05006 -28.34077,-1.47082 -34.02941,1.31873 -6.13488,3.00837 -8.60198,7.63684 -8.60198,16.13791 0,10.77625 1.92884,12.4375 26.3708,22.71232 27.50406,11.56205 32.73995,15.34
 561 38.49433,27.81678 1.98303,4.29772 2.31638,6.77618 2.28779,17.0099 -0.0631,22.60004 -6.84129,33.53179 -24.62544,39.71574 -5.51588,1.91799 -9.30648,2.34136 -23.25665,2.59752 -10.32346,0.18956 -19.88998,-0.21682 -25.13588,-1.06776 z"
+           id="path4196"
+           inkscape:connector-curvature="0"
+           transform="matrix(0.8,0,0,-0.8,0,792)" /><path
+           style="fill:#ed1c24;fill-opacity:1"
+           d="m 586.61724,607.88425 c -9.16066,-2.86545 -16.35018,-9.88467 -21.41233,-20.90508 -2.19769,-4.78442 -2.39241,-6.6512 -2.39241,-22.93585 l 0,-17.7275 3.3955,-6.89998 c 8.02648,-16.31053 25.03594,-23.85687 48.74192,-21.62459 5.1265,0.48273 11.84583,1.36469 14.93184,1.95989 3.08601,0.59521 5.78132,1.08219 5.98958,1.08219 0.89468,0 0.26761,-22.75389 -0.72564,-26.33055 -1.45437,-5.23716 -6.43666,-10.73777 -11.18735,-12.35121 -5.03493,-1.70998 -20.64752,-1.4404 -35.02509,0.60477 -16.75657,2.38357 -15.18326,3.23398 -15.18326,-8.20698 0,-11.49032 -1.79832,-10.15863 17.04678,-12.62346 16.33686,-2.13676 36.0592,-1.67748 43.89072,1.0221 6.28922,2.16794 13.67946,7.02852 17.08802,11.23883 1.37758,1.7016 3.72133,5.97912 5.20833,9.50558 l 2.70365,6.41176 0.30162,58.07291 0.30162,58.07292 -11.49953,0 -11.49954,0 0,-11.52657 0,-11.52658 -6.13079,7.71494 c -6.48996,8.16694 -16.16953,15.53187 -22.85283,17.38814 -5.20587,1.4459 -16.42693,1.23087 -21.69081,-0.41568 z m 27.86465,-22.52211 c 
 5.92393,-3.02217 12.68911,-9.50363 17.60769,-16.86927 l 4.16042,-6.23027 0,-11.74902 c 0,-10.21547 -0.23794,-11.84483 -1.82292,-12.48304 -3.71425,-1.49558 -27.86364,-1.90459 -32.10574,-0.54376 -5.80814,1.8632 -10.51031,5.68811 -13.21637,10.75065 -1.99228,3.72719 -2.33414,5.96862 -2.33414,15.30424 0,10.0299 0.23538,11.33789 2.83658,15.7627 5.30297,9.02069 14.61935,11.28955 24.87448,6.05777 z"
+           id="path4198"
+           inkscape:connector-curvature="0"
+           transform="matrix(0.8,0,0,-0.8,0,792)" /><path
+           style="fill:#ed1c24;fill-opacity:1"
+           d="m 231.5625,628.52654 c -1.71875,-0.24803 -6.28906,-1.19347 -10.15625,-2.10097 l -7.03125,-1.64999 0,-8.33257 c 0,-6.81762 0.27223,-8.2281 1.49735,-7.75798 6.25262,2.39936 12.17881,3.16538 24.54432,3.17259 12.51579,0.007 14.97906,-0.29985 22.39583,-2.79273 20.63492,-6.93569 36.20948,-20.46462 45.8111,-39.79406 6.29736,-12.6775 8.23176,-21.1494 8.20563,-35.9375 -0.0193,-10.92248 -0.33863,-12.89983 -3.62016,-22.41635 -1.99566,-5.78744 -5.65243,-13.56044 -8.2083,-17.44791 -4.01033,-6.09972 -5.02051,-7.01124 -7.77378,-7.01463 -6.69868,-0.008 -14.08999,-7.43461 -14.0981,-14.16498 -0.005,-4.25293 3.8294,-9.84754 8.35021,-12.18306 4.09114,-2.11354 9.1445,-1.79095 13.15997,0.84009 3.17243,2.07865 6.60188,7.5836 6.60704,10.6056 0.002,1.25469 2.79532,6.41257 6.20708,11.46196 11.16726,16.52752 15.86614,31.58711 15.86311,50.84012 -0.003,19.5561 -5.09674,36.41057 -15.87236,52.52031 -13.47326,20.14273 -33.35242,34.39899 -55.2642,39.63247 -8.43402,2.0144 -24.79058,3.36045 -30.61724,2.
 51959 z M 301.95513,477.16346 C 303.62981,475.48878 305,473.40555 305,472.53405 c 0,-2.81683 -4.61833,-7.16947 -7.6071,-7.16947 -7.58445,0 -10.22341,10.35633 -3.47198,13.62543 3.52499,1.70683 4.78796,1.4197 8.03421,-1.82655 z"
+           id="path4200"
+           inkscape:connector-curvature="0"
+           transform="matrix(0.8,0,0,-0.8,0,792)" /><path
+           style="fill:#ed1c24;fill-opacity:1"
+           d="m 176.55722,608.47635 c -3.41397,-2.251 -5.19458,-4.6579 -6.67488,-9.02262 -0.58766,-1.73274 -3.35628,-6.18587 -6.15249,-9.89583 -6.23576,-8.27351 -12.11964,-20.66772 -15.07044,-31.7454 -3.1998,-12.01243 -3.1873,-30.94026 0.0289,-43.75 8.06246,-32.11203 31.30535,-58.18768 60.81216,-68.22371 10.80888,-3.67638 25.53732,-5.67302 35.01673,-4.74699 8.3026,0.81106 17.7745,2.6021 17.7745,3.36098 0,0.30528 -1.65982,3.7137 -3.68848,7.57426 l -3.68848,7.01919 -5.94694,-0.97407 c -7.19905,-1.17916 -21.55976,-0.3054 -29.38444,1.78786 -14.86874,3.97769 -31.36479,15.09191 -40.7105,27.42869 -17.73211,23.40728 -21.47871,53.61195 -9.80403,79.03905 1.8873,4.11048 4.88016,9.57923 6.65081,12.15278 3.00743,4.37116 3.52711,4.67918 7.89425,4.67918 5.68424,0 10.57091,3.13196 12.64983,8.10751 1.87504,4.4876 1.82272,7.10035 -0.22875,11.42351 -3.32576,7.0085 -13.18254,9.93634 -19.47772,5.78561 z m 10.16694,-4.95675 c 2.3422,-0.89878 4.85846,-6.12171 4.04763,-8.40153 -2.00828,-5.64674 -8.66903,-7
 .50147 -12.34646,-3.43796 -5.32506,5.88413 0.85227,14.697 8.29883,11.83949 z"
+           id="path4202"
+           inkscape:connector-curvature="0"
+           transform="matrix(0.8,0,0,-0.8,0,792)" /><path
+           style="fill:#ed1c24;fill-opacity:1"
+           d="m 706.04167,534.89583 0,-71.35416 10.9375,0 10.9375,0 0,10.41666 c 0,6.25 0.41666,10.41667 1.04166,10.41667 0.57292,0 1.04167,-0.41169 1.04167,-0.91486 0,-0.50317 3.07755,-4.2389 6.83901,-8.30162 9.45486,-10.21212 17.32685,-14.0879 29.59305,-14.57017 7.13777,-0.28063 10.04988,0.0896 14.83979,1.88645 7.66645,2.876 14.31055,9.33002 18.09708,17.57935 1.61677,3.5223 3.19361,6.40334 3.50408,6.40232 0.31047,-0.001 1.85362,-2.04246 3.42921,-4.53653 4.22539,-6.68852 13.48833,-14.97754 20.30961,-18.17422 5.31901,-2.49268 7.22132,-2.80764 16.92984,-2.80304 9.34115,0.004 11.57941,0.34744 15.33562,2.35015 10.27636,5.47908 15.75385,12.92337 19.08099,25.93238 1.90016,7.42955 2.04172,11.76315 2.04172,62.50392 l 0,54.52087 -11.97917,0 -11.97916,0 0,-50.99039 c 0,-32.43344 -0.40753,-52.89067 -1.11982,-56.21193 -4.77376,-22.25927 -27.83245,-21.85593 -44.97393,0.78667 L 805,506.37017 l 0,49.93992 0,49.93991 -11.97917,0 -11.97916,0 0,-49.85782 c 0,-28.48258 -0.44128,-52.18469 -1.02929,-55
 .28524 -1.44315,-7.60969 -3.87399,-12.22699 -8.1542,-15.48865 -4.90661,-3.739 -14.0717,-3.8746 -20.91238,-0.30939 -5.54351,2.88915 -13.87129,10.51754 -17.94866,16.44128 L 730,506.10453 l 0,50.07274 0,50.07273 -11.97917,0 -11.97916,0 0,-71.35417 z"
+           id="path4204"
+           inkscape:connector-curvature="0"
+           transform="matrix(0.8,0,0,-0.8,0,792)" /><path
+           style="fill:#ed1c24;fill-opacity:1"
+           d="m 914.5757,597.31794 0,-8.93207 32.8125,-50.96796 c 18.04688,-28.03238 32.8125,-51.43483 32.8125,-52.00544 0,-0.66617 -11.37037,-1.03747 -31.77083,-1.03747 l -31.77083,0 0,-10.41667 0,-10.41666 46.875,0 46.87496,0 0,8.99881 0,8.99881 -32.36223,50.47497 c -17.79924,27.76123 -32.61502,51.13365 -32.92394,51.93869 -0.47513,1.23815 4.51202,1.46372 32.36226,1.46372 l 32.92391,0 0,10.41666 0,10.41667 -47.91663,0 -47.91667,0 0,-8.93206 z"
+           id="path4206"
+           inkscape:connector-curvature="0"
+           transform="matrix(0.8,0,0,-0.8,0,792)" /><path
+           style="fill:#ed1c24;fill-opacity:1"
+           d="m 1068.0679,607.94399 c -5.1656,-1.68724 -12.8253,-7.00827 -15.7152,-10.91707 -6.9671,-9.42359 -8.9502,-18.02125 -8.4453,-36.61428 0.3408,-12.55001 0.6738,-14.67321 3.1036,-19.78764 4.1284,-8.6901 9.3651,-14.02335 17.8814,-18.21115 11.5424,-5.67586 26.4005,-6.52778 46.4401,-2.66271 3.086,0.59521 5.7813,1.08219 5.9895,1.08219 0.2083,0 0.3787,-5.40568 0.3787,-12.01262 0,-13.12683 -1.183,-17.61535 -5.9562,-22.59872 -1.4964,-1.56227 -4.4013,-3.41175 -6.4552,-4.10994 -4.9084,-1.66852 -20.6946,-1.37714 -34.9053,0.64429 -16.7566,2.38357 -15.1833,3.23398 -15.1833,-8.20698 0,-11.49032 -1.7983,-10.15863 17.0468,-12.62346 16.3368,-2.13676 36.0592,-1.67748 43.8907,1.0221 6.2892,2.16794 13.6795,7.02852 17.088,11.23883 1.3776,1.7016 3.7214,5.97912 5.2084,9.50558 l 2.7036,6.41176 0.3016,58.07291 0.3016,58.07292 -11.4995,0 -11.4995,0 0,-11.61368 0,-11.61367 -5.2302,6.90894 c -5.1747,6.83551 -12.7091,13.2793 -19.7698,16.90793 -4.7445,2.43829 -19.6303,3.07865 -25.6745,1.10447 z m 27.956
 9,-22.62892 c 5.837,-2.97781 12.6222,-9.49439 17.5155,-16.8222 l 4.1604,-6.23027 0,-11.74902 c 0,-10.21547 -0.2379,-11.84483 -1.8229,-12.48304 -3.7143,-1.49558 -27.8637,-1.90459 -32.1058,-0.54376 -5.8081,1.8632 -10.5103,5.68811 -13.2163,10.75065 -1.9923,3.72719 -2.3342,5.96862 -2.3342,15.30424 0,9.68692 0.2905,11.44475 2.5408,15.37388 5.3515,9.34409 14.7994,11.73742 25.2625,6.39952 z"
+           id="path4208"
+           inkscape:connector-curvature="0"
+           transform="matrix(0.8,0,0,-0.8,0,792)" /></g></g></g></svg>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/samza/blob/b8058af0/docs/img/samza-logo@2x.png
----------------------------------------------------------------------
diff --git a/docs/img/samza-logo@2x.png b/docs/img/samza-logo@2x.png
new file mode 100644
index 0000000..dec38c9
Binary files /dev/null and b/docs/img/samza-logo@2x.png differ

http://git-wip-us.apache.org/repos/asf/samza/blob/b8058af0/docs/index.md
----------------------------------------------------------------------
diff --git a/docs/index.md b/docs/index.md
index 4e6c19c..ec6f200 100644
--- a/docs/index.md
+++ b/docs/index.md
@@ -16,24 +16,4 @@ layout: default
    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.
--->
-
-## What is Samza?
-
-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.
-
-* **Simple API:** Unlike most low-level messaging system APIs, Samza provides a very simple callback-based "process message" API comparable to MapReduce.
-* **Managed state:** Samza manages snapshotting and restoration of a stream processor's state. When the processor is restarted, Samza restores its state to a consistent snapshot. Samza is built to handle large amounts of state (many gigabytes per partition).
-* **Fault tolerance:** Whenever a machine in the cluster fails, Samza works with YARN to transparently migrate your tasks to another machine.
-* **Durability:** Samza uses Kafka to guarantee that messages are processed in the order they were written to a partition, and that no messages are ever 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, which supports Hadoop's security model, and resource isolation through Linux CGroups.
-
-Check out [Hello Samza](/startup/hello-samza/{{site.version}}) to try Samza. Read the [Background](/learn/documentation/{{site.version}}/introduction/background.html) page to learn more about Samza.
-
-### Apache Software Foundation
-
-Apache Samza is a top level project of the [Apache Software Foundation](http://www.apache.org/).
-
-![Apache Software Foundation Logo](http://www.apache.org/images/feather.gif)
+-->
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/samza/blob/b8058af0/docs/index.md.bak
----------------------------------------------------------------------
diff --git a/docs/index.md.bak b/docs/index.md.bak
new file mode 100644
index 0000000..4e6c19c
--- /dev/null
+++ b/docs/index.md.bak
@@ -0,0 +1,39 @@
+---
+layout: default
+---
+<!--
+   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.
+-->
+
+## What is Samza?
+
+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.
+
+* **Simple API:** Unlike most low-level messaging system APIs, Samza provides a very simple callback-based "process message" API comparable to MapReduce.
+* **Managed state:** Samza manages snapshotting and restoration of a stream processor's state. When the processor is restarted, Samza restores its state to a consistent snapshot. Samza is built to handle large amounts of state (many gigabytes per partition).
+* **Fault tolerance:** Whenever a machine in the cluster fails, Samza works with YARN to transparently migrate your tasks to another machine.
+* **Durability:** Samza uses Kafka to guarantee that messages are processed in the order they were written to a partition, and that no messages are ever 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, which supports Hadoop's security model, and resource isolation through Linux CGroups.
+
+Check out [Hello Samza](/startup/hello-samza/{{site.version}}) to try Samza. Read the [Background](/learn/documentation/{{site.version}}/introduction/background.html) page to learn more about Samza.
+
+### Apache Software Foundation
+
+Apache Samza is a top level project of the [Apache Software Foundation](http://www.apache.org/).
+
+![Apache Software Foundation Logo](http://www.apache.org/images/feather.gif)

http://git-wip-us.apache.org/repos/asf/samza/blob/b8058af0/docs/js/bootstrap.min.js
----------------------------------------------------------------------
diff --git a/docs/js/bootstrap.min.js b/docs/js/bootstrap.min.js
deleted file mode 100644
index b04a0e8..0000000
--- a/docs/js/bootstrap.min.js
+++ /dev/null
@@ -1,6 +0,0 @@
-/*!
- * Bootstrap v3.1.1 (http://getbootstrap.com)
- * Copyright 2011-2014 Twitter, Inc.
- * Licensed under MIT (https://github.com/twbs/bootstrap/blob/master/LICENSE)
- */
-if("undefined"==typeof jQuery)throw new Error("Bootstrap's JavaScript requires jQuery");+function(a){"use strict";function b(){var a=document.createElement("bootstrap"),b={WebkitTransition:"webkitTransitionEnd",MozTransition:"transitionend",OTransition:"oTransitionEnd otransitionend",transition:"transitionend"};for(var c in b)if(void 0!==a.style[c])return{end:b[c]};return!1}a.fn.emulateTransitionEnd=function(b){var c=!1,d=this;a(this).one(a.support.transition.end,function(){c=!0});var e=function(){c||a(d).trigger(a.support.transition.end)};return setTimeout(e,b),this},a(function(){a.support.transition=b()})}(jQuery),+function(a){"use strict";var b='[data-dismiss="alert"]',c=function(c){a(c).on("click",b,this.close)};c.prototype.close=function(b){function c(){f.trigger("closed.bs.alert").remove()}var d=a(this),e=d.attr("data-target");e||(e=d.attr("href"),e=e&&e.replace(/.*(?=#[^\s]*$)/,""));var f=a(e);b&&b.preventDefault(),f.length||(f=d.hasClass("alert")?d:d.parent()),f.trigger(b=a.
 Event("close.bs.alert")),b.isDefaultPrevented()||(f.removeClass("in"),a.support.transition&&f.hasClass("fade")?f.one(a.support.transition.end,c).emulateTransitionEnd(150):c())};var d=a.fn.alert;a.fn.alert=function(b){return this.each(function(){var d=a(this),e=d.data("bs.alert");e||d.data("bs.alert",e=new c(this)),"string"==typeof b&&e[b].call(d)})},a.fn.alert.Constructor=c,a.fn.alert.noConflict=function(){return a.fn.alert=d,this},a(document).on("click.bs.alert.data-api",b,c.prototype.close)}(jQuery),+function(a){"use strict";var b=function(c,d){this.$element=a(c),this.options=a.extend({},b.DEFAULTS,d),this.isLoading=!1};b.DEFAULTS={loadingText:"loading..."},b.prototype.setState=function(b){var c="disabled",d=this.$element,e=d.is("input")?"val":"html",f=d.data();b+="Text",f.resetText||d.data("resetText",d[e]()),d[e](f[b]||this.options[b]),setTimeout(a.proxy(function(){"loadingText"==b?(this.isLoading=!0,d.addClass(c).attr(c,c)):this.isLoading&&(this.isLoading=!1,d.removeClass(c).re
 moveAttr(c))},this),0)},b.prototype.toggle=function(){var a=!0,b=this.$element.closest('[data-toggle="buttons"]');if(b.length){var c=this.$element.find("input");"radio"==c.prop("type")&&(c.prop("checked")&&this.$element.hasClass("active")?a=!1:b.find(".active").removeClass("active")),a&&c.prop("checked",!this.$element.hasClass("active")).trigger("change")}a&&this.$element.toggleClass("active")};var c=a.fn.button;a.fn.button=function(c){return this.each(function(){var d=a(this),e=d.data("bs.button"),f="object"==typeof c&&c;e||d.data("bs.button",e=new b(this,f)),"toggle"==c?e.toggle():c&&e.setState(c)})},a.fn.button.Constructor=b,a.fn.button.noConflict=function(){return a.fn.button=c,this},a(document).on("click.bs.button.data-api","[data-toggle^=button]",function(b){var c=a(b.target);c.hasClass("btn")||(c=c.closest(".btn")),c.button("toggle"),b.preventDefault()})}(jQuery),+function(a){"use strict";var b=function(b,c){this.$element=a(b),this.$indicators=this.$element.find(".carousel-in
 dicators"),this.options=c,this.paused=this.sliding=this.interval=this.$active=this.$items=null,"hover"==this.options.pause&&this.$element.on("mouseenter",a.proxy(this.pause,this)).on("mouseleave",a.proxy(this.cycle,this))};b.DEFAULTS={interval:5e3,pause:"hover",wrap:!0},b.prototype.cycle=function(b){return b||(this.paused=!1),this.interval&&clearInterval(this.interval),this.options.interval&&!this.paused&&(this.interval=setInterval(a.proxy(this.next,this),this.options.interval)),this},b.prototype.getActiveIndex=function(){return this.$active=this.$element.find(".item.active"),this.$items=this.$active.parent().children(),this.$items.index(this.$active)},b.prototype.to=function(b){var c=this,d=this.getActiveIndex();return b>this.$items.length-1||0>b?void 0:this.sliding?this.$element.one("slid.bs.carousel",function(){c.to(b)}):d==b?this.pause().cycle():this.slide(b>d?"next":"prev",a(this.$items[b]))},b.prototype.pause=function(b){return b||(this.paused=!0),this.$element.find(".next, .p
 rev").length&&a.support.transition&&(this.$element.trigger(a.support.transition.end),this.cycle(!0)),this.interval=clearInterval(this.interval),this},b.prototype.next=function(){return this.sliding?void 0:this.slide("next")},b.prototype.prev=function(){return this.sliding?void 0:this.slide("prev")},b.prototype.slide=function(b,c){var d=this.$element.find(".item.active"),e=c||d[b](),f=this.interval,g="next"==b?"left":"right",h="next"==b?"first":"last",i=this;if(!e.length){if(!this.options.wrap)return;e=this.$element.find(".item")[h]()}if(e.hasClass("active"))return this.sliding=!1;var j=a.Event("slide.bs.carousel",{relatedTarget:e[0],direction:g});return this.$element.trigger(j),j.isDefaultPrevented()?void 0:(this.sliding=!0,f&&this.pause(),this.$indicators.length&&(this.$indicators.find(".active").removeClass("active"),this.$element.one("slid.bs.carousel",function(){var b=a(i.$indicators.children()[i.getActiveIndex()]);b&&b.addClass("active")})),a.support.transition&&this.$element.h
 asClass("slide")?(e.addClass(b),e[0].offsetWidth,d.addClass(g),e.addClass(g),d.one(a.support.transition.end,function(){e.removeClass([b,g].join(" ")).addClass("active"),d.removeClass(["active",g].join(" ")),i.sliding=!1,setTimeout(function(){i.$element.trigger("slid.bs.carousel")},0)}).emulateTransitionEnd(1e3*d.css("transition-duration").slice(0,-1))):(d.removeClass("active"),e.addClass("active"),this.sliding=!1,this.$element.trigger("slid.bs.carousel")),f&&this.cycle(),this)};var c=a.fn.carousel;a.fn.carousel=function(c){return this.each(function(){var d=a(this),e=d.data("bs.carousel"),f=a.extend({},b.DEFAULTS,d.data(),"object"==typeof c&&c),g="string"==typeof c?c:f.slide;e||d.data("bs.carousel",e=new b(this,f)),"number"==typeof c?e.to(c):g?e[g]():f.interval&&e.pause().cycle()})},a.fn.carousel.Constructor=b,a.fn.carousel.noConflict=function(){return a.fn.carousel=c,this},a(document).on("click.bs.carousel.data-api","[data-slide], [data-slide-to]",function(b){var c,d=a(this),e=a(d.a
 ttr("data-target")||(c=d.attr("href"))&&c.replace(/.*(?=#[^\s]+$)/,"")),f=a.extend({},e.data(),d.data()),g=d.attr("data-slide-to");g&&(f.interval=!1),e.carousel(f),(g=d.attr("data-slide-to"))&&e.data("bs.carousel").to(g),b.preventDefault()}),a(window).on("load",function(){a('[data-ride="carousel"]').each(function(){var b=a(this);b.carousel(b.data())})})}(jQuery),+function(a){"use strict";var b=function(c,d){this.$element=a(c),this.options=a.extend({},b.DEFAULTS,d),this.transitioning=null,this.options.parent&&(this.$parent=a(this.options.parent)),this.options.toggle&&this.toggle()};b.DEFAULTS={toggle:!0},b.prototype.dimension=function(){var a=this.$element.hasClass("width");return a?"width":"height"},b.prototype.show=function(){if(!this.transitioning&&!this.$element.hasClass("in")){var b=a.Event("show.bs.collapse");if(this.$element.trigger(b),!b.isDefaultPrevented()){var c=this.$parent&&this.$parent.find("> .panel > .in");if(c&&c.length){var d=c.data("bs.collapse");if(d&&d.transition
 ing)return;c.collapse("hide"),d||c.data("bs.collapse",null)}var e=this.dimension();this.$element.removeClass("collapse").addClass("collapsing")[e](0),this.transitioning=1;var f=function(){this.$element.removeClass("collapsing").addClass("collapse in")[e]("auto"),this.transitioning=0,this.$element.trigger("shown.bs.collapse")};if(!a.support.transition)return f.call(this);var g=a.camelCase(["scroll",e].join("-"));this.$element.one(a.support.transition.end,a.proxy(f,this)).emulateTransitionEnd(350)[e](this.$element[0][g])}}},b.prototype.hide=function(){if(!this.transitioning&&this.$element.hasClass("in")){var b=a.Event("hide.bs.collapse");if(this.$element.trigger(b),!b.isDefaultPrevented()){var c=this.dimension();this.$element[c](this.$element[c]())[0].offsetHeight,this.$element.addClass("collapsing").removeClass("collapse").removeClass("in"),this.transitioning=1;var d=function(){this.transitioning=0,this.$element.trigger("hidden.bs.collapse").removeClass("collapsing").addClass("collap
 se")};return a.support.transition?void this.$element[c](0).one(a.support.transition.end,a.proxy(d,this)).emulateTransitionEnd(350):d.call(this)}}},b.prototype.toggle=function(){this[this.$element.hasClass("in")?"hide":"show"]()};var c=a.fn.collapse;a.fn.collapse=function(c){return this.each(function(){var d=a(this),e=d.data("bs.collapse"),f=a.extend({},b.DEFAULTS,d.data(),"object"==typeof c&&c);!e&&f.toggle&&"show"==c&&(c=!c),e||d.data("bs.collapse",e=new b(this,f)),"string"==typeof c&&e[c]()})},a.fn.collapse.Constructor=b,a.fn.collapse.noConflict=function(){return a.fn.collapse=c,this},a(document).on("click.bs.collapse.data-api","[data-toggle=collapse]",function(b){var c,d=a(this),e=d.attr("data-target")||b.preventDefault()||(c=d.attr("href"))&&c.replace(/.*(?=#[^\s]+$)/,""),f=a(e),g=f.data("bs.collapse"),h=g?"toggle":d.data(),i=d.attr("data-parent"),j=i&&a(i);g&&g.transitioning||(j&&j.find('[data-toggle=collapse][data-parent="'+i+'"]').not(d).addClass("collapsed"),d[f.hasClass("in
 ")?"addClass":"removeClass"]("collapsed")),f.collapse(h)})}(jQuery),+function(a){"use strict";function b(b){a(d).remove(),a(e).each(function(){var d=c(a(this)),e={relatedTarget:this};d.hasClass("open")&&(d.trigger(b=a.Event("hide.bs.dropdown",e)),b.isDefaultPrevented()||d.removeClass("open").trigger("hidden.bs.dropdown",e))})}function c(b){var c=b.attr("data-target");c||(c=b.attr("href"),c=c&&/#[A-Za-z]/.test(c)&&c.replace(/.*(?=#[^\s]*$)/,""));var d=c&&a(c);return d&&d.length?d:b.parent()}var d=".dropdown-backdrop",e="[data-toggle=dropdown]",f=function(b){a(b).on("click.bs.dropdown",this.toggle)};f.prototype.toggle=function(d){var e=a(this);if(!e.is(".disabled, :disabled")){var f=c(e),g=f.hasClass("open");if(b(),!g){"ontouchstart"in document.documentElement&&!f.closest(".navbar-nav").length&&a('<div class="dropdown-backdrop"/>').insertAfter(a(this)).on("click",b);var h={relatedTarget:this};if(f.trigger(d=a.Event("show.bs.dropdown",h)),d.isDefaultPrevented())return;f.toggleClass("op
 en").trigger("shown.bs.dropdown",h),e.focus()}return!1}},f.prototype.keydown=function(b){if(/(38|40|27)/.test(b.keyCode)){var d=a(this);if(b.preventDefault(),b.stopPropagation(),!d.is(".disabled, :disabled")){var f=c(d),g=f.hasClass("open");if(!g||g&&27==b.keyCode)return 27==b.which&&f.find(e).focus(),d.click();var h=" li:not(.divider):visible a",i=f.find("[role=menu]"+h+", [role=listbox]"+h);if(i.length){var j=i.index(i.filter(":focus"));38==b.keyCode&&j>0&&j--,40==b.keyCode&&j<i.length-1&&j++,~j||(j=0),i.eq(j).focus()}}}};var g=a.fn.dropdown;a.fn.dropdown=function(b){return this.each(function(){var c=a(this),d=c.data("bs.dropdown");d||c.data("bs.dropdown",d=new f(this)),"string"==typeof b&&d[b].call(c)})},a.fn.dropdown.Constructor=f,a.fn.dropdown.noConflict=function(){return a.fn.dropdown=g,this},a(document).on("click.bs.dropdown.data-api",b).on("click.bs.dropdown.data-api",".dropdown form",function(a){a.stopPropagation()}).on("click.bs.dropdown.data-api",e,f.prototype.toggle).on(
 "keydown.bs.dropdown.data-api",e+", [role=menu], [role=listbox]",f.prototype.keydown)}(jQuery),+function(a){"use strict";var b=function(b,c){this.options=c,this.$element=a(b),this.$backdrop=this.isShown=null,this.options.remote&&this.$element.find(".modal-content").load(this.options.remote,a.proxy(function(){this.$element.trigger("loaded.bs.modal")},this))};b.DEFAULTS={backdrop:!0,keyboard:!0,show:!0},b.prototype.toggle=function(a){return this[this.isShown?"hide":"show"](a)},b.prototype.show=function(b){var c=this,d=a.Event("show.bs.modal",{relatedTarget:b});this.$element.trigger(d),this.isShown||d.isDefaultPrevented()||(this.isShown=!0,this.escape(),this.$element.on("click.dismiss.bs.modal",'[data-dismiss="modal"]',a.proxy(this.hide,this)),this.backdrop(function(){var d=a.support.transition&&c.$element.hasClass("fade");c.$element.parent().length||c.$element.appendTo(document.body),c.$element.show().scrollTop(0),d&&c.$element[0].offsetWidth,c.$element.addClass("in").attr("aria-hidde
 n",!1),c.enforceFocus();var e=a.Event("shown.bs.modal",{relatedTarget:b});d?c.$element.find(".modal-dialog").one(a.support.transition.end,function(){c.$element.focus().trigger(e)}).emulateTransitionEnd(300):c.$element.focus().trigger(e)}))},b.prototype.hide=function(b){b&&b.preventDefault(),b=a.Event("hide.bs.modal"),this.$element.trigger(b),this.isShown&&!b.isDefaultPrevented()&&(this.isShown=!1,this.escape(),a(document).off("focusin.bs.modal"),this.$element.removeClass("in").attr("aria-hidden",!0).off("click.dismiss.bs.modal"),a.support.transition&&this.$element.hasClass("fade")?this.$element.one(a.support.transition.end,a.proxy(this.hideModal,this)).emulateTransitionEnd(300):this.hideModal())},b.prototype.enforceFocus=function(){a(document).off("focusin.bs.modal").on("focusin.bs.modal",a.proxy(function(a){this.$element[0]===a.target||this.$element.has(a.target).length||this.$element.focus()},this))},b.prototype.escape=function(){this.isShown&&this.options.keyboard?this.$element.o
 n("keyup.dismiss.bs.modal",a.proxy(function(a){27==a.which&&this.hide()},this)):this.isShown||this.$element.off("keyup.dismiss.bs.modal")},b.prototype.hideModal=function(){var a=this;this.$element.hide(),this.backdrop(function(){a.removeBackdrop(),a.$element.trigger("hidden.bs.modal")})},b.prototype.removeBackdrop=function(){this.$backdrop&&this.$backdrop.remove(),this.$backdrop=null},b.prototype.backdrop=function(b){var c=this.$element.hasClass("fade")?"fade":"";if(this.isShown&&this.options.backdrop){var d=a.support.transition&&c;if(this.$backdrop=a('<div class="modal-backdrop '+c+'" />').appendTo(document.body),this.$element.on("click.dismiss.bs.modal",a.proxy(function(a){a.target===a.currentTarget&&("static"==this.options.backdrop?this.$element[0].focus.call(this.$element[0]):this.hide.call(this))},this)),d&&this.$backdrop[0].offsetWidth,this.$backdrop.addClass("in"),!b)return;d?this.$backdrop.one(a.support.transition.end,b).emulateTransitionEnd(150):b()}else!this.isShown&&this.
 $backdrop?(this.$backdrop.removeClass("in"),a.support.transition&&this.$element.hasClass("fade")?this.$backdrop.one(a.support.transition.end,b).emulateTransitionEnd(150):b()):b&&b()};var c=a.fn.modal;a.fn.modal=function(c,d){return this.each(function(){var e=a(this),f=e.data("bs.modal"),g=a.extend({},b.DEFAULTS,e.data(),"object"==typeof c&&c);f||e.data("bs.modal",f=new b(this,g)),"string"==typeof c?f[c](d):g.show&&f.show(d)})},a.fn.modal.Constructor=b,a.fn.modal.noConflict=function(){return a.fn.modal=c,this},a(document).on("click.bs.modal.data-api",'[data-toggle="modal"]',function(b){var c=a(this),d=c.attr("href"),e=a(c.attr("data-target")||d&&d.replace(/.*(?=#[^\s]+$)/,"")),f=e.data("bs.modal")?"toggle":a.extend({remote:!/#/.test(d)&&d},e.data(),c.data());c.is("a")&&b.preventDefault(),e.modal(f,this).one("hide",function(){c.is(":visible")&&c.focus()})}),a(document).on("show.bs.modal",".modal",function(){a(document.body).addClass("modal-open")}).on("hidden.bs.modal",".modal",functi
 on(){a(document.body).removeClass("modal-open")})}(jQuery),+function(a){"use strict";var b=function(a,b){this.type=this.options=this.enabled=this.timeout=this.hoverState=this.$element=null,this.init("tooltip",a,b)};b.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},b.prototype.init=function(b,c,d){this.enabled=!0,this.type=b,this.$element=a(c),this.options=this.getOptions(d);for(var e=this.options.trigger.split(" "),f=e.length;f--;){var g=e[f];if("click"==g)this.$element.on("click."+this.type,this.options.selector,a.proxy(this.toggle,this));else if("manual"!=g){var h="hover"==g?"mouseenter":"focusin",i="hover"==g?"mouseleave":"focusout";this.$element.on(h+"."+this.type,this.options.selector,a.proxy(this.enter,this)),this.$element.on(i+"."+this.type,this.options.selector,a.proxy(this.leave,this))}}this.options.selector
 ?this._options=a.extend({},this.options,{trigger:"manual",selector:""}):this.fixTitle()},b.prototype.getDefaults=function(){return b.DEFAULTS},b.prototype.getOptions=function(b){return b=a.extend({},this.getDefaults(),this.$element.data(),b),b.delay&&"number"==typeof b.delay&&(b.delay={show:b.delay,hide:b.delay}),b},b.prototype.getDelegateOptions=function(){var b={},c=this.getDefaults();return this._options&&a.each(this._options,function(a,d){c[a]!=d&&(b[a]=d)}),b},b.prototype.enter=function(b){var c=b instanceof this.constructor?b:a(b.currentTarget)[this.type](this.getDelegateOptions()).data("bs."+this.type);return clearTimeout(c.timeout),c.hoverState="in",c.options.delay&&c.options.delay.show?void(c.timeout=setTimeout(function(){"in"==c.hoverState&&c.show()},c.options.delay.show)):c.show()},b.prototype.leave=function(b){var c=b instanceof this.constructor?b:a(b.currentTarget)[this.type](this.getDelegateOptions()).data("bs."+this.type);return clearTimeout(c.timeout),c.hoverState="o
 ut",c.options.delay&&c.options.delay.hide?void(c.timeout=setTimeout(function(){"out"==c.hoverState&&c.hide()},c.options.delay.hide)):c.hide()},b.prototype.show=function(){var b=a.Event("show.bs."+this.type);if(this.hasContent()&&this.enabled){if(this.$element.trigger(b),b.isDefaultPrevented())return;var c=this,d=this.tip();this.setContent(),this.options.animation&&d.addClass("fade");var e="function"==typeof this.options.placement?this.options.placement.call(this,d[0],this.$element[0]):this.options.placement,f=/\s?auto?\s?/i,g=f.test(e);g&&(e=e.replace(f,"")||"top"),d.detach().css({top:0,left:0,display:"block"}).addClass(e),this.options.container?d.appendTo(this.options.container):d.insertAfter(this.$element);var h=this.getPosition(),i=d[0].offsetWidth,j=d[0].offsetHeight;if(g){var k=this.$element.parent(),l=e,m=document.documentElement.scrollTop||document.body.scrollTop,n="body"==this.options.container?window.innerWidth:k.outerWidth(),o="body"==this.options.container?window.innerHei
 ght:k.outerHeight(),p="body"==this.options.container?0:k.offset().left;e="bottom"==e&&h.top+h.height+j-m>o?"top":"top"==e&&h.top-m-j<0?"bottom":"right"==e&&h.right+i>n?"left":"left"==e&&h.left-i<p?"right":e,d.removeClass(l).addClass(e)}var q=this.getCalculatedOffset(e,h,i,j);this.applyPlacement(q,e),this.hoverState=null;var r=function(){c.$element.trigger("shown.bs."+c.type)};a.support.transition&&this.$tip.hasClass("fade")?d.one(a.support.transition.end,r).emulateTransitionEnd(150):r()}},b.prototype.applyPlacement=function(b,c){var d,e=this.tip(),f=e[0].offsetWidth,g=e[0].offsetHeight,h=parseInt(e.css("margin-top"),10),i=parseInt(e.css("margin-left"),10);isNaN(h)&&(h=0),isNaN(i)&&(i=0),b.top=b.top+h,b.left=b.left+i,a.offset.setOffset(e[0],a.extend({using:function(a){e.css({top:Math.round(a.top),left:Math.round(a.left)})}},b),0),e.addClass("in");var j=e[0].offsetWidth,k=e[0].offsetHeight;if("top"==c&&k!=g&&(d=!0,b.top=b.top+g-k),/bottom|top/.test(c)){var l=0;b.left<0&&(l=-2*b.left,b
 .left=0,e.offset(b),j=e[0].offsetWidth,k=e[0].offsetHeight),this.replaceArrow(l-f+j,j,"left")}else this.replaceArrow(k-g,k,"top");d&&e.offset(b)},b.prototype.replaceArrow=function(a,b,c){this.arrow().css(c,a?50*(1-a/b)+"%":"")},b.prototype.setContent=function(){var a=this.tip(),b=this.getTitle();a.find(".tooltip-inner")[this.options.html?"html":"text"](b),a.removeClass("fade in top bottom left right")},b.prototype.hide=function(){function b(){"in"!=c.hoverState&&d.detach(),c.$element.trigger("hidden.bs."+c.type)}var c=this,d=this.tip(),e=a.Event("hide.bs."+this.type);return this.$element.trigger(e),e.isDefaultPrevented()?void 0:(d.removeClass("in"),a.support.transition&&this.$tip.hasClass("fade")?d.one(a.support.transition.end,b).emulateTransitionEnd(150):b(),this.hoverState=null,this)},b.prototype.fixTitle=function(){var a=this.$element;(a.attr("title")||"string"!=typeof a.attr("data-original-title"))&&a.attr("data-original-title",a.attr("title")||"").attr("title","")},b.prototype.
 hasContent=function(){return this.getTitle()},b.prototype.getPosition=function(){var b=this.$element[0];return a.extend({},"function"==typeof b.getBoundingClientRect?b.getBoundingClientRect():{width:b.offsetWidth,height:b.offsetHeight},this.$element.offset())},b.prototype.getCalculatedOffset=function(a,b,c,d){return"bottom"==a?{top:b.top+b.height,left:b.left+b.width/2-c/2}:"top"==a?{top:b.top-d,left:b.left+b.width/2-c/2}:"left"==a?{top:b.top+b.height/2-d/2,left:b.left-c}:{top:b.top+b.height/2-d/2,left:b.left+b.width}},b.prototype.getTitle=function(){var a,b=this.$element,c=this.options;return a=b.attr("data-original-title")||("function"==typeof c.title?c.title.call(b[0]):c.title)},b.prototype.tip=function(){return this.$tip=this.$tip||a(this.options.template)},b.prototype.arrow=function(){return this.$arrow=this.$arrow||this.tip().find(".tooltip-arrow")},b.prototype.validate=function(){this.$element[0].parentNode||(this.hide(),this.$element=null,this.options=null)},b.prototype.enabl
 e=function(){this.enabled=!0},b.prototype.disable=function(){this.enabled=!1},b.prototype.toggleEnabled=function(){this.enabled=!this.enabled},b.prototype.toggle=function(b){var c=b?a(b.currentTarget)[this.type](this.getDelegateOptions()).data("bs."+this.type):this;c.tip().hasClass("in")?c.leave(c):c.enter(c)},b.prototype.destroy=function(){clearTimeout(this.timeout),this.hide().$element.off("."+this.type).removeData("bs."+this.type)};var c=a.fn.tooltip;a.fn.tooltip=function(c){return this.each(function(){var d=a(this),e=d.data("bs.tooltip"),f="object"==typeof c&&c;(e||"destroy"!=c)&&(e||d.data("bs.tooltip",e=new b(this,f)),"string"==typeof c&&e[c]())})},a.fn.tooltip.Constructor=b,a.fn.tooltip.noConflict=function(){return a.fn.tooltip=c,this}}(jQuery),+function(a){"use strict";var b=function(a,b){this.init("popover",a,b)};if(!a.fn.tooltip)throw new Error("Popover requires tooltip.js");b.DEFAULTS=a.extend({},a.fn.tooltip.Constructor.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>'}),b.prototype=a.extend({},a.fn.tooltip.Constructor.prototype),b.prototype.constructor=b,b.prototype.getDefaults=function(){return b.DEFAULTS},b.prototype.setContent=function(){var a=this.tip(),b=this.getTitle(),c=this.getContent();a.find(".popover-title")[this.options.html?"html":"text"](b),a.find(".popover-content")[this.options.html?"string"==typeof c?"html":"append":"text"](c),a.removeClass("fade top bottom left right in"),a.find(".popover-title").html()||a.find(".popover-title").hide()},b.prototype.hasContent=function(){return this.getTitle()||this.getContent()},b.prototype.getContent=function(){var a=this.$element,b=this.options;return a.attr("data-content")||("function"==typeof b.content?b.content.call(a[0]):b.content)},b.prototype.arrow=function(){return this.$arrow=this.$arrow||this.tip().find(".arrow")},b.prototype.tip=function(){return this.
 $tip||(this.$tip=a(this.options.template)),this.$tip};var c=a.fn.popover;a.fn.popover=function(c){return this.each(function(){var d=a(this),e=d.data("bs.popover"),f="object"==typeof c&&c;(e||"destroy"!=c)&&(e||d.data("bs.popover",e=new b(this,f)),"string"==typeof c&&e[c]())})},a.fn.popover.Constructor=b,a.fn.popover.noConflict=function(){return a.fn.popover=c,this}}(jQuery),+function(a){"use strict";function b(c,d){var e,f=a.proxy(this.process,this);this.$element=a(a(c).is("body")?window:c),this.$body=a("body"),this.$scrollElement=this.$element.on("scroll.bs.scroll-spy.data-api",f),this.options=a.extend({},b.DEFAULTS,d),this.selector=(this.options.target||(e=a(c).attr("href"))&&e.replace(/.*(?=#[^\s]+$)/,"")||"")+" .nav li > a",this.offsets=a([]),this.targets=a([]),this.activeTarget=null,this.refresh(),this.process()}b.DEFAULTS={offset:10},b.prototype.refresh=function(){var b=this.$element[0]==window?"offset":"position";this.offsets=a([]),this.targets=a([]);{var c=this;this.$body.fi
 nd(this.selector).map(function(){var d=a(this),e=d.data("target")||d.attr("href"),f=/^#./.test(e)&&a(e);return f&&f.length&&f.is(":visible")&&[[f[b]().top+(!a.isWindow(c.$scrollElement.get(0))&&c.$scrollElement.scrollTop()),e]]||null}).sort(function(a,b){return a[0]-b[0]}).each(function(){c.offsets.push(this[0]),c.targets.push(this[1])})}},b.prototype.process=function(){var a,b=this.$scrollElement.scrollTop()+this.options.offset,c=this.$scrollElement[0].scrollHeight||this.$body[0].scrollHeight,d=c-this.$scrollElement.height(),e=this.offsets,f=this.targets,g=this.activeTarget;if(b>=d)return g!=(a=f.last()[0])&&this.activate(a);if(g&&b<=e[0])return g!=(a=f[0])&&this.activate(a);for(a=e.length;a--;)g!=f[a]&&b>=e[a]&&(!e[a+1]||b<=e[a+1])&&this.activate(f[a])},b.prototype.activate=function(b){this.activeTarget=b,a(this.selector).parentsUntil(this.options.target,".active").removeClass("active");var c=this.selector+'[data-target="'+b+'"],'+this.selector+'[href="'+b+'"]',d=a(c).parents("li"
 ).addClass("active");d.parent(".dropdown-menu").length&&(d=d.closest("li.dropdown").addClass("active")),d.trigger("activate.bs.scrollspy")};var c=a.fn.scrollspy;a.fn.scrollspy=function(c){return this.each(function(){var d=a(this),e=d.data("bs.scrollspy"),f="object"==typeof c&&c;e||d.data("bs.scrollspy",e=new b(this,f)),"string"==typeof c&&e[c]()})},a.fn.scrollspy.Constructor=b,a.fn.scrollspy.noConflict=function(){return a.fn.scrollspy=c,this},a(window).on("load",function(){a('[data-spy="scroll"]').each(function(){var b=a(this);b.scrollspy(b.data())})})}(jQuery),+function(a){"use strict";var b=function(b){this.element=a(b)};b.prototype.show=function(){var b=this.element,c=b.closest("ul:not(.dropdown-menu)"),d=b.data("target");if(d||(d=b.attr("href"),d=d&&d.replace(/.*(?=#[^\s]*$)/,"")),!b.parent("li").hasClass("active")){var e=c.find(".active:last a")[0],f=a.Event("show.bs.tab",{relatedTarget:e});if(b.trigger(f),!f.isDefaultPrevented()){var g=a(d);this.activate(b.parent("li"),c),this
 .activate(g,g.parent(),function(){b.trigger({type:"shown.bs.tab",relatedTarget:e})})}}},b.prototype.activate=function(b,c,d){function e(){f.removeClass("active").find("> .dropdown-menu > .active").removeClass("active"),b.addClass("active"),g?(b[0].offsetWidth,b.addClass("in")):b.removeClass("fade"),b.parent(".dropdown-menu")&&b.closest("li.dropdown").addClass("active"),d&&d()}var f=c.find("> .active"),g=d&&a.support.transition&&f.hasClass("fade");g?f.one(a.support.transition.end,e).emulateTransitionEnd(150):e(),f.removeClass("in")};var c=a.fn.tab;a.fn.tab=function(c){return this.each(function(){var d=a(this),e=d.data("bs.tab");e||d.data("bs.tab",e=new b(this)),"string"==typeof c&&e[c]()})},a.fn.tab.Constructor=b,a.fn.tab.noConflict=function(){return a.fn.tab=c,this},a(document).on("click.bs.tab.data-api",'[data-toggle="tab"], [data-toggle="pill"]',function(b){b.preventDefault(),a(this).tab("show")})}(jQuery),+function(a){"use strict";var b=function(c,d){this.options=a.extend({},b.DE
 FAULTS,d),this.$window=a(window).on("scroll.bs.affix.data-api",a.proxy(this.checkPosition,this)).on("click.bs.affix.data-api",a.proxy(this.checkPositionWithEventLoop,this)),this.$element=a(c),this.affixed=this.unpin=this.pinnedOffset=null,this.checkPosition()};b.RESET="affix affix-top affix-bottom",b.DEFAULTS={offset:0},b.prototype.getPinnedOffset=function(){if(this.pinnedOffset)return this.pinnedOffset;this.$element.removeClass(b.RESET).addClass("affix");var a=this.$window.scrollTop(),c=this.$element.offset();return this.pinnedOffset=c.top-a},b.prototype.checkPositionWithEventLoop=function(){setTimeout(a.proxy(this.checkPosition,this),1)},b.prototype.checkPosition=function(){if(this.$element.is(":visible")){var c=a(document).height(),d=this.$window.scrollTop(),e=this.$element.offset(),f=this.options.offset,g=f.top,h=f.bottom;"top"==this.affixed&&(e.top+=d),"object"!=typeof f&&(h=g=f),"function"==typeof g&&(g=f.top(this.$element)),"function"==typeof h&&(h=f.bottom(this.$element));va
 r i=null!=this.unpin&&d+this.unpin<=e.top?!1:null!=h&&e.top+this.$element.height()>=c-h?"bottom":null!=g&&g>=d?"top":!1;if(this.affixed!==i){this.unpin&&this.$element.css("top","");var j="affix"+(i?"-"+i:""),k=a.Event(j+".bs.affix");this.$element.trigger(k),k.isDefaultPrevented()||(this.affixed=i,this.unpin="bottom"==i?this.getPinnedOffset():null,this.$element.removeClass(b.RESET).addClass(j).trigger(a.Event(j.replace("affix","affixed"))),"bottom"==i&&this.$element.offset({top:c-h-this.$element.height()}))}}};var c=a.fn.affix;a.fn.affix=function(c){return this.each(function(){var d=a(this),e=d.data("bs.affix"),f="object"==typeof c&&c;e||d.data("bs.affix",e=new b(this,f)),"string"==typeof c&&e[c]()})},a.fn.affix.Constructor=b,a.fn.affix.noConflict=function(){return a.fn.affix=c,this},a(window).on("load",function(){a('[data-spy="affix"]').each(function(){var b=a(this),c=b.data();c.offset=c.offset||{},c.offsetBottom&&(c.offset.bottom=c.offsetBottom),c.offsetTop&&(c.offset.top=c.offsetT
 op),b.affix(c)})})}(jQuery);
\ No newline at end of file


[22/29] samza git commit: SAMZA-1898: New UI layout for the Samza website

Posted by bo...@apache.org.
SAMZA-1898: New UI layout for the Samza website

- This is still a work in progress!

Author: Angela Murrell <am...@gmail.com>

Reviewers: Jagadish<ja...@apache.org>

Closes #635 from amurrell/master


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

Branch: refs/heads/NewKafkaSystemConsumer
Commit: b8058af0a2e665fd295b9469f77e66e9eb04bd1a
Parents: dec1639
Author: Angela Murrell <am...@gmail.com>
Authored: Fri Sep 21 11:43:51 2018 -0700
Committer: Jagadish <jv...@linkedin.com>
Committed: Fri Sep 21 11:43:51 2018 -0700

----------------------------------------------------------------------
 .gitignore                                      |    3 +-
 build.gradle                                    |    2 +
 docs/_case-studies/TEMPLATE.md                  |   32 +
 docs/_case-studies/digitalsmiths.md             |   31 +
 docs/_case-studies/fortscale.md                 |   30 +
 docs/_case-studies/index.md                     |   49 +
 docs/_case-studies/intuit.md                    |   31 +
 docs/_case-studies/linkedin.md                  |   30 +
 docs/_case-studies/netflix.md                   |   30 +
 docs/_case-studies/optimizely.md                |   58 +
 docs/_case-studies/redfin.md                    |   30 +
 docs/_case-studies/state.md                     |   31 +
 docs/_case-studies/uber.md                      |   30 +
 docs/_committers/TEMPLATE.md                    |   30 +
 docs/_committers/angela-murrell.md              |   28 +
 docs/_committers/boris-shkolnik.md              |   28 +
 docs/_committers/chinmay-soman.md               |   28 +
 docs/_committers/chris-riccomini.md             |   28 +
 docs/_committers/garry-turkington.md            |   28 +
 docs/_committers/jagadish-venkatraman.md        |   28 +
 docs/_committers/jake-maes.md                   |   28 +
 docs/_committers/jakob-homan.md                 |   29 +
 docs/_committers/jay-kreps.md                   |   28 +
 docs/_committers/martin-kleppmann.md            |   28 +
 docs/_committers/navina-ramesh.md               |   28 +
 docs/_committers/prateek-maheshwari.md          |   28 +
 docs/_committers/sriram-subramanian.md          |   28 +
 docs/_committers/wei-song.md                    |   28 +
 docs/_committers/xinyu-liu.md                   |   28 +
 docs/_committers/yan-fang.md                    |   28 +
 docs/_committers/yi-pan.md                      |   28 +
 docs/_committers/zhijie-shen.md                 |   28 +
 docs/_config.yml                                |   17 +
 docs/_includes/footer.html                      |   92 +
 docs/_includes/main-navigation.html             |   39 +
 docs/_layouts/case-study.html                   |  150 ++
 docs/_layouts/default.html                      |  309 +--
 docs/_layouts/news.html                         |   81 +
 docs/_layouts/page.html                         |   96 +-
 docs/_layouts/talks-and-meetups.html            |   42 +
 docs/_meetups/dec-2018.md                       |   46 +
 docs/_meetups/july-2018.md                      |   56 +
 docs/_menu/index.html                           |  160 ++
 ...ncing-the-release-of-apache-samza--0.14.0.md |   76 +
 docs/_news/2018-04-28-something-else.md         |   36 +
 ...ncing-the-release-of-apache-samza--0.14.1.md |   83 +
 docs/_news/TEMPLATE.md                          |   38 +
 docs/_news/index.md                             |   54 +
 docs/_powered-by/TEMPLATE.md                    |   23 +
 docs/_powered-by/intuit.md                      |   22 +
 docs/_powered-by/linkedin.md                    |   22 +
 docs/_powered-by/mobileaware.md                 |   22 +
 docs/_releases/0.10.md                          |   21 +
 docs/_releases/0.11.md                          |   21 +
 docs/_releases/0.12.md                          |   21 +
 docs/_releases/0.13.md                          |   21 +
 docs/_releases/0.14.md                          |   21 +
 docs/_releases/0.7.0.md                         |   21 +
 docs/_releases/0.8.md                           |   21 +
 docs/_releases/0.9.md                           |   21 +
 docs/_releases/TEMPLATE.md                      |   21 +
 docs/_talks/TEMPLATE.md                         |   34 +
 docs/_talks/june-2018--bangalore-kafka-group.md |   30 +
 docs/_talks/november-2017-big-data-span-2017.md |   30 +
 .../november-2018--totally-awesome-summit.md    |   30 +
 ...ber-21-2017--dataworks-summit-sydney-2017.md |   33 +
 docs/community/committers-old.md                |  103 +
 docs/community/committers.html                  |   97 +
 docs/community/committers.md                    |  103 -
 docs/css/bootstrap.css.map                      |    1 -
 docs/css/bootstrap.min.css                      |    7 -
 docs/css/font-awesome.min.css                   |    4 -
 docs/css/google-fonts.css                       |   72 +
 docs/css/ionicons.min.css                       |   11 +
 docs/css/main.new.css                           | 2262 ++++++++++++++++++
 docs/css/ropa-sans.css                          |   25 -
 docs/fonts/ionicons.eot                         |  Bin 0 -> 112650 bytes
 docs/fonts/ionicons.svg                         |  713 ++++++
 docs/fonts/ionicons.ttf                         |  Bin 0 -> 112472 bytes
 docs/fonts/ionicons.woff                        |  Bin 0 -> 65912 bytes
 docs/fonts/ionicons.woff2                       |  Bin 0 -> 50632 bytes
 docs/img/favicon/apple-touch-icon-114x114.png   |  Bin 0 -> 14109 bytes
 docs/img/favicon/apple-touch-icon-120x120.png   |  Bin 0 -> 12879 bytes
 docs/img/favicon/apple-touch-icon-144x144.png   |  Bin 0 -> 18520 bytes
 docs/img/favicon/apple-touch-icon-152x152.png   |  Bin 0 -> 20884 bytes
 docs/img/favicon/apple-touch-icon-57x57.png     |  Bin 0 -> 5351 bytes
 docs/img/favicon/apple-touch-icon-60x60.png     |  Bin 0 -> 5270 bytes
 docs/img/favicon/apple-touch-icon-72x72.png     |  Bin 0 -> 7156 bytes
 docs/img/favicon/apple-touch-icon-76x76.png     |  Bin 0 -> 7707 bytes
 docs/img/favicon/favicon-128.png                |  Bin 0 -> 13302 bytes
 docs/img/favicon/favicon-16x16.png              |  Bin 0 -> 824 bytes
 docs/img/favicon/favicon-196x196.png            |  Bin 0 -> 30623 bytes
 docs/img/favicon/favicon-32x32.png              |  Bin 0 -> 2288 bytes
 docs/img/favicon/favicon-96x96.png              |  Bin 0 -> 10467 bytes
 docs/img/favicon/favicon.ico                    |  Bin 0 -> 34494 bytes
 docs/img/favicon/mstile-144x144.png             |  Bin 0 -> 18520 bytes
 docs/img/favicon/mstile-150x150.png             |  Bin 0 -> 43894 bytes
 docs/img/favicon/mstile-310x150.png             |  Bin 0 -> 102518 bytes
 docs/img/favicon/mstile-310x310.png             |  Bin 0 -> 192956 bytes
 docs/img/favicon/mstile-70x70.png               |  Bin 0 -> 13302 bytes
 docs/img/feather.gif                            |  Bin 0 -> 16647 bytes
 docs/img/samza-just-logo-transparent.png        |  Bin 0 -> 260503 bytes
 docs/img/samza-just-logo.svg                    |  128 +
 docs/img/samza-logo-no-text.png                 |  Bin 0 -> 93664 bytes
 docs/img/samza-logo.png                         |  Bin 0 -> 6412 bytes
 docs/img/samza-logo.svg                         |  174 ++
 docs/img/samza-logo@2x.png                      |  Bin 0 -> 16164 bytes
 docs/index.md                                   |   22 +-
 docs/index.md.bak                               |   39 +
 docs/js/bootstrap.min.js                        |    6 -
 docs/js/jquery-1.11.1.min.js                    |    4 -
 docs/js/jquery.tablesorter.min.js               |    4 -
 docs/js/main.new.js                             |  645 +++++
 docs/learn/tutorials/versioned/index.md         |    3 +-
 docs/meetups/index.html                         |  106 +
 docs/powered-by/index.html                      |   48 +
 docs/startup/preview/index.md                   |    2 +-
 .../startup/releases/versioned/release-notes.md |   17 +-
 docs/talks/index.html                           |   85 +
 119 files changed, 7085 insertions(+), 328 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/samza/blob/b8058af0/.gitignore
----------------------------------------------------------------------
diff --git a/.gitignore b/.gitignore
index 531878e..05d33d0 100644
--- a/.gitignore
+++ b/.gitignore
@@ -30,4 +30,5 @@ out/
 *.patch
 **.pyc
 samza-shell/src/main/visualizer/plan.json
-**/hs_err_pid*.log
\ No newline at end of file
+**/hs_err_pid*.log
+.vscode/

http://git-wip-us.apache.org/repos/asf/samza/blob/b8058af0/build.gradle
----------------------------------------------------------------------
diff --git a/build.gradle b/build.gradle
index 8b68205..48a28f2 100644
--- a/build.gradle
+++ b/build.gradle
@@ -67,6 +67,8 @@ rat {
     '**/bootstrap.min.css',
     '**/bootstrap.min.js',
     '**/build/**',
+    '**/google-fonts.css',
+    '**/ionicons.min.css',
     '**/font-awesome.min.css',
     '**/jquery-1.11.1.min.js',
     '**/jquery.tablesorter.min.js',

http://git-wip-us.apache.org/repos/asf/samza/blob/b8058af0/docs/_case-studies/TEMPLATE.md
----------------------------------------------------------------------
diff --git a/docs/_case-studies/TEMPLATE.md b/docs/_case-studies/TEMPLATE.md
new file mode 100644
index 0000000..0f36ecf
--- /dev/null
+++ b/docs/_case-studies/TEMPLATE.md
@@ -0,0 +1,32 @@
+---
+exclude_from_loop: true # useful for drafts
+layout: case-study # the layout to use
+hide_title: true # so we have control in case-study layout, but can still use page
+title: Totally awesome use-case of samza by Company # title of case study page
+study_domain: company.com # just the domain, not the protocol
+menu_title: Company # what shows up in the menu
+excerpt_separator: <!--more-->
+---
+<!--
+   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.
+-->
+
+Testing the excerpt
+
+<!--more-->
+
+
+Markdown content goes here
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/samza/blob/b8058af0/docs/_case-studies/digitalsmiths.md
----------------------------------------------------------------------
diff --git a/docs/_case-studies/digitalsmiths.md b/docs/_case-studies/digitalsmiths.md
new file mode 100644
index 0000000..339a4f0
--- /dev/null
+++ b/docs/_case-studies/digitalsmiths.md
@@ -0,0 +1,31 @@
+---
+layout: case-study # the layout to use
+hide_title: true # so we have control in case-study layout, but can still use page
+title: Totally awesome use-case of samza by DigitalSmiths # title of case study page
+study_domain: digitalsmiths.com # just the domain, not the protocol
+menu_title: DigitalSmiths # what shows up in the menu
+excerpt_separator: <!--more-->
+---
+<!--
+   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.
+-->
+
+Writing a brief sentence, that is different from the same blurb on the powered-by section, that prepares readers for the case study is a good idea.
+
+<!--more-->
+
+
+Markdown content goes here
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/samza/blob/b8058af0/docs/_case-studies/fortscale.md
----------------------------------------------------------------------
diff --git a/docs/_case-studies/fortscale.md b/docs/_case-studies/fortscale.md
new file mode 100644
index 0000000..38ae4d9
--- /dev/null
+++ b/docs/_case-studies/fortscale.md
@@ -0,0 +1,30 @@
+---
+layout: case-study # the layout to use
+hide_title: true # so we have control in case-study layout, but can still use page
+title: Totally awesome use-case of samza by FortScale # title of case study page
+study_domain: fortscale.com # just the domain, not the protocol
+menu_title: FortScale # what shows up in the menu
+excerpt_separator: <!--more-->
+---
+<!--
+   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.
+-->
+
+Testing the excerpt
+
+<!--more-->
+
+Markdown content goes here
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/samza/blob/b8058af0/docs/_case-studies/index.md
----------------------------------------------------------------------
diff --git a/docs/_case-studies/index.md b/docs/_case-studies/index.md
new file mode 100644
index 0000000..2ab9791
--- /dev/null
+++ b/docs/_case-studies/index.md
@@ -0,0 +1,49 @@
+---
+layout: page
+title: Case Studies
+exclude_from_loop: true
+---
+<!--
+   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.
+-->
+
+Explore the many use-cases of the Samza Framework via our case-studies.
+
+<ul class="case-studies">
+
+  {% for company in site.case-studies %}
+    {% if company.exclude_from_loop %}
+        {% continue %}
+    {% endif %}
+    <li>
+      <a href="{{ company.url }}" title="{{ company.menu_title }}">
+        <span style="background-image: url('https://logo.clearbit.com/{{ company.study_domain }}?size=256');"></span>
+      </a>
+      <div class="study-detail">
+        <a href="https://{{ company.study_domain }}" class="external-link" rel="nofollow">
+          <i class="icon ion-md-share-alt"></i> {{ company.menu_title }}
+        </a>
+        {% if company.excerpt %}
+        <div class="study-description">
+        {{ company.excerpt }}
+        </div>
+        {% endif %}
+        <a class="btn" href="{{ company.url }}">View Case Study</a>
+      </div>
+    </li>
+  {% endfor %}
+
+</ul>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/samza/blob/b8058af0/docs/_case-studies/intuit.md
----------------------------------------------------------------------
diff --git a/docs/_case-studies/intuit.md b/docs/_case-studies/intuit.md
new file mode 100644
index 0000000..6aff6cf
--- /dev/null
+++ b/docs/_case-studies/intuit.md
@@ -0,0 +1,31 @@
+---
+layout: case-study # the layout to use
+hide_title: true # so we have control in case-study layout, but can still use page
+title: Totally awesome use-case of samza by Intuit # title of case study page
+study_domain: intuit.com # just the domain, not the protocol
+menu_title: Intuit # what shows up in the menu
+excerpt_separator: <!--more-->
+---
+<!--
+   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.
+-->
+
+Testing the excerpt
+
+<!--more-->
+
+
+Markdown content goes here
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/samza/blob/b8058af0/docs/_case-studies/linkedin.md
----------------------------------------------------------------------
diff --git a/docs/_case-studies/linkedin.md b/docs/_case-studies/linkedin.md
new file mode 100644
index 0000000..3c35b6b
--- /dev/null
+++ b/docs/_case-studies/linkedin.md
@@ -0,0 +1,30 @@
+---
+layout: case-study # the layout to use
+hide_title: true # so we have control in case-study layout, but can still use page
+title: Totally awesome use-case of samza by LinkedIn # title of case study page
+study_domain: linkedin.com # just the domain, not the protocol
+menu_title: LinkedIn # what shows up in the menu
+excerpt_separator: <!--more-->
+---
+<!--
+   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.
+-->
+
+Testing the excerpt
+
+<!--more-->
+
+Markdown content goes here
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/samza/blob/b8058af0/docs/_case-studies/netflix.md
----------------------------------------------------------------------
diff --git a/docs/_case-studies/netflix.md b/docs/_case-studies/netflix.md
new file mode 100644
index 0000000..7f3cb3b
--- /dev/null
+++ b/docs/_case-studies/netflix.md
@@ -0,0 +1,30 @@
+---
+layout: case-study # the layout to use
+hide_title: true # so we have control in case-study layout, but can still use page
+title: Totally awesome use-case of samza by Netflix # title of case study page
+study_domain: netflix.com # just the domain, not the protocol
+menu_title: Netflix # what shows up in the menu
+excerpt_separator: <!--more-->
+---
+<!--
+   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.
+-->
+
+Testing the excerpt
+
+<!--more-->
+
+Markdown content goes here
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/samza/blob/b8058af0/docs/_case-studies/optimizely.md
----------------------------------------------------------------------
diff --git a/docs/_case-studies/optimizely.md b/docs/_case-studies/optimizely.md
new file mode 100644
index 0000000..8b18233
--- /dev/null
+++ b/docs/_case-studies/optimizely.md
@@ -0,0 +1,58 @@
+---
+layout: case-study
+hide_title: true # so we have control in case-study layout, but can still use page
+title: Real Time Session Aggregation at Optimizely
+study_domain: optimizely.com
+menu_title: Optimizely
+excerpt_separator: <!--more-->
+---
+<!--
+   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.
+-->
+
+Testing the excerpt
+
+<!--more-->
+
+Optimizely is a world’s leading experimentation platform, enabling businesses to deliver continuous experimentation and personalization across websites, mobile apps and connected devices. At Optimizely, billions of events are tracked on a daily basis. Session metrics are among the key metrics provided to their end user in real time. Prior to introducing Samza for realtime computation, the engineering team at Optimizely used HBase to store and serve experimentation data, and Druid for personalization data including session metrics. As business requirements evolved, the Druid-based solution became more and more challenging.
+
+-   Long delays in session metrics caused by M/R jobs
+-   Reprocessing of events due to inability to incrementally update Druid index
+-   Difficulties in scaling dimensions and cardinality
+-   Queries expanding long time periods are expensive
+
+The engineering team at Optimizely decided to move away from Druid and focus on HBase as the store, and introduced stream processing to pre-aggregate and deduplicate session events. They evaluated multiple stream processing platforms and chose Samza as their stream processing platform. In their solution, every session event is tagged with an identifier for up to 30 minutes; upon receiving a session event, the Samza job updates session metadata and aggregates counters for the session that is stored in a local RocksDB state store. At the end of each one-minute window, aggregated session metrics are ingested to HBase. With the new solution
+
+-   The median query latency was reduced from 40+ ms to 5 ms
+-   Session metrics are now available in realtime
+-   HBase query response time is improved due to reduced write-rate
+-   HBase storage requirement are drastically reduced
+-   Lower development effort thanks to out-of-the-box Kafka integration
+ 
+Here is a testimonial from Optimizely
+
+“At Optimizely, we have built the world’s leading experimentation platform, which ingests billions of click-stream events a day from millions of visitors for analysis. Apache Samza has been a great asset to Optimizely's Event ingestion pipeline allowing us to perform large scale, real time stream computing such as aggregations (e.g. session computations) and data enrichment on a multiple billion events / day scale. The programming model, durability and the close integration with Apache Kafka fit our needs perfectly” said Vignesh Sukumar, Senior Engineering Manager at Optimizely”
+
+In addition, stream processing is also applied to other use cases such as data enrichment, event stream partitioning and metrics processing at Optimizely.
+
+Key Samza features: *Stateful processing*, *Windowing*, *Kafka-integration*, *Scalability*, *Fault-tolerant*
+
+More information
+
+-   [https://medium.com/engineers-optimizely/from-batching-to-streaming-real-time-session-metrics-using-samza-part-1-aed2051dd7a3](https://medium.com/engineers-optimizely/from-batching-to-streaming-real-time-session-metrics-using-samza-part-1-aed2051dd7a3)
+    
+-   [https://medium.com/engineers-optimizely/from-batching-to-streaming-real-time-session-metrics-using-samza-part-2-b596350a7820](https://medium.com/engineers-optimizely/from-batching-to-streaming-real-time-session-metrics-using-samza-part-2-b596350a7820)
+    
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/samza/blob/b8058af0/docs/_case-studies/redfin.md
----------------------------------------------------------------------
diff --git a/docs/_case-studies/redfin.md b/docs/_case-studies/redfin.md
new file mode 100644
index 0000000..b4e5098
--- /dev/null
+++ b/docs/_case-studies/redfin.md
@@ -0,0 +1,30 @@
+---
+layout: case-study # the layout to use
+hide_title: true # so we have control in case-study layout, but can still use page
+title: Totally awesome use-case of samza by Redfin # title of case study page
+study_domain: redfin.com # just the domain, not the protocol
+menu_title: Redfin # what shows up in the menu
+excerpt_separator: <!--more-->
+---
+<!--
+   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.
+-->
+
+Testing the excerpt
+
+<!--more-->
+
+Markdown content goes here
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/samza/blob/b8058af0/docs/_case-studies/state.md
----------------------------------------------------------------------
diff --git a/docs/_case-studies/state.md b/docs/_case-studies/state.md
new file mode 100644
index 0000000..419620e
--- /dev/null
+++ b/docs/_case-studies/state.md
@@ -0,0 +1,31 @@
+---
+layout: case-study # the layout to use
+hide_title: true # so we have control in case-study layout, but can still use page
+title: Totally awesome use-case of samza by State # title of case study page
+study_domain: state.com # just the domain, not the protocol
+menu_title: State # what shows up in the menu
+excerpt_separator: <!--more-->
+---
+<!--
+   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.
+-->
+
+Testing the excerpt
+
+<!--more-->
+
+
+Markdown content goes here
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/samza/blob/b8058af0/docs/_case-studies/uber.md
----------------------------------------------------------------------
diff --git a/docs/_case-studies/uber.md b/docs/_case-studies/uber.md
new file mode 100644
index 0000000..64e7a4f
--- /dev/null
+++ b/docs/_case-studies/uber.md
@@ -0,0 +1,30 @@
+---
+layout: case-study # the layout to use
+hide_title: true # so we have control in case-study layout, but can still use page
+title: Totally awesome use-case of samza by Uber # title of case study page
+study_domain: uber.com # just the domain, not the protocol
+menu_title: Uber # what shows up in the menu
+excerpt_separator: <!--more-->
+---
+<!--
+   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.
+-->
+
+Testing the excerpt
+
+<!--more-->
+
+Markdown content goes here
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/samza/blob/b8058af0/docs/_committers/TEMPLATE.md
----------------------------------------------------------------------
diff --git a/docs/_committers/TEMPLATE.md b/docs/_committers/TEMPLATE.md
new file mode 100644
index 0000000..b6b759e
--- /dev/null
+++ b/docs/_committers/TEMPLATE.md
@@ -0,0 +1,30 @@
+---
+exclude_from_loop: true # useful for drafting
+name: # Persons name
+website: # optional - well most of these are
+linkedin: # put full https link
+twitter: # just username eg. bobsherman
+image: # overrides github avatar
+github: #just user name eg. bobsherman
+pmc_member: # boolean, true
+apache_member: # boolean, true
+job_title: # optional
+samza_title:  # title if applicable
+order: # try to space these 5 a part for easier editing
+---
+<!--
+   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.
+-->
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/samza/blob/b8058af0/docs/_committers/angela-murrell.md
----------------------------------------------------------------------
diff --git a/docs/_committers/angela-murrell.md b/docs/_committers/angela-murrell.md
new file mode 100644
index 0000000..d1125f6
--- /dev/null
+++ b/docs/_committers/angela-murrell.md
@@ -0,0 +1,28 @@
+---
+name: Angela Murrell
+website: 
+linkedin: https://www.linkedin.com/in/angela-murrell-92689088/
+twitter: angie_splice
+image: 
+github: amurrell
+pmc_member: false
+job_title:
+samza_title:
+order: 100
+---
+<!--
+   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.
+-->
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/samza/blob/b8058af0/docs/_committers/boris-shkolnik.md
----------------------------------------------------------------------
diff --git a/docs/_committers/boris-shkolnik.md b/docs/_committers/boris-shkolnik.md
new file mode 100644
index 0000000..72b00d9
--- /dev/null
+++ b/docs/_committers/boris-shkolnik.md
@@ -0,0 +1,28 @@
+---
+name: Boris Shkolnik
+website: 
+linkedin: https://www.linkedin.com/in/boryas
+twitter: https://twitter.com/sborya
+image: 
+github: sborya
+pmc_member: false
+job_title:
+samza_title:
+order: 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.
+-->
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/samza/blob/b8058af0/docs/_committers/chinmay-soman.md
----------------------------------------------------------------------
diff --git a/docs/_committers/chinmay-soman.md b/docs/_committers/chinmay-soman.md
new file mode 100644
index 0000000..e8ec9ba
--- /dev/null
+++ b/docs/_committers/chinmay-soman.md
@@ -0,0 +1,28 @@
+---
+name: Chinmay Soman
+website: 
+linkedin: https://www.linkedin.com/pub/chinmay-soman/5/610/35
+twitter: ChinmaySoman
+image: 
+github:
+pmc_member: true
+job_title:
+samza_title:
+order: 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.
+-->
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/samza/blob/b8058af0/docs/_committers/chris-riccomini.md
----------------------------------------------------------------------
diff --git a/docs/_committers/chris-riccomini.md b/docs/_committers/chris-riccomini.md
new file mode 100644
index 0000000..89b162e
--- /dev/null
+++ b/docs/_committers/chris-riccomini.md
@@ -0,0 +1,28 @@
+---
+name: Chris Riccomini
+website: 
+linkedin: http://www.linkedin.com/in/riccomini
+twitter: criccomini
+image: 
+github: criccomini
+pmc_member: true
+job_title:
+samza_title:
+order: 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.
+-->
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/samza/blob/b8058af0/docs/_committers/garry-turkington.md
----------------------------------------------------------------------
diff --git a/docs/_committers/garry-turkington.md b/docs/_committers/garry-turkington.md
new file mode 100644
index 0000000..58e2ac3
--- /dev/null
+++ b/docs/_committers/garry-turkington.md
@@ -0,0 +1,28 @@
+---
+name: Garry Turkington
+website: 
+linkedin: https://uk.linkedin.com/in/garryturkington
+twitter: garryturk
+image: 
+github: garryturk
+pmc_member: true
+job_title:
+samza_title: 
+order: 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.
+-->
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/samza/blob/b8058af0/docs/_committers/jagadish-venkatraman.md
----------------------------------------------------------------------
diff --git a/docs/_committers/jagadish-venkatraman.md b/docs/_committers/jagadish-venkatraman.md
new file mode 100644
index 0000000..62e34e6
--- /dev/null
+++ b/docs/_committers/jagadish-venkatraman.md
@@ -0,0 +1,28 @@
+---
+name: Jagadish Venkatraman
+website: 
+linkedin: https://www.linkedin.com/in/jagadishvenkat
+twitter: vjagadish1989
+image: 
+github: vjagadish1989
+pmc_member: true
+job_title:
+samza_title:
+order: 15
+---
+<!--
+   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.
+-->
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/samza/blob/b8058af0/docs/_committers/jake-maes.md
----------------------------------------------------------------------
diff --git a/docs/_committers/jake-maes.md b/docs/_committers/jake-maes.md
new file mode 100644
index 0000000..a5b29eb
--- /dev/null
+++ b/docs/_committers/jake-maes.md
@@ -0,0 +1,28 @@
+---
+name: Jake Maes
+website: 
+linkedin: https://www.linkedin.com/in/jacobmaes
+twitter: jakemaes
+image: 
+github: jmakes
+pmc_member: true
+job_title:
+samza_title: 
+order: 20
+---
+<!--
+   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.
+-->
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/samza/blob/b8058af0/docs/_committers/jakob-homan.md
----------------------------------------------------------------------
diff --git a/docs/_committers/jakob-homan.md b/docs/_committers/jakob-homan.md
new file mode 100644
index 0000000..4c616e5
--- /dev/null
+++ b/docs/_committers/jakob-homan.md
@@ -0,0 +1,29 @@
+---
+name: Jakob Homan
+website: 
+linkedin: https://www.linkedin.com/in/jghoman
+twitter: blueboxtraveler
+image: 
+github: jghoman
+pmc_member: true
+apache_member: true
+job_title:
+samza_title:
+order: 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.
+-->
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/samza/blob/b8058af0/docs/_committers/jay-kreps.md
----------------------------------------------------------------------
diff --git a/docs/_committers/jay-kreps.md b/docs/_committers/jay-kreps.md
new file mode 100644
index 0000000..9969aea
--- /dev/null
+++ b/docs/_committers/jay-kreps.md
@@ -0,0 +1,28 @@
+---
+name: Jay Kreps
+website: 
+linkedin: https://www.linkedin.com/in/jaykreps
+twitter: jaykreps
+image: 
+github: jkreps
+pmc_member: true
+job_title:
+samza_title:
+order: 60
+---
+<!--
+   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.
+-->
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/samza/blob/b8058af0/docs/_committers/martin-kleppmann.md
----------------------------------------------------------------------
diff --git a/docs/_committers/martin-kleppmann.md b/docs/_committers/martin-kleppmann.md
new file mode 100644
index 0000000..0e6f801
--- /dev/null
+++ b/docs/_committers/martin-kleppmann.md
@@ -0,0 +1,28 @@
+---
+name: Martin Kleppmann
+website: 
+linkedin: https://www.linkedin.com/in/martinkleppmann
+twitter: martinkl
+image: 
+github: ept
+pmc_member: true
+job_title:
+samza_title: 
+order: 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.
+-->
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/samza/blob/b8058af0/docs/_committers/navina-ramesh.md
----------------------------------------------------------------------
diff --git a/docs/_committers/navina-ramesh.md b/docs/_committers/navina-ramesh.md
new file mode 100644
index 0000000..dc0217c
--- /dev/null
+++ b/docs/_committers/navina-ramesh.md
@@ -0,0 +1,28 @@
+---
+name: Navina Ramesh
+website: 
+linkedin: https://www.linkedin.com/in/rnavina
+twitter: navina_r
+image: 
+github: navina
+pmc_member: true
+job_title:
+samza_title: 
+order: 10
+---
+<!--
+   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.
+-->
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/samza/blob/b8058af0/docs/_committers/prateek-maheshwari.md
----------------------------------------------------------------------
diff --git a/docs/_committers/prateek-maheshwari.md b/docs/_committers/prateek-maheshwari.md
new file mode 100644
index 0000000..1cdaeeb
--- /dev/null
+++ b/docs/_committers/prateek-maheshwari.md
@@ -0,0 +1,28 @@
+---
+name: Prateek Maheshwari
+website: 
+linkedin: https://www.linkedin.com/in/mprateek
+twitter:
+image: 
+github: prateekm-li
+pmc_member: false
+job_title:
+samza_title:
+order: 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.
+-->
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/samza/blob/b8058af0/docs/_committers/sriram-subramanian.md
----------------------------------------------------------------------
diff --git a/docs/_committers/sriram-subramanian.md b/docs/_committers/sriram-subramanian.md
new file mode 100644
index 0000000..8c8a079
--- /dev/null
+++ b/docs/_committers/sriram-subramanian.md
@@ -0,0 +1,28 @@
+---
+name: Sriram Subramanian
+website: 
+linkedin: https://www.linkedin.com/pub/sriram-subramanian/3/52a/162
+twitter: sriramsub1
+image: 
+github:
+pmc_member: true
+job_title:
+samza_title:
+order: 65
+---
+<!--
+   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.
+-->
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/samza/blob/b8058af0/docs/_committers/wei-song.md
----------------------------------------------------------------------
diff --git a/docs/_committers/wei-song.md b/docs/_committers/wei-song.md
new file mode 100644
index 0000000..5a83f63
--- /dev/null
+++ b/docs/_committers/wei-song.md
@@ -0,0 +1,28 @@
+---
+name: Wei Song
+website: 
+linkedin: https://www.linkedin.com/in/weisong44
+twitter:
+image: 
+github: weisong44
+pmc_member: false
+job_title:
+samza_title: 
+order: 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.
+-->
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/samza/blob/b8058af0/docs/_committers/xinyu-liu.md
----------------------------------------------------------------------
diff --git a/docs/_committers/xinyu-liu.md b/docs/_committers/xinyu-liu.md
new file mode 100644
index 0000000..872afcf
--- /dev/null
+++ b/docs/_committers/xinyu-liu.md
@@ -0,0 +1,28 @@
+---
+name: Xinyu Liu
+website: 
+linkedin: https://www.linkedin.com/in/xinyu-liu-b0b21648
+twitter: 
+image: 
+github: xinyuiscool
+pmc_member: true
+job_title:
+samza_title:
+order: 25
+---
+<!--
+   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.
+-->
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/samza/blob/b8058af0/docs/_committers/yan-fang.md
----------------------------------------------------------------------
diff --git a/docs/_committers/yan-fang.md b/docs/_committers/yan-fang.md
new file mode 100644
index 0000000..19eb423
--- /dev/null
+++ b/docs/_committers/yan-fang.md
@@ -0,0 +1,28 @@
+---
+name: Yan Fang
+website: 
+linkedin: https://www.linkedin.com/in/yanfangus
+twitter: yanfang724
+image: 
+github: 
+pmc_member: true
+job_title:
+samza_title: 
+order: 40
+---
+<!--
+   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.
+-->
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/samza/blob/b8058af0/docs/_committers/yi-pan.md
----------------------------------------------------------------------
diff --git a/docs/_committers/yi-pan.md b/docs/_committers/yi-pan.md
new file mode 100644
index 0000000..b31c0aa
--- /dev/null
+++ b/docs/_committers/yi-pan.md
@@ -0,0 +1,28 @@
+---
+name: Yi Pan
+website: 
+linkedin: https://www.linkedin.com/pub/yi-pan/9/85a/238
+twitter: nickpan47
+image: 
+github: nickpan47
+pmc_member: true
+job_title:
+samza_title: VP of Apache Samza
+order: 5
+---
+<!--
+   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.
+-->
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/samza/blob/b8058af0/docs/_committers/zhijie-shen.md
----------------------------------------------------------------------
diff --git a/docs/_committers/zhijie-shen.md b/docs/_committers/zhijie-shen.md
new file mode 100644
index 0000000..2dc802e
--- /dev/null
+++ b/docs/_committers/zhijie-shen.md
@@ -0,0 +1,28 @@
+---
+name: Zhijie Shen
+website: 
+linkedin: https://www.linkedin.com/in/zjshen
+twitter: zhijieshen
+image: 
+github: 
+pmc_member: true
+job_title:
+samza_title:
+order: 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.
+-->
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/samza/blob/b8058af0/docs/_config.yml
----------------------------------------------------------------------
diff --git a/docs/_config.yml b/docs/_config.yml
index cfd3b0e..ef29e52 100644
--- a/docs/_config.yml
+++ b/docs/_config.yml
@@ -26,3 +26,20 @@ baseurl: http://samza.apache.org
 version: latest
 # this is the version you will go if you click 'switch version' in "latest" pages.
 latest-release: '0.14'
+collections:
+  menu:
+    output: false
+  news:
+    output: true
+  case-studies:
+    output: true
+  talks:
+    output: false
+  meetups:
+    output: false
+  committers:
+    output: false
+  releases:
+    output: false
+  powered-by:
+    output: false
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/samza/blob/b8058af0/docs/_includes/footer.html
----------------------------------------------------------------------
diff --git a/docs/_includes/footer.html b/docs/_includes/footer.html
new file mode 100644
index 0000000..d64ea62
--- /dev/null
+++ b/docs/_includes/footer.html
@@ -0,0 +1,92 @@
+<!--
+   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.
+-->
+
+<footer>
+  <div class="footer-inner">
+    <div class="side-by-side">
+      <div>
+        <div class="footer__heading">Learn More</div>
+        <div class="footer__items">
+          <a class="footer__item" href="/talks/">Talks</a>
+          <a class="footer__item" href="/meetups/">Meetups</a>
+          <a class="footer__item" href="/news/">News</a>
+          <a class="footer__item" href="/learn/documentation/{{site.version}}/introduction/background.html">About</a>
+        </div>
+      </div>
+      <div>
+        <div class="footer__heading">Community</div>
+        <div class="footer__items">
+          <a class="footer__item" href="/community/mailing-lists.html">Contact Us</a>
+          <a class="footer__item" href="/contribute/contributors-corner.html">Contributors' Corner</a>
+          <a class="footer__item" href="/community/committers.html">PMC members and committers</a>
+          <a class="footer__item" href="/powered-by/">Powered By</a>
+        </div>
+      </div>
+
+      <div>
+        <div class="quick-links">
+          <a class="quick-link" href="/startup/download" target="_blank">
+            <i class="icon ion-md-download"></i>
+          </a>
+          <a class="quick-link" href="https://git-wip-us.apache.org/repos/asf?p=samza.git;a=tree" target="_blank">
+            <i class="icon ion-md-code"></i>
+          </a>
+          <a class="quick-link" href="https://twitter.com/samzastream" target="_blank">
+            <i class="icon ion-logo-twitter"></i>
+          </a>
+        </div>
+
+        <p>
+          <script>document.write(new Date().getFullYear());</script> &copy; samza.apache.org</p>
+      </div>
+
+    </div>
+  </div>
+
+</footer>
+
+{% if page.url contains "versioned" %}
+<script>
+  var tryFile = function (url, cb) {
+    var myRequest = new Request(url);
+    fetch(myRequest).then((response, cb) => {
+      console.log(response.status); // returns 200
+      cb(response.status != 404);
+    });
+  }
+
+  tryFile(window.location.pathname, function (status) {
+    // do something with the status
+    console.log(status);
+  });
+</script> 
+{% endif %}
+
+<!-- Google Analytics -->
+<script>
+  (function (i, s, o, g, r, a, m) {
+    i['GoogleAnalyticsObject'] = r; i[r] = i[r] || function () {
+      (i[r].q = i[r].q || []).push(arguments)
+    }, i[r].l = 1 * new Date(); a = s.createElement(o),
+      m = s.getElementsByTagName(o)[0]; a.async = 1; a.src = g; m.parentNode.insertBefore(a, m)
+  })(window, document, 'script', '//www.google-analytics.com/analytics.js', 'ga');
+
+  ga('create', 'UA-43122768-1', 'apache.org');
+  ga('send', 'pageview');
+
+</script>
+<script src="/js/main.new.js"></script>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/samza/blob/b8058af0/docs/_includes/main-navigation.html
----------------------------------------------------------------------
diff --git a/docs/_includes/main-navigation.html b/docs/_includes/main-navigation.html
new file mode 100644
index 0000000..c0517fc
--- /dev/null
+++ b/docs/_includes/main-navigation.html
@@ -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.
+-->
+<div class="main-navigation" data-plugin="menu">
+  <div class="main-navigation__toggle" data-menu-closed>
+    <i class="icon ion-md-menu"></i>
+  </div>
+  <div class="main-navigation__toggle main-navigation__toggle--opened" data-menu-opened>
+    <i class="icon ion-md-close"></i>
+  </div>
+  <div class="main-navigation__inner">
+    <div class="main-navigation__logo">
+      <a href="/">
+        <img class="main-navigation__logo-img" src="/img/samza-logo.png" srcset="/img/samza-logo.png 1x, /img/samza-logo@2x.png 2x"
+          alt="Samza Logo" />
+      </a>
+    </div>
+    <div class="main-navigation__items" data-menu-opened>
+      <a class="main-navigation__item" href="/case-studies/">Case Studies</a>
+      <a class="main-navigation__item" href="/startup/hello-samza/{{site.version}}">Get Started</a>
+      <a class="main-navigation__item" href="/learn/documentation/{{site.version}}/introduction/background.html">Docs</a>
+      <a class="main-navigation__item" href="/startup/download/">Download</a>
+    </div>
+
+  </div>
+</div>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/samza/blob/b8058af0/docs/_layouts/case-study.html
----------------------------------------------------------------------
diff --git a/docs/_layouts/case-study.html b/docs/_layouts/case-study.html
new file mode 100644
index 0000000..a8b85a3
--- /dev/null
+++ b/docs/_layouts/case-study.html
@@ -0,0 +1,150 @@
+<!DOCTYPE html>
+<!--
+   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.
+-->
+<html lang="en">
+
+<head>
+  <meta http-equiv="X-UA-Compatible" content="IE=edge">
+  <meta name="viewport" content="width=device-width, initial-scale=1, shrink-to-fit=no">
+  <meta charset="utf-8">
+  <title>Samza {% if page.title %}- {{page.title}}{% endif %}</title>
+  <link rel="apple-touch-icon-precomposed" sizes="57x57" href="/img/favicon/apple-touch-icon-57x57.png" />
+  <link rel="apple-touch-icon-precomposed" sizes="114x114" href="/img/favicon/apple-touch-icon-114x114.png" />
+  <link rel="apple-touch-icon-precomposed" sizes="72x72" href="/img/favicon/apple-touch-icon-72x72.png" />
+  <link rel="apple-touch-icon-precomposed" sizes="144x144" href="/img/favicon/apple-touch-icon-144x144.png" />
+  <link rel="apple-touch-icon-precomposed" sizes="60x60" href="/img/favicon/apple-touch-icon-60x60.png" />
+  <link rel="apple-touch-icon-precomposed" sizes="120x120" href="/img/favicon/apple-touch-icon-120x120.png" />
+  <link rel="apple-touch-icon-precomposed" sizes="76x76" href="/img/favicon/apple-touch-icon-76x76.png" />
+  <link rel="apple-touch-icon-precomposed" sizes="152x152" href="/img/favicon/apple-touch-icon-152x152.png" />
+  <link rel="icon" type="image/png" href="/img/favicon/favicon-196x196.png" sizes="196x196" />
+  <link rel="icon" type="image/png" href="/img/favicon/favicon-96x96.png" sizes="96x96" />
+  <link rel="icon" type="image/png" href="/img/favicon/favicon-32x32.png" sizes="32x32" />
+  <link rel="icon" type="image/png" href="/img/favicon/favicon-16x16.png" sizes="16x16" />
+  <link rel="icon" type="image/png" href="/img/favicon/favicon-128.png" sizes="128x128" />
+  <meta name="application-name" content="https://samza.apache.org" />
+  <meta name="msapplication-TileColor" content="#FFFFFF" />
+  <meta name="msapplication-TileImage" content="/img/favicon/mstile-144x144.png" />
+  <meta name="msapplication-square70x70logo" content="/img/favicon/mstile-70x70.png" />
+  <meta name="msapplication-square150x150logo" content="/img/favicon/mstile-150x150.png" />
+  <meta name="msapplication-wide310x150logo" content="/img/favicon/mstile-310x150.png" />
+  <meta name="msapplication-square310x310logo" content="/img/favicon/mstile-310x310.png" />
+  <link href="/css/ionicons.min.css" rel="stylesheet">
+  <link href="/css/google-fonts.css" rel="stylesheet">
+  <link href="/css/syntax.css" rel="stylesheet"/>
+  <link rel="stylesheet" href="/css/main.new.css" />
+</head>
+
+<body class="page page--case-study">
+  {% include main-navigation.html %}
+
+  <div class="section section--bottom-flare section--hero">
+      <div class="case-study-hero">
+          <img src="https://logo.clearbit.com/{{ page.study_domain }}?size=256"/>
+          <h2>{{ page.title }}</h2>
+      </div>
+  </div>
+
+  <div class="container">
+      <div class="container__toggle">
+        <i class="icon ion-md-arrow-dropleft-circle container__toggle-icon"></i>
+        <i class="icon ion-md-arrow-dropright-circle container__toggle-icon container__toggle-icon--opened"></i>
+      </div>
+      
+      <!-- There is only one menu, but made it as a no-output collection to grab data only -->
+      {% for menu in site.menu %}
+        {{ menu.content }}
+      {% endfor %}
+      
+      <div class="section">
+        <div class="content">
+          {% if page.hide_title != true %}
+          <h2>{{ page.title }}</h2>
+          {% endif %}
+
+          {% if page.url contains "/learn/documentation" or page.url contains "/releases/" %}
+
+          <div class="releases-list-container">
+
+            <span>Releases</span>
+
+            <ul class="releases-list" data-releases-list>
+              <li class="hide"><a href="{{ page.url | replace:"versioned","latest"}}">latest</a></li>
+              
+              {% assign releases_sorted = site.releases | sort: 'order' | reverse %}
+              {% for release in releases_sorted %}
+
+              <li class="hide"><a href="{{ page.url | replace:"versioned", release.version }}">{{ release.version }}</a></li>
+
+              {% endfor %}
+
+            </ul>
+
+          </div>
+
+          <hr class="releases-list-divider">
+
+          {% endif %}
+          
+          <h3>
+            <a href="https://{{ page.study_domain }}" rel="nofollow" target="_blank">
+              {{ page.study_domain }}
+            </a>
+          </h3>
+          
+          {{ page.content | remove: page.excerpt }}
+          
+          <hr/>
+          
+          <h3>
+            More Case Studies...
+          </h3>
+
+          <div class="case-studies">
+          
+              <div class="case-studies-list">
+          
+              {% for study in site.case-studies %}
+          
+                  {% if study.exclude_from_loop or study.url == page.url %}
+                      {% continue %}
+                  {% endif %}
+          
+                  <div class="case-studies-list__item">
+                      <a class="case-studies-list__item-logo" href="{{ study.url }}" title="{{ study.menu_title }}">
+                        <div class="case-studies-list__item-logo-bg" style="background-image: url('https://logo.clearbit.com/{{ study.study_domain}}?size=512');"></div>
+                      </a>
+                  </div>
+          
+              {% endfor %}
+          
+              </div>
+          
+          </div>
+           
+        </div>
+      </div>
+
+  </div>
+  
+
+  <!-- footer starts here -->
+
+  {% include footer.html %}
+
+</body>
+
+</html>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/samza/blob/b8058af0/docs/_layouts/default.html
----------------------------------------------------------------------
diff --git a/docs/_layouts/default.html b/docs/_layouts/default.html
index 7597657..4256c7e 100644
--- a/docs/_layouts/default.html
+++ b/docs/_layouts/default.html
@@ -16,148 +16,183 @@
    limitations under the License.
 -->
 <html lang="en">
-  <head>
-    <meta charset="utf-8">
-    <title>Samza {% if page.title %}- {{page.title}}{% endif %}</title>
-    <link href='/css/ropa-sans.css' rel='stylesheet' type='text/css'/>
-    <link href="/css/bootstrap.min.css" rel="stylesheet"/>
-    <link href="/css/font-awesome.min.css" rel="stylesheet"/>
-    <link href="/css/main.css" rel="stylesheet"/>
-    <link href="/css/syntax.css" rel="stylesheet"/>
-    <link rel="icon" type="image/png" href="/img/samza-icon.png">
-    <script src="/js/jquery-1.11.1.min.js"></script>
-  </head>
-  <body>
-    <div class="wrapper">
-      <div class="wrapper-content">
-
-        <div class="masthead">
-          <div class="container">
-            <div class="masthead-logo">
-              <a href="/" class="logo">samza</a>
-            </div>
-            <div class="masthead-icons">
-              <div class="pull-right">
-                <a href="/startup/download"><i class="fa fa-arrow-circle-o-down masthead-icon"></i></a>
-                <a href="https://git-wip-us.apache.org/repos/asf?p=samza.git;a=tree" target="_blank"><i class="fa fa-code masthead-icon" style="font-weight: bold;"></i></a>
-                <a href="https://twitter.com/samzastream" target="_blank"><i class="fa fa-twitter masthead-icon"></i></a>
-                <!-- this icon only shows in versioned pages -->
-                {% if page.url contains "versioned" %}
-                  {% if site.version == "latest" %}
-                    {% capture linkUrl %}{{page.url | replace:'versioned', site.latest-release | replace_first: '/', ''}}{% endcapture %}
-                  {% else %}
-                    {% capture linkUrl %}{{page.url | replace:'versioned', 'latest' | replace_first: '/', ''}}{% endcapture %}
-                  {% endif %}
-                  <a href="{{ site.baseurl }}/{{ linkUrl }}"><i id="switch-version-button"></i></a>
-                  {% assign navLink = site.version %} <!-- links for the navigation bar -->
-                {% else %}
-                  {% assign navLink = site.latest-release %}
-                {% endif %}
-
-              </div>
-            </div>
-          </div><!-- /.container -->
+
+<head>
+  <meta http-equiv="X-UA-Compatible" content="IE=edge">
+  <meta name="viewport" content="width=device-width, initial-scale=1, shrink-to-fit=no">
+  <meta charset="utf-8">
+  <title>Samza {% if page.title %}- {{page.title}}{% endif %}</title>
+  <link rel="apple-touch-icon-precomposed" sizes="57x57" href="/img/favicon/apple-touch-icon-57x57.png" />
+  <link rel="apple-touch-icon-precomposed" sizes="114x114" href="/img/favicon/apple-touch-icon-114x114.png" />
+  <link rel="apple-touch-icon-precomposed" sizes="72x72" href="/img/favicon/apple-touch-icon-72x72.png" />
+  <link rel="apple-touch-icon-precomposed" sizes="144x144" href="/img/favicon/apple-touch-icon-144x144.png" />
+  <link rel="apple-touch-icon-precomposed" sizes="60x60" href="/img/favicon/apple-touch-icon-60x60.png" />
+  <link rel="apple-touch-icon-precomposed" sizes="120x120" href="/img/favicon/apple-touch-icon-120x120.png" />
+  <link rel="apple-touch-icon-precomposed" sizes="76x76" href="/img/favicon/apple-touch-icon-76x76.png" />
+  <link rel="apple-touch-icon-precomposed" sizes="152x152" href="/img/favicon/apple-touch-icon-152x152.png" />
+  <link rel="icon" type="image/png" href="/img/favicon/favicon-196x196.png" sizes="196x196" />
+  <link rel="icon" type="image/png" href="/img/favicon/favicon-96x96.png" sizes="96x96" />
+  <link rel="icon" type="image/png" href="/img/favicon/favicon-32x32.png" sizes="32x32" />
+  <link rel="icon" type="image/png" href="/img/favicon/favicon-16x16.png" sizes="16x16" />
+  <link rel="icon" type="image/png" href="/img/favicon/favicon-128.png" sizes="128x128" />
+  <meta name="application-name" content="https://samza.apache.org" />
+  <meta name="msapplication-TileColor" content="#FFFFFF" />
+  <meta name="msapplication-TileImage" content="/img/favicon/mstile-144x144.png" />
+  <meta name="msapplication-square70x70logo" content="/img/favicon/mstile-70x70.png" />
+  <meta name="msapplication-square150x150logo" content="/img/favicon/mstile-150x150.png" />
+  <meta name="msapplication-wide310x150logo" content="/img/favicon/mstile-310x150.png" />
+  <meta name="msapplication-square310x310logo" content="/img/favicon/mstile-310x310.png" />
+  <link href="/css/ionicons.min.css" rel="stylesheet">
+  <link href="/css/google-fonts.css" rel="stylesheet">
+  <link rel="stylesheet" href="/css/main.new.css" />
+</head>
+
+<body class="home">
+  {% include main-navigation.html %}
+
+  <div class="section section--bottom-flare section--hero" id="hero">
+    <canvas id="canvas"></canvas>
+    <div>
+      <h1 class="section__title">
+        Build powerful tools with samza stream processing
+      </h1>
+      <h2 class="section__title section__title--sub">
+        View our
+        <a href="somewhere">case studies</a> to see examples from leading brands
+      </h2>
+      <div class="content">
+        <a class="button" href="/startup/hello-samza/{{site.version}}">
+          Quick Start
+        </a>
+      </div>
+    </div>
+  </div>
+
+
+  <div class="section section--what-is-samza">
+    <div class="section__title">What is Apache Samza?</div>
+    <div class="content">
+      <p>
+        <strong>Apache Samza</strong>, a top level project of the
+        <a href="https://www.apache.org/" target="_blank">Apache Software Foundation</a>, is a distributed stream processing framework. It uses
+        <a target="_blank" href="https://kafka.apache.org">Apache Kafka</a> for messaging, and
+        <a target="_blank" href="https://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.
+      </p>
+    </div>
+  </div>
+
+  <div class="section section--highlight section--bottom-flare section--features">
+    <div class="section__title section__title--center">
+      Features of the Samza Framework
+    </div>
+    <div class="section__items">
+      <div class="section__item">
+        <i class="section__item-icon icon ion-md-flash"></i>
+        <div class="section__item-title">
+          Performance
+        </div>
+        <div class="section__item-features">
+          <div class="section__item-feature">Extremely high throughput</div>
+          <div class="section__item-feature">Low latency streaming</div>
+          <div class="section__item-feature">Async i/O support</div>
+          <div class="section__item-feature">Configurable Parallelism</div>
         </div>
+      </div>
 
-        <div class="container">
-          <div class="menu">
-            <h1><i class="fa fa-rocket"></i> Getting Started</h1>
-            <ul>
-              <li><a href="/startup/hello-samza/{{ navLink }}">Hello Samza</a></li>
-              <li><a href="/startup/download">Download</a></li>
-              <li><a href="/startup/preview">Feature Preview</a></li>
-            </ul>
-
-            <h1><i class="fa fa-book"></i> Learn</h1>
-            <ul>
-              <li><a href="/learn/documentation/{{ navLink }}">Documentation</a></li>
-              <li><a href="/learn/documentation/{{ navLink }}/jobs/configuration-table.html">Configuration</a></li>
-              <li><a href="/learn/documentation/{{ navLink }}/container/metrics-table.html">Metrics</a></li>
-              <li><a href="/learn/documentation/{{ navLink }}/api/javadocs/">Javadocs</a></li>
-              <li><a href="/learn/tutorials/{{ navLink }}">Tutorials</a></li>
-              <li><a href="https://cwiki.apache.org/confluence/display/SAMZA/FAQ">FAQ</a></li>
-              <li><a href="https://cwiki.apache.org/confluence/display/SAMZA/Apache+Samza">Wiki</a></li>
-              <li><a href="https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=51812876">Papers &amp; Talks</a></li>
-              <li><a href="http://blogs.apache.org/samza">Blog</a></li>
-            </ul>
-
-            <h1><i class="fa fa-comments"></i> Community</h1>
-            <ul>
-              <li><a href="/community/mailing-lists.html">Mailing Lists</a></li>
-              <li><a href="/community/irc.html">IRC</a></li>
-              <li><a href="https://issues.apache.org/jira/browse/SAMZA">Bugs</a></li>
-              <li><a href="https://cwiki.apache.org/confluence/display/SAMZA/Powered+By">Powered by</a></li>
-              <li><a href="https://cwiki.apache.org/confluence/display/SAMZA/Ecosystem">Ecosystem</a></li>
-              <li><a href="/community/committers.html">Committers</a></li>
-            </ul>
-
-            <h1><i class="fa fa-code"></i> Contribute</h1>
-            <ul>
-              <li><a href="/contribute/contributors-corner.html">Contributor's Corner</a></li>
-              <li><a href="/contribute/coding-guide.html">Coding Guide</a></li>
-              <li><a href="/contribute/design-documents.html">Design Documents</a></li>
-              <li><a href="/contribute/code.html">Code</a></li>
-              <li><a href="/contribute/tests.html">Tests</a></li>
-            </ul>
-
-            <h1><i class="fa fa-history"></i> Archive</h1>
-            <ul>
-              <li><a href="/archive/index.html#latest">latest</a></li>
-              <li><a href="/archive/index.html#14">0.14</a></li>
-              <li><a href="/archive/index.html#13">0.13</a></li>
-              <li><a href="/archive/index.html#12">0.12</a></li>
-              <li><a href="/archive/index.html#11">0.11</a></li>
-              <li><a href="/archive/index.html#10">0.10</a></li>
-              <li><a href="/archive/index.html#09">0.9</a></li>
-              <li><a href="/archive/index.html#08">0.8</a></li>
-              <li><a href="/archive/index.html#07">0.7</a></li>
-            </ul>
-          </div>
-
-          <div class="content">
-            {{content}}
-          </div>
+      <div class="section__item">
+        <i class="section__item-icon icon ion-md-trending-up"></i>
+        <div class="section__item-title">
+          Massive scale
         </div>
+        <div class="section__item-features">
+          <div class="section__item-feature">Incremental checkpointing</div>
+          <div class="section__item-feature">Host affinity</div>
+          <div class="section__item-feature">Terabytes of state</div>
+          <div class="section__item-feature">Fault tolerance</div>
+        </div>
+      </div>
 
-      </div><!-- /.wrapper-content -->
-    </div><!-- /.wrapper -->
+      <div class="section__item">
+        <i class="section__item-icon icon ion-md-build"></i>
+        <div class="section__item-title">
+          Operability
+        </div>
+        <div class="section__item-features">
+          <div class="section__item-feature">Backpressure handling</div>
+          <div class="section__item-feature">High availability</div>
+          <div class="section__item-feature">Rolling upgrades</div>
+        </div>
+      </div>
 
-    <div class="footer">
-      <div class="container">
-        <!-- nothing for now. -->
+      <div class="section__item">
+        <i class="section__item-icon icon ion-md-cloud"></i>
+        <div class="section__item-title">
+          Powerful APIs
+        </div>
+        <div class="section__item-features">
+          <div class="section__item-feature">Per-message low-level</div>
+          <div class="section__item-feature">Stream based high-level DSL</div>
+          <div class="section__item-feature">Declarative SQL</div>
+          <div class="section__item-feature">Apache BEAM</div>
+        </div>
+      </div>
+
+      <div class="section__item">
+        <i class="section__item-icon icon ion-md-code"></i>
+        <div class="section__item-title">
+          Write once, Run Anywhere
+        </div>
+        <div class="section__item-features">
+          <div class="section__item-feature">Batch vs streaming</div>
+          <div class="section__item-feature">Standalone vs YARN</div>
+          <div class="section__item-feature">Embedded library vs service</div>
+        </div>
       </div>
+
+      <div class="section__item">
+        <i class="section__item-icon icon ion-md-color-wand"></i>
+        <div class="section__item-title">
+          Pluggable architecture
+        </div>
+        <div class="section__item-features">
+          <div class="section__item-feature">Integrate with Kubernetes, Mesos, Yarn</div>
+          <div class="section__item-feature">Consume from EventHubs, Kafka, Kinesis</div>
+        </div>
+      </div>
+
+    </div>
+
+  </div>
+
+  <div class="section section--brands">
+    <div class="section__title section__title--center">
+      Case Studies
+    </div>
+
+    <div class="section__items">
+
+      {% for study in site.case-studies %}
+
+      {% if study.exclude_from_loop %}
+        {% continue %}
+      {% endif %}
+
+      <div class="section__item">
+        <a class="section__item-logo" href="{{ study.url }}" title="{{ study.menu_title }}">
+          <div class="section__item-logo-bg" style="background-image: url('https://logo.clearbit.com/{{ study.study_domain}}?size=256');"></div>
+        </a>
+      </div>
+      
+      {% endfor %}
+
     </div>
 
-  {% if page.url contains "versioned" %}
-    <script>
-      $( document ).ready(function() {
-        if ( $.fn.urlExists( "/{{ linkUrl }}" ) ) {
-          $("#switch-version-button").addClass("fa fa-history masthead-icon");
-        }
-      });
-
-      /* a function to test whether the url exists or not */
-      (function( $ ) {
-        $.fn.urlExists = function(url) {
-          var http = new XMLHttpRequest();
-          http.open('HEAD', url, false);
-          http.send();
-          return http.status != 404;
-        };
-      }( jQuery ));
-    </script>
-  {% endif %}
-
-    <!-- Google Analytics -->
-    <script>
-      (function(i,s,o,g,r,a,m){i['GoogleAnalyticsObject']=r;i[r]=i[r]||function(){
-      (i[r].q=i[r].q||[]).push(arguments)},i[r].l=1*new Date();a=s.createElement(o),
-      m=s.getElementsByTagName(o)[0];a.async=1;a.src=g;m.parentNode.insertBefore(a,m)
-      })(window,document,'script','//www.google-analytics.com/analytics.js','ga');
-
-      ga('create', 'UA-43122768-1', 'apache.org');
-      ga('send', 'pageview');
-
-    </script>
-  </body>
-</html>
+  </div>
+
+  <!-- footer starts here -->
+
+  {% include footer.html %}
+  
+</body>
+
+</html>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/samza/blob/b8058af0/docs/_layouts/news.html
----------------------------------------------------------------------
diff --git a/docs/_layouts/news.html b/docs/_layouts/news.html
new file mode 100644
index 0000000..69c16e5
--- /dev/null
+++ b/docs/_layouts/news.html
@@ -0,0 +1,81 @@
+---
+layout: page
+---
+<!--
+   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.
+-->
+
+<div class="breadcrumbs-list-container">
+
+    <span>{{ page.date | date: "%B %-d, %Y"}}</span>
+
+    <ul class="breadcrumbs-list">
+        <li class=""><a href="/news/">News</a></li>
+        <li class="divider"></li>
+        {% if page.previous and page.previous.exclude_from_loop != true %}
+        <li class="">
+            <i class="icon ion-md-arrow-dropleft"></i>
+            <a href="{{ page.previous.url }}">Previous</a>
+        </li>
+        {% endif %}
+        {% if page.next and page.next.exclude_from_loop != true %}
+        <li class="" style="float: right;">
+            <a href="{{ page.next.url}}">Next</a>
+            <i class="icon ion-md-arrow-dropright"></i>
+        </li>
+        {% endif %}
+    </ul>
+
+</div>
+
+<hr class="breadcrumbs-list-divider">
+
+{{ page.content | remove: page.excerpt }}
+
+{% assign show_continue = false %}
+
+{% if page.previous and page.previous.exclude_from_loop != true %}
+    {% assign show_continue = true %}
+{% endif %}
+
+{% if page.next and page.next.exclude_from_loop != true %}
+    {% assign show_continue = true %}
+{% endif %}
+
+{% if show_continue %}
+<hr/>
+<h5>Continue Reading</h5>
+
+<div class="news-end">
+    {% if page.previous and page.previous.exclude_from_loop != true %}
+    <a class="news-previous news-end-action" href="{{ page.previous.url }}">
+        <div class="news-end-action-title">
+            <i class="icon ion-md-arrow-dropleft-circle"></i>
+            {{ page.previous.title }}
+        </div>
+    </a>
+    {% endif %}
+
+    {% if page.next and page.next.exclude_from_loop != true %}
+    <a class="news-next news-end-action" href="{{ page.next.url }}">
+        <div class="news-end-action-title">
+            {{ page.next.title }}
+            <i class="icon ion-md-arrow-dropright-circle"></i>
+        </div>
+    </a>
+    {% endif %}
+</div>
+{% endif %}
\ No newline at end of file


[20/29] samza git commit: SAMZA-1898: New UI layout for the Samza website

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/samza/blob/b8058af0/docs/css/bootstrap.css.map
----------------------------------------------------------------------
diff --git a/docs/css/bootstrap.css.map b/docs/css/bootstrap.css.map
deleted file mode 100644
index 6bc5a2d..0000000
--- a/docs/css/bootstrap.css.map
+++ /dev/null
@@ -1 +0,0 @@
-{"version":3,"sources":["less/normalize.less","less/print.less","less/scaffolding.less","less/mixins.less","less/variables.less","less/thumbnails.less","less/carousel.less","less/type.less","less/code.less","less/grid.less","less/tables.less","less/forms.less","less/buttons.less","less/button-groups.less","less/component-animations.less","less/glyphicons.less","less/dropdowns.less","less/input-groups.less","less/navs.less","less/navbar.less","less/utilities.less","less/breadcrumbs.less","less/pagination.less","less/pager.less","less/labels.less","less/badges.less","less/jumbotron.less","less/alerts.less","less/progress-bars.less","less/media.less","less/list-group.less","less/panels.less","less/wells.less","less/close.less","less/modals.less","less/tooltip.less","less/popovers.less","less/responsive-utilities.less"],"names":[],"mappings":";AAQA;EACE,uBAAA;EACA,0BAAA;EACA,8BAAA;;AAOF;EACE,SAAA;;AAUF;AACA;AACA;AACA;AACA;AACA;AACA;AACA;AACA;AACA;AACA;AACA;EACE,cAAA;;AAQF;AACA;AACA;AACA
 ;EACE,qBAAA;EACA,wBAAA;;AAQF,KAAK,IAAI;EACP,aAAA;EACA,SAAA;;AAQF;AACA;EACE,aAAA;;AAUF;EACE,uBAAA;;AAOF,CAAC;AACD,CAAC;EACC,UAAA;;AAUF,IAAI;EACF,yBAAA;;AAOF;AACA;EACE,iBAAA;;AAOF;EACE,kBAAA;;AAQF;EACE,cAAA;EACA,gBAAA;;AAOF;EACE,gBAAA;EACA,WAAA;;AAOF;EACE,cAAA;;AAOF;AACA;EACE,cAAA;EACA,cAAA;EACA,kBAAA;EACA,wBAAA;;AAGF;EACE,WAAA;;AAGF;EACE,eAAA;;AAUF;EACE,SAAA;;AAOF,GAAG,IAAI;EACL,gBAAA;;AAUF;EACE,gBAAA;;AAOF;EACE,4BAAA;EACA,uBAAA;EACA,SAAA;;AAOF;EACE,cAAA;;AAOF;AACA;AACA;AACA;EACE,iCAAA;EACA,cAAA;;AAkBF;AACA;AACA;AACA;AACA;EACE,cAAA;EACA,aAAA;EACA,SAAA;;AAOF;EACE,iBAAA;;AAUF;AACA;EACE,oBAAA;;AAWF;AACA,IAAK,MAAK;AACV,KAAK;AACL,KAAK;EACH,0BAAA;EACA,eAAA;;AAOF,MAAM;AACN,IAAK,MAAK;EACR,eAAA;;AAOF,MAAM;AACN,KAAK;EACH,SAAA;EACA,UAAA;;AAQF;EACE,mBAAA;;AAWF,KAAK;AACL,KAAK;EACH,sBAAA;EACA,UAAA;;AASF,KAAK,eAAe;AACpB,KAAK,eAAe;EAClB,YAAA;;AASF,KAAK;EACH,6BAAA;EACA,4BAAA;EACA,+BAAA;EACA,uBAAA;;AASF,KAAK,eAAe;AACpB,KAAK,eAAe;EAClB,wBAAA;;AAOF;EACE,yBAAA;EACA,aAAA;EACA,8BAAA;;AAQF;EACE,SAAA;EACA,UA
 AA;;AAOF;EACE,cAAA;;AAQF;EACE,iBAAA;;AAUF;EACE,yBAAA;EACA,iBAAA;;AAGF;AACA;EACE,UAAA;;AChUF;EA9FE;IACE,4BAAA;IACA,sBAAA;IACA,kCAAA;IACA,2BAAA;;EAGF;EACA,CAAC;IACC,0BAAA;;EAGF,CAAC,MAAM;IACL,SAAS,KAAK,WAAW,GAAzB;;EAGF,IAAI,OAAO;IACT,SAAS,KAAK,YAAY,GAA1B;;EAIF,CAAC,qBAAqB;EACtB,CAAC,WAAW;IACV,SAAS,EAAT;;EAGF;EACA;IACE,sBAAA;IACA,wBAAA;;EAGF;IACE,2BAAA;;EAGF;EACA;IACE,wBAAA;;EAGF;IACE,0BAAA;;EAGF;EACA;EACA;IACE,UAAA;IACA,SAAA;;EAGF;EACA;IACE,uBAAA;;EAKF;IACE,2BAAA;;EAIF;IACE,aAAA;;EAEF,MACE;EADF,MAEE;IACE,iCAAA;;EAGJ,IAEE;EADF,OAAQ,OACN;IACE,iCAAA;;EAGJ;IACE,sBAAA;;EAGF;IACE,oCAAA;;EAEF,eACE;EADF,eAEE;IACE,iCAAA;;;ACtFN;ECyOE,8BAAA;EACG,2BAAA;EACK,sBAAA;;ADxOV,CAAC;AACD,CAAC;ECqOC,8BAAA;EACG,2BAAA;EACK,sBAAA;;ADhOV;EACE,gBAAA;EACA,6CAAA;;AAGF;EACE,aEcwB,8CFdxB;EACA,eAAA;EACA,uBAAA;EACA,cAAA;EACA,yBAAA;;AAIF;AACA;AACA;AACA;EACE,oBAAA;EACA,kBAAA;EACA,oBAAA;;AAMF;EACE,cAAA;EACA,qBAAA;;AAEA,CAAC;AACD,CAAC;EACC,cAAA;EACA,0BAAA;;AAGF,CAAC;ECzBD,oBAAA;EAEA,0CAAA;EACA,oBAAA;;ADiCF;EACE,SAAA;;A
 AMF;EACE,sBAAA;;AAIF;AG1EA,UAUE;AAVF,UAWE,EAAE;ACPJ,eAKE,QAME;AAXJ,eAKE,QAOE,IAAI;EHyWN,cAAA;EACA,eAAA;EACA,YAAA;;AD5SF;EACE,kBAAA;;AAMF;EACE,YAAA;EACA,uBAAA;EACA,yBAAA;EACA,yBAAA;EACA,kBAAA;EC8BA,wCAAA;EACQ,gCAAA;EA+PR,qBAAA;EACA,eAAA;EACA,YAAA;;ADxRF;EACE,kBAAA;;AAMF;EACE,gBAAA;EACA,mBAAA;EACA,SAAA;EACA,6BAAA;;AAQF;EACE,kBAAA;EACA,UAAA;EACA,WAAA;EACA,YAAA;EACA,UAAA;EACA,gBAAA;EACA,MAAM,gBAAN;EACA,SAAA;;AK5HF;AAAI;AAAI;AAAI;AAAI;AAAI;AACpB;AAAK;AAAK;AAAK;AAAK;AAAK;EACvB,oBAAA;EACA,gBAAA;EACA,gBAAA;EACA,cAAA;;AALF,EAOE;AAPE,EAOF;AAPM,EAON;AAPU,EAOV;AAPc,EAOd;AAPkB,EAOlB;AANF,GAME;AANG,GAMH;AANQ,GAMR;AANa,GAMb;AANkB,GAMlB;AANuB,GAMvB;AAPF,EAQE;AARE,EAQF;AARM,EAQN;AARU,EAQV;AARc,EAQd;AARkB,EAQlB;AAPF,GAOE;AAPG,GAOH;AAPQ,GAOR;AAPa,GAOb;AAPkB,GAOlB;AAPuB,GAOvB;EACE,mBAAA;EACA,cAAA;EACA,cAAA;;AAIJ;AAAI;AACJ;AAAI;AACJ;AAAI;EACF,gBAAA;EACA,mBAAA;;AAJF,EAME;AANE,GAMF;AALF,EAKE;AALE,GAKF;AAJF,EAIE;AAJE,GAIF;AANF,EAOE;AAPE,GAOF;AANF,EAME;AANE,GAMF;AALF,EAKE;AALE,GAKF;EACE,cAAA;;AAGJ;AAAI;AACJ
 ;AAAI;AACJ;AAAI;EACF,gBAAA;EACA,mBAAA;;AAJF,EAME;AANE,GAMF;AALF,EAKE;AALE,GAKF;AAJF,EAIE;AAJE,GAIF;AANF,EAOE;AAPE,GAOF;AANF,EAME;AANE,GAMF;AALF,EAKE;AALE,GAKF;EACE,cAAA;;AAIJ;AAAI;EAAM,eAAA;;AACV;AAAI;EAAM,eAAA;;AACV;AAAI;EAAM,eAAA;;AACV;AAAI;EAAM,eAAA;;AACV;AAAI;EAAM,eAAA;;AACV;AAAI;EAAM,eAAA;;AAMV;EACE,gBAAA;;AAGF;EACE,mBAAA;EACA,eAAA;EACA,gBAAA;EACA,gBAAA;;AAKF,QAHqC;EAGrC;IAFI,eAAA;;;AASJ;AACA;EAAU,cAAA;;AAGV;EAAU,kBAAA;;AAGV;EAAuB,gBAAA;;AACvB;EAAuB,iBAAA;;AACvB;EAAuB,kBAAA;;AACvB;EAAuB,mBAAA;;AAGvB;EACE,cAAA;;AAEF;EJofE,cAAA;;AACA,CAAC,aAAC;EACA,cAAA;;AInfJ;EJifE,cAAA;;AACA,CAAC,aAAC;EACA,cAAA;;AIhfJ;EJ8eE,cAAA;;AACA,CAAC,UAAC;EACA,cAAA;;AI7eJ;EJ2eE,cAAA;;AACA,CAAC,aAAC;EACA,cAAA;;AI1eJ;EJweE,cAAA;;AACA,CAAC,YAAC;EACA,cAAA;;AIneJ;EAGE,WAAA;EJqdA,yBAAA;;AACA,CAAC,WAAC;EACA,yBAAA;;AIpdJ;EJkdE,yBAAA;;AACA,CAAC,WAAC;EACA,yBAAA;;AIjdJ;EJ+cE,yBAAA;;AACA,CAAC,QAAC;EACA,yBAAA;;AI9cJ;EJ4cE,yBAAA;;AACA,CAAC,WAAC;EACA,yBAAA;;AI3cJ;EJycE,yBAAA;;AACA,CAAC,UAAC;EACA,yBAAA;;AIncJ;EACE,mBAAA;
 EACA,mBAAA;EACA,gCAAA;;AAQF;AACA;EACE,aAAA;EACA,mBAAA;;AAHF,EAIE;AAHF,EAGE;AAJF,EAKE;AAJF,EAIE;EACE,gBAAA;;AAOJ;EACE,eAAA;EACA,gBAAA;;AAIF;EALE,eAAA;EACA,gBAAA;EAMA,iBAAA;;AAFF,YAIE;EACE,qBAAA;EACA,iBAAA;EACA,kBAAA;;AAKJ;EACE,aAAA;EACA,mBAAA;;AAEF;AACA;EACE,uBAAA;;AAEF;EACE,iBAAA;;AAEF;EACE,cAAA;;AAwBF,QAhB2C;EACzC,cACE;IACE,WAAA;IACA,YAAA;IACA,WAAA;IACA,iBAAA;IJ1IJ,gBAAA;IACA,uBAAA;IACA,mBAAA;;EImIA,cAQE;IACE,kBAAA;;;AAUN,IAAI;AAEJ,IAAI;EACF,YAAA;EACA,iCAAA;;AAEF;EACE,cAAA;EACA,yBAAA;;AAIF;EACE,kBAAA;EACA,gBAAA;EACA,iBAAA;EACA,8BAAA;;AAKE,UAHF,EAGG;AAAD,UAFF,GAEG;AAAD,UADF,GACG;EACC,gBAAA;;AAVN,UAgBE;AAhBF,UAiBE;AAjBF,UAkBE;EACE,cAAA;EACA,cAAA;EACA,uBAAA;EACA,cAAA;;AAEA,UARF,OAQG;AAAD,UAPF,MAOG;AAAD,UANF,OAMG;EACC,SAAS,aAAT;;AAQN;AACA,UAAU;EACR,mBAAA;EACA,eAAA;EACA,+BAAA;EACA,cAAA;EACA,iBAAA;;AAME,mBAHF,OAGG;AAAD,UAXM,WAQR,OAGG;AAAD,mBAFF,MAEG;AAAD,UAXM,WASR,MAEG;AAAD,mBADF,OACG;AAAD,UAXM,WAUR,OACG;EAAU,SAAS,EAAT;;AACX,mBAJF,OAIG;AAAD,UAZM,WAQR,OAIG;AAAD,mBAHF,MAGG;AAAD,UAZM,WASR,M
 AGG;AAAD,mBAFF,OAEG;AAAD,UAZM,WAUR,OAEG;EACC,SAAS,aAAT;;AAMN,UAAU;AACV,UAAU;EACR,SAAS,EAAT;;AAIF;EACE,mBAAA;EACA,kBAAA;EACA,uBAAA;;AC7RF;AACA;AACA;AACA;EACE,sCJkCiD,wBIlCjD;;AAIF;EACE,gBAAA;EACA,cAAA;EACA,cAAA;EACA,yBAAA;EACA,mBAAA;EACA,kBAAA;;AAIF;EACE,gBAAA;EACA,cAAA;EACA,cAAA;EACA,yBAAA;EACA,kBAAA;EACA,8CAAA;;AAIF;EACE,cAAA;EACA,cAAA;EACA,gBAAA;EACA,eAAA;EACA,uBAAA;EACA,qBAAA;EACA,qBAAA;EACA,cAAA;EACA,yBAAA;EACA,yBAAA;EACA,kBAAA;;AAXF,GAcE;EACE,UAAA;EACA,kBAAA;EACA,cAAA;EACA,qBAAA;EACA,6BAAA;EACA,gBAAA;;AAKJ;EACE,iBAAA;EACA,kBAAA;;ACpDF;ENqnBE,kBAAA;EACA,iBAAA;EACA,kBAAA;EACA,mBAAA;;AMlnBA,QAHmC;EAGnC;IAFE,YAAA;;;AAKF,QAHmC;EAGnC;IAFE,YAAA;;;AAKJ,QAHqC;EAGrC;IAFI,aAAA;;;AAUJ;ENimBE,kBAAA;EACA,iBAAA;EACA,kBAAA;EACA,mBAAA;;AM3lBF;ENimBE,kBAAA;EACA,mBAAA;;AAqIE;EACE,kBAAA;EAEA,eAAA;EAEA,kBAAA;EACA,mBAAA;;AAgBF;EACE,WAAA;;AAOJ,KAAK,EAAQ,CAAC;EACZ,WAAA;;AADF,KAAK,EAAQ,CAAC;EACZ,mBAAA;;AADF,KAAK,EAAQ,CAAC;EACZ,mBAAA;;AADF,KAAK,EAAQ,CAAC;EACZ,UAAA;;AADF,KAAK,EAAQ,CAAC;EACZ,mBAAA;;AADF,K
 AAK,EAAQ,CAAC;EACZ,mBAAA;;AADF,KAAK,EAAQ,CAAC;EACZ,UAAA;;AADF,KAAK,EAAQ,CAAC;EACZ,mBAAA;;AADF,KAAK,EAAQ,CAAC;EACZ,mBAAA;;AADF,KAAK,EAAQ,CAAC;EACZ,UAAA;;AADF,KAAK,EAAQ,CAAC;EACZ,mBAAA;;AADF,KAAK,EAAQ,CAAC;EACZ,kBAAA;;AASF,KAAK,EAAQ,MAAM;EACjB,WAAA;;AADF,KAAK,EAAQ,MAAM;EACjB,mBAAA;;AADF,KAAK,EAAQ,MAAM;EACjB,mBAAA;;AADF,KAAK,EAAQ,MAAM;EACjB,UAAA;;AADF,KAAK,EAAQ,MAAM;EACjB,mBAAA;;AADF,KAAK,EAAQ,MAAM;EACjB,mBAAA;;AADF,KAAK,EAAQ,MAAM;EACjB,UAAA;;AADF,KAAK,EAAQ,MAAM;EACjB,mBAAA;;AADF,KAAK,EAAQ,MAAM;EACjB,mBAAA;;AADF,KAAK,EAAQ,MAAM;EACjB,UAAA;;AADF,KAAK,EAAQ,MAAM;EACjB,mBAAA;;AADF,KAAK,EAAQ,MAAM;EACjB,kBAAA;;AADF,KAAK,EAAQ,MAAM;EACjB,SAAA;;AANF,KAAK,EAAQ,MAAM;EACjB,UAAA;;AADF,KAAK,EAAQ,MAAM;EACjB,kBAAA;;AADF,KAAK,EAAQ,MAAM;EACjB,kBAAA;;AADF,KAAK,EAAQ,MAAM;EACjB,SAAA;;AADF,KAAK,EAAQ,MAAM;EACjB,kBAAA;;AADF,KAAK,EAAQ,MAAM;EACjB,kBAAA;;AADF,KAAK,EAAQ,MAAM;EACjB,SAAA;;AADF,KAAK,EAAQ,MAAM;EACjB,kBAAA;;AADF,KAAK,EAAQ,MAAM;EACjB,kBAAA;;AADF,KAAK,EAAQ,MAAM;EACjB,SAAA;;AADF,KAAK,EAAQ,MAAM;EACjB,kBAAA
 ;;AADF,KAAK,EAAQ,MAAM;EACjB,iBAAA;;AADF,KAAK,EAAQ,MAAM;EACjB,QAAA;;AASF,KAAK,EAAQ,QAAQ;EACnB,iBAAA;;AADF,KAAK,EAAQ,QAAQ;EACnB,yBAAA;;AADF,KAAK,EAAQ,QAAQ;EACnB,yBAAA;;AADF,KAAK,EAAQ,QAAQ;EACnB,gBAAA;;AADF,KAAK,EAAQ,QAAQ;EACnB,yBAAA;;AADF,KAAK,EAAQ,QAAQ;EACnB,yBAAA;;AADF,KAAK,EAAQ,QAAQ;EACnB,gBAAA;;AADF,KAAK,EAAQ,QAAQ;EACnB,yBAAA;;AADF,KAAK,EAAQ,QAAQ;EACnB,yBAAA;;AADF,KAAK,EAAQ,QAAQ;EACnB,gBAAA;;AADF,KAAK,EAAQ,QAAQ;EACnB,yBAAA;;AADF,KAAK,EAAQ,QAAQ;EACnB,wBAAA;;AADF,KAAK,EAAQ,QAAQ;EACnB,eAAA;;AMvvBJ,QALmC;ENouB/B;IACE,WAAA;;EAOJ,KAAK,EAAQ,CAAC;IACZ,WAAA;;EADF,KAAK,EAAQ,CAAC;IACZ,mBAAA;;EADF,KAAK,EAAQ,CAAC;IACZ,mBAAA;;EADF,KAAK,EAAQ,CAAC;IACZ,UAAA;;EADF,KAAK,EAAQ,CAAC;IACZ,mBAAA;;EADF,KAAK,EAAQ,CAAC;IACZ,mBAAA;;EADF,KAAK,EAAQ,CAAC;IACZ,UAAA;;EADF,KAAK,EAAQ,CAAC;IACZ,mBAAA;;EADF,KAAK,EAAQ,CAAC;IACZ,mBAAA;;EADF,KAAK,EAAQ,CAAC;IACZ,UAAA;;EADF,KAAK,EAAQ,CAAC;IACZ,mBAAA;;EADF,KAAK,EAAQ,CAAC;IACZ,kBAAA;;EASF,KAAK,EAAQ,MAAM;IACjB,WAAA;;EADF,KAAK,EAAQ,MAAM;IACjB,mBAAA;;EADF,KAAK,EAAQ,MAAM;IACjB
 ,mBAAA;;EADF,KAAK,EAAQ,MAAM;IACjB,UAAA;;EADF,KAAK,EAAQ,MAAM;IACjB,mBAAA;;EADF,KAAK,EAAQ,MAAM;IACjB,mBAAA;;EADF,KAAK,EAAQ,MAAM;IACjB,UAAA;;EADF,KAAK,EAAQ,MAAM;IACjB,mBAAA;;EADF,KAAK,EAAQ,MAAM;IACjB,mBAAA;;EADF,KAAK,EAAQ,MAAM;IACjB,UAAA;;EADF,KAAK,EAAQ,MAAM;IACjB,mBAAA;;EADF,KAAK,EAAQ,MAAM;IACjB,kBAAA;;EADF,KAAK,EAAQ,MAAM;IACjB,SAAA;;EANF,KAAK,EAAQ,MAAM;IACjB,UAAA;;EADF,KAAK,EAAQ,MAAM;IACjB,kBAAA;;EADF,KAAK,EAAQ,MAAM;IACjB,kBAAA;;EADF,KAAK,EAAQ,MAAM;IACjB,SAAA;;EADF,KAAK,EAAQ,MAAM;IACjB,kBAAA;;EADF,KAAK,EAAQ,MAAM;IACjB,kBAAA;;EADF,KAAK,EAAQ,MAAM;IACjB,SAAA;;EADF,KAAK,EAAQ,MAAM;IACjB,kBAAA;;EADF,KAAK,EAAQ,MAAM;IACjB,kBAAA;;EADF,KAAK,EAAQ,MAAM;IACjB,SAAA;;EADF,KAAK,EAAQ,MAAM;IACjB,kBAAA;;EADF,KAAK,EAAQ,MAAM;IACjB,iBAAA;;EADF,KAAK,EAAQ,MAAM;IACjB,QAAA;;EASF,KAAK,EAAQ,QAAQ;IACnB,iBAAA;;EADF,KAAK,EAAQ,QAAQ;IACnB,yBAAA;;EADF,KAAK,EAAQ,QAAQ;IACnB,yBAAA;;EADF,KAAK,EAAQ,QAAQ;IACnB,gBAAA;;EADF,KAAK,EAAQ,QAAQ;IACnB,yBAAA;;EADF,KAAK,EAAQ,QAAQ;IACnB,yBAAA;;EADF,KAAK,EAAQ,QAAQ;IACnB,gBAAA;;EADF,KAA
 K,EAAQ,QAAQ;IACnB,yBAAA;;EADF,KAAK,EAAQ,QAAQ;IACnB,yBAAA;;EADF,KAAK,EAAQ,QAAQ;IACnB,gBAAA;;EADF,KAAK,EAAQ,QAAQ;IACnB,yBAAA;;EADF,KAAK,EAAQ,QAAQ;IACnB,wBAAA;;EADF,KAAK,EAAQ,QAAQ;IACnB,eAAA;;;AM9uBJ,QALmC;EN2tB/B;IACE,WAAA;;EAOJ,KAAK,EAAQ,CAAC;IACZ,WAAA;;EADF,KAAK,EAAQ,CAAC;IACZ,mBAAA;;EADF,KAAK,EAAQ,CAAC;IACZ,mBAAA;;EADF,KAAK,EAAQ,CAAC;IACZ,UAAA;;EADF,KAAK,EAAQ,CAAC;IACZ,mBAAA;;EADF,KAAK,EAAQ,CAAC;IACZ,mBAAA;;EADF,KAAK,EAAQ,CAAC;IACZ,UAAA;;EADF,KAAK,EAAQ,CAAC;IACZ,mBAAA;;EADF,KAAK,EAAQ,CAAC;IACZ,mBAAA;;EADF,KAAK,EAAQ,CAAC;IACZ,UAAA;;EADF,KAAK,EAAQ,CAAC;IACZ,mBAAA;;EADF,KAAK,EAAQ,CAAC;IACZ,kBAAA;;EASF,KAAK,EAAQ,MAAM;IACjB,WAAA;;EADF,KAAK,EAAQ,MAAM;IACjB,mBAAA;;EADF,KAAK,EAAQ,MAAM;IACjB,mBAAA;;EADF,KAAK,EAAQ,MAAM;IACjB,UAAA;;EADF,KAAK,EAAQ,MAAM;IACjB,mBAAA;;EADF,KAAK,EAAQ,MAAM;IACjB,mBAAA;;EADF,KAAK,EAAQ,MAAM;IACjB,UAAA;;EADF,KAAK,EAAQ,MAAM;IACjB,mBAAA;;EADF,KAAK,EAAQ,MAAM;IACjB,mBAAA;;EADF,KAAK,EAAQ,MAAM;IACjB,UAAA;;EADF,KAAK,EAAQ,MAAM;IACjB,mBAAA;;EADF,KAAK,EAAQ,MAAM;IACjB,kBAAA;;EAD
 F,KAAK,EAAQ,MAAM;IACjB,SAAA;;EANF,KAAK,EAAQ,MAAM;IACjB,UAAA;;EADF,KAAK,EAAQ,MAAM;IACjB,kBAAA;;EADF,KAAK,EAAQ,MAAM;IACjB,kBAAA;;EADF,KAAK,EAAQ,MAAM;IACjB,SAAA;;EADF,KAAK,EAAQ,MAAM;IACjB,kBAAA;;EADF,KAAK,EAAQ,MAAM;IACjB,kBAAA;;EADF,KAAK,EAAQ,MAAM;IACjB,SAAA;;EADF,KAAK,EAAQ,MAAM;IACjB,kBAAA;;EADF,KAAK,EAAQ,MAAM;IACjB,kBAAA;;EADF,KAAK,EAAQ,MAAM;IACjB,SAAA;;EADF,KAAK,EAAQ,MAAM;IACjB,kBAAA;;EADF,KAAK,EAAQ,MAAM;IACjB,iBAAA;;EADF,KAAK,EAAQ,MAAM;IACjB,QAAA;;EASF,KAAK,EAAQ,QAAQ;IACnB,iBAAA;;EADF,KAAK,EAAQ,QAAQ;IACnB,yBAAA;;EADF,KAAK,EAAQ,QAAQ;IACnB,yBAAA;;EADF,KAAK,EAAQ,QAAQ;IACnB,gBAAA;;EADF,KAAK,EAAQ,QAAQ;IACnB,yBAAA;;EADF,KAAK,EAAQ,QAAQ;IACnB,yBAAA;;EADF,KAAK,EAAQ,QAAQ;IACnB,gBAAA;;EADF,KAAK,EAAQ,QAAQ;IACnB,yBAAA;;EADF,KAAK,EAAQ,QAAQ;IACnB,yBAAA;;EADF,KAAK,EAAQ,QAAQ;IACnB,gBAAA;;EADF,KAAK,EAAQ,QAAQ;IACnB,yBAAA;;EADF,KAAK,EAAQ,QAAQ;IACnB,wBAAA;;EADF,KAAK,EAAQ,QAAQ;IACnB,eAAA;;;AMvuBJ,QAHmC;ENktB/B;IACE,WAAA;;EAOJ,KAAK,EAAQ,CAAC;IACZ,WAAA;;EADF,KAAK,EAAQ,CAAC;IACZ,mBAAA;;EADF,KAAK,EAAQ,CAAC;
 IACZ,mBAAA;;EADF,KAAK,EAAQ,CAAC;IACZ,UAAA;;EADF,KAAK,EAAQ,CAAC;IACZ,mBAAA;;EADF,KAAK,EAAQ,CAAC;IACZ,mBAAA;;EADF,KAAK,EAAQ,CAAC;IACZ,UAAA;;EADF,KAAK,EAAQ,CAAC;IACZ,mBAAA;;EADF,KAAK,EAAQ,CAAC;IACZ,mBAAA;;EADF,KAAK,EAAQ,CAAC;IACZ,UAAA;;EADF,KAAK,EAAQ,CAAC;IACZ,mBAAA;;EADF,KAAK,EAAQ,CAAC;IACZ,kBAAA;;EASF,KAAK,EAAQ,MAAM;IACjB,WAAA;;EADF,KAAK,EAAQ,MAAM;IACjB,mBAAA;;EADF,KAAK,EAAQ,MAAM;IACjB,mBAAA;;EADF,KAAK,EAAQ,MAAM;IACjB,UAAA;;EADF,KAAK,EAAQ,MAAM;IACjB,mBAAA;;EADF,KAAK,EAAQ,MAAM;IACjB,mBAAA;;EADF,KAAK,EAAQ,MAAM;IACjB,UAAA;;EADF,KAAK,EAAQ,MAAM;IACjB,mBAAA;;EADF,KAAK,EAAQ,MAAM;IACjB,mBAAA;;EADF,KAAK,EAAQ,MAAM;IACjB,UAAA;;EADF,KAAK,EAAQ,MAAM;IACjB,mBAAA;;EADF,KAAK,EAAQ,MAAM;IACjB,kBAAA;;EADF,KAAK,EAAQ,MAAM;IACjB,SAAA;;EANF,KAAK,EAAQ,MAAM;IACjB,UAAA;;EADF,KAAK,EAAQ,MAAM;IACjB,kBAAA;;EADF,KAAK,EAAQ,MAAM;IACjB,kBAAA;;EADF,KAAK,EAAQ,MAAM;IACjB,SAAA;;EADF,KAAK,EAAQ,MAAM;IACjB,kBAAA;;EADF,KAAK,EAAQ,MAAM;IACjB,kBAAA;;EADF,KAAK,EAAQ,MAAM;IACjB,SAAA;;EADF,KAAK,EAAQ,MAAM;IACjB,kBAAA;;EADF,KAAK,EAAQ,
 MAAM;IACjB,kBAAA;;EADF,KAAK,EAAQ,MAAM;IACjB,SAAA;;EADF,KAAK,EAAQ,MAAM;IACjB,kBAAA;;EADF,KAAK,EAAQ,MAAM;IACjB,iBAAA;;EADF,KAAK,EAAQ,MAAM;IACjB,QAAA;;EASF,KAAK,EAAQ,QAAQ;IACnB,iBAAA;;EADF,KAAK,EAAQ,QAAQ;IACnB,yBAAA;;EADF,KAAK,EAAQ,QAAQ;IACnB,yBAAA;;EADF,KAAK,EAAQ,QAAQ;IACnB,gBAAA;;EADF,KAAK,EAAQ,QAAQ;IACnB,yBAAA;;EADF,KAAK,EAAQ,QAAQ;IACnB,yBAAA;;EADF,KAAK,EAAQ,QAAQ;IACnB,gBAAA;;EADF,KAAK,EAAQ,QAAQ;IACnB,yBAAA;;EADF,KAAK,EAAQ,QAAQ;IACnB,yBAAA;;EADF,KAAK,EAAQ,QAAQ;IACnB,gBAAA;;EADF,KAAK,EAAQ,QAAQ;IACnB,yBAAA;;EADF,KAAK,EAAQ,QAAQ;IACnB,wBAAA;;EADF,KAAK,EAAQ,QAAQ;IACnB,eAAA;;;AOtzBJ;EACE,eAAA;EACA,6BAAA;;AAEF;EACE,gBAAA;;AAMF;EACE,WAAA;EACA,mBAAA;;AAFF,MAIE,QAGE,KACE;AARN,MAKE,QAEE,KACE;AARN,MAME,QACE,KACE;AARN,MAIE,QAGE,KAEE;AATN,MAKE,QAEE,KAEE;AATN,MAME,QACE,KAEE;EACE,YAAA;EACA,uBAAA;EACA,mBAAA;EACA,6BAAA;;AAbR,MAkBE,QAAQ,KAAK;EACX,sBAAA;EACA,gCAAA;;AApBJ,MAuBE,UAAU,QAGR,KAAI,YACF;AA3BN,MAwBE,WAAW,QAET,KAAI,YACF;AA3BN,MAyBE,QAAO,YACL,KAAI,YACF;AA3BN,MAuBE,UAAU,QAGR,KAAI,YAEF;AA5BN,MAwBE
 ,WAAW,QAET,KAAI,YAEF;AA5BN,MAyBE,QAAO,YACL,KAAI,YAEF;EACE,aAAA;;AA7BR,MAkCE,QAAQ;EACN,6BAAA;;AAnCJ,MAuCE;EACE,yBAAA;;AAOJ,gBACE,QAGE,KACE;AALN,gBAEE,QAEE,KACE;AALN,gBAGE,QACE,KACE;AALN,gBACE,QAGE,KAEE;AANN,gBAEE,QAEE,KAEE;AANN,gBAGE,QACE,KAEE;EACE,YAAA;;AAWR;EACE,yBAAA;;AADF,eAEE,QAGE,KACE;AANN,eAGE,QAEE,KACE;AANN,eAIE,QACE,KACE;AANN,eAEE,QAGE,KAEE;AAPN,eAGE,QAEE,KAEE;AAPN,eAIE,QACE,KAEE;EACE,yBAAA;;AARR,eAYE,QAAQ,KACN;AAbJ,eAYE,QAAQ,KAEN;EACE,wBAAA;;AAUN,cACE,QAAQ,KAAI,UAAU,KACpB;AAFJ,cACE,QAAQ,KAAI,UAAU,KAEpB;EACE,yBAAA;;AAUN,YACE,QAAQ,KAAI,MACV;AAFJ,YACE,QAAQ,KAAI,MAEV;EACE,yBAAA;;AAUN,KAAM,IAAG;EACP,gBAAA;EACA,WAAA;EACA,qBAAA;;AAKE,KAFF,GAEG;AAAD,KADF,GACG;EACC,gBAAA;EACA,WAAA;EACA,mBAAA;;AP0SJ,MAAO,QAAQ,KAGb,KAAI,CAAC;AAFP,MAAO,QAAQ,KAEb,KAAI,CAAC;AADP,MAAO,QAAQ,KACb,KAAI,CAAC;AAHP,MAAO,QAAQ,KAIb,KAAI,CAAC;AAHP,MAAO,QAAQ,KAGb,KAAI,CAAC;AAFP,MAAO,QAAQ,KAEb,KAAI,CAAC;AACL,MALK,QAAQ,KAKZ,CAAC,MAAS;AAAX,MAJK,QAAQ,KAIZ,CAAC,MAAS;AAAX,MAHK,QAAQ,KAGZ,CAAC,MAAS;AACX,MANK,QAAQ,KAMZ,CAAC
 ,MAAS;AAAX,MALK,QAAQ,KAKZ,CAAC,MAAS;AAAX,MAJK,QAAQ,KAIZ,CAAC,MAAS;EACT,yBAAA;;AAMJ,YAAa,QAAQ,KACnB,KAAI,CAAC,MAAQ;AADf,YAAa,QAAQ,KAEnB,KAAI,CAAC,MAAQ;AACb,YAHW,QAAQ,KAGlB,CAAC,MAAQ,MAAO;AACjB,YAJW,QAAQ,KAIlB,CAAC,MAAQ,MAAO;EACf,yBAAA;;AAlBJ,MAAO,QAAQ,KAGb,KAAI,CAAC;AAFP,MAAO,QAAQ,KAEb,KAAI,CAAC;AADP,MAAO,QAAQ,KACb,KAAI,CAAC;AAHP,MAAO,QAAQ,KAIb,KAAI,CAAC;AAHP,MAAO,QAAQ,KAGb,KAAI,CAAC;AAFP,MAAO,QAAQ,KAEb,KAAI,CAAC;AACL,MALK,QAAQ,KAKZ,CAAC,OAAS;AAAX,MAJK,QAAQ,KAIZ,CAAC,OAAS;AAAX,MAHK,QAAQ,KAGZ,CAAC,OAAS;AACX,MANK,QAAQ,KAMZ,CAAC,OAAS;AAAX,MALK,QAAQ,KAKZ,CAAC,OAAS;AAAX,MAJK,QAAQ,KAIZ,CAAC,OAAS;EACT,yBAAA;;AAMJ,YAAa,QAAQ,KACnB,KAAI,CAAC,OAAQ;AADf,YAAa,QAAQ,KAEnB,KAAI,CAAC,OAAQ;AACb,YAHW,QAAQ,KAGlB,CAAC,OAAQ,MAAO;AACjB,YAJW,QAAQ,KAIlB,CAAC,OAAQ,MAAO;EACf,yBAAA;;AAlBJ,MAAO,QAAQ,KAGb,KAAI,CAAC;AAFP,MAAO,QAAQ,KAEb,KAAI,CAAC;AADP,MAAO,QAAQ,KACb,KAAI,CAAC;AAHP,MAAO,QAAQ,KAIb,KAAI,CAAC;AAHP,MAAO,QAAQ,KAGb,KAAI,CAAC;AAFP,MAAO,QAAQ,KAEb,KAAI,CAAC;AACL,MALK,QAAQ,KAKZ,CAAC,IAAS;AAAX,MAJK,QAAQ,KAIZ,C
 AAC,IAAS;AAAX,MAHK,QAAQ,KAGZ,CAAC,IAAS;AACX,MANK,QAAQ,KAMZ,CAAC,IAAS;AAAX,MALK,QAAQ,KAKZ,CAAC,IAAS;AAAX,MAJK,QAAQ,KAIZ,CAAC,IAAS;EACT,yBAAA;;AAMJ,YAAa,QAAQ,KACnB,KAAI,CAAC,IAAQ;AADf,YAAa,QAAQ,KAEnB,KAAI,CAAC,IAAQ;AACb,YAHW,QAAQ,KAGlB,CAAC,IAAQ,MAAO;AACjB,YAJW,QAAQ,KAIlB,CAAC,IAAQ,MAAO;EACf,yBAAA;;AAlBJ,MAAO,QAAQ,KAGb,KAAI,CAAC;AAFP,MAAO,QAAQ,KAEb,KAAI,CAAC;AADP,MAAO,QAAQ,KACb,KAAI,CAAC;AAHP,MAAO,QAAQ,KAIb,KAAI,CAAC;AAHP,MAAO,QAAQ,KAGb,KAAI,CAAC;AAFP,MAAO,QAAQ,KAEb,KAAI,CAAC;AACL,MALK,QAAQ,KAKZ,CAAC,OAAS;AAAX,MAJK,QAAQ,KAIZ,CAAC,OAAS;AAAX,MAHK,QAAQ,KAGZ,CAAC,OAAS;AACX,MANK,QAAQ,KAMZ,CAAC,OAAS;AAAX,MALK,QAAQ,KAKZ,CAAC,OAAS;AAAX,MAJK,QAAQ,KAIZ,CAAC,OAAS;EACT,yBAAA;;AAMJ,YAAa,QAAQ,KACnB,KAAI,CAAC,OAAQ;AADf,YAAa,QAAQ,KAEnB,KAAI,CAAC,OAAQ;AACb,YAHW,QAAQ,KAGlB,CAAC,OAAQ,MAAO;AACjB,YAJW,QAAQ,KAIlB,CAAC,OAAQ,MAAO;EACf,yBAAA;;AAlBJ,MAAO,QAAQ,KAGb,KAAI,CAAC;AAFP,MAAO,QAAQ,KAEb,KAAI,CAAC;AADP,MAAO,QAAQ,KACb,KAAI,CAAC;AAHP,MAAO,QAAQ,KAIb,KAAI,CAAC;AAHP,MAAO,QAAQ,KAGb,KAAI,CAAC;AAFP,MAAO,QAAQ,KAE
 b,KAAI,CAAC;AACL,MALK,QAAQ,KAKZ,CAAC,MAAS;AAAX,MAJK,QAAQ,KAIZ,CAAC,MAAS;AAAX,MAHK,QAAQ,KAGZ,CAAC,MAAS;AACX,MANK,QAAQ,KAMZ,CAAC,MAAS;AAAX,MALK,QAAQ,KAKZ,CAAC,MAAS;AAAX,MAJK,QAAQ,KAIZ,CAAC,MAAS;EACT,yBAAA;;AAMJ,YAAa,QAAQ,KACnB,KAAI,CAAC,MAAQ;AADf,YAAa,QAAQ,KAEnB,KAAI,CAAC,MAAQ;AACb,YAHW,QAAQ,KAGlB,CAAC,MAAQ,MAAO;AACjB,YAJW,QAAQ,KAIlB,CAAC,MAAQ,MAAO;EACf,yBAAA;;AOpON,QA/DmC;EACjC;IACE,WAAA;IACA,mBAAA;IACA,kBAAA;IACA,kBAAA;IACA,4CAAA;IACA,yBAAA;IACA,iCAAA;;EAPF,iBAUE;IACE,gBAAA;;EAXJ,iBAUE,SAIE,QAGE,KACE;EAlBR,iBAUE,SAKE,QAEE,KACE;EAlBR,iBAUE,SAME,QACE,KACE;EAlBR,iBAUE,SAIE,QAGE,KAEE;EAnBR,iBAUE,SAKE,QAEE,KAEE;EAnBR,iBAUE,SAME,QACE,KAEE;IACE,mBAAA;;EApBV,iBA2BE;IACE,SAAA;;EA5BJ,iBA2BE,kBAIE,QAGE,KACE,KAAI;EAnCZ,iBA2BE,kBAKE,QAEE,KACE,KAAI;EAnCZ,iBA2BE,kBAME,QACE,KACE,KAAI;EAnCZ,iBA2BE,kBAIE,QAGE,KAEE,KAAI;EApCZ,iBA2BE,kBAKE,QAEE,KAEE,KAAI;EApCZ,iBA2BE,kBAME,QACE,KAEE,KAAI;IACF,cAAA;;EArCV,iBA2BE,kBAIE,QAGE,KAKE,KAAI;EAvCZ,iBA2BE,kBAKE,QAEE,KAKE,KAAI;EAvCZ,iBA2BE,kBAME,QACE,KAKE,KAAI;EAv
 CZ,iBA2BE,kBAIE,QAGE,KAME,KAAI;EAxCZ,iBA2BE,kBAKE,QAEE,KAME,KAAI;EAxCZ,iBA2BE,kBAME,QACE,KAME,KAAI;IACF,eAAA;;EAzCV,iBA2BE,kBAsBE,QAEE,KAAI,WACF;EApDR,iBA2BE,kBAuBE,QACE,KAAI,WACF;EApDR,iBA2BE,kBAsBE,QAEE,KAAI,WAEF;EArDR,iBA2BE,kBAuBE,QACE,KAAI,WAEF;IACE,gBAAA;;;ACxNZ;EACE,UAAA;EACA,SAAA;EACA,SAAA;EAIA,YAAA;;AAGF;EACE,cAAA;EACA,WAAA;EACA,UAAA;EACA,mBAAA;EACA,eAAA;EACA,oBAAA;EACA,cAAA;EACA,SAAA;EACA,gCAAA;;AAGF;EACE,qBAAA;EACA,kBAAA;EACA,iBAAA;;AAWF,KAAK;ERsMH,8BAAA;EACG,2BAAA;EACK,sBAAA;;AQnMV,KAAK;AACL,KAAK;EACH,eAAA;EACA,kBAAA;;EACA,mBAAA;;AAIF,KAAK;EACH,cAAA;;AAIF,KAAK;EACH,cAAA;EACA,WAAA;;AAIF,MAAM;AACN,MAAM;EACJ,YAAA;;AAIF,KAAK,aAAa;AAClB,KAAK,cAAc;AACnB,KAAK,iBAAiB;ER7CpB,oBAAA;EAEA,0CAAA;EACA,oBAAA;;AQ+CF;EACE,cAAA;EACA,gBAAA;EACA,eAAA;EACA,uBAAA;EACA,cAAA;;AA0BF;EACE,cAAA;EACA,WAAA;EACA,YAAA;EACA,iBAAA;EACA,eAAA;EACA,uBAAA;EACA,cAAA;EACA,yBAAA;EACA,sBAAA;EACA,yBAAA;EACA,kBAAA;ERHA,wDAAA;EACQ,gDAAA;EAKR,8EAAA;EACQ,sEAAA;;AAmwBR,aAAC;EACC,qBAAA;EACA,UAAA;EA5wBF,sFAAA;EACQ,8EAA
 A;;AAlER,aAAC;EAA+B,cAAA;EACA,UAAA;;AAChC,aAAC;EAA+B,cAAA;;AAChC,aAAC;EAA+B,cAAA;;AQgFhC,aAAC;AACD,aAAC;AACD,QAAQ,UAAW;EACjB,mBAAA;EACA,yBAAA;EACA,UAAA;;AAIF,QAAQ;EACN,YAAA;;AAYJ,KAAK;EACH,wBAAA;;AASF,KAAK;EACH,iBAAA;;AASF;EACE,mBAAA;;AAQF;AACA;EACE,cAAA;EACA,gBAAA;EACA,gBAAA;EACA,mBAAA;EACA,kBAAA;;AANF,MAOE;AANF,SAME;EACE,eAAA;EACA,mBAAA;EACA,eAAA;;AAGJ,MAAO,MAAK;AACZ,aAAc,MAAK;AACnB,SAAU,MAAK;AACf,gBAAiB,MAAK;EACpB,WAAA;EACA,kBAAA;;AAEF,MAAO;AACP,SAAU;EACR,gBAAA;;AAIF;AACA;EACE,qBAAA;EACA,kBAAA;EACA,gBAAA;EACA,sBAAA;EACA,mBAAA;EACA,eAAA;;AAEF,aAAc;AACd,gBAAiB;EACf,aAAA;EACA,iBAAA;;AAYA,KANG,cAMF;AAAD,KALG,iBAKF;AAAD,MAAC;AAAD,aAAC;AAAD,SAAC;AAAD,gBAAC;AACD,QAAQ,UAAW,MAPhB;AAOH,QAAQ,UAAW,MANhB;AAMH,QAAQ,UAAW;AAAnB,QAAQ,UAAW;AAAnB,QAAQ,UAAW;AAAnB,QAAQ,UAAW;EACjB,mBAAA;;AAUJ;ERqpBE,YAAA;EACA,iBAAA;EACA,eAAA;EACA,gBAAA;EACA,kBAAA;;AAEA,MAAM;EACJ,YAAA;EACA,iBAAA;;AAGF,QAAQ;AACR,MAAM,UAAU;EACd,YAAA;;AQ9pBJ;ERipBE,YAAA;EACA,kBAAA;EACA,eAAA;EACA,iBAAA;EACA,kBAAA;;AAEA,MAAM;EACJ,YAAA;EACA,
 iBAAA;;AAGF,QAAQ;AACR,MAAM,UAAU;EACd,YAAA;;AQrpBJ;EAEE,kBAAA;;AAFF,aAKE;EACE,qBAAA;;AANJ,aAUE;EACE,kBAAA;EACA,SAAA;EACA,QAAA;EACA,cAAA;EACA,WAAA;EACA,YAAA;EACA,iBAAA;EACA,kBAAA;;AAKJ,YRsjBE;AQtjBF,YRujBE;AQvjBF,YRwjBE;AQxjBF,YRyjBE;AQzjBF,YR0jBE;AQ1jBF,YR2jBE;EACE,cAAA;;AQ5jBJ,YR+jBE;EACE,qBAAA;EAvuBF,wDAAA;EACQ,gDAAA;;AAwuBN,YAHF,cAGG;EACC,qBAAA;EA1uBJ,yEAAA;EACQ,iEAAA;;AQsKV,YRykBE;EACE,cAAA;EACA,qBAAA;EACA,yBAAA;;AQ5kBJ,YR+kBE;EACE,cAAA;;AQ7kBJ,YRmjBE;AQnjBF,YRojBE;AQpjBF,YRqjBE;AQrjBF,YRsjBE;AQtjBF,YRujBE;AQvjBF,YRwjBE;EACE,cAAA;;AQzjBJ,YR4jBE;EACE,qBAAA;EAvuBF,wDAAA;EACQ,gDAAA;;AAwuBN,YAHF,cAGG;EACC,qBAAA;EA1uBJ,yEAAA;EACQ,iEAAA;;AQyKV,YRskBE;EACE,cAAA;EACA,qBAAA;EACA,yBAAA;;AQzkBJ,YR4kBE;EACE,cAAA;;AQ1kBJ,URgjBE;AQhjBF,URijBE;AQjjBF,URkjBE;AQljBF,URmjBE;AQnjBF,URojBE;AQpjBF,URqjBE;EACE,cAAA;;AQtjBJ,URyjBE;EACE,qBAAA;EAvuBF,wDAAA;EACQ,gDAAA;;AAwuBN,UAHF,cAGG;EACC,qBAAA;EA1uBJ,yEAAA;EACQ,iEAAA;;AQ4KV,URmkBE;EACE,cAAA;EACA,qBAAA;EACA,yBAAA;;AQtkBJ,URykBE;EACE,cAAA;;AQhkBJ;EACE,gB
 AAA;;AASF;EACE,cAAA;EACA,eAAA;EACA,mBAAA;EACA,cAAA;;AAoEF,QAjDqC;EAiDrC,YA/CI;IACE,qBAAA;IACA,gBAAA;IACA,sBAAA;;EA4CN,YAxCI;IACE,qBAAA;IACA,WAAA;IACA,sBAAA;;EAqCN,YAlCI,aAAa;IACX,WAAA;;EAiCN,YA9BI;IACE,gBAAA;IACA,sBAAA;;EA4BN,YAtBI;EAsBJ,YArBI;IACE,qBAAA;IACA,aAAA;IACA,gBAAA;IACA,eAAA;IACA,sBAAA;;EAgBN,YAdI,OAAO,MAAK;EAchB,YAbI,UAAU,MAAK;IACb,WAAA;IACA,cAAA;;EAWN,YAJI,cAAc;IACZ,MAAA;;;AAWN,gBAGE;AAHF,gBAIE;AAJF,gBAKE;AALF,gBAME;AANF,gBAOE;EACE,aAAA;EACA,gBAAA;EACA,gBAAA;;AAVJ,gBAcE;AAdF,gBAeE;EACE,gBAAA;;AAhBJ,gBAoBE;ERyOA,kBAAA;EACA,mBAAA;;AQ9PF,gBAwBE;EACE,gBAAA;;AAUF,QANmC;EAMnC,gBALE;IACE,iBAAA;;;AA/BN,gBAuCE,cAAc;EACZ,MAAA;EACA,WAAA;;AC3aJ;EACE,qBAAA;EACA,gBAAA;EACA,mBAAA;EACA,kBAAA;EACA,sBAAA;EACA,eAAA;EACA,sBAAA;EACA,6BAAA;EACA,mBAAA;ET0gBA,iBAAA;EACA,eAAA;EACA,uBAAA;EACA,kBAAA;EAnSA,yBAAA;EACG,sBAAA;EACC,qBAAA;EACI,iBAAA;;AStON,IAAC;AAAD,IAFD,OAEE;AAAD,IADD,OACE;ETQH,oBAAA;EAEA,0CAAA;EACA,oBAAA;;ASNA,IAAC;AACD,IAAC;EACC,cAAA;EACA,qBAAA;;AAGF,IAAC;AACD,IAAC;EACC,UAAA;EACA,sBA
 AA;ETmFF,wDAAA;EACQ,gDAAA;;AShFR,IAAC;AACD,IAAC;AACD,QAAQ,UAAW;EACjB,mBAAA;EACA,oBAAA;ET+OF,aAAA;EAGA,yBAAA;EAvKA,wBAAA;EACQ,gBAAA;;ASlEV;ET2bE,cAAA;EACA,yBAAA;EACA,qBAAA;;AAEA,YAAC;AACD,YAAC;AACD,YAAC;AACD,YAAC;AACD,KAAM,iBAAgB;EACpB,cAAA;EACA,yBAAA;EACI,qBAAA;;AAEN,YAAC;AACD,YAAC;AACD,KAAM,iBAAgB;EACpB,sBAAA;;AAKA,YAHD;AAGC,YAFD;AAEC,QADM,UAAW;AAEjB,YAJD,SAIE;AAAD,YAHD,UAGE;AAAD,QAFM,UAAW,aAEhB;AACD,YALD,SAKE;AAAD,YAJD,UAIE;AAAD,QAHM,UAAW,aAGhB;AACD,YAND,SAME;AAAD,YALD,UAKE;AAAD,QAJM,UAAW,aAIhB;AACD,YAPD,SAOE;AAAD,YAND,UAME;AAAD,QALM,UAAW,aAKhB;EACC,yBAAA;EACI,qBAAA;;AStdV,YT0dE;EACE,cAAA;EACA,yBAAA;;ASzdJ;ETwbE,cAAA;EACA,yBAAA;EACA,qBAAA;;AAEA,YAAC;AACD,YAAC;AACD,YAAC;AACD,YAAC;AACD,KAAM,iBAAgB;EACpB,cAAA;EACA,yBAAA;EACI,qBAAA;;AAEN,YAAC;AACD,YAAC;AACD,KAAM,iBAAgB;EACpB,sBAAA;;AAKA,YAHD;AAGC,YAFD;AAEC,QADM,UAAW;AAEjB,YAJD,SAIE;AAAD,YAHD,UAGE;AAAD,QAFM,UAAW,aAEhB;AACD,YALD,SAKE;AAAD,YAJD,UAIE;AAAD,QAHM,UAAW,aAGhB;AACD,YAND,SAME;AAAD,YALD,UAKE;AAAD,QAJM,UAAW,aAIhB;AACD,YAPD,SAOE;AA
 AD,YAND,UAME;AAAD,QALM,UAAW,aAKhB;EACC,yBAAA;EACI,qBAAA;;ASndV,YTudE;EACE,cAAA;EACA,yBAAA;;ASrdJ;ETobE,cAAA;EACA,yBAAA;EACA,qBAAA;;AAEA,YAAC;AACD,YAAC;AACD,YAAC;AACD,YAAC;AACD,KAAM,iBAAgB;EACpB,cAAA;EACA,yBAAA;EACI,qBAAA;;AAEN,YAAC;AACD,YAAC;AACD,KAAM,iBAAgB;EACpB,sBAAA;;AAKA,YAHD;AAGC,YAFD;AAEC,QADM,UAAW;AAEjB,YAJD,SAIE;AAAD,YAHD,UAGE;AAAD,QAFM,UAAW,aAEhB;AACD,YALD,SAKE;AAAD,YAJD,UAIE;AAAD,QAHM,UAAW,aAGhB;AACD,YAND,SAME;AAAD,YALD,UAKE;AAAD,QAJM,UAAW,aAIhB;AACD,YAPD,SAOE;AAAD,YAND,UAME;AAAD,QALM,UAAW,aAKhB;EACC,yBAAA;EACI,qBAAA;;AS/cV,YTmdE;EACE,cAAA;EACA,yBAAA;;ASjdJ;ETgbE,cAAA;EACA,yBAAA;EACA,qBAAA;;AAEA,SAAC;AACD,SAAC;AACD,SAAC;AACD,SAAC;AACD,KAAM,iBAAgB;EACpB,cAAA;EACA,yBAAA;EACI,qBAAA;;AAEN,SAAC;AACD,SAAC;AACD,KAAM,iBAAgB;EACpB,sBAAA;;AAKA,SAHD;AAGC,SAFD;AAEC,QADM,UAAW;AAEjB,SAJD,SAIE;AAAD,SAHD,UAGE;AAAD,QAFM,UAAW,UAEhB;AACD,SALD,SAKE;AAAD,SAJD,UAIE;AAAD,QAHM,UAAW,UAGhB;AACD,SAND,SAME;AAAD,SALD,UAKE;AAAD,QAJM,UAAW,UAIhB;AACD,SAPD,SAOE;AAAD,SAND,UAME;AAAD,QALM,UAAW,UAKhB;EACC,yB
 AAA;EACI,qBAAA;;AS3cV,ST+cE;EACE,cAAA;EACA,yBAAA;;AS7cJ;ET4aE,cAAA;EACA,yBAAA;EACA,qBAAA;;AAEA,YAAC;AACD,YAAC;AACD,YAAC;AACD,YAAC;AACD,KAAM,iBAAgB;EACpB,cAAA;EACA,yBAAA;EACI,qBAAA;;AAEN,YAAC;AACD,YAAC;AACD,KAAM,iBAAgB;EACpB,sBAAA;;AAKA,YAHD;AAGC,YAFD;AAEC,QADM,UAAW;AAEjB,YAJD,SAIE;AAAD,YAHD,UAGE;AAAD,QAFM,UAAW,aAEhB;AACD,YALD,SAKE;AAAD,YAJD,UAIE;AAAD,QAHM,UAAW,aAGhB;AACD,YAND,SAME;AAAD,YALD,UAKE;AAAD,QAJM,UAAW,aAIhB;AACD,YAPD,SAOE;AAAD,YAND,UAME;AAAD,QALM,UAAW,aAKhB;EACC,yBAAA;EACI,qBAAA;;ASvcV,YT2cE;EACE,cAAA;EACA,yBAAA;;ASzcJ;ETwaE,cAAA;EACA,yBAAA;EACA,qBAAA;;AAEA,WAAC;AACD,WAAC;AACD,WAAC;AACD,WAAC;AACD,KAAM,iBAAgB;EACpB,cAAA;EACA,yBAAA;EACI,qBAAA;;AAEN,WAAC;AACD,WAAC;AACD,KAAM,iBAAgB;EACpB,sBAAA;;AAKA,WAHD;AAGC,WAFD;AAEC,QADM,UAAW;AAEjB,WAJD,SAIE;AAAD,WAHD,UAGE;AAAD,QAFM,UAAW,YAEhB;AACD,WALD,SAKE;AAAD,WAJD,UAIE;AAAD,QAHM,UAAW,YAGhB;AACD,WAND,SAME;AAAD,WALD,UAKE;AAAD,QAJM,UAAW,YAIhB;AACD,WAPD,SAOE;AAAD,WAND,UAME;AAAD,QALM,UAAW,YAKhB;EACC,yBAAA;EACI,qBAAA;;ASncV,WTucE;EACE,cAAA;EAC
 A,yBAAA;;AShcJ;EACE,cAAA;EACA,mBAAA;EACA,eAAA;EACA,gBAAA;;AAEA;AACA,SAAC;AACD,SAAC;AACD,QAAQ,UAAW;EACjB,6BAAA;ET2BF,wBAAA;EACQ,gBAAA;;ASzBR;AACA,SAAC;AACD,SAAC;AACD,SAAC;EACC,yBAAA;;AAEF,SAAC;AACD,SAAC;EACC,cAAA;EACA,0BAAA;EACA,6BAAA;;AAIA,SAFD,UAEE;AAAD,QADM,UAAW,UAChB;AACD,SAHD,UAGE;AAAD,QAFM,UAAW,UAEhB;EACC,cAAA;EACA,qBAAA;;AASN;ACvBA,aAAc;EVubZ,kBAAA;EACA,eAAA;EACA,iBAAA;EACA,kBAAA;;AS/ZF;AC5BA,aAAc;EVwbZ,iBAAA;EACA,eAAA;EACA,gBAAA;EACA,kBAAA;;AS3ZF;ACjCA,aAAc;EVybZ,gBAAA;EACA,eAAA;EACA,gBAAA;EACA,kBAAA;;ASnZF;EACE,cAAA;EACA,WAAA;EACA,eAAA;EACA,gBAAA;;AAIF,UAAW;EACT,eAAA;;AAOA,KAHG,eAGF;AAAD,KAFG,cAEF;AAAD,KADG,eACF;EACC,WAAA;;AEnJJ;EACE,UAAA;EXqHA,wCAAA;EACQ,gCAAA;;AWpHR,KAAC;EACC,UAAA;;AAIJ;EACE,aAAA;;AACA,SAAC;EACC,cAAA;;AAGJ;EACE,kBAAA;EACA,SAAA;EACA,gBAAA;EXqGA,qCAAA;EACQ,6BAAA;;AYtHV;EACE,aAAa,sBAAb;EACA,qDAAA;EACA,2TAAA;;AAOF;EACE,kBAAA;EACA,QAAA;EACA,qBAAA;EACA,aAAa,sBAAb;EACA,kBAAA;EACA,mBAAA;EACA,cAAA;EACA,mCAAA;EACA,kCAAA;;AAIkC,mBAAC;EAAU,SAAS,KAAT;;AACX,eAAC;EAAU,SA
 AS,KAAT;;AACX,eAAC;EAAU,SAAS,OAAT;;AACX,gBAAC;EAAU,SAAS,OAAT;;AACX,gBAAC;EAAU,SAAS,OAAT;;AACX,mBAAC;EAAU,SAAS,OAAT;;AACX,iBAAC;EAAU,SAAS,OAAT;;AACX,gBAAC;EAAU,SAAS,OAAT;;AACX,gBAAC;EAAU,SAAS,OAAT;;AACX,iBAAC;EAAU,SAAS,OAAT;;AACX,gBAAC;EAAU,SAAS,OAAT;;AACX,eAAC;EAAU,SAAS,OAAT;;AACX,qBAAC;EAAU,SAAS,OAAT;;AACX,eAAC;EAAU,SAAS,OAAT;;AACX,eAAC;EAAU,SAAS,OAAT;;AACX,mBAAC;EAAU,SAAS,OAAT;;AACX,aAAC;EAAU,SAAS,OAAT;;AACX,kBAAC;EAAU,SAAS,OAAT;;AACX,aAAC;EAAU,SAAS,OAAT;;AACX,iBAAC;EAAU,SAAS,OAAT;;AACX,kBAAC;EAAU,SAAS,OAAT;;AACX,mBAAC;EAAU,SAAS,OAAT;;AACX,cAAC;EAAU,SAAS,OAAT;;AACX,iBAAC;EAAU,SAAS,OAAT;;AACX,cAAC;EAAU,SAAS,OAAT;;AACX,gBAAC;EAAU,SAAS,OAAT;;AACX,eAAC;EAAU,SAAS,OAAT;;AACX,eAAC;EAAU,SAAS,OAAT;;AACX,eAAC;EAAU,SAAS,OAAT;;AACX,eAAC;EAAU,SAAS,OAAT;;AACX,uBAAC;EAAU,SAAS,OAAT;;AACX,mBAAC;EAAU,SAAS,OAAT;;AACX,iBAAC;EAAU,SAAS,OAAT;;AACX,gBAAC;EAAU,SAAS,OAAT;;AACX,sBAAC;EAAU,SAAS,OAAT;;AACX,iBAAC;EAAU,SAAS,OAAT;;AACX,kBAAC;EAAU,SAAS,OAAT;;AACX,mBAAC;EAAU,SAAS,OAAT;;AACX,eAAC;EAAU,SAAS,OAAT;;AA
 CX,eAAC;EAAU,SAAS,OAAT;;AACX,qBAAC;EAAU,SAAS,OAAT;;AACX,qBAAC;EAAU,SAAS,OAAT;;AACX,sBAAC;EAAU,SAAS,OAAT;;AACX,oBAAC;EAAU,SAAS,OAAT;;AACX,iBAAC;EAAU,SAAS,OAAT;;AACX,kBAAC;EAAU,SAAS,OAAT;;AACX,cAAC;EAAU,SAAS,OAAT;;AACX,eAAC;EAAU,SAAS,OAAT;;AACX,eAAC;EAAU,SAAS,OAAT;;AACX,mBAAC;EAAU,SAAS,OAAT;;AACX,gBAAC;EAAU,SAAS,OAAT;;AACX,iBAAC;EAAU,SAAS,OAAT;;AACX,eAAC;EAAU,SAAS,OAAT;;AACX,eAAC;EAAU,SAAS,OAAT;;AACX,iBAAC;EAAU,SAAS,OAAT;;AACX,sBAAC;EAAU,SAAS,OAAT;;AACX,qBAAC;EAAU,SAAS,OAAT;;AACX,qBAAC;EAAU,SAAS,OAAT;;AACX,uBAAC;EAAU,SAAS,OAAT;;AACX,sBAAC;EAAU,SAAS,OAAT;;AACX,wBAAC;EAAU,SAAS,OAAT;;AACX,eAAC;EAAU,SAAS,OAAT;;AACX,sBAAC;EAAU,SAAS,OAAT;;AACX,uBAAC;EAAU,SAAS,OAAT;;AACX,yBAAC;EAAU,SAAS,OAAT;;AACX,kBAAC;EAAU,SAAS,OAAT;;AACX,qBAAC;EAAU,SAAS,OAAT;;AACX,iBAAC;EAAU,SAAS,OAAT;;AACX,eAAC;EAAU,SAAS,OAAT;;AACX,eAAC;EAAU,SAAS,OAAT;;AACX,gBAAC;EAAU,SAAS,OAAT;;AACX,gBAAC;EAAU,SAAS,OAAT;;AACX,eAAC;EAAU,SAAS,OAAT;;AACX,wBAAC;EAAU,SAAS,OAAT;;AACX,wBAAC;EAAU,SAAS,OAAT;;AACX,mBAAC;EAAU,SAAS,OAAT;;AACX,eAAC;
 EAAU,SAAS,OAAT;;AACX,gBAAC;EAAU,SAAS,OAAT;;AACX,eAAC;EAAU,SAAS,OAAT;;AACX,kBAAC;EAAU,SAAS,OAAT;;AACX,uBAAC;EAAU,SAAS,OAAT;;AACX,uBAAC;EAAU,SAAS,OAAT;;AACX,gBAAC;EAAU,SAAS,OAAT;;AACX,uBAAC;EAAU,SAAS,OAAT;;AACX,wBAAC;EAAU,SAAS,OAAT;;AACX,oBAAC;EAAU,SAAS,OAAT;;AACX,qBAAC;EAAU,SAAS,OAAT;;AACX,sBAAC;EAAU,SAAS,OAAT;;AACX,kBAAC;EAAU,SAAS,OAAT;;AACX,wBAAC;EAAU,SAAS,OAAT;;AACX,oBAAC;EAAU,SAAS,OAAT;;AACX,qBAAC;EAAU,SAAS,OAAT;;AACX,wBAAC;EAAU,SAAS,OAAT;;AACX,oBAAC;EAAU,SAAS,OAAT;;AACX,qBAAC;EAAU,SAAS,OAAT;;AACX,qBAAC;EAAU,SAAS,OAAT;;AACX,sBAAC;EAAU,SAAS,OAAT;;AACX,mBAAC;EAAU,SAAS,OAAT;;AACX,qBAAC;EAAU,SAAS,OAAT;;AACX,oBAAC;EAAU,SAAS,OAAT;;AACX,sBAAC;EAAU,SAAS,OAAT;;AACX,uBAAC;EAAU,SAAS,OAAT;;AACX,2BAAC;EAAU,SAAS,OAAT;;AACX,eAAC;EAAU,SAAS,OAAT;;AACX,eAAC;EAAU,SAAS,OAAT;;AACX,eAAC;EAAU,SAAS,OAAT;;AACX,mBAAC;EAAU,SAAS,OAAT;;AACX,oBAAC;EAAU,SAAS,OAAT;;AACX,uBAAC;EAAU,SAAS,OAAT;;AACX,gBAAC;EAAU,SAAS,OAAT;;AACX,mBAAC;EAAU,SAAS,OAAT;;AACX,iBAAC;EAAU,SAAS,OAAT;;AACX,kBAAC;EAAU,SAAS,OAAT;;AACX,iBAAC;EA
 AU,SAAS,OAAT;;AACX,qBAAC;EAAU,SAAS,OAAT;;AACX,uBAAC;EAAU,SAAS,OAAT;;AACX,kBAAC;EAAU,SAAS,OAAT;;AACX,wBAAC;EAAU,SAAS,OAAT;;AACX,uBAAC;EAAU,SAAS,OAAT;;AACX,sBAAC;EAAU,SAAS,OAAT;;AACX,0BAAC;EAAU,SAAS,OAAT;;AACX,4BAAC;EAAU,SAAS,OAAT;;AACX,cAAC;EAAU,SAAS,OAAT;;AACX,mBAAC;EAAU,SAAS,OAAT;;AACX,eAAC;EAAU,SAAS,OAAT;;AACX,sBAAC;EAAU,SAAS,OAAT;;AACX,oBAAC;EAAU,SAAS,OAAT;;AACX,sBAAC;EAAU,SAAS,OAAT;;AACX,qBAAC;EAAU,SAAS,OAAT;;AACX,oBAAC;EAAU,SAAS,OAAT;;AACX,kBAAC;EAAU,SAAS,OAAT;;AACX,oBAAC;EAAU,SAAS,OAAT;;AACX,6BAAC;EAAU,SAAS,OAAT;;AACX,4BAAC;EAAU,SAAS,OAAT;;AACX,0BAAC;EAAU,SAAS,OAAT;;AACX,4BAAC;EAAU,SAAS,OAAT;;AACX,gBAAC;EAAU,SAAS,OAAT;;AACX,iBAAC;EAAU,SAAS,OAAT;;AACX,gBAAC;EAAU,SAAS,OAAT;;AACX,iBAAC;EAAU,SAAS,OAAT;;AACX,oBAAC;EAAU,SAAS,OAAT;;AACX,qBAAC;EAAU,SAAS,OAAT;;AACX,oBAAC;EAAU,SAAS,OAAT;;AACX,oBAAC;EAAU,SAAS,OAAT;;AACX,sBAAC;EAAU,SAAS,OAAT;;AACX,eAAC;EAAU,SAAS,OAAT;;AACX,gBAAC;EAAU,SAAS,OAAT;;AACX,kBAAC;EAAU,SAAS,OAAT;;AACX,cAAC;EAAU,SAAS,OAAT;;AACX,cAAC;EAAU,SAAS,OAAT;;AACX,eAAC;EAAU,S
 AAS,OAAT;;AACX,2BAAC;EAAU,SAAS,OAAT;;AACX,+BAAC;EAAU,SAAS,OAAT;;AACX,wBAAC;EAAU,SAAS,OAAT;;AACX,4BAAC;EAAU,SAAS,OAAT;;AACX,6BAAC;EAAU,SAAS,OAAT;;AACX,iCAAC;EAAU,SAAS,OAAT;;AACX,oBAAC;EAAU,SAAS,OAAT;;AACX,iBAAC;EAAU,SAAS,OAAT;;AACX,wBAAC;EAAU,SAAS,OAAT;;AACX,sBAAC;EAAU,SAAS,OAAT;;AACX,iBAAC;EAAU,SAAS,OAAT;;AACX,gBAAC;EAAU,SAAS,OAAT;;AACX,kBAAC;EAAU,SAAS,OAAT;;AACX,qBAAC;EAAU,SAAS,OAAT;;AACX,iBAAC;EAAU,SAAS,OAAT;;AACX,eAAC;EAAU,SAAS,OAAT;;AACX,eAAC;EAAU,SAAS,OAAT;;AACX,gBAAC;EAAU,SAAS,OAAT;;AACX,iBAAC;EAAU,SAAS,OAAT;;AACX,iBAAC;EAAU,SAAS,OAAT;;AACX,eAAC;EAAU,SAAS,OAAT;;AACX,sBAAC;EAAU,SAAS,OAAT;;AACX,uBAAC;EAAU,SAAS,OAAT;;AACX,wBAAC;EAAU,SAAS,OAAT;;AACX,sBAAC;EAAU,SAAS,OAAT;;AACX,sBAAC;EAAU,SAAS,OAAT;;AACX,sBAAC;EAAU,SAAS,OAAT;;AACX,mBAAC;EAAU,SAAS,OAAT;;AACX,kBAAC;EAAU,SAAS,OAAT;;AACX,iBAAC;EAAU,SAAS,OAAT;;AACX,qBAAC;EAAU,SAAS,OAAT;;AACX,mBAAC;EAAU,SAAS,OAAT;;AACX,oBAAC;EAAU,SAAS,OAAT;;AACX,gBAAC;EAAU,SAAS,OAAT;;AACX,gBAAC;EAAU,SAAS,OAAT;;AACX,mBAAC;EAAU,SAAS,OAAT;;AACX,mBAAC;EAAU,SA
 AS,OAAT;;AACX,oBAAC;EAAU,SAAS,OAAT;;AACX,uBAAC;EAAU,SAAS,OAAT;;AACX,sBAAC;EAAU,SAAS,OAAT;;AACX,oBAAC;EAAU,SAAS,OAAT;;AACX,oBAAC;EAAU,SAAS,OAAT;;AACX,oBAAC;EAAU,SAAS,OAAT;;AACX,yBAAC;EAAU,SAAS,OAAT;;AACX,4BAAC;EAAU,SAAS,OAAT;;AACX,yBAAC;EAAU,SAAS,OAAT;;AACX,uBAAC;EAAU,SAAS,OAAT;;AACX,uBAAC;EAAU,SAAS,OAAT;;AACX,yBAAC;EAAU,SAAS,OAAT;;AClO/C;EACE,qBAAA;EACA,QAAA;EACA,SAAA;EACA,gBAAA;EACA,sBAAA;EACA,qBAAA;EACA,mCAAA;EACA,kCAAA;;AAIF;EACE,kBAAA;;AAIF,gBAAgB;EACd,UAAA;;AAIF;EACE,kBAAA;EACA,SAAA;EACA,OAAA;EACA,aAAA;EACA,aAAA;EACA,WAAA;EACA,gBAAA;EACA,cAAA;EACA,eAAA;EACA,gBAAA;EACA,eAAA;EACA,yBAAA;EACA,yBAAA;EACA,qCAAA;EACA,kBAAA;Eb8EA,mDAAA;EACQ,2CAAA;Ea7ER,4BAAA;;AAKA,cAAC;EACC,QAAA;EACA,UAAA;;AAxBJ,cA4BE;EboVA,WAAA;EACA,aAAA;EACA,gBAAA;EACA,yBAAA;;AanXF,cAiCE,KAAK;EACH,cAAA;EACA,iBAAA;EACA,WAAA;EACA,mBAAA;EACA,uBAAA;EACA,cAAA;EACA,mBAAA;;AAMF,cADa,KAAK,IACjB;AACD,cAFa,KAAK,IAEjB;EACC,qBAAA;EACA,cAAA;EACA,yBAAA;;AAMF,cADa,UAAU;AAEvB,cAFa,UAAU,IAEtB;AACD,cAHa,UAAU,IAGtB;EACC,cAAA;EACA,qBAAA
 ;EACA,UAAA;EACA,yBAAA;;AASF,cADa,YAAY;AAEzB,cAFa,YAAY,IAExB;AACD,cAHa,YAAY,IAGxB;EACC,cAAA;;AAKF,cADa,YAAY,IACxB;AACD,cAFa,YAAY,IAExB;EACC,qBAAA;EACA,6BAAA;EACA,sBAAA;EbkPF,mEAAA;EahPE,mBAAA;;AAKJ,KAEE;EACE,cAAA;;AAHJ,KAOE;EACE,UAAA;;AAQJ;EACE,UAAA;EACA,QAAA;;AAQF;EACE,OAAA;EACA,WAAA;;AAIF;EACE,cAAA;EACA,iBAAA;EACA,eAAA;EACA,uBAAA;EACA,cAAA;;AAIF;EACE,eAAA;EACA,OAAA;EACA,QAAA;EACA,SAAA;EACA,MAAA;EACA,YAAA;;AAIF,WAAY;EACV,QAAA;EACA,UAAA;;AAQF,OAGE;AAFF,oBAAqB,UAEnB;EACE,aAAA;EACA,wBAAA;EACA,SAAS,EAAT;;AANJ,OASE;AARF,oBAAqB,UAQnB;EACE,SAAA;EACA,YAAA;EACA,kBAAA;;AAsBJ,QAb2C;EACzC,aACE;IAnEF,UAAA;IACA,QAAA;;EAiEA,aAME;IA9DF,OAAA;IACA,WAAA;;;AH7IF;AACA;EACE,kBAAA;EACA,qBAAA;EACA,sBAAA;;AAJF,UAKE;AAJF,mBAIE;EACE,kBAAA;EACA,WAAA;;AAEA,UAJF,OAIG;AAAD,mBAJF,OAIG;AACD,UALF,OAKG;AAAD,mBALF,OAKG;AACD,UANF,OAMG;AAAD,mBANF,OAMG;AACD,UAPF,OAOG;AAAD,mBAPF,OAOG;EACC,UAAA;;AAEF,UAVF,OAUG;AAAD,mBAVF,OAUG;EAEC,aAAA;;AAMN,UACE,KAAK;AADP,UAEE,KAAK;AAFP,UAGE,WAAW;AAHb,UAIE,WAAW;EACT,iBAAA;;AAKJ;EACE,iBAAA
 ;;AADF,YAIE;AAJF,YAKE;EACE,WAAA;;AANJ,YAQE;AARF,YASE;AATF,YAUE;EACE,gBAAA;;AAIJ,UAAW,OAAM,IAAI,cAAc,IAAI,aAAa,IAAI;EACtD,gBAAA;;AAIF,UAAW,OAAM;EACf,cAAA;;AACA,UAFS,OAAM,YAEd,IAAI,aAAa,IAAI;EV2CtB,6BAAA;EACG,0BAAA;;AUvCL,UAAW,OAAM,WAAW,IAAI;AAChC,UAAW,mBAAkB,IAAI;EV6C/B,4BAAA;EACG,yBAAA;;AUzCL,UAAW;EACT,WAAA;;AAEF,UAAW,aAAY,IAAI,cAAc,IAAI,aAAc;EACzD,gBAAA;;AAEF,UAAW,aAAY,YACrB,OAAM;AADR,UAAW,aAAY,YAErB;EVwBA,6BAAA;EACG,0BAAA;;AUrBL,UAAW,aAAY,WAAY,OAAM;EV4BvC,4BAAA;EACG,yBAAA;;AUxBL,UAAW,iBAAgB;AAC3B,UAAU,KAAM;EACd,UAAA;;AAiBF,UAAW,OAAO;EAChB,iBAAA;EACA,kBAAA;;AAEF,UAAW,UAAU;EACnB,kBAAA;EACA,mBAAA;;AAKF,UAAU,KAAM;EVGd,wDAAA;EACQ,gDAAA;;AUAR,UAJQ,KAAM,iBAIb;EVDD,wBAAA;EACQ,gBAAA;;AUOV,IAAK;EACH,cAAA;;AAGF,OAAQ;EACN,uBAAA;EACA,sBAAA;;AAGF,OAAQ,QAAQ;EACd,uBAAA;;AAOF,mBACE;AADF,mBAEE;AAFF,mBAGE,aAAa;EACX,cAAA;EACA,WAAA;EACA,WAAA;EACA,eAAA;;AAPJ,mBAWE,aAEE;EACE,WAAA;;AAdN,mBAkBE,OAAO;AAlBT,mBAmBE,OAAO;AAnBT,mBAoBE,aAAa;AApBf,mBAqBE,aAAa;EACX,gBAAA;EACA,cAAA;;AAKF,mBADkB,OACjB,IAAI,cAAc,IAA
 I;EACrB,gBAAA;;AAEF,mBAJkB,OAIjB,YAAY,IAAI;EACf,4BAAA;EVvEF,6BAAA;EACC,4BAAA;;AUyED,mBARkB,OAQjB,WAAW,IAAI;EACd,8BAAA;EVnFF,0BAAA;EACC,yBAAA;;AUsFH,mBAAoB,aAAY,IAAI,cAAc,IAAI,aAAc;EAClE,gBAAA;;AAEF,mBAAoB,aAAY,YAAY,IAAI,aAC9C,OAAM;AADR,mBAAoB,aAAY,YAAY,IAAI,aAE9C;EVpFA,6BAAA;EACC,4BAAA;;AUuFH,mBAAoB,aAAY,WAAW,IAAI,cAAe,OAAM;EVhGlE,0BAAA;EACC,yBAAA;;AUwGH;EACE,cAAA;EACA,WAAA;EACA,mBAAA;EACA,yBAAA;;AAJF,oBAKE;AALF,oBAME;EACE,WAAA;EACA,mBAAA;EACA,SAAA;;AATJ,oBAWE,aAAa;EACX,WAAA;;AAMJ,uBAAwB,OAAO,QAAO;AACtC,uBAAwB,OAAO,QAAO;EACpC,aAAA;;AI1NF;EACE,kBAAA;EACA,cAAA;EACA,yBAAA;;AAGA,YAAC;EACC,WAAA;EACA,eAAA;EACA,gBAAA;;AATJ,YAYE;EAGE,kBAAA;EACA,UAAA;EAKA,WAAA;EAEA,WAAA;EACA,gBAAA;;AASJ,eAAgB;AAChB,eAAgB;AAChB,eAAgB,mBAAmB;Edw2BjC,YAAA;EACA,kBAAA;EACA,eAAA;EACA,iBAAA;EACA,kBAAA;;AAEA,MAAM,ech3BQ;Adg3Bd,MAAM,ec/2BQ;Ad+2Bd,MAAM,ec92BQ,mBAAmB;Ed+2B/B,YAAA;EACA,iBAAA;;AAGF,QAAQ,ecr3BM;Adq3Bd,QAAQ,ecp3BM;Ado3Bd,QAAQ,ecn3BM,mBAAmB;Ado3BjC,MAAM,UAAU,ect3BF;Ads3Bd,MAAM,UAAU,ecr3BF;Adq3Bd,MAAM,UAAU,e
 cp3BF,mBAAmB;Edq3B/B,YAAA;;Acp3BJ,eAAgB;AAChB,eAAgB;AAChB,eAAgB,mBAAmB;Edq2BjC,YAAA;EACA,iBAAA;EACA,eAAA;EACA,gBAAA;EACA,kBAAA;;AAEA,MAAM,ec72BQ;Ad62Bd,MAAM,ec52BQ;Ad42Bd,MAAM,ec32BQ,mBAAmB;Ed42B/B,YAAA;EACA,iBAAA;;AAGF,QAAQ,ecl3BM;Adk3Bd,QAAQ,ecj3BM;Adi3Bd,QAAQ,ech3BM,mBAAmB;Adi3BjC,MAAM,UAAU,ecn3BF;Adm3Bd,MAAM,UAAU,ecl3BF;Adk3Bd,MAAM,UAAU,ecj3BF,mBAAmB;Edk3B/B,YAAA;;Ac72BJ;AACA;AACA,YAAa;EACX,mBAAA;;AAEA,kBAAC,IAAI,cAAc,IAAI;AAAvB,gBAAC,IAAI,cAAc,IAAI;AAAvB,YAHW,cAGV,IAAI,cAAc,IAAI;EACrB,gBAAA;;AAIJ;AACA;EACE,SAAA;EACA,mBAAA;EACA,sBAAA;;AAKF;EACE,iBAAA;EACA,eAAA;EACA,mBAAA;EACA,cAAA;EACA,cAAA;EACA,kBAAA;EACA,yBAAA;EACA,yBAAA;EACA,kBAAA;;AAGA,kBAAC;EACC,iBAAA;EACA,eAAA;EACA,kBAAA;;AAEF,kBAAC;EACC,kBAAA;EACA,eAAA;EACA,kBAAA;;AApBJ,kBAwBE,MAAK;AAxBP,kBAyBE,MAAK;EACH,aAAA;;AAKJ,YAAa,cAAa;AAC1B,kBAAkB;AAClB,gBAAgB,YAAa;AAC7B,gBAAgB,YAAa,aAAa;AAC1C,gBAAgB,YAAa;AAC7B,gBAAgB,WAAY,OAAM,IAAI,aAAa,IAAI;AACvD,gBAAgB,WAAY,aAAY,IAAI,aAAc;EdFxD,6BAAA;EACG,0BAAA;;AcIL,kBAAkB;EAChB,eAAA;;AAEF,YAAa
 ,cAAa;AAC1B,kBAAkB;AAClB,gBAAgB,WAAY;AAC5B,gBAAgB,WAAY,aAAa;AACzC,gBAAgB,WAAY;AAC5B,gBAAgB,YAAa,OAAM,IAAI;AACvC,gBAAgB,YAAa,aAAY,IAAI,cAAe;EdN1D,4BAAA;EACG,yBAAA;;AcQL,kBAAkB;EAChB,cAAA;;AAKF;EACE,kBAAA;EAGA,YAAA;EACA,mBAAA;;AALF,gBASE;EACE,kBAAA;;AAVJ,gBASE,OAEE;EACE,iBAAA;;AAGF,gBANF,OAMG;AACD,gBAPF,OAOG;AACD,gBARF,OAQG;EACC,UAAA;;AAKJ,gBAAC,YACC;AADF,gBAAC,YAEC;EACE,kBAAA;;AAGJ,gBAAC,WACC;AADF,gBAAC,WAEC;EACE,iBAAA;;ACtJN;EACE,gBAAA;EACA,eAAA;EACA,gBAAA;;AAHF,IAME;EACE,kBAAA;EACA,cAAA;;AARJ,IAME,KAIE;EACE,kBAAA;EACA,cAAA;EACA,kBAAA;;AACA,IARJ,KAIE,IAIG;AACD,IATJ,KAIE,IAKG;EACC,qBAAA;EACA,yBAAA;;AAKJ,IAhBF,KAgBG,SAAU;EACT,cAAA;;AAEA,IAnBJ,KAgBG,SAAU,IAGR;AACD,IApBJ,KAgBG,SAAU,IAIR;EACC,cAAA;EACA,qBAAA;EACA,6BAAA;EACA,mBAAA;;AAOJ,IADF,MAAM;AAEJ,IAFF,MAAM,IAEH;AACD,IAHF,MAAM,IAGH;EACC,yBAAA;EACA,qBAAA;;AAzCN,IAkDE;EfkVA,WAAA;EACA,aAAA;EACA,gBAAA;EACA,yBAAA;;AevYF,IAyDE,KAAK,IAAI;EACP,eAAA;;AASJ;EACE,gCAAA;;AADF,SAEE;EACE,WAAA;EAEA,mBAAA;;AALJ,SAEE,KAME;EACE,iBAAA;EACA,uBAAA;EACA,6BA
 AA;EACA,0BAAA;;AACA,SAXJ,KAME,IAKG;EACC,qCAAA;;AAMF,SAlBJ,KAiBG,OAAQ;AAEP,SAnBJ,KAiBG,OAAQ,IAEN;AACD,SApBJ,KAiBG,OAAQ,IAGN;EACC,cAAA;EACA,yBAAA;EACA,yBAAA;EACA,gCAAA;EACA,eAAA;;AAKN,SAAC;EAqDD,WAAA;EA8BA,gBAAA;;AAnFA,SAAC,cAuDD;EACE,WAAA;;AAxDF,SAAC,cAuDD,KAEG;EACC,kBAAA;EACA,kBAAA;;AA3DJ,SAAC,cA+DD,YAAY;EACV,SAAA;EACA,UAAA;;AAYJ,QATqC;EASrC,SA7EG,cAqEC;IACE,mBAAA;IACA,SAAA;;EAMN,SA7EG,cAqEC,KAGE;IACE,gBAAA;;;AAzEN,SAAC,cAqFD,KAAK;EAEH,eAAA;EACA,kBAAA;;AAxFF,SAAC,cA2FD,UAAU;AA3FV,SAAC,cA4FD,UAAU,IAAG;AA5Fb,SAAC,cA6FD,UAAU,IAAG;EACX,yBAAA;;AAcJ,QAXqC;EAWrC,SA5GG,cAkGC,KAAK;IACH,gCAAA;IACA,0BAAA;;EAQN,SA5GG,cAsGC,UAAU;EAMd,SA5GG,cAuGC,UAAU,IAAG;EAKjB,SA5GG,cAwGC,UAAU,IAAG;IACX,4BAAA;;;AAhGN,UACE;EACE,WAAA;;AAFJ,UACE,KAIE;EACE,kBAAA;;AANN,UACE,KAOE;EACE,gBAAA;;AAKA,UAbJ,KAYG,OAAQ;AAEP,UAdJ,KAYG,OAAQ,IAEN;AACD,UAfJ,KAYG,OAAQ,IAGN;EACC,cAAA;EACA,yBAAA;;AAQR,YACE;EACE,WAAA;;AAFJ,YACE,KAEE;EACE,eAAA;EACA,cAAA;;AAYN;EACE,WAAA;;AADF,cAGE;EACE,WAAA;;AAJJ,cAGE,KAEG;EACC,kBAAA;EACA,kBAAA;;AAPN,
 cAWE,YAAY;EACV,SAAA;EACA,UAAA;;AAYJ,QATqC;EASrC,cARI;IACE,mBAAA;IACA,SAAA;;EAMN,cARI,KAGE;IACE,gBAAA;;;AASR;EACE,gBAAA;;AADF,mBAGE,KAAK;EAEH,eAAA;EACA,kBAAA;;AANJ,mBASE,UAAU;AATZ,mBAUE,UAAU,IAAG;AAVf,mBAWE,UAAU,IAAG;EACX,yBAAA;;AAcJ,QAXqC;EAWrC,mBAVI,KAAK;IACH,gCAAA;IACA,0BAAA;;EAQN,mBANI,UAAU;EAMd,mBALI,UAAU,IAAG;EAKjB,mBAJI,UAAU,IAAG;IACX,4BAAA;;;AAUN,YACE;EACE,aAAA;;AAFJ,YAIE;EACE,cAAA;;AASJ,SAAU;EAER,gBAAA;Ef3IA,0BAAA;EACC,yBAAA;;AgB1FH;EACE,kBAAA;EACA,gBAAA;EACA,mBAAA;EACA,6BAAA;;AAQF,QAH6C;EAG7C;IAFI,kBAAA;;;AAgBJ,QAH6C;EAG7C;IAFI,WAAA;;;AAeJ;EACE,iBAAA;EACA,mBAAA;EACA,mBAAA;EACA,kBAAA;EACA,iCAAA;EACA,kDAAA;EAEA,iCAAA;;AAEA,gBAAC;EACC,gBAAA;;AA4BJ,QAzB6C;EAyB7C;IAxBI,WAAA;IACA,aAAA;IACA,gBAAA;;EAEA,gBAAC;IACC,yBAAA;IACA,uBAAA;IACA,iBAAA;IACA,4BAAA;;EAGF,gBAAC;IACC,mBAAA;;EAKF,iBAAkB;EAClB,kBAAmB;EACnB,oBAAqB;IACnB,eAAA;IACA,gBAAA;;;AAUN,UAEE;AADF,gBACE;AAFF,UAGE;AAFF,gBAEE;EACE,mBAAA;EACA,kBAAA;;AAMF,QAJ6C;EAI7C,UATA;EASA,gBATA;EASA,UARA;EAQA,gBARA;IAKI,eAAA;IACA,cAAA;;;AAaN;E
 ACE,aAAA;EACA,qBAAA;;AAKF,QAH6C;EAG7C;IAFI,gBAAA;;;AAKJ;AACA;EACE,eAAA;EACA,QAAA;EACA,OAAA;EACA,aAAA;;AAMF,QAH6C;EAG7C;EAAA;IAFI,gBAAA;;;AAGJ;EACE,MAAA;EACA,qBAAA;;AAEF;EACE,SAAA;EACA,gBAAA;EACA,qBAAA;;AAMF;EACE,WAAA;EACA,kBAAA;EACA,eAAA;EACA,iBAAA;EACA,YAAA;;AAEA,aAAC;AACD,aAAC;EACC,qBAAA;;AASJ,QAN6C;EACzC,OAAQ,aAAa;EACrB,OAAQ,mBAAmB;IACzB,kBAAA;;;AAWN;EACE,kBAAA;EACA,YAAA;EACA,kBAAA;EACA,iBAAA;EhBsaA,eAAA;EACA,kBAAA;EgBraA,6BAAA;EACA,sBAAA;EACA,6BAAA;EACA,kBAAA;;AAIA,cAAC;EACC,aAAA;;AAdJ,cAkBE;EACE,cAAA;EACA,WAAA;EACA,WAAA;EACA,kBAAA;;AAtBJ,cAwBE,UAAU;EACR,eAAA;;AAMJ,QAH6C;EAG7C;IAFI,aAAA;;;AAUJ;EACE,mBAAA;;AADF,WAGE,KAAK;EACH,iBAAA;EACA,oBAAA;EACA,iBAAA;;AA2BF,QAxB+C;EAwB/C,WAtBE,MAAM;IACJ,gBAAA;IACA,WAAA;IACA,WAAA;IACA,aAAA;IACA,6BAAA;IACA,SAAA;IACA,gBAAA;;EAeJ,WAtBE,MAAM,eAQJ,KAAK;EAcT,WAtBE,MAAM,eASJ;IACE,0BAAA;;EAYN,WAtBE,MAAM,eAYJ,KAAK;IACH,iBAAA;;EACA,WAdJ,MAAM,eAYJ,KAAK,IAEF;EACD,WAfJ,MAAM,eAYJ,KAAK,IAGF;IACC,sBAAA;;;AAuBV,QAhB6C;EAgB7C;IAfI,WAAA;IACA,SAAA;;EAcJ,WAZI;IACE,
 WAAA;;EAWN,WAZI,KAEE;IACE,iBAAA;IACA,oBAAA;;EAIJ,WAAC,aAAa;IACZ,mBAAA;;;AAkBN,QAN2C;EACzC;ICnQA,sBAAA;;EDoQA;ICvQA,uBAAA;;;ADgRF;EACE,kBAAA;EACA,mBAAA;EACA,kBAAA;EACA,iCAAA;EACA,oCAAA;EhB3KA,4FAAA;EACQ,oFAAA;EAkeR,eAAA;EACA,kBAAA;;AQ3NF,QAjDqC;EAiDrC,YA/CI;IACE,qBAAA;IACA,gBAAA;IACA,sBAAA;;EA4CN,YAxCI;IACE,qBAAA;IACA,WAAA;IACA,sBAAA;;EAqCN,YAlCI,aAAa;IACX,WAAA;;EAiCN,YA9BI;IACE,gBAAA;IACA,sBAAA;;EA4BN,YAtBI;EAsBJ,YArBI;IACE,qBAAA;IACA,aAAA;IACA,gBAAA;IACA,eAAA;IACA,sBAAA;;EAgBN,YAdI,OAAO,MAAK;EAchB,YAbI,UAAU,MAAK;IACb,WAAA;IACA,cAAA;;EAWN,YAJI,cAAc;IACZ,MAAA;;;AQhFJ,QAHiD;EAGjD,YAJA;IAEI,kBAAA;;;AAsBN,QAd6C;EAc7C;IAbI,WAAA;IACA,SAAA;IACA,cAAA;IACA,eAAA;IACA,cAAA;IACA,iBAAA;IhBlMF,wBAAA;IACQ,gBAAA;;EgBqMN,YAAC,aAAa;IACZ,mBAAA;;;AASN,WAAY,KAAK;EACf,aAAA;EhBvOA,0BAAA;EACC,yBAAA;;AgB0OH,oBAAqB,YAAY,KAAK;EhBnOpC,6BAAA;EACC,4BAAA;;AgB2OH;EhBqQE,eAAA;EACA,kBAAA;;AgBnQA,WAAC;EhBkQD,gBAAA;EACA,mBAAA;;AgBhQA,WAAC;EhB+PD,gBAAA;EACA,mBAAA;;AgBtPF;EhBqPE,gBAAA;EACA,mBAAA;;AgBzOF,QAV6C;EAU7C;IATI
 ,WAAA;IACA,iBAAA;IACA,kBAAA;;EAGA,YAAC,aAAa;IACZ,eAAA;;;AASN;EACE,yBAAA;EACA,qBAAA;;AAFF,eAIE;EACE,cAAA;;AACA,eAFF,cAEG;AACD,eAHF,cAGG;EACC,cAAA;EACA,6BAAA;;AATN,eAaE;EACE,cAAA;;AAdJ,eAiBE,YACE,KAAK;EACH,cAAA;;AAEA,eAJJ,YACE,KAAK,IAGF;AACD,eALJ,YACE,KAAK,IAIF;EACC,cAAA;EACA,6BAAA;;AAIF,eAXJ,YAUE,UAAU;AAER,eAZJ,YAUE,UAAU,IAEP;AACD,eAbJ,YAUE,UAAU,IAGP;EACC,cAAA;EACA,yBAAA;;AAIF,eAnBJ,YAkBE,YAAY;AAEV,eApBJ,YAkBE,YAAY,IAET;AACD,eArBJ,YAkBE,YAAY,IAGT;EACC,cAAA;EACA,6BAAA;;AAxCR,eA6CE;EACE,qBAAA;;AACA,eAFF,eAEG;AACD,eAHF,eAGG;EACC,yBAAA;;AAjDN,eA6CE,eAME;EACE,yBAAA;;AApDN,eAwDE;AAxDF,eAyDE;EACE,qBAAA;;AAOE,eAHJ,YAEE,QAAQ;AAEN,eAJJ,YAEE,QAAQ,IAEL;AACD,eALJ,YAEE,QAAQ,IAGL;EACC,yBAAA;EACA,cAAA;;AAiCN,QA7BiD;EA6BjD,eAxCA,YAaI,MAAM,eACJ,KAAK;IACH,cAAA;;EACA,eAhBR,YAaI,MAAM,eACJ,KAAK,IAEF;EACD,eAjBR,YAaI,MAAM,eACJ,KAAK,IAGF;IACC,cAAA;IACA,6BAAA;;EAIF,eAvBR,YAaI,MAAM,eASJ,UAAU;EAER,eAxBR,YAaI,MAAM,eASJ,UAAU,IAEP;EACD,eAzBR,YAaI,MAAM,eASJ,UAAU,IAGP;IACC,cAAA;IACA,yBAAA;;EAIF,eA/BR,YAaI,MAAM,eAiBJ
 ,YAAY;EAEV,eAhCR,YAaI,MAAM,eAiBJ,YAAY,IAET;EACD,eAjCR,YAaI,MAAM,eAiBJ,YAAY,IAGT;IACC,cAAA;IACA,6BAAA;;;AAjGZ,eA6GE;EACE,cAAA;;AACA,eAFF,aAEG;EACC,cAAA;;AAQN;EACE,yBAAA;EACA,qBAAA;;AAFF,eAIE;EACE,cAAA;;AACA,eAFF,cAEG;AACD,eAHF,cAGG;EACC,cAAA;EACA,6BAAA;;AATN,eAaE;EACE,cAAA;;AAdJ,eAiBE,YACE,KAAK;EACH,cAAA;;AAEA,eAJJ,YACE,KAAK,IAGF;AACD,eALJ,YACE,KAAK,IAIF;EACC,cAAA;EACA,6BAAA;;AAIF,eAXJ,YAUE,UAAU;AAER,eAZJ,YAUE,UAAU,IAEP;AACD,eAbJ,YAUE,UAAU,IAGP;EACC,cAAA;EACA,yBAAA;;AAIF,eAnBJ,YAkBE,YAAY;AAEV,eApBJ,YAkBE,YAAY,IAET;AACD,eArBJ,YAkBE,YAAY,IAGT;EACC,cAAA;EACA,6BAAA;;AAxCR,eA8CE;EACE,qBAAA;;AACA,eAFF,eAEG;AACD,eAHF,eAGG;EACC,yBAAA;;AAlDN,eA8CE,eAME;EACE,yBAAA;;AArDN,eAyDE;AAzDF,eA0DE;EACE,qBAAA;;AAME,eAFJ,YACE,QAAQ;AAEN,eAHJ,YACE,QAAQ,IAEL;AACD,eAJJ,YACE,QAAQ,IAGL;EACC,yBAAA;EACA,cAAA;;AAuCN,QAnCiD;EAmCjD,eA7CA,YAYI,MAAM,eACJ;IACE,qBAAA;;EA+BR,eA7CA,YAYI,MAAM,eAIJ;IACE,yBAAA;;EA4BR,eA7CA,YAYI,MAAM,eAOJ,KAAK;IACH,cAAA;;EACA,eArBR,YAYI,MAAM,eAOJ,KAAK,IAEF;EACD,eAtBR,YAYI,MAAM,eAOJ,KAAK,IAGF
 ;IACC,cAAA;IACA,6BAAA;;EAIF,eA5BR,YAYI,MAAM,eAeJ,UAAU;EAER,eA7BR,YAYI,MAAM,eAeJ,UAAU,IAEP;EACD,eA9BR,YAYI,MAAM,eAeJ,UAAU,IAGP;IACC,cAAA;IACA,yBAAA;;EAIF,eApCR,YAYI,MAAM,eAuBJ,YAAY;EAEV,eArCR,YAYI,MAAM,eAuBJ,YAAY,IAET;EACD,eAtCR,YAYI,MAAM,eAuBJ,YAAY,IAGT;IACC,cAAA;IACA,6BAAA;;;AAvGZ,eA8GE;EACE,cAAA;;AACA,eAFF,aAEG;EACC,cAAA;;AE9lBN;EACE,iBAAA;EACA,mBAAA;EACA,gBAAA;EACA,yBAAA;EACA,kBAAA;;AALF,WAOE;EACE,qBAAA;;AARJ,WAOE,KAGE,KAAI;EACF,SAAS,QAAT;EACA,cAAA;EACA,cAAA;;AAbN,WAiBE;EACE,cAAA;;ACpBJ;EACE,qBAAA;EACA,eAAA;EACA,cAAA;EACA,kBAAA;;AAJF,WAME;EACE,eAAA;;AAPJ,WAME,KAEE;AARJ,WAME,KAGE;EACE,kBAAA;EACA,WAAA;EACA,iBAAA;EACA,uBAAA;EACA,qBAAA;EACA,cAAA;EACA,yBAAA;EACA,yBAAA;EACA,iBAAA;;AAEF,WAdF,KAcG,YACC;AADF,WAdF,KAcG,YAEC;EACE,cAAA;EnBqFN,8BAAA;EACG,2BAAA;;AmBlFD,WArBF,KAqBG,WACC;AADF,WArBF,KAqBG,WAEC;EnBuEJ,+BAAA;EACG,4BAAA;;AmBhED,WAFF,KAAK,IAEF;AAAD,WADF,KAAK,OACF;AACD,WAHF,KAAK,IAGF;AAAD,WAFF,KAAK,OAEF;EACC,cAAA;EACA,yBAAA;EACA,qBAAA;;AAMF,WAFF,UAAU;AAER,WADF,UAAU;AAER,WAHF,UAAU,IAGP
 ;AAAD,WAFF,UAAU,OAEP;AACD,WAJF,UAAU,IAIP;AAAD,WAHF,UAAU,OAGP;EACC,UAAA;EACA,cAAA;EACA,yBAAA;EACA,qBAAA;EACA,eAAA;;AAtDN,WA0DE,YACE;AA3DJ,WA0DE,YAEE,OAAM;AA5DV,WA0DE,YAGE,OAAM;AA7DV,WA0DE,YAIE;AA9DJ,WA0DE,YAKE,IAAG;AA/DP,WA0DE,YAME,IAAG;EACD,cAAA;EACA,yBAAA;EACA,qBAAA;EACA,mBAAA;;AASN,cnBodE,KACE;AmBrdJ,cnBodE,KAEE;EACE,kBAAA;EACA,eAAA;;AAEF,cANF,KAMG,YACC;AADF,cANF,KAMG,YAEC;EA7bJ,8BAAA;EACG,2BAAA;;AAgcD,cAZF,KAYG,WACC;AADF,cAZF,KAYG,WAEC;EA3cJ,+BAAA;EACG,4BAAA;;AmBnBL,cnB+cE,KACE;AmBhdJ,cnB+cE,KAEE;EACE,iBAAA;EACA,eAAA;;AAEF,cANF,KAMG,YACC;AADF,cANF,KAMG,YAEC;EA7bJ,8BAAA;EACG,2BAAA;;AAgcD,cAZF,KAYG,WACC;AADF,cAZF,KAYG,WAEC;EA3cJ,+BAAA;EACG,4BAAA;;AoBnGL;EACE,eAAA;EACA,cAAA;EACA,gBAAA;EACA,kBAAA;;AAJF,MAME;EACE,eAAA;;AAPJ,MAME,GAEE;AARJ,MAME,GAGE;EACE,qBAAA;EACA,iBAAA;EACA,yBAAA;EACA,yBAAA;EACA,mBAAA;;AAdN,MAME,GAWE,IAAG;AAjBP,MAME,GAYE,IAAG;EACD,qBAAA;EACA,yBAAA;;AApBN,MAwBE,MACE;AAzBJ,MAwBE,MAEE;EACE,YAAA;;AA3BN,MA+BE,UACE;AAhCJ,MA+BE,UAEE;EACE,WAAA;;AAlCN,MAsCE,UACE;AAvCJ,MAsCE,U
 AEE,IAAG;AAxCP,MAsCE,UAGE,IAAG;AAzCP,MAsCE,UAIE;EACE,cAAA;EACA,yBAAA;EACA,mBAAA;;AC9CN;EACE,eAAA;EACA,uBAAA;EACA,cAAA;EACA,iBAAA;EACA,cAAA;EACA,cAAA;EACA,kBAAA;EACA,mBAAA;EACA,wBAAA;EACA,oBAAA;;AAIE,MADD,MACE;AACD,MAFD,MAEE;EACC,cAAA;EACA,qBAAA;EACA,eAAA;;AAKJ,MAAC;EACC,aAAA;;AAIF,IAAK;EACH,kBAAA;EACA,SAAA;;AAOJ;ErBmhBE,yBAAA;;AAEE,cADD,MACE;AACD,cAFD,MAEE;EACC,yBAAA;;AqBnhBN;ErB+gBE,yBAAA;;AAEE,cADD,MACE;AACD,cAFD,MAEE;EACC,yBAAA;;AqB/gBN;ErB2gBE,yBAAA;;AAEE,cADD,MACE;AACD,cAFD,MAEE;EACC,yBAAA;;AqB3gBN;ErBugBE,yBAAA;;AAEE,WADD,MACE;AACD,WAFD,MAEE;EACC,yBAAA;;AqBvgBN;ErBmgBE,yBAAA;;AAEE,cADD,MACE;AACD,cAFD,MAEE;EACC,yBAAA;;AqBngBN;ErB+fE,yBAAA;;AAEE,aADD,MACE;AACD,aAFD,MAEE;EACC,yBAAA;;AsB1jBN;EACE,qBAAA;EACA,eAAA;EACA,gBAAA;EACA,eAAA;EACA,iBAAA;EACA,cAAA;EACA,cAAA;EACA,wBAAA;EACA,mBAAA;EACA,kBAAA;EACA,yBAAA;EACA,mBAAA;;AAGA,MAAC;EACC,aAAA;;AAIF,IAAK;EACH,kBAAA;EACA,SAAA;;AAEF,OAAQ;EACN,MAAA;EACA,gBAAA;;AAMF,CADD,MACE;AACD,CAFD,MAEE;EACC,cAAA;EACA,qBAAA;EACA,eAAA;;AAKJ,CAAC,gBAAgB,O
 AAQ;AACzB,UAAW,UAAU,IAAI;EACvB,cAAA;EACA,yBAAA;;AAEF,UAAW,KAAK,IAAI;EAClB,gBAAA;;AChDF;EACE,aAAA;EACA,mBAAA;EACA,cAAA;EACA,yBAAA;;AAJF,UAME;AANF,UAOE;EACE,cAAA;;AARJ,UAUE;EACE,mBAAA;EACA,eAAA;EACA,gBAAA;;AAGF,UAAW;EACT,kBAAA;;AAjBJ,UAoBE;EACE,eAAA;;AAiBJ,mBAdgD;EAchD;IAbI,iBAAA;IACA,oBAAA;;EAEA,UAAW;IACT,kBAAA;IACA,mBAAA;;EAQN,UALI;EAKJ,UAJI;IACE,eAAA;;;ArBlCN;EACE,cAAA;EACA,YAAA;EACA,mBAAA;EACA,uBAAA;EACA,yBAAA;EACA,yBAAA;EACA,kBAAA;EFkHA,wCAAA;EACQ,gCAAA;;AE1HV,UAUE;AAVF,UAWE,EAAE;EAEA,iBAAA;EACA,kBAAA;;AAIF,CAAC,UAAC;AACF,CAAC,UAAC;AACF,CAAC,UAAC;EACA,qBAAA;;AArBJ,UAyBE;EACE,YAAA;EACA,cAAA;;AsBzBJ;EACE,aAAA;EACA,mBAAA;EACA,6BAAA;EACA,kBAAA;;AAJF,MAOE;EACE,aAAA;EAEA,cAAA;;AAVJ,MAaE;EACE,iBAAA;;AAdJ,MAkBE;AAlBF,MAmBE;EACE,gBAAA;;AApBJ,MAsBE,IAAI;EACF,eAAA;;AAQJ;EACC,mBAAA;;AADD,kBAIE;EACE,kBAAA;EACA,SAAA;EACA,YAAA;EACA,cAAA;;AAQJ;ExBmXE,yBAAA;EACA,qBAAA;EACA,cAAA;;AwBrXF,cxBuXE;EACE,yBAAA;;AwBxXJ,cxB0XE;EACE,cAAA;;AwBxXJ;ExBgXE,yBAAA;EACA,qBAAA;EACA,cAAA;;AwBlXF,WxBoXE;EACE,yBAAA;;
 AwBrXJ,WxBuXE;EACE,cAAA;;AwBrXJ;ExB6WE,yBAAA;EACA,qBAAA;EACA,cAAA;;AwB/WF,cxBiXE;EACE,yBAAA;;AwBlXJ,cxBoXE;EACE,cAAA;;AwBlXJ;ExB0WE,yBAAA;EACA,qBAAA;EACA,cAAA;;AwB5WF,axB8WE;EACE,yBAAA;;AwB/WJ,axBiXE;EACE,cAAA;;AyBzaJ;EACE;IAAQ,2BAAA;;EACR;IAAQ,wBAAA;;;AAIV;EACE;IAAQ,2BAAA;;EACR;IAAQ,wBAAA;;;AASV;EACE,gBAAA;EACA,YAAA;EACA,mBAAA;EACA,yBAAA;EACA,kBAAA;EzB0FA,sDAAA;EACQ,8CAAA;;AyBtFV;EACE,WAAA;EACA,SAAA;EACA,YAAA;EACA,eAAA;EACA,iBAAA;EACA,cAAA;EACA,kBAAA;EACA,yBAAA;EzB6EA,sDAAA;EACQ,8CAAA;EAKR,mCAAA;EACQ,2BAAA;;AyB9EV,iBAAkB;EzBqSd,kBAAkB,2LAAlB;EACA,kBAAkB,mLAAlB;EyBpSF,0BAAA;;AAIF,SAAS,OAAQ;EzBoJf,0DAAA;EACQ,kDAAA;;AyB5IV;EzBkiBE,yBAAA;;AACA,iBAAkB;EA7QhB,kBAAkB,2LAAlB;EACA,kBAAkB,mLAAlB;;AyBnRJ;EzB8hBE,yBAAA;;AACA,iBAAkB;EA7QhB,kBAAkB,2LAAlB;EACA,kBAAkB,mLAAlB;;AyB/QJ;EzB0hBE,yBAAA;;AACA,iBAAkB;EA7QhB,kBAAkB,2LAAlB;EACA,kBAAkB,mLAAlB;;AyB3QJ;EzBshBE,yBAAA;;AACA,iBAAkB;EA7QhB,kBAAkB,2LAAlB;EACA,kBAAkB,mLAAlB;;A0B/UJ;AACA;EACE,gBAAA;EACA,OAAA;;AAIF;AACA,MAAO;EACL,gBAAA;;AAEF,MAAM;EACJ
 ,aAAA;;AAIF;EACE,cAAA;;AAIF;EACE,eAAA;;AAOF,MACE;EACE,kBAAA;;AAFJ,MAIE;EACE,iBAAA;;AASJ;EACE,eAAA;EACA,gBAAA;;AC7CF;EAEE,mBAAA;EACA,eAAA;;AAQF;EACE,kBAAA;EACA,cAAA;EACA,kBAAA;EAEA,mBAAA;EACA,yBAAA;EACA,yBAAA;;AAGA,gBAAC;E3BqED,4BAAA;EACC,2BAAA;;A2BnED,gBAAC;EACC,gBAAA;E3ByEF,+BAAA;EACC,8BAAA;;A2BxFH,gBAmBE;EACE,YAAA;;AApBJ,gBAsBE,SAAS;EACP,iBAAA;;AAUJ,CAAC;EACC,cAAA;;AADF,CAAC,gBAGC;EACE,cAAA;;AAIF,CARD,gBAQE;AACD,CATD,gBASE;EACC,qBAAA;EACA,yBAAA;;AAIF,CAfD,gBAeE;AACD,CAhBD,gBAgBE,OAAO;AACR,CAjBD,gBAiBE,OAAO;EACN,UAAA;EACA,cAAA;EACA,yBAAA;EACA,qBAAA;;AANF,CAfD,gBAeE,OASC;AARF,CAhBD,gBAgBE,OAAO,MAQN;AAPF,CAjBD,gBAiBE,OAAO,MAON;EACE,cAAA;;AAVJ,CAfD,gBAeE,OAYC;AAXF,CAhBD,gBAgBE,OAAO,MAWN;AAVF,CAjBD,gBAiBE,OAAO,MAUN;EACE,cAAA;;A3BoYJ,iBAAiB;EACf,cAAA;EACA,yBAAA;;AAEA,CAAC,iBAJc;EAKb,cAAA;;AADF,CAAC,iBAJc,OAOb;EAA2B,cAAA;;AAE3B,CALD,iBAJc,OASZ;AACD,CAND,iBAJc,OAUZ;EACC,cAAA;EACA,yBAAA;;AAEF,CAVD,iBAJc,OAcZ;AACD,CAXD,iBAJc,OAeZ,OAAO;AACR,CAZD,iBAJc,OAgBZ,OAAO;EACN,WAAA;EACA,yBAAA;EACA,qBA
 AA;;AAnBN,iBAAiB;EACf,cAAA;EACA,yBAAA;;AAEA,CAAC,iBAJc;EAKb,cAAA;;AADF,CAAC,iBAJc,IAOb;EAA2B,cAAA;;AAE3B,CALD,iBAJc,IASZ;AACD,CAND,iBAJc,IAUZ;EACC,cAAA;EACA,yBAAA;;AAEF,CAVD,iBAJc,IAcZ;AACD,CAXD,iBAJc,IAeZ,OAAO;AACR,CAZD,iBAJc,IAgBZ,OAAO;EACN,WAAA;EACA,yBAAA;EACA,qBAAA;;AAnBN,iBAAiB;EACf,cAAA;EACA,yBAAA;;AAEA,CAAC,iBAJc;EAKb,cAAA;;AADF,CAAC,iBAJc,OAOb;EAA2B,cAAA;;AAE3B,CALD,iBAJc,OASZ;AACD,CAND,iBAJc,OAUZ;EACC,cAAA;EACA,yBAAA;;AAEF,CAVD,iBAJc,OAcZ;AACD,CAXD,iBAJc,OAeZ,OAAO;AACR,CAZD,iBAJc,OAgBZ,OAAO;EACN,WAAA;EACA,yBAAA;EACA,qBAAA;;AAnBN,iBAAiB;EACf,cAAA;EACA,yBAAA;;AAEA,CAAC,iBAJc;EAKb,cAAA;;AADF,CAAC,iBAJc,MAOb;EAA2B,cAAA;;AAE3B,CALD,iBAJc,MASZ;AACD,CAND,iBAJc,MAUZ;EACC,cAAA;EACA,yBAAA;;AAEF,CAVD,iBAJc,MAcZ;AACD,CAXD,iBAJc,MAeZ,OAAO;AACR,CAZD,iBAJc,MAgBZ,OAAO;EACN,WAAA;EACA,yBAAA;EACA,qBAAA;;A2BlYR;EACE,aAAA;EACA,kBAAA;;AAEF;EACE,gBAAA;EACA,gBAAA;;ACtGF;EACE,mBAAA;EACA,yBAAA;EACA,6BAAA;EACA,kBAAA;E5B+GA,iDAAA;EACQ,yCAAA;;A4B3GV;EACE,aAAA;;AAKF;EACE,kBAAA;EACA,oCAAA;E5B4EA,4BAAA;EA
 CC,2BAAA;;A4B/EH,cAKE,YAAY;EACV,cAAA;;AAKJ;EACE,aAAA;EACA,gBAAA;EACA,eAAA;EACA,cAAA;;AAJF,YAME;EACE,cAAA;;AAKJ;EACE,kBAAA;EACA,yBAAA;EACA,6BAAA;E5B4DA,+BAAA;EACC,8BAAA;;A4BnDH,MACE;EACE,gBAAA;;AAFJ,MACE,cAGE;EACE,mBAAA;EACA,gBAAA;;AAIF,MATF,cASG,YACC,iBAAgB;EACd,aAAA;E5B8BN,4BAAA;EACC,2BAAA;;A4B1BC,MAhBF,cAgBG,WACC,iBAAgB;EACd,gBAAA;E5B+BN,+BAAA;EACC,8BAAA;;A4BzBH,cAAe,cACb,iBAAgB;EACd,mBAAA;;AAUJ,MACE;AADF,MAEE,oBAAoB;EAClB,gBAAA;;AAHJ,MAME,SAAQ;AANV,MAOE,oBAAmB,YAAa,SAAQ;E5BHxC,4BAAA;EACC,2BAAA;;A4BLH,MAME,SAAQ,YAIN,QAAO,YAEL,KAAI,YACF,GAAE;AAbV,MAOE,oBAAmB,YAAa,SAAQ,YAGtC,QAAO,YAEL,KAAI,YACF,GAAE;AAbV,MAME,SAAQ,YAKN,QAAO,YACL,KAAI,YACF,GAAE;AAbV,MAOE,oBAAmB,YAAa,SAAQ,YAItC,QAAO,YACL,KAAI,YACF,GAAE;AAbV,MAME,SAAQ,YAIN,QAAO,YAEL,KAAI,YAEF,GAAE;AAdV,MAOE,oBAAmB,YAAa,SAAQ,YAGtC,QAAO,YAEL,KAAI,YAEF,GAAE;AAdV,MAME,SAAQ,YAKN,QAAO,YACL,KAAI,YAEF,GAAE;AAdV,MAOE,oBAAmB,YAAa,SAAQ,YAItC,QAAO,YACL,KAAI,YAEF,GAAE;EACA,2BAAA;;AAfV,MAME,SAAQ,YAIN,QAAO,YAEL,KAAI,YAKF,GAAE;AAjBV,MAOE,oBAAmB,YAAa,S
 AAQ,YAGtC,QAAO,YAEL,KAAI,YAKF,GAAE;AAjBV,MAME,SAAQ,YAKN,QAAO,YACL,KAAI,YAKF,GAAE;AAjBV,MAOE,oBAAmB,YAAa,SAAQ,YAItC,QAAO,YACL,KAAI,YAKF,GAAE;AAjBV,MAME,SAAQ,YAIN,QAAO,YAEL,KAAI,YAMF,GAAE;AAlBV,MAOE,oBAAmB,YAAa,SAAQ,YAGtC,QAAO,YAEL,KAAI,YAMF,GAAE;AAlBV,MAME,SAAQ,YAKN,QAAO,YACL,KAAI,YAMF,GAAE;AAlBV,MAOE,oBAAmB,YAAa,SAAQ,YAItC,QAAO,YACL,KAAI,YAMF,GAAE;EACA,4BAAA;;AAnBV,MAyBE,SAAQ;AAzBV,MA0BE,oBAAmB,WAAY,SAAQ;E5BdvC,+BAAA;EACC,8BAAA;;A4BbH,MAyBE,SAAQ,WAIN,QAAO,WAEL,KAAI,WACF,GAAE;AAhCV,MA0BE,oBAAmB,WAAY,SAAQ,WAGrC,QAAO,WAEL,KAAI,WACF,GAAE;AAhCV,MAyBE,SAAQ,WAKN,QAAO,WACL,KAAI,WACF,GAAE;AAhCV,MA0BE,oBAAmB,WAAY,SAAQ,WAIrC,QAAO,WACL,KAAI,WACF,GAAE;AAhCV,MAyBE,SAAQ,WAIN,QAAO,WAEL,KAAI,WAEF,GAAE;AAjCV,MA0BE,oBAAmB,WAAY,SAAQ,WAGrC,QAAO,WAEL,KAAI,WAEF,GAAE;AAjCV,MAyBE,SAAQ,WAKN,QAAO,WACL,KAAI,WAEF,GAAE;AAjCV,MA0BE,oBAAmB,WAAY,SAAQ,WAIrC,QAAO,WACL,KAAI,WAEF,GAAE;EACA,8BAAA;;AAlCV,MAyBE,SAAQ,WAIN,QAAO,WAEL,KAAI,WAKF,GAAE;AApCV,MA0BE,oBAAmB,WAAY,SAAQ,WAGrC,QAAO,WAEL,KAAI,WAKF,GAAE;AApCV,MAyBE,SAAQ,
 WAKN,QAAO,WACL,KAAI,WAKF,GAAE;AApCV,MA0BE,oBAAmB,WAAY,SAAQ,WAIrC,QAAO,WACL,KAAI,WAKF,GAAE;AApCV,MAyBE,SAAQ,WAIN,QAAO,WAEL,KAAI,WAMF,GAAE;AArCV,MA0BE,oBAAmB,WAAY,SAAQ,WAGrC,QAAO,WAEL,KAAI,WAMF,GAAE;AArCV,MAyBE,SAAQ,WAKN,QAAO,WACL,KAAI,WAMF,GAAE;AArCV,MA0BE,oBAAmB,WAAY,SAAQ,WAIrC,QAAO,WACL,KAAI,WAMF,GAAE;EACA,+BAAA;;AAtCV,MA2CE,cAAc;AA3ChB,MA4CE,cAAc;EACZ,6BAAA;;AA7CJ,MA+CE,SAAS,QAAO,YAAa,KAAI,YAAa;AA/ChD,MAgDE,SAAS,QAAO,YAAa,KAAI,YAAa;EAC5C,aAAA;;AAjDJ,MAmDE;AAnDF,MAoDE,oBAAoB;EAClB,SAAA;;AArDJ,MAmDE,kBAGE,QAGE,KACE,KAAI;AA1DZ,MAoDE,oBAAoB,kBAElB,QAGE,KACE,KAAI;AA1DZ,MAmDE,kBAIE,QAEE,KACE,KAAI;AA1DZ,MAoDE,oBAAoB,kBAGlB,QAEE,KACE,KAAI;AA1DZ,MAmDE,kBAKE,QACE,KACE,KAAI;AA1DZ,MAoDE,oBAAoB,kBAIlB,QACE,KACE,KAAI;AA1DZ,MAmDE,kBAGE,QAGE,KAEE,KAAI;AA3DZ,MAoDE,oBAAoB,kBAElB,QAGE,KAEE,KAAI;AA3DZ,MAmDE,kBAIE,QAEE,KAEE,KAAI;AA3DZ,MAoDE,oBAAoB,kBAGlB,QAEE,KAEE,KAAI;AA3DZ,MAmDE,kBAKE,QACE,KAEE,KAAI;AA3DZ,MAoDE,oBAAoB,kBAIlB,QACE,KAEE,KAAI;EACF,cAAA;;AA5DV,MAmDE,kBAGE,QAGE,KAKE,KAAI;AA9DZ,MAoDE,oBAA
 oB,kBAElB,QAGE,KAKE,KAAI;AA9DZ,MAmDE,kBAIE,QAEE,KAKE,KAAI;AA9DZ,MAoDE,oBAAoB,kBAGlB,QAEE,KAKE,KAAI;AA9DZ,MAmDE,kBAKE,QACE,KAKE,KAAI;AA9DZ,MAoDE,oBAAoB,kBAIlB,QACE,KAKE,KAAI;AA9DZ,MAmDE,kBAGE,QAGE,KAME,KAAI;AA/DZ,MAoDE,oBAAoB,kBAElB,QAGE,KAME,KAAI;AA/DZ,MAmDE,kBAIE,QAEE,KAME,KAAI;AA/DZ,MAoDE,oBAAoB,kBAGlB,QAEE,KAME,KAAI;AA/DZ,MAmDE,kBAKE,QACE,KAME,KAAI;AA/DZ,MAoDE,oBAAoB,kBAIlB,QACE,KAME,KAAI;EACF,eAAA;;AAhEV,MAmDE,kBAiBE,QAEE,KAAI,YACF;AAvER,MAoDE,oBAAoB,kBAgBlB,QAEE,KAAI,YACF;AAvER,MAmDE,kBAkBE,QACE,KAAI,YACF;AAvER,MAoDE,oBAAoB,kBAiBlB,QACE,KAAI,YACF;AAvER,MAmDE,kBAiBE,QAEE,KAAI,YAEF;AAxER,MAoDE,oBAAoB,kBAgBlB,QAEE,KAAI,YAEF;AAxER,MAmDE,kBAkBE,QACE,KAAI,YAEF;AAxER,MAoDE,oBAAoB,kBAiBlB,QACE,KAAI,YAEF;EACE,gBAAA;;AAzEV,MAmDE,kBA0BE,QAEE,KAAI,WACF;AAhFR,MAoDE,oBAAoB,kBAyBlB,QAEE,KAAI,WACF;AAhFR,MAmDE,kBA2BE,QACE,KAAI,WACF;AAhFR,MAoDE,oBAAoB,kBA0BlB,QACE,KAAI,WACF;AAhFR,MAmDE,kBA0BE,QAEE,KAAI,WAEF;AAjFR,MAoDE,oBAAoB,kBAyBlB,QAEE,KAAI,WAEF;AAjFR,MAmDE,kBA2BE,QACE,KAAI,WAEF;AAjFR,MAoDE,o
 BAAoB,kBA0BlB,QACE,KAAI,WAEF;EACE,gBAAA;;AAlFV,MAuFE;EACE,SAAA;EACA,gBAAA;;AAUJ;EACE,mBAAA;;AADF,YAIE;EACE,gBAAA;EACA,kBAAA;EACA,gBAAA;;AAPJ,YAIE,OAIE;EACE,eAAA;;AATN,YAaE;EACE,gBAAA;;AAdJ,YAaE,eAEE,kBAAkB;EAChB,6BAAA;;AAhBN,YAmBE;EACE,aAAA;;AApBJ,YAmBE,cAEE,kBAAkB;EAChB,gCAAA;;AAON;E5BsLE,qBAAA;;AAEA,cAAE;EACA,cAAA;EACA,yBAAA;EACA,qBAAA;;AAHF,cAAE,iBAKA,kBAAkB;EAChB,yBAAA;;AAGJ,cAAE,gBACA,kBAAkB;EAChB,4BAAA;;A4BhMN;E5BmLE,qBAAA;;AAEA,cAAE;EACA,cAAA;EACA,yBAAA;EACA,qBAAA;;AAHF,cAAE,iBAKA,kBAAkB;EAChB,yBAAA;;AAGJ,cAAE,gBACA,kBAAkB;EAChB,4BAAA;;A4B7LN;E5BgLE,qBAAA;;AAEA,cAAE;EACA,cAAA;EACA,yBAAA;EACA,qBAAA;;AAHF,cAAE,iBAKA,kBAAkB;EAChB,yBAAA;;AAGJ,cAAE,gBACA,kBAAkB;EAChB,4BAAA;;A4B1LN;E5B6KE,qBAAA;;AAEA,WAAE;EACA,cAAA;EACA,yBAAA;EACA,qBAAA;;AAHF,WAAE,iBAKA,kBAAkB;EAChB,yBAAA;;AAGJ,WAAE,gBACA,kBAAkB;EAChB,4BAAA;;A4BvLN;E5B0KE,qBAAA;;AAEA,cAAE;EACA,cAAA;EACA,yBAAA;EACA,qBAAA;;AAHF,cAAE,iBAKA,kBAAkB;EAChB,yBAAA;;AAGJ,cAAE,gBACA,kBAAkB;EAChB,4BAAA;;A4BpLN;E5BuKE,qBAAA;;AAEA,aAAE;EACA,cAAA
 ;EACA,yBAAA;EACA,qBAAA;;AAHF,aAAE,iBAKA,kBAAkB;EAChB,yBAAA;;AAGJ,aAAE,gBACA,kBAAkB;EAChB,4BAAA;;A6B5ZN;EACE,gBAAA;EACA,aAAA;EACA,mBAAA;EACA,yBAAA;EACA,yBAAA;EACA,kBAAA;E7B6GA,uDAAA;EACQ,+CAAA;;A6BpHV,KAQE;EACE,kBAAA;EACA,iCAAA;;AAKJ;EACE,aAAA;EACA,kBAAA;;AAEF;EACE,YAAA;EACA,kBAAA;;ACtBF;EACE,YAAA;EACA,eAAA;EACA,iBAAA;EACA,cAAA;EACA,cAAA;EACA,4BAAA;E9BkRA,YAAA;EAGA,yBAAA;;A8BlRA,MAAC;AACD,MAAC;EACC,cAAA;EACA,qBAAA;EACA,eAAA;E9B2QF,YAAA;EAGA,yBAAA;;A8BvQA,MAAM;EACJ,UAAA;EACA,eAAA;EACA,uBAAA;EACA,SAAA;EACA,wBAAA;;ACpBJ;EACE,gBAAA;;AAIF;EACE,aAAA;EACA,cAAA;EACA,kBAAA;EACA,eAAA;EACA,MAAA;EACA,QAAA;EACA,SAAA;EACA,OAAA;EACA,aAAA;EACA,iCAAA;EAIA,UAAA;;AAGA,MAAC,KAAM;E/BiIP,mBAAmB,kBAAnB;EACI,eAAe,kBAAf;EACI,WAAW,kBAAX;EApBR,mDAAA;EACG,6CAAA;EACE,yCAAA;EACG,mCAAA;;A+B9GR,MAAC,GAAI;E/B6HL,mBAAmB,eAAnB;EACI,eAAe,eAAf;EACI,WAAW,eAAX;;A+B3HV;EACE,kBAAA;EACA,WAAA;EACA,YAAA;;AAIF;EACE,kBAAA;EACA,yBAAA;EACA,yBAAA;EACA,oCAAA;EACA,kBAAA;E/BqEA,gDAAA;EACQ,wCAAA;E+BpER,4BAAA;EAEA,aAAA;;AAIF;EACE,eAAA;E
 ACA,MAAA;EACA,QAAA;EACA,SAAA;EACA,OAAA;EACA,aAAA;EACA,yBAAA;;AAEA,eAAC;E/BwND,UAAA;EAGA,wBAAA;;A+B1NA,eAAC;E/BuND,YAAA;EAGA,yBAAA;;A+BrNF;EACE,aAAA;EACA,gCAAA;EACA,yBAAA;;AAGF,aAAc;EACZ,gBAAA;;AAIF;EACE,SAAA;EACA,uBAAA;;AAKF;EACE,kBAAA;EACA,aAAA;;AAIF;EACE,gBAAA;EACA,uBAAA;EACA,iBAAA;EACA,6BAAA;;AAJF,aAQE,KAAK;EACH,gBAAA;EACA,gBAAA;;AAVJ,aAaE,WAAW,KAAK;EACd,iBAAA;;AAdJ,aAiBE,WAAW;EACT,cAAA;;AAmBJ,QAdmC;EAEjC;IACE,YAAA;IACA,iBAAA;;EAEF;I/BPA,iDAAA;IACQ,yCAAA;;E+BWR;IAAY,YAAA;;;AAMd,QAHmC;EACjC;IAAY,YAAA;;;ACnId;EACE,kBAAA;EACA,aAAA;EACA,cAAA;EACA,mBAAA;EACA,eAAA;EACA,gBAAA;EhCiRA,UAAA;EAGA,wBAAA;;AgCjRA,QAAC;EhC8QD,YAAA;EAGA,yBAAA;;AgChRA,QAAC;EAAU,gBAAA;EAAmB,cAAA;;AAC9B,QAAC;EAAU,gBAAA;EAAmB,cAAA;;AAC9B,QAAC;EAAU,eAAA;EAAmB,cAAA;;AAC9B,QAAC;EAAU,iBAAA;EAAmB,cAAA;;AAIhC;EACE,gBAAA;EACA,gBAAA;EACA,cAAA;EACA,kBAAA;EACA,qBAAA;EACA,yBAAA;EACA,kBAAA;;AAIF;EACE,kBAAA;EACA,QAAA;EACA,SAAA;EACA,yBAAA;EACA,mBAAA;;AAGA,QAAC,IAAK;EACJ,SAAA;EACA,SAAA;EACA,iBAAA;EACA,uBAAA;EACA,yBAAA;;AAEF,QAAC,S
 AAU;EACT,SAAA;EACA,SAAA;EACA,uBAAA;EACA,yBAAA;;AAEF,QAAC,UAAW;EACV,SAAA;EACA,UAAA;EACA,uBAAA;EACA,yBAAA;;AAEF,QAAC,MAAO;EACN,QAAA;EACA,OAAA;EACA,gBAAA;EACA,2BAAA;EACA,2BAAA;;AAEF,QAAC,KAAM;EACL,QAAA;EACA,QAAA;EACA,gBAAA;EACA,2BAAA;EACA,0BAAA;;AAEF,QAAC,OAAQ;EACP,MAAA;EACA,SAAA;EACA,iBAAA;EACA,uBAAA;EACA,4BAAA;;AAEF,QAAC,YAAa;EACZ,MAAA;EACA,SAAA;EACA,uBAAA;EACA,4BAAA;;AAEF,QAAC,aAAc;EACb,MAAA;EACA,UAAA;EACA,uBAAA;EACA,4BAAA;;ACvFJ;EACE,kBAAA;EACA,MAAA;EACA,OAAA;EACA,aAAA;EACA,aAAA;EACA,gBAAA;EACA,YAAA;EACA,gBAAA;EACA,yBAAA;EACA,4BAAA;EACA,yBAAA;EACA,oCAAA;EACA,kBAAA;EjCuGA,iDAAA;EACQ,yCAAA;EiCpGR,mBAAA;;AAGA,QAAC;EAAW,iBAAA;;AACZ,QAAC;EAAW,iBAAA;;AACZ,QAAC;EAAW,gBAAA;;AACZ,QAAC;EAAW,kBAAA;;AAGd;EACE,SAAA;EACA,iBAAA;EACA,eAAA;EACA,mBAAA;EACA,iBAAA;EACA,yBAAA;EACA,gCAAA;EACA,0BAAA;;AAGF;EACE,iBAAA;;AAQA,QADO;AAEP,QAFO,SAEN;EACC,kBAAA;EACA,cAAA;EACA,QAAA;EACA,SAAA;EACA,yBAAA;EACA,mBAAA;;AAGJ,QAAS;EACP,kBAAA;;AAEF,QAAS,SAAQ;EACf,kBAAA;EACA,SAAS,EAAT;;AAIA,QAAC,IAAK;EACJ,SAAA;EACA,kBAAA;E
 ACA,sBAAA;EACA,yBAAA;EACA,qCAAA;EACA,aAAA;;AACA,QAPD,IAAK,SAOH;EACC,SAAS,GAAT;EACA,WAAA;EACA,kBAAA;EACA,sBAAA;EACA,yBAAA;;AAGJ,QAAC,MAAO;EACN,QAAA;EACA,WAAA;EACA,iBAAA;EACA,oBAAA;EACA,2BAAA;EACA,uCAAA;;AACA,QAPD,MAAO,SAOL;EACC,SAAS,GAAT;EACA,SAAA;EACA,aAAA;EACA,oBAAA;EACA,2BAAA;;AAGJ,QAAC,OAAQ;EACP,SAAA;EACA,kBAAA;EACA,mBAAA;EACA,4BAAA;EACA,wCAAA;EACA,UAAA;;AACA,QAPD,OAAQ,SAON;EACC,SAAS,GAAT;EACA,QAAA;EACA,kBAAA;EACA,mBAAA;EACA,4BAAA;;AAIJ,QAAC,KAAM;EACL,QAAA;EACA,YAAA;EACA,iBAAA;EACA,qBAAA;EACA,0BAAA;EACA,sCAAA;;AACA,QAPD,KAAM,SAOJ;EACC,SAAS,GAAT;EACA,UAAA;EACA,qBAAA;EACA,0BAAA;EACA,aAAA;;A9B1HN;EACE,kBAAA;;AAGF;EACE,kBAAA;EACA,gBAAA;EACA,WAAA;;AAHF,eAKE;EACE,aAAA;EACA,kBAAA;EH8GF,yCAAA;EACQ,iCAAA;;AGtHV,eAKE,QAME;AAXJ,eAKE,QAOE,IAAI;EAEF,cAAA;;AAdN,eAkBE;AAlBF,eAmBE;AAnBF,eAoBE;EAAU,cAAA;;AApBZ,eAsBE;EACE,OAAA;;AAvBJ,eA0BE;AA1BF,eA2BE;EACE,kBAAA;EACA,MAAA;EACA,WAAA;;AA9BJ,eAiCE;EACE,UAAA;;AAlCJ,eAoCE;EACE,WAAA;;AArCJ,eAuCE,QAAO;AAvCT,eAwCE,QAAO;EACL,OAAA;;AAzCJ,eA4CE,UAAS;EACP,WAA
 A;;AA7CJ,eA+CE,UAAS;EACP,UAAA;;AAQJ;EACE,kBAAA;EACA,MAAA;EACA,OAAA;EACA,SAAA;EACA,UAAA;EHsNA,YAAA;EAGA,yBAAA;EGvNA,eAAA;EACA,cAAA;EACA,kBAAA;EACA,yCAAA;;AAKA,iBAAC;EH8NC,kBAAkB,8BAA8B,mCAAyC,uCAAzF;EACA,kBAAmB,4EAAnB;EACA,2BAAA;EACA,sHAAA;;AG9NF,iBAAC;EACC,UAAA;EACA,QAAA;EHyNA,kBAAkB,8BAA8B,sCAAyC,oCAAzF;EACA,kBAAmB,4EAAnB;EACA,2BAAA;EACA,sHAAA;;AGvNF,iBAAC;AACD,iBAAC;EACC,aAAA;EACA,cAAA;EACA,qBAAA;EH8LF,YAAA;EAGA,yBAAA;;AG9NF,iBAkCE;AAlCF,iBAmCE;AAnCF,iBAoCE;AApCF,iBAqCE;EACE,kBAAA;EACA,QAAA;EACA,UAAA;EACA,qBAAA;;AAzCJ,iBA2CE;AA3CF,iBA4CE;EACE,SAAA;;AA7CJ,iBA+CE;AA/CF,iBAgDE;EACE,UAAA;;AAjDJ,iBAmDE;AAnDF,iBAoDE;EACE,WAAA;EACA,YAAA;EACA,iBAAA;EACA,kBAAA;EACA,kBAAA;;AAIA,iBADF,WACG;EACC,SAAS,OAAT;;AAIF,iBADF,WACG;EACC,SAAS,OAAT;;AAUN;EACE,kBAAA;EACA,YAAA;EACA,SAAA;EACA,WAAA;EACA,UAAA;EACA,iBAAA;EACA,eAAA;EACA,gBAAA;EACA,kBAAA;;AATF,oBAWE;EACE,qBAAA;EACA,WAAA;EACA,YAAA;EACA,WAAA;EACA,mBAAA;EACA,yBAAA;EACA,mBAAA;EACA,eAAA;EAUA,yBAAA;EACA,kCAAA;;AA9BJ,oBAgCE;EACE,SAAA;EACA,WAAA;EACA,YAA
 A;EACA,yBAAA;;AAOJ;EACE,kBAAA;EACA,SAAA;EACA,UAAA;EACA,YAAA;EACA,WAAA;EACA,iBAAA;EACA,oBAAA;EACA,cAAA;EACA,kBAAA;EACA,yCAAA;;AACA,iBAAE;EACA,iBAAA;;AAkCJ,mBA5B8C;EAG5C,iBACE;EADF,iBAEE;EAFF,iBAGE;EAHF,iBAIE;IACE,WAAA;IACA,YAAA;IACA,iBAAA;IACA,kBAAA;IACA,eAAA;;EAKJ;IACE,SAAA;IACA,UAAA;IACA,oBAAA;;EAIF;IACE,YAAA;;;AHlNF,SAAC;AACD,SAAC;AMXH,UNUG;AMVH,UNWG;AMSH,gBNVG;AMUH,gBNTG;AMkBH,INnBG;AMmBH,INlBG;AQsXH,gBAoBE,YR3YC;AQuXH,gBAoBE,YR1YC;AUkBH,YVnBG;AUmBH,YVlBG;AU8HH,mBAWE,aV1IC;AU+HH,mBAWE,aVzIC;AeZH,IfWG;AeXH,IfYG;AgBVH,OhBSG;AgBTH,OhBUG;AgBUH,chBXG;AgBWH,chBVG;AgB6BH,gBhB9BG;AgB8BH,gBhB7BG;AoBfH,MpBcG;AoBdH,MpBeG;A4BLH,W5BIG;A4BJH,W5BKG;A+B+EH,a/BhFG;A+BgFH,a/B/EG;EACC,SAAS,GAAT;EACA,cAAA;;AAEF,SAAC;AMfH,UNeG;AMKH,gBNLG;AMcH,INdG;AQkXH,gBAoBE,YRtYC;AUcH,YVdG;AU0HH,mBAWE,aVrIC;AehBH,IfgBG;AgBdH,OhBcG;AgBMH,chBNG;AgByBH,gBhBzBG;AoBnBH,MpBmBG;A4BTH,W5BSG;A+B2EH,a/B3EG;EACC,WAAA;;AiBdJ;EjB6BE,cAAA;EACA,iBAAA;EACA,kBAAA;;AiB5BF;EACE,uBAAA;;AAEF;EACE,sBAAA;;AAQF;EACE,wBAAA;;AAEF;EACE,yBAA
 A;;AAEF;EACE,kBAAA;;AAEF;EjB8CE,WAAA;EACA,kBAAA;EACA,iBAAA;EACA,6BAAA;EACA,SAAA;;AiBzCF;EACE,wBAAA;EACA,6BAAA;;AAOF;EACE,eAAA;;AiBnCF;EACE,mBAAA;;AAKF;AACA;AACA;AACA;ElCylBE,wBAAA;;AkCjlBF,QAHqC;EAGrC;IlCykBE,yBAAA;;EACA,KAAK;IAAK,cAAA;;EACV,EAAE;IAAQ,kBAAA;;EACV,EAAE;EACF,EAAE;IAAQ,mBAAA;;;AkCxkBZ,QAHqC,uBAAgC;EAGrE;IlCokBE,yBAAA;;EACA,KAAK;IAAK,cAAA;;EACV,EAAE;IAAQ,kBAAA;;EACV,EAAE;EACF,EAAE;IAAQ,mBAAA;;;AkCnkBZ,QAHqC,uBAAgC;EAGrE;IlC+jBE,yBAAA;;EACA,KAAK;IAAK,cAAA;;EACV,EAAE;IAAQ,kBAAA;;EACV,EAAE;EACF,EAAE;IAAQ,mBAAA;;;AkC9jBZ,QAHqC;EAGrC;IlC0jBE,yBAAA;;EACA,KAAK;IAAK,cAAA;;EACV,EAAE;IAAQ,kBAAA;;EACV,EAAE;EACF,EAAE;IAAQ,mBAAA;;;AkCxjBZ,QAHqC;EAGrC;IlC4jBE,wBAAA;;;AkCvjBF,QAHqC,uBAAgC;EAGrE;IlCujBE,wBAAA;;;AkCljBF,QAHqC,uBAAgC;EAGrE;IlCkjBE,wBAAA;;;AkC7iBF,QAHqC;EAGrC;IlC6iBE,wBAAA;;;AkCtiBF;ElCsiBE,wBAAA;;AkChiBF;EAAA;IlCwhBE,yBAAA;;EACA,KAAK;IAAK,cAAA;;EACV,EAAE;IAAQ,kBAAA;;EACV,EAAE;EACF,EAAE;IAAQ,mBAAA;;;AkCthBZ;EAAA;IlC0hBE,wBAAA","sourcesContent":["/*! normalize.css v3.0.0 |
  MIT License | git.io/normalize */\n\n//\n// 1. Set default font family to sans-serif.\n// 2. Prevent iOS text size adjust after orientation change, without disabling\n//    user zoom.\n//\n\nhtml {\n  font-family: sans-serif; // 1\n  -ms-text-size-adjust: 100%; // 2\n  -webkit-text-size-adjust: 100%; // 2\n}\n\n//\n// Remove default margin.\n//\n\nbody {\n  margin: 0;\n}\n\n// HTML5 display definitions\n// ==========================================================================\n\n//\n// Correct `block` display not defined in IE 8/9.\n//\n\narticle,\naside,\ndetails,\nfigcaption,\nfigure,\nfooter,\nheader,\nhgroup,\nmain,\nnav,\nsection,\nsummary {\n  display: block;\n}\n\n//\n// 1. Correct `inline-block` display not defined in IE 8/9.\n// 2. Normalize vertical alignment of `progress` in Chrome, Firefox, and Opera.\n//\n\naudio,\ncanvas,\nprogress,\nvideo {\n  display: inline-block; // 1\n  vertical-align: baseline; // 2\n}\n\n//\n// Prevent modern browsers from displaying `audio
 ` without controls.\n// Remove excess height in iOS 5 devices.\n//\n\naudio:not([controls]) {\n  display: none;\n  height: 0;\n}\n\n//\n// Address `[hidden]` styling not present in IE 8/9.\n// Hide the `template` element in IE, Safari, and Firefox < 22.\n//\n\n[hidden],\ntemplate {\n  display: none;\n}\n\n// Links\n// ==========================================================================\n\n//\n// Remove the gray background color from active links in IE 10.\n//\n\na {\n  background: transparent;\n}\n\n//\n// Improve readability when focused and also mouse hovered in all browsers.\n//\n\na:active,\na:hover {\n  outline: 0;\n}\n\n// Text-level semantics\n// ==========================================================================\n\n//\n// Address styling not present in IE 8/9, Safari 5, and Chrome.\n//\n\nabbr[title] {\n  border-bottom: 1px dotted;\n}\n\n//\n// Address style set to `bolder` in Firefox 4+, Safari 5, and Chrome.\n//\n\nb,\nstrong {\n  font-weight: bold;\n}\n\n//\n
 // Address styling not present in Safari 5 and Chrome.\n//\n\ndfn {\n  font-style: italic;\n}\n\n//\n// Address variable `h1` font-size and margin within `section` and `article`\n// contexts in Firefox 4+, Safari 5, and Chrome.\n//\n\nh1 {\n  font-size: 2em;\n  margin: 0.67em 0;\n}\n\n//\n// Address styling not present in IE 8/9.\n//\n\nmark {\n  background: #ff0;\n  color: #000;\n}\n\n//\n// Address inconsistent and variable font size in all browsers.\n//\n\nsmall {\n  font-size: 80%;\n}\n\n//\n// Prevent `sub` and `sup` affecting `line-height` in all browsers.\n//\n\nsub,\nsup {\n  font-size: 75%;\n  line-height: 0;\n  position: relative;\n  vertical-align: baseline;\n}\n\nsup {\n  top: -0.5em;\n}\n\nsub {\n  bottom: -0.25em;\n}\n\n// Embedded content\n// ==========================================================================\n\n//\n// Remove border when inside `a` element in IE 8/9.\n//\n\nimg {\n  border: 0;\n}\n\n//\n// Correct overflow displayed oddly in IE 9.\n//\n\nsvg:no
 t(:root) {\n  overflow: hidden;\n}\n\n// Grouping content\n// ==========================================================================\n\n//\n// Address margin not present in IE 8/9 and Safari 5.\n//\n\nfigure {\n  margin: 1em 40px;\n}\n\n//\n// Address differences between Firefox and other browsers.\n//\n\nhr {\n  -moz-box-sizing: content-box;\n  box-sizing: content-box;\n  height: 0;\n}\n\n//\n// Contain overflow in all browsers.\n//\n\npre {\n  overflow: auto;\n}\n\n//\n// Address odd `em`-unit font size rendering in all browsers.\n//\n\ncode,\nkbd,\npre,\nsamp {\n  font-family: monospace, monospace;\n  font-size: 1em;\n}\n\n// Forms\n// ==========================================================================\n\n//\n// Known limitation: by default, Chrome and Safari on OS X allow very limited\n// styling of `select`, unless a `border` property is set.\n//\n\n//\n// 1. Correct color not being inherited.\n//    Known issue: affects color of disabled elements.\n// 2. Correct fon
 t properties not being inherited.\n// 3. Address margins set differently in Firefox 4+, Safari 5, and Chrome.\n//\n\nbutton,\ninput,\noptgroup,\nselect,\ntextarea {\n  color: inherit; // 1\n  font: inherit; // 2\n  margin: 0; // 3\n}\n\n//\n// Address `overflow` set to `hidden` in IE 8/9/10.\n//\n\nbutton {\n  overflow: visible;\n}\n\n//\n// Address inconsistent `text-transform` inheritance for `button` and `select`.\n// All other form control elements do not inherit `text-transform` values.\n// Correct `button` style inheritance in Firefox, IE 8+, and Opera\n// Correct `select` style inheritance in Firefox.\n//\n\nbutton,\nselect {\n  text-transform: none;\n}\n\n//\n// 1. Avoid the WebKit bug in Android 4.0.* where (2) destroys native `audio`\n//    and `video` controls.\n// 2. Correct inability to style clickable `input` types in iOS.\n// 3. Improve usability and consistency of cursor style between image-type\n//    `input` and others.\n//\n\nbutton,\nhtml input[type=\"button\"], 
 // 1\ninput[type=\"reset\"],\ninput[type=\"submit\"] {\n  -webkit-appearance: button; // 2\n  cursor: pointer; // 3\n}\n\n//\n// Re-set default cursor for disabled elements.\n//\n\nbutton[disabled],\nhtml input[disabled] {\n  cursor: default;\n}\n\n//\n// Remove inner padding and border in Firefox 4+.\n//\n\nbutton::-moz-focus-inner,\ninput::-moz-focus-inner {\n  border: 0;\n  padding: 0;\n}\n\n//\n// Address Firefox 4+ setting `line-height` on `input` using `!important` in\n// the UA stylesheet.\n//\n\ninput {\n  line-height: normal;\n}\n\n//\n// It's recommended that you don't attempt to style these elements.\n// Firefox's implementation doesn't respect box-sizing, padding, or width.\n//\n// 1. Address box sizing set to `content-box` in IE 8/9/10.\n// 2. Remove excess padding in IE 8/9/10.\n//\n\ninput[type=\"checkbox\"],\ninput[type=\"radio\"] {\n  box-sizing: border-box; // 1\n  padding: 0; // 2\n}\n\n//\n// Fix the cursor style for Chrome's increment/decrement buttons. For cert
 ain\n// `font-size` values of the `input`, it causes the cursor style of the\n// decrement button to change from `default` to `text`.\n//\n\ninput[type=\"number\"]::-webkit-inner-spin-button,\ninput[type=\"number\"]::-webkit-outer-spin-button {\n  height: auto;\n}\n\n//\n// 1. Address `appearance` set to `searchfield` in Safari 5 and Chrome.\n// 2. Address `box-sizing` set to `border-box` in Safari 5 and Chrome\n//    (include `-moz` to future-proof).\n//\n\ninput[type=\"search\"] {\n  -webkit-appearance: textfield; // 1\n  -moz-box-sizing: content-box;\n  -webkit-box-sizing: content-box; // 2\n  box-sizing: content-box;\n}\n\n//\n// Remove inner padding and search cancel button in Safari and Chrome on OS X.\n// Safari (but not Chrome) clips the cancel button when the search input has\n// padding (and `textfield` appearance).\n//\n\ninput[type=\"search\"]::-webkit-search-cancel-button,\ninput[type=\"search\"]::-webkit-search-decoration {\n  -webkit-appearance: none;\n}\n\n//\n// Def
 ine consistent border, margin, and padding.\n//\n\nfieldset {\n  border: 1px solid #c0c0c0;\n  margin: 0 2px;\n  padding: 0.35em 0.625em 0.75em;\n}\n\n//\n// 1. Correct `color` not being inherited in IE 8/9.\n// 2. Remove padding so people aren't caught out if they zero out fieldsets.\n//\n\nlegend {\n  border: 0; // 1\n  padding: 0; // 2\n}\n\n//\n// Remove default vertical scrollbar in IE 8/9.\n//\n\ntextarea {\n  overflow: auto;\n}\n\n//\n// Don't inherit the `font-weight` (applied by a rule above).\n// NOTE: the default cannot safely be changed in Chrome and Safari on OS X.\n//\n\noptgroup {\n  font-weight: bold;\n}\n\n// Tables\n// ==========================================================================\n\n//\n// Remove most spacing between table cells.\n//\n\ntable {\n  border-collapse: collapse;\n  border-spacing: 0;\n}\n\ntd,\nth {\n  padding: 0;\n}","//\n// Basic print styles\n// --------------------------------------------------\n// Source: https://github.com/h5bp/html5-
 boilerplate/blob/master/css/main.css\n\n@media print {\n\n  * {\n    text-shadow: none !important;\n    color: #000 !important; // Black prints faster: h5bp.com/s\n    background: transparent !important;\n    box-shadow: none !important;\n  }\n\n  a,\n  a:visited {\n    text-decoration: underline;\n  }\n\n  a[href]:after {\n    content: \" (\" attr(href) \")\";\n  }\n\n  abbr[title]:after {\n    content: \" (\" attr(title) \")\";\n  }\n\n  // Don't show links for images, or javascript/internal links\n  a[href^=\"javascript:\"]:after,\n  a[href^=\"#\"]:after {\n    content: \"\";\n  }\n\n  pre,\n  blockquote {\n    border: 1px solid #999;\n    page-break-inside: avoid;\n  }\n\n  thead {\n    display: table-header-group; // h5bp.com/t\n  }\n\n  tr,\n  img {\n    page-break-inside: avoid;\n  }\n\n  img {\n    max-width: 100% !important;\n  }\n\n  p,\n  h2,\n  h3 {\n    orphans: 3;\n    widows: 3;\n  }\n\n  h2,\n  h3 {\n    page-break-after: avoid;\n  }\n\n  // Chrome (OSX) fix for http
 s://github.com/twbs/bootstrap/issues/11245\n  // Once fixed, we can just straight up remove this.\n  select {\n    background: #fff !important;\n  }\n\n  // Bootstrap components\n  .navbar {\n    display: none;\n  }\n  .table {\n    td,\n    th {\n      background-color: #fff !important;\n    }\n  }\n  .btn,\n  .dropup > .btn {\n    > .caret {\n      border-top-color: #000 !important;\n    }\n  }\n  .label {\n    border: 1px solid #000;\n  }\n\n  .table {\n    border-collapse: collapse !important;\n  }\n  .table-bordered {\n    th,\n    td {\n      border: 1px solid #ddd !important;\n    }\n  }\n\n}\n","//\n// Scaffolding\n// --------------------------------------------------\n\n\n// Reset the box-sizing\n//\n// Heads up! This reset may cause conflicts with some third-party widgets.\n// For recommendations on resolving such conflicts, see\n// http://getbootstrap.com/getting-started/#third-box-sizing\n* {\n  .box-sizing(border-box);\n}\n*:before,\n*:after {\n  .box-sizing(border-box)
 ;\n}\n\n\n// Body reset\n\nhtml {\n  font-size: 62.5%;\n  -webkit-tap-highlight-color: rgba(0,0,0,0);\n}\n\nbody {\n  font-family: @font-family-base;\n  font-size: @font-size-base;\n  line-height: @line-height-base;\n  color: @text-color;\n  background-color: @body-bg;\n}\n\n// Reset fonts for relevant elements\ninput,\nbutton,\nselect,\ntextarea {\n  font-family: inherit;\n  font-size: inherit;\n  line-height: inherit;\n}\n\n\n// Links\n\na {\n  color: @link-color;\n  text-decoration: none;\n\n  &:hover,\n  &:focus {\n    color: @link-hover-color;\n    text-decoration: underline;\n  }\n\n  &:focus {\n    .tab-focus();\n  }\n}\n\n\n// Figures\n//\n// We reset this here because previously Normalize had no `figure` margins. This\n// ensures we don't break anyone's use of the element.\n\nfigure {\n  margin: 0;\n}\n\n\n// Images\n\nimg {\n  vertical-align: middle;\n}\n\n// Responsive images (ensure images don't scale beyond their parents)\n.img-responsive {\n  .img-responsive();\n}\n\n/
 / Rounded corners\n.img-rounded {\n  border-radius: @border-radius-large;\n}\n\n// Image thumbnails\n//\n// Heads up! This is mixin-ed into thumbnails.less for `.thumbnail`.\n.img-thumbnail {\n  padding: @thumbnail-padding;\n  line-height: @line-height-base;\n  background-color: @thumbnail-bg;\n  border: 1px solid @thumbnail-border;\n  border-radius: @thumbnail-border-radius;\n  .transition(all .2s ease-in-out);\n\n  // Keep them at most 100% wide\n  .img-responsive(inline-block);\n}\n\n// Perfect circle\n.img-circle {\n  border-radius: 50%; // set radius in percents\n}\n\n\n// Horizontal rules\n\nhr {\n  margin-top:    @line-height-computed;\n  margin-bottom: @line-height-computed;\n  border: 0;\n  border-top: 1px solid @hr-border;\n}\n\n\n// Only display content to screen readers\n//\n// See: http://a11yproject.com/posts/how-to-hide-content/\n\n.sr-only {\n  position: absolute;\n  width: 1px;\n  height: 1px;\n  margin: -1px;\n  padding: 0;\n  overflow: hidden;\n  clip: rect(0,0,0,
 0);\n  border: 0;\n}\n","//\n// Mixins\n// --------------------------------------------------\n\n\n// Utilities\n// -------------------------\n\n// Clearfix\n// Source: http://nicolasgallagher.com/micro-clearfix-hack/\n//\n// For modern browsers\n// 1. The space content is one way to avoid an Opera bug when the\n//    contenteditable attribute is included anywhere else in the document.\n//    Otherwise it causes space to appear at the top and bottom of elements\n//    that are clearfixed.\n// 2. The use of `table` rather than `block` is only necessary if using\n//    `:before` to contain the top-margins of child elements.\n.clearfix() {\n  &:before,\n  &:after {\n    content: \" \"; // 1\n    display: table; // 2\n  }\n  &:after {\n    clear: both;\n  }\n}\n\n// WebKit-style focus\n.tab-focus() {\n  // Default\n  outline: thin dotted;\n  // WebKit\n  outline: 5px auto -webkit-focus-ring-color;\n  outline-offset: -2px;\n}\n\n// Center-align a block level element\n.center-block() {\n 
  display: block;\n  margin-left: auto;\n  margin-right: auto;\n}\n\n// Sizing shortcuts\n.size(@width; @height) {\n  width: @width;\n  height: @height;\n}\n.square(@size) {\n  .size(@size; @size);\n}\n\n// Placeholder text\n.placeholder(@color: @input-color-placeholder) {\n  &::-moz-placeholder           { color: @color;   // Firefox\n                                  opacity: 1; } // See https://github.com/twbs/bootstrap/pull/11526\n  &:-ms-input-placeholder       { color: @color; } // Internet Explorer 10+\n  &::-webkit-input-placeholder  { color: @color; } // Safari and Chrome\n}\n\n// Text overflow\n// Requires inline-block or block for proper styling\n.text-overflow() {\n  overflow: hidden;\n  text-overflow: ellipsis;\n  white-space: nowrap;\n}\n\n// CSS image replacement\n//\n// Heads up! v3 launched with with only `.hide-text()`, but per our pattern for\n// mixins being reused as classes with the same name, this doesn't hold up. As\n// of v3.0.1 we have added `.text-hide()` a
 nd deprecated `.hide-text()`. Note\n// that we cannot chain the mixins together in Less, so they are repeated.\n//\n// Source: https://github.com/h5bp/html5-boilerplate/commit/aa0396eae757\n\n// Deprecated as of v3.0.1 (will be removed in v4)\n.hide-text() {\n  font: ~\"0/0\" a;\n  color: transparent;\n  text-shadow: none;\n  background-color: transparent;\n  border: 0;\n}\n// New mixin to use as of v3.0.1\n.text-hide() {\n  .hide-text();\n}\n\n\n\n// CSS3 PROPERTIES\n// --------------------------------------------------\n\n// Single side border-radius\n.border-top-radius(@radius) {\n  border-top-right-radius: @radius;\n   border-top-left-radius: @radius;\n}\n.border-right-radius(@radius) {\n  border-bottom-right-radius: @radius;\n     border-top-right-radius: @radius;\n}\n.border-bottom-radius(@radius) {\n  border-bottom-right-radius: @radius;\n   border-bottom-left-radius: @radius;\n}\n.border-left-radius(@radius) {\n  border-bottom-left-radius: @radius;\n     border-top-left-radi
 us: @radius;\n}\n\n// Drop shadows\n//\n// Note: Deprecated `.box-shadow()` as of v3.1.0 since all of Bootstrap's\n//   supported browsers that have box shadow capabilities now support the\n//   standard `box-shadow` property.\n.box-shadow(@shadow) {\n  -webkit-box-shadow: @shadow; // iOS <4.3 & Android <4.1\n          box-shadow: @shadow;\n}\n\n// Transitions\n.transition(@transition) {\n  -webkit-transition: @transition;\n          transition: @transition;\n}\n.transition-property(@transition-property) {\n  -webkit-transition-property: @transition-property;\n          transition-property: @transition-property;\n}\n.transition-delay(@transition-delay) {\n  -webkit-transition-delay: @transition-delay;\n          transition-delay: @transition-delay;\n}\n.transition-duration(@transition-duration) {\n  -webkit-transition-duration: @transition-duration;\n          transition-duration: @transition-duration;\n}\n.transition-transform(@transition) {\n  -webkit-transition: -webkit-transform
  @transition;\n     -moz-transition: -moz-transform @transition;\n       -o-transition: -o-transform @transition;\n          transition: transform @transition;\n}\n\n// Transformations\n.rotate(@degrees) {\n  -webkit-transform: rotate(@degrees);\n      -ms-transform: rotate(@degrees); // IE9 only\n          transform: rotate(@degrees);\n}\n.scale(@ratio; @ratio-y...) {\n  -webkit-transform: scale(@ratio, @ratio-y);\n      -ms-transform: scale(@ratio, @ratio-y); // IE9 only\n          transform: scale(@ratio, @ratio-y);\n}\n.translate(@x; @y) {\n  -webkit-transform: translate(@x, @y);\n      -ms-transform: translate(@x, @y); // IE9 only\n          transform: translate(@x, @y);\n}\n.skew(@x; @y) {\n  -webkit-transform: skew(@x, @y);\n      -ms-transform: skewX(@x) skewY(@y); // See https://github.com/twbs/bootstrap/issues/4885; IE9+\n          transform: skew(@x, @y);\n}\n.translate3d(@x; @y; @z) {\n  -webkit-transform: translate3d(@x, @y, @z);\n          transform: translate3d(@x, @y
 , @z);\n}\n\n.rotateX(@degrees) {\n  -webkit-transform: rotateX(@degrees);\n      -ms-transform: rotateX(@degrees); // IE9 only\n          transform: rotateX(@degrees);\n}\n.rotateY(@degrees) {\n  -webkit-transform: rotateY(@degrees);\n      -ms-transform: rotateY(@degrees); // IE9 only\n          transform: rotateY(@degrees);\n}\n.perspective(@perspective) {\n  -webkit-perspective: @perspective;\n     -moz-perspective: @perspective;\n          perspective: @perspective;\n}\n.perspective-origin(@perspective) {\n  -webkit-perspective-origin: @perspective;\n     -moz-perspective-origin: @perspective;\n          perspective-origin: @perspective;\n}\n.transform-origin(@origin) {\n  -webkit-transform-origin: @origin;\n     -moz-transform-origin: @origin;\n      -ms-transform-origin: @origin; // IE9 only\n          transform-origin: @origin;\n}\n\n// Animations\n.animation(@animation) {\n  -webkit-animation: @animation;\n          animation: @animation;\n}\n.animation-name(@name) {\n  -we
 bkit-animation-name: @name;\n          animation-name: @name;\n}\n.animation-duration(@duration) {\n  -webkit-animation-duration: @duration;\n          animation-duration: @duration;\n}\n.animation-timing-function(@timing-function) {\n  -webkit-animation-timing-function: @timing-function;\n          animation-timing-function: @timing-function;\n}\n.animation-delay(@delay) {\n  -webkit-animation-delay: @delay;\n          animation-delay: @delay;\n}\n.animation-iteration-count(@iteration-count) {\n  -webkit-animation-iteration-count: @iteration-count;\n          animation-iteration-count: @iteration-count;\n}\n.animation-direction(@direction) {\n  -webkit-animation-direction: @direction;\n          animation-direction: @direction;\n}\n\n// Backface visibility\n// Prevent browsers from flickering when using CSS 3D transforms.\n// Default value is `visible`, but can be changed to `hidden`\n.backface-visibility(@visibility){\n  -webkit-backface-visibility: @visibility;\n     -moz-backfac
 e-visibility: @visibility;\n          backface-visibility: @visibility;\n}\n\n// Box sizing\n.box-sizing(@boxmodel) {\n  -webkit-box-sizing: @boxmodel;\n     -moz-box-sizing: @boxmodel;\n          box-sizing: @boxmodel;\n}\n\n// User select\n// For selecting text on the page\n.user-select(@select) {\n  -webkit-user-select: @select;\n     -moz-user-select: @select;\n      -ms-user-select: @select; // IE10+\n          user-select: @select;\n}\n\n// Resize anything\n.resizable(@direction) {\n  resize: @direction; // Options: horizontal, vertical, both\n  overflow: auto; // Safari fix\n}\n\n// CSS3 Content Columns\n.content-columns(@column-count; @column-gap: @grid-gutter-width) {\n  -webkit-column-count: @column-count;\n     -moz-column-count: @column-count;\n          column-count: @column-count;\n  -webkit-column-gap: @column-gap;\n     -moz-column-gap: @column-gap;\n          column-gap: @column-gap;\n}\n\n// Optional hyphenation\n.hyphens(@mode: auto) {\n  word-wrap: break-word;\n 
  -webkit-hyphens: @mode;\n     -moz-hyphens: @mode;\n      -ms-hyphens: @mode; // IE10+\n       -o-hyphens: @mode;\n          hyphens: @mode;\n}\n\n// Opacity\n.opacity(@opacity) {\n  opacity: @opacity;\n  // IE8 filter\n  @opacity-ie: (@opacity * 100);\n  filter: ~\"alpha(opacity=@{opacity-ie})\";\n}\n\n\n\n// GRADIENTS\n// --------------------------------------------------\n\n#gradient {\n\n  // Horizontal gradient, from left to right\n  //\n  // Creates two color stops, start and end, by specifying a color and position for each color stop.\n  // Color stops are not available in IE9 and below.\n  .horizontal(@start-color: #555; @end-color: #333; @start-percent: 0%; @end-percent: 100%) {\n    background-image: -webkit-linear-gradient(left, color-stop(@start-color @start-percent), color-stop(@end-color @end-percent)); // Safari 5.1-6, Chrome 10+\n    background-image:  linear-gradient(to right, @start-color @start-percent, @end-color @end-percent); // Standard, IE10, Firefox 16+, Op
 era 12.10+, Safari 7+, Chrome 26+\n    background-repeat: repeat-x;\n    filter: e(%(\"progid:DXImageTransform.Microsoft.gradient(startColorstr='%d', endColorstr='%d', GradientType=1)\",argb(@start-color),argb(@end-color))); // IE9 and down\n  }\n\n  // Vertical gradient, from top to bottom\n  //\n  // Creates two color stops, start and end, by specifying a color and position for each color stop.\n  // Color stops are not available in IE9 and below.\n  .vertical(@start-color: #555; @end-color: #333; @start-percent: 0%; @end-percent: 100%) {\n    background-image: -webkit-linear-gradient(top, @start-color @start-percent, @end-color @end-percent);  // Safari 5.1-6, Chrome 10+\n    background-image: linear-gradient(to bottom, @start-color @start-percent, @end-color @end-percent); // Standard, IE10, Firefox 16+, Opera 12.10+, Safari 7+, Chrome 26+\n    background-repeat: repeat-x;\n    filter: e(%(\"progid:DXImageTransform.Microsoft.gradient(startColorstr='%d', endColorstr='%d', Gradien
 tType=0)\",argb(@start-color),argb(@end-color))); // IE9 and down\n  }\n\n  .directional(@start-color: #555; @end-color: #333; @deg: 45deg) {\n    background-repeat: repeat-x;\n    background-image: -webkit-linear-gradient(@deg, @start-color, @end-color); // Safari 5.1-6, Chrome 10+\n    background-image: linear-gradient(@deg, @start-color, @end-color); // Standard, IE10, Firefox 16+, Opera 12.10+, Safari 7+, Chrome 26+\n  }\n  .horizontal-three-colors(@start-color: #00b3ee; @mid-color: #7a43b6; @color-stop: 50%; @end-color: #c3325f) {\n    background-image: -webkit-linear-gradient(left, @start-color, @mid-color @color-stop, @end-color);\n    background-image: linear-gradient(to right, @start-color, @mid-color @color-stop, @end-color);\n    background-repeat: no-repeat;\n    filter: e(%(\"progid:DXImageTransform.Microsoft.gradient(startColorstr='%d', endColorstr='%d', GradientType=1)\",argb(@start-color),argb(@end-color))); // IE9 and down, gets no color-stop at all for proper fallb
 ack\n  }\n  .vertical-three-colors(@start-color: #00b3ee; @mid-color: #7a43b6; @color-stop: 50%; @end-color: #c3325f) {\n    background-image: -webkit-linear-gradient(@start-color, @mid-color @color-stop, @end-color);\n    background-image: linear-gradient(@start-color, @mid-color @color-stop, @end-color);\n    background-repeat: no-repeat;\n    filter: e(%(\"progid:DXImageTransform.Microsoft.gradient(startColorstr='%d', endColorstr='%d', GradientType=0)\",argb(@start-color),argb(@end-color))); // IE9 and down, gets no color-stop at all for proper fallback\n  }\n  .radial(@inner-color: #555; @outer-color: #333) {\n    background-image: -webkit-radial-gradient(circle, @inner-color, @outer-color);\n    background-image: radial-gradient(circle, @inner-color, @outer-color);\n    background-repeat: no-repeat;\n  }\n  .striped(@color: rgba(255,255,255,.15); @angle: 45deg) {\n    background-image: -webkit-linear-gradient(@angle, @color 25%, transparent 25%, transparent 50%, @color 50%, @co
 lor 75%, transparent 75%, transparent);\n    background-image: linear-gradient(@angle, @color 25%, transparent 25%, transparent 50%, @color 50%, @color 75%, transparent 75%, transparent);\n  }\n}\n\n// Reset filters for IE\n//\n// When you need to remove a gradient background, do not forget to use this to reset\n// the IE filter for IE9 and below.\n.reset-filter() {\n  filter: e(%(\"progid:DXImageTransform.Microsoft.gradient(enabled = false)\"));\n}\n\n\n\n// Retina images\n//\n// Short retina mixin for setting background-image and -size\n\n.img-retina(@file-1x; @file-2x; @width-1x; @height-1x) {\n  background-image: url(\"@{file-1x}\");\n\n  @media\n  only screen and (-webkit-min-device-pixel-ratio: 2),\n  only screen and (   min--moz-device-pixel-ratio: 2),\n  only screen and (     -o-min-device-pixel-ratio: 2/1),\n  only screen and (        min-device-pixel-ratio: 2),\n  only screen and (                min-resolution: 192dpi),\n  only screen and (                min-resolution: 
 2dppx) {\n    background-image: url(\"@{file-2x}\");\n    background-size: @width-1x @height-1x;\n  }\n}\n\n\n// Responsive image\n//\n// Keep images from scaling beyond the width of their parents.\n\n.img-responsive(@display: block) {\n  display: @display;\n  max-width: 100%; // Part 1: Set a maximum relative to the parent\n  height: auto; // Part 2: Scale the height according to the width, otherwise you get stretching\n}\n\n\n// COMPONENT MIXINS\n// --------------------------------------------------\n\n// Horizontal dividers\n// -------------------------\n// Dividers (basically an hr) within dropdowns and nav lists\n.nav-divider(@color: #e5e5e5) {\n  height: 1px;\n  margin: ((@line-height-computed / 2) - 1) 0;\n  overflow: hidden;\n  background-color: @color;\n}\n\n// Panels\n// -------------------------\n.panel-variant(@border; @heading-text-color; @heading-bg-color; @heading-border) {\n  border-color: @border;\n\n  & > .panel-heading {\n    color: @heading-text-color;\n    backg
 round-color: @heading-bg-color;\n    border-color: @heading-border;\n\n    + .panel-collapse .panel-body {\n      border-top-color: @border;\n    }\n  }\n  & > .panel-footer {\n    + .panel-collapse .panel-body {\n      border-bottom-color: @border;\n    }\n  }\n}\n\n// Alerts\n// -------------------------\n.alert-variant(@background; @border; @text-color) {\n  background-color: @background;\n  border-color: @border;\n  color: @text-color;\n\n  hr {\n    border-top-color: darken(@border, 5%);\n  }\n  .alert-link {\n    color: darken(@text-color, 10%);\n  }\n}\n\n// Tables\n// -------------------------\n.table-row-variant(@state; @background) {\n  // Exact selectors below required to override `.table-striped` and prevent\n  // inheritance to nested tables.\n  .table > thead > tr,\n  .table > tbody > tr,\n  .table > tfoot > tr {\n    > td.@{state},\n    > th.@{state},\n    &.@{state} > td,\n    &.@{state} > th {\n      background-color: @background;\n    }\n  }\n\n  // Hover states fo
 r `.table-hover`\n  // Note: this is not available for cells or rows within `thead` or `tfoot`.\n  .table-hover > tbody > tr {\n    > td.@{state}:hover,\n    > th.@{state}:hover,\n    &.@{state}:hover > td,\n    &.@{state}:hover > th {\n      background-color: darken(@background, 5%);\n    }\n  }\n}\n\n// List Groups\n// -------------------------\n.list-group-item-variant(@state; @background; @color) {\n  .list-grou

<TRUNCATED>

[27/29] samza git commit: more review comments

Posted by bo...@apache.org.
more review comments


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

Branch: refs/heads/NewKafkaSystemConsumer
Commit: 190a39990a9281511e07876636b6c8784337d35a
Parents: 93ca950
Author: Boris S <bo...@apache.org>
Authored: Tue Sep 25 15:55:15 2018 -0700
Committer: Boris S <bo...@apache.org>
Committed: Tue Sep 25 15:55:15 2018 -0700

----------------------------------------------------------------------
 .../samza/config/KafkaConsumerConfig.java       |  47 ++-
 .../samza/system/kafka/KafkaConsumerProxy.java  | 345 +++++++++----------
 .../samza/system/kafka/KafkaSystemConsumer.java |  49 ++-
 .../kafka/KafkaSystemConsumerMetrics.scala      |   2 +-
 .../samza/config/TestKafkaConsumerConfig.java   |  19 +-
 5 files changed, 235 insertions(+), 227 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/samza/blob/190a3999/samza-kafka/src/main/scala/org/apache/samza/config/KafkaConsumerConfig.java
----------------------------------------------------------------------
diff --git a/samza-kafka/src/main/scala/org/apache/samza/config/KafkaConsumerConfig.java b/samza-kafka/src/main/scala/org/apache/samza/config/KafkaConsumerConfig.java
index 7d2408b..3fa66e5 100644
--- a/samza-kafka/src/main/scala/org/apache/samza/config/KafkaConsumerConfig.java
+++ b/samza-kafka/src/main/scala/org/apache/samza/config/KafkaConsumerConfig.java
@@ -31,6 +31,7 @@ import org.apache.samza.SamzaException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import scala.Option;
+import scala.runtime.AbstractFunction0;
 
 
 /**
@@ -40,9 +41,9 @@ public class KafkaConsumerConfig extends HashMap<String, Object> {
 
   public static final Logger LOG = LoggerFactory.getLogger(KafkaConsumerConfig.class);
 
-  private static final String PRODUCER_CLIENT_ID_PREFIX = "kafka-producer";
-  private static final String CONSUMER_CLIENT_ID_PREFIX = "kafka-consumer";
-  private static final String ADMIN_CLIENT_ID_PREFIX = "samza-admin";
+  static final String PRODUCER_CLIENT_ID_PREFIX = "kafka-producer";
+  static final String CONSUMER_CLIENT_ID_PREFIX = "kafka-consumer";
+  static final String ADMIN_CLIENT_ID_PREFIX = "samza-admin";
 
   /*
    * By default, KafkaConsumer will fetch some big number of available messages for all the partitions.
@@ -55,12 +56,12 @@ public class KafkaConsumerConfig extends HashMap<String, Object> {
   }
 
   /**
-   * This is a help method to create the configs for use in Kafka consumer.
+   * Helper method to create configs for use in Kafka consumer.
    * The values are based on the "consumer" subset of the configs provided by the app and Samza overrides.
    *
-   * @param config - config provided by the app.
-   * @param systemName - system name for which the consumer is configured.
-   * @param clientId - client id to be used in the Kafka consumer.
+   * @param config config provided by the app.
+   * @param systemName system name to get the consumer configuration for.
+   * @param clientId client id to be used in the Kafka consumer.
    * @return KafkaConsumerConfig
    */
   public static KafkaConsumerConfig getKafkaSystemConsumerConfig(Config config, String systemName, String clientId) {
@@ -85,7 +86,7 @@ public class KafkaConsumerConfig extends HashMap<String, Object> {
     consumerProps.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG,
         getAutoOffsetResetValue((String) consumerProps.get(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG)));
 
-    // make sure bootstrap configs are in, if not - get them from the producer
+    // if consumer bootstrap servers are not configured, get them from the producer configs
     if (!subConf.containsKey(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG)) {
       String bootstrapServers =
           config.get(String.format("systems.%s.producer.%s", systemName, ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG));
@@ -119,10 +120,19 @@ public class KafkaConsumerConfig extends HashMap<String, Object> {
   // group id should be unique per job
   static String getConsumerGroupId(Config config) {
     JobConfig jobConfig = new JobConfig(config);
-    Option<String> jobIdOption = jobConfig.getJobId();
-    Option<String> jobNameOption = jobConfig.getName();
-    return (jobNameOption.isDefined() ? jobNameOption.get() : "undefined_job_name") + "-" + (jobIdOption.isDefined()
-        ? jobIdOption.get() : "undefined_job_id");
+    Option jobNameOption = jobConfig.getName();
+    if (jobNameOption.isEmpty()) {
+      throw new ConfigException("Missing job name");
+    }
+    String jobName = (String) jobNameOption.get();
+
+    Option jobIdOption = jobConfig.getJobId();
+    String jobId = "1";
+    if (! jobIdOption.isEmpty()) {
+      jobId = (String) jobIdOption.get();
+    }
+
+    return String.format("%s-%s", jobName, jobId);
   }
 
   // client id should be unique per job
@@ -139,11 +149,18 @@ public class KafkaConsumerConfig extends HashMap<String, Object> {
   }
 
   static String getConsumerClientId(String id, Config config) {
-    if (config.get(JobConfig.JOB_NAME()) == null) {
+    JobConfig jobConfig = new JobConfig(config);
+    Option jobNameOption = jobConfig.getName();
+    if (jobNameOption.isEmpty()) {
       throw new ConfigException("Missing job name");
     }
-    String jobName = config.get(JobConfig.JOB_NAME());
-    String jobId = (config.get(JobConfig.JOB_ID()) != null) ? config.get(JobConfig.JOB_ID()) : "1";
+    String jobName = (String) jobNameOption.get();
+
+    Option jobIdOption = jobConfig.getJobId();
+    String jobId = "1";
+    if (! jobIdOption.isEmpty()) {
+      jobId = (String) jobIdOption.get();
+    }
 
     return String.format("%s-%s-%s", id.replaceAll("\\W", "_"), jobName.replaceAll("\\W", "_"),
         jobId.replaceAll("\\W", "_"));

http://git-wip-us.apache.org/repos/asf/samza/blob/190a3999/samza-kafka/src/main/scala/org/apache/samza/system/kafka/KafkaConsumerProxy.java
----------------------------------------------------------------------
diff --git a/samza-kafka/src/main/scala/org/apache/samza/system/kafka/KafkaConsumerProxy.java b/samza-kafka/src/main/scala/org/apache/samza/system/kafka/KafkaConsumerProxy.java
index d2f7096..04071c1 100644
--- a/samza-kafka/src/main/scala/org/apache/samza/system/kafka/KafkaConsumerProxy.java
+++ b/samza-kafka/src/main/scala/org/apache/samza/system/kafka/KafkaConsumerProxy.java
@@ -30,13 +30,10 @@ import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
-import javax.print.DocFlavor;
-import kafka.common.KafkaException;
 import kafka.common.TopicAndPartition;
 import org.apache.kafka.clients.consumer.Consumer;
 import org.apache.kafka.clients.consumer.ConsumerRecord;
 import org.apache.kafka.clients.consumer.ConsumerRecords;
-import org.apache.kafka.clients.consumer.InvalidOffsetException;
 import org.apache.kafka.common.Metric;
 import org.apache.kafka.common.MetricName;
 import org.apache.kafka.common.TopicPartition;
@@ -53,21 +50,21 @@ import org.slf4j.LoggerFactory;
  * This class is not thread safe. There will be only one instance of this class per KafkaSystemConsumer object.
  * We still need some synchronization around kafkaConsumer. See pollConsumer() method for details.
  */
-/*package private */class KafkaConsumerProxy<K, V> {
+class KafkaConsumerProxy<K, V> {
   private static final Logger LOG = LoggerFactory.getLogger(KafkaConsumerProxy.class);
 
   private static final int SLEEP_MS_WHILE_NO_TOPIC_PARTITION = 100;
 
-  /* package private */ final Thread consumerPollThread;
+  final Thread consumerPollThread;
   private final Consumer<K, V> kafkaConsumer;
   private final KafkaSystemConsumer.KafkaConsumerMessageSink sink;
   private final KafkaSystemConsumerMetrics kafkaConsumerMetrics;
   private final String metricName;
   private final String systemName;
   private final String clientId;
-  private final Map<TopicPartition, SystemStreamPartition> topicPartitions2SSP = new HashMap<>();
+  private final Map<TopicPartition, SystemStreamPartition> topicPartitionToSSP = new HashMap<>();
   private final Map<SystemStreamPartition, MetricName> perPartitionMetrics = new HashMap<>();
-  // list of all the SSPs we poll from, with their next offsets correspondingly.
+  // list of all the SSPs we poll from, with their next(most recently read + 1) offsets correspondingly.
   private final Map<SystemStreamPartition, Long> nextOffsets = new ConcurrentHashMap<>();
   // lags behind the high water mark, as reported by the Kafka consumer.
   private final Map<SystemStreamPartition, Long> latestLags = new HashMap<>();
@@ -76,7 +73,6 @@ import org.slf4j.LoggerFactory;
   private volatile Throwable failureCause = null;
   private final CountDownLatch consumerPollThreadStartLatch = new CountDownLatch(1);
 
-  // package private constructor
   KafkaConsumerProxy(Consumer<K, V> kafkaConsumer, String systemName, String clientId,
       KafkaSystemConsumer.KafkaConsumerMessageSink messageSink, KafkaSystemConsumerMetrics samzaConsumerMetrics,
       String metricName) {
@@ -96,14 +92,46 @@ import org.slf4j.LoggerFactory;
         "Samza KafkaConsumerProxy Poll " + consumerPollThread.getName() + " - " + systemName);
   }
 
-  @Override
-  public String toString() {
-    return String.format("consumerProxy-%s-%s", systemName, clientId);
+  /**
+   * Add new partition to the list of polled partitions.
+   * Bust only be called before {@link KafkaConsumerProxy#start} is called..
+   */
+  public void addTopicPartition(SystemStreamPartition ssp, long nextOffset) {
+    LOG.info(String.format("Adding new topicPartition %s with offset %s to queue for consumer %s", ssp, nextOffset,
+        this));
+    topicPartitionToSSP.put(KafkaSystemConsumer.toTopicPartition(ssp), ssp); //registered SSPs
+
+    // this is already vetted offset so there is no need to validate it
+    nextOffsets.put(ssp, nextOffset);
+
+    kafkaConsumerMetrics.setNumTopicPartitions(metricName, nextOffsets.size());
+  }
+
+  /**
+   * Stop this KafkaConsumerProxy and wait for at most {@code timeoutMs}.
+   * @param timeoutMs maximum time to wait to stop this KafkaConsumerProxy
+   */
+  public void stop(long timeoutMs) {
+    LOG.info("Shutting down KafkaConsumerProxy poll thread {} for {}", consumerPollThread.getName(), this);
+
+    isRunning = false;
+    try {
+      consumerPollThread.join(timeoutMs/2);
+      // join() may timeout
+      // in this case we should interrupt it and wait again
+      if (consumerPollThread.isAlive()) {
+        consumerPollThread.interrupt();
+        consumerPollThread.join(timeoutMs/2);
+      }
+    } catch (InterruptedException e) {
+      LOG.warn("Join in KafkaConsumerProxy has failed", e);
+      consumerPollThread.interrupt();
+    }
   }
 
   public void start() {
     if (!consumerPollThread.isAlive()) {
-      LOG.info("Starting KafkaConsumerProxy polling thread for system " + systemName + " " + this.toString());
+      LOG.info("Starting KafkaConsumerProxy polling thread for " + this.toString());
 
       consumerPollThread.start();
 
@@ -112,70 +140,124 @@ import org.slf4j.LoggerFactory;
         try {
           consumerPollThreadStartLatch.await(3000, TimeUnit.MILLISECONDS);
         } catch (InterruptedException e) {
-          LOG.info("Got InterruptedException", e);
+          LOG.info("Ignoring InterruptedException while waiting for consumer poll thread to start.", e);
         }
       }
     } else {
       LOG.warn("Tried to start an already started KafkaConsumerProxy (%s). Ignoring.", this.toString());
     }
 
-    if (topicPartitions2SSP.size() == 0) {
-      String msg = String.format("Cannot start empty set of TopicPartitions for system %s, clientid %s",
-          systemName, clientId);
+    if (topicPartitionToSSP.size() == 0) {
+      String msg = String.format("Cannot start KafkaConsumerProxy without any registered TopicPartitions for %s", systemName);
       LOG.error(msg);
       throw new SamzaException(msg);
     }
   }
 
-  /**
-   * Stop the thread and wait for it to stop.
-   * @param timeoutMs how long to wait in join
-   */
-  public void stop(long timeoutMs) {
-    LOG.info("Shutting down KafkaConsumerProxy poll thread:" + consumerPollThread.getName());
+  boolean isRunning() {
+    return isRunning;
+  }
 
-    isRunning = false;
-    try {
-      consumerPollThread.join(timeoutMs);
-      // join returns event if the thread didn't finish
-      // in this case we should interrupt it and wait again
-      if (consumerPollThread.isAlive()) {
-        consumerPollThread.interrupt();
-        consumerPollThread.join(timeoutMs);
+  Throwable getFailureCause() {
+    return failureCause;
+  }
+
+  private void initializeLags() {
+    // This is expensive, so only do it once at the beginning. After the first poll, we can rely on metrics for lag.
+    Map<TopicPartition, Long> endOffsets = kafkaConsumer.endOffsets(topicPartitionToSSP.keySet());
+    endOffsets.forEach((tp, offset) -> {
+      SystemStreamPartition ssp = topicPartitionToSSP.get(tp);
+      long startingOffset = nextOffsets.get(ssp);
+      // End offsets are the offset of the newest message + 1
+      // If the message we are about to consume is < end offset, we are starting with a lag.
+      long initialLag = endOffsets.get(tp) - startingOffset;
+
+      LOG.info("Initial lag for SSP {} is {} (end={}, startOffset={})", ssp, initialLag, endOffsets.get(tp), startingOffset);
+      latestLags.put(ssp, initialLag);
+      sink.setIsAtHighWatermark(ssp, initialLag == 0);
+    });
+
+    // initialize lag metrics
+    refreshLagMetrics();
+  }
+
+  // creates a separate thread for getting the messages.
+  private Runnable createProxyThreadRunnable() {
+    Runnable runnable = () -> {
+      isRunning = true;
+
+      try {
+        consumerPollThreadStartLatch.countDown();
+        LOG.info("Starting consumer poll thread {} for system {}", consumerPollThread.getName(), systemName);
+        initializeLags();
+        while (isRunning) {
+          fetchMessages();
+        }
+      } catch (Throwable throwable) {
+        LOG.error(String.format("Error in KafkaConsumerProxy poll thread for system: %s.", systemName), throwable);
+        // KafkaSystemConsumer uses the failureCause to propagate the throwable to the container
+        failureCause = throwable;
+        isRunning = false;
       }
-    } catch (InterruptedException e) {
-      LOG.warn("Join in KafkaConsumerProxy has failed", e);
-      consumerPollThread.interrupt();
-    }
+
+      if (!isRunning) {
+        LOG.info("KafkaConsumerProxy for system {} has stopped.", systemName);
+      }
+    };
+
+    return runnable;
   }
 
-  /**
-   * Add new partition to the list of polled partitions.
-   * This method should be called only at the beginning, before the thread is started.
-   */
-  public void addTopicPartition(SystemStreamPartition ssp, long nextOffset) {
-    LOG.info(String.format("Adding new topic and partition %s, offset = %s to queue for consumer %s", ssp, nextOffset,
-        this));
-    topicPartitions2SSP.put(KafkaSystemConsumer.toTopicPartition(ssp), ssp); //registered SSPs
+  private void fetchMessages() {
+    Set<SystemStreamPartition> sspsToFetch = new HashSet<>();
+    for (SystemStreamPartition ssp : nextOffsets.keySet()) {
+      if (sink.needsMoreMessages(ssp)) {
+        sspsToFetch.add(ssp);
+      }
+    }
+    LOG.debug("pollConsumer for {} SSPs: {}", sspsToFetch.size(), sspsToFetch);
+    if (!sspsToFetch.isEmpty()) {
+      kafkaConsumerMetrics.incClientReads(metricName);
 
-    // this is already vetted offset so there is no need to validate it
-    LOG.info(String.format("Got offset %s for new topic and partition %s.", nextOffset, ssp));
+      Map<SystemStreamPartition, List<IncomingMessageEnvelope>> response;
 
-    nextOffsets.put(ssp, nextOffset);
+      response = pollConsumer(sspsToFetch, 500L);
+
+      // move the responses into the queue
+      for (Map.Entry<SystemStreamPartition, List<IncomingMessageEnvelope>> e : response.entrySet()) {
+        List<IncomingMessageEnvelope> envelopes = e.getValue();
+        if (envelopes != null) {
+          moveMessagesToTheirQueue(e.getKey(), envelopes);
+        }
+      }
+
+      populateCurrentLags(sspsToFetch); // find current lags for for each SSP
+    } else { // nothing to read
+
+      LOG.debug("No topic/partitions need to be fetched for system {} right now. Sleeping {}ms.", systemName,
+          SLEEP_MS_WHILE_NO_TOPIC_PARTITION);
+
+      kafkaConsumerMetrics.incClientSkippedFetchRequests(metricName);
 
-    kafkaConsumerMetrics.setTopicPartitionValue(metricName, nextOffsets.size());
+      try {
+        Thread.sleep(SLEEP_MS_WHILE_NO_TOPIC_PARTITION);
+      } catch (InterruptedException e) {
+        LOG.warn("Sleep in fetchMessages was interrupted");
+      }
+    }
+    refreshLagMetrics();
   }
 
   // the actual polling of the messages from kafka
   private Map<SystemStreamPartition, List<IncomingMessageEnvelope>> pollConsumer(
-      Set<SystemStreamPartition> systemStreamPartitions, long timeout) {
+      Set<SystemStreamPartition> systemStreamPartitions, long timeoutMs) {
 
     // Since we need to poll only from some subset of TopicPartitions (passed as the argument),
     // we need to pause the rest.
     List<TopicPartition> topicPartitionsToPause = new ArrayList<>();
     List<TopicPartition> topicPartitionsToPoll = new ArrayList<>();
 
-    for (Map.Entry<TopicPartition, SystemStreamPartition> e : topicPartitions2SSP.entrySet()) {
+    for (Map.Entry<TopicPartition, SystemStreamPartition> e : topicPartitionToSSP.entrySet()) {
       TopicPartition tp = e.getKey();
       SystemStreamPartition ssp = e.getValue();
       if (systemStreamPartitions.contains(ssp)) {
@@ -186,21 +268,18 @@ import org.slf4j.LoggerFactory;
     }
 
     ConsumerRecords<K, V> records;
-
     try {
       // Synchronize, in case the consumer is used in some other thread (metadata or something else)
       synchronized (kafkaConsumer) {
         // Since we are not polling from ALL the subscribed topics, so we need to "change" the subscription temporarily
         kafkaConsumer.pause(topicPartitionsToPause);
         kafkaConsumer.resume(topicPartitionsToPoll);
-        records = kafkaConsumer.poll(timeout);
-        // resume original set of subscription - may be required for checkpointing
-        kafkaConsumer.resume(topicPartitionsToPause);
+        records = kafkaConsumer.poll(timeoutMs);
       }
     } catch (Exception e) {
       // we may get InvalidOffsetException | AuthorizationException | KafkaException exceptions,
       // but we still just rethrow, and log it up the stack.
-      LOG.error("Caught a Kafka exception in pollConsumer", e);
+      LOG.error("Caught a Kafka exception in pollConsumer for system " + systemName, e);
       throw e;
     }
 
@@ -209,12 +288,11 @@ import org.slf4j.LoggerFactory;
 
   private Map<SystemStreamPartition, List<IncomingMessageEnvelope>> processResults(ConsumerRecords<K, V> records) {
     if (records == null) {
-      throw new SamzaException("ERROR:records is null, after pollConsumer call (in processResults)");
+      throw new SamzaException("Received null 'records' after polling consumer in KafkaConsumerProxy " + this);
     }
 
     Map<SystemStreamPartition, List<IncomingMessageEnvelope>> results = new HashMap<>(records.count());
     // Parse the returned records and convert them into the IncomingMessageEnvelope.
-    // Note. They have been already de-serialized by the consumer.
     for (ConsumerRecord<K, V> record : records) {
       int partition = record.partition();
       String topic = record.topic();
@@ -222,18 +300,18 @@ import org.slf4j.LoggerFactory;
 
       updateMetrics(record, tp);
 
-      SystemStreamPartition ssp = topicPartitions2SSP.get(tp);
-      List<IncomingMessageEnvelope> listMsgs = results.get(ssp);
-      if (listMsgs == null) {
-        listMsgs = new ArrayList<>();
-        results.put(ssp, listMsgs);
+      SystemStreamPartition ssp = topicPartitionToSSP.get(tp);
+      List<IncomingMessageEnvelope> messages = results.get(ssp);
+      if (messages == null) {
+        messages = new ArrayList<>();
+        results.put(ssp, messages);
       }
 
-      final K key = record.key();
-      final Object value = record.value();
-      final IncomingMessageEnvelope imEnvelope =
+      K key = record.key();
+      Object value = record.value();
+      IncomingMessageEnvelope imEnvelope =
           new IncomingMessageEnvelope(ssp, String.valueOf(record.offset()), key, value, getRecordSize(record));
-      listMsgs.add(imEnvelope);
+      messages.add(imEnvelope);
     }
     if (LOG.isDebugEnabled()) {
       LOG.debug("# records per SSP:");
@@ -246,52 +324,6 @@ import org.slf4j.LoggerFactory;
     return results;
   }
 
-   // creates a separate thread for getting the messages.
-  private Runnable createProxyThreadRunnable() {
-    Runnable runnable=  () -> {
-      isRunning = true;
-
-      try {
-        consumerPollThreadStartLatch.countDown();
-        LOG.info("Starting runnable " + consumerPollThread.getName());
-        initializeLags();
-        while (isRunning) {
-          fetchMessages();
-        }
-      } catch (Throwable throwable) {
-        LOG.error(String.format("Error in KafkaConsumerProxy poll thread for system: %s.", systemName), throwable);
-        // KafkaSystemConsumer uses the failureCause to propagate the throwable to the container
-        failureCause = throwable;
-        isRunning = false;
-      }
-
-      if (!isRunning) {
-        LOG.info("KafkaConsumerProxy for system {} has stopped.", systemName);
-      }
-    };
-
-    return runnable;
-  }
-
-  private void initializeLags() {
-    // This is expensive, so only do it once at the beginning. After the first poll, we can rely on metrics for lag.
-    Map<TopicPartition, Long> endOffsets = kafkaConsumer.endOffsets(topicPartitions2SSP.keySet());
-    endOffsets.forEach((tp, offset) -> {
-      SystemStreamPartition ssp = topicPartitions2SSP.get(tp);
-      long startingOffset = nextOffsets.get(ssp);
-      // End offsets are the offset of the newest message + 1
-      // If the message we are about to consume is < end offset, we are starting with a lag.
-      long initialLag = endOffsets.get(tp) - startingOffset;
-
-      LOG.info("Initial lag for SSP {} is {} (end={}, startOffset={})", ssp, initialLag, endOffsets.get(tp), startingOffset);
-      latestLags.put(ssp, initialLag);
-      sink.setIsAtHighWatermark(ssp, initialLag == 0);
-    });
-
-    // initialize lag metrics
-    refreshLatencyMetrics();
-  }
-
   private int getRecordSize(ConsumerRecord<K, V> r) {
     int keySize = (r.key() == null) ? 0 : r.serializedKeySize();
     return keySize + r.serializedValueSize();
@@ -300,10 +332,16 @@ import org.slf4j.LoggerFactory;
   private void updateMetrics(ConsumerRecord<K, V> r, TopicPartition tp) {
     TopicAndPartition tap = KafkaSystemConsumer.toTopicAndPartition(tp);
     SystemStreamPartition ssp = new SystemStreamPartition(systemName, tp.topic(), new Partition(tp.partition()));
-    long currentSSPLag = getLatestLag(ssp); // lag between the current offset and the highwatermark
+
+    Long lag = latestLags.get(ssp);
+    if (lag == null) {
+      throw new SamzaException("Unknown/unregistered ssp in latestLags. ssp=" + ssp + "; system=" + systemName);
+    }
+    long currentSSPLag = lag.longValue(); // lag between the current offset and the highwatermark
     if (currentSSPLag < 0) {
       return;
     }
+
     long recordOffset = r.offset();
     long highWatermark = recordOffset + currentSSPLag; // derived value for the highwatermark
 
@@ -315,7 +353,6 @@ import org.slf4j.LoggerFactory;
     kafkaConsumerMetrics.setHighWatermarkValue(tap, highWatermark);
   }
 
-
   private void moveMessagesToTheirQueue(SystemStreamPartition ssp, List<IncomingMessageEnvelope> envelopes) {
     long nextOffset = nextOffsets.get(ssp);
 
@@ -329,16 +366,6 @@ import org.slf4j.LoggerFactory;
     nextOffsets.put(ssp, nextOffset);
   }
 
-  private void populateMetricNames(Set<SystemStreamPartition> ssps) {
-    HashMap<String, String> tags = new HashMap<>();
-    tags.put("client-id", clientId);// this is required by the KafkaConsumer to get the metrics
-
-    for (SystemStreamPartition ssp : ssps) {
-      TopicPartition tp = KafkaSystemConsumer.toTopicPartition(ssp);
-      perPartitionMetrics.put(ssp, new MetricName(tp + ".records-lag", "consumer-fetch-manager-metrics", "", tags));
-    }
-  }
-
   // The only way to figure out lag for the KafkaConsumer is to look at the metrics after each poll() call.
   // One of the metrics (records-lag) shows how far behind the HighWatermark the consumer is.
   // This method populates the lag information for each SSP into latestLags member variable.
@@ -348,17 +375,23 @@ import org.slf4j.LoggerFactory;
 
     // populate the MetricNames first time
     if (perPartitionMetrics.isEmpty()) {
-      populateMetricNames(ssps);
+      HashMap<String, String> tags = new HashMap<>();
+      tags.put("client-id", clientId); // this is required by the KafkaConsumer to get the metrics
+
+      for (SystemStreamPartition ssp : ssps) {
+        TopicPartition tp = KafkaSystemConsumer.toTopicPartition(ssp);
+        perPartitionMetrics.put(ssp, new MetricName(tp + ".records-lag", "consumer-fetch-manager-metrics", "", tags));
+      }
     }
 
     for (SystemStreamPartition ssp : ssps) {
       MetricName mn = perPartitionMetrics.get(ssp);
-      Metric currentLagM = consumerMetrics.get(mn);
+      Metric currentLagMetric = consumerMetrics.get(mn);
 
       // High watermark is fixed to be the offset of last available message,
       // so the lag is now at least 0, which is the same as Samza's definition.
       // If the lag is not 0, then isAtHead is not true, and kafkaClient keeps polling.
-      long currentLag = (currentLagM != null) ? (long) currentLagM.value() : -1L;
+      long currentLag = (currentLagMetric != null) ? (long) currentLagMetric.value() : -1L;
       latestLags.put(ssp, currentLag);
 
       // calls the setIsAtHead for the BlockingEnvelopeMap
@@ -366,58 +399,7 @@ import org.slf4j.LoggerFactory;
     }
   }
 
-  // Get the latest lag for a specific SSP.
-  private long getLatestLag(SystemStreamPartition ssp) {
-    Long lag = latestLags.get(ssp);
-    if (lag == null) {
-      throw new SamzaException("Unknown/unregistered ssp in latestLags request: " + ssp);
-    }
-    return lag;
-  }
-
-  // Using the consumer to poll the messages from the stream.
-  private void fetchMessages() {
-    Set<SystemStreamPartition> sspsToFetch = new HashSet<>();
-    for (SystemStreamPartition ssp : nextOffsets.keySet()) {
-      if (sink.needsMoreMessages(ssp)) {
-        sspsToFetch.add(ssp);
-      }
-    }
-    LOG.debug("pollConsumer {}", sspsToFetch.size());
-    if (!sspsToFetch.isEmpty()) {
-      kafkaConsumerMetrics.incClientReads(metricName);
-
-      Map<SystemStreamPartition, List<IncomingMessageEnvelope>> response;
-      LOG.debug("pollConsumer from following SSPs: {}; total#={}", sspsToFetch, sspsToFetch.size());
-
-      response = pollConsumer(sspsToFetch, 500L);
-
-      // move the responses into the queue
-      for (Map.Entry<SystemStreamPartition, List<IncomingMessageEnvelope>> e : response.entrySet()) {
-        List<IncomingMessageEnvelope> envelopes = e.getValue();
-        if (envelopes != null) {
-          moveMessagesToTheirQueue(e.getKey(), envelopes);
-        }
-      }
-
-      populateCurrentLags(sspsToFetch); // find current lags for for each SSP
-    } else { // nothing to read
-
-      LOG.debug("No topic/partitions need to be fetched for consumer {} right now. Sleeping {}ms.", kafkaConsumer,
-          SLEEP_MS_WHILE_NO_TOPIC_PARTITION);
-
-      kafkaConsumerMetrics.incClientSkippedFetchRequests(metricName);
-
-      try {
-        Thread.sleep(SLEEP_MS_WHILE_NO_TOPIC_PARTITION);
-      } catch (InterruptedException e) {
-        LOG.warn("Sleep in fetchMessages was interrupted");
-      }
-    }
-    refreshLatencyMetrics();
-  }
-
-  private void refreshLatencyMetrics() {
+  private void refreshLagMetrics() {
     for (Map.Entry<SystemStreamPartition, Long> e : nextOffsets.entrySet()) {
       SystemStreamPartition ssp = e.getKey();
       Long offset = e.getValue();
@@ -433,12 +415,9 @@ import org.slf4j.LoggerFactory;
     }
   }
 
-  boolean isRunning() {
-    return isRunning;
-  }
-
-  Throwable getFailureCause() {
-    return failureCause;
+   @Override
+  public String toString() {
+    return String.format("consumerProxy-%s-%s", systemName, clientId);
   }
 }
 

http://git-wip-us.apache.org/repos/asf/samza/blob/190a3999/samza-kafka/src/main/scala/org/apache/samza/system/kafka/KafkaSystemConsumer.java
----------------------------------------------------------------------
diff --git a/samza-kafka/src/main/scala/org/apache/samza/system/kafka/KafkaSystemConsumer.java b/samza-kafka/src/main/scala/org/apache/samza/system/kafka/KafkaSystemConsumer.java
index 17f29f1..10ce274 100644
--- a/samza-kafka/src/main/scala/org/apache/samza/system/kafka/KafkaSystemConsumer.java
+++ b/samza-kafka/src/main/scala/org/apache/samza/system/kafka/KafkaSystemConsumer.java
@@ -66,10 +66,10 @@ public class KafkaSystemConsumer<K, V> extends BlockingEnvelopeMap implements Sy
   final KafkaConsumerMessageSink messageSink;
 
   // This proxy contains a separate thread, which reads kafka messages (with consumer.poll()) and populates
-  // BlockeingEnvelopMap's buffers.
+  // BlockingEnvelopMap's buffers.
   final private KafkaConsumerProxy proxy;
 
-  // keep registration data until the start - mapping between registered SSPs and topicPartitions, and the offsets
+  // keep registration data until the start - mapping between registered SSPs and topicPartitions, and their offsets
   final Map<TopicPartition, String> topicPartitionsToOffset = new HashMap<>();
   final Map<TopicPartition, SystemStreamPartition> topicPartitionsToSSP = new HashMap<>();
 
@@ -77,10 +77,11 @@ public class KafkaSystemConsumer<K, V> extends BlockingEnvelopeMap implements Sy
   long perPartitionFetchThresholdBytes;
 
   /**
+   * Create a KafkaSystemConsumer for the provided {@code systemName}
    * @param systemName system name for which we create the consumer
-   * @param config config passed into the the app
-   * @param metrics metrics collecting object
-   * @param clock - system clock, allows to override internal clock (System.currentTimeMillis())
+   * @param config application config
+   * @param metrics metrics for this KafkaSystemConsumer
+   * @param clock system clock
    */
   public KafkaSystemConsumer(Consumer<K, V> kafkaConsumer, String systemName, Config config, String clientId,
       KafkaSystemConsumerMetrics metrics, Clock clock) {
@@ -99,11 +100,9 @@ public class KafkaSystemConsumer<K, V> extends BlockingEnvelopeMap implements Sy
     messageSink = new KafkaConsumerMessageSink();
 
     // Create the proxy to do the actual message reading.
-    String metricName = String.format("%s %s", systemName, clientId);
+    String metricName = String.format("%s", systemName);
     proxy = new KafkaConsumerProxy(kafkaConsumer, systemName, clientId, messageSink, metrics, metricName);
     LOG.info("{}: Created KafkaConsumerProxy {} ", this, proxy);
-
-    LOG.info("{}: Created KafkaSystemConsumer {}", this, kafkaConsumer);
   }
 
   /**
@@ -118,7 +117,7 @@ public class KafkaSystemConsumer<K, V> extends BlockingEnvelopeMap implements Sy
     // extract kafka client configs
     KafkaConsumerConfig consumerConfig = KafkaConsumerConfig.getKafkaSystemConsumerConfig(config, systemName, clientId);
 
-    LOG.info("{}:{} KafkaClient properties {}", systemName, clientId, consumerConfig);
+    LOG.info("{}: KafkaClient properties {}", systemName, consumerConfig);
 
     return new KafkaConsumer(consumerConfig);
   }
@@ -130,7 +129,7 @@ public class KafkaSystemConsumer<K, V> extends BlockingEnvelopeMap implements Sy
       return;
     }
     if (stopped.get()) {
-      LOG.warn("{}: Attempting to start a stopped consumer", this);
+      LOG.error("{}: Attempting to start a stopped consumer", this);
       return;
     }
     // initialize the subscriptions for all the registered TopicPartitions
@@ -151,8 +150,7 @@ public class KafkaSystemConsumer<K, V> extends BlockingEnvelopeMap implements Sy
         kafkaConsumer.assign(topicPartitionsToSSP.keySet());
       }
     } catch (Exception e) {
-      LOG.warn("{}: Start subscription failed", this);
-      throw new SamzaException(e);
+      throw new SamzaException("Consumer subscription failed for " + this, e);
     }
   }
 
@@ -164,7 +162,7 @@ public class KafkaSystemConsumer<K, V> extends BlockingEnvelopeMap implements Sy
   void startConsumer() {
     // set the offset for each TopicPartition
     if (topicPartitionsToOffset.size() <= 0) {
-      LOG.warn("{}: Consumer is not subscribed to any SSPs", this);
+      LOG.error ("{}: Consumer is not subscribed to any SSPs", this);
     }
 
     topicPartitionsToOffset.forEach((tp, startingOffsetString) -> {
@@ -204,35 +202,30 @@ public class KafkaSystemConsumer<K, V> extends BlockingEnvelopeMap implements Sy
     long fetchThreshold = FETCH_THRESHOLD;
     if (fetchThresholdOption.isDefined()) {
       fetchThreshold = Long.valueOf(fetchThresholdOption.get());
-      LOG.info("{}: fetchThresholdOption is configured. fetchThreshold={}", this, fetchThreshold);
     }
 
     Option<String> fetchThresholdBytesOption = kafkaConfig.getConsumerFetchThresholdBytes(systemName);
     long fetchThresholdBytes = FETCH_THRESHOLD_BYTES;
     if (fetchThresholdBytesOption.isDefined()) {
       fetchThresholdBytes = Long.valueOf(fetchThresholdBytesOption.get());
-      LOG.info("{}: fetchThresholdBytesOption is configured. fetchThresholdBytes={}", this, fetchThresholdBytes);
     }
 
-    int numTPs = topicPartitionsToSSP.size();
-    if (numTPs != topicPartitionsToOffset.size()) {
+    int numPartitions = topicPartitionsToSSP.size();
+    if (numPartitions != topicPartitionsToOffset.size()) {
       throw new SamzaException("topicPartitionsToSSP.size() doesn't match topicPartitionsToOffset.size()");
     }
 
-    LOG.info("{}: fetchThresholdBytes = {}; fetchThreshold={}; partitions num={}", this, fetchThresholdBytes,
-        fetchThreshold, numTPs);
 
-    if (numTPs > 0) {
-      perPartitionFetchThreshold = fetchThreshold / numTPs;
-      LOG.info("{}: perPartitionFetchThreshold={}", this, perPartitionFetchThreshold);
+    if (numPartitions > 0) {
+      perPartitionFetchThreshold = fetchThreshold / numPartitions;
       if (fetchThresholdBytesEnabled) {
         // currently this feature cannot be enabled, because we do not have the size of the messages available.
         // messages get double buffered, hence divide by 2
-        perPartitionFetchThresholdBytes = (fetchThresholdBytes / 2) / numTPs;
-        LOG.info("{} :perPartitionFetchThresholdBytes is enabled. perPartitionFetchThresholdBytes={}", this,
-            perPartitionFetchThresholdBytes);
+        perPartitionFetchThresholdBytes = (fetchThresholdBytes / 2) / numPartitions;
       }
     }
+    LOG.info("{}: fetchThresholdBytes = {}; fetchThreshold={}; numPartitions={}, perPartitionFetchThreshold={}, perPartitionFetchThresholdBytes(0 if disabled)={}",
+        this, fetchThresholdBytes, fetchThreshold, numPartitions, perPartitionFetchThreshold, perPartitionFetchThresholdBytes);
   }
 
   @Override
@@ -260,8 +253,10 @@ public class KafkaSystemConsumer<K, V> extends BlockingEnvelopeMap implements Sy
     }
   }
 
-  /*
-   record the ssp and the offset. Do not submit it to the consumer yet.
+  /**
+   * record the ssp and the offset. Do not submit it to the consumer yet.
+   * @param systemStreamPartition ssp to register
+   * @param offset offset to register with
    */
   @Override
   public void register(SystemStreamPartition systemStreamPartition, String offset) {

http://git-wip-us.apache.org/repos/asf/samza/blob/190a3999/samza-kafka/src/main/scala/org/apache/samza/system/kafka/KafkaSystemConsumerMetrics.scala
----------------------------------------------------------------------
diff --git a/samza-kafka/src/main/scala/org/apache/samza/system/kafka/KafkaSystemConsumerMetrics.scala b/samza-kafka/src/main/scala/org/apache/samza/system/kafka/KafkaSystemConsumerMetrics.scala
index c4552e6..59a8854 100644
--- a/samza-kafka/src/main/scala/org/apache/samza/system/kafka/KafkaSystemConsumerMetrics.scala
+++ b/samza-kafka/src/main/scala/org/apache/samza/system/kafka/KafkaSystemConsumerMetrics.scala
@@ -55,7 +55,7 @@ class KafkaSystemConsumerMetrics(val systemName: String = "unknown", val registr
 
   // java friendlier interfaces
   // Gauges
-  def setTopicPartitionValue(clientName: String, value: Int) {
+  def setNumTopicPartitions(clientName: String, value: Int) {
     topicPartitions.get(clientName).set(value)
   }
 

http://git-wip-us.apache.org/repos/asf/samza/blob/190a3999/samza-kafka/src/test/java/org/apache/samza/config/TestKafkaConsumerConfig.java
----------------------------------------------------------------------
diff --git a/samza-kafka/src/test/java/org/apache/samza/config/TestKafkaConsumerConfig.java b/samza-kafka/src/test/java/org/apache/samza/config/TestKafkaConsumerConfig.java
index 35a717a..de5d093 100644
--- a/samza-kafka/src/test/java/org/apache/samza/config/TestKafkaConsumerConfig.java
+++ b/samza-kafka/src/test/java/org/apache/samza/config/TestKafkaConsumerConfig.java
@@ -28,7 +28,7 @@ import org.junit.Test;
 
 
 public class TestKafkaConsumerConfig {
-  private final Map<String, String> props = new HashMap<>();
+
   public final static String SYSTEM_NAME = "testSystem";
   public final static String KAFKA_PRODUCER_PROPERTY_PREFIX = "systems." + SYSTEM_NAME + ".producer.";
   public final static String KAFKA_CONSUMER_PROPERTY_PREFIX = "systems." + SYSTEM_NAME + ".consumer.";
@@ -36,6 +36,7 @@ public class TestKafkaConsumerConfig {
 
   @Test
   public void testDefaults() {
+    Map<String, String> props = new HashMap<>();
 
     props.put(KAFKA_CONSUMER_PROPERTY_PREFIX + ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "true"); // should be ignored
     props.put(KAFKA_CONSUMER_PROPERTY_PREFIX + ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG,
@@ -43,6 +44,8 @@ public class TestKafkaConsumerConfig {
     props.put(KAFKA_CONSUMER_PROPERTY_PREFIX + ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG,
         "100"); // should NOT be ignored
 
+    props.put(JobConfig.JOB_NAME(), "jobName");
+
     // if KAFKA_CONSUMER_PROPERTY_PREFIX is set, then PRODUCER should be ignored
     props.put(KAFKA_PRODUCER_PROPERTY_PREFIX + "bootstrap.servers", "ignroeThis:9092");
     props.put(KAFKA_CONSUMER_PROPERTY_PREFIX + "bootstrap.servers", "useThis:9092");
@@ -72,11 +75,23 @@ public class TestKafkaConsumerConfig {
 
     Assert.assertEquals(KafkaConsumerConfig.getConsumerGroupId(config),
         kafkaConsumerConfig.get(ConsumerConfig.GROUP_ID_CONFIG));
+
+    Assert.assertEquals(KafkaConsumerConfig.CONSUMER_CLIENT_ID_PREFIX.replace("-", "_") + "-jobName-1",
+        KafkaConsumerConfig.getConsumerClientId(config));
+    Assert.assertEquals("jobName-1", KafkaConsumerConfig.getConsumerGroupId(config));
+
+    props.put(JobConfig.JOB_ID(), "jobId");
+    config = new MapConfig(props);
+
+    Assert.assertEquals(KafkaConsumerConfig.CONSUMER_CLIENT_ID_PREFIX.replace("-", "_") + "-jobName-jobId",
+        KafkaConsumerConfig.getConsumerClientId(config));
+    Assert.assertEquals("jobName-jobId", KafkaConsumerConfig.getConsumerGroupId(config));
   }
 
   // test stuff that should not be overridden
   @Test
   public void testNotOverride() {
+    Map<String, String> props = new HashMap<>();
 
     // if KAFKA_CONSUMER_PROPERTY_PREFIX is not set, then PRODUCER should be used
     props.put(KAFKA_PRODUCER_PROPERTY_PREFIX + ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "useThis:9092");
@@ -85,6 +100,8 @@ public class TestKafkaConsumerConfig {
     props.put(KAFKA_CONSUMER_PROPERTY_PREFIX + ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG,
         TestKafkaConsumerConfig.class.getName());
 
+    props.put(JobConfig.JOB_NAME(), "jobName");
+
     Config config = new MapConfig(props);
     KafkaConsumerConfig kafkaConsumerConfig =
         KafkaConsumerConfig.getKafkaSystemConsumerConfig(config, SYSTEM_NAME, CLIENT_ID);


[03/29] samza git commit: SAMZA-1859: Zookeeper implementation of MetadataStore.

Posted by bo...@apache.org.
SAMZA-1859: Zookeeper implementation of MetadataStore.

Author: Shanthoosh Venkataraman <sp...@usc.edu>

Reviewers: Prateek Maheshwari <pm...@apache.org>, Daniel Nishumura <dn...@linkedin.com>

Closes #629 from shanthoosh/metadata_store_zk_impl


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

Branch: refs/heads/NewKafkaSystemConsumer
Commit: 160927adad9843cf5110b341cb0b67413f75249d
Parents: 19c6f4f
Author: Shanthoosh Venkataraman <sp...@usc.edu>
Authored: Wed Sep 19 12:16:15 2018 -0700
Committer: Prateek Maheshwari <pm...@apache.org>
Committed: Wed Sep 19 12:16:15 2018 -0700

----------------------------------------------------------------------
 .../samza/metadatastore/MetadataStore.java      |   6 +-
 .../apache/samza/container/LocalityManager.java |   2 +-
 .../grouper/task/TaskAssignmentManager.java     |   2 +-
 .../metadatastore/CoordinatorStreamStore.java   |   2 +-
 .../java/org/apache/samza/zk/ProcessorData.java |  19 +--
 .../samza/zk/ZkJobCoordinatorFactory.java       |  17 +--
 .../java/org/apache/samza/zk/ZkKeyBuilder.java  |  30 +++--
 .../org/apache/samza/zk/ZkMetadataStore.java    | 132 +++++++++++++++++++
 .../apache/samza/zk/ZkMetadataStoreFactory.java |  36 +++++
 .../apache/samza/container/SamzaContainer.scala |  26 ++--
 .../TestCoordinatorStreamStore.java             |   2 +-
 .../org/apache/samza/zk/TestZkKeyBuilder.java   |   2 +-
 .../apache/samza/zk/TestZkMetadataStore.java    | 121 +++++++++++++++++
 13 files changed, 340 insertions(+), 57 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/samza/blob/160927ad/samza-api/src/main/java/org/apache/samza/metadatastore/MetadataStore.java
----------------------------------------------------------------------
diff --git a/samza-api/src/main/java/org/apache/samza/metadatastore/MetadataStore.java b/samza-api/src/main/java/org/apache/samza/metadatastore/MetadataStore.java
index aaa420b..cd04794 100644
--- a/samza-api/src/main/java/org/apache/samza/metadatastore/MetadataStore.java
+++ b/samza-api/src/main/java/org/apache/samza/metadatastore/MetadataStore.java
@@ -19,8 +19,6 @@
 package org.apache.samza.metadatastore;
 
 import org.apache.samza.annotation.InterfaceStability;
-import org.apache.samza.config.Config;
-import org.apache.samza.metrics.MetricsRegistry;
 import java.util.Map;
 
 /**
@@ -32,10 +30,8 @@ public interface MetadataStore {
   /**
    * Initializes the metadata store, if applicable, setting up the underlying resources
    * and connections to the store endpoints.
-   *
-   * @param config the configuration for instantiating the MetadataStore.
    */
-  void init(Config config, MetricsRegistry metricsRegistry);
+  void init();
 
   /**
    * Gets the value associated with the specified {@code key}.

http://git-wip-us.apache.org/repos/asf/samza/blob/160927ad/samza-core/src/main/java/org/apache/samza/container/LocalityManager.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/container/LocalityManager.java b/samza-core/src/main/java/org/apache/samza/container/LocalityManager.java
index c70b15a..20e86d9 100644
--- a/samza-core/src/main/java/org/apache/samza/container/LocalityManager.java
+++ b/samza-core/src/main/java/org/apache/samza/container/LocalityManager.java
@@ -78,7 +78,7 @@ public class LocalityManager {
     this.config = config;
     MetadataStoreFactory metadataStoreFactory = Util.getObj(new JobConfig(config).getMetadataStoreFactory(), MetadataStoreFactory.class);
     this.metadataStore = metadataStoreFactory.getMetadataStore(SetContainerHostMapping.TYPE, config, metricsRegistry);
-    this.metadataStore.init(config, metricsRegistry);
+    this.metadataStore.init();
     this.keySerde = keySerde;
     this.valueSerde = valueSerde;
     this.taskAssignmentManager = new TaskAssignmentManager(config, metricsRegistry, keySerde, valueSerde);

http://git-wip-us.apache.org/repos/asf/samza/blob/160927ad/samza-core/src/main/java/org/apache/samza/container/grouper/task/TaskAssignmentManager.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/container/grouper/task/TaskAssignmentManager.java b/samza-core/src/main/java/org/apache/samza/container/grouper/task/TaskAssignmentManager.java
index 42a6e81..2bfd4c3 100644
--- a/samza-core/src/main/java/org/apache/samza/container/grouper/task/TaskAssignmentManager.java
+++ b/samza-core/src/main/java/org/apache/samza/container/grouper/task/TaskAssignmentManager.java
@@ -84,7 +84,7 @@ public class TaskAssignmentManager {
   }
 
   public void init(Config config, MetricsRegistry metricsRegistry) {
-    this.metadataStore.init(config, metricsRegistry);
+    this.metadataStore.init();
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/samza/blob/160927ad/samza-core/src/main/java/org/apache/samza/coordinator/metadatastore/CoordinatorStreamStore.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/coordinator/metadatastore/CoordinatorStreamStore.java b/samza-core/src/main/java/org/apache/samza/coordinator/metadatastore/CoordinatorStreamStore.java
index d74188b..af5e2f9 100644
--- a/samza-core/src/main/java/org/apache/samza/coordinator/metadatastore/CoordinatorStreamStore.java
+++ b/samza-core/src/main/java/org/apache/samza/coordinator/metadatastore/CoordinatorStreamStore.java
@@ -89,7 +89,7 @@ public class CoordinatorStreamStore implements MetadataStore {
   }
 
   @Override
-  public void init(Config config, MetricsRegistry metricsRegistry) {
+  public void init() {
     if (isInitialized.compareAndSet(false, true)) {
       LOG.info("Starting the coordinator stream system consumer with config: {}.", config);
       registerConsumer();

http://git-wip-us.apache.org/repos/asf/samza/blob/160927ad/samza-core/src/main/java/org/apache/samza/zk/ProcessorData.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/zk/ProcessorData.java b/samza-core/src/main/java/org/apache/samza/zk/ProcessorData.java
index a48a450..91ba33d 100644
--- a/samza-core/src/main/java/org/apache/samza/zk/ProcessorData.java
+++ b/samza-core/src/main/java/org/apache/samza/zk/ProcessorData.java
@@ -21,17 +21,18 @@ package org.apache.samza.zk;
 
 import java.util.Objects;
 import org.apache.samza.SamzaException;
+import org.apache.samza.runtime.LocationId;
 
 /**
  * Represents processor data stored in zookeeper processors node.
  */
 public class ProcessorData {
   private final String processorId;
-  private final String host;
+  private final String locationId;
 
-  public ProcessorData(String host, String processorId) {
+  public ProcessorData(String locationId, String processorId) {
     this.processorId = processorId;
-    this.host = host;
+    this.locationId = locationId;
   }
 
   public ProcessorData(String data) {
@@ -39,16 +40,16 @@ public class ProcessorData {
     if (splt.length != 2) {
       throw new SamzaException("incorrect processor data format = " + data);
     }
-    host = splt[0];
+    locationId = splt[0];
     processorId = splt[1];
   }
 
   public String toString() {
-    return host + " " + processorId;
+    return locationId + " " + processorId;
   }
 
-  public String getHost() {
-    return host;
+  public LocationId getLocationId() {
+    return new LocationId(locationId);
   }
 
   public String getProcessorId() {
@@ -57,7 +58,7 @@ public class ProcessorData {
 
   @Override
   public int hashCode() {
-    return Objects.hash(processorId, host);
+    return Objects.hash(processorId, locationId);
   }
 
   @Override
@@ -65,6 +66,6 @@ public class ProcessorData {
     if (obj == null) return false;
     if (getClass() != obj.getClass()) return false;
     final ProcessorData other = (ProcessorData) obj;
-    return Objects.equals(processorId, other.processorId) && Objects.equals(host, other.host);
+    return Objects.equals(processorId, other.processorId) && Objects.equals(locationId, other.locationId);
   }
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/160927ad/samza-core/src/main/java/org/apache/samza/zk/ZkJobCoordinatorFactory.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/zk/ZkJobCoordinatorFactory.java b/samza-core/src/main/java/org/apache/samza/zk/ZkJobCoordinatorFactory.java
index 6888df0..3dad6c1 100644
--- a/samza-core/src/main/java/org/apache/samza/zk/ZkJobCoordinatorFactory.java
+++ b/samza-core/src/main/java/org/apache/samza/zk/ZkJobCoordinatorFactory.java
@@ -31,7 +31,6 @@ import org.apache.samza.metrics.MetricsRegistryMap;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-
 public class ZkJobCoordinatorFactory implements JobCoordinatorFactory {
 
   private static final Logger LOG = LoggerFactory.getLogger(ZkJobCoordinatorFactory.class);
@@ -40,22 +39,24 @@ public class ZkJobCoordinatorFactory implements JobCoordinatorFactory {
   private static final String DEFAULT_JOB_NAME = "defaultJob";
 
   /**
-   * Method to instantiate an implementation of JobCoordinator
+   * Instantiates an {@link ZkJobCoordinator} using the {@link Config}.
    *
-   * @param config - configs relevant for the JobCoordinator TODO: Separate JC related configs into a "JobCoordinatorConfig"
-   * @return An instance of IJobCoordinator
+   * @param config zookeeper configurations required for instantiating {@link ZkJobCoordinator}
+   * @return An instance of {@link ZkJobCoordinator}
    */
   @Override
   public JobCoordinator getJobCoordinator(Config config) {
+    // TODO: Separate JC related configs into a "ZkJobCoordinatorConfig"
     MetricsRegistry metricsRegistry = new MetricsRegistryMap();
-    ZkUtils zkUtils = getZkUtils(config, metricsRegistry);
-    LOG.debug("Creating ZkJobCoordinator instance with config: {}.", config);
+    String jobCoordinatorZkBasePath = getJobCoordinationZkPath(config);
+    ZkUtils zkUtils = getZkUtils(config, metricsRegistry, jobCoordinatorZkBasePath);
+    LOG.debug("Creating ZkJobCoordinator with config: {}.", config);
     return new ZkJobCoordinator(config, metricsRegistry, zkUtils);
   }
 
-  private ZkUtils getZkUtils(Config config, MetricsRegistry metricsRegistry) {
+  private ZkUtils getZkUtils(Config config, MetricsRegistry metricsRegistry, String coordinatorZkBasePath) {
     ZkConfig zkConfig = new ZkConfig(config);
-    ZkKeyBuilder keyBuilder = new ZkKeyBuilder(getJobCoordinationZkPath(config));
+    ZkKeyBuilder keyBuilder = new ZkKeyBuilder(coordinatorZkBasePath);
     ZkClient zkClient = ZkCoordinationUtilsFactory
         .createZkClient(zkConfig.getZkConnect(), zkConfig.getZkSessionTimeoutMs(), zkConfig.getZkConnectionTimeoutMs());
     return new ZkUtils(keyBuilder, zkClient, zkConfig.getZkConnectionTimeoutMs(), zkConfig.getZkSessionTimeoutMs(), metricsRegistry);

http://git-wip-us.apache.org/repos/asf/samza/blob/160927ad/samza-core/src/main/java/org/apache/samza/zk/ZkKeyBuilder.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/zk/ZkKeyBuilder.java b/samza-core/src/main/java/org/apache/samza/zk/ZkKeyBuilder.java
index 37bff6d..16efe81 100644
--- a/samza-core/src/main/java/org/apache/samza/zk/ZkKeyBuilder.java
+++ b/samza-core/src/main/java/org/apache/samza/zk/ZkKeyBuilder.java
@@ -42,16 +42,18 @@ import com.google.common.base.Strings;
  * This class provides helper methods to easily generate/parse the path in the ZK hierarchy.
  */
 public class ZkKeyBuilder {
+
+  static final String PROCESSORS_PATH = "processors";
+  static final String JOBMODEL_GENERATION_PATH = "jobModelGeneration";
+  static final String JOB_MODEL_UPGRADE_BARRIER_PATH = "jobModelUpgradeBarrier";
+  private static final String TASK_LOCALITY_PATH = "taskLocality";
+
   /**
    * Prefix generated to uniquely identify a particular deployment of a job.
    * TODO: For now, it looks like $jobName-$jobId. We need to add a unique deployment/attempt identifier as well.
    */
   private final String pathPrefix;
 
-  static final String PROCESSORS_PATH = "processors";
-  static final String JOBMODEL_GENERATION_PATH = "JobModelGeneration";
-  static final String JOB_MODEL_UPGRADE_BARRIER = "jobModelUpgradeBarrier";
-
   public ZkKeyBuilder(String pathPrefix) {
     if (pathPrefix != null && !pathPrefix.trim().isEmpty()) {
       this.pathPrefix = pathPrefix.trim();
@@ -60,11 +62,11 @@ public class ZkKeyBuilder {
     }
   }
 
-  public String getRootPath() {
+  String getRootPath() {
     return "/" + pathPrefix;
   }
 
-  public String getProcessorsPath() {
+  String getProcessorsPath() {
     return String.format("/%s/%s", pathPrefix, PROCESSORS_PATH);
   }
 
@@ -77,25 +79,29 @@ public class ZkKeyBuilder {
    * @param path Full ZK path of a registered processor
    * @return String representing the processor ID
    */
-  public static String parseIdFromPath(String path) {
+  static String parseIdFromPath(String path) {
     if (!Strings.isNullOrEmpty(path))
       return path.substring(path.lastIndexOf("/") + 1);
     return null;
   }
 
-  public String getJobModelVersionPath() {
+  String getJobModelVersionPath() {
     return String.format("%s/%s/jobModelVersion", getRootPath(), JOBMODEL_GENERATION_PATH);
   }
 
-  public String getJobModelPathPrefix() {
+  String getJobModelPathPrefix() {
     return String.format("%s/%s/jobModels", getRootPath(), JOBMODEL_GENERATION_PATH);
   }
 
-  public String getJobModelPath(String jobModelVersion) {
+  String getJobModelPath(String jobModelVersion) {
     return String.format("%s/%s", getJobModelPathPrefix(), jobModelVersion);
   }
 
-  public String getJobModelVersionBarrierPrefix() {
-    return String.format("%s/%s/%s/versionBarriers", getRootPath(), JOBMODEL_GENERATION_PATH, JOB_MODEL_UPGRADE_BARRIER);
+  String getJobModelVersionBarrierPrefix() {
+    return String.format("%s/%s/%s/versionBarriers", getRootPath(), JOBMODEL_GENERATION_PATH, JOB_MODEL_UPGRADE_BARRIER_PATH);
+  }
+
+  String getTaskLocalityPath() {
+    return String.format("%s/%s", getRootPath(), TASK_LOCALITY_PATH);
   }
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/160927ad/samza-core/src/main/java/org/apache/samza/zk/ZkMetadataStore.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/zk/ZkMetadataStore.java b/samza-core/src/main/java/org/apache/samza/zk/ZkMetadataStore.java
new file mode 100644
index 0000000..4cfdc8d
--- /dev/null
+++ b/samza-core/src/main/java/org/apache/samza/zk/ZkMetadataStore.java
@@ -0,0 +1,132 @@
+/*
+ * 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.zk;
+
+import java.nio.charset.Charset;
+import java.util.concurrent.TimeUnit;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import org.I0Itec.zkclient.serialize.BytesPushThroughSerializer;
+import org.apache.samza.config.Config;
+import org.apache.samza.config.ZkConfig;
+import org.apache.samza.metrics.MetricsRegistry;
+import org.apache.samza.metadatastore.MetadataStore;
+import org.apache.samza.SamzaException;
+import org.I0Itec.zkclient.ZkClient;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * An implementation of the {@link MetadataStore} interface where the
+ * metadata of the Samza job is stored in zookeeper.
+ */
+public class ZkMetadataStore implements MetadataStore {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ZkMetadataStore.class);
+
+  private final ZkClient zkClient;
+  private final ZkConfig zkConfig;
+  private final String zkBaseDir;
+
+  public ZkMetadataStore(String zkBaseDir, Config config, MetricsRegistry metricsRegistry) {
+    this.zkConfig = new ZkConfig(config);
+    this.zkClient = new ZkClient(zkConfig.getZkConnect(), zkConfig.getZkSessionTimeoutMs(), zkConfig.getZkConnectionTimeoutMs(), new BytesPushThroughSerializer());
+    this.zkBaseDir = zkBaseDir;
+    zkClient.createPersistent(zkBaseDir, true);
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public void init() {
+    zkClient.waitUntilConnected(zkConfig.getZkConnectionTimeoutMs(), TimeUnit.MILLISECONDS);
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public byte[] get(byte[] key) {
+    return zkClient.readData(getZkPathForKey(key), true);
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public void put(byte[] key, byte[] value) {
+    String zkPath = getZkPathForKey(key);
+    zkClient.createPersistent(zkPath, true);
+    zkClient.writeData(zkPath, value);
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public void delete(byte[] key) {
+    zkClient.delete(getZkPathForKey(key));
+  }
+
+  /**
+   * {@inheritDoc}
+   * @throws SamzaException if there're exceptions reading data from zookeeper.
+   */
+  @Override
+  public Map<byte[], byte[]> all() {
+    try {
+      List<String> zkSubDirectories = zkClient.getChildren(zkBaseDir);
+      Map<byte[], byte[]> result = new HashMap<>();
+      for (String zkSubDir : zkSubDirectories) {
+        String completeZkPath = String.format("%s/%s", zkBaseDir, zkSubDir);
+        byte[] value = zkClient.readData(completeZkPath, true);
+        if (value != null) {
+          result.put(completeZkPath.getBytes("UTF-8"), value);
+        }
+      }
+      return result;
+    } catch (Exception e) {
+      String errorMsg = String.format("Error reading path: %s from zookeeper.", zkBaseDir);
+      LOG.error(errorMsg, e);
+      throw new SamzaException(errorMsg, e);
+    }
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public void flush() {
+    // No-op for zookeeper implementation.
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public void close() {
+    zkClient.close();
+  }
+
+  private String getZkPathForKey(byte[] key) {
+    return String.format("%s/%s", zkBaseDir, new String(key, Charset.forName("UTF-8")));
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/160927ad/samza-core/src/main/java/org/apache/samza/zk/ZkMetadataStoreFactory.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/zk/ZkMetadataStoreFactory.java b/samza-core/src/main/java/org/apache/samza/zk/ZkMetadataStoreFactory.java
new file mode 100644
index 0000000..a9c979d
--- /dev/null
+++ b/samza-core/src/main/java/org/apache/samza/zk/ZkMetadataStoreFactory.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.zk;
+
+import org.apache.samza.config.Config;
+import org.apache.samza.metadatastore.MetadataStore;
+import org.apache.samza.metadatastore.MetadataStoreFactory;
+import org.apache.samza.metrics.MetricsRegistry;
+
+/**
+ * Builds the {@link ZkMetadataStore} based upon the provided {@link Config}
+ * and {@link MetricsRegistry}.
+ */
+public class ZkMetadataStoreFactory implements MetadataStoreFactory {
+
+  @Override
+  public MetadataStore getMetadataStore(String namespace, Config config, MetricsRegistry metricsRegistry) {
+    return new ZkMetadataStore(namespace, config, metricsRegistry);
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/160927ad/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
index 68de4a6..7b64f5e 100644
--- a/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala
+++ b/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala
@@ -129,12 +129,6 @@ object SamzaContainer extends Logging {
     val containerName = "samza-container-%s" format containerId
     val maxChangeLogStreamPartitions = jobModel.maxChangeLogStreamPartitions
 
-    var localityManager: LocalityManager = null
-    if (new ClusterManagerConfig(config).getHostAffinityEnabled()) {
-      val registryMap = new MetricsRegistryMap(containerName)
-      localityManager = new LocalityManager(config, registryMap)
-    }
-
     val containerPID = ManagementFactory.getRuntimeMXBean().getName()
 
     info("Setting up Samza container: %s" format containerName)
@@ -719,7 +713,6 @@ object SamzaContainer extends Logging {
       consumerMultiplexer = consumerMultiplexer,
       producerMultiplexer = producerMultiplexer,
       offsetManager = offsetManager,
-      localityManager = localityManager,
       securityManager = securityManager,
       metrics = samzaContainerMetrics,
       reporters = reporters,
@@ -799,7 +792,7 @@ class SamzaContainer(
       startDiagnostics
       startAdmins
       startOffsetManager
-      startLocalityManager
+      storeContainerLocality
       startStores
       startTableManager
       startDiskSpaceMonitor
@@ -841,7 +834,6 @@ class SamzaContainer(
       shutdownDiskSpaceMonitor
       shutdownHostStatisticsMonitor
       shutdownProducers
-      shutdownLocalityManager
       shutdownOffsetManager
       shutdownMetrics
       shutdownSecurityManger
@@ -961,8 +953,10 @@ class SamzaContainer(
     offsetManager.start
   }
 
-  def startLocalityManager {
-    if(localityManager != null) {
+  def storeContainerLocality {
+    val isHostAffinityEnabled: Boolean = new ClusterManagerConfig(containerContext.config).getHostAffinityEnabled
+    if (isHostAffinityEnabled) {
+      val localityManager: LocalityManager = new LocalityManager(containerContext.config, containerContext.metricsRegistry)
       val containerName = "SamzaContainer-" + String.valueOf(containerContext.id)
       info("Registering %s with metadata store" format containerName)
       try {
@@ -978,6 +972,9 @@ class SamzaContainer(
         case unknownException: Throwable =>
           warn("Received an exception when persisting locality info for container %s: " +
             "%s" format (containerContext.id, unknownException.getMessage))
+      } finally {
+        info("Shutting down locality manager.")
+        localityManager.close()
       }
     }
   }
@@ -1145,13 +1142,6 @@ class SamzaContainer(
     taskInstances.values.foreach(_.shutdownTableManager)
   }
 
-  def shutdownLocalityManager {
-    if(localityManager != null) {
-      info("Shutting down locality manager.")
-      localityManager.close()
-    }
-  }
-
   def shutdownOffsetManager {
     info("Shutting down offset manager.")
 

http://git-wip-us.apache.org/repos/asf/samza/blob/160927ad/samza-core/src/test/java/org/apache/samza/coordinator/metadatastore/TestCoordinatorStreamStore.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/coordinator/metadatastore/TestCoordinatorStreamStore.java b/samza-core/src/test/java/org/apache/samza/coordinator/metadatastore/TestCoordinatorStreamStore.java
index da2d984..0e48363 100644
--- a/samza-core/src/test/java/org/apache/samza/coordinator/metadatastore/TestCoordinatorStreamStore.java
+++ b/samza-core/src/test/java/org/apache/samza/coordinator/metadatastore/TestCoordinatorStreamStore.java
@@ -57,7 +57,7 @@ public class TestCoordinatorStreamStore {
     when(CoordinatorStreamUtil.getCoordinatorSystemStream(anyObject())).thenReturn(new SystemStream("test-kafka", "test"));
     when(CoordinatorStreamUtil.getCoordinatorStreamName(anyObject(), anyObject())).thenReturn("test");
     coordinatorStreamStore = new CoordinatorStreamStore(SetTaskContainerMapping.TYPE, new MapConfig(configMap), new MetricsRegistryMap());
-    coordinatorStreamStore.init(new MapConfig(), new MetricsRegistryMap());
+    coordinatorStreamStore.init();
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/samza/blob/160927ad/samza-core/src/test/java/org/apache/samza/zk/TestZkKeyBuilder.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/zk/TestZkKeyBuilder.java b/samza-core/src/test/java/org/apache/samza/zk/TestZkKeyBuilder.java
index 8ddd688..d2175b2 100644
--- a/samza-core/src/test/java/org/apache/samza/zk/TestZkKeyBuilder.java
+++ b/samza-core/src/test/java/org/apache/samza/zk/TestZkKeyBuilder.java
@@ -56,6 +56,6 @@ public class TestZkKeyBuilder {
     Assert.assertEquals("/test/" + ZkKeyBuilder.JOBMODEL_GENERATION_PATH + "/jobModels", builder.getJobModelPathPrefix());
     String version = "2";
     Assert.assertEquals("/test/" + ZkKeyBuilder.JOBMODEL_GENERATION_PATH + "/jobModels/" + version, builder.getJobModelPath(version));
-    Assert.assertEquals("/test/" + ZkKeyBuilder.JOBMODEL_GENERATION_PATH + "/" + ZkKeyBuilder.JOB_MODEL_UPGRADE_BARRIER + "/versionBarriers", builder.getJobModelVersionBarrierPrefix());
+    Assert.assertEquals("/test/" + ZkKeyBuilder.JOBMODEL_GENERATION_PATH + "/" + ZkKeyBuilder.JOB_MODEL_UPGRADE_BARRIER_PATH + "/versionBarriers", builder.getJobModelVersionBarrierPrefix());
   }
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/160927ad/samza-core/src/test/java/org/apache/samza/zk/TestZkMetadataStore.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/zk/TestZkMetadataStore.java b/samza-core/src/test/java/org/apache/samza/zk/TestZkMetadataStore.java
new file mode 100644
index 0000000..5930c65
--- /dev/null
+++ b/samza-core/src/test/java/org/apache/samza/zk/TestZkMetadataStore.java
@@ -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.zk;
+
+import com.google.common.collect.ImmutableMap;
+import java.util.Arrays;
+import org.apache.commons.lang3.RandomStringUtils;
+import org.apache.samza.config.Config;
+import org.apache.samza.config.MapConfig;
+import org.apache.samza.config.ZkConfig;
+import org.apache.samza.metadatastore.MetadataStore;
+import org.apache.samza.metrics.MetricsRegistryMap;
+import org.apache.samza.testUtils.EmbeddedZookeeper;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestZkMetadataStore {
+
+  private static final String LOCALHOST = "127.0.0.1";
+
+  private static EmbeddedZookeeper zkServer;
+
+  private MetadataStore zkMetadataStore;
+
+  @BeforeClass
+  public static void beforeClass() {
+    zkServer = new EmbeddedZookeeper();
+    zkServer.setup();
+  }
+
+  @AfterClass
+  public static void afterClass() {
+    zkServer.teardown();
+  }
+
+  @Before
+  public void beforeTest() {
+    String testZkConnectionString = String.format("%s:%s", LOCALHOST, zkServer.getPort());
+    Config zkConfig = new MapConfig(ImmutableMap.of(ZkConfig.ZK_CONNECT, testZkConnectionString));
+    zkMetadataStore = new ZkMetadataStoreFactory().getMetadataStore(String.format("/%s", RandomStringUtils.randomAlphabetic(5)), zkConfig, new MetricsRegistryMap());
+  }
+
+  @After
+  public void afterTest() {
+    zkMetadataStore.close();
+  }
+
+  @Test
+  public void testReadAfterWrite() throws Exception {
+    byte[] key = "test-key1".getBytes("UTF-8");
+    byte[] value = "test-value1".getBytes("UTF-8");
+    Assert.assertNull(zkMetadataStore.get(key));
+    zkMetadataStore.put(key, value);
+    Assert.assertTrue(Arrays.equals(value, zkMetadataStore.get(key)));
+    Assert.assertEquals(1, zkMetadataStore.all().size());
+  }
+
+  @Test
+  public void testReadAfterDelete() throws Exception {
+    byte[] key = "test-key1".getBytes("UTF-8");
+    byte[] value = "test-value1".getBytes("UTF-8");
+    Assert.assertNull(zkMetadataStore.get(key));
+    zkMetadataStore.put(key, value);
+    Assert.assertTrue(Arrays.equals(value, zkMetadataStore.get(key)));
+    zkMetadataStore.delete(key);
+    Assert.assertNull(zkMetadataStore.get(key));
+    Assert.assertEquals(0, zkMetadataStore.all().size());
+  }
+
+  @Test
+  public void testReadOfNonExistentKey() throws Exception {
+    Assert.assertNull(zkMetadataStore.get("randomKey".getBytes("UTF-8")));
+    Assert.assertEquals(0, zkMetadataStore.all().size());
+  }
+
+  @Test
+  public void testMultipleUpdatesForSameKey() throws Exception {
+    byte[] key = "test-key1".getBytes("UTF-8");
+    byte[] value = "test-value1".getBytes("UTF-8");
+    byte[] value1 = "test-value2".getBytes("UTF-8");
+    zkMetadataStore.put(key, value);
+    zkMetadataStore.put(key, value1);
+    Assert.assertTrue(Arrays.equals(value1, zkMetadataStore.get(key)));
+    Assert.assertEquals(1, zkMetadataStore.all().size());
+  }
+
+  @Test
+  public void testAllEntries() throws Exception {
+    byte[] key = "test-key1".getBytes("UTF-8");
+    byte[] key1 = "test-key2".getBytes("UTF-8");
+    byte[] key2 = "test-key3".getBytes("UTF-8");
+    byte[] value = "test-value1".getBytes("UTF-8");
+    byte[] value1 = "test-value2".getBytes("UTF-8");
+    byte[] value2 = "test-value3".getBytes("UTF-8");
+    zkMetadataStore.put(key, value);
+    zkMetadataStore.put(key1, value1);
+    zkMetadataStore.put(key2, value2);
+    ImmutableMap<byte[], byte[]> expected = ImmutableMap.of(key, value, key1, value1, key2, value2);
+    Assert.assertEquals(expected.size(), zkMetadataStore.all().size());
+  }
+}


[24/29] samza git commit: more review comments

Posted by bo...@apache.org.
more review comments


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

Branch: refs/heads/NewKafkaSystemConsumer
Commit: 4ae563c609aaaf0d2ceaeb90ab9ae33dfdc8d601
Parents: 5397a34
Author: Boris S <bo...@apache.org>
Authored: Mon Sep 24 12:31:25 2018 -0700
Committer: Boris S <bo...@apache.org>
Committed: Mon Sep 24 12:31:25 2018 -0700

----------------------------------------------------------------------
 .../samza/config/KafkaConsumerConfig.java       | 19 ++---
 .../samza/system/kafka/KafkaSystemConsumer.java | 40 +++++----
 .../samza/system/kafka/KafkaSystemFactory.scala |  1 -
 .../samza/config/TestKafkaConsumerConfig.java   | 85 ++++++++------------
 .../system/kafka/TestKafkaSystemAdminJava.java  | 18 ++---
 .../system/kafka/TestKafkaSystemConsumer.java   | 40 ++++-----
 6 files changed, 84 insertions(+), 119 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/samza/blob/4ae563c6/samza-kafka/src/main/scala/org/apache/samza/config/KafkaConsumerConfig.java
----------------------------------------------------------------------
diff --git a/samza-kafka/src/main/scala/org/apache/samza/config/KafkaConsumerConfig.java b/samza-kafka/src/main/scala/org/apache/samza/config/KafkaConsumerConfig.java
index 4bbe00f..7d2408b 100644
--- a/samza-kafka/src/main/scala/org/apache/samza/config/KafkaConsumerConfig.java
+++ b/samza-kafka/src/main/scala/org/apache/samza/config/KafkaConsumerConfig.java
@@ -23,13 +23,11 @@ package org.apache.samza.config;
 
 import java.util.HashMap;
 import java.util.Map;
-import java.util.Properties;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.kafka.clients.consumer.ConsumerConfig;
 import org.apache.kafka.clients.consumer.RangeAssignor;
 import org.apache.kafka.common.serialization.ByteArrayDeserializer;
 import org.apache.samza.SamzaException;
-import org.apache.samza.config.JobConfig;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import scala.Option;
@@ -78,7 +76,6 @@ public class KafkaConsumerConfig extends HashMap<String, Object> {
     consumerProps.put(ConsumerConfig.GROUP_ID_CONFIG, groupId);
     consumerProps.put(ConsumerConfig.CLIENT_ID_CONFIG, clientId);
 
-
     // These are values we enforce in sazma, and they cannot be overwritten.
 
     // Disable consumer auto-commit because Samza controls commits
@@ -86,7 +83,7 @@ public class KafkaConsumerConfig extends HashMap<String, Object> {
 
     // Translate samza config value to kafka config value
     consumerProps.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG,
-        getAutoOffsetResetValue((String)consumerProps.get(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG)));
+        getAutoOffsetResetValue((String) consumerProps.get(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG)));
 
     // make sure bootstrap configs are in, if not - get them from the producer
     if (!subConf.containsKey(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG)) {
@@ -148,9 +145,7 @@ public class KafkaConsumerConfig extends HashMap<String, Object> {
     String jobName = config.get(JobConfig.JOB_NAME());
     String jobId = (config.get(JobConfig.JOB_ID()) != null) ? config.get(JobConfig.JOB_ID()) : "1";
 
-    return String.format("%s-%s-%s", id.replaceAll(
-        "\\W", "_"),
-        jobName.replaceAll("\\W", "_"),
+    return String.format("%s-%s-%s", id.replaceAll("\\W", "_"), jobName.replaceAll("\\W", "_"),
         jobId.replaceAll("\\W", "_"));
   }
 
@@ -172,7 +167,7 @@ public class KafkaConsumerConfig extends HashMap<String, Object> {
     final String KAFKA_OFFSET_NONE = "none";
 
     if (autoOffsetReset == null) {
-     return KAFKA_OFFSET_LATEST; // return default
+      return KAFKA_OFFSET_LATEST; // return default
     }
 
     // accept kafka values directly
@@ -184,15 +179,15 @@ public class KafkaConsumerConfig extends HashMap<String, Object> {
     String newAutoOffsetReset;
     switch (autoOffsetReset) {
       case SAMZA_OFFSET_LARGEST:
-        newAutoOffsetReset =  KAFKA_OFFSET_LATEST;
+        newAutoOffsetReset = KAFKA_OFFSET_LATEST;
         break;
       case SAMZA_OFFSET_SMALLEST:
-        newAutoOffsetReset =  KAFKA_OFFSET_EARLIEST;
+        newAutoOffsetReset = KAFKA_OFFSET_EARLIEST;
         break;
       default:
-        newAutoOffsetReset =  KAFKA_OFFSET_LATEST;
+        newAutoOffsetReset = KAFKA_OFFSET_LATEST;
     }
-    LOG.info("AutoOffsetReset value converted from {} to {}", autoOffsetReset,  newAutoOffsetReset);
+    LOG.info("AutoOffsetReset value converted from {} to {}", autoOffsetReset, newAutoOffsetReset);
     return newAutoOffsetReset;
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/samza/blob/4ae563c6/samza-kafka/src/main/scala/org/apache/samza/system/kafka/KafkaSystemConsumer.java
----------------------------------------------------------------------
diff --git a/samza-kafka/src/main/scala/org/apache/samza/system/kafka/KafkaSystemConsumer.java b/samza-kafka/src/main/scala/org/apache/samza/system/kafka/KafkaSystemConsumer.java
index e5ded8d..17f29f1 100644
--- a/samza-kafka/src/main/scala/org/apache/samza/system/kafka/KafkaSystemConsumer.java
+++ b/samza-kafka/src/main/scala/org/apache/samza/system/kafka/KafkaSystemConsumer.java
@@ -31,11 +31,11 @@ import java.util.concurrent.atomic.AtomicBoolean;
 import kafka.common.TopicAndPartition;
 import org.apache.kafka.clients.consumer.Consumer;
 import org.apache.kafka.clients.consumer.KafkaConsumer;
-import org.apache.samza.config.KafkaConsumerConfig;
 import org.apache.kafka.common.TopicPartition;
 import org.apache.samza.SamzaException;
 import org.apache.samza.config.Config;
 import org.apache.samza.config.KafkaConfig;
+import org.apache.samza.config.KafkaConsumerConfig;
 import org.apache.samza.system.IncomingMessageEnvelope;
 import org.apache.samza.system.SystemConsumer;
 import org.apache.samza.system.SystemStreamPartition;
@@ -69,7 +69,6 @@ public class KafkaSystemConsumer<K, V> extends BlockingEnvelopeMap implements Sy
   // BlockeingEnvelopMap's buffers.
   final private KafkaConsumerProxy proxy;
 
-
   // keep registration data until the start - mapping between registered SSPs and topicPartitions, and the offsets
   final Map<TopicPartition, String> topicPartitionsToOffset = new HashMap<>();
   final Map<TopicPartition, SystemStreamPartition> topicPartitionsToSSP = new HashMap<>();
@@ -102,7 +101,7 @@ public class KafkaSystemConsumer<K, V> extends BlockingEnvelopeMap implements Sy
     // Create the proxy to do the actual message reading.
     String metricName = String.format("%s %s", systemName, clientId);
     proxy = new KafkaConsumerProxy(kafkaConsumer, systemName, clientId, messageSink, metrics, metricName);
-    LOG.info("{}: Created KafkaConsumerProxy {} ", this, proxy );
+    LOG.info("{}: Created KafkaConsumerProxy {} ", this, proxy);
 
     LOG.info("{}: Created KafkaSystemConsumer {}", this, kafkaConsumer);
   }
@@ -117,8 +116,7 @@ public class KafkaSystemConsumer<K, V> extends BlockingEnvelopeMap implements Sy
   public static KafkaConsumer<byte[], byte[]> getKafkaConsumerImpl(String systemName, String clientId, Config config) {
 
     // extract kafka client configs
-    KafkaConsumerConfig consumerConfig =
-        KafkaConsumerConfig.getKafkaSystemConsumerConfig(config, systemName, clientId);
+    KafkaConsumerConfig consumerConfig = KafkaConsumerConfig.getKafkaSystemConsumerConfig(config, systemName, clientId);
 
     LOG.info("{}:{} KafkaClient properties {}", systemName, clientId, consumerConfig);
 
@@ -179,8 +177,8 @@ public class KafkaSystemConsumer<K, V> extends BlockingEnvelopeMap implements Sy
       } catch (Exception e) {
         // all recoverable execptions are handled by the client.
         // if we get here there is nothing left to do but bail out.
-        String msg = String.format("%s: Got Exception while seeking to %s for partition %s",
-            this, startingOffsetString, tp);
+        String msg =
+            String.format("%s: Got Exception while seeking to %s for partition %s", this, startingOffsetString, tp);
         LOG.error(msg, e);
         throw new SamzaException(msg, e);
       }
@@ -217,12 +215,12 @@ public class KafkaSystemConsumer<K, V> extends BlockingEnvelopeMap implements Sy
     }
 
     int numTPs = topicPartitionsToSSP.size();
-    if (numTPs == topicPartitionsToOffset.size()) {
+    if (numTPs != topicPartitionsToOffset.size()) {
       throw new SamzaException("topicPartitionsToSSP.size() doesn't match topicPartitionsToOffset.size()");
     }
 
-    LOG.info("{}: fetchThresholdBytes = {}; fetchThreshold={}; partitions num={}",
-        this, fetchThresholdBytes, fetchThreshold, numTPs);
+    LOG.info("{}: fetchThresholdBytes = {}; fetchThreshold={}; partitions num={}", this, fetchThresholdBytes,
+        fetchThreshold, numTPs);
 
     if (numTPs > 0) {
       perPartitionFetchThreshold = fetchThreshold / numTPs;
@@ -231,8 +229,8 @@ public class KafkaSystemConsumer<K, V> extends BlockingEnvelopeMap implements Sy
         // currently this feature cannot be enabled, because we do not have the size of the messages available.
         // messages get double buffered, hence divide by 2
         perPartitionFetchThresholdBytes = (fetchThresholdBytes / 2) / numTPs;
-        LOG.info("{} :perPartitionFetchThresholdBytes is enabled. perPartitionFetchThresholdBytes={}",
-            this, perPartitionFetchThresholdBytes);
+        LOG.info("{} :perPartitionFetchThresholdBytes is enabled. perPartitionFetchThresholdBytes={}", this,
+            perPartitionFetchThresholdBytes);
       }
     }
   }
@@ -268,9 +266,8 @@ public class KafkaSystemConsumer<K, V> extends BlockingEnvelopeMap implements Sy
   @Override
   public void register(SystemStreamPartition systemStreamPartition, String offset) {
     if (started.get()) {
-      String msg =
-          String.format("%s: Trying to register partition after consumer has been started. ssp=%s",
-              this, systemStreamPartition);
+      String msg = String.format("%s: Trying to register partition after consumer has been started. ssp=%s", this,
+          systemStreamPartition);
       throw new SamzaException(msg);
     }
 
@@ -286,7 +283,6 @@ public class KafkaSystemConsumer<K, V> extends BlockingEnvelopeMap implements Sy
 
     topicPartitionsToSSP.put(tp, systemStreamPartition);
 
-
     String existingOffset = topicPartitionsToOffset.get(tp);
     // register the older (of the two) offset in the consumer, to guarantee we do not miss any messages.
     if (existingOffset == null || compareOffsets(existingOffset, offset) > 0) {
@@ -353,10 +349,10 @@ public class KafkaSystemConsumer<K, V> extends BlockingEnvelopeMap implements Sy
     }
 
     boolean needsMoreMessages(SystemStreamPartition ssp) {
-        LOG.debug("{}: needsMoreMessages from following SSP: {}. fetchLimitByBytes enabled={}; messagesSizeInQueue={};"
-                + "(limit={}); messagesNumInQueue={}(limit={};", this, ssp, fetchThresholdBytesEnabled,
-            getMessagesSizeInQueue(ssp), perPartitionFetchThresholdBytes, getNumMessagesInQueue(ssp),
-            perPartitionFetchThreshold);
+      LOG.debug("{}: needsMoreMessages from following SSP: {}. fetchLimitByBytes enabled={}; messagesSizeInQueue={};"
+              + "(limit={}); messagesNumInQueue={}(limit={};", this, ssp, fetchThresholdBytesEnabled,
+          getMessagesSizeInQueue(ssp), perPartitionFetchThresholdBytes, getNumMessagesInQueue(ssp),
+          perPartitionFetchThreshold);
 
       if (fetchThresholdBytesEnabled) {
         return getMessagesSizeInQueue(ssp) < perPartitionFetchThresholdBytes;
@@ -372,8 +368,8 @@ public class KafkaSystemConsumer<K, V> extends BlockingEnvelopeMap implements Sy
         put(ssp, envelope);
       } catch (InterruptedException e) {
         throw new SamzaException(
-            String.format("%s: Consumer was interrupted while trying to add message with offset %s for ssp %s",
-                this, envelope.getOffset(), ssp));
+            String.format("%s: Consumer was interrupted while trying to add message with offset %s for ssp %s", this,
+                envelope.getOffset(), ssp));
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/samza/blob/4ae563c6/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
index deaee56..ba5390b 100644
--- 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
@@ -22,7 +22,6 @@ package org.apache.samza.system.kafka
 import java.util.Properties
 
 import kafka.utils.ZkUtils
-import org.apache.kafka.clients.consumer.KafkaConsumer
 import org.apache.kafka.clients.producer.KafkaProducer
 import org.apache.samza.SamzaException
 import org.apache.samza.config.ApplicationConfig.ApplicationMode

http://git-wip-us.apache.org/repos/asf/samza/blob/4ae563c6/samza-kafka/src/test/java/org/apache/samza/config/TestKafkaConsumerConfig.java
----------------------------------------------------------------------
diff --git a/samza-kafka/src/test/java/org/apache/samza/config/TestKafkaConsumerConfig.java b/samza-kafka/src/test/java/org/apache/samza/config/TestKafkaConsumerConfig.java
index 719ea22..35a717a 100644
--- a/samza-kafka/src/test/java/org/apache/samza/config/TestKafkaConsumerConfig.java
+++ b/samza-kafka/src/test/java/org/apache/samza/config/TestKafkaConsumerConfig.java
@@ -24,7 +24,6 @@ import org.apache.kafka.clients.consumer.RangeAssignor;
 import org.apache.kafka.common.serialization.ByteArrayDeserializer;
 import org.apache.samza.SamzaException;
 import org.junit.Assert;
-import org.junit.Before;
 import org.junit.Test;
 
 
@@ -35,84 +34,67 @@ public class TestKafkaConsumerConfig {
   public final static String KAFKA_CONSUMER_PROPERTY_PREFIX = "systems." + SYSTEM_NAME + ".consumer.";
   private final static String CLIENT_ID = "clientId";
 
-  @Before
-  public void setProps() {
-
-  }
-
   @Test
   public void testDefaults() {
 
     props.put(KAFKA_CONSUMER_PROPERTY_PREFIX + ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "true"); // should be ignored
-    props.put(KAFKA_CONSUMER_PROPERTY_PREFIX + ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG, "Ignore"); // should be ignored
-    props.put(KAFKA_CONSUMER_PROPERTY_PREFIX + ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG, "100"); // should NOT be ignored
+    props.put(KAFKA_CONSUMER_PROPERTY_PREFIX + ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG,
+        "Ignore"); // should be ignored
+    props.put(KAFKA_CONSUMER_PROPERTY_PREFIX + ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG,
+        "100"); // should NOT be ignored
 
     // if KAFKA_CONSUMER_PROPERTY_PREFIX is set, then PRODUCER should be ignored
     props.put(KAFKA_PRODUCER_PROPERTY_PREFIX + "bootstrap.servers", "ignroeThis:9092");
     props.put(KAFKA_CONSUMER_PROPERTY_PREFIX + "bootstrap.servers", "useThis:9092");
 
     Config config = new MapConfig(props);
-    KafkaConsumerConfig kafkaConsumerConfig = KafkaConsumerConfig.getKafkaSystemConsumerConfig(
-        config, SYSTEM_NAME, CLIENT_ID);
+    KafkaConsumerConfig kafkaConsumerConfig =
+        KafkaConsumerConfig.getKafkaSystemConsumerConfig(config, SYSTEM_NAME, CLIENT_ID);
 
     Assert.assertEquals("false", kafkaConsumerConfig.get(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG));
 
-    Assert.assertEquals(
-        KafkaConsumerConfig.DEFAULT_KAFKA_CONSUMER_MAX_POLL_RECORDS,
+    Assert.assertEquals(KafkaConsumerConfig.DEFAULT_KAFKA_CONSUMER_MAX_POLL_RECORDS,
         kafkaConsumerConfig.get(ConsumerConfig.MAX_POLL_RECORDS_CONFIG));
 
-    Assert.assertEquals(
-        RangeAssignor.class.getName(),
+    Assert.assertEquals(RangeAssignor.class.getName(),
         kafkaConsumerConfig.get(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG));
 
-    Assert.assertEquals(
-        "useThis:9092",
-        kafkaConsumerConfig.get(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG));
-    Assert.assertEquals(
-        "100",
-        kafkaConsumerConfig.get(ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG));
+    Assert.assertEquals("useThis:9092", kafkaConsumerConfig.get(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG));
+    Assert.assertEquals("100", kafkaConsumerConfig.get(ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG));
 
-    Assert.assertEquals(
-        ByteArrayDeserializer.class.getName(),
+    Assert.assertEquals(ByteArrayDeserializer.class.getName(),
         kafkaConsumerConfig.get(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG));
 
-    Assert.assertEquals(
-        ByteArrayDeserializer.class.getName(),
-        kafkaConsumerConfig.get(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG) );
+    Assert.assertEquals(ByteArrayDeserializer.class.getName(),
+        kafkaConsumerConfig.get(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG));
 
-    Assert.assertEquals(
-        CLIENT_ID,
-        kafkaConsumerConfig.get(ConsumerConfig.CLIENT_ID_CONFIG));
+    Assert.assertEquals(CLIENT_ID, kafkaConsumerConfig.get(ConsumerConfig.CLIENT_ID_CONFIG));
 
-    Assert.assertEquals(
-        KafkaConsumerConfig.getConsumerGroupId(config),
+    Assert.assertEquals(KafkaConsumerConfig.getConsumerGroupId(config),
         kafkaConsumerConfig.get(ConsumerConfig.GROUP_ID_CONFIG));
   }
 
-  @Test
   // test stuff that should not be overridden
+  @Test
   public void testNotOverride() {
 
     // if KAFKA_CONSUMER_PROPERTY_PREFIX is not set, then PRODUCER should be used
     props.put(KAFKA_PRODUCER_PROPERTY_PREFIX + ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "useThis:9092");
-    props.put(KAFKA_CONSUMER_PROPERTY_PREFIX + ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, TestKafkaConsumerConfig.class.getName());
-    props.put(KAFKA_CONSUMER_PROPERTY_PREFIX + ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, TestKafkaConsumerConfig.class.getName());
-
+    props.put(KAFKA_CONSUMER_PROPERTY_PREFIX + ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG,
+        TestKafkaConsumerConfig.class.getName());
+    props.put(KAFKA_CONSUMER_PROPERTY_PREFIX + ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG,
+        TestKafkaConsumerConfig.class.getName());
 
     Config config = new MapConfig(props);
-    KafkaConsumerConfig kafkaConsumerConfig = KafkaConsumerConfig.getKafkaSystemConsumerConfig(
-        config, SYSTEM_NAME, CLIENT_ID);
+    KafkaConsumerConfig kafkaConsumerConfig =
+        KafkaConsumerConfig.getKafkaSystemConsumerConfig(config, SYSTEM_NAME, CLIENT_ID);
 
-    Assert.assertEquals(
-        "useThis:9092",
-        kafkaConsumerConfig.get(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG));
+    Assert.assertEquals("useThis:9092", kafkaConsumerConfig.get(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG));
 
-    Assert.assertEquals(
-        TestKafkaConsumerConfig.class.getName(),
+    Assert.assertEquals(TestKafkaConsumerConfig.class.getName(),
         kafkaConsumerConfig.get(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG));
 
-    Assert.assertEquals(
-        TestKafkaConsumerConfig.class.getName(),
+    Assert.assertEquals(TestKafkaConsumerConfig.class.getName(),
         kafkaConsumerConfig.get(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG));
   }
 
@@ -122,30 +104,29 @@ public class TestKafkaConsumerConfig {
 
     map.put(JobConfig.JOB_NAME(), "jobName");
     map.put(JobConfig.JOB_ID(), "jobId");
-    String result =  KafkaConsumerConfig.getConsumerClientId("consumer", new MapConfig(map));
+    String result = KafkaConsumerConfig.getConsumerClientId("consumer", new MapConfig(map));
     Assert.assertEquals("consumer-jobName-jobId", result);
 
-    result =  KafkaConsumerConfig.getConsumerClientId("consumer-", new MapConfig(map));
+    result = KafkaConsumerConfig.getConsumerClientId("consumer-", new MapConfig(map));
     Assert.assertEquals("consumer_-jobName-jobId", result);
 
-    result =  KafkaConsumerConfig.getConsumerClientId("super-duper-consumer", new MapConfig(map));
+    result = KafkaConsumerConfig.getConsumerClientId("super-duper-consumer", new MapConfig(map));
     Assert.assertEquals("super_duper_consumer-jobName-jobId", result);
 
     map.put(JobConfig.JOB_NAME(), " very important!job");
-    result =  KafkaConsumerConfig.getConsumerClientId("consumer", new MapConfig(map));
+    result = KafkaConsumerConfig.getConsumerClientId("consumer", new MapConfig(map));
     Assert.assertEquals("consumer-_very_important_job-jobId", result);
 
     map.put(JobConfig.JOB_ID(), "number-#3");
-    result =  KafkaConsumerConfig.getConsumerClientId("consumer", new MapConfig(map));
+    result = KafkaConsumerConfig.getConsumerClientId("consumer", new MapConfig(map));
     Assert.assertEquals("consumer-_very_important_job-number__3", result);
   }
 
-
-
   @Test(expected = SamzaException.class)
   public void testNoBootstrapServers() {
-    KafkaConsumerConfig kafkaConsumerConfig = KafkaConsumerConfig.getKafkaSystemConsumerConfig(
-        new MapConfig(Collections.emptyMap()), SYSTEM_NAME, "clientId");
+    KafkaConsumerConfig kafkaConsumerConfig =
+        KafkaConsumerConfig.getKafkaSystemConsumerConfig(new MapConfig(Collections.emptyMap()), SYSTEM_NAME,
+            "clientId");
 
     Assert.fail("didn't get exception for the missing config:" + ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG);
   }

http://git-wip-us.apache.org/repos/asf/samza/blob/4ae563c6/samza-kafka/src/test/java/org/apache/samza/system/kafka/TestKafkaSystemAdminJava.java
----------------------------------------------------------------------
diff --git a/samza-kafka/src/test/java/org/apache/samza/system/kafka/TestKafkaSystemAdminJava.java b/samza-kafka/src/test/java/org/apache/samza/system/kafka/TestKafkaSystemAdminJava.java
index 77f47f9..7e968bf 100644
--- a/samza-kafka/src/test/java/org/apache/samza/system/kafka/TestKafkaSystemAdminJava.java
+++ b/samza-kafka/src/test/java/org/apache/samza/system/kafka/TestKafkaSystemAdminJava.java
@@ -19,17 +19,14 @@
 
 package org.apache.samza.system.kafka;
 
-import java.util.*;
 import java.util.HashMap;
 import java.util.Map;
-
+import java.util.Properties;
 import kafka.api.TopicMetadata;
 import org.apache.samza.system.StreamSpec;
 import org.apache.samza.system.StreamValidationException;
 import org.apache.samza.system.SystemAdmin;
 import org.apache.samza.util.ScalaJavaUtil;
-import org.junit.After;
-import org.junit.Before;
 import org.junit.Test;
 import org.mockito.Mockito;
 
@@ -71,7 +68,6 @@ public class TestKafkaSystemAdminJava extends TestKafkaSystemAdmin {
 
     admin.createStream(spec);
     admin.validateStream(spec);
-
   }
 
   @Test
@@ -143,7 +139,8 @@ public class TestKafkaSystemAdminJava extends TestKafkaSystemAdmin {
   public void testCreateStream() {
     StreamSpec spec = new StreamSpec("testId", "testStream", "testSystem", 8);
 
-    assertTrue("createStream should return true if the stream does not exist and then is created.", systemAdmin().createStream(spec));
+    assertTrue("createStream should return true if the stream does not exist and then is created.",
+        systemAdmin().createStream(spec));
     systemAdmin().validateStream(spec);
 
     assertFalse("createStream should return false if the stream already exists.", systemAdmin().createStream(spec));
@@ -162,7 +159,8 @@ public class TestKafkaSystemAdminJava extends TestKafkaSystemAdmin {
     StreamSpec spec1 = new StreamSpec("testId", "testStreamPartition", "testSystem", 8);
     StreamSpec spec2 = new StreamSpec("testId", "testStreamPartition", "testSystem", 4);
 
-    assertTrue("createStream should return true if the stream does not exist and then is created.", systemAdmin().createStream(spec1));
+    assertTrue("createStream should return true if the stream does not exist and then is created.",
+        systemAdmin().createStream(spec1));
 
     systemAdmin().validateStream(spec2);
   }
@@ -172,7 +170,8 @@ public class TestKafkaSystemAdminJava extends TestKafkaSystemAdmin {
     StreamSpec spec1 = new StreamSpec("testId", "testStreamName1", "testSystem", 8);
     StreamSpec spec2 = new StreamSpec("testId", "testStreamName2", "testSystem", 8);
 
-    assertTrue("createStream should return true if the stream does not exist and then is created.", systemAdmin().createStream(spec1));
+    assertTrue("createStream should return true if the stream does not exist and then is created.",
+        systemAdmin().createStream(spec1));
 
     systemAdmin().validateStream(spec2);
   }
@@ -181,7 +180,8 @@ public class TestKafkaSystemAdminJava extends TestKafkaSystemAdmin {
   public void testClearStream() {
     StreamSpec spec = new StreamSpec("testId", "testStreamClear", "testSystem", 8);
 
-    assertTrue("createStream should return true if the stream does not exist and then is created.", systemAdmin().createStream(spec));
+    assertTrue("createStream should return true if the stream does not exist and then is created.",
+        systemAdmin().createStream(spec));
     assertTrue(systemAdmin().clearStream(spec));
 
     scala.collection.immutable.Set<String> topic = new scala.collection.immutable.Set.Set1<>(spec.getPhysicalName());

http://git-wip-us.apache.org/repos/asf/samza/blob/4ae563c6/samza-kafka/src/test/scala/org/apache/samza/system/kafka/TestKafkaSystemConsumer.java
----------------------------------------------------------------------
diff --git a/samza-kafka/src/test/scala/org/apache/samza/system/kafka/TestKafkaSystemConsumer.java b/samza-kafka/src/test/scala/org/apache/samza/system/kafka/TestKafkaSystemConsumer.java
index 9e8ff44..933558c 100644
--- a/samza-kafka/src/test/scala/org/apache/samza/system/kafka/TestKafkaSystemConsumer.java
+++ b/samza-kafka/src/test/scala/org/apache/samza/system/kafka/TestKafkaSystemConsumer.java
@@ -21,24 +21,22 @@
 
 package org.apache.samza.system.kafka;
 
-import java.util.Collections;
 import java.util.HashMap;
 import java.util.Map;
 import org.apache.kafka.clients.consumer.Consumer;
 import org.apache.kafka.clients.consumer.ConsumerConfig;
 import org.apache.kafka.clients.consumer.KafkaConsumer;
-import org.apache.samza.config.KafkaConsumerConfig;
 import org.apache.kafka.common.serialization.ByteArraySerializer;
 import org.apache.samza.Partition;
 import org.apache.samza.config.Config;
 import org.apache.samza.config.KafkaConfig;
+import org.apache.samza.config.KafkaConsumerConfig;
 import org.apache.samza.config.MapConfig;
 import org.apache.samza.system.IncomingMessageEnvelope;
 import org.apache.samza.system.SystemStreamPartition;
 import org.apache.samza.util.Clock;
 import org.apache.samza.util.NoOpMetricsRegistry;
 import org.junit.Assert;
-import org.junit.Before;
 import org.junit.Test;
 
 import static org.junit.Assert.*;
@@ -52,12 +50,7 @@ public class TestKafkaSystemConsumer {
   public final String FETCH_THRESHOLD_MSGS = "50000";
   public final String FETCH_THRESHOLD_BYTES = "100000";
 
-  @Before
-  public void setUp() {
-
-  }
-
-  private KafkaSystemConsumer setupConsumer(String fetchMsg, String fetchBytes) {
+  private KafkaSystemConsumer createConsumer(String fetchMsg, String fetchBytes) {
     final Map<String, String> map = new HashMap<>();
 
     map.put(String.format(KafkaConfig.CONSUMER_FETCH_THRESHOLD(), TEST_SYSTEM), fetchMsg);
@@ -70,8 +63,8 @@ public class TestKafkaSystemConsumer {
         KafkaConsumerConfig.getKafkaSystemConsumerConfig(config, TEST_SYSTEM, TEST_CLIENT_ID);
     final KafkaConsumer<byte[], byte[]> kafkaConsumer = new MockKafkaConsumer(consumerConfig);
 
-    MockKafkaSystmeCosumer newKafkaSystemConsumer =
-        new MockKafkaSystmeCosumer(kafkaConsumer, TEST_SYSTEM, config, TEST_CLIENT_ID,
+    MockKafkaSystemConsumer newKafkaSystemConsumer =
+        new MockKafkaSystemConsumer(kafkaConsumer, TEST_SYSTEM, config, TEST_CLIENT_ID,
             new KafkaSystemConsumerMetrics(TEST_SYSTEM, new NoOpMetricsRegistry()), System::currentTimeMillis);
 
     return newKafkaSystemConsumer;
@@ -80,7 +73,7 @@ public class TestKafkaSystemConsumer {
   @Test
   public void testConfigValidations() {
 
-    final KafkaSystemConsumer consumer = setupConsumer(FETCH_THRESHOLD_MSGS, FETCH_THRESHOLD_BYTES);
+    final KafkaSystemConsumer consumer = createConsumer(FETCH_THRESHOLD_MSGS, FETCH_THRESHOLD_BYTES);
 
     consumer.start();
     // should be no failures
@@ -88,7 +81,7 @@ public class TestKafkaSystemConsumer {
 
   @Test
   public void testFetchThresholdShouldDivideEvenlyAmongPartitions() {
-    final KafkaSystemConsumer consumer = setupConsumer(FETCH_THRESHOLD_MSGS, FETCH_THRESHOLD_BYTES);
+    final KafkaSystemConsumer consumer = createConsumer(FETCH_THRESHOLD_MSGS, FETCH_THRESHOLD_BYTES);
     final int partitionsNum = 50;
     for (int i = 0; i < partitionsNum; i++) {
       consumer.register(new SystemStreamPartition(TEST_SYSTEM, TEST_STREAM, new Partition(i)), "0");
@@ -99,12 +92,14 @@ public class TestKafkaSystemConsumer {
     Assert.assertEquals(Long.valueOf(FETCH_THRESHOLD_MSGS) / partitionsNum, consumer.perPartitionFetchThreshold);
     Assert.assertEquals(Long.valueOf(FETCH_THRESHOLD_BYTES) / 2 / partitionsNum,
         consumer.perPartitionFetchThresholdBytes);
+
+    consumer.stop();
   }
 
   @Test
   public void testConsumerRegisterOlderOffsetOfTheSamzaSSP() {
 
-    KafkaSystemConsumer consumer = setupConsumer(FETCH_THRESHOLD_MSGS, FETCH_THRESHOLD_BYTES);
+    KafkaSystemConsumer consumer = createConsumer(FETCH_THRESHOLD_MSGS, FETCH_THRESHOLD_BYTES);
 
     SystemStreamPartition ssp0 = new SystemStreamPartition(TEST_SYSTEM, TEST_STREAM, new Partition(0));
     SystemStreamPartition ssp1 = new SystemStreamPartition(TEST_SYSTEM, TEST_STREAM, new Partition(1));
@@ -137,7 +132,7 @@ public class TestKafkaSystemConsumer {
         bytesSerde.serialize("", "value1".getBytes()), ime1Size);
     IncomingMessageEnvelope ime11 = new IncomingMessageEnvelope(ssp1, "0", bytesSerde.serialize("", "key11".getBytes()),
         bytesSerde.serialize("", "value11".getBytes()), ime11Size);
-    KafkaSystemConsumer consumer = setupConsumer(FETCH_THRESHOLD_MSGS, FETCH_THRESHOLD_BYTES);
+    KafkaSystemConsumer consumer = createConsumer(FETCH_THRESHOLD_MSGS, FETCH_THRESHOLD_BYTES);
 
     consumer.register(ssp0, "0");
     consumer.register(ssp1, "0");
@@ -156,6 +151,8 @@ public class TestKafkaSystemConsumer {
     Assert.assertEquals(2, consumer.getNumMessagesInQueue(ssp1));
     Assert.assertEquals(ime0Size, consumer.getMessagesSizeInQueue(ssp0));
     Assert.assertEquals(ime1Size + ime11Size, consumer.getMessagesSizeInQueue(ssp1));
+
+    consumer.stop();
   }
 
   @Test
@@ -178,7 +175,7 @@ public class TestKafkaSystemConsumer {
 
     // limit by number of messages 4/2 = 2 per partition
     // limit by number of bytes - disabled
-    KafkaSystemConsumer consumer = setupConsumer("4", "0"); // should disable
+    KafkaSystemConsumer consumer = createConsumer("4", "0"); // should disable
 
     consumer.register(ssp0, "0");
     consumer.register(ssp1, "0");
@@ -197,6 +194,8 @@ public class TestKafkaSystemConsumer {
     Assert.assertEquals(2, consumer.getNumMessagesInQueue(ssp1));
     Assert.assertEquals(ime0Size, consumer.getMessagesSizeInQueue(ssp0));
     Assert.assertEquals(ime1Size + ime11Size, consumer.getMessagesSizeInQueue(ssp1));
+
+    consumer.stop();
   }
 
   // mock kafkaConsumer and SystemConsumer
@@ -206,17 +205,12 @@ public class TestKafkaSystemConsumer {
     }
   }
 
-  static class MockKafkaSystmeCosumer extends KafkaSystemConsumer {
-    public MockKafkaSystmeCosumer(Consumer kafkaConsumer, String systemName, Config config, String clientId,
+  static class MockKafkaSystemConsumer extends KafkaSystemConsumer {
+    public MockKafkaSystemConsumer(Consumer kafkaConsumer, String systemName, Config config, String clientId,
         KafkaSystemConsumerMetrics metrics, Clock clock) {
       super(kafkaConsumer, systemName, config, clientId, metrics, clock);
     }
 
-    //@Override
-    //void createConsumerProxy() {
-    //  this.messageSink = new KafkaConsumerMessageSink();
-    //}
-
     @Override
     void startConsumer() {
     }


[18/29] samza git commit: SAMZA-1898: New UI layout for the Samza website

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/samza/blob/b8058af0/docs/css/font-awesome.min.css
----------------------------------------------------------------------
diff --git a/docs/css/font-awesome.min.css b/docs/css/font-awesome.min.css
deleted file mode 100644
index 3d920fc..0000000
--- a/docs/css/font-awesome.min.css
+++ /dev/null
@@ -1,4 +0,0 @@
-/*!
- *  Font Awesome 4.1.0 by @davegandy - http://fontawesome.io - @fontawesome
- *  License - http://fontawesome.io/license (Font: SIL OFL 1.1, CSS: MIT License)
- */@font-face{font-family:'FontAwesome';src:url('../fonts/fontawesome-webfont.eot?v=4.1.0');src:url('../fonts/fontawesome-webfont.eot?#iefix&v=4.1.0') format('embedded-opentype'),url('../fonts/fontawesome-webfont.woff?v=4.1.0') format('woff'),url('../fonts/fontawesome-webfont.ttf?v=4.1.0') format('truetype'),url('../fonts/fontawesome-webfont.svg?v=4.1.0#fontawesomeregular') format('svg');font-weight:normal;font-style:normal}.fa{display:inline-block;font-family:FontAwesome;font-style:normal;font-weight:normal;line-height:1;-webkit-font-smoothing:antialiased;-moz-osx-font-smoothing:grayscale}.fa-lg{font-size:1.33333333em;line-height:.75em;vertical-align:-15%}.fa-2x{font-size:2em}.fa-3x{font-size:3em}.fa-4x{font-size:4em}.fa-5x{font-size:5em}.fa-fw{width:1.28571429em;text-align:center}.fa-ul{padding-left:0;margin-left:2.14285714em;list-style-type:none}.fa-ul>li{position:relative}.fa-li{position:absolute;left:-2.14285714em;width:2.14285714em;top:.14285714em;text-align:center}.fa-li.fa-l
 g{left:-1.85714286em}.fa-border{padding:.2em .25em .15em;border:solid .08em #eee;border-radius:.1em}.pull-right{float:right}.pull-left{float:left}.fa.pull-left{margin-right:.3em}.fa.pull-right{margin-left:.3em}.fa-spin{-webkit-animation:spin 2s infinite linear;-moz-animation:spin 2s infinite linear;-o-animation:spin 2s infinite linear;animation:spin 2s infinite linear}@-moz-keyframes spin{0%{-moz-transform:rotate(0deg)}100%{-moz-transform:rotate(359deg)}}@-webkit-keyframes spin{0%{-webkit-transform:rotate(0deg)}100%{-webkit-transform:rotate(359deg)}}@-o-keyframes spin{0%{-o-transform:rotate(0deg)}100%{-o-transform:rotate(359deg)}}@keyframes spin{0%{-webkit-transform:rotate(0deg);transform:rotate(0deg)}100%{-webkit-transform:rotate(359deg);transform:rotate(359deg)}}.fa-rotate-90{filter:progid:DXImageTransform.Microsoft.BasicImage(rotation=1);-webkit-transform:rotate(90deg);-moz-transform:rotate(90deg);-ms-transform:rotate(90deg);-o-transform:rotate(90deg);transform:rotate(90deg)}.fa-
 rotate-180{filter:progid:DXImageTransform.Microsoft.BasicImage(rotation=2);-webkit-transform:rotate(180deg);-moz-transform:rotate(180deg);-ms-transform:rotate(180deg);-o-transform:rotate(180deg);transform:rotate(180deg)}.fa-rotate-270{filter:progid:DXImageTransform.Microsoft.BasicImage(rotation=3);-webkit-transform:rotate(270deg);-moz-transform:rotate(270deg);-ms-transform:rotate(270deg);-o-transform:rotate(270deg);transform:rotate(270deg)}.fa-flip-horizontal{filter:progid:DXImageTransform.Microsoft.BasicImage(rotation=0, mirror=1);-webkit-transform:scale(-1, 1);-moz-transform:scale(-1, 1);-ms-transform:scale(-1, 1);-o-transform:scale(-1, 1);transform:scale(-1, 1)}.fa-flip-vertical{filter:progid:DXImageTransform.Microsoft.BasicImage(rotation=2, mirror=1);-webkit-transform:scale(1, -1);-moz-transform:scale(1, -1);-ms-transform:scale(1, -1);-o-transform:scale(1, -1);transform:scale(1, -1)}.fa-stack{position:relative;display:inline-block;width:2em;height:2em;line-height:2em;vertical-al
 ign:middle}.fa-stack-1x,.fa-stack-2x{position:absolute;left:0;width:100%;text-align:center}.fa-stack-1x{line-height:inherit}.fa-stack-2x{font-size:2em}.fa-inverse{color:#fff}.fa-glass:before{content:"\f000"}.fa-music:before{content:"\f001"}.fa-search:before{content:"\f002"}.fa-envelope-o:before{content:"\f003"}.fa-heart:before{content:"\f004"}.fa-star:before{content:"\f005"}.fa-star-o:before{content:"\f006"}.fa-user:before{content:"\f007"}.fa-film:before{content:"\f008"}.fa-th-large:before{content:"\f009"}.fa-th:before{content:"\f00a"}.fa-th-list:before{content:"\f00b"}.fa-check:before{content:"\f00c"}.fa-times:before{content:"\f00d"}.fa-search-plus:before{content:"\f00e"}.fa-search-minus:before{content:"\f010"}.fa-power-off:before{content:"\f011"}.fa-signal:before{content:"\f012"}.fa-gear:before,.fa-cog:before{content:"\f013"}.fa-trash-o:before{content:"\f014"}.fa-home:before{content:"\f015"}.fa-file-o:before{content:"\f016"}.fa-clock-o:before{content:"\f017"}.fa-road:before{conten
 t:"\f018"}.fa-download:before{content:"\f019"}.fa-arrow-circle-o-down:before{content:"\f01a"}.fa-arrow-circle-o-up:before{content:"\f01b"}.fa-inbox:before{content:"\f01c"}.fa-play-circle-o:before{content:"\f01d"}.fa-rotate-right:before,.fa-repeat:before{content:"\f01e"}.fa-refresh:before{content:"\f021"}.fa-list-alt:before{content:"\f022"}.fa-lock:before{content:"\f023"}.fa-flag:before{content:"\f024"}.fa-headphones:before{content:"\f025"}.fa-volume-off:before{content:"\f026"}.fa-volume-down:before{content:"\f027"}.fa-volume-up:before{content:"\f028"}.fa-qrcode:before{content:"\f029"}.fa-barcode:before{content:"\f02a"}.fa-tag:before{content:"\f02b"}.fa-tags:before{content:"\f02c"}.fa-book:before{content:"\f02d"}.fa-bookmark:before{content:"\f02e"}.fa-print:before{content:"\f02f"}.fa-camera:before{content:"\f030"}.fa-font:before{content:"\f031"}.fa-bold:before{content:"\f032"}.fa-italic:before{content:"\f033"}.fa-text-height:before{content:"\f034"}.fa-text-width:before{content:"\f035
 "}.fa-align-left:before{content:"\f036"}.fa-align-center:before{content:"\f037"}.fa-align-right:before{content:"\f038"}.fa-align-justify:before{content:"\f039"}.fa-list:before{content:"\f03a"}.fa-dedent:before,.fa-outdent:before{content:"\f03b"}.fa-indent:before{content:"\f03c"}.fa-video-camera:before{content:"\f03d"}.fa-photo:before,.fa-image:before,.fa-picture-o:before{content:"\f03e"}.fa-pencil:before{content:"\f040"}.fa-map-marker:before{content:"\f041"}.fa-adjust:before{content:"\f042"}.fa-tint:before{content:"\f043"}.fa-edit:before,.fa-pencil-square-o:before{content:"\f044"}.fa-share-square-o:before{content:"\f045"}.fa-check-square-o:before{content:"\f046"}.fa-arrows:before{content:"\f047"}.fa-step-backward:before{content:"\f048"}.fa-fast-backward:before{content:"\f049"}.fa-backward:before{content:"\f04a"}.fa-play:before{content:"\f04b"}.fa-pause:before{content:"\f04c"}.fa-stop:before{content:"\f04d"}.fa-forward:before{content:"\f04e"}.fa-fast-forward:before{content:"\f050"}.f
 a-step-forward:before{content:"\f051"}.fa-eject:before{content:"\f052"}.fa-chevron-left:before{content:"\f053"}.fa-chevron-right:before{content:"\f054"}.fa-plus-circle:before{content:"\f055"}.fa-minus-circle:before{content:"\f056"}.fa-times-circle:before{content:"\f057"}.fa-check-circle:before{content:"\f058"}.fa-question-circle:before{content:"\f059"}.fa-info-circle:before{content:"\f05a"}.fa-crosshairs:before{content:"\f05b"}.fa-times-circle-o:before{content:"\f05c"}.fa-check-circle-o:before{content:"\f05d"}.fa-ban:before{content:"\f05e"}.fa-arrow-left:before{content:"\f060"}.fa-arrow-right:before{content:"\f061"}.fa-arrow-up:before{content:"\f062"}.fa-arrow-down:before{content:"\f063"}.fa-mail-forward:before,.fa-share:before{content:"\f064"}.fa-expand:before{content:"\f065"}.fa-compress:before{content:"\f066"}.fa-plus:before{content:"\f067"}.fa-minus:before{content:"\f068"}.fa-asterisk:before{content:"\f069"}.fa-exclamation-circle:before{content:"\f06a"}.fa-gift:before{content:"\
 f06b"}.fa-leaf:before{content:"\f06c"}.fa-fire:before{content:"\f06d"}.fa-eye:before{content:"\f06e"}.fa-eye-slash:before{content:"\f070"}.fa-warning:before,.fa-exclamation-triangle:before{content:"\f071"}.fa-plane:before{content:"\f072"}.fa-calendar:before{content:"\f073"}.fa-random:before{content:"\f074"}.fa-comment:before{content:"\f075"}.fa-magnet:before{content:"\f076"}.fa-chevron-up:before{content:"\f077"}.fa-chevron-down:before{content:"\f078"}.fa-retweet:before{content:"\f079"}.fa-shopping-cart:before{content:"\f07a"}.fa-folder:before{content:"\f07b"}.fa-folder-open:before{content:"\f07c"}.fa-arrows-v:before{content:"\f07d"}.fa-arrows-h:before{content:"\f07e"}.fa-bar-chart-o:before{content:"\f080"}.fa-twitter-square:before{content:"\f081"}.fa-facebook-square:before{content:"\f082"}.fa-camera-retro:before{content:"\f083"}.fa-key:before{content:"\f084"}.fa-gears:before,.fa-cogs:before{content:"\f085"}.fa-comments:before{content:"\f086"}.fa-thumbs-o-up:before{content:"\f087"}.f
 a-thumbs-o-down:before{content:"\f088"}.fa-star-half:before{content:"\f089"}.fa-heart-o:before{content:"\f08a"}.fa-sign-out:before{content:"\f08b"}.fa-linkedin-square:before{content:"\f08c"}.fa-thumb-tack:before{content:"\f08d"}.fa-external-link:before{content:"\f08e"}.fa-sign-in:before{content:"\f090"}.fa-trophy:before{content:"\f091"}.fa-github-square:before{content:"\f092"}.fa-upload:before{content:"\f093"}.fa-lemon-o:before{content:"\f094"}.fa-phone:before{content:"\f095"}.fa-square-o:before{content:"\f096"}.fa-bookmark-o:before{content:"\f097"}.fa-phone-square:before{content:"\f098"}.fa-twitter:before{content:"\f099"}.fa-facebook:before{content:"\f09a"}.fa-github:before{content:"\f09b"}.fa-unlock:before{content:"\f09c"}.fa-credit-card:before{content:"\f09d"}.fa-rss:before{content:"\f09e"}.fa-hdd-o:before{content:"\f0a0"}.fa-bullhorn:before{content:"\f0a1"}.fa-bell:before{content:"\f0f3"}.fa-certificate:before{content:"\f0a3"}.fa-hand-o-right:before{content:"\f0a4"}.fa-hand-o-le
 ft:before{content:"\f0a5"}.fa-hand-o-up:before{content:"\f0a6"}.fa-hand-o-down:before{content:"\f0a7"}.fa-arrow-circle-left:before{content:"\f0a8"}.fa-arrow-circle-right:before{content:"\f0a9"}.fa-arrow-circle-up:before{content:"\f0aa"}.fa-arrow-circle-down:before{content:"\f0ab"}.fa-globe:before{content:"\f0ac"}.fa-wrench:before{content:"\f0ad"}.fa-tasks:before{content:"\f0ae"}.fa-filter:before{content:"\f0b0"}.fa-briefcase:before{content:"\f0b1"}.fa-arrows-alt:before{content:"\f0b2"}.fa-group:before,.fa-users:before{content:"\f0c0"}.fa-chain:before,.fa-link:before{content:"\f0c1"}.fa-cloud:before{content:"\f0c2"}.fa-flask:before{content:"\f0c3"}.fa-cut:before,.fa-scissors:before{content:"\f0c4"}.fa-copy:before,.fa-files-o:before{content:"\f0c5"}.fa-paperclip:before{content:"\f0c6"}.fa-save:before,.fa-floppy-o:before{content:"\f0c7"}.fa-square:before{content:"\f0c8"}.fa-navicon:before,.fa-reorder:before,.fa-bars:before{content:"\f0c9"}.fa-list-ul:before{content:"\f0ca"}.fa-list-ol:
 before{content:"\f0cb"}.fa-strikethrough:before{content:"\f0cc"}.fa-underline:before{content:"\f0cd"}.fa-table:before{content:"\f0ce"}.fa-magic:before{content:"\f0d0"}.fa-truck:before{content:"\f0d1"}.fa-pinterest:before{content:"\f0d2"}.fa-pinterest-square:before{content:"\f0d3"}.fa-google-plus-square:before{content:"\f0d4"}.fa-google-plus:before{content:"\f0d5"}.fa-money:before{content:"\f0d6"}.fa-caret-down:before{content:"\f0d7"}.fa-caret-up:before{content:"\f0d8"}.fa-caret-left:before{content:"\f0d9"}.fa-caret-right:before{content:"\f0da"}.fa-columns:before{content:"\f0db"}.fa-unsorted:before,.fa-sort:before{content:"\f0dc"}.fa-sort-down:before,.fa-sort-desc:before{content:"\f0dd"}.fa-sort-up:before,.fa-sort-asc:before{content:"\f0de"}.fa-envelope:before{content:"\f0e0"}.fa-linkedin:before{content:"\f0e1"}.fa-rotate-left:before,.fa-undo:before{content:"\f0e2"}.fa-legal:before,.fa-gavel:before{content:"\f0e3"}.fa-dashboard:before,.fa-tachometer:before{content:"\f0e4"}.fa-comment
 -o:before{content:"\f0e5"}.fa-comments-o:before{content:"\f0e6"}.fa-flash:before,.fa-bolt:before{content:"\f0e7"}.fa-sitemap:before{content:"\f0e8"}.fa-umbrella:before{content:"\f0e9"}.fa-paste:before,.fa-clipboard:before{content:"\f0ea"}.fa-lightbulb-o:before{content:"\f0eb"}.fa-exchange:before{content:"\f0ec"}.fa-cloud-download:before{content:"\f0ed"}.fa-cloud-upload:before{content:"\f0ee"}.fa-user-md:before{content:"\f0f0"}.fa-stethoscope:before{content:"\f0f1"}.fa-suitcase:before{content:"\f0f2"}.fa-bell-o:before{content:"\f0a2"}.fa-coffee:before{content:"\f0f4"}.fa-cutlery:before{content:"\f0f5"}.fa-file-text-o:before{content:"\f0f6"}.fa-building-o:before{content:"\f0f7"}.fa-hospital-o:before{content:"\f0f8"}.fa-ambulance:before{content:"\f0f9"}.fa-medkit:before{content:"\f0fa"}.fa-fighter-jet:before{content:"\f0fb"}.fa-beer:before{content:"\f0fc"}.fa-h-square:before{content:"\f0fd"}.fa-plus-square:before{content:"\f0fe"}.fa-angle-double-left:before{content:"\f100"}.fa-angle-do
 uble-right:before{content:"\f101"}.fa-angle-double-up:before{content:"\f102"}.fa-angle-double-down:before{content:"\f103"}.fa-angle-left:before{content:"\f104"}.fa-angle-right:before{content:"\f105"}.fa-angle-up:before{content:"\f106"}.fa-angle-down:before{content:"\f107"}.fa-desktop:before{content:"\f108"}.fa-laptop:before{content:"\f109"}.fa-tablet:before{content:"\f10a"}.fa-mobile-phone:before,.fa-mobile:before{content:"\f10b"}.fa-circle-o:before{content:"\f10c"}.fa-quote-left:before{content:"\f10d"}.fa-quote-right:before{content:"\f10e"}.fa-spinner:before{content:"\f110"}.fa-circle:before{content:"\f111"}.fa-mail-reply:before,.fa-reply:before{content:"\f112"}.fa-github-alt:before{content:"\f113"}.fa-folder-o:before{content:"\f114"}.fa-folder-open-o:before{content:"\f115"}.fa-smile-o:before{content:"\f118"}.fa-frown-o:before{content:"\f119"}.fa-meh-o:before{content:"\f11a"}.fa-gamepad:before{content:"\f11b"}.fa-keyboard-o:before{content:"\f11c"}.fa-flag-o:before{content:"\f11d"}.
 fa-flag-checkered:before{content:"\f11e"}.fa-terminal:before{content:"\f120"}.fa-code:before{content:"\f121"}.fa-mail-reply-all:before,.fa-reply-all:before{content:"\f122"}.fa-star-half-empty:before,.fa-star-half-full:before,.fa-star-half-o:before{content:"\f123"}.fa-location-arrow:before{content:"\f124"}.fa-crop:before{content:"\f125"}.fa-code-fork:before{content:"\f126"}.fa-unlink:before,.fa-chain-broken:before{content:"\f127"}.fa-question:before{content:"\f128"}.fa-info:before{content:"\f129"}.fa-exclamation:before{content:"\f12a"}.fa-superscript:before{content:"\f12b"}.fa-subscript:before{content:"\f12c"}.fa-eraser:before{content:"\f12d"}.fa-puzzle-piece:before{content:"\f12e"}.fa-microphone:before{content:"\f130"}.fa-microphone-slash:before{content:"\f131"}.fa-shield:before{content:"\f132"}.fa-calendar-o:before{content:"\f133"}.fa-fire-extinguisher:before{content:"\f134"}.fa-rocket:before{content:"\f135"}.fa-maxcdn:before{content:"\f136"}.fa-chevron-circle-left:before{content:"
 \f137"}.fa-chevron-circle-right:before{content:"\f138"}.fa-chevron-circle-up:before{content:"\f139"}.fa-chevron-circle-down:before{content:"\f13a"}.fa-html5:before{content:"\f13b"}.fa-css3:before{content:"\f13c"}.fa-anchor:before{content:"\f13d"}.fa-unlock-alt:before{content:"\f13e"}.fa-bullseye:before{content:"\f140"}.fa-ellipsis-h:before{content:"\f141"}.fa-ellipsis-v:before{content:"\f142"}.fa-rss-square:before{content:"\f143"}.fa-play-circle:before{content:"\f144"}.fa-ticket:before{content:"\f145"}.fa-minus-square:before{content:"\f146"}.fa-minus-square-o:before{content:"\f147"}.fa-level-up:before{content:"\f148"}.fa-level-down:before{content:"\f149"}.fa-check-square:before{content:"\f14a"}.fa-pencil-square:before{content:"\f14b"}.fa-external-link-square:before{content:"\f14c"}.fa-share-square:before{content:"\f14d"}.fa-compass:before{content:"\f14e"}.fa-toggle-down:before,.fa-caret-square-o-down:before{content:"\f150"}.fa-toggle-up:before,.fa-caret-square-o-up:before{content:"\
 f151"}.fa-toggle-right:before,.fa-caret-square-o-right:before{content:"\f152"}.fa-euro:before,.fa-eur:before{content:"\f153"}.fa-gbp:before{content:"\f154"}.fa-dollar:before,.fa-usd:before{content:"\f155"}.fa-rupee:before,.fa-inr:before{content:"\f156"}.fa-cny:before,.fa-rmb:before,.fa-yen:before,.fa-jpy:before{content:"\f157"}.fa-ruble:before,.fa-rouble:before,.fa-rub:before{content:"\f158"}.fa-won:before,.fa-krw:before{content:"\f159"}.fa-bitcoin:before,.fa-btc:before{content:"\f15a"}.fa-file:before{content:"\f15b"}.fa-file-text:before{content:"\f15c"}.fa-sort-alpha-asc:before{content:"\f15d"}.fa-sort-alpha-desc:before{content:"\f15e"}.fa-sort-amount-asc:before{content:"\f160"}.fa-sort-amount-desc:before{content:"\f161"}.fa-sort-numeric-asc:before{content:"\f162"}.fa-sort-numeric-desc:before{content:"\f163"}.fa-thumbs-up:before{content:"\f164"}.fa-thumbs-down:before{content:"\f165"}.fa-youtube-square:before{content:"\f166"}.fa-youtube:before{content:"\f167"}.fa-xing:before{content
 :"\f168"}.fa-xing-square:before{content:"\f169"}.fa-youtube-play:before{content:"\f16a"}.fa-dropbox:before{content:"\f16b"}.fa-stack-overflow:before{content:"\f16c"}.fa-instagram:before{content:"\f16d"}.fa-flickr:before{content:"\f16e"}.fa-adn:before{content:"\f170"}.fa-bitbucket:before{content:"\f171"}.fa-bitbucket-square:before{content:"\f172"}.fa-tumblr:before{content:"\f173"}.fa-tumblr-square:before{content:"\f174"}.fa-long-arrow-down:before{content:"\f175"}.fa-long-arrow-up:before{content:"\f176"}.fa-long-arrow-left:before{content:"\f177"}.fa-long-arrow-right:before{content:"\f178"}.fa-apple:before{content:"\f179"}.fa-windows:before{content:"\f17a"}.fa-android:before{content:"\f17b"}.fa-linux:before{content:"\f17c"}.fa-dribbble:before{content:"\f17d"}.fa-skype:before{content:"\f17e"}.fa-foursquare:before{content:"\f180"}.fa-trello:before{content:"\f181"}.fa-female:before{content:"\f182"}.fa-male:before{content:"\f183"}.fa-gittip:before{content:"\f184"}.fa-sun-o:before{content:"
 \f185"}.fa-moon-o:before{content:"\f186"}.fa-archive:before{content:"\f187"}.fa-bug:before{content:"\f188"}.fa-vk:before{content:"\f189"}.fa-weibo:before{content:"\f18a"}.fa-renren:before{content:"\f18b"}.fa-pagelines:before{content:"\f18c"}.fa-stack-exchange:before{content:"\f18d"}.fa-arrow-circle-o-right:before{content:"\f18e"}.fa-arrow-circle-o-left:before{content:"\f190"}.fa-toggle-left:before,.fa-caret-square-o-left:before{content:"\f191"}.fa-dot-circle-o:before{content:"\f192"}.fa-wheelchair:before{content:"\f193"}.fa-vimeo-square:before{content:"\f194"}.fa-turkish-lira:before,.fa-try:before{content:"\f195"}.fa-plus-square-o:before{content:"\f196"}.fa-space-shuttle:before{content:"\f197"}.fa-slack:before{content:"\f198"}.fa-envelope-square:before{content:"\f199"}.fa-wordpress:before{content:"\f19a"}.fa-openid:before{content:"\f19b"}.fa-institution:before,.fa-bank:before,.fa-university:before{content:"\f19c"}.fa-mortar-board:before,.fa-graduation-cap:before{content:"\f19d"}.fa-
 yahoo:before{content:"\f19e"}.fa-google:before{content:"\f1a0"}.fa-reddit:before{content:"\f1a1"}.fa-reddit-square:before{content:"\f1a2"}.fa-stumbleupon-circle:before{content:"\f1a3"}.fa-stumbleupon:before{content:"\f1a4"}.fa-delicious:before{content:"\f1a5"}.fa-digg:before{content:"\f1a6"}.fa-pied-piper-square:before,.fa-pied-piper:before{content:"\f1a7"}.fa-pied-piper-alt:before{content:"\f1a8"}.fa-drupal:before{content:"\f1a9"}.fa-joomla:before{content:"\f1aa"}.fa-language:before{content:"\f1ab"}.fa-fax:before{content:"\f1ac"}.fa-building:before{content:"\f1ad"}.fa-child:before{content:"\f1ae"}.fa-paw:before{content:"\f1b0"}.fa-spoon:before{content:"\f1b1"}.fa-cube:before{content:"\f1b2"}.fa-cubes:before{content:"\f1b3"}.fa-behance:before{content:"\f1b4"}.fa-behance-square:before{content:"\f1b5"}.fa-steam:before{content:"\f1b6"}.fa-steam-square:before{content:"\f1b7"}.fa-recycle:before{content:"\f1b8"}.fa-automobile:before,.fa-car:before{content:"\f1b9"}.fa-cab:before,.fa-taxi:b
 efore{content:"\f1ba"}.fa-tree:before{content:"\f1bb"}.fa-spotify:before{content:"\f1bc"}.fa-deviantart:before{content:"\f1bd"}.fa-soundcloud:before{content:"\f1be"}.fa-database:before{content:"\f1c0"}.fa-file-pdf-o:before{content:"\f1c1"}.fa-file-word-o:before{content:"\f1c2"}.fa-file-excel-o:before{content:"\f1c3"}.fa-file-powerpoint-o:before{content:"\f1c4"}.fa-file-photo-o:before,.fa-file-picture-o:before,.fa-file-image-o:before{content:"\f1c5"}.fa-file-zip-o:before,.fa-file-archive-o:before{content:"\f1c6"}.fa-file-sound-o:before,.fa-file-audio-o:before{content:"\f1c7"}.fa-file-movie-o:before,.fa-file-video-o:before{content:"\f1c8"}.fa-file-code-o:before{content:"\f1c9"}.fa-vine:before{content:"\f1ca"}.fa-codepen:before{content:"\f1cb"}.fa-jsfiddle:before{content:"\f1cc"}.fa-life-bouy:before,.fa-life-saver:before,.fa-support:before,.fa-life-ring:before{content:"\f1cd"}.fa-circle-o-notch:before{content:"\f1ce"}.fa-ra:before,.fa-rebel:before{content:"\f1d0"}.fa-ge:before,.fa-empi
 re:before{content:"\f1d1"}.fa-git-square:before{content:"\f1d2"}.fa-git:before{content:"\f1d3"}.fa-hacker-news:before{content:"\f1d4"}.fa-tencent-weibo:before{content:"\f1d5"}.fa-qq:before{content:"\f1d6"}.fa-wechat:before,.fa-weixin:before{content:"\f1d7"}.fa-send:before,.fa-paper-plane:before{content:"\f1d8"}.fa-send-o:before,.fa-paper-plane-o:before{content:"\f1d9"}.fa-history:before{content:"\f1da"}.fa-circle-thin:before{content:"\f1db"}.fa-header:before{content:"\f1dc"}.fa-paragraph:before{content:"\f1dd"}.fa-sliders:before{content:"\f1de"}.fa-share-alt:before{content:"\f1e0"}.fa-share-alt-square:before{content:"\f1e1"}.fa-bomb:before{content:"\f1e2"}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/samza/blob/b8058af0/docs/css/google-fonts.css
----------------------------------------------------------------------
diff --git a/docs/css/google-fonts.css b/docs/css/google-fonts.css
new file mode 100644
index 0000000..83b941b
--- /dev/null
+++ b/docs/css/google-fonts.css
@@ -0,0 +1,72 @@
+/* latin-ext */
+@font-face {
+  font-family: 'Barlow';
+  font-style: normal;
+  font-weight: 400;
+  src: local('Barlow Regular'), local('Barlow-Regular'), url(https://fonts.gstatic.com/s/barlow/v1/7cHpv4kjgoGqM7E_Ass52Hs.woff2) format('woff2');
+  unicode-range: U+0100-024F, U+0259, U+1E00-1EFF, U+2020, U+20A0-20AB, U+20AD-20CF, U+2113, U+2C60-2C7F, U+A720-A7FF;
+}
+/* latin */
+@font-face {
+  font-family: 'Barlow';
+  font-style: normal;
+  font-weight: 400;
+  src: local('Barlow Regular'), local('Barlow-Regular'), url(https://fonts.gstatic.com/s/barlow/v1/7cHpv4kjgoGqM7E_DMs5.woff2) format('woff2');
+  unicode-range: U+0000-00FF, U+0131, U+0152-0153, U+02BB-02BC, U+02C6, U+02DA, U+02DC, U+2000-206F, U+2074, U+20AC, U+2122, U+2191, U+2193, U+2212, U+2215, U+FEFF, U+FFFD;
+}
+/* cyrillic-ext */
+@font-face {
+  font-family: 'Cuprum';
+  font-style: normal;
+  font-weight: 400;
+  src: local('Cuprum Regular'), local('Cuprum-Regular'), url(https://fonts.gstatic.com/s/cuprum/v9/dg4k_pLmvrkcOkBNK-tCt70.woff2) format('woff2');
+  unicode-range: U+0460-052F, U+1C80-1C88, U+20B4, U+2DE0-2DFF, U+A640-A69F, U+FE2E-FE2F;
+}
+/* cyrillic */
+@font-face {
+  font-family: 'Cuprum';
+  font-style: normal;
+  font-weight: 400;
+  src: local('Cuprum Regular'), local('Cuprum-Regular'), url(https://fonts.gstatic.com/s/cuprum/v9/dg4k_pLmvrkcOkBNIutCt70.woff2) format('woff2');
+  unicode-range: U+0400-045F, U+0490-0491, U+04B0-04B1, U+2116;
+}
+/* vietnamese */
+@font-face {
+  font-family: 'Cuprum';
+  font-style: normal;
+  font-weight: 400;
+  src: local('Cuprum Regular'), local('Cuprum-Regular'), url(https://fonts.gstatic.com/s/cuprum/v9/dg4k_pLmvrkcOkBNKetCt70.woff2) format('woff2');
+  unicode-range: U+0102-0103, U+0110-0111, U+1EA0-1EF9, U+20AB;
+}
+/* latin-ext */
+@font-face {
+  font-family: 'Cuprum';
+  font-style: normal;
+  font-weight: 400;
+  src: local('Cuprum Regular'), local('Cuprum-Regular'), url(https://fonts.gstatic.com/s/cuprum/v9/dg4k_pLmvrkcOkBNKOtCt70.woff2) format('woff2');
+  unicode-range: U+0100-024F, U+0259, U+1E00-1EFF, U+2020, U+20A0-20AB, U+20AD-20CF, U+2113, U+2C60-2C7F, U+A720-A7FF;
+}
+/* latin */
+@font-face {
+  font-family: 'Cuprum';
+  font-style: normal;
+  font-weight: 400;
+  src: local('Cuprum Regular'), local('Cuprum-Regular'), url(https://fonts.gstatic.com/s/cuprum/v9/dg4k_pLmvrkcOkBNJutC.woff2) format('woff2');
+  unicode-range: U+0000-00FF, U+0131, U+0152-0153, U+02BB-02BC, U+02C6, U+02DA, U+02DC, U+2000-206F, U+2074, U+20AC, U+2122, U+2191, U+2193, U+2212, U+2215, U+FEFF, U+FFFD;
+}
+/* arabic */
+@font-face {
+  font-family: 'Tajawal';
+  font-style: normal;
+  font-weight: 400;
+  src: local('Tajawal'), local('Tajawal-Regular'), url(https://fonts.gstatic.com/s/tajawal/v1/Iura6YBj_oCad4k1nzSBC45I.woff2) format('woff2');
+  unicode-range: U+0600-06FF, U+200C-200E, U+2010-2011, U+204F, U+2E41, U+FB50-FDFF, U+FE80-FEFC;
+}
+/* latin */
+@font-face {
+  font-family: 'Tajawal';
+  font-style: normal;
+  font-weight: 400;
+  src: local('Tajawal'), local('Tajawal-Regular'), url(https://fonts.gstatic.com/s/tajawal/v1/Iura6YBj_oCad4k1nzGBCw.woff2) format('woff2');
+  unicode-range: U+0000-00FF, U+0131, U+0152-0153, U+02BB-02BC, U+02C6, U+02DA, U+02DC, U+2000-206F, U+2074, U+20AC, U+2122, U+2191, U+2193, U+2212, U+2215, U+FEFF, U+FFFD;
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/b8058af0/docs/css/ionicons.min.css
----------------------------------------------------------------------
diff --git a/docs/css/ionicons.min.css b/docs/css/ionicons.min.css
new file mode 100644
index 0000000..16cfdcc
--- /dev/null
+++ b/docs/css/ionicons.min.css
@@ -0,0 +1,11 @@
+/*!
+  Ionicons, v4.4.1
+  Created by Ben Sperry for the Ionic Framework, http://ionicons.com/
+  https://twitter.com/benjsperry  https://twitter.com/ionicframework
+  MIT License: https://github.com/driftyco/ionicons
+
+  Android-style icons originally built by Google’s
+  Material Design Icons: https://github.com/google/material-design-icons
+  used under CC BY http://creativecommons.org/licenses/by/4.0/
+  Modified icons to fit ionicon’s grid from original.
+*/@font-face{font-family:"Ionicons";src:url("../fonts/ionicons.eot?v=4.4.1");src:url("../fonts/ionicons.eot?v=4.4.1#iefix") format("embedded-opentype"),url("../fonts/ionicons.woff2?v=4.4.1") format("woff2"),url("../fonts/ionicons.woff?v=4.4.1") format("woff"),url("../fonts/ionicons.ttf?v=4.4.1") format("truetype"),url("../fonts/ionicons.svg?v=4.4.1#Ionicons") format("svg");font-weight:normal;font-style:normal}.ion,.ionicons,.ion-ios-add:before,.ion-ios-add-circle:before,.ion-ios-add-circle-outline:before,.ion-ios-airplane:before,.ion-ios-alarm:before,.ion-ios-albums:before,.ion-ios-alert:before,.ion-ios-american-football:before,.ion-ios-analytics:before,.ion-ios-aperture:before,.ion-ios-apps:before,.ion-ios-appstore:before,.ion-ios-archive:before,.ion-ios-arrow-back:before,.ion-ios-arrow-down:before,.ion-ios-arrow-dropdown:before,.ion-ios-arrow-dropdown-circle:before,.ion-ios-arrow-dropleft:before,.ion-ios-arrow-dropleft-circle:before,.ion-ios-arrow-dropright:before,.ion-ios-arrow-d
 ropright-circle:before,.ion-ios-arrow-dropup:before,.ion-ios-arrow-dropup-circle:before,.ion-ios-arrow-forward:before,.ion-ios-arrow-round-back:before,.ion-ios-arrow-round-down:before,.ion-ios-arrow-round-forward:before,.ion-ios-arrow-round-up:before,.ion-ios-arrow-up:before,.ion-ios-at:before,.ion-ios-attach:before,.ion-ios-backspace:before,.ion-ios-barcode:before,.ion-ios-baseball:before,.ion-ios-basket:before,.ion-ios-basketball:before,.ion-ios-battery-charging:before,.ion-ios-battery-dead:before,.ion-ios-battery-full:before,.ion-ios-beaker:before,.ion-ios-bed:before,.ion-ios-beer:before,.ion-ios-bicycle:before,.ion-ios-bluetooth:before,.ion-ios-boat:before,.ion-ios-body:before,.ion-ios-bonfire:before,.ion-ios-book:before,.ion-ios-bookmark:before,.ion-ios-bookmarks:before,.ion-ios-bowtie:before,.ion-ios-briefcase:before,.ion-ios-browsers:before,.ion-ios-brush:before,.ion-ios-bug:before,.ion-ios-build:before,.ion-ios-bulb:before,.ion-ios-bus:before,.ion-ios-business:before,.ion-io
 s-cafe:before,.ion-ios-calculator:before,.ion-ios-calendar:before,.ion-ios-call:before,.ion-ios-camera:before,.ion-ios-car:before,.ion-ios-card:before,.ion-ios-cart:before,.ion-ios-cash:before,.ion-ios-cellular:before,.ion-ios-chatboxes:before,.ion-ios-chatbubbles:before,.ion-ios-checkbox:before,.ion-ios-checkbox-outline:before,.ion-ios-checkmark:before,.ion-ios-checkmark-circle:before,.ion-ios-checkmark-circle-outline:before,.ion-ios-clipboard:before,.ion-ios-clock:before,.ion-ios-close:before,.ion-ios-close-circle:before,.ion-ios-close-circle-outline:before,.ion-ios-cloud:before,.ion-ios-cloud-circle:before,.ion-ios-cloud-done:before,.ion-ios-cloud-download:before,.ion-ios-cloud-outline:before,.ion-ios-cloud-upload:before,.ion-ios-cloudy:before,.ion-ios-cloudy-night:before,.ion-ios-code:before,.ion-ios-code-download:before,.ion-ios-code-working:before,.ion-ios-cog:before,.ion-ios-color-fill:before,.ion-ios-color-filter:before,.ion-ios-color-palette:before,.ion-ios-color-wand:befor
 e,.ion-ios-compass:before,.ion-ios-construct:before,.ion-ios-contact:before,.ion-ios-contacts:before,.ion-ios-contract:before,.ion-ios-contrast:before,.ion-ios-copy:before,.ion-ios-create:before,.ion-ios-crop:before,.ion-ios-cube:before,.ion-ios-cut:before,.ion-ios-desktop:before,.ion-ios-disc:before,.ion-ios-document:before,.ion-ios-done-all:before,.ion-ios-download:before,.ion-ios-easel:before,.ion-ios-egg:before,.ion-ios-exit:before,.ion-ios-expand:before,.ion-ios-eye:before,.ion-ios-eye-off:before,.ion-ios-fastforward:before,.ion-ios-female:before,.ion-ios-filing:before,.ion-ios-film:before,.ion-ios-finger-print:before,.ion-ios-fitness:before,.ion-ios-flag:before,.ion-ios-flame:before,.ion-ios-flash:before,.ion-ios-flash-off:before,.ion-ios-flashlight:before,.ion-ios-flask:before,.ion-ios-flower:before,.ion-ios-folder:before,.ion-ios-folder-open:before,.ion-ios-football:before,.ion-ios-funnel:before,.ion-ios-gift:before,.ion-ios-git-branch:before,.ion-ios-git-commit:before,.ion-
 ios-git-compare:before,.ion-ios-git-merge:before,.ion-ios-git-network:before,.ion-ios-git-pull-request:before,.ion-ios-glasses:before,.ion-ios-globe:before,.ion-ios-grid:before,.ion-ios-hammer:before,.ion-ios-hand:before,.ion-ios-happy:before,.ion-ios-headset:before,.ion-ios-heart:before,.ion-ios-heart-dislike:before,.ion-ios-heart-empty:before,.ion-ios-heart-half:before,.ion-ios-help:before,.ion-ios-help-buoy:before,.ion-ios-help-circle:before,.ion-ios-help-circle-outline:before,.ion-ios-home:before,.ion-ios-hourglass:before,.ion-ios-ice-cream:before,.ion-ios-image:before,.ion-ios-images:before,.ion-ios-infinite:before,.ion-ios-information:before,.ion-ios-information-circle:before,.ion-ios-information-circle-outline:before,.ion-ios-jet:before,.ion-ios-journal:before,.ion-ios-key:before,.ion-ios-keypad:before,.ion-ios-laptop:before,.ion-ios-leaf:before,.ion-ios-link:before,.ion-ios-list:before,.ion-ios-list-box:before,.ion-ios-locate:before,.ion-ios-lock:before,.ion-ios-log-in:befor
 e,.ion-ios-log-out:before,.ion-ios-magnet:before,.ion-ios-mail:before,.ion-ios-mail-open:before,.ion-ios-mail-unread:before,.ion-ios-male:before,.ion-ios-man:before,.ion-ios-map:before,.ion-ios-medal:before,.ion-ios-medical:before,.ion-ios-medkit:before,.ion-ios-megaphone:before,.ion-ios-menu:before,.ion-ios-mic:before,.ion-ios-mic-off:before,.ion-ios-microphone:before,.ion-ios-moon:before,.ion-ios-more:before,.ion-ios-move:before,.ion-ios-musical-note:before,.ion-ios-musical-notes:before,.ion-ios-navigate:before,.ion-ios-notifications:before,.ion-ios-notifications-off:before,.ion-ios-notifications-outline:before,.ion-ios-nuclear:before,.ion-ios-nutrition:before,.ion-ios-open:before,.ion-ios-options:before,.ion-ios-outlet:before,.ion-ios-paper:before,.ion-ios-paper-plane:before,.ion-ios-partly-sunny:before,.ion-ios-pause:before,.ion-ios-paw:before,.ion-ios-people:before,.ion-ios-person:before,.ion-ios-person-add:before,.ion-ios-phone-landscape:before,.ion-ios-phone-portrait:before,.
 ion-ios-photos:before,.ion-ios-pie:before,.ion-ios-pin:before,.ion-ios-pint:before,.ion-ios-pizza:before,.ion-ios-planet:before,.ion-ios-play:before,.ion-ios-play-circle:before,.ion-ios-podium:before,.ion-ios-power:before,.ion-ios-pricetag:before,.ion-ios-pricetags:before,.ion-ios-print:before,.ion-ios-pulse:before,.ion-ios-qr-scanner:before,.ion-ios-quote:before,.ion-ios-radio:before,.ion-ios-radio-button-off:before,.ion-ios-radio-button-on:before,.ion-ios-rainy:before,.ion-ios-recording:before,.ion-ios-redo:before,.ion-ios-refresh:before,.ion-ios-refresh-circle:before,.ion-ios-remove:before,.ion-ios-remove-circle:before,.ion-ios-remove-circle-outline:before,.ion-ios-reorder:before,.ion-ios-repeat:before,.ion-ios-resize:before,.ion-ios-restaurant:before,.ion-ios-return-left:before,.ion-ios-return-right:before,.ion-ios-reverse-camera:before,.ion-ios-rewind:before,.ion-ios-ribbon:before,.ion-ios-rocket:before,.ion-ios-rose:before,.ion-ios-sad:before,.ion-ios-save:before,.ion-ios-scho
 ol:before,.ion-ios-search:before,.ion-ios-send:before,.ion-ios-settings:before,.ion-ios-share:before,.ion-ios-share-alt:before,.ion-ios-shirt:before,.ion-ios-shuffle:before,.ion-ios-skip-backward:before,.ion-ios-skip-forward:before,.ion-ios-snow:before,.ion-ios-speedometer:before,.ion-ios-square:before,.ion-ios-square-outline:before,.ion-ios-star:before,.ion-ios-star-half:before,.ion-ios-star-outline:before,.ion-ios-stats:before,.ion-ios-stopwatch:before,.ion-ios-subway:before,.ion-ios-sunny:before,.ion-ios-swap:before,.ion-ios-switch:before,.ion-ios-sync:before,.ion-ios-tablet-landscape:before,.ion-ios-tablet-portrait:before,.ion-ios-tennisball:before,.ion-ios-text:before,.ion-ios-thermometer:before,.ion-ios-thumbs-down:before,.ion-ios-thumbs-up:before,.ion-ios-thunderstorm:before,.ion-ios-time:before,.ion-ios-timer:before,.ion-ios-today:before,.ion-ios-train:before,.ion-ios-transgender:before,.ion-ios-trash:before,.ion-ios-trending-down:before,.ion-ios-trending-up:before,.ion-ios-
 trophy:before,.ion-ios-tv:before,.ion-ios-umbrella:before,.ion-ios-undo:before,.ion-ios-unlock:before,.ion-ios-videocam:before,.ion-ios-volume-high:before,.ion-ios-volume-low:before,.ion-ios-volume-mute:before,.ion-ios-volume-off:before,.ion-ios-walk:before,.ion-ios-wallet:before,.ion-ios-warning:before,.ion-ios-watch:before,.ion-ios-water:before,.ion-ios-wifi:before,.ion-ios-wine:before,.ion-ios-woman:before,.ion-logo-android:before,.ion-logo-angular:before,.ion-logo-apple:before,.ion-logo-bitbucket:before,.ion-logo-bitcoin:before,.ion-logo-buffer:before,.ion-logo-chrome:before,.ion-logo-closed-captioning:before,.ion-logo-codepen:before,.ion-logo-css3:before,.ion-logo-designernews:before,.ion-logo-dribbble:before,.ion-logo-dropbox:before,.ion-logo-euro:before,.ion-logo-facebook:before,.ion-logo-flickr:before,.ion-logo-foursquare:before,.ion-logo-freebsd-devil:before,.ion-logo-game-controller-a:before,.ion-logo-game-controller-b:before,.ion-logo-github:before,.ion-logo-google:before
 ,.ion-logo-googleplus:before,.ion-logo-hackernews:before,.ion-logo-html5:before,.ion-logo-instagram:before,.ion-logo-ionic:before,.ion-logo-ionitron:before,.ion-logo-javascript:before,.ion-logo-linkedin:before,.ion-logo-markdown:before,.ion-logo-model-s:before,.ion-logo-no-smoking:before,.ion-logo-nodejs:before,.ion-logo-npm:before,.ion-logo-octocat:before,.ion-logo-pinterest:before,.ion-logo-playstation:before,.ion-logo-polymer:before,.ion-logo-python:before,.ion-logo-reddit:before,.ion-logo-rss:before,.ion-logo-sass:before,.ion-logo-skype:before,.ion-logo-slack:before,.ion-logo-snapchat:before,.ion-logo-steam:before,.ion-logo-tumblr:before,.ion-logo-tux:before,.ion-logo-twitch:before,.ion-logo-twitter:before,.ion-logo-usd:before,.ion-logo-vimeo:before,.ion-logo-vk:before,.ion-logo-whatsapp:before,.ion-logo-windows:before,.ion-logo-wordpress:before,.ion-logo-xbox:before,.ion-logo-xing:before,.ion-logo-yahoo:before,.ion-logo-yen:before,.ion-logo-youtube:before,.ion-md-add:before,.io
 n-md-add-circle:before,.ion-md-add-circle-outline:before,.ion-md-airplane:before,.ion-md-alarm:before,.ion-md-albums:before,.ion-md-alert:before,.ion-md-american-football:before,.ion-md-analytics:before,.ion-md-aperture:before,.ion-md-apps:before,.ion-md-appstore:before,.ion-md-archive:before,.ion-md-arrow-back:before,.ion-md-arrow-down:before,.ion-md-arrow-dropdown:before,.ion-md-arrow-dropdown-circle:before,.ion-md-arrow-dropleft:before,.ion-md-arrow-dropleft-circle:before,.ion-md-arrow-dropright:before,.ion-md-arrow-dropright-circle:before,.ion-md-arrow-dropup:before,.ion-md-arrow-dropup-circle:before,.ion-md-arrow-forward:before,.ion-md-arrow-round-back:before,.ion-md-arrow-round-down:before,.ion-md-arrow-round-forward:before,.ion-md-arrow-round-up:before,.ion-md-arrow-up:before,.ion-md-at:before,.ion-md-attach:before,.ion-md-backspace:before,.ion-md-barcode:before,.ion-md-baseball:before,.ion-md-basket:before,.ion-md-basketball:before,.ion-md-battery-charging:before,.ion-md-bat
 tery-dead:before,.ion-md-battery-full:before,.ion-md-beaker:before,.ion-md-bed:before,.ion-md-beer:before,.ion-md-bicycle:before,.ion-md-bluetooth:before,.ion-md-boat:before,.ion-md-body:before,.ion-md-bonfire:before,.ion-md-book:before,.ion-md-bookmark:before,.ion-md-bookmarks:before,.ion-md-bowtie:before,.ion-md-briefcase:before,.ion-md-browsers:before,.ion-md-brush:before,.ion-md-bug:before,.ion-md-build:before,.ion-md-bulb:before,.ion-md-bus:before,.ion-md-business:before,.ion-md-cafe:before,.ion-md-calculator:before,.ion-md-calendar:before,.ion-md-call:before,.ion-md-camera:before,.ion-md-car:before,.ion-md-card:before,.ion-md-cart:before,.ion-md-cash:before,.ion-md-cellular:before,.ion-md-chatboxes:before,.ion-md-chatbubbles:before,.ion-md-checkbox:before,.ion-md-checkbox-outline:before,.ion-md-checkmark:before,.ion-md-checkmark-circle:before,.ion-md-checkmark-circle-outline:before,.ion-md-clipboard:before,.ion-md-clock:before,.ion-md-close:before,.ion-md-close-circle:before,.
 ion-md-close-circle-outline:before,.ion-md-cloud:before,.ion-md-cloud-circle:before,.ion-md-cloud-done:before,.ion-md-cloud-download:before,.ion-md-cloud-outline:before,.ion-md-cloud-upload:before,.ion-md-cloudy:before,.ion-md-cloudy-night:before,.ion-md-code:before,.ion-md-code-download:before,.ion-md-code-working:before,.ion-md-cog:before,.ion-md-color-fill:before,.ion-md-color-filter:before,.ion-md-color-palette:before,.ion-md-color-wand:before,.ion-md-compass:before,.ion-md-construct:before,.ion-md-contact:before,.ion-md-contacts:before,.ion-md-contract:before,.ion-md-contrast:before,.ion-md-copy:before,.ion-md-create:before,.ion-md-crop:before,.ion-md-cube:before,.ion-md-cut:before,.ion-md-desktop:before,.ion-md-disc:before,.ion-md-document:before,.ion-md-done-all:before,.ion-md-download:before,.ion-md-easel:before,.ion-md-egg:before,.ion-md-exit:before,.ion-md-expand:before,.ion-md-eye:before,.ion-md-eye-off:before,.ion-md-fastforward:before,.ion-md-female:before,.ion-md-filin
 g:before,.ion-md-film:before,.ion-md-finger-print:before,.ion-md-fitness:before,.ion-md-flag:before,.ion-md-flame:before,.ion-md-flash:before,.ion-md-flash-off:before,.ion-md-flashlight:before,.ion-md-flask:before,.ion-md-flower:before,.ion-md-folder:before,.ion-md-folder-open:before,.ion-md-football:before,.ion-md-funnel:before,.ion-md-gift:before,.ion-md-git-branch:before,.ion-md-git-commit:before,.ion-md-git-compare:before,.ion-md-git-merge:before,.ion-md-git-network:before,.ion-md-git-pull-request:before,.ion-md-glasses:before,.ion-md-globe:before,.ion-md-grid:before,.ion-md-hammer:before,.ion-md-hand:before,.ion-md-happy:before,.ion-md-headset:before,.ion-md-heart:before,.ion-md-heart-dislike:before,.ion-md-heart-empty:before,.ion-md-heart-half:before,.ion-md-help:before,.ion-md-help-buoy:before,.ion-md-help-circle:before,.ion-md-help-circle-outline:before,.ion-md-home:before,.ion-md-hourglass:before,.ion-md-ice-cream:before,.ion-md-image:before,.ion-md-images:before,.ion-md-in
 finite:before,.ion-md-information:before,.ion-md-information-circle:before,.ion-md-information-circle-outline:before,.ion-md-jet:before,.ion-md-journal:before,.ion-md-key:before,.ion-md-keypad:before,.ion-md-laptop:before,.ion-md-leaf:before,.ion-md-link:before,.ion-md-list:before,.ion-md-list-box:before,.ion-md-locate:before,.ion-md-lock:before,.ion-md-log-in:before,.ion-md-log-out:before,.ion-md-magnet:before,.ion-md-mail:before,.ion-md-mail-open:before,.ion-md-mail-unread:before,.ion-md-male:before,.ion-md-man:before,.ion-md-map:before,.ion-md-medal:before,.ion-md-medical:before,.ion-md-medkit:before,.ion-md-megaphone:before,.ion-md-menu:before,.ion-md-mic:before,.ion-md-mic-off:before,.ion-md-microphone:before,.ion-md-moon:before,.ion-md-more:before,.ion-md-move:before,.ion-md-musical-note:before,.ion-md-musical-notes:before,.ion-md-navigate:before,.ion-md-notifications:before,.ion-md-notifications-off:before,.ion-md-notifications-outline:before,.ion-md-nuclear:before,.ion-md-nu
 trition:before,.ion-md-open:before,.ion-md-options:before,.ion-md-outlet:before,.ion-md-paper:before,.ion-md-paper-plane:before,.ion-md-partly-sunny:before,.ion-md-pause:before,.ion-md-paw:before,.ion-md-people:before,.ion-md-person:before,.ion-md-person-add:before,.ion-md-phone-landscape:before,.ion-md-phone-portrait:before,.ion-md-photos:before,.ion-md-pie:before,.ion-md-pin:before,.ion-md-pint:before,.ion-md-pizza:before,.ion-md-planet:before,.ion-md-play:before,.ion-md-play-circle:before,.ion-md-podium:before,.ion-md-power:before,.ion-md-pricetag:before,.ion-md-pricetags:before,.ion-md-print:before,.ion-md-pulse:before,.ion-md-qr-scanner:before,.ion-md-quote:before,.ion-md-radio:before,.ion-md-radio-button-off:before,.ion-md-radio-button-on:before,.ion-md-rainy:before,.ion-md-recording:before,.ion-md-redo:before,.ion-md-refresh:before,.ion-md-refresh-circle:before,.ion-md-remove:before,.ion-md-remove-circle:before,.ion-md-remove-circle-outline:before,.ion-md-reorder:before,.ion-
 md-repeat:before,.ion-md-resize:before,.ion-md-restaurant:before,.ion-md-return-left:before,.ion-md-return-right:before,.ion-md-reverse-camera:before,.ion-md-rewind:before,.ion-md-ribbon:before,.ion-md-rocket:before,.ion-md-rose:before,.ion-md-sad:before,.ion-md-save:before,.ion-md-school:before,.ion-md-search:before,.ion-md-send:before,.ion-md-settings:before,.ion-md-share:before,.ion-md-share-alt:before,.ion-md-shirt:before,.ion-md-shuffle:before,.ion-md-skip-backward:before,.ion-md-skip-forward:before,.ion-md-snow:before,.ion-md-speedometer:before,.ion-md-square:before,.ion-md-square-outline:before,.ion-md-star:before,.ion-md-star-half:before,.ion-md-star-outline:before,.ion-md-stats:before,.ion-md-stopwatch:before,.ion-md-subway:before,.ion-md-sunny:before,.ion-md-swap:before,.ion-md-switch:before,.ion-md-sync:before,.ion-md-tablet-landscape:before,.ion-md-tablet-portrait:before,.ion-md-tennisball:before,.ion-md-text:before,.ion-md-thermometer:before,.ion-md-thumbs-down:before,.
 ion-md-thumbs-up:before,.ion-md-thunderstorm:before,.ion-md-time:before,.ion-md-timer:before,.ion-md-today:before,.ion-md-train:before,.ion-md-transgender:before,.ion-md-trash:before,.ion-md-trending-down:before,.ion-md-trending-up:before,.ion-md-trophy:before,.ion-md-tv:before,.ion-md-umbrella:before,.ion-md-undo:before,.ion-md-unlock:before,.ion-md-videocam:before,.ion-md-volume-high:before,.ion-md-volume-low:before,.ion-md-volume-mute:before,.ion-md-volume-off:before,.ion-md-walk:before,.ion-md-wallet:before,.ion-md-warning:before,.ion-md-watch:before,.ion-md-water:before,.ion-md-wifi:before,.ion-md-wine:before,.ion-md-woman:before{display:inline-block;font-family:"Ionicons";speak:none;font-style:normal;font-weight:normal;font-variant:normal;text-transform:none;text-rendering:auto;line-height:1;-webkit-font-smoothing:antialiased;-moz-osx-font-smoothing:grayscale}.ion-ios-add:before{content:"\f102"}.ion-ios-add-circle:before{content:"\f101"}.ion-ios-add-circle-outline:before{conte
 nt:"\f100"}.ion-ios-airplane:before{content:"\f137"}.ion-ios-alarm:before{content:"\f3c8"}.ion-ios-albums:before{content:"\f3ca"}.ion-ios-alert:before{content:"\f104"}.ion-ios-american-football:before{content:"\f106"}.ion-ios-analytics:before{content:"\f3ce"}.ion-ios-aperture:before{content:"\f108"}.ion-ios-apps:before{content:"\f10a"}.ion-ios-appstore:before{content:"\f10c"}.ion-ios-archive:before{content:"\f10e"}.ion-ios-arrow-back:before{content:"\f3cf"}.ion-ios-arrow-down:before{content:"\f3d0"}.ion-ios-arrow-dropdown:before{content:"\f110"}.ion-ios-arrow-dropdown-circle:before{content:"\f125"}.ion-ios-arrow-dropleft:before{content:"\f112"}.ion-ios-arrow-dropleft-circle:before{content:"\f129"}.ion-ios-arrow-dropright:before{content:"\f114"}.ion-ios-arrow-dropright-circle:before{content:"\f12b"}.ion-ios-arrow-dropup:before{content:"\f116"}.ion-ios-arrow-dropup-circle:before{content:"\f12d"}.ion-ios-arrow-forward:before{content:"\f3d1"}.ion-ios-arrow-round-back:before{content:"\f1
 17"}.ion-ios-arrow-round-down:before{content:"\f118"}.ion-ios-arrow-round-forward:before{content:"\f119"}.ion-ios-arrow-round-up:before{content:"\f11a"}.ion-ios-arrow-up:before{content:"\f3d8"}.ion-ios-at:before{content:"\f3da"}.ion-ios-attach:before{content:"\f11b"}.ion-ios-backspace:before{content:"\f11d"}.ion-ios-barcode:before{content:"\f3dc"}.ion-ios-baseball:before{content:"\f3de"}.ion-ios-basket:before{content:"\f11f"}.ion-ios-basketball:before{content:"\f3e0"}.ion-ios-battery-charging:before{content:"\f120"}.ion-ios-battery-dead:before{content:"\f121"}.ion-ios-battery-full:before{content:"\f122"}.ion-ios-beaker:before{content:"\f124"}.ion-ios-bed:before{content:"\f139"}.ion-ios-beer:before{content:"\f126"}.ion-ios-bicycle:before{content:"\f127"}.ion-ios-bluetooth:before{content:"\f128"}.ion-ios-boat:before{content:"\f12a"}.ion-ios-body:before{content:"\f3e4"}.ion-ios-bonfire:before{content:"\f12c"}.ion-ios-book:before{content:"\f3e8"}.ion-ios-bookmark:before{content:"\f12e"}
 .ion-ios-bookmarks:before{content:"\f3ea"}.ion-ios-bowtie:before{content:"\f130"}.ion-ios-briefcase:before{content:"\f3ee"}.ion-ios-browsers:before{content:"\f3f0"}.ion-ios-brush:before{content:"\f132"}.ion-ios-bug:before{content:"\f134"}.ion-ios-build:before{content:"\f136"}.ion-ios-bulb:before{content:"\f138"}.ion-ios-bus:before{content:"\f13a"}.ion-ios-business:before{content:"\f1a3"}.ion-ios-cafe:before{content:"\f13c"}.ion-ios-calculator:before{content:"\f3f2"}.ion-ios-calendar:before{content:"\f3f4"}.ion-ios-call:before{content:"\f13e"}.ion-ios-camera:before{content:"\f3f6"}.ion-ios-car:before{content:"\f140"}.ion-ios-card:before{content:"\f142"}.ion-ios-cart:before{content:"\f3f8"}.ion-ios-cash:before{content:"\f144"}.ion-ios-cellular:before{content:"\f13d"}.ion-ios-chatboxes:before{content:"\f3fa"}.ion-ios-chatbubbles:before{content:"\f146"}.ion-ios-checkbox:before{content:"\f148"}.ion-ios-checkbox-outline:before{content:"\f147"}.ion-ios-checkmark:before{content:"\f3ff"}.ion
 -ios-checkmark-circle:before{content:"\f14a"}.ion-ios-checkmark-circle-outline:before{content:"\f149"}.ion-ios-clipboard:before{content:"\f14c"}.ion-ios-clock:before{content:"\f403"}.ion-ios-close:before{content:"\f406"}.ion-ios-close-circle:before{content:"\f14e"}.ion-ios-close-circle-outline:before{content:"\f14d"}.ion-ios-cloud:before{content:"\f40c"}.ion-ios-cloud-circle:before{content:"\f152"}.ion-ios-cloud-done:before{content:"\f154"}.ion-ios-cloud-download:before{content:"\f408"}.ion-ios-cloud-outline:before{content:"\f409"}.ion-ios-cloud-upload:before{content:"\f40b"}.ion-ios-cloudy:before{content:"\f410"}.ion-ios-cloudy-night:before{content:"\f40e"}.ion-ios-code:before{content:"\f157"}.ion-ios-code-download:before{content:"\f155"}.ion-ios-code-working:before{content:"\f156"}.ion-ios-cog:before{content:"\f412"}.ion-ios-color-fill:before{content:"\f159"}.ion-ios-color-filter:before{content:"\f414"}.ion-ios-color-palette:before{content:"\f15b"}.ion-ios-color-wand:before{conten
 t:"\f416"}.ion-ios-compass:before{content:"\f15d"}.ion-ios-construct:before{content:"\f15f"}.ion-ios-contact:before{content:"\f41a"}.ion-ios-contacts:before{content:"\f161"}.ion-ios-contract:before{content:"\f162"}.ion-ios-contrast:before{content:"\f163"}.ion-ios-copy:before{content:"\f41c"}.ion-ios-create:before{content:"\f165"}.ion-ios-crop:before{content:"\f41e"}.ion-ios-cube:before{content:"\f168"}.ion-ios-cut:before{content:"\f16a"}.ion-ios-desktop:before{content:"\f16c"}.ion-ios-disc:before{content:"\f16e"}.ion-ios-document:before{content:"\f170"}.ion-ios-done-all:before{content:"\f171"}.ion-ios-download:before{content:"\f420"}.ion-ios-easel:before{content:"\f173"}.ion-ios-egg:before{content:"\f175"}.ion-ios-exit:before{content:"\f177"}.ion-ios-expand:before{content:"\f178"}.ion-ios-eye:before{content:"\f425"}.ion-ios-eye-off:before{content:"\f17a"}.ion-ios-fastforward:before{content:"\f427"}.ion-ios-female:before{content:"\f17b"}.ion-ios-filing:before{content:"\f429"}.ion-ios
 -film:before{content:"\f42b"}.ion-ios-finger-print:before{content:"\f17c"}.ion-ios-fitness:before{content:"\f1ab"}.ion-ios-flag:before{content:"\f42d"}.ion-ios-flame:before{content:"\f42f"}.ion-ios-flash:before{content:"\f17e"}.ion-ios-flash-off:before{content:"\f12f"}.ion-ios-flashlight:before{content:"\f141"}.ion-ios-flask:before{content:"\f431"}.ion-ios-flower:before{content:"\f433"}.ion-ios-folder:before{content:"\f435"}.ion-ios-folder-open:before{content:"\f180"}.ion-ios-football:before{content:"\f437"}.ion-ios-funnel:before{content:"\f182"}.ion-ios-gift:before{content:"\f191"}.ion-ios-git-branch:before{content:"\f183"}.ion-ios-git-commit:before{content:"\f184"}.ion-ios-git-compare:before{content:"\f185"}.ion-ios-git-merge:before{content:"\f186"}.ion-ios-git-network:before{content:"\f187"}.ion-ios-git-pull-request:before{content:"\f188"}.ion-ios-glasses:before{content:"\f43f"}.ion-ios-globe:before{content:"\f18a"}.ion-ios-grid:before{content:"\f18c"}.ion-ios-hammer:before{conte
 nt:"\f18e"}.ion-ios-hand:before{content:"\f190"}.ion-ios-happy:before{content:"\f192"}.ion-ios-headset:before{content:"\f194"}.ion-ios-heart:before{content:"\f443"}.ion-ios-heart-dislike:before{content:"\f13f"}.ion-ios-heart-empty:before{content:"\f19b"}.ion-ios-heart-half:before{content:"\f19d"}.ion-ios-help:before{content:"\f446"}.ion-ios-help-buoy:before{content:"\f196"}.ion-ios-help-circle:before{content:"\f198"}.ion-ios-help-circle-outline:before{content:"\f197"}.ion-ios-home:before{content:"\f448"}.ion-ios-hourglass:before{content:"\f103"}.ion-ios-ice-cream:before{content:"\f19a"}.ion-ios-image:before{content:"\f19c"}.ion-ios-images:before{content:"\f19e"}.ion-ios-infinite:before{content:"\f44a"}.ion-ios-information:before{content:"\f44d"}.ion-ios-information-circle:before{content:"\f1a0"}.ion-ios-information-circle-outline:before{content:"\f19f"}.ion-ios-jet:before{content:"\f1a5"}.ion-ios-journal:before{content:"\f189"}.ion-ios-key:before{content:"\f1a7"}.ion-ios-keypad:befo
 re{content:"\f450"}.ion-ios-laptop:before{content:"\f1a8"}.ion-ios-leaf:before{content:"\f1aa"}.ion-ios-link:before{content:"\f22a"}.ion-ios-list:before{content:"\f454"}.ion-ios-list-box:before{content:"\f143"}.ion-ios-locate:before{content:"\f1ae"}.ion-ios-lock:before{content:"\f1b0"}.ion-ios-log-in:before{content:"\f1b1"}.ion-ios-log-out:before{content:"\f1b2"}.ion-ios-magnet:before{content:"\f1b4"}.ion-ios-mail:before{content:"\f1b8"}.ion-ios-mail-open:before{content:"\f1b6"}.ion-ios-mail-unread:before{content:"\f145"}.ion-ios-male:before{content:"\f1b9"}.ion-ios-man:before{content:"\f1bb"}.ion-ios-map:before{content:"\f1bd"}.ion-ios-medal:before{content:"\f1bf"}.ion-ios-medical:before{content:"\f45c"}.ion-ios-medkit:before{content:"\f45e"}.ion-ios-megaphone:before{content:"\f1c1"}.ion-ios-menu:before{content:"\f1c3"}.ion-ios-mic:before{content:"\f461"}.ion-ios-mic-off:before{content:"\f45f"}.ion-ios-microphone:before{content:"\f1c6"}.ion-ios-moon:before{content:"\f468"}.ion-ios-
 more:before{content:"\f1c8"}.ion-ios-move:before{content:"\f1cb"}.ion-ios-musical-note:before{content:"\f46b"}.ion-ios-musical-notes:before{content:"\f46c"}.ion-ios-navigate:before{content:"\f46e"}.ion-ios-notifications:before{content:"\f1d3"}.ion-ios-notifications-off:before{content:"\f1d1"}.ion-ios-notifications-outline:before{content:"\f133"}.ion-ios-nuclear:before{content:"\f1d5"}.ion-ios-nutrition:before{content:"\f470"}.ion-ios-open:before{content:"\f1d7"}.ion-ios-options:before{content:"\f1d9"}.ion-ios-outlet:before{content:"\f1db"}.ion-ios-paper:before{content:"\f472"}.ion-ios-paper-plane:before{content:"\f1dd"}.ion-ios-partly-sunny:before{content:"\f1df"}.ion-ios-pause:before{content:"\f478"}.ion-ios-paw:before{content:"\f47a"}.ion-ios-people:before{content:"\f47c"}.ion-ios-person:before{content:"\f47e"}.ion-ios-person-add:before{content:"\f1e1"}.ion-ios-phone-landscape:before{content:"\f1e2"}.ion-ios-phone-portrait:before{content:"\f1e3"}.ion-ios-photos:before{content:"\f4
 82"}.ion-ios-pie:before{content:"\f484"}.ion-ios-pin:before{content:"\f1e5"}.ion-ios-pint:before{content:"\f486"}.ion-ios-pizza:before{content:"\f1e7"}.ion-ios-planet:before{content:"\f1eb"}.ion-ios-play:before{content:"\f488"}.ion-ios-play-circle:before{content:"\f113"}.ion-ios-podium:before{content:"\f1ed"}.ion-ios-power:before{content:"\f1ef"}.ion-ios-pricetag:before{content:"\f48d"}.ion-ios-pricetags:before{content:"\f48f"}.ion-ios-print:before{content:"\f1f1"}.ion-ios-pulse:before{content:"\f493"}.ion-ios-qr-scanner:before{content:"\f1f3"}.ion-ios-quote:before{content:"\f1f5"}.ion-ios-radio:before{content:"\f1f9"}.ion-ios-radio-button-off:before{content:"\f1f6"}.ion-ios-radio-button-on:before{content:"\f1f7"}.ion-ios-rainy:before{content:"\f495"}.ion-ios-recording:before{content:"\f497"}.ion-ios-redo:before{content:"\f499"}.ion-ios-refresh:before{content:"\f49c"}.ion-ios-refresh-circle:before{content:"\f135"}.ion-ios-remove:before{content:"\f1fc"}.ion-ios-remove-circle:before{c
 ontent:"\f1fb"}.ion-ios-remove-circle-outline:before{content:"\f1fa"}.ion-ios-reorder:before{content:"\f1fd"}.ion-ios-repeat:before{content:"\f1fe"}.ion-ios-resize:before{content:"\f1ff"}.ion-ios-restaurant:before{content:"\f201"}.ion-ios-return-left:before{content:"\f202"}.ion-ios-return-right:before{content:"\f203"}.ion-ios-reverse-camera:before{content:"\f49f"}.ion-ios-rewind:before{content:"\f4a1"}.ion-ios-ribbon:before{content:"\f205"}.ion-ios-rocket:before{content:"\f14b"}.ion-ios-rose:before{content:"\f4a3"}.ion-ios-sad:before{content:"\f207"}.ion-ios-save:before{content:"\f1a6"}.ion-ios-school:before{content:"\f209"}.ion-ios-search:before{content:"\f4a5"}.ion-ios-send:before{content:"\f20c"}.ion-ios-settings:before{content:"\f4a7"}.ion-ios-share:before{content:"\f211"}.ion-ios-share-alt:before{content:"\f20f"}.ion-ios-shirt:before{content:"\f213"}.ion-ios-shuffle:before{content:"\f4a9"}.ion-ios-skip-backward:before{content:"\f215"}.ion-ios-skip-forward:before{content:"\f217"
 }.ion-ios-snow:before{content:"\f218"}.ion-ios-speedometer:before{content:"\f4b0"}.ion-ios-square:before{content:"\f21a"}.ion-ios-square-outline:before{content:"\f15c"}.ion-ios-star:before{content:"\f4b3"}.ion-ios-star-half:before{content:"\f4b1"}.ion-ios-star-outline:before{content:"\f4b2"}.ion-ios-stats:before{content:"\f21c"}.ion-ios-stopwatch:before{content:"\f4b5"}.ion-ios-subway:before{content:"\f21e"}.ion-ios-sunny:before{content:"\f4b7"}.ion-ios-swap:before{content:"\f21f"}.ion-ios-switch:before{content:"\f221"}.ion-ios-sync:before{content:"\f222"}.ion-ios-tablet-landscape:before{content:"\f223"}.ion-ios-tablet-portrait:before{content:"\f24e"}.ion-ios-tennisball:before{content:"\f4bb"}.ion-ios-text:before{content:"\f250"}.ion-ios-thermometer:before{content:"\f252"}.ion-ios-thumbs-down:before{content:"\f254"}.ion-ios-thumbs-up:before{content:"\f256"}.ion-ios-thunderstorm:before{content:"\f4bd"}.ion-ios-time:before{content:"\f4bf"}.ion-ios-timer:before{content:"\f4c1"}.ion-ios
 -today:before{content:"\f14f"}.ion-ios-train:before{content:"\f258"}.ion-ios-transgender:before{content:"\f259"}.ion-ios-trash:before{content:"\f4c5"}.ion-ios-trending-down:before{content:"\f25a"}.ion-ios-trending-up:before{content:"\f25b"}.ion-ios-trophy:before{content:"\f25d"}.ion-ios-tv:before{content:"\f115"}.ion-ios-umbrella:before{content:"\f25f"}.ion-ios-undo:before{content:"\f4c7"}.ion-ios-unlock:before{content:"\f261"}.ion-ios-videocam:before{content:"\f4cd"}.ion-ios-volume-high:before{content:"\f11c"}.ion-ios-volume-low:before{content:"\f11e"}.ion-ios-volume-mute:before{content:"\f263"}.ion-ios-volume-off:before{content:"\f264"}.ion-ios-walk:before{content:"\f266"}.ion-ios-wallet:before{content:"\f18b"}.ion-ios-warning:before{content:"\f268"}.ion-ios-watch:before{content:"\f269"}.ion-ios-water:before{content:"\f26b"}.ion-ios-wifi:before{content:"\f26d"}.ion-ios-wine:before{content:"\f26f"}.ion-ios-woman:before{content:"\f271"}.ion-logo-android:before{content:"\f225"}.ion-l
 ogo-angular:before{content:"\f227"}.ion-logo-apple:before{content:"\f229"}.ion-logo-bitbucket:before{content:"\f193"}.ion-logo-bitcoin:before{content:"\f22b"}.ion-logo-buffer:before{content:"\f22d"}.ion-logo-chrome:before{content:"\f22f"}.ion-logo-closed-captioning:before{content:"\f105"}.ion-logo-codepen:before{content:"\f230"}.ion-logo-css3:before{content:"\f231"}.ion-logo-designernews:before{content:"\f232"}.ion-logo-dribbble:before{content:"\f233"}.ion-logo-dropbox:before{content:"\f234"}.ion-logo-euro:before{content:"\f235"}.ion-logo-facebook:before{content:"\f236"}.ion-logo-flickr:before{content:"\f107"}.ion-logo-foursquare:before{content:"\f237"}.ion-logo-freebsd-devil:before{content:"\f238"}.ion-logo-game-controller-a:before{content:"\f13b"}.ion-logo-game-controller-b:before{content:"\f181"}.ion-logo-github:before{content:"\f239"}.ion-logo-google:before{content:"\f23a"}.ion-logo-googleplus:before{content:"\f23b"}.ion-logo-hackernews:before{content:"\f23c"}.ion-logo-html5:bef
 ore{content:"\f23d"}.ion-logo-instagram:before{content:"\f23e"}.ion-logo-ionic:before{content:"\f150"}.ion-logo-ionitron:before{content:"\f151"}.ion-logo-javascript:before{content:"\f23f"}.ion-logo-linkedin:before{content:"\f240"}.ion-logo-markdown:before{content:"\f241"}.ion-logo-model-s:before{content:"\f153"}.ion-logo-no-smoking:before{content:"\f109"}.ion-logo-nodejs:before{content:"\f242"}.ion-logo-npm:before{content:"\f195"}.ion-logo-octocat:before{content:"\f243"}.ion-logo-pinterest:before{content:"\f244"}.ion-logo-playstation:before{content:"\f245"}.ion-logo-polymer:before{content:"\f15e"}.ion-logo-python:before{content:"\f246"}.ion-logo-reddit:before{content:"\f247"}.ion-logo-rss:before{content:"\f248"}.ion-logo-sass:before{content:"\f249"}.ion-logo-skype:before{content:"\f24a"}.ion-logo-slack:before{content:"\f10b"}.ion-logo-snapchat:before{content:"\f24b"}.ion-logo-steam:before{content:"\f24c"}.ion-logo-tumblr:before{content:"\f24d"}.ion-logo-tux:before{content:"\f2ae"}.i
 on-logo-twitch:before{content:"\f2af"}.ion-logo-twitter:before{content:"\f2b0"}.ion-logo-usd:before{content:"\f2b1"}.ion-logo-vimeo:before{content:"\f2c4"}.ion-logo-vk:before{content:"\f10d"}.ion-logo-whatsapp:before{content:"\f2c5"}.ion-logo-windows:before{content:"\f32f"}.ion-logo-wordpress:before{content:"\f330"}.ion-logo-xbox:before{content:"\f34c"}.ion-logo-xing:before{content:"\f10f"}.ion-logo-yahoo:before{content:"\f34d"}.ion-logo-yen:before{content:"\f34e"}.ion-logo-youtube:before{content:"\f34f"}.ion-md-add:before{content:"\f273"}.ion-md-add-circle:before{content:"\f272"}.ion-md-add-circle-outline:before{content:"\f158"}.ion-md-airplane:before{content:"\f15a"}.ion-md-alarm:before{content:"\f274"}.ion-md-albums:before{content:"\f275"}.ion-md-alert:before{content:"\f276"}.ion-md-american-football:before{content:"\f277"}.ion-md-analytics:before{content:"\f278"}.ion-md-aperture:before{content:"\f279"}.ion-md-apps:before{content:"\f27a"}.ion-md-appstore:before{content:"\f27b"}.i
 on-md-archive:before{content:"\f27c"}.ion-md-arrow-back:before{content:"\f27d"}.ion-md-arrow-down:before{content:"\f27e"}.ion-md-arrow-dropdown:before{content:"\f280"}.ion-md-arrow-dropdown-circle:before{content:"\f27f"}.ion-md-arrow-dropleft:before{content:"\f282"}.ion-md-arrow-dropleft-circle:before{content:"\f281"}.ion-md-arrow-dropright:before{content:"\f284"}.ion-md-arrow-dropright-circle:before{content:"\f283"}.ion-md-arrow-dropup:before{content:"\f286"}.ion-md-arrow-dropup-circle:before{content:"\f285"}.ion-md-arrow-forward:before{content:"\f287"}.ion-md-arrow-round-back:before{content:"\f288"}.ion-md-arrow-round-down:before{content:"\f289"}.ion-md-arrow-round-forward:before{content:"\f28a"}.ion-md-arrow-round-up:before{content:"\f28b"}.ion-md-arrow-up:before{content:"\f28c"}.ion-md-at:before{content:"\f28d"}.ion-md-attach:before{content:"\f28e"}.ion-md-backspace:before{content:"\f28f"}.ion-md-barcode:before{content:"\f290"}.ion-md-baseball:before{content:"\f291"}.ion-md-bask
 et:before{content:"\f292"}.ion-md-basketball:before{content:"\f293"}.ion-md-battery-charging:before{content:"\f294"}.ion-md-battery-dead:before{content:"\f295"}.ion-md-battery-full:before{content:"\f296"}.ion-md-beaker:before{content:"\f297"}.ion-md-bed:before{content:"\f160"}.ion-md-beer:before{content:"\f298"}.ion-md-bicycle:before{content:"\f299"}.ion-md-bluetooth:before{content:"\f29a"}.ion-md-boat:before{content:"\f29b"}.ion-md-body:before{content:"\f29c"}.ion-md-bonfire:before{content:"\f29d"}.ion-md-book:before{content:"\f29e"}.ion-md-bookmark:before{content:"\f29f"}.ion-md-bookmarks:before{content:"\f2a0"}.ion-md-bowtie:before{content:"\f2a1"}.ion-md-briefcase:before{content:"\f2a2"}.ion-md-browsers:before{content:"\f2a3"}.ion-md-brush:before{content:"\f2a4"}.ion-md-bug:before{content:"\f2a5"}.ion-md-build:before{content:"\f2a6"}.ion-md-bulb:before{content:"\f2a7"}.ion-md-bus:before{content:"\f2a8"}.ion-md-business:before{content:"\f1a4"}.ion-md-cafe:before{content:"\f2a9"}.
 ion-md-calculator:before{content:"\f2aa"}.ion-md-calendar:before{content:"\f2ab"}.ion-md-call:before{content:"\f2ac"}.ion-md-camera:before{content:"\f2ad"}.ion-md-car:before{content:"\f2b2"}.ion-md-card:before{content:"\f2b3"}.ion-md-cart:before{content:"\f2b4"}.ion-md-cash:before{content:"\f2b5"}.ion-md-cellular:before{content:"\f164"}.ion-md-chatboxes:before{content:"\f2b6"}.ion-md-chatbubbles:before{content:"\f2b7"}.ion-md-checkbox:before{content:"\f2b9"}.ion-md-checkbox-outline:before{content:"\f2b8"}.ion-md-checkmark:before{content:"\f2bc"}.ion-md-checkmark-circle:before{content:"\f2bb"}.ion-md-checkmark-circle-outline:before{content:"\f2ba"}.ion-md-clipboard:before{content:"\f2bd"}.ion-md-clock:before{content:"\f2be"}.ion-md-close:before{content:"\f2c0"}.ion-md-close-circle:before{content:"\f2bf"}.ion-md-close-circle-outline:before{content:"\f166"}.ion-md-cloud:before{content:"\f2c9"}.ion-md-cloud-circle:before{content:"\f2c2"}.ion-md-cloud-done:before{content:"\f2c3"}.ion-md-
 cloud-download:before{content:"\f2c6"}.ion-md-cloud-outline:before{content:"\f2c7"}.ion-md-cloud-upload:before{content:"\f2c8"}.ion-md-cloudy:before{content:"\f2cb"}.ion-md-cloudy-night:before{content:"\f2ca"}.ion-md-code:before{content:"\f2ce"}.ion-md-code-download:before{content:"\f2cc"}.ion-md-code-working:before{content:"\f2cd"}.ion-md-cog:before{content:"\f2cf"}.ion-md-color-fill:before{content:"\f2d0"}.ion-md-color-filter:before{content:"\f2d1"}.ion-md-color-palette:before{content:"\f2d2"}.ion-md-color-wand:before{content:"\f2d3"}.ion-md-compass:before{content:"\f2d4"}.ion-md-construct:before{content:"\f2d5"}.ion-md-contact:before{content:"\f2d6"}.ion-md-contacts:before{content:"\f2d7"}.ion-md-contract:before{content:"\f2d8"}.ion-md-contrast:before{content:"\f2d9"}.ion-md-copy:before{content:"\f2da"}.ion-md-create:before{content:"\f2db"}.ion-md-crop:before{content:"\f2dc"}.ion-md-cube:before{content:"\f2dd"}.ion-md-cut:before{content:"\f2de"}.ion-md-desktop:before{content:"\f2
 df"}.ion-md-disc:before{content:"\f2e0"}.ion-md-document:before{content:"\f2e1"}.ion-md-done-all:before{content:"\f2e2"}.ion-md-download:before{content:"\f2e3"}.ion-md-easel:before{content:"\f2e4"}.ion-md-egg:before{content:"\f2e5"}.ion-md-exit:before{content:"\f2e6"}.ion-md-expand:before{content:"\f2e7"}.ion-md-eye:before{content:"\f2e9"}.ion-md-eye-off:before{content:"\f2e8"}.ion-md-fastforward:before{content:"\f2ea"}.ion-md-female:before{content:"\f2eb"}.ion-md-filing:before{content:"\f2ec"}.ion-md-film:before{content:"\f2ed"}.ion-md-finger-print:before{content:"\f2ee"}.ion-md-fitness:before{content:"\f1ac"}.ion-md-flag:before{content:"\f2ef"}.ion-md-flame:before{content:"\f2f0"}.ion-md-flash:before{content:"\f2f1"}.ion-md-flash-off:before{content:"\f169"}.ion-md-flashlight:before{content:"\f16b"}.ion-md-flask:before{content:"\f2f2"}.ion-md-flower:before{content:"\f2f3"}.ion-md-folder:before{content:"\f2f5"}.ion-md-folder-open:before{content:"\f2f4"}.ion-md-football:before{conten
 t:"\f2f6"}.ion-md-funnel:before{content:"\f2f7"}.ion-md-gift:before{content:"\f199"}.ion-md-git-branch:before{content:"\f2fa"}.ion-md-git-commit:before{content:"\f2fb"}.ion-md-git-compare:before{content:"\f2fc"}.ion-md-git-merge:before{content:"\f2fd"}.ion-md-git-network:before{content:"\f2fe"}.ion-md-git-pull-request:before{content:"\f2ff"}.ion-md-glasses:before{content:"\f300"}.ion-md-globe:before{content:"\f301"}.ion-md-grid:before{content:"\f302"}.ion-md-hammer:before{content:"\f303"}.ion-md-hand:before{content:"\f304"}.ion-md-happy:before{content:"\f305"}.ion-md-headset:before{content:"\f306"}.ion-md-heart:before{content:"\f308"}.ion-md-heart-dislike:before{content:"\f167"}.ion-md-heart-empty:before{content:"\f1a1"}.ion-md-heart-half:before{content:"\f1a2"}.ion-md-help:before{content:"\f30b"}.ion-md-help-buoy:before{content:"\f309"}.ion-md-help-circle:before{content:"\f30a"}.ion-md-help-circle-outline:before{content:"\f16d"}.ion-md-home:before{content:"\f30c"}.ion-md-hourglass:
 before{content:"\f111"}.ion-md-ice-cream:before{content:"\f30d"}.ion-md-image:before{content:"\f30e"}.ion-md-images:before{content:"\f30f"}.ion-md-infinite:before{content:"\f310"}.ion-md-information:before{content:"\f312"}.ion-md-information-circle:before{content:"\f311"}.ion-md-information-circle-outline:before{content:"\f16f"}.ion-md-jet:before{content:"\f315"}.ion-md-journal:before{content:"\f18d"}.ion-md-key:before{content:"\f316"}.ion-md-keypad:before{content:"\f317"}.ion-md-laptop:before{content:"\f318"}.ion-md-leaf:before{content:"\f319"}.ion-md-link:before{content:"\f22e"}.ion-md-list:before{content:"\f31b"}.ion-md-list-box:before{content:"\f31a"}.ion-md-locate:before{content:"\f31c"}.ion-md-lock:before{content:"\f31d"}.ion-md-log-in:before{content:"\f31e"}.ion-md-log-out:before{content:"\f31f"}.ion-md-magnet:before{content:"\f320"}.ion-md-mail:before{content:"\f322"}.ion-md-mail-open:before{content:"\f321"}.ion-md-mail-unread:before{content:"\f172"}.ion-md-male:before{conte
 nt:"\f323"}.ion-md-man:before{content:"\f324"}.ion-md-map:before{content:"\f325"}.ion-md-medal:before{content:"\f326"}.ion-md-medical:before{content:"\f327"}.ion-md-medkit:before{content:"\f328"}.ion-md-megaphone:before{content:"\f329"}.ion-md-menu:before{content:"\f32a"}.ion-md-mic:before{content:"\f32c"}.ion-md-mic-off:before{content:"\f32b"}.ion-md-microphone:before{content:"\f32d"}.ion-md-moon:before{content:"\f32e"}.ion-md-more:before{content:"\f1c9"}.ion-md-move:before{content:"\f331"}.ion-md-musical-note:before{content:"\f332"}.ion-md-musical-notes:before{content:"\f333"}.ion-md-navigate:before{content:"\f334"}.ion-md-notifications:before{content:"\f338"}.ion-md-notifications-off:before{content:"\f336"}.ion-md-notifications-outline:before{content:"\f337"}.ion-md-nuclear:before{content:"\f339"}.ion-md-nutrition:before{content:"\f33a"}.ion-md-open:before{content:"\f33b"}.ion-md-options:before{content:"\f33c"}.ion-md-outlet:before{content:"\f33d"}.ion-md-paper:before{content:"\f
 33f"}.ion-md-paper-plane:before{content:"\f33e"}.ion-md-partly-sunny:before{content:"\f340"}.ion-md-pause:before{content:"\f341"}.ion-md-paw:before{content:"\f342"}.ion-md-people:before{content:"\f343"}.ion-md-person:before{content:"\f345"}.ion-md-person-add:before{content:"\f344"}.ion-md-phone-landscape:before{content:"\f346"}.ion-md-phone-portrait:before{content:"\f347"}.ion-md-photos:before{content:"\f348"}.ion-md-pie:before{content:"\f349"}.ion-md-pin:before{content:"\f34a"}.ion-md-pint:before{content:"\f34b"}.ion-md-pizza:before{content:"\f354"}.ion-md-planet:before{content:"\f356"}.ion-md-play:before{content:"\f357"}.ion-md-play-circle:before{content:"\f174"}.ion-md-podium:before{content:"\f358"}.ion-md-power:before{content:"\f359"}.ion-md-pricetag:before{content:"\f35a"}.ion-md-pricetags:before{content:"\f35b"}.ion-md-print:before{content:"\f35c"}.ion-md-pulse:before{content:"\f35d"}.ion-md-qr-scanner:before{content:"\f35e"}.ion-md-quote:before{content:"\f35f"}.ion-md-radio:b
 efore{content:"\f362"}.ion-md-radio-button-off:before{content:"\f360"}.ion-md-radio-button-on:before{content:"\f361"}.ion-md-rainy:before{content:"\f363"}.ion-md-recording:before{content:"\f364"}.ion-md-redo:before{content:"\f365"}.ion-md-refresh:before{content:"\f366"}.ion-md-refresh-circle:before{content:"\f228"}.ion-md-remove:before{content:"\f368"}.ion-md-remove-circle:before{content:"\f367"}.ion-md-remove-circle-outline:before{content:"\f176"}.ion-md-reorder:before{content:"\f369"}.ion-md-repeat:before{content:"\f36a"}.ion-md-resize:before{content:"\f36b"}.ion-md-restaurant:before{content:"\f36c"}.ion-md-return-left:before{content:"\f36d"}.ion-md-return-right:before{content:"\f36e"}.ion-md-reverse-camera:before{content:"\f36f"}.ion-md-rewind:before{content:"\f370"}.ion-md-ribbon:before{content:"\f371"}.ion-md-rocket:before{content:"\f179"}.ion-md-rose:before{content:"\f372"}.ion-md-sad:before{content:"\f373"}.ion-md-save:before{content:"\f1a9"}.ion-md-school:before{content:"\f3
 74"}.ion-md-search:before{content:"\f375"}.ion-md-send:before{content:"\f376"}.ion-md-settings:before{content:"\f377"}.ion-md-share:before{content:"\f379"}.ion-md-share-alt:before{content:"\f378"}.ion-md-shirt:before{content:"\f37a"}.ion-md-shuffle:before{content:"\f37b"}.ion-md-skip-backward:before{content:"\f37c"}.ion-md-skip-forward:before{content:"\f37d"}.ion-md-snow:before{content:"\f37e"}.ion-md-speedometer:before{content:"\f37f"}.ion-md-square:before{content:"\f381"}.ion-md-square-outline:before{content:"\f380"}.ion-md-star:before{content:"\f384"}.ion-md-star-half:before{content:"\f382"}.ion-md-star-outline:before{content:"\f383"}.ion-md-stats:before{content:"\f385"}.ion-md-stopwatch:before{content:"\f386"}.ion-md-subway:before{content:"\f387"}.ion-md-sunny:before{content:"\f388"}.ion-md-swap:before{content:"\f389"}.ion-md-switch:before{content:"\f38a"}.ion-md-sync:before{content:"\f38b"}.ion-md-tablet-landscape:before{content:"\f38c"}.ion-md-tablet-portrait:before{content:"\
 f38d"}.ion-md-tennisball:before{content:"\f38e"}.ion-md-text:before{content:"\f38f"}.ion-md-thermometer:before{content:"\f390"}.ion-md-thumbs-down:before{content:"\f391"}.ion-md-thumbs-up:before{content:"\f392"}.ion-md-thunderstorm:before{content:"\f393"}.ion-md-time:before{content:"\f394"}.ion-md-timer:before{content:"\f395"}.ion-md-today:before{content:"\f17d"}.ion-md-train:before{content:"\f396"}.ion-md-transgender:before{content:"\f397"}.ion-md-trash:before{content:"\f398"}.ion-md-trending-down:before{content:"\f399"}.ion-md-trending-up:before{content:"\f39a"}.ion-md-trophy:before{content:"\f39b"}.ion-md-tv:before{content:"\f17f"}.ion-md-umbrella:before{content:"\f39c"}.ion-md-undo:before{content:"\f39d"}.ion-md-unlock:before{content:"\f39e"}.ion-md-videocam:before{content:"\f39f"}.ion-md-volume-high:before{content:"\f123"}.ion-md-volume-low:before{content:"\f131"}.ion-md-volume-mute:before{content:"\f3a1"}.ion-md-volume-off:before{content:"\f3a2"}.ion-md-walk:before{content:"\f
 3a4"}.ion-md-wallet:before{content:"\f18f"}.ion-md-warning:before{content:"\f3a5"}.ion-md-watch:before{content:"\f3a6"}.ion-md-water:before{content:"\f3a7"}.ion-md-wifi:before{content:"\f3a8"}.ion-md-wine:before{content:"\f3a9"}.ion-md-woman:before{content:"\f3aa"}


[28/29] samza git commit: added job name to test

Posted by bo...@apache.org.
added job name to test


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

Branch: refs/heads/NewKafkaSystemConsumer
Commit: da39af728993a83538af6957e9c8ab09cfba1ccd
Parents: 190a399
Author: Boris S <bo...@apache.org>
Authored: Tue Sep 25 16:25:27 2018 -0700
Committer: Boris S <bo...@apache.org>
Committed: Tue Sep 25 16:25:27 2018 -0700

----------------------------------------------------------------------
 .../org/apache/samza/system/kafka/TestKafkaSystemConsumer.java     | 2 ++
 1 file changed, 2 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/samza/blob/da39af72/samza-kafka/src/test/scala/org/apache/samza/system/kafka/TestKafkaSystemConsumer.java
----------------------------------------------------------------------
diff --git a/samza-kafka/src/test/scala/org/apache/samza/system/kafka/TestKafkaSystemConsumer.java b/samza-kafka/src/test/scala/org/apache/samza/system/kafka/TestKafkaSystemConsumer.java
index 933558c..5791545 100644
--- a/samza-kafka/src/test/scala/org/apache/samza/system/kafka/TestKafkaSystemConsumer.java
+++ b/samza-kafka/src/test/scala/org/apache/samza/system/kafka/TestKafkaSystemConsumer.java
@@ -29,6 +29,7 @@ import org.apache.kafka.clients.consumer.KafkaConsumer;
 import org.apache.kafka.common.serialization.ByteArraySerializer;
 import org.apache.samza.Partition;
 import org.apache.samza.config.Config;
+import org.apache.samza.config.JobConfig;
 import org.apache.samza.config.KafkaConfig;
 import org.apache.samza.config.KafkaConsumerConfig;
 import org.apache.samza.config.MapConfig;
@@ -57,6 +58,7 @@ public class TestKafkaSystemConsumer {
     map.put(String.format(KafkaConfig.CONSUMER_FETCH_THRESHOLD_BYTES(), TEST_SYSTEM), fetchBytes);
     map.put(String.format("systems.%s.consumer.%s", TEST_SYSTEM, ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG),
         BOOTSTRAP_SERVER);
+    map.put(JobConfig.JOB_NAME(), "jobName");
 
     Config config = new MapConfig(map);
     KafkaConsumerConfig consumerConfig =


[08/29] samza git commit: Merge branch 'master' into NewConsumer2

Posted by bo...@apache.org.
Merge branch 'master' into NewConsumer2


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

Branch: refs/heads/NewKafkaSystemConsumer
Commit: 1d1fb896db8773cb539835949966f8270005c81a
Parents: 74b6cfa 927adff
Author: Boris S <bo...@apache.org>
Authored: Thu Sep 20 09:39:29 2018 -0700
Committer: Boris S <bo...@apache.org>
Committed: Thu Sep 20 09:39:29 2018 -0700

----------------------------------------------------------------------
 .travis.yml                                     |   4 +-
 build.gradle                                    |   1 +
 gradle/dependency-versions.gradle               |   1 +
 .../samza/metadatastore/MetadataStore.java      |   6 +-
 .../samza/config/ClusterManagerConfig.java      |   1 -
 .../apache/samza/container/LocalityManager.java |   2 +-
 .../grouper/task/GroupByContainerCount.java     |   6 +-
 .../grouper/task/GroupByContainerIds.java       |   4 +-
 .../task/SingleContainerGrouperFactory.java     |   2 +-
 .../grouper/task/TaskAssignmentManager.java     |   2 +-
 .../metadatastore/CoordinatorStreamStore.java   |   2 +-
 .../samza/execution/ExecutionPlanner.java       | 178 +++--
 .../org/apache/samza/execution/JobGraph.java    |  82 +-
 .../samza/execution/JobGraphJsonGenerator.java  |   4 +-
 .../org/apache/samza/execution/JobNode.java     |   5 +-
 .../apache/samza/job/model/ContainerModel.java  |  31 +-
 .../org/apache/samza/job/model/JobModel.java    |   2 -
 .../apache/samza/processor/StreamProcessor.java |   2 +-
 .../model/JsonContainerModelMixIn.java          |  38 +-
 .../serializers/model/JsonJobModelMixIn.java    |   2 +
 .../serializers/model/JsonTaskModelMixIn.java   |   2 +
 .../serializers/model/SamzaObjectMapper.java    |  41 +-
 .../apache/samza/storage/StorageRecovery.java   |   2 +-
 .../samza/system/inmemory/InMemoryManager.java  |   6 +-
 .../system/inmemory/InMemorySystemAdmin.java    |   4 -
 .../system/inmemory/InMemorySystemConsumer.java |   4 -
 .../system/inmemory/InMemorySystemFactory.java  |   4 -
 .../system/inmemory/InMemorySystemProducer.java |   4 -
 .../org/apache/samza/table/TableManager.java    |  34 +-
 .../table/remote/RemoteReadWriteTable.java      |   2 +-
 .../samza/table/remote/RemoteReadableTable.java |   2 +-
 .../table/remote/RemoteTableDescriptor.java     |  46 +-
 .../samza/table/remote/RemoteTableProvider.java |  42 +-
 .../samza/table/remote/TableReadFunction.java   |   7 +
 .../samza/table/remote/TableWriteFunction.java  |   7 +
 .../samza/table/retry/FailsafeAdapter.java      | 103 +++
 .../table/retry/RetriableReadFunction.java      | 102 +++
 .../table/retry/RetriableWriteFunction.java     | 120 +++
 .../apache/samza/table/retry/RetryMetrics.java  |  59 ++
 .../samza/table/retry/TableRetryPolicy.java     | 257 +++++++
 .../java/org/apache/samza/zk/ProcessorData.java |  19 +-
 .../samza/zk/ZkJobCoordinatorFactory.java       |  17 +-
 .../java/org/apache/samza/zk/ZkKeyBuilder.java  |  30 +-
 .../org/apache/samza/zk/ZkMetadataStore.java    | 132 ++++
 .../apache/samza/zk/ZkMetadataStoreFactory.java |  36 +
 .../org/apache/samza/config/JobConfig.scala     |   1 +
 .../apache/samza/container/SamzaContainer.scala |  26 +-
 .../samza/coordinator/JobModelManager.scala     |   2 +-
 .../grouper/task/TestGroupByContainerCount.java |  89 +--
 .../grouper/task/TestGroupByContainerIds.java   |  37 +-
 .../samza/container/mock/ContainerMocks.java    |   6 +-
 .../coordinator/JobModelManagerTestUtil.java    |  13 +-
 .../TestCoordinatorStreamStore.java             |   2 +-
 .../samza/execution/TestExecutionPlanner.java   | 100 +--
 .../apache/samza/execution/TestJobGraph.java    |  38 +-
 .../apache/samza/job/model/TestJobModel.java    |  50 ++
 .../operators/impl/TestOperatorImplGraph.java   |   8 +-
 .../model/TestSamzaObjectMapper.java            | 205 +++--
 .../apache/samza/table/TestTableManager.java    |  11 +-
 .../samza/table/remote/TestRemoteTable.java     | 116 ++-
 .../table/remote/TestRemoteTableDescriptor.java |  10 +-
 .../retry/TestRetriableTableFunctions.java      | 316 ++++++++
 .../samza/table/retry/TestTableRetryPolicy.java |  82 ++
 .../org/apache/samza/zk/TestZkKeyBuilder.java   |   2 +-
 .../apache/samza/zk/TestZkMetadataStore.java    | 121 +++
 .../samza/container/TestSamzaContainer.scala    | 741 ++++---------------
 .../samza/container/TestTaskInstance.scala      | 526 ++++---------
 .../TestTaskInstanceExceptionHandler.scala      | 144 ++++
 .../samza/coordinator/TestJobCoordinator.scala  |  12 +-
 .../samza/system/chooser/MockSystemAdmin.scala  |  30 +
 .../chooser/TestBootstrappingChooser.scala      |   3 +-
 .../system/chooser/TestDefaultChooser.scala     |   1 -
 .../samza/system/hdfs/HdfsSystemAdmin.java      |  12 +
 .../samza/sql/translator/JoinTranslator.java    |   4 +-
 .../samza/example/PageViewCounterExample.java   |   1 -
 .../samza/test/framework/StreamAssert.java      |  73 +-
 .../apache/samza/test/framework/TestRunner.java | 289 ++++----
 .../test/framework/stream/CollectionStream.java | 204 -----
 .../system/CollectionStreamSystemSpec.java      |  90 ---
 .../system/InMemoryInputDescriptor.java         |  42 ++
 .../system/InMemoryOutputDescriptor.java        |  46 ++
 .../system/InMemorySystemDescriptor.java        | 118 +++
 .../AsyncStreamTaskIntegrationTest.java         | 108 +--
 .../StreamApplicationIntegrationTest.java       |  45 +-
 .../framework/StreamTaskIntegrationTest.java    | 112 ++-
 .../processor/TestZkLocalApplicationRunner.java |   6 +-
 .../table/TestLocalTableWithSideInputs.java     |  38 +-
 .../samza/test/table/TestRemoteTable.java       |  27 +-
 .../table/TestTableDescriptorsProvider.java     |  18 +-
 .../webapp/TestApplicationMasterRestClient.java |   2 +-
 90 files changed, 3299 insertions(+), 2085 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/samza/blob/1d1fb896/samza-core/src/main/java/org/apache/samza/storage/StorageRecovery.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/samza/blob/1d1fb896/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/samza/blob/1d1fb896/samza-core/src/main/scala/org/apache/samza/coordinator/JobModelManager.scala
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/samza/blob/1d1fb896/samza-core/src/test/scala/org/apache/samza/coordinator/TestJobCoordinator.scala
----------------------------------------------------------------------