giraph-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From ni...@apache.org
Subject [2/6] GIRAPH-447: Clean up GiraphConfiguration related things (nitay).
Date Thu, 13 Dec 2012 20:03:37 GMT
http://git-wip-us.apache.org/repos/asf/giraph/blob/599f70e3/giraph/src/main/java/org/apache/giraph/ImmutableClassesGiraphConfiguration.java
----------------------------------------------------------------------
diff --git a/giraph/src/main/java/org/apache/giraph/ImmutableClassesGiraphConfiguration.java b/giraph/src/main/java/org/apache/giraph/ImmutableClassesGiraphConfiguration.java
deleted file mode 100644
index da43b11..0000000
--- a/giraph/src/main/java/org/apache/giraph/ImmutableClassesGiraphConfiguration.java
+++ /dev/null
@@ -1,618 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.giraph;
-
-
-import org.apache.giraph.graph.AggregatorWriter;
-import org.apache.giraph.graph.Combiner;
-import org.apache.giraph.graph.DefaultMasterCompute;
-import org.apache.giraph.graph.DefaultVertexResolver;
-import org.apache.giraph.graph.DefaultWorkerContext;
-import org.apache.giraph.graph.EdgeInputFormat;
-import org.apache.giraph.graph.GraphState;
-import org.apache.giraph.graph.MasterCompute;
-import org.apache.giraph.graph.TextAggregatorWriter;
-import org.apache.giraph.graph.Vertex;
-import org.apache.giraph.graph.VertexInputFormat;
-import org.apache.giraph.graph.VertexOutputFormat;
-import org.apache.giraph.graph.VertexResolver;
-import org.apache.giraph.graph.WorkerContext;
-import org.apache.giraph.graph.partition.GraphPartitionerFactory;
-import org.apache.giraph.graph.partition.HashPartitionerFactory;
-import org.apache.giraph.graph.partition.MasterGraphPartitioner;
-import org.apache.giraph.graph.partition.Partition;
-import org.apache.giraph.graph.partition.PartitionStats;
-import org.apache.giraph.graph.partition.SimplePartition;
-import org.apache.giraph.master.MasterObserver;
-import org.apache.giraph.utils.ExtendedByteArrayDataInput;
-import org.apache.giraph.utils.ExtendedByteArrayDataOutput;
-import org.apache.giraph.utils.ExtendedDataInput;
-import org.apache.giraph.utils.ExtendedDataOutput;
-import org.apache.giraph.utils.ReflectionUtils;
-import org.apache.giraph.utils.UnsafeByteArrayInputStream;
-import org.apache.giraph.utils.UnsafeByteArrayOutputStream;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.WritableComparable;
-import org.apache.hadoop.util.Progressable;
-
-import java.util.List;
-
-/**
- * The classes set here are immutable, the remaining configuration is mutable.
- * Classes are immutable and final to provide the best performance for
- * instantiation.  Everything is thread-safe.
- *
- * @param <I> Vertex id
- * @param <V> Vertex data
- * @param <E> Edge data
- * @param <M> Message data
- */
-public class ImmutableClassesGiraphConfiguration<I extends WritableComparable,
-    V extends Writable, E extends Writable, M extends Writable> extends
-    GiraphConfiguration {
-  /** Vertex class - cached for fast access */
-  private final Class<? extends Vertex<I, V, E, M>> vertexClass;
-  /** Vertex id class - cached for fast access */
-  private final Class<I> vertexIdClass;
-  /** Vertex value class - cached for fast access */
-  private final Class<V> vertexValueClass;
-  /** Edge value class - cached for fast access */
-  private final Class<E> edgeValueClass;
-  /** Message value class - cached for fast access */
-  private final Class<M> messageValueClass;
-
-  /** Graph partitioner factory class - cached for fast access */
-  private final Class<? extends GraphPartitionerFactory<I, V, E, M>>
-  graphPartitionerFactoryClass;
-  /** Master graph partitioner - cached for fast access */
-  private final MasterGraphPartitioner<I, V, E, M> masterGraphPartitioner;
-
-  /** Vertex input format class - cached for fast access */
-  private final Class<? extends VertexInputFormat<I, V, E, M>>
-  vertexInputFormatClass;
-  /** Vertex output format class - cached for fast access */
-  private final Class<? extends VertexOutputFormat<I, V, E>>
-  vertexOutputFormatClass;
-  /** Edge input format class - cached for fast access */
-  private final Class<? extends EdgeInputFormat<I, E>>
-  edgeInputFormatClass;
-
-
-  /** Aggregator writer class - cached for fast access */
-  private final Class<? extends AggregatorWriter> aggregatorWriterClass;
-  /** Combiner class - cached for fast access */
-  private final Class<? extends Combiner<I, M>> combinerClass;
-
-  /** Vertex resolver class - cached for fast access */
-  private final Class<? extends VertexResolver<I, V, E, M>> vertexResolverClass;
-  /** Worker context class - cached for fast access */
-  private final Class<? extends WorkerContext> workerContextClass;
-  /** Master compute class - cached for fast access */
-  private final Class<? extends MasterCompute> masterComputeClass;
-
-  /** Partition class - cached for fast accesss */
-  private final Class<? extends Partition<I, V, E, M>> partitionClass;
-
-  /**
-   * Use unsafe serialization? Cached for fast access to instantiate the
-   * extended data input/output classes
-   */
-  private final boolean useUnsafeSerialization;
-
-  /**
-   * Constructor.  Takes the configuration and then gets the classes out of
-   * them for Giraph
-   *
-   * @param conf Configuration
-   */
-  public ImmutableClassesGiraphConfiguration(Configuration conf) {
-    super(conf);
-    // set pre-validated generic parameter types into Configuration
-    vertexClass = (Class<? extends Vertex<I, V, E, M>>)
-        conf.getClass(VERTEX_CLASS, null, Vertex.class);
-    List<Class<?>> classList =
-        org.apache.giraph.utils.ReflectionUtils.<Vertex>getTypeArguments(
-            Vertex.class, vertexClass);
-    vertexIdClass = (Class<I>) classList.get(0);
-    vertexValueClass = (Class<V>) classList.get(1);
-    edgeValueClass = (Class<E>) classList.get(2);
-    messageValueClass = (Class<M>) classList.get(3);
-
-    graphPartitionerFactoryClass =
-        (Class<? extends GraphPartitionerFactory<I, V, E, M>>)
-        conf.getClass(GRAPH_PARTITIONER_FACTORY_CLASS,
-            HashPartitionerFactory.class,
-            GraphPartitionerFactory.class);
-    masterGraphPartitioner =
-        createGraphPartitioner().createMasterGraphPartitioner();
-
-    vertexInputFormatClass = (Class<? extends VertexInputFormat<I, V, E, M>>)
-        conf.getClass(VERTEX_INPUT_FORMAT_CLASS,
-        null, VertexInputFormat.class);
-    vertexOutputFormatClass = (Class<? extends VertexOutputFormat<I, V, E>>)
-        conf.getClass(VERTEX_OUTPUT_FORMAT_CLASS,
-        null, VertexOutputFormat.class);
-    edgeInputFormatClass = (Class<? extends EdgeInputFormat<I, E>>)
-        conf.getClass(EDGE_INPUT_FORMAT_CLASS, null, EdgeInputFormat.class);
-
-    aggregatorWriterClass = conf.getClass(AGGREGATOR_WRITER_CLASS,
-        TextAggregatorWriter.class, AggregatorWriter.class);
-    combinerClass = (Class<? extends Combiner<I, M>>)
-        conf.getClass(VERTEX_COMBINER_CLASS, null, Combiner.class);
-    vertexResolverClass = (Class<? extends VertexResolver<I, V, E, M>>)
-        conf.getClass(VERTEX_RESOLVER_CLASS,
-        DefaultVertexResolver.class, VertexResolver.class);
-    workerContextClass = conf.getClass(WORKER_CONTEXT_CLASS,
-        DefaultWorkerContext.class, WorkerContext.class);
-    masterComputeClass =  conf.getClass(MASTER_COMPUTE_CLASS,
-        DefaultMasterCompute.class, MasterCompute.class);
-
-    partitionClass = (Class<? extends Partition<I, V, E, M>>)
-        conf.getClass(PARTITION_CLASS, SimplePartition.class, Partition.class);
-
-    useUnsafeSerialization = getBoolean(USE_UNSAFE_SERIALIZATION,
-        USE_UNSAFE_SERIALIZATION_DEFAULT);
-  }
-
-  /**
-   * Get the user's subclassed
-   * {@link org.apache.giraph.graph.partition.GraphPartitionerFactory}.
-   *
-   * @return User's graph partitioner
-   */
-  public Class<? extends GraphPartitionerFactory<I, V, E, M>>
-  getGraphPartitionerClass() {
-    return graphPartitionerFactoryClass;
-  }
-
-  /**
-   * Create a user graph partitioner class
-   *
-   * @return Instantiated user graph partitioner class
-   */
-  public GraphPartitionerFactory<I, V, E, M> createGraphPartitioner() {
-    return ReflectionUtils.newInstance(graphPartitionerFactoryClass, this);
-  }
-
-  /**
-   * Create a user graph partitioner partition stats class
-   *
-   * @return Instantiated user graph partition stats class
-   */
-  public PartitionStats createGraphPartitionStats() {
-    return masterGraphPartitioner.createPartitionStats();
-  }
-
-  /**
-   * Does the job have a {@link VertexInputFormat}?
-   *
-   * @return True iff a {@link VertexInputFormat} has been specified.
-   */
-  public boolean hasVertexInputFormat() {
-    return vertexInputFormatClass != null;
-  }
-
-  /**
-   * Get the user's subclassed
-   * {@link org.apache.giraph.graph.VertexInputFormat}.
-   *
-   * @return User's vertex input format class
-   */
-  public Class<? extends VertexInputFormat<I, V, E, M>>
-  getVertexInputFormatClass() {
-    return vertexInputFormatClass;
-  }
-
-  /**
-   * Create a user vertex input format class
-   *
-   * @return Instantiated user vertex input format class
-   */
-  public VertexInputFormat<I, V, E, M>
-  createVertexInputFormat() {
-    return ReflectionUtils.newInstance(vertexInputFormatClass, this);
-  }
-
-  /**
-   * Get the user's subclassed
-   * {@link org.apache.giraph.graph.VertexOutputFormat}.
-   *
-   * @return User's vertex output format class
-   */
-  public Class<? extends VertexOutputFormat<I, V, E>>
-  getVertexOutputFormatClass() {
-    return vertexOutputFormatClass;
-  }
-
-  /**
-   * Create a user vertex output format class
-   *
-   * @return Instantiated user vertex output format class
-   */
-  @SuppressWarnings("rawtypes")
-  public VertexOutputFormat<I, V, E> createVertexOutputFormat() {
-    return ReflectionUtils.newInstance(vertexOutputFormatClass, this);
-  }
-
-  /**
-   * Does the job have an {@link EdgeInputFormat}?
-   *
-   * @return True iff an {@link EdgeInputFormat} has been specified.
-   */
-  public boolean hasEdgeInputFormat() {
-    return edgeInputFormatClass != null;
-  }
-
-  /**
-   * Get the user's subclassed
-   * {@link org.apache.giraph.graph.EdgeInputFormat}.
-   *
-   * @return User's edge input format class
-   */
-  public Class<? extends EdgeInputFormat<I, E>> getEdgeInputFormatClass() {
-    return edgeInputFormatClass;
-  }
-
-  /**
-   * Create a user edge input format class
-   *
-   * @return Instantiated user edge input format class
-   */
-  public EdgeInputFormat<I, E> createEdgeInputFormat() {
-    return ReflectionUtils.newInstance(edgeInputFormatClass, this);
-  }
-
-  /**
-   * Get the user's subclassed {@link org.apache.giraph.graph.AggregatorWriter}.
-   *
-   * @return User's aggregator writer class
-   */
-  public Class<? extends AggregatorWriter> getAggregatorWriterClass() {
-    return aggregatorWriterClass;
-  }
-
-  /**
-   * Create a user aggregator output format class
-   *
-   * @return Instantiated user aggregator writer class
-   */
-  public AggregatorWriter createAggregatorWriter() {
-    return ReflectionUtils.newInstance(aggregatorWriterClass, this);
-  }
-
-  /**
-   * Create a user combiner class
-   *
-   * @return Instantiated user combiner class
-   */
-  @SuppressWarnings("rawtypes")
-  public Combiner<I, M> createCombiner() {
-    return ReflectionUtils.newInstance(combinerClass, this);
-  }
-
-  /**
-   * Check if user set a combiner
-   *
-   * @return True iff user set a combiner class
-   */
-  public boolean useCombiner() {
-    return combinerClass != null;
-  }
-
-  /**
-   * Get the user's subclassed VertexResolver.
-   *
-   * @return User's vertex resolver class
-   */
-  public Class<? extends VertexResolver<I, V, E, M>> getVertexResolverClass() {
-    return vertexResolverClass;
-  }
-
-  /**
-   * Create a user vertex revolver
-   *
-   * @param graphState State of the graph from the worker
-   * @return Instantiated user vertex resolver
-   */
-  @SuppressWarnings("rawtypes")
-  public VertexResolver<I, V, E, M> createVertexResolver(
-                       GraphState<I, V, E, M> graphState) {
-    VertexResolver<I, V, E, M> resolver =
-        ReflectionUtils.newInstance(vertexResolverClass, this);
-    resolver.setGraphState(graphState);
-    return resolver;
-  }
-
-  /**
-   * Get the user's subclassed WorkerContext.
-   *
-   * @return User's worker context class
-   */
-  public Class<? extends WorkerContext> getWorkerContextClass() {
-    return workerContextClass;
-  }
-
-  /**
-   * Create a user worker context
-   *
-   * @param graphState State of the graph from the worker
-   * @return Instantiated user worker context
-   */
-  @SuppressWarnings("rawtypes")
-  public WorkerContext createWorkerContext(GraphState<I, V, E, M> graphState) {
-    WorkerContext workerContext =
-        ReflectionUtils.newInstance(workerContextClass, this);
-    workerContext.setGraphState(graphState);
-    return workerContext;
-  }
-
-  /**
-   * Get the user's subclassed {@link org.apache.giraph.graph.MasterCompute}
-   *
-   * @return User's master class
-   */
-  public Class<? extends MasterCompute> getMasterComputeClass() {
-    return masterComputeClass;
-  }
-
-  /**
-   * Create a user master
-   *
-   * @return Instantiated user master
-   */
-  public MasterCompute createMasterCompute() {
-    return ReflectionUtils.newInstance(masterComputeClass, this);
-  }
-
-  /**
-   * Get the user's subclassed {@link org.apache.giraph.graph.Vertex}
-   *
-   * @return User's vertex class
-   */
-  public Class<? extends Vertex<I, V, E, M>> getVertexClass() {
-    return vertexClass;
-  }
-
-  /**
-   * Create a user vertex
-   *
-   * @return Instantiated user vertex
-   */
-  public Vertex<I, V, E, M> createVertex() {
-    return ReflectionUtils.newInstance(vertexClass, this);
-  }
-
-  /**
-   * Get the user's subclassed vertex index class.
-   *
-   * @return User's vertex index class
-   */
-  public Class<I> getVertexIdClass() {
-    return vertexIdClass;
-  }
-
-  /**
-   * Create a user vertex index
-   *
-   * @return Instantiated user vertex index
-   */
-  public I createVertexId() {
-    try {
-      return vertexIdClass.newInstance();
-    } catch (InstantiationException e) {
-      throw new IllegalArgumentException(
-          "createVertexId: Failed to instantiate", e);
-    } catch (IllegalAccessException e) {
-      throw new IllegalArgumentException(
-          "createVertexId: Illegally accessed", e);
-    }
-  }
-
-  /**
-   * Get the user's subclassed vertex value class.
-   *
-   * @return User's vertex value class
-   */
-  public Class<V> getVertexValueClass() {
-    return vertexValueClass;
-  }
-
-  /**
-   * Create a user vertex value
-   *
-   * @return Instantiated user vertex value
-   */
-  @SuppressWarnings("unchecked")
-  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);
-      }
-    }
-  }
-
-  /**
-   * Create array of MasterObservers.
-   *
-   * @return Instantiated array of MasterObservers.
-   */
-  public MasterObserver[] createMasterObservers() {
-    Class<? extends MasterObserver>[] klasses = getMasterObserverClasses();
-    MasterObserver[] objects = new MasterObserver[klasses.length];
-    for (int i = 0; i < klasses.length; ++i) {
-      objects[i] = ReflectionUtils.newInstance(klasses[i], this);
-    }
-    return objects;
-  }
-
-  /**
-   * Get the user's subclassed edge value class.
-   *
-   * @return User's vertex edge value class
-   */
-  public Class<E> getEdgeValueClass() {
-    return edgeValueClass;
-  }
-
-  /**
-   * Create a user edge value
-   *
-   * @return Instantiated user edge value
-   */
-  public E createEdgeValue() {
-    if (edgeValueClass == NullWritable.class) {
-      return (E) NullWritable.get();
-    } else {
-      try {
-        return edgeValueClass.newInstance();
-      } catch (InstantiationException e) {
-        throw new IllegalArgumentException(
-            "createEdgeValue: Failed to instantiate", e);
-      } catch (IllegalAccessException e) {
-        throw new IllegalArgumentException(
-            "createEdgeValue: Illegally accessed", e);
-      }
-    }
-  }
-
-  /**
-   * Get the user's subclassed vertex message value class.
-   *
-   * @return User's vertex message value class
-   */
-  @SuppressWarnings("unchecked")
-  public Class<M> getMessageValueClass() {
-    return messageValueClass;
-  }
-
-  /**
-   * Create a user vertex message value
-   *
-   * @return Instantiated user vertex message value
-   */
-  public M createMessageValue() {
-    if (messageValueClass == NullWritable.class) {
-      return (M) NullWritable.get();
-    } else {
-      try {
-        return messageValueClass.newInstance();
-      } catch (InstantiationException e) {
-        throw new IllegalArgumentException(
-            "createMessageValue: Failed to instantiate", e);
-      } catch (IllegalAccessException e) {
-        throw new IllegalArgumentException(
-            "createMessageValue: Illegally accessed", e);
-      }
-    }
-  }
-
-  /**
-   * Create a partition
-   *
-   * @param id Partition id
-   * @param progressable Progressable for reporting progress
-   * @return Instantiated partition
-   */
-  public Partition<I, V, E, M> createPartition(
-      int id, Progressable progressable) {
-    Partition<I, V, E, M> partition =
-        ReflectionUtils.newInstance(partitionClass, this);
-    partition.initialize(id, progressable);
-    return partition;
-  }
-
-  /**
-   * Use unsafe serialization?
-   *
-   * @return True if using unsafe serialization, false otherwise.
-   */
-  public boolean useUnsafeSerialization() {
-    return useUnsafeSerialization;
-  }
-
-  /**
-   * Create an extended data output (can be subclassed)
-   *
-   * @return ExtendedDataOutput object
-   */
-  public ExtendedDataOutput createExtendedDataOutput() {
-    if (useUnsafeSerialization) {
-      return new UnsafeByteArrayOutputStream();
-    } else {
-      return new ExtendedByteArrayDataOutput();
-    }
-  }
-
-  /**
-   * Create an extended data output (can be subclassed)
-   *
-   * @param expectedSize Expected size
-   * @return ExtendedDataOutput object
-   */
-  public ExtendedDataOutput createExtendedDataOutput(int expectedSize) {
-    if (useUnsafeSerialization) {
-      return new UnsafeByteArrayOutputStream(expectedSize);
-    } else {
-      return new ExtendedByteArrayDataOutput(expectedSize);
-    }
-  }
-
-  /**
-   * Create an extended data output (can be subclassed)
-   *
-   * @param buf Buffer to use for the output (reuse perhaps)
-   * @param pos How much of the buffer is already used
-   * @return ExtendedDataOutput object
-   */
-  public ExtendedDataOutput createExtendedDataOutput(byte[] buf,
-                                                     int pos) {
-    if (useUnsafeSerialization) {
-      return new UnsafeByteArrayOutputStream(buf, pos);
-    } else {
-      return new ExtendedByteArrayDataOutput(buf, pos);
-    }
-  }
-
-  /**
-   * Create an extended data input (can be subclassed)
-   *
-   * @param buf Buffer to use for the input
-   * @param off Where to start reading in the buffer
-   * @param length Maximum length of the buffer
-   * @return ExtendedDataInput object
-   */
-  public ExtendedDataInput createExtendedDataInput(
-      byte[] buf, int off, int length) {
-    if (useUnsafeSerialization) {
-      return new UnsafeByteArrayInputStream(buf, off, length);
-    } else {
-      return new ExtendedByteArrayDataInput(buf, off, length);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/giraph/blob/599f70e3/giraph/src/main/java/org/apache/giraph/benchmark/PageRankBenchmark.java
----------------------------------------------------------------------
diff --git a/giraph/src/main/java/org/apache/giraph/benchmark/PageRankBenchmark.java b/giraph/src/main/java/org/apache/giraph/benchmark/PageRankBenchmark.java
index a471cc1..83d0537 100644
--- a/giraph/src/main/java/org/apache/giraph/benchmark/PageRankBenchmark.java
+++ b/giraph/src/main/java/org/apache/giraph/benchmark/PageRankBenchmark.java
@@ -22,7 +22,7 @@ import org.apache.commons.cli.CommandLineParser;
 import org.apache.commons.cli.HelpFormatter;
 import org.apache.commons.cli.Options;
 import org.apache.commons.cli.PosixParser;
-import org.apache.giraph.GiraphConfiguration;
+import org.apache.giraph.conf.GiraphConstants;
 import org.apache.giraph.examples.DoubleSumCombiner;
 import org.apache.giraph.graph.GiraphJob;
 import org.apache.giraph.io.PseudoRandomVertexInputFormat;
@@ -139,7 +139,7 @@ public class PageRankBenchmark implements Tool {
       job.getConfiguration().useUnsafeSerialization(true);
     }
     LOG.info("Using class " +
-        job.getConfiguration().get(GiraphConfiguration.VERTEX_CLASS));
+        job.getConfiguration().get(GiraphConstants.VERTEX_CLASS));
     if (!cmd.hasOption('t') ||
         (Integer.parseInt(cmd.getOptionValue('t')) == 2)) {
       job.getConfiguration().setVertexCombinerClass(

http://git-wip-us.apache.org/repos/asf/giraph/blob/599f70e3/giraph/src/main/java/org/apache/giraph/benchmark/RandomMessageBenchmark.java
----------------------------------------------------------------------
diff --git a/giraph/src/main/java/org/apache/giraph/benchmark/RandomMessageBenchmark.java b/giraph/src/main/java/org/apache/giraph/benchmark/RandomMessageBenchmark.java
index 833e0cb..604c4a9 100644
--- a/giraph/src/main/java/org/apache/giraph/benchmark/RandomMessageBenchmark.java
+++ b/giraph/src/main/java/org/apache/giraph/benchmark/RandomMessageBenchmark.java
@@ -23,8 +23,8 @@ import org.apache.commons.cli.CommandLineParser;
 import org.apache.commons.cli.HelpFormatter;
 import org.apache.commons.cli.Options;
 import org.apache.commons.cli.PosixParser;
-import org.apache.giraph.GiraphConfiguration;
 import org.apache.giraph.aggregators.LongSumAggregator;
+import org.apache.giraph.conf.GiraphConstants;
 import org.apache.giraph.graph.DefaultMasterCompute;
 import org.apache.giraph.graph.EdgeListVertex;
 import org.apache.giraph.graph.GiraphJob;
@@ -387,7 +387,7 @@ public class RandomMessageBenchmark implements Tool {
     }
     if (cmd.hasOption('f')) {
       job.getConfiguration().setInt(
-          GiraphConfiguration.MSG_NUM_FLUSH_THREADS,
+          GiraphConstants.MSG_NUM_FLUSH_THREADS,
           Integer.parseInt(cmd.getOptionValue('f')));
     }
     if (job.run(isVerbose)) {

http://git-wip-us.apache.org/repos/asf/giraph/blob/599f70e3/giraph/src/main/java/org/apache/giraph/benchmark/ShortestPathsBenchmark.java
----------------------------------------------------------------------
diff --git a/giraph/src/main/java/org/apache/giraph/benchmark/ShortestPathsBenchmark.java b/giraph/src/main/java/org/apache/giraph/benchmark/ShortestPathsBenchmark.java
index 62398f8..21fc0ac 100644
--- a/giraph/src/main/java/org/apache/giraph/benchmark/ShortestPathsBenchmark.java
+++ b/giraph/src/main/java/org/apache/giraph/benchmark/ShortestPathsBenchmark.java
@@ -23,7 +23,7 @@ import org.apache.commons.cli.CommandLineParser;
 import org.apache.commons.cli.HelpFormatter;
 import org.apache.commons.cli.Options;
 import org.apache.commons.cli.PosixParser;
-import org.apache.giraph.GiraphConfiguration;
+import org.apache.giraph.conf.GiraphConstants;
 import org.apache.giraph.examples.MinimumDoubleCombiner;
 import org.apache.giraph.graph.EdgeListVertex;
 import org.apache.giraph.graph.GiraphJob;
@@ -130,7 +130,7 @@ public class ShortestPathsBenchmark implements Tool {
           HashMapVertexShortestPathsBenchmark.class);
     }
     LOG.info("Using class " +
-        job.getConfiguration().get(GiraphConfiguration.VERTEX_CLASS));
+        job.getConfiguration().get(GiraphConstants.VERTEX_CLASS));
     job.getConfiguration().setVertexInputFormatClass(
         PseudoRandomVertexInputFormat.class);
     if (!cmd.hasOption("nc")) {

http://git-wip-us.apache.org/repos/asf/giraph/blob/599f70e3/giraph/src/main/java/org/apache/giraph/bsp/BspInputFormat.java
----------------------------------------------------------------------
diff --git a/giraph/src/main/java/org/apache/giraph/bsp/BspInputFormat.java b/giraph/src/main/java/org/apache/giraph/bsp/BspInputFormat.java
index 84dcda1..bce84b1 100644
--- a/giraph/src/main/java/org/apache/giraph/bsp/BspInputFormat.java
+++ b/giraph/src/main/java/org/apache/giraph/bsp/BspInputFormat.java
@@ -18,20 +18,20 @@
 
 package org.apache.giraph.bsp;
 
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.giraph.GiraphConfiguration;
-import org.apache.hadoop.mapreduce.RecordReader;
-import org.apache.hadoop.mapreduce.InputFormat;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.giraph.conf.GiraphConstants;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.InputFormat;
 import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
 import org.apache.log4j.Logger;
 
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
 /**
  * This InputFormat supports the BSP model by ensuring that the user specifies
  * how many splits (number of mappers) should be started simultaneously.
@@ -49,15 +49,15 @@ public class BspInputFormat extends InputFormat<Text, Text> {
    * @return Maximum number of tasks
    */
   public static int getMaxTasks(Configuration conf) {
-    int maxWorkers = conf.getInt(GiraphConfiguration.MAX_WORKERS, 0);
+    int maxWorkers = conf.getInt(GiraphConstants.MAX_WORKERS, 0);
     boolean splitMasterWorker =
-        conf.getBoolean(GiraphConfiguration.SPLIT_MASTER_WORKER,
-            GiraphConfiguration.SPLIT_MASTER_WORKER_DEFAULT);
+        conf.getBoolean(GiraphConstants.SPLIT_MASTER_WORKER,
+            GiraphConstants.SPLIT_MASTER_WORKER_DEFAULT);
     int maxTasks = maxWorkers;
     if (splitMasterWorker) {
       int zkServers =
-          conf.getInt(GiraphConfiguration.ZOOKEEPER_SERVER_COUNT,
-              GiraphConfiguration.ZOOKEEPER_SERVER_COUNT_DEFAULT);
+          conf.getInt(GiraphConstants.ZOOKEEPER_SERVER_COUNT,
+              GiraphConstants.ZOOKEEPER_SERVER_COUNT_DEFAULT);
       maxTasks += zkServers;
     }
     if (LOG.isDebugEnabled()) {

http://git-wip-us.apache.org/repos/asf/giraph/blob/599f70e3/giraph/src/main/java/org/apache/giraph/comm/SendMessageCache.java
----------------------------------------------------------------------
diff --git a/giraph/src/main/java/org/apache/giraph/comm/SendMessageCache.java b/giraph/src/main/java/org/apache/giraph/comm/SendMessageCache.java
index 2891ad5..27e1010 100644
--- a/giraph/src/main/java/org/apache/giraph/comm/SendMessageCache.java
+++ b/giraph/src/main/java/org/apache/giraph/comm/SendMessageCache.java
@@ -18,14 +18,9 @@
 
 package org.apache.giraph.comm;
 
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.giraph.GiraphConfiguration;
-import org.apache.giraph.ImmutableClassesGiraphConfiguration;
 import org.apache.giraph.bsp.CentralizedServiceWorker;
+import org.apache.giraph.conf.GiraphConstants;
+import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
 import org.apache.giraph.graph.WorkerInfo;
 import org.apache.giraph.graph.partition.PartitionOwner;
 import org.apache.giraph.utils.ByteArrayVertexIdMessages;
@@ -33,6 +28,12 @@ import org.apache.giraph.utils.PairList;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparable;
 
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+
+import java.util.List;
+import java.util.Map;
+
 /**
  * Aggregates the messages to be send to workers so they can be sent
  * in bulk.  Not thread-safe.
@@ -103,8 +104,8 @@ public class SendMessageCache<I extends WritableComparable,
     float additionalRequestSize =
         conf.getFloat(ADDITIONAL_MSG_REQUEST_SIZE,
             ADDITIONAL_MSG_REQUEST_SIZE_DEFAULT);
-    int requestSize = conf.getInt(GiraphConfiguration.MAX_MSG_REQUEST_SIZE,
-        GiraphConfiguration.MAX_MSG_REQUEST_SIZE_DEFAULT);
+    int requestSize = conf.getInt(GiraphConstants.MAX_MSG_REQUEST_SIZE,
+        GiraphConstants.MAX_MSG_REQUEST_SIZE_DEFAULT);
     int initialRequestSize = (int) (requestSize * (1 + additionalRequestSize));
     initialBufferSizes = new int[maxWorker + 1];
     for (WorkerInfo workerInfo : serviceWorker.getWorkerInfoList()) {

http://git-wip-us.apache.org/repos/asf/giraph/blob/599f70e3/giraph/src/main/java/org/apache/giraph/comm/SendPartitionCache.java
----------------------------------------------------------------------
diff --git a/giraph/src/main/java/org/apache/giraph/comm/SendPartitionCache.java b/giraph/src/main/java/org/apache/giraph/comm/SendPartitionCache.java
index f710f11..a199d0c 100644
--- a/giraph/src/main/java/org/apache/giraph/comm/SendPartitionCache.java
+++ b/giraph/src/main/java/org/apache/giraph/comm/SendPartitionCache.java
@@ -19,7 +19,7 @@ package org.apache.giraph.comm;
 
 import com.google.common.collect.Maps;
 import java.util.Map;
-import org.apache.giraph.ImmutableClassesGiraphConfiguration;
+import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
 import org.apache.giraph.graph.GiraphTransferRegulator;
 import org.apache.giraph.graph.Vertex;
 import org.apache.giraph.graph.partition.Partition;

http://git-wip-us.apache.org/repos/asf/giraph/blob/599f70e3/giraph/src/main/java/org/apache/giraph/comm/ServerData.java
----------------------------------------------------------------------
diff --git a/giraph/src/main/java/org/apache/giraph/comm/ServerData.java b/giraph/src/main/java/org/apache/giraph/comm/ServerData.java
index 38df97f..7e89a41 100644
--- a/giraph/src/main/java/org/apache/giraph/comm/ServerData.java
+++ b/giraph/src/main/java/org/apache/giraph/comm/ServerData.java
@@ -18,12 +18,12 @@
 
 package org.apache.giraph.comm;
 
-import org.apache.giraph.GiraphConfiguration;
-import org.apache.giraph.ImmutableClassesGiraphConfiguration;
 import org.apache.giraph.comm.aggregators.AllAggregatorServerData;
 import org.apache.giraph.comm.aggregators.OwnerAggregatorServerData;
 import org.apache.giraph.comm.messages.MessageStoreByPartition;
 import org.apache.giraph.comm.messages.MessageStoreFactory;
+import org.apache.giraph.conf.GiraphConstants;
+import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
 import org.apache.giraph.graph.VertexMutations;
 import org.apache.giraph.graph.partition.DiskBackedPartitionStore;
 import org.apache.giraph.graph.partition.PartitionStore;
@@ -92,8 +92,8 @@ public class ServerData<I extends WritableComparable,
     this.messageStoreFactory = messageStoreFactory;
     currentMessageStore = messageStoreFactory.newStore();
     incomingMessageStore = messageStoreFactory.newStore();
-    if (configuration.getBoolean(GiraphConfiguration.USE_OUT_OF_CORE_GRAPH,
-        GiraphConfiguration.USE_OUT_OF_CORE_GRAPH_DEFAULT)) {
+    if (configuration.getBoolean(GiraphConstants.USE_OUT_OF_CORE_GRAPH,
+        GiraphConstants.USE_OUT_OF_CORE_GRAPH_DEFAULT)) {
       partitionStore =
           new DiskBackedPartitionStore<I, V, E, M>(configuration, context);
     } else {

http://git-wip-us.apache.org/repos/asf/giraph/blob/599f70e3/giraph/src/main/java/org/apache/giraph/comm/aggregators/AggregatorUtils.java
----------------------------------------------------------------------
diff --git a/giraph/src/main/java/org/apache/giraph/comm/aggregators/AggregatorUtils.java b/giraph/src/main/java/org/apache/giraph/comm/aggregators/AggregatorUtils.java
index 08b9834..ff93804 100644
--- a/giraph/src/main/java/org/apache/giraph/comm/aggregators/AggregatorUtils.java
+++ b/giraph/src/main/java/org/apache/giraph/comm/aggregators/AggregatorUtils.java
@@ -18,7 +18,7 @@
 
 package org.apache.giraph.comm.aggregators;
 
-import org.apache.giraph.ImmutableClassesGiraphConfiguration;
+import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
 import org.apache.giraph.graph.Aggregator;
 import org.apache.giraph.graph.WorkerInfo;
 import org.apache.hadoop.io.Writable;

http://git-wip-us.apache.org/repos/asf/giraph/blob/599f70e3/giraph/src/main/java/org/apache/giraph/comm/messages/ByteArrayMessagesPerVertexStore.java
----------------------------------------------------------------------
diff --git a/giraph/src/main/java/org/apache/giraph/comm/messages/ByteArrayMessagesPerVertexStore.java b/giraph/src/main/java/org/apache/giraph/comm/messages/ByteArrayMessagesPerVertexStore.java
index c54a49a..65caa5d 100644
--- a/giraph/src/main/java/org/apache/giraph/comm/messages/ByteArrayMessagesPerVertexStore.java
+++ b/giraph/src/main/java/org/apache/giraph/comm/messages/ByteArrayMessagesPerVertexStore.java
@@ -24,7 +24,7 @@ import java.io.DataOutput;
 import java.io.IOException;
 import java.util.Map;
 import java.util.concurrent.ConcurrentMap;
-import org.apache.giraph.ImmutableClassesGiraphConfiguration;
+import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
 import org.apache.giraph.bsp.CentralizedServiceWorker;
 import org.apache.giraph.utils.ByteArrayVertexIdMessages;
 import org.apache.giraph.utils.ExtendedDataOutput;

http://git-wip-us.apache.org/repos/asf/giraph/blob/599f70e3/giraph/src/main/java/org/apache/giraph/comm/messages/DiskBackedMessageStore.java
----------------------------------------------------------------------
diff --git a/giraph/src/main/java/org/apache/giraph/comm/messages/DiskBackedMessageStore.java b/giraph/src/main/java/org/apache/giraph/comm/messages/DiskBackedMessageStore.java
index c2d01e8..5684990 100644
--- a/giraph/src/main/java/org/apache/giraph/comm/messages/DiskBackedMessageStore.java
+++ b/giraph/src/main/java/org/apache/giraph/comm/messages/DiskBackedMessageStore.java
@@ -33,7 +33,7 @@ import java.util.concurrent.ConcurrentSkipListMap;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.locks.ReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
-import org.apache.giraph.ImmutableClassesGiraphConfiguration;
+import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
 import org.apache.giraph.utils.ExtendedDataOutput;
 import org.apache.giraph.utils.RepresentativeByteArrayIterable;
 import org.apache.hadoop.io.Writable;

http://git-wip-us.apache.org/repos/asf/giraph/blob/599f70e3/giraph/src/main/java/org/apache/giraph/comm/messages/OneMessagePerVertexStore.java
----------------------------------------------------------------------
diff --git a/giraph/src/main/java/org/apache/giraph/comm/messages/OneMessagePerVertexStore.java b/giraph/src/main/java/org/apache/giraph/comm/messages/OneMessagePerVertexStore.java
index 9871ff0..61fb750 100644
--- a/giraph/src/main/java/org/apache/giraph/comm/messages/OneMessagePerVertexStore.java
+++ b/giraph/src/main/java/org/apache/giraph/comm/messages/OneMessagePerVertexStore.java
@@ -24,7 +24,7 @@ import java.io.IOException;
 import java.util.Collections;
 import java.util.Map;
 import java.util.concurrent.ConcurrentMap;
-import org.apache.giraph.ImmutableClassesGiraphConfiguration;
+import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
 import org.apache.giraph.bsp.CentralizedServiceWorker;
 import org.apache.giraph.graph.Combiner;
 import org.apache.giraph.utils.ByteArrayVertexIdMessages;

http://git-wip-us.apache.org/repos/asf/giraph/blob/599f70e3/giraph/src/main/java/org/apache/giraph/comm/messages/SequentialFileMessageStore.java
----------------------------------------------------------------------
diff --git a/giraph/src/main/java/org/apache/giraph/comm/messages/SequentialFileMessageStore.java b/giraph/src/main/java/org/apache/giraph/comm/messages/SequentialFileMessageStore.java
index 0c8b34a..1805f0b 100644
--- a/giraph/src/main/java/org/apache/giraph/comm/messages/SequentialFileMessageStore.java
+++ b/giraph/src/main/java/org/apache/giraph/comm/messages/SequentialFileMessageStore.java
@@ -18,9 +18,17 @@
 
 package org.apache.giraph.comm.messages;
 
+import org.apache.giraph.conf.GiraphConstants;
+import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
+import org.apache.giraph.utils.EmptyIterable;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
+import org.apache.log4j.Logger;
+
 import com.google.common.collect.Iterables;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Sets;
+
 import java.io.BufferedInputStream;
 import java.io.BufferedOutputStream;
 import java.io.DataInput;
@@ -35,12 +43,6 @@ import java.util.Collection;
 import java.util.List;
 import java.util.SortedSet;
 import java.util.concurrent.atomic.AtomicInteger;
-import org.apache.giraph.GiraphConfiguration;
-import org.apache.giraph.ImmutableClassesGiraphConfiguration;
-import org.apache.giraph.utils.EmptyIterable;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.WritableComparable;
-import org.apache.log4j.Logger;
 
 /**
  * Used for writing and reading collection of messages to the disk. {@link
@@ -374,11 +376,11 @@ public class SequentialFileMessageStore<I extends WritableComparable,
     public Factory(ImmutableClassesGiraphConfiguration config) {
       this.config = config;
       String jobId = config.get("mapred.job.id", "Unknown Job");
-      this.directory = config.get(GiraphConfiguration.MESSAGES_DIRECTORY,
-          GiraphConfiguration.MESSAGES_DIRECTORY_DEFAULT) + jobId +
+      this.directory = config.get(GiraphConstants.MESSAGES_DIRECTORY,
+          GiraphConstants.MESSAGES_DIRECTORY_DEFAULT) + jobId +
           File.separator;
-      this.bufferSize = config.getInt(GiraphConfiguration.MESSAGES_BUFFER_SIZE,
-          GiraphConfiguration.MESSAGES_BUFFER_SIZE_DEFAULT);
+      this.bufferSize = config.getInt(GiraphConstants.MESSAGES_BUFFER_SIZE,
+          GiraphConstants.MESSAGES_BUFFER_SIZE_DEFAULT);
       storeCounter = new AtomicInteger();
       new File(directory).mkdirs();
     }

http://git-wip-us.apache.org/repos/asf/giraph/blob/599f70e3/giraph/src/main/java/org/apache/giraph/comm/messages/SimpleMessageStore.java
----------------------------------------------------------------------
diff --git a/giraph/src/main/java/org/apache/giraph/comm/messages/SimpleMessageStore.java b/giraph/src/main/java/org/apache/giraph/comm/messages/SimpleMessageStore.java
index 1bfa92e..fc6560d 100644
--- a/giraph/src/main/java/org/apache/giraph/comm/messages/SimpleMessageStore.java
+++ b/giraph/src/main/java/org/apache/giraph/comm/messages/SimpleMessageStore.java
@@ -28,7 +28,7 @@ import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.ConcurrentMap;
-import org.apache.giraph.ImmutableClassesGiraphConfiguration;
+import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
 import org.apache.giraph.bsp.CentralizedServiceWorker;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparable;

http://git-wip-us.apache.org/repos/asf/giraph/blob/599f70e3/giraph/src/main/java/org/apache/giraph/comm/netty/ByteCounter.java
----------------------------------------------------------------------
diff --git a/giraph/src/main/java/org/apache/giraph/comm/netty/ByteCounter.java b/giraph/src/main/java/org/apache/giraph/comm/netty/ByteCounter.java
index 9c2035a..cca6c4a 100644
--- a/giraph/src/main/java/org/apache/giraph/comm/netty/ByteCounter.java
+++ b/giraph/src/main/java/org/apache/giraph/comm/netty/ByteCounter.java
@@ -81,7 +81,7 @@ public class ByteCounter extends SimpleChannelHandler implements
   /** Constructor */
   public ByteCounter() {
     // Initialize Metrics
-    GiraphMetrics.getInstance().addSuperstepResetObserver(this);
+    GiraphMetrics.get().addSuperstepResetObserver(this);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/giraph/blob/599f70e3/giraph/src/main/java/org/apache/giraph/comm/netty/NettyClient.java
----------------------------------------------------------------------
diff --git a/giraph/src/main/java/org/apache/giraph/comm/netty/NettyClient.java b/giraph/src/main/java/org/apache/giraph/comm/netty/NettyClient.java
index cad1405..c66c819 100644
--- a/giraph/src/main/java/org/apache/giraph/comm/netty/NettyClient.java
+++ b/giraph/src/main/java/org/apache/giraph/comm/netty/NettyClient.java
@@ -18,34 +18,20 @@
 
 package org.apache.giraph.comm.netty;
 
-import com.google.common.collect.Lists;
-import com.google.common.collect.MapMaker;
-import com.google.common.util.concurrent.ThreadFactoryBuilder;
-import java.io.IOException;
-import java.net.InetSocketAddress;
-import java.util.Collection;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
-import org.apache.giraph.GiraphConfiguration;
-import org.apache.giraph.ImmutableClassesGiraphConfiguration;
 import org.apache.giraph.comm.netty.handler.AddressRequestIdGenerator;
 import org.apache.giraph.comm.netty.handler.ClientRequestId;
 import org.apache.giraph.comm.netty.handler.RequestEncoder;
 import org.apache.giraph.comm.netty.handler.RequestInfo;
 import org.apache.giraph.comm.netty.handler.RequestServerHandler;
 import org.apache.giraph.comm.netty.handler.ResponseClientHandler;
-/*if[HADOOP_NON_SECURE]
-else[HADOOP_NON_SECURE]*/
+/*if_not[HADOOP_NON_SECURE]*/
 import org.apache.giraph.comm.netty.handler.SaslClientHandler;
 import org.apache.giraph.comm.requests.RequestType;
 import org.apache.giraph.comm.requests.SaslTokenMessageRequest;
 /*end[HADOOP_NON_SECURE]*/
 import org.apache.giraph.comm.requests.WritableRequest;
+import org.apache.giraph.conf.GiraphConstants;
+import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
 import org.apache.giraph.graph.TaskInfo;
 import org.apache.giraph.utils.ProgressableUtils;
 import org.apache.giraph.utils.TimedLogger;
@@ -65,9 +51,22 @@ import org.jboss.netty.handler.codec.frame.LengthFieldBasedFrameDecoder;
 import org.jboss.netty.handler.execution.ExecutionHandler;
 import org.jboss.netty.handler.execution.MemoryAwareThreadPoolExecutor;
 
-import static org.jboss.netty.channel.Channels.pipeline;
+import com.google.common.collect.Lists;
+import com.google.common.collect.MapMaker;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
 
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
 
+import static org.jboss.netty.channel.Channels.pipeline;
 
 /**
  * Netty client for sending requests.  Thread-safe.
@@ -87,8 +86,7 @@ public class NettyClient {
   public static final int MAX_REQUESTS_TO_LIST = 10;
   /** 30 seconds to connect by default */
   public static final int MAX_CONNECTION_MILLISECONDS_DEFAULT = 30 * 1000;
-/*if[HADOOP_NON_SECURE]
-else[HADOOP_NON_SECURE]*/
+/*if_not[HADOOP_NON_SECURE]*/
   /** Used to authenticate with other workers acting as servers */
   public static final ChannelLocal<SaslNettyClient> SASL =
       new ChannelLocal<SaslNettyClient>();
@@ -166,14 +164,14 @@ else[HADOOP_NON_SECURE]*/
     this.context = context;
     this.myTaskInfo = myTaskInfo;
     this.channelsPerServer = conf.getInt(
-        GiraphConfiguration.CHANNELS_PER_SERVER,
-        GiraphConfiguration.DEFAULT_CHANNELS_PER_SERVER);
+        GiraphConstants.CHANNELS_PER_SERVER,
+        GiraphConstants.DEFAULT_CHANNELS_PER_SERVER);
     sendBufferSize = conf.getInt(
-        GiraphConfiguration.CLIENT_SEND_BUFFER_SIZE,
-        GiraphConfiguration.DEFAULT_CLIENT_SEND_BUFFER_SIZE);
+        GiraphConstants.CLIENT_SEND_BUFFER_SIZE,
+        GiraphConstants.DEFAULT_CLIENT_SEND_BUFFER_SIZE);
     receiveBufferSize = conf.getInt(
-        GiraphConfiguration.CLIENT_RECEIVE_BUFFER_SIZE,
-        GiraphConfiguration.DEFAULT_CLIENT_RECEIVE_BUFFER_SIZE);
+        GiraphConstants.CLIENT_RECEIVE_BUFFER_SIZE,
+        GiraphConstants.DEFAULT_CLIENT_RECEIVE_BUFFER_SIZE);
 
     limitNumberOfOpenRequests = conf.getBoolean(
         LIMIT_NUMBER_OF_OPEN_REQUESTS,
@@ -191,38 +189,38 @@ else[HADOOP_NON_SECURE]*/
     }
 
     maxRequestMilliseconds = conf.getInt(
-        GiraphConfiguration.MAX_REQUEST_MILLISECONDS,
-        GiraphConfiguration.MAX_REQUEST_MILLISECONDS_DEFAULT);
+        GiraphConstants.MAX_REQUEST_MILLISECONDS,
+        GiraphConstants.MAX_REQUEST_MILLISECONDS_DEFAULT);
 
     maxConnectionFailures = conf.getInt(
-        GiraphConfiguration.NETTY_MAX_CONNECTION_FAILURES,
-        GiraphConfiguration.NETTY_MAX_CONNECTION_FAILURES_DEFAULT);
+        GiraphConstants.NETTY_MAX_CONNECTION_FAILURES,
+        GiraphConstants.NETTY_MAX_CONNECTION_FAILURES_DEFAULT);
 
     waitingRequestMsecs = conf.getInt(
-        GiraphConfiguration.WAITING_REQUEST_MSECS,
-        GiraphConfiguration.WAITING_REQUEST_MSECS_DEFAULT);
+        GiraphConstants.WAITING_REQUEST_MSECS,
+        GiraphConstants.WAITING_REQUEST_MSECS_DEFAULT);
 
     maxPoolSize = conf.getInt(
-        GiraphConfiguration.NETTY_CLIENT_THREADS,
-        GiraphConfiguration.NETTY_CLIENT_THREADS_DEFAULT);
+        GiraphConstants.NETTY_CLIENT_THREADS,
+        GiraphConstants.NETTY_CLIENT_THREADS_DEFAULT);
 
     maxResolveAddressAttempts = conf.getInt(
-        GiraphConfiguration.MAX_RESOLVE_ADDRESS_ATTEMPTS,
-        GiraphConfiguration.MAX_RESOLVE_ADDRESS_ATTEMPTS_DEFAULT);
+        GiraphConstants.MAX_RESOLVE_ADDRESS_ATTEMPTS,
+        GiraphConstants.MAX_RESOLVE_ADDRESS_ATTEMPTS_DEFAULT);
 
     clientRequestIdRequestInfoMap =
         new MapMaker().concurrencyLevel(maxPoolSize).makeMap();
 
     handlerBeforeExecutionHandler = conf.get(
-        GiraphConfiguration.NETTY_CLIENT_EXECUTION_AFTER_HANDLER,
-        GiraphConfiguration.NETTY_CLIENT_EXECUTION_AFTER_HANDLER_DEFAULT);
+        GiraphConstants.NETTY_CLIENT_EXECUTION_AFTER_HANDLER,
+        GiraphConstants.NETTY_CLIENT_EXECUTION_AFTER_HANDLER_DEFAULT);
     boolean useExecutionHandler = conf.getBoolean(
-        GiraphConfiguration.NETTY_CLIENT_USE_EXECUTION_HANDLER,
-        GiraphConfiguration.NETTY_CLIENT_USE_EXECUTION_HANDLER_DEFAULT);
+        GiraphConstants.NETTY_CLIENT_USE_EXECUTION_HANDLER,
+        GiraphConstants.NETTY_CLIENT_USE_EXECUTION_HANDLER_DEFAULT);
     if (useExecutionHandler) {
       int executionThreads = conf.getInt(
-          GiraphConfiguration.NETTY_CLIENT_EXECUTION_THREADS,
-          GiraphConfiguration.NETTY_CLIENT_EXECUTION_THREADS_DEFAULT);
+          GiraphConstants.NETTY_CLIENT_EXECUTION_THREADS,
+          GiraphConstants.NETTY_CLIENT_EXECUTION_THREADS_DEFAULT);
       executionHandler = new ExecutionHandler(
           new MemoryAwareThreadPoolExecutor(
               executionThreads, 1048576, 1048576, 1, TimeUnit.HOURS,
@@ -261,8 +259,7 @@ else[HADOOP_NON_SECURE]*/
     bootstrap.setPipelineFactory(new ChannelPipelineFactory() {
       @Override
       public ChannelPipeline getPipeline() throws Exception {
-/*if[HADOOP_NON_SECURE]
-else[HADOOP_NON_SECURE]*/
+/*if_not[HADOOP_NON_SECURE]*/
         if (conf.authenticate()) {
           LOG.info("Using Netty with authentication.");
 
@@ -282,8 +279,8 @@ else[HADOOP_NON_SECURE]*/
           pipeline.addLast("length-field-based-frame-decoder",
               new LengthFieldBasedFrameDecoder(1024, 0, 4, 0, 4));
           pipeline.addLast("request-encoder", new RequestEncoder(conf.getInt(
-              GiraphConfiguration.NETTY_REQUEST_ENCODER_BUFFER_SIZE,
-              GiraphConfiguration.NETTY_REQUEST_ENCODER_BUFFER_SIZE_DEFAULT)));
+              GiraphConstants.NETTY_REQUEST_ENCODER_BUFFER_SIZE,
+              GiraphConstants.NETTY_REQUEST_ENCODER_BUFFER_SIZE_DEFAULT)));
           // The following pipeline component responds to the server's SASL
           // tokens with its own responses. Both client and server share the
           // same Hadoop Job token, which is used to create the SASL tokens to
@@ -302,8 +299,8 @@ else[HADOOP_NON_SECURE]*/
           pipeline.addLast("responseFrameDecoder",
               new FixedLengthFrameDecoder(RequestServerHandler.RESPONSE_BYTES));
           pipeline.addLast("requestEncoder", new RequestEncoder(conf.getInt(
-              GiraphConfiguration.NETTY_REQUEST_ENCODER_BUFFER_SIZE,
-              GiraphConfiguration.NETTY_REQUEST_ENCODER_BUFFER_SIZE_DEFAULT)));
+              GiraphConstants.NETTY_REQUEST_ENCODER_BUFFER_SIZE,
+              GiraphConstants.NETTY_REQUEST_ENCODER_BUFFER_SIZE_DEFAULT)));
           pipeline.addLast("responseClientHandler",
               new ResponseClientHandler(clientRequestIdRequestInfoMap, conf));
           if (executionHandler != null) {
@@ -311,8 +308,7 @@ else[HADOOP_NON_SECURE]*/
                 "executionHandler", executionHandler);
           }
           return pipeline;
