Skip to content

Commit 42cdae1

Browse files
GenTangjeanlyn
authored andcommitted
[SPARK-5090] [EXAMPLES] The improvement of python converter for hbase
Hi, Following the discussion in http://apache-spark-developers-list.1001551.n3.nabble.com/python-converter-in-HBaseConverter-scala-spark-examples-td10001.html. I made some modification in three files in package examples: 1. HBaseConverters.scala: the new converter will converts all the records in an hbase results into a single string 2. hbase_input.py: as the value string may contain several records, we can use ast package to convert the string into dict 3. HBaseTest.scala: as the package examples use hbase 0.98.7 the original constructor HTableDescriptor is deprecated. The updation to new constructor is made Author: GenTang <[email protected]> Closes apache#3920 from GenTang/master and squashes the following commits: d2153df [GenTang] import JSONObject precisely 4802481 [GenTang] dump the result into a singl String 62df7f0 [GenTang] remove the comment 21de653 [GenTang] return the string in json format 15b1fe3 [GenTang] the modification of comments 5cbbcfc [GenTang] the improvement of pythonconverter ceb31c5 [GenTang] the modification for adapting updation of hbase 3253b61 [GenTang] the modification accompanying the improvement of pythonconverter
1 parent 8b3b1de commit 42cdae1

File tree

2 files changed

+30
-11
lines changed

2 files changed

+30
-11
lines changed

examples/src/main/python/hbase_inputformat.py

Lines changed: 13 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -18,6 +18,7 @@
1818
from __future__ import print_function
1919

2020
import sys
21+
import json
2122

2223
from pyspark import SparkContext
2324

@@ -27,24 +28,24 @@
2728
hbase(main):016:0> create 'test', 'f1'
2829
0 row(s) in 1.0430 seconds
2930
30-
hbase(main):017:0> put 'test', 'row1', 'f1', 'value1'
31+
hbase(main):017:0> put 'test', 'row1', 'f1:a', 'value1'
3132
0 row(s) in 0.0130 seconds
3233
33-
hbase(main):018:0> put 'test', 'row2', 'f1', 'value2'
34+
hbase(main):018:0> put 'test', 'row1', 'f1:b', 'value2'
3435
0 row(s) in 0.0030 seconds
3536
36-
hbase(main):019:0> put 'test', 'row3', 'f1', 'value3'
37+
hbase(main):019:0> put 'test', 'row2', 'f1', 'value3'
3738
0 row(s) in 0.0050 seconds
3839
39-
hbase(main):020:0> put 'test', 'row4', 'f1', 'value4'
40+
hbase(main):020:0> put 'test', 'row3', 'f1', 'value4'
4041
0 row(s) in 0.0110 seconds
4142
4243
hbase(main):021:0> scan 'test'
4344
ROW COLUMN+CELL
44-
row1 column=f1:, timestamp=1401883411986, value=value1
45-
row2 column=f1:, timestamp=1401883415212, value=value2
46-
row3 column=f1:, timestamp=1401883417858, value=value3
47-
row4 column=f1:, timestamp=1401883420805, value=value4
45+
row1 column=f1:a, timestamp=1401883411986, value=value1
46+
row1 column=f1:b, timestamp=1401883415212, value=value2
47+
row2 column=f1:, timestamp=1401883417858, value=value3
48+
row3 column=f1:, timestamp=1401883420805, value=value4
4849
4 row(s) in 0.0240 seconds
4950
"""
5051
if __name__ == "__main__":
@@ -64,6 +65,8 @@
6465
table = sys.argv[2]
6566
sc = SparkContext(appName="HBaseInputFormat")
6667

68+
# Other options for configuring scan behavior are available. More information available at
69+
# https://github.com/apache/hbase/blob/master/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableInputFormat.java
6770
conf = {"hbase.zookeeper.quorum": host, "hbase.mapreduce.inputtable": table}
6871
if len(sys.argv) > 3:
6972
conf = {"hbase.zookeeper.quorum": host, "zookeeper.znode.parent": sys.argv[3],
@@ -78,6 +81,8 @@
7881
keyConverter=keyConv,
7982
valueConverter=valueConv,
8083
conf=conf)
84+
hbase_rdd = hbase_rdd.flatMapValues(lambda v: v.split("\n")).mapValues(json.loads)
85+
8186
output = hbase_rdd.collect()
8287
for (k, v) in output:
8388
print((k, v))

examples/src/main/scala/org/apache/spark/examples/pythonconverters/HBaseConverters.scala

Lines changed: 17 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -18,20 +18,34 @@
1818
package org.apache.spark.examples.pythonconverters
1919

2020
import scala.collection.JavaConversions._
21+
import scala.util.parsing.json.JSONObject
2122

2223
import org.apache.spark.api.python.Converter
2324
import org.apache.hadoop.hbase.client.{Put, Result}
2425
import org.apache.hadoop.hbase.io.ImmutableBytesWritable
2526
import org.apache.hadoop.hbase.util.Bytes
27+
import org.apache.hadoop.hbase.KeyValue.Type
28+
import org.apache.hadoop.hbase.CellUtil
2629

2730
/**
28-
* Implementation of [[org.apache.spark.api.python.Converter]] that converts an
29-
* HBase Result to a String
31+
* Implementation of [[org.apache.spark.api.python.Converter]] that converts all
32+
* the records in an HBase Result to a String
3033
*/
3134
class HBaseResultToStringConverter extends Converter[Any, String] {
3235
override def convert(obj: Any): String = {
36+
import collection.JavaConverters._
3337
val result = obj.asInstanceOf[Result]
34-
Bytes.toStringBinary(result.value())
38+
val output = result.listCells.asScala.map(cell =>
39+
Map(
40+
"row" -> Bytes.toStringBinary(CellUtil.cloneRow(cell)),
41+
"columnFamily" -> Bytes.toStringBinary(CellUtil.cloneFamily(cell)),
42+
"qualifier" -> Bytes.toStringBinary(CellUtil.cloneQualifier(cell)),
43+
"timestamp" -> cell.getTimestamp.toString,
44+
"type" -> Type.codeToType(cell.getTypeByte).toString,
45+
"value" -> Bytes.toStringBinary(CellUtil.cloneValue(cell))
46+
)
47+
)
48+
output.map(JSONObject(_).toString()).mkString("\n")
3549
}
3650
}
3751

0 commit comments

Comments
 (0)