aboutsummaryrefslogtreecommitdiff
path: root/examples
diff options
context:
space:
mode:
authorEthan Jewett <esjewett@gmail.com>2013-05-09 18:33:38 -0500
committerEthan Jewett <esjewett@gmail.com>2013-05-09 18:33:38 -0500
commitee6f6aa6cd028e6a3938dcd5334661c27f493bc6 (patch)
tree19f55db17edf6608fda37c5a435aa58003b75284 /examples
parent012c9e5ab072239e07202abe4775b434be6e32b9 (diff)
downloadspark-ee6f6aa6cd028e6a3938dcd5334661c27f493bc6.tar.gz
spark-ee6f6aa6cd028e6a3938dcd5334661c27f493bc6.tar.bz2
spark-ee6f6aa6cd028e6a3938dcd5334661c27f493bc6.zip
Add hBase example
Diffstat (limited to 'examples')
-rw-r--r--examples/src/main/scala/spark/examples/HBaseTest.scala35
1 files changed, 35 insertions, 0 deletions
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