Sanil15 commented on a change in pull request #1227: SAMZA-2404: Container 
Placement Handler for dispatching container placement messages between 
metastore and JobCoordinator
URL: https://github.com/apache/samza/pull/1227#discussion_r353470304
 
 

 ##########
 File path: 
samza-core/src/main/java/org/apache/samza/clustermanager/container/placement/ContainerPlacementHandler.java
 ##########
 @@ -0,0 +1,287 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.clustermanager.container.placement;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import org.apache.commons.lang3.ArrayUtils;
+import org.apache.samza.SamzaException;
+import org.apache.samza.clustermanager.ContainerProcessManager;
+import org.apache.samza.container.placement.ContainerPlacementMessage;
+import org.apache.samza.container.placement.ContainerPlacementRequestMessage;
+import org.apache.samza.container.placement.ContainerPlacementResponseMessage;
+import 
org.apache.samza.coordinator.metadatastore.NamespaceAwareCoordinatorStreamStore;
+import org.apache.samza.metadatastore.MetadataStore;
+import org.codehaus.jackson.map.ObjectMapper;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Stateless handler that dispatches {@link ContainerPlacementRequestMessage} 
read from {@link MetadataStore} to Job Coordinator
+ * and writes responses {@link 
org.apache.samza.container.placement.ContainerPlacementResponseMessage} from 
Job Coordinator
+ * to Metastore.
+ *
+ * {@link ContainerPlacementRequestMessage} are written under namespace {@code 
REQUEST_STORE_NAMESPACE}
+ * {@link ContainerPlacementResponseMessage} are written under namespace 
{@code RESPONSE_STORE_NAMESPACE}
+ */
+public class ContainerPlacementHandler implements Runnable {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(ContainerPlacementHandler.class);
+
+  private static final Integer VERSION = 1;
+  public static final String REQUEST_STORE_NAMESPACE = 
"samza-request-place-container-v" + VERSION;
+  public static final String RESPONSE_STORE_NAMESPACE = 
"samza-response-place-container-v" + VERSION;
+
+  private final NamespaceAwareCoordinatorStreamStore requestStore;
+  private final NamespaceAwareCoordinatorStreamStore responseStore;
+  /**
+   * {@link ContainerProcessManager} needs to intercept container placement 
actions between ContainerPlacementHandler and
+   * {@link org.apache.samza.clustermanager.ContainerManager} to avoid cyclic 
dependency between
+   * {@link org.apache.samza.clustermanager.ContainerManager} and {@link 
org.apache.samza.clustermanager.ContainerAllocator}
+   */
+  private final Optional<ContainerProcessManager> containerProcessManager;
+  private final Optional<Integer> containerPlacementHandlerSleepMs;
+
+  private final ObjectMapper objectMapper = 
ContainerPlacementMessageObjectMapper.getObjectMapper();
+
+  /**
+   * State that controls the lifecycle of the ContainerPlacementHandler thread
+   */
+  private volatile boolean isRunning;
+
+  private ContainerPlacementHandler(MetadataStore metadataStore, 
Optional<ContainerProcessManager> manager,
+      Optional<Integer> containerPlacementHandlerSleepMs) {
+    Preconditions.checkNotNull(metadataStore, "MetadataStore cannot be null");
+    if (manager.isPresent()) {
+      Preconditions.checkNotNull(manager.get(), "ContainerProcessManager 
cannot be null");
+      Preconditions.checkState(containerPlacementHandlerSleepMs.isPresent());
+    }
+    this.containerProcessManager = manager;
+    this.requestStore = new 
NamespaceAwareCoordinatorStreamStore(metadataStore, REQUEST_STORE_NAMESPACE);
+    this.responseStore = new 
NamespaceAwareCoordinatorStreamStore(metadataStore, RESPONSE_STORE_NAMESPACE);
+    this.requestStore.init();
+    this.responseStore.init();
+    this.containerPlacementHandlerSleepMs = containerPlacementHandlerSleepMs;
+    this.isRunning = true;
+  }
+
+  public ContainerPlacementHandler(MetadataStore metadataStore) {
+    this(metadataStore, Optional.empty(), Optional.empty());
+  }
+
+  public ContainerPlacementHandler(MetadataStore metadataStore, 
ContainerProcessManager manager, Integer containerPlacementHandlerSleepMs) {
+    this(metadataStore, Optional.of(manager), 
Optional.of(containerPlacementHandlerSleepMs));
+  }
+
+  @Override
+  public void run() {
+    while (isRunning && containerProcessManager.isPresent()) {
+      try {
+        for (ContainerPlacementRequestMessage message : 
readAllContainerPlacementRequestMessages()) {
+          // We do not need to dispatch ContainerPlacementResponseMessage 
because they are written from JobCoordinator
+          // in response to a Container Placement Action
+          LOG.info("Received a container placement message {}", message);
+          
containerProcessManager.get().registerContainerPlacementAction(message);
+        }
+        Thread.sleep(containerPlacementHandlerSleepMs.get());
+      } catch (InterruptedException e) {
+        LOG.warn("Got InterruptedException in ContainerPlacementHandler 
thread.", e);
+        Thread.currentThread().interrupt();
+      } catch (Exception e) {
+        LOG.error(
+            "Got unknown Exception while registering ContainerPlacement 
actions in ContainerPlacementHandler thread.",
+            e);
+      }
+    }
+  }
+
+  /**
+   * Writes a {@link ContainerPlacementRequestMessage} to the underlying 
metastore. This method should be used by external controllers
+   * to issue a request to JobCoordinator
+   * @param message container placement request
+   */
+  public void 
writeContainerPlacementRequestMessage(ContainerPlacementRequestMessage message) 
{
+    Preconditions.checkState(isRunning, "Underlying metadata store not 
available");
+    Preconditions.checkNotNull(message);
+    try {
+      requestStore.put(message.getProcessorId(), 
objectMapper.writeValueAsBytes(message));
+    } catch (Exception ex) {
+      throw new SamzaException(
+          String.format("ContainerPlacementRequestMessage might have been not 
written to metastore %s", message), ex);
+    }
+  }
+
+  /**
+   * Writes a {@link ContainerPlacementResponseMessage} to the underlying 
metastore. This method should be used by external controllers
+   * to issue a request to JobCoordinator
+   * @param message
+   */
+  public void 
writeContainerPlacementResponseMessage(ContainerPlacementResponseMessage 
message) {
 
 Review comment:
   Good catch, I will change the visibility to default!

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

Reply via email to