Index: commons/src/main/java/org/apache/hama/commons/io/PipesKeyValueWritable.java =================================================================== --- commons/src/main/java/org/apache/hama/commons/io/PipesKeyValueWritable.java (revision 0) +++ commons/src/main/java/org/apache/hama/commons/io/PipesKeyValueWritable.java (revision 0) @@ -0,0 +1,105 @@ +/** + * 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.hama.commons.io; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; + +import org.apache.hadoop.io.Text; + +/** + * PipesKeyValueWritable containing a Text key-value WritableComparable pair. + * Format: delimiter | key | delimiter | value e.g., :Key:Value with delimiter + * ':' or |Key|Value with delimiter '|' + */ +public final class PipesKeyValueWritable extends KeyValueWritable { + + // private static final Log LOG = + // LogFactory.getLog(PipesKeyValueWritable.class); + + /** + * Delimiter between key and value + */ + private char keyValueDelimiter; + + public PipesKeyValueWritable() { + super(); + } + + public PipesKeyValueWritable(Text key, Text value, char keyValueDelimiter) { + super(key, value); + this.keyValueDelimiter = keyValueDelimiter; + } + + public char getKeyValueDelimiter() { + return keyValueDelimiter; + } + + public void setKeyValueDelimiter(char keyValueDelimiter) { + this.keyValueDelimiter = keyValueDelimiter; + } + + @Override + public void readFields(DataInput in) throws IOException { + String str = Text.readString(in); + // LOG.debug("readFields: '" + str + "'"); + + this.keyValueDelimiter = str.charAt(0); + str = str.substring(1); + String[] result = str.split(String.valueOf(this.keyValueDelimiter), 2); + super.setKey(new Text(result[0])); + super.setValue(new Text(result[1])); + } + + @Override + public void write(DataOutput out) throws IOException { + // LOG.debug("write: '" + this.toString() + "'"); + Text.writeString(out, this.toString()); + } + + @Override + public String toString() { + return this.keyValueDelimiter + super.getKey().toString() + + this.keyValueDelimiter + super.getValue().toString(); + } + + @Override + public int compareTo(KeyValueWritable obj) { + // if key is numeric compare numbers + if ((isNumeric(key.toString())) && (isNumeric(obj.key.toString()))) { + double val1 = Double.parseDouble(key.toString()); + double val2 = Double.parseDouble(obj.key.toString()); + int cmp = Double.compare(val1, val2); + if (cmp != 0) { + return cmp; + } + } else { // else compare key string + int cmp = key.compareTo(obj.key); + if (cmp != 0) { + return cmp; + } + } + // if keys are equal compare value + return value.compareTo(obj.value); + } + + public final static boolean isNumeric(String s) { + return s.matches("[-+]?\\d*\\.?\\d+"); + } +} Index: commons/src/main/java/org/apache/hama/commons/io/VectorWritable.java =================================================================== --- commons/src/main/java/org/apache/hama/commons/io/VectorWritable.java (revision 1542513) +++ commons/src/main/java/org/apache/hama/commons/io/VectorWritable.java (working copy) @@ -28,7 +28,7 @@ /** * Writable for dense vectors. */ -public final class VectorWritable implements WritableComparable { +public class VectorWritable implements WritableComparable { private DoubleVector vector; @@ -45,12 +45,12 @@ } @Override - public final void write(DataOutput out) throws IOException { + public void write(DataOutput out) throws IOException { writeVector(this.vector, out); } @Override - public final void readFields(DataInput in) throws IOException { + public void readFields(DataInput in) throws IOException { this.vector = readVector(in); } Index: commons/src/main/java/org/apache/hama/commons/io/PipesVectorWritable.java =================================================================== --- commons/src/main/java/org/apache/hama/commons/io/PipesVectorWritable.java (revision 0) +++ commons/src/main/java/org/apache/hama/commons/io/PipesVectorWritable.java (revision 0) @@ -0,0 +1,82 @@ +/** + * 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.hama.commons.io; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; + +import org.apache.hadoop.io.Text; +import org.apache.hama.commons.math.DenseDoubleVector; +import org.apache.hama.commons.math.DoubleVector; + +/** + * VectorWritable for Hama Pipes dense vectors. + */ +public final class PipesVectorWritable extends VectorWritable { + + // private static final Log LOG = + // LogFactory.getLog(PipesVectorWritable.class); + + public PipesVectorWritable() { + super(); + } + + public PipesVectorWritable(VectorWritable v) { + super(v); + } + + public PipesVectorWritable(DoubleVector v) { + super(v); + } + + @Override + public final void write(DataOutput out) throws IOException { + writeVector(super.getVector(), out); + } + + @Override + public final void readFields(DataInput in) throws IOException { + super.set(readVector(in)); + } + + public static void writeVector(DoubleVector vector, DataOutput out) + throws IOException { + String str = ""; + for (int i = 0; i < vector.getLength(); i++) { + str += (i < vector.getLength() - 1) ? vector.get(i) + ", " : vector + .get(i); + } + + // LOG.debug("writeVector: '" + str + "'"); + Text.writeString(out, str); + } + + public static DoubleVector readVector(DataInput in) throws IOException { + String str = Text.readString(in); + // LOG.debug("readVector: '" + str + "'"); + + String[] values = str.split(","); + int len = values.length; + DoubleVector vector = new DenseDoubleVector(len); + for (int i = 0; i < len; i++) { + vector.set(i, Double.parseDouble(values[i])); + } + return vector; + } +} Index: commons/src/main/java/org/apache/hama/commons/io/KeyValueWritable.java =================================================================== --- commons/src/main/java/org/apache/hama/commons/io/KeyValueWritable.java (revision 0) +++ commons/src/main/java/org/apache/hama/commons/io/KeyValueWritable.java (revision 0) @@ -0,0 +1,124 @@ +/** + * 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.hama.commons.io; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; + +import org.apache.hadoop.io.WritableComparable; + +/** + * KeyValueWritable containing a key-value WritableComparable pair. + * + * @param the class of key + * @param the class of value + */ +public class KeyValueWritable + implements WritableComparable> { + + protected K key = null; + protected V value = null; + + public KeyValueWritable() { + } + + public KeyValueWritable(K key, V value) { + this.key = key; + this.value = value; + } + + public K getKey() { + return key; + } + + public void setKey(K key) { + this.key = key; + } + + public V getValue() { + return value; + } + + public void setValue(V value) { + this.value = value; + } + + @Override + public void readFields(DataInput in) throws IOException { + key.readFields(in); + value.readFields(in); + } + + @Override + public void write(DataOutput out) throws IOException { + key.write(out); + value.write(out); + } + + @Override + public int hashCode() { + final int prime = 31; + int result = 1; + result = prime * result + ((key == null) ? 0 : key.hashCode()); + result = prime * result + ((value == null) ? 0 : value.hashCode()); + return result; + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + if (obj == null) { + return false; + } + if (getClass() != obj.getClass()) { + return false; + } + + KeyValueWritable other = (KeyValueWritable) obj; + if (key == null) { + if (other.key != null) { + return false; + } + } else if (!key.equals(other.key)) { + return false; + } + + if (value == null) { + if (other.value != null) { + return false; + } + } else if (!value.equals(other.value)) { + return false; + } + return true; + } + + @Override + public int compareTo(KeyValueWritable obj) { + int cmp = key.compareTo(obj.key); + if (cmp != 0) { + return cmp; + } + // if keys are equal compare value + return value.compareTo(obj.value); + } + +} Index: examples/src/main/java/org/apache/hama/examples/util/Generator.java =================================================================== --- examples/src/main/java/org/apache/hama/examples/util/Generator.java (revision 1542513) +++ examples/src/main/java/org/apache/hama/examples/util/Generator.java (working copy) @@ -27,6 +27,8 @@ System.out .println(" fastgen: Generate random matrix, which can be used as a input of graph examples and is faster than symmetric."); System.out.println(" square: Generate random square matrix."); + System.out + .println(" vectorwritablematrix: Generate a random matrix, consisting of VectorWritables."); System.exit(1); } @@ -37,6 +39,8 @@ SymmetricMatrixGen.main(newArgs); } else if (args[0].equals("fastgen")) { FastGraphGen.main(newArgs); + } else if (args[0].equals("vectorwritablematrix")) { + VectorWritableMatrixGen.main(newArgs); } else if (args[0].equals("square")) { System.out.println("Not implemented yet."); // SquareMatrixGen.main(newArgs); Index: examples/src/main/java/org/apache/hama/examples/util/VectorWritableMatrixGen.java =================================================================== --- examples/src/main/java/org/apache/hama/examples/util/VectorWritableMatrixGen.java (revision 0) +++ examples/src/main/java/org/apache/hama/examples/util/VectorWritableMatrixGen.java (revision 0) @@ -0,0 +1,179 @@ +/** + * 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.hama.examples.util; + +import java.io.IOException; +import java.math.BigDecimal; +import java.util.Random; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.io.IntWritable; +import org.apache.hadoop.io.SequenceFile; +import org.apache.hama.HamaConfiguration; +import org.apache.hama.commons.io.PipesVectorWritable; +import org.apache.hama.commons.io.VectorWritable; +import org.apache.hama.commons.math.DenseDoubleVector; + +public class VectorWritableMatrixGen { + private static final Log LOG = LogFactory + .getLog(VectorWritableMatrixGen.class); + + public static void main(String[] args) throws InterruptedException, + IOException, ClassNotFoundException { + + boolean saveTransposed = false; + boolean usePipesVectorWritable = false; + double minValue = 0; + double maxValue = 1000; + int precision = 3; + + // check arguments + if (args.length < 3) { + System.out + .println("Usage: " + + " [] []" + + " [] []" + + " []"); + System.out + .println("e.g., hama jar hama-examples-*.jar gen vectorwritablematrix 10 10 /tmp/matrix.seq"); + System.out + .println(" hama jar hama-examples-*.jar gen vectorwritablematrix 10 10 /tmp/matrix.seq" + + " false false 0 10 2"); + System.exit(1); + } + + int rowSize = Integer.parseInt(args[0]); + int colSize = Integer.parseInt(args[1]); + Path outputPath = new Path(args[2]); + + if (args.length > 3) { + saveTransposed = Boolean.parseBoolean(args[3]); + if (args.length > 4) { + usePipesVectorWritable = Boolean.parseBoolean(args[4]); + if (args.length > 5) { + minValue = Double.parseDouble(args[5]); + if (args.length > 6) { + maxValue = Double.parseDouble(args[6]); + if (args.length > 7) { + precision = Integer.parseInt(args[7]); + } + } + } + } + } + + LOG.debug("rowSize: " + rowSize + " colSize: " + colSize + " outputPath: " + + outputPath); + LOG.debug("saveTransposed: " + saveTransposed + " usePipesVectorWritable: " + + usePipesVectorWritable); + LOG.debug("minValue: " + minValue + " maxValue: " + maxValue + + " precision: " + precision); + + // create random double matrix + double[][] matrix = createRandomMatrix(rowSize, colSize, new Random(), + minValue, maxValue, precision); + + // write matrix to dfs + writeMatrix(matrix, outputPath, saveTransposed, usePipesVectorWritable); + } + + public static double[][] createRandomMatrix(int rows, int columns, + Random rand, double rangeMin, double rangeMax, int precision) { + + LOG.debug("createRandomMatrix rows: " + rows + " cols: " + columns); + + final double[][] matrix = new double[rows][columns]; + + for (int i = 0; i < rows; i++) { + for (int j = 0; j < columns; j++) { + double randomValue = rangeMin + (rangeMax - rangeMin) + * rand.nextDouble(); + + matrix[i][j] = new BigDecimal(randomValue).setScale(precision, + BigDecimal.ROUND_DOWN).doubleValue(); + } + } + return matrix; + } + + public static Path writeMatrix(double[][] matrix, Path path, + boolean saveTransposed, boolean usePipesVectorWritable) { + + LOG.debug("writeMatrix path: " + path + " saveTransposed: " + + saveTransposed + " usePipesVectorWritable: " + usePipesVectorWritable); + + // transpose matrix before saving + if (saveTransposed) { + int rows = matrix.length; + int columns = matrix[0].length; + double[][] transposed = new double[columns][rows]; + for (int i = 0; i < rows; i++) { + for (int j = 0; j < columns; j++) { + transposed[j][i] = matrix[i][j]; + } + } + matrix = transposed; + } + + // Write matrix to DFS + HamaConfiguration conf = new HamaConfiguration(); + SequenceFile.Writer writer = null; + try { + FileSystem fs = FileSystem.get(conf); + // use PipesVectorWritable if specified + if (usePipesVectorWritable) { + writer = new SequenceFile.Writer(fs, conf, path, IntWritable.class, + PipesVectorWritable.class); + + for (int i = 0; i < matrix.length; i++) { + DenseDoubleVector rowVector = new DenseDoubleVector(matrix[i]); + writer.append(new IntWritable(i), new PipesVectorWritable(rowVector)); + LOG.debug("IntWritable: " + i + " PipesVectorWritable: " + + rowVector.toString()); + } + + } else { + writer = new SequenceFile.Writer(fs, conf, path, IntWritable.class, + VectorWritable.class); + + for (int i = 0; i < matrix.length; i++) { + DenseDoubleVector rowVector = new DenseDoubleVector(matrix[i]); + writer.append(new IntWritable(i), new VectorWritable(rowVector)); + LOG.debug("IntWritable: " + i + " VectorWritable: " + + rowVector.toString()); + } + } + + } catch (IOException e) { + e.printStackTrace(); + } finally { + if (writer != null) { + try { + writer.close(); + } catch (IOException e) { + e.printStackTrace(); + } + } + } + return path; + } + +} Index: src/assemble/bin.xml =================================================================== --- src/assemble/bin.xml (revision 1542513) +++ src/assemble/bin.xml (working copy) @@ -103,14 +103,6 @@ ../hama-${project.version}/lib - ../c++/target/native - - *.a - - 755 - ../hama-${project.version}/lib/native - - ../bin hama Index: core/pom.xml =================================================================== --- core/pom.xml (revision 1542513) +++ core/pom.xml (working copy) @@ -209,6 +209,7 @@ org.codehaus.mojo build-helper-maven-plugin + 1.8 add-source Index: core/src/main/java/org/apache/hama/bsp/BSPTask.java =================================================================== --- core/src/main/java/org/apache/hama/bsp/BSPTask.java (revision 1542513) +++ core/src/main/java/org/apache/hama/bsp/BSPTask.java (working copy) @@ -33,8 +33,6 @@ import org.apache.hadoop.util.ReflectionUtils; import org.apache.hama.Constants; import org.apache.hama.ipc.BSPPeerProtocol; -import org.apache.hama.pipes.PipesApplicable; -import org.apache.hama.pipes.PipesBSP; /** * Base class for tasks. @@ -166,10 +164,6 @@ LOG.debug("bsp.work.class: " + workClass.toString()); - /* Setup PipesApplication if workClass is matching */ - if (PipesBSP.class.equals(workClass)) - ((PipesApplicable) bsp).setApplication(job.getPipesApplication()); - // The policy is to throw the first exception and log the remaining. Exception firstException = null; try { Index: core/src/main/java/org/apache/hama/bsp/PartitioningRunner.java =================================================================== --- core/src/main/java/org/apache/hama/bsp/PartitioningRunner.java (revision 1542513) +++ core/src/main/java/org/apache/hama/bsp/PartitioningRunner.java (working copy) @@ -18,6 +18,7 @@ package org.apache.hama.bsp; import java.io.IOException; +import java.lang.reflect.Constructor; import java.util.HashMap; import java.util.LinkedList; import java.util.List; @@ -37,6 +38,7 @@ import org.apache.hama.Constants; import org.apache.hama.bsp.sync.SyncException; import org.apache.hama.commons.util.KeyValuePair; +import org.apache.hama.pipes.PipesPartitioner; public class PartitioningRunner extends BSP { @@ -47,7 +49,8 @@ private FileSystem fs = null; private Path partitionDir; private RecordConverter converter; - private Map>> values = new HashMap>>(); + private Map>> values = new HashMap>>(); + private PipesPartitioner pipesPartitioner = null; @Override public final void setup( @@ -103,7 +106,7 @@ * needed. */ public Map newMap(); - + /** * @return a list implementation, so order will not be changed in subclasses */ @@ -143,7 +146,7 @@ @Override public List> newList() { - return new LinkedList>(); + return new LinkedList>(); } } @@ -172,17 +175,20 @@ int index = converter.getPartitionId(outputPair, partitioner, conf, peer, desiredNum); - + LinkedList> list = values.get(index); if (list == null) { - list = (LinkedList>) converter.newList(); + list = (LinkedList>) converter + .newList(); values.put(index, list); } - list.add(new KeyValuePair(pair.getKey(), pair.getValue())); + list.add(new KeyValuePair(pair.getKey(), pair + .getValue())); } // The reason of use of Memory is to reduce file opens - for (Map.Entry>> e : values.entrySet()) { + for (Map.Entry>> e : values + .entrySet()) { Path destFile = new Path(partitionDir + "/part-" + e.getKey() + "/file-" + peer.getPeerIndex()); SequenceFile.Writer writer = SequenceFile.createWriter(fs, conf, @@ -240,15 +246,43 @@ } } + @Override + public void cleanup( + BSPPeer peer) + throws IOException { + if (this.pipesPartitioner != null) { + this.pipesPartitioner.cleanup(); + } + } + public static int getMergeProcessorID(int partitionID, int peerNum) { return partitionID % peerNum; } @SuppressWarnings("rawtypes") public Partitioner getPartitioner() { - return ReflectionUtils.newInstance(conf.getClass( + Class partitionerClass = conf.getClass( Constants.RUNTIME_PARTITIONING_CLASS, HashPartitioner.class, - Partitioner.class), conf); + Partitioner.class); + + LOG.debug(Constants.RUNTIME_PARTITIONING_CLASS + ": " + + partitionerClass.toString()); + + // Check for Hama Pipes Partitioner + Partitioner partitioner = null; + if (PipesPartitioner.class.equals(partitionerClass)) { + try { + Constructor ctor = partitionerClass + .getConstructor(Configuration.class); + partitioner = ctor.newInstance(conf); + this.pipesPartitioner = (PipesPartitioner) partitioner; + } catch (Exception e) { + LOG.error(e); + } + } else { + partitioner = ReflectionUtils.newInstance(partitionerClass, conf); + } + return partitioner; } private static String getPartitionName(int i) { Index: core/src/main/java/org/apache/hama/bsp/BSPJobClient.java =================================================================== --- core/src/main/java/org/apache/hama/bsp/BSPJobClient.java (revision 1542513) +++ core/src/main/java/org/apache/hama/bsp/BSPJobClient.java (working copy) @@ -52,7 +52,6 @@ import org.apache.hadoop.io.Writable; import org.apache.hadoop.io.WritableUtils; import org.apache.hadoop.io.compress.CompressionCodec; -import org.apache.hama.ipc.RPC; import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.util.Shell; import org.apache.hadoop.util.Tool; @@ -61,6 +60,7 @@ import org.apache.hama.HamaConfiguration; import org.apache.hama.ipc.HamaRPCProtocolVersion; import org.apache.hama.ipc.JobSubmissionProtocol; +import org.apache.hama.ipc.RPC; /** * BSPJobClient is the primary interface for the user-job to interact with the @@ -429,7 +429,8 @@ job.get(Constants.RUNTIME_PARTITIONING_CLASS)); } BSPJob partitioningJob = new BSPJob(conf); - LOG.debug("partitioningJob input: " + partitioningJob.get(Constants.JOB_INPUT_DIR)); + LOG.debug("partitioningJob input: " + + partitioningJob.get(Constants.JOB_INPUT_DIR)); partitioningJob.setInputFormat(job.getInputFormat().getClass()); partitioningJob.setInputKeyClass(job.getInputKeyClass()); partitioningJob.setInputValueClass(job.getInputValueClass()); @@ -768,9 +769,6 @@ // TODO if error found, kill job // running.killJob(); jc.close(); - - // Added cleanup for Client PipesApp and DistributedCache - job.cleanup(); } /** Index: core/src/main/java/org/apache/hama/bsp/BSPJob.java =================================================================== --- core/src/main/java/org/apache/hama/bsp/BSPJob.java (revision 1542513) +++ core/src/main/java/org/apache/hama/bsp/BSPJob.java (working copy) @@ -22,10 +22,7 @@ import java.net.URLDecoder; import java.util.Enumeration; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.io.BytesWritable; import org.apache.hadoop.io.LongWritable; import org.apache.hadoop.io.Text; import org.apache.hadoop.io.Writable; @@ -34,10 +31,6 @@ import org.apache.hama.HamaConfiguration; import org.apache.hama.bsp.message.compress.BSPMessageCompressor; import org.apache.hama.bsp.message.compress.BSPMessageCompressorFactory; -import org.apache.hama.pipes.PipesApplicable; -import org.apache.hama.pipes.PipesApplication; -import org.apache.hama.pipes.PipesPartitioner; -import org.apache.hama.pipes.util.DistributedCacheUtil; /** * A BSP job configuration. @@ -54,9 +47,6 @@ private BSPJobClient jobClient; private RunningJob info; - private PipesApplication pipesApp = null; - private static final Log LOG = LogFactory.getLog(BSPJob.class); - public BSPJob() throws IOException { this(new HamaConfiguration()); } @@ -260,23 +250,6 @@ return conf.getBoolean(name, defaultValue); } - public final PipesApplication getPipesApplication() { - if (pipesApp == null) - pipesApp = new PipesApplication(); - - return pipesApp; - } - - public void cleanup() { - try { - // Close client pipesApplication - if (this.getPipesApplication() != null) - this.getPipesApplication().cleanup(); - } catch (IOException e) { - LOG.error(e); - } - } - public void setNumBspTask(int tasks) { conf.setInt("bsp.peers.num", tasks); } @@ -416,36 +389,6 @@ } @SuppressWarnings("rawtypes") - public Partitioner getPartitioner() { - - Class partitionerClass = conf.getClass( - Constants.RUNTIME_PARTITIONING_CLASS, HashPartitioner.class, - Partitioner.class); - - LOG.info("DEBUG: " + Constants.RUNTIME_PARTITIONING_CLASS + ": " - + partitionerClass.toString()); - - Partitioner partitioner = ReflectionUtils.newInstance(partitionerClass, - conf); - - /* PipesPartitioner usage */ - if (PipesPartitioner.class.equals(partitionerClass)) { - ((PipesApplicable) partitioner) - .setApplication(this.getPipesApplication()); - - try { - DistributedCacheUtil.moveLocalFiles(conf); - this.getPipesApplication().start(conf); - } catch (IOException e) { - LOG.error(e); - } catch (InterruptedException e) { - LOG.error(e); - } - } - return partitioner; - } - - @SuppressWarnings("rawtypes") public OutputFormat getOutputFormat() { return ReflectionUtils.newInstance(conf.getClass( Constants.OUTPUT_FORMAT_CLASS, TextOutputFormat.class, Index: core/src/main/java/org/apache/hama/pipes/PipesApplication.java =================================================================== --- core/src/main/java/org/apache/hama/pipes/PipesApplication.java (revision 1542513) +++ core/src/main/java/org/apache/hama/pipes/PipesApplication.java (working copy) @@ -37,7 +37,6 @@ import org.apache.hadoop.filecache.DistributedCache; import org.apache.hadoop.fs.FileUtil; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.io.BytesWritable; import org.apache.hadoop.io.Writable; import org.apache.hadoop.util.StringUtils; import org.apache.hama.bsp.BSPPeer; @@ -54,10 +53,9 @@ * Adapted from Hadoop Pipes. * */ -public class PipesApplication { +public class PipesApplication { - private static final Log LOG = LogFactory.getLog(PipesApplication.class - .getName()); + private static final Log LOG = LogFactory.getLog(PipesApplication.class); private ServerSocket serverSocket; private Process process; private Socket clientSocket; @@ -72,7 +70,7 @@ } /* Build Environment based on the Configuration */ - private Map setupEnvironment(Configuration conf) + public Map setupEnvironment(Configuration conf) throws IOException { Map env = new HashMap(); @@ -220,7 +218,7 @@ clientSocket = serverSocket.accept(); LOG.debug("DEBUG: Client connected! - start BinaryProtocol!"); - downlink = new BinaryProtocol(conf, + downlink = new BinaryProtocol(conf, clientSocket.getOutputStream(), clientSocket.getInputStream()); downlink.start(); @@ -249,8 +247,8 @@ * @throws IOException */ @SuppressWarnings({ "rawtypes", "unchecked" }) - public void start(BSPPeer peer) - throws IOException, InterruptedException { + public void start(BSPPeer peer) throws IOException, + InterruptedException { Map env = setupEnvironment(peer.getConfiguration()); List cmd = setupCommand(peer.getConfiguration()); @@ -291,7 +289,7 @@ clientSocket = serverSocket.accept(); LOG.debug("DEBUG: Client connected! - start BinaryProtocol!"); - downlink = new BinaryProtocol(peer, + downlink = new BinaryProtocol(peer, clientSocket.getOutputStream(), clientSocket.getInputStream()); } Index: core/src/main/java/org/apache/hama/pipes/PipesBSP.java =================================================================== --- core/src/main/java/org/apache/hama/pipes/PipesBSP.java (revision 1542513) +++ core/src/main/java/org/apache/hama/pipes/PipesBSP.java (working copy) @@ -32,10 +32,10 @@ * runtimes. */ public class PipesBSP - extends BSP implements PipesApplicable { + extends BSP { private static final Log LOG = LogFactory.getLog(PipesBSP.class); - private PipesApplication application; + private PipesApplication application = new PipesApplication(); @Override public void setup(BSPPeer peer) @@ -97,12 +97,4 @@ } } - @SuppressWarnings("unchecked") - @Override - public void setApplication( - PipesApplication pipesApp) { - - this.application = (PipesApplication) pipesApp; - } - } Index: core/src/main/java/org/apache/hama/pipes/PipesApplicable.java =================================================================== --- core/src/main/java/org/apache/hama/pipes/PipesApplicable.java (revision 1542513) +++ core/src/main/java/org/apache/hama/pipes/PipesApplicable.java (working copy) @@ -1,31 +0,0 @@ -/** - * 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.hama.pipes; - -import org.apache.hadoop.io.Writable; - -/** - * This interface is implemented in Classes which should have access to the - * PipesApplication object. e.g., PipesBSP and PipesPartitioner - */ -public interface PipesApplicable { - - public void setApplication( - PipesApplication pipesApp); - -} Index: core/src/main/java/org/apache/hama/pipes/Submitter.java =================================================================== --- core/src/main/java/org/apache/hama/pipes/Submitter.java (revision 1542513) +++ core/src/main/java/org/apache/hama/pipes/Submitter.java (working copy) @@ -40,6 +40,7 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.io.BytesWritable; import org.apache.hadoop.io.Text; import org.apache.hadoop.util.GenericOptionsParser; import org.apache.hadoop.util.Tool; @@ -230,9 +231,12 @@ setIfUnset(job.getConfiguration(), "bsp.input.value.class", textClassname); setIfUnset(job.getConfiguration(), "bsp.output.key.class", textClassname); setIfUnset(job.getConfiguration(), "bsp.output.value.class", textClassname); + setIfUnset(job.getConfiguration(), "bsp.message.class", + BytesWritable.class.getName()); setIfUnset(job.getConfiguration(), "bsp.job.name", "Hama Pipes Job"); + // DEBUG Output LOG.debug("isJavaRecordReader: " + getIsJavaRecordReader(job.getConfiguration())); LOG.debug("BspClass: " + job.getBspClass().getName()); @@ -240,14 +244,11 @@ LOG.debug("InputFormat: " + job.getInputFormat()); LOG.debug("InputKeyClass: " + job.getInputKeyClass().getName()); LOG.debug("InputValueClass: " + job.getInputValueClass().getName()); + LOG.debug("InputFormat: " + job.getOutputFormat()); LOG.debug("OutputKeyClass: " + job.getOutputKeyClass().getName()); LOG.debug("OutputValueClass: " + job.getOutputValueClass().getName()); + LOG.debug("MessageClass: " + job.get("bsp.message.class")); - if ((!job.getOutputKeyClass().getName().equals(textClassname)) - || (!job.getOutputValueClass().getName().equals(textClassname))) - throw new IllegalArgumentException( - "Hama Pipes does only support Text as Key/Value output!"); - LOG.debug("bsp.master.address: " + job.getConfiguration().get("bsp.master.address")); LOG.debug("bsp.local.tasks.maximum: " @@ -258,7 +259,8 @@ String exec = getExecutable(job.getConfiguration()); if (exec == null) { - throw new IllegalArgumentException("No application defined."); + throw new IllegalArgumentException( + "No application defined. (Set property hama.pipes.executable)"); } URI[] fileCache = DistributedCache.getCacheFiles(job.getConfiguration()); @@ -273,7 +275,7 @@ try { fileCache[0] = new URI(exec); } catch (URISyntaxException e) { - IOException ie = new IOException("Problem parsing execable URI " + exec); + IOException ie = new IOException("Problem parsing executable URI " + exec); ie.initCause(e); throw ie; } Index: core/src/main/java/org/apache/hama/pipes/PipesPartitioner.java =================================================================== --- core/src/main/java/org/apache/hama/pipes/PipesPartitioner.java (revision 1542513) +++ core/src/main/java/org/apache/hama/pipes/PipesPartitioner.java (working copy) @@ -21,7 +21,8 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.io.Writable; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.io.BytesWritable; import org.apache.hama.bsp.Partitioner; /** @@ -30,13 +31,30 @@ * BinaryProtocol -> C++ Partitioner and back * */ -public class PipesPartitioner implements Partitioner, - PipesApplicable { +public class PipesPartitioner implements Partitioner { - private static final Log LOG = LogFactory.getLog(PipesPartitioner.class - .getName()); - private PipesApplication application = null; + private static final Log LOG = LogFactory.getLog(PipesPartitioner.class); + private PipesApplication application = new PipesApplication(); + public PipesPartitioner(Configuration conf) { + LOG.debug("Start Pipes client for PipesPartitioner."); + try { + application.start(conf); + } catch (IOException e) { + LOG.error(e); + } catch (InterruptedException e) { + LOG.error(e); + } + } + + public void cleanup() { + try { + application.cleanup(); + } catch (IOException e) { + LOG.error(e); + } + } + /** * Partitions a specific key value mapping to a bucket. * @@ -50,26 +68,20 @@ public int getPartition(K key, V value, int numTasks) { int returnVal = 0; try { - // LOG.info("pipesApp==null: " + ((pipesApp == null) ? "true" : "false")); - // LOG.info("pipesApp.getDownlink()==null: " - // + ((pipesApp.getDownlink() == null) ? "true" : "false")); - // LOG.info("Class: "+value.getClass().toString()); - if ((application != null) && (application.getDownlink() != null)) - returnVal = application.getDownlink().getPartition(key.toString(), - value.toString(), numTasks); + if ((application != null) && (application.getDownlink() != null)) { + returnVal = application.getDownlink() + .getPartition(key, value, numTasks); + } else { + LOG.warn("PipesApplication or application.getDownlink() might be null! (application==null): " + + ((application == null) ? "true" : "false")); + } } catch (IOException e) { LOG.error(e); } + LOG.debug("getPartition returns: " + returnVal); return returnVal; } - - @Override - public void setApplication( - PipesApplication pipesApp) { - - this.application = pipesApp; - } - + } Index: core/src/main/java/org/apache/hama/pipes/protocol/StreamingProtocol.java =================================================================== --- core/src/main/java/org/apache/hama/pipes/protocol/StreamingProtocol.java (revision 1542513) +++ core/src/main/java/org/apache/hama/pipes/protocol/StreamingProtocol.java (working copy) @@ -49,7 +49,7 @@ * @param output value. */ public class StreamingProtocol - extends BinaryProtocol { + extends BinaryProtocol { private static final Pattern PROTOCOL_STRING_PATTERN = Pattern.compile("="); @@ -62,21 +62,22 @@ } @Override - public UplinkReader getUplinkReader( + public UplinkReader getUplinkReader( BSPPeer peer, InputStream in) throws IOException { - return new StreamingUplinkReaderThread(peer, in); + return new StreamingUplinkReaderThread(this, peer, in); } - + public class StreamingUplinkReaderThread extends - UplinkReader { + UplinkReader { private BufferedReader reader; public StreamingUplinkReaderThread( + BinaryProtocol binaryProtocol, BSPPeer peer, InputStream stream) throws IOException { - super(null, peer, stream); + super(binaryProtocol, peer, stream); reader = new BufferedReader(new InputStreamReader(inStream)); } @@ -164,7 +165,7 @@ } @Override - public int readCommand() throws IOException { + protected int readCommand() throws IOException { String readLine = reader.readLine(); if (readLine != null && !readLine.isEmpty()) { String[] split = PROTOCOL_STRING_PATTERN.split(readLine, 2); @@ -211,6 +212,10 @@ } + /* ************************************************************ */ + /* Override Implementation of DownwardProtocol */ + /* ************************************************************ */ + @Override public void start() throws IOException { writeLine(MessageType.START, null); @@ -277,14 +282,14 @@ } public void writeLine(String msg) throws IOException { - stream.write((msg + "\n").getBytes()); - stream.flush(); + outStream.write((msg + "\n").getBytes()); + outStream.flush(); } public void writeLine(MessageType type, String msg) throws IOException { - stream.write((getProtocolString(type) + (msg == null ? "" : msg) + "\n") + outStream.write((getProtocolString(type) + (msg == null ? "" : msg) + "\n") .getBytes()); - stream.flush(); + outStream.flush(); } public String getProtocolString(MessageType type) { Index: core/src/main/java/org/apache/hama/pipes/protocol/UplinkReader.java =================================================================== --- core/src/main/java/org/apache/hama/pipes/protocol/UplinkReader.java (revision 1542513) +++ core/src/main/java/org/apache/hama/pipes/protocol/UplinkReader.java (working copy) @@ -34,6 +34,8 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.BytesWritable; +import org.apache.hadoop.io.IntWritable; +import org.apache.hadoop.io.LongWritable; import org.apache.hadoop.io.NullWritable; import org.apache.hadoop.io.SequenceFile; import org.apache.hadoop.io.Text; @@ -46,24 +48,23 @@ import org.apache.hama.bsp.sync.SyncException; import org.apache.hama.commons.util.KeyValuePair; -public class UplinkReader +public class UplinkReader extends Thread { private static final Log LOG = LogFactory.getLog(UplinkReader.class); - protected DataInputStream inStream; - private K2 key; - private V2 value; - - private BinaryProtocol binProtocol; - private BSPPeer peer = null; + private BinaryProtocol binProtocol; + private BSPPeer peer = null; private Configuration conf; - - private Map>> sequenceFileReaders; - private Map>> sequenceFileWriters; - @SuppressWarnings("unchecked") - public UplinkReader(BinaryProtocol binaryProtocol, + protected DataInputStream inStream; + protected DataOutputStream outStream; + + private Map>> sequenceFileReaders; + private Map>> sequenceFileWriters; + + public UplinkReader( + BinaryProtocol binaryProtocol, Configuration conf, InputStream stream) throws IOException { this.binProtocol = binaryProtocol; @@ -72,18 +73,15 @@ this.inStream = new DataInputStream(new BufferedInputStream(stream, BinaryProtocol.BUFFER_SIZE)); - this.key = (K2) ReflectionUtils.newInstance((Class) conf - .getClass("bsp.output.key.class", Object.class), conf); + this.outStream = binProtocol.getOutputStream(); - this.value = (V2) ReflectionUtils.newInstance((Class) conf - .getClass("bsp.output.value.class", Object.class), conf); - - this.sequenceFileReaders = new HashMap>>(); - this.sequenceFileWriters = new HashMap>>(); + this.sequenceFileReaders = new HashMap>>(); + this.sequenceFileWriters = new HashMap>>(); } - public UplinkReader(BinaryProtocol binaryProtocol, - BSPPeer peer, InputStream stream) + public UplinkReader( + BinaryProtocol binaryProtocol, + BSPPeer peer, InputStream stream) throws IOException { this(binaryProtocol, peer.getConfiguration(), stream); this.peer = peer; @@ -93,10 +91,6 @@ return this.peer != null; } - public void closeConnection() throws IOException { - inStream.close(); - } - @Override public void run() { while (true) { @@ -108,7 +102,7 @@ int cmd = readCommand(); if (cmd == -1) continue; - LOG.debug("Handling uplink command " + cmd); + LOG.debug("Handling uplink command: " + MessageType.values()[cmd]); if (cmd == MessageType.WRITE_KEYVALUE.code && isPeerAvailable()) { // INCOMING writeKeyValue(); @@ -182,65 +176,67 @@ } } + // onError is overwritten by StreamingProtocol in Hama Streaming protected void onError(Throwable e) { LOG.error(StringUtils.stringifyException(e)); } - public int readCommand() throws IOException { - return WritableUtils.readVInt(inStream); + // readCommand is overwritten by StreamingProtocol in Hama Streaming + protected int readCommand() throws IOException { + return WritableUtils.readVInt(this.inStream); } + public void closeConnection() throws IOException { + this.inStream.close(); + } + public void reopenInput() throws IOException { LOG.debug("Got MessageType.REOPEN_INPUT"); peer.reopenInput(); } public void getSuperstepCount() throws IOException { - DataOutputStream stream = binProtocol.getStream(); - WritableUtils.writeVInt(stream, MessageType.GET_SUPERSTEP_COUNT.code); - WritableUtils.writeVLong(stream, peer.getSuperstepCount()); + WritableUtils.writeVInt(this.outStream, + MessageType.GET_SUPERSTEP_COUNT.code); + WritableUtils.writeVLong(this.outStream, peer.getSuperstepCount()); binProtocol.flush(); - LOG.debug("Responded MessageType.GET_SUPERSTEP_COUNT - SuperstepCount: " + peer.getSuperstepCount()); } public void getPeerCount() throws IOException { - DataOutputStream stream = binProtocol.getStream(); - WritableUtils.writeVInt(stream, MessageType.GET_PEER_COUNT.code); - WritableUtils.writeVInt(stream, peer.getNumPeers()); + WritableUtils.writeVInt(this.outStream, MessageType.GET_PEER_COUNT.code); + WritableUtils.writeVInt(this.outStream, peer.getNumPeers()); binProtocol.flush(); LOG.debug("Responded MessageType.GET_PEER_COUNT - NumPeers: " + peer.getNumPeers()); } public void getPeerIndex() throws IOException { - DataOutputStream stream = binProtocol.getStream(); - WritableUtils.writeVInt(stream, MessageType.GET_PEER_INDEX.code); - WritableUtils.writeVInt(stream, peer.getPeerIndex()); + WritableUtils.writeVInt(this.outStream, MessageType.GET_PEER_INDEX.code); + WritableUtils.writeVInt(this.outStream, peer.getPeerIndex()); binProtocol.flush(); LOG.debug("Responded MessageType.GET_PEER_INDEX - PeerIndex: " + peer.getPeerIndex()); } public void getPeerName() throws IOException { - DataOutputStream stream = binProtocol.getStream(); - int id = WritableUtils.readVInt(inStream); + int id = WritableUtils.readVInt(this.inStream); LOG.debug("Got MessageType.GET_PEERNAME id: " + id); - WritableUtils.writeVInt(stream, MessageType.GET_PEERNAME.code); + WritableUtils.writeVInt(this.outStream, MessageType.GET_PEERNAME.code); if (id == -1) { // -1 indicates get own PeerName - Text.writeString(stream, peer.getPeerName()); + Text.writeString(this.outStream, peer.getPeerName()); LOG.debug("Responded MessageType.GET_PEERNAME - Get Own PeerName: " + peer.getPeerName()); } else if ((id < -1) || (id >= peer.getNumPeers())) { // if no PeerName for this index is found write emptyString - Text.writeString(stream, ""); + Text.writeString(this.outStream, ""); LOG.debug("Responded MessageType.GET_PEERNAME - Empty PeerName!"); } else { - Text.writeString(stream, peer.getPeerName(id)); + Text.writeString(this.outStream, peer.getPeerName(id)); LOG.debug("Responded MessageType.GET_PEERNAME - PeerName: " + peer.getPeerName(id)); } @@ -248,13 +244,12 @@ } public void getAllPeerNames() throws IOException { - DataOutputStream stream = binProtocol.getStream(); LOG.debug("Got MessageType.GET_ALL_PEERNAME"); - WritableUtils.writeVInt(stream, MessageType.GET_ALL_PEERNAME.code); - WritableUtils.writeVInt(stream, peer.getAllPeerNames().length); - for (String s : peer.getAllPeerNames()) - Text.writeString(stream, s); - + WritableUtils.writeVInt(this.outStream, MessageType.GET_ALL_PEERNAME.code); + WritableUtils.writeVInt(this.outStream, peer.getAllPeerNames().length); + for (String s : peer.getAllPeerNames()) { + Text.writeString(this.outStream, s); + } binProtocol.flush(); LOG.debug("Responded MessageType.GET_ALL_PEERNAME - peerNamesCount: " + peer.getAllPeerNames().length); @@ -266,44 +261,59 @@ } public void getMessage() throws IOException { - DataOutputStream stream = binProtocol.getStream(); LOG.debug("Got MessageType.GET_MSG"); - WritableUtils.writeVInt(stream, MessageType.GET_MSG.code); - BytesWritable msg = peer.getCurrentMessage(); - if (msg != null) - binProtocol.writeObject(msg); - + WritableUtils.writeVInt(this.outStream, MessageType.GET_MSG.code); + Writable message = peer.getCurrentMessage(); + if (message != null) { + binProtocol.writeObject(message); + } binProtocol.flush(); - LOG.debug("Responded MessageType.GET_MSG - Message(BytesWritable) ");// +msg); + LOG.debug("Responded MessageType.GET_MSG - Message: " + + ((message.toString().length() < 10) ? message.toString() : message + .toString().substring(0, 9) + "...")); } public void getMessageCount() throws IOException { - DataOutputStream stream = binProtocol.getStream(); - WritableUtils.writeVInt(stream, MessageType.GET_MSG_COUNT.code); - WritableUtils.writeVInt(stream, peer.getNumCurrentMessages()); + WritableUtils.writeVInt(this.outStream, MessageType.GET_MSG_COUNT.code); + WritableUtils.writeVInt(this.outStream, peer.getNumCurrentMessages()); binProtocol.flush(); LOG.debug("Responded MessageType.GET_MSG_COUNT - Count: " + peer.getNumCurrentMessages()); } - public void sendMessage() throws IOException { - String peerName = Text.readString(inStream); - BytesWritable msg = new BytesWritable(); - readObject(msg); - LOG.debug("Got MessageType.SEND_MSG to peerName: " + peerName); - peer.send(peerName, msg); + public void incrementCounter() throws IOException { + String group = Text.readString(this.inStream); + String name = Text.readString(this.inStream); + long amount = WritableUtils.readVLong(this.inStream); + peer.incrementCounter(group, name, amount); } - public void incrementCounter() throws IOException { - // int id = WritableUtils.readVInt(inStream); - String group = Text.readString(inStream); - String name = Text.readString(inStream); - long amount = WritableUtils.readVLong(inStream); - peer.incrementCounter(name, group, amount); + @SuppressWarnings("unchecked") + public void sendMessage() throws IOException, InstantiationException, + IllegalAccessException { + String peerName = Text.readString(this.inStream); + + M message = (M) ReflectionUtils.newInstance((Class) conf + .getClass("bsp.message.class", BytesWritable.class), conf); + + LOG.debug("Got MessageType.SEND_MSG peerName: " + peerName + + " messageClass: " + message.getClass().getName()); + + readObject(message); + + peer.send(peerName, message); + + LOG.debug("Done MessageType.SEND_MSG to peerName: " + + peerName + + " messageClass: " + + message.getClass().getName() + + " Message: " + + ((message.toString().length() < 10) ? message.toString() : message + .toString().substring(0, 9) + "...")); } public void readKeyValue() throws IOException { - DataOutputStream stream = binProtocol.getStream(); + boolean nullinput = peer.getConfiguration().get( Constants.INPUT_FORMAT_CLASS) == null || peer.getConfiguration().get(Constants.INPUT_FORMAT_CLASS) @@ -311,54 +321,73 @@ if (!nullinput) { - KeyValuePair pair = peer.readNext(); + KeyValuePair pair = peer.readNext(); - WritableUtils.writeVInt(stream, MessageType.READ_KEYVALUE.code); if (pair != null) { - binProtocol.writeObject(new Text(pair.getKey().toString())); - String valueStr = pair.getValue().toString(); - binProtocol.writeObject(new Text(valueStr)); + WritableUtils.writeVInt(this.outStream, MessageType.READ_KEYVALUE.code); + binProtocol.writeObject((Writable) pair.getKey()); + binProtocol.writeObject((Writable) pair.getValue()); - LOG.debug("Responded MessageType.READ_KEYVALUE - Key: " - + pair.getKey() + LOG.debug("Responded MessageType.READ_KEYVALUE -" + + " Key: " + + ((pair.getKey().toString().length() < 10) ? pair.getKey() + .toString() : pair.getKey().toString().substring(0, 9) + "...") + " Value: " - + ((valueStr.length() < 10) ? valueStr : valueStr.substring(0, 9) + + ((pair.getValue().toString().length() < 10) ? pair.getValue() + .toString() : pair.getValue().toString().substring(0, 9) + "...")); } else { - Text.writeString(stream, ""); - Text.writeString(stream, ""); - LOG.debug("Responded MessageType.READ_KEYVALUE - EMPTY KeyValue Pair"); + WritableUtils.writeVInt(this.outStream, MessageType.END_OF_DATA.code); + LOG.debug("Responded MessageType.READ_KEYVALUE - END_OF_DATA"); } binProtocol.flush(); } else { - /* TODO */ - /* Send empty Strings to show no KeyValue pair is available */ - WritableUtils.writeVInt(stream, MessageType.READ_KEYVALUE.code); - Text.writeString(stream, ""); - Text.writeString(stream, ""); + WritableUtils.writeVInt(this.outStream, MessageType.END_OF_DATA.code); binProtocol.flush(); - LOG.debug("Responded MessageType.READ_KEYVALUE - EMPTY KeyValue Pair"); + LOG.debug("Responded MessageType.READ_KEYVALUE - END_OF_DATA"); } } + @SuppressWarnings("unchecked") public void writeKeyValue() throws IOException { - readObject(key); // string or binary only - readObject(value); // string or binary only - if (LOG.isDebugEnabled()) - LOG.debug("Got MessageType.WRITE_KEYVALUE - Key: " + key + " Value: " - + value); - peer.write(key, value); + + KEYOUT keyOut = (KEYOUT) ReflectionUtils.newInstance( + (Class) conf.getClass("bsp.output.key.class", + Object.class), conf); + + VALUEOUT valueOut = (VALUEOUT) ReflectionUtils.newInstance( + (Class) conf.getClass("bsp.output.value.class", + Object.class), conf); + + LOG.debug("Got MessageType.WRITE_KEYVALUE keyOutClass: " + + keyOut.getClass().getName() + " valueOutClass: " + valueOut.getClass().getName()); + + readObject((Writable) keyOut); + readObject((Writable) valueOut); + + peer.write(keyOut, valueOut); + + LOG.debug("Done MessageType.WRITE_KEYVALUE -" + + " Key: " + + ((keyOut.toString().length() < 10) ? keyOut.toString() : keyOut + .toString().substring(0, 9) + "...") + + " Value: " + + ((valueOut.toString().length() < 10) ? valueOut.toString() : valueOut + .toString().substring(0, 9) + "...")); } public void seqFileOpen() throws IOException { - String path = Text.readString(inStream); + String path = Text.readString(this.inStream); // option - read = "r" or write = "w" - String option = Text.readString(inStream); - // key and value Type stored in the SequenceFile - String keyType = Text.readString(inStream); - String valueType = Text.readString(inStream); + String option = Text.readString(this.inStream); + // key and value class stored in the SequenceFile + String keyClass = Text.readString(this.inStream); + String valueClass = Text.readString(this.inStream); + LOG.debug("GOT MessageType.SEQFILE_OPEN - Option: " + option); + LOG.debug("GOT MessageType.SEQFILE_OPEN - KeyClass: " + keyClass); + LOG.debug("GOT MessageType.SEQFILE_OPEN - ValueClass: " + valueClass); int fileID = -1; @@ -367,98 +396,165 @@ SequenceFile.Reader reader; try { reader = new SequenceFile.Reader(fs, new Path(path), conf); + + // try to load key and value class + Class sequenceKeyClass = conf.getClassLoader().loadClass(keyClass); + Class sequenceValueClass = conf.getClassLoader().loadClass( + valueClass); + + // try to instantiate key and value class + Writable sequenceKeyWritable = (Writable) ReflectionUtils.newInstance( + sequenceKeyClass, conf); + Writable sequenceValueWritable = (Writable) ReflectionUtils + .newInstance(sequenceValueClass, conf); + + // put new fileID and key and value Writable instances into HashMap fileID = reader.hashCode(); sequenceFileReaders .put( fileID, - new AbstractMap.SimpleEntry>( - reader, new AbstractMap.SimpleEntry( - keyType, valueType))); + new AbstractMap.SimpleEntry>( + reader, new AbstractMap.SimpleEntry( + sequenceKeyWritable, sequenceValueWritable))); + } catch (IOException e) { fileID = -1; + } catch (ClassNotFoundException e) { + fileID = -1; } } else if (option.equals("w")) { SequenceFile.Writer writer; try { - writer = new SequenceFile.Writer(fs, conf, new Path(path), Text.class, - Text.class); + + // try to load key and value class + Class sequenceKeyClass = conf.getClassLoader().loadClass(keyClass); + Class sequenceValueClass = conf.getClassLoader().loadClass( + valueClass); + + writer = new SequenceFile.Writer(fs, conf, new Path(path), + sequenceKeyClass, sequenceValueClass); + + // try to instantiate key and value class + Writable sequenceKeyWritable = (Writable) ReflectionUtils.newInstance( + sequenceKeyClass, conf); + Writable sequenceValueWritable = (Writable) ReflectionUtils + .newInstance(sequenceValueClass, conf); + + // put new fileID and key and value Writable instances into HashMap fileID = writer.hashCode(); sequenceFileWriters .put( fileID, - new AbstractMap.SimpleEntry>( - writer, new AbstractMap.SimpleEntry( - keyType, valueType))); + new AbstractMap.SimpleEntry>( + writer, new AbstractMap.SimpleEntry( + sequenceKeyWritable, sequenceValueWritable))); + } catch (IOException e) { fileID = -1; + } catch (ClassNotFoundException e) { + fileID = -1; } } - DataOutputStream stream = binProtocol.getStream(); - WritableUtils.writeVInt(stream, MessageType.SEQFILE_OPEN.code); - WritableUtils.writeVInt(stream, fileID); + WritableUtils.writeVInt(this.outStream, MessageType.SEQFILE_OPEN.code); + WritableUtils.writeVInt(this.outStream, fileID); binProtocol.flush(); LOG.debug("Responded MessageType.SEQFILE_OPEN - FileID: " + fileID); } - public void seqFileReadNext() throws IOException, ClassNotFoundException { - int fileID = WritableUtils.readVInt(inStream); - // LOG.debug("GOT MessageType.SEQFILE_READNEXT - FileID: " + fileID); + public void seqFileReadNext() throws IOException { + int fileID = WritableUtils.readVInt(this.inStream); + LOG.debug("GOT MessageType.SEQFILE_READNEXT - FileID: " + fileID); - Class keyType = conf.getClassLoader().loadClass( - sequenceFileReaders.get(fileID).getValue().getKey()); - Writable key = (Writable) ReflectionUtils.newInstance(keyType, conf); + // check if fileID is available in sequenceFileReader + if (sequenceFileReaders.containsKey(fileID)) { - Class valueType = conf.getClassLoader().loadClass( - sequenceFileReaders.get(fileID).getValue().getValue()); - Writable value = (Writable) ReflectionUtils.newInstance(valueType, conf); + Writable sequenceKeyWritable = sequenceFileReaders.get(fileID).getValue() + .getKey(); + Writable sequenceValueWritable = sequenceFileReaders.get(fileID) + .getValue().getValue(); - if (sequenceFileReaders.containsKey(fileID)) - sequenceFileReaders.get(fileID).getKey().next(key, value); + // try to read next key/value pair from SequenceFile.Reader + if (sequenceFileReaders.get(fileID).getKey() + .next(sequenceKeyWritable, sequenceValueWritable)) { - // RESPOND - DataOutputStream stream = binProtocol.getStream(); - WritableUtils.writeVInt(stream, MessageType.SEQFILE_READNEXT.code); - try { - String k = key.toString(); - String v = value.toString(); - Text.writeString(stream, k); - Text.writeString(stream, v); - LOG.debug("Responded MessageType.SEQFILE_READNEXT - key: " + k - + " value: " + ((v.length() < 10) ? v : v.substring(0, 9) + "...")); + WritableUtils.writeVInt(this.outStream, + MessageType.SEQFILE_READNEXT.code); + binProtocol.writeObject(sequenceKeyWritable); + binProtocol.writeObject(sequenceValueWritable); - } catch (NullPointerException e) { // key or value is null + LOG.debug("Responded MessageType.SEQFILE_READNEXT -" + + " Key: " + + ((sequenceKeyWritable.toString().length() < 10) ? sequenceKeyWritable + .toString() : sequenceKeyWritable.toString().substring(0, 9) + + "...") + + " Value: " + + ((sequenceValueWritable.toString().length() < 10) ? sequenceValueWritable + .toString() : sequenceValueWritable.toString().substring(0, 9) + + "...")); - Text.writeString(stream, ""); - Text.writeString(stream, ""); - LOG.debug("Responded MessageType.SEQFILE_READNEXT - EMPTY KeyValue Pair"); + } else { // false when at end of file + + WritableUtils.writeVInt(this.outStream, MessageType.END_OF_DATA.code); + LOG.debug("Responded MessageType.SEQFILE_READNEXT - END_OF_DATA"); + } + binProtocol.flush(); + + } else { // no fileID stored + LOG.warn("SequenceFileReader: FileID " + fileID + " not found!"); + WritableUtils.writeVInt(this.outStream, MessageType.END_OF_DATA.code); + LOG.debug("Responded MessageType.SEQFILE_READNEXT - END_OF_DATA"); + binProtocol.flush(); } - binProtocol.flush(); } public void seqFileAppend() throws IOException { - int fileID = WritableUtils.readVInt(inStream); - String keyStr = Text.readString(inStream); - String valueStr = Text.readString(inStream); + int fileID = WritableUtils.readVInt(this.inStream); + LOG.debug("GOT MessageType.SEQFILE_APPEND - FileID: " + fileID); boolean result = false; + + // check if fileID is available in sequenceFileWriter if (sequenceFileWriters.containsKey(fileID)) { - sequenceFileWriters.get(fileID).getKey() - .append(new Text(keyStr), new Text(valueStr)); - result = true; + + Writable sequenceKeyWritable = sequenceFileReaders.get(fileID).getValue() + .getKey(); + Writable sequenceValueWritable = sequenceFileReaders.get(fileID) + .getValue().getValue(); + + // try to read key and value + readObject(sequenceKeyWritable); + readObject(sequenceValueWritable); + + if ((sequenceKeyWritable != null) && (sequenceValueWritable != null)) { + + // append to sequenceFile + sequenceFileWriters.get(fileID).getKey() + .append(sequenceKeyWritable, sequenceValueWritable); + + LOG.debug("Stored data: Key: " + + ((sequenceKeyWritable.toString().length() < 10) ? sequenceKeyWritable + .toString() : sequenceKeyWritable.toString().substring(0, 9) + + "...") + + " Value: " + + ((sequenceValueWritable.toString().length() < 10) ? sequenceValueWritable + .toString() : sequenceValueWritable.toString().substring(0, 9) + + "...")); + + result = true; + } } // RESPOND - DataOutputStream stream = binProtocol.getStream(); - WritableUtils.writeVInt(stream, MessageType.SEQFILE_APPEND.code); - WritableUtils.writeVInt(stream, result ? 1 : 0); + WritableUtils.writeVInt(this.outStream, MessageType.SEQFILE_APPEND.code); + WritableUtils.writeVInt(this.outStream, result ? 1 : 0); binProtocol.flush(); LOG.debug("Responded MessageType.SEQFILE_APPEND - Result: " + result); } public void seqFileClose() throws IOException { - int fileID = WritableUtils.readVInt(inStream); + int fileID = WritableUtils.readVInt(this.inStream); boolean result = false; @@ -471,15 +567,14 @@ } // RESPOND - DataOutputStream stream = binProtocol.getStream(); - WritableUtils.writeVInt(stream, MessageType.SEQFILE_CLOSE.code); - WritableUtils.writeVInt(stream, result ? 1 : 0); + WritableUtils.writeVInt(this.outStream, MessageType.SEQFILE_CLOSE.code); + WritableUtils.writeVInt(this.outStream, result ? 1 : 0); binProtocol.flush(); LOG.debug("Responded MessageType.SEQFILE_CLOSE - Result: " + result); } public void partitionResponse() throws IOException { - int partResponse = WritableUtils.readVInt(inStream); + int partResponse = WritableUtils.readVInt(this.inStream); synchronized (binProtocol.resultLock) { binProtocol.setResult(partResponse); LOG.debug("Received MessageType.PARTITION_RESPONSE - Result: " @@ -488,29 +583,61 @@ } } + /** + * Read the given object from stream. If it is a IntWritable, LongWritable, + * FloatWritable, DoubleWritable, Text or BytesWritable, read it directly. + * Otherwise, read it to a buffer and then write the length and data to the + * stream. + * + * @param obj the object to read + * @throws IOException + */ protected void readObject(Writable obj) throws IOException { - int numBytes = readCommand(); byte[] buffer; + // For BytesWritable and Text, use the specified length to set the length // this causes the "obvious" translations to work. So that if you emit // a string "abc" from C++, it shows up as "abc". - if (obj instanceof BytesWritable) { + + if (obj instanceof Text) { + int numBytes = WritableUtils.readVInt(this.inStream); buffer = new byte[numBytes]; - inStream.readFully(buffer); + this.inStream.readFully(buffer); + ((Text) obj).set(buffer); + + } else if (obj instanceof BytesWritable) { + int numBytes = WritableUtils.readVInt(this.inStream); + buffer = new byte[numBytes]; + this.inStream.readFully(buffer); ((BytesWritable) obj).set(buffer, 0, numBytes); - } else if (obj instanceof Text) { - buffer = new byte[numBytes]; - inStream.readFully(buffer); - ((Text) obj).set(buffer); + + } else if (obj instanceof IntWritable) { + LOG.debug("read IntWritable"); + ((IntWritable) obj).set(WritableUtils.readVInt(this.inStream)); + + } else if (obj instanceof LongWritable) { + ((LongWritable) obj).set(WritableUtils.readVLong(this.inStream)); + + // else if ((obj instanceof FloatWritable) || (obj instanceof + // DoubleWritable)) + } else if (obj instanceof NullWritable) { - throw new IOException( - "Cannot read data into NullWritable! Check OutputClasses!"); + throw new IOException("Cannot read data into NullWritable!"); + } else { - /* TODO */ - /* IntWritable, DoubleWritable */ - throw new IOException( - "Hama Pipes does only support Text as Key/Value output!"); - // obj.readFields(inStream); + // Note: other types are transfered as String which should be implemented + // in Writable itself + try { + LOG.debug("reading other type"); + // try reading object + obj.readFields(this.inStream); + // String s = Text.readString(inStream); + + } catch (IOException e) { + + throw new IOException("Hama Pipes is not able to read " + + obj.getClass().getName(), e); + } } } } Index: core/src/main/java/org/apache/hama/pipes/protocol/BinaryProtocol.java =================================================================== --- core/src/main/java/org/apache/hama/pipes/protocol/BinaryProtocol.java (revision 1542513) +++ core/src/main/java/org/apache/hama/pipes/protocol/BinaryProtocol.java (working copy) @@ -33,13 +33,13 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.io.BytesWritable; -import org.apache.hadoop.io.DataOutputBuffer; +import org.apache.hadoop.io.IntWritable; +import org.apache.hadoop.io.LongWritable; import org.apache.hadoop.io.Text; import org.apache.hadoop.io.Writable; import org.apache.hadoop.io.WritableUtils; import org.apache.hama.bsp.BSPPeer; import org.apache.hama.pipes.Submitter; -import org.apache.hama.pipes.protocol.UplinkReader; /** * This protocol is a binary implementation of the Hama Pipes protocol. @@ -47,8 +47,8 @@ * Adapted from Hadoop Pipes. * */ -public class BinaryProtocol - implements DownwardProtocol { +public class BinaryProtocol implements + DownwardProtocol { protected static final Log LOG = LogFactory.getLog(BinaryProtocol.class .getName()); @@ -57,19 +57,16 @@ * The buffer size for the command socket */ protected static final int BUFFER_SIZE = 128 * 1024; + protected final DataOutputStream outStream; + /* protected final peer is only needed by the Streaming Protocol */ + protected final BSPPeer peer; - protected final DataOutputStream stream; - protected final DataOutputBuffer buffer = new DataOutputBuffer(); - - private UplinkReader uplink; - public final Object hasTaskLock = new Object(); private boolean hasTask = false; public final Object resultLock = new Object(); private Integer resultInt = null; - /* Protected final peer is only needed by the Streaming Protocol */ - protected final BSPPeer peer; + private UplinkReader uplink; private Configuration conf; /** @@ -91,11 +88,13 @@ if (Submitter.getKeepCommandFile(conf)) { out = new TeeOutputStream("downlink.data", out); } - stream = new DataOutputStream(new BufferedOutputStream(out, BUFFER_SIZE)); - uplink = new UplinkReader(this, conf, in); - uplink.setName("pipe-uplink-handler"); - uplink.start(); + this.outStream = new DataOutputStream(new BufferedOutputStream(out, + BUFFER_SIZE)); + + this.uplink = new UplinkReader(this, conf, in); + this.uplink.setName("pipe-uplink-handler"); + this.uplink.start(); } /** @@ -108,8 +107,8 @@ * @param in The input stream to communicate on. * @throws IOException */ - public BinaryProtocol(BSPPeer peer, - OutputStream out, InputStream in) throws IOException { + public BinaryProtocol(BSPPeer peer, OutputStream out, + InputStream in) throws IOException { this.peer = peer; this.conf = peer.getConfiguration(); @@ -117,20 +116,22 @@ if (Submitter.getKeepCommandFile(conf)) { out = new TeeOutputStream("downlink.data", out); } - stream = new DataOutputStream(new BufferedOutputStream(out, BUFFER_SIZE)); - uplink = getUplinkReader(peer, in); - uplink.setName("pipe-uplink-handler"); - uplink.start(); + this.outStream = new DataOutputStream(new BufferedOutputStream(out, + BUFFER_SIZE)); + + this.uplink = getUplinkReader(peer, in); + this.uplink.setName("pipe-uplink-handler"); + this.uplink.start(); } - public UplinkReader getUplinkReader( - BSPPeer peer, InputStream in) throws IOException { - return new UplinkReader(this, peer, in); + public UplinkReader getUplinkReader( + BSPPeer peer, InputStream in) throws IOException { + return new UplinkReader(this, peer, in); } public boolean isHasTask() { - return hasTask; + return this.hasTask; } public synchronized void setHasTask(boolean hasTask) { @@ -141,8 +142,8 @@ this.resultInt = result; } - public DataOutputStream getStream() { - return stream; + public DataOutputStream getOutputStream() { + return this.outStream; } /** @@ -189,8 +190,8 @@ @Override public void start() throws IOException { LOG.debug("starting downlink"); - WritableUtils.writeVInt(stream, MessageType.START.code); - WritableUtils.writeVInt(stream, CURRENT_PROTOCOL_VERSION); + WritableUtils.writeVInt(this.outStream, MessageType.START.code); + WritableUtils.writeVInt(this.outStream, CURRENT_PROTOCOL_VERSION); flush(); LOG.debug("Sent MessageType.START"); setBSPJobConf(conf); @@ -198,15 +199,15 @@ @Override public void setBSPJobConf(Configuration conf) throws IOException { - WritableUtils.writeVInt(stream, MessageType.SET_BSPJOB_CONF.code); + WritableUtils.writeVInt(this.outStream, MessageType.SET_BSPJOB_CONF.code); List> list = new ArrayList>(); for (Entry entry : conf) { list.add(entry); } - WritableUtils.writeVInt(stream, list.size()); + WritableUtils.writeVInt(this.outStream, list.size()); for (Entry entry : list) { - Text.writeString(stream, entry.getKey()); - Text.writeString(stream, entry.getValue()); + Text.writeString(this.outStream, entry.getKey()); + Text.writeString(this.outStream, entry.getValue()); } flush(); LOG.debug("Sent MessageType.SET_BSPJOB_CONF including " + list.size() @@ -216,9 +217,9 @@ @Override public void setInputTypes(String keyType, String valueType) throws IOException { - WritableUtils.writeVInt(stream, MessageType.SET_INPUT_TYPES.code); - Text.writeString(stream, keyType); - Text.writeString(stream, valueType); + WritableUtils.writeVInt(this.outStream, MessageType.SET_INPUT_TYPES.code); + Text.writeString(this.outStream, keyType); + Text.writeString(this.outStream, valueType); flush(); LOG.debug("Sent MessageType.SET_INPUT_TYPES"); } @@ -227,9 +228,9 @@ public void runSetup(boolean pipedInput, boolean pipedOutput) throws IOException { - WritableUtils.writeVInt(stream, MessageType.RUN_SETUP.code); - WritableUtils.writeVInt(stream, pipedInput ? 1 : 0); - WritableUtils.writeVInt(stream, pipedOutput ? 1 : 0); + WritableUtils.writeVInt(this.outStream, MessageType.RUN_SETUP.code); + WritableUtils.writeVInt(this.outStream, pipedInput ? 1 : 0); + WritableUtils.writeVInt(this.outStream, pipedOutput ? 1 : 0); flush(); setHasTask(true); LOG.debug("Sent MessageType.RUN_SETUP"); @@ -239,9 +240,9 @@ public void runBsp(boolean pipedInput, boolean pipedOutput) throws IOException { - WritableUtils.writeVInt(stream, MessageType.RUN_BSP.code); - WritableUtils.writeVInt(stream, pipedInput ? 1 : 0); - WritableUtils.writeVInt(stream, pipedOutput ? 1 : 0); + WritableUtils.writeVInt(this.outStream, MessageType.RUN_BSP.code); + WritableUtils.writeVInt(this.outStream, pipedInput ? 1 : 0); + WritableUtils.writeVInt(this.outStream, pipedOutput ? 1 : 0); flush(); setHasTask(true); LOG.debug("Sent MessageType.RUN_BSP"); @@ -251,32 +252,37 @@ public void runCleanup(boolean pipedInput, boolean pipedOutput) throws IOException { - WritableUtils.writeVInt(stream, MessageType.RUN_CLEANUP.code); - WritableUtils.writeVInt(stream, pipedInput ? 1 : 0); - WritableUtils.writeVInt(stream, pipedOutput ? 1 : 0); + WritableUtils.writeVInt(this.outStream, MessageType.RUN_CLEANUP.code); + WritableUtils.writeVInt(this.outStream, pipedInput ? 1 : 0); + WritableUtils.writeVInt(this.outStream, pipedOutput ? 1 : 0); flush(); setHasTask(true); LOG.debug("Sent MessageType.RUN_CLEANUP"); } @Override - public int getPartition(String key, String value, int numTasks) - throws IOException { + public int getPartition(K1 key, V1 value, int numTasks) throws IOException { - WritableUtils.writeVInt(stream, MessageType.PARTITION_REQUEST.code); - Text.writeString(stream, key); - Text.writeString(stream, value); - WritableUtils.writeVInt(stream, numTasks); + WritableUtils.writeVInt(this.outStream, MessageType.PARTITION_REQUEST.code); + writeObject((Writable) key); + writeObject((Writable) value); + WritableUtils.writeVInt(this.outStream, numTasks); flush(); - LOG.debug("Sent MessageType.PARTITION_REQUEST - key: " + key + " value: " - + value.substring(0, 10) + "..." + " numTasks: " + numTasks); + LOG.debug("Sent MessageType.PARTITION_REQUEST - key: " + + ((key.toString().length() < 10) ? key.toString() : key.toString() + .substring(0, 9) + "...") + + " value: " + + ((value.toString().length() < 10) ? value.toString() : value + .toString().substring(0, 9) + "...") + " numTasks: " + numTasks); + int resultVal = 0; - synchronized (resultLock) { + synchronized (this.resultLock) { try { - while (resultInt == null) - resultLock.wait(); + while (resultInt == null) { + this.resultLock.wait(); + } resultVal = resultInt; resultInt = null; @@ -290,14 +296,14 @@ @Override public void abort() throws IOException { - WritableUtils.writeVInt(stream, MessageType.ABORT.code); + WritableUtils.writeVInt(this.outStream, MessageType.ABORT.code); flush(); LOG.debug("Sent MessageType.ABORT"); } @Override public void flush() throws IOException { - stream.flush(); + this.outStream.flush(); } /** @@ -310,22 +316,26 @@ public void close() throws IOException, InterruptedException { // runCleanup(pipedInput,pipedOutput); LOG.debug("closing connection"); - endOfInput(); - uplink.interrupt(); - uplink.join(); + // Only send closing message back in Hama Pipes NOT in Hama Streaming + boolean streamingEnabled = conf.getBoolean("hama.streaming.enabled", false); + if (!streamingEnabled) { + endOfInput(); + } + this.uplink.interrupt(); + this.uplink.join(); - uplink.closeConnection(); - stream.close(); + this.uplink.closeConnection(); + this.outStream.close(); } @Override public boolean waitForFinish() throws IOException, InterruptedException { // LOG.debug("waitForFinish... "+hasTask); - synchronized (hasTaskLock) { + synchronized (this.hasTaskLock) { try { - while (hasTask) - hasTaskLock.wait(); + while (this.hasTask) + this.hasTaskLock.wait(); } catch (InterruptedException e) { LOG.error(e); @@ -336,40 +346,58 @@ } public void endOfInput() throws IOException { - WritableUtils.writeVInt(stream, MessageType.CLOSE.code); + WritableUtils.writeVInt(this.outStream, MessageType.CLOSE.code); flush(); LOG.debug("Sent close command"); LOG.debug("Sent MessageType.CLOSE"); } /** - * Write the given object to the stream. If it is a Text or BytesWritable, - * write it directly. Otherwise, write it to a buffer and then write the - * length and data to the stream. + * Write the given object to the stream. If it is a IntWritable, LongWritable, + * FloatWritable, DoubleWritable, Text or BytesWritable, write it directly. + * Otherwise, write it to a buffer and then write the length and data to the + * stream. * * @param obj the object to write * @throws IOException */ protected void writeObject(Writable obj) throws IOException { - // For Text and BytesWritable, encode them directly, so that they end up + // For basic types IntWritable, LongWritable, FloatWritable, DoubleWritable, + // Text and BytesWritable, encode them directly, so that they end up // in C++ as the natural translations. if (obj instanceof Text) { Text t = (Text) obj; int len = t.getLength(); - WritableUtils.writeVInt(stream, len); - stream.write(t.getBytes(), 0, len); + WritableUtils.writeVInt(this.outStream, len); + this.outStream.write(t.getBytes(), 0, len); + } else if (obj instanceof BytesWritable) { BytesWritable b = (BytesWritable) obj; int len = b.getLength(); - WritableUtils.writeVInt(stream, len); - stream.write(b.getBytes(), 0, len); + WritableUtils.writeVInt(this.outStream, len); + this.outStream.write(b.getBytes(), 0, len); + + } else if (obj instanceof IntWritable) { + WritableUtils.writeVInt(this.outStream, ((IntWritable) obj).get()); + + } else if (obj instanceof LongWritable) { + WritableUtils.writeVLong(this.outStream, ((LongWritable) obj).get()); + + // else if ((obj instanceof FloatWritable) || (obj instanceof + // DoubleWritable)) + } else { - buffer.reset(); - obj.write(buffer); - int length = buffer.getLength(); - WritableUtils.writeVInt(stream, length); - stream.write(buffer.getData(), 0, length); + // Note: other types are transfered as String which should be implemented + // in Writable itself + + // DataOutputBuffer buffer = new DataOutputBuffer(); + // buffer.reset(); + // obj.write(buffer); + // int length = buffer.getLength(); + // WritableUtils.writeVInt(stream, length); + // stream.write(buffer.getData(), 0, length); + + obj.write(this.outStream); } } - } Index: core/src/main/java/org/apache/hama/pipes/protocol/MessageType.java =================================================================== --- core/src/main/java/org/apache/hama/pipes/protocol/MessageType.java (revision 1542513) +++ core/src/main/java/org/apache/hama/pipes/protocol/MessageType.java (working copy) @@ -32,7 +32,7 @@ REOPEN_INPUT(17), CLEAR(18), CLOSE(19), ABORT(20), DONE(21), TASK_DONE(22), REGISTER_COUNTER(23), INCREMENT_COUNTER(24), SEQFILE_OPEN(25), SEQFILE_READNEXT(26), SEQFILE_APPEND(27), SEQFILE_CLOSE(28), - PARTITION_REQUEST(29), PARTITION_RESPONSE(30), LOG(31); + PARTITION_REQUEST(29), PARTITION_RESPONSE(30), LOG(31), END_OF_DATA(32); final int code; Index: core/src/main/java/org/apache/hama/pipes/protocol/DownwardProtocol.java =================================================================== --- core/src/main/java/org/apache/hama/pipes/protocol/DownwardProtocol.java (revision 1542513) +++ core/src/main/java/org/apache/hama/pipes/protocol/DownwardProtocol.java (working copy) @@ -21,7 +21,6 @@ import java.io.IOException; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.io.Writable; /** * The abstract description of the downward (from Java to C++) Pipes protocol. @@ -31,7 +30,7 @@ * Adapted from Hadoop Pipes. * */ -public interface DownwardProtocol { +public interface DownwardProtocol { /** * Start communication @@ -82,7 +81,7 @@ * * @throws IOException */ - int getPartition(String key, String value, int numTasks) throws IOException; + int getPartition(K1 key, V1 value, int numTasks) throws IOException; /** * The task should stop as soon as possible, because something has gone wrong. Index: c++/src/CMakeLists.txt =================================================================== --- c++/src/CMakeLists.txt (revision 1542513) +++ c++/src/CMakeLists.txt (working copy) @@ -34,7 +34,7 @@ function(output_directory TGT DIR) SET_TARGET_PROPERTIES(${TGT} PROPERTIES RUNTIME_OUTPUT_DIRECTORY "${CMAKE_BINARY_DIR}/${DIR}") - SET_TARGET_PROPERTIES(${TGT} PROPERTIES + SET_TARGET_PROPERTIES(${TGT} PROPERTIES ARCHIVE_OUTPUT_DIRECTORY "${CMAKE_BINARY_DIR}/${DIR}") SET_TARGET_PROPERTIES(${TGT} PROPERTIES LIBRARY_OUTPUT_DIRECTORY "${CMAKE_BINARY_DIR}/${DIR}") @@ -49,15 +49,15 @@ # Example programs add_executable(summation main/native/examples/impl/summation.cc) -target_link_libraries(summation hamapipes hadooputils) +target_link_libraries(summation hadooputils) output_directory(summation examples) add_executable(piestimator main/native/examples/impl/piestimator.cc) -target_link_libraries(piestimator hamapipes hadooputils) +target_link_libraries(piestimator hadooputils) output_directory(piestimator examples) add_executable(matrixmultiplication main/native/examples/impl/matrixmultiplication.cc) -target_link_libraries(matrixmultiplication DenseDoubleVector hamapipes hadooputils) +target_link_libraries(matrixmultiplication DenseDoubleVector hadooputils) output_directory(matrixmultiplication examples) add_library(DenseDoubleVector @@ -70,11 +70,11 @@ main/native/utils/impl/SerialUtils.cc ) -add_library(hamapipes STATIC - main/native/pipes/impl/HamaPipes.cc -) +#add_library(hamapipes STATIC +# main/native/pipes/impl/Pipes.cc +#) -target_link_libraries(hamapipes +#target_link_libraries(hamapipes # ${OPENSSL_LIBRARIES} - pthread -) +# pthread +#) Index: c++/src/main/native/examples/input/MatrixA.seq =================================================================== Cannot display: file marked as a binary type. svn:mime-type = application/octet-stream Index: c++/src/main/native/examples/input/MatrixB_transposed.seq =================================================================== Cannot display: file marked as a binary type. svn:mime-type = application/octet-stream Index: c++/src/main/native/examples/impl/DenseDoubleVector.cc =================================================================== --- c++/src/main/native/examples/impl/DenseDoubleVector.cc (revision 1542513) +++ c++/src/main/native/examples/impl/DenseDoubleVector.cc (working copy) @@ -32,25 +32,26 @@ namespace math { - DenseDoubleVector::DenseDoubleVector(int len) : size(len), vector(new double[len]) { + DenseDoubleVector::DenseDoubleVector(int len) : vector(new double[len]), size(len) { } - DenseDoubleVector::DenseDoubleVector(int len, double val) : size(len), vector(new double[len]) { + DenseDoubleVector::DenseDoubleVector(int len, double val) : vector(new double[len]), size(len) { for (int i=0; i { + private: + string master_task_; - void setup(BSPContext& context) { + public: + SummationBSP(BSPContext& context) { } + + void setup(BSPContext& context) { // Choose one as a master - masterTask = context.getPeerName(context.getNumPeers() / 2); + master_task_ = context.getPeerName(context.getNumPeers() / 2); } - void bsp(BSPContext& context) { + void bsp(BSPContext& context) { - double intermediateSum = 0.0; + double intermediate_sum = 0.0; string key; string value; while(context.readNext(key,value)) { - intermediateSum += HadoopUtils::toDouble(value); + intermediate_sum += HadoopUtils::toDouble(value); } - context.sendMessage(masterTask, HadoopUtils::toString(intermediateSum)); + context.sendMessage(master_task_, intermediate_sum); context.sync(); } - void cleanup(BSPContext& context) { - if (context.getPeerName().compare(masterTask)==0) { + void cleanup(BSPContext& context) { + if (context.getPeerName().compare(master_task_)==0) { double sum = 0.0; - int msgCount = context.getNumCurrentMessages(); - for (int i=0; i()); + return HamaPipes::runTask(HamaPipes::TemplateFactory()); } Index: c++/src/main/native/examples/impl/piestimator.cc =================================================================== --- c++/src/main/native/examples/impl/piestimator.cc (revision 1542513) +++ c++/src/main/native/examples/impl/piestimator.cc (working copy) @@ -33,31 +33,32 @@ using HamaPipes::BSPContext; using namespace HadoopUtils; -class PiEstimatorBSP: public BSP { -private: - string masterTask; - long iterations; // iterations_per_bsp_task -public: - PiEstimatorBSP(BSPContext& context) { - iterations = 1000000L; +class PiEstimatorBSP: public BSP { + private: + string master_task_; + long iterations_; // iterations_per_bsp_task + + public: + PiEstimatorBSP(BSPContext& context) { + iterations_ = 1000000L; } inline double closed_interval_rand(double x0, double x1) { return x0 + (x1 - x0) * rand() / ((double) RAND_MAX); } - void setup(BSPContext& context) { + void setup(BSPContext& context) { // Choose one as a master - masterTask = context.getPeerName(context.getNumPeers() / 2); + master_task_ = context.getPeerName(context.getNumPeers() / 2); } - void bsp(BSPContext& context) { + void bsp(BSPContext& context) { /* initialize random seed */ srand(time(NULL)); int in = 0; - for (long i = 0; i < iterations; i++) { + for (long i = 0; i < iterations_; i++) { double x = 2.0 * closed_interval_rand(0, 1) - 1.0; double y = 2.0 * closed_interval_rand(0, 1) - 1.0; if (sqrt(x * x + y * y) < 1.0) { @@ -65,28 +66,26 @@ } } - context.sendMessage(masterTask, toString(in)); + context.sendMessage(master_task_, in); context.sync(); } - void cleanup(BSPContext& context) { - if (context.getPeerName().compare(masterTask)==0) { + void cleanup(BSPContext& context) { + if (context.getPeerName().compare(master_task_)==0) { - long totalHits = 0; - int msgCount = context.getNumCurrentMessages(); - string received; - for (int i=0; i()); + return HamaPipes::runTask(HamaPipes::TemplateFactory()); } Index: c++/src/main/native/examples/impl/matrixmultiplication.cc =================================================================== --- c++/src/main/native/examples/impl/matrixmultiplication.cc (revision 1542513) +++ c++/src/main/native/examples/impl/matrixmultiplication.cc (working copy) @@ -24,119 +24,121 @@ #include #include #include -#include #include #include using std::string; -using std::cout; +using std::vector; using HamaPipes::BSP; using HamaPipes::BSPJob; using HamaPipes::Partitioner; using HamaPipes::BSPContext; -using namespace HadoopUtils; using math::DenseDoubleVector; -class MatrixMultiplicationBSP: public BSP { +class MatrixMultiplicationBSP: public BSP { private: - string masterTask; - int seqFileID; - string HAMA_MAT_MULT_B_PATH; + string master_task_; + int seq_file_id_; + string HAMA_MAT_MULT_B_PATH_; + public: - MatrixMultiplicationBSP(BSPContext& context) { - seqFileID = 0; - HAMA_MAT_MULT_B_PATH = "hama.mat.mult.B.path"; + MatrixMultiplicationBSP(BSPContext& context) { + seq_file_id_ = 0; + HAMA_MAT_MULT_B_PATH_ = "hama.mat.mult.B.path"; } - void setup(BSPContext& context) { + void setup(BSPContext& context) { // Choose one as a master - masterTask = context.getPeerName(context.getNumPeers() / 2); + master_task_ = context.getPeerName(context.getNumPeers() / 2); reopenMatrixB(context); } - void bsp(BSPContext& context) { + void bsp(BSPContext& context) { - string aRowKey; - string aRowVectorStr; + int a_row_key = 0; + string a_row_vector_str; // while for each row of matrix A - while(context.readNext(aRowKey, aRowVectorStr)) { + while(context.readNext(a_row_key, a_row_vector_str)) { - DenseDoubleVector *aRowVector = new DenseDoubleVector(aRowVectorStr); - DenseDoubleVector *colValues = NULL; + DenseDoubleVector *a_row_vector = new DenseDoubleVector(a_row_vector_str); - string bColKey; - string bColVectorStr; + int b_col_key = 0; + string b_col_vector_str; + // dynamic column values, depend on matrix B cols + vector col_values; + // while for each col of matrix B - while (context.sequenceFileReadNext(seqFileID,bColKey,bColVectorStr)) { + while (context.sequenceFileReadNext(seq_file_id_, b_col_key, b_col_vector_str)) { - DenseDoubleVector *bColVector = new DenseDoubleVector(bColVectorStr); + DenseDoubleVector *b_col_vector = new DenseDoubleVector(b_col_vector_str); - if (colValues == NULL) - colValues = new DenseDoubleVector(bColVector->getDimension()); + double dot = a_row_vector->dot(b_col_vector); - double dot = aRowVector->dot(bColVector); - - colValues->set(toInt(bColKey), dot); + col_values.push_back(dot); } + DenseDoubleVector *col_values_vector = new DenseDoubleVector(col_values.size(), col_values.data()); + // Submit one calculated row + // :key:value1,value2,value3 std::stringstream message; - message << aRowKey << ":" << colValues->toString(); - context.sendMessage(masterTask, message.str()); + message << ":" << a_row_key << ":" << col_values_vector->toString(); + context.sendMessage(master_task_, message.str()); reopenMatrixB(context); } - context.sequenceFileClose(seqFileID); + context.sequenceFileClose(seq_file_id_); context.sync(); } - void cleanup(BSPContext& context) { - if (context.getPeerName().compare(masterTask)==0) { + void cleanup(BSPContext& context) { + if (context.getPeerName().compare(master_task_)==0) { - int msgCount = context.getNumCurrentMessages(); + int msg_count = context.getNumCurrentMessages(); - for (int i=0; i& context) { + if (seq_file_id_!=0) { + context.sequenceFileClose(seq_file_id_); } const BSPJob* job = context.getBSPJob(); - string path = job->get(HAMA_MAT_MULT_B_PATH); + string path = job->get(HAMA_MAT_MULT_B_PATH_); - seqFileID = context.sequenceFileOpen(path,"r", - "org.apache.hadoop.io.IntWritable", - "org.apache.hama.ml.writable.VectorWritable"); + seq_file_id_ = context.sequenceFileOpen(path,"r", + "org.apache.hadoop.io.IntWritable", + "org.apache.hama.commons.io.PipesVectorWritable"); } }; -class MatrixRowPartitioner: public Partitioner { +class MatrixRowPartitioner: public Partitioner { public: - MatrixRowPartitioner(BSPContext& context) { } + MatrixRowPartitioner(BSPContext& context) { } - int partition(const string& key,const string& value, int32_t numTasks) { - return toInt(key) % numTasks; + int partition(const int& key,const string& value, int32_t num_tasks) { + return key % num_tasks; } }; int main(int argc, char *argv[]) { - return HamaPipes::runTask(HamaPipes::TemplateFactory()); + return HamaPipes::runTask(HamaPipes::TemplateFactory()); } Index: c++/src/main/native/examples/impl/DenseDoubleVector.hh =================================================================== --- c++/src/main/native/examples/impl/DenseDoubleVector.hh (revision 1542513) +++ c++/src/main/native/examples/impl/DenseDoubleVector.hh (working copy) @@ -32,7 +32,7 @@ /// Creates a new vector with the given length and default value. DenseDoubleVector(int length, double val); // Creates a new vector with the given array. - DenseDoubleVector(double arr[]); + DenseDoubleVector(int length, double arr[]); DenseDoubleVector(const string values); ~DenseDoubleVector(); // Destructor Index: c++/src/main/native/examples/conf/summation.xml =================================================================== --- c++/src/main/native/examples/conf/summation.xml (revision 1542513) +++ c++/src/main/native/examples/conf/summation.xml (working copy) @@ -34,10 +34,30 @@ org.apache.hama.bsp.KeyValueTextInputFormat + bsp.input.key.class + org.apache.hadoop.io.Text + + + bsp.input.value.class + org.apache.hadoop.io.Text + + bsp.output.format.class - org.apache.hama.bsp.TextOutputFormat + org.apache.hama.bsp.SequenceFileOutputFormat + bsp.output.key.class + org.apache.hadoop.io.Text + + + bsp.output.value.class + org.apache.hadoop.io.DoubleWritable + + + bsp.message.class + org.apache.hadoop.io.DoubleWritable + + hama.pipes.logging false Index: c++/src/main/native/examples/conf/piestimator.xml =================================================================== --- c++/src/main/native/examples/conf/piestimator.xml (revision 1542513) +++ c++/src/main/native/examples/conf/piestimator.xml (working copy) @@ -35,9 +35,21 @@ bsp.output.format.class - org.apache.hama.bsp.TextOutputFormat + org.apache.hama.bsp.SequenceFileOutputFormat + bsp.output.key.class + org.apache.hadoop.io.Text + + + bsp.output.value.class + org.apache.hadoop.io.DoubleWritable + + + bsp.message.class + org.apache.hadoop.io.IntWritable + + hama.pipes.logging false Index: c++/src/main/native/examples/conf/matrixmultiplication.xml =================================================================== --- c++/src/main/native/examples/conf/matrixmultiplication.xml (revision 1542513) +++ c++/src/main/native/examples/conf/matrixmultiplication.xml (working copy) @@ -30,10 +30,6 @@ true - bsp.input.format.class - org.apache.hama.bsp.SequenceFileInputFormat - - bsp.input.dir /examples/input/matrixmultiplication/MatrixA.seq @@ -42,15 +38,47 @@ /examples/input/matrixmultiplication/MatrixB_transposed.seq - hama.pipes.logging - false + bsp.input.format.class + org.apache.hama.bsp.SequenceFileInputFormat - hama.messenger.queue.class - org.apache.hama.bsp.message.queue.SortedMessageQueue + bsp.input.key.class + org.apache.hadoop.io.IntWritable + bsp.input.value.class + org.apache.hama.commons.io.PipesVectorWritable + + + bsp.output.format.class + org.apache.hama.bsp.SequenceFileOutputFormat + + + bsp.output.key.class + org.apache.hadoop.io.IntWritable + + + bsp.output.value.class + org.apache.hama.commons.io.PipesVectorWritable + + + bsp.message.class + org.apache.hama.commons.io.PipesKeyValueWritable + + + hama.messenger.xfer.queue.class + org.apache.hama.bsp.message.queue.SortedMessageTransferProtocol + + bsp.input.partitioner.class org.apache.hama.pipes.PipesPartitioner + + hama.pipes.logging + false + + + bsp.peers.num + 3 + Index: c++/src/main/native/examples/README.txt =================================================================== --- c++/src/main/native/examples/README.txt (revision 1542513) +++ c++/src/main/native/examples/README.txt (working copy) @@ -1,113 +1,127 @@ #################################################################### -# Hama Pipes Examples # +# Hama Pipes Examples README # #################################################################### -# - Summation # -# - PiEstimator # -# - MatrixMultiplication # +# Hama Pipes includes the following three examples: # +# - 1) Summation # +# - 2) PiEstimator # +# - 3) MatrixMultiplication # #################################################################### To run the examples, first compile them: -% mvn install +% mvn install -and then copy the binaries to dfs: - -% hadoop fs -put c++/target/native/examples/summation \ - /examples/bin/summation - -create an input directory with text files: - -% hadoop fs -put my-data in-dir - -and run the word count example: - -% hama pipes \ - -conf c++/src/main/native/examples/conf/summation.xml \ - -input in-dir -output out-dir - #################################################################### +# 1) Summation Example # +#################################################################### -# Summation Example +First copy summation binary to dfs % hadoop fs -put c++/target/native/examples/summation \ /examples/bin/summation +Generate input data and copy to dfs + % echo -e "key1\t1.0\nkey2\t2.0\nkey3\t3.0\nkey4\t4.0\n\ key5\t5.0\nkey6\t6.0\nkey7\t7.0\nkey8\t8.0\nkey9\t9.0\n\ key10\t10.0" > summation.txt && hadoop fs -put summation.txt \ /examples/input/summation/input.txt && rm summation.txt +Run summation example + % hama pipes \ -conf c++/src/main/native/examples/conf/summation.xml \ -input /examples/input/summation \ -output /examples/output/summation +View input and output data + % hadoop fs -cat /examples/input/summation/input.txt -% hadoop fs -cat /examples/output/summation/part-00000 +% hama seqdumper -seqFile /examples/output/summation/part-00000 +You should see +# Input Path: /examples/output/summation/part-00000 +# Key class: class org.apache.hadoop.io.Text +# Value Class: class org.apache.hadoop.io.DoubleWritable +# Key: Sum: Value: 55.0 +# Count: 1 + +Delete input and output folder + % hadoop fs -rmr /examples/input/summation % hadoop fs -rmr /examples/output/summation #################################################################### +# 2) PiEstimator Example # +#################################################################### -# PiEstimator Example +First copy piestimator binary to dfs % hadoop fs -put c++/target/native/examples/piestimator \ /examples/bin/piestimator +Run piestimator example because no input data is needed + % hama pipes \ -conf c++/src/main/native/examples/conf/piestimator.xml \ -output /examples/output/piestimator -% hadoop fs -cat /examples/output/piestimator/part-00001 +View output data +% hama seqdumper -seqFile /examples/output/piestimator/part-00001 + +You should see +# Input Path: /examples/output/piestimator/part-00001 +# Key class: class org.apache.hadoop.io.Text +# Value Class: class org.apache.hadoop.io.DoubleWritable +# Key: Estimated value of PI: Value: 3.139116 +# Count: 1 + +Delete output folder + % hadoop fs -rmr /examples/output/piestimator #################################################################### +# 3) MatrixMultiplication Example # +#################################################################### -# MatrixMultiplication Example +First copy matrixmultiplication binary to dfs % hadoop fs -put c++/target/native/examples/matrixmultiplication \ /examples/bin/matrixmultiplication -% hadoop fs -put c++/src/main/native/examples/input/MatrixA.seq \ - /examples/input/matrixmultiplication/MatrixA.seq +Generate input data -% hadoop fs -put \ - c++/src/main/native/examples/input/MatrixB_transposed.seq \ - /examples/input/matrixmultiplication/MatrixB_transposed.seq +% hama jar dist/target/hama-*/hama-*/hama-examples-*.jar \ + gen vectorwritablematrix 4 4 \ + /examples/input/matrixmultiplication/MatrixA.seq \ + false true 0 10 0 +% hama jar dist/target/hama-*/hama-*/hama-examples-*.jar \ + gen vectorwritablematrix 4 4 \ + /examples/input/matrixmultiplication/MatrixB_transposed.seq \ + false true 0 10 0 + +Run matrixmultiplication example + % hama pipes \ -conf c++/src/main/native/examples/conf/matrixmultiplication.xml \ -output /examples/output/matrixmultiplication +View input and output data + % hama seqdumper \ -seqFile /examples/input/matrixmultiplication/MatrixA.seq % hama seqdumper \ - -seqFile /examples/input/matrixmultiplication/MatrixB_transposed.seq + -seqFile \ + /examples/input/matrixmultiplication/MatrixB_transposed.seq -% hadoop fs -cat /examples/output/matrixmultiplication/part-00000 +% hama seqdumper -seqFile \ + /examples/output/matrixmultiplication/part-00001 -# Matrix A -# 9 4 1 9 -# 1 8 6 3 -# 8 3 3 9 -# 7 1 9 6 +Delete output folder -# Matrix B (not transposed) -# 2 1 6 5 -# 7 8 9 5 -# 2 1 5 8 -# 7 4 4 9 - -# Resulting Matrix C -# 111.0 78.0 131.0 154.0 -# 91.0 83.0 120.0 120.0 -# 106.0 71.0 126.0 160.0 -# 81.0 48.0 120.0 166.0 - % hadoop fs -rmr /examples/output/matrixmultiplication #################################################################### - Index: c++/src/main/native/pipes/api/hama/TemplateFactory.hh =================================================================== --- c++/src/main/native/pipes/api/hama/TemplateFactory.hh (revision 1542513) +++ c++/src/main/native/pipes/api/hama/TemplateFactory.hh (working copy) @@ -19,27 +19,31 @@ #define HAMA_PIPES_TEMPLATE_FACTORY_HH namespace HamaPipes { - - template - class TemplateFactory2: public Factory { + + /* Generic template factory specification */ + template + class TemplateFactory2: public Factory { public: - BSP* createBSP(BSPContext& context) const { + BSP* createBSP(BSPContext& context) const { return new BSP(context); } }; - - template - class TemplateFactory: public TemplateFactory2 { + + /* Template factory including partitioner specification */ + template + class TemplateFactory: public TemplateFactory2 { public: - partitioner* createPartitioner(BSPContext& context) const { - return new partitioner(context); - } + partitioner* createPartitioner(BSPContext& context) const { + return new partitioner(context); + } }; - template - class TemplateFactory - : public TemplateFactory2 { + + /* Template factory without partitioner specification */ + template + class TemplateFactory + : public TemplateFactory2 { }; - + } #endif Index: c++/src/main/native/pipes/api/hama/Pipes.hh =================================================================== --- c++/src/main/native/pipes/api/hama/Pipes.hh (revision 1542513) +++ c++/src/main/native/pipes/api/hama/Pipes.hh (working copy) @@ -18,357 +18,709 @@ #ifndef HAMA_PIPES_HH #define HAMA_PIPES_HH -#ifdef SWIG -%module (directors="1") HamaPipes -%include "std_string.i" -%feature("director") BSP; -%feature("director") Partitioner; -%feature("director") RecordReader; -%feature("director") RecordWriter; -%feature("director") Factory; -#else +#include +#include +#include +#include +#include /* ostringstream */ +#include +#include /* printf */ +#include /* getenv */ #include +#include +#include +#include +#include /* typeid */ +#include /* sleep */ #include -#endif -#include +#include "hadoop/SerialUtils.hh" +#include "hadoop/StringUtils.hh" +#define stringify( name ) # name + +using std::map; using std::string; using std::vector; +using std::pair; +using namespace HadoopUtils; + namespace HamaPipes { - -/** - * This interface defines the interface between application code and the - * foreign code interface to Hadoop Map/Reduce. - */ - -/** - * A BSPJob defines the properties for a job. - */ -class BSPJob { -public: - virtual bool hasKey(const string& key) const = 0; - virtual const string& get(const string& key) const = 0; - virtual int getInt(const string& key) const = 0; - virtual float getFloat(const string& key) const = 0; - virtual bool getBoolean(const string&key) const = 0; - virtual ~BSPJob() {} -}; - -/** - * Task context provides the information about the task and job. - */ -class TaskContext { -public: + + // global varibales + bool logging; + + /********************************************/ + /*************** MESSAGE_TYPE ***************/ + /********************************************/ + enum MESSAGE_TYPE { + START_MESSAGE, SET_BSPJOB_CONF, SET_INPUT_TYPES, + RUN_SETUP, RUN_BSP, RUN_CLEANUP, + READ_KEYVALUE, WRITE_KEYVALUE, + GET_MSG, GET_MSG_COUNT, + SEND_MSG, SYNC, + GET_ALL_PEERNAME, GET_PEERNAME, + GET_PEER_INDEX, GET_PEER_COUNT, GET_SUPERSTEP_COUNT, + REOPEN_INPUT, CLEAR, + CLOSE, ABORT, + DONE, TASK_DONE, + REGISTER_COUNTER, INCREMENT_COUNTER, + SEQFILE_OPEN, SEQFILE_READNEXT, + SEQFILE_APPEND, SEQFILE_CLOSE, + PARTITION_REQUEST, PARTITION_RESPONSE, + LOG, END_OF_DATA + }; + + /* Only needed for debugging output */ + const char* messageTypeNames[] = { + stringify( START_MESSAGE ), stringify( SET_BSPJOB_CONF ), stringify( SET_INPUT_TYPES ), + stringify( RUN_SETUP ), stringify( RUN_BSP ), stringify( RUN_CLEANUP ), + stringify( READ_KEYVALUE ), stringify( WRITE_KEYVALUE ), + stringify( GET_MSG ), stringify( GET_MSG_COUNT ), + stringify( SEND_MSG ), stringify( SYNC ), + stringify( GET_ALL_PEERNAME ), stringify( GET_PEERNAME ), + stringify( GET_PEER_INDEX ), stringify( GET_PEER_COUNT ), stringify( GET_SUPERSTEP_COUNT ), + stringify( REOPEN_INPUT ), stringify( CLEAR ), + stringify( CLOSE ), stringify( ABORT ), + stringify( DONE ), stringify( TASK_DONE ), + stringify( REGISTER_COUNTER ), stringify( INCREMENT_COUNTER ), + stringify( SEQFILE_OPEN ), stringify( SEQFILE_READNEXT ), + stringify( SEQFILE_APPEND ), stringify( SEQFILE_CLOSE ), + stringify( PARTITION_REQUEST ), stringify( PARTITION_RESPONSE ), + stringify( LOG ), stringify( END_OF_DATA ) + }; + /** - * Counter to keep track of a property and its value. + * Generic KeyValuePair including is_empty */ - class Counter { - private: - int id; - public: - Counter(int counterId) : id(counterId) {} - Counter(const Counter& counter) : id(counter.id) {} - - int getId() const { return id; } + template + struct KeyValuePair : pair { + typedef pair base_t; + bool is_empty; + + KeyValuePair() : is_empty(false) {} + explicit KeyValuePair(bool x) : is_empty(x) {} + KeyValuePair(const K& k, const V& v) : base_t(k, v), is_empty(false) {} + + template + KeyValuePair(const pair &p) : base_t(p), is_empty(false) {} + + template + KeyValuePair(const KeyValuePair &p) : base_t(p), is_empty(p.is_empty) {} }; /** - * Get the BSPJob for the current task. + * Override Generic KeyValuePair << operator */ - virtual const BSPJob* getBSPJob() = 0; - + template + OS &operator<<(OS &os, const KeyValuePair& p) { + os << ""; + return os; + } + + /********************************************/ + /****************** BSPJob ******************/ + /********************************************/ /** - * Get the current key. - * @return the current key + * A BSPJob defines the properties for a job. */ - //virtual const string& getInputKey() = 0; - - /** - * Get the current value. - * @return the current value - */ - //virtual const string& getInputValue() = 0; - - /** - * Generate an output record - */ - //virtual void emit(const string& key, const string& value) = 0; - - /** - * Mark your task as having made progress without changing the status - * message. - */ - //virtual void progress() = 0; - - /** - * Set the status message and call progress. - */ - //virtual void setStatus(const string& status) = 0; - - /** - * Register a counter with the given group and name. - */ - //virtual Counter* getCounter(const string& group, const string& name) = 0; - - /** - * Increment the value of the counter with the given amount. - */ - //virtual void incrementCounter(const Counter* counter, uint64_t amount) = 0; - virtual void incrementCounter(const string& group, const string& name, uint64_t amount) = 0; + class BSPJob { + public: + virtual bool hasKey(const string& key) const = 0; + virtual const string& get(const string& key) const = 0; + virtual int getInt(const string& key) const = 0; + virtual float getFloat(const string& key) const = 0; + virtual bool getBoolean(const string& key) const = 0; + virtual ~BSPJob() {} + }; - virtual ~TaskContext() {} -}; + /********************************************/ + /**************** BSPJobImpl ****************/ + /********************************************/ + class BSPJobImpl: public BSPJob { + private: + map values_; + public: + void set(const string& key, const string& value) { + values_[key] = value; + } + virtual bool hasKey(const string& key) const { + return values_.find(key) != values_.end(); + } -/** - * SequenceFile Connector - */ -class SequenceFileConnector { -public: - /** - * Open SequenceFile with option "r" or "w" - * key and value type of the values stored in the SequenceFile - * @return the corresponding fileID - */ - virtual int sequenceFileOpen(const string& path, const string& option, const string& keyType, const string& valueType) = 0; + virtual const string& get(const string& key) const { + map::const_iterator itr = values_.find(key); + if (itr == values_.end()) { + throw Error("Key " + key + " not found in BSPJob"); + } + return itr->second; + } - /** - * Read next key/value pair from the SequenceFile with fileID - */ - virtual bool sequenceFileReadNext(int fileID, string& key, string& value) = 0; - - /** - * Append the next key/value pair to the SequenceFile with fileID - */ - virtual bool sequenceFileAppend(int fileID, const string& key, const string& value) = 0; + virtual int getInt(const string& key) const { + const string& val = get(key); + return toInt(val); + } + virtual float getFloat(const string& key) const { + const string& val = get(key); + return toFloat(val); + } + + virtual bool getBoolean(const string& key) const { + const string& val = get(key); + return toBool(val); + } + }; + + /********************************************/ + /**************** TaskContext ***************/ + /********************************************/ /** - * Close SequenceFile + * Task context provides the information about the task and job. */ - virtual bool sequenceFileClose(int fileID) = 0; -}; - - -class BSPContext: public TaskContext, public SequenceFileConnector { -public: - - /** - * Access the InputSplit of the mapper. - */ - //virtual const string& getInputSplit() = 0; - - /** - * Get the name of the key class of the input to this task. - */ - virtual const string& getInputKeyClass() = 0; - - /** - * Get the name of the value class of the input to this task. - */ - virtual const string& getInputValueClass() = 0; + class TaskContext { + public: + /** + * Counter to keep track of a property and its value. + */ + class Counter { + private: + int id_; + public: + Counter(int counter_id) : id_(counter_id) {} + Counter(const Counter& counter) : id_(counter.id_) {} + + int getId() const { return id_; } + }; + /** + * Get the BSPJob for the current task. + */ + virtual const BSPJob* getBSPJob() = 0; + /** + * Get the current key. + * @return the current key + */ + //virtual const string& getInputKey() = 0; - /** - * Send a data with a tag to another BSPSlave corresponding to hostname. - * Messages sent by this method are not guaranteed to be received in a sent - * order. - */ - virtual void sendMessage(const string& peerName, const string& msg) = 0; + /** + * Get the current value. + * @return the current value + */ + //virtual const string& getInputValue() = 0; - /** - * @return A message from the peer's received messages queue (a FIFO). - */ - virtual const string& getCurrentMessage() = 0; + /** + * Generate an output record + */ + //virtual void emit(const string& key, const string& value) = 0; - /** - * @return The number of messages in the peer's received messages queue. - */ - virtual int getNumCurrentMessages() = 0; + /** + * Mark your task as having made progress without changing the status + * message. + */ + //virtual void progress() = 0; - /** - * Barrier Synchronization. - * - * Sends all the messages in the outgoing message queues to the corresponding - * remote peers. - */ - virtual void sync() = 0; + /** + * Set the status message and call progress. + */ + //virtual void setStatus(const string& status) = 0; - /** - * @return the count of current super-step - */ - virtual long getSuperstepCount() = 0; - - /** - * @return the name of this peer in the format "hostname:port". - */ - virtual const string& getPeerName() = 0; + /** + * Register a counter with the given group and name. + */ + //virtual Counter* getCounter(const string& group, const string& name) = 0; - /** - * @return the name of n-th peer from sorted array by name. - */ - virtual const string& getPeerName(int index) = 0; + /** + * Increment the value of the counter with the given amount. + */ + //virtual void incrementCounter(const Counter* counter, uint64_t amount) = 0; + virtual void incrementCounter(const string& group, const string& name, uint64_t amount) = 0; - /** - * @return the index of this peer from sorted array by name. - */ - virtual int getPeerIndex() = 0; + virtual ~TaskContext() {} + }; + + /********************************************/ + /************* DownwardProtocol *************/ + /********************************************/ + template + class DownwardProtocol { + public: + virtual void start(int protocol_version) = 0; + virtual void setBSPJob(vector values) = 0; + virtual void setInputTypes(string key_type, string value_type) = 0; - /** - * @return the names of all the peers executing tasks from the same job - * (including this peer). - */ - virtual vector getAllPeerNames() = 0; + virtual void runBsp(bool piped_input, bool piped_output) = 0; + virtual void runCleanup(bool piped_input, bool piped_output) = 0; + virtual void runSetup(bool piped_input, bool piped_output) = 0; - /** - * @return the number of peers - */ - virtual int getNumPeers() = 0; + virtual void runPartition(const K1& key, const V1& value, int32_t num_tasks) = 0; - /** - * Clears all queues entries. - */ - virtual void clear() = 0; + virtual void close() = 0; + virtual void abort() = 0; + virtual ~DownwardProtocol() {} + }; + + /********************************************/ + /************** UpwardProtocol **************/ + /********************************************/ + template + class UpwardProtocol { + public: + virtual void sendCommand(int32_t cmd) = 0; - /** - * Writes a key/value pair to the output collector - */ - virtual void write(const string& key, const string& value) = 0; + template + void sendCommand(int32_t cmd, T value) { + static_cast(this)->template sendCommand(cmd, value); + } + template + void sendCommand(int32_t cmd, const T values[], int size) { + static_cast(this)->template sendCommand(cmd, values, size); + } + + template + void sendCommand(int32_t cmd, T1 value1, T2 value2) { + static_cast(this)->template sendCommand(cmd, value1, value2); + } + + template + void sendCommand(int32_t cmd, T1 value, const T2 values[], int size) { + static_cast(this)->template sendCommand(cmd, value, values, size); + } + + //virtual void registerCounter(int id, const string& group, const string& name) = 0; + //virtual void incrementCounter(const TaskContext::Counter* counter, uint64_t amount) = 0; + virtual void incrementCounter(const string& group, const string& name, uint64_t amount) = 0; + virtual ~UpwardProtocol() {} + }; + + /* Forward definition of BinaryUpwardProtocol to pass to UpwardProtocol */ + class BinaryUpwardProtocol; + + /********************************************/ + /***************** Protocol *****************/ + /********************************************/ + template + class Protocol { + public: + + template + T getResult(int32_t expected_response_cmd) { + return static_cast(this)->template getResult(expected_response_cmd); + } + + template + vector getVectorResult(int32_t expected_response_cmd) { + return static_cast(this)->template getVectorResult(expected_response_cmd); + } + + template + KeyValuePair getKeyValueResult(int32_t expected_response_cmd) { + return static_cast(this)->template getKeyValueResult(expected_response_cmd); + } + + virtual void nextEvent() = 0; + virtual UpwardProtocol* getUplink() = 0; + virtual ~Protocol(){} + }; + + /********************************************/ + /*********** SequenceFileConnector **********/ + /********************************************/ /** - * Deserializes the next input key value into the given objects; + * SequenceFile Connector */ - virtual bool readNext(string& key, string& value) = 0; + /* Using Curiously recurring template pattern(CTRP) */ + /* because virtual template function not possible */ + template + class SequenceFileConnector { + public: + /** + * Open SequenceFile with option "r" or "w" + * key and value type of the values stored in the SequenceFile + * @return the corresponding fileID + */ + virtual int32_t sequenceFileOpen(const string& path, const string& option, const string& key_type, const string& value_type) = 0; /** + * Close SequenceFile + */ + virtual bool sequenceFileClose(int32_t file_id) = 0; + + /** + * Read next key/value pair from the SequenceFile with fileID + * Using Curiously recurring template pattern(CTRP) + */ + template + bool sequenceFileReadNext(int32_t file_id, K& key, V& value) { + return static_cast(this)->template sequenceFileReadNext(file_id, key, value); + } + + /** + * Append the next key/value pair to the SequenceFile with fileID + * Using Curiously recurring template pattern(CTRP) + */ + template + bool sequenceFileAppend(int32_t file_id, const K& key, const V& value) { + return static_cast(this)->template sequenceFileAppend(file_id, key, value); + } + }; + + /* Forward definition of BSPContextImpl to pass to SequenceFileConnector */ + template + class BSPContextImpl; + + + template + class BSPContext: public TaskContext, public SequenceFileConnector > { + public: + + /** + * Access the InputSplit of the mapper. + */ + //virtual const string& getInputSplit() = 0; + + /** + * Get the name of the key class of the input to this task. + */ + virtual string getInputKeyClass() = 0; + + /** + * Get the name of the value class of the input to this task. + */ + virtual string getInputValueClass() = 0; + + /** + * Send a data with a tag to another BSPSlave corresponding to hostname. + * Messages sent by this method are not guaranteed to be received in a sent + * order. + */ + virtual void sendMessage(const string& peer_name, const M& msg) = 0; + + /** + * @return A message from the peer's received messages queue (a FIFO). + */ + virtual M getCurrentMessage() = 0; + + /** + * @return The number of messages in the peer's received messages queue. + */ + virtual int getNumCurrentMessages() = 0; + + /** + * Barrier Synchronization. + * + * Sends all the messages in the outgoing message queues to the corresponding + * remote peers. + */ + virtual void sync() = 0; + + /** + * @return the count of current super-step + */ + virtual long getSuperstepCount() = 0; + + /** + * @return the name of this peer in the format "hostname:port". + */ + virtual string getPeerName() = 0; + + /** + * @return the name of n-th peer from sorted array by name. + */ + virtual string getPeerName(int index) = 0; + + /** + * @return the index of this peer from sorted array by name. + */ + virtual int getPeerIndex() = 0; + + /** + * @return the names of all the peers executing tasks from the same job + * (including this peer). + */ + virtual vector getAllPeerNames() = 0; + + /** + * @return the number of peers + */ + virtual int getNumPeers() = 0; + + /** + * Clears all queues entries. + */ + virtual void clear() = 0; + + /** + * Writes a key/value pair to the output collector + */ + virtual void write(const K2& key, const V2& value) = 0; + + /** + * Deserializes the next input key value into the given objects; + */ + virtual bool readNext(K1& key, V1& value) = 0; + + /** * Reads the next key value pair and returns it as a pair. It may reuse a * {@link KeyValuePair} instance to save garbage collection time. - * + * * @return null if there are no records left. * @throws IOException */ //public KeyValuePair readNext() throws IOException; + /** + * Closes the input and opens it right away, so that the file pointer is at + * the beginning again. + */ + virtual void reopenInput() = 0; + + }; + + class Closable { + public: + virtual void close() {} + virtual ~Closable() {} + }; + /** - * Closes the input and opens it right away, so that the file pointer is at - * the beginning again. + * The application's BSP class to do bsp. */ - virtual void reopenInput() = 0; + template + class BSP: public Closable { + public: + /** + * This method is called before the BSP method. It can be used for setup + * purposes. + */ + virtual void setup(BSPContext& context) = 0; -}; + /** + * This method is your computation method, the main work of your BSP should be + * done here. + */ + virtual void bsp(BSPContext& context) = 0; -class Closable { -public: - virtual void close() {} - virtual ~Closable() {} -}; - -/** - * The application's BSP class to do bsp. - */ -class BSP: public Closable { -public: + /** + * This method is called after the BSP method. It can be used for cleanup + * purposes. Cleanup is guranteed to be called after the BSP runs, even in + * case of exceptions. + */ + virtual void cleanup(BSPContext& context) = 0; + }; + /** - * This method is called before the BSP method. It can be used for setup - * purposes. + * User code to decide where each key should be sent. */ - virtual void setup(BSPContext& context) = 0; - + template + class Partitioner { + public: + + virtual int partition(const K1& key, const V1& value, int32_t num_tasks) = 0; + virtual ~Partitioner() {} + }; + /** - * This method is your computation method, the main work of your BSP should be - * done here. + * For applications that want to read the input directly for the map function + * they can define RecordReaders in C++. */ - virtual void bsp(BSPContext& context) = 0; + template + class RecordReader: public Closable { + public: + virtual bool next(K& key, V& value) = 0; + /** + * The progress of the record reader through the split as a value between + * 0.0 and 1.0. + */ + virtual float getProgress() = 0; + }; + /** - * This method is called after the BSP method. It can be used for cleanup - * purposes. Cleanup is guranteed to be called after the BSP runs, even in - * case of exceptions. + * An object to write key/value pairs as they are emited from the reduce. */ - virtual void cleanup(BSPContext& context) = 0; -}; - -/** - * User code to decide where each key should be sent. - */ -class Partitioner { -public: + template + class RecordWriter: public Closable { + public: + virtual void emit(const K& key, const V& value) = 0; + }; + + /** + * A factory to create the necessary application objects. + */ + template + class Factory { + public: + virtual BSP* createBSP(BSPContext& context) const = 0; - virtual int partition(const string& key,const string& value, int32_t numTasks) = 0; - virtual ~Partitioner() {} -}; + /** + * Create an application partitioner object. + * @return the new partitioner or NULL, if the default partitioner should be + * used. + */ + virtual Partitioner* createPartitioner(BSPContext& context) const { + return NULL; + } -/** - * For applications that want to read the input directly for the map function - * they can define RecordReaders in C++. - */ -class RecordReader: public Closable { -public: - virtual bool next(string& key, string& value) = 0; - + /** + * Create an application record reader. + * @return the new RecordReader or NULL, if the Java RecordReader should be + * used. + */ + virtual RecordReader* createRecordReader(BSPContext& context) const { + return NULL; + } + + /** + * Create an application record writer. + * @return the new RecordWriter or NULL, if the Java RecordWriter should be + * used. + */ + virtual RecordWriter* createRecordWriter(BSPContext& context) const { + return NULL; + } + + virtual ~Factory() {} + }; + /** - * The progress of the record reader through the split as a value between - * 0.0 and 1.0. + * Generic toString */ - virtual float getProgress() = 0; -}; - -/** - * An object to write key/value pairs as they are emited from the reduce. - */ -class RecordWriter: public Closable { -public: - virtual void emit(const string& key, - const string& value) = 0; -}; - -/** - * A factory to create the necessary application objects. - */ -class Factory { -public: - virtual BSP* createBSP(BSPContext& context) const = 0; - + template + string toString(const T& t) + { + std::ostringstream oss; + oss << t; + return oss.str(); + } /** - * Create an application partitioner object. - * @return the new partitioner or NULL, if the default partitioner should be - * used. + * Generic toString template specializations */ - virtual Partitioner* createPartitioner(BSPContext& context) const { - return NULL; + template <> string toString(const string& t) { + return t; } - + /** - * Create an application record reader. - * @return the new RecordReader or NULL, if the Java RecordReader should be - * used. + * Generic serialization */ - virtual RecordReader* createRecordReader(BSPContext& context) const { - return NULL; + template + void serialize(T t, OutStream& stream) { + serializeString(toString(t), stream); } - + /** - * Create an application record writer. - * @return the new RecordWriter or NULL, if the Java RecordWriter should be - * used. + * Generic serialization template specializations */ - virtual RecordWriter* createRecordWriter(BSPContext& context) const { - return NULL; + template <> void serialize(int32_t t, OutStream& stream) { + serializeInt(t, stream); + if (logging) { + fprintf(stderr,"HamaPipes::BinaryProtocol::serializeInt '%d'\n", t); + } } - - virtual ~Factory() {} -}; - -/** - * Run the assigned task in the framework. - * The user's main function should set the various functions using the - * set* functions above and then call this. - * @return true, if the task succeeded. - */ -bool runTask(const Factory& factory); - + template <> void serialize(int64_t t, OutStream& stream) { + serializeLong(t, stream); + if (logging) { + fprintf(stderr,"HamaPipes::BinaryProtocol::serializeLong '%ld'\n", (long)t); + } + } + template <> void serialize(float t, OutStream& stream) { + serializeFloat(t, stream); + if (logging) { + fprintf(stderr,"HamaPipes::BinaryProtocol::serializeFloat '%f'\n", t); + } + } + template <> void serialize(double t, OutStream& stream) { + serializeDouble(t, stream); + if (logging) { + fprintf(stderr,"HamaPipes::BinaryProtocol::serializeDouble '%f'\n", t); + } + } + template <> void serialize(string t, OutStream& stream) { + serializeString(t, stream); + if (logging) { + fprintf(stderr,"HamaPipes::BinaryProtocol::serializeString '%s'\n", t.c_str()); + } + } + + /** + * Generic deserialization + */ + template + T deserialize(InStream& stream) { + string str = "Not able to deserialize type: "; + throw Error(str.append(typeid(T).name())); + } + + /** + * Generic deserialization template specializations + */ + template <> int32_t deserialize(InStream& stream) { + int32_t result = deserializeInt(stream); + if (logging) { + fprintf(stderr,"HamaPipes::BinaryProtocol::deserializeInt result: '%d'\n", + result); + } + return result; + } + template <> int64_t deserialize(InStream& stream) { + int64_t result = deserializeLong(stream); + if (logging) { + fprintf(stderr,"HamaPipes::BinaryProtocol::deserializeLong result: '%ld'\n", + (long)result); + } + return result; + } + template <> float deserialize(InStream& stream) { + float result = deserializeFloat(stream); + if (logging) { + fprintf(stderr,"HamaPipes::BinaryProtocol::deserializeFloat result: '%f'\n", + result); + } + return result; + } + template <> double deserialize(InStream& stream) { + double result = deserializeDouble(stream); + if (logging) { + fprintf(stderr,"HamaPipes::BinaryProtocol::deserializeDouble result: '%f'\n", + result); + } + return result; + } + template <> string deserialize(InStream& stream) { + string result = deserializeString(stream); + + if (logging) { + if (result.empty()) { + fprintf(stderr,"HamaPipes::BinaryProtocol::deserializeString returns EMPTY string! Maybe wrong Serialization?\n"); + } else { + fprintf(stderr,"HamaPipes::BinaryProtocol::deserializeString result: '%s'\n", + ((result.length()<10)?result.c_str():result.substr(0,9).append("...").c_str())); + } + } + return result; + } + + /** + * Run the assigned task in the framework. + * The user's main function should set the various functions using the + * set* functions above and then call this. + * @return true, if the task succeeded. + */ + template + bool runTask(const Factory& factory); + + // Include implementation in header because of templates + #include "../../impl/Pipes.cc" } #endif Index: c++/src/main/native/pipes/impl/Pipes.cc =================================================================== --- c++/src/main/native/pipes/impl/Pipes.cc (revision 0) +++ c++/src/main/native/pipes/impl/Pipes.cc (revision 0) @@ -0,0 +1,1356 @@ +/** + * 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. + */ + +/********************************************/ +/*********** BinaryUpwardProtocol ***********/ +/********************************************/ +class BinaryUpwardProtocol: public UpwardProtocol { +private: + FileOutStream* out_stream_; +public: + BinaryUpwardProtocol(FILE* out_stream) { + out_stream_ = new FileOutStream(); + HADOOP_ASSERT(out_stream_->open(out_stream), "problem opening stream"); + } + + /* local function */ + void sendCommand(int32_t cmd, bool flush) { + serializeInt(cmd, *out_stream_); + if (flush) { + out_stream_->flush(); + } + if(logging) { + fprintf(stderr,"HamaPipes::BinaryUpwardProtocol sent CMD: %s\n", + messageTypeNames[cmd]); + } + } + + template + void sendCommand(int32_t cmd, T value) { + sendCommand(cmd, false); + // Write out generic value + serialize(value, *out_stream_); + out_stream_->flush(); + if(logging) { + fprintf(stderr,"HamaPipes::BinaryUpwardProtocol sent CMD: %s with Value: '%s'\n", + messageTypeNames[cmd], toString(value).c_str()); + } + } + + template + void sendCommand(int32_t cmd, const T values[], int size) { + sendCommand(cmd, false); + // Write out generic values + for (int i=0; i(values[i], *out_stream_); + if(logging) { + fprintf(stderr,"HamaPipes::BinaryUpwardProtocol sent CMD: %s with Param%d: '%s'\n", + messageTypeNames[cmd], i+1, toString(values[i]).c_str()); + } + } + out_stream_->flush(); + } + + template + void sendCommand(int32_t cmd, T1 value1, T2 value2) { + sendCommand(cmd, false); + // Write out generic value1 + serialize(value1, *out_stream_); + if(logging) { + fprintf(stderr,"HamaPipes::BinaryUpwardProtocol sent CMD: %s with Param1: '%s'\n", + messageTypeNames[cmd], toString(value1).c_str()); + } + // Write out generic value2 + serialize(value2, *out_stream_); + if(logging) { + fprintf(stderr,"HamaPipes::BinaryUpwardProtocol sent CMD: %s with Param2: '%s'\n", + messageTypeNames[cmd], toString(value2).c_str()); + } + out_stream_->flush(); + } + + template + void sendCommand(int32_t cmd, T1 value, const T2 values[], int size) { + sendCommand(cmd, false); + // Write out generic value + serialize(value, *out_stream_); + if(logging) { + fprintf(stderr,"HamaPipes::BinaryUpwardProtocol sent CMD: %s with Param%d: '%s'\n", + messageTypeNames[cmd], 0, toString(value).c_str()); + } + // Write out generic values + for (int i=0; i(values[i], *out_stream_); + if(logging) { + fprintf(stderr,"HamaPipes::BinaryUpwardProtocol sent CMD: %s with Param%d: '%s'\n", + messageTypeNames[cmd], i+1, toString(value).c_str()); + } + } + out_stream_->flush(); + } + + virtual void sendCommand(int32_t cmd) { + sendCommand(cmd, true); + } + + /* + virtual void registerCounter(int id, const string& group, + const string& name) { + serializeInt(REGISTER_COUNTER, *stream); + serializeInt(id, *stream); + serializeString(group, *stream); + serializeString(name, *stream); + } + + virtual void incrementCounter(const TaskContext::Counter* counter, + uint64_t amount) { + serializeInt(INCREMENT_COUNTER, *stream); + serializeInt(counter->getId(), *stream); + serializeLong(amount, *stream); + } + */ + + virtual void incrementCounter(const string& group, const string& name, uint64_t amount) { + serializeInt(INCREMENT_COUNTER, *out_stream_); + serializeString(group, *out_stream_); + serializeString(name, *out_stream_); + serializeLong(amount, *out_stream_); + out_stream_->flush(); + if(logging) { + fprintf(stderr,"HamaPipes::BinaryUpwardProtocol sent incrementCounter\n"); + } + } + + ~BinaryUpwardProtocol() { + delete out_stream_; + } +}; + +/********************************************/ +/************** BinaryProtocol **************/ +/********************************************/ +template +class BinaryProtocol: public Protocol< BinaryProtocol > { +private: + FileInStream* in_stream_; + DownwardProtocol* handler_; + BinaryUpwardProtocol* uplink_; + +public: + BinaryProtocol(FILE* in_stream, DownwardProtocol* handler, FILE* uplink) { + in_stream_ = new FileInStream(); + in_stream_->open(in_stream); + uplink_ = new BinaryUpwardProtocol(uplink); + handler_ = handler; + } + + UpwardProtocol* getUplink() { + return uplink_; + } + + /** + * Wait for next event, but don't expect a response for + * a previously sent command + */ + void nextEvent() { + // read command + int32_t cmd; + cmd = deserializeInt(*in_stream_); + + switch (cmd) { + + case START_MESSAGE: { + int32_t protocol_version; + protocol_version = deserialize(*in_stream_); + if(logging) { + fprintf(stderr,"HamaPipes::BinaryProtocol::nextEvent - got START_MESSAGE protocol_version: %d\n", + protocol_version); + } + handler_->start(protocol_version); + break; + } + // setup BSP Job Configuration + case SET_BSPJOB_CONF: { + int32_t entries; + entries = deserialize(*in_stream_); + if(logging) { + fprintf(stderr,"HamaPipes::BinaryProtocol::nextEvent - got SET_BSPJOB_CONF entries: %d\n", + entries); + } + vector properties(entries*2); + for(int i=0; i < entries*2; ++i) { + string item; + item = deserialize(*in_stream_); + properties.push_back(item); + if(logging) { + fprintf(stderr,"HamaPipes::BinaryProtocol::nextEvent - got SET_BSPJOB_CONF add NewEntry: %s\n", + item.c_str()); + } + } + if(logging) { + fprintf(stderr,"HamaPipes::BinaryProtocol::nextEvent - got all Configuration %d entries!\n", + entries); + } + handler_->setBSPJob(properties); + break; + } + case SET_INPUT_TYPES: { + string key_type; + string value_type; + key_type = deserialize(*in_stream_); + value_type = deserialize(*in_stream_); + if(logging) { + fprintf(stderr,"HamaPipes::BinaryProtocol::nextEvent - got SET_INPUT_TYPES keyType: %s valueType: %s\n", + key_type.c_str(), value_type.c_str()); + } + handler_->setInputTypes(key_type, value_type); + break; + } + case RUN_SETUP: { + if(logging) { + fprintf(stderr,"HamaPipes::BinaryProtocol::nextEvent - got RUN_SETUP\n"); + } + int32_t piped_input; + int32_t piped_output; + piped_input = deserialize(*in_stream_); + piped_output = deserialize(*in_stream_); + handler_->runSetup(piped_input, piped_output); + break; + } + case RUN_BSP: { + if(logging) { + fprintf(stderr,"HamaPipes::BinaryProtocol::nextEvent - got RUN_BSP\n"); + } + int32_t piped_input; + int32_t piped_output; + piped_input = deserialize(*in_stream_); + piped_output = deserialize(*in_stream_); + handler_->runBsp(piped_input, piped_output); + break; + } + case RUN_CLEANUP: { + if(logging) { + fprintf(stderr,"HamaPipes::BinaryProtocol::nextEvent - got RUN_CLEANUP\n"); + } + int32_t piped_input; + int32_t piped_output; + piped_input = deserialize(*in_stream_); + piped_output = deserialize(*in_stream_); + handler_->runCleanup(piped_input, piped_output); + break; + } + case PARTITION_REQUEST: { + if(logging) { + fprintf(stderr,"HamaPipes::BinaryProtocol::nextEvent - got PARTITION_REQUEST\n"); + } + + K1 partion_key; + V1 partion_value; + int32_t num_tasks; + + partion_key = deserialize(*in_stream_); + partion_value = deserialize(*in_stream_); + num_tasks = deserialize(*in_stream_); + + handler_->runPartition(partion_key, partion_value, num_tasks); + break; + } + case CLOSE: { + if(logging) { + fprintf(stderr,"HamaPipes::BinaryProtocol::nextEvent - got CLOSE\n"); + } + handler_->close(); + break; + } + case ABORT: { + if(logging) { + fprintf(stderr,"HamaPipes::BinaryProtocol::nextEvent - got ABORT\n"); + } + handler_->abort(); + break; + } + default: { + HADOOP_ASSERT(false, "Unknown binary command " + toString(cmd)); + fprintf(stderr,"HamaPipes::BinaryProtocol::nextEvent - Unknown binary command: %d\n", + cmd); + } + } + } + + /** + * Wait for next event, which should be a response for + * a previously sent command (expected_response_cmd) + * and return the generic result + */ + template + T getResult(int32_t expected_response_cmd) { + + T result = T(); + + // read response command + int32_t cmd; + cmd = deserializeInt(*in_stream_); + + // check if response is expected + if (expected_response_cmd == cmd) { + + switch (cmd) { + + case GET_MSG_COUNT: { + T msg_count; + msg_count = deserialize(*in_stream_); + if(logging) { + fprintf(stderr,"HamaPipes::BinaryProtocol::nextEvent - got GET_MSG_COUNT msg_count: '%s'\n", + toString(msg_count).c_str()); + } + return msg_count; + } + case GET_MSG: { + T msg; + msg = deserialize(*in_stream_); + if(logging) { + fprintf(stderr,"HamaPipes::BinaryProtocol::nextEvent - got GET_MSG msg: '%s'\n", + toString(msg).c_str()); + } + return msg; + } + case GET_PEERNAME: { + T peername; + peername = deserialize(*in_stream_); + if(logging) { + fprintf(stderr,"HamaPipes::BinaryProtocol::nextEvent - got GET_PEERNAME peername: %s\n", + toString(peername).c_str()); + } + return peername; + } + case GET_PEER_INDEX: { + T peer_index = deserialize(*in_stream_); + if(logging) { + fprintf(stderr,"HamaPipes::BinaryProtocol::nextEvent - got GET_PEER_INDEX peer_index: '%s'\n", + toString(peer_index).c_str()); + } + return peer_index; + } + case GET_PEER_COUNT: { + T peer_count = deserialize(*in_stream_); + if(logging) { + fprintf(stderr,"HamaPipes::BinaryProtocol::nextEvent - got GET_PEER_COUNT peer_count: '%s'\n", + toString(peer_count).c_str()); + } + return peer_count; + } + case GET_SUPERSTEP_COUNT: { + T superstep_count = deserialize(*in_stream_); + if(logging) { + fprintf(stderr,"HamaPipes::BinaryProtocol::nextEvent - got GET_SUPERSTEP_COUNT superstep_count: '%s'\n", + toString(superstep_count).c_str()); + } + return superstep_count; + } + + case SEQFILE_OPEN: { + T file_id = deserialize(*in_stream_); + if(logging) { + fprintf(stderr,"HamaPipes::BinaryProtocol::nextEvent - got SEQFILE_OPEN file_id: '%s'\n", + toString(file_id).c_str()); + } + return file_id; + } + case SEQFILE_APPEND: { + result = deserialize(*in_stream_); + if(logging) { + fprintf(stderr,"HamaPipes::BinaryProtocol::nextEvent - got SEQFILE_APPEND result: '%s'\n", + toString(result).c_str()); + } + return result; + } + case SEQFILE_CLOSE: { + result = deserialize(*in_stream_); + if(logging) { + fprintf(stderr,"HamaPipes::BinaryProtocol::nextEvent - got SEQFILE_CLOSE result: '%s'\n", + toString(result).c_str()); + } + return result; + } + } + // Not expected response + } else { + + /* + case CLOSE: { + if(logging)fprintf(stderr,"HamaPipes::BinaryProtocol::nextEvent - got CLOSE\n"); + handler_->close(); + break; + } + case ABORT: { + if(logging)fprintf(stderr,"HamaPipes::BinaryProtocol::nextEvent - got ABORT\n"); + handler_->abort(); + break; + } + */ + HADOOP_ASSERT(false, "Unknown binary command " + toString(cmd)); + fprintf(stderr,"HamaPipes::BinaryProtocol::nextEvent(%d) - Unknown binary command: %d\n", + expected_response_cmd, cmd); + } + return result; + } + + /** + * Wait for next event, which should be a response for + * a previously sent command (expected_response_cmd) + * and return the generic vector result list + */ + template + vector getVectorResult(int32_t expected_response_cmd) { + + vector results; + + // read response command + int32_t cmd; + cmd = deserializeInt(*in_stream_); + + // check if response is expected + if (expected_response_cmd == cmd) { + + switch (cmd) { + case GET_ALL_PEERNAME: { + vector peernames; + T peername; + int32_t peername_count = deserialize(*in_stream_); + if(logging) { + fprintf(stderr,"HamaPipes::BinaryProtocol::nextEvent - got GET_ALL_PEERNAME peername_count: %d\n", + peername_count); + } + for (int i=0; i(*in_stream_); + peernames.push_back(peername); + if(logging) { + fprintf(stderr,"HamaPipes::BinaryProtocol::nextEvent - got GET_ALL_PEERNAME peername: '%s'\n", + toString(peername).c_str()); + } + } + return peernames; + } + } + } else { + HADOOP_ASSERT(false, "Unknown binary command " + toString(cmd)); + fprintf(stderr,"HamaPipes::BinaryProtocol::nextEvent(%d) - Unknown binary command: %d\n", + expected_response_cmd, cmd); + } + return results; + } + + /** + * Wait for next event, which should be a response for + * a previously sent command (expected_response_cmd) + * and return the generic KeyValuePair or an empty one + * if no data is available + */ + template + KeyValuePair getKeyValueResult(int32_t expected_response_cmd) { + + KeyValuePair key_value_pair; + + // read response command + int32_t cmd; + cmd = deserializeInt(*in_stream_); + + // check if response is expected or END_OF_DATA + if ((expected_response_cmd == cmd) || (cmd == END_OF_DATA) ) { + + switch (cmd) { + + case READ_KEYVALUE: { + K key = deserialize(*in_stream_); + V value = deserialize(*in_stream_); + + if(logging) { + string k = toString(key); + string v = toString(value); + fprintf(stderr,"HamaPipes::BinaryProtocol::nextEvent - got READ_KEYVALUE key: '%s' value: '%s'\n", + ((k.length()<10)?k.c_str():k.substr(0,9).append("...").c_str()), + ((v.length()<10)?v.c_str():v.substr(0,9).append("...").c_str()) ); + } + + key_value_pair = pair(key, value); + return key_value_pair; + } + case SEQFILE_READNEXT: { + K key = deserialize(*in_stream_); + V value = deserialize(*in_stream_); + + if(logging) { + string k = toString(key); + string v = toString(value); + fprintf(stderr,"HamaPipes::BinaryProtocol::nextEvent - got SEQFILE_READNEXT key: '%s' value: '%s'\n", + ((k.length()<10)?k.c_str():k.substr(0,9).append("...").c_str()), + ((v.length()<10)?v.c_str():v.substr(0,9).append("...").c_str()) ); + } + + key_value_pair = pair(key, value); + return key_value_pair; + } + case END_OF_DATA: { + key_value_pair = KeyValuePair(true); + if(logging) { + fprintf(stderr,"HamaPipes::BinaryProtocol::nextEvent - got END_OF_DATA\n"); + } + } + } + } else { + key_value_pair = KeyValuePair(true); + fprintf(stderr,"HamaPipes::BinaryProtocol::nextEvent(expected_cmd = %d) - Unknown binary command: %d\n", + expected_response_cmd, cmd); + fprintf(stderr,"ERORR: Please verfiy serialization! The key or value type could possibly not be deserialized!\n"); + HADOOP_ASSERT(false, "Unknown binary command " + toString(cmd)); + } + return key_value_pair; + } + + virtual ~BinaryProtocol() { + delete in_stream_; + delete uplink_; + delete handler_; + } +}; + +/********************************************/ +/************** BSPContextImpl **************/ +/********************************************/ +template +class BSPContextImpl: public BSPContext, public DownwardProtocol { +private: + const Factory* factory_; + BSPJob* job_; + BSP* bsp_; + Partitioner* partitioner_; + RecordReader* reader_; + RecordWriter* writer_; + Protocol< BinaryProtocol >* protocol_; + UpwardProtocol* uplink_; + + bool done_; + bool has_task_; + pthread_mutex_t mutex_done_; + std::vector registered_counter_ids_; + + pair inputClass_; + //string* inputSplit_; + +public: + + BSPContextImpl(const Factory& factory) { + + factory_ = &factory; + job_ = NULL; + bsp_ = NULL; + partitioner_ = NULL; + reader_ = NULL; + writer_ = NULL; + protocol_ = NULL; + + done_ = false; + has_task_ = false; + pthread_mutex_init(&mutex_done_, NULL); + + //inputSplit_ = NULL; + } + + + /********************************************/ + /*********** DownwardProtocol IMPL **********/ + /********************************************/ + virtual void start(int protocol_version) { + if (protocol_version != 0) { + throw Error("Protocol version " + toString(protocol_version) + + " not supported"); + } + partitioner_ = factory_->createPartitioner(*this); + } + + virtual void setBSPJob(vector values) { + int len = values.size(); + BSPJobImpl* result = new BSPJobImpl(); + HADOOP_ASSERT(len % 2 == 0, "Odd length of job conf values"); + for(int i=0; i < len; i += 2) { + result->set(values[i], values[i+1]); + } + job_ = result; + } + + virtual void setInputTypes(string key_type, string value_type) { + inputClass_ = pair(key_type, value_type); + } + + /* local method */ + void setupReaderWriter(bool piped_input, bool piped_output) { + + if(logging) { + fprintf(stderr,"HamaPipes::BSPContextImpl::setupReaderWriter - pipedInput: %s pipedOutput: %s\n", + (piped_input)?"true":"false", (piped_output)?"true":"false"); + } + + if (piped_input && reader_==NULL) { + reader_ = factory_->createRecordReader(*this); + HADOOP_ASSERT((reader_ == NULL) == piped_input, + piped_input ? "RecordReader defined when not needed.": + "RecordReader not defined"); + + //if (reader != NULL) { + // value = new string(); + //} + } + + if (piped_output && writer_==NULL) { + writer_ = factory_->createRecordWriter(*this); + HADOOP_ASSERT((writer_ == NULL) == piped_output, + piped_output ? "RecordWriter defined when not needed.": + "RecordWriter not defined"); + } + } + + virtual void runSetup(bool piped_input, bool piped_output) { + setupReaderWriter(piped_input, piped_output); + + if (bsp_ == NULL) { + bsp_ = factory_->createBSP(*this); + } + + if (bsp_ != NULL) { + has_task_ = true; + bsp_->setup(*this); + has_task_ = false; + uplink_->sendCommand(TASK_DONE); + } + } + + virtual void runBsp(bool piped_input, bool piped_output) { + setupReaderWriter(piped_input, piped_output); + + if (bsp_ == NULL) { + bsp_ = factory_->createBSP(*this); + } + + if (bsp_ != NULL) { + has_task_ = true; + bsp_->bsp(*this); + has_task_ = false; + uplink_->sendCommand(TASK_DONE); + } + } + + virtual void runCleanup(bool piped_input, bool piped_output) { + setupReaderWriter(piped_input, piped_output); + + if (bsp_ != NULL) { + has_task_ = true; + bsp_->cleanup(*this); + has_task_ = false; + uplink_->sendCommand(TASK_DONE); + } + } + + /********************************************/ + /******* Partitioner *******/ + /********************************************/ + virtual void runPartition(const K1& key, const V1& value, int32_t num_tasks) { + if (partitioner_ != NULL) { + int part = partitioner_->partition(key, value, num_tasks); + uplink_->sendCommand(PARTITION_RESPONSE, part); + } else { + if(logging) { + fprintf(stderr,"HamaPipes::BSPContextImpl::runPartition Partitioner is NULL!\n"); + } + } + } + + virtual void close() { + pthread_mutex_lock(&mutex_done_); + done_ = true; + has_task_ = false; + pthread_mutex_unlock(&mutex_done_); + + if(logging) { + fprintf(stderr,"HamaPipes::BSPContextImpl::close - done: %s hasTask: %s\n", + (done_)?"true":"false",(has_task_)?"true":"false"); + } + } + + virtual void abort() { + throw Error("Aborted by driver"); + } + + /********************************************/ + /************** TaskContext IMPL ************/ + /********************************************/ + + /** + * Get the BSPJob for the current task. + */ + virtual const BSPJob* getBSPJob() { + return job_; + } + + /** + * Get the current key. + * @return the current key or NULL if called before the first map or reduce + */ + //virtual const string& getInputKey() { + // return key; + //} + + /** + * Get the current value. + * @return the current value or NULL if called before the first map or + * reduce + */ + //virtual const string& getInputValue() { + // return *value; + //} + + /** + * Register a counter with the given group and name. + */ + /* + virtual Counter* getCounter(const std::string& group, + const std::string& name) { + int id = registeredCounterIds.size(); + registeredCounterIds.push_back(id); + uplink->registerCounter(id, group, name); + return new Counter(id); + }*/ + + /** + * Increment the value of the counter with the given amount. + */ + virtual void incrementCounter(const string& group, const string& name, uint64_t amount) { + uplink_->incrementCounter(group, name, amount); + } + + /********************************************/ + /************** BSPContext IMPL *************/ + /********************************************/ + + /** + * Access the InputSplit of the bsp. + */ + //virtual const string& getInputSplit() { + // return *inputSplit_; + //} + + /** + * Get the name of the key class of the input to this task. + */ + virtual string getInputKeyClass() { + return inputClass_.first; + } + + /** + * Get the name of the value class of the input to this task. + */ + virtual string getInputValueClass() { + return inputClass_.second; + } + + /** + * Send a data with a tag to another BSPSlave corresponding to hostname. + * Messages sent by this method are not guaranteed to be received in a sent + * order. + */ + virtual void sendMessage(const string& peer_name, const M& msg) { + uplink_->sendCommand(SEND_MSG, peer_name, msg); + } + + /** + * @return A message from the peer's received messages queue (a FIFO). + */ + virtual M getCurrentMessage() { + uplink_->sendCommand(GET_MSG); + + M message = protocol_->template getResult(GET_MSG); + + if(logging) { + fprintf(stderr,"HamaPipes::BSPContextImpl::getMessage - result: %s\n", + toString(message).c_str()); + } + return message; + } + + /** + * @return The number of messages in the peer's received messages queue. + */ + virtual int getNumCurrentMessages() { + uplink_->sendCommand(GET_MSG_COUNT); + + int result = protocol_->template getResult(GET_MSG_COUNT); + + if(logging) { + fprintf(stderr,"HamaPipes::BSPContextImpl::getNumCurrentMessages - result: %d\n", + result); + } + return result; + } + + /** + * Barrier Synchronization. + * + * Sends all the messages in the outgoing message queues to the corresponding + * remote peers. + */ + virtual void sync() { + uplink_->sendCommand(SYNC); + } + + /** + * @return the name of this peer in the format "hostname:port". + */ + virtual string getPeerName() { + // submit id=-1 to receive own peername + uplink_->sendCommand(GET_PEERNAME, -1); + + string result = protocol_->template getResult(GET_PEERNAME); + + if(logging) { + fprintf(stderr,"HamaPipes::BSPContextImpl::getPeerName - result: %s\n", + result.c_str()); + } + return result; + } + + /** + * @return the name of n-th peer from sorted array by name. + */ + virtual string getPeerName(int index) { + uplink_->sendCommand(GET_PEERNAME, index); + + string result = protocol_->template getResult(GET_PEERNAME); + + if(logging) { + fprintf(stderr,"HamaPipes::BSPContextImpl::getPeerName - result: %s\n", + result.c_str()); + } + + return result; + } + + /** + * @return the names of all the peers executing tasks from the same job + * (including this peer). + */ + virtual vector getAllPeerNames() { + uplink_->sendCommand(GET_ALL_PEERNAME); + + vector results = protocol_->template getVectorResult(GET_ALL_PEERNAME); + + return results; + } + + /** + * @return the index of this peer from sorted array by name. + */ + virtual int getPeerIndex() { + uplink_->sendCommand(GET_PEER_INDEX); + + int result = protocol_->template getResult(GET_PEER_INDEX); + + if(logging) { + fprintf(stderr,"HamaPipes::BSPContextImpl::getPeerIndex - result: %d\n", + result); + } + return result; + } + + /** + * @return the number of peers + */ + virtual int getNumPeers() { + uplink_->sendCommand(GET_PEER_COUNT); + + int result = protocol_->template getResult(GET_PEER_COUNT); + + if(logging) { + fprintf(stderr,"HamaPipes::BSPContextImpl::getNumPeers - result: %d\n", + result); + } + return result; + } + + /** + * @return the count of current super-step + */ + virtual long getSuperstepCount() { + uplink_->sendCommand(GET_SUPERSTEP_COUNT); + + long result = protocol_->template getResult(GET_SUPERSTEP_COUNT); + + if(logging) { + fprintf(stderr,"HamaPipes::BSPContextImpl::getSuperstepCount - result: %ld\n", + result); + } + return result; + } + + /** + * Clears all queues entries. + */ + virtual void clear() { + uplink_->sendCommand(CLEAR); + } + + /** + * Writes a key/value pair to the output collector + */ + virtual void write(const K2& key, const V2& value) { + if (writer_ != NULL) { + writer_->emit(key, value); + } else { + uplink_->sendCommand(WRITE_KEYVALUE, key, value); + } + } + + /** + * Deserializes the next input key value into the given objects; + */ + virtual bool readNext(K1& key, V1& value) { + + uplink_->sendCommand(READ_KEYVALUE); + + // TODO + // check if value is array [0, 1, 2, ...], and remove brackets + /* + int len = current_value_.length(); + if ( (current_value_[0]=='[') && + (current_value_[len-1]==']') ) { + value = current_value_.substr(1,len-2); + } else { + value = current_value_; + } + + if (logging && key.empty() && value.empty()) { + fprintf(stderr,"HamaPipes::BSPContextImpl::readNext - Empty KeyValuePair\n"); + } + + return (!key.empty() && !value.empty()); + */ + + KeyValuePair key_value_pair; + key_value_pair = protocol_->template getKeyValueResult(READ_KEYVALUE); + + if (!key_value_pair.is_empty) { + key = key_value_pair.first; + value = key_value_pair.second; + } + + if (logging && key_value_pair.is_empty) { + fprintf(stderr,"HamaPipes::BSPContextImpl::readNext - END_OF_DATA\n"); + } + + return (!key_value_pair.is_empty); + } + + /** + * Closes the input and opens it right away, so that the file pointer is at + * the beginning again. + */ + virtual void reopenInput() { + uplink_->sendCommand(REOPEN_INPUT); + } + + + /********************************************/ + /******* SequenceFileConnector IMPL *******/ + /********************************************/ + + /** + * Open SequenceFile with opion "r" or "w" + * @return the corresponding fileID + */ + virtual int32_t sequenceFileOpen(const string& path, const string& option, + const string& key_type, const string& value_type) { + if (logging) { + fprintf(stderr,"HamaPipes::BSPContextImpl::sequenceFileOpen - Path: %s\n", + path.c_str()); + } + + if ( (option.compare("r")==0) || (option.compare("w")==0)) { + + string values[] = {path, option, key_type, value_type}; + uplink_->sendCommand(SEQFILE_OPEN, values, 4); + + int result = protocol_->template getResult(SEQFILE_OPEN); + + if(logging) { + fprintf(stderr,"HamaPipes::BSPContextImpl::sequenceFileOpen - result: %d\n", + result); + } + return result; + } else { + //Error wrong option + fprintf(stderr,"HamaPipes::BSPContextImpl::sequenceFileOpen wrong option: %s!\n", + option.c_str()); + return -1; + } + } + + /** + * Close SequenceFile + */ + virtual bool sequenceFileClose(int32_t file_id) { + uplink_->sendCommand(SEQFILE_CLOSE, file_id); + + int result = protocol_->template getResult(SEQFILE_CLOSE); + + if (logging && result==0) { + fprintf(stderr,"HamaPipes::BSPContextImpl::sequenceFileClose - Nothing was closed!\n"); + } else if (logging) { + fprintf(stderr,"HamaPipes::BSPContextImpl::sequenceFileClose - File was successfully closed!\n"); + } + + return (result==1); + } + + /** + * Read next key/value pair from the SequenceFile with fileID + * Using Curiously recurring template pattern(CTRP) + */ + template + bool sequenceFileReadNext(int32_t file_id, K& key, V& value) { + + // send request + uplink_->sendCommand(SEQFILE_READNEXT, file_id); + + // TODO + /* + // check if value is array [0, 1, 2, ...], and remove brackets + int len = current_value_.length(); + if ( (current_value_[0]=='[') && + (current_value_[len-1]==']') ) { + value = current_value_(1,len-2); + } else { + value = current_value_; + } + */ + + // get response + KeyValuePair key_value_pair; + key_value_pair = protocol_->template getKeyValueResult(SEQFILE_READNEXT); + + if (!key_value_pair.is_empty) { + key = key_value_pair.first; + value = key_value_pair.second; + } + + if (logging && key_value_pair.is_empty) { + fprintf(stderr,"HamaPipes::BSPContextImpl::readNext - END_OF_DATA\n"); + } + + return (!key_value_pair.is_empty); + } + + /** + * Append the next key/value pair to the SequenceFile with fileID + * Using Curiously recurring template pattern(CTRP) + */ + template + bool sequenceFileAppend(int32_t file_id, const K& key, const V& value) { + string values[] = {key, value}; + uplink_->sendCommand(SEQFILE_APPEND, file_id, values, 2); + + int result = protocol_->template getResult(SEQFILE_APPEND); + + if (logging && result==0) { + fprintf(stderr,"HamaPipes::BSPContextImpl::sequenceFileAppend - Nothing appended!\n"); + } else if (logging) { + fprintf(stderr,"HamaPipes::BSPContextImpl::sequenceFileAppend - Successfully appended!\n"); + } + + return (result==1); + } + + /********************************************/ + /*************** Other STUFF ***************/ + /********************************************/ + + void setProtocol(Protocol< BinaryProtocol >* protocol, UpwardProtocol* uplink) { + protocol_ = protocol; + uplink_ = uplink; + } + + bool isDone() { + pthread_mutex_lock(&mutex_done_); + bool done_copy = done_; + pthread_mutex_unlock(&mutex_done_); + return done_copy; + } + + /** + * Advance to the next value. + */ + /* + bool nextValue() { + if (isNewKey || done) { + return false; + } + isNewValue = false; + //progress(); + protocol->nextEvent(); + return isNewValue; + } + */ + void waitForTask() { + while (!done_ && !has_task_) { + if(logging) { + fprintf(stderr,"HamaPipes::BSPContextImpl::waitForTask - done: %s hasTask: %s\n", + (done_)?"true":"false", (has_task_)?"true":"false"); + } + protocol_->nextEvent(); + } + } + /* + bool nextKey() { + if (reader == NULL) { + while (!isNewKey) { + nextValue(); + if (done) { + return false; + } + } + key = *newKey; + } else { + if (!reader->next(key, const_cast(*value))) { + pthread_mutex_lock(&mutexDone); + done = true; + pthread_mutex_unlock(&mutexDone); + return false; + } + //progressFloat = reader->getProgress(); + } + isNewKey = false; + + if (bsp != NULL) { + bsp->bsp(*this); + } + return true; + } + */ + void closeAll() { + if (reader_) { + reader_->close(); + } + + if (bsp_) { + bsp_->close(); + } + + if (writer_) { + writer_->close(); + } + } + + virtual ~BSPContextImpl() { + delete job_; + //delete inputSplit_; + //if (reader) { + // delete value; + //} + delete reader_; + delete bsp_; + delete writer_; + pthread_mutex_destroy(&mutex_done_); + } +}; + +/** + * Ping the parent every 5 seconds to know if it is alive + */ +template +void* ping(void* ptr) { + BSPContextImpl* context = (BSPContextImpl*) ptr; + char* portStr = getenv("hama.pipes.command.port"); + int MAX_RETRIES = 3; + int remaining_retries = MAX_RETRIES; + while (!context->isDone()) { + try{ + sleep(5); + int sock = -1; + if (portStr) { + sock = socket(PF_INET, SOCK_STREAM, 0); + HADOOP_ASSERT(sock != - 1, + string("problem creating socket: ") + strerror(errno)); + sockaddr_in addr; + addr.sin_family = AF_INET; + addr.sin_port = htons(toInt(portStr)); + addr.sin_addr.s_addr = htonl(INADDR_LOOPBACK); + if(logging) { + fprintf(stderr,"HamaPipes::ping - connected to GroomServer Port: %s\n", + portStr); + } + HADOOP_ASSERT(connect(sock, (sockaddr*) &addr, sizeof(addr)) == 0, + string("problem connecting command socket: ") + + strerror(errno)); + + } + if (sock != -1) { + int result = shutdown(sock, SHUT_RDWR); + HADOOP_ASSERT(result == 0, "problem shutting socket"); + result = close(sock); + HADOOP_ASSERT(result == 0, "problem closing socket"); + } + remaining_retries = MAX_RETRIES; + } catch (Error& err) { + if (!context->isDone()) { + fprintf(stderr, "Hama Pipes Exception: in ping %s\n", + err.getMessage().c_str()); + remaining_retries -= 1; + if (remaining_retries == 0) { + exit(1); + } + } else { + return NULL; + } + } + } + return NULL; +} + +/** + * Run the assigned task in the framework. + * The user's main function should set the various functions using the + * set* functions above and then call this. + * @return true, if the task succeeded. + */ +template +bool runTask(const Factory& factory) { + try { + HADOOP_ASSERT(getenv("hama.pipes.logging")!=NULL, "No environment found!"); + + logging = (toInt(getenv("hama.pipes.logging"))==0)?false:true; + if (logging) { + fprintf(stderr,"HamaPipes::runTask - logging is: %s\n", + ((logging)?"true":"false")); + } + + BSPContextImpl* context = new BSPContextImpl(factory); + Protocol< BinaryProtocol >* protocol; + + char* port_str = getenv("hama.pipes.command.port"); + int sock = -1; + FILE* in_stream = NULL; + FILE* out_stream = NULL; + char *bufin = NULL; + char *bufout = NULL; + if (port_str) { + sock = socket(PF_INET, SOCK_STREAM, 0); + HADOOP_ASSERT(sock != - 1, + string("problem creating socket: ") + strerror(errno)); + sockaddr_in addr; + addr.sin_family = AF_INET; + addr.sin_port = htons(toInt(port_str)); + addr.sin_addr.s_addr = htonl(INADDR_LOOPBACK); + HADOOP_ASSERT(connect(sock, (sockaddr*) &addr, sizeof(addr)) == 0, + string("problem connecting command socket: ") + + strerror(errno)); + + in_stream = fdopen(sock, "r"); + out_stream = fdopen(sock, "w"); + + // increase buffer size + int bufsize = 128*1024; + int setbuf; + bufin = new char[bufsize]; + bufout = new char[bufsize]; + setbuf = setvbuf(in_stream, bufin, _IOFBF, bufsize); + HADOOP_ASSERT(setbuf == 0, string("problem with setvbuf for in_stream: ") + + strerror(errno)); + setbuf = setvbuf(out_stream, bufout, _IOFBF, bufsize); + HADOOP_ASSERT(setbuf == 0, string("problem with setvbuf for out_stream: ") + + strerror(errno)); + + protocol = new BinaryProtocol(in_stream, context, out_stream); + if(logging) { + fprintf(stderr,"HamaPipes::runTask - connected to GroomServer Port: %s\n", + port_str); + } + + } else if (getenv("hama.pipes.command.file")) { + char* filename = getenv("hama.pipes.command.file"); + string out_filename = filename; + out_filename += ".out"; + in_stream = fopen(filename, "r"); + out_stream = fopen(out_filename.c_str(), "w"); + protocol = new BinaryProtocol(in_stream, context, out_stream); + } else { + //protocol = new TextProtocol(stdin, context, stdout); + fprintf(stderr,"HamaPipes::runTask - Protocol couldn't be initialized!\n"); + return -1; + } + + context->setProtocol(protocol, protocol->getUplink()); + + //pthread_t pingThread; + //pthread_create(&pingThread, NULL, ping, (void*)(context)); + + context->waitForTask(); + + //while (!context->isDone()) { + //context->nextKey(); + //} + + context->closeAll(); + protocol->getUplink()->sendCommand(DONE); + + //pthread_join(pingThread,NULL); + + // Cleanup + delete context; + delete protocol; + + if (in_stream != NULL) { + fflush(in_stream); + } + if (out_stream != NULL) { + fflush(out_stream); + } + + fflush(stdout); + + if (sock != -1) { + int result = shutdown(sock, SHUT_RDWR); + HADOOP_ASSERT(result == 0, "problem shutting socket"); + result = close(sock); + HADOOP_ASSERT(result == 0, "problem closing socket"); + } + + //TODO REFACTOR + if (in_stream != NULL) { + //fclose(stream); + } + if (out_stream != NULL) { + //fclose(outStream); + } + delete bufin; + delete bufout; + + return true; + + } catch (Error& err) { + fprintf(stderr, "Hama Pipes Exception: %s\n", + err.getMessage().c_str()); + return false; + } +} Index: c++/src/main/native/pipes/impl/HamaPipes.cc =================================================================== --- c++/src/main/native/pipes/impl/HamaPipes.cc (revision 1542513) +++ c++/src/main/native/pipes/impl/HamaPipes.cc (working copy) @@ -1,1357 +0,0 @@ -/** - * 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. - */ - -#include "hama/Pipes.hh" -#include "hadoop/SerialUtils.hh" -#include "hadoop/StringUtils.hh" - -#include -#include - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#define stringify( name ) # name - -using std::map; -using std::string; -using std::vector; -using std::cout; -using std::endl; - -using namespace HadoopUtils; - -namespace HamaPipes { - - bool logging; - - /********************************************/ - /****************** BSPJob ******************/ - /********************************************/ - class BSPJobImpl: public BSPJob { - private: - map values; - public: - void set(const string& key, const string& value) { - values[key] = value; - } - - virtual bool hasKey(const string& key) const { - return values.find(key) != values.end(); - } - - virtual const string& get(const string& key) const { - map::const_iterator itr = values.find(key); - if (itr == values.end()) { - throw Error("Key " + key + " not found in BSPJob"); - } - return itr->second; - } - - virtual int getInt(const string& key) const { - const string& val = get(key); - return toInt(val); - } - - virtual float getFloat(const string& key) const { - const string& val = get(key); - return toFloat(val); - } - - virtual bool getBoolean(const string&key) const { - const string& val = get(key); - return toBool(val); - } - }; - - /********************************************/ - /************* DownwardProtocol *************/ - /********************************************/ - class DownwardProtocol { - public: - virtual void start(int protocol) = 0; - virtual void setBSPJob(vector values) = 0; - virtual void setInputTypes(string keyType, string valueType) = 0; - virtual void setKeyValue(const string& _key, const string& _value) = 0; - - virtual void runBsp(bool pipedInput, bool pipedOutput) = 0; - virtual void runCleanup(bool pipedInput, bool pipedOutput) = 0; - virtual void runSetup(bool pipedInput, bool pipedOutput) = 0; - virtual void runPartition(const string& key, const string& value, int32_t numTasks) = 0; - - virtual void setNewResult(int32_t value) = 0; - virtual void setNewResult(int64_t value) = 0; - virtual void setNewResult(const string& value) = 0; - virtual void setNewResult(vector value) = 0; - - //virtual void reduceKey(const string& key) = 0; - //virtual void reduceValue(const string& value) = 0; - virtual void close() = 0; - virtual void abort() = 0; - virtual ~DownwardProtocol() {} - }; - - /********************************************/ - /************** UpwardProtocol **************/ - /********************************************/ - class UpwardProtocol { - public: - virtual void sendCMD(int32_t cmd) = 0; - virtual void sendCMD(int32_t cmd, int32_t value) = 0; - virtual void sendCMD(int32_t cmd, int32_t value, const string values[], int size) = 0; - virtual void sendCMD(int32_t cmd, const string& value) = 0; - virtual void sendCMD(int32_t cmd, const string values[], int size) = 0; - - //virtual void registerCounter(int id, const string& group, const string& name) = 0; - //virtual void incrementCounter(const TaskContext::Counter* counter, uint64_t amount) = 0; - virtual void incrementCounter(const string& group, const string& name, uint64_t amount) = 0; - virtual ~UpwardProtocol() {} - }; - - /********************************************/ - /***************** Protocol *****************/ - /********************************************/ - class Protocol { - public: - virtual void nextEvent() = 0; - virtual UpwardProtocol* getUplink() = 0; - virtual ~Protocol(){} - }; - - /********************************************/ - /*************** MESSAGE_TYPE ***************/ - /********************************************/ - enum MESSAGE_TYPE { - START_MESSAGE, SET_BSPJOB_CONF, SET_INPUT_TYPES, - RUN_SETUP, RUN_BSP, RUN_CLEANUP, - READ_KEYVALUE, WRITE_KEYVALUE, - GET_MSG, GET_MSG_COUNT, - SEND_MSG, SYNC, - GET_ALL_PEERNAME, GET_PEERNAME, - GET_PEER_INDEX, GET_PEER_COUNT, GET_SUPERSTEP_COUNT, - REOPEN_INPUT, CLEAR, - CLOSE, ABORT, - DONE, TASK_DONE, - REGISTER_COUNTER, INCREMENT_COUNTER, - SEQFILE_OPEN, SEQFILE_READNEXT, - SEQFILE_APPEND, SEQFILE_CLOSE, - PARTITION_REQUEST, PARTITION_RESPONSE - }; - - /* Only needed for debugging output */ - const char* messageTypeNames[] = { - stringify( START_MESSAGE ), stringify( SET_BSPJOB_CONF ), stringify( SET_INPUT_TYPES ), - stringify( RUN_SETUP ), stringify( RUN_BSP ), stringify( RUN_CLEANUP ), - stringify( READ_KEYVALUE ), stringify( WRITE_KEYVALUE ), - stringify( GET_MSG ), stringify( GET_MSG_COUNT ), - stringify( SEND_MSG ), stringify( SYNC ), - stringify( GET_ALL_PEERNAME ), stringify( GET_PEERNAME ), - stringify( GET_PEER_INDEX ), stringify( GET_PEER_COUNT ), stringify( GET_SUPERSTEP_COUNT ), - stringify( REOPEN_INPUT ), stringify( CLEAR ), - stringify( CLOSE ), stringify( ABORT ), - stringify( DONE ), stringify( TASK_DONE ), - stringify( REGISTER_COUNTER ), stringify( INCREMENT_COUNTER ), - stringify( SEQFILE_OPEN ), stringify( SEQFILE_READNEXT ), - stringify( SEQFILE_APPEND ), stringify( SEQFILE_CLOSE ), - stringify( PARTITION_REQUEST ), stringify( PARTITION_RESPONSE ) - }; - - /********************************************/ - /*********** BinaryUpwardProtocol ***********/ - /********************************************/ - class BinaryUpwardProtocol: public UpwardProtocol { - private: - FileOutStream* stream; - public: - BinaryUpwardProtocol(FILE* _stream) { - stream = new FileOutStream(); - HADOOP_ASSERT(stream->open(_stream), "problem opening stream"); - - } - - virtual void sendCMD(int32_t cmd) { - serializeInt(cmd, *stream); - stream->flush(); - if(logging)fprintf(stderr,"HamaPipes::BinaryUpwardProtocol sent CMD: %s\n", - messageTypeNames[cmd]); - } - - virtual void sendCMD(int32_t cmd, int32_t value) { - serializeInt(cmd, *stream); - serializeInt(value, *stream); - stream->flush(); - if(logging)fprintf(stderr,"HamaPipes::BinaryUpwardProtocol sent CMD: %s with Value: %d\n", - messageTypeNames[cmd],value); - } - - virtual void sendCMD(int32_t cmd, const string& value) { - serializeInt(cmd, *stream); - serializeString(value, *stream); - stream->flush(); - if(logging)fprintf(stderr,"HamaPipes::BinaryUpwardProtocol sent CMD: %s with Value: %s\n", - messageTypeNames[cmd],value.c_str()); - } - - virtual void sendCMD(int32_t cmd, const string values[], int size) { - serializeInt(cmd, *stream); - for (int i=0; iflush(); - } - - virtual void sendCMD(int32_t cmd, int32_t value, const string values[], int size) { - serializeInt(cmd, *stream); - serializeInt(value, *stream); - for (int i=0; iflush(); - } - - /* - virtual void registerCounter(int id, const string& group, - const string& name) { - serializeInt(REGISTER_COUNTER, *stream); - serializeInt(id, *stream); - serializeString(group, *stream); - serializeString(name, *stream); - } - - virtual void incrementCounter(const TaskContext::Counter* counter, - uint64_t amount) { - serializeInt(INCREMENT_COUNTER, *stream); - serializeInt(counter->getId(), *stream); - serializeLong(amount, *stream); - } - */ - - virtual void incrementCounter(const string& group, const string& name, uint64_t amount) { - serializeInt(INCREMENT_COUNTER, *stream); - serializeString(group, *stream); - serializeString(name, *stream); - serializeLong(amount, *stream); - stream->flush(); - if(logging)fprintf(stderr,"HamaPipes::BinaryUpwardProtocol sent incrementCounter\n"); - } - - ~BinaryUpwardProtocol() { - delete stream; - } - }; - - /********************************************/ - /************** BinaryProtocol **************/ - /********************************************/ - class BinaryProtocol: public Protocol { - private: - FileInStream* downStream; - DownwardProtocol* handler; - BinaryUpwardProtocol * uplink; - - string key; - string value; - - public: - BinaryProtocol(FILE* down, DownwardProtocol* _handler, FILE* up) { - downStream = new FileInStream(); - downStream->open(down); - uplink = new BinaryUpwardProtocol(up); - handler = _handler; - } - - UpwardProtocol* getUplink() { - return uplink; - } - - - virtual void nextEvent() { - int32_t cmd; - cmd = deserializeInt(*downStream); - - switch (cmd) { - - case START_MESSAGE: { - int32_t prot; - prot = deserializeInt(*downStream); - if(logging) - fprintf(stderr,"HamaPipes::BinaryProtocol::nextEvent - got START_MESSAGE prot: %d\n", prot); - handler->start(prot); - break; - } - /* SET BSP Job Configuration / Environment */ - case SET_BSPJOB_CONF: { - int32_t entries; - entries = deserializeInt(*downStream); - if(logging) - fprintf(stderr,"HamaPipes::BinaryProtocol::nextEvent - got SET_BSPJOB_CONF entries: %d\n", - entries); - vector result(entries*2); - for(int i=0; i < entries*2; ++i) { - string item; - deserializeString(item, *downStream); - result.push_back(item); - if(logging) - fprintf(stderr,"HamaPipes::BinaryProtocol::nextEvent - got SET_BSPJOB_CONF add NewEntry: %s\n", - item.c_str()); - } - if(logging) - fprintf(stderr,"HamaPipes::BinaryProtocol::nextEvent - got all Configuration %d entries!\n", - entries); - handler->setBSPJob(result); - break; - } - case SET_INPUT_TYPES: { - string keyType; - string valueType; - deserializeString(keyType, *downStream); - deserializeString(valueType, *downStream); - if(logging) - fprintf(stderr,"HamaPipes::BinaryProtocol::nextEvent - got SET_INPUT_TYPES keyType: %s valueType: %s\n", - keyType.c_str(),valueType.c_str()); - handler->setInputTypes(keyType, valueType); - break; - } - case READ_KEYVALUE: { - deserializeString(key, *downStream); - deserializeString(value, *downStream); - if(logging) - fprintf(stderr,"HamaPipes::BinaryProtocol::nextEvent - got READ_KEYVALUE key: %s value: %s\n", - key.c_str(),((value.length()<10)?value.c_str():value.substr(0,9).c_str())); - handler->setKeyValue(key, value); - break; - } - case RUN_SETUP: { - if(logging)fprintf(stderr,"HamaPipes::BinaryProtocol::nextEvent - got RUN_SETUP\n"); - int32_t pipedInput; - int32_t pipedOutput; - pipedInput = deserializeInt(*downStream); - pipedOutput = deserializeInt(*downStream); - handler->runSetup(pipedInput, pipedOutput); - break; - } - case RUN_BSP: { - if(logging)fprintf(stderr,"HamaPipes::BinaryProtocol::nextEvent - got RUN_BSP\n"); - int32_t pipedInput; - int32_t pipedOutput; - pipedInput = deserializeInt(*downStream); - pipedOutput = deserializeInt(*downStream); - handler->runBsp(pipedInput, pipedOutput); - break; - } - case RUN_CLEANUP: { - if(logging)fprintf(stderr,"HamaPipes::BinaryProtocol::nextEvent - got RUN_CLEANUP\n"); - int32_t pipedInput; - int32_t pipedOutput; - pipedInput = deserializeInt(*downStream); - pipedOutput = deserializeInt(*downStream); - handler->runCleanup(pipedInput, pipedOutput); - break; - } - - case PARTITION_REQUEST: { - if(logging)fprintf(stderr,"HamaPipes::BinaryProtocol::nextEvent - got PARTITION_REQUEST\n"); - string partionKey; - string partionValue; - int32_t numTasks; - deserializeString(partionKey, *downStream); - deserializeString(partionValue, *downStream); - numTasks = deserializeInt(*downStream); - handler->runPartition(partionKey, partionValue, numTasks); - break; - } - - case GET_MSG_COUNT: { - int32_t msgCount = deserializeInt(*downStream); - if(logging) - fprintf(stderr,"HamaPipes::BinaryProtocol::nextEvent - got GET_MSG_COUNT msgCount: %d\n", - msgCount); - handler->setNewResult(msgCount); - break; - } - case GET_MSG: { - string msg; - deserializeString(msg,*downStream); - if(logging) - fprintf(stderr,"HamaPipes::BinaryProtocol::nextEvent - got GET_MSG msg: %s\n", - msg.c_str()); - handler->setNewResult(msg); - break; - } - case GET_PEERNAME: { - string peername; - deserializeString(peername,*downStream); - if(logging) - fprintf(stderr,"HamaPipes::BinaryProtocol::nextEvent - got GET_PEERNAME peername: %s\n", - peername.c_str()); - handler->setNewResult(peername); - break; - } - case GET_ALL_PEERNAME: { - vector peernames; - int32_t peernameCount = deserializeInt(*downStream); - if(logging) - fprintf(stderr,"HamaPipes::BinaryProtocol::nextEvent - got GET_ALL_PEERNAME peernameCount: %d\n", - peernameCount); - string peername; - for (int i=0; isetNewResult(peernames); - break; - } - case GET_PEER_INDEX: { - int32_t peerIndex = deserializeInt(*downStream); - if(logging) - fprintf(stderr,"HamaPipes::BinaryProtocol::nextEvent - got GET_PEER_INDEX peerIndex: %d\n", - peerIndex); - handler->setNewResult(peerIndex); - break; - } - case GET_PEER_COUNT: { - int32_t peerCount = deserializeInt(*downStream); - if(logging) - fprintf(stderr,"HamaPipes::BinaryProtocol::nextEvent - got GET_PEER_COUNT peerCount: %d\n", - peerCount); - handler->setNewResult(peerCount); - break; - } - case GET_SUPERSTEP_COUNT: { - int64_t superstepCount = deserializeLong(*downStream); - if(logging) - fprintf(stderr,"HamaPipes::BinaryProtocol::nextEvent - got GET_SUPERSTEP_COUNT superstepCount: %ld\n", - (long)superstepCount); - handler->setNewResult(superstepCount); - break; - } - - - case SEQFILE_OPEN: { - int32_t fileID = deserializeInt(*downStream); - if(logging) - fprintf(stderr,"HamaPipes::BinaryProtocol::nextEvent - got SEQFILE_OPEN fileID: %d\n",fileID); - handler->setNewResult(fileID); - break; - } - case SEQFILE_READNEXT: { - deserializeString(key, *downStream); - deserializeString(value, *downStream); - if(logging) - fprintf(stderr,"HamaPipes::BinaryProtocol::nextEvent - got SEQFILE_READNEXT key: %s value: %s\n", - key.c_str(),((value.length()<10)?value.c_str():value.substr(0,9).c_str())); - handler->setKeyValue(key, value); - break; - } - case SEQFILE_APPEND: { - int32_t result = deserializeInt(*downStream); - if(logging) - fprintf(stderr,"HamaPipes::BinaryProtocol::nextEvent - got SEQFILE_APPEND result: %d\n",result); - handler->setNewResult(result); - break; - } - case SEQFILE_CLOSE: { - int32_t result = deserializeInt(*downStream); - if(logging) - fprintf(stderr,"HamaPipes::BinaryProtocol::nextEvent - got SEQFILE_CLOSE result: %d\n",result); - handler->setNewResult(result); - break; - } - - - case CLOSE: { - if(logging)fprintf(stderr,"HamaPipes::BinaryProtocol::nextEvent - got CLOSE\n"); - handler->close(); - break; - } - case ABORT: { - if(logging)fprintf(stderr,"HamaPipes::BinaryProtocol::nextEvent - got ABORT\n"); - handler->abort(); - break; - } - default: - HADOOP_ASSERT(false, "Unknown binary command " + toString(cmd)); - fprintf(stderr,"HamaPipes::BinaryProtocol::nextEvent - Unknown binary command: %d\n",cmd); - } - } - - virtual ~BinaryProtocol() { - delete downStream; - delete uplink; - } - }; - - /********************************************/ - /************** BSPContextImpl **************/ - /********************************************/ - class BSPContextImpl: public BSPContext, public DownwardProtocol { - private: - bool done; - BSPJob* job; - //string key; - //const string* newKey; - //const string* value; - bool hasTask; - //bool isNewKey; - //bool isNewValue; - string* inputKeyClass; - string* inputValueClass; - - //string status; - //float progressFloat; - //uint64_t lastProgress; - //bool statusSet; - - Protocol* protocol; - UpwardProtocol *uplink; - - //string* inputSplit; - - RecordReader* reader; - RecordWriter* writer; - - BSP* bsp; - Partitioner* partitioner; - - const Factory* factory; - pthread_mutex_t mutexDone; - std::vector registeredCounterIds; - - int32_t resultInt; - bool isNewResultInt; - int64_t resultLong; - bool isNewResultLong; - string resultString; - bool isNewResultString; - vector resultVector; - bool isNewResultVector; - - bool isNewKeyValuePair; - string currentKey; - string currentValue; - - public: - - BSPContextImpl(const Factory& _factory) { - //statusSet = false; - done = false; - //newKey = NULL; - factory = &_factory; - job = NULL; - - inputKeyClass = NULL; - inputValueClass = NULL; - - //inputSplit = NULL; - - bsp = NULL; - reader = NULL; - writer = NULL; - partitioner = NULL; - protocol = NULL; - //isNewKey = false; - //isNewValue = false; - //lastProgress = 0; - //progressFloat = 0.0f; - hasTask = false; - pthread_mutex_init(&mutexDone, NULL); - - isNewResultInt = false; - isNewResultString = false, - isNewResultVector = false; - - isNewKeyValuePair = false; - } - - - /********************************************/ - /*********** DownwardProtocol IMPL **********/ - /********************************************/ - virtual void start(int protocol) { - if (protocol != 0) { - throw Error("Protocol version " + toString(protocol) + - " not supported"); - } - partitioner = factory->createPartitioner(*this); - } - - virtual void setBSPJob(vector values) { - int len = values.size(); - BSPJobImpl* result = new BSPJobImpl(); - HADOOP_ASSERT(len % 2 == 0, "Odd length of job conf values"); - for(int i=0; i < len; i += 2) { - result->set(values[i], values[i+1]); - } - job = result; - } - - virtual void setInputTypes(string keyType, string valueType) { - inputKeyClass = new string(keyType); - inputValueClass = new string(valueType); - } - - virtual void setKeyValue(const string& _key, const string& _value) { - currentKey = _key; - currentValue = _value; - isNewKeyValuePair = true; - } - - /* private Method */ - void setupReaderWriter(bool pipedInput, bool pipedOutput) { - - if(logging) - fprintf(stderr,"HamaPipes::BSPContextImpl::setupReaderWriter - pipedInput: %s pipedOutput: %s\n", - (pipedInput)?"true":"false",(pipedOutput)?"true":"false"); - - if (pipedInput && reader==NULL) { - reader = factory->createRecordReader(*this); - HADOOP_ASSERT((reader == NULL) == pipedInput, - pipedInput ? "RecordReader defined when not needed.": - "RecordReader not defined"); - - //if (reader != NULL) { - // value = new string(); - //} - } - - if (pipedOutput && writer==NULL) { - writer = factory->createRecordWriter(*this); - HADOOP_ASSERT((writer == NULL) == pipedOutput, - pipedOutput ? "RecordWriter defined when not needed.": - "RecordWriter not defined"); - } - } - - virtual void runSetup(bool pipedInput, bool pipedOutput) { - setupReaderWriter(pipedInput,pipedOutput); - - if (bsp == NULL) - bsp = factory->createBSP(*this); - - if (bsp != NULL) { - hasTask = true; - bsp->setup(*this); - hasTask = false; - uplink->sendCMD(TASK_DONE); - } - } - - virtual void runBsp(bool pipedInput, bool pipedOutput) { - setupReaderWriter(pipedInput,pipedOutput); - - if (bsp == NULL) - bsp = factory->createBSP(*this); - - if (bsp != NULL) { - hasTask = true; - bsp->bsp(*this); - hasTask = false; - uplink->sendCMD(TASK_DONE); - } - } - - virtual void runCleanup(bool pipedInput, bool pipedOutput) { - setupReaderWriter(pipedInput,pipedOutput); - - if (bsp != NULL) { - hasTask = true; - bsp->cleanup(*this); - hasTask = false; - uplink->sendCMD(TASK_DONE); - } - } - - /********************************************/ - /******* Partitioner *******/ - /********************************************/ - virtual void runPartition(const string& key, const string& value, int32_t numTasks){ - if (partitioner != NULL) { - int part = partitioner->partition(key, value, numTasks); - uplink->sendCMD(PARTITION_RESPONSE, part); - } else { - if(logging) - fprintf(stderr,"HamaPipes::BSPContextImpl::runPartition Partitioner is NULL!\n"); - } - } - - virtual void setNewResult(int32_t _value) { - resultInt = _value; - isNewResultInt = true; - } - - virtual void setNewResult(int64_t _value) { - resultLong = _value; - isNewResultLong = true; - } - - virtual void setNewResult(const string& _value) { - resultString = _value; - isNewResultString = true; - } - - virtual void setNewResult(vector _value) { - resultVector = _value; - isNewResultVector = true; - } - - virtual void close() { - pthread_mutex_lock(&mutexDone); - done = true; - hasTask = false; - pthread_mutex_unlock(&mutexDone); - if(logging) - fprintf(stderr,"HamaPipes::BSPContextImpl::close - done: %s hasTask: %s\n", - (done)?"true":"false",(hasTask)?"true":"false"); - } - - virtual void abort() { - throw Error("Aborted by driver"); - } - - /********************************************/ - /************** TaskContext IMPL ************/ - /********************************************/ - - /** - * Get the BSPJob for the current task. - */ - virtual const BSPJob* getBSPJob() { - return job; - } - - /** - * Get the current key. - * @return the current key or NULL if called before the first map or reduce - */ - //virtual const string& getInputKey() { - // return key; - //} - - /** - * Get the current value. - * @return the current value or NULL if called before the first map or - * reduce - */ - //virtual const string& getInputValue() { - // return *value; - //} - - /** - * Register a counter with the given group and name. - */ - /* - virtual Counter* getCounter(const std::string& group, - const std::string& name) { - int id = registeredCounterIds.size(); - registeredCounterIds.push_back(id); - uplink->registerCounter(id, group, name); - return new Counter(id); - }*/ - - /** - * Increment the value of the counter with the given amount. - */ - virtual void incrementCounter(const string& group, const string& name, uint64_t amount) { - uplink->incrementCounter(group, name, amount); - } - - /********************************************/ - /************** BSPContext IMPL *************/ - /********************************************/ - - /** - * Access the InputSplit of the bsp. - */ - //virtual const string& getInputSplit() { - // return *inputSplit; - //} - - /** - * Get the name of the key class of the input to this task. - */ - virtual const string& getInputKeyClass() { - return *inputKeyClass; - } - - /** - * Get the name of the value class of the input to this task. - */ - virtual const string& getInputValueClass() { - return *inputValueClass; - } - - /** - * Send a data with a tag to another BSPSlave corresponding to hostname. - * Messages sent by this method are not guaranteed to be received in a sent - * order. - */ - virtual void sendMessage(const string& peerName, const string& msg) { - string values[] = {peerName, msg}; - uplink->sendCMD(SEND_MSG,values, 2); - } - - /** - * @return A message from the peer's received messages queue (a FIFO). - */ - virtual const string& getCurrentMessage() { - uplink->sendCMD(GET_MSG); - - while (!isNewResultString) - protocol->nextEvent(); - - isNewResultString = false; - if(logging) - fprintf(stderr,"HamaPipes::BSPContextImpl::getMessage - NewResultString: %s\n", - resultString.c_str()); - return resultString; - } - - /** - * @return The number of messages in the peer's received messages queue. - */ - virtual int getNumCurrentMessages() { - uplink->sendCMD(GET_MSG_COUNT); - - while (!isNewResultInt) - protocol->nextEvent(); - - isNewResultInt = false; - return resultInt; - } - - /** - * Barrier Synchronization. - * - * Sends all the messages in the outgoing message queues to the corresponding - * remote peers. - */ - virtual void sync() { - uplink->sendCMD(SYNC); - } - - /** - * @return the name of this peer in the format "hostname:port". - */ - virtual const string& getPeerName() { - uplink->sendCMD(GET_PEERNAME,-1); - - while (!isNewResultString) - protocol->nextEvent(); - - if(logging) - fprintf(stderr,"HamaPipes::BSPContextImpl::getPeerName - NewResultString: %s\n", - resultString.c_str()); - isNewResultString = false; - return resultString; - } - - /** - * @return the name of n-th peer from sorted array by name. - */ - virtual const string& getPeerName(int index) { - uplink->sendCMD(GET_PEERNAME,index); - - while (!isNewResultString) - protocol->nextEvent(); - - if(logging) - fprintf(stderr,"HamaPipes::BSPContextImpl::getPeerName - NewResultString: %s\n", - resultString.c_str()); - isNewResultString = false; - return resultString; - } - - /** - * @return the names of all the peers executing tasks from the same job - * (including this peer). - */ - virtual vector getAllPeerNames() { - uplink->sendCMD(GET_ALL_PEERNAME); - - while (!isNewResultVector) - protocol->nextEvent(); - - isNewResultVector = false; - return resultVector; - } - - /** - * @return the index of this peer from sorted array by name. - */ - virtual int getPeerIndex() { - uplink->sendCMD(GET_PEER_INDEX); - - while (!isNewResultInt) - protocol->nextEvent(); - - isNewResultInt = false; - return resultInt; - } - - /** - * @return the number of peers - */ - virtual int getNumPeers() { - uplink->sendCMD(GET_PEER_COUNT); - - while (!isNewResultInt) - protocol->nextEvent(); - - isNewResultInt = false; - return resultInt; - } - - /** - * @return the count of current super-step - */ - virtual long getSuperstepCount() { - uplink->sendCMD(GET_SUPERSTEP_COUNT); - - while (!isNewResultLong) - protocol->nextEvent(); - - isNewResultLong = false; - return resultLong; - } - - /** - * Clears all queues entries. - */ - virtual void clear() { - uplink->sendCMD(CLEAR); - } - - /** - * Writes a key/value pair to the output collector - */ - virtual void write(const string& key, const string& value) { - if (writer != NULL) { - writer->emit(key, value); - } else { - string values[] = {key, value}; - uplink->sendCMD(WRITE_KEYVALUE, values, 2); - } - } - - /** - * Deserializes the next input key value into the given objects; - */ - virtual bool readNext(string& _key, string& _value) { - uplink->sendCMD(READ_KEYVALUE); - - while (!isNewKeyValuePair) - protocol->nextEvent(); - - isNewKeyValuePair = false; - - _key = currentKey; - - // check if value is array [0, 1, 2, ...], and remove brackets - int len = currentValue.length(); - if ( (currentValue[0]=='[') && - (currentValue[len-1]==']') ) { - _value = currentValue.substr(1,len-2); - } else { - _value = currentValue; - } - - if (logging && _key.empty() && _value.empty()) { - fprintf(stderr,"HamaPipes::BSPContextImpl::readNext - Empty KeyValuePair\n"); - } - - return (!_key.empty() && !_value.empty()); - } - - /** - * Closes the input and opens it right away, so that the file pointer is at - * the beginning again. - */ - virtual void reopenInput() { - uplink->sendCMD(REOPEN_INPUT); - } - - - /********************************************/ - /******* SequenceFileConnector IMPL *******/ - /********************************************/ - - /** - * Open SequenceFile with opion "r" or "w" - * @return the corresponding fileID - */ - virtual int sequenceFileOpen(const string& path, const string& option, - const string& keyType, const string& valueType) { - if (logging) - fprintf(stderr,"HamaPipes::BSPContextImpl::sequenceFileOpen - Path: %s\n", - path.c_str()); - - if ( (option.compare("r")==0) || (option.compare("w")==0)) { - - string values[] = {path, option, keyType, valueType}; - uplink->sendCMD(SEQFILE_OPEN,values, 4); - - while (!isNewResultInt) - protocol->nextEvent(); - - isNewResultInt = false; - return resultInt; - } else { - fprintf(stderr,"HamaPipes::BSPContextImpl::sequenceFileOpen wrong option: %s!\n", - option.c_str()); - return -1; //Error wrong option - } - } - - /** - * Read next key/value pair from the SequenceFile with fileID - */ - virtual bool sequenceFileReadNext(int fileID, string& _key, string& _value) { - - uplink->sendCMD(SEQFILE_READNEXT,fileID); - - while (!isNewKeyValuePair) - protocol->nextEvent(); - - isNewKeyValuePair = false; - - _key = currentKey; - // check if value is array [0, 1, 2, ...], and remove brackets - int len = currentValue.length(); - if ( (currentValue[0]=='[') && - (currentValue[len-1]==']') ) { - _value = currentValue.substr(1,len-2); - } else { - _value = currentValue; - } - - if (logging && _key.empty() && _value.empty()) - fprintf(stderr,"HamaPipes::BSPContextImpl::sequenceFileReadNext - Empty KeyValuePair\n"); - - return (!_key.empty() && !_value.empty()); - } - - /** - * Append the next key/value pair to the SequenceFile with fileID - */ - virtual bool sequenceFileAppend(int fileID, const string& key, const string& value) { - string values[] = {key, value}; - uplink->sendCMD(SEQFILE_APPEND,fileID, values, 2); - - while (!isNewResultInt) - protocol->nextEvent(); - - isNewResultInt = false; - return (resultInt==1); - } - - /** - * Close SequenceFile - */ - virtual bool sequenceFileClose(int fileID) { - uplink->sendCMD(SEQFILE_CLOSE,fileID); - - while (!isNewResultInt) - protocol->nextEvent(); - - if (logging && resultInt==0) - fprintf(stderr,"HamaPipes::BSPContextImpl::sequenceFileClose - Nothing was closed!\n"); - else if (logging) - fprintf(stderr,"HamaPipes::BSPContextImpl::sequenceFileClose - File was successfully closed!\n"); - - isNewResultInt = false; - return (resultInt==1); - } - - /********************************************/ - /*************** Other STUFF ***************/ - /********************************************/ - - void setProtocol(Protocol* _protocol, UpwardProtocol* _uplink) { - protocol = _protocol; - uplink = _uplink; - } - - bool isDone() { - pthread_mutex_lock(&mutexDone); - bool doneCopy = done; - pthread_mutex_unlock(&mutexDone); - return doneCopy; - } - - /** - * Advance to the next value. - */ - /* - bool nextValue() { - if (isNewKey || done) { - return false; - } - isNewValue = false; - //progress(); - protocol->nextEvent(); - return isNewValue; - } - */ - void waitForTask() { - while (!done && !hasTask) { - if(logging)fprintf(stderr,"HamaPipes::BSPContextImpl::waitForTask - done: %s hasTask: %s\n", - (done)?"true":"false",(hasTask)?"true":"false"); - protocol->nextEvent(); - } - } - /* - bool nextKey() { - if (reader == NULL) { - while (!isNewKey) { - nextValue(); - if (done) { - return false; - } - } - key = *newKey; - } else { - if (!reader->next(key, const_cast(*value))) { - pthread_mutex_lock(&mutexDone); - done = true; - pthread_mutex_unlock(&mutexDone); - return false; - } - //progressFloat = reader->getProgress(); - } - isNewKey = false; - - if (bsp != NULL) { - bsp->bsp(*this); - } - return true; - } - */ - void closeAll() { - if (reader) { - reader->close(); - } - - if (bsp) { - bsp->close(); - } - - if (writer) { - writer->close(); - } - } - - virtual ~BSPContextImpl() { - delete job; - delete inputKeyClass; - delete inputValueClass; - //delete inputSplit; - //if (reader) { - // delete value; - //} - delete reader; - delete bsp; - delete writer; - pthread_mutex_destroy(&mutexDone); - } - }; - - /** - * Ping the parent every 5 seconds to know if it is alive - */ - void* ping(void* ptr) { - BSPContextImpl* context = (BSPContextImpl*) ptr; - char* portStr = getenv("hama.pipes.command.port"); - int MAX_RETRIES = 3; - int remaining_retries = MAX_RETRIES; - while (!context->isDone()) { - try{ - sleep(5); - int sock = -1; - if (portStr) { - sock = socket(PF_INET, SOCK_STREAM, 0); - HADOOP_ASSERT(sock != - 1, - string("problem creating socket: ") + strerror(errno)); - sockaddr_in addr; - addr.sin_family = AF_INET; - addr.sin_port = htons(toInt(portStr)); - addr.sin_addr.s_addr = htonl(INADDR_LOOPBACK); - if(logging) - fprintf(stderr,"HamaPipes::ping - connected to GroomServer Port: %s\n", - portStr); - HADOOP_ASSERT(connect(sock, (sockaddr*) &addr, sizeof(addr)) == 0, - string("problem connecting command socket: ") + - strerror(errno)); - - } - if (sock != -1) { - int result = shutdown(sock, SHUT_RDWR); - HADOOP_ASSERT(result == 0, "problem shutting socket"); - result = close(sock); - HADOOP_ASSERT(result == 0, "problem closing socket"); - } - remaining_retries = MAX_RETRIES; - } catch (Error& err) { - if (!context->isDone()) { - fprintf(stderr, "Hama Pipes Exception: in ping %s\n", - err.getMessage().c_str()); - remaining_retries -= 1; - if (remaining_retries == 0) { - exit(1); - } - } else { - return NULL; - } - } - } - return NULL; - } - - /** - * Run the assigned task in the framework. - * The user's main function should set the various functions using the - * set* functions above and then call this. - * @return true, if the task succeeded. - */ - bool runTask(const Factory& factory) { - try { - HADOOP_ASSERT(getenv("hama.pipes.logging")!=NULL,"No environment found!"); - - logging = (toInt(getenv("hama.pipes.logging"))==0)?false:true; - if(logging) - fprintf(stderr,"HamaPipes::runTask - logging is: %s\n", - ((logging)?"true":"false")); - - BSPContextImpl* context = new BSPContextImpl(factory); - Protocol* connection; - - char* portStr = getenv("hama.pipes.command.port"); - int sock = -1; - FILE* stream = NULL; - FILE* outStream = NULL; - char *bufin = NULL; - char *bufout = NULL; - if (portStr) { - sock = socket(PF_INET, SOCK_STREAM, 0); - HADOOP_ASSERT(sock != - 1, - string("problem creating socket: ") + strerror(errno)); - sockaddr_in addr; - addr.sin_family = AF_INET; - addr.sin_port = htons(toInt(portStr)); - addr.sin_addr.s_addr = htonl(INADDR_LOOPBACK); - HADOOP_ASSERT(connect(sock, (sockaddr*) &addr, sizeof(addr)) == 0, - string("problem connecting command socket: ") + - strerror(errno)); - - stream = fdopen(sock, "r"); - outStream = fdopen(sock, "w"); - - // increase buffer size - int bufsize = 128*1024; - int setbuf; - bufin = new char[bufsize]; - bufout = new char[bufsize]; - setbuf = setvbuf(stream, bufin, _IOFBF, bufsize); - HADOOP_ASSERT(setbuf == 0, string("problem with setvbuf for inStream: ") - + strerror(errno)); - setbuf = setvbuf(outStream, bufout, _IOFBF, bufsize); - HADOOP_ASSERT(setbuf == 0, string("problem with setvbuf for outStream: ") - + strerror(errno)); - - connection = new BinaryProtocol(stream, context, outStream); - if(logging) - fprintf(stderr,"HamaPipes::runTask - connected to GroomServer Port: %s\n", - portStr); - - } else if (getenv("hama.pipes.command.file")) { - char* filename = getenv("hama.pipes.command.file"); - string outFilename = filename; - outFilename += ".out"; - stream = fopen(filename, "r"); - outStream = fopen(outFilename.c_str(), "w"); - connection = new BinaryProtocol(stream, context, outStream); - } else { - //connection = new TextProtocol(stdin, context, stdout); - fprintf(stderr,"HamaPipes::runTask - Connection couldn't be initialized!\n"); - return -1; - } - - context->setProtocol(connection, connection->getUplink()); - - //pthread_t pingThread; - //pthread_create(&pingThread, NULL, ping, (void*)(context)); - - context->waitForTask(); - - //while (!context->isDone()) { - //context->nextKey(); - //} - - context->closeAll(); - connection->getUplink()->sendCMD(DONE); - - //pthread_join(pingThread,NULL); - - delete context; - delete connection; - if (stream != NULL) { - fflush(stream); - } - if (outStream != NULL) { - fflush(outStream); - } - fflush(stdout); - if (sock != -1) { - int result = shutdown(sock, SHUT_RDWR); - HADOOP_ASSERT(result == 0, "problem shutting socket"); - result = close(sock); - HADOOP_ASSERT(result == 0, "problem closing socket"); - } - if (stream != NULL) { - //fclose(stream); - } - if (outStream != NULL) { - //fclose(outStream); - } - delete bufin; - delete bufout; - return true; - } catch (Error& err) { - fprintf(stderr, "Hama Pipes Exception: %s\n", - err.getMessage().c_str()); - return false; - } - } -} - Index: c++/src/main/native/utils/api/hadoop/SerialUtils.hh =================================================================== --- c++/src/main/native/utils/api/hadoop/SerialUtils.hh (revision 1542513) +++ c++/src/main/native/utils/api/hadoop/SerialUtils.hh (working copy) @@ -22,7 +22,7 @@ #include namespace HadoopUtils { - + /** * A simple exception class that records a message for the user. */ @@ -30,38 +30,38 @@ private: std::string error; public: - + /** * Create an error object with the given message. */ Error(const std::string& msg); - + /** * Construct an error object with the given message that was created on * the given file, line, and functino. */ - Error(const std::string& msg, + Error(const std::string& msg, const std::string& file, int line, const std::string& function); - + /** * Get the error message. */ const std::string& getMessage() const; }; - + /** * Check to make sure that the condition is true, and throw an exception * if it is not. The exception will contain the message and a description * of the source location. */ - #define HADOOP_ASSERT(CONDITION, MESSAGE) \ - { \ - if (!(CONDITION)) { \ - throw HadoopUtils::Error((MESSAGE), __FILE__, __LINE__, \ - __PRETTY_FUNCTION__); \ - } \ - } - +#define HADOOP_ASSERT(CONDITION, MESSAGE) \ +{ \ +if (!(CONDITION)) { \ +throw HadoopUtils::Error((MESSAGE), __FILE__, __LINE__, \ +__PRETTY_FUNCTION__); \ +} \ +} + /** * An interface for an input stream. */ @@ -76,7 +76,7 @@ virtual void read(void *buf, size_t len) = 0; virtual ~InStream() {} }; - + /** * An interface for an output stream. */ @@ -95,7 +95,7 @@ virtual void flush() = 0; virtual ~OutStream() {} }; - + /** * A class to read a file as a stream. */ @@ -118,18 +118,18 @@ */ bool isOwned; }; - + /** * A class to write a stream to a file. */ class FileOutStream: public OutStream { public: - + /** * Create a stream that isn't bound to anything. */ FileOutStream(); - + /** * Create the given file, potentially overwriting an existing file. */ @@ -144,7 +144,7 @@ FILE *mFile; bool isOwned; }; - + /** * A stream that reads from a string. */ @@ -156,15 +156,17 @@ const std::string& buffer; std::string::const_iterator itr; }; - + void serializeInt(int32_t t, OutStream& stream); int32_t deserializeInt(InStream& stream); void serializeLong(int64_t t, OutStream& stream); int64_t deserializeLong(InStream& stream); void serializeFloat(float t, OutStream& stream); float deserializeFloat(InStream& stream); + void serializeDouble(double t, OutStream& stream); + double deserializeDouble(InStream& stream); void serializeString(const std::string& t, OutStream& stream); - void deserializeString(std::string& t, InStream& stream); + std::string deserializeString(InStream& stream); } #endif Index: c++/src/main/native/utils/impl/StringUtils.cc =================================================================== --- c++/src/main/native/utils/impl/StringUtils.cc (revision 1542513) +++ c++/src/main/native/utils/impl/StringUtils.cc (working copy) @@ -32,14 +32,14 @@ using std::vector; namespace HadoopUtils { - + // Added by Apache Hama Pipes string toString(double x) { std::stringstream ss; ss << std::setprecision(16) << x; return ss.str(); } - + // Added by Apache Hama Pipes double toDouble(const string& val) { const char* begin = val.c_str(); @@ -47,36 +47,36 @@ double result = strtod(begin, &end); size_t s = end - begin; if(s < val.size()) { - throw Error("Problem converting "+val+" to double. (result:" + throw Error("Problem converting '"+val+"' to double. (result:" +toString(result)+")"); } return result; } - + string toString(int32_t x) { char str[100]; sprintf(str, "%d", x); return str; } - + int toInt(const string& val) { int result; char trash; int num = sscanf(val.c_str(), "%d%c", &result, &trash); HADOOP_ASSERT(num == 1, - "Problem converting " + val + " to integer."); + "Problem converting '" + val + "' to integer."); return result; } - + float toFloat(const string& val) { float result; char trash; int num = sscanf(val.c_str(), "%f%c", &result, &trash); HADOOP_ASSERT(num == 1, - "Problem converting " + val + " to float."); + "Problem converting '" + val + "' to float."); return result; } - + bool toBool(const string& val) { if (val == "true") { return true; @@ -84,10 +84,10 @@ return false; } else { HADOOP_ASSERT(false, - "Problem converting " + val + " to boolean."); + "Problem converting '" + val + "' to boolean."); } } - + /** * Get the current time in the number of milliseconds since 1970. */ @@ -98,15 +98,15 @@ HADOOP_ASSERT(sys != -1, strerror(errno)); return tv.tv_sec * 1000 + tv.tv_usec / 1000; } - + vector splitString(const std::string& str, - const char* separator) { + const char* separator) { vector result; string::size_type prev_pos=0; string::size_type pos=0; while ((pos = str.find_first_of(separator, prev_pos)) != string::npos) { if (prev_pos < pos) { - result.push_back(str.substr(prev_pos, pos-prev_pos)); + result.push_back(str.substr(prev_pos, pos-prev_pos)); } prev_pos = pos + 1; } @@ -115,7 +115,7 @@ } return result; } - + string quoteString(const string& str, const char* deliminators) { @@ -123,31 +123,31 @@ for(int i=result.length() -1; i >= 0; --i) { char ch = result[i]; if (!isprint(ch) || - ch == '\\' || + ch == '\\' || strchr(deliminators, ch)) { switch (ch) { - case '\\': - result.replace(i, 1, "\\\\"); - break; - case '\t': - result.replace(i, 1, "\\t"); - break; - case '\n': - result.replace(i, 1, "\\n"); - break; - case ' ': - result.replace(i, 1, "\\s"); - break; - default: - char buff[4]; - sprintf(buff, "\\%02x", static_cast(result[i])); - result.replace(i, 1, buff); + case '\\': + result.replace(i, 1, "\\\\"); + break; + case '\t': + result.replace(i, 1, "\\t"); + break; + case '\n': + result.replace(i, 1, "\\n"); + break; + case ' ': + result.replace(i, 1, "\\s"); + break; + default: + char buff[4]; + sprintf(buff, "\\%02x", static_cast(result[i])); + result.replace(i, 1, buff); } } } return result; } - + string unquoteString(const string& str) { string result(str); string::size_type current = result.find('\\'); @@ -158,8 +158,8 @@ if (isxdigit(result[current+1])) { num_chars = 2; HADOOP_ASSERT(current + num_chars < result.size(), - "escape pattern \\ is missing second digit in '" - + str + "'"); + "escape pattern \\ is missing second digit in '" + + str + "'"); char sub_str[3]; sub_str[0] = result[current+1]; sub_str[1] = result[current+2]; @@ -167,27 +167,27 @@ char* end_ptr = NULL; long int int_val = strtol(sub_str, &end_ptr, 16); HADOOP_ASSERT(*end_ptr == '\0' && int_val >= 0, - "escape pattern \\ is broken in '" + str + "'"); + "escape pattern \\ is broken in '" + str + "'"); new_ch = static_cast(int_val); } else { num_chars = 1; switch(result[current+1]) { - case '\\': - new_ch = '\\'; - break; - case 't': - new_ch = '\t'; - break; - case 'n': - new_ch = '\n'; - break; - case 's': - new_ch = ' '; - break; - default: - string msg("unknow n escape character '"); - msg += result[current+1]; - HADOOP_ASSERT(false, msg + "' found in '" + str + "'"); + case '\\': + new_ch = '\\'; + break; + case 't': + new_ch = '\t'; + break; + case 'n': + new_ch = '\n'; + break; + case 's': + new_ch = ' '; + break; + default: + string msg("unknow n escape character '"); + msg += result[current+1]; + HADOOP_ASSERT(false, msg + "' found in '" + str + "'"); } } result.replace(current, 1 + num_chars, 1, new_ch); @@ -198,5 +198,5 @@ } return result; } - + } Index: c++/src/main/native/utils/impl/SerialUtils.cc =================================================================== --- c++/src/main/native/utils/impl/SerialUtils.cc (revision 1542513) +++ c++/src/main/native/utils/impl/SerialUtils.cc (working copy) @@ -27,41 +27,41 @@ using std::string; namespace HadoopUtils { - + Error::Error(const std::string& msg): error(msg) { } - - Error::Error(const std::string& msg, - const std::string& file, int line, + + Error::Error(const std::string& msg, + const std::string& file, int line, const std::string& function) { - error = msg + " at " + file + ":" + toString(line) + - " in " + function; + error = msg + " at " + file + ":" + toString(line) + + " in " + function; } - + const std::string& Error::getMessage() const { return error; } - + FileInStream::FileInStream() { mFile = NULL; isOwned = false; } - + bool FileInStream::open(const std::string& name) { mFile = fopen(name.c_str(), "rb"); isOwned = true; return (mFile != NULL); } - + bool FileInStream::open(FILE* file) { mFile = file; isOwned = false; return (mFile != NULL); } - + void FileInStream::read(void *buf, size_t len) { size_t result = fread(buf, len, 1, mFile); @@ -73,12 +73,12 @@ } } } - + bool FileInStream::skip(size_t nbytes) { return (0==fseek(mFile, nbytes, SEEK_CUR)); } - + bool FileInStream::close() { int ret = 0; @@ -88,20 +88,20 @@ mFile = NULL; return (ret==0); } - + FileInStream::~FileInStream() { if (mFile != NULL) { close(); } } - + FileOutStream::FileOutStream() { mFile = NULL; isOwned = false; } - + bool FileOutStream::open(const std::string& name, bool overwrite) { if (!overwrite) { @@ -115,26 +115,26 @@ isOwned = true; return (mFile != NULL); } - + bool FileOutStream::open(FILE* file) { mFile = file; isOwned = false; return (mFile != NULL); } - + void FileOutStream::write(const void* buf, size_t len) { size_t result = fwrite(buf, len, 1, mFile); HADOOP_ASSERT(result == 1, string("write error to file: ") + strerror(errno)); } - + bool FileOutStream::advance(size_t nbytes) { return (0==fseek(mFile, nbytes, SEEK_CUR)); } - + bool FileOutStream::close() { int ret = 0; @@ -144,23 +144,23 @@ mFile = NULL; return (ret == 0); } - + void FileOutStream::flush() { fflush(mFile); } - + FileOutStream::~FileOutStream() { if (mFile != NULL) { close(); } } - + StringInStream::StringInStream(const std::string& str): buffer(str) { itr = buffer.begin(); } - + void StringInStream::read(void *buf, size_t buflen) { size_t bytes = 0; char* output = (char*) buf; @@ -174,11 +174,11 @@ } HADOOP_ASSERT(bytes == buflen, "unexpected end of string reached"); } - + void serializeInt(int32_t t, OutStream& stream) { serializeLong(t,stream); } - + void serializeLong(int64_t t, OutStream& stream) { if (t >= -112 && t <= 127) { @@ -186,22 +186,22 @@ stream.write(&b, 1); return; } - + int8_t len = -112; if (t < 0) { t ^= -1ll; // reset the sign bit len = -120; } - + uint64_t tmp = t; while (tmp != 0) { tmp = tmp >> 8; len--; } - - stream.write(&len, 1); + + stream.write(&len, 1); len = (len < -120) ? -(len + 120) : -(len + 112); - + for (uint32_t idx = len; idx != 0; idx--) { uint32_t shiftbits = (idx - 1) * 8; uint64_t mask = 0xFFll << shiftbits; @@ -209,11 +209,11 @@ stream.write(&b, 1); } } - + int32_t deserializeInt(InStream& stream) { return deserializeLong(stream); } - + int64_t deserializeLong(InStream& stream) { int8_t b; @@ -242,7 +242,7 @@ } return t; } - + void serializeFloat(float t, OutStream& stream) { char buf[sizeof(float)]; @@ -251,16 +251,38 @@ xdr_float(&xdrs, &t); stream.write(buf, sizeof(float)); } - - void deserializeFloat(float& t, InStream& stream) + + float deserializeFloat(InStream& stream) { + float t; char buf[sizeof(float)]; stream.read(buf, sizeof(float)); XDR xdrs; xdrmem_create(&xdrs, buf, sizeof(float), XDR_DECODE); xdr_float(&xdrs, &t); + return t; } - + + void serializeDouble(double t, OutStream& stream) + { + char buf[sizeof(double)]; + XDR xdrs; + xdrmem_create(&xdrs, buf, sizeof(double), XDR_ENCODE); + xdr_double(&xdrs, &t); + stream.write(buf, sizeof(double)); + } + + double deserializeDouble(InStream& stream) + { + double t; + char buf[sizeof(double)]; + stream.read(buf, sizeof(double)); + XDR xdrs; + xdrmem_create(&xdrs, buf, sizeof(double), XDR_DECODE); + xdr_double(&xdrs, &t); + return t; + } + void serializeString(const std::string& t, OutStream& stream) { serializeInt(t.length(), stream); @@ -268,9 +290,10 @@ stream.write(t.data(), t.length()); } } - - void deserializeString(std::string& t, InStream& stream) + + std::string deserializeString(InStream& stream) { + std::string t; int32_t len = deserializeInt(stream); if (len > 0) { // resize the string to the right length @@ -289,6 +312,6 @@ } else { t.clear(); } + return t; } - } Index: c++/README.txt =================================================================== --- c++/README.txt (revision 0) +++ c++/README.txt (revision 0) @@ -0,0 +1,25 @@ +#################################################################### +# Hama Pipes README # +#################################################################### +# Hama Pipes includes the following three examples: # +# - 1) Summation # +# - 2) PiEstimator # +# - 3) MatrixMultiplication # +# in c++/src/main/native/examples # +# Please see c++/src/main/native/examples/README.txt # +#################################################################### + +Please use the following command to compile: + +% g++ -m64 -Ic++/src/main/native/utils/api \ + -Ic++/src/main/native/pipes/api \ + -Lc++/target/native \ + -lhadooputils -lpthread \ + PROGRAM.cc \ + -o PROGRAM \ + -g -Wall -O2 + +Attention: The paths have to be adjusted, if you are not operating + in the Hama source folder. + +####################################################################