HBASE-14184 Fix indention and type-o in JavaHBaseContext (Ted Malaska)

This commit is contained in:
tedyu 2015-08-07 11:02:24 -07:00
parent f06daaf010
commit e53d2481ee
1 changed files with 288 additions and 293 deletions

View File

@ -17,22 +17,15 @@
package org.apache.hadoop.hbase.spark package org.apache.hadoop.hbase.spark
import org.apache.hadoop.hbase.TableName
import org.apache.spark.api.java.JavaSparkContext
import org.apache.hadoop.conf.Configuration import org.apache.hadoop.conf.Configuration
import org.apache.spark.api.java.JavaRDD import org.apache.hadoop.hbase.TableName
import org.apache.spark.api.java.function.VoidFunction import org.apache.hadoop.hbase.client.{Connection, Delete, Get, Put, Result, Scan}
import org.apache.spark.api.java.function.Function
import org.apache.hadoop.hbase.client.Connection
import org.apache.spark.streaming.api.java.JavaDStream
import org.apache.spark.api.java.function.FlatMapFunction
import scala.collection.JavaConversions._
import org.apache.hadoop.hbase.client.Put
import org.apache.hadoop.hbase.client.Delete
import org.apache.hadoop.hbase.client.Get
import org.apache.hadoop.hbase.client.Result
import org.apache.hadoop.hbase.client.Scan
import org.apache.hadoop.hbase.io.ImmutableBytesWritable import org.apache.hadoop.hbase.io.ImmutableBytesWritable
import org.apache.spark.api.java.{JavaRDD, JavaSparkContext}
import org.apache.spark.api.java.function.{FlatMapFunction, Function, VoidFunction}
import org.apache.spark.streaming.api.java.JavaDStream
import scala.collection.JavaConversions._
import scala.reflect.ClassTag import scala.reflect.ClassTag
/** /**
@ -61,11 +54,12 @@ class JavaHBaseContext(@transient jsc: JavaSparkContext,
* with HBase * with HBase
*/ */
def foreachPartition[T](javaRdd: JavaRDD[T], def foreachPartition[T](javaRdd: JavaRDD[T],
f: VoidFunction[(java.util.Iterator[T], Connection)] ) = { f: VoidFunction[(java.util.Iterator[T], Connection)]) = {
hbaseContext.foreachPartition(javaRdd.rdd, hbaseContext.foreachPartition(javaRdd.rdd,
(it:Iterator[T], conn:Connection) => (it: Iterator[T], conn: Connection) => {
{ f.call((it, conn)) }) f.call((it, conn))
})
} }
/** /**
@ -84,7 +78,7 @@ class JavaHBaseContext(@transient jsc: JavaSparkContext,
def foreachPartition[T](javaDstream: JavaDStream[T], def foreachPartition[T](javaDstream: JavaDStream[T],
f: VoidFunction[(Iterator[T], Connection)]) = { f: VoidFunction[(Iterator[T], Connection)]) = {
hbaseContext.foreachPartition(javaDstream.dstream, hbaseContext.foreachPartition(javaDstream.dstream,
(it:Iterator[T], conn: Connection) => f.call(it, conn)) (it: Iterator[T], conn: Connection) => f.call(it, conn))
} }
/** /**
@ -105,9 +99,9 @@ class JavaHBaseContext(@transient jsc: JavaSparkContext,
* @return Returns a new RDD generated by the user definition * @return Returns a new RDD generated by the user definition
* function just like normal mapPartition * function just like normal mapPartition
*/ */
def mapPartitions[T,R](javaRdd: JavaRDD[T], def mapPartitions[T, R](javaRdd: JavaRDD[T],
f: FlatMapFunction[(java.util.Iterator[T], f: FlatMapFunction[(java.util.Iterator[T],
Connection),R] ): JavaRDD[R] = { Connection), R]): JavaRDD[R] = {
def fn = (it: Iterator[T], conn: Connection) => def fn = (it: Iterator[T], conn: Connection) =>
asScalaIterator( asScalaIterator(
@ -115,7 +109,7 @@ class JavaHBaseContext(@transient jsc: JavaSparkContext,
) )
JavaRDD.fromRDD(hbaseContext.mapPartitions(javaRdd.rdd, JavaRDD.fromRDD(hbaseContext.mapPartitions(javaRdd.rdd,
(iterator:Iterator[T], connection:Connection) => (iterator: Iterator[T], connection: Connection) =>
fn(iterator, connection))(fakeClassTag[R]))(fakeClassTag[R]) fn(iterator, connection))(fakeClassTag[R]))(fakeClassTag[R])
} }
@ -144,7 +138,7 @@ class JavaHBaseContext(@transient jsc: JavaSparkContext,
JavaDStream[U] = { JavaDStream[U] = {
JavaDStream.fromDStream(hbaseContext.streamMapPartitions(javaDstream.dstream, JavaDStream.fromDStream(hbaseContext.streamMapPartitions(javaDstream.dstream,
(it: Iterator[T], conn: Connection) => (it: Iterator[T], conn: Connection) =>
mp.call(it, conn) )(fakeClassTag[U]))(fakeClassTag[U]) mp.call(it, conn))(fakeClassTag[U]))(fakeClassTag[U])
} }
/** /**
@ -155,16 +149,16 @@ class JavaHBaseContext(@transient jsc: JavaSparkContext,
* The complexity of managing the HConnection is * The complexity of managing the HConnection is
* removed from the developer * removed from the developer
* *
* @param javaDdd Original JavaRDD with data to iterate over * @param javaRdd Original JavaRDD with data to iterate over
* @param tableName The name of the table to put into * @param tableName The name of the table to put into
* @param f Function to convert a value in the JavaRDD * @param f Function to convert a value in the JavaRDD
* to a HBase Put * to a HBase Put
*/ */
def bulkPut[T](javaDdd: JavaRDD[T], def bulkPut[T](javaRdd: JavaRDD[T],
tableName: TableName, tableName: TableName,
f: Function[(T), Put]) { f: Function[(T), Put]) {
hbaseContext.bulkPut(javaDdd.rdd, tableName, (t:T) => f.call(t)) hbaseContext.bulkPut(javaRdd.rdd, tableName, (t: T) => f.call(t))
} }
/** /**
@ -183,10 +177,10 @@ class JavaHBaseContext(@transient jsc: JavaSparkContext,
*/ */
def streamBulkPut[T](javaDstream: JavaDStream[T], def streamBulkPut[T](javaDstream: JavaDStream[T],
tableName: TableName, tableName: TableName,
f: Function[T,Put]) = { f: Function[T, Put]) = {
hbaseContext.streamBulkPut(javaDstream.dstream, hbaseContext.streamBulkPut(javaDstream.dstream,
tableName, tableName,
(t:T) => f.call(t)) (t: T) => f.call(t))
} }
/** /**
@ -205,8 +199,8 @@ class JavaHBaseContext(@transient jsc: JavaSparkContext,
* @param batchSize The number of deletes to batch before sending to HBase * @param batchSize The number of deletes to batch before sending to HBase
*/ */
def bulkDelete[T](javaRdd: JavaRDD[T], tableName: TableName, def bulkDelete[T](javaRdd: JavaRDD[T], tableName: TableName,
f: Function[T, Delete], batchSize:Integer) { f: Function[T, Delete], batchSize: Integer) {
hbaseContext.bulkDelete(javaRdd.rdd, tableName, (t:T) => f.call(t), batchSize) hbaseContext.bulkDelete(javaRdd.rdd, tableName, (t: T) => f.call(t), batchSize)
} }
/** /**
@ -218,18 +212,18 @@ class JavaHBaseContext(@transient jsc: JavaSparkContext,
* The complexity of managing the HConnection is * The complexity of managing the HConnection is
* removed from the developer * removed from the developer
* *
* @param javaDstream Original DStream with data to iterate over * @param javaDStream Original DStream with data to iterate over
* @param tableName The name of the table to delete from * @param tableName The name of the table to delete from
* @param f Function to convert a value in the JavaDStream to a * @param f Function to convert a value in the JavaDStream to a
* HBase Delete * HBase Delete
* @param batchSize The number of deletes to be sent at once * @param batchSize The number of deletes to be sent at once
*/ */
def streamBulkDelete[T](javaDstream: JavaDStream[T], def streamBulkDelete[T](javaDStream: JavaDStream[T],
tableName: TableName, tableName: TableName,
f: Function[T, Delete], f: Function[T, Delete],
batchSize: Integer) = { batchSize: Integer) = {
hbaseContext.streamBulkDelete(javaDstream.dstream, tableName, hbaseContext.streamBulkDelete(javaDStream.dstream, tableName,
(t:T) => f.call(t), (t: T) => f.call(t),
batchSize) batchSize)
} }
@ -247,10 +241,10 @@ class JavaHBaseContext(@transient jsc: JavaSparkContext,
* @param convertResult This will convert the HBase Result object to * @param convertResult This will convert the HBase Result object to
* what ever the user wants to put in the resulting * what ever the user wants to put in the resulting
* JavaRDD * JavaRDD
* return new JavaRDD that is created by the Get to HBase * @return New JavaRDD that is created by the Get to HBase
*/ */
def bulkGet[T, U](tableName: TableName, def bulkGet[T, U](tableName: TableName,
batchSize:Integer, batchSize: Integer,
javaRdd: JavaRDD[T], javaRdd: JavaRDD[T],
makeGet: Function[T, Get], makeGet: Function[T, Get],
convertResult: Function[Result, U]): JavaRDD[U] = { convertResult: Function[Result, U]): JavaRDD[U] = {
@ -258,8 +252,10 @@ class JavaHBaseContext(@transient jsc: JavaSparkContext,
JavaRDD.fromRDD(hbaseContext.bulkGet[T, U](tableName, JavaRDD.fromRDD(hbaseContext.bulkGet[T, U](tableName,
batchSize, batchSize,
javaRdd.rdd, javaRdd.rdd,
(t:T) => makeGet.call(t), (t: T) => makeGet.call(t),
(r:Result) => {convertResult.call(r)})(fakeClassTag[U]))(fakeClassTag[U]) (r: Result) => {
convertResult.call(r)
})(fakeClassTag[U]))(fakeClassTag[U])
} }
@ -279,29 +275,29 @@ class JavaHBaseContext(@transient jsc: JavaSparkContext,
* @param convertResult This will convert the HBase Result object to * @param convertResult This will convert the HBase Result object to
* what ever the user wants to put in the resulting * what ever the user wants to put in the resulting
* JavaDStream * JavaDStream
* return new JavaDStream that is created by the Get to HBase * @return New JavaDStream that is created by the Get to HBase
*/ */
def streamBulkGet[T, U](tableName:TableName, def streamBulkGet[T, U](tableName: TableName,
batchSize:Integer, batchSize: Integer,
javaDStream: JavaDStream[T], javaDStream: JavaDStream[T],
makeGet: Function[T, Get], makeGet: Function[T, Get],
convertResult: Function[Result, U]) { convertResult: Function[Result, U]) {
JavaDStream.fromDStream(hbaseContext.streamBulkGet(tableName, JavaDStream.fromDStream(hbaseContext.streamBulkGet(tableName,
batchSize, batchSize,
javaDStream.dstream, javaDStream.dstream,
(t:T) => makeGet.call(t), (t: T) => makeGet.call(t),
(r:Result) => convertResult.call(r) )(fakeClassTag[U]))(fakeClassTag[U]) (r: Result) => convertResult.call(r))(fakeClassTag[U]))(fakeClassTag[U])
} }
/** /**
* This function will use the native HBase TableInputFormat with the * This function will use the native HBase TableInputFormat with the
* given scan object to generate a new JavaRDD * given scan object to generate a new JavaRDD
* *
* @param tableName the name of the table to scan * @param tableName The name of the table to scan
* @param scans the HBase scan object to use to read data from HBase * @param scans The HBase scan object to use to read data from HBase
* @param f function to convert a Result object from HBase into * @param f Function to convert a Result object from HBase into
* what the user wants in the final generated JavaRDD * What the user wants in the final generated JavaRDD
* @return new JavaRDD with results from scan * @return New JavaRDD with results from scan
*/ */
def hbaseRDD[U](tableName: TableName, def hbaseRDD[U](tableName: TableName,
scans: Scan, scans: Scan,
@ -310,7 +306,7 @@ class JavaHBaseContext(@transient jsc: JavaSparkContext,
JavaRDD.fromRDD( JavaRDD.fromRDD(
hbaseContext.hbaseRDD[U](tableName, hbaseContext.hbaseRDD[U](tableName,
scans, scans,
(v:(ImmutableBytesWritable, Result)) => (v: (ImmutableBytesWritable, Result)) =>
f.call(v._1, v._2))(fakeClassTag[U]))(fakeClassTag[U]) f.call(v._1, v._2))(fakeClassTag[U]))(fakeClassTag[U])
} }
@ -318,10 +314,9 @@ class JavaHBaseContext(@transient jsc: JavaSparkContext,
* A overloaded version of HBaseContext hbaseRDD that define the * A overloaded version of HBaseContext hbaseRDD that define the
* type of the resulting JavaRDD * type of the resulting JavaRDD
* *
* @param tableName the name of the table to scan * @param tableName The name of the table to scan
* @param scans the HBase scan object to use to read data from HBase * @param scans The HBase scan object to use to read data from HBase
* @return New JavaRDD with results from scan * @return New JavaRDD with results from scan
*
*/ */
def hbaseRDD(tableName: TableName, def hbaseRDD(tableName: TableName,
scans: Scan): scans: Scan):