You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hama.apache.org by ed...@apache.org on 2013/12/12 08:06:53 UTC

svn commit: r1550360 - in /hama/trunk: ./ core/src/main/java/org/apache/hama/bsp/ core/src/test/java/org/apache/hama/bsp/ core/src/test/java/org/apache/hama/bsp/message/ examples/src/main/java/org/apache/hama/examples/ graph/src/main/java/org/apache/ha...

Author: edwardyoon
Date: Thu Dec 12 07:06:53 2013
New Revision: 1550360

URL: http://svn.apache.org/r1550360
Log:
HAMA-830: KMeans and NeuralNetwork doesn't load config file

Modified:
    hama/trunk/CHANGES.txt
    hama/trunk/core/src/main/java/org/apache/hama/bsp/BSPPeer.java
    hama/trunk/core/src/main/java/org/apache/hama/bsp/BSPPeerImpl.java
    hama/trunk/core/src/main/java/org/apache/hama/bsp/LocalBSPRunner.java
    hama/trunk/core/src/test/java/org/apache/hama/bsp/TestCheckpoint.java
    hama/trunk/core/src/test/java/org/apache/hama/bsp/message/TestHadoopMessageManager.java
    hama/trunk/examples/src/main/java/org/apache/hama/examples/BipartiteMatching.java
    hama/trunk/examples/src/main/java/org/apache/hama/examples/Kmeans.java
    hama/trunk/examples/src/main/java/org/apache/hama/examples/NeuralNetwork.java
    hama/trunk/examples/src/main/java/org/apache/hama/examples/PageRank.java
    hama/trunk/examples/src/main/java/org/apache/hama/examples/SSSP.java
    hama/trunk/examples/src/main/java/org/apache/hama/examples/SpMV.java
    hama/trunk/graph/src/main/java/org/apache/hama/graph/GraphJobRunner.java
    hama/trunk/graph/src/main/java/org/apache/hama/graph/Vertex.java
    hama/trunk/graph/src/main/java/org/apache/hama/graph/VertexInterface.java
    hama/trunk/graph/src/test/java/org/apache/hama/graph/example/PageRank.java
    hama/trunk/ml/src/main/java/org/apache/hama/ml/semiclustering/SemiClusteringVertex.java

Modified: hama/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hama/trunk/CHANGES.txt?rev=1550360&r1=1550359&r2=1550360&view=diff
==============================================================================
--- hama/trunk/CHANGES.txt (original)
+++ hama/trunk/CHANGES.txt Thu Dec 12 07:06:53 2013
@@ -11,6 +11,7 @@ Release 0.7.0 (unreleased changes)
 
   BUG FIXES
 
+   HAMA-830: KMeans and NeuralNetwork doesn't load config file (edwardyoon)
    HAMA-812: In local mode BSPJobClient.close throws Exception (Martin Illecker)
    HAMA-821: Fix bugs in KMeans example and make output more readable (edwardyoon)
 

Modified: hama/trunk/core/src/main/java/org/apache/hama/bsp/BSPPeer.java
URL: http://svn.apache.org/viewvc/hama/trunk/core/src/main/java/org/apache/hama/bsp/BSPPeer.java?rev=1550360&r1=1550359&r2=1550360&view=diff
==============================================================================
--- hama/trunk/core/src/main/java/org/apache/hama/bsp/BSPPeer.java (original)
+++ hama/trunk/core/src/main/java/org/apache/hama/bsp/BSPPeer.java Thu Dec 12 07:06:53 2013
@@ -19,9 +19,9 @@ 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.Constants;
+import org.apache.hama.HamaConfiguration;
 import org.apache.hama.bsp.Counters.Counter;
 import org.apache.hama.bsp.sync.SyncException;
 import org.apache.hama.commons.util.KeyValuePair;
@@ -140,7 +140,7 @@ public interface BSPPeer<K1, V1, K2, V2,
   /**
    * @return the jobs configuration
    */
