You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@giraph.apache.org by ma...@apache.org on 2013/03/14 01:39:43 UTC

git commit: GIRAPH-562: Implement getConf/setConf in AbstractHive classes (majakabiljo)

Updated Branches:
  refs/heads/trunk a0433d21d -> a2dfd6792


GIRAPH-562: Implement getConf/setConf in AbstractHive classes (majakabiljo)


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

Branch: refs/heads/trunk
Commit: a2dfd6792db0c0232f56f29510111a0e213cc46b
Parents: a0433d2
Author: Maja Kabiljo <ma...@maja-mbp.thefacebook.com>
Authored: Wed Mar 13 17:37:35 2013 -0700
Committer: Maja Kabiljo <ma...@maja-mbp.thefacebook.com>
Committed: Wed Mar 13 17:38:49 2013 -0700

----------------------------------------------------------------------
 CHANGELOG                                          |    2 +
 .../DefaultImmutableClassesGiraphConfigurable.java |   47 +++++++++++++++
 .../giraph/hive/input/edge/AbstractHiveToEdge.java |    9 +--
 .../input/vertex/AbstractHiveToVertexEdges.java    |    9 +--
 .../input/vertex/AbstractHiveToVertexValue.java    |    8 +-
 .../giraph/hive/output/AbstractVertexToHive.java   |    9 +--
 6 files changed, 65 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/giraph/blob/a2dfd679/CHANGELOG
----------------------------------------------------------------------
diff --git a/CHANGELOG b/CHANGELOG
index b4802d1..3603786 100644
--- a/CHANGELOG
+++ b/CHANGELOG
@@ -1,6 +1,8 @@
 Giraph Change Log
 
 Release 0.2.0 - unreleased
+  GIRAPH-562: Implement getConf/setConf in AbstractHive classes (majakabiljo)
+
   GIRAPH-561: Only print vertex/edge input classes if not null (aching)
 
   GIRAPH-554: Set PartitionContext in InternalVertexRunner (majakabiljo)

http://git-wip-us.apache.org/repos/asf/giraph/blob/a2dfd679/giraph-core/src/main/java/org/apache/giraph/conf/DefaultImmutableClassesGiraphConfigurable.java
----------------------------------------------------------------------
diff --git a/giraph-core/src/main/java/org/apache/giraph/conf/DefaultImmutableClassesGiraphConfigurable.java b/giraph-core/src/main/java/org/apache/giraph/conf/DefaultImmutableClassesGiraphConfigurable.java
new file mode 100644
index 0000000..b3bdbf7
--- /dev/null
+++ b/giraph-core/src/main/java/org/apache/giraph/conf/DefaultImmutableClassesGiraphConfigurable.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.giraph.conf;
+
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
+
+/**
+ * Default implementation of ImmutableClassesGiraphConfigurable
+ *
+ * @param <I> Vertex id
+ * @param <V> Vertex data
+ * @param <E> Edge data
+ * @param <M> Message data
+ */
+public class DefaultImmutableClassesGiraphConfigurable<I extends WritableComparable,
+    V extends Writable, E extends Writable, M extends Writable> implements
+    ImmutableClassesGiraphConfigurable<I, V, E, M> {
+  /** Configuration */
+  private ImmutableClassesGiraphConfiguration<I, V, E, M> conf;
+
+  @Override
+  public void setConf(ImmutableClassesGiraphConfiguration<I, V, E, M> conf) {
+    this.conf = conf;
+  }
+
+  @Override
+  public ImmutableClassesGiraphConfiguration<I, V, E, M> getConf() {
+    return conf;
+  }
+}

http://git-wip-us.apache.org/repos/asf/giraph/blob/a2dfd679/giraph-hive/src/main/java/org/apache/giraph/hive/input/edge/AbstractHiveToEdge.java
----------------------------------------------------------------------
diff --git a/giraph-hive/src/main/java/org/apache/giraph/hive/input/edge/AbstractHiveToEdge.java b/giraph-hive/src/main/java/org/apache/giraph/hive/input/edge/AbstractHiveToEdge.java
index c418085..f29fea7 100644
--- a/giraph-hive/src/main/java/org/apache/giraph/hive/input/edge/AbstractHiveToEdge.java
+++ b/giraph-hive/src/main/java/org/apache/giraph/hive/input/edge/AbstractHiveToEdge.java
@@ -17,7 +17,7 @@
  */
 package org.apache.giraph.hive.input.edge;
 
-import org.apache.giraph.conf.ImmutableClassesGiraphConfigurable;
+import org.apache.giraph.conf.DefaultImmutableClassesGiraphConfigurable;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparable;
 
@@ -32,7 +32,6 @@ import com.facebook.giraph.hive.HiveTableSchemaAware;
  * @param <M> Message Value
  */
 public abstract class AbstractHiveToEdge<I extends WritableComparable,
-    V extends Writable, E extends Writable, M extends Writable> implements
-    HiveTableSchemaAware,
-    HiveToEdge<I, E>,
-    ImmutableClassesGiraphConfigurable<I, V, E, M> { }
+    V extends Writable, E extends Writable, M extends Writable>
+    extends DefaultImmutableClassesGiraphConfigurable<I, V, E, M>
+    implements HiveTableSchemaAware, HiveToEdge<I, E> { }

