You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hama.apache.org by tj...@apache.org on 2012/02/24 10:34:55 UTC

svn commit: r1293145 - in /incubator/hama/trunk: ./ core/src/main/java/org/apache/hama/bsp/ examples/src/main/java/org/apache/hama/examples/ yarn/src/main/java/org/apache/hama/bsp/

Author: tjungblut
Date: Fri Feb 24 09:34:54 2012
New Revision: 1293145

URL: http://svn.apache.org/viewvc?rev=1293145&view=rev
Log:
[HAMA-483]: Remove old and deprecated BSP API (tjungblut)

Modified:
    incubator/hama/trunk/CHANGES.txt
    incubator/hama/trunk/core/src/main/java/org/apache/hama/bsp/BSP.java
    incubator/hama/trunk/core/src/main/java/org/apache/hama/bsp/BSPInterface.java
    incubator/hama/trunk/core/src/main/java/org/apache/hama/bsp/LocalBSPRunner.java
    incubator/hama/trunk/examples/src/main/java/org/apache/hama/examples/RandBench.java
    incubator/hama/trunk/yarn/src/main/java/org/apache/hama/bsp/YarnSerializePrinting.java

Modified: incubator/hama/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/incubator/hama/trunk/CHANGES.txt?rev=1293145&r1=1293144&r2=1293145&view=diff
==============================================================================
--- incubator/hama/trunk/CHANGES.txt (original)
+++ incubator/hama/trunk/CHANGES.txt Fri Feb 24 09:34:54 2012
@@ -12,6 +12,7 @@ Release 0.5 - Unreleased
 
   IMPROVEMENTS
     
+    HAMA-483: Remove old and deprecated BSP API (tjungblut)    
     HAMA-514: Add maven-gpg-plugin to parent POM file (edwardyoon)
     HAMA-510: Add sendMessageToNeighbors() to Vertex (tjungblut)
     HAMA-502: Message API Improvement (edwardyoon)

Modified: incubator/hama/trunk/core/src/main/java/org/apache/hama/bsp/BSP.java
URL: http://svn.apache.org/viewvc/incubator/hama/trunk/core/src/main/java/org/apache/hama/bsp/BSP.java?rev=1293145&r1=1293144&r2=1293145&view=diff
==============================================================================
--- incubator/hama/trunk/core/src/main/java/org/apache/hama/bsp/BSP.java (original)
+++ incubator/hama/trunk/core/src/main/java/org/apache/hama/bsp/BSP.java Fri Feb 24 09:34:54 2012
@@ -19,7 +19,6 @@ package org.apache.hama.bsp;
 
 import java.io.IOException;
 
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.Writable;
 import org.apache.hama.bsp.sync.SyncException;
 
@@ -29,8 +28,6 @@ import org.apache.hama.bsp.sync.SyncExce
 public abstract class BSP<K1, V1, K2, V2, M extends Writable> implements
     BSPInterface<K1, V1, K2, V2, M> {
 
-  protected Configuration conf;
-
   /**
    * This method is your computation method, the main work of your BSP should be
    * done here.
@@ -67,26 +64,4 @@ public abstract class BSP<K1, V1, K2, V2
 
   }
 
-  /**
-   * Returns the configuration of this BSP Job.
-   * 
-   * @deprecated Use BSPPeer.getConfiguration() instead. Will be removed in
-   *             0.5.0.
-   */
-  @Deprecated
-  @Override
-  public Configuration getConf() {
-    return conf;
-  }
-
-  /**
-   * Sets the configuration of this BSP Job.
-   * 
-   * @deprecated Won't be used anymore.
-   */
-  @Deprecated
-  @Override
-  public void setConf(Configuration conf) {
-    this.conf = conf;
-  }
 }

Modified: incubator/hama/trunk/core/src/main/java/org/apache/hama/bsp/BSPInterface.java
URL: http://svn.apache.org/viewvc/incubator/hama/trunk/core/src/main/java/org/apache/hama/bsp/BSPInterface.java?rev=1293145&r1=1293144&r2=1293145&view=diff
==============================================================================
--- incubator/hama/trunk/core/src/main/java/org/apache/hama/bsp/BSPInterface.java (original)
+++ incubator/hama/trunk/core/src/main/java/org/apache/hama/bsp/BSPInterface.java Fri Feb 24 09:34:54 2012
@@ -17,12 +17,11 @@
  */
 package org.apache.hama.bsp;
 
-import org.apache.hadoop.conf.Configurable;
 
 /**
  * Interface BSP defines the basic operations needed to implement the BSP
  * algorithm.
  */
-public interface BSPInterface<K1, V1, K2, V2, M> extends Configurable {
+public interface BSPInterface<K1, V1, K2, V2, M> {
 
 }

Modified: incubator/hama/trunk/core/src/main/java/org/apache/hama/bsp/LocalBSPRunner.java
URL: http://svn.apache.org/viewvc/incubator/hama/trunk/core/src/main/java/org/apache/hama/bsp/LocalBSPRunner.java?rev=1293145&r1=1293144&r2=1293145&view=diff
==============================================================================
--- incubator/hama/trunk/core/src/main/java/org/apache/hama/bsp/LocalBSPRunner.java (original)
+++ incubator/hama/trunk/core/src/main/java/org/apache/hama/bsp/LocalBSPRunner.java Fri Feb 24 09:34:54 2012
@@ -143,7 +143,7 @@ public class LocalBSPRunner implements J
         splitFile.close();
       }
     }
-    
+
     threadPool = (ThreadPoolExecutor) Executors.newFixedThreadPool(numBspTask);
 
     peerNames = new String[numBspTask];
