Return-Path: X-Original-To: apmail-hama-commits-archive@www.apache.org Delivered-To: apmail-hama-commits-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 3DFC5108A9 for ; Thu, 12 Dec 2013 07:07:58 +0000 (UTC) Received: (qmail 42351 invoked by uid 500); 12 Dec 2013 07:07:57 -0000 Delivered-To: apmail-hama-commits-archive@hama.apache.org Received: (qmail 42328 invoked by uid 500); 12 Dec 2013 07:07:53 -0000 Mailing-List: contact commits-help@hama.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@hama.apache.org Delivered-To: mailing list commits@hama.apache.org Received: (qmail 42320 invoked by uid 99); 12 Dec 2013 07:07:52 -0000 Received: from athena.apache.org (HELO athena.apache.org) (140.211.11.136) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 12 Dec 2013 07:07:52 +0000 X-ASF-Spam-Status: No, hits=-2000.0 required=5.0 tests=ALL_TRUSTED X-Spam-Check-By: apache.org Received: from [140.211.11.4] (HELO eris.apache.org) (140.211.11.4) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 12 Dec 2013 07:07:50 +0000 Received: from eris.apache.org (localhost [127.0.0.1]) by eris.apache.org (Postfix) with ESMTP id 8300923888FE; Thu, 12 Dec 2013 07:06:54 +0000 (UTC) Content-Type: text/plain; charset="utf-8" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit Subject: 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... Date: Thu, 12 Dec 2013 07:06:53 -0000 To: commits@hama.apache.org From: edwardyoon@apache.org X-Mailer: svnmailer-1.0.9 Message-Id: <20131212070654.8300923888FE@eris.apache.org> X-Virus-Checked: Checked by ClamAV on apache.org 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 { - 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 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: -g [ ]"); 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 { @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 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 indeces = new ArrayList(); List values = new ArrayList(); @@ -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 combiner; private Partitioner 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