You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@giraph.apache.org by jg...@apache.org on 2012/08/10 20:49:46 UTC

svn commit: r1371807 - in /giraph/trunk: ./ src/main/java/org/apache/giraph/ src/main/java/org/apache/giraph/graph/ src/main/java/org/apache/giraph/utils/ src/test/java/org/apache/giraph/

Author: jghoman
Date: Fri Aug 10 18:49:45 2012
New Revision: 1371807

URL: http://svn.apache.org/viewvc?rev=1371807&view=rev
Log:
GIRAPH-258. Check type compatibility before submitting job. Contributed by Eli Reisman.

Added:
    giraph/trunk/src/main/java/org/apache/giraph/graph/GiraphTypeValidator.java
Modified:
    giraph/trunk/CHANGELOG
    giraph/trunk/src/main/java/org/apache/giraph/GiraphRunner.java
    giraph/trunk/src/main/java/org/apache/giraph/graph/GraphMapper.java
    giraph/trunk/src/main/java/org/apache/giraph/utils/ReflectionUtils.java
    giraph/trunk/src/test/java/org/apache/giraph/TestVertexTypes.java

Modified: giraph/trunk/CHANGELOG
URL: http://svn.apache.org/viewvc/giraph/trunk/CHANGELOG?rev=1371807&r1=1371806&r2=1371807&view=diff
==============================================================================
--- giraph/trunk/CHANGELOG (original)
+++ giraph/trunk/CHANGELOG Fri Aug 10 18:49:45 2012
@@ -2,6 +2,9 @@ Giraph Change Log
 
 Release 0.2.0 - unreleased
 
+  GIRAPH-258: Check type compatibility before submitting job. 
+  (Eli Reisman via jghoman)
+
   GIRAPH-218: Consolidate all I/O Format classes under one roof in lib/ directory.
   (Eli Reisman via jghoman)
 

Modified: giraph/trunk/src/main/java/org/apache/giraph/GiraphRunner.java
URL: http://svn.apache.org/viewvc/giraph/trunk/src/main/java/org/apache/giraph/GiraphRunner.java?rev=1371807&r1=1371806&r2=1371807&view=diff
==============================================================================
--- giraph/trunk/src/main/java/org/apache/giraph/GiraphRunner.java (original)
+++ giraph/trunk/src/main/java/org/apache/giraph/GiraphRunner.java Fri Aug 10 18:49:45 2012
@@ -24,6 +24,7 @@ import org.apache.commons.cli.HelpFormat
 import org.apache.commons.cli.Options;
 import org.apache.giraph.examples.Algorithm;
 import org.apache.giraph.graph.GiraphJob;
+import org.apache.giraph.graph.GiraphTypeValidator;
 import org.apache.giraph.graph.Vertex;
 import org.apache.giraph.utils.AnnotationUtils;
 import org.apache.hadoop.conf.Configuration;
@@ -217,6 +218,13 @@ public class GiraphRunner implements Too
       }
     }
 
+    // validate generic parameters chosen are correct or
+    // throw IllegalArgumentException, halting execution.
+    @SuppressWarnings("rawtypes")
+    GiraphTypeValidator<?, ?, ?, ?> validator =
+      new GiraphTypeValidator(job.getConfiguration());
+    validator.validateClassTypes();
+
     job.setWorkerConfiguration(workers, workers, 100.0f);
 
     boolean verbose = !cmd.hasOption('q');

