forked from cloudera-labs/SparkOnHBase
-
Notifications
You must be signed in to change notification settings - Fork 0
/
JavaHBaseContext.scala
415 lines (385 loc) · 15.9 KB
/
JavaHBaseContext.scala
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
package com.cloudera.spark.hbase
import org.apache.spark.api.java.JavaSparkContext
import org.apache.hadoop.conf.Configuration
import org.apache.spark.api.java.JavaRDD
import org.apache.spark.api.java.function.VoidFunction
import org.apache.spark.api.java.function.Function
import org.apache.hadoop.hbase.client.HConnection
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.Increment
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 scala.reflect.ClassTag
class JavaHBaseContext(@transient jsc: JavaSparkContext,
@transient config: Configuration) extends Serializable {
val hbc = new HBaseContext(jsc.sc, config)
/**
* A simple enrichment of the traditional Spark javaRdd foreachPartition.
* This function differs from the original in that it offers the
* developer access to a already connected HConnection object
*
* Note: Do not close the HConnection object. All HConnection
* management is handled outside this method
*
* @param javaRdd Original javaRdd with data to iterate over
* @param f Function to be given a iterator to iterate through
* the RDD values and a HConnection object to interact
* with HBase
*/
def foreachPartition[T](javaRdd: JavaRDD[T],
f: VoidFunction[(java.util.Iterator[T], HConnection)] ) = {
hbc.foreachPartition(javaRdd.rdd,
(iterator:Iterator[T], hConnection) =>
{ f.call((iterator, hConnection))})
}
def foreach[T](javaRdd: JavaRDD[T],
f: VoidFunction[(T, HConnection)] ) = {
hbc.foreachPartition(javaRdd.rdd,
(iterator:Iterator[T], hConnection) =>
iterator.foreach(a => f.call((a, hConnection))))
//{ f.call((iterator, hConnection))})
}
/**
* A simple enrichment of the traditional Spark Streaming dStream foreach
* This function differs from the original in that it offers the
* developer access to a already connected HConnection object
*
* Note: Do not close the HConnection object. All HConnection
* management is handled outside this method
*
* @param javaDstream Original DStream with data to iterate over
* @param f Function to be given a iterator to iterate through
* the JavaDStream values and a HConnection object to
* interact with HBase
*/
def foreachRDD[T](javaDstream: JavaDStream[T],
f: VoidFunction[(Iterator[T], HConnection)]) = {
hbc.foreachRDD(javaDstream.dstream, (it:Iterator[T], hc: HConnection) => f.call(it, hc))
}
/**
* A simple enrichment of the traditional Spark JavaRDD mapPartition.
* This function differs from the original in that it offers the
* developer access to a already connected HConnection object
*
* Note: Do not close the HConnection object. All HConnection
* management is handled outside this method
*
* Note: Make sure to partition correctly to avoid memory issue when
* getting data from HBase
*
* @param javaRdd Original JavaRdd with data to iterate over
* @param mp Function to be given a iterator to iterate through
* the RDD values and a HConnection object to interact
* with HBase
* @return Returns a new RDD generated by the user definition
* function just like normal mapPartition
*/
def mapPartition[T,R](javaRdd: JavaRDD[T],
mp: FlatMapFunction[(java.util.Iterator[T], HConnection),R] ): JavaRDD[R] = {
def fn = (x: Iterator[T], hc: HConnection) =>
asScalaIterator(
mp.call((asJavaIterator(x), hc)).iterator()
)
JavaRDD.fromRDD(hbc.mapPartition(javaRdd.rdd,
(iterator:Iterator[T], hConnection:HConnection) =>
fn(iterator, hConnection))(fakeClassTag[R]))(fakeClassTag[R])
}
/**
* A simple enrichment of the traditional Spark Streaming JavaDStream
* mapPartition.
*
* This function differs from the original in that it offers the
* developer access to a already connected HConnection object
*
* Note: Do not close the HConnection object. All HConnection
* management is handled outside this method
*
* Note: Make sure to partition correctly to avoid memory issue when
* getting data from HBase
*
* @param javaDstream Original JavaDStream with data to iterate over
* @param mp Function to be given a iterator to iterate through
* the JavaDStream values and a HConnection object to
* interact with HBase
* @return Returns a new JavaDStream generated by the user
* definition function just like normal mapPartition
*/
def streamMap[T, U](javaDstream: JavaDStream[T],
mp: Function[(Iterator[T], HConnection), Iterator[U]]): JavaDStream[U] = {
JavaDStream.fromDStream(hbc.streamMap(javaDstream.dstream,
(it: Iterator[T], hc: HConnection) =>
mp.call(it, hc) )(fakeClassTag[U]))(fakeClassTag[U])
}
/**
* A simple abstraction over the HBaseContext.foreachPartition method.
*
* It allow addition support for a user to take JavaRDD
* and generate puts and send them to HBase.
* The complexity of managing the HConnection is
* removed from the developer
*
* @param javaRdd Original JavaRDD with data to iterate over
* @param tableName The name of the table to put into
* @param f Function to convert a value in the JavaRDD
* to a HBase Put
* @param autoFlush If autoFlush should be turned on
*/
def bulkPut[T](javaRdd: JavaRDD[T],
tableName: String,
f: Function[(T), Put],
autoFlush: Boolean) {
hbc.bulkPut(javaRdd.rdd, tableName, (t:T) => f.call(t), autoFlush)
}
/**
* A simple abstraction over the HBaseContext.streamMapPartition method.
*
* It allow addition support for a user to take a JavaDStream and
* generate puts and send them to HBase.
*
* The complexity of managing the HConnection is
* removed from the developer
*
* @param javaDstream Original DStream with data to iterate over
* @param tableName The name of the table to put into
* @param f Function to convert a value in
* the JavaDStream to a HBase Put
* @param autoFlush If autoFlush should be turned on
*/
def streamBulkPut[T](javaDstream: JavaDStream[T],
tableName: String,
f: Function[T,Put],
autoFlush: Boolean) = {
hbc.streamBulkPut(javaDstream.dstream,
tableName,
(t:T) => f.call(t),
autoFlush)
}
/**
* A simple abstraction over the HBaseContext.foreachPartition method.
*
* It allow addition support for a user to take RDD
* and generate checkAndPuts and send them to HBase.
* The complexity of managing the HConnection is
* removed from the developer
*
* @param javaRdd Original RDD with data to iterate over
* @param tableName The name of the table to put into
* @param f Function to convert a value in the RDD to
* a HBase checkAndPut
* @param autoFlush If autoFlush should be turned on
*/
def bulkCheckAndPut[T](javaRdd: JavaRDD[T],
tableName: String,
f: Function[T,(Array[Byte], Array[Byte], Array[Byte], Array[Byte], Put)],
autoFlush: Boolean) {
hbc.bulkCheckAndPut(javaRdd.rdd, tableName, (t:T) => f.call(t), autoFlush)
}
/**
* A simple abstraction over the HBaseContext.foreachPartition method.
*
* It allow addition support for a user to take a JavaRDD and
* generate increments and send them to HBase.
*
* The complexity of managing the HConnection is
* removed from the developer
*
* @param javaRdd Original JavaRDD with data to iterate over
* @param tableName The name of the table to increment to
* @param f function to convert a value in the JavaRDD to a
* HBase Increments
* @param batchSize The number of increments to batch before sending to HBase
*/
def bulkIncrement[T](javaRdd: JavaRDD[T], tableName: String,
f: Function[T,Increment], batchSize:Integer) {
hbc.bulkIncrement(javaRdd.rdd, tableName, (t:T) => f.call(t), batchSize)
}
/**
* A simple abstraction over the HBaseContext.foreachPartition method.
*
* It allow addition support for a user to take a JavaRDD and
* generate delete and send them to HBase.
*
* The complexity of managing the HConnection is
* removed from the developer
*
* @param javaRdd Original JavaRDD with data to iterate over
* @param tableName The name of the table to delete from
* @param f Function to convert a value in the JavaRDD to a
* HBase Deletes
* @param batchSize The number of delete to batch before sending to HBase
*/
def bulkDelete[T](javaRdd: JavaRDD[T], tableName: String,
f: Function[T, Delete], batchSize:Integer) {
hbc.bulkDelete(javaRdd.rdd, tableName, (t:T) => f.call(t), batchSize)
}
/**
* A simple abstraction over the HBaseContext.streamBulkMutation method.
*
* It allow addition support for a user to take a DStream and
* generate Increments and send them to HBase.
*
* The complexity of managing the HConnection is
* removed from the developer
*
* @param javaDstream Original JavaDStream with data to iterate over
* @param tableName The name of the table to increments into
* @param f Function to convert a value in the JavaDStream to a
* HBase Increments
* @param batchSize The number of increments to batch before sending to HBase
*/
def streamBulkIncrement[T](javaDstream: JavaDStream[T],
tableName: String,
f: Function[T, Increment],
batchSize: Integer) = {
hbc.streamBulkIncrement(javaDstream.dstream, tableName,
(t:T) => f.call(t),
batchSize)
}
/**
* A simple abstraction over the HBaseContext.streamBulkMutation method.
*
* It allow addition support for a user to take a JavaDStream and
* generate Delete and send them to HBase.
*
* The complexity of managing the HConnection is
* removed from the developer
*
* @param javaDstream Original DStream with data to iterate over
* @param tableName The name of the table to delete from
* @param f function to convert a value in the JavaDStream to a
* HBase Delete
*/
def streamBulkDelete[T](javaDstream: JavaDStream[T],
tableName: String,
f: Function[T, Delete],
batchSize: Integer) = {
hbc.streamBulkDelete(javaDstream.dstream, tableName,
(t:T) => f.call(t),
batchSize)
}
/**
* A simple abstraction over the bulkCheckDelete method.
*
* It allow addition support for a user to take a JavaDStream and
* generate CheckAndDelete and send them to HBase.
*
* The complexity of managing the HConnection is
* removed from the developer
*
* @param javaDstream Original JavaDStream with data to iterate over
* @param tableName The name of the table to delete from
* @param f function to convert a value in the JavaDStream to a
* HBase Delete
*/
def streamBulkCheckAndDelete[T](javaDstream: JavaDStream[T],
tableName: String,
f: Function[T, (Array[Byte], Array[Byte], Array[Byte], Array[Byte], Delete)]) = {
hbc.streamBulkCheckAndDelete(javaDstream.dstream, tableName,
(t:T) => f.call(t))
}
/**
* A simple abstraction over the HBaseContext.mapPartition method.
*
* It allow addition support for a user to take a JavaRDD and generates a
* new RDD based on Gets and the results they bring back from HBase
*
* @param javaRdd Original JavaRDD with data to iterate over
* @param tableName The name of the table to get from
* @param makeGet Function to convert a value in the JavaRDD to a
* HBase Get
* @param convertResult This will convert the HBase Result object to
* what ever the user wants to put in the resulting
* JavaRDD
* return new JavaRDD that is created by the Get to HBase
*/
def bulkGet[T, U](tableName: String,
batchSize:Integer,
javaRdd: JavaRDD[T],
makeGet: Function[T, Get],
convertResult: Function[Result, U]): JavaRDD[U] = {
JavaRDD.fromRDD(hbc.bulkGet(tableName,
batchSize,
javaRdd.rdd,
(t:T) => makeGet.call(t),
(r:Result) => {convertResult.call(r)}))(fakeClassTag[U])
}
/**
* A simple abstraction over the HBaseContext.streamMap method.
*
* It allow addition support for a user to take a DStream and
* generates a new DStream based on Gets and the results
* they bring back from HBase
*
* @param javaDStream Original DStream with data to iterate over
* @param tableName The name of the table to get from
* @param makeGet Function to convert a value in the JavaDStream to a
* HBase Get
* @param convertResult This will convert the HBase Result object to
* what ever the user wants to put in the resulting
* JavaDStream
* return new JavaDStream that is created by the Get to HBase
*/
def streamBulkGet[T, U](tableName:String,
batchSize:Integer,
javaDStream: JavaDStream[T],
makeGet: Function[T, Get],
convertResult: Function[Result, U]) {
JavaDStream.fromDStream(hbc.streamBulkGet(tableName,
batchSize,
javaDStream.dstream,
(t:T) => makeGet.call(t),
(r:Result) => convertResult.call(r) )(fakeClassTag[U]))(fakeClassTag[U])
}
/**
* This function will use the native HBase TableInputFormat with the
* given scan object to generate a new JavaRDD
*
* @param tableName the name of the table to scan
* @param scans the HBase scan object to use to read data from HBase
* @param f function to convert a Result object from HBase into
* what the user wants in the final generated JavaRDD
* @return new JavaRDD with results from scan
*/
def hbaseRDD[U](tableName: String,
scans: Scan,
f: Function[(ImmutableBytesWritable, Result), U]):
JavaRDD[U] = {
JavaRDD.fromRDD(
hbc.hbaseRDD[U](tableName,
scans,
(v:(ImmutableBytesWritable, Result)) =>
f.call(v._1, v._2))(fakeClassTag[U]))(fakeClassTag[U])
}
/**
* A overloaded version of HBaseContext hbaseRDD that predefines the
* type of the outputing JavaRDD
*
* @param tableName the name of the table to scan
* @param scans the HBase scan object to use to read data from HBase
* @return New JavaRDD with results from scan
*
*/
def hbaseRDD(tableName: String,
scans: Scan): JavaRDD[(Array[Byte], java.util.List[(Array[Byte], Array[Byte], Array[Byte])])] = {
JavaRDD.fromRDD(hbc.hbaseRDD(tableName, scans))
}
/**
* Produces a ClassTag[T], which is actually just a casted ClassTag[AnyRef].
*
* This method is used to keep ClassTags out of the external Java API, as the Java compiler
* cannot produce them automatically. While this ClassTag-faking does please the compiler,
* it can cause problems at runtime if the Scala API relies on ClassTags for correctness.
*
* Often, though, a ClassTag[AnyRef] will not lead to incorrect behavior, just worse performance
* or security issues. For instance, an Array[AnyRef] can hold any type T, but may lose primitive
* specialization.
*/
private[spark]
def fakeClassTag[T]: ClassTag[T] = ClassTag.AnyRef.asInstanceOf[ClassTag[T]]
}