You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@samza.apache.org by GitBox <gi...@apache.org> on 2020/12/08 23:28:52 UTC

[GitHub] [samza] mynameborat commented on a change in pull request #1446: SAMZA-2605: Make Standby Container Requests Rack Aware

mynameborat commented on a change in pull request #1446:
URL: https://github.com/apache/samza/pull/1446#discussion_r538881117



##########
File path: samza-core/src/main/java/org/apache/samza/clustermanager/FaultDomainManager.java
##########
@@ -0,0 +1,71 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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;
+
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * This interface gets fault domain information of different nodes from the cluster manager (Yarn/Kubernetes/etc.).
+ *  It also provides other functionality like exposing all the available fault domains, checking if two hosts belong to
+ *  the same fault domain, and getting the valid fault domains that a standby container can be placed on.
+ */
+public interface FaultDomainManager {

Review comment:
       The interface exposes too many functionalities that are either not necessary upstream or conflating.
    
   1. `computeNodeToFaultDomainMap`
   2. `getAllowedFaultDomainsForSchedulingContainer`
   3. `getNodeToFaultDomainMap`
   
   The notion of allowed fault domain for the container is not determined by the cluster manager's fault domain manager rather `StandbyContainerManager`. Hence [2] shouldn't be exposed as a functionality.
   [3] seems redundant given you have `getFaultDomainOfNode`
   [1] is completely internal to the manager and shouldn't be exposed.

##########
File path: samza-core/src/main/java/org/apache/samza/clustermanager/ContainerProcessManager.java
##########
@@ -114,6 +114,11 @@
    */
   private final ClusterResourceManager clusterResourceManager;
 
+  /**
+   * An interface to get information about nodes and the fault domains they reside on.
+   */
+  private final FaultDomainManager faultDomainManager;

Review comment:
       can be removed since its only used to wire the `ContainerManager`.
   

##########
File path: samza-core/src/main/java/org/apache/samza/clustermanager/FaultDomainManager.java
##########
@@ -0,0 +1,71 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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;
+
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * This interface gets fault domain information of different nodes from the cluster manager (Yarn/Kubernetes/etc.).
+ *  It also provides other functionality like exposing all the available fault domains, checking if two hosts belong to
+ *  the same fault domain, and getting the valid fault domains that a standby container can be placed on.
+ */
+public interface FaultDomainManager {
+
+  /**
+   * This method returns all the fault domain values in a cluster for RUNNING nodes.
+   * @return a set of {@link FaultDomain}s
+   */
+  Set<FaultDomain> getAllFaultDomains();
+
+  /**
+   * This method returns the fault domain a particular node resides on.
+   * @param host the host
+   * @return the {@link FaultDomain}
+   */
+  FaultDomain getFaultDomainOfNode(String host);
+
+  /**
+   * This method checks if the two hostnames provided reside on the same fault domain.
+   * @param host1 hostname
+   * @param host2 hostname
+   * @return true if the hosts exist on the same fault domain
+   */
+  boolean checkHostsOnSameFaultDomain(String host1, String host2);

Review comment:
       Given nodes can belong to multiple fault domain, what does this functionality offer? Overlapping fault domain = `true` or absolute 1:1 mapping as `true`.
   
   Why do you need this functionality in the first place? I thought we were using the set difference approach to get the fault domain of active and then find the set difference between available fault domain and active fault domain and use that for requesting resources.

##########
File path: samza-core/src/main/java/org/apache/samza/clustermanager/ContainerProcessManager.java
##########
@@ -183,6 +191,7 @@ public ContainerProcessManager(Config config, SamzaApplicationState state, Metri
       SamzaApplicationState state,
       MetricsRegistryMap registry,
       ClusterResourceManager resourceManager,
+      FaultDomainManager faultDomainManager,

Review comment:
       nit: add new parameters to the end of the signature

##########
File path: samza-yarn/src/main/java/org/apache/samza/job/yarn/RackManagerFactory.java
##########
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.job.yarn;
+
+import org.apache.samza.clustermanager.FaultDomainManager;
+import org.apache.samza.clustermanager.FaultDomainManagerFactory;
+
+/**
+ * A factory to build a {@link RackManager}.
+ */
+public class RackManagerFactory implements FaultDomainManagerFactory {

Review comment:
       Refer to comment above on this being `YarnFaultDomainManager`. Guess that will change this as well.

##########
File path: samza-core/src/main/java/org/apache/samza/clustermanager/FaultDomainManager.java
##########
@@ -0,0 +1,71 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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;
+
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * This interface gets fault domain information of different nodes from the cluster manager (Yarn/Kubernetes/etc.).
+ *  It also provides other functionality like exposing all the available fault domains, checking if two hosts belong to
+ *  the same fault domain, and getting the valid fault domains that a standby container can be placed on.
+ */
+public interface FaultDomainManager {
+
+  /**
+   * This method returns all the fault domain values in a cluster for RUNNING nodes.
+   * @return a set of {@link FaultDomain}s
+   */
+  Set<FaultDomain> getAllFaultDomains();
+
+  /**
+   * This method returns the fault domain a particular node resides on.
+   * @param host the host
+   * @return the {@link FaultDomain}
+   */
+  FaultDomain getFaultDomainOfNode(String host);

Review comment:
       Should be `Set<FaultDomain>` given a node can belong to multiple fault domain

##########
File path: samza-core/src/main/java/org/apache/samza/clustermanager/FaultDomainManagerFactory.java
##########
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.clustermanager;
+
+/**
+ * A factory to build a {@link FaultDomainManager}.
+ */
+public interface FaultDomainManagerFactory {
+
+  public FaultDomainManager getFaultDomainManager();

Review comment:
       Should we take `Config` and `MetricsRegistry` as bare minimum parameters to enable us add metrics or use cut off switch within or configure the potential behavior as it evolves.

##########
File path: samza-yarn/src/main/java/org/apache/samza/job/yarn/RackManager.java
##########
@@ -0,0 +1,116 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.samza.job.yarn;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.hadoop.yarn.api.records.NodeReport;
+import org.apache.hadoop.yarn.api.records.NodeState;
+import org.apache.hadoop.yarn.client.api.impl.YarnClientImpl;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.samza.clustermanager.FaultDomain;
+import org.apache.samza.clustermanager.FaultDomainManager;
+import org.apache.samza.clustermanager.FaultDomainType;
+
+public class RackManager implements FaultDomainManager {

Review comment:
       I suppose this needs to be `YarnFaultDomainManager`.
   `FaultDomainManager` by itself is capable of handling all types of fault domain for a given cluster type. Hence  `RackManager` by itself doesn’t seem suitable abstraction rather there should be one for YARN which is capable of reporting all sort of fault domains supported by YARN

##########
File path: samza-core/src/main/java/org/apache/samza/clustermanager/FaultDomainManager.java
##########
@@ -0,0 +1,71 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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;
+
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * This interface gets fault domain information of different nodes from the cluster manager (Yarn/Kubernetes/etc.).
+ *  It also provides other functionality like exposing all the available fault domains, checking if two hosts belong to
+ *  the same fault domain, and getting the valid fault domains that a standby container can be placed on.
+ */
+public interface FaultDomainManager {
+
+  /**
+   * This method returns all the fault domain values in a cluster for RUNNING nodes.
+   * @return a set of {@link FaultDomain}s
+   */
+  Set<FaultDomain> getAllFaultDomains();
+
+  /**
+   * This method returns the fault domain a particular node resides on.
+   * @param host the host
+   * @return the {@link FaultDomain}
+   */
+  FaultDomain getFaultDomainOfNode(String host);

Review comment:
       +1 use consistent terminologies. 




----------------------------------------------------------------
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:
users@infra.apache.org