Skip to content

SPARK-1127 Add saveAsHBase to PairRDDFunctions #123

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Closed
wants to merge 2 commits into from
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
15 changes: 15 additions & 0 deletions core/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -36,6 +36,21 @@
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-client</artifactId>
</dependency>
<dependency>
<groupId>org.apache.hbase</groupId>
<artifactId>hbase</artifactId>
<type>test-jar</type>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.hbase</groupId>
<artifactId>hbase</artifactId>
</dependency>
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-test</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>net.java.dev.jets3t</groupId>
<artifactId>jets3t</artifactId>
Expand Down
106 changes: 106 additions & 0 deletions core/src/main/scala/org/apache/spark/SparkHBaseWriter.scala
Original file line number Diff line number Diff line change
@@ -0,0 +1,106 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.spark

import org.apache.hadoop.hbase.client.{Put, HTable}
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.hbase.util.Bytes
import org.apache.commons.codec.binary.Hex
import org.apache.hadoop.io.Text
import org.apache.hadoop.hbase.HConstants

/**
* Internal helper class that saves an RDD using a HBase OutputFormat. This is only public
* because we need to access this class from the `spark` package to use some package-private HBase
* functions, but this class should not be used directly by users.
*
* Saves the RDD using a JobConf, which should contain an output key class, an output value class,
* a filename to write to, etc, exactly like in a HBase MapReduce job.
*/
private[apache]
class SparkHBaseWriter(conf: HBaseConf)
extends Logging {

private var htable: HTable = null

val zkHost = conf.zkHost
val zkPort = conf.zkPort
val zkNode = conf.zkNode
val table = conf.table
val rowkeyType = conf.rowkeyType
val columns = conf.columns
val delimiter = conf.delimiter

def setup() {
val conf = new Configuration()
conf.set(HConstants.ZOOKEEPER_QUORUM, zkHost)
conf.set(HConstants.ZOOKEEPER_CLIENT_PORT, zkPort)
conf.set(HConstants.ZOOKEEPER_ZNODE_PARENT, zkNode)
htable = new HTable(conf, table)
}

def toByteArr(field: String, kind: String) = kind match {
case HBaseType.Boolean => Bytes.toBytes(field.toBoolean)
case HBaseType.Int => Bytes.toBytes(field.toInt)
case HBaseType.Long => Bytes.toBytes(field.toLong)
case HBaseType.Float => Bytes.toBytes(field.toFloat)
case HBaseType.Double => Bytes.toBytes(field.toDouble)
case HBaseType.String => Bytes.toBytes(field)
case HBaseType.Bytes => Hex.decodeHex(field.toCharArray)
}

def parseRecord(record: String) = {
val fields = record.split(delimiter)
val put = new Put(toByteArr(fields(0), rowkeyType))

List.range(1, fields.size) foreach {
i => put.add(columns(i - 1).family, columns(i - 1).qualifier, toByteArr(fields(i), columns(i - 1).typ))
}

put
}

def write(record: Text) {
val put = parseRecord(record.toString)
htable.put(put)
}

def close() {
htable.close()
}
}

private[apache]
object HBaseType {
val Boolean = "bool"
val Int = "int"
val Long = "long"
val Float = "float"
val Double = "double"
val String = "string"
val Bytes = "bytes"
}

private[apache]
class HBaseColumn(val family: Array[Byte], val qualifier: Array[Byte], val typ: String)
extends Serializable

private[apache]
class HBaseConf(val zkHost: String, val zkPort: String, val zkNode: String,
val table: String, val rowkeyType: String, val columns: List[HBaseColumn], val delimiter: Char)
extends Serializable
18 changes: 18 additions & 0 deletions core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala
Original file line number Diff line number Diff line change
Expand Up @@ -36,6 +36,7 @@ import org.apache.hadoop.io.compress.CompressionCodec
import org.apache.hadoop.mapred.{FileOutputCommitter, FileOutputFormat, JobConf, OutputFormat}
import org.apache.hadoop.mapreduce.{OutputFormat => NewOutputFormat, Job => NewAPIHadoopJob, RecordWriter => NewRecordWriter, JobContext, SparkHadoopMapReduceUtil}
import org.apache.hadoop.mapreduce.lib.output.{FileOutputFormat => NewFileOutputFormat}
import org.apache.hadoop.io.Text

// SparkHadoopWriter and SparkHadoopMapReduceUtil are actually source files defined in Spark.
import org.apache.hadoop.mapred.SparkHadoopWriter
Expand Down Expand Up @@ -746,6 +747,23 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)])
writer.commitJob()
}

