You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@giraph.apache.org by ap...@apache.org on 2013/03/25 22:59:13 UTC

git commit: VertexValueFactory

Updated Branches:
  refs/heads/trunk a4d333039 -> 95ce243fd


VertexValueFactory


Project: http://git-wip-us.apache.org/repos/asf/giraph/repo
Commit: http://git-wip-us.apache.org/repos/asf/giraph/commit/95ce243f
Tree: http://git-wip-us.apache.org/repos/asf/giraph/tree/95ce243f
Diff: http://git-wip-us.apache.org/repos/asf/giraph/diff/95ce243f

Branch: refs/heads/trunk
Commit: 95ce243fd8be15aed3155ba8bd1791a846df8a32
Parents: a4d3330
Author: Alessandro Presta <al...@fb.com>
Authored: Fri Mar 22 18:48:26 2013 -0700
Committer: Alessandro Presta <al...@fb.com>
Committed: Mon Mar 25 14:58:30 2013 -0700

----------------------------------------------------------------------
 CHANGELOG                                          |    2 +
 .../java/org/apache/giraph/conf/GiraphClasses.java |   34 ++++++++-
 .../apache/giraph/conf/GiraphConfiguration.java    |   12 +++
 .../org/apache/giraph/conf/GiraphConstants.java    |    4 +-
 .../conf/ImmutableClassesGiraphConfiguration.java  |   40 +++++++----
 .../giraph/graph/DefaultVertexValueFactory.java    |   58 +++++++++++++++
 .../apache/giraph/graph/VertexValueFactory.java    |   47 ++++++++++++
 .../giraph/job/GiraphConfigurationValidator.java   |   31 +++++++-
 .../apache/giraph/utils/ConfigurationUtils.java    |    8 ++
 .../apache/giraph/utils/InternalVertexRunner.java  |    1 +
 .../java/org/apache/giraph/io/TestEdgeInput.java   |   23 ++++++
 .../org/apache/giraph/vertex/TestVertexTypes.java  |   43 ++++++++++-
 12 files changed, 282 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/giraph/blob/95ce243f/CHANGELOG
----------------------------------------------------------------------
diff --git a/CHANGELOG b/CHANGELOG
index 50ed932..84939a4 100644
--- a/CHANGELOG
+++ b/CHANGELOG
@@ -1,6 +1,8 @@
 Giraph Change Log
 
 Release 0.2.0 - unreleased
+  GIRAPH-586: Customizable default vertex value (apresta)
+
   GIRAPH-580: NPE in HiveGiraphRunner when the vertex output format is
   not defined (aching)
 

http://git-wip-us.apache.org/repos/asf/giraph/blob/95ce243f/giraph-core/src/main/java/org/apache/giraph/conf/GiraphClasses.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/conf/GiraphClasses.java b/giraph-core/src/main/java/org/apache/giraph/conf/GiraphClasses.java
index 5090250..d67d3a5 100644
--- a/giraph-core/src/main/java/org/apache/giraph/conf/GiraphClasses.java
+++ b/giraph-core/src/main/java/org/apache/giraph/conf/GiraphClasses.java
@@ -23,7 +23,10 @@ import org.apache.giraph.combiner.Combiner;
 import org.apache.giraph.edge.ByteArrayEdges;
 import org.apache.giraph.edge.VertexEdges;
 import org.apache.giraph.graph.DefaultVertexResolver;
+import org.apache.giraph.graph.DefaultVertexValueFactory;
+import org.apache.giraph.graph.Vertex;
 import org.apache.giraph.graph.VertexResolver;
+import org.apache.giraph.graph.VertexValueFactory;
 import org.apache.giraph.io.EdgeInputFormat;
 import org.apache.giraph.io.VertexInputFormat;
 import org.apache.giraph.io.VertexOutputFormat;
@@ -36,7 +39,6 @@ import org.apache.giraph.partition.Partition;
 import org.apache.giraph.partition.PartitionContext;
 import org.apache.giraph.partition.SimplePartition;
 import org.apache.giraph.utils.ReflectionUtils;
-import org.apache.giraph.graph.Vertex;
 import org.apache.giraph.worker.DefaultWorkerContext;
 import org.apache.giraph.worker.WorkerContext;
 import org.apache.hadoop.conf.Configuration;