-  public Configuration getConfiguration();
+  public HamaConfiguration getConfiguration();
 
   /**
    * Get the {@link Counter} of the given group with the given name.

Modified: hama/trunk/core/src/main/java/org/apache/hama/bsp/BSPPeerImpl.java
URL: http://svn.apache.org/viewvc/hama/trunk/core/src/main/java/org/apache/hama/bsp/BSPPeerImpl.java?rev=1550360&r1=1550359&r2=1550360&view=diff
==============================================================================
--- hama/trunk/core/src/main/java/org/apache/hama/bsp/BSPPeerImpl.java (original)
+++ hama/trunk/core/src/main/java/org/apache/hama/bsp/BSPPeerImpl.java Thu Dec 12 07:06:53 2013
@@ -26,7 +26,6 @@ import java.util.Map.Entry;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.filecache.DistributedCache;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.LocalFileSystem;
@@ -36,6 +35,7 @@ import org.apache.hadoop.io.DataInputBuf
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hama.Constants;
+import org.apache.hama.HamaConfiguration;
 import org.apache.hama.bsp.Counters.Counter;
 import org.apache.hama.bsp.ft.AsyncRcvdMsgCheckpointImpl;
 import org.apache.hama.bsp.ft.BSPFaultTolerantService;
@@ -63,7 +63,7 @@ public final class BSPPeerImpl<K1, V1, K
     COMPRESSED_MESSAGES, SUPERSTEP_SUM, TASK_INPUT_RECORDS, TASK_OUTPUT_RECORDS, IO_BYTES_READ, MESSAGE_BYTES_TRANSFERED, MESSAGE_BYTES_RECEIVED, TOTAL_MESSAGES_SENT, TOTAL_MESSAGES_RECEIVED, COMPRESSED_BYTES_SENT, COMPRESSED_BYTES_RECEIVED, TIME_IN_SYNC_MS
   }
 
-  private final Configuration conf;
+  private final HamaConfiguration conf;
   private final FileSystem fs;
   private BSPJob bspJob;
 
@@ -110,7 +110,7 @@ public final class BSPPeerImpl<K1, V1, K
    * @param conf is the configuration file.
    * @param dfs is the Hadoop FileSystem.
    */