def saveAsHBaseTable(zkHost: String, zkPort: String, zkNode: String,
table: String, rowkeyType: String, columns: List[HBaseColumn], delimiter: Char) {
val conf = new HBaseConf(zkHost, zkPort, zkNode, table, rowkeyType, columns, delimiter)

def writeToHBase(context: TaskContext, iter: Iterator[(K, V)]) {
val writer = new SparkHBaseWriter(conf)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Where is writer closed ?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Sorry, I forgot to add exception handle and close table later. As discussion in SPARK-1127, I would try to provide a higher-level abstraction for this.

writer.setup()

while(iter.hasNext) {
val record = iter.next()
writer.write(record._2.asInstanceOf[Text])
}
}

self.context.runJob(self, writeToHBase _)
}

/**
* Return an RDD with the keys of each tuple.
*/
Expand Down
23 changes: 23 additions & 0 deletions core/src/test/scala/org/apache/spark/FileSuite.scala
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,8 @@ import org.apache.hadoop.mapred.FileAlreadyExistsException
import org.scalatest.FunSuite

import org.apache.spark.SparkContext._
import org.apache.hadoop.hbase.util.Bytes
import org.apache.hadoop.hbase.{HConstants, HBaseTestingUtility}
import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat

class FileSuite extends FunSuite with LocalSparkContext {
Expand Down Expand Up @@ -197,6 +199,27 @@ class FileSuite extends FunSuite with LocalSparkContext {
assert(output.map(_.toString).collect().toList === List("(1,a)", "(2,aa)", "(3,aaa)"))
}

test("write SequenceFile using HBase") {
sc = new SparkContext("local", "test")
val nums = sc.makeRDD(1 to 3).map(x => (x, new Text("a" + x + " 1.0")))

val table = "test"
val rowkeyType = HBaseType.String
val cfBytes = Bytes.toBytes("cf")
val columns = List[HBaseColumn](new HBaseColumn(cfBytes, Bytes.toBytes("qual0"), HBaseType.Float))
val delimiter = ' '

val util = new HBaseTestingUtility()
util.startMiniCluster()
util.createTable(Bytes.toBytes(table), cfBytes)
val conf = util.getConfiguration
val zkHost = conf.get(HConstants.ZOOKEEPER_QUORUM)
val zkPort = conf.get(HConstants.ZOOKEEPER_CLIENT_PORT)
val zkNode = conf.get(HConstants.ZOOKEEPER_ZNODE_PARENT)

nums.saveAsHBaseTable(zkHost, zkPort, zkNode, table, rowkeyType, columns, delimiter)
}

test("file caching") {
sc = new SparkContext("local", "test")
val tempDir = Files.createTempDir()
Expand Down
18 changes: 18 additions & 0 deletions pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -464,6 +464,24 @@
</exclusion>
</exclusions>
</dependency>
<dependency>
<groupId>org.apache.hbase</groupId>
<artifactId>hbase</artifactId>
<version>${hbase.version}</version>
<type>test-jar</type>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.hbase</groupId>
<artifactId>hbase</artifactId>
<version>${hbase.version}</version>
</dependency>
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-test</artifactId>
<version>${hadoop.version}</version>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.avro</groupId>
<artifactId>avro</artifactId>
Expand Down
5 changes: 5 additions & 0 deletions project/SparkBuild.scala
Original file line number Diff line number Diff line change
Expand Up @@ -91,6 +91,8 @@ object SparkBuild extends Build {
lazy val hadoopClient = if (hadoopVersion.startsWith("0.20.") || hadoopVersion == "1.0.0") "hadoop-core" else "hadoop-client"
val maybeAvro = if (hadoopVersion.startsWith("0.23.") && isYarnEnabled) Seq("org.apache.avro" % "avro" % "1.7.4") else Seq()

lazy val hbaseVersion = Properties.envOrElse("SPARK_HBASE_VERSION", HBASE_VERSION)

// Conditionally include the java 8 sub-project
lazy val javaVersion = System.getProperty("java.specification.version")
lazy val isJava8Enabled = javaVersion.toDouble >= "1.8".toDouble
Expand Down Expand Up @@ -292,6 +294,9 @@ object SparkBuild extends Build {
"net.java.dev.jets3t" % "jets3t" % "0.7.1" excludeAll(excludeCommonsLogging),
"org.apache.derby" % "derby" % "10.4.2.0" % "test",
"org.apache.hadoop" % hadoopClient % hadoopVersion excludeAll(excludeNetty, excludeAsm, excludeCommonsLogging, excludeSLF4J, excludeOldAsm),
"org.apache.hadoop" % "hadoop-test" % hadoopVersion % "test",
"org.apache.hbase" % "hbase" % hbaseVersion,
"org.apache.hbase" % "hbase" % hbaseVersion % "test" classifier "tests",
"org.apache.curator" % "curator-recipes" % "2.4.0" excludeAll(excludeNetty),
"com.codahale.metrics" % "metrics-core" % "3.0.0",
"com.codahale.metrics" % "metrics-jvm" % "3.0.0",
Expand Down