-/*if[HADOOP_NON_SECURE]
-else[HADOOP_NON_SECURE]*/
+/*if_not[HADOOP_NON_SECURE]*/
         }
 /*end[HADOOP_NON_SECURE]*/
       }
@@ -455,8 +451,7 @@ else[HADOOP_NON_SECURE]*/
     }
   }
 
-/*if[HADOOP_NON_SECURE]
-else[HADOOP_NON_SECURE]*/
+/*if_not[HADOOP_NON_SECURE]*/
   /**
    * Authenticate all servers in addressChannelMap.
    */
@@ -635,8 +630,7 @@ else[HADOOP_NON_SECURE]*/
       byteCounter.resetAll();
     }
     boolean registerRequest = true;
-/*if[HADOOP_NON_SECURE]
-else[HADOOP_NON_SECURE]*/
+/*if_not[HADOOP_NON_SECURE]*/
     if (request.getType() == RequestType.SASL_TOKEN_MESSAGE_REQUEST) {
       registerRequest = false;
     }

http://git-wip-us.apache.org/repos/asf/giraph/blob/599f70e3/giraph/src/main/java/org/apache/giraph/comm/netty/NettyMasterClient.java
----------------------------------------------------------------------
diff --git a/giraph/src/main/java/org/apache/giraph/comm/netty/NettyMasterClient.java b/giraph/src/main/java/org/apache/giraph/comm/netty/NettyMasterClient.java
index 857e4b9..c575172 100644
--- a/giraph/src/main/java/org/apache/giraph/comm/netty/NettyMasterClient.java
+++ b/giraph/src/main/java/org/apache/giraph/comm/netty/NettyMasterClient.java
@@ -18,7 +18,7 @@
 
 package org.apache.giraph.comm.netty;
 
