Author: apresta
Date: Tue Nov 20 05:43:29 2012
New Revision: 1411549
URL: http://svn.apache.org/viewvc?rev=1411549&view=rev
Log:
GIRAPH-405: Edge-based input from HCatalog (apresta)
Modified:
giraph/trunk/CHANGELOG
giraph/trunk/giraph-formats-contrib/pom.xml
giraph/trunk/giraph-formats-contrib/src/main/java/org/apache/giraph/io/hcatalog/HCatalogVertexInputFormat.java
giraph/trunk/giraph-formats-contrib/src/main/java/org/apache/giraph/io/hcatalog/HCatalogVertexOutputFormat.java
giraph/trunk/giraph-formats-contrib/src/main/java/org/apache/giraph/io/hcatalog/HiveGiraphRunner.java
giraph/trunk/pom.xml
Modified: giraph/trunk/CHANGELOG
URL:
http://svn.apache.org/viewvc/giraph/trunk/CHANGELOG?rev=1411549&r1=1411548&r2=1411549&view=diff
==============================================================================
--- giraph/trunk/CHANGELOG (original)
+++ giraph/trunk/CHANGELOG Tue Nov 20 05:43:29 2012
@@ -1,6 +1,8 @@
Giraph Change Log
Release 0.2.0 - unreleased
+ GIRAPH-405: Edge-based input from HCatalog (apresta)
+
GIRAPH-433: Vertex#initialize() shouldn't be called with null edges (apresta)
GIRAPH-416: MasterObserver for user post-application customization (nitay)
Modified: giraph/trunk/giraph-formats-contrib/pom.xml
URL:
http://svn.apache.org/viewvc/giraph/trunk/giraph-formats-contrib/pom.xml?rev=1411549&r1=1411548&r2=1411549&view=diff
==============================================================================
--- giraph/trunk/giraph-formats-contrib/pom.xml (original)
+++ giraph/trunk/giraph-formats-contrib/pom.xml Tue Nov 20 05:43:29 2012
@@ -194,6 +194,14 @@ under the License.
<artifactId>hive-common</artifactId>
</dependency>
<dependency>
+ <groupId>org.apache.hive</groupId>
+ <artifactId>hive-exec</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hive</groupId>
+ <artifactId>hive-metastore</artifactId>
+ </dependency>
+ <dependency>
<groupId>org.apache.zookeeper</groupId>
<artifactId>zookeeper</artifactId>
</dependency>
Modified:
giraph/trunk/giraph-formats-contrib/src/main/java/org/apache/giraph/io/hcatalog/HCatalogVertexInputFormat.java
URL:
http://svn.apache.org/viewvc/giraph/trunk/giraph-formats-contrib/src/main/java/org/apache/giraph/io/hcatalog/HCatalogVertexInputFormat.java?rev=1411549&r1=1411548&r2=1411549&view=diff
==============================================================================
---
giraph/trunk/giraph-formats-contrib/src/main/java/org/apache/giraph/io/hcatalog/HCatalogVertexInputFormat.java
(original)
+++
giraph/trunk/giraph-formats-contrib/src/main/java/org/apache/giraph/io/hcatalog/HCatalogVertexInputFormat.java
Tue Nov 20 05:43:29 2012
@@ -18,10 +18,6 @@
package org.apache.giraph.io.hcatalog;
-import java.io.IOException;
-import java.util.List;
-import java.util.Map;
-
import org.apache.giraph.ImmutableClassesGiraphConfiguration;
import org.apache.giraph.graph.Vertex;
import org.apache.giraph.graph.VertexInputFormat;
@@ -33,11 +29,14 @@ import org.apache.hadoop.mapreduce.JobCo
import org.apache.hadoop.mapreduce.RecordReader;
import org.apache.hadoop.mapreduce.TaskAttemptContext;
import org.apache.hcatalog.data.HCatRecord;
-import org.apache.hcatalog.mapreduce.HCatInputFormat;
+import org.apache.log4j.Logger;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
-import org.apache.log4j.Logger;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
/**
* Abstract class that users should subclass to load data from a Hive or Pig
@@ -47,10 +46,12 @@ import org.apache.log4j.Logger;
* stored in the input table.
* <p>
* The desired database and table name to load from can be specified via
- * {@link HCatInputFormat#setInput(org.apache.hadoop.mapreduce.Job,
+ * {@link GiraphHCatInputFormat#setVertexInput(org.apache.hadoop.mapreduce.Job,
* org.apache.hcatalog.mapreduce.InputJobInfo)}
* as you setup your vertex input format with
- * {@link GiraphJob#setVertexInputFormatClass(Class)}.
+ * {@link org.apache.giraph.GiraphConfiguration#
+ * setVertexInputFormatClass(Class)}.
+ *
* @param <I> Vertex id
* @param <V> Vertex value
* @param <E> Edge value
@@ -65,15 +66,15 @@ public abstract class HCatalogVertexInpu
M extends Writable>
extends VertexInputFormat<I, V, E, M> {
/**
- * H catalog input format.
+ * HCatalog input format.
*/
- private HCatInputFormat hCatInputFormat = new HCatInputFormat();
+ private GiraphHCatInputFormat hCatInputFormat = new GiraphHCatInputFormat();
@Override
public final List<InputSplit> getSplits(
final JobContext context, final int numWorkers)
throws IOException, InterruptedException {
- return hCatInputFormat.getSplits(context);
+ return hCatInputFormat.getVertexSplits(context);
}
/**
@@ -162,7 +163,7 @@ public abstract class HCatalogVertexInpu
}
/**
- * create vertex writer instance.
+ * create vertex reader instance.
* @return HCatalogVertexReader
*/
protected abstract HCatalogVertexReader createVertexReader();
@@ -175,7 +176,7 @@ public abstract class HCatalogVertexInpu
try {
HCatalogVertexReader reader = createVertexReader();
reader.initialize(hCatInputFormat.
- createRecordReader(split, context));
+ createVertexRecordReader(split, context));
return reader;
} catch (InterruptedException e) {
throw new IllegalStateException(
Modified:
giraph/trunk/giraph-formats-contrib/src/main/java/org/apache/giraph/io/hcatalog/HCatalogVertexOutputFormat.java
URL:
http://svn.apache.org/viewvc/giraph/trunk/giraph-formats-contrib/src/main/java/org/apache/giraph/io/hcatalog/HCatalogVertexOutputFormat.java?rev=1411549&r1=1411548&r2=1411549&view=diff
==============================================================================
---
giraph/trunk/giraph-formats-contrib/src/main/java/org/apache/giraph/io/hcatalog/HCatalogVertexOutputFormat.java
(original)
+++
giraph/trunk/giraph-formats-contrib/src/main/java/org/apache/giraph/io/hcatalog/HCatalogVertexOutputFormat.java
Tue Nov 20 05:43:29 2012
@@ -18,8 +18,6 @@
package org.apache.giraph.io.hcatalog;
-import java.io.IOException;
-
import org.apache.giraph.graph.Vertex;
import org.apache.giraph.graph.VertexOutputFormat;
import org.apache.giraph.graph.VertexWriter;
@@ -33,6 +31,8 @@ import org.apache.hcatalog.data.DefaultH
import org.apache.hcatalog.data.HCatRecord;
import org.apache.hcatalog.mapreduce.HCatOutputFormat;
+import java.io.IOException;
+
/**
* Abstract class that users should subclass to store data to Hive or Pig
table.
* You can easily implement a {@link HCatalogVertexWriter} by extending
@@ -43,8 +43,9 @@ import org.apache.hcatalog.mapreduce.HCa
* {@link HCatOutputFormat#setOutput(org.apache.hadoop.mapreduce.Job,
* org.apache.hcatalog.mapreduce.OutputJobInfo)}
* as you setup your vertex output format with
- * {@link GiraphJob#setVertexOutputFormatClass(Class)}. You must create the
- * output table.
+ * {@link org.apache.giraph.GiraphConfiguration#
+ * setVertexOutputFormatClass(Class)}. You must create the output table.
+ *
* @param <I> Vertex id
* @param <V> Vertex value
* @param <E> Edge value
Modified:
giraph/trunk/giraph-formats-contrib/src/main/java/org/apache/giraph/io/hcatalog/HiveGiraphRunner.java
URL:
http://svn.apache.org/viewvc/giraph/trunk/giraph-formats-contrib/src/main/java/org/apache/giraph/io/hcatalog/HiveGiraphRunner.java?rev=1411549&r1=1411548&r2=1411549&view=diff
==============================================================================
---
giraph/trunk/giraph-formats-contrib/src/main/java/org/apache/giraph/io/hcatalog/HiveGiraphRunner.java
(original)
+++
giraph/trunk/giraph-formats-contrib/src/main/java/org/apache/giraph/io/hcatalog/HiveGiraphRunner.java
Tue Nov 20 05:43:29 2012
@@ -18,31 +18,33 @@
package org.apache.giraph.io.hcatalog;
-import java.io.File;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.List;
-import java.util.Map;
-
import org.apache.commons.cli.CommandLine;
import org.apache.commons.cli.CommandLineParser;
import org.apache.commons.cli.GnuParser;
import org.apache.commons.cli.HelpFormatter;
import org.apache.commons.cli.Options;
import org.apache.commons.cli.ParseException;
-import org.apache.giraph.graph.Vertex;
+import org.apache.giraph.graph.EdgeInputFormat;
import org.apache.giraph.graph.GiraphJob;
+import org.apache.giraph.graph.Vertex;
+import org.apache.giraph.graph.VertexInputFormat;
+import org.apache.giraph.graph.VertexOutputFormat;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hive.conf.HiveConf;
import org.apache.hadoop.util.Tool;
import org.apache.hadoop.util.ToolRunner;
-import org.apache.hcatalog.mapreduce.HCatInputFormat;
import org.apache.hcatalog.mapreduce.HCatOutputFormat;
import org.apache.hcatalog.mapreduce.InputJobInfo;
import org.apache.hcatalog.mapreduce.OutputJobInfo;
+import org.apache.log4j.Logger;
import com.google.common.collect.Lists;
-import org.apache.log4j.Logger;
+
+import java.io.File;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
/**
* Hive Giraph Runner
@@ -69,13 +71,21 @@ public class HiveGiraphRunner implements
*/
protected String dbName;
/**
- * input table name
+ * vertex input table name
+ */
+ protected String vertexInputTableName;
+ /**
+ * vertex input table filter
+ */
+ protected String vertexInputTableFilterExpr;
+ /**
+ * edge input table name
*/
- protected String inputTableName;
+ protected String edgeInputTableName;
/**
- * input table filter
+ * edge input table filter
*/
- protected String inputTableFilterExpr;
+ protected String edgeInputTableFilterExpr;
/**
* output table name
*/
@@ -88,34 +98,36 @@ public class HiveGiraphRunner implements
/**
* vertex class.
*/
- @SuppressWarnings("rawtypes")
private Class<? extends Vertex> vertexClass;
/**
- * vertex input format internal.
- */
- @SuppressWarnings("rawtypes")
- private Class<? extends HCatalogVertexInputFormat> vertexInputFormatClass;
+ * vertex input format internal.
+ */
+ private Class<? extends VertexInputFormat> vertexInputFormatClass;
+ /**
+ * edge input format internal.
+ */
+ private Class<? extends EdgeInputFormat> edgeInputFormatClass;
/**
* vertex output format internal.
*/
- @SuppressWarnings("rawtypes")
- private Class<? extends HCatalogVertexOutputFormat> vertexOutputFormatClass;
+ private Class<? extends VertexOutputFormat> vertexOutputFormatClass;
/**
- * giraph runner class.
- * @param vertexClass vertec class
- * @param vertexInputFormatClass input format
- * @param vertexOutputFormatClass output format
+ * Giraph runner class.
+ *
+ * @param vertexClass Vertex class
+ * @param vertexInputFormatClass Vertex input format
+ * @param edgeInputFormatClass Edge input format
+ * @param vertexOutputFormatClass Output format
*/
protected HiveGiraphRunner(
- @SuppressWarnings("rawtypes")
- Class<? extends Vertex> vertexClass,
- @SuppressWarnings("rawtypes")
- Class<? extends HCatalogVertexInputFormat> vertexInputFormatClass,
- @SuppressWarnings("rawtypes")
- Class<? extends HCatalogVertexOutputFormat> vertexOutputFormatClass)
{
+ Class<? extends Vertex> vertexClass,
+ Class<? extends VertexInputFormat> vertexInputFormatClass,
+ Class<? extends EdgeInputFormat> edgeInputFormatClass,
+ Class<? extends VertexOutputFormat> vertexOutputFormatClass) {
this.vertexClass = vertexClass;
this.vertexInputFormatClass = vertexInputFormatClass;
+ this.edgeInputFormatClass = edgeInputFormatClass;
this.vertexOutputFormatClass = vertexOutputFormatClass;
this.conf = new HiveConf(getClass());
}
@@ -126,13 +138,13 @@ public class HiveGiraphRunner implements
* @throws Exception any errors from Hive Giraph Runner
*/
public static void main(String[] args) throws Exception {
- System.exit(ToolRunner
- .run(new HiveGiraphRunner(null, null, null), args));
+ System.exit(ToolRunner.run(
+ new HiveGiraphRunner(null, null, null, null), args));
}
@Override
public final int run(String[] args) throws Exception {
- // process args
+ // process args
try {
processArguments(args);
} catch (InterruptedException e) {
@@ -150,16 +162,26 @@ public class HiveGiraphRunner implements
job.getConfiguration().setVertexClass(vertexClass);
// setup input from Hive
- InputJobInfo inputJobInfo = InputJobInfo.create(dbName, inputTableName,
- inputTableFilterExpr);
- HCatInputFormat.setInput(job.getInternalJob(), inputJobInfo);
- job.getConfiguration().setVertexInputFormatClass(vertexInputFormatClass);
+ if (vertexInputFormatClass != null) {
+ InputJobInfo vertexInputJobInfo = InputJobInfo.create(dbName,
+ vertexInputTableName, vertexInputTableFilterExpr);
+ GiraphHCatInputFormat.setVertexInput(job.getInternalJob(),
+ vertexInputJobInfo);
+ job.getConfiguration().setVertexInputFormatClass(vertexInputFormatClass);
+ }
+ if (edgeInputFormatClass != null) {
+ InputJobInfo edgeInputJobInfo = InputJobInfo.create(dbName,
+ edgeInputTableName, edgeInputTableFilterExpr);
+ GiraphHCatInputFormat.setEdgeInput(job.getInternalJob(),
+ edgeInputJobInfo);
+ job.getConfiguration().setEdgeInputFormatClass(edgeInputFormatClass);
+ }
// setup output to Hive
HCatOutputFormat.setOutput(job.getInternalJob(), OutputJobInfo.create(
- dbName, outputTableName, outputTablePartitionValues));
+ dbName, outputTableName, outputTablePartitionValues));
HCatOutputFormat.setSchema(job.getInternalJob(),
- HCatOutputFormat.getTableSchema(job.getInternalJob()));
+ HCatOutputFormat.getTableSchema(job.getInternalJob()));
if (skipOutput) {
LOG.warn("run: Warning - Output will be skipped!");
} else {
@@ -178,20 +200,20 @@ public class HiveGiraphRunner implements
* @param conf Configuration argument
*/
private static void adjustConfigurationForHive(Configuration conf) {
- // when output partitions are used, workers register them to the
- // metastore at cleanup stage, and on HiveConf's initialization, it
- // looks for hive-site.xml from.
+ // when output partitions are used, workers register them to the
+ // metastore at cleanup stage, and on HiveConf's initialization, it
+ // looks for hive-site.xml from.
addToStringCollection(conf, "tmpfiles", conf.getClassLoader()
- .getResource("hive-site.xml").toString());
+ .getResource("hive-site.xml").toString());
- // Also, you need hive.aux.jars as well
- // addToStringCollection(conf, "tmpjars",
- // conf.getStringCollection("hive.aux.jars.path"));
+ // Also, you need hive.aux.jars as well
+ // addToStringCollection(conf, "tmpjars",
+ // conf.getStringCollection("hive.aux.jars.path"));
- // Or, more effectively, we can provide all the jars client needed to
- // the workers as well
+ // Or, more effectively, we can provide all the jars client needed to
+ // the workers as well
String[] hadoopJars = System.getenv("HADOOP_CLASSPATH").split(
- File.pathSeparator);
+ File.pathSeparator);
List<String> hadoopJarURLs = Lists.newArrayList();
for (String jarPath : hadoopJars) {
File file = new File(jarPath);
@@ -220,25 +242,34 @@ public class HiveGiraphRunner implements
options.addOption("w", "workers", true, "Number of workers");
if (vertexClass == null) {
options.addOption(null, "vertexClass", true,
- "Giraph Vertex class to use");
+ "Giraph Vertex class to use");
}
if (vertexInputFormatClass == null) {
options.addOption(null, "vertexInputFormatClass", true,
- "Giraph HCatalogVertexInputFormat class to use");
+ "Giraph HCatalogVertexInputFormat class to use");
+ }
+ if (edgeInputFormatClass == null) {
+ options.addOption(null, "edgeInputFormatClass", true,
+ "Giraph HCatalogEdgeInputFormat class to use");
}
if (vertexOutputFormatClass == null) {
options.addOption(null, "vertexOutputFormatClass", true,
- "Giraph HCatalogVertexOutputFormat class to use");
+ "Giraph HCatalogVertexOutputFormat class to use");
}
options.addOption("db", "database", true, "Hive database name");
- options.addOption("i", "inputTable", true, "Input table name");
- options.addOption("I", "inputFilter", true,
- "Input table filter expression (e.g., \"a<2 AND b='two'\"");
+ options.addOption("vi", "vertexInputTable", true,
+ "Vertex input table name");
+ options.addOption("VI", "vertexInputFilter", true,
+ "Vertex input table filter expression (e.g., \"a<2 AND b='two'\"");
+ options.addOption("ei", "edgeInputTable", true,
+ "Edge input table name");
+ options.addOption("EI", "edgeInputFilter", true,
+ "Edge input table filter expression (e.g., \"a<2 AND b='two'\"");
options.addOption("o", "outputTable", true, "Output table name");
options.addOption("O", "outputPartition", true,
- "Output table partition values (e.g., \"a=1,b=two\")");
+ "Output table partition values (e.g., \"a=1,b=two\")");
options.addOption("s", "skipOutput", false, "Skip output?");
addMoreOptions(options);
@@ -253,17 +284,23 @@ public class HiveGiraphRunner implements
// Giraph classes
if (cmdln.hasOption("vertexClass")) {
vertexClass = findClass(cmdln.getOptionValue("vertexClass"),
- Vertex.class);
+ Vertex.class);
}
if (cmdln.hasOption("vertexInputFormatClass")) {
vertexInputFormatClass = findClass(
- cmdln.getOptionValue("vertexInputFormatClass"),
- HCatalogVertexInputFormat.class);
+ cmdln.getOptionValue("vertexInputFormatClass"),
+ HCatalogVertexInputFormat.class);
}
+ if (cmdln.hasOption("edgeInputFormatClass")) {
+ edgeInputFormatClass = findClass(
+ cmdln.getOptionValue("edgeInputFormatClass"),
+ HCatalogEdgeInputFormat.class);
+ }
+
if (cmdln.hasOption("vertexOutputFormatClass")) {
vertexOutputFormatClass = findClass(
- cmdln.getOptionValue("vertexOutputFormatClass"),
- HCatalogVertexOutputFormat.class);
+ cmdln.getOptionValue("vertexOutputFormatClass"),
+ HCatalogVertexOutputFormat.class);
}
if (cmdln.hasOption("skipOutput")) {
@@ -272,34 +309,42 @@ public class HiveGiraphRunner implements
if (vertexClass == null) {
throw new IllegalArgumentException(
- "Need the Giraph Vertex class name (-vertexClass) to use");
+ "Need the Giraph Vertex class name (-vertexClass) to use");
}
- if (vertexInputFormatClass == null) {
+ if (vertexInputFormatClass == null && edgeInputFormatClass == null) {
throw new IllegalArgumentException(
- "Need the Giraph VertexInputFormat " +
- "class name (-vertexInputFormatClass) to use");
+ "Need at least one of Giraph VertexInputFormat " +
+ "class name (-vertexInputFormatClass) and " +
+ "EdgeInputFormat class name (-edgeInputFormatClass)");
}
if (vertexOutputFormatClass == null) {
throw new IllegalArgumentException(
- "Need the Giraph VertexOutputFormat " +
- "class name (-vertexOutputFormatClass) to use");
+ "Need the Giraph VertexOutputFormat " +
+ "class name (-vertexOutputFormatClass) to use");
}
if (!cmdln.hasOption("workers")) {
throw new IllegalArgumentException(
- "Need to choose the number of workers (-w)");
+ "Need to choose the number of workers (-w)");
+ }
+ if (!cmdln.hasOption("vertexInputTable") &&
+ vertexInputFormatClass != null) {
+ throw new IllegalArgumentException(
+ "Need to set the vertex input table name (-vi)");
}
- if (!cmdln.hasOption("inputTable")) {
+ if (!cmdln.hasOption("edgeInputTable") &&
+ edgeInputFormatClass != null) {
throw new IllegalArgumentException(
- "Need to set the input table name (-i). " +
- "One example is 'dim_friendlist'");
+ "Need to set the edge input table name (-ei)");
}
if (!cmdln.hasOption("outputTable")) {
throw new IllegalArgumentException(
- "Need to set the output table name (-o).");
+ "Need to set the output table name (-o)");
}
dbName = cmdln.getOptionValue("dbName", "default");
- inputTableName = cmdln.getOptionValue("inputTable");
- inputTableFilterExpr = cmdln.getOptionValue("inputFilter");
+ vertexInputTableName = cmdln.getOptionValue("vertexInputTable");
+ vertexInputTableFilterExpr = cmdln.getOptionValue("vertexInputFilter");
+ edgeInputTableName = cmdln.getOptionValue("edgeInputTable");
+ edgeInputTableFilterExpr = cmdln.getOptionValue("edgeInputFilter");
outputTableName = cmdln.getOptionValue("outputTable");
outputTablePartitionValues = HiveUtils.parsePartitionValues(cmdln
.getOptionValue("outputPartition"));
@@ -366,8 +411,7 @@ public class HiveGiraphRunner implements
}
return null;
} catch (ClassNotFoundException e) {
- throw new IllegalArgumentException(className +
- ": Invalid class name");
+ throw new IllegalArgumentException(className + ": Invalid class name");
}
}
@@ -412,21 +456,35 @@ public class HiveGiraphRunner implements
String prefix = "\t";
LOG.info(prefix + "-vertexClass=" +
vertexClass.getCanonicalName());
- LOG.info(prefix + "-vertexInputFormatClass=" +
- vertexInputFormatClass.getCanonicalName());
+ if (vertexInputFormatClass != null) {
+ LOG.info(prefix + "-vertexInputFormatClass=" +
+ vertexInputFormatClass.getCanonicalName());
+ }
+ if (edgeInputFormatClass != null) {
+ LOG.info(prefix + "-edgeInputFormatClass=" +
+ edgeInputFormatClass.getCanonicalName());
+ }
LOG.info(prefix + "-vertexOutputFormatClass=" +
vertexOutputFormatClass.getCanonicalName());
- LOG.info(prefix + "-inputTable=" + inputTableName);
- if (inputTableFilterExpr != null) {
- LOG.info(prefix + "-inputFilter=\"" +
- inputTableFilterExpr + "\"");
+ if (vertexInputTableName != null) {
+ LOG.info(prefix + "-vertexInputTable=" + vertexInputTableName);
+ }
+ if (vertexInputTableFilterExpr != null) {
+ LOG.info(prefix + "-vertexInputFilter=\"" +
+ vertexInputTableFilterExpr + "\"");
+ }
+ if (edgeInputTableName != null) {
+ LOG.info(prefix + "-edgeInputTable=" + edgeInputTableName);
+ }
+ if (edgeInputTableFilterExpr != null) {
+ LOG.info(prefix + "-edgeInputFilter=\"" +
+ edgeInputTableFilterExpr + "\"");
}
LOG.info(prefix + "-outputTable=" + outputTableName);
if (outputTablePartitionValues != null) {
LOG.info(prefix + "-outputPartition=\"" +
- outputTablePartitionValues + "\"");
+ outputTablePartitionValues + "\"");
}
LOG.info(prefix + "-workers=" + workers);
}
-
}
Modified: giraph/trunk/pom.xml
URL:
http://svn.apache.org/viewvc/giraph/trunk/pom.xml?rev=1411549&r1=1411548&r2=1411549&view=diff
==============================================================================
--- giraph/trunk/pom.xml (original)
+++ giraph/trunk/pom.xml Tue Nov 20 05:43:29 2012
@@ -225,6 +225,7 @@ under the License.
<hbase.version>0.90.5</hbase.version>
<jackson.version>1.8.0</jackson.version>
<slf4j.version>1.7.2</slf4j.version>
+ <hive.version>0.9.0</hive.version>
<forHadoop>for-hadoop-${hadoop.version}</forHadoop>
</properties>
@@ -746,7 +747,17 @@ under the License.
<dependency>
<groupId>org.apache.hive</groupId>
<artifactId>hive-common</artifactId>
- <version>0.9.0</version>
+ <version>${hive.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hive</groupId>
+ <artifactId>hive-exec</artifactId>
+ <version>${hive.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hive</groupId>
+ <artifactId>hive-metastore</artifactId>
+ <version>${hive.version}</version>
</dependency>
<dependency>
<groupId>org.apache.mahout</groupId>