diff --git a/examples/pom.xml b/examples/pom.xml index c42d2bcdb9ed55bcae303fa8844a8827e8a07ba1..0fbb5a3d5d1a1eece1d08af30bdd7624ba9e2916 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 0000000000000000000000000000000000000000..6e910154d4241940f69127e08631eaac856994b4 --- /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 diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 0ea23b446fd31baad7011873b09c183feba169bb..bf090d7fccd00ff918b39822299c3a9029d44c7f 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -201,7 +201,11 @@ object SparkBuild extends Build { def examplesSettings = sharedSettings ++ Seq( name := "spark-examples", - libraryDependencies ++= Seq("com.twitter" % "algebird-core_2.9.2" % "0.1.11") + resolvers ++= Seq("Apache HBase" at "https://repository.apache.org/content/repositories/releases"), + libraryDependencies ++= Seq( + "com.twitter" % "algebird-core_2.9.2" % "0.1.11", + "org.apache.hbase" % "hbase" % "0.94.6" excludeAll(excludeNetty) + ) ) def bagelSettings = sharedSettings ++ Seq(name := "spark-bagel")