-import org.apache.giraph.ImmutableClassesGiraphConfiguration;
+import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
 import org.apache.giraph.bsp.CentralizedServiceMaster;
 import org.apache.giraph.comm.MasterClient;
 import org.apache.giraph.comm.aggregators.AggregatorUtils;

http://git-wip-us.apache.org/repos/asf/giraph/blob/599f70e3/giraph/src/main/java/org/apache/giraph/comm/netty/NettyMasterServer.java
----------------------------------------------------------------------
diff --git a/giraph/src/main/java/org/apache/giraph/comm/netty/NettyMasterServer.java b/giraph/src/main/java/org/apache/giraph/comm/netty/NettyMasterServer.java
index 37daea8..1f04bcf 100644
--- a/giraph/src/main/java/org/apache/giraph/comm/netty/NettyMasterServer.java
+++ b/giraph/src/main/java/org/apache/giraph/comm/netty/NettyMasterServer.java
@@ -18,7 +18,7 @@
 
 package org.apache.giraph.comm.netty;
 
-import org.apache.giraph.ImmutableClassesGiraphConfiguration;
+import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
 import org.apache.giraph.bsp.CentralizedServiceMaster;
 import org.apache.giraph.comm.netty.handler.MasterRequestServerHandler;
 import org.apache.giraph.comm.MasterServer;