@@ -246,8 +246,6 @@ public class LocalBSPRunner implements J
       BSPPeerImpl peer = new BSPPeerImpl(job, conf, new TaskAttemptID(
           new TaskID(job.getJobID(), id), id), new LocalUmbilical(), id,
           splitname, realBytes, new Counters());
-
-      bsp.setConf(conf);
       try {
         bsp.setup(peer);
         bsp.bsp(peer);
@@ -302,7 +300,8 @@ public class LocalBSPRunner implements J
 
   }
 
-  public static class LocalMessageManager<M extends Writable> implements MessageManager<M> {
+  public static class LocalMessageManager<M extends Writable> implements
+      MessageManager<M> {
 
     private static final ConcurrentHashMap<InetSocketAddress, LocalMessageManager> managerMap = new ConcurrentHashMap<InetSocketAddress, LocalBSPRunner.LocalMessageManager>();
 
@@ -336,8 +335,7 @@ public class LocalBSPRunner implements J
         inetSocketAddress = BSPNetUtils.getAddress(peerName);
         socketCache.put(peerName, inetSocketAddress);
       }
-      LinkedList<M> msgs = localOutgoingMessages
-          .get(inetSocketAddress);
+      LinkedList<M> msgs = localOutgoingMessages.get(inetSocketAddress);
       if (msgs == null) {
         msgs = new LinkedList<M>();
       }
@@ -358,7 +356,7 @@ public class LocalBSPRunner implements J
     public Iterator<Entry<InetSocketAddress, LinkedList<M>>> getMessageIterator() {
       return localOutgoingMessages.entrySet().iterator();
     }
-    
+
     @Override
     public void clearOutgoingQueues() {
       localOutgoingMessages.clear();

Modified: incubator/hama/trunk/examples/src/main/java/org/apache/hama/examples/RandBench.java
URL: http://svn.apache.org/viewvc/incubator/hama/trunk/examples/src/main/java/org/apache/hama/examples/RandBench.java?rev=1293145&r1=1293144&r2=1293145&view=diff
==============================================================================
--- incubator/hama/trunk/examples/src/main/java/org/apache/hama/examples/RandBench.java (original)
+++ incubator/hama/trunk/examples/src/main/java/org/apache/hama/examples/RandBench.java Fri Feb 24 09:34:54 2012
@@ -77,9 +77,9 @@ public class RandBench {
     @Override
     public void setup(
         BSPPeer<NullWritable, NullWritable, NullWritable, NullWritable, BytesWritable> peer) {
-      this.sizeOfMsg = conf.getInt(SIZEOFMSG, 1);
-      this.nCommunications = conf.getInt(N_COMMUNICATIONS, 1);
-      this.nSupersteps = conf.getInt(N_SUPERSTEPS, 1);
+      this.sizeOfMsg = peer.getConfiguration().getInt(SIZEOFMSG, 1);
+      this.nCommunications = peer.getConfiguration().getInt(N_COMMUNICATIONS, 1);
+      this.nSupersteps = peer.getConfiguration().getInt(N_SUPERSTEPS, 1);
     }
   }
 

Modified: incubator/hama/trunk/yarn/src/main/java/org/apache/hama/bsp/YarnSerializePrinting.java
URL: http://svn.apache.org/viewvc/incubator/hama/trunk/yarn/src/main/java/org/apache/hama/bsp/YarnSerializePrinting.java?rev=1293145&r1=1293144&r2=1293145&view=diff
==============================================================================
--- incubator/hama/trunk/yarn/src/main/java/org/apache/hama/bsp/YarnSerializePrinting.java (original)
+++ incubator/hama/trunk/yarn/src/main/java/org/apache/hama/bsp/YarnSerializePrinting.java Fri Feb 24 09:34:54 2012
@@ -21,7 +21,6 @@ import java.io.IOException;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.NullWritable;
 import org.apache.hama.HamaConfiguration;
 import org.apache.hama.bsp.sync.SyncException;
@@ -31,7 +30,6 @@ public class YarnSerializePrinting {
   public static class HelloBSP extends
       BSP<NullWritable, NullWritable, NullWritable, NullWritable, NullWritable> {
     public static final Log LOG = LogFactory.getLog(HelloBSP.class);
-    private Configuration conf;
     private final static int PRINT_INTERVAL = 1000;
     private int num;
 
@@ -39,15 +37,13 @@ public class YarnSerializePrinting {
     public void bsp(
         BSPPeer<NullWritable, NullWritable, NullWritable, NullWritable, NullWritable> bspPeer)
         throws IOException, SyncException, InterruptedException {
-      num = conf.getInt("bsp.peers.num", 0);
+      num = bspPeer.getConfiguration().getInt("bsp.peers.num", 0);
       LOG.info(bspPeer.getAllPeerNames());
       int i = 0;
       for (String otherPeer : bspPeer.getAllPeerNames()) {
         String peerName = bspPeer.getPeerName();
         if (peerName.equals(otherPeer)) {
-          LOG
-              .info("Hello BSP from " + (i + 1) + " of " + num + ": "
-                  + peerName);
+          LOG.info("Hello BSP from " + (i + 1) + " of " + num + ": " + peerName);
         }
 
         Thread.sleep(PRINT_INTERVAL);
@@ -55,16 +51,6 @@ public class YarnSerializePrinting {
         i++;
       }
     }
-
-    @Override
-    public Configuration getConf() {
-      return conf;
-    }
-
-    @Override
-    public void setConf(Configuration conf) {
-      this.conf = conf;
-    }
   }
 
   public static void main(String[] args) throws IOException,