-
Notifications
You must be signed in to change notification settings - Fork 28.9k
SPARK-1127 Add spark-hbase. #194
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
Closed
Changes from all commits
Commits
Show all changes
11 commits
Select commit
Hold shift + click to select a range
fd8d121
Add spark-hbase and class-level documents.
haosdent b9a894c
Fix visibility of SparkHBaseWriter.
haosdent c8d4e9e
Fix document error in log4j.properties
haosdent 4e9fff2
Handle htable.close() exception scenrio.
haosdent 1a020d1
Fix syntax errors in scala document.
haosdent 4b4c5a7
Close htable and shutdown mini cluster in test case.
haosdent 448f667
Move try/catch block of close() out from SparkHBaseWriter
haosdent 67a48a8
Add methods to save SchemaRDD as HBase table.
haosdent 177958a
Add deps in sbt.
haosdent a09a6c2
Fix scalastyle errors.
haosdent 30f7343
Remove unecessary imports.
haosdent File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,104 @@ | ||
| <?xml version="1.0" encoding="UTF-8"?> | ||
| <!-- | ||
| ~ 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. | ||
| --> | ||
|
|
||
| <project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" | ||
| xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> | ||
| <modelVersion>4.0.0</modelVersion> | ||
| <parent> | ||
| <groupId>org.apache.spark</groupId> | ||
| <artifactId>spark-parent</artifactId> | ||
| <version>1.0.0-SNAPSHOT</version> | ||
| <relativePath>../../pom.xml</relativePath> | ||
| </parent> | ||
|
|
||
| <groupId>org.apache.spark</groupId> | ||
| <artifactId>spark-nosql-hbase</artifactId> | ||
| <packaging>jar</packaging> | ||
| <name>Spark Project External HBase</name> | ||
| <url>http://spark.apache.org/</url> | ||
|
|
||
| <dependencies> | ||
| <dependency> | ||
| <groupId>org.apache.spark</groupId> | ||
| <artifactId>spark-core_${scala.binary.version}</artifactId> | ||
| <version>${project.version}</version> | ||
| </dependency> | ||
| <dependency> | ||
| <groupId>org.apache.spark</groupId> | ||
| <artifactId>spark-core_${scala.binary.version}</artifactId> | ||
| <version>${project.version}</version> | ||
| <type>test-jar</type> | ||
| <scope>test</scope> | ||
| </dependency> | ||
| <dependency> | ||
| <groupId>org.apache.spark</groupId> | ||
| <artifactId>spark-sql_${scala.binary.version}</artifactId> | ||
| <version>${project.version}</version> | ||
| </dependency> | ||
| <dependency> | ||
| <groupId>org.apache.spark</groupId> | ||
| <artifactId>spark-sql_${scala.binary.version}</artifactId> | ||
| <version>${project.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.hbase</groupId> | ||
| <artifactId>hbase</artifactId> | ||
| <version>${hbase.version}</version> | ||
| <type>test-jar</type> | ||
| <scope>test</scope> | ||
| </dependency> | ||
| <dependency> | ||
| <groupId>org.apache.hadoop</groupId> | ||
| <artifactId>hadoop-client</artifactId> | ||
| <version>${hadoop.version}</version> | ||
| </dependency> | ||
| <dependency> | ||
| <groupId>org.apache.hadoop</groupId> | ||
| <artifactId>hadoop-test</artifactId> | ||
| <version>${hadoop.version}</version> | ||
| <scope>test</scope> | ||
| </dependency> | ||
| <dependency> | ||
| <groupId>org.scalatest</groupId> | ||
| <artifactId>scalatest_${scala.binary.version}</artifactId> | ||
| <scope>test</scope> | ||
| </dependency> | ||
| <dependency> | ||
| <groupId>org.scalacheck</groupId> | ||
| <artifactId>scalacheck_${scala.binary.version}</artifactId> | ||
| <scope>test</scope> | ||
| </dependency> | ||
| </dependencies> | ||
| <build> | ||
| <outputDirectory>target/scala-${scala.binary.version}/classes</outputDirectory> | ||
| <testOutputDirectory>target/scala-${scala.binary.version}/test-classes</testOutputDirectory> | ||
| <plugins> | ||
| <plugin> | ||
| <groupId>org.scalatest</groupId> | ||
| <artifactId>scalatest-maven-plugin</artifactId> | ||
| </plugin> | ||
| </plugins> | ||
| </build> | ||
| </project> | ||
132 changes: 132 additions & 0 deletions
132
external/hbase/src/main/scala/org/apache/spark/nosql/hbase/HBaseUtils.scala
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,132 @@ | ||
| /* | ||
| * 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.nosql.hbase | ||
|
|
||
| import org.apache.hadoop.io.Text | ||
| import org.apache.spark.rdd.RDD | ||
| import org.apache.spark.Logging | ||
| import org.apache.spark.sql.{Row, SchemaRDD} | ||
| import org.apache.spark.sql.catalyst.types.DataType | ||
| import org.apache.hadoop.hbase.util.Bytes | ||
|
|
||
| /** | ||
| * A public object that provides HBase support. | ||
| * You could save RDD into HBase through | ||
| * [[org.apache.spark.nosql.hbase.HBaseUtils.saveAsHBaseTable]] method. | ||
| */ | ||
| object HBaseUtils | ||
| extends Logging { | ||
|
|
||
| /** | ||
| * Save [[org.apache.spark.rdd.RDD[Text]]] as a HBase table | ||
| * | ||
| * The format of records in RDD should look like this: | ||
| * rowkey|delimiter|column|delimiter|column|delimiter|... | ||
| * For example (if delimiter is ","): | ||
| * 0001,apple,banana | ||
| * "0001" is rowkey field while "apple" and "banana" are column fields. | ||
| * | ||
| * @param rdd [[org.apache.spark.rdd.RDD[Text]]] | ||
| * @param zkHost the zookeeper hosts. e.g. "10.232.98.10,10.232.98.11,10.232.98.12" | ||
| * @param zkPort the zookeeper client listening port. e.g. "2181" | ||
| * @param zkNode the zookeeper znode of HBase. e.g. "hbase-apache" | ||
| * @param table the name of table which we save records | ||
| * @param rowkeyType the type of rowkey. [[org.apache.spark.sql.catalyst.types.DataType]] | ||
| * @param columns the column list. [[org.apache.spark.nosql.hbase.HBaseColumn]] | ||
| * @param delimiter the delimiter which used to split record into fields | ||
| */ | ||
| def saveAsHBaseTable(rdd: RDD[Text], | ||
| zkHost: String, zkPort: String, zkNode: String, table: String, | ||
| rowkeyType: DataType, columns: List[HBaseColumn], delimiter: Char) { | ||
| val conf = new HBaseConf(zkHost, zkPort, zkNode, table, rowkeyType, columns, delimiter) | ||
|
|
||
| def writeToHBase(iter: Iterator[Text]) { | ||
| val writer = new SparkHBaseWriter(conf) | ||
|
|
||
| try { | ||
| writer.init() | ||
|
|
||
| while (iter.hasNext) { | ||
| val record = iter.next() | ||
| writer.write(record) | ||
| } | ||
| } finally { | ||
| try { | ||
| writer.close() | ||
| } catch { | ||
| case ex: Exception => logWarning("Close HBase table failed.", ex) | ||
| } | ||
| } | ||
| } | ||
|
|
||
| rdd.foreachPartition(writeToHBase) | ||
| } | ||
|
|
||
| /** | ||
| * Save [[org.apache.spark.sql.SchemaRDD]] as a HBase table | ||
| * | ||
| * The first field of Row would be save as rowkey in HBase. | ||
| * All fields of Row use the @param family as the column family. | ||
| * | ||
| * @param rdd [[org.apache.spark.sql.SchemaRDD]] | ||
| * @param zkHost the zookeeper hosts. e.g. "10.232.98.10,10.232.98.11,10.232.98.12" | ||
| * @param zkPort the zookeeper client listening port. e.g. "2181" | ||
| * @param zkNode the zookeeper znode of HBase. e.g. "hbase-apache" | ||
| * @param table the name of table which we save records | ||
| * @param family the fixed column family which we save records | ||
| */ | ||
| def saveAsHBaseTable(rdd: SchemaRDD, | ||
| zkHost: String, zkPort: String, zkNode: String, | ||
| table: String, family: Array[Byte]) { | ||
| // Convert attributes informations in SchemaRDD to List[HBaseColumn] | ||
| val attributes = rdd.logicalPlan.output | ||
| var i = 0 | ||
| // Assume first field in Row is rowkey | ||
| val rowkeyType = attributes(i).dataType | ||
|
|
||
| var columns = List.empty[HBaseColumn] | ||
| for (i <- 1 to attributes.length - 1) { | ||
| val attribute = attributes(i) | ||
| val qualifier = Bytes.toBytes(attribute.name) | ||
| columns = columns :+ new HBaseColumn(family, qualifier, attribute.dataType) | ||
| } | ||
|
|
||
| val conf = new HBaseConf(zkHost, zkPort, zkNode, table, rowkeyType, columns, ',') | ||
|
|
||
| def writeToHBase(iter: Iterator[Row]) { | ||
| val writer = new SparkHBaseWriter(conf) | ||
|
|
||
| try { | ||
| writer.init() | ||
|
|
||
| while (iter.hasNext) { | ||
| val record = iter.next() | ||
| writer.write(record) | ||
| } | ||
| } finally { | ||
| try { | ||
| writer.close() | ||
| } catch { | ||
| case ex: Exception => logWarning("Close HBase table failed.", ex) | ||
| } | ||
| } | ||
| } | ||
|
|
||
| rdd.foreachPartition(writeToHBase) | ||
| } | ||
| } |
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Don't these come automatically from spark-core? Is there a reason you need to declare them here?