http://git-wip-us.apache.org/repos/asf/giraph/blob/599f70e3/giraph/src/main/java/org/apache/giraph/comm/netty/NettyServer.java
----------------------------------------------------------------------
diff --git a/giraph/src/main/java/org/apache/giraph/comm/netty/NettyServer.java b/giraph/src/main/java/org/apache/giraph/comm/netty/NettyServer.java
index f8a5b46..971c7c5 100644
--- a/giraph/src/main/java/org/apache/giraph/comm/netty/NettyServer.java
+++ b/giraph/src/main/java/org/apache/giraph/comm/netty/NettyServer.java
@@ -18,27 +18,18 @@
 
 package org.apache.giraph.comm.netty;
 
-import java.net.InetAddress;
-import java.net.InetSocketAddress;
-import java.net.UnknownHostException;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-
-/*if[HADOOP_NON_SECURE]
-else[HADOOP_NON_SECURE]*/
+/*if_not[HADOOP_NON_SECURE]*/
 import org.apache.giraph.comm.netty.handler.AuthorizeServerHandler;
 /*end[HADOOP_NON_SECURE]*/
-import org.apache.giraph.comm.netty.handler.WorkerRequestReservedMap;
 import org.apache.giraph.comm.netty.handler.RequestDecoder;
 import org.apache.giraph.comm.netty.handler.RequestServerHandler;
 import org.apache.giraph.comm.netty.handler.ResponseEncoder;