@@ -70,6 +72,9 @@ public class GiraphClasses<I extends WritableComparable,
   /** Vertex edges class - cached for fast access */
   protected Class<? extends VertexEdges<I, E>> vertexEdgesClass;
 
+  /** Vertex value factory class - cached for fast access */
+  protected Class<? extends VertexValueFactory<V>> vertexValueFactoryClass;
+
   /** Graph partitioner factory class - cached for fast access */
   protected Class<? extends GraphPartitionerFactory<I, V, E, M>>
   graphPartitionerFactoryClass;
@@ -109,6 +114,8 @@ public class GiraphClasses<I extends WritableComparable,
     // downcast.
     vertexEdgesClass = (Class<? extends VertexEdges<I, E>>) (Object)
         ByteArrayEdges.class;
+    vertexValueFactoryClass = (Class<? extends VertexValueFactory<V>>) (Object)
+        DefaultVertexValueFactory.class;
     graphPartitionerFactoryClass =
         (Class<? extends GraphPartitionerFactory<I, V, E, M>>) (Object)
             HashPartitionerFactory.class;
@@ -149,6 +156,9 @@ public class GiraphClasses<I extends WritableComparable,
     vertexEdgesClass = (Class<? extends VertexEdges<I, E>>)
         conf.getClass(VERTEX_EDGES_CLASS, ByteArrayEdges.class,
             VertexEdges.class);
+    vertexValueFactoryClass = (Class<? extends VertexValueFactory<V>>)
+        conf.getClass(VERTEX_VALUE_FACTORY_CLASS,
+            DefaultVertexValueFactory.class);
 
     graphPartitionerFactoryClass =
         (Class<? extends GraphPartitionerFactory<I, V, E, M>>)
@@ -239,6 +249,15 @@ public class GiraphClasses<I extends WritableComparable,
   }
 
   /**
+   * Get vertex value factory class
+   *
+   * @return Vertex value factory class
+   */
+  public Class<? extends VertexValueFactory<V>> getVertexValueFactoryClass() {
+    return vertexValueFactoryClass;
+  }
+
+  /**
    * Get the GraphPartitionerFactory
    *
    * @return GraphPartitionerFactory
@@ -500,6 +519,19 @@ public class GiraphClasses<I extends WritableComparable,
   }
 
   /**
+   * Set VertexValueFactory class held
+   *
+   * @param vertexValueFactoryClass Vertex value factory class to set
+   * @return this
+   */
+  public GiraphClasses setVertexValueFactoryClass(
+      Class<? extends VertexValueFactory> vertexValueFactoryClass) {
+    this.vertexValueFactoryClass = (Class<? extends VertexValueFactory<V>>)
+        vertexValueFactoryClass;
+    return this;
+  }
+
+  /**
    * Set GraphPartitionerFactory class held
    *
    * @param klass GraphPartitionerFactory to set

http://git-wip-us.apache.org/repos/asf/giraph/blob/95ce243f/giraph-core/src/main/java/org/apache/giraph/conf/GiraphConfiguration.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/conf/GiraphConfiguration.java b/giraph-core/src/main/java/org/apache/giraph/conf/GiraphConfiguration.java
index 6886d58..ae276f6 100644
--- a/giraph-core/src/main/java/org/apache/giraph/conf/GiraphConfiguration.java
+++ b/giraph-core/src/main/java/org/apache/giraph/conf/GiraphConfiguration.java
@@ -22,6 +22,7 @@ import org.apache.giraph.aggregators.AggregatorWriter;
 import org.apache.giraph.combiner.Combiner;
 import org.apache.giraph.edge.VertexEdges;
 import org.apache.giraph.graph.VertexResolver;
+import org.apache.giraph.graph.VertexValueFactory;
 import org.apache.giraph.io.EdgeInputFormat;
 import org.apache.giraph.io.VertexInputFormat;
 import org.apache.giraph.io.VertexOutputFormat;
@@ -75,6 +76,17 @@ public class GiraphConfiguration extends Configuration
   }
 
   /**
+   * Set the vertex value factory class
+   *
+   * @param vertexValueFactoryClass Creates default vertex values
+   */
+  public final void setVertexValueFactoryClass(
+      Class<? extends VertexValueFactory> vertexValueFactoryClass) {
+    setClass(VERTEX_VALUE_FACTORY_CLASS, vertexValueFactoryClass,
+        VertexValueFactory.class);
+  }
+
+  /**
    * Set the vertex edges class
    *
    * @param vertexEdgesClass Determines the way edges are stored

http://git-wip-us.apache.org/repos/asf/giraph/blob/95ce243f/giraph-core/src/main/java/org/apache/giraph/conf/GiraphConstants.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/conf/GiraphConstants.java b/giraph-core/src/main/java/org/apache/giraph/conf/GiraphConstants.java
index ad9073d..fe5278b 100644
--- a/giraph-core/src/main/java/org/apache/giraph/conf/GiraphConstants.java
+++ b/giraph-core/src/main/java/org/apache/giraph/conf/GiraphConstants.java
@@ -24,7 +24,9 @@ package org.apache.giraph.conf;
 public interface GiraphConstants {
   /** Vertex class - required */
   String VERTEX_CLASS = "giraph.vertexClass";
