aboutsummaryrefslogtreecommitdiff
path: root/examples
diff options
context:
space:
mode:
authorGenTang <gen.tang86@gmail.com>2015-05-22 23:37:03 -0700
committerDavies Liu <davies@databricks.com>2015-05-22 23:37:03 -0700
commit4583cf4be17155c68178155acf6866d7cc8f7df0 (patch)
treeda036545e4fd42dc265a316801d900addb526ecd /examples
parent368b8c2b5ed8b06b00ac87059f75915b13ba3b8d (diff)
downloadspark-4583cf4be17155c68178155acf6866d7cc8f7df0.tar.gz
spark-4583cf4be17155c68178155acf6866d7cc8f7df0.tar.bz2
spark-4583cf4be17155c68178155acf6866d7cc8f7df0.zip
[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 <gen.tang86@gmail.com> Closes #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
Diffstat (limited to 'examples')
-rw-r--r--examples/src/main/python/hbase_inputformat.py21
-rw-r--r--examples/src/main/scala/org/apache/spark/examples/pythonconverters/HBaseConverters.scala20
2 files changed, 30 insertions, 11 deletions
diff --git a/examples/src/main/python/hbase_inputformat.py b/examples/src/main/python/hbase_inputformat.py
index 5b82a14fba..c5ae5d043b 100644
--- a/examples/src/main/python/hbase_inputformat.py
+++ b/examples/src/main/python/hbase_inputformat.py
@@ -18,6 +18,7 @@
from __future__ import print_function
import sys
+import json
from pyspark import SparkContext
@@ -27,24 +28,24 @@ Create test data in HBase first:
hbase(main):016:0> create 'test', 'f1'
0 row(s) in 1.0430 seconds
-hbase(main):017:0> put 'test', 'row1', 'f1', 'value1'
+hbase(main):017:0> put 'test', 'row1', 'f1:a', 'value1'
0 row(s) in 0.0130 seconds
-hbase(main):018:0> put 'test', 'row2', 'f1', 'value2'
+hbase(main):018:0> put 'test', 'row1', 'f1:b', 'value2'
0 row(s) in 0.0030 seconds
-hbase(main):019:0> put 'test', 'row3', 'f1', 'value3'
+hbase(main):019:0> put 'test', 'row2', 'f1', 'value3'
0 row(s) in 0.0050 seconds
-hbase(main):020:0> put 'test', 'row4', 'f1', 'value4'
+hbase(main):020:0> put 'test', 'row3', 'f1', 'value4'
0 row(s) in 0.0110 seconds
hbase(main):021:0> scan 'test'
ROW COLUMN+CELL
- row1 column=f1:, timestamp=1401883411986, value=value1
- row2 column=f1:, timestamp=1401883415212, value=value2
- row3 column=f1:, timestamp=1401883417858, value=value3
- row4 column=f1:, timestamp=1401883420805, value=value4
+ row1 column=f1:a, timestamp=1401883411986, value=value1
+ row1 column=f1:b, timestamp=1401883415212, value=value2
+ row2 column=f1:, timestamp=1401883417858, value=value3
+ row3 column=f1:, timestamp=1401883420805, value=value4
4 row(s) in 0.0240 seconds
"""
if __name__ == "__main__":
@@ -64,6 +65,8 @@ if __name__ == "__main__":
table = sys.argv[2]
sc = SparkContext(appName="HBaseInputFormat")
+ # Other options for configuring scan behavior are available. More information available at
+ # https://github.com/apache/hbase/blob/master/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableInputFormat.java
conf = {"hbase.zookeeper.quorum": host, "hbase.mapreduce.inputtable": table}
if len(sys.argv) > 3:
conf = {"hbase.zookeeper.quorum": host, "zookeeper.znode.parent": sys.argv[3],
@@ -78,6 +81,8 @@ if __name__ == "__main__":
keyConverter=keyConv,
valueConverter=valueConv,
conf=conf)
+ hbase_rdd = hbase_rdd.flatMapValues(lambda v: v.split("\n")).mapValues(json.loads)
+
output = hbase_rdd.collect()
for (k, v) in output:
print((k, v))
diff --git a/examples/src/main/scala/org/apache/spark/examples/pythonconverters/HBaseConverters.scala b/examples/src/main/scala/org/apache/spark/examples/pythonconverters/HBaseConverters.scala
index 273bee0a8b..90d48a6410 100644
--- a/examples/src/main/scala/org/apache/spark/examples/pythonconverters/HBaseConverters.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/pythonconverters/HBaseConverters.scala
@@ -18,20 +18,34 @@
package org.apache.spark.examples.pythonconverters
import scala.collection.JavaConversions._
+import scala.util.parsing.json.JSONObject
import org.apache.spark.api.python.Converter
import org.apache.hadoop.hbase.client.{Put, Result}
import org.apache.hadoop.hbase.io.ImmutableBytesWritable
import org.apache.hadoop.hbase.util.Bytes
+import org.apache.hadoop.hbase.KeyValue.Type
+import org.apache.hadoop.hbase.CellUtil
/**
- * Implementation of [[org.apache.spark.api.python.Converter]] that converts an
- * HBase Result to a String
+ * Implementation of [[org.apache.spark.api.python.Converter]] that converts all
+ * the records in an HBase Result to a String
*/
class HBaseResultToStringConverter extends Converter[Any, String] {
override def convert(obj: Any): String = {
+ import collection.JavaConverters._
val result = obj.asInstanceOf[Result]
- Bytes.toStringBinary(result.value())
+ val output = result.listCells.asScala.map(cell =>
+ Map(
+ "row" -> Bytes.toStringBinary(CellUtil.cloneRow(cell)),
+ "columnFamily" -> Bytes.toStringBinary(CellUtil.cloneFamily(cell)),
+ "qualifier" -> Bytes.toStringBinary(CellUtil.cloneQualifier(cell)),
+ "timestamp" -> cell.getTimestamp.toString,
+ "type" -> Type.codeToType(cell.getTypeByte).toString,
+ "value" -> Bytes.toStringBinary(CellUtil.cloneValue(cell))
+ )
+ )
+ output.map(JSONObject(_).toString()).mkString("\n")
}
}