-/*if[HADOOP_NON_SECURE]
-else[HADOOP_NON_SECURE]*/
+/*if_not[HADOOP_NON_SECURE]*/
 import org.apache.giraph.comm.netty.handler.SaslServerHandler;
 /*end[HADOOP_NON_SECURE]*/
-import java.util.concurrent.TimeUnit;
-import org.apache.giraph.GiraphConfiguration;
-import org.apache.giraph.ImmutableClassesGiraphConfiguration;
+import org.apache.giraph.comm.netty.handler.WorkerRequestReservedMap;
+import org.apache.giraph.conf.GiraphConstants;
+import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
 import org.apache.giraph.graph.TaskInfo;
 import org.apache.giraph.utils.ProgressableUtils;
 import org.apache.hadoop.util.Progressable;
@@ -58,11 +49,18 @@ import org.jboss.netty.channel.group.ChannelGroup;
 import org.jboss.netty.channel.group.DefaultChannelGroup;
 import org.jboss.netty.channel.socket.nio.NioServerSocketChannelFactory;
 import org.jboss.netty.handler.codec.frame.LengthFieldBasedFrameDecoder;
-
-import com.google.common.util.concurrent.ThreadFactoryBuilder;
 import org.jboss.netty.handler.execution.ExecutionHandler;
 import org.jboss.netty.handler.execution.MemoryAwareThreadPoolExecutor;
 
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.net.UnknownHostException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+
 import static org.jboss.netty.channel.Channels.pipeline;
 
 /**
@@ -73,8 +71,7 @@ public class NettyServer {
   public static final int MAXIMUM_THREAD_POOL_SIZE_DEFAULT = 32;
 
 
-/*if[HADOOP_NON_SECURE]
-else[HADOOP_NON_SECURE]*/
+/*if_not[HADOOP_NON_SECURE]*/
   /** Used to authenticate with netty clients */
   public static final ChannelLocal<SaslNettyServer>
   CHANNEL_SASL_NETTY_SERVERS =