Added: giraph/trunk/src/main/java/org/apache/giraph/graph/GiraphTypeValidator.java
URL: http://svn.apache.org/viewvc/giraph/trunk/src/main/java/org/apache/giraph/graph/GiraphTypeValidator.java?rev=1371807&view=auto
==============================================================================
--- giraph/trunk/src/main/java/org/apache/giraph/graph/GiraphTypeValidator.java (added)
+++ giraph/trunk/src/main/java/org/apache/giraph/graph/GiraphTypeValidator.java Fri Aug 10 18:49:45 2012
@@ -0,0 +1,237 @@
+/*
+ * 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.utils.ReflectionUtils;
+
+import org.apache.hadoop.io.WritableComparable;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.conf.Configuration;
+
+import java.lang.reflect.Type;
+import java.util.List;
+
+import org.apache.log4j.Logger;
+
+/**
+ * GiraphTypeValidator attempts to verify the consistency of
+ * user-chosen InputFormat, OutputFormat, and Vertex type
+ * parameters before the job run actually begins.
+ *
+ * @param <I> the Vertex ID type
+ * @param <V> the Vertex Value type
+ * @param <E> the Edge Value type
+ * @param <M> the Message type
+ */
+public class GiraphTypeValidator<I extends WritableComparable,
+  V extends Writable, E extends Writable, M extends Writable> {
+  /**
+   * Class logger object.
+   */
+  private static Logger LOG =
+    Logger.getLogger(GiraphTypeValidator.class);
+
+  /** I param vertex index in classList */
+  private static final int ID_PARAM_INDEX = 0;
+  /** V param vertex index in classList */
+  private static final int VALUE_PARAM_INDEX = 1;
+  /** E param vertex index in classList */
+  private static final int EDGE_PARAM_INDEX = 2;
+  /** M param vertex index in classList */
+  private static final int MSG_PARAM_INDEX = 3;
+  /** M param vertex combiner index in classList */
+  private static final int MSG_COMBINER_PARAM_INDEX = 1;
+
+  /** Vertex Index Type */
+  private Type vertexIndexType;
+  /** Vertex Index Type */
+  private Type vertexValueType;
+  /** Vertex Index Type */
+  private Type edgeValueType;
+  /** Vertex Index Type */
+  private Type messageValueType;
+
+  /**
+   * The Configuration object for use in the validation test.
+   */
+  private Configuration conf;
+
+  /**
+   * Constructor to execute the validation test, throws
+   * unchecked exception to end job run on failure.
+   *
+   * @param conf the Configuration for this run.
+   */
+  public GiraphTypeValidator(Configuration conf) {
+    this.conf = conf;
+  }
+
+  /**
+   * Make sure that all registered classes have matching types.  This
+   * is a little tricky due to type erasure, cannot simply get them from
+   * the class type arguments.  Also, set the vertex index, vertex value,
+   * edge value and message value classes.
+   */
+  public void validateClassTypes() {
+    Class<? extends Vertex<I, V, E, M>> vertexClass =
+      BspUtils.<I, V, E, M>getVertexClass(conf);
+    List<Class<?>> classList = ReflectionUtils.<Vertex>getTypeArguments(
+      Vertex.class, vertexClass);
+    vertexIndexType = classList.get(ID_PARAM_INDEX);
+    vertexValueType = classList.get(VALUE_PARAM_INDEX);
+    edgeValueType = classList.get(EDGE_PARAM_INDEX);
+    messageValueType = classList.get(MSG_PARAM_INDEX);
+    verifyVertexInputFormatGenericTypes();
+    verifyVertexOutputFormatGenericTypes();
+    verifyVertexResolverGenericTypes();
+    verifyVertexCombinerGenericTypes();
+  }
+
+  /** Verify matching generic types in VertexInputFormat. */
+  private void verifyVertexInputFormatGenericTypes() {
+    Class<? extends VertexInputFormat<I, V, E, M>> vertexInputFormatClass =
+      BspUtils.<I, V, E, M>getVertexInputFormatClass(conf);
+    List<Class<?>> classList =
+      ReflectionUtils.<VertexInputFormat>getTypeArguments(
+        VertexInputFormat.class, vertexInputFormatClass);
+    if (classList.get(ID_PARAM_INDEX) == null) {
+      LOG.warn("Input format vertex index type is not known");
+    } else if (!vertexIndexType.equals(classList.get(ID_PARAM_INDEX))) {
+      throw new IllegalArgumentException(
+        "checkClassTypes: Vertex index types don't match, " +
+          "vertex - " + vertexIndexType +
+          ", vertex input format - " + classList.get(ID_PARAM_INDEX));
+    }
+    if (classList.get(VALUE_PARAM_INDEX) == null) {
+      LOG.warn("Input format vertex value type is not known");
+    } else if (!vertexValueType.equals(classList.get(VALUE_PARAM_INDEX))) {
+      throw new IllegalArgumentException(
+        "checkClassTypes: Vertex value types don't match, " +
+          "vertex - " + vertexValueType +
+          ", vertex input format - " + classList.get(VALUE_PARAM_INDEX));
+    }
+    if (classList.get(EDGE_PARAM_INDEX) == null) {
+      LOG.warn("Input format edge value type is not known");
+    } else if (!edgeValueType.equals(classList.get(EDGE_PARAM_INDEX))) {
+      throw new IllegalArgumentException(
+        "checkClassTypes: Edge value types don't match, " +
+          "vertex - " + edgeValueType +
+          ", vertex input format - " + classList.get(EDGE_PARAM_INDEX));
+    }
+  }
+
+  /** If there is a combiner type, verify its generic params match the job. */
+  private void verifyVertexCombinerGenericTypes() {
+    Class<? extends VertexCombiner<I, M>> vertexCombinerClass =
+      BspUtils.<I, M>getVertexCombinerClass(conf);
+    if (vertexCombinerClass != null) {
+      List<Class<?>> classList =
+        ReflectionUtils.<VertexCombiner>getTypeArguments(
+          VertexCombiner.class, vertexCombinerClass);
+      if (!vertexIndexType.equals(classList.get(ID_PARAM_INDEX))) {
+        throw new IllegalArgumentException(
+          "checkClassTypes: Vertex index types don't match, " +
+            "vertex - " + vertexIndexType +
+            ", vertex combiner - " + classList.get(ID_PARAM_INDEX));
+      }
+      if (!messageValueType.equals(classList.get(MSG_COMBINER_PARAM_INDEX))) {
+        throw new IllegalArgumentException(
+          "checkClassTypes: Message value types don't match, " +
+            "vertex - " + messageValueType +
+            ", vertex combiner - " + classList.get(MSG_COMBINER_PARAM_INDEX));
+      }
+    }
+  }
+
+  /** Verify that the output format's generic params match the job. */
+  private void verifyVertexOutputFormatGenericTypes() {
+    Class<? extends VertexOutputFormat<I, V, E>>
+      vertexOutputFormatClass =
+      BspUtils.<I, V, E>getVertexOutputFormatClass(conf);
+    if (vertexOutputFormatClass != null) {
+      List<Class<?>> classList =
+        ReflectionUtils.<VertexOutputFormat>getTypeArguments(
+          VertexOutputFormat.class, vertexOutputFormatClass);
+      if (classList.get(ID_PARAM_INDEX) == null) {
+        LOG.warn("Output format vertex index type is not known");
+      } else if (!vertexIndexType.equals(classList.get(ID_PARAM_INDEX))) {
+        throw new IllegalArgumentException(
+          "checkClassTypes: Vertex index types don't match, " +
+            "vertex - " + vertexIndexType +
+            ", vertex output format - " + classList.get(ID_PARAM_INDEX));
+      }
+      if (classList.get(VALUE_PARAM_INDEX) == null) {
+        LOG.warn("Output format vertex value type is not known");
+      } else if (!vertexValueType.equals(classList.get(VALUE_PARAM_INDEX))) {
+        throw new IllegalArgumentException(
+          "checkClassTypes: Vertex value types don't match, " +
+            "vertex - " + vertexValueType +
+            ", vertex output format - " + classList.get(VALUE_PARAM_INDEX));
+      }
+      if (classList.get(EDGE_PARAM_INDEX) == null) {
+        LOG.warn("Output format edge value type is not known");
+      } else if (!edgeValueType.equals(classList.get(EDGE_PARAM_INDEX))) {
+        throw new IllegalArgumentException(
+          "checkClassTypes: Edge value types don't match, " +
+            "vertex - " + edgeValueType +
+            ", vertex output format - " + classList.get(EDGE_PARAM_INDEX));
+      }
+    }
+  }
+
+  /** If there is a vertex resolver,
+   * validate the generic parameter types. */
+  private void verifyVertexResolverGenericTypes() {
+    Class<? extends VertexResolver<I, V, E, M>>
+      vertexResolverClass =
+      BspUtils.<I, V, E, M>getVertexResolverClass(conf);
+    List<Class<?>> classList =
+      ReflectionUtils.<VertexResolver>getTypeArguments(
+        VertexResolver.class, vertexResolverClass);
+    if (classList.get(ID_PARAM_INDEX) != null &&
+      !vertexIndexType.equals(classList.get(ID_PARAM_INDEX))) {
+      throw new IllegalArgumentException(
+        "checkClassTypes: Vertex index types don't match, " +
+          "vertex - " + vertexIndexType +
+          ", vertex resolver - " + classList.get(ID_PARAM_INDEX));
+    }
+    if (classList.get(VALUE_PARAM_INDEX) != null &&
+      !vertexValueType.equals(classList.get(VALUE_PARAM_INDEX))) {
+      throw new IllegalArgumentException(
+        "checkClassTypes: Vertex value types don't match, " +
+          "vertex - " + vertexValueType +
+          ", vertex resolver - " + classList.get(VALUE_PARAM_INDEX));
+    }
+    if (classList.get(EDGE_PARAM_INDEX) != null &&
+      !edgeValueType.equals(classList.get(EDGE_PARAM_INDEX))) {
+      throw new IllegalArgumentException(
+        "checkClassTypes: Edge value types don't match, " +
+          "vertex - " + edgeValueType +
+          ", vertex resolver - " + classList.get(EDGE_PARAM_INDEX));
+    }
+    if (classList.get(MSG_PARAM_INDEX) != null &&
+      !messageValueType.equals(classList.get(MSG_PARAM_INDEX))) {
+      throw new IllegalArgumentException(
+        "checkClassTypes: Message value types don't match, " +
+          "vertex - " + messageValueType +
+          ", vertex resolver - " + classList.get(MSG_PARAM_INDEX));
+    }
+  }
+}
+