-  protected BSPPeerImpl(final Configuration conf, FileSystem dfs) {
+  protected BSPPeerImpl(final HamaConfiguration conf, FileSystem dfs) {
     this.conf = conf;
     this.fs = dfs;
   }
@@ -122,12 +122,12 @@ public final class BSPPeerImpl<K1, V1, K
    * @param dfs is the Hadoop FileSystem.
    * @param counters is the counters from outside.
    */
-  public BSPPeerImpl(final Configuration conf, FileSystem dfs, Counters counters) {
+  public BSPPeerImpl(final HamaConfiguration conf, FileSystem dfs, Counters counters) {
     this(conf, dfs);
     this.counters = counters;
   }
 
-  public BSPPeerImpl(BSPJob job, Configuration conf, TaskAttemptID taskId,
+  public BSPPeerImpl(BSPJob job, HamaConfiguration conf, TaskAttemptID taskId,
       BSPPeerProtocol umbilical, int partition, String splitClass,
       BytesWritable split, Counters counters) throws Exception {
     this(job, conf, taskId, umbilical, partition, splitClass, split, counters,
@@ -145,7 +145,7 @@ public final class BSPPeerImpl<K1, V1, K
    * @throws Exception
    */
   @SuppressWarnings("unchecked")
-  public BSPPeerImpl(BSPJob job, Configuration conf, TaskAttemptID taskId,
+  public BSPPeerImpl(BSPJob job, HamaConfiguration conf, TaskAttemptID taskId,
       BSPPeerProtocol umbilical, int partition, String splitClass,
       BytesWritable split, Counters counters, long superstep,
       TaskStatus.State state) throws Exception {
@@ -591,7 +591,7 @@ public final class BSPPeerImpl<K1, V1, K
    * @return the conf
    */
   @Override
-  public final Configuration getConfiguration() {
+  public final HamaConfiguration getConfiguration() {
     return conf;
   }
 

Modified: hama/trunk/core/src/main/java/org/apache/hama/bsp/LocalBSPRunner.java
URL: http://svn.apache.org/viewvc/hama/trunk/core/src/main/java/org/apache/hama/bsp/LocalBSPRunner.java?rev=1550360&r1=1550359&r2=1550360&view=diff
==============================================================================
--- hama/trunk/core/src/main/java/org/apache/hama/bsp/LocalBSPRunner.java (original)
+++ hama/trunk/core/src/main/java/org/apache/hama/bsp/LocalBSPRunner.java Thu Dec 12 07:06:53 2013
@@ -150,8 +150,8 @@ public class LocalBSPRunner implements J
     peerNames = new String[numBspTask];
     for (int i = 0; i < numBspTask; i++) {
       peerNames[i] = "local:" + i;
-      completionService.submit(new BSPRunner(new Configuration(conf), job, i,
-          splits));
+      completionService.submit(new BSPRunner(new HamaConfiguration(conf), job,
+          i, splits));
       globalCounters.incrCounter(JobInProgress.JobCounter.LAUNCHED_TASKS, 1L);
     }
 
@@ -211,14 +211,15 @@ public class LocalBSPRunner implements J
   @SuppressWarnings({ "rawtypes" })
   static class BSPRunner implements Callable<BSPPeerImpl> {
 
-    private final Configuration conf;
+    private final HamaConfiguration conf;
     private final BSPJob job;
     private final int id;
     private final BSP bsp;
     private final RawSplit[] splits;
     private BSPPeerImpl peer;
 
-    public BSPRunner(Configuration conf, BSPJob job, int id, RawSplit[] splits) {
+    public BSPRunner(HamaConfiguration conf, BSPJob job, int id,
+        RawSplit[] splits) {
       super();
       this.conf = conf;
       this.job = job;

Modified: hama/trunk/core/src/test/java/org/apache/hama/bsp/TestCheckpoint.java
URL: http://svn.apache.org/viewvc/hama/trunk/core/src/test/java/org/apache/hama/bsp/TestCheckpoint.java?rev=1550360&r1=1550359&r2=1550360&view=diff
==============================================================================
--- hama/trunk/core/src/test/java/org/apache/hama/bsp/TestCheckpoint.java (original)
+++ hama/trunk/core/src/test/java/org/apache/hama/bsp/TestCheckpoint.java Thu Dec 12 07:06:53 2013
@@ -46,6 +46,7 @@ import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hama.Constants;
+import org.apache.hama.HamaConfiguration;
 import org.apache.hama.bsp.Counters.Counter;
 import org.apache.hama.bsp.ft.AsyncRcvdMsgCheckpointImpl;
 import org.apache.hama.bsp.ft.FaultTolerantPeerService;
@@ -267,7 +268,7 @@ public class TestCheckpoint extends Test
     }
 
     @Override
-    public Configuration getConfiguration() {
+    public HamaConfiguration getConfiguration() {
       return null;
     }
 

Modified: hama/trunk/core/src/test/java/org/apache/hama/bsp/message/TestHadoopMessageManager.java
URL: http://svn.apache.org/viewvc/hama/trunk/core/src/test/java/org/apache/hama/bsp/message/TestHadoopMessageManager.java?rev=1550360&r1=1550359&r2=1550360&view=diff
==============================================================================
--- hama/trunk/core/src/test/java/org/apache/hama/bsp/message/TestHadoopMessageManager.java (original)
+++ hama/trunk/core/src/test/java/org/apache/hama/bsp/message/TestHadoopMessageManager.java Thu Dec 12 07:06:53 2013
@@ -23,11 +23,11 @@ import java.util.Map.Entry;
 
 import junit.framework.TestCase;
 
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.NullWritable;
 import org.apache.hama.Constants;
+import org.apache.hama.HamaConfiguration;
 import org.apache.hama.bsp.BSPMessageBundle;
 import org.apache.hama.bsp.BSPPeer;
 import org.apache.hama.bsp.BSPPeerImpl;
@@ -48,7 +48,7 @@ public class TestHadoopMessageManager ex
   public static volatile int increment = 1;
 
   public void testMemoryMessaging() throws Exception {
-    Configuration conf = new Configuration();
+    HamaConfiguration conf = new HamaConfiguration();
     conf.setClass(MessageManager.TRANSFER_QUEUE_TYPE_CLASS,
         MemoryTransferProtocol.class, MessageTransferQueue.class);
     conf.set(DiskQueue.DISK_QUEUE_PATH_KEY, TMP_OUTPUT_PATH);
@@ -56,14 +56,14 @@ public class TestHadoopMessageManager ex
   }
 
   public void testDiskMessaging() throws Exception {
-    Configuration conf = new Configuration();
+    HamaConfiguration conf = new HamaConfiguration();
     conf.set(DiskQueue.DISK_QUEUE_PATH_KEY, TMP_OUTPUT_PATH);
     conf.setClass(MessageManager.TRANSFER_QUEUE_TYPE_CLASS,
         DiskTransferProtocolQueue.class, MessageTransferQueue.class);
     messagingInternal(conf);
   }
 
-  private static void messagingInternal(Configuration conf) throws Exception {
+  private static void messagingInternal(HamaConfiguration conf) throws Exception {
     conf.set(MessageManagerFactory.MESSAGE_MANAGER_CLASS,
         "org.apache.hama.bsp.message.HadoopMessageManagerImpl");
     MessageManager<IntWritable> messageManager = MessageManagerFactory

Modified: hama/trunk/examples/src/main/java/org/apache/hama/examples/BipartiteMatching.java
URL: http://svn.apache.org/viewvc/hama/trunk/examples/src/main/java/org/apache/hama/examples/BipartiteMatching.java?rev=1550360&r1=1550359&r2=1550360&view=diff
==============================================================================
--- hama/trunk/examples/src/main/java/org/apache/hama/examples/BipartiteMatching.java (original)
+++ hama/trunk/examples/src/main/java/org/apache/hama/examples/BipartiteMatching.java Thu Dec 12 07:06:53 2013
@@ -58,7 +58,7 @@ public final class BipartiteMatching {
     private final static Text RIGHT = new Text("R");
 
     @Override
-    public void setup(Configuration conf) {
+    public void setup(HamaConfiguration conf) {
       this.getPeer().getNumCurrentMessages();
     }
 

Modified: hama/trunk/examples/src/main/java/org/apache/hama/examples/Kmeans.java
URL: http://svn.apache.org/viewvc/hama/trunk/examples/src/main/java/org/apache/hama/examples/Kmeans.java?rev=1550360&r1=1550359&r2=1550360&view=diff
==============================================================================
--- hama/trunk/examples/src/main/java/org/apache/hama/examples/Kmeans.java (original)
+++ hama/trunk/examples/src/main/java/org/apache/hama/examples/Kmeans.java Thu Dec 12 07:06:53 2013
@@ -19,10 +19,10 @@ package org.apache.hama.examples;
 
 import java.util.List;
 
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.NullWritable;
+import org.apache.hama.HamaConfiguration;
 import org.apache.hama.bsp.BSPJob;
 import org.apache.hama.commons.io.VectorWritable;
 import org.apache.hama.ml.kmeans.KMeansBSP;
@@ -61,7 +61,8 @@ public class Kmeans {
           .println("USAGE: <INPUT_PATH> <OUTPUT_PATH> <MAXITERATIONS> <K (how many centers)> -g [<COUNT> <DIMENSION OF VECTORS>]");
       return;
     }
-    Configuration conf = new Configuration();
+    HamaConfiguration conf = new HamaConfiguration();
+    
     Path in = new Path(args[0]);
     Path out = new Path(args[1]);
     FileSystem fs = FileSystem.get(conf);

Modified: hama/trunk/examples/src/main/java/org/apache/hama/examples/NeuralNetwork.java
URL: http://svn.apache.org/viewvc/hama/trunk/examples/src/main/java/org/apache/hama/examples/NeuralNetwork.java?rev=1550360&r1=1550359&r2=1550360&view=diff
==============================================================================
--- hama/trunk/examples/src/main/java/org/apache/hama/examples/NeuralNetwork.java (original)
+++ hama/trunk/examples/src/main/java/org/apache/hama/examples/NeuralNetwork.java Thu Dec 12 07:06:53 2013
@@ -25,9 +25,9 @@ import java.net.URI;
 import java.util.HashMap;
 import java.util.Map;
 
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hama.HamaConfiguration;
 import org.apache.hama.commons.math.DenseDoubleVector;
 import org.apache.hama.commons.math.DoubleVector;
 import org.apache.hama.commons.math.FunctionFactory;
@@ -50,6 +50,7 @@ public class NeuralNetwork {
         printUsage();
         return;
       }
+      HamaConfiguration conf = new HamaConfiguration();
 
       String featureDataPath = args[1];
       String resultDataPath = args[2];
@@ -58,8 +59,7 @@ public class NeuralNetwork {
       SmallLayeredNeuralNetwork ann = new SmallLayeredNeuralNetwork(modelPath);
 
       // process data in streaming approach
-      FileSystem fs = FileSystem.get(new URI(featureDataPath),
-          new Configuration());
+      FileSystem fs = FileSystem.get(new URI(featureDataPath), conf);
       BufferedReader br = new BufferedReader(new InputStreamReader(
           fs.open(new Path(featureDataPath))));
       Path outputPath = new Path(resultDataPath);

Modified: hama/trunk/examples/src/main/java/org/apache/hama/examples/PageRank.java
URL: http://svn.apache.org/viewvc/hama/trunk/examples/src/main/java/org/apache/hama/examples/PageRank.java?rev=1550360&r1=1550359&r2=1550360&view=diff
==============================================================================
--- hama/trunk/examples/src/main/java/org/apache/hama/examples/PageRank.java (original)
+++ hama/trunk/examples/src/main/java/org/apache/hama/examples/PageRank.java Thu Dec 12 07:06:53 2013
@@ -48,7 +48,7 @@ public class PageRank {
     static double MAXIMUM_CONVERGENCE_ERROR = 0.001;
 
     @Override
-    public void setup(Configuration conf) {
+    public void setup(HamaConfiguration conf) {
       String val = conf.get("hama.pagerank.alpha");
       if (val != null) {
         DAMPING_FACTOR = Double.parseDouble(val);

Modified: hama/trunk/examples/src/main/java/org/apache/hama/examples/SSSP.java
URL: http://svn.apache.org/viewvc/hama/trunk/examples/src/main/java/org/apache/hama/examples/SSSP.java?rev=1550360&r1=1550359&r2=1550360&view=diff
==============================================================================
--- hama/trunk/examples/src/main/java/org/apache/hama/examples/SSSP.java (original)
+++ hama/trunk/examples/src/main/java/org/apache/hama/examples/SSSP.java Thu Dec 12 07:06:53 2013
@@ -20,7 +20,6 @@ package org.apache.hama.examples;
 import java.io.IOException;
 import java.util.Iterator;
 
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.LongWritable;
@@ -43,7 +42,7 @@ public class SSSP {
       Vertex<Text, IntWritable, IntWritable> {
 
     @Override
-    public void setup(Configuration conf) {
+    public void setup(HamaConfiguration conf) {
       this.setValue(new IntWritable(Integer.MAX_VALUE));
     }
 

Modified: hama/trunk/examples/src/main/java/org/apache/hama/examples/SpMV.java
URL: http://svn.apache.org/viewvc/hama/trunk/examples/src/main/java/org/apache/hama/examples/SpMV.java?rev=1550360&r1=1550359&r2=1550360&view=diff
==============================================================================
--- hama/trunk/examples/src/main/java/org/apache/hama/examples/SpMV.java (original)
+++ hama/trunk/examples/src/main/java/org/apache/hama/examples/SpMV.java Thu Dec 12 07:06:53 2013
@@ -23,7 +23,6 @@ import java.util.List;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -81,7 +80,7 @@ public class SpMV {
    * IMPORTANT: This can be a bottle neck. Problem can be here{@core
    * WritableUtil.convertSpMVOutputToDenseVector()}
    */
-  private static void convertToDenseVector(Configuration conf)
+  private static void convertToDenseVector(HamaConfiguration conf)
       throws IOException {
     String resultPath = convertSpMVOutputToDenseVector(
         conf.get(outputPathString), conf);
@@ -104,7 +103,7 @@ public class SpMV {
         BSPPeer<IntWritable, SparseVectorWritable, IntWritable, DoubleWritable, NullWritable> peer)
         throws IOException, SyncException, InterruptedException {
       // reading input vector, which represented as matrix row
-      Configuration conf = peer.getConfiguration();
+      HamaConfiguration conf = (HamaConfiguration) peer.getConfiguration();
       v = new DenseVectorWritable();
       readFromFile(conf.get(inputVectorPathString), v, conf);
       peer.sync();
@@ -234,7 +233,7 @@ public class SpMV {
    * @throws IOException
    */
   public static String convertSpMVOutputToDenseVector(
-      String SpMVoutputPathString, Configuration conf) throws IOException {
+      String SpMVoutputPathString, HamaConfiguration conf) throws IOException {
     List<Integer> indeces = new ArrayList<Integer>();
     List<Double> values = new ArrayList<Double>();
 
@@ -270,7 +269,7 @@ public class SpMV {
   }
 
   public static void readFromFile(String pathString, Writable result,
-      Configuration conf) throws IOException {
+      HamaConfiguration conf) throws IOException {
     FileSystem fs = FileSystem.get(conf);
     SequenceFile.Reader reader = null;
     Path path = new Path(pathString);
@@ -306,7 +305,7 @@ public class SpMV {
    * @throws IOException
    */
   public static void writeToFile(String pathString, Writable result,
-      Configuration conf) throws IOException {
+      HamaConfiguration conf) throws IOException {
     FileSystem fs = FileSystem.get(conf);
     SequenceFile.Writer writer = null;
     try {

Modified: hama/trunk/graph/src/main/java/org/apache/hama/graph/GraphJobRunner.java
URL: http://svn.apache.org/viewvc/hama/trunk/graph/src/main/java/org/apache/hama/graph/GraphJobRunner.java?rev=1550360&r1=1550359&r2=1550360&view=diff
==============================================================================
--- hama/trunk/graph/src/main/java/org/apache/hama/graph/GraphJobRunner.java (original)
+++ hama/trunk/graph/src/main/java/org/apache/hama/graph/GraphJobRunner.java Thu Dec 12 07:06:53 2013
@@ -31,6 +31,7 @@ import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparable;
 import org.apache.hama.Constants;
+import org.apache.hama.HamaConfiguration;
 import org.apache.hama.bsp.BSP;
 import org.apache.hama.bsp.BSPPeer;
 import org.apache.hama.bsp.Combiner;
@@ -79,7 +80,7 @@ public final class GraphJobRunner<V exte
   public static final String MESSAGE_COMBINER_CLASS_KEY = "hama.vertex.message.combiner.class";
   public static final String VERTEX_CLASS_KEY = "hama.graph.vertex.class";
 
-  private Configuration conf;
+  private HamaConfiguration conf;
   private Combiner<M> combiner;
   private Partitioner<V, M> partitioner;
 

Modified: hama/trunk/graph/src/main/java/org/apache/hama/graph/Vertex.java
URL: http://svn.apache.org/viewvc/hama/trunk/graph/src/main/java/org/apache/hama/graph/Vertex.java?rev=1550360&r1=1550359&r2=1550360&view=diff
==============================================================================
--- hama/trunk/graph/src/main/java/org/apache/hama/graph/Vertex.java (original)
+++ hama/trunk/graph/src/main/java/org/apache/hama/graph/Vertex.java Thu Dec 12 07:06:53 2013
@@ -23,11 +23,11 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.MapWritable;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparable;
+import org.apache.hama.HamaConfiguration;
 import org.apache.hama.bsp.BSPPeer;
 import org.apache.hama.bsp.Partitioner;
 
@@ -59,7 +59,7 @@ public abstract class Vertex<V extends W
 
   private boolean votedToHalt = false;
 
-  public Configuration getConf() {
+  public HamaConfiguration getConf() {
     return runner.getPeer().getConfiguration();
   }
 
@@ -69,7 +69,7 @@ public abstract class Vertex<V extends W
   }
 
   @Override
-  public void setup(Configuration conf) {
+  public void setup(HamaConfiguration conf) {
   }
 
   @Override

Modified: hama/trunk/graph/src/main/java/org/apache/hama/graph/VertexInterface.java
URL: http://svn.apache.org/viewvc/hama/trunk/graph/src/main/java/org/apache/hama/graph/VertexInterface.java?rev=1550360&r1=1550359&r2=1550360&view=diff
==============================================================================
--- hama/trunk/graph/src/main/java/org/apache/hama/graph/VertexInterface.java (original)
+++ hama/trunk/graph/src/main/java/org/apache/hama/graph/VertexInterface.java Thu Dec 12 07:06:53 2013
@@ -20,9 +20,9 @@ package org.apache.hama.graph;
 import java.io.IOException;
 import java.util.List;
 
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparable;
+import org.apache.hama.HamaConfiguration;
 
 /**
  * The vertex interface.
@@ -40,7 +40,7 @@ public interface VertexInterface<V exten
   /**
    * Used to setup a vertex.
    */
-  public void setup(Configuration conf);
+  public void setup(HamaConfiguration conf);
 
   /**
    * @return the unique identification for the vertex.

Modified: hama/trunk/graph/src/test/java/org/apache/hama/graph/example/PageRank.java
URL: http://svn.apache.org/viewvc/hama/trunk/graph/src/test/java/org/apache/hama/graph/example/PageRank.java?rev=1550360&r1=1550359&r2=1550360&view=diff
==============================================================================
--- hama/trunk/graph/src/test/java/org/apache/hama/graph/example/PageRank.java (original)
+++ hama/trunk/graph/src/test/java/org/apache/hama/graph/example/PageRank.java Thu Dec 12 07:06:53 2013
@@ -48,7 +48,7 @@ public class PageRank {
     static double MAXIMUM_CONVERGENCE_ERROR = 0.001;
 
     @Override
-    public void setup(Configuration conf) {
+    public void setup(HamaConfiguration conf) {
       String val = conf.get("hama.pagerank.alpha");
       if (val != null) {
         DAMPING_FACTOR = Double.parseDouble(val);

Modified: hama/trunk/ml/src/main/java/org/apache/hama/ml/semiclustering/SemiClusteringVertex.java
URL: http://svn.apache.org/viewvc/hama/trunk/ml/src/main/java/org/apache/hama/ml/semiclustering/SemiClusteringVertex.java?rev=1550360&r1=1550359&r2=1550360&view=diff
==============================================================================
--- hama/trunk/ml/src/main/java/org/apache/hama/ml/semiclustering/SemiClusteringVertex.java (original)
+++ hama/trunk/ml/src/main/java/org/apache/hama/ml/semiclustering/SemiClusteringVertex.java Thu Dec 12 07:06:53 2013
@@ -18,15 +18,21 @@
 
 package org.apache.hama.ml.semiclustering;
 
-import org.apache.hadoop.conf.Configuration;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.TreeSet;
+
 import org.apache.hadoop.io.DoubleWritable;
 import org.apache.hadoop.io.Text;
+import org.apache.hama.HamaConfiguration;
 import org.apache.hama.graph.Edge;
 import org.apache.hama.graph.Vertex;
 
-import java.io.IOException;
-import java.util.*;
-
 /**
  * SemiClusteringVertex Class defines each vertex in a Graph job and the
  * compute() method is the function which is applied on each Vertex in the graph
@@ -40,7 +46,7 @@ public class SemiClusteringVertex extend
   private int graphJobVertexMaxClusterCount;
 
   @Override
-  public void setup(Configuration conf) {
+  public void setup(HamaConfiguration conf) {
     semiClusterMaximumVertexCount = conf.getInt("semicluster.max.vertex.count",
         10);
     graphJobMessageSentCount = conf.getInt(