@@ -103,8 +100,7 @@ else[HADOOP_NON_SECURE]*/
   private final int tcpBacklog;
   /** Factory for {@link RequestServerHandler} */
   private final RequestServerHandler.Factory requestServerHandlerFactory;
-/*if[HADOOP_NON_SECURE]
-else[HADOOP_NON_SECURE]*/
+/*if_not[HADOOP_NON_SECURE]*/
   /** Factory for {@link RequestServerHandler} */
   private SaslServerHandler.Factory saslServerHandlerFactory;
 /*end[HADOOP_NON_SECURE]*/
@@ -141,17 +137,16 @@ else[HADOOP_NON_SECURE]*/
     this.conf = conf;
     this.progressable = progressable;
     this.requestServerHandlerFactory = requestServerHandlerFactory;
-    /*if[HADOOP_NON_SECURE]
-    else[HADOOP_NON_SECURE]*/
+    /*if_not[HADOOP_NON_SECURE]*/
     this.saslServerHandlerFactory = new SaslServerHandler.Factory();
     /*end[HADOOP_NON_SECURE]*/
     this.myTaskInfo = myTaskInfo;
     sendBufferSize = conf.getInt(
-        GiraphConfiguration.SERVER_SEND_BUFFER_SIZE,
-        GiraphConfiguration.DEFAULT_SERVER_SEND_BUFFER_SIZE);
+        GiraphConstants.SERVER_SEND_BUFFER_SIZE,
+        GiraphConstants.DEFAULT_SERVER_SEND_BUFFER_SIZE);
     receiveBufferSize = conf.getInt(
-        GiraphConfiguration.SERVER_RECEIVE_BUFFER_SIZE,
-        GiraphConfiguration.DEFAULT_SERVER_RECEIVE_BUFFER_SIZE);
+        GiraphConstants.SERVER_RECEIVE_BUFFER_SIZE,
+        GiraphConstants.DEFAULT_SERVER_RECEIVE_BUFFER_SIZE);
 
     workerRequestReservedMap = new WorkerRequestReservedMap(conf);
 
@@ -169,12 +164,12 @@ else[HADOOP_NON_SECURE]*/
     }
 
     maxPoolSize = conf.getInt(
-        GiraphConfiguration.NETTY_SERVER_THREADS,
-        GiraphConfiguration.NETTY_SERVER_THREADS_DEFAULT);
+        GiraphConstants.NETTY_SERVER_THREADS,
+        GiraphConstants.NETTY_SERVER_THREADS_DEFAULT);
 
-    tcpBacklog = conf.getInt(GiraphConfiguration.TCP_BACKLOG,
-        conf.getInt(GiraphConfiguration.MAX_WORKERS,
-            GiraphConfiguration.TCP_BACKLOG_DEFAULT));
+    tcpBacklog = conf.getInt(GiraphConstants.TCP_BACKLOG,
+        conf.getInt(GiraphConstants.MAX_WORKERS,
+            GiraphConstants.TCP_BACKLOG_DEFAULT));
 
     channelFactory = new NioServerSocketChannelFactory(
         bossExecutorService,
@@ -182,11 +177,11 @@ else[HADOOP_NON_SECURE]*/
         maxPoolSize);
 
     handlerBeforeExecutionHandler = conf.get(
-        GiraphConfiguration.NETTY_SERVER_EXECUTION_AFTER_HANDLER,
-        GiraphConfiguration.NETTY_SERVER_EXECUTION_AFTER_HANDLER_DEFAULT);
+        GiraphConstants.NETTY_SERVER_EXECUTION_AFTER_HANDLER,
+        GiraphConstants.NETTY_SERVER_EXECUTION_AFTER_HANDLER_DEFAULT);
     boolean useExecutionHandler = conf.getBoolean(
-        GiraphConfiguration.NETTY_SERVER_USE_EXECUTION_HANDLER,
-        GiraphConfiguration.NETTY_SERVER_USE_EXECUTION_HANDLER_DEFAULT);
+        GiraphConstants.NETTY_SERVER_USE_EXECUTION_HANDLER,
+        GiraphConstants.NETTY_SERVER_USE_EXECUTION_HANDLER_DEFAULT);
     if (useExecutionHandler) {
       int executionThreads = conf.getNettyServerExecutionThreads();
       executionHandler = new ExecutionHandler(
@@ -204,8 +199,7 @@ else[HADOOP_NON_SECURE]*/
     }
   }
 
-/*if[HADOOP_NON_SECURE]
-else[HADOOP_NON_SECURE]*/
+/*if_not[HADOOP_NON_SECURE]*/
   /**
    * Constructor for creating the server
    *
@@ -249,8 +243,7 @@ else[HADOOP_NON_SECURE]*/
     bootstrap.setPipelineFactory(new ChannelPipelineFactory() {
       @Override
       public ChannelPipeline getPipeline() throws Exception {
-/*if[HADOOP_NON_SECURE]
-else[HADOOP_NON_SECURE]*/
+/*if_not[HADOOP_NON_SECURE]*/
         if (conf.authenticate()) {
           LOG.info("start: Will use Netty pipeline with " +
               "authentication and authorization of clients.");
@@ -300,8 +293,7 @@ else[HADOOP_NON_SECURE]*/
                 "executionHandler", executionHandler);
           }
           return pipeline;
-/*if[HADOOP_NON_SECURE]
-else[HADOOP_NON_SECURE]*/
+/*if_not[HADOOP_NON_SECURE]*/
         }
 /*end[HADOOP_NON_SECURE]*/
       }
@@ -310,19 +302,19 @@ else[HADOOP_NON_SECURE]*/
     int taskId = conf.getTaskPartition();
     int numTasks = conf.getInt("mapred.map.tasks", 1);
     // Number of workers + 1 for master
-    int numServers = conf.getInt(GiraphConfiguration.MAX_WORKERS, numTasks) + 1;
+    int numServers = conf.getInt(GiraphConstants.MAX_WORKERS, numTasks) + 1;
     int portIncrementConstant =
         (int) Math.pow(10, Math.ceil(Math.log10(numServers)));