http://git-wip-us.apache.org/repos/asf/giraph/blob/a2dfd679/giraph-hive/src/main/java/org/apache/giraph/hive/input/vertex/AbstractHiveToVertexEdges.java
----------------------------------------------------------------------
diff --git a/giraph-hive/src/main/java/org/apache/giraph/hive/input/vertex/AbstractHiveToVertexEdges.java b/giraph-hive/src/main/java/org/apache/giraph/hive/input/vertex/AbstractHiveToVertexEdges.java
index 746e9e3..7b01dac 100644
--- a/giraph-hive/src/main/java/org/apache/giraph/hive/input/vertex/AbstractHiveToVertexEdges.java
+++ b/giraph-hive/src/main/java/org/apache/giraph/hive/input/vertex/AbstractHiveToVertexEdges.java
@@ -17,7 +17,7 @@
  */
 package org.apache.giraph.hive.input.vertex;
 
-import org.apache.giraph.conf.ImmutableClassesGiraphConfigurable;
+import org.apache.giraph.conf.DefaultImmutableClassesGiraphConfigurable;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparable;
 
@@ -32,7 +32,6 @@ import com.facebook.giraph.hive.HiveTableSchemaAware;
  * @param <M> Message Value
  */
 public abstract class AbstractHiveToVertexEdges<I extends WritableComparable,
-    V extends Writable, E extends Writable, M extends Writable> implements
-    HiveTableSchemaAware,
-    HiveToVertexEdges<I, E>,
-    ImmutableClassesGiraphConfigurable<I, V, E, M> { }
+    V extends Writable, E extends Writable, M extends Writable>
+    extends DefaultImmutableClassesGiraphConfigurable<I, V, E, M>
+    implements HiveTableSchemaAware, HiveToVertexEdges<I, E> { }

http://git-wip-us.apache.org/repos/asf/giraph/blob/a2dfd679/giraph-hive/src/main/java/org/apache/giraph/hive/input/vertex/AbstractHiveToVertexValue.java
----------------------------------------------------------------------
diff --git a/giraph-hive/src/main/java/org/apache/giraph/hive/input/vertex/AbstractHiveToVertexValue.java b/giraph-hive/src/main/java/org/apache/giraph/hive/input/vertex/AbstractHiveToVertexValue.java
index 9636d2a..5c279b5 100644
--- a/giraph-hive/src/main/java/org/apache/giraph/hive/input/vertex/AbstractHiveToVertexValue.java
+++ b/giraph-hive/src/main/java/org/apache/giraph/hive/input/vertex/AbstractHiveToVertexValue.java
@@ -17,7 +17,7 @@
  */
 package org.apache.giraph.hive.input.vertex;
 
-import org.apache.giraph.conf.ImmutableClassesGiraphConfigurable;
+import org.apache.giraph.conf.DefaultImmutableClassesGiraphConfigurable;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparable;
 
@@ -32,6 +32,6 @@ import com.facebook.giraph.hive.HiveTableSchemaAware;
  * @param <M> Message Value
  */
 public abstract class AbstractHiveToVertexValue<I extends WritableComparable,
-    V extends Writable, E extends Writable, M extends Writable> implements
-    HiveTableSchemaAware, HiveToVertexValue<I, V>,
-    ImmutableClassesGiraphConfigurable<I, V, E, M> { }
+    V extends Writable, E extends Writable, M extends Writable>
+    extends DefaultImmutableClassesGiraphConfigurable<I, V, E, M>
+    implements HiveTableSchemaAware, HiveToVertexValue<I, V> { }

http://git-wip-us.apache.org/repos/asf/giraph/blob/a2dfd679/giraph-hive/src/main/java/org/apache/giraph/hive/output/AbstractVertexToHive.java
----------------------------------------------------------------------
diff --git a/giraph-hive/src/main/java/org/apache/giraph/hive/output/AbstractVertexToHive.java b/giraph-hive/src/main/java/org/apache/giraph/hive/output/AbstractVertexToHive.java
index f2f2267..8e3f1ca 100644
--- a/giraph-hive/src/main/java/org/apache/giraph/hive/output/AbstractVertexToHive.java
+++ b/giraph-hive/src/main/java/org/apache/giraph/hive/output/AbstractVertexToHive.java
@@ -17,7 +17,7 @@
  */
 package org.apache.giraph.hive.output;
 
-import org.apache.giraph.conf.ImmutableClassesGiraphConfigurable;
+import org.apache.giraph.conf.DefaultImmutableClassesGiraphConfigurable;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparable;
 
@@ -32,7 +32,6 @@ import com.facebook.giraph.hive.HiveTableSchemaAware;
  * @param <M> Message Value
  */
 public abstract class AbstractVertexToHive<I extends WritableComparable,
-    V extends Writable, E extends Writable, M extends Writable> implements
-    HiveTableSchemaAware,
-    VertexToHive<I, V, E>,
-    ImmutableClassesGiraphConfigurable<I, V, E, M> { }
+    V extends Writable, E extends Writable, M extends Writable>
+    extends DefaultImmutableClassesGiraphConfigurable<I, V, E, M>
+    implements HiveTableSchemaAware, VertexToHive<I, V, E> { }