-  /** Vertex edges class - required */
+  /** Vertex value factory class - optional */
+  String VERTEX_VALUE_FACTORY_CLASS = "giraph.vertexValueFactoryClass";
+  /** Vertex edges class - optional */
   String VERTEX_EDGES_CLASS = "giraph.vertexEdgesClass";
 
   /** Class for Master - optional */

http://git-wip-us.apache.org/repos/asf/giraph/blob/95ce243f/giraph-core/src/main/java/org/apache/giraph/conf/ImmutableClassesGiraphConfiguration.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/conf/ImmutableClassesGiraphConfiguration.java b/giraph-core/src/main/java/org/apache/giraph/conf/ImmutableClassesGiraphConfiguration.java
index 4fedc46..2a3466d 100644
--- a/giraph-core/src/main/java/org/apache/giraph/conf/ImmutableClassesGiraphConfiguration.java
+++ b/giraph-core/src/main/java/org/apache/giraph/conf/ImmutableClassesGiraphConfiguration.java
@@ -24,6 +24,7 @@ import org.apache.giraph.edge.ReusableEdge;
 import org.apache.giraph.edge.ReuseObjectsVertexEdges;
 import org.apache.giraph.graph.GraphState;
 import org.apache.giraph.graph.VertexResolver;
+import org.apache.giraph.graph.VertexValueFactory;
 import org.apache.giraph.io.EdgeInputFormat;
 import org.apache.giraph.io.VertexInputFormat;
 import org.apache.giraph.io.VertexOutputFormat;