-    int bindPort = conf.getInt(GiraphConfiguration.IPC_INITIAL_PORT,
-        GiraphConfiguration.IPC_INITIAL_PORT_DEFAULT) +
+    int bindPort = conf.getInt(GiraphConstants.IPC_INITIAL_PORT,
+        GiraphConstants.IPC_INITIAL_PORT_DEFAULT) +
         taskId;
     int bindAttempts = 0;
     final int maxIpcPortBindAttempts =
-        conf.getInt(GiraphConfiguration.MAX_IPC_PORT_BIND_ATTEMPTS,
-            GiraphConfiguration.MAX_IPC_PORT_BIND_ATTEMPTS_DEFAULT);
+        conf.getInt(GiraphConstants.MAX_IPC_PORT_BIND_ATTEMPTS,
+            GiraphConstants.MAX_IPC_PORT_BIND_ATTEMPTS_DEFAULT);
     final boolean failFirstPortBindingAttempt =
-        conf.getBoolean(GiraphConfiguration.FAIL_FIRST_IPC_PORT_BIND_ATTEMPT,
-            GiraphConfiguration.FAIL_FIRST_IPC_PORT_BIND_ATTEMPT_DEFAULT);
+        conf.getBoolean(GiraphConstants.FAIL_FIRST_IPC_PORT_BIND_ATTEMPT,
+            GiraphConstants.FAIL_FIRST_IPC_PORT_BIND_ATTEMPT_DEFAULT);
 
     // Simple handling of port collisions on the same machine while
     // preserving debugability from the port number alone.

http://git-wip-us.apache.org/repos/asf/giraph/blob/599f70e3/giraph/src/main/java/org/apache/giraph/comm/netty/NettyWorkerAggregatorRequestProcessor.java
----------------------------------------------------------------------
diff --git a/giraph/src/main/java/org/apache/giraph/comm/netty/NettyWorkerAggregatorRequestProcessor.java b/giraph/src/main/java/org/apache/giraph/comm/netty/NettyWorkerAggregatorRequestProcessor.java
index 4e05ebe..6838321 100644
--- a/giraph/src/main/java/org/apache/giraph/comm/netty/NettyWorkerAggregatorRequestProcessor.java
+++ b/giraph/src/main/java/org/apache/giraph/comm/netty/NettyWorkerAggregatorRequestProcessor.java
@@ -18,7 +18,7 @@
 
 package org.apache.giraph.comm.netty;
 
-import org.apache.giraph.ImmutableClassesGiraphConfiguration;
+import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
 import org.apache.giraph.bsp.CentralizedServiceWorker;
 import org.apache.giraph.comm.aggregators.WorkerAggregatorRequestProcessor;
 import org.apache.giraph.comm.WorkerClient;

http://git-wip-us.apache.org/repos/asf/giraph/blob/599f70e3/giraph/src/main/java/org/apache/giraph/comm/netty/NettyWorkerClient.java
----------------------------------------------------------------------
diff --git a/giraph/src/main/java/org/apache/giraph/comm/netty/NettyWorkerClient.java b/giraph/src/main/java/org/apache/giraph/comm/netty/NettyWorkerClient.java
index 14d8399..4aff7b9 100644
--- a/giraph/src/main/java/org/apache/giraph/comm/netty/NettyWorkerClient.java
+++ b/giraph/src/main/java/org/apache/giraph/comm/netty/NettyWorkerClient.java
@@ -18,7 +18,7 @@
 
 package org.apache.giraph.comm.netty;
 
-import org.apache.giraph.ImmutableClassesGiraphConfiguration;
+import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
 import org.apache.giraph.bsp.CentralizedServiceWorker;
 import org.apache.giraph.comm.WorkerClient;
 import org.apache.giraph.comm.requests.WritableRequest;

http://git-wip-us.apache.org/repos/asf/giraph/blob/599f70e3/giraph/src/main/java/org/apache/giraph/comm/netty/NettyWorkerClientRequestProcessor.java
----------------------------------------------------------------------
diff --git a/giraph/src/main/java/org/apache/giraph/comm/netty/NettyWorkerClientRequestProcessor.java b/giraph/src/main/java/org/apache/giraph/comm/netty/NettyWorkerClientRequestProcessor.java
index 9a1dd02..e5ab649 100644
--- a/giraph/src/main/java/org/apache/giraph/comm/netty/NettyWorkerClientRequestProcessor.java
+++ b/giraph/src/main/java/org/apache/giraph/comm/netty/NettyWorkerClientRequestProcessor.java
@@ -17,10 +17,6 @@
  */
 package org.apache.giraph.comm.netty;
 
-import java.io.IOException;
-import java.util.Map;
-import org.apache.giraph.GiraphConfiguration;
-import org.apache.giraph.ImmutableClassesGiraphConfiguration;
 import org.apache.giraph.bsp.CentralizedServiceWorker;
 import org.apache.giraph.comm.SendMessageCache;
 import org.apache.giraph.comm.SendMutationsCache;
@@ -35,6 +31,8 @@ import org.apache.giraph.comm.requests.SendVertexRequest;
 import org.apache.giraph.comm.requests.SendWorkerMessagesRequest;
 import org.apache.giraph.comm.requests.WorkerRequest;
 import org.apache.giraph.comm.requests.WritableRequest;
+import org.apache.giraph.conf.GiraphConstants;
+import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
 import org.apache.giraph.graph.BspService;
 import org.apache.giraph.graph.Edge;
 import org.apache.giraph.graph.Vertex;
@@ -51,6 +49,9 @@ import org.apache.hadoop.io.WritableComparable;
 import org.apache.hadoop.mapreduce.Mapper;
 import org.apache.log4j.Logger;
 
