You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@samza.apache.org by xi...@apache.org on 2017/09/12 21:19:42 UTC

[01/26] samza git commit: SAMZA-1374: Implement Leader Election using Lease Blob in Azure

Repository: samza
Updated Branches:
  refs/heads/0.14.0 79200c735 -> ebce13e74


SAMZA-1374: Implement Leader Election using Lease Blob in Azure

PR 1: AzureClient + AzureConfig
PR 2: LeaseBlobManager
PR 3: BlobUtils + JobModelBundle
PR 4: TableUtils + ProcessorEntity
**PR 5: AzureLeaderElector** (current PR)

Author: PawasChhokra <Jaimatadi1$>
Author: PawasChhokra <pa...@gmail.com>

Reviewers: Navina Ramesh <na...@apache.org>, Shanthoosh Venkataraman <sv...@linkedin.com>

Closes #259 from PawasChhokra/LeaderElection


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

Branch: refs/heads/0.14.0
Commit: 403042394945deba1affa7269c7ead3129326cbd
Parents: bf4c761
Author: Pawas Chhokra <pa...@gmail.com>
Authored: Tue Aug 8 18:26:50 2017 -0700
Committer: navina <na...@apache.org>
Committed: Tue Aug 8 18:26:50 2017 -0700

----------------------------------------------------------------------
 .../main/java/org/apache/samza/AzureClient.java |  10 ++
 .../org/apache/samza/AzureLeaderElector.java    | 111 +++++++++++++++++++
 2 files changed, 121 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/samza/blob/40304239/samza-azure/src/main/java/org/apache/samza/AzureClient.java
----------------------------------------------------------------------
diff --git a/samza-azure/src/main/java/org/apache/samza/AzureClient.java b/samza-azure/src/main/java/org/apache/samza/AzureClient.java
index 7c12055..2248d12 100644
--- a/samza-azure/src/main/java/org/apache/samza/AzureClient.java
+++ b/samza-azure/src/main/java/org/apache/samza/AzureClient.java
@@ -20,6 +20,9 @@
 package org.apache.samza;
 
 import com.microsoft.azure.storage.CloudStorageAccount;
+import com.microsoft.azure.storage.RetryLinearRetry;
+import com.microsoft.azure.storage.RetryPolicy;
+import com.microsoft.azure.storage.blob.BlobRequestOptions;
 import com.microsoft.azure.storage.blob.CloudBlobClient;
 import com.microsoft.azure.storage.table.CloudTableClient;
 import java.net.URISyntaxException;
@@ -47,7 +50,14 @@ public class AzureClient {
   AzureClient(String storageConnectionString) {
     try {
       account = CloudStorageAccount.parse(storageConnectionString);
+
       blobClient = account.createCloudBlobClient();
+      // Set retry policy for operations on the blob. In this case, every failed operation on the blob will be retried thrice, after 5 second intervals.
+      BlobRequestOptions options = new BlobRequestOptions();
+      RetryPolicy retryPolicy = new RetryLinearRetry(5000, 3);
+      options.setRetryPolicyFactory(retryPolicy);
+      blobClient.setDefaultRequestOptions(options);
+
       tableClient = account.createCloudTableClient();
     } catch (IllegalArgumentException | URISyntaxException e) {
       LOG.error("Connection string {} specifies an invalid URI.", storageConnectionString);

http://git-wip-us.apache.org/repos/asf/samza/blob/40304239/samza-azure/src/main/java/org/apache/samza/AzureLeaderElector.java
----------------------------------------------------------------------
diff --git a/samza-azure/src/main/java/org/apache/samza/AzureLeaderElector.java b/samza-azure/src/main/java/org/apache/samza/AzureLeaderElector.java
new file mode 100644
index 0000000..efa8ea1
--- /dev/null
+++ b/samza-azure/src/main/java/org/apache/samza/AzureLeaderElector.java
@@ -0,0 +1,111 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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;
+
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.samza.coordinator.LeaderElector;
+import org.apache.samza.coordinator.LeaderElectorListener;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Class to facilitate leader election in Azure.
+ * The processor that acquires the lease on the blob becomes the leader.
+ * The lease ID is null initially. It is generated by Azure when the processor acquires the lease, and updated accordingly.
+ * Every processor requires a valid active lease ID in order to perform successful write and delete operations on the blob.
+ * Read operations from the blob are not dependent on the lease ID.
+ */
+public class AzureLeaderElector implements LeaderElector {
+
+  private static final Logger LOG = LoggerFactory.getLogger(AzureLeaderElector.class);
+  private static final int LEASE_TIME_IN_SEC = 60;
+  private final LeaseBlobManager leaseBlobManager;
+  private LeaderElectorListener leaderElectorListener = null;
+  private final AtomicReference<String> leaseId;
+  private final AtomicBoolean isLeader;
+
+  public AzureLeaderElector(LeaseBlobManager leaseBlobManager) {
+    this.isLeader = new AtomicBoolean(false);
+    this.leaseBlobManager = leaseBlobManager;
+    this.leaseId = new AtomicReference<>(null);
+  }
+
+  @Override
+  public void setLeaderElectorListener(LeaderElectorListener listener) {
+    this.leaderElectorListener = listener;
+  }
+
+  /**
+   * Tries to become the leader by acquiring a lease on the blob.
+   * The acquireLease operation has a retry policy where upon failure, the operation is tried 3 times at 5 second intervals.
+   * Invokes the listener on becoming the leader.
+   */
+  @Override
+  public void tryBecomeLeader() {
+    try {
+      leaseId.getAndSet(leaseBlobManager.acquireLease(LEASE_TIME_IN_SEC, leaseId.get()));
+      if (leaseId.get() != null) {
+        LOG.info("Became leader with lease ID {}.", leaseId.get());
+        isLeader.set(true);
+        if (leaderElectorListener != null) {
+          leaderElectorListener.onBecomingLeader();
+        }
+      }
+    } catch (AzureException e) {
+      LOG.error("Error while trying to acquire lease.", e);
+    }
+  }
+
+  /**
+   * Releases the lease in order to resign leadership. It also stops all schedulers scheduled by the leader.
+   * The releaseLease operation has a retry policy where upon failure, the operation is tried 3 times at 5 second intervals.
+   */
+  @Override
+  public void resignLeadership() {
+    if (isLeader.get()) {
+      leaseBlobManager.releaseLease(leaseId.get());
+      isLeader.set(false);
+      LOG.info("Resigning leadership with lease ID {}", leaseId.get());
+      leaseId.getAndSet(null);
+    } else {
+      LOG.info("Can't release the lease because it is not the leader and does not hold an active lease.");
+    }
+  }
+
+  /**
+   * Checks whether it's a leader
+   * @return true if it is the leader, false otherwise
+   */
+  @Override
+  public boolean amILeader() {
+    return isLeader.get();
+  }
+
+  public AtomicReference<String> getLeaseId() {
+    return leaseId;
+  }
+
+  public LeaseBlobManager getLeaseBlobManager() {
+    return this.leaseBlobManager;
+  }
+
+}


[13/26] samza git commit: SAMZA-1401: update NOTICE for 0.13.1 release with RocksDB BSD+patents license

Posted by xi...@apache.org.
SAMZA-1401: update NOTICE for 0.13.1 release with RocksDB BSD+patents license

Please see details in SAMZA-1401.

Author: Fred Ji <ha...@gmail.com>

Reviewers: Navina Ramesh <na...@apache.org>

Closes #279 from fredji97/rocksdb


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

Branch: refs/heads/0.14.0
Commit: 1d253c757c74d5a5364d0b270d037361c60d1ee6
Parents: e21ff71
Author: Fred Ji <ha...@gmail.com>
Authored: Thu Aug 17 17:54:40 2017 -0700
Committer: navina <na...@apache.org>
Committed: Thu Aug 17 17:54:40 2017 -0700

----------------------------------------------------------------------
 NOTICE | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/samza/blob/1d253c75/NOTICE
----------------------------------------------------------------------
diff --git a/NOTICE b/NOTICE
index 2ee5fdc..8024af7 100644
--- a/NOTICE
+++ b/NOTICE
@@ -4,3 +4,4 @@ Copyright 2014 The Apache Software Foundation
 This product includes software developed at The Apache Software
 Foundation (http://www.apache.org/).
 
+This product uses RocksDB which carries Facebook BSD+patents license.


[05/26] samza git commit: SAMZA-1388: Flaky test - TestStatefulTask#testShouldStartAndRestore

Posted by xi...@apache.org.
SAMZA-1388: Flaky test - TestStatefulTask#testShouldStartAndRestore

I believe the problem originated from SAMZA-173.

The core issue is testShouldRestoreStore was not updated to expect 6 messages after 2 more messages were added to testShouldStartTaskForFirstTime.

Fixed the issue and refactored the code so the 2 methods wouldn't disagree again in the future.

Author: Jacob Maes <jm...@linkedin.com>

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

Closes #269 from jmakes/samza-1388


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

Branch: refs/heads/0.14.0
Commit: b1277d8b4cbda3a12f00dd2c8e86980086f2c640
Parents: 46b3601
Author: Jacob Maes <jm...@linkedin.com>
Authored: Fri Aug 11 15:50:12 2017 -0700
Committer: Jacob Maes <jm...@linkedin.com>
Committed: Fri Aug 11 15:50:12 2017 -0700

----------------------------------------------------------------------
 .../test/integration/TestStatefulTask.scala     | 79 +++++++-------------
 1 file changed, 29 insertions(+), 50 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/samza/blob/b1277d8b/samza-test/src/test/scala/org/apache/samza/test/integration/TestStatefulTask.scala
----------------------------------------------------------------------
diff --git a/samza-test/src/test/scala/org/apache/samza/test/integration/TestStatefulTask.scala b/samza-test/src/test/scala/org/apache/samza/test/integration/TestStatefulTask.scala
index e5b6756..734487b 100644
--- a/samza-test/src/test/scala/org/apache/samza/test/integration/TestStatefulTask.scala
+++ b/samza-test/src/test/scala/org/apache/samza/test/integration/TestStatefulTask.scala
@@ -30,8 +30,16 @@ import org.junit.{AfterClass, BeforeClass, Test}
 import scala.collection.JavaConverters._
 
 object TestStatefulTask {
-    val STORE_NAME = "mystore"
-    val STATE_TOPIC_STREAM = "mystoreChangelog"
+  val STORE_NAME = "mystore"
+  val STATE_TOPIC_STREAM = "mystoreChangelog"
+
+  // Messages with one dupe and one delete. A negative string means delete. See StateStoreTestTask.testProcess()
+  val MESSAGES_SEND_1 = List("1", "2", "3", "2", "99", "-99")
+  val MESSAGES_RECV_1 = List("1", "2", "3", "2", "99", null)
+  val STORE_CONTENTS_1 = List("1", "2", "3")
+
+  val MESSAGES_SEND_2 = List("4", "5", "5")
+  val MESSAGES_RECV_2 = List("4", "5", "5")
 
   @BeforeClass
   def beforeSetupServers {
@@ -47,13 +55,13 @@ object TestStatefulTask {
 /**
  * Test that does the following:
  * 1. Start a single partition of TestStateStoreTask using ThreadJobFactory.
- * 2. Send four messages to input (1,2,3,2), which contain one dupe (2).
+ * 2. Send MESSAGES_SEND_1, which contains a dupe and a delete.
  * 3. Validate that all messages were received by TestStateStoreTask.
- * 4. Validate that TestStateStoreTask called store.put() for all four messages, and that the messages ended up in the mystore topic.
+ * 4. Validate that TestStateStoreTask called store.put() for all messages, and that the messages ended up in the mystore topic.
  * 5. Kill the job.
  * 6. Start the job again.
- * 7. Validate that the job restored all messages (1,2,3) to the store.
- * 8. Send three more messages to input (4,5,5), and validate that TestStateStoreTask receives them.
+ * 7. Validate that the job restored all messages STORE_CONTENTS_1 to the store.
+ * 8. Send three more messages to input MESSAGES_SEND_2, and validate that TestStateStoreTask receives them.
  * 9. Kill the job again.
  */
 class TestStatefulTask extends StreamTaskTestUtil {
@@ -86,23 +94,12 @@ class TestStatefulTask extends StreamTaskTestUtil {
     assertEquals(0, task.received.size)
 
     // Send some messages to input stream.
-    send(task, "1")
-    send(task, "2")
-    send(task, "3")
-    send(task, "2")
-    send(task, "99")
-    send(task, "-99")
+    TestStatefulTask.MESSAGES_SEND_1.foreach(m => send(task, m))
 
     // Validate that messages appear in store stream.
     val messages = readAll(TestStatefulTask.STATE_TOPIC_STREAM, 5, "testShouldStartTaskForFirstTime")
 
-    assertEquals(6, messages.length)
-    assertEquals("1", messages(0))
-    assertEquals("2", messages(1))
-    assertEquals("3", messages(2))
-    assertEquals("2", messages(3))
-    assertEquals("99", messages(4))
-    assertNull(messages(5))
+    assertEquals(TestStatefulTask.MESSAGES_RECV_1, messages)
 
     stopJob(job)
   }
@@ -111,52 +108,34 @@ class TestStatefulTask extends StreamTaskTestUtil {
     val (job, task) = startJob
 
     // Validate that restored has expected data.
-    assertEquals(3, task.asInstanceOf[StateStoreTestTask].restored.size)
-    assertTrue(task.asInstanceOf[StateStoreTestTask].restored.contains("1"))
-    assertTrue(task.asInstanceOf[StateStoreTestTask].restored.contains("2"))
-    assertTrue(task.asInstanceOf[StateStoreTestTask].restored.contains("3"))
+    assertEquals(TestStatefulTask.STORE_CONTENTS_1.length, task.asInstanceOf[StateStoreTestTask].restored.size)
+    TestStatefulTask.STORE_CONTENTS_1.foreach(m =>  assertTrue(task.asInstanceOf[StateStoreTestTask].restored.contains(m)))
 
     var count = 0
 
-    // We should get the original four messages in the stream (1,2,3,2).
+    // We should get the original size messages in the stream (1,2,3,2,99,-99).
     // Note that this will trigger four new outgoing messages to the STATE_TOPIC.
-    while (task.received.size < 4 && count < 100) {
+    while (task.received.size < TestStatefulTask.MESSAGES_RECV_1.length && count < 100) {
       Thread.sleep(600)
       count += 1
     }
 
     assertTrue("Timed out waiting to received messages. Received thus far: " + task.received.size, count < 100)
 
-    // Reset the count down latch after the 4 messages come in.
+    // Reset the count down latch after the 6 messages come in.
     task.awaitMessage
 
     // Send some messages to input stream.
-    send(task, "4")
-    send(task, "5")
-    send(task, "5")
+    TestStatefulTask.MESSAGES_SEND_2.foreach(m => send(task, m))
+
+    val expectedMessagesRcvd =  TestStatefulTask.MESSAGES_RECV_1 ++ // From initial start.
+                                TestStatefulTask.MESSAGES_RECV_1 ++ // From second startup.
+                                TestStatefulTask.MESSAGES_RECV_2    // From sending in this method.
 
     // Validate that messages appear in store stream.
-    val messages = readAll(TestStatefulTask.STATE_TOPIC_STREAM, 14, "testShouldRestoreStore")
-
-    assertEquals(15, messages.length)
-    // From initial start.
-    assertEquals("1", messages(0))
-    assertEquals("2", messages(1))
-    assertEquals("3", messages(2))
-    assertEquals("2", messages(3))
-    assertEquals("99", messages(4))
-    assertNull(messages(5))
-    // From second startup.
-    assertEquals("1", messages(6))
-    assertEquals("2", messages(7))
-    assertEquals("3", messages(8))
-    assertEquals("2", messages(9))
-    assertEquals("99", messages(10))
-    assertNull(messages(11))
-    // From sending in this method.
-    assertEquals("4", messages(12))
-    assertEquals("5", messages(13))
-    assertEquals("5", messages(14))
+    val messages = readAll(TestStatefulTask.STATE_TOPIC_STREAM, expectedMessagesRcvd.length-1, "testShouldRestoreStore")
+
+    assertEquals(expectedMessagesRcvd, messages)
 
     stopJob(job)
   }


[26/26] samza git commit: Fix some integration tests after merging from master

Posted by xi...@apache.org.
Fix some integration tests after merging from master


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

Branch: refs/heads/0.14.0
Commit: ebce13e7461d80f2331b4005cb287719cdad88ad
Parents: 1701ea8
Author: Xinyu Liu <xi...@xiliu-ld1.linkedin.biz>
Authored: Tue Sep 12 14:16:27 2017 -0700
Committer: Xinyu Liu <xi...@xiliu-ld1.linkedin.biz>
Committed: Tue Sep 12 14:16:27 2017 -0700

----------------------------------------------------------------------
 .../PassthroughCoordinationUtilsFactory.java    | 30 ++++++++++++++++++++
 .../EndOfStreamIntegrationTest.java             |  3 +-
 .../WatermarkIntegrationTest.java               |  2 ++
 3 files changed, 34 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/samza/blob/ebce13e7/samza-core/src/main/java/org/apache/samza/standalone/PassthroughCoordinationUtilsFactory.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/standalone/PassthroughCoordinationUtilsFactory.java b/samza-core/src/main/java/org/apache/samza/standalone/PassthroughCoordinationUtilsFactory.java
new file mode 100644
index 0000000..ea38c88
--- /dev/null
+++ b/samza-core/src/main/java/org/apache/samza/standalone/PassthroughCoordinationUtilsFactory.java
@@ -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.standalone;
+
+import org.apache.samza.config.Config;
+import org.apache.samza.coordinator.CoordinationUtils;
+import org.apache.samza.coordinator.CoordinationUtilsFactory;
+
+public class PassthroughCoordinationUtilsFactory implements CoordinationUtilsFactory {
+  @Override
+  public CoordinationUtils getCoordinationUtils(String groupId, String participantId, Config updatedConfig) {
+    return null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/ebce13e7/samza-test/src/test/java/org/apache/samza/test/controlmessages/EndOfStreamIntegrationTest.java
----------------------------------------------------------------------
diff --git a/samza-test/src/test/java/org/apache/samza/test/controlmessages/EndOfStreamIntegrationTest.java b/samza-test/src/test/java/org/apache/samza/test/controlmessages/EndOfStreamIntegrationTest.java
index 26abb13..f313348 100644
--- a/samza-test/src/test/java/org/apache/samza/test/controlmessages/EndOfStreamIntegrationTest.java
+++ b/samza-test/src/test/java/org/apache/samza/test/controlmessages/EndOfStreamIntegrationTest.java
@@ -31,6 +31,7 @@ import org.apache.samza.config.MapConfig;
 import org.apache.samza.config.TaskConfig;
 import org.apache.samza.container.grouper.task.SingleContainerGrouperFactory;
 import org.apache.samza.runtime.LocalApplicationRunner;
+import org.apache.samza.standalone.PassthroughCoordinationUtilsFactory;
 import org.apache.samza.standalone.PassthroughJobCoordinatorFactory;
 import org.apache.samza.test.controlmessages.TestData.PageView;
 import org.apache.samza.test.controlmessages.TestData.PageViewJsonSerdeFactory;
@@ -48,7 +49,6 @@ import static org.junit.Assert.assertEquals;
  */
 public class EndOfStreamIntegrationTest extends AbstractIntegrationTestHarness {
 
-
   private static final String[] PAGEKEYS = {"inbox", "home", "search", "pymk", "group", "job"};
 
   @Test
@@ -71,6 +71,7 @@ public class EndOfStreamIntegrationTest extends AbstractIntegrationTestHarness {
 
     configs.put(JobConfig.JOB_NAME(), "test-eos-job");
     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());
 

http://git-wip-us.apache.org/repos/asf/samza/blob/ebce13e7/samza-test/src/test/java/org/apache/samza/test/controlmessages/WatermarkIntegrationTest.java
----------------------------------------------------------------------
diff --git a/samza-test/src/test/java/org/apache/samza/test/controlmessages/WatermarkIntegrationTest.java b/samza-test/src/test/java/org/apache/samza/test/controlmessages/WatermarkIntegrationTest.java
index 58da8bd..2eb72fc 100644
--- a/samza-test/src/test/java/org/apache/samza/test/controlmessages/WatermarkIntegrationTest.java
+++ b/samza-test/src/test/java/org/apache/samza/test/controlmessages/WatermarkIntegrationTest.java
@@ -50,6 +50,7 @@ import org.apache.samza.runtime.LocalApplicationRunner;
 import org.apache.samza.runtime.TestLocalApplicationRunner;
 import org.apache.samza.serializers.IntegerSerdeFactory;
 import org.apache.samza.serializers.StringSerdeFactory;
+import org.apache.samza.standalone.PassthroughCoordinationUtilsFactory;
 import org.apache.samza.standalone.PassthroughJobCoordinatorFactory;
 import org.apache.samza.system.IncomingMessageEnvelope;
 import org.apache.samza.system.SystemAdmin;
@@ -124,6 +125,7 @@ public class WatermarkIntegrationTest extends AbstractIntegrationTestHarness {
 
     configs.put(JobConfig.JOB_NAME(), "test-watermark-job");
     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());
 


[07/26] samza git commit: SAMZA-1385: Fix zookeeper path conflict between LocalApplicationRunner and ZkJobCoordinator

Posted by xi...@apache.org.
SAMZA-1385: Fix zookeeper path conflict between LocalApplicationRunner and ZkJobCoordinator

Tested the fix w/ sample page view adclick joiner job.
navina sborya nickpan47 can you please take a look at the RB?

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

Reviewers: Navina Ramesh <na...@apache.org>, Yi Pan <ni...@gmail.com>, Shanthoosh Venkataraman <sv...@linkedin.com>

Closes #265 from bharathkk/master


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

Branch: refs/heads/0.14.0
Commit: ef6bb8196c669e651ef6628c7b74a5594713ee82
Parents: a0aae52
Author: Bharath Kumarasubramanian <bk...@linkedin.com>
Authored: Mon Aug 14 22:02:24 2017 -0700
Committer: navina <na...@apache.org>
Committed: Mon Aug 14 22:02:24 2017 -0700

----------------------------------------------------------------------
 .../samza/runtime/LocalApplicationRunner.java   |  80 +++++++++---
 .../samza/zk/ZkJobCoordinatorFactory.java       |  20 ++-
 .../org/apache/samza/zk/ZkProcessorLatch.java   |  11 +-
 .../runtime/TestLocalApplicationRunner.java     | 127 ++++++++++++++++---
 4 files changed, 195 insertions(+), 43 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/samza/blob/ef6bb819/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java b/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java
index b0bfc8a..588e657 100644
--- a/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java
+++ b/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java
@@ -26,6 +26,7 @@ import java.util.UUID;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicReference;
 import org.apache.samza.SamzaException;
@@ -57,13 +58,12 @@ import org.slf4j.LoggerFactory;
 public class LocalApplicationRunner extends AbstractApplicationRunner {
 
   private static final Logger LOG = LoggerFactory.getLogger(LocalApplicationRunner.class);
-  // Latch id that's used for awaiting the init of application before creating the StreamProcessors
-  private static final String INIT_LATCH_ID = "init";
+  private static final String APPLICATION_RUNNER_ZK_PATH_SUFFIX = "/ApplicationRunnerData";
   // Latch timeout is set to 10 min
   private static final long LATCH_TIMEOUT_MINUTES = 10;
+  private static final long LEADER_ELECTION_WAIT_TIME_MS = 1000;
 
   private final String uid;
-  private final CoordinationUtils coordinationUtils;
   private final Set<StreamProcessor> processors = ConcurrentHashMap.newKeySet();
   private final CountDownLatch shutdownLatch = new CountDownLatch(1);
   private final AtomicInteger numProcessorsToStart = new AtomicInteger();
@@ -122,9 +122,6 @@ public class LocalApplicationRunner extends AbstractApplicationRunner {
         }
       }
 
-      if (coordinationUtils != null) {
-        coordinationUtils.reset();
-      }
       shutdownLatch.countDown();
     }
   }
@@ -132,7 +129,6 @@ public class LocalApplicationRunner extends AbstractApplicationRunner {
   public LocalApplicationRunner(Config config) {
     super(config);
     uid = UUID.randomUUID().toString();
-    coordinationUtils = createCoordinationUtils();
   }
 
   @Override
@@ -159,10 +155,14 @@ public class LocalApplicationRunner extends AbstractApplicationRunner {
     try {
       // 1. initialize and plan
       ExecutionPlan plan = getExecutionPlan(app);
-      writePlanJsonFile(plan.getPlanAsJson());
+
+      String executionPlanJson = plan.getPlanAsJson();
+      writePlanJsonFile(executionPlanJson);
 
       // 2. create the necessary streams
-      createStreams(plan.getIntermediateStreams());
+      // TODO: System generated intermediate streams should have robust naming scheme. Refer JIRA-1391
+      String planId = String.valueOf(executionPlanJson.hashCode());
+      createStreams(planId, plan.getIntermediateStreams());
 
       // 3. create the StreamProcessors
       if (plan.getJobConfigs().isEmpty()) {
@@ -216,7 +216,8 @@ public class LocalApplicationRunner extends AbstractApplicationRunner {
     // TODO: we will need a better way to package the configs with application runner
     if (ZkJobCoordinatorFactory.class.getName().equals(jobCoordinatorFactoryClassName)) {
       ApplicationConfig appConfig = new ApplicationConfig(config);
-      return new ZkCoordinationServiceFactory().getCoordinationService(appConfig.getGlobalAppId(), uid, config);
+      return new ZkCoordinationServiceFactory().getCoordinationService(
+          appConfig.getGlobalAppId() + APPLICATION_RUNNER_ZK_PATH_SUFFIX, uid, config);
     } else {
       return null;
     }
@@ -227,20 +228,32 @@ public class LocalApplicationRunner extends AbstractApplicationRunner {
    * If {@link CoordinationUtils} is provided, this function will first invoke leader election, and the leader
    * will create the streams. All the runner processes will wait on the latch that is released after the leader finishes
    * stream creation.
+   * @param planId a unique identifier representing the plan used for coordination purpose
    * @param intStreams list of intermediate {@link StreamSpec}s
-   * @throws Exception exception for latch timeout
+   * @throws TimeoutException exception for latch timeout
    */
-  /* package private */ void createStreams(List<StreamSpec> intStreams) throws Exception {
+  /* package private */ void createStreams(String planId, List<StreamSpec> intStreams) throws TimeoutException {
     if (!intStreams.isEmpty()) {
+      // Move the scope of coordination utils within stream creation to address long idle connection problem.
+      // Refer SAMZA-1385 for more details
+      CoordinationUtils coordinationUtils = createCoordinationUtils();
       if (coordinationUtils != null) {
-        Latch initLatch = coordinationUtils.getLatch(1, INIT_LATCH_ID);
-        LeaderElector leaderElector = coordinationUtils.getLeaderElector();
-        leaderElector.setLeaderElectorListener(() -> {
-            getStreamManager().createStreams(intStreams);
-            initLatch.countDown();
-          });
-        leaderElector.tryBecomeLeader();
-        initLatch.await(LATCH_TIMEOUT_MINUTES, TimeUnit.MINUTES);
+        Latch initLatch = coordinationUtils.getLatch(1, planId);
+
+        try {
+          // check if the processor needs to go through leader election and stream creation
+          if (shouldContestInElectionForStreamCreation(initLatch)) {
+            LeaderElector leaderElector = coordinationUtils.getLeaderElector();
+            leaderElector.setLeaderElectorListener(() -> {
+                getStreamManager().createStreams(intStreams);
+                initLatch.countDown();
+              });
+            leaderElector.tryBecomeLeader();
+            initLatch.await(LATCH_TIMEOUT_MINUTES, TimeUnit.MINUTES);
+          }
+        } finally {
+          coordinationUtils.reset();
+        }
       } else {
         // each application process will try creating the streams, which
         // requires stream creation to be idempotent
@@ -272,4 +285,31 @@ public class LocalApplicationRunner extends AbstractApplicationRunner {
           taskFactory.getClass().getCanonicalName()));
     }
   }
+
+  /**
+   * In order to fix SAMZA-1385, we are limiting the scope of coordination util within stream creation phase and destroying
+   * the coordination util right after. By closing the zk connection, we clean up the ephemeral node used for leader election.
+   * It creates the following issues whenever a new process joins after the ephemeral node is gone.
+   *    1. It is unnecessary to re-conduct leader election for stream creation in the same application lifecycle
+   *    2. Underlying systems may not support check for stream existence prior to creation which could have potential problems.
+   * As in interim solution, we reuse the same latch as a marker to determine if create streams phase is done for the
+   * application lifecycle using {@link Latch#await(long, TimeUnit)}
+   *
+   * @param streamCreationLatch latch used for stream creation
+   * @return true if processor needs to be part of election
+   *         false otherwise
+   */
+  private boolean shouldContestInElectionForStreamCreation(Latch streamCreationLatch) {
+    boolean eligibleForElection = true;
+
+    try {
+      streamCreationLatch.await(LEADER_ELECTION_WAIT_TIME_MS, TimeUnit.MILLISECONDS);
+      // case we didn't time out suggesting that latch already exists
+      eligibleForElection = false;
+    } catch (TimeoutException e) {
+      LOG.info("Timed out waiting for the latch! Going to enter leader election section to create streams");
+    }
+
+    return eligibleForElection;
+  }
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/ef6bb819/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 85e3b4a..08d826e 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
@@ -22,6 +22,7 @@ package org.apache.samza.zk;
 import org.I0Itec.zkclient.ZkClient;
 import org.apache.samza.config.ApplicationConfig;
 import org.apache.samza.config.Config;
+import org.apache.samza.config.JobConfig;
 import org.apache.samza.config.ZkConfig;
 import org.apache.samza.coordinator.JobCoordinator;
 import org.apache.samza.coordinator.JobCoordinatorFactory;
@@ -30,9 +31,13 @@ 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);
+  private static final String JOB_COORDINATOR_ZK_PATH_FORMAT = "%s/%s-%s-coordinationData";
+  private static final String DEFAULT_JOB_ID = "1";
+  private static final String DEFAULT_JOB_NAME = "defaultJob";
 
   /**
    * Method to instantiate an implementation of JobCoordinator
@@ -50,8 +55,21 @@ public class ZkJobCoordinatorFactory implements JobCoordinatorFactory {
 
   private ZkUtils getZkUtils(Config config, MetricsRegistry metricsRegistry) {
     ZkConfig zkConfig = new ZkConfig(config);
-    ZkKeyBuilder keyBuilder = new ZkKeyBuilder(new ApplicationConfig(config).getGlobalAppId());
+    ZkKeyBuilder keyBuilder = new ZkKeyBuilder(getJobCoordinationZkPath(config));
     ZkClient zkClient = ZkCoordinationServiceFactory.createZkClient(zkConfig.getZkConnect(), zkConfig.getZkSessionTimeoutMs(), zkConfig.getZkConnectionTimeoutMs());
     return new ZkUtils(keyBuilder, zkClient, zkConfig.getZkConnectionTimeoutMs(), metricsRegistry);
   }
+
+  private String getJobCoordinationZkPath(Config config) {
+    JobConfig jobConfig = new JobConfig(config);
+    String appId = new ApplicationConfig(config).getGlobalAppId();
+    String jobName = jobConfig.getName().isDefined()
+        ? jobConfig.getName().get()
+        : DEFAULT_JOB_NAME;
+    String jobId = jobConfig.getJobId().isDefined()
+        ? jobConfig.getJobId().get()
+        : DEFAULT_JOB_ID;
+
+    return String.format(JOB_COORDINATOR_ZK_PATH_FORMAT, appId, jobName, jobId);
+  }
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/ef6bb819/samza-core/src/main/java/org/apache/samza/zk/ZkProcessorLatch.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/zk/ZkProcessorLatch.java b/samza-core/src/main/java/org/apache/samza/zk/ZkProcessorLatch.java
index decdd7d..166c627 100644
--- a/samza-core/src/main/java/org/apache/samza/zk/ZkProcessorLatch.java
+++ b/samza-core/src/main/java/org/apache/samza/zk/ZkProcessorLatch.java
@@ -20,6 +20,7 @@ package org.apache.samza.zk;
 
 import java.util.concurrent.TimeUnit;
 
+import java.util.concurrent.TimeoutException;
 import org.apache.samza.coordinator.Latch;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -53,8 +54,14 @@ public class ZkProcessorLatch implements Latch {
   }
 
   @Override
-  public void await(long timeout, TimeUnit timeUnit) {
-    zkUtils.getZkClient().waitUntilExists(targetPath, timeUnit, timeout);
+  public void await(long timeout, TimeUnit timeUnit) throws TimeoutException {
+    // waitUntilExists signals timeout by returning false as opposed to throwing exception. We internally need to map
+    // the non-existence to a TimeoutException in order to respect the contract defined in Latch interface
+    boolean targetPathExists = zkUtils.getZkClient().waitUntilExists(targetPath, timeUnit, timeout);
+
+    if (!targetPathExists) {
+      throw new TimeoutException("Timed out waiting for the targetPath");
+    }
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/samza/blob/ef6bb819/samza-core/src/test/java/org/apache/samza/runtime/TestLocalApplicationRunner.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/runtime/TestLocalApplicationRunner.java b/samza-core/src/test/java/org/apache/samza/runtime/TestLocalApplicationRunner.java
index a04bd3b..6c7827e 100644
--- a/samza-core/src/test/java/org/apache/samza/runtime/TestLocalApplicationRunner.java
+++ b/samza-core/src/test/java/org/apache/samza/runtime/TestLocalApplicationRunner.java
@@ -19,6 +19,15 @@
 
 package org.apache.samza.runtime;
 
+import com.google.common.collect.ImmutableList;
+import java.lang.reflect.Field;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.stream.Collectors;
 import org.apache.samza.application.StreamApplication;
 import org.apache.samza.config.ApplicationConfig;
 import org.apache.samza.config.JobConfig;
@@ -38,31 +47,31 @@ import org.apache.samza.system.StreamSpec;
 import org.junit.Test;
 import org.mockito.ArgumentCaptor;
 
-import java.lang.reflect.Field;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.*;
 import static org.mockito.Matchers.anyInt;
 import static org.mockito.Matchers.anyObject;
 import static org.mockito.Matchers.anyString;
-import static org.mockito.Mockito.doAnswer;
-import static org.mockito.Mockito.doNothing;
-import static org.mockito.Mockito.doReturn;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.spy;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.when;
+import static org.mockito.Mockito.*;
 
 
 public class TestLocalApplicationRunner {
 
+  private static final String PLAN_JSON = "{"
+      + "\"jobs\":[{"
+      + "\"jobName\":\"test-application\","
+      + "\"jobId\":\"1\","
+      + "\"operatorGraph\":{"
+      + "\"intermediateStreams\":{%s},"
+      + "\"applicationName\":\"test-application\",\"applicationId\":\"1\"}";
+  private static final String STREAM_SPEC_JSON_FORMAT = "\"%s\":{"
+      + "\"streamSpec\":{"
+      + "\"id\":\"%s\","
+      + "\"systemName\":\"%s\","
+      + "\"physicalName\":\"%s\","
+      + "\"partitionCount\":2},"
+      + "\"sourceJobs\":[\"test-app\"],"
+      + "\"targetJobs\":[\"test-target-app\"]},";
+
   @Test
   public void testStreamCreation() throws Exception {
     Map<String, String> config = new HashMap<>();
@@ -180,8 +189,10 @@ public class TestLocalApplicationRunner {
       @Override
       public void await(long timeout, TimeUnit tu)
           throws TimeoutException {
-        // in this test, latch is released before wait
-        assertTrue(done);
+        // in this test, latch is released after countDown is invoked
+        if (!done) {
+          throw new TimeoutException("timed out waiting for the target path");
+        }
       }
 
       @Override
@@ -343,4 +354,80 @@ public class TestLocalApplicationRunner {
     assertEquals(spy.status(app), ApplicationStatus.UnsuccessfulFinish);
   }
 
+  /**
+   * A test case to verify if the plan results in different hash if there is change in topological sort order.
+   * Note: the overall JOB PLAN remains the same outside the scope of intermediate streams the sake of these test cases.
+   */
+  @Test
+  public void testPlanIdWithShuffledStreamSpecs() {
+    List<StreamSpec> streamSpecs = ImmutableList.of(
+      new StreamSpec("test-stream-1", "stream-1", "testStream"),
+        new StreamSpec("test-stream-2", "stream-2", "testStream"),
+        new StreamSpec("test-stream-3", "stream-3", "testStream"));
+    String planIdBeforeShuffle = getExecutionPlanId(streamSpecs);
+
+    List<StreamSpec> shuffledStreamSpecs = ImmutableList.of(
+        new StreamSpec("test-stream-2", "stream-2", "testStream"),
+        new StreamSpec("test-stream-1", "stream-1", "testStream"),
+        new StreamSpec("test-stream-3", "stream-3", "testStream"));
+
+    assertNotEquals("Expected both of the latch ids to be different", planIdBeforeShuffle,
+        getExecutionPlanId(shuffledStreamSpecs));
+  }
+
+  /**
+   * A test case to verify if the plan results in same hash in case of same plan.
+   * Note: the overall JOB PLAN remains the same outside the scope of intermediate streams the sake of these test cases.
+   */
+  @Test
+  public void testGeneratePlanIdWithSameStreamSpecs() {
+    List<StreamSpec> streamSpecs = ImmutableList.of(
+        new StreamSpec("test-stream-1", "stream-1", "testStream"),
+        new StreamSpec("test-stream-2", "stream-2", "testStream"),
+        new StreamSpec("test-stream-3", "stream-3", "testStream"));
+    String planIdForFirstAttempt = getExecutionPlanId(streamSpecs);
+    String planIdForSecondAttempt = getExecutionPlanId(streamSpecs);
+
+    assertEquals("Expected latch ids to match!", "1447946713", planIdForFirstAttempt);
+    assertEquals("Expected latch ids to match for the second attempt!", planIdForFirstAttempt, planIdForSecondAttempt);
+  }
+
+  /**
+   * A test case to verify plan results in different hash in case of different intermediate stream.
+   * Note: the overall JOB PLAN remains the same outside the scope of intermediate streams the sake of these test cases.
+   */
+  @Test
+  public void testGeneratePlanIdWithDifferentStreamSpecs() {
+    List<StreamSpec> streamSpecs = ImmutableList.of(
+        new StreamSpec("test-stream-1", "stream-1", "testStream"),
+        new StreamSpec("test-stream-2", "stream-2", "testStream"),
+        new StreamSpec("test-stream-3", "stream-3", "testStream"));
+    String planIdBeforeShuffle = getExecutionPlanId(streamSpecs);
+
+    List<StreamSpec> updatedStreamSpecs = ImmutableList.of(
+        new StreamSpec("test-stream-1", "stream-1", "testStream"),
+        new StreamSpec("test-stream-4", "stream-4", "testStream"),
+        new StreamSpec("test-stream-3", "stream-3", "testStream"));
+
+    assertNotEquals("Expected both of the latch ids to be different", planIdBeforeShuffle,
+        getExecutionPlanId(updatedStreamSpecs));
+  }
+
+  private String getExecutionPlanId(List<StreamSpec> updatedStreamSpecs) {
+    String intermediateStreamJson = updatedStreamSpecs.stream()
+        .map(this::streamSpecToJson)
+        .collect(Collectors.joining(","));
+
+    int planId = String.format(PLAN_JSON, intermediateStreamJson).hashCode();
+
+    return String.valueOf(planId);
+  }
+
+  private String streamSpecToJson(StreamSpec streamSpec) {
+    return String.format(STREAM_SPEC_JSON_FORMAT,
+        streamSpec.getId(),
+        streamSpec.getId(),
+        streamSpec.getSystemName(),
+        streamSpec.getPhysicalName());
+  }
 }


[22/26] samza git commit: Introduces CoordinationUtilsFactory to create different implementations of CoordinationUtils

Posted by xi...@apache.org.
Introduces CoordinationUtilsFactory to create different implementations of CoordinationUtils

Some refactoring and cleanup.

Author: Boris Shkolnik <bo...@apache.org>

Reviewers: Navina Ramesh <na...@apache.org>

Closes #282 from sborya/CoordinationUtilsFactory


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

Branch: refs/heads/0.14.0
Commit: 81b4e6361f8c05e2a670ba6263c3786d0eea9f62
Parents: 6574386
Author: Boris Shkolnik <bo...@apache.org>
Authored: Tue Aug 29 13:23:47 2017 -0700
Committer: navina <na...@apache.org>
Committed: Tue Aug 29 13:23:47 2017 -0700

----------------------------------------------------------------------
 build.gradle                                    |  4 +-
 .../AzureCoordinationServiceFactory.java        | 30 ------
 .../coordinator/AzureCoordinationUtils.java     |  9 +-
 .../AzureCoordinationUtilsFactory.java          | 30 ++++++
 .../samza/config/JobCoordinatorConfig.java      | 23 +++++
 .../coordinator/CoordinationServiceFactory.java | 36 --------
 .../samza/coordinator/CoordinationUtils.java    | 11 +--
 .../coordinator/CoordinationUtilsFactory.java   | 47 ++++++++++
 .../samza/runtime/LocalApplicationRunner.java   | 29 ++----
 .../samza/zk/ZkCoordinationServiceFactory.java  | 89 ------------------
 .../apache/samza/zk/ZkCoordinationUtils.java    | 21 +++--
 .../samza/zk/ZkCoordinationUtilsFactory.java    | 89 ++++++++++++++++++
 .../samza/zk/ZkJobCoordinatorFactory.java       |  3 +-
 .../org/apache/samza/zk/ZkProcessorLatch.java   |  8 +-
 .../runtime/TestLocalApplicationRunner.java     | 96 +++++++++++---------
 .../apache/samza/zk/TestZkLeaderElector.java    |  3 +-
 .../org/apache/samza/zk/TestZkNamespace.java    |  8 +-
 .../apache/samza/zk/TestZkProcessorLatch.java   |  2 +-
 .../processor/TestZkLocalApplicationRunner.java |  2 +-
 19 files changed, 285 insertions(+), 255 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/samza/blob/81b4e636/build.gradle
----------------------------------------------------------------------
diff --git a/build.gradle b/build.gradle
index 16fe2cf..319e51f 100644
--- a/build.gradle
+++ b/build.gradle
@@ -155,7 +155,9 @@ project(":samza-core_$scalaVersion") {
 
   dependencies {
     compile project(':samza-api')
-    compile "com.101tec:zkclient:$zkClientVersion"
+    compile("com.101tec:zkclient:$zkClientVersion") {
+      exclude module: 'junit:junit'
+    }
     compile "com.google.guava:guava:$guavaVersion"
     compile "net.sf.jopt-simple:jopt-simple:$joptSimpleVersion"
     compile "org.apache.commons:commons-collections4:$apacheCommonsCollections4Version"

http://git-wip-us.apache.org/repos/asf/samza/blob/81b4e636/samza-azure/src/main/java/org/apache/samza/coordinator/AzureCoordinationServiceFactory.java
----------------------------------------------------------------------
diff --git a/samza-azure/src/main/java/org/apache/samza/coordinator/AzureCoordinationServiceFactory.java b/samza-azure/src/main/java/org/apache/samza/coordinator/AzureCoordinationServiceFactory.java
deleted file mode 100644
index 8016fbe..0000000
--- a/samza-azure/src/main/java/org/apache/samza/coordinator/AzureCoordinationServiceFactory.java
+++ /dev/null
@@ -1,30 +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.coordinator;
-
-import org.apache.samza.config.Config;
-
-public class AzureCoordinationServiceFactory implements CoordinationServiceFactory {
-
-  @Override
-  public CoordinationUtils getCoordinationService(String groupId, String participantId, Config config) {
-    return new AzureCoordinationUtils(config);
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/81b4e636/samza-azure/src/main/java/org/apache/samza/coordinator/AzureCoordinationUtils.java
----------------------------------------------------------------------
diff --git a/samza-azure/src/main/java/org/apache/samza/coordinator/AzureCoordinationUtils.java b/samza-azure/src/main/java/org/apache/samza/coordinator/AzureCoordinationUtils.java
index f76dde3..dbd945f 100644
--- a/samza-azure/src/main/java/org/apache/samza/coordinator/AzureCoordinationUtils.java
+++ b/samza-azure/src/main/java/org/apache/samza/coordinator/AzureCoordinationUtils.java
@@ -36,10 +36,6 @@ public class AzureCoordinationUtils implements CoordinationUtils {
   }
 
   @Override
-  public void reset() {}
-
-
-  @Override
   public LeaderElector getLeaderElector() throws UnsupportedOperationException {
     return null;
   }
@@ -49,6 +45,11 @@ public class AzureCoordinationUtils implements CoordinationUtils {
     return null;
   }
 
+  @Override
+  public void close() {
+
+  }
+
   public DistributedLock getLock(String initLockName) {
     BlobUtils blob = new BlobUtils(client, azureConfig.getAzureContainerName(),
         azureConfig.getAzureBlobName() + initLockName, azureConfig.getAzureBlobLength());

http://git-wip-us.apache.org/repos/asf/samza/blob/81b4e636/samza-azure/src/main/java/org/apache/samza/coordinator/AzureCoordinationUtilsFactory.java
----------------------------------------------------------------------
diff --git a/samza-azure/src/main/java/org/apache/samza/coordinator/AzureCoordinationUtilsFactory.java b/samza-azure/src/main/java/org/apache/samza/coordinator/AzureCoordinationUtilsFactory.java
new file mode 100644
index 0000000..00ec016
--- /dev/null
+++ b/samza-azure/src/main/java/org/apache/samza/coordinator/AzureCoordinationUtilsFactory.java
@@ -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.coordinator;
+
+import org.apache.samza.config.Config;
+
+public class AzureCoordinationUtilsFactory implements CoordinationUtilsFactory {
+
+  @Override
+  public CoordinationUtils getCoordinationUtils(String groupId, String participantId, Config config) {
+    return new AzureCoordinationUtils(config);
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/81b4e636/samza-core/src/main/java/org/apache/samza/config/JobCoordinatorConfig.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/config/JobCoordinatorConfig.java b/samza-core/src/main/java/org/apache/samza/config/JobCoordinatorConfig.java
index 57632ca..a04038a 100644
--- a/samza-core/src/main/java/org/apache/samza/config/JobCoordinatorConfig.java
+++ b/samza-core/src/main/java/org/apache/samza/config/JobCoordinatorConfig.java
@@ -20,14 +20,37 @@
 package org.apache.samza.config;
 
 import com.google.common.base.Strings;
+import org.apache.samza.SamzaException;
+import org.apache.samza.zk.ZkCoordinationUtilsFactory;
+import org.apache.samza.zk.ZkJobCoordinatorFactory;
+
 
 public class JobCoordinatorConfig extends MapConfig {
   public static final String JOB_COORDINATOR_FACTORY = "job.coordinator.factory";
+  public static final String JOB_COORDINATION_UTILS_FACTORY = "job.coordination.utils.factory";
+  public final static String DEFAULT_COORDINATION_UTILS_FACTORY = ZkCoordinationUtilsFactory.class.getName();
 
   public JobCoordinatorConfig(Config config) {
     super(config);
   }
 
+  public String getJobCoordinationUtilsFactoryClassName() {
+    String jobCoordinatorFactoryClassName = get(JobCoordinatorConfig.JOB_COORDINATOR_FACTORY, "");
+
+    String className = get(JOB_COORDINATION_UTILS_FACTORY, "");
+
+    if (!Strings.isNullOrEmpty(className)) {
+      return className;
+    }
+
+    // TODO: we will need a better way to package the configs with application runner
+    if (ZkJobCoordinatorFactory.class.getName().equals(jobCoordinatorFactoryClassName)) {
+      return DEFAULT_COORDINATION_UTILS_FACTORY;
+    }
+
+    throw new SamzaException("Cannot determine which CoordinationUtilsFactory to load");
+  }
+
   public String getJobCoordinatorFactoryClassName() {
     String jobCoordinatorFactoryClassName = get(JOB_COORDINATOR_FACTORY);
     if (Strings.isNullOrEmpty(jobCoordinatorFactoryClassName)) {

http://git-wip-us.apache.org/repos/asf/samza/blob/81b4e636/samza-core/src/main/java/org/apache/samza/coordinator/CoordinationServiceFactory.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/coordinator/CoordinationServiceFactory.java b/samza-core/src/main/java/org/apache/samza/coordinator/CoordinationServiceFactory.java
deleted file mode 100644
index 497d3e0..0000000
--- a/samza-core/src/main/java/org/apache/samza/coordinator/CoordinationServiceFactory.java
+++ /dev/null
@@ -1,36 +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.coordinator;
-
-import org.apache.samza.config.Config;
-
-
-/**
- * factory to instantiate a c{@link CoordinationUtils} service
- */
-public interface CoordinationServiceFactory {
-  /**
-   * get a unique service instance
-   * @param groupId - unique id to identify the service
-   * @param participantId - a unique id that identifies the participant in the service
-   * @param updatedConfig - configs, to define the details of the service
-   * @return a unique service instance
-   */
-  CoordinationUtils getCoordinationService(String groupId, String participantId, Config updatedConfig);
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/81b4e636/samza-core/src/main/java/org/apache/samza/coordinator/CoordinationUtils.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/coordinator/CoordinationUtils.java b/samza-core/src/main/java/org/apache/samza/coordinator/CoordinationUtils.java
index 150b3d4..4ba44b5 100644
--- a/samza-core/src/main/java/org/apache/samza/coordinator/CoordinationUtils.java
+++ b/samza-core/src/main/java/org/apache/samza/coordinator/CoordinationUtils.java
@@ -31,14 +31,13 @@ import org.apache.samza.annotation.InterfaceStability;
 @InterfaceStability.Evolving
 public interface CoordinationUtils {
 
-  /**
-   * reset the internal structure. Does not happen automatically with stop()
-   */
-  void reset();
-
-
   // facilities for group coordination
   LeaderElector getLeaderElector(); // leaderElector is unique based on the groupId
 
   Latch getLatch(int size, String latchId);
+
+  /**
+   * performs necessary cleanup and closes ALL the utils.
+   */
+  void close();
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/81b4e636/samza-core/src/main/java/org/apache/samza/coordinator/CoordinationUtilsFactory.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/coordinator/CoordinationUtilsFactory.java b/samza-core/src/main/java/org/apache/samza/coordinator/CoordinationUtilsFactory.java
new file mode 100644
index 0000000..914216b
--- /dev/null
+++ b/samza-core/src/main/java/org/apache/samza/coordinator/CoordinationUtilsFactory.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.coordinator;
+
+import org.apache.samza.config.Config;
+import org.apache.samza.config.JobCoordinatorConfig;
+import org.apache.samza.util.*;
+
+
+/**
+ * factory to instantiate a c{@link CoordinationUtils} service
+ */
+public interface CoordinationUtilsFactory {
+
+  public static CoordinationUtilsFactory getCoordinationUtilsFactory(Config config) {
+    // load the class
+    JobCoordinatorConfig jcConfig = new JobCoordinatorConfig(config);
+    String coordinationUtilsFactoryClass =   jcConfig.getJobCoordinationUtilsFactoryClassName();
+
+    return ClassLoaderHelper.fromClassName(coordinationUtilsFactoryClass, CoordinationUtilsFactory.class);
+  }
+
+  /**
+   * get a unique service instance
+   * @param groupId - unique id to identify the service
+   * @param participantId - a unique id that identifies the participant in the service
+   * @param updatedConfig - configs, to define the details of the service
+   * @return a unique service instance
+   */
+  CoordinationUtils getCoordinationUtils(String groupId, String participantId, Config updatedConfig);
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/81b4e636/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java b/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java
index 588e657..fc11cf5 100644
--- a/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java
+++ b/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java
@@ -34,9 +34,9 @@ import org.apache.samza.application.StreamApplication;
 import org.apache.samza.config.ApplicationConfig;
 import org.apache.samza.config.Config;
 import org.apache.samza.config.JobConfig;
-import org.apache.samza.config.JobCoordinatorConfig;
 import org.apache.samza.config.TaskConfig;
 import org.apache.samza.coordinator.CoordinationUtils;
+import org.apache.samza.coordinator.CoordinationUtilsFactory;
 import org.apache.samza.coordinator.Latch;
 import org.apache.samza.coordinator.LeaderElector;
 import org.apache.samza.execution.ExecutionPlan;
@@ -47,8 +47,6 @@ import org.apache.samza.system.StreamSpec;
 import org.apache.samza.task.AsyncStreamTaskFactory;
 import org.apache.samza.task.StreamTaskFactory;
 import org.apache.samza.task.TaskFactoryUtil;
-import org.apache.samza.zk.ZkCoordinationServiceFactory;
-import org.apache.samza.zk.ZkJobCoordinatorFactory;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -207,23 +205,6 @@ public class LocalApplicationRunner extends AbstractApplicationRunner {
   }
 
   /**
-   * Create the {@link CoordinationUtils} needed by the application runner, or null if it's not configured.
-   * @return an instance of {@link CoordinationUtils}
-   */
-  /* package private */ CoordinationUtils createCoordinationUtils() {
-    String jobCoordinatorFactoryClassName = config.get(JobCoordinatorConfig.JOB_COORDINATOR_FACTORY, "");
-
-    // TODO: we will need a better way to package the configs with application runner
-    if (ZkJobCoordinatorFactory.class.getName().equals(jobCoordinatorFactoryClassName)) {
-      ApplicationConfig appConfig = new ApplicationConfig(config);
-      return new ZkCoordinationServiceFactory().getCoordinationService(
-          appConfig.getGlobalAppId() + APPLICATION_RUNNER_ZK_PATH_SUFFIX, uid, config);
-    } else {
-      return null;
-    }
-  }
-
-  /**
    * Create intermediate streams using {@link org.apache.samza.execution.StreamManager}.
    * If {@link CoordinationUtils} is provided, this function will first invoke leader election, and the leader
    * will create the streams. All the runner processes will wait on the latch that is released after the leader finishes
@@ -236,7 +217,10 @@ public class LocalApplicationRunner extends AbstractApplicationRunner {
     if (!intStreams.isEmpty()) {
       // Move the scope of coordination utils within stream creation to address long idle connection problem.
       // Refer SAMZA-1385 for more details
-      CoordinationUtils coordinationUtils = createCoordinationUtils();
+
+      String coordinationId = new ApplicationConfig(config).getGlobalAppId() + APPLICATION_RUNNER_ZK_PATH_SUFFIX;
+      CoordinationUtils coordinationUtils =
+          CoordinationUtilsFactory.getCoordinationUtilsFactory(config).getCoordinationUtils(coordinationId, uid, config);
       if (coordinationUtils != null) {
         Latch initLatch = coordinationUtils.getLatch(1, planId);
 
@@ -252,7 +236,8 @@ public class LocalApplicationRunner extends AbstractApplicationRunner {
             initLatch.await(LATCH_TIMEOUT_MINUTES, TimeUnit.MINUTES);
           }
         } finally {
-          coordinationUtils.reset();
+          if (initLatch != null)
+            coordinationUtils.close();
         }
       } else {
         // each application process will try creating the streams, which

http://git-wip-us.apache.org/repos/asf/samza/blob/81b4e636/samza-core/src/main/java/org/apache/samza/zk/ZkCoordinationServiceFactory.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/zk/ZkCoordinationServiceFactory.java b/samza-core/src/main/java/org/apache/samza/zk/ZkCoordinationServiceFactory.java
deleted file mode 100644
index 1dd5ec9..0000000
--- a/samza-core/src/main/java/org/apache/samza/zk/ZkCoordinationServiceFactory.java
+++ /dev/null
@@ -1,89 +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.zk;
-
-import com.google.common.base.Strings;
-import org.I0Itec.zkclient.ZkClient;
-import org.apache.samza.SamzaException;
-import org.apache.samza.config.Config;
-import org.apache.samza.config.ZkConfig;
-import org.apache.samza.coordinator.CoordinationServiceFactory;
-import org.apache.samza.coordinator.CoordinationUtils;
-import org.apache.samza.util.NoOpMetricsRegistry;
-import org.apache.zookeeper.client.ConnectStringParser;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-
-public class ZkCoordinationServiceFactory implements CoordinationServiceFactory {
-  private static final Logger LOG = LoggerFactory.getLogger(ZkCoordinationServiceFactory.class);
-
-  public CoordinationUtils getCoordinationService(String groupId, String participantId, Config config) {
-    ZkConfig zkConfig = new ZkConfig(config);
-
-    ZkClient zkClient =
-        createZkClient(zkConfig.getZkConnect(), zkConfig.getZkSessionTimeoutMs(), zkConfig.getZkConnectionTimeoutMs());
-
-    ZkUtils zkUtils = new ZkUtils(new ZkKeyBuilder(groupId), zkClient, zkConfig.getZkConnectionTimeoutMs(), new NoOpMetricsRegistry());
-    return new ZkCoordinationUtils(participantId, zkConfig, zkUtils);
-  }
-
-  /**
-   * helper method to create zkClient
-   * @param connectString - zkConnect string
-   * @param sessionTimeoutMS - session timeout
-   * @param connectionTimeoutMs - connection timeout
-   * @return zkClient object
-   */
-  public static ZkClient createZkClient(String connectString, int sessionTimeoutMS, int connectionTimeoutMs) {
-    ZkClient zkClient;
-    try {
-      zkClient = new ZkClient(connectString, sessionTimeoutMS, connectionTimeoutMs);
-    } catch (Exception e) {
-      // ZkClient constructor may throw a variety of different exceptions, not all of them Zk based.
-      throw new SamzaException("zkClient failed to connect to ZK at :" + connectString, e);
-    }
-
-    // make sure the namespace in zk exists (if specified)
-    validateZkNameSpace(connectString, zkClient);
-
-    return zkClient;
-  }
-
-  /**
-   * if ZkConnectString contains namespace path at the end, but it does not exist we should fail
-   * @param zkConnect - connect string
-   * @param zkClient - zkClient object to talk to the ZK
-   */
-  public static void validateZkNameSpace(String zkConnect, ZkClient zkClient) {
-    ConnectStringParser parser = new ConnectStringParser(zkConnect);
-
-    String path = parser.getChrootPath();
-    if (Strings.isNullOrEmpty(path)) {
-      return; // no namespace path
-    }
-
-    LOG.info("connectString = " + zkConnect + "; path =" + path);
-
-    // if namespace specified (path above) but "/" does not exists, we will fail
-    if (!zkClient.exists("/")) {
-      throw new SamzaException("Zookeeper namespace: " + path + " does not exist for zk at " + zkConnect);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/81b4e636/samza-core/src/main/java/org/apache/samza/zk/ZkCoordinationUtils.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/zk/ZkCoordinationUtils.java b/samza-core/src/main/java/org/apache/samza/zk/ZkCoordinationUtils.java
index f5dda2e..05886db 100644
--- a/samza-core/src/main/java/org/apache/samza/zk/ZkCoordinationUtils.java
+++ b/samza-core/src/main/java/org/apache/samza/zk/ZkCoordinationUtils.java
@@ -41,16 +41,6 @@ public class ZkCoordinationUtils implements CoordinationUtils {
   }
 
   @Override
-  public void reset() {
-    try {
-      zkUtils.close();
-    } catch (ZkInterruptedException ex) {
-      // Swallowing due to occurrence in the last stage of lifecycle(Not actionable).
-      LOG.error("Exception in reset: ", ex);
-    }
-  }
-
-  @Override
   public LeaderElector getLeaderElector() {
     return new ZkLeaderElector(processorIdStr, zkUtils);
   }
@@ -60,6 +50,17 @@ public class ZkCoordinationUtils implements CoordinationUtils {
     return new ZkProcessorLatch(size, latchId, processorIdStr, zkUtils);
   }
 
+  @Override
+  public void close() {
+    try {
+      if (zkUtils != null)
+        zkUtils.close();
+    } catch (ZkInterruptedException ex) {
+      // Swallowing due to occurrence in the last stage of lifecycle(Not actionable).
+      LOG.error("Exception in close(): ", ex);
+    }
+  }
+
   // TODO - SAMZA-1128 CoordinationService should directly depend on ZkUtils and DebounceTimer
   public ZkUtils getZkUtils() {
     return zkUtils;

http://git-wip-us.apache.org/repos/asf/samza/blob/81b4e636/samza-core/src/main/java/org/apache/samza/zk/ZkCoordinationUtilsFactory.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/zk/ZkCoordinationUtilsFactory.java b/samza-core/src/main/java/org/apache/samza/zk/ZkCoordinationUtilsFactory.java
new file mode 100644
index 0000000..ded6a38
--- /dev/null
+++ b/samza-core/src/main/java/org/apache/samza/zk/ZkCoordinationUtilsFactory.java
@@ -0,0 +1,89 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.base.Strings;
+import org.I0Itec.zkclient.ZkClient;
+import org.apache.samza.SamzaException;
+import org.apache.samza.config.Config;
+import org.apache.samza.config.ZkConfig;
+import org.apache.samza.coordinator.CoordinationUtilsFactory;
+import org.apache.samza.coordinator.CoordinationUtils;
+import org.apache.samza.util.NoOpMetricsRegistry;
+import org.apache.zookeeper.client.ConnectStringParser;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class ZkCoordinationUtilsFactory implements CoordinationUtilsFactory {
+  private static final Logger LOG = LoggerFactory.getLogger(ZkCoordinationUtilsFactory.class);
+
+  public CoordinationUtils getCoordinationUtils(String groupId, String participantId, Config config) {
+    ZkConfig zkConfig = new ZkConfig(config);
+
+    ZkClient zkClient =
+        createZkClient(zkConfig.getZkConnect(), zkConfig.getZkSessionTimeoutMs(), zkConfig.getZkConnectionTimeoutMs());
+
+    ZkUtils zkUtils = new ZkUtils(new ZkKeyBuilder(groupId), zkClient, zkConfig.getZkConnectionTimeoutMs(), new NoOpMetricsRegistry());
+    return new ZkCoordinationUtils(participantId, zkConfig, zkUtils);
+  }
+
+  /**
+   * helper method to create zkClient
+   * @param connectString - zkConnect string
+   * @param sessionTimeoutMS - session timeout
+   * @param connectionTimeoutMs - connection timeout
+   * @return zkClient object
+   */
+  public static ZkClient createZkClient(String connectString, int sessionTimeoutMS, int connectionTimeoutMs) {
+    ZkClient zkClient;
+    try {
+      zkClient = new ZkClient(connectString, sessionTimeoutMS, connectionTimeoutMs);
+    } catch (Exception e) {
+      // ZkClient constructor may throw a variety of different exceptions, not all of them Zk based.
+      throw new SamzaException("zkClient failed to connect to ZK at :" + connectString, e);
+    }
+
+    // make sure the namespace in zk exists (if specified)
+    validateZkNameSpace(connectString, zkClient);
+
+    return zkClient;
+  }
+
+  /**
+   * if ZkConnectString contains namespace path at the end, but it does not exist we should fail
+   * @param zkConnect - connect string
+   * @param zkClient - zkClient object to talk to the ZK
+   */
+  public static void validateZkNameSpace(String zkConnect, ZkClient zkClient) {
+    ConnectStringParser parser = new ConnectStringParser(zkConnect);
+
+    String path = parser.getChrootPath();
+    if (Strings.isNullOrEmpty(path)) {
+      return; // no namespace path
+    }
+
+    LOG.info("connectString = " + zkConnect + "; path =" + path);
+
+    // if namespace specified (path above) but "/" does not exists, we will fail
+    if (!zkClient.exists("/")) {
+      throw new SamzaException("Zookeeper namespace: " + path + " does not exist for zk at " + zkConnect);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/81b4e636/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 563bf4c..c967a21 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
@@ -56,7 +56,8 @@ public class ZkJobCoordinatorFactory implements JobCoordinatorFactory {
   private ZkUtils getZkUtils(Config config, MetricsRegistry metricsRegistry) {
     ZkConfig zkConfig = new ZkConfig(config);
     ZkKeyBuilder keyBuilder = new ZkKeyBuilder(getJobCoordinationZkPath(config));
-    ZkClient zkClient = ZkCoordinationServiceFactory.createZkClient(zkConfig.getZkConnect(), zkConfig.getZkSessionTimeoutMs(), zkConfig.getZkConnectionTimeoutMs());
+    ZkClient zkClient = ZkCoordinationUtilsFactory
+        .createZkClient(zkConfig.getZkConnect(), zkConfig.getZkSessionTimeoutMs(), zkConfig.getZkConnectionTimeoutMs());
     return new ZkUtils(keyBuilder, zkClient, zkConfig.getZkConnectionTimeoutMs(), metricsRegistry);
   }
 

http://git-wip-us.apache.org/repos/asf/samza/blob/81b4e636/samza-core/src/main/java/org/apache/samza/zk/ZkProcessorLatch.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/zk/ZkProcessorLatch.java b/samza-core/src/main/java/org/apache/samza/zk/ZkProcessorLatch.java
index 77defa4..6e8236e 100644
--- a/samza-core/src/main/java/org/apache/samza/zk/ZkProcessorLatch.java
+++ b/samza-core/src/main/java/org/apache/samza/zk/ZkProcessorLatch.java
@@ -31,7 +31,7 @@ import org.slf4j.LoggerFactory;
  * When Nth node is created await() call returns.
  */
 public class ZkProcessorLatch implements Latch {
-  private static final Logger LOGGER = LoggerFactory.getLogger(ZkProcessorLatch.class);
+  private static final Logger LOG = LoggerFactory.getLogger(ZkProcessorLatch.class);
 
   private final ZkUtils zkUtils;
   private final String participantId;
@@ -50,7 +50,7 @@ public class ZkProcessorLatch implements Latch {
     zkUtils.validatePaths(new String[] {latchPath});
     targetPath =  String.format("%s/%010d", latchPath, size - 1);
 
-    LOGGER.debug("ZkProcessorLatch targetPath " + targetPath);
+    LOG.debug("ZkProcessorLatch targetPath " + targetPath);
   }
 
   @Override
@@ -66,8 +66,8 @@ public class ZkProcessorLatch implements Latch {
 
   @Override
   public void countDown() {
-    // create persistent (should be ephemeral? Probably not)
+    // create persistent node
     String path = zkUtils.getZkClient().createPersistentSequential(latchPath + "/", participantId);
-    LOGGER.debug("ZKProcessorLatch countDown created " + path);
+    LOG.debug("ZKProcessorLatch countDown created " + path);
   }
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/81b4e636/samza-core/src/test/java/org/apache/samza/runtime/TestLocalApplicationRunner.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/runtime/TestLocalApplicationRunner.java b/samza-core/src/test/java/org/apache/samza/runtime/TestLocalApplicationRunner.java
index ecacf25..38f58fd 100644
--- a/samza-core/src/test/java/org/apache/samza/runtime/TestLocalApplicationRunner.java
+++ b/samza-core/src/test/java/org/apache/samza/runtime/TestLocalApplicationRunner.java
@@ -34,6 +34,7 @@ import org.apache.samza.config.JobConfig;
 import org.apache.samza.config.MapConfig;
 import org.apache.samza.config.TaskConfig;
 import org.apache.samza.coordinator.CoordinationUtils;
+import org.apache.samza.coordinator.CoordinationUtilsFactory;
 import org.apache.samza.coordinator.Latch;
 import org.apache.samza.coordinator.LeaderElector;
 import org.apache.samza.coordinator.LeaderElectorListener;
@@ -45,35 +46,35 @@ import org.apache.samza.processor.StreamProcessor;
 import org.apache.samza.processor.StreamProcessorLifecycleListener;
 import org.apache.samza.system.StreamSpec;
 import org.junit.Test;
+import org.junit.runner.RunWith;
 import org.mockito.ArgumentCaptor;
+import org.powermock.core.classloader.annotations.PrepareForTest;
+import org.powermock.modules.junit4.PowerMockRunner;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
 import static org.mockito.Matchers.anyInt;
 import static org.mockito.Matchers.anyObject;
 import static org.mockito.Matchers.anyString;
 import static org.mockito.Mockito.*;
+import static org.powermock.api.mockito.PowerMockito.mockStatic;
 
 
+@RunWith(PowerMockRunner.class)
+@PrepareForTest(CoordinationUtilsFactory.class)
 public class TestLocalApplicationRunner {
 
-  private static final String PLAN_JSON = "{"
-      + "\"jobs\":[{"
-      + "\"jobName\":\"test-application\","
-      + "\"jobId\":\"1\","
-      + "\"operatorGraph\":{"
-      + "\"intermediateStreams\":{%s},"
-      + "\"applicationName\":\"test-application\",\"applicationId\":\"1\"}";
-  private static final String STREAM_SPEC_JSON_FORMAT = "\"%s\":{"
-      + "\"streamSpec\":{"
-      + "\"id\":\"%s\","
-      + "\"systemName\":\"%s\","
-      + "\"physicalName\":\"%s\","
-      + "\"partitionCount\":2},"
-      + "\"sourceJobs\":[\"test-app\"],"
-      + "\"targetJobs\":[\"test-target-app\"]},";
+  private static final String PLAN_JSON =
+      "{" + "\"jobs\":[{" + "\"jobName\":\"test-application\"," + "\"jobId\":\"1\"," + "\"operatorGraph\":{"
+          + "\"intermediateStreams\":{%s}," + "\"applicationName\":\"test-application\",\"applicationId\":\"1\"}";
+  private static final String STREAM_SPEC_JSON_FORMAT =
+      "\"%s\":{" + "\"streamSpec\":{" + "\"id\":\"%s\"," + "\"systemName\":\"%s\"," + "\"physicalName\":\"%s\","
+          + "\"partitionCount\":2}," + "\"sourceJobs\":[\"test-app\"]," + "\"targetJobs\":[\"test-target-app\"]},";
 
   @Test
-  public void testStreamCreation() throws Exception {
+  public void testStreamCreation()
+      throws Exception {
     Map<String, String> config = new HashMap<>();
     LocalApplicationRunner runner = new LocalApplicationRunner(new MapConfig(config));
     StreamApplication app = mock(StreamApplication.class);
@@ -109,6 +110,10 @@ public class TestLocalApplicationRunner {
     };
     when(planner.plan(anyObject())).thenReturn(plan);
 
+    mockStatic(CoordinationUtilsFactory.class);
+    CoordinationUtilsFactory coordinationUtilsFactory = mock(CoordinationUtilsFactory.class);
+    when(CoordinationUtilsFactory.getCoordinationUtilsFactory(anyObject())).thenReturn(coordinationUtilsFactory);
+
     LocalApplicationRunner spy = spy(runner);
     try {
       spy.run(app);
@@ -123,7 +128,8 @@ public class TestLocalApplicationRunner {
   }
 
   @Test
-  public void testStreamCreationWithCoordination() throws Exception {
+  public void testStreamCreationWithCoordination()
+      throws Exception {
     Map<String, String> config = new HashMap<>();
     LocalApplicationRunner runner = new LocalApplicationRunner(new MapConfig(config));
     StreamApplication app = mock(StreamApplication.class);
@@ -176,7 +182,8 @@ public class TestLocalApplicationRunner {
       }
 
       @Override
-      public void resignLeadership() {}
+      public void resignLeadership() {
+      }
 
       @Override
       public boolean amILeader() {
@@ -186,6 +193,7 @@ public class TestLocalApplicationRunner {
 
     Latch latch = new Latch() {
       boolean done = false;
+
       @Override
       public void await(long timeout, TimeUnit tu)
           throws TimeoutException {
@@ -200,9 +208,15 @@ public class TestLocalApplicationRunner {
         done = true;
       }
     };
+
+    mockStatic(CoordinationUtilsFactory.class);
+    CoordinationUtilsFactory coordinationUtilsFactory = mock(CoordinationUtilsFactory.class);
+    when(CoordinationUtilsFactory.getCoordinationUtilsFactory(anyObject())).thenReturn(coordinationUtilsFactory);
+
     when(coordinationUtils.getLeaderElector()).thenReturn(leaderElector);
     when(coordinationUtils.getLatch(anyInt(), anyString())).thenReturn(latch);
-    doReturn(coordinationUtils).when(spy).createCoordinationUtils();
+    when(coordinationUtilsFactory.getCoordinationUtils(anyString(), anyString(), anyObject()))
+        .thenReturn(coordinationUtils);
 
     try {
       spy.run(app);
@@ -217,12 +231,12 @@ public class TestLocalApplicationRunner {
   }
 
   @Test
-  public void testRunStreamTask() throws Exception {
+  public void testRunStreamTask()
+      throws Exception {
     final Map<String, String> config = new HashMap<>();
     config.put(ApplicationConfig.APP_PROCESSOR_ID_GENERATOR_CLASS, UUIDGenerator.class.getName());
     config.put(TaskConfig.TASK_CLASS(), "org.apache.samza.test.processor.IdentityStreamTask");
 
-
     LocalApplicationRunner runner = new LocalApplicationRunner(new MapConfig(config));
 
     StreamProcessor sp = mock(StreamProcessor.class);
@@ -243,10 +257,11 @@ public class TestLocalApplicationRunner {
     spy.runTask();
 
     assertEquals(ApplicationStatus.SuccessfulFinish, spy.status(null));
-
   }
+
   @Test
-  public void testRunComplete() throws Exception {
+  public void testRunComplete()
+      throws Exception {
     final Map<String, String> config = new HashMap<>();
     config.put(ApplicationConfig.APP_PROCESSOR_ID_GENERATOR_CLASS, UUIDGenerator.class.getName());
     LocalApplicationRunner runner = new LocalApplicationRunner(new MapConfig(config));
@@ -289,7 +304,6 @@ public class TestLocalApplicationRunner {
         return null;
       }).when(sp).start();
 
-
     LocalApplicationRunner spy = spy(runner);
     doReturn(sp).when(spy).createStreamProcessor(anyObject(), anyObject(), captor.capture());
 
@@ -299,7 +313,8 @@ public class TestLocalApplicationRunner {
   }
 
   @Test
-  public void testRunFailure() throws Exception {
+  public void testRunFailure()
+      throws Exception {
     final Map<String, String> config = new HashMap<>();
     config.put(ApplicationConfig.PROCESSOR_ID, "0");
     LocalApplicationRunner runner = new LocalApplicationRunner(new MapConfig(config));
@@ -335,13 +350,13 @@ public class TestLocalApplicationRunner {
     ArgumentCaptor<StreamProcessorLifecycleListener> captor =
         ArgumentCaptor.forClass(StreamProcessorLifecycleListener.class);
 
-    doAnswer(i -> {
+    doAnswer(i ->
+      {
         StreamProcessorLifecycleListener listener = captor.getValue();
         listener.onFailure(t);
         return null;
       }).when(sp).start();
 
-
     LocalApplicationRunner spy = spy(runner);
     doReturn(sp).when(spy).createStreamProcessor(anyObject(), anyObject(), captor.capture());
 
@@ -360,14 +375,12 @@ public class TestLocalApplicationRunner {
    */
   @Test
   public void testPlanIdWithShuffledStreamSpecs() {
-    List<StreamSpec> streamSpecs = ImmutableList.of(
-      new StreamSpec("test-stream-1", "stream-1", "testStream"),
+    List<StreamSpec> streamSpecs = ImmutableList.of(new StreamSpec("test-stream-1", "stream-1", "testStream"),
         new StreamSpec("test-stream-2", "stream-2", "testStream"),
         new StreamSpec("test-stream-3", "stream-3", "testStream"));
     String planIdBeforeShuffle = getExecutionPlanId(streamSpecs);
 
-    List<StreamSpec> shuffledStreamSpecs = ImmutableList.of(
-        new StreamSpec("test-stream-2", "stream-2", "testStream"),
+    List<StreamSpec> shuffledStreamSpecs = ImmutableList.of(new StreamSpec("test-stream-2", "stream-2", "testStream"),
         new StreamSpec("test-stream-1", "stream-1", "testStream"),
         new StreamSpec("test-stream-3", "stream-3", "testStream"));
 
@@ -381,8 +394,7 @@ public class TestLocalApplicationRunner {
    */
   @Test
   public void testGeneratePlanIdWithSameStreamSpecs() {
-    List<StreamSpec> streamSpecs = ImmutableList.of(
-        new StreamSpec("test-stream-1", "stream-1", "testStream"),
+    List<StreamSpec> streamSpecs = ImmutableList.of(new StreamSpec("test-stream-1", "stream-1", "testStream"),
         new StreamSpec("test-stream-2", "stream-2", "testStream"),
         new StreamSpec("test-stream-3", "stream-3", "testStream"));
     String planIdForFirstAttempt = getExecutionPlanId(streamSpecs);
@@ -398,14 +410,12 @@ public class TestLocalApplicationRunner {
    */
   @Test
   public void testGeneratePlanIdWithDifferentStreamSpecs() {
-    List<StreamSpec> streamSpecs = ImmutableList.of(
-        new StreamSpec("test-stream-1", "stream-1", "testStream"),
+    List<StreamSpec> streamSpecs = ImmutableList.of(new StreamSpec("test-stream-1", "stream-1", "testStream"),
         new StreamSpec("test-stream-2", "stream-2", "testStream"),
         new StreamSpec("test-stream-3", "stream-3", "testStream"));
     String planIdBeforeShuffle = getExecutionPlanId(streamSpecs);
 
-    List<StreamSpec> updatedStreamSpecs = ImmutableList.of(
-        new StreamSpec("test-stream-1", "stream-1", "testStream"),
+    List<StreamSpec> updatedStreamSpecs = ImmutableList.of(new StreamSpec("test-stream-1", "stream-1", "testStream"),
         new StreamSpec("test-stream-4", "stream-4", "testStream"),
         new StreamSpec("test-stream-3", "stream-3", "testStream"));
 
@@ -414,9 +424,8 @@ public class TestLocalApplicationRunner {
   }
 
   private String getExecutionPlanId(List<StreamSpec> updatedStreamSpecs) {
-    String intermediateStreamJson = updatedStreamSpecs.stream()
-        .map(this::streamSpecToJson)
-        .collect(Collectors.joining(","));
+    String intermediateStreamJson =
+        updatedStreamSpecs.stream().map(this::streamSpecToJson).collect(Collectors.joining(","));
 
     int planId = String.format(PLAN_JSON, intermediateStreamJson).hashCode();
 
@@ -424,10 +433,7 @@ public class TestLocalApplicationRunner {
   }
 
   private String streamSpecToJson(StreamSpec streamSpec) {
-    return String.format(STREAM_SPEC_JSON_FORMAT,
-        streamSpec.getId(),
-        streamSpec.getId(),
-        streamSpec.getSystemName(),
+    return String.format(STREAM_SPEC_JSON_FORMAT, streamSpec.getId(), streamSpec.getId(), streamSpec.getSystemName(),
         streamSpec.getPhysicalName());
   }
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/81b4e636/samza-core/src/test/java/org/apache/samza/zk/TestZkLeaderElector.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/zk/TestZkLeaderElector.java b/samza-core/src/test/java/org/apache/samza/zk/TestZkLeaderElector.java
index 010d138..74b9abd 100644
--- a/samza-core/src/test/java/org/apache/samza/zk/TestZkLeaderElector.java
+++ b/samza-core/src/test/java/org/apache/samza/zk/TestZkLeaderElector.java
@@ -433,7 +433,8 @@ public class TestZkLeaderElector {
   }
 
   private ZkUtils getZkUtilsWithNewClient() {
-    ZkClient zkClient = ZkCoordinationServiceFactory.createZkClient(testZkConnectionString, SESSION_TIMEOUT_MS, CONNECTION_TIMEOUT_MS);
+    ZkClient zkClient = ZkCoordinationUtilsFactory
+        .createZkClient(testZkConnectionString, SESSION_TIMEOUT_MS, CONNECTION_TIMEOUT_MS);
     return new ZkUtils(
         KEY_BUILDER,
         zkClient,

http://git-wip-us.apache.org/repos/asf/samza/blob/81b4e636/samza-core/src/test/java/org/apache/samza/zk/TestZkNamespace.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/zk/TestZkNamespace.java b/samza-core/src/test/java/org/apache/samza/zk/TestZkNamespace.java
index 3ce203e..1defccb 100644
--- a/samza-core/src/test/java/org/apache/samza/zk/TestZkNamespace.java
+++ b/samza-core/src/test/java/org/apache/samza/zk/TestZkNamespace.java
@@ -90,7 +90,7 @@ public class TestZkNamespace {
     String zkConnect = "127.0.0.1:" + zkServer.getPort() + zkNameSpace;
     createNamespace(zkNameSpace);
     initZk(zkConnect);
-    ZkCoordinationServiceFactory.validateZkNameSpace(zkConnect, zkClient);
+    ZkCoordinationUtilsFactory.validateZkNameSpace(zkConnect, zkClient);
 
     zkClient.createPersistent("/test");
     zkClient.createPersistent("/test/test1");
@@ -106,7 +106,7 @@ public class TestZkNamespace {
     try {
       String zkConnect = "127.0.0.1:" + zkServer.getPort() + "/zkNameSpace";
       initZk(zkConnect);
-      ZkCoordinationServiceFactory.validateZkNameSpace(zkConnect, zkClient);
+      ZkCoordinationUtilsFactory.validateZkNameSpace(zkConnect, zkClient);
       Assert.fail("1.Should fail with exception, because namespace doesn't exist");
     } catch (SamzaException e) {
       // expected
@@ -120,7 +120,7 @@ public class TestZkNamespace {
     try {
       String zkConnect = "127.0.0.1:" + zkServer.getPort() + "/zkNameSpace/xyz";
       initZk(zkConnect);
-      ZkCoordinationServiceFactory.validateZkNameSpace(zkConnect, zkClient);
+      ZkCoordinationUtilsFactory.validateZkNameSpace(zkConnect, zkClient);
       Assert.fail("2.Should fail with exception, because namespace doesn't exist");
     } catch (SamzaException e) {
       // expected
@@ -134,7 +134,7 @@ public class TestZkNamespace {
     // should succeed, because no namespace provided
     String zkConnect = "127.0.0.1:" + zkServer.getPort() + "";
     initZk(zkConnect);
-    ZkCoordinationServiceFactory.validateZkNameSpace(zkConnect, zkClient);
+    ZkCoordinationUtilsFactory.validateZkNameSpace(zkConnect, zkClient);
     tearDownZk();
   }
 

http://git-wip-us.apache.org/repos/asf/samza/blob/81b4e636/samza-core/src/test/java/org/apache/samza/zk/TestZkProcessorLatch.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/zk/TestZkProcessorLatch.java b/samza-core/src/test/java/org/apache/samza/zk/TestZkProcessorLatch.java
index b2a5533..674287b 100644
--- a/samza-core/src/test/java/org/apache/samza/zk/TestZkProcessorLatch.java
+++ b/samza-core/src/test/java/org/apache/samza/zk/TestZkProcessorLatch.java
@@ -215,7 +215,7 @@ public class TestZkProcessorLatch {
 
   }
   private ZkUtils getZkUtilsWithNewClient(String processorId) {
-    ZkClient zkClient = ZkCoordinationServiceFactory
+    ZkClient zkClient = ZkCoordinationUtilsFactory
         .createZkClient(testZkConnectionString, SESSION_TIMEOUT_MS, CONNECTION_TIMEOUT_MS);
     return new ZkUtils(
         KEY_BUILDER,

http://git-wip-us.apache.org/repos/asf/samza/blob/81b4e636/samza-test/src/test/java/org/apache/samza/test/processor/TestZkLocalApplicationRunner.java
----------------------------------------------------------------------
diff --git a/samza-test/src/test/java/org/apache/samza/test/processor/TestZkLocalApplicationRunner.java b/samza-test/src/test/java/org/apache/samza/test/processor/TestZkLocalApplicationRunner.java
index 76fd046..fb8f17a 100644
--- a/samza-test/src/test/java/org/apache/samza/test/processor/TestZkLocalApplicationRunner.java
+++ b/samza-test/src/test/java/org/apache/samza/test/processor/TestZkLocalApplicationRunner.java
@@ -23,6 +23,7 @@ import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
+import java.util.ArrayList;
 import kafka.admin.AdminUtils;
 import kafka.server.KafkaServer;
 import kafka.utils.TestUtils;
@@ -61,7 +62,6 @@ import org.slf4j.LoggerFactory;
 import scala.collection.JavaConverters;
 
 import java.io.Serializable;
-import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
 import java.util.Map;


[12/26] samza git commit: SAMZA-1400: disabling flaky test testTwoStreamProcessors in TestZkStreamProcessorSession

Posted by xi...@apache.org.
SAMZA-1400: disabling flaky test testTwoStreamProcessors in TestZkStreamProcessorSession

In this SAMZA-1400, We are disabling this flaky one in master and will cherry pick for 0.13.1. We have created a ticket SAMZA-1399 for fixing it in later build.

navina sborya Please take a look.

Author: Fred Ji <ha...@gmail.com>

Reviewers: Xinyu Liu <xi...@apache.org>

Closes #278 from fredji97/SAMZA1400


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

Branch: refs/heads/0.14.0
Commit: e21ff714637238e9fe1011a9f4f20699f1e9e471
Parents: afe3bf8
Author: Fred Ji <ha...@gmail.com>
Authored: Thu Aug 17 11:16:10 2017 -0700
Committer: Xinyu Liu <xi...@xiliu-ld.linkedin.biz>
Committed: Thu Aug 17 11:16:10 2017 -0700

----------------------------------------------------------------------
 .../org/apache/samza/processor/TestZkStreamProcessorSession.java  | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/samza/blob/e21ff714/samza-test/src/test/java/org/apache/samza/processor/TestZkStreamProcessorSession.java
----------------------------------------------------------------------
diff --git a/samza-test/src/test/java/org/apache/samza/processor/TestZkStreamProcessorSession.java b/samza-test/src/test/java/org/apache/samza/processor/TestZkStreamProcessorSession.java
index 6aab5e3..880d766 100644
--- a/samza-test/src/test/java/org/apache/samza/processor/TestZkStreamProcessorSession.java
+++ b/samza-test/src/test/java/org/apache/samza/processor/TestZkStreamProcessorSession.java
@@ -43,7 +43,8 @@ public class TestZkStreamProcessorSession extends TestZkStreamProcessorBase {
     testStreamProcessorWithSessionRestart(new String[]{"1"});
   }
 
-  @Test
+  // TODO: SAMZA-1399 fix the flaky test testTwoStreamProcessors and re-enable it
+  // @Test
   public void testTwoStreamProcessors() {
     testStreamProcessorWithSessionRestart(new String[]{"2", "3"});
   }


[17/26] samza git commit: SAMZA-1382: added Zk communication protocol version verification

Posted by xi...@apache.org.
SAMZA-1382: added Zk communication protocol version verification

Author: Boris Shkolnik <bo...@apache.org>
Author: Boris Shkolnik <bs...@linkedin.com>

Reviewers: Xinyu Liu <xi...@apache.org>

Closes #255 from sborya/ZkCommunicationVersion


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

Branch: refs/heads/0.14.0
Commit: 80d82b6a51f6dc0b65f766b63704b30c780073ac
Parents: 67f7214
Author: Boris Shkolnik <bo...@apache.org>
Authored: Tue Aug 22 17:55:54 2017 -0700
Committer: Xinyu Liu <xi...@xiliu-ld.linkedin.biz>
Committed: Tue Aug 22 17:55:54 2017 -0700

----------------------------------------------------------------------
 .../samza/zk/ZkBarrierForVersionUpgrade.java    |  2 +-
 .../org/apache/samza/zk/ZkControllerImpl.java   | 19 ++++++-
 .../org/apache/samza/zk/ZkLeaderElector.java    |  2 +-
 .../org/apache/samza/zk/ZkProcessorLatch.java   |  4 +-
 .../main/java/org/apache/samza/zk/ZkUtils.java  | 41 +++++++++++++-
 .../apache/samza/zk/TestZkLeaderElector.java    |  4 +-
 .../java/org/apache/samza/zk/TestZkUtils.java   | 57 ++++++++++++++++++--
 7 files changed, 115 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/samza/blob/80d82b6a/samza-core/src/main/java/org/apache/samza/zk/ZkBarrierForVersionUpgrade.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/zk/ZkBarrierForVersionUpgrade.java b/samza-core/src/main/java/org/apache/samza/zk/ZkBarrierForVersionUpgrade.java
index 3257ee1..abea299 100644
--- a/samza-core/src/main/java/org/apache/samza/zk/ZkBarrierForVersionUpgrade.java
+++ b/samza-core/src/main/java/org/apache/samza/zk/ZkBarrierForVersionUpgrade.java
@@ -83,7 +83,7 @@ public class ZkBarrierForVersionUpgrade {
   public void create(final String version, List<String> participants) {
     String barrierRoot = keyBuilder.getBarrierRoot();
     String barrierParticipantsPath = keyBuilder.getBarrierParticipantsPath(version);
-    zkUtils.makeSurePersistentPathsExists(new String[]{
+    zkUtils.validatePaths(new String[]{
         barrierRoot,
         keyBuilder.getBarrierPath(version),
         barrierParticipantsPath,

http://git-wip-us.apache.org/repos/asf/samza/blob/80d82b6a/samza-core/src/main/java/org/apache/samza/zk/ZkControllerImpl.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/zk/ZkControllerImpl.java b/samza-core/src/main/java/org/apache/samza/zk/ZkControllerImpl.java
index 3af5042..6305616 100644
--- a/samza-core/src/main/java/org/apache/samza/zk/ZkControllerImpl.java
+++ b/samza-core/src/main/java/org/apache/samza/zk/ZkControllerImpl.java
@@ -46,8 +46,8 @@ public class ZkControllerImpl implements ZkController {
 
   private void init() {
     ZkKeyBuilder keyBuilder = zkUtils.getKeyBuilder();
-    zkUtils.makeSurePersistentPathsExists(
-        new String[]{keyBuilder.getProcessorsPath(), keyBuilder.getJobModelVersionPath(), keyBuilder
+
+    zkUtils.validatePaths(new String[]{keyBuilder.getProcessorsPath(), keyBuilder.getJobModelVersionPath(), keyBuilder
             .getJobModelPathPrefix()});
   }
 
@@ -57,6 +57,21 @@ public class ZkControllerImpl implements ZkController {
     // possibly split into two method - becomeLeader() and becomeParticipant()
     zkLeaderElector.tryBecomeLeader();
 
+    // make sure we are connection to a job that uses the same ZK communication protocol version.
+    try {
+      zkUtils.validateZkVersion();
+    } catch (SamzaException e) {
+      // IMPORTANT: Mismatch of the version, means we are trying to join a job, started by processors with different version.
+      // If there are no processors running, this is the place to do the migration to the new
+      // ZK structure.
+      // If some processors are running, then this processor should fail with an error to tell the user to stop all
+      // the processors before upgrading to this new version.
+      // TODO migration here
+      // for now we just rethrow the exception
+      throw e;
+    }
+
+
     // subscribe to JobModel version updates
     zkUtils.subscribeToJobModelVersionChange(new ZkJobModelVersionChangeHandler(zkUtils));
   }

http://git-wip-us.apache.org/repos/asf/samza/blob/80d82b6a/samza-core/src/main/java/org/apache/samza/zk/ZkLeaderElector.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/zk/ZkLeaderElector.java b/samza-core/src/main/java/org/apache/samza/zk/ZkLeaderElector.java
index 97430cb..f4c1e94 100644
--- a/samza-core/src/main/java/org/apache/samza/zk/ZkLeaderElector.java
+++ b/samza-core/src/main/java/org/apache/samza/zk/ZkLeaderElector.java
@@ -63,7 +63,7 @@ public class ZkLeaderElector implements LeaderElector {
     this.hostName = getHostName();
     this.previousProcessorChangeListener = new PreviousProcessorChangeListener(zkUtils);
 
-    zkUtils.makeSurePersistentPathsExists(new String[]{keyBuilder.getProcessorsPath()});
+    zkUtils.validatePaths(new String[]{keyBuilder.getProcessorsPath()});
   }
 
   @VisibleForTesting

http://git-wip-us.apache.org/repos/asf/samza/blob/80d82b6a/samza-core/src/main/java/org/apache/samza/zk/ZkProcessorLatch.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/zk/ZkProcessorLatch.java b/samza-core/src/main/java/org/apache/samza/zk/ZkProcessorLatch.java
index 166c627..77defa4 100644
--- a/samza-core/src/main/java/org/apache/samza/zk/ZkProcessorLatch.java
+++ b/samza-core/src/main/java/org/apache/samza/zk/ZkProcessorLatch.java
@@ -46,8 +46,8 @@ public class ZkProcessorLatch implements Latch {
     ZkKeyBuilder keyBuilder = this.zkUtils.getKeyBuilder();
 
     latchPath = String.format("%s/%s", keyBuilder.getRootPath(), LATCH_PATH + "_" + latchId);
-    // TODO: Verify that makeSurePersistentPathsExists doesn't fail with exceptions
-    zkUtils.makeSurePersistentPathsExists(new String[] {latchPath});
+    // TODO: Verify that validatePaths doesn't fail with exceptions
+    zkUtils.validatePaths(new String[] {latchPath});
     targetPath =  String.format("%s/%010d", latchPath, size - 1);
 
     LOGGER.debug("ZkProcessorLatch targetPath " + targetPath);

http://git-wip-us.apache.org/repos/asf/samza/blob/80d82b6a/samza-core/src/main/java/org/apache/samza/zk/ZkUtils.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/zk/ZkUtils.java b/samza-core/src/main/java/org/apache/samza/zk/ZkUtils.java
index 5df7114..6ca9052 100644
--- a/samza-core/src/main/java/org/apache/samza/zk/ZkUtils.java
+++ b/samza-core/src/main/java/org/apache/samza/zk/ZkUtils.java
@@ -32,7 +32,9 @@ import java.util.stream.Collectors;
 import org.I0Itec.zkclient.IZkChildListener;
 import org.I0Itec.zkclient.IZkDataListener;
 import org.I0Itec.zkclient.ZkClient;
+import org.I0Itec.zkclient.exception.ZkBadVersionException;
 import org.I0Itec.zkclient.exception.ZkInterruptedException;
+import org.I0Itec.zkclient.exception.ZkNodeExistsException;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.samza.SamzaException;
 import org.apache.samza.job.model.JobModel;
@@ -74,6 +76,8 @@ import org.slf4j.LoggerFactory;
  */
 public class ZkUtils {
   private static final Logger LOG = LoggerFactory.getLogger(ZkUtils.class);
+  /* package private */static final String ZK_PROTOCOL_VERSION = "1.0";
+
 
   private final ZkClient zkClient;
   private volatile String ephemeralPath = null;
@@ -438,11 +442,46 @@ public class ZkUtils {
         "(actual data version after update = " + stat.getVersion() + ")");
   }
 
+  // validate that Zk protocol currently used by the job is the same as in this participant
+  public void validateZkVersion() {
+
+    // Version of the protocol is written into root znode. If root does not exist yet we need to create one.
+    String rootPath = keyBuilder.getRootPath();
+    if (!zkClient.exists(rootPath)) {
+      try {
+        // attempt to create the root with the correct version
+        zkClient.createPersistent(rootPath, ZK_PROTOCOL_VERSION);
+        LOG.info("Created zk root node: " + rootPath + " with zk version " + ZK_PROTOCOL_VERSION);
+        return;
+      } catch (ZkNodeExistsException e) {
+        // ignoring
+        LOG.warn("root path " + rootPath + " already exists.");
+      }
+    }
+    // if exists, verify the version
+    Stat stat = new Stat();
+    String version = (String) zkClient.readData(rootPath, stat);
+    if (version == null) {
+      // for backward compatibility, if no value - assume 1.0
+      try {
+        zkClient.writeData(rootPath, "1.0", stat.getVersion());
+      } catch (ZkBadVersionException e) {
+        // if the write failed with ZkBadVersionException it means someone else already wrote a version, so we can ignore it.
+      }
+      // re-read the updated version
+      version = (String) zkClient.readData(rootPath);
+    }
+    LOG.info("Current version for zk root node: " + rootPath + " is " + version + ", expected version is " + ZK_PROTOCOL_VERSION);
+    if (!version.equals(ZK_PROTOCOL_VERSION)) {
+      throw new SamzaException("ZK Protocol mismatch. Expected " + ZK_PROTOCOL_VERSION + "; found " + version);
+    }
+  }
+
   /**
    * verify that given paths exist in ZK
    * @param paths - paths to verify or create
    */
-  public void makeSurePersistentPathsExists(String[] paths) {
+  public void validatePaths(String[] paths) {
     for (String path : paths) {
       if (!zkClient.exists(path)) {
         zkClient.createPersistent(path, true);

http://git-wip-us.apache.org/repos/asf/samza/blob/80d82b6a/samza-core/src/test/java/org/apache/samza/zk/TestZkLeaderElector.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/zk/TestZkLeaderElector.java b/samza-core/src/test/java/org/apache/samza/zk/TestZkLeaderElector.java
index 3ff9175..010d138 100644
--- a/samza-core/src/test/java/org/apache/samza/zk/TestZkLeaderElector.java
+++ b/samza-core/src/test/java/org/apache/samza/zk/TestZkLeaderElector.java
@@ -100,7 +100,7 @@ public class TestZkLeaderElector {
     when(mockZkUtils.registerProcessorAndGetId(any())).
         thenReturn(KEY_BUILDER.getProcessorsPath() + "/0000000000");
     when(mockZkUtils.getSortedActiveProcessorsZnodes()).thenReturn(activeProcessors);
-    Mockito.doNothing().when(mockZkUtils).makeSurePersistentPathsExists(any(String[].class));
+    Mockito.doNothing().when(mockZkUtils).validatePaths(any(String[].class));
 
     ZkKeyBuilder kb = mock(ZkKeyBuilder.class);
     when(kb.getProcessorsPath()).thenReturn("");
@@ -119,7 +119,7 @@ public class TestZkLeaderElector {
     String processorId = "1";
     ZkUtils mockZkUtils = mock(ZkUtils.class);
     when(mockZkUtils.getSortedActiveProcessorsZnodes()).thenReturn(new ArrayList<String>());
-    Mockito.doNothing().when(mockZkUtils).makeSurePersistentPathsExists(any(String[].class));
+    Mockito.doNothing().when(mockZkUtils).validatePaths(any(String[].class));
 
     ZkKeyBuilder kb = mock(ZkKeyBuilder.class);
     when(kb.getProcessorsPath()).thenReturn("");

http://git-wip-us.apache.org/repos/asf/samza/blob/80d82b6a/samza-core/src/test/java/org/apache/samza/zk/TestZkUtils.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/zk/TestZkUtils.java b/samza-core/src/test/java/org/apache/samza/zk/TestZkUtils.java
index b5953d1..3c8f67e 100644
--- a/samza-core/src/test/java/org/apache/samza/zk/TestZkUtils.java
+++ b/samza-core/src/test/java/org/apache/samza/zk/TestZkUtils.java
@@ -18,6 +18,7 @@
  */
 package org.apache.samza.zk;
 
+import java.lang.reflect.Field;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
@@ -30,6 +31,7 @@ import org.I0Itec.zkclient.IZkDataListener;
 import org.I0Itec.zkclient.ZkClient;
 import org.I0Itec.zkclient.ZkConnection;
 import org.I0Itec.zkclient.exception.ZkNodeExistsException;
+import org.apache.commons.lang3.reflect.FieldUtils;
 import org.apache.samza.SamzaException;
 import org.apache.samza.config.MapConfig;
 import org.apache.samza.job.model.ContainerModel;
@@ -45,6 +47,7 @@ import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.ExpectedException;
 
+
 public class TestZkUtils {
   private static EmbeddedZookeeper zkServer = null;
   private static final ZkKeyBuilder KEY_BUILDER = new ZkKeyBuilder("test");
@@ -116,12 +119,58 @@ public class TestZkUtils {
   }
 
   @Test
+  public void testZKProtocolVersion() {
+    // first time connect, version should be set to ZkUtils.ZK_PROTOCOL_VERSION
+    ZkLeaderElector le = new ZkLeaderElector("1", zkUtils);
+    ZkControllerImpl zkController = new ZkControllerImpl("1", zkUtils, null, le);
+    zkController.register();
+    String root = zkUtils.getKeyBuilder().getRootPath();
+    String ver = (String) zkUtils.getZkClient().readData(root);
+    Assert.assertEquals(ZkUtils.ZK_PROTOCOL_VERSION, ver);
+
+    // do it again (in case original value was null
+    zkController = new ZkControllerImpl("1", zkUtils, null, le);
+    zkController.register();
+    ver = (String) zkUtils.getZkClient().readData(root);
+    Assert.assertEquals(ZkUtils.ZK_PROTOCOL_VERSION, ver);
+
+    // now negative case
+    zkUtils.getZkClient().writeData(root, "2.0");
+    try {
+      zkController = new ZkControllerImpl("1", zkUtils, null, le);
+      zkController.register();
+      Assert.fail("Expected to fail because of version mismatch 2.0 vs 1.0");
+    } catch (SamzaException e) {
+      // expected
+    }
+
+    // validate future values, let's say that current version should be 3.0
+    try {
+      Field f = zkUtils.getClass().getDeclaredField("ZK_PROTOCOL_VERSION");
+      FieldUtils.removeFinalModifier(f);
+      f.set(null, "3.0");
+    } catch (Exception e) {
+      System.out.println(e);
+      Assert.fail();
+    }
+
+    try {
+      zkController = new ZkControllerImpl("1", zkUtils, null, le);
+      zkController.register();
+      Assert.fail("Expected to fail because of version mismatch 2.0 vs 3.0");
+    } catch (SamzaException e) {
+      // expected
+    }
+  }
+
+  @Test
   public void testGetProcessorsIDs() {
     Assert.assertEquals(0, zkUtils.getSortedActiveProcessorsIDs().size());
     zkUtils.registerProcessorAndGetId(new ProcessorData("host1", "1"));
     List<String> l = zkUtils.getSortedActiveProcessorsIDs();
     Assert.assertEquals(1, l.size());
-    new ZkUtils(KEY_BUILDER, zkClient, SESSION_TIMEOUT_MS, new NoOpMetricsRegistry()).registerProcessorAndGetId(new ProcessorData("host2", "2"));
+    new ZkUtils(KEY_BUILDER, zkClient, SESSION_TIMEOUT_MS, new NoOpMetricsRegistry()).registerProcessorAndGetId(
+        new ProcessorData("host2", "2"));
     l = zkUtils.getSortedActiveProcessorsIDs();
     Assert.assertEquals(2, l.size());
 
@@ -149,8 +198,7 @@ public class TestZkUtils {
     Assert.assertFalse(zkUtils.exists(root));
 
     // create the paths
-    zkUtils.makeSurePersistentPathsExists(
-        new String[]{root, keyBuilder.getJobModelVersionPath(), keyBuilder.getProcessorsPath()});
+    zkUtils.validatePaths(new String[]{root, keyBuilder.getJobModelVersionPath(), keyBuilder.getProcessorsPath()});
     Assert.assertTrue(zkUtils.exists(root));
     Assert.assertTrue(zkUtils.exists(keyBuilder.getJobModelVersionPath()));
     Assert.assertTrue(zkUtils.exists(keyBuilder.getProcessorsPath()));
@@ -213,8 +261,7 @@ public class TestZkUtils {
     String version = "1";
     String oldVersion = "0";
 
-    zkUtils.makeSurePersistentPathsExists(
-        new String[]{root, keyBuilder.getJobModelPathPrefix(), keyBuilder.getJobModelVersionPath()});
+    zkUtils.validatePaths(new String[]{root, keyBuilder.getJobModelPathPrefix(), keyBuilder.getJobModelVersionPath()});
 
     zkUtils.publishJobModelVersion(oldVersion, version);
     Assert.assertEquals(version, zkUtils.getJobModelVersion());


[23/26] samza git commit: SAMZA-1385: Coordination utils factory with distributed lock

Posted by xi...@apache.org.
SAMZA-1385: Coordination utils factory with distributed lock

this PR includes some changes from another PR. I will re-merge it again, after the other PR is in.

Author: Boris Shkolnik <bo...@apache.org>

Reviewers: Xinyu Liu <xi...@apache.org>

Closes #284 from sborya/CoordinationUtilsFactory_withDistributedLock


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

Branch: refs/heads/0.14.0
Commit: dd07e07421bd57e0f927b8974131b949f5ac2e71
Parents: 81b4e63
Author: Boris Shkolnik <bo...@apache.org>
Authored: Tue Aug 29 15:37:17 2017 -0700
Committer: Xinyu Liu <xi...@xiliu-ld1.linkedin.biz>
Committed: Tue Aug 29 15:37:17 2017 -0700

----------------------------------------------------------------------
 .../coordinator/AzureCoordinationUtils.java     |  13 ++-
 .../org/apache/samza/coordinator/AzureLock.java |   6 +-
 .../samza/coordinator/CoordinationUtils.java    |   5 +-
 .../coordinator/DistributedLockWithState.java   |  42 +++++++
 .../samza/runtime/LocalApplicationRunner.java   |  94 ++++++---------
 .../apache/samza/zk/ZkCoordinationUtils.java    |   5 +
 .../samza/zk/ZkCoordinationUtilsFactory.java    |   2 +-
 .../org/apache/samza/zk/ZkDistributedLock.java  | 117 +++++++++++++++++++
 .../runtime/TestApplicationRunnerMain.java      |   2 +-
 .../runtime/TestLocalApplicationRunner.java     |  57 ++-------
 10 files changed, 220 insertions(+), 123 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/samza/blob/dd07e074/samza-azure/src/main/java/org/apache/samza/coordinator/AzureCoordinationUtils.java
----------------------------------------------------------------------
diff --git a/samza-azure/src/main/java/org/apache/samza/coordinator/AzureCoordinationUtils.java b/samza-azure/src/main/java/org/apache/samza/coordinator/AzureCoordinationUtils.java
index dbd945f..2a42514 100644
--- a/samza-azure/src/main/java/org/apache/samza/coordinator/AzureCoordinationUtils.java
+++ b/samza-azure/src/main/java/org/apache/samza/coordinator/AzureCoordinationUtils.java
@@ -46,13 +46,14 @@ public class AzureCoordinationUtils implements CoordinationUtils {
   }
 
   @Override
-  public void close() {
-
-  }
-
-  public DistributedLock getLock(String initLockName) {
+  public DistributedLockWithState getLockWithState(String lockId) {
     BlobUtils blob = new BlobUtils(client, azureConfig.getAzureContainerName(),
-        azureConfig.getAzureBlobName() + initLockName, azureConfig.getAzureBlobLength());
+        azureConfig.getAzureBlobName() + lockId, azureConfig.getAzureBlobLength());
     return new AzureLock(blob);
   }
+
+  @Override
+  public void close() {
+    
+  }
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/dd07e074/samza-azure/src/main/java/org/apache/samza/coordinator/AzureLock.java
----------------------------------------------------------------------
diff --git a/samza-azure/src/main/java/org/apache/samza/coordinator/AzureLock.java b/samza-azure/src/main/java/org/apache/samza/coordinator/AzureLock.java
index 0ef1b83..c0d3ff2 100644
--- a/samza-azure/src/main/java/org/apache/samza/coordinator/AzureLock.java
+++ b/samza-azure/src/main/java/org/apache/samza/coordinator/AzureLock.java
@@ -33,7 +33,7 @@ import org.slf4j.LoggerFactory;
 /**
  * Distributed lock primitive for Azure.
  */
-public class AzureLock implements DistributedLock {
+public class AzureLock implements DistributedLockWithState {
 
   private static final Logger LOG = LoggerFactory.getLogger(AzureLock.class);
   private static final int LEASE_TIME_IN_SEC = 60;
@@ -55,7 +55,7 @@ public class AzureLock implements DistributedLock {
    * @return true if the lock was acquired successfully, false if lock acquire operation is unsuccessful even after subsequent tries within the timeout range.
    */
   @Override
-  public boolean lock(long timeout, TimeUnit unit) {
+  public boolean lockIfNotSet(long timeout, TimeUnit unit) {
     //Start timer for timeout
     long startTime = System.currentTimeMillis();
     long lockTimeout = TimeUnit.MILLISECONDS.convert(timeout, unit);
@@ -87,7 +87,7 @@ public class AzureLock implements DistributedLock {
    * Unlocks, by releasing the lease on the blob.
    */
   @Override
-  public void unlock() {
+  public void unlockAndSet() {
     boolean status = leaseBlobManager.releaseLease(leaseId.get());
     if (status) {
       LOG.info("Unlocked successfully.");

http://git-wip-us.apache.org/repos/asf/samza/blob/dd07e074/samza-core/src/main/java/org/apache/samza/coordinator/CoordinationUtils.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/coordinator/CoordinationUtils.java b/samza-core/src/main/java/org/apache/samza/coordinator/CoordinationUtils.java
index 4ba44b5..9ebd2e2 100644
--- a/samza-core/src/main/java/org/apache/samza/coordinator/CoordinationUtils.java
+++ b/samza-core/src/main/java/org/apache/samza/coordinator/CoordinationUtils.java
@@ -27,6 +27,7 @@ import org.apache.samza.annotation.InterfaceStability;
  * This service provide three primitives:
  *   - LeaderElection
  *   - Latch
+ *   - LockWithState (does not lock if state is set)
  */
 @InterfaceStability.Evolving
 public interface CoordinationUtils {
@@ -36,8 +37,10 @@ public interface CoordinationUtils {
 
   Latch getLatch(int size, String latchId);
 
+  DistributedLockWithState getLockWithState(String lockId);
+
   /**
-   * performs necessary cleanup and closes ALL the utils.
+   * utilites cleanup
    */
   void close();
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/dd07e074/samza-core/src/main/java/org/apache/samza/coordinator/DistributedLockWithState.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/coordinator/DistributedLockWithState.java b/samza-core/src/main/java/org/apache/samza/coordinator/DistributedLockWithState.java
new file mode 100644
index 0000000..0de7813
--- /dev/null
+++ b/samza-core/src/main/java/org/apache/samza/coordinator/DistributedLockWithState.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.coordinator;
+
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+
+public interface DistributedLockWithState {
+
+  /**
+   * Trie to acquire the lock, but first check if the state flag is set. If it is set, return false.
+   * If the flag is not set, and lock is acquired - return true.
+   * Throw TimeOutException if could not acquire the lock.
+   * @param timeout Duration of lock acquiring timeout.
+   * @param unit Time Unit of the timeout defined above.
+   * @return true if lock is acquired successfully, false if state is already set.
+   */
+  boolean lockIfNotSet(long timeout, TimeUnit unit) throws TimeoutException;
+
+  /**
+   * Release the lock and set the state
+   */
+  void unlockAndSet();
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/samza/blob/dd07e074/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java b/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java
index fc11cf5..077c124 100644
--- a/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java
+++ b/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java
@@ -37,8 +37,7 @@ import org.apache.samza.config.JobConfig;
 import org.apache.samza.config.TaskConfig;
 import org.apache.samza.coordinator.CoordinationUtils;
 import org.apache.samza.coordinator.CoordinationUtilsFactory;
-import org.apache.samza.coordinator.Latch;
-import org.apache.samza.coordinator.LeaderElector;
+import org.apache.samza.coordinator.DistributedLockWithState;
 import org.apache.samza.execution.ExecutionPlan;
 import org.apache.samza.job.ApplicationStatus;
 import org.apache.samza.processor.StreamProcessor;
@@ -56,10 +55,7 @@ import org.slf4j.LoggerFactory;
 public class LocalApplicationRunner extends AbstractApplicationRunner {
 
   private static final Logger LOG = LoggerFactory.getLogger(LocalApplicationRunner.class);
-  private static final String APPLICATION_RUNNER_ZK_PATH_SUFFIX = "/ApplicationRunnerData";
-  // Latch timeout is set to 10 min
-  private static final long LATCH_TIMEOUT_MINUTES = 10;
-  private static final long LEADER_ELECTION_WAIT_TIME_MS = 1000;
+  private static final String APPLICATION_RUNNER_PATH_SUFFIX = "/ApplicationRunnerData";
 
   private final String uid;
   private final Set<StreamProcessor> processors = ConcurrentHashMap.newKeySet();
@@ -214,36 +210,39 @@ public class LocalApplicationRunner extends AbstractApplicationRunner {
    * @throws TimeoutException exception for latch timeout
    */
   /* package private */ void createStreams(String planId, List<StreamSpec> intStreams) throws TimeoutException {
-    if (!intStreams.isEmpty()) {
-      // Move the scope of coordination utils within stream creation to address long idle connection problem.
-      // Refer SAMZA-1385 for more details
-
-      String coordinationId = new ApplicationConfig(config).getGlobalAppId() + APPLICATION_RUNNER_ZK_PATH_SUFFIX;
-      CoordinationUtils coordinationUtils =
-          CoordinationUtilsFactory.getCoordinationUtilsFactory(config).getCoordinationUtils(coordinationId, uid, config);
-      if (coordinationUtils != null) {
-        Latch initLatch = coordinationUtils.getLatch(1, planId);
-
-        try {
-          // check if the processor needs to go through leader election and stream creation
-          if (shouldContestInElectionForStreamCreation(initLatch)) {
-            LeaderElector leaderElector = coordinationUtils.getLeaderElector();
-            leaderElector.setLeaderElectorListener(() -> {
-                getStreamManager().createStreams(intStreams);
-                initLatch.countDown();
-              });
-            leaderElector.tryBecomeLeader();
-            initLatch.await(LATCH_TIMEOUT_MINUTES, TimeUnit.MINUTES);
-          }
-        } finally {
-          if (initLatch != null)
-            coordinationUtils.close();
-        }
-      } else {
-        // each application process will try creating the streams, which
-        // requires stream creation to be idempotent
+    if (intStreams.isEmpty()) {
+      LOG.info("Set of intermediate streams is empty. Nothing to create.");
+      return;
+    }
+    LOG.info("A single processor must create the intermediate streams. Processor {} will attempt to acquire the lock.", uid);
+    // Move the scope of coordination utils within stream creation to address long idle connection problem.
+    // Refer SAMZA-1385 for more details
+    String coordinationId = new ApplicationConfig(config).getGlobalAppId() + APPLICATION_RUNNER_PATH_SUFFIX;
+    CoordinationUtils coordinationUtils =
+        CoordinationUtilsFactory.getCoordinationUtilsFactory(config).getCoordinationUtils(coordinationId, uid, config);
+    if (coordinationUtils == null) {
+      LOG.warn("Processor {} failed to create utils. Each processor will attempt to create streams.", uid);
+      // each application process will try creating the streams, which
+      // requires stream creation to be idempotent
+      getStreamManager().createStreams(intStreams);
+      return;
+    }
+
+    DistributedLockWithState lockWithState = coordinationUtils.getLockWithState(planId);
+    try {
+      // check if the processor needs to go through leader election and stream creation
+      if (lockWithState.lockIfNotSet(1000, TimeUnit.MILLISECONDS)) {
+        LOG.info("lock acquired for streams creation by " + uid);
         getStreamManager().createStreams(intStreams);
+        lockWithState.unlockAndSet();
+      } else {
+        LOG.info("Processor {} did not obtain the lock for streams creation. They must've been created by another processor.", uid);
       }
+    } catch (TimeoutException e) {
+      String msg = String.format("Processor {} failed to get the lock for stream initialization", uid);
+      throw new SamzaException(msg, e);
+    } finally {
+      coordinationUtils.close();
     }
   }
 
@@ -270,31 +269,4 @@ public class LocalApplicationRunner extends AbstractApplicationRunner {
           taskFactory.getClass().getCanonicalName()));
     }
   }
-
-  /**
-   * In order to fix SAMZA-1385, we are limiting the scope of coordination util within stream creation phase and destroying
-   * the coordination util right after. By closing the zk connection, we clean up the ephemeral node used for leader election.
-   * It creates the following issues whenever a new process joins after the ephemeral node is gone.
-   *    1. It is unnecessary to re-conduct leader election for stream creation in the same application lifecycle
-   *    2. Underlying systems may not support check for stream existence prior to creation which could have potential problems.
-   * As in interim solution, we reuse the same latch as a marker to determine if create streams phase is done for the
-   * application lifecycle using {@link Latch#await(long, TimeUnit)}
-   *
-   * @param streamCreationLatch latch used for stream creation
-   * @return true if processor needs to be part of election
-   *         false otherwise
-   */
-  private boolean shouldContestInElectionForStreamCreation(Latch streamCreationLatch) {
-    boolean eligibleForElection = true;
-
-    try {
-      streamCreationLatch.await(LEADER_ELECTION_WAIT_TIME_MS, TimeUnit.MILLISECONDS);
-      // case we didn't time out suggesting that latch already exists
-      eligibleForElection = false;
-    } catch (TimeoutException e) {
-      LOG.info("Timed out waiting for the latch! Going to enter leader election section to create streams");
-    }
-
-    return eligibleForElection;
-  }
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/dd07e074/samza-core/src/main/java/org/apache/samza/zk/ZkCoordinationUtils.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/zk/ZkCoordinationUtils.java b/samza-core/src/main/java/org/apache/samza/zk/ZkCoordinationUtils.java
index 05886db..3d4a2d1 100644
--- a/samza-core/src/main/java/org/apache/samza/zk/ZkCoordinationUtils.java
+++ b/samza-core/src/main/java/org/apache/samza/zk/ZkCoordinationUtils.java
@@ -21,6 +21,7 @@ package org.apache.samza.zk;
 import org.I0Itec.zkclient.exception.ZkInterruptedException;
 import org.apache.samza.config.ZkConfig;
 import org.apache.samza.coordinator.CoordinationUtils;
+import org.apache.samza.coordinator.DistributedLockWithState;
 import org.apache.samza.coordinator.Latch;
 import org.apache.samza.coordinator.LeaderElector;
 import org.slf4j.Logger;
@@ -51,6 +52,10 @@ public class ZkCoordinationUtils implements CoordinationUtils {
   }
 
   @Override
+  public DistributedLockWithState getLockWithState(String lockId) {
+    return new ZkDistributedLock(processorIdStr, zkUtils, lockId);
+  }
+
   public void close() {
     try {
       if (zkUtils != null)

http://git-wip-us.apache.org/repos/asf/samza/blob/dd07e074/samza-core/src/main/java/org/apache/samza/zk/ZkCoordinationUtilsFactory.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/zk/ZkCoordinationUtilsFactory.java b/samza-core/src/main/java/org/apache/samza/zk/ZkCoordinationUtilsFactory.java
index ded6a38..8dd42c1 100644
--- a/samza-core/src/main/java/org/apache/samza/zk/ZkCoordinationUtilsFactory.java
+++ b/samza-core/src/main/java/org/apache/samza/zk/ZkCoordinationUtilsFactory.java
@@ -23,8 +23,8 @@ import org.I0Itec.zkclient.ZkClient;
 import org.apache.samza.SamzaException;
 import org.apache.samza.config.Config;
 import org.apache.samza.config.ZkConfig;
-import org.apache.samza.coordinator.CoordinationUtilsFactory;
 import org.apache.samza.coordinator.CoordinationUtils;
+import org.apache.samza.coordinator.CoordinationUtilsFactory;
 import org.apache.samza.util.NoOpMetricsRegistry;
 import org.apache.zookeeper.client.ConnectStringParser;
 import org.slf4j.Logger;

http://git-wip-us.apache.org/repos/asf/samza/blob/dd07e074/samza-core/src/main/java/org/apache/samza/zk/ZkDistributedLock.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/zk/ZkDistributedLock.java b/samza-core/src/main/java/org/apache/samza/zk/ZkDistributedLock.java
new file mode 100644
index 0000000..cfb4641
--- /dev/null
+++ b/samza-core/src/main/java/org/apache/samza/zk/ZkDistributedLock.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.zk;
+
+import java.util.List;
+import java.util.Random;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import org.apache.samza.SamzaException;
+import org.apache.samza.coordinator.DistributedLockWithState;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Distributed lock primitive for Zookeeper.
+ */
+public class ZkDistributedLock implements DistributedLockWithState {
+
+  public static final Logger LOG = LoggerFactory.getLogger(ZkDistributedLock.class);
+  private static final String STATE_INITED = "sate_initialized";
+  private final ZkUtils zkUtils;
+  private final String lockPath;
+  private final String participantId;
+  private final ZkKeyBuilder keyBuilder;
+  private final Random random = new Random();
+  private String nodePath = null;
+  private final String statePath;
+
+  public ZkDistributedLock(String participantId, ZkUtils zkUtils, String lockId) {
+    this.zkUtils = zkUtils;
+    this.participantId = participantId;
+    this.keyBuilder = zkUtils.getKeyBuilder();
+    lockPath = String.format("%s/stateLock_%s", keyBuilder.getRootPath(), lockId);
+    statePath = String.format("%s/%s_%s", lockPath, STATE_INITED, lockId);
+    zkUtils.validatePaths(new String[] {lockPath});
+  }
+
+  /**
+   * Tries to acquire a lock in order to create intermediate streams. On failure to acquire lock, it keeps trying until the lock times out.
+   * Creates a sequential ephemeral node to acquire the lock. If the path of this node has the lowest sequence number, the processor has acquired the lock.
+   * @param timeout Duration of lock acquiring timeout.
+   * @param unit Unit of the timeout defined above.
+   * @return true if lock is acquired successfully, false if it times out.
+   */
+  @Override
+  public boolean lockIfNotSet(long timeout, TimeUnit unit)
+      throws TimeoutException {
+
+    nodePath = zkUtils.getZkClient().createEphemeralSequential(lockPath + "/", participantId);
+
+    //Start timer for timeout
+    long startTime = System.currentTimeMillis();
+    long lockTimeout = TimeUnit.MILLISECONDS.convert(timeout, unit);
+
+    while ((System.currentTimeMillis() - startTime) < lockTimeout) {
+
+      if (zkUtils.getZkClient().exists(statePath)) {
+        // state already set, no point locking
+        return false;
+      }
+
+      List<String> children = zkUtils.getZkClient().getChildren(lockPath);
+      int index = children.indexOf(ZkKeyBuilder.parseIdFromPath(nodePath));
+
+      if (children.size() == 0 || index == -1) {
+        throw new SamzaException("Looks like we are no longer connected to Zk. Need to reconnect!");
+      }
+      // Acquires lock when the node has the lowest sequence number and returns.
+      if (index == 0) {
+        LOG.info("Acquired lock for participant id: {}", participantId);
+        return true;
+      } else {
+        try {
+          Thread.sleep(random.nextInt(1000));
+        } catch (InterruptedException e) {
+          Thread.interrupted();
+        }
+        LOG.info("Trying to acquire lock again...");
+      }
+    }
+    throw new TimeoutException("could not acquire lock for " + timeout + " " + unit.toString());
+  }
+
+  /**
+   * Unlocks, by deleting the ephemeral sequential node created to acquire the lock.
+   */
+  @Override
+  public void unlockAndSet() {
+    // set state
+    zkUtils.getZkClient().createPersistent(statePath, true);
+
+    if (nodePath != null) {
+      zkUtils.getZkClient().delete(nodePath);
+      nodePath = null;
+      LOG.info("Ephemeral lock node deleted. Unlocked!");
+    } else {
+      LOG.warn("Ephemeral lock node you want to delete doesn't exist");
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/samza/blob/dd07e074/samza-core/src/test/java/org/apache/samza/runtime/TestApplicationRunnerMain.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/runtime/TestApplicationRunnerMain.java b/samza-core/src/test/java/org/apache/samza/runtime/TestApplicationRunnerMain.java
index d22fbae..eb0ebe9 100644
--- a/samza-core/src/test/java/org/apache/samza/runtime/TestApplicationRunnerMain.java
+++ b/samza-core/src/test/java/org/apache/samza/runtime/TestApplicationRunnerMain.java
@@ -25,7 +25,7 @@ import org.apache.samza.job.ApplicationStatus;
 import org.apache.samza.operators.StreamGraph;
 import org.junit.Test;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
 
 
 public class TestApplicationRunnerMain {

http://git-wip-us.apache.org/repos/asf/samza/blob/dd07e074/samza-core/src/test/java/org/apache/samza/runtime/TestLocalApplicationRunner.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/runtime/TestLocalApplicationRunner.java b/samza-core/src/test/java/org/apache/samza/runtime/TestLocalApplicationRunner.java
index 38f58fd..f9c1252 100644
--- a/samza-core/src/test/java/org/apache/samza/runtime/TestLocalApplicationRunner.java
+++ b/samza-core/src/test/java/org/apache/samza/runtime/TestLocalApplicationRunner.java
@@ -25,8 +25,6 @@ import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
 import java.util.stream.Collectors;
 import org.apache.samza.application.StreamApplication;
 import org.apache.samza.config.ApplicationConfig;
@@ -35,9 +33,7 @@ import org.apache.samza.config.MapConfig;
 import org.apache.samza.config.TaskConfig;
 import org.apache.samza.coordinator.CoordinationUtils;
 import org.apache.samza.coordinator.CoordinationUtilsFactory;
-import org.apache.samza.coordinator.Latch;
-import org.apache.samza.coordinator.LeaderElector;
-import org.apache.samza.coordinator.LeaderElectorListener;
+import org.apache.samza.coordinator.DistributedLockWithState;
 import org.apache.samza.execution.ExecutionPlan;
 import org.apache.samza.execution.ExecutionPlanner;
 import org.apache.samza.execution.StreamManager;
@@ -54,7 +50,6 @@ import org.powermock.modules.junit4.PowerMockRunner;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
-import static org.mockito.Matchers.anyInt;
 import static org.mockito.Matchers.anyObject;
 import static org.mockito.Matchers.anyString;
 import static org.mockito.Mockito.*;
@@ -168,53 +163,13 @@ public class TestLocalApplicationRunner {
     LocalApplicationRunner spy = spy(runner);
 
     CoordinationUtils coordinationUtils = mock(CoordinationUtils.class);
-    LeaderElector leaderElector = new LeaderElector() {
-      private LeaderElectorListener leaderElectorListener;
-
-      @Override
-      public void setLeaderElectorListener(LeaderElectorListener listener) {
-        this.leaderElectorListener = listener;
-      }
-
-      @Override
-      public void tryBecomeLeader() {
-        leaderElectorListener.onBecomingLeader();
-      }
-
-      @Override
-      public void resignLeadership() {
-      }
-
-      @Override
-      public boolean amILeader() {
-        return false;
-      }
-    };
-
-    Latch latch = new Latch() {
-      boolean done = false;
-
-      @Override
-      public void await(long timeout, TimeUnit tu)
-          throws TimeoutException {
-        // in this test, latch is released after countDown is invoked
-        if (!done) {
-          throw new TimeoutException("timed out waiting for the target path");
-        }
-      }
-
-      @Override
-      public void countDown() {
-        done = true;
-      }
-    };
-
-    mockStatic(CoordinationUtilsFactory.class);
     CoordinationUtilsFactory coordinationUtilsFactory = mock(CoordinationUtilsFactory.class);
+    mockStatic(CoordinationUtilsFactory.class);
     when(CoordinationUtilsFactory.getCoordinationUtilsFactory(anyObject())).thenReturn(coordinationUtilsFactory);
 
-    when(coordinationUtils.getLeaderElector()).thenReturn(leaderElector);
-    when(coordinationUtils.getLatch(anyInt(), anyString())).thenReturn(latch);
+    DistributedLockWithState lock = mock(DistributedLockWithState.class);
+    when(lock.lockIfNotSet(anyLong(), anyObject())).thenReturn(true);
+    when(coordinationUtils.getLockWithState(anyString())).thenReturn(lock);
     when(coordinationUtilsFactory.getCoordinationUtils(anyString(), anyString(), anyObject()))
         .thenReturn(coordinationUtils);
 
@@ -384,6 +339,7 @@ public class TestLocalApplicationRunner {
         new StreamSpec("test-stream-1", "stream-1", "testStream"),
         new StreamSpec("test-stream-3", "stream-3", "testStream"));
 
+
     assertFalse("Expected both of the latch ids to be different",
         planIdBeforeShuffle.equals(getExecutionPlanId(shuffledStreamSpecs)));
   }
@@ -419,6 +375,7 @@ public class TestLocalApplicationRunner {
         new StreamSpec("test-stream-4", "stream-4", "testStream"),
         new StreamSpec("test-stream-3", "stream-3", "testStream"));
 
+
     assertFalse("Expected both of the latch ids to be different",
         planIdBeforeShuffle.equals(getExecutionPlanId(updatedStreamSpecs)));
   }


[21/26] samza git commit: SAMZA-1410 make tests pass in different Junit versions for testPlanIdWithShuffledStreamSpecs and testGeneratePlanIdWithDifferentStreamSpec

Posted by xi...@apache.org.
SAMZA-1410 make tests pass in different Junit versions for testPlanIdWithShuffledStreamSpecs and testGeneratePlanIdWithDifferentStreamSpec

More details are in https://issues.apache.org/jira/browse/SAMZA-1410.

gradlew build and test passed.

Author: Fred Ji <ha...@gmail.com>

Reviewers: Xinyu Liu <xi...@apache.org>

Closes #289 from fredji97/assertNotEquals


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

Branch: refs/heads/0.14.0
Commit: 65743863626a3d6621ba7a4b101c7a91b807bce2
Parents: 3636be0
Author: Fred Ji <ha...@gmail.com>
Authored: Mon Aug 28 14:17:32 2017 -0700
Committer: Xinyu Liu <xi...@xiliu-ld1.linkedin.biz>
Committed: Mon Aug 28 14:17:32 2017 -0700

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


http://git-wip-us.apache.org/repos/asf/samza/blob/65743863/samza-core/src/test/java/org/apache/samza/runtime/TestLocalApplicationRunner.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/runtime/TestLocalApplicationRunner.java b/samza-core/src/test/java/org/apache/samza/runtime/TestLocalApplicationRunner.java
index 6c7827e..ecacf25 100644
--- a/samza-core/src/test/java/org/apache/samza/runtime/TestLocalApplicationRunner.java
+++ b/samza-core/src/test/java/org/apache/samza/runtime/TestLocalApplicationRunner.java
@@ -371,8 +371,8 @@ public class TestLocalApplicationRunner {
         new StreamSpec("test-stream-1", "stream-1", "testStream"),
         new StreamSpec("test-stream-3", "stream-3", "testStream"));
 
-    assertNotEquals("Expected both of the latch ids to be different", planIdBeforeShuffle,
-        getExecutionPlanId(shuffledStreamSpecs));
+    assertFalse("Expected both of the latch ids to be different",
+        planIdBeforeShuffle.equals(getExecutionPlanId(shuffledStreamSpecs)));
   }
 
   /**
@@ -409,8 +409,8 @@ public class TestLocalApplicationRunner {
         new StreamSpec("test-stream-4", "stream-4", "testStream"),
         new StreamSpec("test-stream-3", "stream-3", "testStream"));
 
-    assertNotEquals("Expected both of the latch ids to be different", planIdBeforeShuffle,
-        getExecutionPlanId(updatedStreamSpecs));
+    assertFalse("Expected both of the latch ids to be different",
+        planIdBeforeShuffle.equals(getExecutionPlanId(updatedStreamSpecs)));
   }
 
   private String getExecutionPlanId(List<StreamSpec> updatedStreamSpecs) {


[15/26] samza git commit: SAMZA-1378: Introduce and Implement Scheduler Interface for Polling in Azure

Posted by xi...@apache.org.
SAMZA-1378: Introduce and Implement Scheduler Interface for Polling in Azure

PR 1: AzureClient + AzureConfig
PR 2: LeaseBlobManager
PR 3: BlobUtils + JobModelBundle
PR 4: TableUtils + ProcessorEntity
PR 5: AzureLeaderElector
PR 6: Added all schedulers (current PR)

Author: PawasChhokra <Jaimatadi1$>
Author: PawasChhokra <pa...@gmail.com>

Reviewers: Navina Ramesh <na...@apache.org>

Closes #261 from PawasChhokra/AzureSchedulers


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

Branch: refs/heads/0.14.0
Commit: c3b447ecb343ddc4e48296448127fda5dfafe913
Parents: 1d253c7
Author: Pawas Chhokra <pa...@gmail.com>
Authored: Fri Aug 18 14:38:46 2017 -0700
Committer: navina <na...@apache.org>
Committed: Fri Aug 18 14:38:46 2017 -0700

----------------------------------------------------------------------
 .../main/java/org/apache/samza/AzureClient.java |  18 +-
 .../main/java/org/apache/samza/AzureConfig.java |  73 ---
 .../org/apache/samza/AzureLeaderElector.java    | 111 ----
 .../main/java/org/apache/samza/BlobUtils.java   | 280 ----------
 .../java/org/apache/samza/JobModelBundle.java   |  61 ---
 .../java/org/apache/samza/LeaseBlobManager.java |  98 ----
 .../java/org/apache/samza/ProcessorEntity.java  |  58 ---
 .../main/java/org/apache/samza/TableUtils.java  | 198 --------
 .../org/apache/samza/config/AzureConfig.java    |  68 +++
 .../samza/coordinator/AzureJobCoordinator.java  | 509 +++++++++++++++++++
 .../samza/coordinator/AzureLeaderElector.java   | 109 ++++
 .../samza/coordinator/data/BarrierState.java    |  27 +
 .../samza/coordinator/data/JobModelBundle.java  |  61 +++
 .../samza/coordinator/data/ProcessorEntity.java |  62 +++
 .../scheduler/HeartbeatScheduler.java           |  81 +++
 .../scheduler/JMVersionUpgradeScheduler.java    |  99 ++++
 .../LeaderBarrierCompleteScheduler.java         | 118 +++++
 .../scheduler/LeaderLivenessCheckScheduler.java | 120 +++++
 .../scheduler/LivenessCheckScheduler.java       | 108 ++++
 .../scheduler/RenewLeaseScheduler.java          |  79 +++
 .../scheduler/SchedulerStateChangeListener.java |  29 ++
 .../coordinator/scheduler/TaskScheduler.java    |  35 ++
 .../java/org/apache/samza/util/BlobUtils.java   | 284 +++++++++++
 .../org/apache/samza/util/LeaseBlobManager.java |  99 ++++
 .../java/org/apache/samza/util/TableUtils.java  | 205 ++++++++
 25 files changed, 2105 insertions(+), 885 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/samza/blob/c3b447ec/samza-azure/src/main/java/org/apache/samza/AzureClient.java
----------------------------------------------------------------------
diff --git a/samza-azure/src/main/java/org/apache/samza/AzureClient.java b/samza-azure/src/main/java/org/apache/samza/AzureClient.java
index 2248d12..04f8fd3 100644
--- a/samza-azure/src/main/java/org/apache/samza/AzureClient.java
+++ b/samza-azure/src/main/java/org/apache/samza/AzureClient.java
@@ -25,6 +25,7 @@ import com.microsoft.azure.storage.RetryPolicy;
 import com.microsoft.azure.storage.blob.BlobRequestOptions;
 import com.microsoft.azure.storage.blob.CloudBlobClient;
 import com.microsoft.azure.storage.table.CloudTableClient;
+import com.microsoft.azure.storage.table.TableRequestOptions;
 import java.net.URISyntaxException;
 import java.security.InvalidKeyException;
 import org.slf4j.Logger;
@@ -44,21 +45,26 @@ public class AzureClient {
   /**
    * Creates a reference to the Azure Storage account according to the connection string that the client passes.
    * Also creates references to Azure Blob Storage and Azure Table Storage.
-   * @param storageConnectionString Connection string to conenct to Azure Storage Account, format: "DefaultEndpointsProtocol=<https>;AccountName=<>;AccountKey=<>"
+   * @param storageConnectionString Connection string to connect to Azure Storage Account
+   *                                Format: DefaultEndpointsProtocol=https;AccountName="Insert your account name";AccountKey="Insert your account key"
    * @throws AzureException If an Azure storage service error occurred, or when the storageConnectionString is invalid.
    */
-  AzureClient(String storageConnectionString) {
+  public AzureClient(String storageConnectionString) {
     try {
       account = CloudStorageAccount.parse(storageConnectionString);
+      RetryPolicy retryPolicy = new RetryLinearRetry(5000,  3);
 
       blobClient = account.createCloudBlobClient();
       // Set retry policy for operations on the blob. In this case, every failed operation on the blob will be retried thrice, after 5 second intervals.
-      BlobRequestOptions options = new BlobRequestOptions();
-      RetryPolicy retryPolicy = new RetryLinearRetry(5000, 3);
-      options.setRetryPolicyFactory(retryPolicy);
-      blobClient.setDefaultRequestOptions(options);
+      BlobRequestOptions blobOptions = new BlobRequestOptions();
+      blobOptions.setRetryPolicyFactory(retryPolicy);
+      blobClient.setDefaultRequestOptions(blobOptions);
 
+      // Set retry policy for operations on the table. In this case, every failed operation on the table will be retried thrice, after 5 second intervals.
       tableClient = account.createCloudTableClient();
+      TableRequestOptions tableOptions = new TableRequestOptions();
+      tableOptions.setRetryPolicyFactory(retryPolicy);
+      tableClient.setDefaultRequestOptions(tableOptions);
     } catch (IllegalArgumentException | URISyntaxException e) {
       LOG.error("Connection string {} specifies an invalid URI.", storageConnectionString);
       LOG.error("Please confirm the connection string is in the Azure connection string format.");

http://git-wip-us.apache.org/repos/asf/samza/blob/c3b447ec/samza-azure/src/main/java/org/apache/samza/AzureConfig.java
----------------------------------------------------------------------
diff --git a/samza-azure/src/main/java/org/apache/samza/AzureConfig.java b/samza-azure/src/main/java/org/apache/samza/AzureConfig.java
deleted file mode 100644
index 47873a7..0000000
--- a/samza-azure/src/main/java/org/apache/samza/AzureConfig.java
+++ /dev/null
@@ -1,73 +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;
-
-import org.apache.samza.config.ApplicationConfig;
-import org.apache.samza.config.Config;
-import org.apache.samza.config.ConfigException;
-import org.apache.samza.config.MapConfig;
-
-/**
- * Config class for reading all user defined parameters for Azure driven coordination services.
- */
-public class AzureConfig extends MapConfig {
-
-  // Connection string for Azure Storage Account, format: "DefaultEndpointsProtocol=<https>;AccountName=<>;AccountKey=<>"
-  public static final String AZURE_STORAGE_CONNECT = "azure.storage.connect";
-  public static final String AZURE_PAGEBLOB_LENGTH = "job.coordinator.azure.blob.length";
-  public static final long DEFAULT_AZURE_PAGEBLOB_LENGTH = 5120000;
-
-  private static String containerName;
-  private static String blobName;
-  private static String tableName;
-
-  public AzureConfig(Config config) {
-    super(config);
-    ApplicationConfig appConfig = new ApplicationConfig(config);
-    //Remove all non-alphanumeric characters from id as table name does not allow them.
-    String id = appConfig.getGlobalAppId().replaceAll("[^A-Za-z0-9]", "");
-    containerName = "samzacontainer" + id;
-    blobName = "samzablob" + id;
-    tableName = "samzatable" + id;
-  }
-
-  public String getAzureConnect() {
-    if (!containsKey(AZURE_STORAGE_CONNECT)) {
-      throw new ConfigException("Missing " + AZURE_STORAGE_CONNECT + " config!");
-    }
-    return get(AZURE_STORAGE_CONNECT);
-  }
-
-  public String getAzureContainerName() {
-    return containerName;
-  }
-
-  public String getAzureBlobName() {
-    return blobName;
-  }
-
-  public long getAzureBlobLength() {
-    return getLong(AZURE_PAGEBLOB_LENGTH, DEFAULT_AZURE_PAGEBLOB_LENGTH);
-  }
-
-  public String getAzureTableName() {
-    return tableName;
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/samza/blob/c3b447ec/samza-azure/src/main/java/org/apache/samza/AzureLeaderElector.java
----------------------------------------------------------------------
diff --git a/samza-azure/src/main/java/org/apache/samza/AzureLeaderElector.java b/samza-azure/src/main/java/org/apache/samza/AzureLeaderElector.java
deleted file mode 100644
index efa8ea1..0000000
--- a/samza-azure/src/main/java/org/apache/samza/AzureLeaderElector.java
+++ /dev/null
@@ -1,111 +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;
-
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicReference;
-import org.apache.samza.coordinator.LeaderElector;
-import org.apache.samza.coordinator.LeaderElectorListener;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-
-/**
- * Class to facilitate leader election in Azure.
- * The processor that acquires the lease on the blob becomes the leader.
- * The lease ID is null initially. It is generated by Azure when the processor acquires the lease, and updated accordingly.
- * Every processor requires a valid active lease ID in order to perform successful write and delete operations on the blob.
- * Read operations from the blob are not dependent on the lease ID.
- */
-public class AzureLeaderElector implements LeaderElector {
-
-  private static final Logger LOG = LoggerFactory.getLogger(AzureLeaderElector.class);
-  private static final int LEASE_TIME_IN_SEC = 60;
-  private final LeaseBlobManager leaseBlobManager;
-  private LeaderElectorListener leaderElectorListener = null;
-  private final AtomicReference<String> leaseId;
-  private final AtomicBoolean isLeader;
-
-  public AzureLeaderElector(LeaseBlobManager leaseBlobManager) {
-    this.isLeader = new AtomicBoolean(false);
-    this.leaseBlobManager = leaseBlobManager;
-    this.leaseId = new AtomicReference<>(null);
-  }
-
-  @Override
-  public void setLeaderElectorListener(LeaderElectorListener listener) {
-    this.leaderElectorListener = listener;
-  }
-
-  /**
-   * Tries to become the leader by acquiring a lease on the blob.
-   * The acquireLease operation has a retry policy where upon failure, the operation is tried 3 times at 5 second intervals.
-   * Invokes the listener on becoming the leader.
-   */
-  @Override
-  public void tryBecomeLeader() {
-    try {
-      leaseId.getAndSet(leaseBlobManager.acquireLease(LEASE_TIME_IN_SEC, leaseId.get()));
-      if (leaseId.get() != null) {
-        LOG.info("Became leader with lease ID {}.", leaseId.get());
-        isLeader.set(true);
-        if (leaderElectorListener != null) {
-          leaderElectorListener.onBecomingLeader();
-        }
-      }
-    } catch (AzureException e) {
-      LOG.error("Error while trying to acquire lease.", e);
-    }
-  }
-
-  /**
-   * Releases the lease in order to resign leadership. It also stops all schedulers scheduled by the leader.
-   * The releaseLease operation has a retry policy where upon failure, the operation is tried 3 times at 5 second intervals.
-   */
-  @Override
-  public void resignLeadership() {
-    if (isLeader.get()) {
-      leaseBlobManager.releaseLease(leaseId.get());
-      isLeader.set(false);
-      LOG.info("Resigning leadership with lease ID {}", leaseId.get());
-      leaseId.getAndSet(null);
-    } else {
-      LOG.info("Can't release the lease because it is not the leader and does not hold an active lease.");
-    }
-  }
-
-  /**
-   * Checks whether it's a leader
-   * @return true if it is the leader, false otherwise
-   */
-  @Override
-  public boolean amILeader() {
-    return isLeader.get();
-  }
-
-  public AtomicReference<String> getLeaseId() {
-    return leaseId;
-  }
-
-  public LeaseBlobManager getLeaseBlobManager() {
-    return this.leaseBlobManager;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/c3b447ec/samza-azure/src/main/java/org/apache/samza/BlobUtils.java
----------------------------------------------------------------------
diff --git a/samza-azure/src/main/java/org/apache/samza/BlobUtils.java b/samza-azure/src/main/java/org/apache/samza/BlobUtils.java
deleted file mode 100644
index a798384..0000000
--- a/samza-azure/src/main/java/org/apache/samza/BlobUtils.java
+++ /dev/null
@@ -1,280 +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;
-
-import com.microsoft.azure.storage.AccessCondition;
-import com.microsoft.azure.storage.StorageException;
-import com.microsoft.azure.storage.blob.CloudBlobClient;
-import com.microsoft.azure.storage.blob.CloudBlobContainer;
-import com.microsoft.azure.storage.blob.CloudPageBlob;
-import java.io.ByteArrayInputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.net.URISyntaxException;
-import java.util.Arrays;
-import java.util.List;
-import org.apache.samza.job.model.JobModel;
-import org.apache.samza.serializers.model.SamzaObjectMapper;
-import org.eclipse.jetty.http.HttpStatus;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-
-/**
- * Client side class that has reference to Azure blob storage.
- * Used for writing and reading from the blob.
- * Every write requires a valid lease ID.
- */
-public class BlobUtils {
-
-  private static final Logger LOG = LoggerFactory.getLogger(BlobUtils.class);
-  private static final long JOB_MODEL_BLOCK_SIZE = 1024000;
-  private static final long BARRIER_STATE_BLOCK_SIZE = 1024;
-  private static final long PROCESSOR_LIST_BLOCK_SIZE = 1024;
-  private CloudBlobClient blobClient;
-  private CloudBlobContainer container;
-  private CloudPageBlob blob;
-
-  /**
-   * Creates an object of BlobUtils. It creates the container and page blob if they don't exist already.
-   * @param client Client handle for access to Azure Storage account.
-   * @param containerName Name of container inside which we want the blob to reside.
-   * @param blobName Name of the blob to be managed.
-   * @param length Length of the page blob.
-   * @throws AzureException If an Azure storage service error occurred, or when the container name or blob name is invalid.
-   */
-  public BlobUtils(AzureClient client, String containerName, String blobName, long length) {
-    this.blobClient = client.getBlobClient();
-    try {
-      this.container = blobClient.getContainerReference(containerName);
-      container.createIfNotExists();
-      this.blob = container.getPageBlobReference(blobName);
-      if (!blob.exists()) {
-        blob.create(length, AccessCondition.generateIfNotExistsCondition(), null, null);
-      }
-    } catch (URISyntaxException e) {
-      LOG.error("Container name: " + containerName + " or blob name: " + blobName + " invalid.", e);
-      throw new AzureException(e);
-    } catch (StorageException e) {
-      int httpStatusCode = e.getHttpStatusCode();
-      if (httpStatusCode == HttpStatus.CONFLICT_409) {
-        LOG.info("The blob you're trying to create exists already.", e);
-      } else {
-        LOG.error("Azure Storage Exception!", e);
-        throw new AzureException(e);
-      }
-    }
-  }
-
-  /**
-   * Writes the job model to the blob.
-   * Write is successful only if the lease ID passed is valid and the processor holds the lease.
-   * Called by the leader.
-   * @param prevJM Previous job model version that the processor was operating on.
-   * @param currJM Current job model version that the processor is operating on.
-   * @param prevJMV Previous job model version that the processor was operating on.
-   * @param currJMV Current job model version that the processor is operating on.
-   * @param leaseId LeaseID of the lease that the processor holds on the blob. Null if there is no lease.
-   * @return true if write to the blob is successful, false if leaseID is null or an Azure storage service error or IO exception occurred.
-   */
-  public boolean publishJobModel(JobModel prevJM, JobModel currJM, String prevJMV, String currJMV, String leaseId) {
-    try {
-      if (leaseId == null) {
-        return false;
-      }
-      JobModelBundle bundle = new JobModelBundle(prevJM, currJM, prevJMV, currJMV);
-      byte[] data = SamzaObjectMapper.getObjectMapper().writerWithDefaultPrettyPrinter().writeValueAsBytes(bundle);
-      byte[] pageData = Arrays.copyOf(data, (int) JOB_MODEL_BLOCK_SIZE);
-      InputStream is = new ByteArrayInputStream(pageData);
-      blob.uploadPages(is, 0, JOB_MODEL_BLOCK_SIZE, AccessCondition.generateLeaseCondition(leaseId), null, null);
-      LOG.info("Uploaded {} jobModel to blob", bundle.getCurrJobModel());
-      return true;
-    } catch (StorageException | IOException e) {
-      LOG.error("JobModel publish failed for version = " + currJMV, e);
-      return false;
-    }
-  }
-
-  /**
-   * Reads the current job model from the blob.
-   * @return The current job model published on the blob. Returns null when job model details not found on the blob.
-   * @throws AzureException in getJobModelBundle() if an Azure storage service error occurred.
-   * @throws SamzaException in getJobModelBundle() if data retrieved from blob could not be parsed by SamzaObjectMapper.
-   */
-  public JobModel getJobModel() {
-    LOG.info("Reading the job model from blob.");
-    JobModelBundle jmBundle = getJobModelBundle();
-    if (jmBundle == null) {
-      LOG.error("Job Model details don't exist on the blob.");
-      return null;
-    }
-    JobModel jm = jmBundle.getCurrJobModel();
-    return jm;
-  }
-
-  /**
-   * Reads the current job model version from the blob .
-   * @return Current job model version published on the blob. Returns null when job model details not found on the blob.
-   * @throws AzureException in getJobModelBundle() if an Azure storage service error occurred.
-   * @throws SamzaException in getJobModelBundle() if data retrieved from blob could not be parsed by SamzaObjectMapper.
-   */
-  public String getJobModelVersion() {
-    LOG.info("Reading the job model version from blob.");
-    JobModelBundle jmBundle = getJobModelBundle();
-    if (jmBundle == null) {
-      LOG.error("Job Model details don't exist on the blob.");
-      return null;
-    }
-    String jmVersion = jmBundle.getCurrJobModelVersion();
-    return jmVersion;
-  }
-
-  /**
-   * Writes the barrier state to the blob.
-   * Write is successful only if the lease ID passed is valid and the processor holds the lease.
-   * Called only by the leader.
-   * @param state Barrier state to be published to the blob.
-   * @param leaseId LeaseID of the valid lease that the processor holds on the blob. Null if there is no lease.
-   * @return true if write to the blob is successful, false if leaseID is null or an Azure storage service error or IO exception occurred.
-   */
-  public boolean publishBarrierState(String state, String leaseId) {
-    try {
-      if (leaseId == null) {
-        return false;
-      }
-      byte[] data = SamzaObjectMapper.getObjectMapper().writerWithDefaultPrettyPrinter().writeValueAsBytes(state);
-      byte[] pageData = Arrays.copyOf(data, (int) BARRIER_STATE_BLOCK_SIZE);
-      InputStream is = new ByteArrayInputStream(pageData);
-
-      //uploadPages is only successful when the AccessCondition provided has an active and valid lease ID. It fails otherwise.
-      blob.uploadPages(is, JOB_MODEL_BLOCK_SIZE, BARRIER_STATE_BLOCK_SIZE, AccessCondition.generateLeaseCondition(leaseId), null, null);
-      LOG.info("Uploaded barrier state {} to blob", state);
-      return true;
-    } catch (StorageException | IOException e) {
-      LOG.error("Barrier state " + state + " publish failed", e);
-      return false;
-    }
-  }
-
-  /**
-   * Reads the current barrier state from the blob.
-   * @return Barrier state published on the blob.
-   * @throws AzureException If an Azure storage service error occurred.
-   * @throws SamzaException If data retrieved from blob could not be parsed by SamzaObjectMapper.
-   */
-  public String getBarrierState() {
-    LOG.info("Reading the barrier state from blob.");
-    byte[] data = new byte[(int) BARRIER_STATE_BLOCK_SIZE];
-    try {
-      blob.downloadRangeToByteArray(JOB_MODEL_BLOCK_SIZE, BARRIER_STATE_BLOCK_SIZE, data, 0);
-    } catch (StorageException e) {
-      LOG.error("Failed to read barrier state from blob.", e);
-      throw new AzureException(e);
-    }
-    String state;
-    try {
-      state = SamzaObjectMapper.getObjectMapper().readValue(data, String.class);
-    } catch (IOException e) {
-      LOG.error("Failed to parse byte data: " + data + " for barrier state retrieved from the blob.", e);
-      throw new SamzaException(e);
-    }
-    return state;
-  }
-
-  /**
-   * Writes the list of live processors in the system to the blob.
-   * Write is successful only if the lease ID passed is valid and the processor holds the lease.
-   * Called only by the leader.
-   * @param processors List of live processors to be published on the blob.
-   * @param leaseId LeaseID of the valid lease that the processor holds on the blob. Null if there is no lease.
-   * @return true if write to the blob is successful, false if leaseID is null or an Azure storage service error or IO exception occurred.
-   */
-  public boolean publishLiveProcessorList(List<String> processors, String leaseId) {
-    try {
-      if (leaseId == null) {
-        return false;
-      }
-      byte[] data = SamzaObjectMapper.getObjectMapper().writerWithDefaultPrettyPrinter().writeValueAsBytes(processors);
-      byte[] pageData = Arrays.copyOf(data, (int) BARRIER_STATE_BLOCK_SIZE);
-      InputStream is = new ByteArrayInputStream(pageData);
-      blob.uploadPages(is, JOB_MODEL_BLOCK_SIZE + BARRIER_STATE_BLOCK_SIZE, PROCESSOR_LIST_BLOCK_SIZE, AccessCondition.generateLeaseCondition(leaseId), null, null);
-      LOG.info("Uploaded list of live processors to blob.");
-      return true;
-    } catch (StorageException | IOException e) {
-      LOG.error("Processor list: " + processors + "publish failed", e);
-      return false;
-    }
-  }
-
-  /**
-   * Reads the list of live processors published on the blob.
-   * @return String list of live processors.
-   * @throws AzureException If an Azure storage service error occurred.
-   * @throws SamzaException If data retrieved from blob could not be parsed by SamzaObjectMapper.
-   */
-  public List<String> getLiveProcessorList() {
-    LOG.info("Read the the list of live processors from blob.");
-    byte[] data = new byte[(int) PROCESSOR_LIST_BLOCK_SIZE];
-    try {
-      blob.downloadRangeToByteArray(JOB_MODEL_BLOCK_SIZE + BARRIER_STATE_BLOCK_SIZE, PROCESSOR_LIST_BLOCK_SIZE, data, 0);
-    } catch (StorageException e) {
-      LOG.error("Failed to read the list of live processors from the blob.", new AzureException(e));
-      throw new AzureException(e);
-    }
-    List<String> list;
-    try {
-      list = SamzaObjectMapper.getObjectMapper().readValue(data, List.class);
-    } catch (IOException e) {
-      LOG.error("Failed to parse byte data: " + data + " for live processor list retrieved from the blob", new SamzaException(e));
-      throw new SamzaException(e);
-    }
-    return list;
-  }
-
-  public CloudBlobClient getBlobClient() {
-    return this.blobClient;
-  }
-
-  public CloudBlobContainer getBlobContainer() {
-    return this.container;
-  }
-
-  public CloudPageBlob getBlob() {
-    return this.blob;
-  }
-
-  private JobModelBundle getJobModelBundle() {
-    byte[] data = new byte[(int) JOB_MODEL_BLOCK_SIZE];
-    try {
-      blob.downloadRangeToByteArray(0, JOB_MODEL_BLOCK_SIZE, data, 0);
-    } catch (StorageException e) {
-      LOG.error("Failed to read JobModel details from the blob.", e);
-      throw new AzureException(e);
-    }
-    try {
-      JobModelBundle jmBundle = SamzaObjectMapper.getObjectMapper().readValue(data, JobModelBundle.class);
-      return jmBundle;
-    } catch (IOException e) {
-      LOG.error("Failed to parse byte data: " + data + " for JobModel details retrieved from the blob", e);
-      throw new SamzaException(e);
-    }
-  }
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/samza/blob/c3b447ec/samza-azure/src/main/java/org/apache/samza/JobModelBundle.java
----------------------------------------------------------------------
diff --git a/samza-azure/src/main/java/org/apache/samza/JobModelBundle.java b/samza-azure/src/main/java/org/apache/samza/JobModelBundle.java
deleted file mode 100644
index 3ff971f..0000000
--- a/samza-azure/src/main/java/org/apache/samza/JobModelBundle.java
+++ /dev/null
@@ -1,61 +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;
-
-import org.apache.samza.job.model.JobModel;
-
-
-/**
- * Bundle class for current and previous - job model and job model version.
- * Used for publishing updated data to the blob in one go.
- */
-public class JobModelBundle {
-
-  private JobModel prevJobModel;
-  private JobModel currJobModel;
-  private String prevJobModelVersion;
-  private String currJobModelVersion;
-
-  public JobModelBundle() {}
-
-  public JobModelBundle(JobModel prevJM, JobModel currJM, String prevJMV, String currJMV) {
-    prevJobModel = prevJM;
-    currJobModel = currJM;
-    prevJobModelVersion = prevJMV;
-    currJobModelVersion = currJMV;
-  }
-
-  public JobModel getCurrJobModel() {
-    return currJobModel;
-  }
-
-  public JobModel getPrevJobModel() {
-    return prevJobModel;
-  }
-
-  public String getCurrJobModelVersion() {
-    return currJobModelVersion;
-  }
-
-  public String getPrevJobModelVersion() {
-    return prevJobModelVersion;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/c3b447ec/samza-azure/src/main/java/org/apache/samza/LeaseBlobManager.java
----------------------------------------------------------------------
diff --git a/samza-azure/src/main/java/org/apache/samza/LeaseBlobManager.java b/samza-azure/src/main/java/org/apache/samza/LeaseBlobManager.java
deleted file mode 100644
index 5375662..0000000
--- a/samza-azure/src/main/java/org/apache/samza/LeaseBlobManager.java
+++ /dev/null
@@ -1,98 +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;
-
-import com.microsoft.azure.storage.AccessCondition;
-import com.microsoft.azure.storage.StorageException;
-import com.microsoft.azure.storage.blob.CloudPageBlob;
-import org.eclipse.jetty.http.HttpStatus;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-
-/**
- * Helper class for lease blob operations.
- */
-public class LeaseBlobManager {
-
-  private static final Logger LOG = LoggerFactory.getLogger(LeaseBlobManager.class);
-  private CloudPageBlob leaseBlob;
-
-  public LeaseBlobManager(CloudPageBlob leaseBlob) {
-    this.leaseBlob = leaseBlob;
-  }
-
-  /**
-   * Acquires a lease on a blob. The lease ID is NULL initially.
-   * @param leaseTimeInSec The time in seconds you want to acquire the lease for.
-   * @param leaseId Proposed ID you want to acquire the lease with, null if not proposed.
-   * @return String that represents lease ID.  Null if acquireLease is unsuccessful because the blob is leased already.
-   * @throws AzureException If a Azure storage service error occurred. This includes the case where the blob you're trying to lease does not exist.
-   */
-  public String acquireLease(int leaseTimeInSec, String leaseId) {
-    try {
-      String id = leaseBlob.acquireLease(leaseTimeInSec, leaseId);
-      LOG.info("Acquired lease with lease id = " + id);
-      return id;
-    } catch (StorageException storageException) {
-      int httpStatusCode = storageException.getHttpStatusCode();
-      if (httpStatusCode == HttpStatus.CONFLICT_409) {
-        LOG.info("The blob you're trying to acquire is leased already.", storageException);
-      } else if (httpStatusCode == HttpStatus.NOT_FOUND_404) {
-        LOG.error("The blob you're trying to lease does not exist.", storageException);
-        throw new AzureException(storageException);
-      } else {
-        LOG.error("Error acquiring lease!", storageException);
-        throw new AzureException(storageException);
-      }
-    }
-    return null;
-  }
-
-  /**
-   * Renews the lease on the blob.
-   * @param leaseId ID of the lease to be renewed.
-   * @return True if lease was renewed successfully, false otherwise.
-   */
-  public boolean renewLease(String leaseId) {
-    try {
-      leaseBlob.renewLease(AccessCondition.generateLeaseCondition(leaseId));
-      return true;
-    } catch (StorageException storageException) {
-      LOG.error("Wasn't able to renew lease with lease id: " + leaseId, storageException);
-      return false;
-    }
-  }
-
-  /**
-   * Releases the lease on the blob.
-   * @param leaseId ID of the lease to be released.
-   * @return True if released successfully, false otherwise.
-   */
-  public boolean releaseLease(String leaseId) {
-    try {
-      leaseBlob.releaseLease(AccessCondition.generateLeaseCondition(leaseId));
-      return true;
-    } catch (StorageException storageException) {
-      LOG.error("Wasn't able to release lease with lease id: " + leaseId, storageException);
-      return false;
-    }
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/samza/blob/c3b447ec/samza-azure/src/main/java/org/apache/samza/ProcessorEntity.java
----------------------------------------------------------------------
diff --git a/samza-azure/src/main/java/org/apache/samza/ProcessorEntity.java b/samza-azure/src/main/java/org/apache/samza/ProcessorEntity.java
deleted file mode 100644
index 5145821..0000000
--- a/samza-azure/src/main/java/org/apache/samza/ProcessorEntity.java
+++ /dev/null
@@ -1,58 +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;
-
-import com.microsoft.azure.storage.table.TableServiceEntity;
-
-
-/**
- * Table schema for Azure processor table.
- * Denotes a row in the table with PARTITION KEY = Job Model Version and ROW KEY = Processor ID
- * Other fields include integer liveness value to which each processor heartbeats,
- * and boolean isLeader value which denotes whether the processor is a leader or not.
- */
-public class ProcessorEntity extends TableServiceEntity {
-  private int liveness;
-  private boolean isLeader;
-
-  public ProcessorEntity() {}
-
-  public ProcessorEntity(String jobModelVersion, String processorId) {
-    this.partitionKey = jobModelVersion;
-    this.rowKey = processorId;
-    this.isLeader = false;
-  }
-
-  /**
-   * Updates heartbeat by updating the liveness value in the table.
-   * @param value
-   */
-  public void setLiveness(int value) {
-    liveness = value;
-  }
-
-  public void setIsLeader(boolean leader) {
-    isLeader = leader;
-  }
-
-  public boolean getIsLeader() {
-    return isLeader;
-  }
-}

http://git-wip-us.apache.org/repos/asf/samza/blob/c3b447ec/samza-azure/src/main/java/org/apache/samza/TableUtils.java
----------------------------------------------------------------------
diff --git a/samza-azure/src/main/java/org/apache/samza/TableUtils.java b/samza-azure/src/main/java/org/apache/samza/TableUtils.java
deleted file mode 100644
index e49fd90..0000000
--- a/samza-azure/src/main/java/org/apache/samza/TableUtils.java
+++ /dev/null
@@ -1,198 +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;
-
-import com.microsoft.azure.storage.StorageException;
-import com.microsoft.azure.storage.table.CloudTable;
-import com.microsoft.azure.storage.table.CloudTableClient;
-import com.microsoft.azure.storage.table.TableOperation;
-import com.microsoft.azure.storage.table.TableQuery;
-import java.net.URISyntaxException;
-import java.util.HashSet;
-import java.util.Random;
-import java.util.Set;
-import java.util.concurrent.atomic.AtomicReference;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-
-/**
- *  Client side class that has a reference to Azure Table Storage.
- *  Enables the user to add or delete information from the table, make updates to the table and retrieve information from the table.
- *  Every row in a table is uniquely identified by a combination of the PARTIITON KEY and ROW KEY.
- *  PARTITION KEY = Group ID = Job Model Version (for this case).
- *  ROW KEY = Unique entity ID for a group = Processor ID (for this case).
- */
-public class TableUtils {
-
-  private static final Logger LOG = LoggerFactory.getLogger(TableUtils.class);
-  private static final String PARTITION_KEY = "PartitionKey";
-  private static final long CHECK_LIVENESS_DELAY = 30;
-  private static final String INITIAL_STATE = "unassigned";
-  private CloudTableClient tableClient;
-  private CloudTable table;
-
-  public TableUtils(AzureClient client, String tableName) {
-    tableClient = client.getTableClient();
-    try {
-      table = tableClient.getTableReference(tableName);
-      table.createIfNotExists();
-    } catch (URISyntaxException e) {
-      LOG.error("\nConnection string specifies an invalid URI.", new SamzaException(e));
-    } catch (StorageException e) {
-      LOG.error("Azure storage exception.", new SamzaException(e));
-    }
-  }
-
-  /**
-   * Add a row which denotes an active processor to the processor table.
-   * @param jmVersion Job model version that the processor is operating on.
-   * @param pid Unique processor ID.
-   * @param liveness Random heartbeat value.
-   * @param isLeader Denotes whether the processor is a leader or not.
-   * @throws AzureException If an Azure storage service error occurred.
-   */
-  public void addProcessorEntity(String jmVersion, String pid, int liveness, boolean isLeader) {
-    ProcessorEntity entity = new ProcessorEntity(jmVersion, pid);
-    entity.setIsLeader(isLeader);
-    entity.setLiveness(liveness);
-    TableOperation add = TableOperation.insert(entity);
-    try {
-      table.execute(add);
-    } catch (StorageException e) {
-      LOG.error("Azure storage exception while adding processor entity with job model version: " + jmVersion + "and pid: " + pid, e);
-      throw new AzureException(e);
-    }
-  }
-
-  /**
-   * Retrieve a particular row in the processor table, given the partition key and the row key.
-   * @param jmVersion Job model version of the processor row to be retrieved.
-   * @param pid Unique processor ID of the processor row to be retrieved.
-   * @return An instance of required processor entity. Null if does not exist.
-   * @throws AzureException If an Azure storage service error occurred.
-   */
-  public ProcessorEntity getEntity(String jmVersion, String pid) {
-    try {
-      TableOperation retrieveEntity = TableOperation.retrieve(jmVersion, pid, ProcessorEntity.class);
-      ProcessorEntity entity = table.execute(retrieveEntity).getResultAsType();
-      return entity;
-    } catch (StorageException e) {
-      LOG.error("Azure storage exception while retrieving processor entity with job model version: " + jmVersion + "and pid: " + pid, e);
-      throw new AzureException(e);
-    }
-  }
-
-  /**
-   * Updates the liveness value of a particular processor with a randomly generated integer, which in turn updates the last modified since timestamp of the row.
-   * @param jmVersion Job model version of the processor row to be updated.
-   * @param pid Unique processor ID of the processor row to be updated.
-   */
-  public void updateHeartbeat(String jmVersion, String pid) {
-    try {
-      Random rand = new Random();
-      int value = rand.nextInt(10000) + 2;
-      TableOperation retrieveEntity = TableOperation.retrieve(jmVersion, pid, ProcessorEntity.class);
-      ProcessorEntity entity = table.execute(retrieveEntity).getResultAsType();
-      entity.setLiveness(value);
-      TableOperation update = TableOperation.replace(entity);
-      table.execute(update);
-    } catch (StorageException e) {
-      LOG.error("Azure storage exception while updating heartbeat for job model version: " + jmVersion + "and pid: " + pid, e);
-    }
-  }
-
-  /**
-   * Updates the isLeader value when the processor starts or stops being a leader.
-   * @param jmVersion Job model version of the processor row to be updated.
-   * @param pid Unique processor ID of the processor row to be updated.
-   * @param isLeader Denotes whether the processor is a leader or not.
-   * @throws AzureException If an Azure storage service error occurred.
-   */
-  public void updateIsLeader(String jmVersion, String pid, boolean isLeader) {
-    try {
-      TableOperation retrieveEntity = TableOperation.retrieve(jmVersion, pid, ProcessorEntity.class);
-      ProcessorEntity entity = table.execute(retrieveEntity).getResultAsType();
-      entity.setIsLeader(isLeader);
-      TableOperation update = TableOperation.replace(entity);
-      table.execute(update);
-    } catch (StorageException e) {
-      LOG.error("Azure storage exception while updating isLeader value for job model version: " + jmVersion + "and pid: " + pid, e);
-      throw new AzureException(e);
-    }
-  }
-
-  /**
-   * Deletes a specified row in the processor table.
-   * @param jmVersion Job model version of the processor row to be deleted.
-   * @param pid Unique processor ID of the processor row to be deleted.
-   * @throws AzureException If an Azure storage service error occurred.
-   */
-  public void deleteProcessorEntity(String jmVersion, String pid) {
-    try {
-      TableOperation retrieveEntity = TableOperation.retrieve(jmVersion, pid, ProcessorEntity.class);
-      ProcessorEntity entity = table.execute(retrieveEntity).getResultAsType();
-      TableOperation remove = TableOperation.delete(entity);
-      table.execute(remove);
-    } catch (StorageException e) {
-      LOG.error("Azure storage exception while deleting processor entity with job model version: " + jmVersion + "and pid: " + pid, e);
-      throw new AzureException(e);
-    }
-  }
-
-  /**
-   * Retrieve all rows in a table with the given partition key.
-   * @param partitionKey Job model version of the processors to be retrieved.
-   * @return Iterable list of processor entities.
-   */
-  public Iterable<ProcessorEntity> getEntitiesWithPartition(String partitionKey) {
-    String partitionFilter = TableQuery.generateFilterCondition(PARTITION_KEY, TableQuery.QueryComparisons.EQUAL, partitionKey);
-    TableQuery<ProcessorEntity> partitionQuery = TableQuery.from(ProcessorEntity.class).where(partitionFilter);
-    return table.execute(partitionQuery);
-  }
-
-  /**
-   * Gets the list of all active processors that are heartbeating to the processor table.
-   * @param currentJMVersion Current job model version that the processors in the application are operating on.
-   * @return List of ids of currently active processors in the application, retrieved from the processor table.
-   */
-  public Set<String> getActiveProcessorsList(AtomicReference<String> currentJMVersion) {
-    Iterable<ProcessorEntity> tableList = getEntitiesWithPartition(currentJMVersion.get());
-    Set<String> activeProcessorsList = new HashSet<>();
-    for (ProcessorEntity entity: tableList) {
-      if (System.currentTimeMillis() - entity.getTimestamp().getTime() <= CHECK_LIVENESS_DELAY * 1000) {
-        activeProcessorsList.add(entity.getRowKey());
-      }
-    }
-
-    Iterable<ProcessorEntity> unassignedList = getEntitiesWithPartition(INITIAL_STATE);
-    for (ProcessorEntity entity: unassignedList) {
-      if (System.currentTimeMillis() - entity.getTimestamp().getTime() <= CHECK_LIVENESS_DELAY * 1000) {
-        activeProcessorsList.add(entity.getRowKey());
-      }
-    }
-    return activeProcessorsList;
-  }
-
-  public CloudTable getTable() {
-    return table;
-  }
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/samza/blob/c3b447ec/samza-azure/src/main/java/org/apache/samza/config/AzureConfig.java
----------------------------------------------------------------------
diff --git a/samza-azure/src/main/java/org/apache/samza/config/AzureConfig.java b/samza-azure/src/main/java/org/apache/samza/config/AzureConfig.java
new file mode 100644
index 0000000..dc96d2d
--- /dev/null
+++ b/samza-azure/src/main/java/org/apache/samza/config/AzureConfig.java
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.samza.config;
+
+/**
+ * Config class for reading all user defined parameters for Azure driven coordination services.
+ */
+public class AzureConfig extends MapConfig {
+
+  // Connection string for Azure Storage Account, format: "DefaultEndpointsProtocol=<https>;AccountName=<>;AccountKey=<>"
+  public static final String AZURE_STORAGE_CONNECT = "azure.storage.connect";
+  public static final String AZURE_PAGEBLOB_LENGTH = "job.coordinator.azure.blob.length";
+  public static final long DEFAULT_AZURE_PAGEBLOB_LENGTH = 5120000;
+
+  private static String containerName;
+  private static String blobName;
+  private static String tableName;
+
+  public AzureConfig(Config config) {
+    super(config);
+    ApplicationConfig appConfig = new ApplicationConfig(config);
+    //Remove all non-alphanumeric characters from id as table name does not allow them.
+    String id = appConfig.getGlobalAppId().replaceAll("[^A-Za-z0-9]", "");
+    containerName = "samzacontainer" + id;
+    blobName = "samzablob" + id;
+    tableName = "samzatable" + id;
+  }
+
+  public String getAzureConnect() {
+    if (!containsKey(AZURE_STORAGE_CONNECT)) {
+      throw new ConfigException("Missing " + AZURE_STORAGE_CONNECT + " config!");
+    }
+    return get(AZURE_STORAGE_CONNECT);
+  }
+
+  public String getAzureContainerName() {
+    return containerName;
+  }
+
+  public String getAzureBlobName() {
+    return blobName;
+  }
+
+  public long getAzureBlobLength() {
+    return getLong(AZURE_PAGEBLOB_LENGTH, DEFAULT_AZURE_PAGEBLOB_LENGTH);
+  }
+
+  public String getAzureTableName() {
+    return tableName;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/samza/blob/c3b447ec/samza-azure/src/main/java/org/apache/samza/coordinator/AzureJobCoordinator.java
----------------------------------------------------------------------
diff --git a/samza-azure/src/main/java/org/apache/samza/coordinator/AzureJobCoordinator.java b/samza-azure/src/main/java/org/apache/samza/coordinator/AzureJobCoordinator.java
new file mode 100644
index 0000000..9438690
--- /dev/null
+++ b/samza-azure/src/main/java/org/apache/samza/coordinator/AzureJobCoordinator.java
@@ -0,0 +1,509 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.coordinator;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.samza.AzureClient;
+import org.apache.samza.config.AzureConfig;
+import org.apache.samza.coordinator.data.BarrierState;
+import org.apache.samza.config.ApplicationConfig;
+import org.apache.samza.config.Config;
+import org.apache.samza.config.ConfigException;
+import org.apache.samza.config.JobConfig;
+import org.apache.samza.config.TaskConfig;
+import org.apache.samza.container.TaskName;
+import org.apache.samza.container.grouper.stream.SystemStreamPartitionGrouper;
+import org.apache.samza.container.grouper.stream.SystemStreamPartitionGrouperFactory;
+import org.apache.samza.job.model.JobModel;
+import org.apache.samza.runtime.ProcessorIdGenerator;
+import org.apache.samza.coordinator.scheduler.HeartbeatScheduler;
+import org.apache.samza.coordinator.scheduler.JMVersionUpgradeScheduler;
+import org.apache.samza.coordinator.scheduler.LeaderBarrierCompleteScheduler;
+import org.apache.samza.coordinator.scheduler.LeaderLivenessCheckScheduler;
+import org.apache.samza.coordinator.scheduler.LivenessCheckScheduler;
+import org.apache.samza.coordinator.scheduler.RenewLeaseScheduler;
+import org.apache.samza.coordinator.scheduler.SchedulerStateChangeListener;
+import org.apache.samza.system.StreamMetadataCache;
+import org.apache.samza.system.SystemStream;
+import org.apache.samza.system.SystemStreamMetadata;
+import org.apache.samza.system.SystemStreamPartition;
+import org.apache.samza.util.BlobUtils;
+import org.apache.samza.util.ClassLoaderHelper;
+import org.apache.samza.util.LeaseBlobManager;
+import org.apache.samza.util.TableUtils;
+import org.apache.samza.util.Util;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.collection.JavaConverters;
+
+
+/**
+ * Class that provides coordination mechanism for Samza standalone in Azure.
+ * Handles processor lifecycle through Azure blob and table storage. Orchestrates leader election.
+ * The leader job coordinator generates partition mapping, writes shared data to the blob and manages rebalancing.
+ */
+public class AzureJobCoordinator implements JobCoordinator {
+  private static final Logger LOG = LoggerFactory.getLogger(AzureJobCoordinator.class);
+  private static final int METADATA_CACHE_TTL_MS = 5000;
+  private static final String INITIAL_STATE = "UNASSIGNED";
+  private final Consumer<String> errorHandler;
+  private final AzureLeaderElector azureLeaderElector;
+  private final BlobUtils leaderBlob;
+  private final TableUtils table;
+  private final Config config;
+  private final String processorId;
+  private final AzureClient client;
+  private final AtomicReference<String> currentJMVersion;
+  private final AtomicBoolean versionUpgradeDetected;
+  private final HeartbeatScheduler heartbeat;
+  private final JMVersionUpgradeScheduler versionUpgrade;
+  private final LeaderLivenessCheckScheduler leaderAlive;
+  private LivenessCheckScheduler liveness;
+  private RenewLeaseScheduler renewLease;
+  private LeaderBarrierCompleteScheduler leaderBarrierScheduler;
+  private StreamMetadataCache streamMetadataCache = null;
+  private JobCoordinatorListener coordinatorListener = null;
+  private JobModel jobModel = null;
+
+  /**
+   * Creates an instance of Azure job coordinator, along with references to Azure leader elector, Azure Blob and Azure Table.
+   * @param config User defined config
+   */
+  public AzureJobCoordinator(Config config) {
+    //TODO: Cleanup previous values in the table when barrier times out.
+    this.config = config;
+    processorId = createProcessorId(config);
+    currentJMVersion = new AtomicReference<>(INITIAL_STATE);
+    AzureConfig azureConfig = new AzureConfig(config);
+    client = new AzureClient(azureConfig.getAzureConnect());
+    leaderBlob = new BlobUtils(client, azureConfig.getAzureContainerName(), azureConfig.getAzureBlobName(), azureConfig.getAzureBlobLength());
+    errorHandler = (errorMsg) -> {
+      LOG.error(errorMsg);
+      stop();
+    };
+    table = new TableUtils(client, azureConfig.getAzureTableName(), INITIAL_STATE);
+    azureLeaderElector = new AzureLeaderElector(new LeaseBlobManager(leaderBlob.getBlob()));
+    azureLeaderElector.setLeaderElectorListener(new AzureLeaderElectorListener());
+    versionUpgradeDetected = new AtomicBoolean(false);
+    heartbeat = new HeartbeatScheduler(errorHandler, table, currentJMVersion, processorId);
+    versionUpgrade = new JMVersionUpgradeScheduler(errorHandler, leaderBlob, currentJMVersion, versionUpgradeDetected, processorId);
+    leaderAlive = new LeaderLivenessCheckScheduler(errorHandler, table, leaderBlob, currentJMVersion, INITIAL_STATE);
+    leaderBarrierScheduler = null;
+    renewLease = null;
+    liveness = null;
+  }
+
+  @Override
+  public void start() {
+
+    LOG.info("Starting Azure job coordinator.");
+    streamMetadataCache = StreamMetadataCache.apply(METADATA_CACHE_TTL_MS, config);
+    table.addProcessorEntity(INITIAL_STATE, processorId, false);
+
+    // Start scheduler for heartbeating
+    LOG.info("Starting scheduler for heartbeating.");
+    heartbeat.scheduleTask();
+
+    azureLeaderElector.tryBecomeLeader();
+
+    // Start scheduler to check for job model version upgrades
+    LOG.info("Starting scheduler to check for job model version upgrades.");
+    versionUpgrade.setStateChangeListener(createJMVersionUpgradeListener());
+    versionUpgrade.scheduleTask();
+
+    // Start scheduler to check for leader liveness
+    LOG.info("Starting scheduler to check for leader liveness.");
+    leaderAlive.setStateChangeListener(createLeaderLivenessListener());
+    leaderAlive.scheduleTask();
+  }
+
+  @Override
+  public void stop() {
+    LOG.info("Shutting down Azure job coordinator.");
+
+    if (coordinatorListener != null) {
+      coordinatorListener.onJobModelExpired();
+    }
+
+    // Resign leadership
+    if (azureLeaderElector.amILeader()) {
+      azureLeaderElector.resignLeadership();
+    }
+
+    // Shutdown all schedulers
+    shutdownSchedulers();
+
+    if (coordinatorListener != null) {
+      coordinatorListener.onCoordinatorStop();
+    }
+  }
+
+  @Override
+  public String getProcessorId() {
+    return processorId;
+  }
+
+  @Override
+  public void setListener(JobCoordinatorListener listener) {
+    this.coordinatorListener = listener;
+  }
+
+  @Override
+  public JobModel getJobModel() {
+    return jobModel;
+  }
+
+  private void shutdownSchedulers() {
+    if (renewLease != null) {
+      renewLease.shutdown();
+    }
+    if (leaderBarrierScheduler != null) {
+      leaderBarrierScheduler.shutdown();
+    }
+    if (liveness != null) {
+      liveness.shutdown();
+    }
+    heartbeat.shutdown();
+    leaderAlive.shutdown();
+    versionUpgrade.shutdown();
+  }
+
+  /**
+   * Creates a listener for LeaderBarrierCompleteScheduler class.
+   * Invoked by the leader when it detects that rebalancing has completed by polling the processor table.
+   * Updates the barrier state on the blob to denote that the barrier has completed.
+   * Cancels all future tasks scheduled by the LeaderBarrierComplete scheduler to check if barrier has completed.
+   * @return an instance of SchedulerStateChangeListener.
+   */
+  private SchedulerStateChangeListener createLeaderBarrierCompleteListener(String nextJMVersion, AtomicBoolean barrierTimeout) {
+    return () -> {
+      versionUpgradeDetected.getAndSet(false);
+      String state;
+      if (barrierTimeout.get()) {
+        LOG.error("Barrier timed out for version {}", nextJMVersion);
+        state = BarrierState.TIMEOUT.name() + " " + nextJMVersion;
+      } else {
+        LOG.info("Leader detected barrier completion.");
+        state = BarrierState.END.name() + " " + nextJMVersion;
+      }
+      if (!leaderBlob.publishBarrierState(state, azureLeaderElector.getLeaseId().get())) {
+        LOG.info("Leader failed to publish the job model {}. Stopping the processor with PID: .", jobModel, processorId);
+        stop();
+        table.deleteProcessorEntity(currentJMVersion.get(), processorId);
+      }
+      leaderBarrierScheduler.shutdown();
+    };
+  }
+
+  /**
+   * Creates a listener for LivenessCheckScheduler class.
+   * Invoked by the leader when the list of active processors in the system changes.
+   * @return an instance of SchedulerStateChangeListener.
+   */
+  private SchedulerStateChangeListener createLivenessListener(AtomicReference<List<String>> liveProcessors) {
+    return () -> {
+      LOG.info("Leader detected change in list of live processors.");
+      doOnProcessorChange(liveProcessors.get());
+    };
+  }
+
+  /**
+   * Creates a listener for JMVersionUpgradeScheduler class.
+   * Invoked when the processor detects a job model version upgrade on the blob.
+   * Stops listening for job model version upgrades until rebalancing achieved.
+   * @return an instance of SchedulerStateChangeListener.
+   */
+  private SchedulerStateChangeListener createJMVersionUpgradeListener() {
+    return () -> {
+      LOG.info("Job model version upgrade detected.");
+      versionUpgradeDetected.getAndSet(true);
+      onNewJobModelAvailable(leaderBlob.getJobModelVersion());
+    };
+  }
+
+  /**
+   * Creates a listener for LeaderLivenessCheckScheduler class.
+   * Invoked when an existing leader dies. Enables the JC to participate in leader election again.
+   * @return an instance of SchedulerStateChangeListener.
+   */
+  private SchedulerStateChangeListener createLeaderLivenessListener() {
+    return () -> {
+      LOG.info("Existing leader died.");
+      azureLeaderElector.tryBecomeLeader();
+    };
+  }
+
+  /**
+   * For each input stream specified in config, exactly determine its
+   * partitions, returning a set of SystemStreamPartitions containing them all.
+   */
+  private Set<SystemStreamPartition> getInputStreamPartitions() {
+    TaskConfig taskConfig = new TaskConfig(config);
+    scala.collection.immutable.Set<SystemStream> inputSystemStreams = taskConfig.getInputStreams();
+
+    // Get the set of partitions for each SystemStream from the stream metadata
+    Set<SystemStreamPartition>
+        sspSet = JavaConverters.mapAsJavaMapConverter(streamMetadataCache.getStreamMetadata(inputSystemStreams, true)).asJava()
+        .entrySet()
+        .stream()
+        .flatMap(this::mapSSMToSSP)
+        .collect(Collectors.toSet());
+
+    return sspSet;
+  }
+
+  private Stream<SystemStreamPartition> mapSSMToSSP(Map.Entry<SystemStream, SystemStreamMetadata> ssMs) {
+    return ssMs.getValue()
+        .getSystemStreamPartitionMetadata()
+        .keySet()
+        .stream()
+        .map(partition -> new SystemStreamPartition(ssMs.getKey(), partition));
+  }
+
+  /**
+   * Gets a SystemStreamPartitionGrouper object from the configuration.
+   */
+  private SystemStreamPartitionGrouper getSystemStreamPartitionGrouper() {
+    JobConfig jobConfig = new JobConfig(config);
+    String factoryString = jobConfig.getSystemStreamPartitionGrouperFactory();
+    SystemStreamPartitionGrouper grouper = Util.<SystemStreamPartitionGrouperFactory>getObj(factoryString).getSystemStreamPartitionGrouper(jobConfig);
+    return grouper;
+  }
+
+  private int getMaxNumTasks() {
+    // Do grouping to fetch TaskName to SSP mapping
+    Set<SystemStreamPartition> allSystemStreamPartitions = getInputStreamPartitions();
+    SystemStreamPartitionGrouper grouper = getSystemStreamPartitionGrouper();
+    Map<TaskName, Set<SystemStreamPartition>> groups = grouper.group(allSystemStreamPartitions);
+    LOG.info("SystemStreamPartitionGrouper " + grouper.toString() + " has grouped the SystemStreamPartitions into " + Integer.toString(groups.size()) +
+        " tasks with the following taskNames: {}", groups.keySet());
+    return groups.size();
+  }
+
+  /**
+   * Called only by the leader, either when the processor becomes the leader, or when the list of live processors changes.
+   * @param currentProcessorIds New updated list of processor IDs which caused the rebalancing.
+   */
+  private void doOnProcessorChange(List<String> currentProcessorIds) {
+    // if list of processors is empty - it means we are called from 'onBecomeLeader'
+
+    // Check if number of processors is greater than number of tasks
+    List<String> initialProcessorIds = new ArrayList<>(currentProcessorIds);
+    int numTasks = getMaxNumTasks();
+    if (currentProcessorIds.size() > numTasks) {
+      int iterator = 0;
+      while (currentProcessorIds.size() != numTasks) {
+        if (!currentProcessorIds.get(iterator).equals(processorId)) {
+          currentProcessorIds.remove(iterator);
+          iterator++;
+        }
+      }
+    }
+    LOG.info("currentProcessorIds = {}", currentProcessorIds);
+    LOG.info("initialProcessorIds = {}", initialProcessorIds);
+
+    String nextJMVersion;
+    String prevJMVersion = currentJMVersion.get();
+    JobModel prevJobModel = jobModel;
+    AtomicBoolean barrierTimeout = new AtomicBoolean(false);
+
+    if (currentProcessorIds.isEmpty()) {
+      if (currentJMVersion.get().equals(INITIAL_STATE)) {
+        nextJMVersion = "1";
+      } else {
+        nextJMVersion = Integer.toString(Integer.valueOf(prevJMVersion) + 1);
+      }
+      currentProcessorIds = new ArrayList<>(table.getActiveProcessorsList(currentJMVersion));
+      initialProcessorIds = currentProcessorIds;
+    } else {
+      //Check if previous barrier not reached, then previous barrier times out.
+      String blobJMV = leaderBlob.getJobModelVersion();
+      nextJMVersion = Integer.toString(Integer.valueOf(prevJMVersion) + 1);
+      if (blobJMV != null && Integer.valueOf(blobJMV) > Integer.valueOf(prevJMVersion)) {
+        prevJMVersion = blobJMV;
+        prevJobModel = leaderBlob.getJobModel();
+        nextJMVersion = Integer.toString(Integer.valueOf(blobJMV) + 1);
+        versionUpgradeDetected.getAndSet(false);
+        leaderBarrierScheduler.shutdown();
+        leaderBlob.publishBarrierState(BarrierState.TIMEOUT.name() + " " + blobJMV, azureLeaderElector.getLeaseId().get());
+      }
+    }
+
+    // Generate the new JobModel
+    JobModel newJobModel = JobModelManager.readJobModel(this.config, Collections.emptyMap(),
+        null, streamMetadataCache, currentProcessorIds);
+    LOG.info("pid=" + processorId + "Generated new Job Model. Version = " + nextJMVersion);
+
+    // Publish the new job model
+    boolean jmWrite = leaderBlob.publishJobModel(prevJobModel, newJobModel, prevJMVersion, nextJMVersion, azureLeaderElector.getLeaseId().get());
+    // Publish barrier state
+    boolean barrierWrite = leaderBlob.publishBarrierState(BarrierState.START.name() + " " + nextJMVersion, azureLeaderElector.getLeaseId().get());
+    barrierTimeout.set(false);
+    // Publish list of processors this function was called with
+    boolean processorWrite = leaderBlob.publishLiveProcessorList(initialProcessorIds, azureLeaderElector.getLeaseId().get());
+
+    //Shut down processor if write fails even after retries. These writes have an inherent retry policy.
+    if (!jmWrite || !barrierWrite || !processorWrite) {
+      LOG.info("Leader failed to publish the job model {}. Stopping the processor with PID: .", jobModel, processorId);
+      stop();
+      table.deleteProcessorEntity(currentJMVersion.get(), processorId);
+    }
+
+    LOG.info("pid=" + processorId + "Published new Job Model. Version = " + nextJMVersion);
+
+    // Start scheduler to check if barrier reached
+    long startTime = System.currentTimeMillis();
+    leaderBarrierScheduler = new LeaderBarrierCompleteScheduler(errorHandler, table, nextJMVersion, initialProcessorIds, startTime, barrierTimeout, currentJMVersion, processorId);
+    leaderBarrierScheduler.setStateChangeListener(createLeaderBarrierCompleteListener(nextJMVersion, barrierTimeout));
+    leaderBarrierScheduler.scheduleTask();
+  }
+
+  /**
+   * Called when the JC detects a job model version upgrade on the shared blob.
+   * @param nextJMVersion The new job model version after rebalancing.
+   */
+  private void onNewJobModelAvailable(final String nextJMVersion) {
+    LOG.info("pid=" + processorId + "new JobModel available with job model version {}", nextJMVersion);
+
+    //Get the new job model from blob
+    jobModel = leaderBlob.getJobModel();
+    LOG.info("pid=" + processorId + ": new JobModel available. ver=" + nextJMVersion + "; jm = " + jobModel);
+
+    if (!jobModel.getContainers().containsKey(processorId)) {
+      LOG.info("JobModel: {} does not contain the processorId: {}. Stopping the processor.", jobModel, processorId);
+      stop();
+      table.deleteProcessorEntity(currentJMVersion.get(), processorId);
+    } else {
+      //Stop current work
+      if (coordinatorListener != null) {
+        coordinatorListener.onJobModelExpired();
+      }
+      // Add entry with new job model version to the processor table
+      table.addProcessorEntity(nextJMVersion, processorId, azureLeaderElector.amILeader());
+
+      // Start polling blob to check if barrier reached
+      Random random = new Random();
+      String blobBarrierState = leaderBlob.getBarrierState();
+      while (true) {
+        if (blobBarrierState.equals(BarrierState.END.name() + " " + nextJMVersion)) {
+          LOG.info("Barrier completion detected by the worker for barrier version {}.", nextJMVersion);
+          versionUpgradeDetected.getAndSet(false);
+          onNewJobModelConfirmed(nextJMVersion);
+          break;
+        } else if (blobBarrierState.equals(BarrierState.TIMEOUT.name() + " " + nextJMVersion) ||
+            (Integer.valueOf(leaderBlob.getJobModelVersion()) > Integer.valueOf(nextJMVersion))) {
+          LOG.info("Barrier timed out for version number {}", nextJMVersion);
+          versionUpgradeDetected.getAndSet(false);
+          break;
+        } else {
+          try {
+            Thread.sleep(random.nextInt(5000));
+          } catch (InterruptedException e) {
+            Thread.interrupted();
+          }
+          LOG.info("Checking for barrier state on the blob again...");
+          blobBarrierState = leaderBlob.getBarrierState();
+        }
+      }
+    }
+  }
+
+  /**
+   * Called when the JC detects that the barrier has completed by checking the barrier state on the blob.
+   * @param nextJMVersion The new job model version after rebalancing.
+   */
+  private void onNewJobModelConfirmed(final String nextJMVersion) {
+    LOG.info("pid=" + processorId + "new version " + nextJMVersion + " of the job model got confirmed");
+
+    // Delete previous value
+    if (table.getEntity(currentJMVersion.get(), processorId) != null) {
+      table.deleteProcessorEntity(currentJMVersion.get(), processorId);
+    }
+    if (table.getEntity(INITIAL_STATE, processorId) != null) {
+      table.deleteProcessorEntity(INITIAL_STATE, processorId);
+    }
+
+    //Start heartbeating to new entry only when barrier reached.
+    //Changing the current job model version enables that since we are heartbeating to a row identified by the current job model version.
+    currentJMVersion.getAndSet(nextJMVersion);
+
+    //Start the container with the new model
+    if (coordinatorListener != null) {
+      coordinatorListener.onNewJobModel(processorId, jobModel);
+    }
+  }
+
+  private String createProcessorId(Config config) {
+    // TODO: This check to be removed after 0.13+
+    ApplicationConfig appConfig = new ApplicationConfig(config);
+    if (appConfig.getProcessorId() != null) {
+      return appConfig.getProcessorId();
+    } else if (StringUtils.isNotBlank(appConfig.getAppProcessorIdGeneratorClass())) {
+      ProcessorIdGenerator idGenerator =
+          ClassLoaderHelper.fromClassName(appConfig.getAppProcessorIdGeneratorClass(), ProcessorIdGenerator.class);
+      return idGenerator.generateProcessorId(config);
+    } else {
+      throw new ConfigException(String
+          .format("Expected either %s or %s to be configured", ApplicationConfig.PROCESSOR_ID,
+              ApplicationConfig.APP_PROCESSOR_ID_GENERATOR_CLASS));
+    }
+  }
+
+  public class AzureLeaderElectorListener implements LeaderElectorListener {
+    /**
+     * Keep renewing the lease and do the required tasks as a leader.
+     */
+    @Override
+    public void onBecomingLeader() {
+      // Update table to denote that it is a leader.
+      table.updateIsLeader(currentJMVersion.get(), processorId, true);
+
+      // Schedule a task to renew the lease after a fixed time interval
+      LOG.info("Starting scheduler to keep renewing lease held by the leader.");
+      renewLease = new RenewLeaseScheduler((errorMsg) -> {
+          LOG.error(errorMsg);
+          table.updateIsLeader(currentJMVersion.get(), processorId, false);
+          azureLeaderElector.resignLeadership();
+          renewLease.shutdown();
+          liveness.shutdown();
+        }, azureLeaderElector.getLeaseBlobManager(), azureLeaderElector.getLeaseId());
+      renewLease.scheduleTask();
+
+      doOnProcessorChange(new ArrayList<>());
+
+      // Start scheduler to check for change in list of live processors
+      LOG.info("Starting scheduler to check for change in list of live processors in the system.");
+      liveness = new LivenessCheckScheduler(errorHandler, table, leaderBlob, currentJMVersion, processorId);
+      liveness.setStateChangeListener(createLivenessListener(liveness.getLiveProcessors()));
+      liveness.scheduleTask();
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/samza/blob/c3b447ec/samza-azure/src/main/java/org/apache/samza/coordinator/AzureLeaderElector.java
----------------------------------------------------------------------
diff --git a/samza-azure/src/main/java/org/apache/samza/coordinator/AzureLeaderElector.java b/samza-azure/src/main/java/org/apache/samza/coordinator/AzureLeaderElector.java
new file mode 100644
index 0000000..c93f1d0
--- /dev/null
+++ b/samza-azure/src/main/java/org/apache/samza/coordinator/AzureLeaderElector.java
@@ -0,0 +1,109 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.coordinator;
+
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.samza.AzureException;
+import org.apache.samza.util.LeaseBlobManager;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Class to facilitate leader election in Azure.
+ * The processor that acquires the lease on the blob becomes the leader.
+ * The lease ID is null initially. It is generated by Azure when the processor acquires the lease, and updated accordingly.
+ * Every processor requires a valid active lease ID in order to perform successful write and delete operations on the blob.
+ * Read operations from the blob are not dependent on the lease ID.
+ */
+public class AzureLeaderElector implements LeaderElector {
+
+  private static final Logger LOG = LoggerFactory.getLogger(AzureLeaderElector.class);
+  private static final int LEASE_TIME_IN_SEC = 60;
+  private final LeaseBlobManager leaseBlobManager;
+  private LeaderElectorListener leaderElectorListener = null;
+  private final AtomicReference<String> leaseId;
+  private final AtomicBoolean isLeader;
+
+  public AzureLeaderElector(LeaseBlobManager leaseBlobManager) {
+    this.isLeader = new AtomicBoolean(false);
+    this.leaseBlobManager = leaseBlobManager;
+    this.leaseId = new AtomicReference<>(null);
+  }
+
+  @Override
+  public void setLeaderElectorListener(LeaderElectorListener listener) {
+    this.leaderElectorListener = listener;
+  }
+
+  /**
+   * Tries to become the leader by acquiring a lease on the blob.
+   * The acquireLease operation has a retry policy where upon failure, the operation is tried 3 times at 5 second intervals.
+   * Invokes the listener on becoming the leader.
+   * @throws AzureException If a Azure storage service error occurred. This includes the case where the blob you're trying to lease does not exist.
+   */
+  @Override
+  public void tryBecomeLeader() throws AzureException {
+    leaseId.getAndSet(leaseBlobManager.acquireLease(LEASE_TIME_IN_SEC, leaseId.get()));
+    if (leaseId.get() != null) {
+      LOG.info("Became leader with lease ID {}.", leaseId.get());
+      isLeader.set(true);
+      if (leaderElectorListener != null) {
+        leaderElectorListener.onBecomingLeader();
+      }
+    } else {
+      LOG.info("Unable to become the leader. Continuing as a worker.");
+    }
+  }
+
+  /**
+   * Releases the lease in order to resign leadership. It also stops all schedulers scheduled by the leader.
+   * The releaseLease operation has a retry policy where upon failure, the operation is tried 3 times at 5 second intervals.
+   */
+  @Override
+  public void resignLeadership() {
+    if (isLeader.get()) {
+      leaseBlobManager.releaseLease(leaseId.get());
+      isLeader.set(false);
+      LOG.info("Resigning leadership with lease ID {}", leaseId.get());
+      leaseId.getAndSet(null);
+    } else {
+      LOG.info("Can't release the lease because it is not the leader and does not hold an active lease.");
+    }
+  }
+
+  /**
+   * Checks whether it's a leader
+   * @return true if it is the leader, false otherwise
+   */
+  @Override
+  public boolean amILeader() {
+    return isLeader.get();
+  }
+
+  public AtomicReference<String> getLeaseId() {
+    return leaseId;
+  }
+
+  public LeaseBlobManager getLeaseBlobManager() {
+    return this.leaseBlobManager;
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/c3b447ec/samza-azure/src/main/java/org/apache/samza/coordinator/data/BarrierState.java
----------------------------------------------------------------------
diff --git a/samza-azure/src/main/java/org/apache/samza/coordinator/data/BarrierState.java b/samza-azure/src/main/java/org/apache/samza/coordinator/data/BarrierState.java
new file mode 100644
index 0000000..1c144de
--- /dev/null
+++ b/samza-azure/src/main/java/org/apache/samza/coordinator/data/BarrierState.java
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.samza.coordinator.data;
+
+/**
+ * Enum depicting different barrier states.
+ */
+public enum BarrierState {
+  START, END, TIMEOUT
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/c3b447ec/samza-azure/src/main/java/org/apache/samza/coordinator/data/JobModelBundle.java
----------------------------------------------------------------------
diff --git a/samza-azure/src/main/java/org/apache/samza/coordinator/data/JobModelBundle.java b/samza-azure/src/main/java/org/apache/samza/coordinator/data/JobModelBundle.java
new file mode 100644
index 0000000..d05e0a5
--- /dev/null
+++ b/samza-azure/src/main/java/org/apache/samza/coordinator/data/JobModelBundle.java
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.samza.coordinator.data;
+
+import org.apache.samza.job.model.JobModel;
+
+
+/**
+ * Bundle class for current and previous - job model and job model version.
+ * Used for publishing updated data to the blob in one go.
+ */
+public class JobModelBundle {
+
+  private JobModel prevJobModel;
+  private JobModel currJobModel;
+  private String prevJobModelVersion;
+  private String currJobModelVersion;
+
+  public JobModelBundle() {}
+
+  public JobModelBundle(JobModel prevJM, JobModel currJM, String prevJMV, String currJMV) {
+    prevJobModel = prevJM;
+    currJobModel = currJM;
+    prevJobModelVersion = prevJMV;
+    currJobModelVersion = currJMV;
+  }
+
+  public JobModel getCurrJobModel() {
+    return currJobModel;
+  }
+
+  public JobModel getPrevJobModel() {
+    return prevJobModel;
+  }
+
+  public String getCurrJobModelVersion() {
+    return currJobModelVersion;
+  }
+
+  public String getPrevJobModelVersion() {
+    return prevJobModelVersion;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/c3b447ec/samza-azure/src/main/java/org/apache/samza/coordinator/data/ProcessorEntity.java
----------------------------------------------------------------------
diff --git a/samza-azure/src/main/java/org/apache/samza/coordinator/data/ProcessorEntity.java b/samza-azure/src/main/java/org/apache/samza/coordinator/data/ProcessorEntity.java
new file mode 100644
index 0000000..9323bde
--- /dev/null
+++ b/samza-azure/src/main/java/org/apache/samza/coordinator/data/ProcessorEntity.java
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.samza.coordinator.data;
+
+import com.microsoft.azure.storage.table.TableServiceEntity;
+import java.util.Random;
+
+
+/**
+ * Table schema for Azure processor table.
+ * Denotes a row in the table with PARTITION KEY = Job Model Version and ROW KEY = Processor ID
+ * Other fields include integer liveness value to which each processor heartbeats,
+ * and boolean isLeader value which denotes whether the processor is a leader or not.
+ */
+public class ProcessorEntity extends TableServiceEntity {
+  private Random rand = new Random();
+  private int liveness;
+  private boolean isLeader;
+
+  public ProcessorEntity() {}
+
+  public ProcessorEntity(String jobModelVersion, String processorId) {
+    this.partitionKey = jobModelVersion;
+    this.rowKey = processorId;
+    this.isLeader = false;
+    this.liveness = rand.nextInt(10000) + 2;
+  }
+
+  /**
+   * Updates heartbeat by updating the liveness value in the table.
+   * Sets the liveness field to a random integer value in order to update the last modified since timestamp of the row in the table.
+   * This asserts to the leader that the processor is alive.
+   */
+  public void updateLiveness() {
+    liveness = rand.nextInt(10000) + 2;
+  }
+
+  public void setIsLeader(boolean leader) {
+    isLeader = leader;
+  }
+
+  public boolean getIsLeader() {
+    return isLeader;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/samza/blob/c3b447ec/samza-azure/src/main/java/org/apache/samza/coordinator/scheduler/HeartbeatScheduler.java
----------------------------------------------------------------------
diff --git a/samza-azure/src/main/java/org/apache/samza/coordinator/scheduler/HeartbeatScheduler.java b/samza-azure/src/main/java/org/apache/samza/coordinator/scheduler/HeartbeatScheduler.java
new file mode 100644
index 0000000..2abb380
--- /dev/null
+++ b/samza-azure/src/main/java/org/apache/samza/coordinator/scheduler/HeartbeatScheduler.java
@@ -0,0 +1,81 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.samza.coordinator.scheduler;
+
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Consumer;
+import org.apache.samza.util.TableUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Scheduler invoked by each processor for heartbeating to a row of the table.
+ * Heartbeats every 5 seconds.
+ * The row is determined by the job model version and processor id passed to the scheduler.
+ * All time units are in SECONDS.
+ */
+public class HeartbeatScheduler implements TaskScheduler {
+
+  private static final Logger LOG = LoggerFactory.getLogger(HeartbeatScheduler.class);
+  private static final long HEARTBEAT_DELAY_SEC = 5;
+  private static final ThreadFactory PROCESSOR_THREAD_FACTORY =
+      new ThreadFactoryBuilder().setNameFormat("HeartbeatScheduler-%d").build();
+  private final ScheduledExecutorService scheduler = Executors.newSingleThreadScheduledExecutor(PROCESSOR_THREAD_FACTORY);
+  private final String processorId;
+  private final TableUtils table;
+  private final AtomicReference<String> currentJMVersion;
+  private final Consumer<String> errorHandler;
+
+  public HeartbeatScheduler(Consumer<String> errorHandler, TableUtils table, AtomicReference<String> currentJMVersion, final String pid) {
+    this.table = table;
+    this.currentJMVersion = currentJMVersion;
+    processorId = pid;
+    this.errorHandler = errorHandler;
+  }
+
+  @Override
+  public ScheduledFuture scheduleTask() {
+    return scheduler.scheduleWithFixedDelay(() -> {
+        try {
+          String currJVM = currentJMVersion.get();
+          LOG.info("Updating heartbeat for processor ID: " + processorId + " and job model version: " + currJVM);
+          table.updateHeartbeat(currJVM, processorId);
+        } catch (Exception e) {
+          errorHandler.accept("Exception in Heartbeat Scheduler. Stopping the processor...");
+        }
+      }, HEARTBEAT_DELAY_SEC, HEARTBEAT_DELAY_SEC, TimeUnit.SECONDS);
+  }
+
+  @Override
+  public void setStateChangeListener(SchedulerStateChangeListener listener) {}
+
+  @Override
+  public void shutdown() {
+    LOG.info("Shutting down HeartbeatScheduler");
+    scheduler.shutdownNow();
+  }
+}
\ No newline at end of file


[24/26] samza git commit: SAMZA-1413: Config for CoordinationUtilsFactory class name.

Posted by xi...@apache.org.
SAMZA-1413: Config for CoordinationUtilsFactory class name.

Author: Boris Shkolnik <bo...@apache.org>
Author: Boris Shkolnik <bs...@bshkolni-ld1.linkedin.biz>

Reviewers: Navina Ramesh <na...@apache.org>, Fred Ji <fj...@linkedin.com>

Closes #290 from sborya/configUtilsFactoryClassName


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

Branch: refs/heads/0.14.0
Commit: fb39a5142283df4dd2c097063d8d85c92a564392
Parents: dd07e07
Author: Boris Shkolnik <bo...@apache.org>
Authored: Wed Aug 30 16:56:28 2017 -0700
Committer: navina <na...@apache.org>
Committed: Wed Aug 30 16:56:28 2017 -0700

----------------------------------------------------------------------
 .../versioned/jobs/configuration-table.html     | 14 ++++-
 .../coordinator/AzureCoordinationUtils.java     |  1 -
 .../samza/config/JobCoordinatorConfig.java      | 20 +++----
 .../samza/config/TestJobCoordinatorConfig.java  | 58 ++++++++++++++++++++
 4 files changed, 80 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/samza/blob/fb39a514/docs/learn/documentation/versioned/jobs/configuration-table.html
----------------------------------------------------------------------
diff --git a/docs/learn/documentation/versioned/jobs/configuration-table.html b/docs/learn/documentation/versioned/jobs/configuration-table.html
index dc1df30..9b4e279 100644
--- a/docs/learn/documentation/versioned/jobs/configuration-table.html
+++ b/docs/learn/documentation/versioned/jobs/configuration-table.html
@@ -429,7 +429,19 @@
                         Required only for non-cluster-managed applications. Please see the required value for <a href=#task-name-grouper-factory>task-name-grouper-factory </a>
                     </td>
                 </tr>
-
+                <tr>
+                    <td class="property" id="job.coordination.utils.factory">job.coordination.utils.factory</td>
+                    <td class="default">org.apache.samza.zk.ZkCoordinationUtilsFactory</td>
+                    <td class="description">
+                        Class to use to create CoordinationUtils. Currently available values are:
+                        <dl>
+                            <dt><code>org.apache.samza.zk.ZkCoordinationUtilsFactory</code></dt>
+                            <dd>ZooKeeper based coordination utils.</dd>
+                            <dt><code>org.apache.samza.coordinator.AzureCoordinationUtilsFactory</code></dt>
+                            <dd>Azure based coordination utils.</dd>
+                        These coordination utils are currently used for intermediate stream creation.
+                    </td>
+                </tr>
                 <tr>
                                               <!-- change link to StandAlone design/tutorial doc. SAMZA-1299 -->
                 <th colspan="3" class="section" id="ZkBasedJobCoordination"><a href="../index.html">Zookeeper-based job configuration</a></th>

http://git-wip-us.apache.org/repos/asf/samza/blob/fb39a514/samza-azure/src/main/java/org/apache/samza/coordinator/AzureCoordinationUtils.java
----------------------------------------------------------------------
diff --git a/samza-azure/src/main/java/org/apache/samza/coordinator/AzureCoordinationUtils.java b/samza-azure/src/main/java/org/apache/samza/coordinator/AzureCoordinationUtils.java
index 2a42514..b689f3e 100644
--- a/samza-azure/src/main/java/org/apache/samza/coordinator/AzureCoordinationUtils.java
+++ b/samza-azure/src/main/java/org/apache/samza/coordinator/AzureCoordinationUtils.java
@@ -54,6 +54,5 @@ public class AzureCoordinationUtils implements CoordinationUtils {
 
   @Override
   public void close() {
-    
   }
 }

http://git-wip-us.apache.org/repos/asf/samza/blob/fb39a514/samza-core/src/main/java/org/apache/samza/config/JobCoordinatorConfig.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/config/JobCoordinatorConfig.java b/samza-core/src/main/java/org/apache/samza/config/JobCoordinatorConfig.java
index a04038a..700a107 100644
--- a/samza-core/src/main/java/org/apache/samza/config/JobCoordinatorConfig.java
+++ b/samza-core/src/main/java/org/apache/samza/config/JobCoordinatorConfig.java
@@ -22,8 +22,6 @@ package org.apache.samza.config;
 import com.google.common.base.Strings;
 import org.apache.samza.SamzaException;
 import org.apache.samza.zk.ZkCoordinationUtilsFactory;
-import org.apache.samza.zk.ZkJobCoordinatorFactory;
-
 
 public class JobCoordinatorConfig extends MapConfig {
   public static final String JOB_COORDINATOR_FACTORY = "job.coordinator.factory";
@@ -35,20 +33,20 @@ public class JobCoordinatorConfig extends MapConfig {
   }
 
   public String getJobCoordinationUtilsFactoryClassName() {
-    String jobCoordinatorFactoryClassName = get(JobCoordinatorConfig.JOB_COORDINATOR_FACTORY, "");
-
-    String className = get(JOB_COORDINATION_UTILS_FACTORY, "");
+    String className = get(JOB_COORDINATION_UTILS_FACTORY, DEFAULT_COORDINATION_UTILS_FACTORY);
 
-    if (!Strings.isNullOrEmpty(className)) {
-      return className;
+    if (Strings.isNullOrEmpty(className)) {
+      throw new SamzaException("Empty config for " + JOB_COORDINATION_UTILS_FACTORY + " = " + className);
     }
 
-    // TODO: we will need a better way to package the configs with application runner
-    if (ZkJobCoordinatorFactory.class.getName().equals(jobCoordinatorFactoryClassName)) {
-      return DEFAULT_COORDINATION_UTILS_FACTORY;
+    try {
+      Class.forName(className);
+    } catch (ClassNotFoundException e) {
+      throw new SamzaException(
+          "Failed to validate config value for " + JOB_COORDINATION_UTILS_FACTORY + " = " + className, e);
     }
 
-    throw new SamzaException("Cannot determine which CoordinationUtilsFactory to load");
+    return className;
   }
 
   public String getJobCoordinatorFactoryClassName() {

http://git-wip-us.apache.org/repos/asf/samza/blob/fb39a514/samza-core/src/test/scala/org/apache/samza/config/TestJobCoordinatorConfig.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/scala/org/apache/samza/config/TestJobCoordinatorConfig.java b/samza-core/src/test/scala/org/apache/samza/config/TestJobCoordinatorConfig.java
new file mode 100644
index 0000000..2ef92b5
--- /dev/null
+++ b/samza-core/src/test/scala/org/apache/samza/config/TestJobCoordinatorConfig.java
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.samza.config;
+
+import java.util.HashMap;
+import java.util.Map;
+import junit.framework.Assert;
+import org.apache.samza.SamzaException;
+import org.apache.samza.zk.ZkCoordinationUtilsFactory;
+import org.junit.Test;
+
+
+public class TestJobCoordinatorConfig {
+
+  private final static String NONEXISTING_FACTORY_CLASS = "AnotherFactory";
+  private final static String ANOTHER_FACTORY_CLASS = TestJobCoordinatorConfig.class.getName(); // any valid name
+
+  @Test
+  public void testJobCoordinationUtilsFactoryConfig() {
+
+    Map<String, String> map = new HashMap<>();
+    JobCoordinatorConfig jConfig = new JobCoordinatorConfig(new MapConfig(map));
+
+    // test default value
+    Assert.assertEquals(ZkCoordinationUtilsFactory.class.getName(), jConfig.getJobCoordinationUtilsFactoryClassName());
+
+    map.put(JobCoordinatorConfig.JOB_COORDINATION_UTILS_FACTORY, ANOTHER_FACTORY_CLASS);
+    jConfig = new JobCoordinatorConfig(new MapConfig(map));
+    Assert.assertEquals(ANOTHER_FACTORY_CLASS, jConfig.getJobCoordinationUtilsFactoryClassName());
+
+    // failure case
+    map.put(JobCoordinatorConfig.JOB_COORDINATION_UTILS_FACTORY, NONEXISTING_FACTORY_CLASS);
+    jConfig = new JobCoordinatorConfig(new MapConfig(map));
+    try {
+      jConfig.getJobCoordinationUtilsFactoryClassName();
+      Assert.fail("Failed to validate loading of fake class: " + NONEXISTING_FACTORY_CLASS);
+    } catch (SamzaException e) {
+      // expected
+    }
+  }
+}


[16/26] samza git commit: SAMZA-1375: Implement Lock for Azure using Lease Blob

Posted by xi...@apache.org.
SAMZA-1375: Implement Lock for Azure using Lease Blob

Author: PawasChhokra <pa...@gmail.com>
Author: PawasChhokra <Jaimatadi1$>

Reviewers: Navina Ramesh <na...@apache.org>

Closes #280 from PawasChhokra/AzureDistributedLock


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

Branch: refs/heads/0.14.0
Commit: 67f7214aace6f9f98492b53f0a10d703aa8706c1
Parents: c3b447e
Author: Pawas Chhokra <pa...@gmail.com>
Authored: Fri Aug 18 17:49:52 2017 -0700
Committer: navina <na...@apache.org>
Committed: Fri Aug 18 17:49:52 2017 -0700

----------------------------------------------------------------------
 .../AzureCoordinationServiceFactory.java        |  30 ++++++
 .../coordinator/AzureCoordinationUtils.java     |  57 +++++++++++
 .../coordinator/AzureJobCoordinatorFactory.java |  29 ++++++
 .../org/apache/samza/coordinator/AzureLock.java | 100 +++++++++++++++++++
 .../samza/coordinator/DistributedLock.java      |  39 ++++++++
 5 files changed, 255 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/samza/blob/67f7214a/samza-azure/src/main/java/org/apache/samza/coordinator/AzureCoordinationServiceFactory.java
----------------------------------------------------------------------
diff --git a/samza-azure/src/main/java/org/apache/samza/coordinator/AzureCoordinationServiceFactory.java b/samza-azure/src/main/java/org/apache/samza/coordinator/AzureCoordinationServiceFactory.java
new file mode 100644
index 0000000..8016fbe
--- /dev/null
+++ b/samza-azure/src/main/java/org/apache/samza/coordinator/AzureCoordinationServiceFactory.java
@@ -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.coordinator;
+
+import org.apache.samza.config.Config;
+
+public class AzureCoordinationServiceFactory implements CoordinationServiceFactory {
+
+  @Override
+  public CoordinationUtils getCoordinationService(String groupId, String participantId, Config config) {
+    return new AzureCoordinationUtils(config);
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/67f7214a/samza-azure/src/main/java/org/apache/samza/coordinator/AzureCoordinationUtils.java
----------------------------------------------------------------------
diff --git a/samza-azure/src/main/java/org/apache/samza/coordinator/AzureCoordinationUtils.java b/samza-azure/src/main/java/org/apache/samza/coordinator/AzureCoordinationUtils.java
new file mode 100644
index 0000000..f76dde3
--- /dev/null
+++ b/samza-azure/src/main/java/org/apache/samza/coordinator/AzureCoordinationUtils.java
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.coordinator;
+
+import org.apache.samza.AzureClient;
+import org.apache.samza.config.AzureConfig;
+import org.apache.samza.config.Config;
+import org.apache.samza.util.BlobUtils;
+
+
+public class AzureCoordinationUtils implements CoordinationUtils {
+
+  private final AzureConfig azureConfig;
+  private final AzureClient client;
+
+  public AzureCoordinationUtils(Config config) {
+    azureConfig = new AzureConfig(config);
+    this.client = new AzureClient(azureConfig.getAzureConnect());
+  }
+
+  @Override
+  public void reset() {}
+
+
+  @Override
+  public LeaderElector getLeaderElector() throws UnsupportedOperationException {
+    return null;
+  }
+
+  @Override
+  public Latch getLatch(int size, String latchId) throws UnsupportedOperationException {
+    return null;
+  }
+
+  public DistributedLock getLock(String initLockName) {
+    BlobUtils blob = new BlobUtils(client, azureConfig.getAzureContainerName(),
+        azureConfig.getAzureBlobName() + initLockName, azureConfig.getAzureBlobLength());
+    return new AzureLock(blob);
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/67f7214a/samza-azure/src/main/java/org/apache/samza/coordinator/AzureJobCoordinatorFactory.java
----------------------------------------------------------------------
diff --git a/samza-azure/src/main/java/org/apache/samza/coordinator/AzureJobCoordinatorFactory.java b/samza-azure/src/main/java/org/apache/samza/coordinator/AzureJobCoordinatorFactory.java
new file mode 100644
index 0000000..8b3d357
--- /dev/null
+++ b/samza-azure/src/main/java/org/apache/samza/coordinator/AzureJobCoordinatorFactory.java
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.coordinator;
+
+import org.apache.samza.config.Config;
+
+public class AzureJobCoordinatorFactory implements JobCoordinatorFactory {
+  @Override
+  public JobCoordinator getJobCoordinator(Config config) {
+    return new AzureJobCoordinator(config);
+  }
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/67f7214a/samza-azure/src/main/java/org/apache/samza/coordinator/AzureLock.java
----------------------------------------------------------------------
diff --git a/samza-azure/src/main/java/org/apache/samza/coordinator/AzureLock.java b/samza-azure/src/main/java/org/apache/samza/coordinator/AzureLock.java
new file mode 100644
index 0000000..0ef1b83
--- /dev/null
+++ b/samza-azure/src/main/java/org/apache/samza/coordinator/AzureLock.java
@@ -0,0 +1,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.
+ */
+
+package org.apache.samza.coordinator;
+
+import java.util.Random;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.samza.AzureException;
+import org.apache.samza.util.BlobUtils;
+import org.apache.samza.util.LeaseBlobManager;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Distributed lock primitive for Azure.
+ */
+public class AzureLock implements DistributedLock {
+
+  private static final Logger LOG = LoggerFactory.getLogger(AzureLock.class);
+  private static final int LEASE_TIME_IN_SEC = 60;
+  private AtomicBoolean hasLock;
+  private AtomicReference<String> leaseId;
+  private final LeaseBlobManager leaseBlobManager;
+
+  public AzureLock(BlobUtils blobUtils) {
+    this.hasLock = new AtomicBoolean(false);
+    leaseBlobManager = new LeaseBlobManager(blobUtils.getBlob());
+    this.leaseId = new AtomicReference<>(null);
+  }
+
+  /**
+   * Tries to acquire a lock in order to create intermediate streams. On failure to acquire lock, it keeps trying until the lock times out.
+   * The lock is acquired when the blob is leased successfully.
+   * @param timeout Duration after which timeout occurs.
+   * @param unit Time Unit of the timeout defined above.
+   * @return true if the lock was acquired successfully, false if lock acquire operation is unsuccessful even after subsequent tries within the timeout range.
+   */
+  @Override
+  public boolean lock(long timeout, TimeUnit unit) {
+    //Start timer for timeout
+    long startTime = System.currentTimeMillis();
+    long lockTimeout = TimeUnit.MILLISECONDS.convert(timeout, unit);
+    Random random = new Random();
+
+    while ((System.currentTimeMillis() - startTime) < lockTimeout) {
+      try {
+        leaseId.getAndSet(leaseBlobManager.acquireLease(LEASE_TIME_IN_SEC, leaseId.get()));
+      } catch (AzureException e) {
+        return false;
+      }
+      if (leaseId.get() != null) {
+        LOG.info("Acquired lock!");
+        hasLock.getAndSet(true);
+        return true;
+      } else {
+        try {
+          Thread.sleep(random.nextInt(1000));
+        } catch (InterruptedException e) {
+          Thread.interrupted();
+        }
+        LOG.info("Trying to acquire lock again...");
+      }
+    }
+    return false;
+  }
+
+  /**
+   * Unlocks, by releasing the lease on the blob.
+   */
+  @Override
+  public void unlock() {
+    boolean status = leaseBlobManager.releaseLease(leaseId.get());
+    if (status) {
+      LOG.info("Unlocked successfully.");
+      hasLock.getAndSet(false);
+      leaseId.getAndSet(null);
+    } else {
+      LOG.info("Unable to unlock.");
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/samza/blob/67f7214a/samza-azure/src/main/java/org/apache/samza/coordinator/DistributedLock.java
----------------------------------------------------------------------
diff --git a/samza-azure/src/main/java/org/apache/samza/coordinator/DistributedLock.java b/samza-azure/src/main/java/org/apache/samza/coordinator/DistributedLock.java
new file mode 100644
index 0000000..6972cd9
--- /dev/null
+++ b/samza-azure/src/main/java/org/apache/samza/coordinator/DistributedLock.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.samza.coordinator;
+
+import java.util.concurrent.TimeUnit;
+
+
+public interface DistributedLock {
+
+  /**
+   * Tries to acquire the lock
+   * @param timeout Duration of lock acquiring timeout.
+   * @param unit Time Unit of the timeout defined above.
+   * @return true if lock is acquired successfully, false if it times out.
+   */
+  boolean lock(long timeout, TimeUnit unit);
+
+  /**
+   * Releases the lock
+   */
+  void unlock();
+}
\ No newline at end of file


[18/26] samza git commit: Fix broken link for KeyValueStore in Samza's feature preview web-page

Posted by xi...@apache.org.
Fix broken link for KeyValueStore in Samza's feature preview web-page


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

Branch: refs/heads/0.14.0
Commit: 2a4ccfdd72513a8a3809c3d01e8fd0d9c23dd1e1
Parents: 80d82b6
Author: Jagadish <ja...@apache.org>
Authored: Wed Aug 23 21:47:50 2017 -0700
Committer: Jagadish <ja...@apache.org>
Committed: Wed Aug 23 21:47:50 2017 -0700

----------------------------------------------------------------------
 docs/startup/preview/index.md | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/samza/blob/2a4ccfdd/docs/startup/preview/index.md
----------------------------------------------------------------------
diff --git a/docs/startup/preview/index.md b/docs/startup/preview/index.md
index 619923d..b1ca0ac 100644
--- a/docs/startup/preview/index.md
+++ b/docs/startup/preview/index.md
@@ -116,7 +116,7 @@ For more details on running Samza in embedded mode, take a look at the [flexible
 
 #### IV. Processor
 
-The lowest execution unit of a Samza application is the processor. It reads the configs generated from the [ApplicationRunner](/learn/documentation/{{site.version}}/api/javadocs/org/apache/samza/runtime/ApplicationRunner.html) and processes the input stream partitions assigned by the JobCoordinator. It can access local state using a [KeyValueStore](/learn/documentation/{{site.version}}/api/javadocs/org/apache/samza/storage/KeyValueStore.html) implementation (e.g. RocksDB or in-memory) and remote state (e.g. REST service) using multithreading.
+The lowest execution unit of a Samza application is the processor. It reads the configs generated from the [ApplicationRunner](/learn/documentation/{{site.version}}/api/javadocs/org/apache/samza/runtime/ApplicationRunner.html) and processes the input stream partitions assigned by the JobCoordinator. It can access local state using a [KeyValueStore](/learn/documentation/{{site.version}}/api/javadocs/org/apache/samza/storage/kv/KeyValueStore.html) implementation (e.g. RocksDB or in-memory) and remote state (e.g. REST service) using multithreading.
 
 ---
 


[14/26] samza git commit: SAMZA-1378: Introduce and Implement Scheduler Interface for Polling in Azure

Posted by xi...@apache.org.
http://git-wip-us.apache.org/repos/asf/samza/blob/c3b447ec/samza-azure/src/main/java/org/apache/samza/coordinator/scheduler/JMVersionUpgradeScheduler.java
----------------------------------------------------------------------
diff --git a/samza-azure/src/main/java/org/apache/samza/coordinator/scheduler/JMVersionUpgradeScheduler.java b/samza-azure/src/main/java/org/apache/samza/coordinator/scheduler/JMVersionUpgradeScheduler.java
new file mode 100644
index 0000000..ded014f
--- /dev/null
+++ b/samza-azure/src/main/java/org/apache/samza/coordinator/scheduler/JMVersionUpgradeScheduler.java
@@ -0,0 +1,99 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.samza.coordinator.scheduler;
+
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import java.util.List;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Consumer;
+import org.apache.samza.coordinator.data.BarrierState;
+import org.apache.samza.util.BlobUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Scheduler invoked by each processor to check for job model version upgrades on the blob.
+ * Checks every 5 seconds.
+ * The processor polls the leader blob in order to track this.
+ * All time units are in SECONDS.
+ */
+public class JMVersionUpgradeScheduler implements TaskScheduler {
+  private static final Logger LOG = LoggerFactory.getLogger(JMVersionUpgradeScheduler.class);
+  private static final long JMV_UPGRADE_DELAY_SEC = 5;
+  private static final ThreadFactory
+      PROCESSOR_THREAD_FACTORY = new ThreadFactoryBuilder().setNameFormat("JMVersionUpgradeScheduler-%d").build();
+  private final ScheduledExecutorService scheduler = Executors.newSingleThreadScheduledExecutor(PROCESSOR_THREAD_FACTORY);
+  private final BlobUtils blob;
+  private final AtomicReference<String> currentJMVersion;
+  private final AtomicBoolean versionUpgradeDetected;
+  private final String processorId;
+  private final Consumer<String> errorHandler;
+  private SchedulerStateChangeListener listener = null;
+
+  public JMVersionUpgradeScheduler(Consumer<String> errorHandler, BlobUtils blob,
+      AtomicReference<String> currentJMVersion, AtomicBoolean versionUpgradeDetected, String processorId) {
+    this.blob = blob;
+    this.currentJMVersion = currentJMVersion;
+    this.versionUpgradeDetected = versionUpgradeDetected;
+    this.processorId = processorId;
+    this.errorHandler = errorHandler;
+  }
+
+  @Override
+  public ScheduledFuture scheduleTask() {
+    return scheduler.scheduleWithFixedDelay(() -> {
+        try {
+          LOG.info("Checking for job model version upgrade");
+          // Read job model version from the blob.
+          String blobJMV = blob.getJobModelVersion();
+          LOG.info("Job Model Version seen on the blob: {}", blobJMV);
+          String blobBarrierState = blob.getBarrierState();
+          String currentJMV = currentJMVersion.get();
+          LOG.info("Current Job Model Version that the job coordinator is working on: {}", currentJMV);
+          String expectedBarrierState = BarrierState.START.toString() + " " + blobJMV;
+          List<String> processorList = blob.getLiveProcessorList();
+          // Check if the job model version on the blob is consistent with the job model version that the processor is operating on.
+          if (processorList != null && processorList.contains(processorId) && !currentJMV.equals(blobJMV) && blobBarrierState.equals(expectedBarrierState) && !versionUpgradeDetected.get()) {
+            listener.onStateChange();
+          }
+        } catch (Exception e) {
+          errorHandler.accept("Exception in Job Model Version Upgrade Scheduler. Stopping the processor...");
+        }
+      }, JMV_UPGRADE_DELAY_SEC, JMV_UPGRADE_DELAY_SEC, TimeUnit.SECONDS);
+  }
+
+  @Override
+  public void setStateChangeListener(SchedulerStateChangeListener listener) {
+    this.listener = listener;
+  }
+
+  @Override
+  public void shutdown() {
+    LOG.info("Shutting down JMVersionUpgradeScheduler Scheduler.");
+    scheduler.shutdownNow();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/samza/blob/c3b447ec/samza-azure/src/main/java/org/apache/samza/coordinator/scheduler/LeaderBarrierCompleteScheduler.java
----------------------------------------------------------------------
diff --git a/samza-azure/src/main/java/org/apache/samza/coordinator/scheduler/LeaderBarrierCompleteScheduler.java b/samza-azure/src/main/java/org/apache/samza/coordinator/scheduler/LeaderBarrierCompleteScheduler.java
new file mode 100644
index 0000000..7386fa9
--- /dev/null
+++ b/samza-azure/src/main/java/org/apache/samza/coordinator/scheduler/LeaderBarrierCompleteScheduler.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.coordinator.scheduler;
+
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Consumer;
+import org.apache.samza.coordinator.data.ProcessorEntity;
+import org.apache.samza.util.TableUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Scheduler class invoked by the leader to check if the barrier has completed.
+ * Checks every 15 seconds.
+ * The leader polls the Azure processor table in order to track this.
+ * The barrier is completed if all processors that are listed alive on the blob, have entries in the Azure table with the new job model version.
+ * All time units are in SECONDS.
+ */
+public class LeaderBarrierCompleteScheduler implements TaskScheduler {
+
+  private static final Logger LOG = LoggerFactory.getLogger(LeaderBarrierCompleteScheduler.class);
+  private static final long BARRIER_REACHED_DELAY_SEC = 5;
+  private static final long BARRIER_TIMEOUT_SEC = 30;
+  private static final ThreadFactory
+      PROCESSOR_THREAD_FACTORY = new ThreadFactoryBuilder().setNameFormat("LeaderBarrierCompleteScheduler-%d").build();
+  private final ScheduledExecutorService scheduler = Executors.newSingleThreadScheduledExecutor(PROCESSOR_THREAD_FACTORY);
+  private final TableUtils table;
+  private final String nextJMVersion;
+  private final Set<String> blobProcessorSet;
+  private final long startTime;
+  private final AtomicBoolean barrierTimeout;
+  private final Consumer<String> errorHandler;
+  private final String processorId;
+  private final AtomicReference<String> currentJMVersion;
+  private SchedulerStateChangeListener listener = null;
+
+  public LeaderBarrierCompleteScheduler(Consumer<String> errorHandler, TableUtils table, String nextJMVersion,
+      List<String> blobProcessorList, long startTime, AtomicBoolean barrierTimeout, AtomicReference<String> currentJMVersion, final String pid) {
+    this.table = table;
+    this.nextJMVersion = nextJMVersion;
+    this.blobProcessorSet = new HashSet<>(blobProcessorList);
+    this.startTime = startTime;
+    this.barrierTimeout = barrierTimeout;
+    this.errorHandler = errorHandler;
+    this.processorId = pid;
+    this.currentJMVersion = currentJMVersion;
+  }
+
+  @Override
+  public ScheduledFuture scheduleTask() {
+    return scheduler.scheduleWithFixedDelay(() -> {
+        try {
+          if (!table.getEntity(currentJMVersion.get(), processorId).getIsLeader()) {
+            LOG.info("Not the leader anymore. Shutting down LeaderBarrierCompleteScheduler.");
+            barrierTimeout.getAndSet(true);
+            listener.onStateChange();
+          } else {
+            LOG.info("Leader checking for barrier state");
+            // Get processor IDs listed in the table that have the new job model verion.
+            Iterable<ProcessorEntity> tableList = table.getEntitiesWithPartition(nextJMVersion);
+            Set<String> tableProcessors = new HashSet<>();
+            for (ProcessorEntity entity : tableList) {
+              tableProcessors.add(entity.getRowKey());
+            }
+            LOG.info("List of live processors as seen on the blob = {}", blobProcessorSet);
+            LOG.info("List of live processors as seen in the table = {}", tableProcessors);
+            if ((System.currentTimeMillis() - startTime) > (BARRIER_TIMEOUT_SEC * 1000)) {
+              barrierTimeout.getAndSet(true);
+              listener.onStateChange();
+            } else if (blobProcessorSet.equals(tableProcessors)) {
+              listener.onStateChange();
+            }
+          }
+        } catch (Exception e) {
+          errorHandler.accept("Exception in LeaderBarrierCompleteScheduler. Stopping the processor...");
+        }
+      }, BARRIER_REACHED_DELAY_SEC, BARRIER_REACHED_DELAY_SEC, TimeUnit.SECONDS);
+  }
+
+  @Override
+  public void setStateChangeListener(SchedulerStateChangeListener listener) {
+    this.listener = listener;
+  }
+
+  @Override
+  public void shutdown() {
+    LOG.info("Shutting down LeaderBarrierCompleteScheduler Scheduler.");
+    scheduler.shutdownNow();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/samza/blob/c3b447ec/samza-azure/src/main/java/org/apache/samza/coordinator/scheduler/LeaderLivenessCheckScheduler.java
----------------------------------------------------------------------
diff --git a/samza-azure/src/main/java/org/apache/samza/coordinator/scheduler/LeaderLivenessCheckScheduler.java b/samza-azure/src/main/java/org/apache/samza/coordinator/scheduler/LeaderLivenessCheckScheduler.java
new file mode 100644
index 0000000..e0fa448
--- /dev/null
+++ b/samza-azure/src/main/java/org/apache/samza/coordinator/scheduler/LeaderLivenessCheckScheduler.java
@@ -0,0 +1,120 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.coordinator.scheduler;
+
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Consumer;
+import org.apache.samza.util.BlobUtils;
+import org.apache.samza.coordinator.data.ProcessorEntity;
+import org.apache.samza.util.TableUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Scheduler class invoked by each processor to check if the leader is alive.
+ * Checks every 30 seconds.
+ * If a processor row hasn't been updated since 30 seconds, the system assumes that the processor has died.
+ * All time units are in SECONDS.
+ */
+public class LeaderLivenessCheckScheduler implements TaskScheduler {
+  private static final Logger LOG = LoggerFactory.getLogger(LeaderLivenessCheckScheduler.class);
+  private static final long LIVENESS_CHECK_DELAY_SEC = 10;
+  private static final long LIVENESS_DEBOUNCE_TIME_SEC = 30;
+  private static final ThreadFactory
+      PROCESSOR_THREAD_FACTORY = new ThreadFactoryBuilder().setNameFormat("LeaderLivenessCheckScheduler-%d").build();
+  private final ScheduledExecutorService scheduler = Executors.newSingleThreadScheduledExecutor(PROCESSOR_THREAD_FACTORY);
+  private final TableUtils table;
+  private final AtomicReference<String> currentJMVersion;
+  private final BlobUtils blob;
+  private final Consumer<String> errorHandler;
+  private final String initialState;
+  private SchedulerStateChangeListener listener = null;
+
+  public LeaderLivenessCheckScheduler(Consumer<String> errorHandler, TableUtils table, BlobUtils blob, AtomicReference<String> currentJMVersion, String initialState) {
+    this.table = table;
+    this.blob = blob;
+    this.currentJMVersion = currentJMVersion;
+    this.initialState = initialState;
+    this.errorHandler = errorHandler;
+  }
+
+  @Override
+  public ScheduledFuture scheduleTask() {
+    return scheduler.scheduleWithFixedDelay(() -> {
+        try {
+          LOG.info("Checking for leader liveness");
+          if (!checkIfLeaderAlive()) {
+            listener.onStateChange();
+          }
+        } catch (Exception e) {
+          errorHandler.accept("Exception in Leader Liveness Check Scheduler. Stopping the processor...");
+        }
+      }, LIVENESS_CHECK_DELAY_SEC, LIVENESS_CHECK_DELAY_SEC, TimeUnit.SECONDS);
+  }
+
+  @Override
+  public void setStateChangeListener(SchedulerStateChangeListener listener) {
+    this.listener = listener;
+  }
+
+  private boolean checkIfLeaderAlive() {
+    String currJMV = currentJMVersion.get();
+    String blobJMV = blob.getJobModelVersion();
+    //Get the leader processor row from the table.
+    Iterable<ProcessorEntity> tableList = table.getEntitiesWithPartition(currJMV);
+    ProcessorEntity leader = null, nextLeader = null;
+    for (ProcessorEntity entity: tableList) {
+      if (entity.getIsLeader()) {
+        leader = entity;
+        break;
+      }
+    }
+    int currJMVInt = 0;
+    if (!currJMV.equals(initialState)) {
+      currJMVInt = Integer.valueOf(currJMV);
+    }
+    if (Integer.valueOf(blobJMV) > currJMVInt) {
+      for (ProcessorEntity entity : table.getEntitiesWithPartition(blobJMV)) {
+        if (entity.getIsLeader()) {
+          nextLeader = entity;
+          break;
+        }
+      }
+    }
+    // Check if row hasn't been updated since 30 seconds.
+    if ((leader == null || (System.currentTimeMillis() - leader.getTimestamp().getTime() >= (
+        LIVENESS_DEBOUNCE_TIME_SEC * 1000))) && nextLeader == null) {
+      return false;
+    }
+    return true;
+  }
+
+  @Override
+  public void shutdown() {
+    LOG.info("Shutting down LeaderLivenessCheckScheduler Scheduler.");
+    scheduler.shutdownNow();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/samza/blob/c3b447ec/samza-azure/src/main/java/org/apache/samza/coordinator/scheduler/LivenessCheckScheduler.java
----------------------------------------------------------------------
diff --git a/samza-azure/src/main/java/org/apache/samza/coordinator/scheduler/LivenessCheckScheduler.java b/samza-azure/src/main/java/org/apache/samza/coordinator/scheduler/LivenessCheckScheduler.java
new file mode 100644
index 0000000..d4715f3
--- /dev/null
+++ b/samza-azure/src/main/java/org/apache/samza/coordinator/scheduler/LivenessCheckScheduler.java
@@ -0,0 +1,108 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.samza.coordinator.scheduler;
+
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Consumer;
+import org.apache.samza.util.BlobUtils;
+import org.apache.samza.util.TableUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Scheduler class invoked by the leader to check for changes in the list of live processors.
+ * Checks every 30 seconds.
+ * If a processor row hasn't been updated since 30 seconds, the system assumes that the processor has died.
+ * All time units are in SECONDS.
+ */
+public class LivenessCheckScheduler implements TaskScheduler {
+
+  private static final Logger LOG = LoggerFactory.getLogger(LivenessCheckScheduler.class);
+  private static final long LIVENESS_CHECK_DELAY_SEC = 5;
+  private static final ThreadFactory
+      PROCESSOR_THREAD_FACTORY = new ThreadFactoryBuilder().setNameFormat("LivenessCheckScheduler-%d").build();
+  private final ScheduledExecutorService scheduler = Executors.newSingleThreadScheduledExecutor(PROCESSOR_THREAD_FACTORY);
+  private final TableUtils table;
+  private final BlobUtils blob;
+  private final AtomicReference<String> currentJMVersion;
+  private final AtomicReference<List<String>> liveProcessorsList = new AtomicReference<>(null);
+  private final Consumer<String> errorHandler;
+  private SchedulerStateChangeListener listener = null;
+  private final String processorId;
+
+  public LivenessCheckScheduler(Consumer<String> errorHandler, TableUtils table, BlobUtils blob, AtomicReference<String> currentJMVersion, final String pid) {
+    this.table = table;
+    this.blob = blob;
+    this.currentJMVersion = currentJMVersion;
+    this.errorHandler = errorHandler;
+    this.processorId = pid;
+  }
+
+  @Override
+  public ScheduledFuture scheduleTask() {
+    return scheduler.scheduleWithFixedDelay(() -> {
+        try {
+          if (!table.getEntity(currentJMVersion.get(), processorId).getIsLeader()) {
+            LOG.info("Not the leader anymore. Shutting down LivenessCheckScheduler.");
+            scheduler.shutdownNow();
+            return;
+          }
+          LOG.info("Checking for list of live processors");
+          //Get the list of live processors published on the blob.
+          Set<String> currProcessors = new HashSet<>(blob.getLiveProcessorList());
+          //Get the list of live processors from the table. This is the current system state.
+          Set<String> liveProcessors = table.getActiveProcessorsList(currentJMVersion);
+          //Invoke listener if the table list is not consistent with the blob list.
+          if (!liveProcessors.equals(currProcessors)) {
+            liveProcessorsList.getAndSet(new ArrayList<>(liveProcessors));
+            listener.onStateChange();
+          }
+        } catch (Exception e) {
+          errorHandler.accept("Exception in Liveness Check Scheduler. Stopping the processor...");
+        }
+      }, LIVENESS_CHECK_DELAY_SEC, LIVENESS_CHECK_DELAY_SEC, TimeUnit.SECONDS);
+  }
+
+  @Override
+  public void setStateChangeListener(SchedulerStateChangeListener listener) {
+    this.listener = listener;
+  }
+
+  public AtomicReference<List<String>> getLiveProcessors() {
+    return liveProcessorsList;
+  }
+
+  @Override
+  public void shutdown() {
+    LOG.info("Shutting down LivenessCheckScheduler Scheduler.");
+    scheduler.shutdownNow();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/samza/blob/c3b447ec/samza-azure/src/main/java/org/apache/samza/coordinator/scheduler/RenewLeaseScheduler.java
----------------------------------------------------------------------
diff --git a/samza-azure/src/main/java/org/apache/samza/coordinator/scheduler/RenewLeaseScheduler.java b/samza-azure/src/main/java/org/apache/samza/coordinator/scheduler/RenewLeaseScheduler.java
new file mode 100644
index 0000000..f158122
--- /dev/null
+++ b/samza-azure/src/main/java/org/apache/samza/coordinator/scheduler/RenewLeaseScheduler.java
@@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.coordinator.scheduler;
+
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Consumer;
+import org.apache.samza.util.LeaseBlobManager;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Scheduler class to keep renewing the lease once an entity has acquired it.
+ * Renews every 45 seconds.
+ * All time units are in SECONDS.
+ */
+public class RenewLeaseScheduler implements TaskScheduler {
+
+  private static final Logger LOG = LoggerFactory.getLogger(RenewLeaseScheduler.class);
+  private static final long RENEW_LEASE_DELAY_SEC = 45;
+  private static final ThreadFactory
+      PROCESSOR_THREAD_FACTORY = new ThreadFactoryBuilder().setNameFormat("RenewLeaseScheduler-%d").build();
+  private final ScheduledExecutorService scheduler = Executors.newSingleThreadScheduledExecutor(PROCESSOR_THREAD_FACTORY);
+  private final LeaseBlobManager leaseBlobManager;
+  private final AtomicReference<String> leaseId;
+  private final Consumer<String> errorHandler;
+
+  public RenewLeaseScheduler(Consumer<String> errorHandler, LeaseBlobManager leaseBlobManager, AtomicReference<String> leaseId) {
+    this.leaseBlobManager = leaseBlobManager;
+    this.leaseId = leaseId;
+    this.errorHandler = errorHandler;
+  }
+
+  @Override
+  public ScheduledFuture scheduleTask() {
+    return scheduler.scheduleWithFixedDelay(() -> {
+        try {
+          LOG.info("Renewing lease");
+          boolean status = leaseBlobManager.renewLease(leaseId.get());
+          if (!status) {
+            errorHandler.accept("Unable to renew lease. Continuing as non-leader.");
+          }
+        } catch (Exception e) {
+          errorHandler.accept("Exception in Renew Lease Scheduler. Continuing as non-leader.");
+        }
+      }, RENEW_LEASE_DELAY_SEC, RENEW_LEASE_DELAY_SEC, TimeUnit.SECONDS);
+  }
+
+  @Override
+  public void setStateChangeListener(SchedulerStateChangeListener listener) {}
+
+  @Override
+  public void shutdown() {
+    LOG.info("Shutting down RenewLeaseScheduler Scheduler.");
+    scheduler.shutdownNow();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/samza/blob/c3b447ec/samza-azure/src/main/java/org/apache/samza/coordinator/scheduler/SchedulerStateChangeListener.java
----------------------------------------------------------------------
diff --git a/samza-azure/src/main/java/org/apache/samza/coordinator/scheduler/SchedulerStateChangeListener.java b/samza-azure/src/main/java/org/apache/samza/coordinator/scheduler/SchedulerStateChangeListener.java
new file mode 100644
index 0000000..95fc4e1
--- /dev/null
+++ b/samza-azure/src/main/java/org/apache/samza/coordinator/scheduler/SchedulerStateChangeListener.java
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.coordinator.scheduler;
+
+/**
+ * Listener interface for Azure Job Coordinator, to track state changes and take necessary actions.
+ */
+public interface SchedulerStateChangeListener {
+
+  void onStateChange();
+
+}

http://git-wip-us.apache.org/repos/asf/samza/blob/c3b447ec/samza-azure/src/main/java/org/apache/samza/coordinator/scheduler/TaskScheduler.java
----------------------------------------------------------------------
diff --git a/samza-azure/src/main/java/org/apache/samza/coordinator/scheduler/TaskScheduler.java b/samza-azure/src/main/java/org/apache/samza/coordinator/scheduler/TaskScheduler.java
new file mode 100644
index 0000000..63d6e24
--- /dev/null
+++ b/samza-azure/src/main/java/org/apache/samza/coordinator/scheduler/TaskScheduler.java
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.samza.coordinator.scheduler;
+
+import java.util.concurrent.ScheduledFuture;
+
+
+/**
+ * Interface for scheduling tasks for Azure Job Coordinator.
+ */
+public interface TaskScheduler {
+
+  ScheduledFuture scheduleTask();
+
+  void setStateChangeListener(SchedulerStateChangeListener listener);
+
+  void shutdown();
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/samza/blob/c3b447ec/samza-azure/src/main/java/org/apache/samza/util/BlobUtils.java
----------------------------------------------------------------------
diff --git a/samza-azure/src/main/java/org/apache/samza/util/BlobUtils.java b/samza-azure/src/main/java/org/apache/samza/util/BlobUtils.java
new file mode 100644
index 0000000..85e4273
--- /dev/null
+++ b/samza-azure/src/main/java/org/apache/samza/util/BlobUtils.java
@@ -0,0 +1,284 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.samza.util;
+
+import com.microsoft.azure.storage.AccessCondition;
+import com.microsoft.azure.storage.StorageException;
+import com.microsoft.azure.storage.blob.CloudBlobClient;
+import com.microsoft.azure.storage.blob.CloudBlobContainer;
+import com.microsoft.azure.storage.blob.CloudPageBlob;
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.URISyntaxException;
+import java.util.Arrays;
+import java.util.List;
+import org.apache.samza.AzureClient;
+import org.apache.samza.AzureException;
+import org.apache.samza.coordinator.data.JobModelBundle;
+import org.apache.samza.SamzaException;
+import org.apache.samza.job.model.JobModel;
+import org.apache.samza.serializers.model.SamzaObjectMapper;
+import org.eclipse.jetty.http.HttpStatus;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Client side class that has reference to Azure blob storage.
+ * Used for writing and reading from the blob.
+ * Every write requires a valid lease ID.
+ */
+public class BlobUtils {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BlobUtils.class);
+  private static final long JOB_MODEL_BLOCK_SIZE = 1024000;
+  private static final long BARRIER_STATE_BLOCK_SIZE = 1024;
+  private static final long PROCESSOR_LIST_BLOCK_SIZE = 1024;
+  private CloudBlobClient blobClient;
+  private CloudBlobContainer container;
+  private CloudPageBlob blob;
+
+  /**
+   * Creates an object of BlobUtils. It creates the container and page blob if they don't exist already.
+   * @param client Client handle for access to Azure Storage account.
+   * @param containerName Name of container inside which we want the blob to reside.
+   * @param blobName Name of the blob to be managed.
+   * @param length Length of the page blob.
+   * @throws AzureException If an Azure storage service error occurred, or when the container name or blob name is invalid.
+   */
+  public BlobUtils(AzureClient client, String containerName, String blobName, long length) {
+    this.blobClient = client.getBlobClient();
+    try {
+      this.container = blobClient.getContainerReference(containerName);
+      container.createIfNotExists();
+      this.blob = container.getPageBlobReference(blobName);
+      if (!blob.exists()) {
+        blob.create(length, AccessCondition.generateIfNotExistsCondition(), null, null);
+      }
+    } catch (URISyntaxException e) {
+      LOG.error("Container name: " + containerName + " or blob name: " + blobName + " invalid.", e);
+      throw new AzureException(e);
+    } catch (StorageException e) {
+      int httpStatusCode = e.getHttpStatusCode();
+      if (httpStatusCode == HttpStatus.CONFLICT_409) {
+        LOG.info("The blob you're trying to create exists already.", e);
+      } else {
+        LOG.error("Azure Storage Exception!", e);
+        throw new AzureException(e);
+      }
+    }
+  }
+
+  /**
+   * Writes the job model to the blob.
+   * Write is successful only if the lease ID passed is valid and the processor holds the lease.
+   * Called by the leader.
+   * @param prevJM Previous job model version that the processor was operating on.
+   * @param currJM Current job model version that the processor is operating on.
+   * @param prevJMV Previous job model version that the processor was operating on.
+   * @param currJMV Current job model version that the processor is operating on.
+   * @param leaseId LeaseID of the lease that the processor holds on the blob. Null if there is no lease.
+   * @return true if write to the blob is successful, false if leaseID is null or an Azure storage service error or IO exception occurred.
+   */
+  public boolean publishJobModel(JobModel prevJM, JobModel currJM, String prevJMV, String currJMV, String leaseId) {
+    try {
+      if (leaseId == null) {
+        return false;
+      }
+      JobModelBundle bundle = new JobModelBundle(prevJM, currJM, prevJMV, currJMV);
+      byte[] data = SamzaObjectMapper.getObjectMapper().writerWithDefaultPrettyPrinter().writeValueAsBytes(bundle);
+      byte[] pageData = Arrays.copyOf(data, (int) JOB_MODEL_BLOCK_SIZE);
+      InputStream is = new ByteArrayInputStream(pageData);
+      blob.uploadPages(is, 0, JOB_MODEL_BLOCK_SIZE, AccessCondition.generateLeaseCondition(leaseId), null, null);
+      LOG.info("Uploaded {} jobModel to blob", bundle.getCurrJobModel());
+      return true;
+    } catch (StorageException | IOException e) {
+      LOG.error("JobModel publish failed for version = " + currJMV, e);
+      return false;
+    }
+  }
+
+  /**
+   * Reads the current job model from the blob.
+   * @return The current job model published on the blob. Returns null when job model details not found on the blob.
+   * @throws AzureException in getJobModelBundle() if an Azure storage service error occurred.
+   * @throws SamzaException in getJobModelBundle() if data retrieved from blob could not be parsed by SamzaObjectMapper.
+   */
+  public JobModel getJobModel() {
+    LOG.info("Reading the job model from blob.");
+    JobModelBundle jmBundle = getJobModelBundle();
+    if (jmBundle == null) {
+      LOG.error("Job Model details don't exist on the blob.");
+      return null;
+    }
+    JobModel jm = jmBundle.getCurrJobModel();
+    return jm;
+  }
+
+  /**
+   * Reads the current job model version from the blob .
+   * @return Current job model version published on the blob. Returns null when job model details not found on the blob.
+   * @throws AzureException in getJobModelBundle() if an Azure storage service error occurred.
+   * @throws SamzaException in getJobModelBundle() if data retrieved from blob could not be parsed by SamzaObjectMapper.
+   */
+  public String getJobModelVersion() {
+    LOG.info("Reading the job model version from blob.");
+    JobModelBundle jmBundle = getJobModelBundle();
+    if (jmBundle == null) {
+      LOG.error("Job Model details don't exist on the blob.");
+      return null;
+    }
+    String jmVersion = jmBundle.getCurrJobModelVersion();
+    return jmVersion;
+  }
+
+  /**
+   * Writes the barrier state to the blob.
+   * Write is successful only if the lease ID passed is valid and the processor holds the lease.
+   * Called only by the leader.
+   * @param state Barrier state to be published to the blob.
+   * @param leaseId LeaseID of the valid lease that the processor holds on the blob. Null if there is no lease.
+   * @return true if write to the blob is successful, false if leaseID is null or an Azure storage service error or IO exception occurred.
+   */
+  public boolean publishBarrierState(String state, String leaseId) {
+    try {
+      if (leaseId == null) {
+        return false;
+      }
+      byte[] data = SamzaObjectMapper.getObjectMapper().writerWithDefaultPrettyPrinter().writeValueAsBytes(state);
+      byte[] pageData = Arrays.copyOf(data, (int) BARRIER_STATE_BLOCK_SIZE);
+      InputStream is = new ByteArrayInputStream(pageData);
+
+      //uploadPages is only successful when the AccessCondition provided has an active and valid lease ID. It fails otherwise.
+      blob.uploadPages(is, JOB_MODEL_BLOCK_SIZE, BARRIER_STATE_BLOCK_SIZE, AccessCondition.generateLeaseCondition(leaseId), null, null);
+      LOG.info("Uploaded barrier state {} to blob", state);
+      return true;
+    } catch (StorageException | IOException e) {
+      LOG.error("Barrier state " + state + " publish failed", e);
+      return false;
+    }
+  }
+
+  /**
+   * Reads the current barrier state from the blob.
+   * @return Barrier state published on the blob.
+   * @throws AzureException If an Azure storage service error occurred.
+   * @throws SamzaException If data retrieved from blob could not be parsed by SamzaObjectMapper.
+   */
+  public String getBarrierState() {
+    LOG.info("Reading the barrier state from blob.");
+    byte[] data = new byte[(int) BARRIER_STATE_BLOCK_SIZE];
+    try {
+      blob.downloadRangeToByteArray(JOB_MODEL_BLOCK_SIZE, BARRIER_STATE_BLOCK_SIZE, data, 0);
+    } catch (StorageException e) {
+      LOG.error("Failed to read barrier state from blob.", e);
+      throw new AzureException(e);
+    }
+    String state;
+    try {
+      state = SamzaObjectMapper.getObjectMapper().readValue(data, String.class);
+    } catch (IOException e) {
+      LOG.error("Failed to parse byte data: " + data + " for barrier state retrieved from the blob.", e);
+      throw new SamzaException(e);
+    }
+    return state;
+  }
+
+  /**
+   * Writes the list of live processors in the system to the blob.
+   * Write is successful only if the lease ID passed is valid and the processor holds the lease.
+   * Called only by the leader.
+   * @param processors List of live processors to be published on the blob.
+   * @param leaseId LeaseID of the valid lease that the processor holds on the blob. Null if there is no lease.
+   * @return true if write to the blob is successful, false if leaseID is null or an Azure storage service error or IO exception occurred.
+   */
+  public boolean publishLiveProcessorList(List<String> processors, String leaseId) {
+    try {
+      if (leaseId == null) {
+        return false;
+      }
+      byte[] data = SamzaObjectMapper.getObjectMapper().writerWithDefaultPrettyPrinter().writeValueAsBytes(processors);
+      byte[] pageData = Arrays.copyOf(data, (int) BARRIER_STATE_BLOCK_SIZE);
+      InputStream is = new ByteArrayInputStream(pageData);
+      blob.uploadPages(is, JOB_MODEL_BLOCK_SIZE + BARRIER_STATE_BLOCK_SIZE, PROCESSOR_LIST_BLOCK_SIZE, AccessCondition.generateLeaseCondition(leaseId), null, null);
+      LOG.info("Uploaded list of live processors to blob.");
+      return true;
+    } catch (StorageException | IOException e) {
+      LOG.error("Processor list: " + processors + "publish failed", e);
+      return false;
+    }
+  }
+
+  /**
+   * Reads the list of live processors published on the blob.
+   * @return String list of live processors.
+   * @throws AzureException If an Azure storage service error occurred.
+   * @throws SamzaException If data retrieved from blob could not be parsed by SamzaObjectMapper.
+   */
+  public List<String> getLiveProcessorList() {
+    LOG.info("Read the the list of live processors from blob.");
+    byte[] data = new byte[(int) PROCESSOR_LIST_BLOCK_SIZE];
+    try {
+      blob.downloadRangeToByteArray(JOB_MODEL_BLOCK_SIZE + BARRIER_STATE_BLOCK_SIZE, PROCESSOR_LIST_BLOCK_SIZE, data, 0);
+    } catch (StorageException e) {
+      LOG.error("Failed to read the list of live processors from the blob.", new AzureException(e));
+      throw new AzureException(e);
+    }
+    List<String> list;
+    try {
+      list = SamzaObjectMapper.getObjectMapper().readValue(data, List.class);
+    } catch (IOException e) {
+      LOG.error("Failed to parse byte data: " + data + " for live processor list retrieved from the blob", new SamzaException(e));
+      throw new SamzaException(e);
+    }
+    return list;
+  }
+
+  public CloudBlobClient getBlobClient() {
+    return this.blobClient;
+  }
+
+  public CloudBlobContainer getBlobContainer() {
+    return this.container;
+  }
+
+  public CloudPageBlob getBlob() {
+    return this.blob;
+  }
+
+  private JobModelBundle getJobModelBundle() {
+    byte[] data = new byte[(int) JOB_MODEL_BLOCK_SIZE];
+    try {
+      blob.downloadRangeToByteArray(0, JOB_MODEL_BLOCK_SIZE, data, 0);
+    } catch (StorageException e) {
+      LOG.error("Failed to read JobModel details from the blob.", e);
+      throw new AzureException(e);
+    }
+    try {
+      JobModelBundle jmBundle = SamzaObjectMapper.getObjectMapper().readValue(data, JobModelBundle.class);
+      return jmBundle;
+    } catch (IOException e) {
+      LOG.error("Failed to parse byte data: " + data + " for JobModel details retrieved from the blob", e);
+      throw new SamzaException(e);
+    }
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/samza/blob/c3b447ec/samza-azure/src/main/java/org/apache/samza/util/LeaseBlobManager.java
----------------------------------------------------------------------
diff --git a/samza-azure/src/main/java/org/apache/samza/util/LeaseBlobManager.java b/samza-azure/src/main/java/org/apache/samza/util/LeaseBlobManager.java
new file mode 100644
index 0000000..dffb7ae
--- /dev/null
+++ b/samza-azure/src/main/java/org/apache/samza/util/LeaseBlobManager.java
@@ -0,0 +1,99 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.samza.util;
+
+import com.microsoft.azure.storage.AccessCondition;
+import com.microsoft.azure.storage.StorageException;
+import com.microsoft.azure.storage.blob.CloudPageBlob;
+import org.apache.samza.AzureException;
+import org.eclipse.jetty.http.HttpStatus;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Helper class for lease blob operations.
+ */
+public class LeaseBlobManager {
+
+  private static final Logger LOG = LoggerFactory.getLogger(LeaseBlobManager.class);
+  private final CloudPageBlob leaseBlob;
+
+  public LeaseBlobManager(CloudPageBlob leaseBlob) {
+    this.leaseBlob = leaseBlob;
+  }
+
+  /**
+   * Acquires a lease on a blob. The lease ID is NULL initially.
+   * @param leaseTimeInSec The time in seconds you want to acquire the lease for.
+   * @param leaseId Proposed ID you want to acquire the lease with, null if not proposed.
+   * @return String that represents lease ID.  Null if acquireLease is unsuccessful because the blob is leased already.
+   * @throws AzureException If a Azure storage service error occurred. This includes the case where the blob you're trying to lease does not exist.
+   */
+  public String acquireLease(int leaseTimeInSec, String leaseId) {
+    try {
+      String id = leaseBlob.acquireLease(leaseTimeInSec, leaseId);
+      LOG.info("Acquired lease with lease id = " + id);
+      return id;
+    } catch (StorageException storageException) {
+      int httpStatusCode = storageException.getHttpStatusCode();
+      if (httpStatusCode == HttpStatus.CONFLICT_409) {
+        LOG.info("The blob you're trying to acquire is leased already.", storageException.getMessage());
+      } else if (httpStatusCode == HttpStatus.NOT_FOUND_404) {
+        LOG.error("The blob you're trying to lease does not exist.", storageException);
+        throw new AzureException(storageException);
+      } else {
+        LOG.error("Error acquiring lease!", storageException);
+        throw new AzureException(storageException);
+      }
+    }
+    return null;
+  }
+
+  /**
+   * Renews the lease on the blob.
+   * @param leaseId ID of the lease to be renewed.
+   * @return True if lease was renewed successfully, false otherwise.
+   */
+  public boolean renewLease(String leaseId) {
+    try {
+      leaseBlob.renewLease(AccessCondition.generateLeaseCondition(leaseId));
+      return true;
+    } catch (StorageException storageException) {
+      LOG.error("Wasn't able to renew lease with lease id: " + leaseId, storageException);
+      return false;
+    }
+  }
+
+  /**
+   * Releases the lease on the blob.
+   * @param leaseId ID of the lease to be released.
+   * @return True if released successfully, false otherwise.
+   */
+  public boolean releaseLease(String leaseId) {
+    try {
+      leaseBlob.releaseLease(AccessCondition.generateLeaseCondition(leaseId));
+      return true;
+    } catch (StorageException storageException) {
+      LOG.error("Wasn't able to release lease with lease id: " + leaseId, storageException);
+      return false;
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/samza/blob/c3b447ec/samza-azure/src/main/java/org/apache/samza/util/TableUtils.java
----------------------------------------------------------------------
diff --git a/samza-azure/src/main/java/org/apache/samza/util/TableUtils.java b/samza-azure/src/main/java/org/apache/samza/util/TableUtils.java
new file mode 100644
index 0000000..f49ce27
--- /dev/null
+++ b/samza-azure/src/main/java/org/apache/samza/util/TableUtils.java
@@ -0,0 +1,205 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.samza.util;
+
+import com.microsoft.azure.storage.StorageException;
+import com.microsoft.azure.storage.table.CloudTable;
+import com.microsoft.azure.storage.table.CloudTableClient;
+import com.microsoft.azure.storage.table.TableOperation;
+import com.microsoft.azure.storage.table.TableQuery;
+import java.net.URISyntaxException;
+import java.util.HashSet;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.samza.AzureClient;
+import org.apache.samza.AzureException;
+import org.apache.samza.coordinator.data.ProcessorEntity;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ *  Client side class that has a reference to Azure Table Storage.
+ *  Enables the user to add or delete information from the table, make updates to the table and retrieve information from the table.
+ *  Every row in a table is uniquely identified by a combination of the PARTIITON KEY and ROW KEY.
+ *  PARTITION KEY = Group ID = Job Model Version (for this case).
+ *  ROW KEY = Unique entity ID for a group = Processor ID (for this case).
+ */
+public class TableUtils {
+
+  private static final Logger LOG = LoggerFactory.getLogger(TableUtils.class);
+  private static final String PARTITION_KEY = "PartitionKey";
+  private static final long LIVENESS_DEBOUNCE_TIME_SEC = 30;
+  private final String initialState;
+  private final CloudTableClient tableClient;
+  private final CloudTable table;
+
+  public TableUtils(AzureClient client, String tableName, String initialState) {
+    this.initialState = initialState;
+    tableClient = client.getTableClient();
+    try {
+      table = tableClient.getTableReference(tableName);
+      table.createIfNotExists();
+    } catch (URISyntaxException e) {
+      LOG.error("\nConnection string specifies an invalid URI.", e);
+      throw new AzureException(e);
+    } catch (StorageException e) {
+      LOG.error("Azure storage exception.", e);
+      throw new AzureException(e);
+    }
+  }
+
+  /**
+   * Add a row which denotes an active processor to the processor table.
+   * @param jmVersion Job model version that the processor is operating on.
+   * @param pid Unique processor ID.
+   * @param isLeader Denotes whether the processor is a leader or not.
+   * @throws AzureException If an Azure storage service error occurred.
+   */
+  public void addProcessorEntity(String jmVersion, String pid, boolean isLeader) {
+    ProcessorEntity entity = new ProcessorEntity(jmVersion, pid);
+    entity.setIsLeader(isLeader);
+    entity.updateLiveness();
+    TableOperation add = TableOperation.insert(entity);
+    try {
+      table.execute(add);
+    } catch (StorageException e) {
+      LOG.error("Azure storage exception while adding processor entity with job model version: " + jmVersion + "and pid: " + pid, e);
+      throw new AzureException(e);
+    }
+  }
+
+  /**
+   * Retrieve a particular row in the processor table, given the partition key and the row key.
+   * @param jmVersion Job model version of the processor row to be retrieved.
+   * @param pid Unique processor ID of the processor row to be retrieved.
+   * @return An instance of required processor entity. Null if does not exist.
+   * @throws AzureException If an Azure storage service error occurred.
+   */
+  public ProcessorEntity getEntity(String jmVersion, String pid) {
+    try {
+      TableOperation retrieveEntity = TableOperation.retrieve(jmVersion, pid, ProcessorEntity.class);
+      ProcessorEntity entity = table.execute(retrieveEntity).getResultAsType();
+      return entity;
+    } catch (StorageException e) {
+      LOG.error("Azure storage exception while retrieving processor entity with job model version: " + jmVersion + "and pid: " + pid, e);
+      throw new AzureException(e);
+    }
+  }
+
+  /**
+   * Updates the liveness value of a particular processor with a randomly generated integer, which in turn updates the last modified since timestamp of the row.
+   * @param jmVersion Job model version of the processor row to be updated.
+   * @param pid Unique processor ID of the processor row to be updated.
+   */
+  public void updateHeartbeat(String jmVersion, String pid) {
+    try {
+      Random rand = new Random();
+      TableOperation retrieveEntity = TableOperation.retrieve(jmVersion, pid, ProcessorEntity.class);
+      ProcessorEntity entity = table.execute(retrieveEntity).getResultAsType();
+      entity.updateLiveness();
+      TableOperation update = TableOperation.replace(entity);
+      table.execute(update);
+    } catch (StorageException e) {
+      LOG.error("Azure storage exception while updating heartbeat for job model version: " + jmVersion + "and pid: " + pid, e);
+    }
+  }
+
+  /**
+   * Updates the isLeader value when the processor starts or stops being a leader.
+   * @param jmVersion Job model version of the processor row to be updated.
+   * @param pid Unique processor ID of the processor row to be updated.
+   * @param isLeader Denotes whether the processor is a leader or not.
+   * @throws AzureException If an Azure storage service error occurred.
+   */
+  public void updateIsLeader(String jmVersion, String pid, boolean isLeader) {
+    try {
+      TableOperation retrieveEntity = TableOperation.retrieve(jmVersion, pid, ProcessorEntity.class);
+      ProcessorEntity entity = table.execute(retrieveEntity).getResultAsType();
+      entity.setIsLeader(isLeader);
+      TableOperation update = TableOperation.replace(entity);
+      table.execute(update);
+    } catch (StorageException e) {
+      LOG.error("Azure storage exception while updating isLeader value for job model version: " + jmVersion + "and pid: " + pid, e);
+      throw new AzureException(e);
+    }
+  }
+
+  /**
+   * Deletes a specified row in the processor table.
+   * @param jmVersion Job model version of the processor row to be deleted.
+   * @param pid Unique processor ID of the processor row to be deleted.
+   * @throws AzureException If an Azure storage service error occurred.
+   */
+  public void deleteProcessorEntity(String jmVersion, String pid) {
+    try {
+      TableOperation retrieveEntity = TableOperation.retrieve(jmVersion, pid, ProcessorEntity.class);
+      ProcessorEntity entity = table.execute(retrieveEntity).getResultAsType();
+      TableOperation remove = TableOperation.delete(entity);
+      table.execute(remove);
+    } catch (StorageException e) {
+      LOG.error("Azure storage exception while deleting processor entity with job model version: " + jmVersion + "and pid: " + pid, e);
+      throw new AzureException(e);
+    }
+  }
+
+  /**
+   * Retrieve all rows in a table with the given partition key.
+   * @param partitionKey Job model version of the processors to be retrieved.
+   * @return Iterable list of processor entities.
+   */
+  public Iterable<ProcessorEntity> getEntitiesWithPartition(String partitionKey) {
+    String partitionFilter = TableQuery.generateFilterCondition(PARTITION_KEY, TableQuery.QueryComparisons.EQUAL, partitionKey);
+    TableQuery<ProcessorEntity> partitionQuery = TableQuery.from(ProcessorEntity.class).where(partitionFilter);
+    return table.execute(partitionQuery);
+  }
+
+  /**
+   * Gets the list of all active processors that are heartbeating to the processor table.
+   * @param currentJMVersion Current job model version that the processors in the application are operating on.
+   * @return List of ids of currently active processors in the application, retrieved from the processor table.
+   */
+  public Set<String> getActiveProcessorsList(AtomicReference<String> currentJMVersion) {
+    Iterable<ProcessorEntity> tableList = getEntitiesWithPartition(currentJMVersion.get());
+    Set<String> activeProcessorsList = new HashSet<>();
+    for (ProcessorEntity entity: tableList) {
+      if (System.currentTimeMillis() - entity.getTimestamp().getTime() <= (LIVENESS_DEBOUNCE_TIME_SEC * 1000)) {
+        activeProcessorsList.add(entity.getRowKey());
+      }
+    }
+
+    Iterable<ProcessorEntity> unassignedList = getEntitiesWithPartition(initialState);
+    for (ProcessorEntity entity: unassignedList) {
+      long temp = System.currentTimeMillis() - entity.getTimestamp().getTime();
+      LOG.info("Time elapsed since last heartbeat: {}", temp);
+      if (temp <= (LIVENESS_DEBOUNCE_TIME_SEC * 1000)) {
+        activeProcessorsList.add(entity.getRowKey());
+      }
+    }
+    LOG.info("Active processors list: {}", activeProcessorsList);
+    return activeProcessorsList;
+  }
+
+  public CloudTable getTable() {
+    return table;
+  }
+
+}
\ No newline at end of file


[09/26] samza git commit: SAMZA-1396 TestZkLocalApplicationRunner tests fails after SAMZA-1385

Posted by xi...@apache.org.
SAMZA-1396 TestZkLocalApplicationRunner tests fails after SAMZA-1385

* Fixes ZkPath issues
* Fixes appname / jobname mismatch

Author: Navina Ramesh <na...@apache.org>

Reviewers: Xinyu Liu <xi...@apache.org>, Bharath Kumarasubramanian <co...@gmail.com>

Closes #274 from navina/SAMZA-1396


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

Branch: refs/heads/0.14.0
Commit: 119e2fa0126a66f949cba8d2e2d8cbb2a36cc1ec
Parents: 89989fd
Author: Navina Ramesh <na...@apache.org>
Authored: Wed Aug 16 14:42:18 2017 -0700
Committer: navina <na...@apache.org>
Committed: Wed Aug 16 14:42:18 2017 -0700

----------------------------------------------------------------------
 .../samza/zk/ZkJobCoordinatorFactory.java       |  2 +-
 .../processor/TestZkLocalApplicationRunner.java | 62 ++++++++++++--------
 2 files changed, 39 insertions(+), 25 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/samza/blob/119e2fa0/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 08d826e..563bf4c 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
@@ -60,7 +60,7 @@ public class ZkJobCoordinatorFactory implements JobCoordinatorFactory {
     return new ZkUtils(keyBuilder, zkClient, zkConfig.getZkConnectionTimeoutMs(), metricsRegistry);
   }
 
-  private String getJobCoordinationZkPath(Config config) {
+  public static String getJobCoordinationZkPath(Config config) {
     JobConfig jobConfig = new JobConfig(config);
     String appId = new ApplicationConfig(config).getGlobalAppId();
     String jobName = jobConfig.getName().isDefined()

http://git-wip-us.apache.org/repos/asf/samza/blob/119e2fa0/samza-test/src/test/java/org/apache/samza/test/processor/TestZkLocalApplicationRunner.java
----------------------------------------------------------------------
diff --git a/samza-test/src/test/java/org/apache/samza/test/processor/TestZkLocalApplicationRunner.java b/samza-test/src/test/java/org/apache/samza/test/processor/TestZkLocalApplicationRunner.java
index 9ca48ad..76fd046 100644
--- a/samza-test/src/test/java/org/apache/samza/test/processor/TestZkLocalApplicationRunner.java
+++ b/samza-test/src/test/java/org/apache/samza/test/processor/TestZkLocalApplicationRunner.java
@@ -23,15 +23,6 @@ import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
-import java.util.UUID;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.atomic.AtomicBoolean;
 import kafka.admin.AdminUtils;
 import kafka.server.KafkaServer;
 import kafka.utils.TestUtils;
@@ -58,6 +49,7 @@ import org.apache.samza.runtime.LocalApplicationRunner;
 import org.apache.samza.test.StandaloneIntegrationTestHarness;
 import org.apache.samza.test.StandaloneTestUtils;
 import org.apache.samza.util.NoOpMetricsRegistry;
+import org.apache.samza.zk.ZkJobCoordinatorFactory;
 import org.apache.samza.zk.ZkKeyBuilder;
 import org.apache.samza.zk.ZkUtils;
 import org.junit.Rule;
@@ -68,7 +60,18 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import scala.collection.JavaConverters;
 
-import static org.junit.Assert.*;
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.UUID;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 
 /**
  * Integration tests for {@link LocalApplicationRunner}.
@@ -87,7 +90,6 @@ public class TestZkLocalApplicationRunner extends StandaloneIntegrationTestHarne
   private static final String TEST_TASK_GROUPER_FACTORY = "org.apache.samza.container.grouper.task.GroupByContainerIdsFactory";
   private static final String TEST_JOB_COORDINATOR_FACTORY = "org.apache.samza.zk.ZkJobCoordinatorFactory";
   private static final String TEST_SYSTEM_FACTORY = "org.apache.samza.system.kafka.KafkaSystemFactory";
-  private static final String TEST_JOB_NAME = "test-job";
   private static final String TASK_SHUTDOWN_MS = "3000";
   private static final String JOB_DEBOUNCE_TIME_MS = "1000";
   private static final String[] PROCESSOR_IDS = new String[] {"0000000000", "0000000001", "0000000002"};
@@ -118,16 +120,23 @@ public class TestZkLocalApplicationRunner extends StandaloneIntegrationTestHarne
     testStreamAppId = String.format("test-app-id-%s", uniqueTestId);
     inputKafkaTopic = String.format("test-input-topic-%s", uniqueTestId);
     outputKafkaTopic = String.format("test-output-topic-%s", uniqueTestId);
+
+    // Set up stream application config map with the given testStreamAppName, testStreamAppId and test kafka system
+    // TODO: processorId should typically come up from a processorID generator as processor.id will be deprecated in 0.14.0+
+    Map<String, String> configMap =
+        buildStreamApplicationConfigMap(TEST_SYSTEM, inputKafkaTopic, testStreamAppName, testStreamAppId);
+    configMap.put(JobConfig.PROCESSOR_ID(), PROCESSOR_IDS[0]);
+    applicationConfig1 = new ApplicationConfig(new MapConfig(configMap));
+    configMap.put(JobConfig.PROCESSOR_ID(), PROCESSOR_IDS[1]);
+    applicationConfig2 = new ApplicationConfig(new MapConfig(configMap));
+    configMap.put(JobConfig.PROCESSOR_ID(), PROCESSOR_IDS[2]);
+    applicationConfig3 = new ApplicationConfig(new MapConfig(configMap));
+
     ZkClient zkClient = new ZkClient(zkConnect());
-    ZkKeyBuilder zkKeyBuilder = new ZkKeyBuilder(String.format("app-%s-%s", testStreamAppName, testStreamAppId));
+    ZkKeyBuilder zkKeyBuilder = new ZkKeyBuilder(ZkJobCoordinatorFactory.getJobCoordinationZkPath(applicationConfig1));
     zkUtils = new ZkUtils(zkKeyBuilder, zkClient, ZK_CONNECTION_TIMEOUT_MS, new NoOpMetricsRegistry());
     zkUtils.connect();
 
-    // Set up stream application configs with different processorIds and same testStreamAppName, testStreamAppId.
-    applicationConfig1 = buildStreamApplicationConfig(TEST_SYSTEM, inputKafkaTopic, PROCESSOR_IDS[0], testStreamAppName, testStreamAppId);
-    applicationConfig2 = buildStreamApplicationConfig(TEST_SYSTEM, inputKafkaTopic, PROCESSOR_IDS[1], testStreamAppName, testStreamAppId);
-    applicationConfig3 = buildStreamApplicationConfig(TEST_SYSTEM, inputKafkaTopic, PROCESSOR_IDS[2], testStreamAppName, testStreamAppId);
-
     // Create local application runners.
     applicationRunner1 = new LocalApplicationRunner(applicationConfig1);
     applicationRunner2 = new LocalApplicationRunner(applicationConfig2);
@@ -164,8 +173,8 @@ public class TestZkLocalApplicationRunner extends StandaloneIntegrationTestHarne
     }
   }
 
-  private ApplicationConfig buildStreamApplicationConfig(String systemName, String inputTopic,
-                                                         String processorId, String appName, String appId) {
+  private Map<String, String> buildStreamApplicationConfigMap(String systemName, String inputTopic,
+                                                              String appName, String appId) {
     Map<String, String> samzaContainerConfig = ImmutableMap.<String, String>builder()
         .put(TaskConfig.INPUT_STREAMS(), inputTopic)
         .put(JobConfig.JOB_DEFAULT_SYSTEM(), systemName)
@@ -174,17 +183,17 @@ public class TestZkLocalApplicationRunner extends StandaloneIntegrationTestHarne
         .put(JobConfig.SSP_GROUPER_FACTORY(), TEST_SSP_GROUPER_FACTORY)
         .put(TaskConfig.GROUPER_FACTORY(), TEST_TASK_GROUPER_FACTORY)
         .put(JobCoordinatorConfig.JOB_COORDINATOR_FACTORY, TEST_JOB_COORDINATOR_FACTORY)
-        .put(JobConfig.PROCESSOR_ID(), processorId)
         .put(ApplicationConfig.APP_NAME, appName)
         .put(ApplicationConfig.APP_ID, appId)
         .put(String.format("systems.%s.samza.factory", systemName), TEST_SYSTEM_FACTORY)
-        .put(JobConfig.JOB_NAME(), TEST_JOB_NAME)
+        .put(JobConfig.JOB_NAME(), appName)
+        .put(JobConfig.JOB_ID(), appId)
         .put(TaskConfigJava.TASK_SHUTDOWN_MS, TASK_SHUTDOWN_MS)
         .put(JobConfig.JOB_DEBOUNCE_TIME_MS(), JOB_DEBOUNCE_TIME_MS)
         .build();
     Map<String, String> applicationConfig = Maps.newHashMap(samzaContainerConfig);
     applicationConfig.putAll(StandaloneTestUtils.getKafkaSystemConfigs(systemName, bootstrapServers(), zkConnect(), null, StandaloneTestUtils.SerdeAlias.STRING, true));
-    return new ApplicationConfig(new MapConfig(applicationConfig));
+    return applicationConfig;
   }
 
   @Test
@@ -354,9 +363,14 @@ public class TestZkLocalApplicationRunner extends StandaloneIntegrationTestHarne
      * are for ZkJobCoordinator output(onNewJobModelConfirmed, onNewJobModelAvailable). Increasing DefaultDebounceTime to make sure that streamApplication dies & rejoins before expiry.
      */
     Map<String, String> debounceTimeConfig = ImmutableMap.of(JobConfig.JOB_DEBOUNCE_TIME_MS(), "40000");
+    Map<String, String> configMap = buildStreamApplicationConfigMap(TEST_SYSTEM, inputKafkaTopic, testStreamAppName, testStreamAppId);
+    configMap.put(JobConfig.JOB_DEBOUNCE_TIME_MS(), "40000");
+
+    configMap.put(JobConfig.PROCESSOR_ID(), PROCESSOR_IDS[0]);
+    Config applicationConfig1 = new MapConfig(configMap);
 
-    Config applicationConfig1 = new MapConfig(ImmutableList.of(buildStreamApplicationConfig(TEST_SYSTEM, inputKafkaTopic, PROCESSOR_IDS[0], testStreamAppName, testStreamAppId), debounceTimeConfig));
-    Config applicationConfig2 = new MapConfig(ImmutableList.of(buildStreamApplicationConfig(TEST_SYSTEM, inputKafkaTopic, PROCESSOR_IDS[1], testStreamAppName, testStreamAppId), debounceTimeConfig));
+    configMap.put(JobConfig.PROCESSOR_ID(), PROCESSOR_IDS[1]);
+    Config applicationConfig2 = new MapConfig(configMap);
 
     LocalApplicationRunner applicationRunner1 = new LocalApplicationRunner(applicationConfig1);
     LocalApplicationRunner applicationRunner2 = new LocalApplicationRunner(applicationConfig2);


[08/26] samza git commit: SAMZA-1394 add Fred Ji's PGP key for release purpose

Posted by xi...@apache.org.
SAMZA-1394 add Fred Ji's PGP key for release purpose

Author: Fred Ji <fj...@linkedin.com>

Reviewers: Xinyu Liu <xi...@apache.org>

Closes #273 from fredji97/master


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

Branch: refs/heads/0.14.0
Commit: 89989fdab8b4cd18f1d06f347d5eca2185f34880
Parents: ef6bb81
Author: Fred Ji <fj...@linkedin.com>
Authored: Wed Aug 16 09:53:13 2017 -0700
Committer: Xinyu Liu <xi...@xiliu-ld.linkedin.biz>
Committed: Wed Aug 16 09:53:13 2017 -0700

----------------------------------------------------------------------
 KEYS | 57 +++++++++++++++++++++++++++++++++++++++++++++++++++++++++
 1 file changed, 57 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/samza/blob/89989fda/KEYS
----------------------------------------------------------------------
diff --git a/KEYS b/KEYS
index c22172b..558f6e4 100644
--- a/KEYS
+++ b/KEYS
@@ -595,3 +595,60 @@ lsgwJyywRJ6sxTijA3tMSXjFSxVfa9isfeTAGNlsvwm/lsOPdNDpxpRE9T7VuRoa
 4T0llIDYLg==
 =66aV
 -----END PGP PUBLIC KEY BLOCK-----
+pub   4096R/A211312E 2017-08-15 [expires: 2021-08-15]
+uid       [ultimate] Fred Ji <ha...@gmail.com>
+sig 3        A211312E 2017-08-15  Fred Ji <ha...@gmail.com>
+sub   4096R/BCE71616 2017-08-15 [expires: 2021-08-15]
+sig          A211312E 2017-08-15  Fred Ji <ha...@gmail.com>
+
+-----BEGIN PGP PUBLIC KEY BLOCK-----
+
+mQINBFmTSN8BEACpkPAUsmaRBkgGvnU6E6Dnv9UzZHVHJaDctVO6xLi1wnbTPyaN
+tz9MJx3hyYuy/rKh/l3TWLPTvYlwO6PmaXz2DlMRvIWVdztdB87up8XNjXCeDrtv
+SbAezRnPelqx5gsyZ3tgtLOmxXmFmrpa3fqrnIvG22xf0N3+AYhOdEDGzHWiSAjF
+9RUiCWQHLPeQHOlUO7nhK90kkBYamSREJpggpzouwuHiew+MzcIji2U/vPStHmOd
+QVfQZcFnLO6Jp0FLCHxGtFFqhUYc2mgtUK/NExto5PEWHAdFuBr1FY3F5PbcZl+y
+DkWCVbWkfEi/UZ/L5fH5VlMrYynWSQKOmKgPVnYUGfQpU0nHUC8TQVDxrXQV8TKv
+6FV9TlW2rJbAjTCp76EGG7rcGFtgiZ1RkJf8WI7FHDD7VS2EgKmJCFoekA4J9pro
+EbHEtQohYzuMgTNmbNZN2vc8iMxHN+YDrlOxElDOjAtLl6OeBw16FBcp+mksKmHR
+3FXYFRM+S+WrB+jmiIwolbWpSsQ6whcqSx/P1N/yiX/6PjozkkhulGVcbh7gLhRF
+p0DCFs3MIXt7qKjgPEnEiKJ0zlyT2ODmTT2biYLsCwfCEQ92Fzg7PCrDevqMSxFP
+0JU5w4QtLTP6/oYSf1fS7f8iynI8v/pgQDFdGARcYJ5suBkVRST6aG5VCwARAQAB
+tB5GcmVkIEppIDxoYWlmZW5nLmppQGdtYWlsLmNvbT6JAj0EEwEKACcFAlmTSN8C
+GwMFCQeGH4AFCwkIBwMFFQoJCAsFFgIDAQACHgECF4AACgkQ7f2PmqIRMS7fCBAA
+gvJA7z0D7VIxin7WOQdBt7+C2Mt6bkYhEDSW7z1yYju8T8zg5N7nY5b9zMMtmATg
+f/cMuKP3Zu7Ztf0QOU9hHQ0NFZA5Ejx3KwKL3ozTmH8Zlq7PJy03ydSfYgbEyKbD
+LBBdx6hTn3P16qqofAfPA8/zaXWzuOtp5f7sIN0aPEiLUAL3RoplLrCCJGvJrUyl
+Qj7YKa6/nfJOMR2vGl1sU4vzVwjcO/uJ1ohqzYPE8+FgESzr6V6kQb0JB9fs9XXc
+9dThBHZx0L7WmTyMbX9sfYilnXQV9Gw+poc0i0UUM5UoS7lRq7EW7EdC6TTJzafE
+CKP6BxLvYgzHJJfsM0oBiddodooKjrkfLS0fHaBCPP5FpP5YLZhiyu1r9PvblCHV
+eEKjhGb11tWBcza0eLpl8VUc79p0R+hpkhYcJAQ6hnSQvd0+g5qP0MmGop3M+g3D
+wG0YoplWoIAyzmUXxekbDgswDYDYQtKgtwjxWXmkkN4Wj/Wzxoi/3xJNapehdGqZ
+M00w+Lni92U8wRMu5PMtvj8M6oNdCZw2aCpOw5+CdAyp9rKCvM3sCuj9y6qNdgSC
+8vEAN7bOg4f5JMia3bDHzQpfw4ZG7TdVHfNYJML7WA9n3PZ3TNpkHyc4Dh5Gxa8p
+2rsH8c+wreH0F1tIbly1lqcdtP3374LS35xp7TShAjW5Ag0EWZNI3wEQAMWLLWZV
+rFVMhtc0Pb/edJVzK7X+PCm10vjd5u+hz2PkOKjR1CKhcUrWdX1tbQ45wZWqy4Or
+K8kHLUTep22wGoKfqJ6X9elN0/yv3mIhQg1VJ/5EGPevBrx7h2ABQ/a4W2x2OGZB
+ncDSos8pmDYrSlqZ+kNDqJNQxa0P2feV2swJi3Ygj1DT8DaHQgQ/MNoAHDcUULCV
+cCQuS8dbDkEEs27rEJRG0cvI2+0cpqn36zZjWuFprg4qfrL0grkv+XkpgLzArsFU
+dND186as6ZrThlq0ZOmeadP6Z2d259X3OG4+wQVK2WrSo6OUIOZz1KQ+om3yQy81
+t5dYyNfOJ/KPCATHKpdBBR5HRUtgE8nNg3nn5eTCjQzD6nwNE6zuHXwhx03UlbQN
+tTwQRYU6ADG+ZybUuPZpTLRvd+5q5t5E4GZdbWaKoLz/j3Qe253UddyEX2QDNJjF
+Hvws6Qxc4mlQjLl4uLPMzo/pvIclD44PV3IMWyVq8dGiIBgyl7DKCLm1sqV/Kw7Y
+fWpZu1sRvM37PLZJWL+PQZJCacWZnz8i8vc3x3+BhEQosZrNKyA5s5IRE2ox0eOS
+9S1sOP7mNnYhlnyevuWyRD+0vxkVnHzqCMRLYkdBZEHlakFEWPbSKJ9cFBuf8zfZ
+KUlKZoZL2R2v5lNhYcyiAaS+Vz1w84D8lN6NABEBAAGJAiUEGAEKAA8FAlmTSN8C
+GwwFCQeGH4AACgkQ7f2PmqIRMS6DuhAAmfE5gIaPpYvOg9kU3OFRN7tk3ZJq3tjj
+uDXKyvfOi7rg2kLXuCiXtWT9B6pqNVZt+qi65RRHciNA8o7oTG6PLsykXSYrl9ok
+XtWPGq1yMkQS7fZpyBI+G/VJIR4fIQgWXIZgJvgvCdmLbXqlJ3mnOr/cP0MB1yXE
+5vpAWJmHGA9i9WeR+qmjUJ8zxm80ap/u2lPjtE955jS+w/gS9foL6Q+r7uKEhVlc
+gsR0sAtME/MZJ9DCcAcRdUw151vfctq5Ro7SwhRuxWc4E3QbFDoT7ivb9YeIx9AE
+fWbPFsRJPSY22jNNSZi5PFd2naLAuC+rQ10F1trGr+JnlVOU49YaLbVYl6HbbIde
+Sxlr57aIfkDoJsoIpXGq1NcBiOuCDhO/1ODHFWFY0b8LYzSrB4Frzr9ZKtHBnaQ7
+bqRSoSWbtv2DnDlvqEBnC6TImaSlHa/RUQVouKUga8SQ9Z0QDoNapfW7f5rt7IGs
+vaxQlMGlbMfCJB8Pw69dDx9eqKkjw8IoKItxFKMy28Mv43MgzMNDjjbPeNOLFV9M
+UyR9daQyCWQbu+UWn67OBUxJBnJGDjzcIoEKsFU+xt/GkExBXzL4JxiXFS+cv3dg
+P83WClBM5LS+ArY8Z81uX751Sh8JtNEGzdRS2jR3s+cBGsH37XdPceVCUTFLGL/d
+C+FsNOOomcE=
+=3dsB
+-----END PGP PUBLIC KEY BLOCK-----


[10/26] samza git commit: SAMZA-1397; log.debug loop to run only if debug is enabled

Posted by xi...@apache.org.
SAMZA-1397; log.debug loop to run only if debug is enabled

Author: Boris Shkolnik <bo...@apache.org>

Reviewers: Jagadish <jagadish@apache.org?

Closes #275 from sborya/LocalityLogDebug


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

Branch: refs/heads/0.14.0
Commit: 202af33bfe7d60ec9b2f90854abffde4bd55d2e6
Parents: 119e2fa
Author: Boris Shkolnik <bo...@apache.org>
Authored: Wed Aug 16 15:08:51 2017 -0700
Committer: Jagadish <ja...@apache.org>
Committed: Wed Aug 16 15:08:51 2017 -0700

----------------------------------------------------------------------
 .../main/java/org/apache/samza/container/LocalityManager.java  | 6 ++++--
 1 file changed, 4 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/samza/blob/202af33b/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 bafebcc..033002c 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
@@ -109,8 +109,10 @@ public class LocalityManager extends AbstractCoordinatorStreamManager {
     }
     containerToHostMapping = Collections.unmodifiableMap(allMappings);
 
-    for (Map.Entry<String, Map<String, String>> entry : containerToHostMapping.entrySet()) {
-      log.debug(String.format("Locality for container %s: %s", entry.getKey(), entry.getValue()));
+    if (log.isDebugEnabled()) {
+      for (Map.Entry<String, Map<String, String>> entry : containerToHostMapping.entrySet()) {
+        log.debug(String.format("Locality for container %s: %s", entry.getKey(), entry.getValue()));
+      }
     }
 
     return allMappings;


[06/26] samza git commit: SAMZA-1390; Update SamzaMonitorService to spawn deamon threads

Posted by xi...@apache.org.
SAMZA-1390; Update SamzaMonitorService to spawn deamon threads

Observed in LinkedIn production setup that samza-rest jvm process doesn’t stop after main
thread death(due to jetty server failures) since non-deamon threads spawned for
`SamzaMonitorService` are alive.

This affects samza-rest jvm process lifecycle management. To fix this, plugging
in ThreadFactory which sets Thread name format & marks them as daemon.

Author: Shanthoosh Venkataraman <sv...@linkedin.com>

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

Closes #270 from shanthoosh/make_samza_rest_non_daemon


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

Branch: refs/heads/0.14.0
Commit: a0aae5292c31c8bc260bb15c03396279a30f19ef
Parents: b1277d8
Author: Shanthoosh Venkataraman <sv...@linkedin.com>
Authored: Mon Aug 14 17:01:09 2017 -0700
Committer: Jagadish <ja...@apache.org>
Committed: Mon Aug 14 17:01:09 2017 -0700

----------------------------------------------------------------------
 .../java/org/apache/samza/rest/SamzaRestService.java  | 14 ++++++++++++--
 1 file changed, 12 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/samza/blob/a0aae529/samza-rest/src/main/java/org/apache/samza/rest/SamzaRestService.java
----------------------------------------------------------------------
diff --git a/samza-rest/src/main/java/org/apache/samza/rest/SamzaRestService.java b/samza-rest/src/main/java/org/apache/samza/rest/SamzaRestService.java
index de6febb..2f940e3 100644
--- a/samza-rest/src/main/java/org/apache/samza/rest/SamzaRestService.java
+++ b/samza-rest/src/main/java/org/apache/samza/rest/SamzaRestService.java
@@ -18,7 +18,9 @@
  */
 package org.apache.samza.rest;
 
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
 import java.util.Map;
+import java.util.concurrent.ThreadFactory;
 import joptsimple.OptionSet;
 import org.apache.samza.config.MapConfig;
 import org.apache.samza.config.MetricsConfig;
@@ -84,6 +86,7 @@ public class SamzaRestService {
    */
   public static void main(String[] args)
       throws Exception {
+    ScheduledExecutorSchedulingProvider schedulingProvider = null;
     try {
       SamzaRestConfig config = parseConfig(args);
       ReadableMetricsRegistry metricsRegistry = new MetricsRegistryMap();
@@ -99,8 +102,11 @@ public class SamzaRestService {
       restService.addServlet(container, "/*");
 
       // Schedule monitors to run
-      ScheduledExecutorService schedulingService = Executors.newScheduledThreadPool(1);
-      ScheduledExecutorSchedulingProvider schedulingProvider = new ScheduledExecutorSchedulingProvider(schedulingService);
+      ThreadFactory threadFactory = new ThreadFactoryBuilder().setDaemon(true)
+                                                              .setNameFormat("MonitorThread-%d")
+                                                              .build();
+      ScheduledExecutorService schedulingService = Executors.newScheduledThreadPool(1, threadFactory);
+      schedulingProvider = new ScheduledExecutorSchedulingProvider(schedulingService);
       SamzaMonitorService monitorService = new SamzaMonitorService(config,
                                                                    metricsRegistry,
                                                                    schedulingProvider);
@@ -110,6 +116,10 @@ public class SamzaRestService {
       monitorService.stop();
     } catch (Throwable t) {
       log.error("Exception in main.", t);
+    } finally {
+      if (schedulingProvider != null){
+        schedulingProvider.stop();
+      }
     }
   }
 


[11/26] samza git commit: SAMZA-1355 : Enable standalone integration tests conditionally in build.

Posted by xi...@apache.org.
SAMZA-1355 : Enable standalone integration tests conditionally in build.

It's run only when includeSamzaTest gradle project property is set.

Author: Shanthoosh Venkataraman <sv...@linkedin.com>

Reviewers: Xinyu Liu <xi...@apache.org>

Closes #237 from shanthoosh/disable_samza_test_build_with_a_flag


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

Branch: refs/heads/0.14.0
Commit: afe3bf8616edcb5e6d16a457ba54234607c6b25b
Parents: 202af33
Author: Shanthoosh Venkataraman <sv...@linkedin.com>
Authored: Wed Aug 16 16:25:49 2017 -0700
Committer: Xinyu Liu <xi...@xiliu-ld.linkedin.biz>
Committed: Wed Aug 16 16:25:49 2017 -0700

----------------------------------------------------------------------
 README.md                |  2 +-
 bin/check-all.sh         |  2 +-
 docs/contribute/tests.md |  2 +-
 settings.gradle          | 17 +++++++++++++++++
 4 files changed, 20 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/samza/blob/afe3bf86/README.md
----------------------------------------------------------------------
diff --git a/README.md b/README.md
index 4526ae8..cab1826 100644
--- a/README.md
+++ b/README.md
@@ -34,7 +34,7 @@ Samza builds with [Scala](http://www.scala-lang.org/) 2.10 or 2.11 and [YARN](ht
 
 ### Testing Samza
 
-To run all tests:
+To run all unit tests:
 
     ./gradlew clean test
 

http://git-wip-us.apache.org/repos/asf/samza/blob/afe3bf86/bin/check-all.sh
----------------------------------------------------------------------
diff --git a/bin/check-all.sh b/bin/check-all.sh
index 2f9f03c..ceb1e99 100755
--- a/bin/check-all.sh
+++ b/bin/check-all.sh
@@ -81,7 +81,7 @@ do
     for yarn_version in "${YARNs[@]}"
     do
       echo "------------- Running check task against JDK${jdk_number}/Scala ${scala_version}/YARN ${yarn_version}"
-      ${gradle_file} -PscalaVersion=${scala_version} -PyarnVersion=${yarn_version} -Dorg.gradle.java.home=${!i} clean check $@
+      ${gradle_file} -PscalaVersion=${scala_version} -PyarnVersion=${yarn_version} -Dorg.gradle.java.home=${!i} -PrunIntegrationTests clean check $@
       echo "------------- Finished running check task against JDK${jdk_number}/Scala ${scala_version}/YARN ${yarn_version}"
     done
   done

http://git-wip-us.apache.org/repos/asf/samza/blob/afe3bf86/docs/contribute/tests.md
----------------------------------------------------------------------
diff --git a/docs/contribute/tests.md b/docs/contribute/tests.md
index 9fe728c..048e3be 100644
--- a/docs/contribute/tests.md
+++ b/docs/contribute/tests.md
@@ -25,7 +25,7 @@ Samza's unit tests are written on top of [JUnit](http://junit.org/), and license
 
 To run all tests, and license checks:
 
-    ./gradlew clean check
+    ./gradlew clean check -PrunIntegrationTests
 
 To run a single test:
 

http://git-wip-us.apache.org/repos/asf/samza/blob/afe3bf86/settings.gradle
----------------------------------------------------------------------
diff --git a/settings.gradle b/settings.gradle
index a4eba94..603cd35 100644
--- a/settings.gradle
+++ b/settings.gradle
@@ -46,3 +46,20 @@ rootProject.children.each {
     it.name = it.name + "_" + scalaVersion
   }
 }
+
+/**
+ * Skips execution of all integration tests in project 'samza-test'.
+ * To run integration tests in samza-test: './gradlew clean build -PrunIntegrationTests'
+ */
+gradle.taskGraph.whenReady { taskGraph ->
+  taskGraph.getAllTasks().each { task ->
+    def project = task.getProject()
+    task.onlyIf {
+      /**
+       * Behaves as pass through filter for all tasks when `runIntegrationTests` property is turned on.
+       * Filters 'test' task of 'samza-test' project otherwise.
+       */
+      project.hasProperty("runIntegrationTests") || !(project.getName().contains("samza-test") && task.getName() == "test")
+    }
+  }
+}


[20/26] samza git commit: SAMZA-1404: Add warning in case of potential process starvation due to longer window method

Posted by xi...@apache.org.
SAMZA-1404: Add warning in case of potential process starvation due to longer window method

Currently, we use the average windowNs as the lower bound for window trigger time to determine if user needs to warned. We could potentially make this complicated by also including average commit ns and some delta to be more accurate.

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

Reviewers: Jagadish V <vj...@gmail.com>

Closes #281 from bharathkk/master


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

Branch: refs/heads/0.14.0
Commit: 3636be0316ba1ba53f961dacc5e865798119ab20
Parents: 3a13438
Author: Bharath Kumarasubramanian <bk...@linkedin.com>
Authored: Mon Aug 28 12:27:47 2017 -0700
Committer: Xinyu Liu <xi...@xiliu-ld1.linkedin.biz>
Committed: Mon Aug 28 12:27:47 2017 -0700

----------------------------------------------------------------------
 .../main/java/org/apache/samza/task/AsyncRunLoop.java | 14 ++++++++++++++
 1 file changed, 14 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/samza/blob/3636be03/samza-core/src/main/java/org/apache/samza/task/AsyncRunLoop.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/task/AsyncRunLoop.java b/samza-core/src/main/java/org/apache/samza/task/AsyncRunLoop.java
index e5c40df..478f109 100644
--- a/samza-core/src/main/java/org/apache/samza/task/AsyncRunLoop.java
+++ b/samza-core/src/main/java/org/apache/samza/task/AsyncRunLoop.java
@@ -443,6 +443,20 @@ public class AsyncRunLoop implements Runnable, Throttleable {
             long startTime = clock.nanoTime();
             task.window(coordinator);
             containerMetrics.windowNs().update(clock.nanoTime() - startTime);
+
+            // A window() that executes for more than task.window.ms, will starve the next process() call
+            // when the application has job.thread.pool.size > 1. This is due to prioritizing window() ahead of process()
+            // to guarantee window() will fire close to its trigger interval time.
+            // We warn the users if the average window execution time is greater than equals to window trigger interval.
+            long lowerBoundForWindowTriggerTimeInMs = TimeUnit.NANOSECONDS
+                .toMillis((long) containerMetrics.windowNs().getSnapshot().getAverage());
+            if (windowMs <= lowerBoundForWindowTriggerTimeInMs) {
+              log.warn(
+                  "window() call might potentially starve process calls."
+                      + " Consider setting task.window.ms > {} ms",
+                  lowerBoundForWindowTriggerTimeInMs);
+            }
+
             coordinatorRequests.update(coordinator);
 
             state.doneWindow();


[25/26] samza git commit: Merge branch 'master' into 0.14.0

Posted by xi...@apache.org.
Merge branch 'master' into 0.14.0


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

Branch: refs/heads/0.14.0
Commit: 1701ea84a2e029b3297687b5fc814998371b1a6f
Parents: 79200c7 fb39a51
Author: Xinyu Liu <xi...@xiliu-ld1.linkedin.biz>
Authored: Tue Sep 12 11:32:36 2017 -0700
Committer: Xinyu Liu <xi...@xiliu-ld1.linkedin.biz>
Committed: Tue Sep 12 11:32:36 2017 -0700

----------------------------------------------------------------------
 KEYS                                            |  57 +++
 NOTICE                                          |   1 +
 README.md                                       |   2 +-
 bin/check-all.sh                                |   2 +-
 build.gradle                                    |   4 +-
 docs/contribute/tests.md                        |   2 +-
 .../versioned/jobs/configuration-table.html     |  14 +-
 .../versioned/hello-samza-high-level-yarn.md    |   2 +-
 .../versioned/hello-samza-high-level-zk.md      |   2 +-
 .../versioned/samza-rest-getting-started.md     |   2 +-
 docs/startup/download/index.md                  |  17 +-
 docs/startup/hello-samza/versioned/index.md     |   2 +-
 docs/startup/preview/index.md                   |   2 +-
 .../main/java/org/apache/samza/AzureClient.java |  20 +-
 .../main/java/org/apache/samza/AzureConfig.java |  73 ---
 .../main/java/org/apache/samza/BlobUtils.java   | 280 ----------
 .../java/org/apache/samza/JobModelBundle.java   |  61 ---
 .../java/org/apache/samza/LeaseBlobManager.java |  98 ----
 .../java/org/apache/samza/ProcessorEntity.java  |  58 ---
 .../main/java/org/apache/samza/TableUtils.java  | 198 --------
 .../org/apache/samza/config/AzureConfig.java    |  68 +++
 .../coordinator/AzureCoordinationUtils.java     |  58 +++
 .../AzureCoordinationUtilsFactory.java          |  30 ++
 .../samza/coordinator/AzureJobCoordinator.java  | 509 +++++++++++++++++++
 .../coordinator/AzureJobCoordinatorFactory.java |  29 ++
 .../samza/coordinator/AzureLeaderElector.java   | 109 ++++
 .../org/apache/samza/coordinator/AzureLock.java | 100 ++++
 .../samza/coordinator/DistributedLock.java      |  39 ++
 .../samza/coordinator/data/BarrierState.java    |  27 +
 .../samza/coordinator/data/JobModelBundle.java  |  61 +++
 .../samza/coordinator/data/ProcessorEntity.java |  62 +++
 .../scheduler/HeartbeatScheduler.java           |  81 +++
 .../scheduler/JMVersionUpgradeScheduler.java    |  99 ++++
 .../LeaderBarrierCompleteScheduler.java         | 118 +++++
 .../scheduler/LeaderLivenessCheckScheduler.java | 120 +++++
 .../scheduler/LivenessCheckScheduler.java       | 108 ++++
 .../scheduler/RenewLeaseScheduler.java          |  79 +++
 .../scheduler/SchedulerStateChangeListener.java |  29 ++
 .../coordinator/scheduler/TaskScheduler.java    |  35 ++
 .../java/org/apache/samza/util/BlobUtils.java   | 284 +++++++++++
 .../org/apache/samza/util/LeaseBlobManager.java |  99 ++++
 .../java/org/apache/samza/util/TableUtils.java  | 205 ++++++++
 .../samza/config/JobCoordinatorConfig.java      |  21 +
 .../apache/samza/container/LocalityManager.java |   6 +-
 .../coordinator/CoordinationServiceFactory.java |  36 --
 .../samza/coordinator/CoordinationUtils.java    |  14 +-
 .../coordinator/CoordinationUtilsFactory.java   |  47 ++
 .../coordinator/DistributedLockWithState.java   |  42 ++
 .../samza/runtime/LocalApplicationRunner.java   |  91 ++--
 .../org/apache/samza/task/AsyncRunLoop.java     |  14 +
 .../samza/zk/ZkBarrierForVersionUpgrade.java    |   2 +-
 .../org/apache/samza/zk/ZkControllerImpl.java   |  19 +-
 .../samza/zk/ZkCoordinationServiceFactory.java  |  89 ----
 .../apache/samza/zk/ZkCoordinationUtils.java    |  26 +-
 .../samza/zk/ZkCoordinationUtilsFactory.java    |  89 ++++
 .../org/apache/samza/zk/ZkDistributedLock.java  | 117 +++++
 .../samza/zk/ZkJobCoordinatorFactory.java       |  23 +-
 .../org/apache/samza/zk/ZkLeaderElector.java    |   2 +-
 .../org/apache/samza/zk/ZkProcessorLatch.java   |  23 +-
 .../main/java/org/apache/samza/zk/ZkUtils.java  |  41 +-
 .../apache/samza/checkpoint/OffsetManager.scala |  50 +-
 .../apache/samza/container/TaskInstance.scala   |   4 +-
 .../runtime/TestApplicationRunnerMain.java      |   2 +-
 .../runtime/TestLocalApplicationRunner.java     | 185 ++++---
 .../org/apache/samza/task/TestAsyncRunLoop.java |  24 +-
 .../apache/samza/zk/TestZkLeaderElector.java    |   7 +-
 .../org/apache/samza/zk/TestZkNamespace.java    |   8 +-
 .../apache/samza/zk/TestZkProcessorLatch.java   |   2 +-
 .../java/org/apache/samza/zk/TestZkUtils.java   |  57 ++-
 .../samza/checkpoint/TestOffsetManager.scala    |  58 ++-
 .../samza/config/TestJobCoordinatorConfig.java  |  58 +++
 .../samza/container/TestTaskInstance.scala      |  62 ++-
 .../org/apache/samza/rest/SamzaRestService.java |  14 +-
 .../processor/TestZkStreamProcessorSession.java |   3 +-
 .../processor/TestZkLocalApplicationRunner.java |  60 ++-
 .../test/integration/TestStatefulTask.scala     |  79 ++-
 settings.gradle                                 |  17 +
 77 files changed, 3444 insertions(+), 1194 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/samza/blob/1701ea84/samza-core/src/main/java/org/apache/samza/runtime/LocalApplicationRunner.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/samza/blob/1701ea84/samza-core/src/main/java/org/apache/samza/task/AsyncRunLoop.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/samza/blob/1701ea84/samza-core/src/main/scala/org/apache/samza/container/TaskInstance.scala
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/samza/blob/1701ea84/samza-core/src/test/java/org/apache/samza/runtime/TestLocalApplicationRunner.java
----------------------------------------------------------------------
diff --cc samza-core/src/test/java/org/apache/samza/runtime/TestLocalApplicationRunner.java
index 4be4e73,f9c1252..5b2c661
--- a/samza-core/src/test/java/org/apache/samza/runtime/TestLocalApplicationRunner.java
+++ b/samza-core/src/test/java/org/apache/samza/runtime/TestLocalApplicationRunner.java
@@@ -19,7 -19,13 +19,14 @@@
  
  package org.apache.samza.runtime;
  
+ import com.google.common.collect.ImmutableList;
+ import java.lang.reflect.Field;
+ import java.util.Collections;
+ import java.util.HashMap;
+ import java.util.List;
+ import java.util.Map;
+ import java.util.stream.Collectors;
 +import java.util.Set;
  import org.apache.samza.application.StreamApplication;
  import org.apache.samza.config.ApplicationConfig;
  import org.apache.samza.config.JobConfig;
@@@ -343,8 -324,73 +325,77 @@@ public class TestLocalApplicationRunne
      assertEquals(spy.status(app), ApplicationStatus.UnsuccessfulFinish);
    }
  
 +  public static Set<StreamProcessor> getProcessors(LocalApplicationRunner runner) {
 +    return runner.getProcessors();
 +  }
 +
+   /**
+    * A test case to verify if the plan results in different hash if there is change in topological sort order.
+    * Note: the overall JOB PLAN remains the same outside the scope of intermediate streams the sake of these test cases.
+    */
+   @Test
+   public void testPlanIdWithShuffledStreamSpecs() {
+     List<StreamSpec> streamSpecs = ImmutableList.of(new StreamSpec("test-stream-1", "stream-1", "testStream"),
+         new StreamSpec("test-stream-2", "stream-2", "testStream"),
+         new StreamSpec("test-stream-3", "stream-3", "testStream"));
+     String planIdBeforeShuffle = getExecutionPlanId(streamSpecs);
+ 
+     List<StreamSpec> shuffledStreamSpecs = ImmutableList.of(new StreamSpec("test-stream-2", "stream-2", "testStream"),
+         new StreamSpec("test-stream-1", "stream-1", "testStream"),
+         new StreamSpec("test-stream-3", "stream-3", "testStream"));
+ 
+ 
+     assertFalse("Expected both of the latch ids to be different",
+         planIdBeforeShuffle.equals(getExecutionPlanId(shuffledStreamSpecs)));
+   }
+ 
+   /**
+    * A test case to verify if the plan results in same hash in case of same plan.
+    * Note: the overall JOB PLAN remains the same outside the scope of intermediate streams the sake of these test cases.
+    */
+   @Test
+   public void testGeneratePlanIdWithSameStreamSpecs() {
+     List<StreamSpec> streamSpecs = ImmutableList.of(new StreamSpec("test-stream-1", "stream-1", "testStream"),
+         new StreamSpec("test-stream-2", "stream-2", "testStream"),
+         new StreamSpec("test-stream-3", "stream-3", "testStream"));
+     String planIdForFirstAttempt = getExecutionPlanId(streamSpecs);
+     String planIdForSecondAttempt = getExecutionPlanId(streamSpecs);
+ 
+     assertEquals("Expected latch ids to match!", "1447946713", planIdForFirstAttempt);
+     assertEquals("Expected latch ids to match for the second attempt!", planIdForFirstAttempt, planIdForSecondAttempt);
+   }
+ 
+   /**
+    * A test case to verify plan results in different hash in case of different intermediate stream.
+    * Note: the overall JOB PLAN remains the same outside the scope of intermediate streams the sake of these test cases.
+    */
+   @Test
+   public void testGeneratePlanIdWithDifferentStreamSpecs() {
+     List<StreamSpec> streamSpecs = ImmutableList.of(new StreamSpec("test-stream-1", "stream-1", "testStream"),
+         new StreamSpec("test-stream-2", "stream-2", "testStream"),
+         new StreamSpec("test-stream-3", "stream-3", "testStream"));
+     String planIdBeforeShuffle = getExecutionPlanId(streamSpecs);
+ 
+     List<StreamSpec> updatedStreamSpecs = ImmutableList.of(new StreamSpec("test-stream-1", "stream-1", "testStream"),
+         new StreamSpec("test-stream-4", "stream-4", "testStream"),
+         new StreamSpec("test-stream-3", "stream-3", "testStream"));
+ 
+ 
+     assertFalse("Expected both of the latch ids to be different",
+         planIdBeforeShuffle.equals(getExecutionPlanId(updatedStreamSpecs)));
+   }
+ 
+   private String getExecutionPlanId(List<StreamSpec> updatedStreamSpecs) {
+     String intermediateStreamJson =
+         updatedStreamSpecs.stream().map(this::streamSpecToJson).collect(Collectors.joining(","));
+ 
+     int planId = String.format(PLAN_JSON, intermediateStreamJson).hashCode();
+ 
+     return String.valueOf(planId);
+   }
+ 
+   private String streamSpecToJson(StreamSpec streamSpec) {
+     return String.format(STREAM_SPEC_JSON_FORMAT, streamSpec.getId(), streamSpec.getId(), streamSpec.getSystemName(),
+         streamSpec.getPhysicalName());
+   }
  }

http://git-wip-us.apache.org/repos/asf/samza/blob/1701ea84/samza-core/src/test/java/org/apache/samza/task/TestAsyncRunLoop.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/samza/blob/1701ea84/samza-core/src/test/scala/org/apache/samza/checkpoint/TestOffsetManager.scala
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/samza/blob/1701ea84/samza-core/src/test/scala/org/apache/samza/container/TestTaskInstance.scala
----------------------------------------------------------------------
diff --cc samza-core/src/test/scala/org/apache/samza/container/TestTaskInstance.scala
index dcb06d3,4958a57..81f3ed1
--- a/samza-core/src/test/scala/org/apache/samza/container/TestTaskInstance.scala
+++ b/samza-core/src/test/scala/org/apache/samza/container/TestTaskInstance.scala
@@@ -19,34 -19,16 +19,32 @@@
  
  package org.apache.samza.container
  
 +
 +import java.util
 +import java.util
 +import java.util.Collections
  import java.util.concurrent.ConcurrentHashMap
 +import com.google.common.collect.Multimap
 +import org.apache.samza.SamzaException
+ 
  import org.apache.samza.Partition
 +import org.apache.samza.checkpoint.OffsetManager
 +import org.apache.samza.config.Config
 +import org.apache.samza.config.MapConfig
 +import org.apache.samza.control.ControlMessageUtils
 +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.metrics.Counter
 +import org.apache.samza.metrics.Metric
 +import org.apache.samza.metrics.MetricsRegistryMap
+ 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.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.storage.TaskStorageManager
  import org.apache.samza.system.SystemStreamMetadata.SystemStreamPartitionMetadata
- import org.apache.samza.system.SystemStreamPartition
+ import org.apache.samza.system._
  import org.apache.samza.system.chooser.RoundRobinChooser
  import org.apache.samza.task._
  import org.junit.Assert._
@@@ -365,34 -350,47 +366,77 @@@ class TestTaskInstance 
    }
  
    @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]).asInstanceOf[StreamTask],
+       taskName,
+       new MapConfig,
+       new TaskInstanceMetrics,
+       null,
+       Mockito.mock(classOf[SystemConsumers]),
+       collector,
+       Mockito.mock(classOf[SamzaContainerContext]),
+       offsetManager,
+       storageManager,
+       systemStreamPartitions = Set(systemStreamPartition))
+ 
+     taskInstance.commit
+ 
+     // We must first get a snapshot of the checkpoint so it doesn't change while we flush. SAMZA-1384
+     mockOrder.verify(offsetManager).buildCheckpoint(taskName)
+     // 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
+     // 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()
+     // Finally, checkpoint the inputs with the snapshotted checkpoint captured at the beginning of commit
+     mockOrder.verify(offsetManager).writeCheckpoint(taskName, checkpoint)
+   }
++
++  @Test
 +  def testBuildInputToTasks = {
 +    val system: String = "test-system"
 +    val stream0: String = "test-stream-0"
 +    val stream1: String = "test-stream-1"
 +
 +    val ssp0: SystemStreamPartition = new SystemStreamPartition(system, stream0, new Partition(0))
 +    val ssp1: SystemStreamPartition = new SystemStreamPartition(system, stream0, new Partition(1))
 +    val ssp2: SystemStreamPartition = new SystemStreamPartition(system, stream1, new Partition(0))
 +
 +    val task0: TaskName = new TaskName("Task 0")
 +    val task1: TaskName = new TaskName("Task 1")
 +    val ssps: util.Set[SystemStreamPartition] = new util.HashSet[SystemStreamPartition]
 +    ssps.add(ssp0)
 +    ssps.add(ssp2)
 +    val tm0: TaskModel = new TaskModel(task0, ssps, new Partition(0))
 +    val cm0: ContainerModel = new ContainerModel("c0", 0, Collections.singletonMap(task0, tm0))
 +    val tm1: TaskModel = new TaskModel(task1, Collections.singleton(ssp1), new Partition(1))
 +    val cm1: ContainerModel = new ContainerModel("c1", 1, Collections.singletonMap(task1, tm1))
 +
 +    val cms: util.Map[String, ContainerModel] = new util.HashMap[String, ContainerModel]
 +    cms.put(cm0.getProcessorId, cm0)
 +    cms.put(cm1.getProcessorId, cm1)
 +
 +    val jobModel: JobModel = new JobModel(new MapConfig, cms, null)
 +    val streamToTasks: Multimap[SystemStream, String] = TaskInstance.buildInputToTasks(jobModel)
 +    assertEquals(streamToTasks.get(ssp0.getSystemStream).size, 2)
 +    assertEquals(streamToTasks.get(ssp2.getSystemStream).size, 1)
 +  }
  }
  
  class MockSystemAdmin extends SystemAdmin {


[19/26] samza git commit: SAMZA-1408 update master doc to use 0.14.0-SNAPSHOT after 0.13.1 release

Posted by xi...@apache.org.
SAMZA-1408 update master doc to use 0.14.0-SNAPSHOT after 0.13.1 release

Author: Fred Ji <ha...@gmail.com>

Reviewers: Yi Pan <ni...@gmail.com>

Closes #285 from fredji97/master_doc and squashes the following commits:

58cc775 [Fred Ji] SAMZA-1408 keep the doc unchanged for minor release in master and update PR based on feedback
fc41efe [Fred Ji] SAMZA-1408 update master doc to use 0.14.0-SNAPSHOT after 0.13.1 release


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

Branch: refs/heads/0.14.0
Commit: 3a134386f18c2ac2b957191f258022a817e21ef0
Parents: 2a4ccfd
Author: Fred Ji <ha...@gmail.com>
Authored: Fri Aug 25 17:21:21 2017 -0700
Committer: Yi Pan (Data Infrastructure) <ni...@gmail.com>
Committed: Fri Aug 25 17:21:21 2017 -0700

----------------------------------------------------------------------
 .../versioned/hello-samza-high-level-yarn.md       |  2 +-
 .../versioned/hello-samza-high-level-zk.md         |  2 +-
 .../versioned/samza-rest-getting-started.md        |  2 +-
 docs/startup/download/index.md                     | 17 +++++++++--------
 docs/startup/hello-samza/versioned/index.md        |  2 +-
 5 files changed, 13 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/samza/blob/3a134386/docs/learn/tutorials/versioned/hello-samza-high-level-yarn.md
----------------------------------------------------------------------
diff --git a/docs/learn/tutorials/versioned/hello-samza-high-level-yarn.md b/docs/learn/tutorials/versioned/hello-samza-high-level-yarn.md
index 5108099..fbd1e05 100644
--- a/docs/learn/tutorials/versioned/hello-samza-high-level-yarn.md
+++ b/docs/learn/tutorials/versioned/hello-samza-high-level-yarn.md
@@ -63,7 +63,7 @@ Then, you can continue w/ the following command in hello-samza project:
 {% highlight bash %}
 mvn clean package
 mkdir -p deploy/samza
-tar -xvf ./target/hello-samza-0.13.1-SNAPSHOT-dist.tar.gz -C deploy/samza
+tar -xvf ./target/hello-samza-0.14.0-SNAPSHOT-dist.tar.gz -C deploy/samza
 {% endhighlight %}
 
 ### Run a Samza Application

http://git-wip-us.apache.org/repos/asf/samza/blob/3a134386/docs/learn/tutorials/versioned/hello-samza-high-level-zk.md
----------------------------------------------------------------------
diff --git a/docs/learn/tutorials/versioned/hello-samza-high-level-zk.md b/docs/learn/tutorials/versioned/hello-samza-high-level-zk.md
index bc0da6e..ebe5595 100644
--- a/docs/learn/tutorials/versioned/hello-samza-high-level-zk.md
+++ b/docs/learn/tutorials/versioned/hello-samza-high-level-zk.md
@@ -59,7 +59,7 @@ With the environment setup complete, let us move on to building the hello-samza
 {% highlight bash %}
 mvn clean package
 mkdir -p deploy/samza
-tar -xvf ./target/hello-samza-0.13.1-SNAPSHOT-dist.tar.gz -C deploy/samza
+tar -xvf ./target/hello-samza-0.14.0-SNAPSHOT-dist.tar.gz -C deploy/samza
 {% endhighlight %}
 
 We are now all set to deploy the application locally.

http://git-wip-us.apache.org/repos/asf/samza/blob/3a134386/docs/learn/tutorials/versioned/samza-rest-getting-started.md
----------------------------------------------------------------------
diff --git a/docs/learn/tutorials/versioned/samza-rest-getting-started.md b/docs/learn/tutorials/versioned/samza-rest-getting-started.md
index 1459893..b889dd3 100644
--- a/docs/learn/tutorials/versioned/samza-rest-getting-started.md
+++ b/docs/learn/tutorials/versioned/samza-rest-getting-started.md
@@ -48,7 +48,7 @@ Run the following commands:
 {% highlight bash %}
 cd samza-rest/build/distributions/
 mkdir -p deploy/samza-rest
-tar -xvf ./samza-rest-0.13.1-SNAPSHOT.tgz -C deploy/samza-rest
+tar -xvf ./samza-rest-0.14.0-SNAPSHOT.tgz -C deploy/samza-rest
 {% endhighlight %}
 
 #### Configure the Installations Path

http://git-wip-us.apache.org/repos/asf/samza/blob/3a134386/docs/startup/download/index.md
----------------------------------------------------------------------
diff --git a/docs/startup/download/index.md b/docs/startup/download/index.md
index 48715c7..b39f90f 100644
--- a/docs/startup/download/index.md
+++ b/docs/startup/download/index.md
@@ -29,6 +29,7 @@ Starting from 2016, Samza will begin requiring JDK8 or higher. Please see [this
 
 ### Source Releases
 
+ * [samza-sources-0.13.1.tgz](http://www.apache.org/dyn/closer.lua/samza/0.13.1)
  * [samza-sources-0.13.0.tgz](http://www.apache.org/dyn/closer.lua/samza/0.13.0)
  * [samza-sources-0.12.0.tgz](http://www.apache.org/dyn/closer.lua/samza/0.12.0)
  * [samza-sources-0.11.0.tgz](http://www.apache.org/dyn/closer.lua/samza/0.11.0)
@@ -51,12 +52,12 @@ A Maven-based Samza project can pull in all required dependencies Samza dependen
 <dependency>
   <groupId>org.apache.samza</groupId>
   <artifactId>samza-api</artifactId>
-  <version>0.13.0</version>
+  <version>0.13.1</version>
 </dependency>
 <dependency>
   <groupId>org.apache.samza</groupId>
   <artifactId>samza-core_2.11</artifactId>
-  <version>0.13.0</version>
+  <version>0.13.1</version>
   <scope>runtime</scope>
 </dependency>
 <dependency>
@@ -64,37 +65,37 @@ A Maven-based Samza project can pull in all required dependencies Samza dependen
   <artifactId>samza-shell</artifactId>
   <classifier>dist</classifier>
   <type>tgz</type>
-  <version>0.13.0</version>
+  <version>0.13.1</version>
   <scope>runtime</scope>
 </dependency>
 <dependency>
   <groupId>org.apache.samza</groupId>
   <artifactId>samza-yarn_2.11</artifactId>
-  <version>0.13.0</version>
+  <version>0.13.1</version>
   <scope>runtime</scope>
 </dependency>
 <dependency>
   <groupId>org.apache.samza</groupId>
   <artifactId>samza-kv_2.11</artifactId>
-  <version>0.13.0</version>
+  <version>0.13.1</version>
   <scope>runtime</scope>
 </dependency>
 <dependency>
   <groupId>org.apache.samza</groupId>
   <artifactId>samza-kv-rocksdb_2.11</artifactId>
-  <version>0.13.0</version>
+  <version>0.13.1</version>
   <scope>runtime</scope>
 </dependency>
 <dependency>
   <groupId>org.apache.samza</groupId>
   <artifactId>samza-kv-inmemory_2.11</artifactId>
-  <version>0.13.0</version>
+  <version>0.13.1</version>
   <scope>runtime</scope>
 </dependency>
 <dependency>
   <groupId>org.apache.samza</groupId>
   <artifactId>samza-kafka_2.11</artifactId>
-  <version>0.13.0</version>
+  <version>0.13.1</version>
   <scope>runtime</scope>
 </dependency>
 {% endhighlight %}

http://git-wip-us.apache.org/repos/asf/samza/blob/3a134386/docs/startup/hello-samza/versioned/index.md
----------------------------------------------------------------------
diff --git a/docs/startup/hello-samza/versioned/index.md b/docs/startup/hello-samza/versioned/index.md
index aab0ff5..e94e2a5 100644
--- a/docs/startup/hello-samza/versioned/index.md
+++ b/docs/startup/hello-samza/versioned/index.md
@@ -61,7 +61,7 @@ Then, you can continue w/ the following command in hello-samza project:
 {% highlight bash %}
 mvn clean package
 mkdir -p deploy/samza
-tar -xvf ./target/hello-samza-0.13.1-SNAPSHOT-dist.tar.gz -C deploy/samza
+tar -xvf ./target/hello-samza-0.14.0-SNAPSHOT-dist.tar.gz -C deploy/samza
 {% endhighlight %}
 
 ### Run a Samza Job


[04/26] samza git commit: SAMZA-1389: Fix ZkProcessorLatch await(timeout, TimeUnit) api.

Posted by xi...@apache.org.
SAMZA-1389: Fix ZkProcessorLatch await(timeout, TimeUnit) api.

Use passed in timeUnit value for zkClient.waitUnitExists method rather than hardcoding with  `TimeUnit.MILLISECONDS`.

Author: Shanthoosh Venkataraman <sv...@linkedin.com>

Reviewers: Boris Shkolnik <bo...@apache.org>,Fred Ji <fr...@yahoo.com>,Jagadish <jv...@linkedin.com>

Closes #268 from shanthoosh/fix_zklatch_impl


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

Branch: refs/heads/0.14.0
Commit: 46b3601f1a3bbe792338f160f8283ff32b65547a
Parents: 06702af
Author: Shanthoosh Venkataraman <sv...@linkedin.com>
Authored: Fri Aug 11 11:51:32 2017 -0700
Committer: Jacob Maes <jm...@linkedin.com>
Committed: Fri Aug 11 11:51:32 2017 -0700

----------------------------------------------------------------------
 .../src/main/java/org/apache/samza/zk/ZkProcessorLatch.java      | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/samza/blob/46b3601f/samza-core/src/main/java/org/apache/samza/zk/ZkProcessorLatch.java
----------------------------------------------------------------------
diff --git a/samza-core/src/main/java/org/apache/samza/zk/ZkProcessorLatch.java b/samza-core/src/main/java/org/apache/samza/zk/ZkProcessorLatch.java
index ddc8976..decdd7d 100644
--- a/samza-core/src/main/java/org/apache/samza/zk/ZkProcessorLatch.java
+++ b/samza-core/src/main/java/org/apache/samza/zk/ZkProcessorLatch.java
@@ -53,8 +53,8 @@ public class ZkProcessorLatch implements Latch {
   }
 
   @Override
-  public void await(long timeout, TimeUnit tu) {
-    zkUtils.getZkClient().waitUntilExists(targetPath, TimeUnit.MILLISECONDS, timeout);
+  public void await(long timeout, TimeUnit timeUnit) {
+    zkUtils.getZkClient().waitUntilExists(targetPath, timeUnit, timeout);
   }
 
   @Override


[03/26] samza git commit: SAMZA-1387: Unable to Start Samza App Because Regex Check

Posted by xi...@apache.org.
SAMZA-1387: Unable to Start Samza App Because Regex Check

Author: Jacob Maes <jm...@linkedin.com>

Reviewers: Fred Ji <fr...@yahoo.com>

Closes #266 from jmakes/samza-1387


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

Branch: refs/heads/0.14.0
Commit: 06702af8fda1d016ae55461c404b55b84b20ffd2
Parents: 8f7f567
Author: Jacob Maes <jm...@linkedin.com>
Authored: Fri Aug 11 09:28:20 2017 -0700
Committer: Jacob Maes <jm...@linkedin.com>
Committed: Fri Aug 11 09:28:20 2017 -0700

----------------------------------------------------------------------
 .../samza/system/kafka/KafkaSystemAdmin.scala    |  7 ++++---
 .../system/kafka/TestKafkaSystemAdminJava.java   | 19 +++++++++++++++++++
 2 files changed, 23 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/samza/blob/06702af8/samza-kafka/src/main/scala/org/apache/samza/system/kafka/KafkaSystemAdmin.scala
----------------------------------------------------------------------
diff --git a/samza-kafka/src/main/scala/org/apache/samza/system/kafka/KafkaSystemAdmin.scala b/samza-kafka/src/main/scala/org/apache/samza/system/kafka/KafkaSystemAdmin.scala
index af77d5b..1e59b61 100644
--- a/samza-kafka/src/main/scala/org/apache/samza/system/kafka/KafkaSystemAdmin.scala
+++ b/samza-kafka/src/main/scala/org/apache/samza/system/kafka/KafkaSystemAdmin.scala
@@ -38,8 +38,9 @@ import scala.collection.JavaConverters._
 
 object KafkaSystemAdmin extends Logging {
   // Use a dummy string for the stream id. The physical name and partition count are all that matter for changelog creation, so the dummy string should not be used.
-  // We cannot use the topic name, as it may include special chars which are not allowed in stream IDs. See SAMZA-1317
+  // We cannot use the topic name, as it may include special chars which are not allowed in stream IDs. See SAMZA-1317 and 1387
   val CHANGELOG_STREAMID = "unused-temp-changelog-stream-id"
+  val COORDINATOR_STREAMID = "unused-temp-coordinator-stream-id"
 
   /**
    * A helper method that takes oldest, newest, and upcoming offsets for each
@@ -331,7 +332,7 @@ class KafkaSystemAdmin(
   override def createCoordinatorStream(streamName: String) {
     info("Attempting to create coordinator stream %s." format streamName)
 
-    val streamSpec = new KafkaStreamSpec(streamName, streamName, systemName, 1, coordinatorStreamReplicationFactor, coordinatorStreamProperties)
+    val streamSpec = new KafkaStreamSpec(COORDINATOR_STREAMID, streamName, systemName, 1, coordinatorStreamReplicationFactor, coordinatorStreamProperties)
 
     if (createStream(streamSpec)) {
       info("Created coordinator stream %s." format streamName)
@@ -496,7 +497,7 @@ class KafkaSystemAdmin(
   class KafkaChangelogException(s: String, t: Throwable) extends SamzaException(s, t) {
     def this(s: String) = this(s, null)
   }
-  
+
   override def createChangelogStream(topicName: String, numKafkaChangelogPartitions: Int) = {
     val topicMeta = topicMetaInformation.getOrElse(topicName, throw new KafkaChangelogException("Unable to find topic information for topic " + topicName))
     val spec = new KafkaStreamSpec(CHANGELOG_STREAMID, topicName, systemName, numKafkaChangelogPartitions, topicMeta.replicationFactor, topicMeta.kafkaProps)

http://git-wip-us.apache.org/repos/asf/samza/blob/06702af8/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 ce59b40..33c4017 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
@@ -51,6 +51,25 @@ public class TestKafkaSystemAdminJava extends TestKafkaSystemAdmin {
   }
 
   @Test
+  public void testCreateCoordinatorStreamDelegatesToCreateStream_specialCharsInTopicName() {
+    final String STREAM = "test.Coord_inator.Stream";
+
+    SystemAdmin admin = Mockito.spy(createSystemAdmin());
+    StreamSpec spec = new StreamSpec(KafkaSystemAdmin.CHANGELOG_STREAMID(), STREAM, SYSTEM());
+    admin.createCoordinatorStream(STREAM);
+    admin.validateStream(spec);
+
+    ArgumentCaptor<StreamSpec> specCaptor = ArgumentCaptor.forClass(StreamSpec.class);
+    Mockito.verify(admin).createStream(specCaptor.capture());
+
+    StreamSpec internalSpec = specCaptor.getValue();
+    assertTrue(internalSpec instanceof KafkaStreamSpec);  // KafkaStreamSpec is used to carry replication factor
+    assertEquals(KafkaSystemAdmin.COORDINATOR_STREAMID(), internalSpec.getId());
+    assertEquals(SYSTEM(), internalSpec.getSystemName());
+    assertEquals(STREAM, internalSpec.getPhysicalName());
+  }
+
+  @Test
   public void testCreateChangelogStreamDelegatesToCreateStream() {
     final String STREAM = "testChangeLogStream";
     final int PARTITIONS = 12;


[02/26] samza git commit: SAMZA-1384: Race condition with async commit affects checkpoint correctness

Posted by xi...@apache.org.
SAMZA-1384: Race condition with async commit affects checkpoint correctness

Author: Jacob Maes <jm...@linkedin.com>

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

Closes #263 from jmakes/samza-1384


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

Branch: refs/heads/0.14.0
Commit: 8f7f56744c2824426962e1ca4f382b1877e4a9b0
Parents: 4030423
Author: Jacob Maes <jm...@linkedin.com>
Authored: Thu Aug 10 15:44:42 2017 -0700
Committer: Jacob Maes <jm...@linkedin.com>
Committed: Thu Aug 10 15:44:42 2017 -0700

----------------------------------------------------------------------
 .../apache/samza/checkpoint/OffsetManager.scala | 50 +++++++++-----
 .../apache/samza/container/TaskInstance.scala   |  4 +-
 .../org/apache/samza/task/TestAsyncRunLoop.java | 24 ++++---
 .../samza/checkpoint/TestOffsetManager.scala    | 58 ++++++++++++++--
 .../samza/container/TestTaskInstance.scala      | 69 +++++++++++++++-----
 5 files changed, 154 insertions(+), 51 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/samza/blob/8f7f5674/samza-core/src/main/scala/org/apache/samza/checkpoint/OffsetManager.scala
----------------------------------------------------------------------
diff --git a/samza-core/src/main/scala/org/apache/samza/checkpoint/OffsetManager.scala b/samza-core/src/main/scala/org/apache/samza/checkpoint/OffsetManager.scala
index 783340a..8c739d4 100644
--- a/samza-core/src/main/scala/org/apache/samza/checkpoint/OffsetManager.scala
+++ b/samza-core/src/main/scala/org/apache/samza/checkpoint/OffsetManager.scala
@@ -20,20 +20,20 @@
 package org.apache.samza.checkpoint
 
 
+
+import java.util.HashMap
 import java.util.concurrent.ConcurrentHashMap
 
-import org.apache.samza.system.SystemStream
-import org.apache.samza.system.SystemStreamPartition
-import org.apache.samza.system.SystemStreamMetadata
-import org.apache.samza.system.SystemStreamMetadata.OffsetType
 import org.apache.samza.SamzaException
-import scala.collection.JavaConverters._
-import org.apache.samza.util.Logging
 import org.apache.samza.config.Config
 import org.apache.samza.config.StreamConfig.Config2Stream
 import org.apache.samza.config.SystemConfig.Config2System
-import org.apache.samza.system.SystemAdmin
 import org.apache.samza.container.TaskName
+import org.apache.samza.system.SystemStreamMetadata.OffsetType
+import org.apache.samza.system.{SystemAdmin, SystemStream, SystemStreamMetadata, SystemStreamPartition}
+import org.apache.samza.util.Logging
+
+import scala.collection.JavaConverters._
 import scala.collection._
 
 /**
@@ -217,25 +217,42 @@ class OffsetManager(
   }
 
   /**
-   * Checkpoint all offsets for a given TaskName using the CheckpointManager.
-   */
-  def checkpoint(taskName: TaskName) {
+    * Gets a snapshot of all the current offsets for the specified task. This is useful to
+    * ensure there are no concurrent updates to the offsets between when this method is
+    * invoked and the corresponding call to [[OffsetManager.writeCheckpoint()]]
+    */
+  def buildCheckpoint(taskName: TaskName): Checkpoint = {
     if (checkpointManager != null || checkpointListeners.nonEmpty) {
-      debug("Checkpointing offsets for taskName %s." format taskName)
+      debug("Getting checkpoint offsets for taskName %s." format taskName)
 
-      val sspsForTaskName = systemStreamPartitions.getOrElse(taskName, throw new SamzaException("No such SystemStreamPartition set " + taskName + " registered for this checkpointmanager")).toSet
+      val sspsForTaskName = systemStreamPartitions.getOrElse(taskName, throw new SamzaException("No SSPs registered for task: " + taskName)).toSet
       val sspToOffsets = lastProcessedOffsets.get(taskName)
-      val partitionOffsets = if(sspToOffsets != null) {
+
+      val partitionOffsets = if (sspToOffsets != null) {
+        // Filter the offsets in case the task model changed since the last checkpoint was written.
         sspToOffsets.asScala.filterKeys(sspsForTaskName.contains)
       } else {
         warn(taskName + " is not found... ")
         Map[SystemStreamPartition, String]()
       }
 
-      val checkpoint = new Checkpoint(partitionOffsets.asJava)
+      new Checkpoint(new HashMap(partitionOffsets.asJava)) // Copy into new Map to prevent mutation
+    } else {
+      debug("Returning null checkpoint for taskName %s because no checkpoint manager/callback is defined." format taskName)
+      null
+    }
+  }
+
+  /**
+    * Write the specified checkpoint for the given task.
+    */
+  def writeCheckpoint(taskName: TaskName, checkpoint: Checkpoint) {
+    if (checkpoint != null && (checkpointManager != null || checkpointListeners.nonEmpty)) {
+      debug("Writing checkpoint for taskName %s with offsets %s." format (taskName, checkpoint))
 
       if(checkpointManager != null) {
         checkpointManager.writeCheckpoint(taskName, checkpoint)
+        val sspToOffsets = checkpoint.getOffsets
         if(sspToOffsets != null) {
           sspToOffsets.asScala.foreach {
             case (ssp, cp) => offsetManagerMetrics.checkpointedOffsets.get(ssp).set(cp)
@@ -244,15 +261,12 @@ class OffsetManager(
       }
 
       // invoke checkpoint listeners
-      //partitionOffsets.groupBy(_._1.getSystem).foreach {
-      partitionOffsets.groupBy { case (ssp, _) => ssp.getSystem }.foreach {
+      checkpoint.getOffsets.asScala.groupBy { case (ssp, _) => ssp.getSystem }.foreach {
         case (systemName:String, offsets: Map[SystemStreamPartition, String]) => {
           // Option is empty if there is no checkpointListener for this systemName
           checkpointListeners.get(systemName).foreach(_.onCheckpoint(offsets.asJava))
         }
       }
-    } else {
-      debug("Skipping checkpointing for taskName %s because no checkpoint manager/callback is defined." format taskName)
     }
   }
 

http://git-wip-us.apache.org/repos/asf/samza/blob/8f7f5674/samza-core/src/main/scala/org/apache/samza/container/TaskInstance.scala
----------------------------------------------------------------------
diff --git a/samza-core/src/main/scala/org/apache/samza/container/TaskInstance.scala b/samza-core/src/main/scala/org/apache/samza/container/TaskInstance.scala
index 84e993b..65fefda 100644
--- a/samza-core/src/main/scala/org/apache/samza/container/TaskInstance.scala
+++ b/samza-core/src/main/scala/org/apache/samza/container/TaskInstance.scala
@@ -206,6 +206,8 @@ class TaskInstance(
   def commit {
     metrics.commits.inc
 
+    val checkpoint = offsetManager.buildCheckpoint(taskName)
+
     trace("Flushing producers for taskName: %s" format taskName)
 
     collector.flush
@@ -218,7 +220,7 @@ class TaskInstance(
 
     trace("Checkpointing offsets for taskName: %s" format taskName)
 
-    offsetManager.checkpoint(taskName)
+    offsetManager.writeCheckpoint(taskName, checkpoint)
   }
 
   def shutdownTask {

http://git-wip-us.apache.org/repos/asf/samza/blob/8f7f5674/samza-core/src/test/java/org/apache/samza/task/TestAsyncRunLoop.java
----------------------------------------------------------------------
diff --git a/samza-core/src/test/java/org/apache/samza/task/TestAsyncRunLoop.java b/samza-core/src/test/java/org/apache/samza/task/TestAsyncRunLoop.java
index 1afc26a..6694f26 100644
--- a/samza-core/src/test/java/org/apache/samza/task/TestAsyncRunLoop.java
+++ b/samza-core/src/test/java/org/apache/samza/task/TestAsyncRunLoop.java
@@ -50,8 +50,7 @@ import scala.Option;
 import scala.collection.JavaConverters;
 
 import static org.junit.Assert.assertEquals;
-import static org.mockito.Matchers.anyLong;
-import static org.mockito.Matchers.anyObject;
+import static org.mockito.Matchers.*;
 import static org.mockito.Mockito.atLeastOnce;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.never;
@@ -368,8 +367,10 @@ public class TestAsyncRunLoop {
     task0ProcessedMessagesLatch.await();
     task1ProcessedMessagesLatch.await();
 
-    verify(offsetManager).checkpoint(taskName0);
-    verify(offsetManager, never()).checkpoint(taskName1);
+    verify(offsetManager).buildCheckpoint(taskName0);
+    verify(offsetManager).writeCheckpoint(taskName0, any());
+    verify(offsetManager, never()).buildCheckpoint(taskName1);
+    verify(offsetManager, never()).writeCheckpoint(taskName1, any());
   }
 
   //@Test
@@ -398,8 +399,10 @@ public class TestAsyncRunLoop {
     task0ProcessedMessagesLatch.await();
     task1ProcessedMessagesLatch.await();
 
-    verify(offsetManager).checkpoint(taskName0);
-    verify(offsetManager).checkpoint(taskName1);
+    verify(offsetManager).buildCheckpoint(taskName0);
+    verify(offsetManager).writeCheckpoint(taskName0, any());
+    verify(offsetManager).buildCheckpoint(taskName1);
+    verify(offsetManager).writeCheckpoint(taskName1, any());
   }
 
   //@Test
@@ -552,8 +555,10 @@ public class TestAsyncRunLoop {
     task0ProcessedMessagesLatch.await();
     task1ProcessedMessagesLatch.await();
 
-    verify(offsetManager).checkpoint(taskName0);
-    verify(offsetManager).checkpoint(taskName1);
+    verify(offsetManager).buildCheckpoint(taskName0);
+    verify(offsetManager).writeCheckpoint(taskName0, any());
+    verify(offsetManager).buildCheckpoint(taskName1);
+    verify(offsetManager).writeCheckpoint(taskName1, any());
   }
 
   // TODO: Add assertions.
@@ -641,7 +646,8 @@ public class TestAsyncRunLoop {
           secondMsgCompletionLatch.countDown();
           // OffsetManager.update with firstMsg offset, task.commit has happened when second message callback has not completed.
           verify(offsetManager).update(taskName0, firstMsg.getSystemStreamPartition(), firstMsg.getOffset());
-          verify(offsetManager, atLeastOnce()).checkpoint(taskName0);
+          verify(offsetManager, atLeastOnce()).buildCheckpoint(taskName0);
+          verify(offsetManager, atLeastOnce()).writeCheckpoint(taskName0, any());
         }
       } catch (Exception e) {
         e.printStackTrace();

http://git-wip-us.apache.org/repos/asf/samza/blob/8f7f5674/samza-core/src/test/scala/org/apache/samza/checkpoint/TestOffsetManager.scala
----------------------------------------------------------------------
diff --git a/samza-core/src/test/scala/org/apache/samza/checkpoint/TestOffsetManager.scala b/samza-core/src/test/scala/org/apache/samza/checkpoint/TestOffsetManager.scala
index abfc63f..54a08f6 100644
--- a/samza-core/src/test/scala/org/apache/samza/checkpoint/TestOffsetManager.scala
+++ b/samza-core/src/test/scala/org/apache/samza/checkpoint/TestOffsetManager.scala
@@ -82,7 +82,7 @@ class TestOffsetManager {
     assertEquals("46", offsetManager.getStartingOffset(taskName, systemStreamPartition).get)
     // Should not update null offset
     offsetManager.update(taskName, systemStreamPartition, null)
-    offsetManager.checkpoint(taskName)
+    checkpoint(offsetManager, taskName)
     val expectedCheckpoint = new Checkpoint(Map(systemStreamPartition -> "47").asJava)
     assertEquals(expectedCheckpoint, checkpointManager.readLastCheckpoint(taskName))
   }
@@ -102,14 +102,14 @@ class TestOffsetManager {
     offsetManager.register(taskName, Set(systemStreamPartition))
     offsetManager.start
     // Should get offset 45 back from the checkpoint manager, which is last processed, and system admin should return 46 as starting offset.
-    offsetManager.checkpoint(taskName)
+    checkpoint(offsetManager, taskName)
     assertEquals("45", offsetManager.offsetManagerMetrics.checkpointedOffsets.get(systemStreamPartition).getValue)
     offsetManager.update(taskName, systemStreamPartition, "46")
     offsetManager.update(taskName, systemStreamPartition, "47")
-    offsetManager.checkpoint(taskName)
+    checkpoint(offsetManager, taskName)
     assertEquals("47", offsetManager.offsetManagerMetrics.checkpointedOffsets.get(systemStreamPartition).getValue)
     offsetManager.update(taskName, systemStreamPartition, "48")
-    offsetManager.checkpoint(taskName)
+    checkpoint(offsetManager, taskName)
     assertEquals("48", offsetManager.offsetManagerMetrics.checkpointedOffsets.get(systemStreamPartition).getValue)
   }
 
@@ -269,7 +269,7 @@ class TestOffsetManager {
 
     offsetManager.start
     // Should get offset 45 back from the checkpoint manager, which is last processed, and system admin should return 46 as starting offset.
-    offsetManager.checkpoint(taskName)
+    checkpoint(offsetManager, taskName)
     assertEquals("45", offsetManager.offsetManagerMetrics.checkpointedOffsets.get(systemStreamPartition).getValue)
     assertEquals("100", offsetManager.offsetManagerMetrics.checkpointedOffsets.get(systemStreamPartition2).getValue)
     assertEquals("45", consumer.recentCheckpoint.get(systemStreamPartition))
@@ -278,7 +278,7 @@ class TestOffsetManager {
 
     offsetManager.update(taskName, systemStreamPartition, "46")
     offsetManager.update(taskName, systemStreamPartition, "47")
-    offsetManager.checkpoint(taskName)
+    checkpoint(offsetManager, taskName)
     assertEquals("47", offsetManager.offsetManagerMetrics.checkpointedOffsets.get(systemStreamPartition).getValue)
     assertEquals("100", offsetManager.offsetManagerMetrics.checkpointedOffsets.get(systemStreamPartition2).getValue)
     assertEquals("47", consumer.recentCheckpoint.get(systemStreamPartition))
@@ -286,7 +286,7 @@ class TestOffsetManager {
 
     offsetManager.update(taskName, systemStreamPartition, "48")
     offsetManager.update(taskName, systemStreamPartition2, "101")
-    offsetManager.checkpoint(taskName)
+    checkpoint(offsetManager, taskName)
     assertEquals("48", offsetManager.offsetManagerMetrics.checkpointedOffsets.get(systemStreamPartition).getValue)
     assertEquals("101", offsetManager.offsetManagerMetrics.checkpointedOffsets.get(systemStreamPartition2).getValue)
     assertEquals("48", consumer.recentCheckpoint.get(systemStreamPartition))
@@ -294,6 +294,50 @@ class TestOffsetManager {
     offsetManager.stop
   }
 
+  /**
+    * If task.max.concurrency > 1 and task.async.commit == true, a task could update its offsets at the same time as
+    * TaskInstance.commit(). This makes it possible to checkpoint offsets which did not successfully flush.
+    *
+    * This is prevented by using separate methods to get a checkpoint and write that checkpoint. See SAMZA-1384
+    */
+  @Test
+  def testConcurrentCheckpointAndUpdate{
+    val taskName = new TaskName("c")
+    val systemStream = new SystemStream("test-system", "test-stream")
+    val partition = new Partition(0)
+    val systemStreamPartition = new SystemStreamPartition(systemStream, partition)
+    val testStreamMetadata = new SystemStreamMetadata(systemStream.getStream, Map(partition -> new SystemStreamPartitionMetadata("0", "1", "2")).asJava)
+    val systemStreamMetadata = Map(systemStream -> testStreamMetadata)
+    val checkpointManager = getCheckpointManager(systemStreamPartition, taskName)
+    val systemAdmins = Map("test-system" -> getSystemAdmin)
+    val offsetManager = OffsetManager(systemStreamMetadata, new MapConfig, checkpointManager, systemAdmins, Map(), new OffsetManagerMetrics)
+    offsetManager.register(taskName, Set(systemStreamPartition))
+    offsetManager.start
+
+    // Should get offset 45 back from the checkpoint manager, which is last processed, and system admin should return 46 as starting offset.
+    checkpoint(offsetManager, taskName)
+    assertEquals("45", offsetManager.offsetManagerMetrics.checkpointedOffsets.get(systemStreamPartition).getValue)
+
+    offsetManager.update(taskName, systemStreamPartition, "46")
+    // Get checkpoint snapshot like we do at the beginning of TaskInstance.commit()
+    val checkpoint46 = offsetManager.buildCheckpoint(taskName)
+    offsetManager.update(taskName, systemStreamPartition, "47") // Offset updated before checkpoint
+    offsetManager.writeCheckpoint(taskName, checkpoint46)
+    assertEquals(Some("47"), offsetManager.getLastProcessedOffset(taskName, systemStreamPartition))
+    assertEquals("46", offsetManager.offsetManagerMetrics.checkpointedOffsets.get(systemStreamPartition).getValue)
+
+    // Now write the checkpoint for the latest offset
+    val checkpoint47 = offsetManager.buildCheckpoint(taskName)
+    offsetManager.writeCheckpoint(taskName, checkpoint47)
+    assertEquals(Some("47"), offsetManager.getLastProcessedOffset(taskName, systemStreamPartition))
+    assertEquals("47", offsetManager.offsetManagerMetrics.checkpointedOffsets.get(systemStreamPartition).getValue)
+  }
+
+  // Utility method to create and write checkpoint in one statement
+  def checkpoint(offsetManager: OffsetManager, taskName: TaskName): Unit = {
+    offsetManager.writeCheckpoint(taskName, offsetManager.buildCheckpoint(taskName))
+  }
+
   class SystemConsumerWithCheckpointCallback extends SystemConsumer with CheckpointListener{
     var recentCheckpoint: java.util.Map[SystemStreamPartition, String] = java.util.Collections.emptyMap[SystemStreamPartition, String]
     override def start() {}

http://git-wip-us.apache.org/repos/asf/samza/blob/8f7f5674/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 40974a6..4958a57 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,31 +20,25 @@
 package org.apache.samza.container
 
 import java.util.concurrent.ConcurrentHashMap
-import org.apache.samza.SamzaException
+
 import org.apache.samza.Partition
-import org.apache.samza.checkpoint.OffsetManager
-import org.apache.samza.config.Config
-import org.apache.samza.config.MapConfig
-import org.apache.samza.metrics.Counter
-import org.apache.samza.metrics.Metric
-import org.apache.samza.metrics.MetricsRegistryMap
+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.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.storage.TaskStorageManager
 import org.apache.samza.system.SystemStreamMetadata.SystemStreamPartitionMetadata
-import org.apache.samza.system.SystemStreamPartition
+import org.apache.samza.system._
 import org.apache.samza.system.chooser.RoundRobinChooser
 import org.apache.samza.task._
 import org.junit.Assert._
 import org.junit.Test
+import org.mockito.Matchers._
+import org.mockito.Mockito
+import org.mockito.Mockito._
 import org.scalatest.Assertions.intercept
+
 import scala.collection.JavaConverters._
-import org.apache.samza.system.SystemAdmin
 import scala.collection.mutable.ListBuffer
 
 class TestTaskInstance {
@@ -354,6 +348,49 @@ class TestTaskInstance {
     val expected = List(envelope1, envelope2, envelope4)
     assertEquals(expected, result.toList)
   }
+
+  @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]).asInstanceOf[StreamTask],
+      taskName,
+      new MapConfig,
+      new TaskInstanceMetrics,
+      null,
+      Mockito.mock(classOf[SystemConsumers]),
+      collector,
+      Mockito.mock(classOf[SamzaContainerContext]),
+      offsetManager,
+      storageManager,
+      systemStreamPartitions = Set(systemStreamPartition))
+
+    taskInstance.commit
+
+    // We must first get a snapshot of the checkpoint so it doesn't change while we flush. SAMZA-1384
+    mockOrder.verify(offsetManager).buildCheckpoint(taskName)
+    // 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
+    // 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()
+    // Finally, checkpoint the inputs with the snapshotted checkpoint captured at the beginning of commit
+    mockOrder.verify(offsetManager).writeCheckpoint(taskName, checkpoint)
+  }
 }
 
 class MockSystemAdmin extends SystemAdmin {