You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@giraph.apache.org by er...@apache.org on 2012/10/14 10:20:48 UTC
svn commit: r1398033 - in /giraph/trunk: ./
giraph-formats-contrib/src/main/java/org/apache/giraph/io/hbase/
giraph-formats-contrib/src/test/java/org/apache/giraph/io/hbase/
giraph-formats-contrib/src/test/java/org/apache/giraph/io/hbase/edgemarker/
Author: ereisman
Date: Sun Oct 14 08:20:48 2012
New Revision: 1398033
URL: http://svn.apache.org/viewvc?rev=1398033&view=rev
Log:
GIRAPH-368: HBaseVertexInput/Output formats handle setConf() internally (bfem via ereisman)
Modified:
giraph/trunk/CHANGELOG
giraph/trunk/giraph-formats-contrib/src/main/java/org/apache/giraph/io/hbase/HBaseVertexInputFormat.java
giraph/trunk/giraph-formats-contrib/src/main/java/org/apache/giraph/io/hbase/HBaseVertexOutputFormat.java
giraph/trunk/giraph-formats-contrib/src/test/java/org/apache/giraph/io/hbase/TestHBaseRootMarkerVertextFormat.java
giraph/trunk/giraph-formats-contrib/src/test/java/org/apache/giraph/io/hbase/edgemarker/TableEdgeInputFormat.java
giraph/trunk/giraph-formats-contrib/src/test/java/org/apache/giraph/io/hbase/edgemarker/TableEdgeOutputFormat.java
Modified: giraph/trunk/CHANGELOG
URL: http://svn.apache.org/viewvc/giraph/trunk/CHANGELOG?rev=1398033&r1=1398032&r2=1398033&view=diff
==============================================================================
--- giraph/trunk/CHANGELOG (original)
+++ giraph/trunk/CHANGELOG Sun Oct 14 08:20:48 2012
@@ -1,6 +1,8 @@
Giraph Change Log
Release 0.2.0 - unreleased
+ GIRAPH-368: HBase Vertex I/O formats handle setConf() internally (bfem via ereisman)
+
GIRAPH-367: Expose WorkerInfo to clients (Nitay Joffe via ereisman)
GIRAPH-370: AccumuloVertexOutputFormat public visibility for
Modified: giraph/trunk/giraph-formats-contrib/src/main/java/org/apache/giraph/io/hbase/HBaseVertexInputFormat.java
URL: http://svn.apache.org/viewvc/giraph/trunk/giraph-formats-contrib/src/main/java/org/apache/giraph/io/hbase/HBaseVertexInputFormat.java?rev=1398033&r1=1398032&r2=1398033&view=diff
==============================================================================
--- giraph/trunk/giraph-formats-contrib/src/main/java/org/apache/giraph/io/hbase/HBaseVertexInputFormat.java (original)
+++ giraph/trunk/giraph-formats-contrib/src/main/java/org/apache/giraph/io/hbase/HBaseVertexInputFormat.java Sun Oct 14 08:20:48 2012
@@ -19,7 +19,6 @@ package org.apache.giraph.io.hbase;
import org.apache.giraph.graph.VertexInputFormat;
import org.apache.giraph.graph.VertexReader;
-import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
import org.apache.hadoop.hbase.mapreduce.TableInputFormat;
@@ -29,6 +28,7 @@ import org.apache.hadoop.mapreduce.Input
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.List;
@@ -61,21 +61,17 @@ public abstract class HBaseVertexInputF
M extends Writable>
extends VertexInputFormat<I, V, E, M> {
- /**
+
+ /**
* delegate HBase table input format
*/
protected static final TableInputFormat BASE_FORMAT =
new TableInputFormat();
-
- /**
- * static method to initialize
- * base table input format
- * with Configuration.
- * @param conf job configuration
- */
- public static void setConf(Configuration conf) {
- BASE_FORMAT.setConf(conf);
- }
+ /**
+ * logger
+ */
+ private static final Logger LOG =
+ Logger.getLogger(HBaseVertexInputFormat.class);
/**
* Takes an instance of RecordReader that supports
@@ -105,12 +101,15 @@ public abstract class HBaseVertexInputF
private TaskAttemptContext context;
/**
- * constructor used for subclassing vertex record reader.
- * @param reader HBase record reader instance
+ * sets the base TableInputFOrmat and creates a record reader.
+ * @param split InputSplit
+ * @param context Context
+ * @throws IOException
*/
- public HBaseVertexReader(RecordReader<ImmutableBytesWritable,
- Result> reader) {
- this.reader = reader;
+ public HBaseVertexReader(InputSplit split, TaskAttemptContext context)
+ throws IOException {
+ BASE_FORMAT.setConf(context.getConfiguration());
+ this.reader = BASE_FORMAT.createRecordReader(split, context);
}
/**
@@ -182,6 +181,7 @@ public abstract class HBaseVertexInputF
public List<InputSplit> getSplits(
JobContext context, int numWorkers)
throws IOException, InterruptedException {
+ BASE_FORMAT.setConf(context.getConfiguration());
return BASE_FORMAT.getSplits(context);
}
}
Modified: giraph/trunk/giraph-formats-contrib/src/main/java/org/apache/giraph/io/hbase/HBaseVertexOutputFormat.java
URL: http://svn.apache.org/viewvc/giraph/trunk/giraph-formats-contrib/src/main/java/org/apache/giraph/io/hbase/HBaseVertexOutputFormat.java?rev=1398033&r1=1398032&r2=1398033&view=diff
==============================================================================
--- giraph/trunk/giraph-formats-contrib/src/main/java/org/apache/giraph/io/hbase/HBaseVertexOutputFormat.java (original)
+++ giraph/trunk/giraph-formats-contrib/src/main/java/org/apache/giraph/io/hbase/HBaseVertexOutputFormat.java Sun Oct 14 08:20:48 2012
@@ -20,7 +20,6 @@ package org.apache.giraph.io.hbase;
import org.apache.giraph.graph.VertexOutputFormat;
import org.apache.giraph.graph.VertexWriter;
-import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
import org.apache.hadoop.hbase.mapreduce.TableOutputFormat;
import org.apache.hadoop.io.Writable;
@@ -92,12 +91,13 @@ public abstract class HBaseVertexOutputF
Writable> recordWriter;
/**
- * Constructor for subclasses to implement recordWriter
- * @param recordWriter subclass instance
+ * Sets up base table output format and creates a record writer.
+ * @param context task attempt context
*/
- public HBaseVertexWriter(RecordWriter<ImmutableBytesWritable,
- Writable> recordWriter) {
- this.recordWriter = recordWriter;
+ public HBaseVertexWriter(TaskAttemptContext context)
+ throws IOException, InterruptedException {
+ BASE_FORMAT.setConf(context.getConfiguration());
+ this.recordWriter = BASE_FORMAT.getRecordWriter(context);
}
/**
@@ -145,15 +145,6 @@ public abstract class HBaseVertexOutputF
}
/**
- * setConf
- *
- * @param conf Injected configuration instance
- */
- public static void setConf(Configuration conf) {
- BASE_FORMAT.setConf(conf);
- }
-
- /**
* checkOutputSpecs
*
* @param context information about the job
@@ -176,6 +167,7 @@ public abstract class HBaseVertexOutputF
public OutputCommitter getOutputCommitter(
TaskAttemptContext context)
throws IOException, InterruptedException {
+ BASE_FORMAT.setConf(context.getConfiguration());
return BASE_FORMAT.getOutputCommitter(context);
}
}
Modified: giraph/trunk/giraph-formats-contrib/src/test/java/org/apache/giraph/io/hbase/TestHBaseRootMarkerVertextFormat.java
URL: http://svn.apache.org/viewvc/giraph/trunk/giraph-formats-contrib/src/test/java/org/apache/giraph/io/hbase/TestHBaseRootMarkerVertextFormat.java?rev=1398033&r1=1398032&r2=1398033&view=diff
==============================================================================
--- giraph/trunk/giraph-formats-contrib/src/test/java/org/apache/giraph/io/hbase/TestHBaseRootMarkerVertextFormat.java (original)
+++ giraph/trunk/giraph-formats-contrib/src/test/java/org/apache/giraph/io/hbase/TestHBaseRootMarkerVertextFormat.java Sun Oct 14 08:20:48 2012
@@ -140,8 +140,6 @@ public class TestHBaseRootMarkerVertextF
//now operate over HBase using Vertex I/O formats
conf.set(TableInputFormat.INPUT_TABLE, TABLE_NAME);
conf.set(TableOutputFormat.OUTPUT_TABLE, TABLE_NAME);
- HBaseVertexInputFormat.setConf(conf);
- HBaseVertexOutputFormat.setConf(conf);
GiraphJob giraphJob = new GiraphJob(conf, getCallingMethodName());
GiraphConfiguration giraphConf = giraphJob.getConfiguration();
Modified: giraph/trunk/giraph-formats-contrib/src/test/java/org/apache/giraph/io/hbase/edgemarker/TableEdgeInputFormat.java
URL: http://svn.apache.org/viewvc/giraph/trunk/giraph-formats-contrib/src/test/java/org/apache/giraph/io/hbase/edgemarker/TableEdgeInputFormat.java?rev=1398033&r1=1398032&r2=1398033&view=diff
==============================================================================
--- giraph/trunk/giraph-formats-contrib/src/test/java/org/apache/giraph/io/hbase/edgemarker/TableEdgeInputFormat.java (original)
+++ giraph/trunk/giraph-formats-contrib/src/test/java/org/apache/giraph/io/hbase/edgemarker/TableEdgeInputFormat.java Sun Oct 14 08:20:48 2012
@@ -51,8 +51,7 @@ public class TableEdgeInputFormat extend
createVertexReader(InputSplit split,
TaskAttemptContext context) throws IOException {
- return new TableEdgeVertexReader(
- BASE_FORMAT.createRecordReader(split, context));
+ return new TableEdgeVertexReader(split, context);
}
@@ -65,9 +64,8 @@ public class TableEdgeInputFormat extend
private final byte[] CF = Bytes.toBytes("cf");
private final byte[] CHILDREN = Bytes.toBytes("children");
- public TableEdgeVertexReader(
- RecordReader<ImmutableBytesWritable, Result> recordReader) {
- super(recordReader);
+ public TableEdgeVertexReader(InputSplit split, TaskAttemptContext context) throws IOException {
+ super(split, context);
}
Modified: giraph/trunk/giraph-formats-contrib/src/test/java/org/apache/giraph/io/hbase/edgemarker/TableEdgeOutputFormat.java
URL: http://svn.apache.org/viewvc/giraph/trunk/giraph-formats-contrib/src/test/java/org/apache/giraph/io/hbase/edgemarker/TableEdgeOutputFormat.java?rev=1398033&r1=1398032&r2=1398033&view=diff
==============================================================================
--- giraph/trunk/giraph-formats-contrib/src/test/java/org/apache/giraph/io/hbase/edgemarker/TableEdgeOutputFormat.java (original)
+++ giraph/trunk/giraph-formats-contrib/src/test/java/org/apache/giraph/io/hbase/edgemarker/TableEdgeOutputFormat.java Sun Oct 14 08:20:48 2012
@@ -39,9 +39,7 @@ public class TableEdgeOutputFormat
public VertexWriter<Text, Text, Text>
createVertexWriter(TaskAttemptContext context)
throws IOException, InterruptedException {
- RecordWriter<ImmutableBytesWritable, Writable> writer =
- BASE_FORMAT.getRecordWriter(context);
- return new TableEdgeVertexWriter(writer);
+ return new TableEdgeVertexWriter(context);
}
/*
@@ -54,9 +52,9 @@ public class TableEdgeOutputFormat
private final byte[] CF = Bytes.toBytes("cf");
private final byte[] PARENT = Bytes.toBytes("parent");
- public TableEdgeVertexWriter(
- RecordWriter<ImmutableBytesWritable, Writable> writer) {
- super(writer);
+ public TableEdgeVertexWriter(TaskAttemptContext context)
+ throws IOException, InterruptedException {
+ super(context);
}
/*
Record the vertex value as a the value for a new qualifier 'parent'.