+import java.io.IOException;
+import java.util.Map;
+
 /**
  * Aggregate requests and sends them to the thread-safe NettyClient.  This
  * class is not thread-safe and expected to be used and then thrown away after
@@ -112,17 +113,17 @@ public class NettyWorkerClientRequestProcessor<I extends WritableComparable,
     sendMessageCache =
         new SendMessageCache<I, M>(configuration, serviceWorker);
     maxMessagesSizePerWorker = configuration.getInt(
-        GiraphConfiguration.MAX_MSG_REQUEST_SIZE,
-        GiraphConfiguration.MAX_MSG_REQUEST_SIZE_DEFAULT);
+        GiraphConstants.MAX_MSG_REQUEST_SIZE,
+        GiraphConstants.MAX_MSG_REQUEST_SIZE_DEFAULT);
     maxMutationsPerPartition = configuration.getInt(
-        GiraphConfiguration.MAX_MUTATIONS_PER_REQUEST,
-        GiraphConfiguration.MAX_MUTATIONS_PER_REQUEST_DEFAULT);
+        GiraphConstants.MAX_MUTATIONS_PER_REQUEST,
+        GiraphConstants.MAX_MUTATIONS_PER_REQUEST_DEFAULT);
     this.serviceWorker = serviceWorker;
     this.serverData = serviceWorker.getServerData();
 
     // Per-Superstep Metrics.
     // Since this object is not long lived we just initialize the metrics here.
-    GiraphMetrics gmr = GiraphMetrics.getInstance();
+    GiraphMetrics gmr = GiraphMetrics.get();
     msgsSentInSuperstep = new ValueGauge<Long>(gmr.perSuperstep(), "msgs-sent");
   }
 

http://git-wip-us.apache.org/repos/asf/giraph/blob/599f70e3/giraph/src/main/java/org/apache/giraph/comm/netty/NettyWorkerServer.java
----------------------------------------------------------------------
diff --git a/giraph/src/main/java/org/apache/giraph/comm/netty/NettyWorkerServer.java b/giraph/src/main/java/org/apache/giraph/comm/netty/NettyWorkerServer.java
index b162563..8012397 100644
--- a/giraph/src/main/java/org/apache/giraph/comm/netty/NettyWorkerServer.java
+++ b/giraph/src/main/java/org/apache/giraph/comm/netty/NettyWorkerServer.java
@@ -18,8 +18,6 @@
 
 package org.apache.giraph.comm.netty;
 
-import org.apache.giraph.GiraphConfiguration;
-import org.apache.giraph.ImmutableClassesGiraphConfiguration;
 import org.apache.giraph.bsp.CentralizedServiceWorker;
 import org.apache.giraph.comm.ServerData;
 import org.apache.giraph.comm.WorkerServer;
@@ -33,6 +31,8 @@ import org.apache.giraph.comm.messages.MessageStoreFactory;
 import org.apache.giraph.comm.messages.OneMessagePerVertexStore;
 import org.apache.giraph.comm.messages.SequentialFileMessageStore;
 import org.apache.giraph.comm.netty.handler.WorkerRequestServerHandler;
+import org.apache.giraph.conf.GiraphConstants;
+import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
 import org.apache.giraph.graph.GraphState;
 import org.apache.giraph.graph.Vertex;
 import org.apache.giraph.graph.VertexMutations;
@@ -104,8 +104,8 @@ public class NettyWorkerServer<I extends WritableComparable,
   private MessageStoreFactory<I, M, MessageStoreByPartition<I, M>>
   createMessageStoreFactory() {
     boolean useOutOfCoreMessaging = conf.getBoolean(
-        GiraphConfiguration.USE_OUT_OF_CORE_MESSAGES,
-        GiraphConfiguration.USE_OUT_OF_CORE_MESSAGES_DEFAULT);
+        GiraphConstants.USE_OUT_OF_CORE_MESSAGES,
+        GiraphConstants.USE_OUT_OF_CORE_MESSAGES_DEFAULT);
     if (!useOutOfCoreMessaging) {
       if (conf.useCombiner()) {
         if (LOG.isInfoEnabled()) {
@@ -123,8 +123,8 @@ public class NettyWorkerServer<I extends WritableComparable,
       }
     } else {
       int maxMessagesInMemory = conf.getInt(
-          GiraphConfiguration.MAX_MESSAGES_IN_MEMORY,
-          GiraphConfiguration.MAX_MESSAGES_IN_MEMORY_DEFAULT);
+          GiraphConstants.MAX_MESSAGES_IN_MEMORY,
+          GiraphConstants.MAX_MESSAGES_IN_MEMORY_DEFAULT);
       if (LOG.isInfoEnabled()) {
         LOG.info("createMessageStoreFactory: Using DiskBackedMessageStore, " +
             "maxMessagesInMemory = " + maxMessagesInMemory);

http://git-wip-us.apache.org/repos/asf/giraph/blob/599f70e3/giraph/src/main/java/org/apache/giraph/comm/netty/handler/MasterRequestServerHandler.java
----------------------------------------------------------------------
diff --git a/giraph/src/main/java/org/apache/giraph/comm/netty/handler/MasterRequestServerHandler.java b/giraph/src/main/java/org/apache/giraph/comm/netty/handler/MasterRequestServerHandler.java
index 8038947..b3f0121 100644
--- a/giraph/src/main/java/org/apache/giraph/comm/netty/handler/MasterRequestServerHandler.java
+++ b/giraph/src/main/java/org/apache/giraph/comm/netty/handler/MasterRequestServerHandler.java
@@ -18,7 +18,7 @@
 
 package org.apache.giraph.comm.netty.handler;
 
-import org.apache.giraph.ImmutableClassesGiraphConfiguration;
+import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
 import org.apache.giraph.comm.requests.MasterRequest;
 import org.apache.giraph.graph.MasterAggregatorHandler;
 import org.apache.giraph.graph.TaskInfo;

http://git-wip-us.apache.org/repos/asf/giraph/blob/599f70e3/giraph/src/main/java/org/apache/giraph/comm/netty/handler/RequestDecoder.java
----------------------------------------------------------------------
diff --git a/giraph/src/main/java/org/apache/giraph/comm/netty/handler/RequestDecoder.java b/giraph/src/main/java/org/apache/giraph/comm/netty/handler/RequestDecoder.java
index 4f13647..0bf21e5 100644
--- a/giraph/src/main/java/org/apache/giraph/comm/netty/handler/RequestDecoder.java
+++ b/giraph/src/main/java/org/apache/giraph/comm/netty/handler/RequestDecoder.java
@@ -18,7 +18,7 @@
 
 package org.apache.giraph.comm.netty.handler;
 
-import org.apache.giraph.ImmutableClassesGiraphConfiguration;
+import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
 import org.apache.giraph.comm.netty.ByteCounter;
 import org.apache.giraph.comm.requests.RequestType;
 import org.apache.giraph.comm.requests.WritableRequest;

http://git-wip-us.apache.org/repos/asf/giraph/blob/599f70e3/giraph/src/main/java/org/apache/giraph/comm/netty/handler/RequestServerHandler.java
----------------------------------------------------------------------
diff --git a/giraph/src/main/java/org/apache/giraph/comm/netty/handler/RequestServerHandler.java b/giraph/src/main/java/org/apache/giraph/comm/netty/handler/RequestServerHandler.java
index 0efdabd..cd26ea2 100644
--- a/giraph/src/main/java/org/apache/giraph/comm/netty/handler/RequestServerHandler.java
+++ b/giraph/src/main/java/org/apache/giraph/comm/netty/handler/RequestServerHandler.java
@@ -18,9 +18,9 @@
 
 package org.apache.giraph.comm.netty.handler;
 
-import org.apache.giraph.GiraphConfiguration;
-import org.apache.giraph.ImmutableClassesGiraphConfiguration;
 import org.apache.giraph.comm.requests.WritableRequest;
+import org.apache.giraph.conf.GiraphConstants;
+import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
 import org.apache.giraph.graph.TaskInfo;
 import org.apache.giraph.utils.SystemTime;
 import org.apache.giraph.utils.Time;
@@ -72,8 +72,8 @@ public abstract class RequestServerHandler<R> extends
       TaskInfo myTaskInfo) {
     this.workerRequestReservedMap = workerRequestReservedMap;
     closeFirstRequest = conf.getBoolean(
-        GiraphConfiguration.NETTY_SIMULATE_FIRST_REQUEST_CLOSED,
-        GiraphConfiguration.NETTY_SIMULATE_FIRST_REQUEST_CLOSED_DEFAULT);
+        GiraphConstants.NETTY_SIMULATE_FIRST_REQUEST_CLOSED,
+        GiraphConstants.NETTY_SIMULATE_FIRST_REQUEST_CLOSED_DEFAULT);
     this.myTaskInfo = myTaskInfo;
   }
 

http://git-wip-us.apache.org/repos/asf/giraph/blob/599f70e3/giraph/src/main/java/org/apache/giraph/comm/netty/handler/ResponseClientHandler.java
----------------------------------------------------------------------
diff --git a/giraph/src/main/java/org/apache/giraph/comm/netty/handler/ResponseClientHandler.java b/giraph/src/main/java/org/apache/giraph/comm/netty/handler/ResponseClientHandler.java
index faba08f..1803be4 100644
--- a/giraph/src/main/java/org/apache/giraph/comm/netty/handler/ResponseClientHandler.java
+++ b/giraph/src/main/java/org/apache/giraph/comm/netty/handler/ResponseClientHandler.java
@@ -18,7 +18,7 @@
 
 package org.apache.giraph.comm.netty.handler;
 
-import org.apache.giraph.GiraphConfiguration;
+import org.apache.giraph.conf.GiraphConstants;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.log4j.Logger;
 import org.jboss.netty.buffer.ChannelBuffer;
@@ -60,8 +60,8 @@ public class ResponseClientHandler extends SimpleChannelUpstreamHandler {
       Configuration conf) {
     this.workerIdOutstandingRequestMap = workerIdOutstandingRequestMap;
     dropFirstResponse = conf.getBoolean(
-        GiraphConfiguration.NETTY_SIMULATE_FIRST_RESPONSE_FAILED,
-        GiraphConfiguration.NETTY_SIMULATE_FIRST_RESPONSE_FAILED_DEFAULT);
+        GiraphConstants.NETTY_SIMULATE_FIRST_RESPONSE_FAILED,
+        GiraphConstants.NETTY_SIMULATE_FIRST_RESPONSE_FAILED_DEFAULT);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/giraph/blob/599f70e3/giraph/src/main/java/org/apache/giraph/comm/netty/handler/SaslServerHandler.java
----------------------------------------------------------------------
diff --git a/giraph/src/main/java/org/apache/giraph/comm/netty/handler/SaslServerHandler.java b/giraph/src/main/java/org/apache/giraph/comm/netty/handler/SaslServerHandler.java
index 61eaff4..d06fd09 100644
--- a/giraph/src/main/java/org/apache/giraph/comm/netty/handler/SaslServerHandler.java
+++ b/giraph/src/main/java/org/apache/giraph/comm/netty/handler/SaslServerHandler.java
@@ -18,13 +18,13 @@
 
 package org.apache.giraph.comm.netty.handler;
 
-import org.apache.giraph.GiraphConfiguration;
 import org.apache.giraph.comm.netty.NettyServer;
 import org.apache.giraph.comm.netty.SaslNettyServer;
 import org.apache.giraph.comm.requests.RequestType;
 import org.apache.giraph.comm.requests.SaslCompleteRequest;
 import org.apache.giraph.comm.requests.SaslTokenMessageRequest;
 import org.apache.giraph.comm.requests.WritableRequest;
+import org.apache.giraph.conf.GiraphConstants;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapreduce.security.TokenCache;
@@ -75,8 +75,8 @@ public class SaslServerHandler extends
     SaslNettyServer.init(conf);
     setupSecretManager(conf);
     closeFirstRequest = conf.getBoolean(
-        GiraphConfiguration.NETTY_SIMULATE_FIRST_REQUEST_CLOSED,
-        GiraphConfiguration.NETTY_SIMULATE_FIRST_REQUEST_CLOSED_DEFAULT);
+        GiraphConstants.NETTY_SIMULATE_FIRST_REQUEST_CLOSED,
+        GiraphConstants.NETTY_SIMULATE_FIRST_REQUEST_CLOSED_DEFAULT);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/giraph/blob/599f70e3/giraph/src/main/java/org/apache/giraph/comm/netty/handler/WorkerRequestReservedMap.java
----------------------------------------------------------------------
diff --git a/giraph/src/main/java/org/apache/giraph/comm/netty/handler/WorkerRequestReservedMap.java b/giraph/src/main/java/org/apache/giraph/comm/netty/handler/WorkerRequestReservedMap.java
index f0191ab..cf45292 100644
--- a/giraph/src/main/java/org/apache/giraph/comm/netty/handler/WorkerRequestReservedMap.java
+++ b/giraph/src/main/java/org/apache/giraph/comm/netty/handler/WorkerRequestReservedMap.java
@@ -18,14 +18,15 @@
 
 package org.apache.giraph.comm.netty.handler;
 
-import com.google.common.collect.MapMaker;
-import java.util.concurrent.ConcurrentMap;
-
-import org.apache.giraph.GiraphConfiguration;
-import org.apache.giraph.utils.IncreasingBitSet;
 import org.apache.giraph.comm.netty.NettyServer;
+import org.apache.giraph.conf.GiraphConstants;
+import org.apache.giraph.utils.IncreasingBitSet;
 import org.apache.hadoop.conf.Configuration;
 
+import com.google.common.collect.MapMaker;
+
+import java.util.concurrent.ConcurrentMap;
+
 /**
  * Provides a thread-safe map for checking worker and request id pairs
  */
@@ -41,7 +42,7 @@ public class WorkerRequestReservedMap {
    */
   public WorkerRequestReservedMap(Configuration conf) {
     workerRequestReservedMap = new MapMaker().concurrencyLevel(
-        conf.getInt(GiraphConfiguration.MSG_NUM_FLUSH_THREADS,
+        conf.getInt(GiraphConstants.MSG_NUM_FLUSH_THREADS,
             NettyServer.MAXIMUM_THREAD_POOL_SIZE_DEFAULT)).makeMap();
   }
 

http://git-wip-us.apache.org/repos/asf/giraph/blob/599f70e3/giraph/src/main/java/org/apache/giraph/comm/netty/handler/WorkerRequestServerHandler.java
----------------------------------------------------------------------
diff --git a/giraph/src/main/java/org/apache/giraph/comm/netty/handler/WorkerRequestServerHandler.java b/giraph/src/main/java/org/apache/giraph/comm/netty/handler/WorkerRequestServerHandler.java
index ceafca8..b4e7dda 100644
--- a/giraph/src/main/java/org/apache/giraph/comm/netty/handler/WorkerRequestServerHandler.java
+++ b/giraph/src/main/java/org/apache/giraph/comm/netty/handler/WorkerRequestServerHandler.java
@@ -18,7 +18,7 @@
 
 package org.apache.giraph.comm.netty.handler;
 
-import org.apache.giraph.ImmutableClassesGiraphConfiguration;
+import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
 import org.apache.giraph.comm.ServerData;
 import org.apache.giraph.comm.requests.WorkerRequest;
 import org.apache.giraph.graph.TaskInfo;

http://git-wip-us.apache.org/repos/asf/giraph/blob/599f70e3/giraph/src/main/java/org/apache/giraph/comm/requests/WritableRequest.java
----------------------------------------------------------------------
diff --git a/giraph/src/main/java/org/apache/giraph/comm/requests/WritableRequest.java b/giraph/src/main/java/org/apache/giraph/comm/requests/WritableRequest.java
index fc7f875..fad20b0 100644
--- a/giraph/src/main/java/org/apache/giraph/comm/requests/WritableRequest.java
+++ b/giraph/src/main/java/org/apache/giraph/comm/requests/WritableRequest.java
@@ -21,8 +21,8 @@ package org.apache.giraph.comm.requests;
 import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
-import org.apache.giraph.ImmutableClassesGiraphConfigurable;
-import org.apache.giraph.ImmutableClassesGiraphConfiguration;
+import org.apache.giraph.conf.ImmutableClassesGiraphConfigurable;
+import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparable;
 


Mime
View raw message