You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@giraph.apache.org by ik...@apache.org on 2015/06/11 04:50:40 UTC
[1/5] git commit: updated refs/heads/trunk to 819d6d3
Repository: giraph
Updated Branches:
refs/heads/trunk 06a1084af -> 819d6d38d
http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-core/src/main/java/org/apache/giraph/edge/LongDiffNullArrayEdges.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/edge/LongDiffNullArrayEdges.java b/giraph-core/src/main/java/org/apache/giraph/edge/LongDiffNullArrayEdges.java
new file mode 100644
index 0000000..e82c3a8
--- /dev/null
+++ b/giraph-core/src/main/java/org/apache/giraph/edge/LongDiffNullArrayEdges.java
@@ -0,0 +1,463 @@
+/*
+ * 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.giraph.edge;
+
+import it.unimi.dsi.fastutil.bytes.ByteArrays;
+import it.unimi.dsi.fastutil.longs.LongArrayList;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInput;
+import java.io.DataInputStream;
+import java.io.DataOutput;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.BitSet;
+import java.util.Iterator;
+
+import javax.annotation.concurrent.NotThreadSafe;
+
+import org.apache.giraph.utils.EdgeIterables;
+import org.apache.giraph.utils.Trimmable;
+import org.apache.giraph.utils.Varint;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.NullWritable;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * Implementation of {@link org.apache.giraph.edge.OutEdges} with long ids
+ * and null edge values, backed by a dynamic primitive array.
+ * Parallel edges are allowed.
+ * Note: this implementation is optimized for space usage,
+ * but random access and edge removals are expensive.
+ * Users of this class should explicitly call {@link #trim()} function
+ * to compact in-memory representation after all updates are done.
+ * Compacting object is expensive so should only be done once after bulk update.
+ * Compaction can also be caused by serialization attempt or
+ * by calling {@link #iterator()}
+ */
+@NotThreadSafe
+public class LongDiffNullArrayEdges
+ extends ConfigurableOutEdges<LongWritable, NullWritable>
+ implements ReuseObjectsOutEdges<LongWritable, NullWritable>,
+ MutableOutEdges<LongWritable, NullWritable>, Trimmable {
+
+ /**
+ * Array of target vertex ids.
+ */
+ private byte[] compressedData;
+ /**
+ * Number of edges stored in compressed array.
+ * There may be some extra edges in transientData or there may be some edges
+ * removed. These will not count here. To get real number of elements stored
+ * in this object @see {@link #size()}
+ */
+ private int size;
+
+ /**
+ * Last updates are stored here. We clear them out after object is compacted.
+ */
+ private TransientChanges transientData;
+
+ @Override
+ public void initialize(Iterable<Edge<LongWritable, NullWritable>> edges) {
+ reset();
+ EdgeIterables.initialize(this, edges);
+ trim();
+ }
+
+ @Override
+ public void initialize(int capacity) {
+ reset();
+ if (capacity > 0) {
+ transientData = new TransientChanges(capacity);
+ }
+ }
+
+ @Override
+ public void initialize() {
+ reset();
+ }
+
+ @Override
+ public void add(Edge<LongWritable, NullWritable> edge) {
+ checkTransientData();
+ transientData.add(edge.getTargetVertexId().get());
+ }
+
+
+ @Override
+ public void remove(LongWritable targetVertexId) {
+ checkTransientData();
+ long target = targetVertexId.get();
+
+ if (size > 0) {
+ LongsDiffReader reader = new LongsDiffReader(compressedData);
+ for (int i = 0; i < size; i++) {
+ long cur = reader.readNext();
+ if (cur == target) {
+ transientData.markRemoved(i);
+ } else if (cur > target) {
+ break;
+ }
+ }
+ }
+ transientData.removeAdded(target);
+ }
+
+ @Override
+ public int size() {
+ int result = size;
+ if (transientData != null) {
+ result += transientData.size();
+ }
+ return result;
+ }
+
+ @Override
+ public Iterator<Edge<LongWritable, NullWritable>> iterator() {
+ // Returns an iterator that reuses objects.
+ // The downcast is fine because all concrete Edge implementations are
+ // mutable, but we only expose the mutation functionality when appropriate.
+ return (Iterator) mutableIterator();
+ }
+
+ @Override
+ public Iterator<MutableEdge<LongWritable, NullWritable>> mutableIterator() {
+ trim();
+ return new Iterator<MutableEdge<LongWritable, NullWritable>>() {
+ /** Current position in the array. */
+ private int position;
+ private final LongsDiffReader reader =
+ new LongsDiffReader(compressedData);
+
+ /** Representative edge object. */
+ private final MutableEdge<LongWritable, NullWritable> representativeEdge =
+ EdgeFactory.createReusable(new LongWritable());
+
+ @Override
+ public boolean hasNext() {
+ return position < size;
+ }
+
+ @Override
+ public MutableEdge<LongWritable, NullWritable> next() {
+ position++;
+ representativeEdge.getTargetVertexId().set(reader.readNext());
+ return representativeEdge;
+ }
+
+ @Override
+ public void remove() {
+ removeAt(position - 1);
+ }
+ };
+ }
+
+ @Override
+ public void write(DataOutput out) throws IOException {
+ trim();
+ Varint.writeUnsignedVarInt(compressedData.length, out);
+ Varint.writeUnsignedVarInt(size, out);
+ out.write(compressedData);
+ }
+
+ @Override
+ public void readFields(DataInput in) throws IOException {
+ reset();
+ compressedData = new byte[Varint.readUnsignedVarInt(in)];
+ // We can actually calculate size after data array is read,
+ // the trade-off is memory vs speed
+ size = Varint.readUnsignedVarInt(in);
+ in.readFully(compressedData);
+ }
+
+ /**
+ * This function takes all recent updates and stores them efficiently.
+ * It is safe to call this function multiple times.
+ */
+ @Override
+ public void trim() {
+ if (transientData == null) {
+ // We don't have any updates to this object. Return quickly.
+ return;
+ }
+
+ // Beware this array is longer than the number of elements we interested in
+ long[] transientValues = transientData.sortedValues();
+ int pCompressed = 0;
+ int pTransient = 0;
+
+ LongsDiffReader reader = new LongsDiffReader(compressedData);
+ LongsDiffWriter writer = new LongsDiffWriter();
+
+ long curValue = size > 0 ? reader.readNext() : Long.MAX_VALUE;
+
+ // Here we merge freshly added elements and old elements, we also want
+ // to prune removed elements. Both arrays are sorted so in order to merge
+ // them, we move to pointers and store result in the new array
+ while (pTransient < transientData.numberOfAddedElements() ||
+ pCompressed < size) {
+ if (pTransient < transientData.numberOfAddedElements() &&
+ curValue >= transientValues[pTransient]) {
+ writer.writeNext(transientValues[pTransient]);
+ pTransient++;
+ } else {
+ if (!transientData.isRemoved(pCompressed)) {
+ writer.writeNext(curValue);
+ }
+ pCompressed++;
+ if (pCompressed < size) {
+ curValue = reader.readNext();
+ } else {
+ curValue = Long.MAX_VALUE;
+ }
+ }
+ }
+
+ compressedData = writer.toByteArray();
+ size += transientData.size();
+ transientData = null;
+ }
+
+
+ /**
+ * Remove edge at position i.
+ *
+ * @param i Position of edge to be removed
+ */
+ private void removeAt(int i) {
+ checkTransientData();
+ if (i < size) {
+ transientData.markRemoved(i);
+ } else {
+ transientData.removeAddedAt(i - size);
+ }
+ }
+
+ /**
+ * Check if transient data needs to be created.
+ */
+ private void checkTransientData() {
+ if (transientData == null) {
+ transientData = new TransientChanges();
+ }
+ }
+
+ /**
+ * Reset object to completely empty state.
+ */
+ private void reset() {
+ compressedData = ByteArrays.EMPTY_ARRAY;
+ size = 0;
+ transientData = null;
+ }
+
+ /**
+ * Reading array of longs diff encoded from byte array.
+ */
+ private static class LongsDiffReader {
+ /** Input stream */
+ private final DataInput input;
+ /** last read value */
+ private long current;
+ /** True if we haven't read any numbers yet */
+ private boolean first = true;
+
+ /**
+ * Construct LongsDiffReader
+ * @param compressedData Input byte array
+ */
+ LongsDiffReader(byte[] compressedData) {
+ input = new DataInputStream(new ByteArrayInputStream(compressedData));
+ }
+
+ /**
+ * Read next value from reader
+ * @return next value
+ */
+ long readNext() {
+ try {
+ if (first) {
+ current = input.readLong();
+ first = false;
+ } else {
+ current += Varint.readUnsignedVarLong(input);
+ }
+ return current;
+ } catch (IOException e) {
+ throw new IllegalStateException(e);
+ }
+ }
+ }
+
+ /**
+ * Writing array of longs diff encoded into the byte array.
+ */
+ private static class LongsDiffWriter {
+ /** Byte array stream containing result */
+ private final ByteArrayOutputStream resultStream =
+ new ByteArrayOutputStream();
+ /** Wrapping resultStream into DataOutputStream */
+ private final DataOutputStream out = new DataOutputStream(resultStream);
+ /** last value written */
+ private long lastWritten;
+ /** True if we haven't written any numbers yet */
+ private boolean first = true;
+
+ /**
+ * Write next value to writer
+ * @param value Value to be written
+ */
+ void writeNext(long value) {
+ try {
+ if (first) {
+ out.writeLong(value);
+ first = false;
+ } else {
+ Preconditions.checkState(value >= lastWritten,
+ "Values need to be in order");
+ Preconditions.checkState((value - lastWritten) >= 0,
+ "In order to use this class, difference of consecutive IDs " +
+ "cannot overflow longs");
+ Varint.writeUnsignedVarLong(value - lastWritten, out);
+ }
+ lastWritten = value;
+ } catch (IOException e) {
+ throw new IllegalStateException(e);
+ }
+ }
+
+ /**
+ * Get resulting byte array
+ * @return resulting byte array
+ */
+ byte[] toByteArray() {
+ return resultStream.toByteArray();
+ }
+ }
+
+ /**
+ * Temporary storage for all updates.
+ * We don't want to update compressed array frequently so we only update it
+ * on request at the same time we allow temporary updates to persist in this
+ * class.
+ */
+ private static class TransientChanges {
+ /** Neighbors that were added since last flush */
+ private final LongArrayList neighborsAdded;
+ /** Removed indices in original array */
+ private final BitSet removed = new BitSet();
+ /** Number of values removed */
+ private int removedCount;
+
+ /**
+ * Construct transient changes with given capacity
+ * @param capacity capacity
+ */
+ private TransientChanges(int capacity) {
+ neighborsAdded = new LongArrayList(capacity);
+ }
+
+ /**
+ * Construct transient changes
+ */
+ private TransientChanges() {
+ neighborsAdded = new LongArrayList();
+ }
+
+ /**
+ * Add new value
+ * @param value value to add
+ */
+ private void add(long value) {
+ neighborsAdded.add(value);
+ }
+
+ /**
+ * Mark given index to remove
+ * @param index Index to remove
+ */
+ private void markRemoved(int index) {
+ if (!removed.get(index)) {
+ removedCount++;
+ removed.set(index);
+ }
+ }
+
+ /**
+ * Remove value from neighborsAdded
+ * @param index Position to remove from
+ */
+ private void removeAddedAt(int index) {
+ // The order of the edges is irrelevant, so we can simply replace
+ // the deleted edge with the rightmost element, thus achieving constant
+ // time.
+ if (index == neighborsAdded.size() - 1) {
+ neighborsAdded.popLong();
+ } else {
+ neighborsAdded.set(index, neighborsAdded.popLong());
+ }
+ }
+
+ /**
+ * Number of added elements
+ * @return number of added elements
+ */
+ private int numberOfAddedElements() {
+ return neighborsAdded.size();
+ }
+
+ /**
+ * Remove added value
+ * @param target value to remove
+ */
+ private void removeAdded(long target) {
+ neighborsAdded.rem(target);
+ }
+
+ /**
+ * Additional size in transient changes
+ * @return additional size
+ */
+ private int size() {
+ return neighborsAdded.size() - removedCount;
+ }
+
+ /**
+ * Sorted added values
+ * @return sorted added values
+ */
+ private long[] sortedValues() {
+ long[] ret = neighborsAdded.elements();
+ Arrays.sort(ret, 0, neighborsAdded.size());
+ return ret;
+ }
+
+ /**
+ * Check if index was removed
+ * @param i Index to check
+ * @return Whether it was removed
+ */
+ private boolean isRemoved(int i) {
+ return removed.get(i);
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-core/src/main/java/org/apache/giraph/utils/ConfigurationObjectUtils.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/utils/ConfigurationObjectUtils.java b/giraph-core/src/main/java/org/apache/giraph/utils/ConfigurationObjectUtils.java
new file mode 100644
index 0000000..ae31bb2
--- /dev/null
+++ b/giraph-core/src/main/java/org/apache/giraph/utils/ConfigurationObjectUtils.java
@@ -0,0 +1,118 @@
+/*
+ * 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.giraph.utils;
+
+import org.apache.giraph.writable.kryo.KryoWritableWrapper;
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * Utility methods for dealing with Hadoop configuration
+ */
+public class ConfigurationObjectUtils {
+ /** Hide constructor */
+ private ConfigurationObjectUtils() {
+ }
+
+ /**
+ * Encode bytes to a hex String
+ *
+ * @param bytes byte[]
+ * @return encoded String
+ */
+ public static String encodeBytes(byte[] bytes) {
+ StringBuilder strBuf = new StringBuilder();
+ for (int i = 0; i < bytes.length; i++) {
+ strBuf.append((char) (((bytes[i] >> 4) & 0xF) + ('a')));
+ strBuf.append((char) (((bytes[i]) & 0xF) + ('a')));
+ }
+ return strBuf.toString();
+ }
+
+ /**
+ * Decode bytes from a hex String
+ *
+ * @param str String to decode
+ * @return decoded byte[]
+ */
+ public static byte[] decodeBytes(String str) {
+ byte[] bytes = new byte[str.length() / 2];
+ for (int i = 0; i < str.length(); i += 2) {
+ char c = str.charAt(i);
+ bytes[i / 2] = (byte) ((c - 'a') << 4);
+ c = str.charAt(i + 1);
+ bytes[i / 2] += c - 'a';
+ }
+ return bytes;
+ }
+
+ /**
+ * Set byte array to a conf option
+ *
+ * @param data Byte array
+ * @param confOption Conf option
+ * @param conf Configuration
+ */
+ public static void setByteArray(byte[] data, String confOption,
+ Configuration conf) {
+ conf.set(confOption, encodeBytes(data));
+ }
+
+ /**
+ * Get byte array from a conf option
+ *
+ * @param confOption Conf option
+ * @param conf Configuration
+ * @return Byte array
+ */
+ public static byte[] getByteArray(String confOption,
+ Configuration conf) {
+ return decodeBytes(conf.get(confOption));
+ }
+
+ /**
+ * Set object in a conf option using kryo
+ *
+ * @param object Object to set
+ * @param confOption Conf option
+ * @param conf Configuration
+ * @param <T> Type of the object
+ */
+ public static <T> void setObjectKryo(T object, String confOption,
+ Configuration conf) {
+ setByteArray(WritableUtils.toByteArrayUnsafe(
+ new KryoWritableWrapper<>(object)),
+ confOption, conf);
+ }
+
+ /**
+ * Get object from a conf option using kryo
+ *
+ * @param confOption Conf option
+ * @param conf Configuration
+ * @return Object from conf
+ * @param <T> Type of the object
+ */
+ public static <T> T getObjectKryo(String confOption,
+ Configuration conf) {
+ KryoWritableWrapper<T> wrapper = new KryoWritableWrapper<>();
+ WritableUtils.fromByteArrayUnsafe(
+ getByteArray(confOption, conf), wrapper,
+ new UnsafeReusableByteArrayInput());
+ return wrapper.get();
+ }
+}
http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-core/src/main/java/org/apache/giraph/utils/DefaultOutputCommitter.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/utils/DefaultOutputCommitter.java b/giraph-core/src/main/java/org/apache/giraph/utils/DefaultOutputCommitter.java
new file mode 100644
index 0000000..ef273b0
--- /dev/null
+++ b/giraph-core/src/main/java/org/apache/giraph/utils/DefaultOutputCommitter.java
@@ -0,0 +1,72 @@
+/*
+ * 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.giraph.utils;
+
+import java.io.IOException;
+
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.OutputCommitter;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+
+/**
+ * Output committer which has abstract commit method
+ */
+public abstract class DefaultOutputCommitter extends OutputCommitter {
+ /**
+ * For cleaning up the job's output after job completion. Note that this
+ * is invoked for jobs with final run state as
+ * {@link org.apache.hadoop.mapreduce.JobStatus.State#SUCCEEDED}
+ *
+ * @param jobContext Context of the job whose output is being written.
+ */
+ public abstract void commit(JobContext jobContext) throws IOException;
+
+ @Override
+ public final void setupJob(JobContext jobContext) throws IOException {
+ }
+
+ @Override
+ public final void setupTask(TaskAttemptContext taskContext)
+ throws IOException {
+ }
+
+ @Override
+ public final void commitJob(JobContext jobContext)
+ throws IOException {
+ super.commitJob(jobContext);
+ commit(jobContext);
+ }
+
+ @Override
+ public final boolean needsTaskCommit(TaskAttemptContext taskContext)
+ throws IOException {
+ // Digraph does not require a task commit and there is a bug in Corona
+ // which triggers t5688706
+ // Avoiding the task commit should work around this.
+ return false;
+ }
+
+ @Override
+ public final void commitTask(TaskAttemptContext context) throws IOException {
+ }
+
+ @Override
+ public final void abortTask(TaskAttemptContext taskContext)
+ throws IOException {
+ }
+}
http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-core/src/main/java/org/apache/giraph/utils/WritableUtils.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/utils/WritableUtils.java b/giraph-core/src/main/java/org/apache/giraph/utils/WritableUtils.java
index 68ed89a..2229c2f 100644
--- a/giraph-core/src/main/java/org/apache/giraph/utils/WritableUtils.java
+++ b/giraph-core/src/main/java/org/apache/giraph/utils/WritableUtils.java
@@ -34,6 +34,7 @@ import java.util.List;
import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
import org.apache.giraph.edge.Edge;
import org.apache.giraph.edge.OutEdges;
+import org.apache.giraph.factories.ValueFactory;
import org.apache.giraph.graph.Vertex;
import org.apache.giraph.zk.ZooKeeperExt;
import org.apache.giraph.zk.ZooKeeperExt.PathStat;
@@ -950,4 +951,190 @@ public class WritableUtils {
}
return copy;
}
+
+ /**
+ * Create a copy of Writable object, by serializing and deserializing it.
+ *
+ * @param original Original value of which to make a copy
+ * @return Copy of the original value
+ * @param <T> Type of the object
+ */
+ public static final <T extends Writable> T createCopy(T original) {
+ return (T) createCopy(original, original.getClass(), null);
+ }
+
+ /**
+ * Create a copy of Writable object, by serializing and deserializing it.
+ *
+ * @param original Original value of which to make a copy
+ * @param outputClass Expected copy class, needs to match original
+ * @param conf Configuration
+ * @return Copy of the original value
+ * @param <T> Type of the object
+ */
+ public static final <T extends Writable>
+ T createCopy(T original, Class<? extends T> outputClass,
+ ImmutableClassesGiraphConfiguration conf) {
+ T result = WritableUtils.createWritable(outputClass, conf);
+ copyInto(original, result);
+ return result;
+ }
+
+ /**
+ * Create a copy of Writable object, by serializing and deserializing it.
+ *
+ * @param original Original value of which to make a copy
+ * @param classFactory Factory to create new empty object from
+ * @param conf Configuration
+ * @return Copy of the original value
+ * @param <T> Type of the object
+ */
+ public static final <T extends Writable>
+ T createCopy(T original, ValueFactory<T> classFactory,
+ ImmutableClassesGiraphConfiguration conf) {
+ T result = classFactory.newInstance();
+ copyInto(original, result);
+ return result;
+ }
+
+ /**
+ * Serialize given writable object, and return it's size.
+ *
+ * @param w Writable object
+ * @return it's size after serialization
+ */
+ public static int size(Writable w) {
+ try {
+ ExtendedByteArrayDataOutput out = new ExtendedByteArrayDataOutput();
+ w.write(out);
+ return out.getPos();
+ } catch (IOException e) {
+ throw new RuntimeException(e);
+ }
+ }
+
+ /**
+ * Serialize given writable to byte array,
+ * using new instance of ExtendedByteArrayDataOutput.
+ *
+ * @param w Writable object
+ * @return array of bytes
+ * @param <T> Type of the object
+ */
+ public static <T extends Writable> byte[] toByteArray(T w) {
+ try {
+ ExtendedByteArrayDataOutput out = new ExtendedByteArrayDataOutput();
+ w.write(out);
+ return out.toByteArray();
+ } catch (IOException e) {
+ throw new RuntimeException(e);
+ }
+ }
+
+ /**
+ * Deserialize from given byte array into given writable,
+ * using new instance of ExtendedByteArrayDataInput.
+ *
+ * @param data Byte array representing writable
+ * @param to Object to fill
+ * @param <T> Type of the object
+ */
+ public static <T extends Writable> void fromByteArray(byte[] data, T to) {
+ try {
+ ExtendedByteArrayDataInput in =
+ new ExtendedByteArrayDataInput(data, 0, data.length);
+ to.readFields(in);
+
+ if (in.available() != 0) {
+ throw new RuntimeException(
+ "Serialization encountered issues, " + in.available() +
+ " bytes left to be read");
+ }
+ } catch (IOException e) {
+ throw new RuntimeException(e);
+ }
+ }
+
+ /**
+ * Serialize given writable to byte array,
+ * using new instance of UnsafeByteArrayOutputStream.
+ *
+ * @param w Writable object
+ * @return array of bytes
+ * @param <T> Type of the object
+ */
+ public static <T extends Writable> byte[] toByteArrayUnsafe(T w) {
+ try {
+ UnsafeByteArrayOutputStream out = new UnsafeByteArrayOutputStream();
+ w.write(out);
+ return out.toByteArray();
+ } catch (IOException e) {
+ throw new RuntimeException(e);
+ }
+ }
+
+ /**
+ * Deserialize from given byte array into given writable,
+ * using given reusable UnsafeReusableByteArrayInput.
+ *
+ * @param data Byte array representing writable
+ * @param to Object to fill
+ * @param reusableInput Reusable input to use
+ * @param <T> Type of the object
+ */
+ public static <T extends Writable> void fromByteArrayUnsafe(
+ byte[] data, T to, UnsafeReusableByteArrayInput reusableInput) {
+ try {
+ reusableInput.initialize(data, 0, data.length);
+ to.readFields(reusableInput);
+
+ if (reusableInput.available() != 0) {
+ throw new RuntimeException(
+ "Serialization encountered issues, " + reusableInput.available() +
+ " bytes left to be read");
+ }
+ } catch (IOException e) {
+ throw new RuntimeException(e);
+ }
+ }
+
+ /**
+ * First write a boolean saying whether an object is not null,
+ * and if it's not write the object
+ *
+ * @param object Object to write
+ * @param out DataOutput to write to
+ * @param <T> Object type
+ */
+ public static <T extends Writable> void writeIfNotNullAndObject(T object,
+ DataOutput out) throws IOException {
+ out.writeBoolean(object != null);
+ if (object != null) {
+ object.write(out);
+ }
+ }
+
+ /**
+ * First read a boolean saying whether an object is not null,
+ * and if it's not read the object
+ *
+ * @param reusableObject Reuse this object instance
+ * @param objectClass Class of the object, to create if reusableObject is null
+ * @param in DataInput to read from
+ * @param <T> Object type
+ * @return Object, or null
+ */
+ public static <T extends Writable> T readIfNotNullAndObject(T reusableObject,
+ Class<T> objectClass, DataInput in) throws IOException {
+ if (in.readBoolean()) {
+ if (reusableObject == null) {
+ reusableObject = ReflectionUtils.newInstance(objectClass);
+ }
+ reusableObject.readFields(in);
+ return reusableObject;
+ } else {
+ return null;
+ }
+ }
+
}
http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-core/src/test/java/org/apache/giraph/edge/LongDiffNullArrayEdgesTest.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/test/java/org/apache/giraph/edge/LongDiffNullArrayEdgesTest.java b/giraph-core/src/test/java/org/apache/giraph/edge/LongDiffNullArrayEdgesTest.java
new file mode 100644
index 0000000..e23f592
--- /dev/null
+++ b/giraph-core/src/test/java/org/apache/giraph/edge/LongDiffNullArrayEdgesTest.java
@@ -0,0 +1,260 @@
+/*
+ * 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.giraph.edge;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.io.ByteArrayOutputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.giraph.conf.GiraphConfiguration;
+import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
+import org.apache.giraph.utils.UnsafeByteArrayInputStream;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.Writable;
+import org.junit.Assert;
+import org.junit.Test;
+
+import com.google.common.collect.Lists;
+
+
+public class LongDiffNullArrayEdgesTest {
+ private static Edge<LongWritable, NullWritable> createEdge(long id) {
+ return EdgeFactory.create(new LongWritable(id));
+ }
+
+ private static void assertEdges(LongDiffNullArrayEdges edges, long... expected) {
+ int index = 0;
+ for (Edge<LongWritable, NullWritable> edge : edges) {
+ Assert.assertEquals(expected[index], edge.getTargetVertexId().get());
+ index++;
+ }
+ Assert.assertEquals(expected.length, index);
+ }
+
+ @Test
+ public void testEdges() {
+ LongDiffNullArrayEdges edges = getEdges();
+
+ List<Edge<LongWritable, NullWritable>> initialEdges = Lists.newArrayList(
+ createEdge(1), createEdge(2), createEdge(4));
+
+ edges.initialize(initialEdges);
+ assertEdges(edges, 1, 2, 4);
+
+ edges.add(EdgeFactory.createReusable(new LongWritable(3)));
+ assertEdges(edges, 1, 2, 3, 4);
+
+ edges.remove(new LongWritable(2));
+ assertEdges(edges, 1, 3, 4);
+ }
+
+ @Test
+ public void testPositiveAndNegativeEdges() {
+ LongDiffNullArrayEdges edges = getEdges();
+
+ List<Edge<LongWritable, NullWritable>> initialEdges = Lists.newArrayList(
+ createEdge(1), createEdge(-2), createEdge(3), createEdge(-4));
+
+ edges.initialize(initialEdges);
+ assertEdges(edges, -4, -2, 1, 3);
+
+ edges.add(EdgeFactory.createReusable(new LongWritable(5)));
+ assertEdges(edges, -4, -2, 1, 3, 5);
+
+ edges.remove(new LongWritable(-2));
+ assertEdges(edges, -4, 1, 3, 5);
+ }
+
+ @Test
+ public void testMutateEdges() {
+ LongDiffNullArrayEdges edges = getEdges();
+
+ edges.initialize();
+
+ // Add 10 edges with id i, for i = 0..9
+ for (int i = 0; i < 10; ++i) {
+ edges.add(createEdge(i));
+ }
+
+ // Use the mutable iterator to remove edges with even id
+ Iterator<MutableEdge<LongWritable, NullWritable>> edgeIt =
+ edges.mutableIterator();
+ while (edgeIt.hasNext()) {
+ if (edgeIt.next().getTargetVertexId().get() % 2 == 0) {
+ edgeIt.remove();
+ }
+ }
+
+ // We should now have 5 edges
+ assertEquals(5, edges.size());
+ // The edge ids should be all odd
+ for (Edge<LongWritable, NullWritable> edge : edges) {
+ assertEquals(1, edge.getTargetVertexId().get() % 2);
+ }
+ }
+
+ @Test
+ public void testSerialization() throws IOException {
+ LongDiffNullArrayEdges edges = getEdges();
+
+ edges.initialize();
+
+ // Add 10 edges with id i, for i = 0..9
+ for (int i = 0; i < 10; ++i) {
+ edges.add(createEdge(i));
+ }
+
+ edges.trim();
+
+ // Use the mutable iterator to remove edges with even id
+ Iterator<MutableEdge<LongWritable, NullWritable>> edgeIt =
+ edges.mutableIterator();
+ while (edgeIt.hasNext()) {
+ if (edgeIt.next().getTargetVertexId().get() % 2 == 0) {
+ edgeIt.remove();
+ }
+ }
+
+ // We should now have 5 edges
+ assertEdges(edges, 1, 3, 5, 7, 9);
+
+ ByteArrayOutputStream arrayStream = new ByteArrayOutputStream();
+ DataOutputStream tempBuffer = new DataOutputStream(arrayStream);
+
+ edges.write(tempBuffer);
+
+ byte[] binary = arrayStream.toByteArray();
+
+ assertTrue("Serialized version should not be empty ", binary.length > 0);
+
+ edges = getEdges();
+ edges.readFields(new UnsafeByteArrayInputStream(binary));
+
+ assertEquals(5, edges.size());
+
+ for (Edge<LongWritable, NullWritable> edge : edges) {
+ assertEquals(1, edge.getTargetVertexId().get() % 2);
+ }
+ }
+
+ @Test
+ public void testParallelEdges() {
+ LongDiffNullArrayEdges edges = getEdges();
+
+ List<Edge<LongWritable, NullWritable>> initialEdges = Lists.newArrayList(
+ createEdge(2), createEdge(2), createEdge(2));
+
+ edges.initialize(initialEdges);
+ assertEquals(3, edges.size());
+
+ edges.remove(new LongWritable(2));
+ assertEquals(0, edges.size());
+
+ edges.add(EdgeFactory.create(new LongWritable(2)));
+ assertEquals(1, edges.size());
+
+ edges.trim();
+ assertEquals(1, edges.size());
+ }
+
+ @Test
+ public void testEdgeValues() {
+ LongDiffNullArrayEdges edges = getEdges();
+ Set<Long> testValues = new HashSet<Long>();
+ testValues.add(0L);
+ testValues.add((long) Integer.MAX_VALUE);
+ testValues.add(Long.MAX_VALUE);
+
+ // shouldn't be working with negative IDs
+ // testValues.add((long) Integer.MIN_VALUE);
+ // testValues.add(Long.MIN_VALUE);
+
+ List<Edge<LongWritable, NullWritable>> initialEdges =
+ new ArrayList<Edge<LongWritable, NullWritable>>();
+ for(Long id : testValues) {
+ initialEdges.add(createEdge(id));
+ }
+
+ edges.initialize(initialEdges);
+ edges.trim();
+
+ Iterator<MutableEdge<LongWritable, NullWritable>> edgeIt =
+ edges.mutableIterator();
+ while (edgeIt.hasNext()) {
+ long value = edgeIt.next().getTargetVertexId().get();
+ assertTrue("Unknown edge found " + value, testValues.remove(value));
+ }
+ }
+
+ private LongDiffNullArrayEdges getEdges() {
+ GiraphConfiguration gc = new GiraphConfiguration();
+ ImmutableClassesGiraphConfiguration<LongWritable, Writable, NullWritable> conf =
+ new ImmutableClassesGiraphConfiguration<LongWritable, Writable, NullWritable>(gc);
+ LongDiffNullArrayEdges ret = new LongDiffNullArrayEdges();
+ ret.setConf(new ImmutableClassesGiraphConfiguration<LongWritable, Writable, NullWritable>(conf));
+ return ret;
+ }
+
+ @Test
+ public void testAddedSmalerValues() {
+ LongDiffNullArrayEdges edges = getEdges();
+
+ List<Edge<LongWritable, NullWritable>> initialEdges = Lists.newArrayList(
+ createEdge(100));
+
+ edges.initialize(initialEdges);
+
+ edges.trim();
+
+ for (int i=0; i<16; i++) {
+ edges.add(createEdge(i));
+ }
+
+ edges.trim();
+
+ assertEquals(17, edges.size());
+ }
+
+ @Test(expected=IllegalStateException.class)
+ public void testFailSafeOnPotentialOverflow() {
+ LongDiffNullArrayEdges edges = getEdges();
+
+ List<Edge<LongWritable, NullWritable>> initialEdges = Lists.newArrayList(
+ createEdge(5223372036854775807L), createEdge(-4223372036854775807L));
+ edges.initialize(initialEdges);
+ }
+
+ @Test
+ public void testAvoidOverflowWithZero() {
+ LongDiffNullArrayEdges edges = getEdges();
+
+ List<Edge<LongWritable, NullWritable>> initialEdges = Lists.newArrayList(
+ createEdge(5223372036854775807L), createEdge(-4223372036854775807L), createEdge(0));
+ edges.initialize(initialEdges);
+ assertEdges(edges, -4223372036854775807L, 0, 5223372036854775807L);
+ }
+}
http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 63a9bae..4198f13 100644
--- a/pom.xml
+++ b/pom.xml
@@ -275,6 +275,8 @@ under the License.
<!-- This lets modules skip unit tests. More details: GIRAPH-957 -->
<surefire.skip>false</surefire.skip>
+ <checkstyle.config.path>${top.dir}/checkstyle.xml</checkstyle.config.path>
+
<dep.avro.version>1.7.6</dep.avro.version>
<dep.accumulo.version>1.4.0</dep.accumulo.version>
<dep.asm.version>3.2</dep.asm.version>
@@ -541,7 +543,7 @@ under the License.
<artifactId>maven-checkstyle-plugin</artifactId>
<version>2.15</version>
<configuration>
- <configLocation>${top.dir}/checkstyle.xml</configLocation>
+ <configLocation>${checkstyle.config.path}</configLocation>
<consoleOutput>true</consoleOutput>
<enableRulesSummary>false</enableRulesSummary>
<headerLocation>${top.dir}/license-header.txt</headerLocation>
@@ -837,7 +839,7 @@ under the License.
<artifactId>maven-checkstyle-plugin</artifactId>
<version>2.10</version>
<configuration>
- <configLocation>${top.dir}/checkstyle.xml</configLocation>
+ <configLocation>${checkstyle.config.path}</configLocation>
<consoleOutput>true</consoleOutput>
<enableRulesSummary>false</enableRulesSummary>
<headerLocation>${top.dir}/license-header.txt</headerLocation>
@@ -1713,6 +1715,11 @@ under the License.
</dependency>
<dependency>
<groupId>org.apache.giraph</groupId>
+ <artifactId>giraph-block-app</artifactId>
+ <version>${project.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.giraph</groupId>
<artifactId>giraph-examples</artifactId>
<version>${project.version}</version>
</dependency>
@@ -2276,6 +2283,7 @@ under the License.
<modules>
<module>giraph-core</module>
+ <module>giraph-block-app</module>
<module>giraph-examples</module>
</modules>
[5/5] git commit: updated refs/heads/trunk to 819d6d3
Posted by ik...@apache.org.
[GIRAPH-1013] Add core of Blocks Framework
Summary:
Add all classes in the core of the Framework.
This is the full execution engine of the framework.
New module giraph-block-app is created for it, and all framework
classes are going into org.apache.giraph.block_app.framework,
and all non-framework-internal classes are going to go into
subpackages of org.apache.giraph.block_app (i.e. utilities for
writing applications, etc)
Test Plan: mvn clean install
Reviewers: maja.kabiljo, dionysis.logothetis, sergey.edunov
Reviewed By: sergey.edunov
Differential Revision: https://reviews.facebook.net/D39639
Project: http://git-wip-us.apache.org/repos/asf/giraph/repo
Commit: http://git-wip-us.apache.org/repos/asf/giraph/commit/819d6d38
Tree: http://git-wip-us.apache.org/repos/asf/giraph/tree/819d6d38
Diff: http://git-wip-us.apache.org/repos/asf/giraph/diff/819d6d38
Branch: refs/heads/trunk
Commit: 819d6d38d6cb7073786f4b7ed11763fff5200ded
Parents: 06a1084
Author: Igor Kabiljo <ik...@fb.com>
Authored: Fri Jun 5 00:19:41 2015 -0700
Committer: Igor Kabiljo <ik...@fb.com>
Committed: Wed Jun 10 19:50:06 2015 -0700
----------------------------------------------------------------------
checkstyle-relaxed.xml | 272 +++++++++++
giraph-block-app/pom.xml | 124 +++++
giraph-block-app/src/main/assembly/compile.xml | 39 ++
.../framework/AbstractBlockFactory.java | 203 ++++++++
.../block_app/framework/BlockFactory.java | 83 ++++
.../giraph/block_app/framework/BlockUtils.java | 206 +++++++++
.../block_app/framework/BulkConfigurator.java | 32 ++
.../block_app/framework/api/BlockApi.java | 47 ++
.../block_app/framework/api/BlockConfApi.java | 32 ++
.../block_app/framework/api/BlockMasterApi.java | 58 +++
.../block_app/framework/api/BlockOutputApi.java | 31 ++
.../api/BlockOutputHandleAccessor.java | 33 ++
.../block_app/framework/api/BlockWorkerApi.java | 35 ++
.../framework/api/BlockWorkerContextApi.java | 45 ++
.../api/BlockWorkerContextReceiveApi.java | 29 ++
.../api/BlockWorkerContextSendApi.java | 39 ++
.../framework/api/BlockWorkerReceiveApi.java | 33 ++
.../framework/api/BlockWorkerSendApi.java | 116 +++++
.../framework/api/BlockWorkerValueAccessor.java | 36 ++
.../giraph/block_app/framework/api/Counter.java | 33 ++
.../framework/api/CreateReducersApi.java | 83 ++++
.../block_app/framework/api/StatusReporter.java | 31 ++
.../framework/api/giraph/BlockComputation.java | 54 +++
.../api/giraph/BlockMasterApiWrapper.java | 170 +++++++
.../api/giraph/BlockMasterCompute.java | 72 +++
.../api/giraph/BlockWorkerApiWrapper.java | 180 +++++++
.../api/giraph/BlockWorkerContext.java | 102 ++++
.../giraph/BlockWorkerContextApiWrapper.java | 84 ++++
.../framework/api/giraph/package-info.java | 22 +
.../block_app/framework/api/package-info.java | 36 ++
.../giraph/block_app/framework/block/Block.java | 59 +++
.../block_app/framework/block/EmptyBlock.java | 39 ++
.../framework/block/FilteringBlock.java | 113 +++++
.../block_app/framework/block/IfBlock.java | 70 +++
.../block_app/framework/block/RepeatBlock.java | 87 ++++
.../framework/block/RepeatUntilBlock.java | 83 ++++
.../framework/block/SequenceBlock.java | 60 +++
.../block_app/framework/block/package-info.java | 22 +
.../framework/internal/BlockCounters.java | 79 ++++
.../framework/internal/BlockMasterLogic.java | 173 +++++++
.../internal/BlockWorkerContextLogic.java | 91 ++++
.../framework/internal/BlockWorkerLogic.java | 68 +++
.../framework/internal/BlockWorkerPieces.java | 180 +++++++
.../framework/internal/PairedPieceAndStage.java | 111 +++++
.../framework/internal/package-info.java | 22 +
.../framework/output/BlockOutputDesc.java | 50 ++
.../framework/output/BlockOutputFormat.java | 107 +++++
.../framework/output/BlockOutputHandle.java | 119 +++++
.../framework/output/BlockOutputOption.java | 52 +++
.../framework/output/BlockOutputWriter.java | 26 ++
.../framework/output/package-info.java | 21 +
.../block_app/framework/package-info.java | 25 +
.../framework/piece/AbstractPiece.java | 287 ++++++++++++
.../framework/piece/DefaultParentPiece.java | 311 +++++++++++++
.../giraph/block_app/framework/piece/Piece.java | 59 +++
.../framework/piece/PieceWithWorkerContext.java | 54 +++
.../framework/piece/delegate/DelegatePiece.java | 277 +++++++++++
.../piece/delegate/FilteringPiece.java | 157 +++++++
.../framework/piece/delegate/package-info.java | 21 +
.../piece/global_comm/BroadcastHandle.java | 30 ++
.../piece/global_comm/ReduceUtilsObject.java | 62 +++
.../ReducerAndBroadcastWrapperHandle.java | 61 +++
.../piece/global_comm/ReducerHandle.java | 41 ++
.../piece/global_comm/array/ArrayHandle.java | 36 ++
.../global_comm/array/BroadcastArrayHandle.java | 35 ++
.../global_comm/array/ReducerArrayHandle.java | 43 ++
.../piece/global_comm/array/package-info.java | 21 +
.../internal/CreateReducersApiWrapper.java | 73 +++
.../internal/ReducersForPieceHandler.java | 250 ++++++++++
.../internal/VertexSenderObserver.java | 28 ++
.../global_comm/internal/package-info.java | 22 +
.../global_comm/map/BroadcastMapHandle.java | 36 ++
.../piece/global_comm/map/MapHandle.java | 31 ++
.../piece/global_comm/map/ReducerMapHandle.java | 43 ++
.../piece/global_comm/map/package-info.java | 21 +
.../piece/global_comm/package-info.java | 22 +
.../piece/interfaces/VertexPostprocessor.java | 34 ++
.../piece/interfaces/VertexReceiver.java | 54 +++
.../piece/interfaces/VertexSender.java | 45 ++
.../piece/interfaces/package-info.java | 21 +
.../piece/messages/ObjectMessageClasses.java | 119 +++++
.../piece/messages/SupplierFromConf.java | 71 +++
.../framework/piece/messages/package-info.java | 21 +
.../block_app/framework/piece/package-info.java | 24 +
.../org/apache/giraph/function/Consumer.java | 34 ++
.../org/apache/giraph/function/Function.java | 38 ++
.../apache/giraph/function/PairConsumer.java | 34 ++
.../apache/giraph/function/PairFunction.java | 40 ++
.../org/apache/giraph/function/Supplier.java | 37 ++
.../apache/giraph/function/package-info.java | 26 ++
.../giraph/function/primitive/IntSupplier.java | 27 ++
.../giraph/function/primitive/package-info.java | 22 +
.../function/vertex/ConsumerWithVertex.java | 48 ++
.../function/vertex/FunctionWithVertex.java | 52 +++
.../function/vertex/SupplierFromVertex.java | 49 ++
.../giraph/function/vertex/package-info.java | 22 +
.../framework/block/BlockTestingUtils.java | 158 +++++++
.../block_app/framework/block/TestIfBlock.java | 88 ++++
.../framework/block/TestRepeatBlock.java | 80 ++++
.../framework/block/TestRepeatUntilBlock.java | 103 +++++
.../giraph/edge/LongDiffNullArrayEdges.java | 463 +++++++++++++++++++
.../giraph/utils/ConfigurationObjectUtils.java | 118 +++++
.../giraph/utils/DefaultOutputCommitter.java | 72 +++
.../org/apache/giraph/utils/WritableUtils.java | 187 ++++++++
.../giraph/edge/LongDiffNullArrayEdgesTest.java | 260 +++++++++++
pom.xml | 12 +-
106 files changed, 8395 insertions(+), 2 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/checkstyle-relaxed.xml
----------------------------------------------------------------------
diff --git a/checkstyle-relaxed.xml b/checkstyle-relaxed.xml
new file mode 100644
index 0000000..11bcfe0
--- /dev/null
+++ b/checkstyle-relaxed.xml
@@ -0,0 +1,272 @@
+<?xml version="1.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.
+-->
+
+<!--
+ This version of checkstyle is based on the Hadoop and common-math
+ checkstyle configurations. It is a best effort attempt to try to match
+ the CODE_CONVENTIONS and Oracle "Code Conventions for the Java
+ Programming Language". See the following link:
+
+ http://www.oracle.com/technetwork/java/codeconvtoc-136057.html
+
+ The documentation for checkstyle is available at
+
+ http://checkstyle.sourceforge.net
+-->
+
+<!DOCTYPE module PUBLIC "-//Puppy Crawl//DTD Check Configuration 1.1//EN" "http://www.puppycrawl.com/dtds/configuration_1_1.dtd">
+
+<!-- Apache giraph customization of default Checkstyle behavior -->
+<module name="Checker">
+ <property name="localeLanguage" value="en"/>
+
+ <!-- Checks for headers -->
+ <!-- See http://checkstyle.sf.net/config_header.html -->
+ <!-- Verify that EVERY source file has the appropriate license -->
+ <module name="Header">
+ <property name="headerFile" value="${checkstyle.header.file}"/>
+ <property name="fileExtensions" value="java"/>
+ </module>
+
+ <!-- Checks for Javadoc comments (checker). -->
+ <!-- See http://checkstyle.sf.net/config_javadoc.html -->
+ <!-- Require package javadoc -->
+ <module name="JavadocPackage"/>
+
+ <!-- Miscellaneous other checks (checker). -->
+ <!-- See http://checkstyle.sf.net/config_misc.html -->
+ <!-- Require files to end with newline characters -->
+ <module name="NewlineAtEndOfFile"/>
+
+ <!-- Checks for whitespace (tree walker) -->
+ <!-- See http://checkstyle.sf.net/config_whitespace.html -->
+ <!-- No tabs allowed! -->
+ <module name="FileTabCharacter"/>
+
+ <module name="TreeWalker">
+ <property name="cacheFile" value="target/checkstyle-cachefile"/>
+
+ <!-- Checks for blocks. You know, those {}'s -->
+ <!-- See http://checkstyle.sf.net/config_blocks.html -->
+ <!-- No empty blocks (i.e. catch) -->
+ <module name="EmptyBlock"/>
+ <module name="AvoidNestedBlocks"/>
+ <!-- No if/else/do/for/while without braces -->
+ <module name="NeedBraces"/>
+ <module name="LeftCurly"/>
+ <module name="RightCurly"/>
+
+ <!-- Checks for class design -->
+ <!-- See http://checkstyle.sf.net/config_design.html -->
+ <!-- Utility class should not be instantiated, they must have a
+ private constructor -->
+ <module name="HideUtilityClassConstructor"/>
+ <!-- Interfaces must be types (not just constants) -->
+ <module name="InterfaceIsType"/>
+ <!-- No public fields -->
+ <module name="VisibilityModifier">
+ <property name="protectedAllowed" value="true"/>
+ <property name="publicMemberPattern" value="^$"/>
+ </module>
+
+ <!-- Checks for common coding problems -->
+ <!-- See http://checkstyle.sf.net/config_coding.html -->
+ <module name="EmptyStatement"/>
+ <!-- Require hash code override when equals is -->
+ <module name="EqualsHashCode"/>
+ <!-- Method parameters and local variables should not hide
+ fields, except in constructors and setters -->
+ <module name="HiddenField">
+ <property name="ignoreConstructorParameter" value="true" />
+ <property name="ignoreSetter" value="true" />
+ <property name="tokens" value="VARIABLE_DEF"/>
+ </module>
+ <!-- Disallow unnecessary instantiation of Boolean, String -->
+ <module name="IllegalInstantiation">
+ <property name="classes" value="java.lang.Boolean, java.lang.String"/>
+ </module>
+ <module name="InnerAssignment"/>
+ <!-- Switch statements should be complete and with independent cases -->
+ <module name="FallThrough" />
+ <module name="MissingSwitchDefault" />
+ <module name="SimplifyBooleanExpression"/>
+ <module name="SimplifyBooleanReturn"/>
+ <!-- Only one statment per line allowed -->
+ <module name="OneStatementPerLine"/>
+ <!-- Use a consistent way to put declarations -->
+ <module name="DeclarationOrder" />
+ <!-- Don't add up parentheses when they are not required -->
+ <module name="UnnecessaryParentheses" />
+ <!-- Don't use too widespread catch (Exception, Throwable,
+ RuntimeException) -->
+ <module name="IllegalCatch" />
+ <!-- Don't use = or != for string comparisons -->
+ <module name="StringLiteralEquality" />
+ <!-- Don't declare multiple variables in the same statement -->
+ <module name="MultipleVariableDeclarations" />
+ <!-- String literals more than one character long should not be
+ repeated several times -->
+ <!-- the "unchecked" string is also accepted to allow
+ @SuppressWarnings("unchecked") -->
+ <!-- Disabling for now until we have a better ignoreStringsRegexp -->
+ <!--
+ <module name="MultipleStringLiterals" >
+ <property name="ignoreStringsRegexp" value='^(("")|(".")|("unchecked"))$'/>
+ </module>
+ -->
+
+ <!-- Checks for imports -->
+ <!-- See http://checkstyle.sf.net/config_import.html -->
+ <module name="RedundantImport"/>
+ <!-- Import should be explicit, really needed and only from pure
+ java packages -->
+ <module name="AvoidStarImport" />
+ <module name="UnusedImports" />
+ <module name="IllegalImport" />
+
+ <!-- Checks for Javadoc comments (tree walker). -->
+ <!-- See http://checkstyle.sf.net/config_javadoc.html -->
+ <!-- Javadoc must be formatted correctly -->
+ <module name="JavadocStyle">
+ <property name="checkFirstSentence" value="false"/>
+ </module>
+ <!-- Must have class / interface header comments -->
+ <module name="JavadocType"/>
+ <!-- Require method javadocs, allow undeclared RTE, allow missing
+ javadoc on getters and setters -->
+ <module name="JavadocMethod">
+ <property name="allowMissingJavadoc" value="true"/>
+ <property name="allowUndeclaredRTE" value="true"/>
+ <property name="allowMissingThrowsTags" value="true"/>
+ <property name="allowMissingPropertyJavadoc" value="true"/>
+ <property name="allowMissingParamTags" value="true"/>
+ <property name="allowMissingReturnTag" value="true"/>
+ </module>
+
+ <!-- Miscellaneous other checks (tree walker). -->
+ <!-- See http://checkstyle.sf.net/config_misc.html -->
+ <!-- Java style arrays -->
+ <module name="ArrayTypeStyle"/>
+ <!-- Indentation -->
+ <module name="Indentation">
+ <property name="basicOffset" value="2"/>
+ <property name="braceAdjustment" value="0"/>
+ <property name="caseIndent" value="0"/>
+ <property name="throwsIndent" value="2"/>
+ <property name="lineWrappingIndentation" value="0"/>
+ <property name="arrayInitIndent" value="2"/>
+ </module>
+ <!-- Turn this on to see what needs to be done
+ <module name="TodoComment"/>
+ -->
+ <module name="UpperEll"/>
+
+ <!-- Modifier Checks -->
+ <!-- See http://checkstyle.sf.net/config_modifiers.html -->
+ <!-- Use a consistent way to put modifiers -->
+ <module name="ModifierOrder"/>
+ <module name="RedundantModifier"/>
+
+ <!-- Checks for Naming Conventions. -->
+ <!-- See http://checkstyle.sf.net/config_naming.html -->
+ <!-- Constant names should obey the traditional all uppercase
+ naming convention -->
+ <module name="ConstantName"/>
+ <module name="LocalFinalVariableName"/>
+ <module name="LocalVariableName"/>
+ <module name="MemberName"/>
+ <module name="MethodName"/>
+ <module name="PackageName"/>
+ <module name="ParameterName"/>
+ <module name="StaticVariableName">
+ <property name="format" value="^[A-Z0-9_]*$"/>
+ </module>
+ <module name="TypeName"/>
+
+ <!-- Checks for regexp expressions. -->
+ <!-- See http://checkstyle.sf.net/config_regexp.html -->
+ <!-- No trailing whitespace -->
+ <module name="Regexp">
+ <property name="format" value="[ \t]+$"/>
+ <property name="illegalPattern" value="true"/>
+ <property name="message" value="Trailing whitespace"/>
+ </module>
+ <!-- No System.out.println() statements -->
+ <module name="Regexp">
+ <!-- No sysouts -->
+ <property name="format" value="System\.out\.println"/>
+ <property name="illegalPattern" value="true"/>
+ </module>
+ <!-- Authors should be in pom.xml file -->
+ <module name="Regexp">
+ <property name="format" value="@author"/>
+ <property name="illegalPattern" value="true"/>
+ <property name="message" value="developers names should be in pom file"/>
+ </module>
+
+ <!-- Checks for Size Violations. -->
+ <!-- See http://checkstyle.sf.net/config_sizes.html -->
+ <!-- Lines cannot exceed 80 chars -->
+ <module name="LineLength">
+ <property name="max" value="80"/>
+ <property name="ignorePattern" value="^import"/>
+ </module>
+ <!-- Over time, we will revised this down -->
+ <module name="MethodLength">
+ <property name="max" value="200"/>
+ </module>
+ <module name="ParameterNumber">
+ <property name="max" value="8"/>
+ </module>
+
+ <!-- Checks for whitespace (tree walker) -->
+ <!-- See http://checkstyle.sf.net/config_whitespace.html -->
+ <module name="EmptyForIteratorPad"/>
+ <!-- Spacing around methods -->
+ <module name="MethodParamPad">
+ <property name="option" value="nospace"/>
+ <property name="allowLineBreaks" value="true"/>
+ </module>
+ <!-- No whitespace before a token -->
+ <module name="NoWhitespaceBefore"/>
+ <!-- Whitespace after tokens is required -->
+ <module name="WhitespaceAfter"/>
+ <!-- Whitespace around tokens is required -->
+ <module name="WhitespaceAround"/>
+ <module name="ParenPad"/>
+ <module name="TypecastParenPad"/>
+ <!-- No extra whitespace around types -->
+ <module name="GenericWhitespace"/>
+ <!-- Operator must be at end of wrapped line -->
+ <module name="OperatorWrap">
+ <property name="option" value="eol"/>
+ </module>
+
+ <!-- Required for SuppressionCommentFilter below -->
+ <module name="FileContentsHolder"/>
+ </module>
+
+ <!-- Setup special comments to suppress specific checks from source files -->
+ <module name="SuppressionCommentFilter">
+ <property name="offCommentFormat" value="CHECKSTYLE\: stop ([\w\|]+)"/>
+ <property name="onCommentFormat" value="CHECKSTYLE\: resume ([\w\|]+)"/>
+ <property name="checkFormat" value="$1"/>
+ </module>
+</module>
+
http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/pom.xml
----------------------------------------------------------------------
diff --git a/giraph-block-app/pom.xml b/giraph-block-app/pom.xml
new file mode 100644
index 0000000..1f653bb
--- /dev/null
+++ b/giraph-block-app/pom.xml
@@ -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.
+-->
+
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+ <modelVersion>4.0.0</modelVersion>
+
+ <parent>
+ <groupId>org.apache.giraph</groupId>
+ <artifactId>giraph-parent</artifactId>
+ <version>1.2.0-SNAPSHOT</version>
+ </parent>
+ <artifactId>giraph-block-app</artifactId>
+ <packaging>jar</packaging>
+
+ <name>Apache Giraph Blocks Framework</name>
+ <url>http://giraph.apache.org/giraph-block-app/</url>
+ <description>Giraph Blocks Framework and utilities for writing applications</description>
+
+ <properties>
+ <top.dir>${project.basedir}/..</top.dir>
+ <checkstyle.config.path>${top.dir}/checkstyle-relaxed.xml</checkstyle.config.path>
+ </properties>
+
+ <build>
+ <plugins>
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-assembly-plugin</artifactId>
+ </plugin>
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-checkstyle-plugin</artifactId>
+ </plugin>
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-jar-plugin</artifactId>
+ </plugin>
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-javadoc-plugin</artifactId>
+ </plugin>
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-site-plugin</artifactId>
+ <configuration>
+ <siteDirectory>${project.basedir}/src/site</siteDirectory>
+ </configuration>
+ </plugin>
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-surefire-plugin</artifactId>
+ <version>2.6</version>
+ <configuration>
+ <skip>${surefire.skip}</skip>
+ <systemProperties>
+ <property>
+ <name>prop.jarLocation</name>
+ <value>${top.dir}/giraph-core/target/giraph-${project.version}-${forHadoop}-jar-with-dependencies.jar</value>
+ </property>
+ </systemProperties>
+ </configuration>
+ </plugin>
+ <plugin>
+ <groupId>org.codehaus.mojo</groupId>
+ <artifactId>findbugs-maven-plugin</artifactId>
+ </plugin>
+ </plugins>
+ </build>
+
+ <dependencies>
+ <!-- compile dependencies. sorted lexicographically. -->
+ <dependency>
+ <groupId>it.unimi.dsi</groupId>
+ <artifactId>fastutil</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.giraph</groupId>
+ <artifactId>giraph-core</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>com.google.guava</groupId>
+ <artifactId>guava</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.python</groupId>
+ <artifactId>jython</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>log4j</groupId>
+ <artifactId>log4j</artifactId>
+ </dependency>
+
+ <!-- runtime dependency -->
+ <dependency>
+ <groupId>org.slf4j</groupId>
+ <artifactId>slf4j-api</artifactId>
+ <scope>runtime</scope>
+ </dependency>
+
+ <!-- test dependencies. sorted lexicographically. -->
+ <dependency>
+ <groupId>junit</groupId>
+ <artifactId>junit</artifactId>
+ <scope>test</scope>
+ </dependency>
+ </dependencies>
+</project>
http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/assembly/compile.xml
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/assembly/compile.xml b/giraph-block-app/src/main/assembly/compile.xml
new file mode 100644
index 0000000..fcaffa6
--- /dev/null
+++ b/giraph-block-app/src/main/assembly/compile.xml
@@ -0,0 +1,39 @@
+<!--
+ 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.
+-->
+<assembly xmlns="http://maven.apache.org/plugins/maven-assembly-plugin/assembly/1.1.0"
+ xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+ xsi:schemaLocation="http://maven.apache.org/plugins/maven-assembly-plugin/assembly/1.1.0 http://maven.apache.org/xsd/assembly-1.1.0.xsd">
+ <id>jar-with-dependencies</id>
+ <formats>
+ <format>jar</format>
+ </formats>
+ <includeBaseDirectory>false</includeBaseDirectory>
+
+ <dependencySets>
+ <dependencySet>
+ <useProjectArtifact>true</useProjectArtifact>
+ <outputDirectory>/</outputDirectory>
+ <unpackOptions>
+ <excludes>
+ <exclude>META-INF/LICENSE</exclude>
+ </excludes>
+ </unpackOptions>
+ <unpack>true</unpack>
+ <scope>runtime</scope>
+ </dependencySet>
+ </dependencySets>
+</assembly>
http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/AbstractBlockFactory.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/AbstractBlockFactory.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/AbstractBlockFactory.java
new file mode 100644
index 0000000..66ad775
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/AbstractBlockFactory.java
@@ -0,0 +1,203 @@
+/*
+ * 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.giraph.block_app.framework;
+
+import java.util.List;
+
+import org.apache.giraph.comm.messages.MessageEncodeAndStoreType;
+import org.apache.giraph.conf.GiraphConfiguration;
+import org.apache.giraph.conf.GiraphConstants;
+import org.apache.giraph.conf.StrConfOption;
+import org.apache.giraph.edge.IdAndNullArrayEdges;
+import org.apache.giraph.edge.IdAndValueArrayEdges;
+import org.apache.giraph.edge.LongDiffNullArrayEdges;
+import org.apache.giraph.types.ops.PrimitiveIdTypeOps;
+import org.apache.giraph.types.ops.TypeOps;
+import org.apache.giraph.types.ops.TypeOpsUtils;
+import org.apache.giraph.utils.ReflectionUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
+
+/**
+ * Default block factory abstract class, providing default methods that need
+ * to be/can be overridden for specifying required/most common parameters,
+ * to simplify setting properties.
+ *
+ * @param <S> Execution stage type
+ */
+public abstract class AbstractBlockFactory<S> implements BlockFactory<S> {
+ /**
+ * Comma separated list of BulkConfigurators, that are going to be called
+ * to simplify specifying of large number of properties.
+ */
+ public static final StrConfOption CONFIGURATORS = new StrConfOption(
+ "digraph.block_factory_configurators", null, "");
+
+ @Override
+ public List<String> getGcJavaOpts(Configuration conf) {
+ return null;
+ }
+
+ @Override
+ public final void initConfig(GiraphConfiguration conf) {
+ initConfigurators(conf);
+ GiraphConstants.VERTEX_ID_CLASS.setIfUnset(conf, getVertexIDClass(conf));
+ GiraphConstants.VERTEX_VALUE_CLASS.setIfUnset(
+ conf, getVertexValueClass(conf));
+ GiraphConstants.EDGE_VALUE_CLASS.setIfUnset(conf, getEdgeValueClass(conf));
+ GiraphConstants.RESOLVER_CREATE_VERTEX_ON_MSGS.setIfUnset(
+ conf, shouldCreateVertexOnMsgs(conf));
+ if (shouldSendOneMessageToAll(conf)) {
+ GiraphConstants.MESSAGE_ENCODE_AND_STORE_TYPE.setIfUnset(
+ conf, MessageEncodeAndStoreType.EXTRACT_BYTEARRAY_PER_PARTITION);
+ }
+
+ BlockUtils.BLOCK_WORKER_CONTEXT_VALUE_CLASS.setIfUnset(
+ conf, getWorkerContextValueClass(conf));
+
+ // optimize edge structure, if available and not set already
+ if (!GiraphConstants.VERTEX_EDGES_CLASS.contains(conf)) {
+ @SuppressWarnings("rawtypes")
+ Class<? extends WritableComparable> vertexIDClass =
+ GiraphConstants.VERTEX_ID_CLASS.get(conf);
+ Class<? extends Writable> edgeValueClass =
+ GiraphConstants.EDGE_VALUE_CLASS.get(conf);
+
+
+ @SuppressWarnings("rawtypes")
+ PrimitiveIdTypeOps<? extends WritableComparable> idTypeOps =
+ TypeOpsUtils.getPrimitiveIdTypeOpsOrNull(vertexIDClass);
+ if (edgeValueClass.equals(NullWritable.class)) {
+ if (vertexIDClass.equals(LongWritable.class)) {
+ GiraphConstants.VERTEX_EDGES_CLASS.set(
+ conf, LongDiffNullArrayEdges.class);
+ } else if (idTypeOps != null) {
+ GiraphConstants.VERTEX_EDGES_CLASS.set(
+ conf, IdAndNullArrayEdges.class);
+ }
+ } else {
+ TypeOps<?> edgeValueTypeOps =
+ TypeOpsUtils.getTypeOpsOrNull(edgeValueClass);
+ if (edgeValueTypeOps != null) {
+ GiraphConstants.VERTEX_EDGES_CLASS.set(
+ conf, IdAndValueArrayEdges.class);
+ }
+ }
+ }
+
+ additionalInitConfig(conf);
+ }
+
+ @Override
+ public void registerOutputs(GiraphConfiguration conf) {
+ }
+
+ private void initConfigurators(GiraphConfiguration conf) {
+ String configurators = CONFIGURATORS.get(conf);
+ if (configurators != null) {
+ String[] split = configurators.split(",");
+ for (String configurator : split) {
+ runConfigurator(conf, configurator);
+ }
+ }
+ }
+
+ private void runConfigurator(GiraphConfiguration conf, String configurator) {
+ String[] packages = getConvenienceConfiguratorPackages();
+ String[] prefixes = new String[packages.length + 1];
+ prefixes[0] = "";
+ for (int i = 0; i < packages.length; i++) {
+ prefixes[i + 1] = packages[i] + ".";
+ }
+
+ for (String prefix : prefixes) {
+ try {
+ @SuppressWarnings({ "unchecked", "rawtypes" })
+ Class<BulkConfigurator> confClass =
+ (Class) Class.forName(prefix + configurator);
+ BulkConfigurator c = ReflectionUtils.newInstance(confClass);
+ c.configure(conf);
+ return;
+ // CHECKSTYLE: stop EmptyBlock
+ // ignore ClassNotFoundException, and continue the loop
+ } catch (ClassNotFoundException e) {
+ }
+ // CHECKSTYLE: resume EmptyBlock
+ }
+ throw new IllegalStateException(
+ "Configurator " + configurator + " not found");
+ }
+
+ /**
+ * Additional configuration initialization, other then overriding
+ * class specification.
+ */
+ protected void additionalInitConfig(GiraphConfiguration conf) {
+ }
+
+ /**
+ * Concrete vertex id class application will use.
+ */
+ @SuppressWarnings("rawtypes")
+ protected abstract Class<? extends WritableComparable> getVertexIDClass(
+ GiraphConfiguration conf);
+
+ /**
+ * Concrete vertex value class application will use.
+ */
+ protected abstract Class<? extends Writable> getVertexValueClass(
+ GiraphConfiguration conf);
+
+ /**
+ * Concrete edge value class application will use.
+ */
+ protected abstract Class<? extends Writable> getEdgeValueClass(
+ GiraphConfiguration conf);
+
+ /**
+ * Concrete worker context value class application will use, if overridden.
+ */
+ protected Class<?> getWorkerContextValueClass(GiraphConfiguration conf) {
+ return Object.class;
+ }
+
+ /**
+ * Override if vertices shouldn't be created by default, if message is sent
+ * to a vertex that doesn't exist.
+ */
+ protected boolean shouldCreateVertexOnMsgs(GiraphConfiguration conf) {
+ return true;
+ }
+
+ // TODO - see if it should be deprecated
+ protected boolean shouldSendOneMessageToAll(GiraphConfiguration conf) {
+ return false;
+ }
+
+ /**
+ * Provide list of strings representing packages where configurators will
+ * be searched for, allowing that full path is not required for
+ * CONFIGURATORS option.
+ */
+ protected String[] getConvenienceConfiguratorPackages() {
+ return new String[] { };
+ }
+}
http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/BlockFactory.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/BlockFactory.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/BlockFactory.java
new file mode 100644
index 0000000..652eb5e
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/BlockFactory.java
@@ -0,0 +1,83 @@
+/*
+ * 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.giraph.block_app.framework;
+
+import java.util.List;
+
+import org.apache.giraph.block_app.framework.block.Block;
+import org.apache.giraph.conf.GiraphConfiguration;
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * Class describing a particular application.
+ * Everything except input and output should be fully encapsulated within
+ * this class. For any application, it should be enough to only specify
+ * particular BlockFactory.
+ *
+ * Given configuration, it creates a block that represents a full Giraph job.
+ *
+ * Recommended is to extend AbstractBlockFactory directly for most cases.
+ *
+ * @param <S> Execution stage type
+ */
+public interface BlockFactory<S> {
+ /**
+ * Based on provided configuration, updates it, such that all necessary
+ * properties are initialized.
+ */
+ void initConfig(GiraphConfiguration conf);
+
+ /**
+ * Create a block (representing a full Giraph job), based on the given
+ * configuration. Configuration should be treated as immutable at this point.
+ *
+ * If there are issues in configuration, it is very cheap to throw
+ * from this method - as Giraph job will not even start.
+ * This function will be called two times - once before starting
+ * of the Giraph job, to fail early if anything is incorrectly configured.
+ * Second time will be on Master, which will return Block instance
+ * on which createIterator will be called once, which should return
+ * current application run.
+ * initConfig will be called only once, before starting Giraph job itself.
+ * Master will contain configuration already modified by initConfig.
+ */
+ Block createBlock(GiraphConfiguration conf);
+
+ /**
+ * Create an empty instance of execution stage object.
+ *
+ * Can be used by application to be aware of what was executed before.
+ * Most common example is counting iterations, or for having a boolean whether
+ * some important event happened.
+ *
+ * Execution stage should be immutable object, with creating a new
+ * object when different value is needed.
+ */
+ S createExecutionStage(GiraphConfiguration conf);
+
+ /**
+ * Get special GC Java options. If returns null, default options are used.
+ */
+ List<String> getGcJavaOpts(Configuration conf);
+
+ /**
+ * Register outputs to use during the application (vs output at the end of
+ * the application), based on provided configuration.
+ */
+ void registerOutputs(GiraphConfiguration conf);
+}
http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/BlockUtils.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/BlockUtils.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/BlockUtils.java
new file mode 100644
index 0000000..df260f5
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/BlockUtils.java
@@ -0,0 +1,206 @@
+/*
+ * 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.giraph.block_app.framework;
+
+import static org.apache.giraph.utils.ReflectionUtils.getTypeArguments;
+
+import java.lang.reflect.Field;
+import java.lang.reflect.Modifier;
+
+import org.apache.giraph.block_app.framework.api.giraph.BlockComputation;
+import org.apache.giraph.block_app.framework.api.giraph.BlockMasterCompute;
+import org.apache.giraph.block_app.framework.api.giraph.BlockWorkerContext;
+import org.apache.giraph.block_app.framework.block.Block;
+import org.apache.giraph.block_app.framework.piece.AbstractPiece;
+import org.apache.giraph.block_app.framework.piece.Piece;
+import org.apache.giraph.conf.ClassConfOption;
+import org.apache.giraph.conf.GiraphConfiguration;
+import org.apache.giraph.conf.GiraphConstants;
+import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
+import org.apache.giraph.conf.MessageClasses;
+import org.apache.giraph.function.Consumer;
+import org.apache.giraph.types.NoMessage;
+import org.apache.giraph.utils.ReflectionUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.log4j.Logger;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * Utility functions for block applications
+ */
+@SuppressWarnings({ "rawtypes", "unchecked" })
+public class BlockUtils {
+ /** Property describing BlockFactory to use for current application run */
+ public static final ClassConfOption<BlockFactory> BLOCK_FACTORY_CLASS =
+ ClassConfOption.create("digraph.block_factory", null, BlockFactory.class,
+ "block factory describing giraph job");
+
+ /** Property describing BlockFactory to use for current application run */
+ public static final ClassConfOption<Object> BLOCK_WORKER_CONTEXT_VALUE_CLASS =
+ ClassConfOption.create(
+ "digraph.block_worker_context_value_class", null, Object.class,
+ "block worker context value class");
+
+ private static final Logger LOG = Logger.getLogger(BlockUtils.class);
+
+ /** Dissallow constructor */
+ private BlockUtils() { }
+
+ /**
+ * Create new BlockFactory that is specified in the configuration.
+ */
+ public static <S> BlockFactory<S> createBlockFactory(Configuration conf) {
+ return ReflectionUtils.newInstance(BLOCK_FACTORY_CLASS.get(conf));
+ }
+
+ /**
+ * Set which BlockFactory class to be used for the application.
+ * (generally useful within tests only)
+ */
+ public static void setBlockFactoryClass(Configuration conf,
+ Class<? extends BlockFactory<?>> clazz) {
+ BLOCK_FACTORY_CLASS.set(conf, clazz);
+ }
+
+ /**
+ * Set block factory, and initialize configs with it.
+ * Should be used only if there are no configuration options set after
+ * this method call.
+ */
+ public static void setAndInitBlockFactoryClass(GiraphConfiguration conf,
+ Class<? extends BlockFactory<?>> clazz) {
+ BLOCK_FACTORY_CLASS.set(conf, clazz);
+ initAndCheckConfig(conf);
+ }
+
+ /**
+ * Initializes configuration, such that running it executes block application.
+ *
+ * Additionally, checks types of all pieces with a block application.
+ */
+ public static void initAndCheckConfig(GiraphConfiguration conf) {
+ conf.setMasterComputeClass(BlockMasterCompute.class);
+ conf.setComputationClass(BlockComputation.class);
+ conf.setWorkerContextClass(BlockWorkerContext.class);
+
+ Preconditions.checkState(
+ GiraphConstants.OUTGOING_MESSAGE_VALUE_CLASS.get(conf) == null,
+ "Message types should only be specified in Pieces, " +
+ "but outgoing was specified globally");
+ Preconditions.checkState(
+ GiraphConstants.OUTGOING_MESSAGE_VALUE_FACTORY_CLASS
+ .isDefaultValue(conf),
+ "Message types should only be specified in Pieces, " +
+ "but factory was specified globally");
+ Preconditions.checkState(
+ GiraphConstants.MESSAGE_COMBINER_CLASS.get(conf) == null,
+ "Message combiner should only be specified in Pieces, " +
+ "but was specified globally");
+
+ BlockFactory<?> blockFactory = createBlockFactory(conf);
+ blockFactory.initConfig(conf);
+
+ Preconditions.checkState(
+ GiraphConstants.OUTGOING_MESSAGE_VALUE_CLASS.get(conf) == null,
+ "Outgoing message type was specified in blockFactory.initConfig");
+ Preconditions.checkState(
+ GiraphConstants.OUTGOING_MESSAGE_VALUE_FACTORY_CLASS
+ .isDefaultValue(conf),
+ "Outgoing message factory type was specified in " +
+ "blockFactory.initConfig");
+ Preconditions.checkState(
+ GiraphConstants.MESSAGE_COMBINER_CLASS.get(conf) == null,
+ "Message combiner type was specified in blockFactory.initConfig");
+
+ GiraphConstants.OUTGOING_MESSAGE_VALUE_CLASS.set(conf, NoMessage.class);
+
+ final ImmutableClassesGiraphConfiguration immConf =
+ new ImmutableClassesGiraphConfiguration<>(conf);
+
+ // Create blocks to detect issues before creating a Giraph job
+ // They will not be used here
+ Block executionBlock = blockFactory.createBlock(immConf);
+ LOG.info("Executing application - " + executionBlock);
+
+ final Class<?> vertexIdClass = GiraphConstants.VERTEX_ID_CLASS.get(conf);
+ final Class<?> vertexValueClass =
+ GiraphConstants.VERTEX_VALUE_CLASS.get(conf);
+ final Class<?> edgeValueClass =
+ GiraphConstants.EDGE_VALUE_CLASS.get(conf);
+ final Class<?> workerContextValueClass =
+ BlockUtils.BLOCK_WORKER_CONTEXT_VALUE_CLASS.get(conf);
+ final Class<?> executionStageClass =
+ blockFactory.createExecutionStage(conf).getClass();
+
+ // Check for type inconsistencies
+ executionBlock.forAllPossiblePieces(new Consumer<AbstractPiece>() {
+ @Override
+ public void apply(AbstractPiece piece) {
+ if (!piece.getClass().equals(Piece.class)) {
+ Class<?>[] classList = getTypeArguments(
+ AbstractPiece.class, piece.getClass());
+ Preconditions.checkArgument(classList.length == 7);
+
+ ReflectionUtils.verifyTypes(
+ vertexIdClass, classList[0], "vertexId", piece.getClass());
+ ReflectionUtils.verifyTypes(
+ vertexValueClass, classList[1], "vertexValue", piece.getClass());
+ ReflectionUtils.verifyTypes(
+ edgeValueClass, classList[2], "edgeValue", piece.getClass());
+
+ MessageClasses classes = piece.getMessageClasses(immConf);
+ Class<?> messageType = classes.getMessageClass();
+ if (messageType == null) {
+ messageType = NoMessage.class;
+ }
+ ReflectionUtils.verifyTypes(
+ messageType, classList[3], "message", piece.getClass());
+
+ ReflectionUtils.verifyTypes(
+ workerContextValueClass, classList[4],
+ "workerContextValue", piece.getClass());
+ // No need to check worker context message class at all
+
+ ReflectionUtils.verifyTypes(
+ executionStageClass, classList[6],
+ "executionStage", piece.getClass());
+ }
+ }
+ });
+
+ // check for non 'static final' fields in BlockFactories
+ Class<?> bfClass = blockFactory.getClass();
+ while (!bfClass.equals(Object.class)) {
+ for (Field field : bfClass.getDeclaredFields()) {
+ if (!Modifier.isStatic(field.getModifiers()) ||
+ !Modifier.isFinal(field.getModifiers())) {
+ throw new IllegalStateException("BlockFactory (" + bfClass +
+ ") cannot have any mutable (non 'static final') fields as a " +
+ "safety measure, as createBlock function is called from a " +
+ "different context then all other functions, use conf argument " +
+ "instead, or make it 'static final'. Field present: " + field);
+ }
+ }
+ bfClass = bfClass.getSuperclass();
+ }
+
+ // Register outputs
+ blockFactory.registerOutputs(conf);
+ }
+}
http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/BulkConfigurator.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/BulkConfigurator.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/BulkConfigurator.java
new file mode 100644
index 0000000..348c907
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/BulkConfigurator.java
@@ -0,0 +1,32 @@
+/*
+ * 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.giraph.block_app.framework;
+
+import org.apache.giraph.conf.GiraphConfiguration;
+
+/**
+ * Function that modifies configuration.
+ *
+ * Allows for multi-option configuration to be specified in a common classes.
+ */
+public interface BulkConfigurator {
+ /**
+ * Modify given configuration.
+ */
+ void configure(GiraphConfiguration conf);
+}
http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/BlockApi.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/BlockApi.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/BlockApi.java
new file mode 100644
index 0000000..a0c92ad
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/BlockApi.java
@@ -0,0 +1,47 @@
+/*
+ * 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.giraph.block_app.framework.api;
+
+
+/**
+ * Basic block computation API for accessing items
+ * present on both workers and master.
+ */
+public interface BlockApi extends BlockConfApi {
+ /**
+ * Get the total (all workers) number of vertices that
+ * existed at the start of the current piece.
+ *
+ * Recommended to avoid it, as it introduces global dependencies,
+ * code will not be able to work on top of a subgraphs any more.
+ * This number should be easily computable via reducer or aggregator.
+ */
+ @Deprecated
+ long getTotalNumVertices();
+
+ /**
+ * Get the total (all workers) number of edges that
+ * existed at the start of the current piece.
+ *
+ * Recommended to avoid it, as it introduces global dependencies,
+ * code will not be able to work on top of a subgraphs any more.
+ * This number should be easily computable via reducer or aggregator.
+ */
+ @Deprecated
+ long getTotalNumEdges();
+}
http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/BlockConfApi.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/BlockConfApi.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/BlockConfApi.java
new file mode 100644
index 0000000..98e51f8
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/BlockConfApi.java
@@ -0,0 +1,32 @@
+/*
+ * 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.giraph.block_app.framework.api;
+
+import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
+
+/**
+ * Block computation API for accessing configuration.
+ */
+public interface BlockConfApi {
+ /**
+ * Return the configuration used by this object.
+ *
+ * @return Set configuration
+ */
+ ImmutableClassesGiraphConfiguration<?, ?, ?> getConf();
+}
http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/BlockMasterApi.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/BlockMasterApi.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/BlockMasterApi.java
new file mode 100644
index 0000000..3d04584
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/BlockMasterApi.java
@@ -0,0 +1,58 @@
+/*
+ * 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.giraph.block_app.framework.api;
+
+import org.apache.giraph.aggregators.Aggregator;
+import org.apache.giraph.block_app.framework.piece.global_comm.BroadcastHandle;
+import org.apache.giraph.master.MasterAggregatorUsage;
+import org.apache.giraph.master.MasterGlobalCommUsage;
+import org.apache.hadoop.io.Writable;
+
+/**
+ * Block computation API available for the master methods.
+ *
+ * Interface to the MasterCompute methods.
+ */
+public interface BlockMasterApi extends MasterAggregatorUsage,
+ MasterGlobalCommUsage, StatusReporter, BlockApi, BlockOutputApi {
+ /**
+ * No need to use it, and introduces global dependencies.
+ *
+ * Store data locally within the piece, or use ObjectHolder.
+ */
+ @Deprecated
+ @Override
+ <A extends Writable>
+ boolean registerPersistentAggregator(
+ String name, Class<? extends Aggregator<A>> aggregatorClass
+ ) throws InstantiationException, IllegalAccessException;
+
+ /**
+ * Broadcast given value to all workers for next computation.
+ * @param value Value to broadcast
+ */
+ <T extends Writable> BroadcastHandle<T> broadcast(T value);
+
+ /**
+ * Call this to log a line to command line of the job. Use in moderation -
+ * it's a synchronous call to Job client
+ *
+ * @param line Line to print
+ */
+ void logToCommandLine(String line);
+}
http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/BlockOutputApi.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/BlockOutputApi.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/BlockOutputApi.java
new file mode 100644
index 0000000..8c33623
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/BlockOutputApi.java
@@ -0,0 +1,31 @@
+/*
+ * 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.giraph.block_app.framework.api;
+
+import org.apache.giraph.block_app.framework.output.BlockOutputDesc;
+import org.apache.giraph.block_app.framework.output.BlockOutputWriter;
+
+/**
+ * Block worker output api
+ */
+public interface BlockOutputApi {
+ <OW extends BlockOutputWriter, OD extends BlockOutputDesc<OW>>
+ OD getOutputDesc(String confOption);
+
+ <OW extends BlockOutputWriter> OW getWriter(String confOption);
+}
http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/BlockOutputHandleAccessor.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/BlockOutputHandleAccessor.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/BlockOutputHandleAccessor.java
new file mode 100644
index 0000000..074637d
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/BlockOutputHandleAccessor.java
@@ -0,0 +1,33 @@
+/*
+ * 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.giraph.block_app.framework.api;
+
+import org.apache.giraph.block_app.framework.output.BlockOutputHandle;
+
+/**
+ * Function for accessing BlockOutputHandle.
+ *
+ * Intentionally hidden from APIs, to allow usage only
+ * within DefaultParentPiece.
+ */
+public interface BlockOutputHandleAccessor {
+ /**
+ * Get global block output handle.
+ */
+ BlockOutputHandle getBlockOutputHandle();
+}
http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/BlockWorkerApi.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/BlockWorkerApi.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/BlockWorkerApi.java
new file mode 100644
index 0000000..727bf08
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/BlockWorkerApi.java
@@ -0,0 +1,35 @@
+/*
+ * 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.giraph.block_app.framework.api;
+
+import org.apache.giraph.aggregators.AggregatorUsage;
+import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
+import org.apache.hadoop.io.WritableComparable;
+
+/**
+ * Block computation API available for worker methods.
+ *
+ * Interface to the Computation methods.
+ * @param <I> vertex Id type.
+ */
+@SuppressWarnings("rawtypes")
+public interface BlockWorkerApi<I extends WritableComparable>
+ extends AggregatorUsage, BlockApi {
+ @Override
+ ImmutableClassesGiraphConfiguration<I, ?, ?> getConf();
+}
http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/BlockWorkerContextApi.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/BlockWorkerContextApi.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/BlockWorkerContextApi.java
new file mode 100644
index 0000000..d5918b5
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/BlockWorkerContextApi.java
@@ -0,0 +1,45 @@
+/*
+ * 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.giraph.block_app.framework.api;
+
+import org.apache.giraph.aggregators.AggregatorUsage;
+import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
+
+/**
+ * Block computation API available for worker context methods.
+ *
+ * Interface to the WorkerContext methods.
+ */
+public interface BlockWorkerContextApi extends AggregatorUsage, BlockApi {
+ @Override
+ ImmutableClassesGiraphConfiguration<?, ?, ?> getConf();
+
+ /**
+ * Get number of workers
+ *
+ * @return Number of workers
+ */
+ int getWorkerCount();
+
+ /**
+ * Get index for this worker
+ *
+ * @return Index of this worker
+ */
+ int getMyWorkerIndex();
+}
http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/BlockWorkerContextReceiveApi.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/BlockWorkerContextReceiveApi.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/BlockWorkerContextReceiveApi.java
new file mode 100644
index 0000000..a8242b2
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/BlockWorkerContextReceiveApi.java
@@ -0,0 +1,29 @@
+/*
+ * 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.giraph.block_app.framework.api;
+
+import org.apache.giraph.worker.WorkerBroadcastUsage;
+
+/**
+ * Block computation API available for worker context receive methods.
+ *
+ * Interface to the WorkerContext methods.
+ */
+public interface BlockWorkerContextReceiveApi
+ extends BlockWorkerContextApi, WorkerBroadcastUsage {
+}
http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/BlockWorkerContextSendApi.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/BlockWorkerContextSendApi.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/BlockWorkerContextSendApi.java
new file mode 100644
index 0000000..769562d
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/BlockWorkerContextSendApi.java
@@ -0,0 +1,39 @@
+/*
+ * 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.giraph.block_app.framework.api;
+
+import org.apache.giraph.worker.WorkerAggregatorUsage;
+import org.apache.hadoop.io.Writable;
+
+/**
+ * Block computation API available for worker send methods.
+ *
+ * Interface to the WorkerContext methods.
+ *
+ * @param <WM> Worker message type
+ */
+public interface BlockWorkerContextSendApi<WM extends Writable>
+ extends BlockWorkerContextApi, WorkerAggregatorUsage {
+ /**
+ * Send message to another worker
+ *
+ * @param message Message to send
+ * @param workerIndex Index of the worker to send the message to
+ */
+ void sendMessageToWorker(WM message, int workerIndex);
+}
http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/BlockWorkerReceiveApi.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/BlockWorkerReceiveApi.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/BlockWorkerReceiveApi.java
new file mode 100644
index 0000000..6db51bd
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/BlockWorkerReceiveApi.java
@@ -0,0 +1,33 @@
+/*
+ * 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.giraph.block_app.framework.api;
+
+import org.apache.giraph.worker.WorkerBroadcastUsage;
+import org.apache.hadoop.io.WritableComparable;
+
+/**
+ * Block computation API available for worker receive methods.
+ *
+ * Interface to the Computation methods.
+ *
+ * @param <I> vertex Id type.
+ */
+@SuppressWarnings("rawtypes")
+public interface BlockWorkerReceiveApi<I extends WritableComparable>
+ extends BlockWorkerApi<I>, WorkerBroadcastUsage, BlockOutputApi {
+}
http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/BlockWorkerSendApi.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/BlockWorkerSendApi.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/BlockWorkerSendApi.java
new file mode 100644
index 0000000..a72150f
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/BlockWorkerSendApi.java
@@ -0,0 +1,116 @@
+/*
+ * 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.giraph.block_app.framework.api;
+
+import java.util.Iterator;
+
+import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
+import org.apache.giraph.edge.Edge;
+import org.apache.giraph.edge.OutEdges;
+import org.apache.giraph.graph.Vertex;
+import org.apache.giraph.worker.WorkerAggregatorUsage;
+import org.apache.giraph.worker.WorkerReduceUsage;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
+
+/**
+ * Block computation API available for worker send methods.
+ *
+ * Interface to the Computation methods.
+ *
+ * @param <I> Vertex id type
+ * @param <V> Vertex value type
+ * @param <E> Edge value type
+ * @param <M> Message type
+ */
+@SuppressWarnings("rawtypes")
+public interface BlockWorkerSendApi<I extends WritableComparable,
+ V extends Writable, E extends Writable, M extends Writable>
+ extends BlockWorkerApi<I>, WorkerAggregatorUsage, WorkerReduceUsage {
+ @Override
+ ImmutableClassesGiraphConfiguration<I, V, E> getConf();
+
+ /**
+ * Send a message to a vertex id.
+ *
+ * @param id Vertex id to send the message to
+ * @param message Message data to send
+ */
+ void sendMessage(I id, M message);
+
+ /**
+ * Send a message to all edges.
+ *
+ * @param vertex Vertex whose edges to send the message to.
+ * @param message Message sent to all edges.
+ */
+ void sendMessageToAllEdges(Vertex<I, V, E> vertex, M message);
+
+ /**
+ * Send a message to multiple target vertex ids in the iterator.
+ *
+ * @param vertexIdIterator An iterator to multiple target vertex ids.
+ * @param message Message sent to all targets in the iterator.
+ */
+ void sendMessageToMultipleEdges(Iterator<I> vertexIdIterator, M message);
+
+ /**
+ * Sends a request to create a vertex that will be available
+ * in the receive phase.
+ *
+ * @param id Vertex id
+ * @param value Vertex value
+ * @param edges Initial edges
+ */
+ void addVertexRequest(I id, V value, OutEdges<I, E> edges);
+
+ /**
+ * Sends a request to create a vertex that will be available
+ * in the receive phase.
+ *
+ * @param id Vertex id
+ * @param value Vertex value
+ */
+ void addVertexRequest(I id, V value);
+
+ /**
+ * Request to remove a vertex from the graph
+ * (applied just prior to the next receive phase).
+ *
+ * @param vertexId Id of the vertex to be removed.
+ */
+ void removeVertexRequest(I vertexId);
+
+ /**
+ * Request to add an edge of a vertex in the graph
+ * (processed just prior to the next receive phase)
+ *
+ * @param sourceVertexId Source vertex id of edge
+ * @param edge Edge to add
+ */
+ void addEdgeRequest(I sourceVertexId, Edge<I, E> edge);
+
+ /**
+ * Request to remove all edges from a given source vertex to a given target
+ * vertex (processed just prior to the next receive phase).
+ *
+ * @param sourceVertexId Source vertex id
+ * @param targetVertexId Target vertex id
+ */
+ void removeEdgesRequest(I sourceVertexId, I targetVertexId);
+}
http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/BlockWorkerValueAccessor.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/BlockWorkerValueAccessor.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/BlockWorkerValueAccessor.java
new file mode 100644
index 0000000..f7cddf3
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/BlockWorkerValueAccessor.java
@@ -0,0 +1,36 @@
+/*
+ * 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.giraph.block_app.framework.api;
+
+/**
+ * Function for accessing WorkerValue.
+ *
+ * Intentionally hidden from APIs, to allow usage only
+ * within PieceWithWorkerContext.
+ */
+public interface BlockWorkerValueAccessor {
+ /**
+ * Get global worker value.
+ * Value returned can be accessed from may threads, and so all
+ * accesses to it should be done in a thread-safe manner!
+ *
+ * This is the only place in Blocks Framework where you need
+ * to take care of concurrency.
+ */
+ Object getWorkerValue();
+}
http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/Counter.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/Counter.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/Counter.java
new file mode 100644
index 0000000..730b0ab
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/Counter.java
@@ -0,0 +1,33 @@
+/*
+ * 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.giraph.block_app.framework.api;
+
+/** Interface wrapping around org.appache.hadoop.mapreduce.Counter */
+public interface Counter {
+ /**
+ * Set this counter by the given value
+ * @param value the value to set
+ */
+ void setValue(long value);
+
+ /**
+ * Increment this counter by the given value
+ * @param incr the value to increase this counter by
+ */
+ void increment(long incr);
+}
http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/CreateReducersApi.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/CreateReducersApi.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/CreateReducersApi.java
new file mode 100644
index 0000000..77b3b1e
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/CreateReducersApi.java
@@ -0,0 +1,83 @@
+/*
+ * 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.giraph.block_app.framework.api;
+
+import org.apache.giraph.block_app.framework.piece.global_comm.ReducerHandle;
+import org.apache.giraph.reducers.ReduceOperation;
+import org.apache.hadoop.io.Writable;
+
+/**
+ * Api for creating reducer handles.
+ */
+public interface CreateReducersApi extends BlockConfApi {
+
+ /**
+ * Create local reducer, returning a handle to it.
+ *
+ * Local reducer means that each worker thread has it's own local partially
+ * reduced value, which are at the end reduced all together.
+ * Preferable, unless it cannot be used, because all copies of the object
+ * do not fit the memory.
+ */
+ <S, R extends Writable> ReducerHandle<S, R> createLocalReducer(
+ ReduceOperation<S, R> reduceOp);
+
+ /**
+ * Create local reducer, returning a handle to it.
+ *
+ * Local reducer means that each worker thread has it's own local partially
+ * reduced value, which are at the end reduced all together.
+ * Preferable, unless it cannot be used, because all copies of the object
+ * do not fit the memory.
+ */
+ <S, R extends Writable> ReducerHandle<S, R> createLocalReducer(
+ ReduceOperation<S, R> reduceOp, R globalInitialValue);
+
+ /**
+ * Create global reducer, returning a handle to it.
+ *
+ * Global reducer means that there is only one value for each worker,
+ * and each call to reduce will have to obtain a global lock, and incur
+ * synchronization costs.
+ * Use only when objects are so large, that having many copies cannot
+ * fit into memory.
+ */
+ <S, R extends Writable> ReducerHandle<S, R> createGlobalReducer(
+ ReduceOperation<S, R> reduceOp);
+
+ /**
+ * Create global reducer, returning a handle to it.
+ *
+ * Global reducer means that there is only one value for each worker,
+ * and each call to reduce will have to obtain a global lock, and incur
+ * synchronization costs.
+ * Use only when objects are so large, that having many copies cannot
+ * fit into memory.
+ */
+ <S, R extends Writable> ReducerHandle<S, R> createGlobalReducer(
+ ReduceOperation<S, R> reduceOp, R globalInitialValue);
+
+ /**
+ * Function that creates a reducer - abstracting away whether it is
+ * local or global reducer
+ */
+ public interface CreateReducerFunctionApi {
+ <S, R extends Writable> ReducerHandle<S, R> createReducer(
+ ReduceOperation<S, R> reduceOp);
+ }
+}
http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/StatusReporter.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/StatusReporter.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/StatusReporter.java
new file mode 100644
index 0000000..3ce1862
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/StatusReporter.java
@@ -0,0 +1,31 @@
+/*
+ * 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.giraph.block_app.framework.api;
+
+/**
+ * Interface wrapping around functions from
+ * org.appache.hadoop.mapreduce.StatusReporter
+ */
+public interface StatusReporter {
+ /** Get specified counter handler */
+ Counter getCounter(String group, String name);
+ /** Report progress to the Hadoop framework. */
+ void progress();
+ /** Set the current status of the task to the given string. */
+ void setStatus(String status);
+}
http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/giraph/BlockComputation.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/giraph/BlockComputation.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/giraph/BlockComputation.java
new file mode 100644
index 0000000..930b62b
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/giraph/BlockComputation.java
@@ -0,0 +1,54 @@
+/*
+ * 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.giraph.block_app.framework.api.giraph;
+
+import java.io.IOException;
+
+import org.apache.giraph.block_app.framework.internal.BlockWorkerLogic;
+import org.apache.giraph.block_app.framework.internal.BlockWorkerPieces;
+import org.apache.giraph.graph.AbstractComputation;
+import org.apache.giraph.graph.Vertex;
+
+
+/**
+ * Computation that executes receiver and sender blocks passed
+ * into BlockWorkerPieces.
+ */
+@SuppressWarnings({ "rawtypes", "unchecked" })
+public final class BlockComputation extends AbstractComputation {
+ private BlockWorkerLogic workerLogic;
+
+ @Override
+ public void preSuperstep() {
+ BlockWorkerPieces workerPieces =
+ BlockWorkerPieces.getNextWorkerPieces(this);
+ workerLogic = new BlockWorkerLogic(workerPieces);
+ BlockWorkerApiWrapper workerApi = new BlockWorkerApiWrapper<>(this);
+ workerLogic.preSuperstep(workerApi, workerApi);
+ }
+
+ @Override
+ public void compute(Vertex vertex, Iterable messages) throws IOException {
+ workerLogic.compute(vertex, messages);
+ }
+
+ @Override
+ public void postSuperstep() {
+ workerLogic.postSuperstep();
+ }
+}
[3/5] git commit: updated refs/heads/trunk to 819d6d3
Posted by ik...@apache.org.
http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/output/BlockOutputHandle.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/output/BlockOutputHandle.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/output/BlockOutputHandle.java
new file mode 100644
index 0000000..fd38520
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/output/BlockOutputHandle.java
@@ -0,0 +1,119 @@
+/*
+ * 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.giraph.block_app.framework.output;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Queue;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ConcurrentLinkedQueue;
+
+import org.apache.giraph.block_app.framework.api.BlockOutputApi;
+import org.apache.giraph.conf.GiraphConstants;
+import org.apache.giraph.utils.CallableFactory;
+import org.apache.giraph.utils.ProgressableUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.util.Progressable;
+
+/**
+ * Handler for blocks output - keeps track of outputs and writers created
+ */
+@SuppressWarnings("unchecked")
+public class BlockOutputHandle implements BlockOutputApi {
+ private transient Configuration conf;
+ private transient Progressable progressable;
+ private final Map<String, BlockOutputDesc> outputDescMap;
+ private final Map<String, Queue<BlockOutputWriter>> freeWriters;
+ private final Map<String, Queue<BlockOutputWriter>> occupiedWriters;
+
+ public BlockOutputHandle(String jobIdentifier, Configuration conf,
+ Progressable hadoopProgressable) {
+ outputDescMap = BlockOutputFormat.createInitAndCheckOutputDescsMap(
+ conf, jobIdentifier);
+ freeWriters = new HashMap<>();
+ occupiedWriters = new HashMap<>();
+ for (String confOption : outputDescMap.keySet()) {
+ freeWriters.put(confOption,
+ new ConcurrentLinkedQueue<BlockOutputWriter>());
+ occupiedWriters.put(confOption,
+ new ConcurrentLinkedQueue<BlockOutputWriter>());
+ }
+ initialize(conf, hadoopProgressable);
+ }
+
+ public void initialize(Configuration conf, Progressable progressable) {
+ this.conf = conf;
+ this.progressable = progressable;
+ }
+
+
+ @Override
+ public <OW extends BlockOutputWriter, OD extends BlockOutputDesc<OW>>
+ OD getOutputDesc(String confOption) {
+ return (OD) outputDescMap.get(confOption);
+ }
+
+ @Override
+ public <OW extends BlockOutputWriter> OW getWriter(String confOption) {
+ OW outputWriter = (OW) freeWriters.get(confOption).poll();
+ if (outputWriter == null) {
+ outputWriter = (OW) outputDescMap.get(confOption).createOutputWriter(
+ conf, progressable);
+ }
+ occupiedWriters.get(confOption).add(outputWriter);
+ return outputWriter;
+ }
+
+ public void returnAllWriters() {
+ for (Map.Entry<String, Queue<BlockOutputWriter>> entry :
+ occupiedWriters.entrySet()) {
+ freeWriters.get(entry.getKey()).addAll(entry.getValue());
+ entry.getValue().clear();
+ }
+ }
+
+ public void closeAllWriters() {
+ final Queue<BlockOutputWriter> allWriters = new ConcurrentLinkedQueue<>();
+ for (Queue<BlockOutputWriter> blockOutputWriters : freeWriters.values()) {
+ allWriters.addAll(blockOutputWriters);
+ }
+ if (allWriters.isEmpty()) {
+ return;
+ }
+ // Closing writers can take time - use multiple threads and call progress
+ CallableFactory<Void> callableFactory = new CallableFactory<Void>() {
+ @Override
+ public Callable<Void> newCallable(int callableId) {
+ return new Callable<Void>() {
+ @Override
+ public Void call() throws Exception {
+ BlockOutputWriter writer = allWriters.poll();
+ while (writer != null) {
+ writer.close();
+ writer = allWriters.poll();
+ }
+ return null;
+ }
+ };
+ }
+ };
+ ProgressableUtils.getResultsWithNCallables(callableFactory,
+ Math.min(GiraphConstants.NUM_OUTPUT_THREADS.get(conf),
+ allWriters.size()), "close-writers-%d", progressable);
+ }
+}
http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/output/BlockOutputOption.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/output/BlockOutputOption.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/output/BlockOutputOption.java
new file mode 100644
index 0000000..5f82612
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/output/BlockOutputOption.java
@@ -0,0 +1,52 @@
+/*
+ * 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.giraph.block_app.framework.output;
+
+import org.apache.giraph.block_app.framework.api.BlockOutputApi;
+import org.apache.giraph.conf.GiraphConfiguration;
+
+/**
+ * Block output option, with apis to use from application code
+ *
+ * @param <OD> Output description type
+ * @param <OW> Output writer type
+ */
+public class BlockOutputOption<OD extends BlockOutputDesc<OW>,
+ OW extends BlockOutputWriter> {
+ private final String key;
+
+ public BlockOutputOption(String key) {
+ this.key = key;
+ }
+
+ public String getKey() {
+ return key;
+ }
+
+ public void register(OD outputDesc, GiraphConfiguration conf) {
+ BlockOutputFormat.addOutputDesc(outputDesc, key, conf);
+ }
+
+ public OD getOutputDesc(BlockOutputApi outputApi) {
+ return outputApi.<OW, OD>getOutputDesc(key);
+ }
+
+ public OW getWriter(BlockOutputApi outputApi) {
+ return outputApi.getWriter(key);
+ }
+}
http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/output/BlockOutputWriter.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/output/BlockOutputWriter.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/output/BlockOutputWriter.java
new file mode 100644
index 0000000..5574ab4
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/output/BlockOutputWriter.java
@@ -0,0 +1,26 @@
+/*
+ * 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.giraph.block_app.framework.output;
+
+import java.io.Closeable;
+
+/**
+ * Block output writer
+ */
+public interface BlockOutputWriter extends Closeable {
+}
http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/output/package-info.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/output/package-info.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/output/package-info.java
new file mode 100644
index 0000000..11193b1
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/output/package-info.java
@@ -0,0 +1,21 @@
+/*
+ * 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.
+ */
+/**
+ * Multi-output support for Block Applications
+ */
+package org.apache.giraph.block_app.framework.output;
http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/package-info.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/package-info.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/package-info.java
new file mode 100644
index 0000000..0b6934e
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/package-info.java
@@ -0,0 +1,25 @@
+/*
+ * 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.
+ */
+/**
+ * Block application abstraction package.
+ *
+ * Giraph application is represented as a collection of pieces,
+ * aggregated via blocks, ultimately into a single block, that
+ * represents complete application execution.
+ */
+package org.apache.giraph.block_app.framework;
http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/AbstractPiece.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/AbstractPiece.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/AbstractPiece.java
new file mode 100644
index 0000000..882f4f1
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/AbstractPiece.java
@@ -0,0 +1,287 @@
+/*
+ * 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.giraph.block_app.framework.piece;
+
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.giraph.block_app.framework.api.BlockMasterApi;
+import org.apache.giraph.block_app.framework.api.BlockWorkerContextReceiveApi;
+import org.apache.giraph.block_app.framework.api.BlockWorkerContextSendApi;
+import org.apache.giraph.block_app.framework.api.BlockWorkerReceiveApi;
+import org.apache.giraph.block_app.framework.api.BlockWorkerSendApi;
+import org.apache.giraph.block_app.framework.block.Block;
+import org.apache.giraph.block_app.framework.piece.interfaces.VertexPostprocessor;
+import org.apache.giraph.block_app.framework.piece.interfaces.VertexReceiver;
+import org.apache.giraph.block_app.framework.piece.interfaces.VertexSender;
+import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
+import org.apache.giraph.conf.MessageClasses;
+import org.apache.giraph.function.Consumer;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
+
+import com.google.common.collect.Iterators;
+
+/**
+ * Parent of all Pieces, contains comprehensive list of methods Piece
+ * can support. Specific subclasses should be extended directly,
+ * to simplify usage - most frequently for example Piece class.
+ *
+ * Single unit of execution, capturing:
+ * - sending and then receiving messages from vertices
+ * - sending data to be aggregated from workers to master
+ * - sending values from master, via aggregators, to workers
+ * - sending and receiving worker messages
+ *
+ *
+ * Order of execution is:
+ * - On master, once at the start of the application
+ * -- registerAggregators (deprecated, use registerReducers instead)
+ *
+ * - After masterCompute of previous piece, on master:
+ * -- registerReducers
+ *
+ * - Send logic on workers:
+ * -- getVertexSender per each worker thread, and on object returned by it:
+ * --- vertexSend on each vertex
+ * --- postprocess on each worker thread
+ * -- workerContextSend per worker
+ *
+ * - Logic on master:
+ * -- masterCompute
+ *
+ * - Receive logic on workers:
+ * -- workerContextReceive per worker
+ * -- getVertexReceiver per each worker thread, and on object returned by it:
+ * --- vertexReceive on each vertex
+ * --- postprocess on each worker thread
+ *
+ * And before everything, during initialization, registerAggregators.
+ *
+ * Only masterCompute and registerReducers/registerAggregators should modify
+ * the Piece, all of the worker methods should treat Piece as read-only.
+ *
+ * Each piece should be encapsulated unit of execution. Vertex value should be
+ * used as a single implicit "communication" channel between different pieces,
+ * all other dependencies should be explicitly defined and passed through
+ * constructor, via interfaces (as explained below).
+ * I.e. state of the vertex value is invariant that Pieces act upon.
+ * Best is not to depend on explicit vertex value class, but on interface that
+ * provides all needed functions, so that pieces can be freely combined,
+ * as long as vertex value implements appropriate ones.
+ * Similarly, use most abstract class you need - if Piece doesn't depend
+ * on edge value, don't use NullWritable, but Writable. Or if it doesn't
+ * depend on ExecutionStage, use Object for it.
+ *
+ * All other external dependencies should be explicitly passed through
+ * constructor, through interfaces.
+ *
+ * All Pieces will be created within one context - on the master.
+ * They are then going to be replicated across all workers, and across all
+ * threads within each worker, and will see everything that happens in global
+ * context (masterCompute) before them, including any state master has.
+ * Through ObjectHolder/ObjectTransfer, you can pass data between Pieces in
+ * global context, and from global context to worker functions of a Piece
+ * that happens in the future.
+ *
+ * VertexReceiver of previous Piece and VertexSender of next Piece live in
+ * the same context, and vertexReceive of the next Piece is executed
+ * immediately after vertexSend of the previous piece, before vertexSend is
+ * called on the next vertex.
+ * This detail allows you to have external dependency on each other through
+ * memory only mediator objects - like ObjectTransfer.
+ *
+ * All other logic going to live in different contexts,
+ * specifically VertexSender and VertexReceiver of the same Piece,
+ * or workerContextSend and VertexSender of the same Piece, and cannot interact
+ * with each other outside of changing the state of the graph or using
+ * global communication api.
+ *
+ * All methods on this class (or objects it returns) will be called serially,
+ * so there is no need for any Thread synchronization.
+ * Each Thread will have a complete deep copy of the Piece, to achieve that,
+ * so all static fields must be written to be Thread safe!
+ * (i.e. either immutable, or have synchronized/locked access to them)
+ *
+ * @param <I> Vertex id type
+ * @param <V> Vertex value type
+ * @param <E> Edge value type
+ * @param <M> Message type
+ * @param <WV> Worker value type
+ * @param <WM> Worker message type
+ * @param <S> Execution stage type
+ */
+@SuppressWarnings({ "rawtypes" })
+public abstract class AbstractPiece<I extends WritableComparable,
+ V extends Writable, E extends Writable, M extends Writable, WV,
+ WM extends Writable, S> implements Block {
+
+ // Overridable functions
+
+ // registerReducers(CreateReducersApi reduceApi, S executionStage)
+
+ /**
+ * Add automatic handling of reducers to registerReducers.
+ * Only for internal use.
+ */
+ public abstract void wrappedRegisterReducers(
+ BlockMasterApi masterApi, S executionStage);
+
+ // getVertexSender(BlockWorkerSendApi<I, V, E, M> workerApi, S executionStage)
+
+ /**
+ * Add automatic handling of reducers to getVertexSender.
+ *
+ * Only for Framework internal use.
+ */
+ public abstract InnerVertexSender getWrappedVertexSender(
+ final BlockWorkerSendApi<I, V, E, M> workerApi, S executionStage);
+
+ /**
+ * Override to have worker context send computation.
+ *
+ * Called once per worker, after all vertices have been processed with
+ * getVertexSender.
+ */
+ public void workerContextSend(
+ BlockWorkerContextSendApi<WM> workerContextApi, S executionStage,
+ WV workerValue) {
+ }
+
+ /**
+ * Function that is called on master, after send phase, before receive phase.
+ *
+ * It can:
+ * - read aggregators sent from worker
+ * - do global processing
+ * - send data to workers through aggregators
+ */
+ public void masterCompute(BlockMasterApi masterApi, S executionStage) {
+ }
+
+ /**
+ * Override to have worker context receive computation.
+ *
+ * Called once per worker, before all vertices are going to be processed
+ * with getVertexReceiver.
+ */
+ public void workerContextReceive(
+ BlockWorkerContextReceiveApi workerContextApi, S executionStage,
+ WV workerValue, List<WM> workerMessages) {
+ }
+
+ /**
+ * Override to do vertex receive processing.
+ *
+ * Creates handler that defines what should be executed on worker
+ * for each vertex during receive phase.
+ *
+ * This logic executed last.
+ * This function is called once on each worker on each thread, in parallel,
+ * on their copy of Piece object to create functions handler.
+ *
+ * If returned object implements Postprocessor interface, then corresponding
+ * postprocess() function is going to be called once, after all vertices
+ * corresponding thread needed to process are done.
+ */
+ public VertexReceiver<I, V, E, M> getVertexReceiver(
+ BlockWorkerReceiveApi<I> workerApi, S executionStage) {
+ return null;
+ }
+
+ /**
+ * Returns MessageClasses definition for messages being sent by this Piece.
+ */
+ public abstract MessageClasses<I, M> getMessageClasses(
+ ImmutableClassesGiraphConfiguration conf);
+
+ /**
+ * Override to provide different next execution stage for
+ * Pieces that come after it.
+ *
+ * Execution stage should be immutable, and this function should be
+ * returning a new object, if it needs to return different value.
+ *
+ * It affects pieces that come after this piece,
+ * and isn't applied to execution stage this piece sees.
+ */
+ public S nextExecutionStage(S executionStage) {
+ return executionStage;
+ }
+
+ /**
+ * Override to register any potential aggregators used by this piece.
+ *
+ * @deprecated Use registerReducers instead.
+ */
+ @Deprecated
+ public void registerAggregators(BlockMasterApi masterApi)
+ throws InstantiationException, IllegalAccessException {
+ }
+
+ // Inner classes
+
+ /** Inner class to provide clean use without specifying types */
+ public abstract class InnerVertexSender
+ implements VertexSender<I, V, E>, VertexPostprocessor {
+ @Override
+ public void postprocess() { }
+ }
+
+ /** Inner class to provide clean use without specifying types */
+ public abstract class InnerVertexReceiver
+ implements VertexReceiver<I, V, E, M>, VertexPostprocessor {
+ @Override
+ public void postprocess() { }
+ }
+
+ // Internal implementation
+
+ @Override
+ public final Iterator<AbstractPiece> iterator() {
+ return Iterators.<AbstractPiece>singletonIterator(this);
+ }
+
+ @Override
+ public void forAllPossiblePieces(Consumer<AbstractPiece> consumer) {
+ consumer.apply(this);
+ }
+
+ @Override
+ public String toString() {
+ String name = getClass().getSimpleName();
+ if (name.isEmpty()) {
+ name = getClass().getName();
+ }
+ return name;
+ }
+
+
+ // make hashCode and equals final, forcing them to be based on
+ // reference identity.
+ @Override
+ public final int hashCode() {
+ return super.hashCode();
+ }
+
+ @Override
+ public final boolean equals(Object obj) {
+ return super.equals(obj);
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/DefaultParentPiece.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/DefaultParentPiece.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/DefaultParentPiece.java
new file mode 100644
index 0000000..0963efb
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/DefaultParentPiece.java
@@ -0,0 +1,311 @@
+/*
+ * 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.giraph.block_app.framework.piece;
+
+import org.apache.giraph.block_app.framework.api.BlockMasterApi;
+import org.apache.giraph.block_app.framework.api.BlockWorkerSendApi;
+import org.apache.giraph.block_app.framework.api.CreateReducersApi;
+import org.apache.giraph.block_app.framework.piece.global_comm.ReduceUtilsObject;
+import org.apache.giraph.block_app.framework.piece.global_comm.ReducerHandle;
+import org.apache.giraph.block_app.framework.piece.global_comm.internal.CreateReducersApiWrapper;
+import org.apache.giraph.block_app.framework.piece.global_comm.internal.ReducersForPieceHandler;
+import org.apache.giraph.block_app.framework.piece.interfaces.VertexPostprocessor;
+import org.apache.giraph.block_app.framework.piece.interfaces.VertexSender;
+import org.apache.giraph.block_app.framework.piece.messages.ObjectMessageClasses;
+import org.apache.giraph.block_app.framework.piece.messages.SupplierFromConf;
+import org.apache.giraph.block_app.framework.piece.messages.SupplierFromConf.DefaultMessageFactorySupplierFromConf;
+import org.apache.giraph.block_app.framework.piece.messages.SupplierFromConf.SupplierFromConfByCopy;
+import org.apache.giraph.combiner.MessageCombiner;
+import org.apache.giraph.comm.messages.MessageEncodeAndStoreType;
+import org.apache.giraph.conf.EnumConfOption;
+import org.apache.giraph.conf.GiraphConfigurationSettable;
+import org.apache.giraph.conf.GiraphConstants;
+import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
+import org.apache.giraph.conf.MessageClasses;
+import org.apache.giraph.factories.MessageValueFactory;
+import org.apache.giraph.graph.Vertex;
+import org.apache.giraph.types.NoMessage;
+import org.apache.hadoop.io.DoubleWritable;
+import org.apache.hadoop.io.FloatWritable;
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * Additional abstract implementations for all pieces to be used.
+ * Code here is not in AbstractPiece only to allow for non-standard
+ * non-user-defined pieces. <br>
+ * Only logic used by the underlying framework directly is in AbstractPiece
+ * itself.
+ *
+ * @param <I> Vertex id type
+ * @param <V> Vertex value type
+ * @param <E> Edge value type
+ * @param <M> Message type
+ * @param <WV> Worker value type
+ * @param <WM> Worker message type
+ * @param <S> Execution stage type
+ */
+@SuppressWarnings({ "rawtypes", "unchecked" })
+public abstract class DefaultParentPiece<I extends WritableComparable,
+ V extends Writable, E extends Writable, M extends Writable, WV,
+ WM extends Writable, S> extends AbstractPiece<I, V, E, M, WV, WM, S> {
+ // TODO move to GiraphConstants
+ /**
+ * This option will tell which message encode & store enum to force, when
+ * combining is not enabled.
+ *
+ * MESSAGE_ENCODE_AND_STORE_TYPE and this property are basically upper
+ * and lower bound on message store type, when looking them in order from
+ * not doing anything special, to most advanced type:
+ * BYTEARRAY_PER_PARTITION,
+ * EXTRACT_BYTEARRAY_PER_PARTITION,
+ * POINTER_LIST_PER_VERTEX
+ * resulting encode type is going to be:
+ * pieceEncodingType = piece.allowOneMessageToManyIdsEncoding() ?
+ * POINTER_LIST_PER_VERTEX : BYTEARRAY_PER_PARTITION)
+ * Math.max(index(minForce), Math.min(index(maxAllowed), index(pieceType);
+ *
+ * This is useful to force all pieces onto particular message store, even
+ * if they do not overrideallowOneMessageToManyIdsEncoding, though that might
+ * be rarely needed.
+ * This option might be more useful for fully local computation,
+ * where overall job behavior is quite different.
+ */
+ public static final EnumConfOption<MessageEncodeAndStoreType>
+ MESSAGE_ENCODE_AND_STORE_TYPE_MIN_FORCE =
+ EnumConfOption.create("giraph.messageEncodeAndStoreTypeMinForce",
+ MessageEncodeAndStoreType.class,
+ MessageEncodeAndStoreType.BYTEARRAY_PER_PARTITION,
+ "Select the message_encode_and_store_type min force to use");
+
+ private final ReduceUtilsObject reduceUtils = new ReduceUtilsObject();
+ private ReducersForPieceHandler reducersHandler;
+
+ // Overridable functions
+
+ /**
+ * Override to register any potential reducers used by this piece,
+ * through calls to {@code reduceApi}, which will return reducer handles
+ * for simple.
+ * <br/>
+ * Tip: Without defining a field, first write here name of the field and what
+ * you want to reduce, like:
+ * <br/>
+ * {@code totalSum = reduceApi.createLocalReducer(SumReduce.DOUBLE); }
+ * <br/>
+ * and then use tools your IDE provides to generate field signature itself,
+ * which might be slightly complex:
+ * <br/>
+ * {@code ReducerHandle<DoubleWritable, DoubleWritable> totalSum; }
+ */
+ public void registerReducers(CreateReducersApi reduceApi, S executionStage) {
+ }
+
+ /**
+ * Override to do vertex send processing.
+ *
+ * Creates handler that defines what should be executed on worker
+ * during send phase.
+ *
+ * This logic gets executed first.
+ * This function is called once on each worker on each thread, in parallel,
+ * on their copy of Piece object to create functions handler.
+ *
+ * If returned object implements Postprocessor interface, then corresponding
+ * postprocess() function is going to be called once, after all vertices
+ * corresponding thread needed to process are done.
+ */
+ public VertexSender<I, V, E> getVertexSender(
+ BlockWorkerSendApi<I, V, E, M> workerApi, S executionStage) {
+ return null;
+ }
+
+ /**
+ * Override to specify type of the message this Piece sends, if it does
+ * send messages.
+ *
+ * If not overwritten, no messages can be sent.
+ */
+ protected Class<M> getMessageClass() {
+ return null;
+ }
+
+ /**
+ * Override to specify message value factory to be used,
+ * which creates objects into which messages will be deserialized.
+ *
+ * If not overwritten, or null is returned, DefaultMessageValueFactory
+ * will be used.
+ */
+ protected MessageValueFactory<M> getMessageFactory(
+ ImmutableClassesGiraphConfiguration conf) {
+ return null;
+ }
+
+ /**
+ * Override to specify message combiner to be used, if any.
+ *
+ * Message combiner itself should be immutable
+ * (i.e. it will be call simultanously from multiple threads)
+ */
+ protected MessageCombiner<? super I, M> getMessageCombiner(
+ ImmutableClassesGiraphConfiguration conf) {
+ return null;
+ }
+
+ /**
+ * Override to specify that this Piece allows one to many ids encoding to be
+ * used for messages.
+ * You should override this function, if you are sending identical message to
+ * all targets, and message itself is not extremely small.
+ */
+ protected boolean allowOneMessageToManyIdsEncoding() {
+ return false;
+ }
+
+ @Override
+ public MessageClasses<I, M> getMessageClasses(
+ ImmutableClassesGiraphConfiguration conf) {
+ Class<M> messageClass = null;
+ MessageValueFactory<M> messageFactory = getMessageFactory(conf);
+ MessageCombiner<? super I, M> messageCombiner = getMessageCombiner(conf);
+
+ if (messageFactory != null) {
+ messageClass = (Class) messageFactory.newInstance().getClass();
+ } else if (messageCombiner != null) {
+ messageClass = (Class) messageCombiner.createInitialMessage().getClass();
+ }
+
+ if (messageClass != null) {
+ Preconditions.checkState(getMessageClass() == null,
+ "Piece %s defines getMessageFactory or getMessageCombiner, " +
+ "so it doesn't need to define getMessageClass.",
+ toString());
+ } else {
+ messageClass = getMessageClass();
+ if (messageClass == null) {
+ messageClass = (Class) NoMessage.class;
+ }
+ }
+
+ SupplierFromConf<MessageValueFactory<M>> messageFactorySupplier;
+ if (messageFactory != null) {
+ messageFactorySupplier =
+ new SupplierFromConfByCopy<MessageValueFactory<M>>(messageFactory);
+ } else {
+ messageFactorySupplier =
+ new DefaultMessageFactorySupplierFromConf<>(messageClass);
+ }
+
+ SupplierFromConf<? extends MessageCombiner<? super I, M>>
+ messageCombinerSupplier;
+ if (messageCombiner != null) {
+ messageCombinerSupplier = new SupplierFromConfByCopy<>(messageCombiner);
+ } else {
+ messageCombinerSupplier = null;
+ }
+
+ int maxAllowed =
+ GiraphConstants.MESSAGE_ENCODE_AND_STORE_TYPE.get(conf).ordinal();
+ int minForce = MESSAGE_ENCODE_AND_STORE_TYPE_MIN_FORCE.get(conf).ordinal();
+ Preconditions.checkState(maxAllowed >= minForce);
+
+ int pieceEncodeType = (allowOneMessageToManyIdsEncoding() ?
+ MessageEncodeAndStoreType.POINTER_LIST_PER_VERTEX :
+ MessageEncodeAndStoreType.BYTEARRAY_PER_PARTITION).ordinal();
+ // bound piece type with boundaries:
+ pieceEncodeType = Math.max(minForce, Math.min(maxAllowed, pieceEncodeType));
+
+ MessageEncodeAndStoreType messageEncodeAndStoreType =
+ MessageEncodeAndStoreType.values()[pieceEncodeType];
+
+ if (messageFactory instanceof GiraphConfigurationSettable) {
+ throw new IllegalStateException(
+ messageFactory.getClass() + " MessageFactory in " + this +
+ " Piece implements GiraphConfigurationSettable");
+ }
+ if (messageCombiner instanceof GiraphConfigurationSettable) {
+ throw new IllegalStateException(
+ messageCombiner.getClass() + " MessageCombiner in " + this +
+ " Piece implements GiraphConfigurationSettable");
+ }
+
+ return new ObjectMessageClasses<>(
+ messageClass, messageFactorySupplier,
+ messageCombinerSupplier, messageEncodeAndStoreType);
+ }
+
+ // Internal implementation
+
+ @Override
+ public final InnerVertexSender getWrappedVertexSender(
+ final BlockWorkerSendApi<I, V, E, M> workerApi, S executionStage) {
+ reducersHandler.vertexSenderWorkerPreprocess(workerApi);
+ final VertexSender<I, V, E> functions =
+ getVertexSender(workerApi, executionStage);
+ return new InnerVertexSender() {
+ @Override
+ public void vertexSend(Vertex<I, V, E> vertex) {
+ if (functions != null) {
+ functions.vertexSend(vertex);
+ }
+ }
+ @Override
+ public void postprocess() {
+ if (functions instanceof VertexPostprocessor) {
+ ((VertexPostprocessor) functions).postprocess();
+ }
+ reducersHandler.vertexSenderWorkerPostprocess(workerApi);
+ }
+ };
+ }
+
+ @Override
+ public final void wrappedRegisterReducers(
+ BlockMasterApi masterApi, S executionStage) {
+ reducersHandler = new ReducersForPieceHandler();
+ registerReducers(new CreateReducersApiWrapper(
+ masterApi, reducersHandler), executionStage);
+ }
+
+ // utility functions:
+ // TODO Java8 - move these as default functions to VertexSender interface
+ protected final void reduceDouble(
+ ReducerHandle<DoubleWritable, ?> reduceHandle, double value) {
+ reduceUtils.reduceDouble(reduceHandle, value);
+ }
+
+ protected final void reduceFloat(
+ ReducerHandle<FloatWritable, ?> reduceHandle, float value) {
+ reduceUtils.reduceFloat(reduceHandle, value);
+ }
+
+ protected final void reduceLong(
+ ReducerHandle<LongWritable, ?> reduceHandle, long value) {
+ reduceUtils.reduceLong(reduceHandle, value);
+ }
+
+ protected final void reduceInt(
+ ReducerHandle<IntWritable, ?> reduceHandle, int value) {
+ reduceUtils.reduceInt(reduceHandle, value);
+ }
+}
http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/Piece.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/Piece.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/Piece.java
new file mode 100644
index 0000000..3ad66d1
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/Piece.java
@@ -0,0 +1,59 @@
+/*
+ * 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.giraph.block_app.framework.piece;
+
+import java.util.List;
+
+import org.apache.giraph.block_app.framework.api.BlockWorkerContextReceiveApi;
+import org.apache.giraph.block_app.framework.api.BlockWorkerContextSendApi;
+import org.apache.giraph.types.NoMessage;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
+
+/**
+ * Piece that should be extended in common usecases, when we want to be:
+ * - sending and then receiving messages from vertices
+ * - sending data to be aggregated from workers to master
+ * - sending values from master, via aggregators, to workers
+ *
+ * (basically - we don't want to use WorkerContext)
+ *
+ * @param <I> Vertex id type
+ * @param <V> Vertex value type
+ * @param <E> Edge value type
+ * @param <M> Message type
+ * @param <S> Execution stage type
+ */
+@SuppressWarnings("rawtypes")
+public class Piece<I extends WritableComparable, V extends Writable,
+ E extends Writable, M extends Writable, S>
+ extends DefaultParentPiece<I, V, E, M, Object, NoMessage, S> {
+
+ // Disallowing use of Worker Context functions:
+ @Override
+ public final void workerContextSend(
+ BlockWorkerContextSendApi<NoMessage> workerContextApi,
+ S executionStage, Object workerValue) {
+ }
+
+ @Override
+ public final void workerContextReceive(
+ BlockWorkerContextReceiveApi workerContextApi,
+ S executionStage, Object workerValue, List<NoMessage> workerMessages) {
+ }
+}
http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/PieceWithWorkerContext.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/PieceWithWorkerContext.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/PieceWithWorkerContext.java
new file mode 100644
index 0000000..a5d0c8c
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/PieceWithWorkerContext.java
@@ -0,0 +1,54 @@
+/*
+ * 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.giraph.block_app.framework.piece;
+
+import org.apache.giraph.block_app.framework.api.BlockWorkerApi;
+import org.apache.giraph.block_app.framework.api.BlockWorkerValueAccessor;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
+
+/**
+ * Piece that should be extended when WorkerContext is used.
+ *
+ * @param <I> Vertex id type
+ * @param <V> Vertex value type
+ * @param <E> Edge value type
+ * @param <M> Message type
+ * @param <WV> Worker value type
+ * @param <WM> Worker message type
+ * @param <S> Execution stage type
+ */
+@SuppressWarnings("rawtypes")
+public class PieceWithWorkerContext<I extends WritableComparable,
+ V extends Writable, E extends Writable, M extends Writable, WV,
+ WM extends Writable, S>
+ extends DefaultParentPiece<I, V, E, M, WV, WM, S> {
+
+ /**
+ * Get global worker value.
+ * Value returned can be accessed from may threads, and so all
+ * accesses to it should be done in a thread-safe manner!
+ *
+ * This is the only place in Blocks Framework where you need
+ * to take care of concurrency.
+ */
+ @SuppressWarnings("unchecked")
+ public WV getWorkerValue(BlockWorkerApi<I> workerApi) {
+ return (WV) ((BlockWorkerValueAccessor) workerApi).getWorkerValue();
+ }
+}
http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/delegate/DelegatePiece.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/delegate/DelegatePiece.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/delegate/DelegatePiece.java
new file mode 100644
index 0000000..23c2d29
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/delegate/DelegatePiece.java
@@ -0,0 +1,277 @@
+/*
+ * 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.giraph.block_app.framework.piece.delegate;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.giraph.block_app.framework.api.BlockMasterApi;
+import org.apache.giraph.block_app.framework.api.BlockWorkerContextReceiveApi;
+import org.apache.giraph.block_app.framework.api.BlockWorkerContextSendApi;
+import org.apache.giraph.block_app.framework.api.BlockWorkerReceiveApi;
+import org.apache.giraph.block_app.framework.api.BlockWorkerSendApi;
+import org.apache.giraph.block_app.framework.piece.AbstractPiece;
+import org.apache.giraph.block_app.framework.piece.interfaces.VertexPostprocessor;
+import org.apache.giraph.block_app.framework.piece.interfaces.VertexReceiver;
+import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
+import org.apache.giraph.conf.MessageClasses;
+import org.apache.giraph.function.Consumer;
+import org.apache.giraph.graph.Vertex;
+import org.apache.giraph.types.NoMessage;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * Delegate Piece which allows combining multiple pieces in same iteration:
+ * new DelegatePiece(new LogicPiece(), new StatsPiece())
+ * You should be careful when doing so, since those pieces must not interact,
+ * and only one can send messages.
+ * Execution of any of the Piece methods by the framework is going to trigger
+ * sequential execution of that method on all of the pieces this DelegatePiece
+ * wraps. That means for example, getVertexSender is going to be called on all
+ * pieces before masterCompute is called on all pieces, which is called before
+ * getVertexReceiver on all pieces.
+ *
+ * Also, via overriding, it provides an abstract class for filtering. I.e. if
+ * you want piece that filters out calls to masterCompute, you can have:
+ * new FilterMasterPiece(new LogicPiece()),
+ * with FilterMasterPiece extends DelegatePiece, and only overrides getMaster
+ * function and DelegateMasterPiece class.
+ *
+ * @param <I> Vertex id type
+ * @param <V> Vertex value type
+ * @param <E> Edge value type
+ * @param <M> Message type
+ * @param <WV> Worker value type
+ * @param <WM> Worker message type
+ * @param <S> Execution stage type
+ */
+@SuppressWarnings("rawtypes")
+public class DelegatePiece<I extends WritableComparable, V extends Writable,
+ E extends Writable, M extends Writable, WV, WM extends Writable, S>
+ extends AbstractPiece<I, V, E, M, WV, WM, S> {
+
+ private final List<AbstractPiece<I, V, E, M, WV, WM, S>> innerPieces;
+
+ @SafeVarargs
+ @SuppressWarnings("unchecked")
+ public DelegatePiece(AbstractPiece<? super I, ? super V, ? super E,
+ ? super M, ? super WV, ? super WM, ? super S>... innerPieces) {
+ // Pieces are contravariant, but Java generics cannot express that,
+ // so use unchecked cast inside to allow callers to be typesafe
+ this.innerPieces = new ArrayList(Arrays.asList(innerPieces));
+ }
+
+ @SuppressWarnings("unchecked")
+ public DelegatePiece(AbstractPiece<? super I, ? super V, ? super E,
+ ? super M, ? super WV, ? super WM, ? super S> innerPiece) {
+ // Pieces are contravariant, but Java generics cannot express that,
+ // so use unchecked cast inside to allow callers to be typesafe
+ this.innerPieces = new ArrayList(Arrays.asList(innerPiece));
+ }
+
+ protected DelegateWorkerSendFunctions delegateWorkerSendFunctions(
+ ArrayList<InnerVertexSender> workerSendFunctions,
+ BlockWorkerSendApi<I, V, E, M> workerApi, S executionStage) {
+ return new DelegateWorkerSendFunctions(workerSendFunctions);
+ }
+
+ protected DelegateWorkerReceiveFunctions delegateWorkerReceiveFunctions(
+ ArrayList<VertexReceiver<I, V, E, M>> workerReceiveFunctions,
+ BlockWorkerReceiveApi<I> workerApi, S executionStage) {
+ return new DelegateWorkerReceiveFunctions(workerReceiveFunctions);
+ }
+
+ @Override
+ public InnerVertexSender getWrappedVertexSender(
+ BlockWorkerSendApi<I, V, E, M> workerApi, S executionStage) {
+ ArrayList<InnerVertexSender> workerSendFunctions =
+ new ArrayList<>(innerPieces.size());
+ for (AbstractPiece<I, V, E, M, WV, WM, S> innerPiece : innerPieces) {
+ workerSendFunctions.add(
+ innerPiece.getWrappedVertexSender(workerApi, executionStage));
+ }
+ return delegateWorkerSendFunctions(
+ workerSendFunctions, workerApi, executionStage);
+ }
+
+ @Override
+ public InnerVertexReceiver getVertexReceiver(
+ BlockWorkerReceiveApi<I> workerApi, S executionStage) {
+ ArrayList<VertexReceiver<I, V, E, M>> workerReceiveFunctions =
+ new ArrayList<>(innerPieces.size());
+ for (AbstractPiece<I, V, E, M, WV, WM, S> innerPiece : innerPieces) {
+ workerReceiveFunctions.add(
+ innerPiece.getVertexReceiver(workerApi, executionStage));
+ }
+ return delegateWorkerReceiveFunctions(
+ workerReceiveFunctions, workerApi, executionStage);
+ }
+
+ /** Delegating WorkerSendPiece */
+ protected class DelegateWorkerSendFunctions extends InnerVertexSender {
+ private final ArrayList<InnerVertexSender> workerSendFunctions;
+
+ public DelegateWorkerSendFunctions(
+ ArrayList<InnerVertexSender> workerSendFunctions) {
+ this.workerSendFunctions = workerSendFunctions;
+ }
+
+ @Override
+ public void vertexSend(Vertex<I, V, E> vertex) {
+ for (InnerVertexSender functions : workerSendFunctions) {
+ if (functions != null) {
+ functions.vertexSend(vertex);
+ }
+ }
+ }
+
+ @Override
+ public void postprocess() {
+ for (InnerVertexSender functions : workerSendFunctions) {
+ if (functions != null) {
+ functions.postprocess();
+ }
+ }
+ }
+ }
+
+ /** Delegating WorkerReceivePiece */
+ protected class DelegateWorkerReceiveFunctions extends InnerVertexReceiver {
+ private final ArrayList<VertexReceiver<I, V, E, M>>
+ workerReceiveFunctions;
+
+ public DelegateWorkerReceiveFunctions(
+ ArrayList<VertexReceiver<I, V, E, M>> workerReceiveFunctions) {
+ this.workerReceiveFunctions = workerReceiveFunctions;
+ }
+
+ @Override
+ public void vertexReceive(Vertex<I, V, E> vertex, Iterable<M> messages) {
+ for (VertexReceiver<I, V, E, M> functions :
+ workerReceiveFunctions) {
+ if (functions != null) {
+ functions.vertexReceive(vertex, messages);
+ }
+ }
+ }
+
+ @Override
+ public void postprocess() {
+ for (VertexReceiver<I, V, E, M> functions :
+ workerReceiveFunctions) {
+ if (functions instanceof VertexPostprocessor) {
+ ((VertexPostprocessor) functions).postprocess();
+ }
+ }
+ }
+ }
+
+ @Override
+ public void masterCompute(BlockMasterApi api, S executionStage) {
+ for (AbstractPiece<I, V, E, M, WV, WM, S> piece : innerPieces) {
+ piece.masterCompute(api, executionStage);
+ }
+ }
+
+ @Override
+ public void workerContextSend(
+ BlockWorkerContextSendApi<WM> workerContextApi, S executionStage,
+ WV workerValue) {
+ for (AbstractPiece<I, V, E, M, WV, WM, S> piece : innerPieces) {
+ piece.workerContextSend(workerContextApi, executionStage, workerValue);
+ }
+ }
+
+ @Override
+ public void workerContextReceive(
+ BlockWorkerContextReceiveApi workerContextApi, S executionStage,
+ WV workerValue, List<WM> workerMessages) {
+ for (AbstractPiece<I, V, E, M, WV, WM, S> piece : innerPieces) {
+ piece.workerContextReceive(
+ workerContextApi, executionStage, workerValue, workerMessages);
+ }
+ }
+
+ @Override
+ public S nextExecutionStage(S executionStage) {
+ for (AbstractPiece<I, V, E, M, WV, WM, S> innerPiece : innerPieces) {
+ executionStage = innerPiece.nextExecutionStage(executionStage);
+ }
+ return executionStage;
+ }
+
+ @Override
+ public MessageClasses<I, M> getMessageClasses(
+ ImmutableClassesGiraphConfiguration conf) {
+ MessageClasses<I, M> messageClasses = null;
+ MessageClasses<I, M> firstMessageClasses = null;
+ for (AbstractPiece<I, V, E, M, WV, WM, S> innerPiece : innerPieces) {
+ MessageClasses<I, M> cur = innerPiece.getMessageClasses(conf);
+ Preconditions.checkState(cur != null);
+ if (!cur.getMessageClass().equals(NoMessage.class)) {
+ if (messageClasses != null) {
+ throw new RuntimeException(
+ "Only one piece combined through delegate (" +
+ toString() + ") can send messages");
+ }
+ messageClasses = cur;
+ }
+ if (firstMessageClasses == null) {
+ firstMessageClasses = cur;
+ }
+ }
+ return messageClasses != null ? messageClasses : firstMessageClasses;
+ }
+
+ @Override
+ public void forAllPossiblePieces(Consumer<AbstractPiece> consumer) {
+ for (AbstractPiece<I, V, E, M, WV, WM, S> innerPiece : innerPieces) {
+ innerPiece.forAllPossiblePieces(consumer);
+ }
+ }
+
+ @SuppressWarnings("deprecation")
+ @Override
+ public void registerAggregators(BlockMasterApi master)
+ throws InstantiationException, IllegalAccessException {
+ for (AbstractPiece<I, V, E, M, WV, WM, S> innerPiece : innerPieces) {
+ innerPiece.registerAggregators(master);
+ }
+ }
+
+ @Override
+ public void wrappedRegisterReducers(
+ BlockMasterApi masterApi, S executionStage) {
+ for (AbstractPiece<I, V, E, M, WV, WM, S> innerPiece : innerPieces) {
+ innerPiece.wrappedRegisterReducers(masterApi, executionStage);
+ }
+ }
+
+ protected String delegationName() {
+ return "Delegate";
+ }
+
+ @Override
+ public String toString() {
+ return delegationName() + innerPieces.toString();
+ }
+}
http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/delegate/FilteringPiece.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/delegate/FilteringPiece.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/delegate/FilteringPiece.java
new file mode 100644
index 0000000..5c702c5
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/delegate/FilteringPiece.java
@@ -0,0 +1,157 @@
+/*
+ * 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.giraph.block_app.framework.piece.delegate;
+
+import java.util.ArrayList;
+
+import org.apache.giraph.block_app.framework.api.BlockWorkerReceiveApi;
+import org.apache.giraph.block_app.framework.api.BlockWorkerSendApi;
+import org.apache.giraph.block_app.framework.piece.AbstractPiece;
+import org.apache.giraph.block_app.framework.piece.interfaces.VertexReceiver;
+import org.apache.giraph.function.vertex.SupplierFromVertex;
+import org.apache.giraph.graph.Vertex;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * Piece which uses a provided suppliers to decide whether or not to run
+ * receive/send piece part on a certain vertex.
+ *
+ * @param <I> Vertex id type
+ * @param <V> Vertex value type
+ * @param <E> Edge value type
+ * @param <M> Message type
+ * @param <WV> Worker value type
+ * @param <WM> Worker message type
+ * @param <S> Execution stage type
+ */
+@SuppressWarnings({ "rawtypes" })
+public class FilteringPiece<I extends WritableComparable, V extends Writable,
+ E extends Writable, M extends Writable, WV, WM extends Writable, S>
+ extends DelegatePiece<I, V, E, M, WV, WM, S> {
+ private final SupplierFromVertex<I, V, E, Boolean> toCallSend;
+ private final SupplierFromVertex<I, V, E, Boolean> toCallReceive;
+
+ /**
+ * Creates filtering piece which uses passed {@code toCallSend} to filter
+ * calls to {@code vertexSend}, and passed {@code toCallReceive} to filter
+ * calls to {@code vertexReceive}, on passed {@code innerPiece}.
+ */
+ @SuppressWarnings("unchecked")
+ public FilteringPiece(
+ SupplierFromVertex<? super I, ? super V, ? super E, Boolean> toCallSend,
+ SupplierFromVertex<? super I, ? super V, ? super E, Boolean>
+ toCallReceive,
+ AbstractPiece<? super I, ? super V, ? super E, ? super M,
+ ? super WV, ? super WM, ? super S> innerPiece) {
+ super(innerPiece);
+ // Suppliers are contravariant on vertex types,
+ // but Java generics cannot express that,
+ // so use unchecked cast inside to allow callers to be typesafe
+ this.toCallSend = (SupplierFromVertex) toCallSend;
+ this.toCallReceive = (SupplierFromVertex) toCallReceive;
+ Preconditions.checkArgument(
+ toCallSend != null || toCallReceive != null,
+ "Both send and receive filter cannot be null");
+ }
+
+ /**
+ * Creates filtering piece, where both vertexSend and vertexReceive is
+ * filtered based on same supplier.
+ */
+ public FilteringPiece(
+ SupplierFromVertex<? super I, ? super V, ? super E, Boolean>
+ toCallSendAndReceive,
+ AbstractPiece<? super I, ? super V, ? super E, ? super M,
+ ? super WV, ? super WM, ? super S> innerPiece) {
+ this(toCallSendAndReceive, toCallSendAndReceive, innerPiece);
+ }
+
+ /**
+ * Creates filtering piece, that filters only vertexReceive function,
+ * and always calls vertexSend function.
+ */
+ public static <I extends WritableComparable, V extends Writable,
+ E extends Writable, M extends Writable, WV, WM extends Writable, S>
+ FilteringPiece<I, V, E, M, WV, WM, S> createReceiveFiltering(
+ SupplierFromVertex<? super I, ? super V, ? super E, Boolean>
+ toCallReceive,
+ AbstractPiece<? super I, ? super V, ? super E, ? super M,
+ ? super WV, ? super WM, ? super S> innerPiece) {
+ return new FilteringPiece<>(null, toCallReceive, innerPiece);
+ }
+
+ /**
+ * Creates filtering block, that filters only vertexSend function,
+ * and always calls vertexReceive function.
+ */
+ public static <I extends WritableComparable, V extends Writable,
+ E extends Writable, M extends Writable, WV, WM extends Writable, S>
+ FilteringPiece<I, V, E, M, WV, WM, S> createSendFiltering(
+ SupplierFromVertex<? super I, ? super V, ? super E, Boolean> toCallSend,
+ AbstractPiece<? super I, ? super V, ? super E, ? super M, ? super WV,
+ ? super WM, ? super S> innerPiece) {
+ return new FilteringPiece<>(toCallSend, null, innerPiece);
+ }
+
+ @Override
+ protected DelegateWorkerSendFunctions delegateWorkerSendFunctions(
+ ArrayList<InnerVertexSender> workerSendFunctions,
+ BlockWorkerSendApi<I, V, E, M> workerApi, S executionStage) {
+ return new DelegateWorkerSendFunctions(workerSendFunctions) {
+ @Override
+ public void vertexSend(Vertex<I, V, E> vertex) {
+ if (toCallSend == null || toCallSend.get(vertex)) {
+ super.vertexSend(vertex);
+ }
+ }
+ };
+ }
+
+ @Override
+ protected DelegateWorkerReceiveFunctions delegateWorkerReceiveFunctions(
+ ArrayList<VertexReceiver<I, V, E, M>> workerReceiveFunctions,
+ BlockWorkerReceiveApi<I> workerApi, S executionStage) {
+ return new DelegateWorkerReceiveFunctions(workerReceiveFunctions) {
+ @Override
+ public void vertexReceive(Vertex<I, V, E> vertex, Iterable<M> messages) {
+ if (toCallReceive == null || toCallReceive.get(vertex)) {
+ super.vertexReceive(vertex, messages);
+ }
+ }
+ };
+ }
+
+ @Override
+ protected String delegationName() {
+ if (toCallSend != null && toCallReceive != null) {
+ if (toCallSend != toCallReceive) {
+ return "AsymFilter";
+ }
+ return "Filter";
+ } else if (toCallSend != null) {
+ return "SendFilter";
+ } else if (toCallReceive != null) {
+ return "ReceiveFilter";
+ } else {
+ throw new IllegalStateException("Both Send and Receive filters are null");
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/delegate/package-info.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/delegate/package-info.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/delegate/package-info.java
new file mode 100644
index 0000000..f367e6c
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/delegate/package-info.java
@@ -0,0 +1,21 @@
+/*
+ * 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.
+ */
+/**
+ * Pieces that delegate their work to a set of one or multiple other Pieces.
+ */
+package org.apache.giraph.block_app.framework.piece.delegate;
http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/BroadcastHandle.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/BroadcastHandle.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/BroadcastHandle.java
new file mode 100644
index 0000000..f18d1f4
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/BroadcastHandle.java
@@ -0,0 +1,30 @@
+/*
+ * 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.giraph.block_app.framework.piece.global_comm;
+
+import org.apache.giraph.worker.WorkerBroadcastUsage;
+
+/**
+ * Handle to a broadcast.
+ *
+ * @param <T> Value type
+ */
+public interface BroadcastHandle<T> {
+ /** Get broadcasted value */
+ T getBroadcast(WorkerBroadcastUsage worker);
+}
http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/ReduceUtilsObject.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/ReduceUtilsObject.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/ReduceUtilsObject.java
new file mode 100644
index 0000000..bbec1c6
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/ReduceUtilsObject.java
@@ -0,0 +1,62 @@
+/*
+ * 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.giraph.block_app.framework.piece.global_comm;
+
+import org.apache.hadoop.io.DoubleWritable;
+import org.apache.hadoop.io.FloatWritable;
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.io.LongWritable;
+
+/**
+ * Utility object with common primitive reduce operations,
+ * without need to create reusable objects within the piece.
+ */
+public class ReduceUtilsObject {
+ private final DoubleWritable reusableDouble = new DoubleWritable();
+ private final FloatWritable reusableFloat = new FloatWritable();
+ private final LongWritable reusableLong = new LongWritable();
+ private final IntWritable reusableInt = new IntWritable();
+
+ // utility functions:
+ public void reduceDouble(
+ ReducerHandle<DoubleWritable, ?> reduceHandle, double value) {
+ DoubleWritable tmp = reusableDouble;
+ tmp.set(value);
+ reduceHandle.reduce(tmp);
+ }
+
+ public void reduceFloat(
+ ReducerHandle<FloatWritable, ?> reduceHandle, float value) {
+ FloatWritable tmp = reusableFloat;
+ tmp.set(value);
+ reduceHandle.reduce(tmp);
+ }
+
+ public void reduceLong(
+ ReducerHandle<LongWritable, ?> reduceHandle, long value) {
+ LongWritable tmp = reusableLong;
+ tmp.set(value);
+ reduceHandle.reduce(tmp);
+ }
+
+ public void reduceInt(ReducerHandle<IntWritable, ?> reduceHandle, int value) {
+ IntWritable tmp = reusableInt;
+ tmp.set(value);
+ reduceHandle.reduce(tmp);
+ }
+}
http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/ReducerAndBroadcastWrapperHandle.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/ReducerAndBroadcastWrapperHandle.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/ReducerAndBroadcastWrapperHandle.java
new file mode 100644
index 0000000..921c863
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/ReducerAndBroadcastWrapperHandle.java
@@ -0,0 +1,61 @@
+/*
+ * 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.giraph.block_app.framework.piece.global_comm;
+
+import org.apache.giraph.block_app.framework.api.BlockMasterApi;
+import org.apache.giraph.master.MasterGlobalCommUsage;
+import org.apache.giraph.worker.WorkerBroadcastUsage;
+
+/**
+ * Handle that wraps both reducerHandle and broadcastHandle, so callers
+ * don't need to have two fields.
+ *
+ * @param <S> Single value type
+ * @param <R> Reduced value type
+ */
+public class ReducerAndBroadcastWrapperHandle<S, R> {
+ private ReducerHandle<S, R> reducerHandle;
+ private BroadcastHandle<R> broadcastHandle;
+
+ /** Set reducer handle to just registered handle */
+ public void registeredReducer(ReducerHandle<S, R> reducerHandle) {
+ this.reducerHandle = reducerHandle;
+ }
+
+ /** Reduce single value */
+ public void reduce(S valueToReduce) {
+ reducerHandle.reduce(valueToReduce);
+ }
+
+ /** Get reduced value */
+ public R getReducedValue(MasterGlobalCommUsage master) {
+ return reducerHandle.getReducedValue(master);
+ }
+
+ /**
+ * Broadcast reduced value from master
+ */
+ public void broadcastValue(BlockMasterApi master) {
+ broadcastHandle = reducerHandle.broadcastValue(master);
+ }
+
+ /** Get broadcasted value */
+ public R getBroadcast(WorkerBroadcastUsage worker) {
+ return broadcastHandle.getBroadcast(worker);
+ }
+}
http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/ReducerHandle.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/ReducerHandle.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/ReducerHandle.java
new file mode 100644
index 0000000..dae40f2
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/ReducerHandle.java
@@ -0,0 +1,41 @@
+/*
+ * 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.giraph.block_app.framework.piece.global_comm;
+
+import org.apache.giraph.block_app.framework.api.BlockMasterApi;
+import org.apache.giraph.master.MasterGlobalCommUsage;
+
+/**
+ * Handle to a reducer.
+ *
+ * @param <S> Single value type
+ * @param <R> Reduced value type
+ */
+public interface ReducerHandle<S, R> {
+ /** Reduce single value */
+ void reduce(S valueToReduce);
+ /** Get reduced value */
+ R getReducedValue(MasterGlobalCommUsage master);
+
+ /**
+ * Broadcast reduced value from master
+ *
+ * @return Handle to the broadcasted value.
+ */
+ BroadcastHandle<R> broadcastValue(BlockMasterApi master);
+}
http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/array/ArrayHandle.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/array/ArrayHandle.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/array/ArrayHandle.java
new file mode 100644
index 0000000..7ee54cb
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/array/ArrayHandle.java
@@ -0,0 +1,36 @@
+/*
+ * 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.giraph.block_app.framework.piece.global_comm.array;
+
+/**
+ * Handle to array of handles underneath
+ *
+ * @param <T> Value type
+ */
+public interface ArrayHandle<T> {
+ /**
+ * Get value at index.
+ */
+ T get(int index);
+
+ /**
+ * Size of this array if defined up front, or throws
+ * UnsupportedOperationException if size is dynamic
+ */
+ int getStaticSize();
+}
http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/array/BroadcastArrayHandle.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/array/BroadcastArrayHandle.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/array/BroadcastArrayHandle.java
new file mode 100644
index 0000000..bf0d333
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/array/BroadcastArrayHandle.java
@@ -0,0 +1,35 @@
+/*
+ * 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.giraph.block_app.framework.piece.global_comm.array;
+
+import org.apache.giraph.block_app.framework.piece.global_comm.BroadcastHandle;
+import org.apache.giraph.worker.WorkerBroadcastUsage;
+
+/**
+ * Handle to array of broadcasts
+ *
+ * @param <T> Value type
+ */
+public interface BroadcastArrayHandle<T>
+ extends ArrayHandle<BroadcastHandle<T>> {
+
+ /**
+ * Number of elements that were broadcasted.
+ */
+ int getBroadcastedSize(WorkerBroadcastUsage worker);
+}
http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/array/ReducerArrayHandle.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/array/ReducerArrayHandle.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/array/ReducerArrayHandle.java
new file mode 100644
index 0000000..a4b99ac
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/array/ReducerArrayHandle.java
@@ -0,0 +1,43 @@
+/*
+ * 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.giraph.block_app.framework.piece.global_comm.array;
+
+import org.apache.giraph.block_app.framework.api.BlockMasterApi;
+import org.apache.giraph.block_app.framework.piece.global_comm.ReducerHandle;
+
+/**
+ * Handle to array of reducers
+ *
+ * @param <S> Single value type
+ * @param <R> Reduced value type
+ */
+public interface ReducerArrayHandle<S, R>
+ extends ArrayHandle<ReducerHandle<S, R>> {
+
+ /**
+ * Number of elements that were reduced.
+ */
+ int getReducedSize(BlockMasterApi master);
+
+ /**
+ * Broadcast whole array of reducers to master
+ *
+ * @return Handle to the broadcasted array.
+ */
+ BroadcastArrayHandle<R> broadcastValue(BlockMasterApi master);
+}
http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/array/package-info.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/array/package-info.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/array/package-info.java
new file mode 100644
index 0000000..a8beb85
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/array/package-info.java
@@ -0,0 +1,21 @@
+/*
+ * 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.
+ */
+/**
+ * Interfaces representing arrays of individual handles.
+ */
+package org.apache.giraph.block_app.framework.piece.global_comm.array;
http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/internal/CreateReducersApiWrapper.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/internal/CreateReducersApiWrapper.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/internal/CreateReducersApiWrapper.java
new file mode 100644
index 0000000..c2cc0f2
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/internal/CreateReducersApiWrapper.java
@@ -0,0 +1,73 @@
+/*
+ * 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.giraph.block_app.framework.piece.global_comm.internal;
+
+import org.apache.giraph.block_app.framework.api.BlockMasterApi;
+import org.apache.giraph.block_app.framework.api.CreateReducersApi;
+import org.apache.giraph.block_app.framework.piece.global_comm.ReducerHandle;
+import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
+import org.apache.giraph.reducers.ReduceOperation;
+import org.apache.hadoop.io.Writable;
+
+/**
+ * Wrapping masterApi and reducers handler into API for creating reducer
+ * handles.
+ */
+public class CreateReducersApiWrapper implements CreateReducersApi {
+ private final BlockMasterApi masterApi;
+ private final ReducersForPieceHandler reducersApi;
+
+ public CreateReducersApiWrapper(
+ BlockMasterApi masterApi, ReducersForPieceHandler reducersApi) {
+ this.masterApi = masterApi;
+ this.reducersApi = reducersApi;
+ }
+
+ @Override
+ public <S, R extends Writable> ReducerHandle<S, R> createLocalReducer(
+ ReduceOperation<S, R> reduceOp) {
+ return reducersApi.createLocalReducer(
+ masterApi, reduceOp, reduceOp.createInitialValue());
+ }
+
+ @Override
+ public <S, R extends Writable> ReducerHandle<S, R> createLocalReducer(
+ ReduceOperation<S, R> reduceOp, R globalInitialValue) {
+ return reducersApi.createLocalReducer(
+ masterApi, reduceOp, globalInitialValue);
+ }
+
+ @Override
+ public <S, R extends Writable> ReducerHandle<S, R> createGlobalReducer(
+ ReduceOperation<S, R> reduceOp) {
+ return reducersApi.createGlobalReducer(
+ masterApi, reduceOp, reduceOp.createInitialValue());
+ }
+
+ @Override
+ public <S, R extends Writable> ReducerHandle<S, R> createGlobalReducer(
+ ReduceOperation<S, R> reduceOp, R globalInitialValue) {
+ return reducersApi.createGlobalReducer(
+ masterApi, reduceOp, globalInitialValue);
+ }
+
+ @Override
+ public ImmutableClassesGiraphConfiguration<?, ?, ?> getConf() {
+ return masterApi.getConf();
+ }
+}
[4/5] git commit: updated refs/heads/trunk to 819d6d3
Posted by ik...@apache.org.
http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/giraph/BlockMasterApiWrapper.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/giraph/BlockMasterApiWrapper.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/giraph/BlockMasterApiWrapper.java
new file mode 100644
index 0000000..ede6005
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/giraph/BlockMasterApiWrapper.java
@@ -0,0 +1,170 @@
+/*
+ * 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.giraph.block_app.framework.api.giraph;
+
+import org.apache.giraph.aggregators.Aggregator;
+import org.apache.giraph.block_app.framework.api.BlockMasterApi;
+import org.apache.giraph.block_app.framework.api.BlockOutputApi;
+import org.apache.giraph.block_app.framework.api.BlockOutputHandleAccessor;
+import org.apache.giraph.block_app.framework.api.Counter;
+import org.apache.giraph.block_app.framework.output.BlockOutputDesc;
+import org.apache.giraph.block_app.framework.output.BlockOutputHandle;
+import org.apache.giraph.block_app.framework.output.BlockOutputWriter;
+import org.apache.giraph.block_app.framework.piece.global_comm.BroadcastHandle;
+import org.apache.giraph.block_app.framework.piece.global_comm.internal.ReducersForPieceHandler.BroadcastHandleImpl;
+import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
+import org.apache.giraph.master.MasterCompute;
+import org.apache.giraph.reducers.ReduceOperation;
+import org.apache.hadoop.io.Writable;
+
+/**
+ * Giraph implementation of BlockMasterApi, that delegates all calls
+ * to MasterCompute.
+ */
+final class BlockMasterApiWrapper implements BlockMasterApi,
+ BlockOutputApi, BlockOutputHandleAccessor {
+ private final MasterCompute master;
+ private final BlockOutputHandle outputHandle;
+
+ public BlockMasterApiWrapper(MasterCompute master,
+ BlockOutputHandle outputHandle) {
+ this.master = master;
+ this.outputHandle = outputHandle;
+ }
+
+ @Override
+ public ImmutableClassesGiraphConfiguration<?, ?, ?> getConf() {
+ return master.getConf();
+ }
+
+ @Override
+ public void setStatus(String status) {
+ master.getContext().setStatus(status);
+ }
+
+ @Override
+ public void progress() {
+ master.getContext().progress();
+ }
+
+ @Override
+ public Counter getCounter(String group, String name) {
+ final org.apache.hadoop.mapreduce.Counter counter =
+ master.getContext().getCounter(group, name);
+ return new Counter() {
+ @Override
+ public void increment(long incr) {
+ counter.increment(incr);
+ }
+
+ @Override
+ public void setValue(long value) {
+ counter.setValue(value);
+ }
+ };
+ }
+
+ @Override
+ public <R extends Writable> R getReduced(String name) {
+ return master.getReduced(name);
+ }
+
+ @Override
+ public void broadcast(String name, Writable value) {
+ master.broadcast(name, value);
+ }
+
+ @Override
+ public <S, R extends Writable> void registerReducer(
+ String name, ReduceOperation<S, R> reduceOp) {
+ master.registerReducer(name, reduceOp);
+ }
+
+ @Override
+ public <S, R extends Writable> void registerReducer(
+ String name, ReduceOperation<S, R> reduceOp, R globalInitialValue) {
+ master.registerReducer(name, reduceOp, globalInitialValue);
+ }
+
+ @Override
+ public <A extends Writable> A getAggregatedValue(String name) {
+ return master.getAggregatedValue(name);
+ }
+
+ @Override
+ public <A extends Writable>
+ boolean registerAggregator(
+ String name, Class<? extends Aggregator<A>> aggregatorClass
+ ) throws InstantiationException, IllegalAccessException {
+ return master.registerAggregator(name, aggregatorClass);
+ }
+
+ @Override
+ public <A extends Writable>
+ boolean registerPersistentAggregator(
+ String name, Class<? extends Aggregator<A>> aggregatorClass
+ ) throws InstantiationException,
+ IllegalAccessException {
+ return master.registerPersistentAggregator(name, aggregatorClass);
+ }
+
+ @Override
+ public <A extends Writable> void setAggregatedValue(String name, A value) {
+ master.setAggregatedValue(name, value);
+ }
+
+ @Override
+ public <T extends Writable> BroadcastHandle<T> broadcast(T object) {
+ BroadcastHandleImpl<T> handle = new BroadcastHandleImpl<>();
+ master.broadcast(handle.getName(), object);
+ return handle;
+ }
+
+ @Override
+ @Deprecated
+ public long getTotalNumEdges() {
+ return master.getTotalNumEdges();
+ }
+
+ @Override
+ @Deprecated
+ public long getTotalNumVertices() {
+ return master.getTotalNumVertices();
+ }
+
+ @Override
+ public void logToCommandLine(String line) {
+ master.logToCommandLine(line);
+ }
+
+ @Override
+ public <OW extends BlockOutputWriter,
+ OD extends BlockOutputDesc<OW>> OD getOutputDesc(String confOption) {
+ return outputHandle.<OW, OD>getOutputDesc(confOption);
+ }
+
+ @Override
+ public <OW extends BlockOutputWriter> OW getWriter(String confOption) {
+ return outputHandle.getWriter(confOption);
+ }
+
+ @Override
+ public BlockOutputHandle getBlockOutputHandle() {
+ return outputHandle;
+ }
+}
http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/giraph/BlockMasterCompute.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/giraph/BlockMasterCompute.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/giraph/BlockMasterCompute.java
new file mode 100644
index 0000000..69cf9f8
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/giraph/BlockMasterCompute.java
@@ -0,0 +1,72 @@
+/*
+ * 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.giraph.block_app.framework.api.giraph;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.giraph.block_app.framework.internal.BlockMasterLogic;
+import org.apache.giraph.block_app.framework.internal.BlockWorkerPieces;
+import org.apache.giraph.block_app.framework.output.BlockOutputHandle;
+import org.apache.giraph.master.MasterCompute;
+import org.apache.giraph.writable.kryo.KryoWritableWrapper;
+
+/**
+ * MasterCompute class which executes block computation.
+ *
+ * @param <S> Execution stage type
+ */
+public final class BlockMasterCompute<S> extends MasterCompute {
+ private BlockMasterLogic<S> blockMasterLogic = new BlockMasterLogic<>();
+
+ @Override
+ public void initialize() throws InstantiationException,
+ IllegalAccessException {
+ blockMasterLogic.initialize(getConf(), new BlockMasterApiWrapper(this,
+ new BlockOutputHandle(getContext().getJobID().toString(),
+ getConf(), getContext())));
+ }
+
+ @Override
+ public void compute() {
+ BlockWorkerPieces<S> workerPieces =
+ blockMasterLogic.computeNext(getSuperstep());
+ if (workerPieces == null) {
+ haltComputation();
+ } else {
+ BlockWorkerPieces.setNextWorkerPieces(this, workerPieces);
+ }
+ }
+
+ @Override
+ public void write(DataOutput out) throws IOException {
+ new KryoWritableWrapper<>(blockMasterLogic).write(out);
+ }
+
+ @Override
+ public void readFields(DataInput in) throws IOException {
+ KryoWritableWrapper<BlockMasterLogic<S>> object =
+ new KryoWritableWrapper<>();
+ object.readFields(in);
+ blockMasterLogic = object.get();
+ blockMasterLogic.initializeAfterRead(new BlockMasterApiWrapper(this,
+ new BlockOutputHandle(getContext().getJobID().toString(),
+ getConf(), getContext())));
+ }
+}
http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/giraph/BlockWorkerApiWrapper.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/giraph/BlockWorkerApiWrapper.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/giraph/BlockWorkerApiWrapper.java
new file mode 100644
index 0000000..6e839f9
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/giraph/BlockWorkerApiWrapper.java
@@ -0,0 +1,180 @@
+/*
+ * 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.giraph.block_app.framework.api.giraph;
+
+import java.io.IOException;
+import java.util.Iterator;
+
+import org.apache.giraph.block_app.framework.api.BlockOutputApi;
+import org.apache.giraph.block_app.framework.api.BlockWorkerReceiveApi;
+import org.apache.giraph.block_app.framework.api.BlockWorkerSendApi;
+import org.apache.giraph.block_app.framework.api.BlockWorkerValueAccessor;
+import org.apache.giraph.block_app.framework.output.BlockOutputDesc;
+import org.apache.giraph.block_app.framework.output.BlockOutputWriter;
+import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
+import org.apache.giraph.edge.Edge;
+import org.apache.giraph.edge.OutEdges;
+import org.apache.giraph.graph.Computation;
+import org.apache.giraph.graph.Vertex;
+import org.apache.giraph.types.NoMessage;
+import org.apache.giraph.worker.WorkerGlobalCommUsage;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
+
+/**
+ * Giraph implementation of BlockWorkerReceiveApi and BlockWorkerSendAPI,
+ * passing all calls to Computation.
+ *
+ * @param <I> Vertex id type
+ * @param <V> Vertex value type
+ * @param <E> Edge value type
+ * @param <M> Message type
+ */
+@SuppressWarnings("rawtypes")
+final class BlockWorkerApiWrapper<I extends WritableComparable,
+ V extends Writable, E extends Writable, M extends Writable>
+ implements BlockWorkerReceiveApi<I>, BlockWorkerSendApi<I, V, E, M>,
+ BlockWorkerValueAccessor, WorkerGlobalCommUsage, BlockOutputApi {
+ private final Computation<I, V, E, NoMessage, M> worker;
+
+ public BlockWorkerApiWrapper(Computation<I, V, E, NoMessage, M> worker) {
+ this.worker = worker;
+ }
+
+ @Override
+ public ImmutableClassesGiraphConfiguration<I, V, E> getConf() {
+ return worker.getConf();
+ }
+
+ @Override
+ public <A extends Writable> void aggregate(String name, A value) {
+ worker.aggregate(name, value);
+ }
+
+ @Override
+ public <A extends Writable> A getAggregatedValue(String name) {
+ return worker.getAggregatedValue(name);
+ }
+
+ @Override
+ public <B extends Writable> B getBroadcast(String name) {
+ return worker.getBroadcast(name);
+ }
+
+ @Override
+ public void reduce(String name, Object value) {
+ worker.reduce(name, value);
+ }
+
+ @Override
+ public void reduceMerge(String name, Writable value) {
+ worker.reduceMerge(name, value);
+ }
+
+ @Override
+ public void sendMessage(I id, M message) {
+ worker.sendMessage(id, message);
+ }
+
+ @Override
+ public void sendMessageToAllEdges(Vertex<I, V, E> vertex, M message) {
+ worker.sendMessageToAllEdges(vertex, message);
+ }
+
+ @Override
+ public void sendMessageToMultipleEdges(
+ Iterator<I> vertexIdIterator, M message) {
+ worker.sendMessageToMultipleEdges(vertexIdIterator, message);
+ }
+
+ @Override
+ public void addVertexRequest(I id, V value) {
+ try {
+ worker.addVertexRequest(id, value);
+ } catch (IOException e) {
+ throw new RuntimeException(e);
+ }
+ }
+
+ @Override
+ public void addVertexRequest(I id, V value, OutEdges<I, E> edges) {
+ try {
+ worker.addVertexRequest(id, value, edges);
+ } catch (IOException e) {
+ throw new RuntimeException(e);
+ }
+ }
+
+ @Override
+ public void removeVertexRequest(I vertexId) {
+ try {
+ worker.removeVertexRequest(vertexId);
+ } catch (IOException e) {
+ throw new RuntimeException(e);
+ }
+ }
+
+ @Override
+ public void addEdgeRequest(I sourceVertexId, Edge<I, E> edge) {
+ try {
+ worker.addEdgeRequest(sourceVertexId, edge);
+ } catch (IOException e) {
+ throw new RuntimeException(e);
+ }
+ }
+
+ @Override
+ public void removeEdgesRequest(I sourceVertexId, I targetVertexId) {
+ try {
+ worker.removeEdgesRequest(sourceVertexId, targetVertexId);
+ } catch (IOException e) {
+ throw new RuntimeException(e);
+ }
+ }
+
+ private BlockWorkerContext getBlockWorkerContext() {
+ return (BlockWorkerContext) worker.getWorkerContext();
+ }
+
+ @Override
+ public Object getWorkerValue() {
+ return getBlockWorkerContext().getWorkerValue();
+ }
+
+ @Override
+ public long getTotalNumEdges() {
+ return worker.getTotalNumEdges();
+ }
+
+ @Override
+ public long getTotalNumVertices() {
+ return worker.getTotalNumVertices();
+ }
+
+ @Override
+ public <OW extends BlockOutputWriter, OD extends BlockOutputDesc<OW>>
+ OD getOutputDesc(String confOption) {
+ return getBlockWorkerContext().getOutputHandle().<OW, OD>getOutputDesc(
+ confOption);
+ }
+
+ @Override
+ public <OW extends BlockOutputWriter> OW getWriter(String confOption) {
+ return getBlockWorkerContext().getOutputHandle().getWriter(confOption);
+ }
+}
http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/giraph/BlockWorkerContext.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/giraph/BlockWorkerContext.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/giraph/BlockWorkerContext.java
new file mode 100644
index 0000000..1a4f8d8
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/giraph/BlockWorkerContext.java
@@ -0,0 +1,102 @@
+/*
+ * 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.giraph.block_app.framework.api.giraph;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.giraph.block_app.framework.internal.BlockWorkerContextLogic;
+import org.apache.giraph.block_app.framework.internal.BlockWorkerPieces;
+import org.apache.giraph.block_app.framework.output.BlockOutputHandle;
+import org.apache.giraph.worker.WorkerContext;
+import org.apache.giraph.writable.kryo.HadoopKryo;
+import org.apache.giraph.writable.kryo.markers.KryoIgnoreWritable;
+import org.apache.hadoop.io.Writable;
+import org.apache.log4j.Logger;
+
+/**
+ * WorkerContext that executes receiver and sender blocks passed
+ * into BlockWorkerPieces.
+ */
+public final class BlockWorkerContext extends WorkerContext
+ implements KryoIgnoreWritable {
+ public static final Logger LOG = Logger.getLogger(BlockWorkerContext.class);
+
+ private BlockWorkerContextLogic workerLogic;
+
+ @Override
+ public void preApplication()
+ throws InstantiationException, IllegalAccessException {
+ workerLogic = new BlockWorkerContextLogic();
+ workerLogic.preApplication(new BlockWorkerContextApiWrapper<>(this),
+ new BlockOutputHandle(getContext().getJobID().toString(),
+ getConf(), getContext()));
+ }
+
+ @Override
+ public void preSuperstep() {
+ List<Writable> messages = getAndClearMessagesFromOtherWorkers();
+ BlockWorkerContextApiWrapper<Writable> workerApi =
+ new BlockWorkerContextApiWrapper<>(this);
+ BlockWorkerPieces<Object> workerPieces =
+ BlockWorkerPieces.getNextWorkerPieces(this);
+
+ LOG.info("PassedComputation in " + getSuperstep() +
+ " superstep executing " + workerPieces);
+
+ workerLogic.preSuperstep(
+ workerApi, workerApi, workerPieces, getSuperstep(), messages);
+ }
+
+ @Override
+ public void postSuperstep() {
+ workerLogic.postSuperstep();
+ }
+
+ @Override
+ public void postApplication() {
+ workerLogic.postApplication();
+ }
+
+ public Object getWorkerValue() {
+ return workerLogic.getWorkerValue();
+ }
+
+ public BlockOutputHandle getOutputHandle() {
+ return workerLogic.getOutputHandle();
+ }
+
+ // Cannot extend KryoWritable directly, since WorkerContext is
+ // abstract class, not interface... Additionally conf in parent
+ // class cannot be made transient.
+ // So just add serialization of two individual fields.
+ // (and adding KryoIgnoreWritable to avoid wrapping it twice)
+
+ @Override
+ public void write(DataOutput out) throws IOException {
+ HadoopKryo.writeClassAndObject(out, workerLogic);
+ }
+
+ @Override
+ public void readFields(DataInput in) throws IOException {
+ workerLogic = HadoopKryo.readClassAndObject(in);
+ workerLogic.getOutputHandle().initialize(getConf(), getContext());
+ }
+}
http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/giraph/BlockWorkerContextApiWrapper.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/giraph/BlockWorkerContextApiWrapper.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/giraph/BlockWorkerContextApiWrapper.java
new file mode 100644
index 0000000..c52b6a5
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/giraph/BlockWorkerContextApiWrapper.java
@@ -0,0 +1,84 @@
+/*
+ * 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.giraph.block_app.framework.api.giraph;
+
+import org.apache.giraph.block_app.framework.api.BlockWorkerContextReceiveApi;
+import org.apache.giraph.block_app.framework.api.BlockWorkerContextSendApi;
+import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
+import org.apache.giraph.worker.WorkerContext;
+import org.apache.hadoop.io.Writable;
+
+/**
+ * Giraph implementation of BlockWorkerContextReceiveApi and
+ * BlockWorkerContextSendApi, passing all calls to WorkerContext.
+ *
+ * @param <WM> Worker message type
+ */
+final class BlockWorkerContextApiWrapper<WM extends Writable>
+ implements BlockWorkerContextReceiveApi, BlockWorkerContextSendApi<WM> {
+ private final WorkerContext workerContext;
+
+ public BlockWorkerContextApiWrapper(WorkerContext workerContext) {
+ this.workerContext = workerContext;
+ }
+
+ @Override
+ public ImmutableClassesGiraphConfiguration<?, ?, ?> getConf() {
+ return workerContext.getConf();
+ }
+
+ @Override
+ public int getWorkerCount() {
+ return workerContext.getWorkerCount();
+ }
+
+ @Override
+ public int getMyWorkerIndex() {
+ return workerContext.getMyWorkerIndex();
+ }
+
+ @Override
+ public <A extends Writable> A getAggregatedValue(String name) {
+ return workerContext.getAggregatedValue(name);
+ }
+
+ @Override
+ public <A extends Writable> void aggregate(String name, A value) {
+ workerContext.aggregate(name, value);
+ }
+
+ @Override
+ public void sendMessageToWorker(WM message, int workerIndex) {
+ workerContext.sendMessageToWorker(message, workerIndex);
+ }
+
+ @Override
+ public <B extends Writable> B getBroadcast(String name) {
+ return workerContext.getBroadcast(name);
+ }
+
+ @Override
+ public long getTotalNumEdges() {
+ return workerContext.getTotalNumEdges();
+ }
+
+ @Override
+ public long getTotalNumVertices() {
+ return workerContext.getTotalNumVertices();
+ }
+}
http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/giraph/package-info.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/giraph/package-info.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/giraph/package-info.java
new file mode 100644
index 0000000..e20fb8e
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/giraph/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+/**
+ * Giraph implementation of graph processing system API used by
+ * Blocks Framework.
+ */
+package org.apache.giraph.block_app.framework.api.giraph;
http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/package-info.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/package-info.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/package-info.java
new file mode 100644
index 0000000..c10e5d2
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/api/package-info.java
@@ -0,0 +1,36 @@
+/*
+ * 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.
+ */
+/**
+ * Interfaces representing full API to the underlying graph processing system.
+ *
+ * Framework implementation is fully contained within package
+ * org.apache.giraph.block_app.framework, given implementation of interfaces
+ * defined here.
+ *
+ * He have two such implementations:
+ * - one relying on Giraph, distributed graph processing system,
+ * connecting all methods to it's internals
+ * - one having a fully contained local implementation, executing applications
+ * on a single machine. Avoiding overheads of Giraph being distributed,
+ * it allows very efficient evaluation on small graphs, especially useful for
+ * fast unit tests.
+ *
+ * You could potentially use a different graph processing system, to execute
+ * any Block Application, by implementing these interfaces.
+ */
+package org.apache.giraph.block_app.framework.api;
http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/block/Block.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/block/Block.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/block/Block.java
new file mode 100644
index 0000000..6d5287c
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/block/Block.java
@@ -0,0 +1,59 @@
+/*
+ * 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.giraph.block_app.framework.block;
+
+import java.util.Iterator;
+
+import org.apache.giraph.block_app.framework.piece.AbstractPiece;
+import org.apache.giraph.function.Consumer;
+
+/**
+ * Composable unit of execution. Used to combine other Blocks into
+ * bigger units. Each Piece represents a Block itself.
+ *
+ * Execution is represented as an iterator across Pieces.
+ *
+ * The whole application run is represented by a single block at the end.
+ */
+@SuppressWarnings("rawtypes")
+public interface Block extends Iterable<AbstractPiece> {
+ /**
+ * Create iterator representing all pieces needed to be executed
+ * in this block.
+ *
+ * After Iterator.next call returns, master compute of returned Piece is
+ * guaranteed to be called before calling hasNext/next on the iterator.
+ * (allows for iterators logic to depend on the execution dynamically,
+ * and not be only static)
+ */
+ @Override
+ Iterator<AbstractPiece> iterator();
+
+ /**
+ * Calls consumer for each Piece:
+ * - in no particular order
+ * - potentially calling multiple times on same Piece
+ * - even if Piece might never be returned in the iterator
+ * - it will be called at least once for every piece that is
+ * going to be returned by iterator
+ *
+ * Can be used for static analysis/introspection of the block,
+ * without actually executing them.
+ */
+ void forAllPossiblePieces(Consumer<AbstractPiece> consumer);
+}
http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/block/EmptyBlock.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/block/EmptyBlock.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/block/EmptyBlock.java
new file mode 100644
index 0000000..1a57402
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/block/EmptyBlock.java
@@ -0,0 +1,39 @@
+/*
+ * 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.giraph.block_app.framework.block;
+
+import java.util.Collections;
+import java.util.Iterator;
+
+import org.apache.giraph.block_app.framework.piece.AbstractPiece;
+import org.apache.giraph.function.Consumer;
+
+/**
+ * Block without any pieces
+ */
+@SuppressWarnings("rawtypes")
+public final class EmptyBlock implements Block {
+ @Override
+ public Iterator<AbstractPiece> iterator() {
+ return Collections.emptyIterator();
+ }
+
+ @Override
+ public void forAllPossiblePieces(Consumer<AbstractPiece> consumer) {
+ }
+}
http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/block/FilteringBlock.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/block/FilteringBlock.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/block/FilteringBlock.java
new file mode 100644
index 0000000..5631417
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/block/FilteringBlock.java
@@ -0,0 +1,113 @@
+/*
+ * 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.giraph.block_app.framework.block;
+
+import java.util.Iterator;
+
+import org.apache.giraph.block_app.framework.piece.AbstractPiece;
+import org.apache.giraph.block_app.framework.piece.delegate.FilteringPiece;
+import org.apache.giraph.function.Consumer;
+import org.apache.giraph.function.vertex.SupplierFromVertex;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
+
+import com.google.common.base.Function;
+import com.google.common.collect.Iterators;
+
+/**
+ * Block which filters out calls to vertexSend/vertexReceive functions
+ * of all pieces in a given block.
+ * Filtering happens based on toCallSend and toCallReceive suppliers
+ * that are passed in, as every piece is just wrapped with FilteringPiece.
+ *
+ * @param <I> Vertex id type
+ * @param <V> Vertex value type
+ * @param <E> Edge value type
+ */
+@SuppressWarnings({ "rawtypes", "unchecked" })
+public final class FilteringBlock<I extends WritableComparable,
+ V extends Writable, E extends Writable>
+ implements Block {
+ private final SupplierFromVertex<I, V, E, Boolean> toCallSend;
+ private final SupplierFromVertex<I, V, E, Boolean> toCallReceive;
+ private final Block block;
+
+ /**
+ * Creates filtering block which uses passed {@code toCallSend} to filter
+ * calls to {@code vertexSend}, and passed {@code toCallReceive} to filter
+ * calls to {@code vertexReceive}, on all pieces within passed {@code block}.
+ */
+ public FilteringBlock(
+ SupplierFromVertex<I, V, E, Boolean> toCallSend,
+ SupplierFromVertex<I, V, E, Boolean> toCallReceive,
+ Block block) {
+ this.toCallSend = toCallSend;
+ this.toCallReceive = toCallReceive;
+ this.block = block;
+ }
+
+ /**
+ * Creates filtering block, where both vertexSend and vertexReceive is
+ * filtered based on same supplier.
+ */
+ public FilteringBlock(
+ SupplierFromVertex<I, V, E, Boolean> toCallSendAndReceive, Block block) {
+ this(toCallSendAndReceive, toCallSendAndReceive, block);
+ }
+
+ /**
+ * Creates filtering block, that filters only vertexReceive function,
+ * and always calls vertexSend function.
+ */
+ public static
+ <I extends WritableComparable, V extends Writable, E extends Writable>
+ Block createReceiveFiltering(
+ SupplierFromVertex<I, V, E, Boolean> toCallReceive,
+ Block innerBlock) {
+ return new FilteringBlock<>(null, toCallReceive, innerBlock);
+ }
+
+ /**
+ * Creates filtering block, that filters only vertexSend function,
+ * and always calls vertexReceive function.
+ */
+ public static
+ <I extends WritableComparable, V extends Writable, E extends Writable>
+ Block createSendFiltering(
+ SupplierFromVertex<I, V, E, Boolean> toCallSend,
+ Block innerBlock) {
+ return new FilteringBlock<>(toCallSend, null, innerBlock);
+ }
+
+ @Override
+ public Iterator<AbstractPiece> iterator() {
+ return Iterators.transform(
+ block.iterator(),
+ new Function<AbstractPiece, AbstractPiece>() {
+ @Override
+ public AbstractPiece apply(AbstractPiece input) {
+ return new FilteringPiece<>(toCallSend, toCallReceive, input);
+ }
+ });
+ }
+
+ @Override
+ public void forAllPossiblePieces(Consumer<AbstractPiece> consumer) {
+ block.forAllPossiblePieces(consumer);
+ }
+}
http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/block/IfBlock.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/block/IfBlock.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/block/IfBlock.java
new file mode 100644
index 0000000..e73392d
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/block/IfBlock.java
@@ -0,0 +1,70 @@
+/*
+ * 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.giraph.block_app.framework.block;
+
+import java.util.Iterator;
+
+import org.apache.giraph.block_app.framework.piece.AbstractPiece;
+import org.apache.giraph.function.Consumer;
+import org.apache.giraph.function.Supplier;
+
+/**
+ * Block that executes one of two branches based on a condition
+ */
+@SuppressWarnings("rawtypes")
+public final class IfBlock implements Block {
+ private final Block thenBlock;
+ private final Block elseBlock;
+ private final Supplier<Boolean> condition;
+
+ public IfBlock(
+ Supplier<Boolean> condition, Block thenBlock, Block elseBlock) {
+ this.condition = condition;
+ this.thenBlock = thenBlock;
+ this.elseBlock = elseBlock;
+ }
+
+ public IfBlock(Supplier<Boolean> condition, Block thenBlock) {
+ this.condition = condition;
+ this.thenBlock = thenBlock;
+ this.elseBlock = new EmptyBlock();
+ }
+
+ @Override
+ public Iterator<AbstractPiece> iterator() {
+ if (Boolean.TRUE.equals(condition.get())) {
+ return thenBlock.iterator();
+ } else {
+ return elseBlock.iterator();
+ }
+ }
+
+ @Override
+ public void forAllPossiblePieces(Consumer<AbstractPiece> consumer) {
+ thenBlock.forAllPossiblePieces(consumer);
+ elseBlock.forAllPossiblePieces(consumer);
+ }
+
+ @Override
+ public String toString() {
+ if (elseBlock instanceof EmptyBlock) {
+ return "IfBlock(" + thenBlock + ")";
+ }
+ return "IfBlock(" + thenBlock + " , " + elseBlock + ")";
+ }
+}
http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/block/RepeatBlock.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/block/RepeatBlock.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/block/RepeatBlock.java
new file mode 100644
index 0000000..9f4f4a0
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/block/RepeatBlock.java
@@ -0,0 +1,87 @@
+/*
+ * 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.giraph.block_app.framework.block;
+
+import java.util.Collections;
+import java.util.Iterator;
+
+import org.apache.giraph.block_app.framework.piece.AbstractPiece;
+import org.apache.giraph.function.Consumer;
+import org.apache.giraph.function.primitive.IntSupplier;
+
+import com.google.common.collect.Iterables;
+
+/**
+ * Block that repeats another block given number of times.
+ */
+@SuppressWarnings("rawtypes")
+public final class RepeatBlock implements Block {
+ private final Block block;
+ private final IntSupplier repeatTimes;
+
+ public RepeatBlock(final int repeatTimes, Block block) {
+ this.block = block;
+ this.repeatTimes = new IntSupplier() {
+ @Override
+ public int get() {
+ return repeatTimes;
+ }
+ };
+ }
+
+ /**
+ * Creates a repeat block, that before starting execution takes number of
+ * iterations from the given supplier.
+ *
+ * This allows number of iterations to be dynamic, and depend on
+ * execution that happens before.
+ * Note - it doesn't allow for number of repetitions to change during the
+ * loop itself - as it is supplier is called only when this block gets
+ * its turn.
+ */
+ public RepeatBlock(IntSupplier repeatTimes, Block block) {
+ this.block = block;
+ this.repeatTimes = repeatTimes;
+ }
+
+ /**
+ * Create a repeat block that executes unlimited number of times.
+ *
+ * Should rarely be used, as it will cause application never to finish,
+ * unless other unconventional ways of termination are used.
+ */
+ public static Block unlimited(Block block) {
+ return new RepeatBlock(Integer.MAX_VALUE, block);
+ }
+
+ @Override
+ public Iterator<AbstractPiece> iterator() {
+ return Iterables.concat(
+ Collections.nCopies(repeatTimes.get(), block)).iterator();
+ }
+
+ @Override
+ public void forAllPossiblePieces(Consumer<AbstractPiece> consumer) {
+ block.forAllPossiblePieces(consumer);
+ }
+
+ @Override
+ public String toString() {
+ return "RepeatBlock(" + repeatTimes + " * " + block + ")";
+ }
+}
http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/block/RepeatUntilBlock.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/block/RepeatUntilBlock.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/block/RepeatUntilBlock.java
new file mode 100644
index 0000000..13e8833
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/block/RepeatUntilBlock.java
@@ -0,0 +1,83 @@
+/*
+ * 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.giraph.block_app.framework.block;
+
+import java.util.Collections;
+import java.util.Iterator;
+
+import org.apache.giraph.block_app.framework.piece.AbstractPiece;
+import org.apache.giraph.function.Consumer;
+import org.apache.giraph.function.Supplier;
+
+import com.google.common.collect.AbstractIterator;
+import com.google.common.collect.Iterables;
+
+/**
+ * Block that repeats another block until toQuit supplier returns true,
+ * but at most given number of times.
+ *
+ * If toQuit returns true on first run, block is not going
+ * to be executed at all.
+ */
+@SuppressWarnings("rawtypes")
+public final class RepeatUntilBlock implements Block {
+ private final Block block;
+ private final int repeatTimes;
+ private final Supplier<Boolean> toQuit;
+
+ public RepeatUntilBlock(
+ int repeatTimes, Block block, Supplier<Boolean> toQuit) {
+ this.block = block;
+ this.repeatTimes = repeatTimes;
+ this.toQuit = toQuit;
+ }
+
+ /**
+ * Repeat unlimited number of times, until toQuit supplier returns true.
+ */
+ public static Block unlimited(Block block, Supplier<Boolean> toQuit) {
+ return new RepeatUntilBlock(Integer.MAX_VALUE, block, toQuit);
+ }
+
+ @Override
+ public Iterator<AbstractPiece> iterator() {
+ // nCopies uses constant memory, creating a looped list with single element
+ final Iterator<AbstractPiece> repeatIterator =
+ Iterables.concat(Collections.nCopies(repeatTimes, block)).iterator();
+ return new AbstractIterator<AbstractPiece>() {
+ @Override
+ protected AbstractPiece computeNext() {
+ if (Boolean.TRUE.equals(toQuit.get()) || !repeatIterator.hasNext()) {
+ return endOfData();
+ }
+
+ return repeatIterator.next();
+ }
+ };
+ }
+
+ @Override
+ public void forAllPossiblePieces(Consumer<AbstractPiece> consumer) {
+ block.forAllPossiblePieces(consumer);
+ }
+
+ @Override
+ public String toString() {
+ return "RepeatUntilBlock(" + repeatTimes + " * " + block + ")";
+ }
+}
http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/block/SequenceBlock.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/block/SequenceBlock.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/block/SequenceBlock.java
new file mode 100644
index 0000000..d768f0b
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/block/SequenceBlock.java
@@ -0,0 +1,60 @@
+/*
+ * 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.giraph.block_app.framework.block;
+
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.giraph.block_app.framework.piece.AbstractPiece;
+import org.apache.giraph.function.Consumer;
+
+import com.google.common.collect.Iterables;
+
+/**
+ * Block that executes provided blocks sequentially.
+ */
+@SuppressWarnings("rawtypes")
+public final class SequenceBlock implements Block {
+ private final Block[] blocks;
+
+ public SequenceBlock(Block... blocks) {
+ this.blocks = blocks.clone();
+ }
+
+ public SequenceBlock(List<? extends Block> blocks) {
+ this.blocks = blocks.toArray(new Block[blocks.size()]);
+ }
+
+ @Override
+ public Iterator<AbstractPiece> iterator() {
+ return Iterables.concat(Arrays.asList(blocks)).iterator();
+ }
+
+ @Override
+ public void forAllPossiblePieces(Consumer<AbstractPiece> consumer) {
+ for (Block block : blocks) {
+ block.forAllPossiblePieces(consumer);
+ }
+ }
+
+ @Override
+ public String toString() {
+ return "SequenceBlock" + Arrays.toString(blocks);
+ }
+}
http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/block/package-info.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/block/package-info.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/block/package-info.java
new file mode 100644
index 0000000..64adc35
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/block/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+/**
+ * Block interface as a composable unit of execution, and its common
+ * implementations.
+ */
+package org.apache.giraph.block_app.framework.block;
http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/internal/BlockCounters.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/internal/BlockCounters.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/internal/BlockCounters.java
new file mode 100644
index 0000000..6a2fb39
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/internal/BlockCounters.java
@@ -0,0 +1,79 @@
+/*
+ * 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.giraph.block_app.framework.internal;
+
+import java.lang.reflect.Field;
+
+import org.apache.giraph.block_app.framework.api.StatusReporter;
+
+/** Utility class for Blocks Framework related counters */
+public class BlockCounters {
+ public static final String GROUP = "Blocks Framework";
+
+ private BlockCounters() { }
+
+ /**
+ * Takes all fields from stage object, and puts them into counters,
+ * if possible.
+ * Only fields that are convertible to long via widening are set
+ * (i.e. long/int/short/byte)
+ */
+ public static void setStageCounters(
+ String prefix, Object stage, StatusReporter reporter) {
+ if (stage != null && reporter != null) {
+ Class<?> clazz = stage.getClass();
+
+ while (clazz != null) {
+ Field[] fields = clazz.getDeclaredFields();
+
+ Field.setAccessible(fields, true);
+ for (Field field : fields) {
+ try {
+ long value = field.getLong(stage);
+ reporter.getCounter(
+ GROUP, prefix + field.getName()).setValue(value);
+
+ // CHECKSTYLE: stop EmptyBlock - ignore any exceptions
+ } catch (IllegalArgumentException | IllegalAccessException e) {
+ }
+ // CHECKSTYLE: resume EmptyBlock
+ }
+ clazz = clazz.getSuperclass();
+ }
+ }
+ }
+
+ public static void setMasterTimeCounter(
+ PairedPieceAndStage<?> masterPiece, long superstep,
+ long millis, StatusReporter reporter) {
+ reporter.getCounter(
+ GROUP + " Master Timers",
+ String.format(
+ "In %6.1f %s (s)", superstep - 0.5, masterPiece.getPiece())
+ ).setValue(millis / 1000);
+ }
+
+ public static void setWorkerTimeCounter(
+ BlockWorkerPieces<?> workerPieces, long superstep,
+ long millis, StatusReporter reporter) {
+ reporter.getCounter(
+ GROUP + " Worker Timers",
+ String.format("In %6d %s (s)", superstep, workerPieces.toStringShort())
+ ).setValue(millis / 1000);
+ }
+}
http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/internal/BlockMasterLogic.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/internal/BlockMasterLogic.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/internal/BlockMasterLogic.java
new file mode 100644
index 0000000..3b87372
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/internal/BlockMasterLogic.java
@@ -0,0 +1,173 @@
+/*
+ * 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.giraph.block_app.framework.internal;
+
+import java.util.HashSet;
+import java.util.Iterator;
+
+import org.apache.giraph.block_app.framework.BlockFactory;
+import org.apache.giraph.block_app.framework.BlockUtils;
+import org.apache.giraph.block_app.framework.api.BlockMasterApi;
+import org.apache.giraph.block_app.framework.api.BlockOutputHandleAccessor;
+import org.apache.giraph.block_app.framework.block.Block;
+import org.apache.giraph.block_app.framework.piece.AbstractPiece;
+import org.apache.giraph.conf.GiraphConfiguration;
+import org.apache.giraph.function.Consumer;
+import org.apache.log4j.Logger;
+import org.python.google.common.base.Preconditions;
+
+/**
+ * Block execution logic on master, iterating over Pieces of the
+ * application Block, executing master logic, and providing what needs to be
+ * executed on the workers.
+ *
+ * @param <S> Execution stage type
+ */
+@SuppressWarnings("rawtypes")
+public class BlockMasterLogic<S> {
+ private static final Logger LOG = Logger.getLogger(BlockMasterLogic.class);
+
+ private Iterator<AbstractPiece> pieceIterator;
+ private PairedPieceAndStage<S> previousPiece;
+ private transient BlockMasterApi masterApi;
+ private long lastTimestamp = -1;
+ private BlockWorkerPieces previousWorkerPieces;
+ private boolean computationDone;
+
+ public void initialize(
+ GiraphConfiguration conf, final BlockMasterApi masterApi)
+ throws InstantiationException, IllegalAccessException {
+ this.masterApi = masterApi;
+ this.computationDone = false;
+
+ BlockFactory<S> factory = BlockUtils.createBlockFactory(conf);
+ Block executionBlock = factory.createBlock(conf);
+ LOG.info("Executing application - " + executionBlock);
+
+ // We register all possible aggregators at the beginning
+ executionBlock.forAllPossiblePieces(new Consumer<AbstractPiece>() {
+ private final HashSet<AbstractPiece> registeredPieces = new HashSet<>();
+ @SuppressWarnings("deprecation")
+ @Override
+ public void apply(AbstractPiece piece) {
+ // no need to regiser the same piece twice.
+ if (registeredPieces.add(piece)) {
+ try {
+ piece.registerAggregators(masterApi);
+ } catch (InstantiationException | IllegalAccessException e) {
+ throw new RuntimeException(e);
+ }
+ }
+ }
+ });
+
+ pieceIterator = executionBlock.iterator();
+ // Invariant is that ReceiveWorkerPiece of previousPiece has already been
+ // executed and that previousPiece.nextExecutionStage() should be used for
+ // iterating. So passing piece as null, and initial state as current state,
+ // so that nothing get's executed in first half, and calculateNextState
+ // returns initial state.
+ previousPiece = new PairedPieceAndStage<>(
+ null, factory.createExecutionStage(conf));
+ }
+
+ /**
+ * Initialize object after deserializing it.
+ * BlockMasterApi is not serializable, so it is transient, and set via this
+ * method afterwards.
+ */
+ public void initializeAfterRead(BlockMasterApi masterApi) {
+ this.masterApi = masterApi;
+ }
+
+ /**
+ * Executes operations on master (master compute and registering reducers),
+ * and calculates next pieces to be exectued on workers.
+ *
+ * @param superstep Current superstep
+ * @return Next BlockWorkerPieces to be executed on workers, or null
+ * if computation should be halted.
+ */
+ public BlockWorkerPieces<S> computeNext(long superstep) {
+ long beforeMaster = System.currentTimeMillis();
+ if (lastTimestamp != -1) {
+ BlockCounters.setWorkerTimeCounter(
+ previousWorkerPieces, superstep - 1,
+ beforeMaster - lastTimestamp, masterApi);
+ }
+
+ if (previousPiece == null) {
+ postApplication();
+ return null;
+ } else {
+ LOG.info(
+ "Master executing " + previousPiece + ", in superstep " + superstep);
+ previousPiece.masterCompute(masterApi);
+ ((BlockOutputHandleAccessor) masterApi).getBlockOutputHandle().
+ returnAllWriters();
+ long afterMaster = System.currentTimeMillis();
+
+ if (previousPiece.getPiece() != null) {
+ BlockCounters.setMasterTimeCounter(
+ previousPiece, superstep, afterMaster - beforeMaster, masterApi);
+ }
+
+ PairedPieceAndStage<S> nextPiece;
+ if (pieceIterator.hasNext()) {
+ nextPiece = new PairedPieceAndStage<S>(
+ pieceIterator.next(), previousPiece.nextExecutionStage());
+ nextPiece.registerReducers(masterApi);
+ } else {
+ nextPiece = null;
+ }
+ BlockCounters.setStageCounters(
+ "Master finished stage: ", previousPiece.getExecutionStage(),
+ masterApi);
+ LOG.info(
+ "Master passing next " + nextPiece + ", in superstep " + superstep);
+
+ // if there is nothing more to compute, no need for additional superstep
+ // this can only happen if application uses no pieces.
+ BlockWorkerPieces<S> result;
+ if (previousPiece.getPiece() == null && nextPiece == null) {
+ postApplication();
+ result = null;
+ } else {
+ result = new BlockWorkerPieces<>(previousPiece, nextPiece);
+ LOG.info("Master in " + superstep + " superstep passing " +
+ result + " to be executed");
+ }
+
+ previousPiece = nextPiece;
+ lastTimestamp = afterMaster;
+ previousWorkerPieces = result;
+ return result;
+ }
+ }
+
+ /**
+ * Clean up any master state, after application has finished.
+ */
+ private void postApplication() {
+ ((BlockOutputHandleAccessor) masterApi).getBlockOutputHandle().
+ closeAllWriters();
+ Preconditions.checkState(!computationDone);
+ computationDone = true;
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/internal/BlockWorkerContextLogic.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/internal/BlockWorkerContextLogic.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/internal/BlockWorkerContextLogic.java
new file mode 100644
index 0000000..8b8e174
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/internal/BlockWorkerContextLogic.java
@@ -0,0 +1,91 @@
+/*
+ * 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.giraph.block_app.framework.internal;
+
+import java.util.List;
+
+import org.apache.giraph.block_app.framework.BlockUtils;
+import org.apache.giraph.block_app.framework.api.BlockWorkerContextApi;
+import org.apache.giraph.block_app.framework.api.BlockWorkerContextReceiveApi;
+import org.apache.giraph.block_app.framework.api.BlockWorkerContextSendApi;
+import org.apache.giraph.block_app.framework.output.BlockOutputHandle;
+import org.apache.hadoop.io.Writable;
+import org.apache.log4j.Logger;
+
+/**
+ * Block execution logic on WorkerContext.
+ */
+@SuppressWarnings({ "rawtypes" })
+public class BlockWorkerContextLogic {
+ public static final Logger LOG =
+ Logger.getLogger(BlockWorkerContextLogic.class);
+
+ private Object workerValue;
+ private BlockWorkerPieces workerPieces;
+ private BlockOutputHandle outputHandle;
+
+ private transient BlockWorkerContextSendApi sendApi;
+
+ public BlockWorkerContextLogic() {
+ }
+
+ public void preApplication(BlockWorkerContextApi api,
+ BlockOutputHandle outputHandle) {
+ workerValue =
+ BlockUtils.BLOCK_WORKER_CONTEXT_VALUE_CLASS.newInstance(api.getConf());
+ this.outputHandle = outputHandle;
+ }
+
+ public Object getWorkerValue() {
+ return workerValue;
+ }
+
+ public BlockOutputHandle getOutputHandle() {
+ return outputHandle;
+ }
+
+ @SuppressWarnings("unchecked")
+ public void preSuperstep(
+ BlockWorkerContextReceiveApi receiveApi,
+ BlockWorkerContextSendApi sendApi,
+ BlockWorkerPieces workerPieces, long superstep,
+ List<Writable> messages) {
+ LOG.info("Worker executing " + workerPieces + " in " + superstep +
+ " superstep");
+ this.sendApi = sendApi;
+ this.workerPieces = workerPieces;
+ if (workerPieces.getReceiver() != null) {
+ workerPieces.getReceiver().workerContextReceive(
+ receiveApi, workerValue, messages);
+ }
+ }
+
+ public void postSuperstep() {
+ if (workerPieces.getSender() != null) {
+ workerPieces.getSender().workerContextSend(sendApi, workerValue);
+ }
+ workerPieces = null;
+ sendApi = null;
+ outputHandle.returnAllWriters();
+ }
+
+ public void postApplication() {
+ outputHandle.closeAllWriters();
+ // TODO add support through conf for postApplication, if needed.
+ }
+}
http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/internal/BlockWorkerLogic.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/internal/BlockWorkerLogic.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/internal/BlockWorkerLogic.java
new file mode 100644
index 0000000..844160c
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/internal/BlockWorkerLogic.java
@@ -0,0 +1,68 @@
+/*
+ * 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.giraph.block_app.framework.internal;
+
+import org.apache.giraph.block_app.framework.api.BlockWorkerReceiveApi;
+import org.apache.giraph.block_app.framework.api.BlockWorkerSendApi;
+import org.apache.giraph.block_app.framework.piece.AbstractPiece.InnerVertexSender;
+import org.apache.giraph.block_app.framework.piece.interfaces.VertexPostprocessor;
+import org.apache.giraph.block_app.framework.piece.interfaces.VertexReceiver;
+import org.apache.giraph.graph.Vertex;
+
+/**
+ * Block execution logic on workers.
+ */
+@SuppressWarnings({ "rawtypes", "unchecked" })
+public class BlockWorkerLogic {
+ private final BlockWorkerPieces pieces;
+
+ private transient VertexReceiver receiveFunctions;
+ private transient InnerVertexSender sendFunctions;
+
+ public BlockWorkerLogic(BlockWorkerPieces pieces) {
+ this.pieces = pieces;
+ }
+
+ public void preSuperstep(
+ BlockWorkerReceiveApi receiveApi, BlockWorkerSendApi sendApi) {
+ if (pieces.getReceiver() != null) {
+ receiveFunctions = pieces.getReceiver().getVertexReceiver(receiveApi);
+ }
+ if (pieces.getSender() != null) {
+ sendFunctions = pieces.getSender().getVertexSender(sendApi);
+ }
+ }
+
+ public void compute(Vertex vertex, Iterable messages) {
+ if (receiveFunctions != null) {
+ receiveFunctions.vertexReceive(vertex, messages);
+ }
+ if (sendFunctions != null) {
+ sendFunctions.vertexSend(vertex);
+ }
+ }
+
+ public void postSuperstep() {
+ if (receiveFunctions instanceof VertexPostprocessor) {
+ ((VertexPostprocessor) receiveFunctions).postprocess();
+ }
+ if (sendFunctions != null) {
+ sendFunctions.postprocess();
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/internal/BlockWorkerPieces.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/internal/BlockWorkerPieces.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/internal/BlockWorkerPieces.java
new file mode 100644
index 0000000..3b38cfa
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/internal/BlockWorkerPieces.java
@@ -0,0 +1,180 @@
+/*
+ * 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.giraph.block_app.framework.internal;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Objects;
+
+import org.apache.giraph.comm.messages.MessageEncodeAndStoreType;
+import org.apache.giraph.conf.DefaultMessageClasses;
+import org.apache.giraph.conf.GiraphConstants;
+import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
+import org.apache.giraph.conf.MessageClasses;
+import org.apache.giraph.factories.DefaultMessageValueFactory;
+import org.apache.giraph.master.MasterCompute;
+import org.apache.giraph.types.NoMessage;
+import org.apache.giraph.utils.UnsafeReusableByteArrayInput;
+import org.apache.giraph.utils.WritableUtils;
+import org.apache.giraph.worker.WorkerGlobalCommUsage;
+import org.apache.giraph.writable.kryo.KryoWritableWrapper;
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.io.Writable;
+import org.apache.log4j.Logger;
+
+/**
+ * Pair of pieces to be executed on workers in a superstep
+ *
+ * @param <S> Execution stage type
+ */
+@SuppressWarnings({ "rawtypes", "unchecked" })
+public class BlockWorkerPieces<S> {
+ private static final Logger LOG = Logger.getLogger(BlockWorkerPieces.class);
+
+ /** Aggregator holding next worker computation */
+ private static final
+ String NEXT_WORKER_PIECES = "giraph.blocks.next_worker_pieces";
+
+ private final PairedPieceAndStage<S> receiver;
+ private final PairedPieceAndStage<S> sender;
+
+ public BlockWorkerPieces(
+ PairedPieceAndStage<S> receiver, PairedPieceAndStage<S> sender) {
+ this.receiver = receiver;
+ this.sender = sender;
+ }
+
+ public PairedPieceAndStage<S> getReceiver() {
+ return receiver;
+ }
+
+ public PairedPieceAndStage<S> getSender() {
+ return sender;
+ }
+
+ public MessageClasses getOutgoingMessageClasses(
+ ImmutableClassesGiraphConfiguration conf) {
+ MessageClasses messageClasses;
+ if (sender == null || sender.getPiece() == null) {
+ messageClasses = new DefaultMessageClasses(
+ NoMessage.class,
+ DefaultMessageValueFactory.class,
+ null,
+ MessageEncodeAndStoreType.BYTEARRAY_PER_PARTITION);
+ } else {
+ messageClasses = sender.getPiece().getMessageClasses(conf);
+ }
+
+ messageClasses.verifyConsistent(conf);
+ return messageClasses;
+ }
+
+ @Override
+ public String toString() {
+ return "[receiver=" + receiver + ",sender=" + sender + "]";
+ }
+
+ public String toStringShort() {
+ String receiverString =
+ Objects.toString(receiver != null ? receiver.getPiece() : null);
+ String senderString =
+ Objects.toString(sender != null ? sender.getPiece() : null);
+ if (receiverString.equals(senderString)) {
+ return "[receiver&sender=" + receiverString + "]";
+ } else {
+ return "[receiver=" + receiverString + ",sender=" + senderString + "]";
+ }
+ }
+
+ /**
+ * Sets which WorkerComputation is going to be executed in the next superstep.
+ */
+ public static <S> void setNextWorkerPieces(
+ MasterCompute master, BlockWorkerPieces<S> nextWorkerPieces) {
+ Writable toBroadcast = new KryoWritableWrapper<>(nextWorkerPieces);
+ byte[] data = WritableUtils.toByteArrayUnsafe(toBroadcast);
+
+ // TODO: extract splitting logic into common utility
+ int overhead = 4096;
+ int singleSize = Math.max(
+ overhead,
+ GiraphConstants.MAX_MSG_REQUEST_SIZE.get(master.getConf()) - overhead);
+
+ ArrayList<byte[]> splittedData = new ArrayList<>();
+ if (data.length < singleSize) {
+ splittedData.add(data);
+ } else {
+ for (int start = 0; start < data.length; start += singleSize) {
+ splittedData.add(Arrays.copyOfRange(
+ data, start, Math.min(data.length, start + singleSize)));
+ }
+ }
+
+ LOG.info("Next worker piece - total serialized size: " + data.length +
+ ", split into " + splittedData.size());
+ master.getContext().getCounter(
+ "PassedWorker Stats", "total serialized size")
+ .increment(data.length);
+ master.getContext().getCounter(
+ "PassedWorker Stats", "split parts")
+ .increment(splittedData.size());
+
+ master.broadcast(NEXT_WORKER_PIECES, new IntWritable(splittedData.size()));
+
+ for (int i = 0; i < splittedData.size(); i++) {
+ master.broadcast(NEXT_WORKER_PIECES + "_part_" + i,
+ KryoWritableWrapper.wrapIfNeeded(splittedData.get(i)));
+ }
+
+ master.setOutgoingMessageClasses(
+ nextWorkerPieces.getOutgoingMessageClasses(master.getConf()));
+ }
+
+ public static <S> BlockWorkerPieces<S> getNextWorkerPieces(
+ WorkerGlobalCommUsage worker) {
+ int splits = worker.<IntWritable>getBroadcast(NEXT_WORKER_PIECES).get();
+
+ int totalLength = 0;
+ ArrayList<byte[]> splittedData = new ArrayList<>();
+ for (int i = 0; i < splits; i++) {
+ byte[] cur = KryoWritableWrapper.<byte[]>unwrapIfNeeded(
+ worker.getBroadcast(NEXT_WORKER_PIECES + "_part_" + i));
+ splittedData.add(cur);
+ totalLength += cur.length;
+ }
+
+ byte[] merged;
+ if (splits == 1) {
+ merged = splittedData.get(0);
+ } else {
+ merged = new byte[totalLength];
+ int index = 0;
+ for (int i = 0; i < splits; i++) {
+ System.arraycopy(
+ splittedData.get(i), 0, merged, index, splittedData.get(i).length);
+ index += splittedData.get(i).length;
+ }
+ }
+
+ KryoWritableWrapper<BlockWorkerPieces<S>> wrapper =
+ new KryoWritableWrapper<>();
+ WritableUtils.fromByteArrayUnsafe(
+ merged, wrapper, new UnsafeReusableByteArrayInput());
+ return wrapper.get();
+ }
+}
http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/internal/PairedPieceAndStage.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/internal/PairedPieceAndStage.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/internal/PairedPieceAndStage.java
new file mode 100644
index 0000000..90fe83e
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/internal/PairedPieceAndStage.java
@@ -0,0 +1,111 @@
+/*
+ * 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.giraph.block_app.framework.internal;
+
+import java.util.List;
+
+import org.apache.giraph.block_app.framework.api.BlockMasterApi;
+import org.apache.giraph.block_app.framework.api.BlockWorkerContextReceiveApi;
+import org.apache.giraph.block_app.framework.api.BlockWorkerContextSendApi;
+import org.apache.giraph.block_app.framework.api.BlockWorkerReceiveApi;
+import org.apache.giraph.block_app.framework.api.BlockWorkerSendApi;
+import org.apache.giraph.block_app.framework.piece.AbstractPiece;
+import org.apache.giraph.block_app.framework.piece.AbstractPiece.InnerVertexSender;
+import org.apache.giraph.block_app.framework.piece.interfaces.VertexReceiver;
+import org.apache.hadoop.io.Writable;
+
+/**
+ * Object holding piece with it's corresponding execution stage.
+ *
+ * @param <S> Execution stage type
+ */
+@SuppressWarnings({ "rawtypes", "unchecked" })
+public class PairedPieceAndStage<S> {
+ private final AbstractPiece piece;
+ private final S executionStage;
+
+ public PairedPieceAndStage(AbstractPiece piece, S executionStage) {
+ this.piece = piece;
+ this.executionStage = executionStage;
+ }
+
+ public S nextExecutionStage() {
+ // if piece is null, then it cannot change the execution stage
+ return piece != null ?
+ (S) piece.nextExecutionStage(executionStage) : executionStage;
+ }
+
+ public S getExecutionStage() {
+ return executionStage;
+ }
+
+ public void registerReducers(BlockMasterApi masterApi) {
+ if (piece != null) {
+ piece.wrappedRegisterReducers(masterApi, executionStage);
+ }
+ }
+
+ public InnerVertexSender getVertexSender(BlockWorkerSendApi sendApi) {
+ if (piece != null) {
+ return piece.getWrappedVertexSender(sendApi, executionStage);
+ }
+ return null;
+ }
+
+ public void masterCompute(BlockMasterApi masterApi) {
+ if (piece != null) {
+ piece.masterCompute(masterApi, executionStage);
+ }
+ }
+
+ public VertexReceiver getVertexReceiver(
+ BlockWorkerReceiveApi receiveApi) {
+ if (piece != null) {
+ return piece.getVertexReceiver(receiveApi, executionStage);
+ }
+ return null;
+ }
+
+ public void workerContextSend(
+ BlockWorkerContextSendApi workerContextApi, Object workerValue) {
+ if (piece != null) {
+ piece.workerContextSend(workerContextApi, executionStage, workerValue);
+ }
+ }
+
+ public void workerContextReceive(
+ BlockWorkerContextReceiveApi workerContextApi,
+ Object workerValue, List<Writable> workerMessages) {
+ if (piece != null) {
+ piece.workerContextReceive(
+ workerContextApi, executionStage, workerValue, workerMessages);
+ }
+ }
+
+ /**
+ * @return the piece
+ */
+ public AbstractPiece getPiece() {
+ return piece;
+ }
+
+ @Override
+ public String toString() {
+ return "Piece " + piece + " in stage " + executionStage;
+ }
+}
http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/internal/package-info.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/internal/package-info.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/internal/package-info.java
new file mode 100644
index 0000000..3ebe8f7
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/internal/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+/**
+ * Implementation of execution logic, guiding internal execution of
+ * Block Application.
+ */
+package org.apache.giraph.block_app.framework.internal;
http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/output/BlockOutputDesc.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/output/BlockOutputDesc.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/output/BlockOutputDesc.java
new file mode 100644
index 0000000..6f2a3dd
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/output/BlockOutputDesc.java
@@ -0,0 +1,50 @@
+/*
+ * 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.giraph.block_app.framework.output;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.util.Progressable;
+
+/**
+ * Output description
+ *
+ * @param <OW> Writer type
+ */
+public interface BlockOutputDesc<OW extends BlockOutputWriter> {
+ /**
+ * Initialize output and perform any necessary checks
+ *
+ * @param jobIdentifier Unique identifier of the job
+ * @param conf Configuration
+ */
+ void initializeAndCheck(String jobIdentifier, Configuration conf);
+
+ /**
+ * Create writer
+ *
+ * @param conf Configuration
+ * @param hadoopProgressable Progressable to call progress on
+ * @return Writer
+ */
+ OW createOutputWriter(Configuration conf, Progressable hadoopProgressable);
+
+ /**
+ * Commit everything
+ */
+ void commit();
+}
http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/output/BlockOutputFormat.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/output/BlockOutputFormat.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/output/BlockOutputFormat.java
new file mode 100644
index 0000000..818a311
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/output/BlockOutputFormat.java
@@ -0,0 +1,107 @@
+/*
+ * 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.giraph.block_app.framework.output;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.giraph.bsp.BspOutputFormat;
+import org.apache.giraph.conf.GiraphConfiguration;
+import org.apache.giraph.conf.GiraphConstants;
+import org.apache.giraph.conf.StrConfOption;
+import org.apache.giraph.utils.ConfigurationObjectUtils;
+import org.apache.giraph.utils.DefaultOutputCommitter;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.OutputCommitter;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+
+/**
+ * Hadoop output format to use with block output.
+ * It keeps track of all registered outputs, and knows how to create them.
+ */
+public class BlockOutputFormat extends BspOutputFormat {
+ private static final StrConfOption OUTPUT_CONF_OPTIONS = new StrConfOption(
+ "digraph.outputConfOptions", "",
+ "List of conf options for outputs used");
+
+ public static <OD> void addOutputDesc(OD outputDesc, String confOption,
+ GiraphConfiguration conf) {
+ GiraphConstants.HADOOP_OUTPUT_FORMAT_CLASS.set(conf,
+ BlockOutputFormat.class);
+ String currentOutputs = OUTPUT_CONF_OPTIONS.get(conf);
+ if (!currentOutputs.isEmpty()) {
+ currentOutputs = currentOutputs + ",";
+ }
+ OUTPUT_CONF_OPTIONS.set(conf, currentOutputs + confOption);
+ ConfigurationObjectUtils.setObjectKryo(outputDesc, confOption, conf);
+ }
+
+ private static String[] getOutputConfOptions(Configuration conf) {
+ String outputConfOptions = OUTPUT_CONF_OPTIONS.get(conf);
+ return outputConfOptions.isEmpty() ?
+ new String[0] : outputConfOptions.split(",");
+ }
+
+ public static <OW extends BlockOutputWriter, OD extends BlockOutputDesc<OW>>
+ OD createInitAndCheckOutputDesc(String confOption, Configuration conf,
+ String jobIdentifier) {
+ OD outputDesc = ConfigurationObjectUtils.getObjectKryo(confOption, conf);
+ outputDesc.initializeAndCheck(jobIdentifier, conf);
+ return outputDesc;
+ }
+
+ public static Map<String, BlockOutputDesc>
+ createInitAndCheckOutputDescsMap(Configuration conf, String jobIdentifier) {
+ String[] outputConfOptions = getOutputConfOptions(conf);
+ Map<String, BlockOutputDesc> ret = new HashMap<>(outputConfOptions.length);
+ for (String outputConfOption : outputConfOptions) {
+ ret.put(outputConfOption,
+ createInitAndCheckOutputDesc(outputConfOption, conf, jobIdentifier));
+ }
+ return ret;
+ }
+
+ public static Map<String, BlockOutputDesc> createInitAndCheckOutputDescsMap(
+ JobContext jobContext) {
+ return createInitAndCheckOutputDescsMap(jobContext.getConfiguration(),
+ jobContext.getJobID().toString());
+ }
+
+ @Override
+ public void checkOutputSpecs(JobContext jobContext)
+ throws IOException, InterruptedException {
+ createInitAndCheckOutputDescsMap(jobContext);
+ }
+
+ @Override
+ public OutputCommitter getOutputCommitter(
+ TaskAttemptContext context) throws IOException, InterruptedException {
+ return new DefaultOutputCommitter() {
+ @Override
+ public void commit(JobContext jobContext) throws IOException {
+ Map<String, BlockOutputDesc> map =
+ createInitAndCheckOutputDescsMap(jobContext);
+ for (BlockOutputDesc outputDesc : map.values()) {
+ outputDesc.commit();
+ }
+ }
+ };
+ }
+}
[2/5] git commit: updated refs/heads/trunk to 819d6d3
Posted by ik...@apache.org.
http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/internal/ReducersForPieceHandler.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/internal/ReducersForPieceHandler.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/internal/ReducersForPieceHandler.java
new file mode 100644
index 0000000..f06dd89
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/internal/ReducersForPieceHandler.java
@@ -0,0 +1,250 @@
+/*
+ * 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.giraph.block_app.framework.piece.global_comm.internal;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.giraph.block_app.framework.api.BlockMasterApi;
+import org.apache.giraph.block_app.framework.piece.global_comm.BroadcastHandle;
+import org.apache.giraph.block_app.framework.piece.global_comm.ReducerHandle;
+import org.apache.giraph.master.MasterGlobalCommUsage;
+import org.apache.giraph.reducers.ReduceOperation;
+import org.apache.giraph.reducers.Reducer;
+import org.apache.giraph.utils.WritableUtils;
+import org.apache.giraph.worker.WorkerBroadcastUsage;
+import org.apache.giraph.worker.WorkerReduceUsage;
+import org.apache.hadoop.io.Writable;
+
+/**
+ * All logic for transforming Giraph's reducer API to reducer handles.
+ * Contains state of active reducers, and is kept within a Piece.
+ */
+public class ReducersForPieceHandler implements VertexSenderObserver {
+ private static final AtomicInteger HANDLER_COUNTER = new AtomicInteger();
+ private static final AtomicInteger BROADCAST_COUNTER = new AtomicInteger();
+
+ private final int handleIndex = HANDLER_COUNTER.incrementAndGet();
+ private final AtomicInteger reduceCounter = new AtomicInteger();
+
+ private final ArrayList<VertexSenderObserver> observers = new ArrayList<>();
+
+ @Override
+ public void vertexSenderWorkerPreprocess(WorkerReduceUsage usage) {
+ for (VertexSenderObserver observer : observers) {
+ observer.vertexSenderWorkerPreprocess(usage);
+ }
+ }
+
+ @Override
+ public void vertexSenderWorkerPostprocess(WorkerReduceUsage usage) {
+ for (VertexSenderObserver observer : observers) {
+ observer.vertexSenderWorkerPostprocess(usage);
+ }
+ }
+
+ public <S, R extends Writable> ReducerHandle<S, R> createLocalReducer(
+ MasterGlobalCommUsage master, ReduceOperation<S, R> reduceOp,
+ R globalInitialValue) {
+ LocalReduceHandle<S, R> handle = new LocalReduceHandle<>(reduceOp);
+ master.registerReducer(handle.getName(), reduceOp, globalInitialValue);
+ observers.add(handle);
+ return handle;
+ }
+
+ public <S, R extends Writable> ReducerHandle<S, R> createGlobalReducer(
+ MasterGlobalCommUsage master, ReduceOperation<S, R> reduceOp,
+ R globalInitialValue) {
+ ReduceHandleImpl<S, R> handle = new GlobalReduceHandle<>(reduceOp);
+ master.registerReducer(handle.getName(), reduceOp, globalInitialValue);
+ observers.add(handle);
+ return handle;
+ }
+
+ /**
+ * Implementation of BroadcastHandle
+ *
+ * @param <T> Value type
+ */
+ public static class BroadcastHandleImpl<T> implements BroadcastHandle<T> {
+ private final String name;
+
+ public BroadcastHandleImpl() {
+ this.name = "_utils.broadcast." + BROADCAST_COUNTER.incrementAndGet();
+ }
+
+ public String getName() {
+ return name;
+ }
+
+ @Override
+ public T getBroadcast(WorkerBroadcastUsage worker) {
+ return worker.getBroadcast(name);
+ }
+ }
+
+ /**
+ * Parent implementation of ReducerHandle
+ *
+ * @param <S> Single value type
+ * @param <R> Reduced value type
+ */
+ public abstract class ReduceHandleImpl<S, R extends Writable>
+ implements ReducerHandle<S, R>, VertexSenderObserver {
+ protected final ReduceOperation<S, R> reduceOp;
+ private final String name;
+
+ private ReduceHandleImpl(ReduceOperation<S, R> reduceOp) {
+ this.reduceOp = reduceOp;
+ name = "_utils." + handleIndex +
+ ".reduce." + reduceCounter.incrementAndGet();
+ }
+
+ public String getName() {
+ return name;
+ }
+
+ @Override
+ public R getReducedValue(MasterGlobalCommUsage master) {
+ return master.getReduced(name);
+ }
+
+ @Override
+ public BroadcastHandle<R> broadcastValue(BlockMasterApi master) {
+ return unwrapHandle(master.broadcast(
+ new WrappedReducedValue<>(reduceOp, getReducedValue(master))));
+ }
+ }
+
+ private static <R extends Writable> BroadcastHandle<R> unwrapHandle(
+ final BroadcastHandle<WrappedReducedValue<R>> handle) {
+ return new BroadcastHandle<R>() {
+ @Override
+ public R getBroadcast(WorkerBroadcastUsage worker) {
+ return handle.getBroadcast(worker).getValue();
+ }
+ };
+ }
+
+ /**
+ * Wrapper that makes reduced values self-serializable,
+ * and allows them to be broadcasted.
+ *
+ * @param <R> Reduced value type
+ */
+ public static class WrappedReducedValue<R extends Writable>
+ implements Writable {
+ private ReduceOperation<?, R> reduceOp;
+ private R value;
+
+ public WrappedReducedValue() {
+ }
+
+ public WrappedReducedValue(ReduceOperation<?, R> reduceOp, R value) {
+ this.reduceOp = reduceOp;
+ this.value = value;
+ }
+
+ @Override
+ public void write(DataOutput out) throws IOException {
+ WritableUtils.writeWritableObject(reduceOp, out);
+ value.write(out);
+ }
+
+ @Override
+ public void readFields(DataInput in) throws IOException {
+ reduceOp = WritableUtils.readWritableObject(in, null);
+ value = reduceOp.createInitialValue();
+ value.readFields(in);
+ }
+
+ public R getValue() {
+ return value;
+ }
+ }
+
+ /**
+ * Global Reduce Handle is implementation of ReducerHandle, that will keep
+ * only one value for each worker, and each call to reduce will have
+ * to obtain a global lock, and incur synchronization costs.
+ * Use only when objects are so large, that having many copies cannot fit
+ * into memory.
+ *
+ * @param <S> Single value type
+ * @param <R> Reduced value type
+ */
+ public class GlobalReduceHandle<S, R extends Writable>
+ extends ReduceHandleImpl<S, R> {
+ private transient WorkerReduceUsage usage;
+
+ public GlobalReduceHandle(ReduceOperation<S, R> reduceOp) {
+ super(reduceOp);
+ }
+
+ @Override
+ public void vertexSenderWorkerPreprocess(WorkerReduceUsage usage) {
+ this.usage = usage;
+ }
+
+ @Override
+ public void reduce(S valueToReduce) {
+ usage.reduce(getName(), valueToReduce);
+ }
+
+ @Override
+ public void vertexSenderWorkerPostprocess(WorkerReduceUsage usage) {
+ }
+ }
+
+ /**
+ * Local Reduce Handle is implementation of ReducerHandle, that will make a
+ * partially reduced value on each worker thread, which are at the end
+ * reduced all together.
+ * This is preferred implementation, unless it cannot be used due to memory
+ * overhead, because all partially reduced values will not fit the memory.
+ *
+ * @param <S> Single value type
+ * @param <R> Reduced value type
+ */
+ public class LocalReduceHandle<S, R extends Writable>
+ extends ReduceHandleImpl<S, R> {
+ private transient Reducer<S, R> reducer;
+
+ public LocalReduceHandle(ReduceOperation<S, R> reduceOp) {
+ super(reduceOp);
+ }
+
+ @Override
+ public void vertexSenderWorkerPreprocess(WorkerReduceUsage usage) {
+ this.reducer = new Reducer<>(reduceOp);
+ }
+
+ @Override
+ public void reduce(S valueToReduce) {
+ reducer.reduce(valueToReduce);
+ }
+
+ @Override
+ public void vertexSenderWorkerPostprocess(WorkerReduceUsage usage) {
+ usage.reduceMerge(getName(), reducer.getCurrentValue());
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/internal/VertexSenderObserver.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/internal/VertexSenderObserver.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/internal/VertexSenderObserver.java
new file mode 100644
index 0000000..5b3485f
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/internal/VertexSenderObserver.java
@@ -0,0 +1,28 @@
+/*
+ * 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.giraph.block_app.framework.piece.global_comm.internal;
+
+import org.apache.giraph.worker.WorkerReduceUsage;
+
+/**
+ * Observer able to hook into vertex sender pre/post processing
+ */
+public interface VertexSenderObserver {
+ void vertexSenderWorkerPreprocess(WorkerReduceUsage usage);
+ void vertexSenderWorkerPostprocess(WorkerReduceUsage usage);
+}
http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/internal/package-info.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/internal/package-info.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/internal/package-info.java
new file mode 100644
index 0000000..1ba7c8f
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/internal/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+/**
+ * Reducer and Broadcast Handles internal implementation for automatic handling
+ * of global communication within Pieces, hiding a lot of it's complexities.
+ */
+package org.apache.giraph.block_app.framework.piece.global_comm.internal;
http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/map/BroadcastMapHandle.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/map/BroadcastMapHandle.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/map/BroadcastMapHandle.java
new file mode 100644
index 0000000..50d7818
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/map/BroadcastMapHandle.java
@@ -0,0 +1,36 @@
+/*
+ * 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.giraph.block_app.framework.piece.global_comm.map;
+
+import org.apache.giraph.block_app.framework.piece.global_comm.BroadcastHandle;
+import org.apache.giraph.worker.WorkerBroadcastUsage;
+
+/**
+ * Handle to map of broadcasts
+ *
+ * @param <K> Key type
+ * @param <V> Value type
+ */
+public interface BroadcastMapHandle<K, V>
+ extends MapHandle<K, BroadcastHandle<V>> {
+
+ /**
+ * Number of elements that were broadcasted.
+ */
+ int getBroadcastedSize(WorkerBroadcastUsage worker);
+}
http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/map/MapHandle.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/map/MapHandle.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/map/MapHandle.java
new file mode 100644
index 0000000..db01e77
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/map/MapHandle.java
@@ -0,0 +1,31 @@
+/*
+ * 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.giraph.block_app.framework.piece.global_comm.map;
+
+/**
+ * Handle to map of handles underneath
+ *
+ * @param <K> Key type
+ * @param <V> Value type
+ */
+public interface MapHandle<K, V> {
+ /**
+ * Get value for key.
+ */
+ V get(K key);
+}
http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/map/ReducerMapHandle.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/map/ReducerMapHandle.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/map/ReducerMapHandle.java
new file mode 100644
index 0000000..5c31179
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/map/ReducerMapHandle.java
@@ -0,0 +1,43 @@
+/*
+ * 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.giraph.block_app.framework.piece.global_comm.map;
+
+import org.apache.giraph.block_app.framework.api.BlockMasterApi;
+import org.apache.giraph.block_app.framework.piece.global_comm.ReducerHandle;
+
+/**
+ * Handle to array of reducers
+ *
+ * @param <I> Key type
+ * @param <S> Single value type
+ * @param <R> Reduced value type
+ */
+public interface ReducerMapHandle<I, S, R>
+ extends MapHandle<I, ReducerHandle<S, R>> {
+ /**
+ * Number of elements that were reduced.
+ */
+ int getReducedSize(BlockMasterApi master);
+
+ /**
+ * Broadcast whole map of reducers to master
+ *
+ * @return Handle to the broadcasted map.
+ */
+ BroadcastMapHandle<I, R> broadcastValue(BlockMasterApi master);
+}
http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/map/package-info.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/map/package-info.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/map/package-info.java
new file mode 100644
index 0000000..092f864
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/map/package-info.java
@@ -0,0 +1,21 @@
+/*
+ * 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.
+ */
+/**
+ * Interfaces representing map of individual handles
+ */
+package org.apache.giraph.block_app.framework.piece.global_comm.map;
http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/package-info.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/package-info.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/package-info.java
new file mode 100644
index 0000000..0d40741
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/global_comm/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+/**
+ * Interfaces for Reducer and Broadcast Handles for automatic handling
+ * of global communication within Pieces, hiding a lot of it's complexities.
+ */
+package org.apache.giraph.block_app.framework.piece.global_comm;
http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/interfaces/VertexPostprocessor.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/interfaces/VertexPostprocessor.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/interfaces/VertexPostprocessor.java
new file mode 100644
index 0000000..b6cc749
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/interfaces/VertexPostprocessor.java
@@ -0,0 +1,34 @@
+/*
+ * 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.giraph.block_app.framework.piece.interfaces;
+
+import org.apache.giraph.writable.kryo.markers.NonKryoWritable;
+
+/**
+ * Interface containing a single function - postprocess.
+ *
+ * Marked to not allow seriazliation, as it should be created on the worker,
+ * so should never be serialiized, disallow only for catching problems early.
+ */
+public interface VertexPostprocessor extends NonKryoWritable {
+ /**
+ * Override to finish computation. This method is executed exactly once
+ * after computation for all vertices in the partition is complete.
+ */
+ void postprocess();
+}
http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/interfaces/VertexReceiver.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/interfaces/VertexReceiver.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/interfaces/VertexReceiver.java
new file mode 100644
index 0000000..26912ee
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/interfaces/VertexReceiver.java
@@ -0,0 +1,54 @@
+/*
+ * 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.giraph.block_app.framework.piece.interfaces;
+
+import org.apache.giraph.graph.Vertex;
+import org.apache.giraph.writable.kryo.markers.NonKryoWritable;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
+
+/**
+ * Interface representing actions that happen on worker, for each vertex,
+ * during receive phase:
+ * <ul>
+ * <li> to receive messages from vertices </li>
+ * <li> to receive data from master through aggregators </li>
+ * </ul>
+ *
+ * Marked to not allow seriazliation, as it should be created on the worker,
+ * so should never be serialiized, disallow only for catching problems early.
+ *
+ * @param <I> Vertex id type
+ * @param <V> Vertex value type
+ * @param <E> Edge value type
+ * @param <M> Message type
+ */
+@SuppressWarnings("rawtypes")
+public interface VertexReceiver<I extends WritableComparable,
+ V extends Writable, E extends Writable, M extends Writable>
+ extends NonKryoWritable {
+ /**
+ * Must be defined by user to do computation on a single Vertex.
+ *
+ * @param vertex Vertex
+ * @param messages Messages that were sent to this vertex in the previous
+ * superstep. Each message is only guaranteed to have
+ * a life expectancy as long as next() is not called.
+ */
+ void vertexReceive(Vertex<I, V, E> vertex, Iterable<M> messages);
+}
http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/interfaces/VertexSender.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/interfaces/VertexSender.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/interfaces/VertexSender.java
new file mode 100644
index 0000000..0587032
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/interfaces/VertexSender.java
@@ -0,0 +1,45 @@
+/*
+ * 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.giraph.block_app.framework.piece.interfaces;
+
+import org.apache.giraph.graph.Vertex;
+import org.apache.giraph.writable.kryo.markers.NonKryoWritable;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
+
+/**
+ * Interface representing actions that happen on worker, for each vertex,
+ * during send phase:
+ * <ul>
+ * <li> to send messages to vertices </li>
+ * <li> to send data for aggregation on master </li>
+ * </ul>
+ *
+ * Marked to not allow seriazliation, as it should be created on the worker,
+ * so should never be serialiized, disallow only for catching problems early.
+ *
+ * @param <I> Vertex id type
+ * @param <V> Vertex value type
+ * @param <E> Edge value type
+ */
+@SuppressWarnings("rawtypes")
+public interface VertexSender<I extends WritableComparable,
+ V extends Writable, E extends Writable> extends NonKryoWritable {
+ /** Must be defined by user to do computation on a single Vertex. */
+ void vertexSend(Vertex<I, V, E> vertex);
+}
http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/interfaces/package-info.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/interfaces/package-info.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/interfaces/package-info.java
new file mode 100644
index 0000000..db05e78
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/interfaces/package-info.java
@@ -0,0 +1,21 @@
+/*
+ * 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.
+ */
+/**
+ * Vertex processing functions for Pieces
+ */
+package org.apache.giraph.block_app.framework.piece.interfaces;
http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/messages/ObjectMessageClasses.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/messages/ObjectMessageClasses.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/messages/ObjectMessageClasses.java
new file mode 100644
index 0000000..dd977e6
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/messages/ObjectMessageClasses.java
@@ -0,0 +1,119 @@
+/*
+ * 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.giraph.block_app.framework.piece.messages;
+
+import org.apache.giraph.combiner.MessageCombiner;
+import org.apache.giraph.comm.messages.MessageEncodeAndStoreType;
+import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
+import org.apache.giraph.conf.MessageClasses;
+import org.apache.giraph.factories.MessageValueFactory;
+import org.apache.giraph.utils.ReflectionUtils;
+import org.apache.giraph.writable.kryo.KryoWritable;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * MessageClasses implementation that provides factory and combiner instances
+ * through a provided supplier.
+ *
+ * @param <I> Vertex id type
+ * @param <M> Message type
+ */
+public class ObjectMessageClasses<I extends WritableComparable,
+ M extends Writable> extends KryoWritable implements MessageClasses<I, M> {
+ private final Class<M> messageClass;
+ private final SupplierFromConf<MessageValueFactory<M>>
+ messageValueFactorySupplier;
+ private final SupplierFromConf<? extends MessageCombiner<? super I, M>>
+ messageCombinerSupplier;
+ private final MessageEncodeAndStoreType messageEncodeAndStoreType;
+
+ public ObjectMessageClasses() {
+ this(null, null, null, null);
+ }
+
+ public ObjectMessageClasses(Class<M> messageClass,
+ SupplierFromConf<MessageValueFactory<M>> messageValueFactorySupplier,
+ SupplierFromConf<? extends MessageCombiner<? super I, M>>
+ messageCombinerSupplier,
+ MessageEncodeAndStoreType messageEncodeAndStoreType) {
+ this.messageClass = messageClass;
+ this.messageValueFactorySupplier = messageValueFactorySupplier;
+ this.messageCombinerSupplier = messageCombinerSupplier;
+ this.messageEncodeAndStoreType = messageEncodeAndStoreType;
+ }
+
+ @Override
+ public Class<M> getMessageClass() {
+ return messageClass;
+ }
+
+ @Override
+ public MessageValueFactory<M> createMessageValueFactory(
+ ImmutableClassesGiraphConfiguration conf) {
+ return Preconditions.checkNotNull(messageValueFactorySupplier.apply(conf));
+ }
+
+ @Override
+ public MessageCombiner<? super I, M> createMessageCombiner(
+ ImmutableClassesGiraphConfiguration<I, ? extends Writable,
+ ? extends Writable> conf) {
+ return messageCombinerSupplier != null ?
+ Preconditions.checkNotNull(messageCombinerSupplier.apply(conf)) : null;
+ }
+
+ @Override
+ public boolean useMessageCombiner() {
+ return messageCombinerSupplier != null;
+ }
+
+ @Override
+ public MessageEncodeAndStoreType getMessageEncodeAndStoreType() {
+ return messageEncodeAndStoreType;
+ }
+
+ @Override
+ public MessageClasses<I, M> createCopyForNewSuperstep() {
+ return new ObjectMessageClasses<>(
+ messageClass, messageValueFactorySupplier,
+ messageCombinerSupplier, messageEncodeAndStoreType);
+ }
+
+ @Override
+ public void verifyConsistent(ImmutableClassesGiraphConfiguration conf) {
+ MessageValueFactory<M> messageValueFactory =
+ messageValueFactorySupplier.apply(conf);
+ Preconditions.checkState(
+ messageValueFactory.newInstance().getClass().equals(messageClass));
+
+ if (messageCombinerSupplier != null) {
+ MessageCombiner<? super I, M> messageCombiner =
+ messageCombinerSupplier.apply(conf);
+ Preconditions.checkState(messageCombiner.createInitialMessage()
+ .getClass().equals(messageClass));
+ Class<?>[] combinerTypes = ReflectionUtils.getTypeArguments(
+ MessageCombiner.class, messageCombiner.getClass());
+ ReflectionUtils.verifyTypes(conf.getVertexIdClass(), combinerTypes[0],
+ "Vertex id", messageCombiner.getClass());
+ ReflectionUtils.verifyTypes(messageClass, combinerTypes[1],
+ "Outgoing message", messageCombiner.getClass());
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/messages/SupplierFromConf.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/messages/SupplierFromConf.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/messages/SupplierFromConf.java
new file mode 100644
index 0000000..00c86cd
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/messages/SupplierFromConf.java
@@ -0,0 +1,71 @@
+/*
+ * 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.giraph.block_app.framework.piece.messages;
+
+import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
+import org.apache.giraph.factories.DefaultMessageValueFactory;
+import org.apache.giraph.factories.MessageValueFactory;
+import org.apache.giraph.function.Function;
+import org.apache.giraph.writable.kryo.HadoopKryo;
+import org.apache.hadoop.io.Writable;
+
+/**
+ * Supplier from configuration
+ * @param <T> Type of object returned
+ */
+public interface SupplierFromConf<T>
+ extends Function<ImmutableClassesGiraphConfiguration, T> {
+
+ /**
+ * Supplier from configuration, by copying given instance every time.
+ *
+ * @param <T> Type of object returned
+ */
+ public static class SupplierFromConfByCopy<T> implements SupplierFromConf<T> {
+ private final T value;
+
+ public SupplierFromConfByCopy(T value) {
+ this.value = value;
+ }
+
+ @Override
+ public T apply(ImmutableClassesGiraphConfiguration conf) {
+ return HadoopKryo.createCopy(value);
+ }
+ }
+
+ /**
+ * Supplier from configuration returning DefaultMessageValueFactory instances.
+ *
+ * @param <M> Message type
+ */
+ public static class DefaultMessageFactorySupplierFromConf<M extends Writable>
+ implements SupplierFromConf<MessageValueFactory<M>> {
+ private final Class<M> messageClass;
+
+ public DefaultMessageFactorySupplierFromConf(Class<M> messageClass) {
+ this.messageClass = messageClass;
+ }
+
+ @Override
+ public MessageValueFactory<M> apply(
+ ImmutableClassesGiraphConfiguration conf) {
+ return new DefaultMessageValueFactory<>(messageClass, conf);
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/messages/package-info.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/messages/package-info.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/messages/package-info.java
new file mode 100644
index 0000000..ba3014c
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/messages/package-info.java
@@ -0,0 +1,21 @@
+/*
+ * 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.
+ */
+/**
+ * Utility classes for handling of messages within Pieces
+ */
+package org.apache.giraph.block_app.framework.piece.messages;
http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/package-info.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/package-info.java b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/package-info.java
new file mode 100644
index 0000000..fbc6e92
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/block_app/framework/piece/package-info.java
@@ -0,0 +1,24 @@
+/*
+ * 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.
+ */
+/**
+ * Single execution object - Piece, and related classes.
+ *
+ * AbstractPiece is parent class of all Pieces. Most frequentlly
+ * users should extend Piece class itself
+ */
+package org.apache.giraph.block_app.framework.piece;
http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/function/Consumer.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/function/Consumer.java b/giraph-block-app/src/main/java/org/apache/giraph/function/Consumer.java
new file mode 100644
index 0000000..2a0e36a
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/function/Consumer.java
@@ -0,0 +1,34 @@
+/*
+ * 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.giraph.function;
+
+import java.io.Serializable;
+
+
+/**
+ * Function:
+ * (T) -> void
+ *
+ * @param <T> Argument type
+ */
+public interface Consumer<T> extends Serializable {
+ /**
+ * Applies this function to {@code input}
+ */
+ void apply(T input);
+}
http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/function/Function.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/function/Function.java b/giraph-block-app/src/main/java/org/apache/giraph/function/Function.java
new file mode 100644
index 0000000..41046ba
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/function/Function.java
@@ -0,0 +1,38 @@
+/*
+ * 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.giraph.function;
+
+import java.io.Serializable;
+
+
+/**
+ * Function:
+ * (F) -> T
+ *
+ * @param <F> Argument type
+ * @param <T> Result type
+ */
+public interface Function<F, T> extends Serializable {
+ /**
+ * Returns the result of applying this function to given {@code input}.
+ *
+ * The returned object may or may not be a new instance,
+ * depending on the implementation.
+ */
+ T apply(F input);
+}
http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/function/PairConsumer.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/function/PairConsumer.java b/giraph-block-app/src/main/java/org/apache/giraph/function/PairConsumer.java
new file mode 100644
index 0000000..012ec82
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/function/PairConsumer.java
@@ -0,0 +1,34 @@
+/*
+ * 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.giraph.function;
+
+import java.io.Serializable;
+
+/**
+ * Function:
+ * (T1, T2) -> void
+ *
+ * @param <T1> First argument type
+ * @param <T2> Second argument type
+ */
+public interface PairConsumer<T1, T2> extends Serializable {
+ /**
+ * Applies this function to {@code input1} and {@code input2}
+ */
+ void apply(T1 input1, T2 input2);
+}
http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/function/PairFunction.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/function/PairFunction.java b/giraph-block-app/src/main/java/org/apache/giraph/function/PairFunction.java
new file mode 100644
index 0000000..bfff400
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/function/PairFunction.java
@@ -0,0 +1,40 @@
+/*
+ * 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.giraph.function;
+
+import java.io.Serializable;
+
+
+/**
+ * Function:
+ * (F1, F2) -> T
+ *
+ * @param <F1> First argument type
+ * @param <F2> Second argument type
+ * @param <T> Result type
+ */
+public interface PairFunction<F1, F2, T> extends Serializable {
+ /**
+ * Returns the result of applying this function to given
+ * {@code input1} and {@code input2}.
+ *
+ * The returned object may or may not be a new instance,
+ * depending on the implementation.
+ */
+ T apply(F1 input1, F2 input2);
+}
http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/function/Supplier.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/function/Supplier.java b/giraph-block-app/src/main/java/org/apache/giraph/function/Supplier.java
new file mode 100644
index 0000000..1813b54
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/function/Supplier.java
@@ -0,0 +1,37 @@
+/*
+ * 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.giraph.function;
+
+import java.io.Serializable;
+
+/**
+ * Function:
+ * () -> T
+ * <br>
+ * Specialization of com.google.common.base.Supplier, that is also
+ * Serializable.
+ *
+ * @param <T> Result type
+ */
+public interface Supplier<T> extends Serializable {
+ /**
+ * Retrieves an instance of the appropriate type. The returned object may or
+ * may not be a new instance, depending on the implementation.
+ */
+ T get();
+}
http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/function/package-info.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/function/package-info.java b/giraph-block-app/src/main/java/org/apache/giraph/function/package-info.java
new file mode 100644
index 0000000..b089da6
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/function/package-info.java
@@ -0,0 +1,26 @@
+/*
+ * 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 for interfaces representing different functions,
+ * that all extends Serializable in order for Kryo to be able
+ * to serialize them.
+ *
+ * Even when same interface is present in Guava, we do not extend it
+ * due to @Nullable annotations adding requirement of handling nulls.
+ */
+package org.apache.giraph.function;
http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/function/primitive/IntSupplier.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/function/primitive/IntSupplier.java b/giraph-block-app/src/main/java/org/apache/giraph/function/primitive/IntSupplier.java
new file mode 100644
index 0000000..2cf74e1
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/function/primitive/IntSupplier.java
@@ -0,0 +1,27 @@
+/*
+ * 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.giraph.function.primitive;
+
+/**
+ * Primitive specialization of Function:
+ * () -> int
+ */
+public interface IntSupplier {
+ /** Retrieves an int value. */
+ int get();
+}
http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/function/primitive/package-info.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/function/primitive/package-info.java b/giraph-block-app/src/main/java/org/apache/giraph/function/primitive/package-info.java
new file mode 100644
index 0000000..fdf40ff
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/function/primitive/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+/**
+ * Primitive specializations of interfaces from org.apache.giraph.function
+ * package.
+ */
+package org.apache.giraph.function.primitive;
http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/function/vertex/ConsumerWithVertex.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/function/vertex/ConsumerWithVertex.java b/giraph-block-app/src/main/java/org/apache/giraph/function/vertex/ConsumerWithVertex.java
new file mode 100644
index 0000000..87e7f9b
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/function/vertex/ConsumerWithVertex.java
@@ -0,0 +1,48 @@
+/*
+ * 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.giraph.function.vertex;
+
+import java.io.Serializable;
+
+import org.apache.giraph.function.PairConsumer;
+import org.apache.giraph.graph.Vertex;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
+
+
+/**
+ * Function:
+ * (vertex, T) -> void
+ *
+ * A class that can consume objects of a single type, when given a vertex.
+ *
+ * @param <I> Vertex id type
+ * @param <V> Vertex value type
+ * @param <E> Edge value type
+ * @param <T> Argument type
+ */
+@SuppressWarnings("rawtypes")
+public interface ConsumerWithVertex<I extends WritableComparable,
+ V extends Writable, E extends Writable, T>
+ extends PairConsumer<Vertex<I, V, E>, T>, Serializable {
+ /**
+ * Applies this function to {@code vertex} and {@code input}
+ */
+ @Override
+ void apply(Vertex<I, V, E> vertex, T value);
+}
http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/function/vertex/FunctionWithVertex.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/function/vertex/FunctionWithVertex.java b/giraph-block-app/src/main/java/org/apache/giraph/function/vertex/FunctionWithVertex.java
new file mode 100644
index 0000000..fdab5de
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/function/vertex/FunctionWithVertex.java
@@ -0,0 +1,52 @@
+/*
+ * 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.giraph.function.vertex;
+
+import java.io.Serializable;
+
+import org.apache.giraph.function.PairFunction;
+import org.apache.giraph.graph.Vertex;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
+
+/**
+ * Function:
+ * (vertex, F) -> T
+ *
+ * Determines an output value based on a vertex and an input value.
+ *
+ * @param <I> Vertex id type
+ * @param <V> Vertex value type
+ * @param <E> Edge value type
+ * @param <F> Argument type
+ * @param <T> Result type
+ */
+@SuppressWarnings("rawtypes")
+public interface FunctionWithVertex<I extends WritableComparable,
+ V extends Writable, E extends Writable, F, T>
+ extends PairFunction<Vertex<I, V, E>, F, T>, Serializable {
+ /**
+ * Returns the result of applying this function to given
+ * {@code vertex} and {@code input}.
+ *
+ * The returned object may or may not be a new instance,
+ * depending on the implementation.
+ */
+ @Override
+ T apply(Vertex<I, V, E> vertex, F input);
+}
http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/function/vertex/SupplierFromVertex.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/function/vertex/SupplierFromVertex.java b/giraph-block-app/src/main/java/org/apache/giraph/function/vertex/SupplierFromVertex.java
new file mode 100644
index 0000000..bc0f9c1
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/function/vertex/SupplierFromVertex.java
@@ -0,0 +1,49 @@
+/*
+ * 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.giraph.function.vertex;
+
+import java.io.Serializable;
+
+import org.apache.giraph.graph.Vertex;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
+
+/**
+ * Function:
+ * (vertex) -> T
+ *
+ * A class that can supply objects of a single type, when given a vertex.
+ *
+ * (doesn't extend Function<Vertex<I, V, E>, T>, because of different
+ * method names)
+ *
+ * @param <I> Vertex id type
+ * @param <V> Vertex value type
+ * @param <E> Edge value type
+ * @param <T> Result type
+ */
+@SuppressWarnings("rawtypes")
+public interface SupplierFromVertex<I extends WritableComparable,
+ V extends Writable, E extends Writable, T> extends Serializable {
+ /**
+ * Retrieves an instance of the appropriate type, given a vertex.
+ * The returned object may or may not be a new instance,
+ * depending on the implementation.
+ */
+ T get(Vertex<I, V, E> vertex);
+}
http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/main/java/org/apache/giraph/function/vertex/package-info.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/main/java/org/apache/giraph/function/vertex/package-info.java b/giraph-block-app/src/main/java/org/apache/giraph/function/vertex/package-info.java
new file mode 100644
index 0000000..bd5b019
--- /dev/null
+++ b/giraph-block-app/src/main/java/org/apache/giraph/function/vertex/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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 for interfaces representing functions additionally
+ * performed on vertex values.
+ */
+package org.apache.giraph.function.vertex;
http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/test/java/org/apache/giraph/block_app/framework/block/BlockTestingUtils.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/test/java/org/apache/giraph/block_app/framework/block/BlockTestingUtils.java b/giraph-block-app/src/test/java/org/apache/giraph/block_app/framework/block/BlockTestingUtils.java
new file mode 100644
index 0000000..6487d95
--- /dev/null
+++ b/giraph-block-app/src/test/java/org/apache/giraph/block_app/framework/block/BlockTestingUtils.java
@@ -0,0 +1,158 @@
+/*
+ * 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.giraph.block_app.framework.block;
+
+import static org.junit.Assert.assertEquals;
+import it.unimi.dsi.fastutil.ints.IntArrayList;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.Random;
+
+import org.apache.giraph.block_app.framework.piece.AbstractPiece;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Iterables;
+
+@SuppressWarnings({"unchecked", "rawtypes"})
+public class BlockTestingUtils {
+
+ BlockTestingUtils() { }
+
+ private static final int NUM_TRIALS = 10;
+ private static final int REPEAT_TIMES = 10;
+
+ private static int testSequential(Iterator<? extends AbstractPiece> referenceImpl,
+ Iterator<? extends AbstractPiece> testImpl) {
+ int length = 0;
+
+ CheckIterator checkIterator = new CheckIterator(referenceImpl, testImpl);
+ while (checkIterator.hasNext()) {
+ checkIterator.next();
+ length++;
+ }
+
+ System.out.println("Length is : " + length);
+ return length;
+ }
+
+ private static boolean anyHasNext(ArrayList<? extends Iterator> arr) {
+ for (Iterator t : arr) {
+ if (t.hasNext()) {
+ return true;
+ }
+ }
+ return false;
+ }
+
+ private static void testRandom(int length,
+ Iterable<? extends AbstractPiece> referenceImpl,
+ Iterable<? extends AbstractPiece> testImpl) {
+ Random rand = new Random();
+
+ ArrayList<CheckIterator<AbstractPiece>> arr = new ArrayList<>();
+ IntArrayList lengths = new IntArrayList(NUM_TRIALS);
+ for (int i = 0; i < NUM_TRIALS; i++) {
+ lengths.add(0);
+ }
+ for (int i = 0; i < NUM_TRIALS; i++) {
+ arr.add(new CheckIterator(referenceImpl.iterator(), testImpl.iterator()));
+ }
+
+ int totalCount = 0;
+ while (anyHasNext(arr)) {
+ int index = rand.nextInt(NUM_TRIALS);
+ while (!arr.get(index).hasNext()) {
+ index = rand.nextInt(NUM_TRIALS);
+ }
+ CheckIterator it = arr.get(index);
+ it.next();
+ int itLength = lengths.getInt(index);
+ lengths.set(index, itLength + 1);
+ totalCount++;
+ }
+ assertEquals("TotalCount should be length * NUM_TRIALS", length * NUM_TRIALS, totalCount);
+ System.out.println("Final count is : " + totalCount);
+ }
+
+ /**
+ * Tests both the length of the iterator returned by the block, as-well as the deterministic behavior
+ * expected by calling .iterator() against the referenceImpl.
+ * @param referenceImpl : A list of pieces in the expected order
+ * @param testImpl : A list of pieces to test against (the Block)
+ */
+ public static void testIndependence(Iterable<? extends AbstractPiece> referenceImpl,
+ Iterable<? extends AbstractPiece> testImpl) {
+ int length = testSequential(referenceImpl.iterator(), testImpl.iterator());
+ testRandom(length, referenceImpl, testImpl);
+ }
+
+ /**
+ * Test how the block interacts with a repeatBlock. The expected result is to
+ * see the pieces in referenceImpl show up REPEAT_TIMES many times.
+ * @param referenceImpl : A list of pieces in the expected order
+ * @param block : The block to test
+ */
+ public static void testNestedRepeatBlock(Iterable<? extends AbstractPiece> referenceImpl, Block block) {
+ Block repeatBlock = new RepeatBlock(
+ REPEAT_TIMES,
+ block
+ );
+ testIndependence(
+ Iterables.concat(Collections.nCopies(REPEAT_TIMES, referenceImpl)),
+ repeatBlock
+ );
+ }
+
+ public static class CheckIterator<T> implements Iterator {
+
+ private final Iterator<T> fst;
+ private final Iterator<T> snd;
+
+ public CheckIterator(Iterator<T> fst, Iterator<T> snd) {
+ this.fst = fst;
+ this.snd = snd;
+ }
+
+ @Override
+ public boolean hasNext() {
+ boolean fstHasNxt = fst.hasNext();
+ boolean sndHasNxt = snd.hasNext();
+ Preconditions.checkArgument(fstHasNxt == sndHasNxt, "Expect hasNext() on " +
+ "both iterators to be identical. Got: " + fst.hasNext() + " and " + snd.hasNext());
+ return fstHasNxt;
+ }
+
+ @Override
+ public Object next() {
+ T fstNxt = fst.next();
+ T sndNxt = snd.next();
+ Preconditions.checkArgument(fstNxt == sndNxt, "Expect objs returned by " +
+ "both iterators to be identical. Got: " + fstNxt + " and " + sndNxt);
+ return fstNxt;
+ }
+
+ @Override
+ public void remove() {
+ throw new RuntimeException("Not implemented");
+ }
+
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/test/java/org/apache/giraph/block_app/framework/block/TestIfBlock.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/test/java/org/apache/giraph/block_app/framework/block/TestIfBlock.java b/giraph-block-app/src/test/java/org/apache/giraph/block_app/framework/block/TestIfBlock.java
new file mode 100644
index 0000000..0dacae1
--- /dev/null
+++ b/giraph-block-app/src/test/java/org/apache/giraph/block_app/framework/block/TestIfBlock.java
@@ -0,0 +1,88 @@
+/*
+ * 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.giraph.block_app.framework.block;
+
+
+import java.util.Arrays;
+
+import org.apache.giraph.block_app.framework.piece.Piece;
+import org.apache.giraph.function.Supplier;
+import org.junit.Test;
+
+public class TestIfBlock {
+
+ private static final Supplier<Boolean> TRUE_SUPPLIER = new Supplier<Boolean>() {
+ @Override
+ public Boolean get() {
+ return true;
+ }
+ };
+
+ private static final Supplier<Boolean> FALSE_SUPPLIER = new Supplier<Boolean>() {
+ @Override
+ public Boolean get() {
+ return false;
+ }
+ };
+
+ @Test
+ // Test short-circuiting the if -> then
+ public void testIfBlockThen() throws Exception {
+ Piece piece1 = new Piece();
+ Piece piece2 = new Piece();
+ Block ifBlock = new IfBlock(
+ TRUE_SUPPLIER,
+ new SequenceBlock(piece1, piece2)
+ );
+
+ BlockTestingUtils.testIndependence(
+ Arrays.asList(piece1, piece2),
+ ifBlock);
+ }
+
+ @Test
+ // Test short-circuiting the if -> else
+ public void testIfBlockElse() throws Exception {
+ Piece piece1 = new Piece();
+ Piece piece2 = new Piece();
+ Block ifBlock = new IfBlock(
+ FALSE_SUPPLIER,
+ new EmptyBlock(),
+ new SequenceBlock(piece1, piece2)
+ );
+
+ BlockTestingUtils.testIndependence(
+ Arrays.asList(piece1, piece2),
+ ifBlock);
+ }
+
+ @Test
+ public void testIfNestedInRepeat() throws Exception {
+ Piece piece1 = new Piece();
+ Piece piece2 = new Piece();
+ Block ifBlock = new IfBlock(
+ TRUE_SUPPLIER,
+ new SequenceBlock(piece1, piece2)
+ );
+
+ BlockTestingUtils.testNestedRepeatBlock(
+ Arrays.asList(piece1, piece2),
+ ifBlock);
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/test/java/org/apache/giraph/block_app/framework/block/TestRepeatBlock.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/test/java/org/apache/giraph/block_app/framework/block/TestRepeatBlock.java b/giraph-block-app/src/test/java/org/apache/giraph/block_app/framework/block/TestRepeatBlock.java
new file mode 100644
index 0000000..1e096ba
--- /dev/null
+++ b/giraph-block-app/src/test/java/org/apache/giraph/block_app/framework/block/TestRepeatBlock.java
@@ -0,0 +1,80 @@
+/*
+ * 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.giraph.block_app.framework.block;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.giraph.block_app.framework.piece.AbstractPiece;
+import org.apache.giraph.block_app.framework.piece.Piece;
+import org.junit.Test;
+
+import com.google.common.collect.Iterables;
+
+/**
+ * Tests repeatBlock's correctness
+ */
+public class TestRepeatBlock {
+
+ public static final int REPEAT_TIMES = 5;
+
+ @Test
+ public void testRepeatBlockBasic() throws Exception {
+ Piece piece1 = new Piece();
+ Piece piece2 = new Piece();
+ Block innerBlock = new SequenceBlock(piece1, piece2);
+ Block repeatBlock = new RepeatBlock(
+ REPEAT_TIMES,
+ innerBlock
+ );
+ BlockTestingUtils.testIndependence(
+ Iterables.concat(Collections.nCopies(REPEAT_TIMES, Arrays.asList(piece1, piece2))),
+ repeatBlock);
+ }
+
+ @Test
+ public void testNestedRepeatBlock() throws Exception {
+ Piece piece1 = new Piece();
+ Piece piece2 = new Piece();
+ Block innerBlock = new SequenceBlock(piece1, piece2);
+ Block repeatBlock = new RepeatBlock(
+ REPEAT_TIMES,
+ innerBlock
+ );
+ BlockTestingUtils.testNestedRepeatBlock(
+ Iterables.concat(Collections.nCopies(REPEAT_TIMES, Arrays.asList(piece1, piece2))),
+ repeatBlock);
+ }
+
+ @Test
+ public void testRepeatBlockEmpty() throws Exception {
+ Block innerBlock = new EmptyBlock();
+ Block repeatBlock = new RepeatBlock(
+ REPEAT_TIMES,
+ innerBlock
+ );
+ List<? extends AbstractPiece> referenceImpl = Collections.emptyList();
+ BlockTestingUtils.testIndependence(
+ // Concatenating EmptyIterator = just EmptyIterator. No obj's to
+ // compare against either
+ referenceImpl,
+ repeatBlock);
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/giraph/blob/819d6d38/giraph-block-app/src/test/java/org/apache/giraph/block_app/framework/block/TestRepeatUntilBlock.java
----------------------------------------------------------------------
diff --git a/giraph-block-app/src/test/java/org/apache/giraph/block_app/framework/block/TestRepeatUntilBlock.java b/giraph-block-app/src/test/java/org/apache/giraph/block_app/framework/block/TestRepeatUntilBlock.java
new file mode 100644
index 0000000..242d376
--- /dev/null
+++ b/giraph-block-app/src/test/java/org/apache/giraph/block_app/framework/block/TestRepeatUntilBlock.java
@@ -0,0 +1,103 @@
+/*
+ * 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.giraph.block_app.framework.block;
+
+import static org.junit.Assert.assertEquals;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Iterator;
+
+import org.apache.giraph.block_app.framework.piece.AbstractPiece;
+import org.apache.giraph.block_app.framework.piece.Piece;
+import org.apache.giraph.function.Supplier;
+import org.junit.Test;
+
+import com.google.common.collect.Iterables;
+
+/**
+ * Tests repeatUntilBlock's correctness
+ */
+public class TestRepeatUntilBlock {
+
+ public static final int REPEAT_TIMES = 5;
+
+ private static final Supplier<Boolean> falseSupplier = new Supplier<Boolean>() {
+ @Override
+ public Boolean get() {
+ return false;
+ }
+ };
+
+ @Test
+ public void testRepeatUntilBlockBasic() throws Exception {
+ Piece piece1 = new Piece();
+ Piece piece2 = new Piece();
+ Block innerBlock = new SequenceBlock(piece1, piece2);
+ Block repeatBlock = new RepeatUntilBlock(
+ REPEAT_TIMES,
+ innerBlock,
+ falseSupplier
+ );
+ BlockTestingUtils.testIndependence(
+ Iterables.concat(Collections.nCopies(REPEAT_TIMES, Arrays.asList(piece1, piece2))),
+ repeatBlock);
+ }
+
+ @Test
+ public void testNestedRepeatUntilBlock() throws Exception {
+ Piece piece1 = new Piece();
+ Piece piece2 = new Piece();
+ Block innerBlock = new SequenceBlock(piece1, piece2);
+ Block repeatBlock = new RepeatUntilBlock(
+ REPEAT_TIMES,
+ innerBlock,
+ falseSupplier
+ );
+ BlockTestingUtils.testNestedRepeatBlock(
+ Iterables.concat(Collections.nCopies(REPEAT_TIMES, Arrays.asList(piece1, piece2))),
+ repeatBlock);
+ }
+
+ @Test
+ public void testRepeatUntilBlockUnlimited() throws Exception {
+ Block innerBlock = new SequenceBlock(new Piece());
+ // Can't test with testIndependence - spin up our own test inline
+ Supplier<Boolean> countingSupplier = new Supplier<Boolean>() {
+ private int i = 0;
+
+ @Override
+ public Boolean get() {
+ i++;
+ return i > REPEAT_TIMES;
+ }
+ };
+ Block repeatBlock = RepeatUntilBlock.unlimited(
+ innerBlock,
+ countingSupplier
+ );
+ int count = 0;
+ Iterator<AbstractPiece> it = repeatBlock.iterator();
+ while (it.hasNext()) {
+ it.next();
+ count++;
+ }
+ assertEquals("Count must be equal to REPEAT_TIMES", REPEAT_TIMES, count);
+ }
+
+}