aboutsummaryrefslogtreecommitdiff
path: root/examples
diff options
context:
space:
mode:
authorMatei Zaharia <matei@eecs.berkeley.edu>2013-06-18 04:21:17 -0700
committerMatei Zaharia <matei@eecs.berkeley.edu>2013-06-18 04:21:17 -0700
commit73f4c7d2d188e61b361ce772d9525767b7c48680 (patch)
tree1e5c06981687b1c114e4baf809aa468e80f83b7a /examples
parent9933836cc7ebcac9f7beda7acc4a152bce2269af (diff)
parent3217d486f7fdd590250f2efee567e4779e130d34 (diff)
downloadspark-73f4c7d2d188e61b361ce772d9525767b7c48680.tar.gz
spark-73f4c7d2d188e61b361ce772d9525767b7c48680.tar.bz2
spark-73f4c7d2d188e61b361ce772d9525767b7c48680.zip
Merge pull request #605 from esjewett/SPARK-699
Add hBase example (retry of pull request #596)
Diffstat (limited to 'examples')
-rw-r--r--examples/pom.xml10
-rw-r--r--examples/src/main/scala/spark/examples/HBaseTest.scala35
2 files changed, 45 insertions, 0 deletions
diff --git a/examples/pom.xml b/examples/pom.xml
index c42d2bcdb9..0fbb5a3d5d 100644
--- a/examples/pom.xml
+++ b/examples/pom.xml
@@ -67,6 +67,11 @@
<artifactId>hadoop-core</artifactId>
<scope>provided</scope>
</dependency>
+ <dependency>
+ <groupId>org.apache.hbase</groupId>
+ <artifactId>hbase</artifactId>
+ <version>0.94.6</version>
+ </dependency>
</dependencies>
<build>
<plugins>
@@ -105,6 +110,11 @@
<artifactId>hadoop-client</artifactId>
<scope>provided</scope>
</dependency>
+ <dependency>
+ <groupId>org.apache.hbase</groupId>
+ <artifactId>hbase</artifactId>
+ <version>0.94.6</version>
+ </dependency>
</dependencies>
<build>
<plugins>
diff --git a/examples/src/main/scala/spark/examples/HBaseTest.scala b/examples/src/main/scala/spark/examples/HBaseTest.scala
new file mode 100644
index 0000000000..6e910154d4
--- /dev/null
+++ b/examples/src/main/scala/spark/examples/HBaseTest.scala
@@ -0,0 +1,35 @@
+package spark.examples
+
+import spark._
+import spark.rdd.NewHadoopRDD
+import org.apache.hadoop.hbase.{HBaseConfiguration, HTableDescriptor}
+import org.apache.hadoop.hbase.client.HBaseAdmin
+import org.apache.hadoop.hbase.mapreduce.TableInputFormat
+
+object HBaseTest {
+ def main(args: Array[String]) {
+ val sc = new SparkContext(args(0), "HBaseTest",
+ System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
+
+ val conf = HBaseConfiguration.create()
+
+ // Other options for configuring scan behavior are available. More information available at
+ // http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/mapreduce/TableInputFormat.html
+ conf.set(TableInputFormat.INPUT_TABLE, args(1))
+
+ // Initialize hBase table if necessary
+ val admin = new HBaseAdmin(conf)
+ if(!admin.isTableAvailable(args(1))) {
+ val tableDesc = new HTableDescriptor(args(1))
+ admin.createTable(tableDesc)
+ }
+
+ val hBaseRDD = sc.newAPIHadoopRDD(conf, classOf[TableInputFormat],
+ classOf[org.apache.hadoop.hbase.io.ImmutableBytesWritable],
+ classOf[org.apache.hadoop.hbase.client.Result])
+
+ hBaseRDD.count()
+
+ System.exit(0)
+ }
+} \ No newline at end of file