Modified: giraph/trunk/src/main/java/org/apache/giraph/graph/GraphMapper.java
URL: http://svn.apache.org/viewvc/giraph/trunk/src/main/java/org/apache/giraph/graph/GraphMapper.java?rev=1371807&r1=1371806&r2=1371807&view=diff
==============================================================================
--- giraph/trunk/src/main/java/org/apache/giraph/graph/GraphMapper.java (original)
+++ giraph/trunk/src/main/java/org/apache/giraph/graph/GraphMapper.java Fri Aug 10 18:49:45 2012
@@ -150,13 +150,41 @@ public class GraphMapper<I extends Writa
   }
 
   /**
-   * Copied from JobConf to get the location of this jar.  Workaround for
-   * things like Oozie map-reduce jobs.
-   *
-   * @param myClass Class to search the class loader path for to locate
-   *        the relevant jar file
-   * @return Location of the jar file containing myClass
+   * Set the concrete, user-defined choices about generic methods
+   * (validated earlier in GiraphRunner) into the Configuration.
+   * @param conf the Configuration object for this job run.
    */
+  public void determineClassTypes(Configuration conf) {
+    Class<? extends Vertex<I, V, E, M>> vertexClass =
+      BspUtils.<I, V, E, M>getVertexClass(conf);
+    List<Class<?>> classList = ReflectionUtils.<Vertex>getTypeArguments(
+      Vertex.class, vertexClass);
+    Type vertexIndexType = classList.get(0);
+    Type vertexValueType = classList.get(1);
+    Type edgeValueType = classList.get(2);
+    Type messageValueType = classList.get(3);
+    conf.setClass(GiraphJob.VERTEX_ID_CLASS,
+      (Class<?>) vertexIndexType,
+      WritableComparable.class);
+    conf.setClass(GiraphJob.VERTEX_VALUE_CLASS,
+      (Class<?>) vertexValueType,
+      Writable.class);
+    conf.setClass(GiraphJob.EDGE_VALUE_CLASS,
+      (Class<?>) edgeValueType,
+      Writable.class);
+    conf.setClass(GiraphJob.MESSAGE_VALUE_CLASS,
+      (Class<?>) messageValueType,
+      Writable.class);
+  }
+
+    /**
+    * Copied from JobConf to get the location of this jar.  Workaround for
+    * things like Oozie map-reduce jobs.
+    *
+    * @param myClass Class to search the class loader path for to locate
+    *        the relevant jar file
+    * @return Location of the jar file containing myClass
+    */
   private static String findContainingJar(Class<?> myClass) {
     ClassLoader loader = myClass.getClassLoader();
     String classFile =
@@ -181,152 +209,6 @@ public class GraphMapper<I extends Writa
   }
 
   /**
-   * Make sure that all registered classes have matching types.  This
-   * is a little tricky due to type erasure, cannot simply get them from
-   * the class type arguments.  Also, set the vertex index, vertex value,
-   * edge value and message value classes.
-   *
-   * @param conf Configuration to get the various classes
-   */
-  public void determineClassTypes(Configuration conf) {
-    Class<? extends Vertex<I, V, E, M>> vertexClass =
-        BspUtils.<I, V, E, M>getVertexClass(conf);
-    List<Class<?>> classList = ReflectionUtils.<Vertex>getTypeArguments(
-        Vertex.class, vertexClass);
-    Type vertexIndexType = classList.get(0);
-    Type vertexValueType = classList.get(1);
-    Type edgeValueType = classList.get(2);
-    Type messageValueType = classList.get(3);
-
-    Class<? extends VertexInputFormat<I, V, E, M>> vertexInputFormatClass =
-        BspUtils.<I, V, E, M>getVertexInputFormatClass(conf);
-    classList = ReflectionUtils.<VertexInputFormat>getTypeArguments(
-        VertexInputFormat.class, vertexInputFormatClass);
-    if (classList.get(0) == null) {
-      LOG.warn("Input format vertex index type is not known");
-    } else if (!vertexIndexType.equals(classList.get(0))) {
-      throw new IllegalArgumentException(
-          "checkClassTypes: Vertex index types don't match, " +
-              "vertex - " + vertexIndexType +
-              ", vertex input format - " + classList.get(0));
-    }
-    if (classList.get(1) == null) {
-      LOG.warn("Input format vertex value type is not known");
-    } else if (!vertexValueType.equals(classList.get(1))) {
-      throw new IllegalArgumentException(
-          "checkClassTypes: Vertex value types don't match, " +
-              "vertex - " + vertexValueType +
-              ", vertex input format - " + classList.get(1));
-    }
-    if (classList.get(2) == null) {
-      LOG.warn("Input format edge value type is not known");
-    } else if (!edgeValueType.equals(classList.get(2))) {
-      throw new IllegalArgumentException(
-          "checkClassTypes: Edge value types don't match, " +
-              "vertex - " + edgeValueType +
-              ", vertex input format - " + classList.get(2));
-    }
-    // If has vertex combiner class, check
-    Class<? extends VertexCombiner<I, M>> vertexCombinerClass =
-        BspUtils.<I, M>getVertexCombinerClass(conf);
-    if (vertexCombinerClass != null) {
-      classList = ReflectionUtils.<VertexCombiner>getTypeArguments(
-          VertexCombiner.class, vertexCombinerClass);
-      if (!vertexIndexType.equals(classList.get(0))) {
-        throw new IllegalArgumentException(
-            "checkClassTypes: Vertex index types don't match, " +
-                "vertex - " + vertexIndexType +
-                ", vertex combiner - " + classList.get(0));
-      }
-      if (!messageValueType.equals(classList.get(1))) {
-        throw new IllegalArgumentException(
-            "checkClassTypes: Message value types don't match, " +
-                "vertex - " + vertexValueType +
-                ", vertex combiner - " + classList.get(1));
-      }
-    }
-    // If has vertex output format class, check
-    Class<? extends VertexOutputFormat<I, V, E>>
-    vertexOutputFormatClass =
-      BspUtils.<I, V, E>getVertexOutputFormatClass(conf);
-    if (vertexOutputFormatClass != null) {
-      classList =
-          ReflectionUtils.<VertexOutputFormat>getTypeArguments(
-              VertexOutputFormat.class, vertexOutputFormatClass);
-      if (classList.get(0) == null) {
-        LOG.warn("Output format vertex index type is not known");
-      } else if (!vertexIndexType.equals(classList.get(0))) {
-        throw new IllegalArgumentException(
-            "checkClassTypes: Vertex index types don't match, " +
-                "vertex - " + vertexIndexType +
-                ", vertex output format - " + classList.get(0));
-      }
-      if (classList.get(1) == null) {
-        LOG.warn("Output format vertex value type is not known");
-      } else if (!vertexValueType.equals(classList.get(1))) {
-        throw new IllegalArgumentException(
-            "checkClassTypes: Vertex value types don't match, " +
-                "vertex - " + vertexValueType +
-                ", vertex output format - " + classList.get(1));
-      }
-      if (classList.get(2) == null) {
-        LOG.warn("Output format edge value type is not known");
-      } else if (!edgeValueType.equals(classList.get(2))) {
-        throw new IllegalArgumentException(
-            "checkClassTypes: Edge value types don't match, " +
-                "vertex - " + vertexIndexType +
-                ", vertex output format - " + classList.get(2));
-      }
-    }
-    // Vertex resolver might never select the types
-    Class<? extends VertexResolver<I, V, E, M>>
-    vertexResolverClass =
-      BspUtils.<I, V, E, M>getVertexResolverClass(conf);
-    classList = ReflectionUtils.<VertexResolver>getTypeArguments(
-        VertexResolver.class, vertexResolverClass);
-    if (classList.get(0) != null &&
-        !vertexIndexType.equals(classList.get(0))) {
-      throw new IllegalArgumentException(
-          "checkClassTypes: Vertex index types don't match, " +
-              "vertex - " + vertexIndexType +
-              ", vertex resolver - " + classList.get(0));
-    }
-    if (classList.get(1) != null &&
-        !vertexValueType.equals(classList.get(1))) {
-      throw new IllegalArgumentException(
-          "checkClassTypes: Vertex value types don't match, " +
-              "vertex - " + vertexValueType +
-              ", vertex resolver - " + classList.get(1));
-    }
-    if (classList.get(2) != null &&
-        !edgeValueType.equals(classList.get(2))) {
-      throw new IllegalArgumentException(
-          "checkClassTypes: Edge value types don't match, " +
-              "vertex - " + edgeValueType +
-              ", vertex resolver - " + classList.get(2));
-    }
-    if (classList.get(3) != null &&
-        !messageValueType.equals(classList.get(3))) {
-      throw new IllegalArgumentException(
-          "checkClassTypes: Message value types don't match, " +
-              "vertex - " + edgeValueType +
-              ", vertex resolver - " + classList.get(3));
-    }
-    conf.setClass(GiraphJob.VERTEX_ID_CLASS,
-        (Class<?>) vertexIndexType,
-        WritableComparable.class);
-    conf.setClass(GiraphJob.VERTEX_VALUE_CLASS,
-        (Class<?>) vertexValueType,
-        Writable.class);
-    conf.setClass(GiraphJob.EDGE_VALUE_CLASS,
-        (Class<?>) edgeValueType,
-        Writable.class);
-    conf.setClass(GiraphJob.MESSAGE_VALUE_CLASS,
-        (Class<?>) messageValueType,
-        Writable.class);
-  }
-
-  /**
    * Figure out what functions this mapper should do.  Basic logic is as
    * follows:
    * 1) If not split master, everyone does the everything and/or running
@@ -391,7 +273,7 @@ public class GraphMapper<I extends Writa
       conf.set("mapreduce.job.credentials.binary",
           System.getenv("HADOOP_TOKEN_FILE_LOCATION"));
     }
-    // Ensure the user classes have matching types and figure them out
+    // set pre-validated generic parameter types into Configuration
     determineClassTypes(conf);
 
     // Do some initial setup (possibly starting up a Zookeeper service)

Modified: giraph/trunk/src/main/java/org/apache/giraph/utils/ReflectionUtils.java
URL: http://svn.apache.org/viewvc/giraph/trunk/src/main/java/org/apache/giraph/utils/ReflectionUtils.java?rev=1371807&r1=1371806&r2=1371807&view=diff
==============================================================================
--- giraph/trunk/src/main/java/org/apache/giraph/utils/ReflectionUtils.java (original)
+++ giraph/trunk/src/main/java/org/apache/giraph/utils/ReflectionUtils.java Fri Aug 10 18:49:45 2012
@@ -24,6 +24,7 @@ import java.lang.reflect.GenericArrayTyp
 import java.lang.reflect.ParameterizedType;
 import java.lang.reflect.Type;
 import java.lang.reflect.TypeVariable;
+
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;

Modified: giraph/trunk/src/test/java/org/apache/giraph/TestVertexTypes.java
URL: http://svn.apache.org/viewvc/giraph/trunk/src/test/java/org/apache/giraph/TestVertexTypes.java?rev=1371807&r1=1371806&r2=1371807&view=diff
==============================================================================
--- giraph/trunk/src/test/java/org/apache/giraph/TestVertexTypes.java (original)
+++ giraph/trunk/src/test/java/org/apache/giraph/TestVertexTypes.java Fri Aug 10 18:49:45 2012
@@ -22,7 +22,7 @@ import org.apache.giraph.io.GeneratedVer
 import org.apache.giraph.examples.SimpleSuperstepVertex.SimpleSuperstepVertexInputFormat;
 import org.apache.giraph.graph.EdgeListVertex;
 import org.apache.giraph.graph.GiraphJob;
-import org.apache.giraph.graph.GraphMapper;
+import org.apache.giraph.graph.GiraphTypeValidator;
 import org.apache.giraph.graph.Vertex;
 import org.apache.giraph.graph.VertexCombiner;
 import org.apache.giraph.graph.VertexInputFormat;
@@ -36,7 +36,7 @@ import org.apache.hadoop.io.FloatWritabl
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.LongWritable;
 import org.junit.Test;
-
+import org.junit.Assert;
 import java.io.IOException;
 
 
@@ -77,7 +77,6 @@ public class TestVertexTypes {
      */
     private static class GeneratedVertexMatchCombiner extends
             VertexCombiner<LongWritable, FloatWritable> {
-
         @Override
         public Iterable<FloatWritable> combine(LongWritable vertexIndex,
                 Iterable<FloatWritable> msgList) throws IOException {
@@ -90,7 +89,6 @@ public class TestVertexTypes {
      */
     private static class GeneratedVertexMismatchCombiner extends
             VertexCombiner<LongWritable, DoubleWritable> {
-
         @Override
         public Iterable<DoubleWritable> combine(LongWritable vertexIndex,
                 Iterable<DoubleWritable> msgList)
@@ -102,8 +100,6 @@ public class TestVertexTypes {
     @Test
     public void testMatchingType() throws SecurityException,
             NoSuchMethodException, NoSuchFieldException {
-        @SuppressWarnings("rawtypes")
-        GraphMapper<?, ?, ?, ?> mapper = new GraphMapper();
         Configuration conf = new Configuration();
         conf.setClass(GiraphJob.VERTEX_CLASS,
                       GeneratedVertexMatch.class,
@@ -114,14 +110,15 @@ public class TestVertexTypes {
         conf.setClass(GiraphJob.VERTEX_COMBINER_CLASS,
                       GeneratedVertexMatchCombiner.class,
                       VertexCombiner.class);
-        mapper.determineClassTypes(conf);
+      @SuppressWarnings("rawtypes")
+      GiraphTypeValidator<?, ?, ?, ?> validator =
+        new GiraphTypeValidator(conf);
+      validator.validateClassTypes();
     }
 
     @Test
     public void testDerivedMatchingType() throws SecurityException,
             NoSuchMethodException, NoSuchFieldException {
-        @SuppressWarnings("rawtypes")
-        GraphMapper<?, ?, ?, ?> mapper = new GraphMapper();
         Configuration conf = new Configuration();
         conf.setClass(GiraphJob.VERTEX_CLASS,
                       DerivedVertexMatch.class,
@@ -129,14 +126,15 @@ public class TestVertexTypes {
         conf.setClass(GiraphJob.VERTEX_INPUT_FORMAT_CLASS,
                       SimpleSuperstepVertexInputFormat.class,
                       VertexInputFormat.class);
-        mapper.determineClassTypes(conf);
+        @SuppressWarnings("rawtypes")
+        GiraphTypeValidator<?, ?, ?, ?> validator =
+          new GiraphTypeValidator(conf);
+        validator.validateClassTypes();
     }
 
     @Test
     public void testDerivedInputFormatType() throws SecurityException,
             NoSuchMethodException, NoSuchFieldException {
-        @SuppressWarnings("rawtypes")
-        GraphMapper<?, ?, ?, ?> mapper = new GraphMapper();
         Configuration conf = new Configuration();
         conf.setClass(GiraphJob.VERTEX_CLASS,
                       DerivedVertexMatch.class,
@@ -144,57 +142,49 @@ public class TestVertexTypes {
         conf.setClass(GiraphJob.VERTEX_INPUT_FORMAT_CLASS,
                       SimpleSuperstepVertexInputFormat.class,
                       VertexInputFormat.class);
-        mapper.determineClassTypes(conf);
+      @SuppressWarnings("rawtypes")
+      GiraphTypeValidator<?, ?, ?, ?> validator =
+        new GiraphTypeValidator(conf);
+      validator.validateClassTypes();
     }
 
-    @Test
+    @Test(expected = IllegalArgumentException.class)
     public void testMismatchingVertex() throws SecurityException,
-            NoSuchMethodException, NoSuchFieldException {
-        @SuppressWarnings("rawtypes")
-        GraphMapper<?, ?, ?, ?> mapper = new GraphMapper();
-        Configuration conf = new Configuration();
-        conf.setClass(GiraphJob.VERTEX_CLASS,
-                      GeneratedVertexMismatch.class,
-                      Vertex.class);
+      NoSuchMethodException, NoSuchFieldException {
+      Configuration conf = new Configuration();
+      conf.setClass(GiraphJob.VERTEX_CLASS,
+        GeneratedVertexMismatch.class,
+        Vertex.class);
         conf.setClass(GiraphJob.VERTEX_INPUT_FORMAT_CLASS,
-                      SimpleSuperstepVertexInputFormat.class,
-                      VertexInputFormat.class);
-        try {
-            mapper.determineClassTypes(conf);
-            throw new RuntimeException(
-                "testMismatchingVertex: Should have caught an exception!");
-        } catch (IllegalArgumentException e) {
-        }
+          SimpleSuperstepVertexInputFormat.class,
+          VertexInputFormat.class);
+        @SuppressWarnings("rawtypes")
+        GiraphTypeValidator<?, ?, ?, ?> validator =
+          new GiraphTypeValidator(conf);
+        validator.validateClassTypes();
     }
 
-    @Test
+    @Test(expected = IllegalArgumentException.class)
     public void testMismatchingCombiner() throws SecurityException,
-            NoSuchMethodException, NoSuchFieldException {
-        @SuppressWarnings("rawtypes")
-        GraphMapper<?, ?, ?, ?> mapper = new GraphMapper();
-        Configuration conf = new Configuration();
-        conf.setClass(GiraphJob.VERTEX_CLASS,
-                      GeneratedVertexMatch.class,
-                      Vertex.class);
-        conf.setClass(GiraphJob.VERTEX_INPUT_FORMAT_CLASS,
-                      SimpleSuperstepVertexInputFormat.class,
-                      VertexInputFormat.class);
-        conf.setClass(GiraphJob.VERTEX_COMBINER_CLASS,
-                      GeneratedVertexMismatchCombiner.class,
-                      VertexCombiner.class);
-        try {
-            mapper.determineClassTypes(conf);
-            throw new RuntimeException(
-                "testMismatchingCombiner: Should have caught an exception!");
-        } catch (IllegalArgumentException e) {
-        }
+      NoSuchMethodException, NoSuchFieldException {
+      Configuration conf = new Configuration();
+      conf.setClass(GiraphJob.VERTEX_CLASS,
+        GeneratedVertexMatch.class, Vertex.class);
+      conf.setClass(GiraphJob.VERTEX_INPUT_FORMAT_CLASS,
+        SimpleSuperstepVertexInputFormat.class,
+        VertexInputFormat.class);
+      conf.setClass(GiraphJob.VERTEX_COMBINER_CLASS,
+        GeneratedVertexMismatchCombiner.class,
+        VertexCombiner.class);
+      @SuppressWarnings("rawtypes")
+      GiraphTypeValidator<?, ?, ?, ?> validator =
+        new GiraphTypeValidator(conf);
+      validator.validateClassTypes();
     }
 
     @Test
     public void testJsonBase64FormatType() throws SecurityException,
             NoSuchMethodException, NoSuchFieldException {
-        @SuppressWarnings("rawtypes")
-        GraphMapper<?, ?, ?, ?> mapper = new GraphMapper();
         Configuration conf = new Configuration();
         conf.setClass(GiraphJob.VERTEX_CLASS,
                       GeneratedVertexMatch.class,
@@ -205,6 +195,9 @@ public class TestVertexTypes {
         conf.setClass(GiraphJob.VERTEX_OUTPUT_FORMAT_CLASS,
                       JsonBase64VertexOutputFormat.class,
                       VertexOutputFormat.class);
-        mapper.determineClassTypes(conf);
+        @SuppressWarnings("rawtypes")
+        GiraphTypeValidator<?, ?, ?, ?> validator =
+          new GiraphTypeValidator(conf);
+        validator.validateClassTypes();
     }
 }