Browse Source

YARN-6353. Clean up OrderingPolicy javadoc (Daniel Templeton via Varun Saxena)

(cherry picked from commit 35034653d02ac8156338d7267e5975d2d66272d5)
Varun Saxena 8 years ago
parent
commit
e2cabb4d21

+ 61 - 44
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/policy/OrderingPolicy.java

@@ -19,15 +19,13 @@
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler.policy;
 
 import java.util.*;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.*;
 
 
 /**
  * OrderingPolicy is used by the scheduler to order SchedulableEntities for
- * container assignment and preemption
+ * container assignment and preemption.
+ * @param <S> the type of {@link SchedulableEntity} that will be compared
  */
 public interface OrderingPolicy<S extends SchedulableEntity> {
   /*
@@ -35,80 +33,99 @@ public interface OrderingPolicy<S extends SchedulableEntity> {
    * synchronization of all use of the SchedulableEntity Collection and
    * Iterators for correctness and to avoid concurrent modification issues
    */
-  
+
   /**
-   * Get the collection of SchedulableEntities which are managed by this
-   * OrderingPolicy - should include processes returned by the Assignment and
-   * Preemption iterator with no guarantees regarding order
+   * Get the collection of {@link SchedulableEntity} Objects which are managed
+   * by this OrderingPolicy - should include processes returned by the
+   * Assignment and Preemption iterator with no guarantees regarding order.
+   * @return a collection of {@link SchedulableEntity} objects
    */
   public Collection<S> getSchedulableEntities();
-  
+
   /**
-   * Return an iterator over the collection of SchedulableEntities which orders
-   * them for container assignment
+   * Return an iterator over the collection of {@link SchedulableEntity}
+   * objects which orders them for container assignment.
+   * @return an iterator over the collection of {@link SchedulableEntity}
+   * objects
    */
   public Iterator<S> getAssignmentIterator();
-  
+
   /**
-   * Return an iterator over the collection of SchedulableEntities which orders
-   * them for preemption
+   * Return an iterator over the collection of {@link SchedulableEntity}
+   * objects which orders them for preemption.
+   * @return an iterator over the collection of {@link SchedulableEntity}
    */
   public Iterator<S> getPreemptionIterator();
-  
+
   /**
-   * Add a SchedulableEntity to be managed for allocation and preemption 
-   * ordering
+   * Add a {@link SchedulableEntity} to be managed for allocation and preemption
+   * ordering.
+   * @param s the {@link SchedulableEntity} to add
    */
   public void addSchedulableEntity(S s);
-  
+
   /**
-   * Remove a SchedulableEntity from management for allocation and preemption 
-   * ordering
+   * Remove a {@link SchedulableEntity} from management for allocation and
+   * preemption ordering.
+   * @param s the {@link SchedulableEntity} to remove
+   * @return whether the {@link SchedulableEntity} was present before this
+   * operation
    */
   public boolean removeSchedulableEntity(S s);
-  
+
   /**
-   * Add a collection of SchedulableEntities to be managed for allocation 
-   * and preemption ordering
+   * Add a collection of {@link SchedulableEntity} objects to be managed for
+   * allocation and preemption ordering.
+   * @param sc the collection of {@link SchedulableEntity} objects to add
    */
   public void addAllSchedulableEntities(Collection<S> sc);
-  
+
   /**
-   * Get the number of SchedulableEntities managed for allocation and
-   * preemption ordering
+   * Get the number of {@link SchedulableEntity} objects managed for allocation
+   * and preemption ordering.
+   * @return the number of {@link SchedulableEntity} objects
    */
   public int getNumSchedulableEntities();
-  
+
   /**
    * Provides configuration information for the policy from the scheduler
-   * configuration
+   * configuration.
+   * @param conf a map of scheduler configuration properties and values
    */
   public void configure(Map<String, String> conf);
-  
+
   /**
-   * The passed SchedulableEntity has been allocated the passed Container,
-   * take appropriate action (depending on comparator, a reordering of the
-   * SchedulableEntity may be required)
+   * Notify the {@code OrderingPolicy} that the {@link SchedulableEntity}
+   * has been allocated the given {@link RMContainer}, enabling the
+   * {@code OrderingPolicy} to take appropriate action. Depending on the
+   * comparator, a reordering of the {@link SchedulableEntity} may be required.
+   * @param schedulableEntity the {@link SchedulableEntity}
+   * @param r the allocated {@link RMContainer}
    */
-  public void containerAllocated(S schedulableEntity, 
-    RMContainer r);
-  
+  public void containerAllocated(S schedulableEntity, RMContainer r);
+
   /**
-   * The passed SchedulableEntity has released the passed Container,
-   * take appropriate action (depending on comparator, a reordering of the
-   * SchedulableEntity may be required)
+   * Notify the {@code OrderingPolicy} that the {@link SchedulableEntity}
+   * has released the given {@link RMContainer}, enabling the
+   * {@code OrderingPolicy} to take appropriate action. Depending on the
+   * comparator, a reordering of the {@link SchedulableEntity} may be required.
+   * @param schedulableEntity the {@link SchedulableEntity}
+   * @param r the released {@link RMContainer}
    */
-  public void containerReleased(S schedulableEntity, 
-    RMContainer r);
-  
+  public void containerReleased(S schedulableEntity, RMContainer r);
+
   /**
-   * Demand Updated for the passed schedulableEntity, reorder if needed.
+   * Notify the {@code OrderingPolicy} that the demand for the
+   * {@link SchedulableEntity} has been updated, enabling the
+   * {@code OrderingPolicy} to reorder the {@link SchedulableEntity} if needed.
+   * @param schedulableEntity the updated {@link SchedulableEntity}
    */
   void demandUpdated(S schedulableEntity);
 
   /**
-   * Display information regarding configuration and status
+   * Return information regarding configuration and status.
+   * @return configuration and status information
    */
   public String getInfo();
-  
+
 }