@@ -68,6 +69,8 @@ public class ImmutableClassesGiraphConfiguration<I extends WritableComparable,
     extends GiraphConfiguration {
   /** Holder for all the classes */
   private final GiraphClasses classes;
+  /** Vertex value factory. */
+  private final VertexValueFactory<V> vertexValueFactory;
 
   /**
    * Use unsafe serialization? Cached for fast access to instantiate the
@@ -86,6 +89,19 @@ public class ImmutableClassesGiraphConfiguration<I extends WritableComparable,
     classes = new GiraphClasses(conf);
     useUnsafeSerialization = getBoolean(USE_UNSAFE_SERIALIZATION,
         USE_UNSAFE_SERIALIZATION_DEFAULT);
+    try {
+      vertexValueFactory = (VertexValueFactory<V>)
+          classes.getVertexValueFactoryClass().newInstance();
+    } catch (InstantiationException e) {
+      throw new IllegalArgumentException(
+          "ImmutableClassesGiraphConfiguration: Failed to instantiate class " +
+              classes.getVertexValueFactoryClass(), e);
+    } catch (IllegalAccessException e) {
+      throw new IllegalArgumentException(
+          "ImmutableClassesGiraphConfiguration: Illegally accessed class " +
+              classes.getVertexValueFactoryClass(), e);
+    }
+    vertexValueFactory.initialize(this);
   }
 
   /**
@@ -403,20 +419,16 @@ public class ImmutableClassesGiraphConfiguration<I extends WritableComparable,
    */
   @SuppressWarnings("unchecked")
   public V createVertexValue() {
-    Class<V> klass = getVertexValueClass();
-    if (klass == NullWritable.class) {
-      return (V) NullWritable.get();
-    } else {
-      try {
-        return klass.newInstance();
-      } catch (InstantiationException e) {
-        throw new IllegalArgumentException(
-            "createVertexValue: Failed to instantiate", e);
-      } catch (IllegalAccessException e) {
-        throw new IllegalArgumentException(
-            "createVertexValue: Illegally accessed", e);
-      }
-    }
+    return vertexValueFactory.createVertexValue();
+  }
+
+  /**
+   * Get the user's subclassed vertex value factory class
+   *
+   * @return User's vertex value factory class
+   */
+  public Class<? extends VertexValueFactory<V>> getVertexValueFactoryClass() {
+    return classes.getVertexValueFactoryClass();
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/giraph/blob/95ce243f/giraph-core/src/main/java/org/apache/giraph/graph/DefaultVertexValueFactory.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/graph/DefaultVertexValueFactory.java b/giraph-core/src/main/java/org/apache/giraph/graph/DefaultVertexValueFactory.java
new file mode 100644
index 0000000..47902d1
--- /dev/null
+++ b/giraph-core/src/main/java/org/apache/giraph/graph/DefaultVertexValueFactory.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.graph;
+
+import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.Writable;
+
+/**
+ * Default {@link VertexValueFactory} that simply uses the default
+ * constructor for the vertex value class.
+ *
+ * @param <V> Vertex value
+ */
+public class DefaultVertexValueFactory<V extends Writable>
+    implements VertexValueFactory<V> {
+  /** Cached vertex value class. */
+  private Class<V> vertexValueClass;
+
+  @Override
+  public void initialize(
+      ImmutableClassesGiraphConfiguration<?, V, ?, ?> configuration) {
+    vertexValueClass = configuration.getVertexValueClass();
+  }
+
+  @Override
+  public V createVertexValue() {
+    if (vertexValueClass == NullWritable.class) {
+      return (V) NullWritable.get();
+    } else {
+      try {
+        return vertexValueClass.newInstance();
+      } catch (InstantiationException e) {
+        throw new IllegalArgumentException(
+            "createVertexValue: Failed to instantiate", e);
+      } catch (IllegalAccessException e) {
+        throw new IllegalArgumentException(
+            "createVertexValue: Illegally accessed", e);
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/95ce243f/giraph-core/src/main/java/org/apache/giraph/graph/VertexValueFactory.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/graph/VertexValueFactory.java b/giraph-core/src/main/java/org/apache/giraph/graph/VertexValueFactory.java
new file mode 100644
index 0000000..e62bb01
--- /dev/null
+++ b/giraph-core/src/main/java/org/apache/giraph/graph/VertexValueFactory.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.graph;
+
+import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
+import org.apache.hadoop.io.Writable;
+
+/**
+ * Factory class to create default vertex values.
+ * A user can extend this class in order to customize the creation of new
+ * vertex values when a vertex is created by the infrastructure
+ * (e.g., if edges for a vertex are read).
+ *
+ * @param <V> Vertex value
+ */
+public interface VertexValueFactory<V extends Writable> {
+  /**
+   * Initialize the factory from the configuration.
+   *
+   * @param configuration Configuration
+   */
+  void initialize(
+      ImmutableClassesGiraphConfiguration<?, V, ?, ?> configuration);
+
+  /**
+   * Create a new vertex value.
+   *
+   * @return New vertex value.
+   */
+  V createVertexValue();
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/95ce243f/giraph-core/src/main/java/org/apache/giraph/job/GiraphConfigurationValidator.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/job/GiraphConfigurationValidator.java b/giraph-core/src/main/java/org/apache/giraph/job/GiraphConfigurationValidator.java
index b08c74d..eace06f 100644
--- a/giraph-core/src/main/java/org/apache/giraph/job/GiraphConfigurationValidator.java
+++ b/giraph-core/src/main/java/org/apache/giraph/job/GiraphConfigurationValidator.java
@@ -21,15 +21,17 @@ package org.apache.giraph.job;
 import org.apache.giraph.combiner.Combiner;
 import org.apache.giraph.conf.GiraphConstants;
 import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
+import org.apache.giraph.edge.ByteArrayEdges;
+import org.apache.giraph.edge.VertexEdges;
 import org.apache.giraph.graph.DefaultVertexResolver;
+import org.apache.giraph.graph.DefaultVertexValueFactory;
+import org.apache.giraph.graph.Vertex;
 import org.apache.giraph.graph.VertexResolver;
+import org.apache.giraph.graph.VertexValueFactory;
 import org.apache.giraph.io.EdgeInputFormat;
 import org.apache.giraph.io.VertexInputFormat;
 import org.apache.giraph.io.VertexOutputFormat;
 import org.apache.giraph.utils.ReflectionUtils;
-import org.apache.giraph.edge.ByteArrayEdges;
-import org.apache.giraph.graph.Vertex;
-import org.apache.giraph.edge.VertexEdges;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparable;
@@ -71,6 +73,8 @@ public class GiraphConfigurationValidator<I extends WritableComparable,
   private static final int EDGE_PARAM_EDGE_INPUT_FORMAT_INDEX = 1;
   /** E param vertex edges index in classList */
   private static final int EDGE_PARAM_VERTEX_EDGES_INDEX = 1;
+  /** V param vertex value factory index in classList */
+  private static final int VALUE_PARAM_VERTEX_VALUE_FACTORY_INDEX = 0;
 
   /** Vertex Index Type */
   private Type vertexIndexType;
@@ -118,6 +122,7 @@ public class GiraphConfigurationValidator<I extends WritableComparable,
     verifyVertexOutputFormatGenericTypes();
     verifyVertexResolverGenericTypes();
     verifyVertexCombinerGenericTypes();
+    verifyVertexValueFactoryGenericTypes();
   }
 
   /**
@@ -315,6 +320,26 @@ public class GiraphConfigurationValidator<I extends WritableComparable,
     }
   }
 
+  /** Verify that the vertex value factory's type matches the job */
+  private void verifyVertexValueFactoryGenericTypes() {
+    Class<? extends VertexValueFactory<V>>
+        vvfClass = conf.getVertexValueFactoryClass();
+    if (DefaultVertexValueFactory.class.equals(vvfClass)) {
+      return;
+    }
+    List<Class<?>> classList = ReflectionUtils.getTypeArguments(
+        VertexValueFactory.class, vvfClass);
+    if (classList.get(VALUE_PARAM_VERTEX_VALUE_FACTORY_INDEX) != null &&
+        !vertexValueType.equals(
+            classList.get(VALUE_PARAM_VERTEX_VALUE_FACTORY_INDEX))) {
+      throw new IllegalArgumentException(
+          "checkClassTypes: Vertex value types don't match, " +
+              "vertex - " + vertexValueType +
+              ", vertex value factory - " +
+              classList.get(VALUE_PARAM_VERTEX_VALUE_FACTORY_INDEX));
+    }
+  }
+
   /** If there is a vertex resolver,
    * validate the generic parameter types. */
   private void verifyVertexResolverGenericTypes() {

http://git-wip-us.apache.org/repos/asf/giraph/blob/95ce243f/giraph-core/src/main/java/org/apache/giraph/utils/ConfigurationUtils.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/utils/ConfigurationUtils.java b/giraph-core/src/main/java/org/apache/giraph/utils/ConfigurationUtils.java
index 41238d0..cb2a2f7 100644
--- a/giraph-core/src/main/java/org/apache/giraph/utils/ConfigurationUtils.java
+++ b/giraph-core/src/main/java/org/apache/giraph/utils/ConfigurationUtils.java
@@ -30,6 +30,7 @@ import org.apache.giraph.aggregators.AggregatorWriter;
 import org.apache.giraph.combiner.Combiner;
 import org.apache.giraph.conf.GiraphConfiguration;
 import org.apache.giraph.conf.GiraphConstants;
+import org.apache.giraph.graph.VertexValueFactory;
 import org.apache.giraph.io.EdgeInputFormat;
 import org.apache.giraph.io.VertexInputFormat;
 import org.apache.giraph.io.VertexOutputFormat;
@@ -80,6 +81,8 @@ public final class ConfigurationUtils {
     OPTIONS.addOption("mc", "masterCompute", true, "MasterCompute class");
     OPTIONS.addOption("cf", "cacheFile", true, "Files for distributed cache");
     OPTIONS.addOption("pc", "partitionClass", true, "Partition class");
+    OPTIONS.addOption("vvf", "vertexValueFactoryClass", true,
+        "Vertex value factory class");
     OPTIONS.addOption("ca", "customArguments", true, "provide custom" +
         " arguments for the job configuration in the form:" +
         " -ca <param1>=<value1>,<param2>=<value2> -ca <param3>=<value3> etc." +
@@ -250,6 +253,11 @@ public final class ConfigurationUtils {
           (Class<? extends Partition>)
               Class.forName(cmd.getOptionValue("pc")));
     }
+    if (cmd.hasOption("vvf")) {
+      giraphConfiguration.setVertexValueFactoryClass(
+          (Class<? extends VertexValueFactory>)
+              Class.forName(cmd.getOptionValue("vvf")));
+    }
     if (cmd.hasOption("ca")) {
       for (String caOptionValue : cmd.getOptionValues("ca")) {
         for (String paramValue :

http://git-wip-us.apache.org/repos/asf/giraph/blob/95ce243f/giraph-core/src/main/java/org/apache/giraph/utils/InternalVertexRunner.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/utils/InternalVertexRunner.java b/giraph-core/src/main/java/org/apache/giraph/utils/InternalVertexRunner.java
index 7e0b955..abf6950 100644
--- a/giraph-core/src/main/java/org/apache/giraph/utils/InternalVertexRunner.java
+++ b/giraph-core/src/main/java/org/apache/giraph/utils/InternalVertexRunner.java
@@ -126,6 +126,7 @@ public class InternalVertexRunner {
       GiraphConfiguration conf = job.getConfiguration();
       conf.setVertexClass(classes.getVertexClass());
       conf.setVertexEdgesClass(classes.getVertexEdgesClass());
+      conf.setVertexValueFactoryClass(classes.getVertexValueFactoryClass());
       if (classes.hasVertexInputFormat()) {
         conf.setVertexInputFormatClass(classes.getVertexInputFormatClass());
       }

http://git-wip-us.apache.org/repos/asf/giraph/blob/95ce243f/giraph-core/src/test/java/org/apache/giraph/io/TestEdgeInput.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/test/java/org/apache/giraph/io/TestEdgeInput.java b/giraph-core/src/test/java/org/apache/giraph/io/TestEdgeInput.java
index 55c0c86..bfc7e8b 100644
--- a/giraph-core/src/test/java/org/apache/giraph/io/TestEdgeInput.java
+++ b/giraph-core/src/test/java/org/apache/giraph/io/TestEdgeInput.java
@@ -22,6 +22,8 @@ import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Maps;
 import org.apache.giraph.BspCase;
 import org.apache.giraph.conf.GiraphClasses;
+import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
+import org.apache.giraph.graph.VertexValueFactory;
 import org.apache.giraph.io.formats.IdWithValueTextOutputFormat;
 import org.apache.giraph.io.formats.IntIntTextVertexValueInputFormat;
 import org.apache.giraph.io.formats.IntNullReverseTextEdgeInputFormat;
@@ -152,6 +154,15 @@ public class TestEdgeInput extends BspCase {
     // A vertex with edges but no initial value should have the default value
     assertEquals(0, (int) values.get(5));
 
+    // Run a job with a custom VertexValueFactory
+    classes.setVertexValueFactoryClass(TestVertexValueFactory.class);
+    results = InternalVertexRunner.run(classes, emptyParams,
+        vertices, edges);
+    values = parseResults(results);
+    // A vertex with edges but no initial value should have been constructed
+    // by the custom factory
+    assertEquals(3, (int) values.get(5));
+
     classes = new GiraphClasses();
     classes.setVertexClass(TestVertexWithNumEdges.class);
     classes.setVertexEdgesClass(ByteArrayEdges.class);
@@ -189,6 +200,18 @@ public class TestEdgeInput extends BspCase {
     }
   }
 
+  public static class TestVertexValueFactory
+      implements VertexValueFactory<IntWritable> {
+    @Override
+    public void initialize(ImmutableClassesGiraphConfiguration<?, IntWritable,
+        ?, ?> configuration) { }
+
+    @Override
+    public IntWritable createVertexValue() {
+      return new IntWritable(3);
+    }
+  }
+
   private static Map<Integer, Integer> parseResults(Iterable<String> results) {
     Map<Integer, Integer> values = Maps.newHashMap();
     for (String line : results) {

http://git-wip-us.apache.org/repos/asf/giraph/blob/95ce243f/giraph-examples/src/test/java/org/apache/giraph/vertex/TestVertexTypes.java
----------------------------------------------------------------------
diff --git a/giraph-examples/src/test/java/org/apache/giraph/vertex/TestVertexTypes.java b/giraph-examples/src/test/java/org/apache/giraph/vertex/TestVertexTypes.java
index c47c291..5ca55b6 100644
--- a/giraph-examples/src/test/java/org/apache/giraph/vertex/TestVertexTypes.java
+++ b/giraph-examples/src/test/java/org/apache/giraph/vertex/TestVertexTypes.java
@@ -18,19 +18,20 @@
 
 package org.apache.giraph.vertex;
 
+import org.apache.giraph.combiner.Combiner;
 import org.apache.giraph.conf.GiraphConstants;
 import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
 import org.apache.giraph.edge.ByteArrayEdges;
 import org.apache.giraph.edge.VertexEdges;
 import org.apache.giraph.examples.SimpleSuperstepVertex.SimpleSuperstepVertexInputFormat;
-import org.apache.giraph.combiner.Combiner;
 import org.apache.giraph.graph.Vertex;
-import org.apache.giraph.job.GiraphConfigurationValidator;
+import org.apache.giraph.graph.VertexValueFactory;
 import org.apache.giraph.io.VertexInputFormat;
 import org.apache.giraph.io.VertexOutputFormat;
 import org.apache.giraph.io.formats.GeneratedVertexInputFormat;
 import org.apache.giraph.io.formats.JsonBase64VertexInputFormat;
 import org.apache.giraph.io.formats.JsonBase64VertexOutputFormat;
+import org.apache.giraph.job.GiraphConfigurationValidator;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.DoubleWritable;
 import org.apache.hadoop.io.FloatWritable;
@@ -106,6 +107,23 @@ public class TestVertexTypes {
     }
 
     /**
+     * Mismatches the {@link GeneratedVertexMatch}
+     */
+    private static class GeneratedVertexMismatchValueFactory implements
+        VertexValueFactory<DoubleWritable> {
+
+      @Override
+      public void initialize(
+          ImmutableClassesGiraphConfiguration<?, DoubleWritable, ?, ?>
+              configuration) {}
+
+      @Override
+      public DoubleWritable createVertexValue() {
+        return new DoubleWritable();
+      }
+    }
+
+    /**
      * Just populate a conf with testing defaults that won't
      * upset the GiraphConfigurationValidator.
      * */
@@ -223,6 +241,27 @@ public class TestVertexTypes {
       validator.validateConfiguration();
     }
 
+    @Test(expected = IllegalArgumentException.class)
+    public void testMismatchingVertexValueFactory() throws SecurityException,
+        NoSuchMethodException, NoSuchFieldException {
+      Configuration conf = getDefaultTestConf() ;
+      conf.setClass(GiraphConstants.VERTEX_CLASS,
+          GeneratedVertexMatch.class, Vertex.class);
+      conf.setClass(GiraphConstants.VERTEX_EDGES_CLASS,
+          ByteArrayEdges.class,
+          VertexEdges.class);
+      conf.setClass(GiraphConstants.VERTEX_INPUT_FORMAT_CLASS,
+          SimpleSuperstepVertexInputFormat.class,
+          VertexInputFormat.class);
+      conf.setClass(GiraphConstants.VERTEX_VALUE_FACTORY_CLASS,
+          GeneratedVertexMismatchValueFactory.class,
+          VertexValueFactory.class);
+      @SuppressWarnings("rawtypes")
+      GiraphConfigurationValidator<?, ?, ?, ?> validator =
+          new GiraphConfigurationValidator(conf);
+      validator.validateConfiguration();
+    }
+
     @Test
     public void testJsonBase64FormatType() throws SecurityException,
             NoSuchMethodException, NoSuchFieldException {