├── .cache ├── SparkOnHBase.Design.Doc.docx ├── .settings ├── org.eclipse.jdt.ui.prefs ├── org.eclipse.m2e.core.prefs ├── org.eclipse.core.resources.prefs └── org.eclipse.jdt.core.prefs ├── README.md ├── src └── main │ ├── scala │ └── org │ │ └── apache │ │ ├── spark │ │ └── SparkHadoopMapReduceUtilExtended.scala │ │ └── hadoop │ │ └── hbase │ │ └── spark │ │ ├── ByteArrayWrapper.scala │ │ ├── ByteArrayComparable.scala │ │ ├── FamilyHFileWriteOptions.scala │ │ ├── KeyFamilyQualifier.scala │ │ ├── BulkLoadPartitioner.scala │ │ ├── FamiliesQualifiersValues.scala │ │ ├── example │ │ ├── hbasecontext │ │ │ ├── HBaseDistributedScanExample.scala │ │ │ ├── HBaseBulkDeleteExample.scala │ │ │ ├── HBaseStreamingBulkPutExample.scala │ │ │ ├── HBaseBulkPutExampleFromFile.scala │ │ │ ├── HBaseBulkPutExample.scala │ │ │ ├── HBaseBulkPutTimestampExample.scala │ │ │ └── HBaseBulkGetExample.scala │ │ └── rdd │ │ │ ├── HBaseBulkDeleteExample.scala │ │ │ ├── HBaseBulkPutExample.scala │ │ │ ├── HBaseBulkGetExample.scala │ │ │ ├── HBaseMapPartitionExample.scala │ │ │ └── HBaseForeachPartitionExample.scala │ │ ├── ColumnFamilyQualifierMapKeyWrapper.scala │ │ ├── HBaseScanRDD.scala │ │ ├── HBaseDStreamFunctions.scala │ │ ├── DynamicLogicExpression.scala │ │ ├── HBaseRDDFunctions.scala │ │ └── JavaHBaseContext.scala │ └── java │ └── org │ └── apache │ └── hadoop │ └── hbase │ └── spark │ └── SparkSQLPushDownFilter.java ├── .project ├── .classpath ├── pom.unittest.but.no.cluster.xml ├── LICENSE.txt └── pom.xml /.cache: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/tmalaska/SparkOnHBase/HEAD/.cache -------------------------------------------------------------------------------- /SparkOnHBase.Design.Doc.docx: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/tmalaska/SparkOnHBase/HEAD/SparkOnHBase.Design.Doc.docx -------------------------------------------------------------------------------- /.settings/org.eclipse.jdt.ui.prefs: -------------------------------------------------------------------------------- 1 | eclipse.preferences.version=1 2 | formatter_profile=_ted 3 | formatter_settings_version=12 4 | -------------------------------------------------------------------------------- /.settings/org.eclipse.m2e.core.prefs: -------------------------------------------------------------------------------- 1 | activeProfiles= 2 | eclipse.preferences.version=1 3 | resolveWorkspaceProjects=true 4 | version=1 5 | -------------------------------------------------------------------------------- /.settings/org.eclipse.core.resources.prefs: -------------------------------------------------------------------------------- 1 | eclipse.preferences.version=1 2 | encoding//src/main/java=UTF-8 3 | encoding//src/test/java=UTF-8 4 | encoding/=UTF-8 5 | -------------------------------------------------------------------------------- /README.md: -------------------------------------------------------------------------------- 1 | # SparkOnHBase 2 | ## Overview 3 | This is a back port of the HBase Spark Module but this has fixes to work on kerberos 4 | 5 | Documentation on how to use this code is at the following link 6 | 7 | https://hbase.apache.org/book.html#spark -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/SparkHadoopMapReduceUtilExtended.scala: -------------------------------------------------------------------------------- 1 | package org.apache.spark 2 | 3 | import org.apache.spark.mapreduce.SparkHadoopMapReduceUtil 4 | 5 | trait SparkHadoopMapReduceUtilExtended extends SparkHadoopMapReduceUtil{ 6 | 7 | } -------------------------------------------------------------------------------- /.project: -------------------------------------------------------------------------------- 1 | 2 | 3 | spark.hbase 4 | 5 | 6 | 7 | 8 | 9 | org.eclipse.m2e.core.maven2Builder 10 | 11 | 12 | 13 | 14 | org.scala-ide.sdt.core.scalabuilder 15 | 16 | 17 | 18 | 19 | 20 | org.scala-ide.sdt.core.scalanature 21 | org.eclipse.jdt.core.javanature 22 | org.eclipse.m2e.core.maven2Nature 23 | 24 | 25 | -------------------------------------------------------------------------------- /.classpath: -------------------------------------------------------------------------------- 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 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/hadoop/hbase/spark/ByteArrayWrapper.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | package org.apache.hadoop.hbase.spark 18 | 19 | import java.io.Serializable 20 | 21 | import org.apache.hadoop.hbase.util.Bytes 22 | 23 | /** 24 | * This is a wrapper over a byte array so it can work as 25 | * a key in a hashMap 26 | * 27 | * @param value The Byte Array value 28 | */ 29 | class ByteArrayWrapper (var value:Array[Byte]) 30 | extends Comparable[ByteArrayWrapper] with Serializable { 31 | override def compareTo(valueOther: ByteArrayWrapper): Int = { 32 | Bytes.compareTo(value,valueOther.value) 33 | } 34 | override def equals(o2: Any): Boolean = { 35 | o2 match { 36 | case wrapper: ByteArrayWrapper => 37 | Bytes.equals(value, wrapper.value) 38 | case _ => 39 | false 40 | } 41 | } 42 | override def hashCode():Int = { 43 | Bytes.hashCode(value) 44 | } 45 | } 46 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/hadoop/hbase/spark/ByteArrayComparable.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * 3 | * Licensed to the Apache Software Foundation (ASF) under one 4 | * or more contributor license agreements. See the NOTICE file 5 | * distributed with this work for additional information 6 | * regarding copyright ownership. The ASF licenses this file 7 | * to you under the Apache License, Version 2.0 (the 8 | * "License"); you may not use this file except in compliance 9 | * with the License. You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, software 14 | * distributed under the License is distributed on an "AS IS" BASIS, 15 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | * See the License for the specific language governing permissions and 17 | * limitations under the License. 18 | */ 19 | 20 | package org.apache.hadoop.hbase.spark 21 | 22 | import org.apache.hadoop.hbase.util.Bytes 23 | 24 | class ByteArrayComparable(val bytes:Array[Byte], val offset:Int = 0, var length:Int = -1) 25 | extends Comparable[ByteArrayComparable] { 26 | 27 | if (length == -1) { 28 | length = bytes.length 29 | } 30 | 31 | override def compareTo(o: ByteArrayComparable): Int = { 32 | Bytes.compareTo(bytes, offset, length, o.bytes, o.offset, o.length) 33 | } 34 | 35 | override def hashCode(): Int = { 36 | Bytes.hashCode(bytes, offset, length) 37 | } 38 | 39 | override def equals (obj: Any): Boolean = { 40 | obj match { 41 | case b: ByteArrayComparable => 42 | Bytes.equals(bytes, offset, length, b.bytes, b.offset, b.length) 43 | case _ => 44 | false 45 | } 46 | } 47 | } 48 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/hadoop/hbase/spark/FamilyHFileWriteOptions.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.hadoop.hbase.spark 19 | 20 | import java.io.Serializable 21 | 22 | /** 23 | * This object will hold optional data for how a given column family's 24 | * writer will work 25 | * 26 | * @param compression String to define the Compression to be used in the HFile 27 | * @param bloomType String to define the bloom type to be used in the HFile 28 | * @param blockSize The block size to be used in the HFile 29 | * @param dataBlockEncoding String to define the data block encoding to be used 30 | * in the HFile 31 | */ 32 | class FamilyHFileWriteOptions( val compression:String, 33 | val bloomType: String, 34 | val blockSize: Int, 35 | val dataBlockEncoding: String) extends Serializable 36 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/hadoop/hbase/spark/KeyFamilyQualifier.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.hadoop.hbase.spark 19 | 20 | import java.io.Serializable 21 | 22 | import org.apache.hadoop.hbase.util.Bytes 23 | 24 | /** 25 | * This is the key to be used for sorting and shuffling. 26 | * 27 | * We will only partition on the rowKey but we will sort on all three 28 | * 29 | * @param rowKey Record RowKey 30 | * @param family Record ColumnFamily 31 | * @param qualifier Cell Qualifier 32 | */ 33 | class KeyFamilyQualifier(val rowKey:Array[Byte], val family:Array[Byte], val qualifier:Array[Byte]) 34 | extends Comparable[KeyFamilyQualifier] with Serializable { 35 | override def compareTo(o: KeyFamilyQualifier): Int = { 36 | var result = Bytes.compareTo(rowKey, o.rowKey) 37 | if (result == 0) { 38 | result = Bytes.compareTo(family, o.family) 39 | if (result == 0) result = Bytes.compareTo(qualifier, o.qualifier) 40 | } 41 | result 42 | } 43 | override def toString: String = { 44 | Bytes.toString(rowKey) + ":" + Bytes.toString(family) + ":" + Bytes.toString(qualifier) 45 | } 46 | } 47 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/hadoop/hbase/spark/BulkLoadPartitioner.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.hadoop.hbase.spark 19 | 20 | import java.util 21 | import java.util.Comparator 22 | 23 | import org.apache.hadoop.hbase.util.Bytes 24 | import org.apache.spark.Partitioner 25 | 26 | /** 27 | * A Partitioner implementation that will separate records to different 28 | * HBase Regions based on region splits 29 | * 30 | * @param startKeys The start keys for the given table 31 | */ 32 | class BulkLoadPartitioner(startKeys:Array[Array[Byte]]) 33 | extends Partitioner { 34 | 35 | override def numPartitions: Int = startKeys.length 36 | 37 | override def getPartition(key: Any): Int = { 38 | 39 | val comparator: Comparator[Array[Byte]] = new Comparator[Array[Byte]] { 40 | override def compare(o1: Array[Byte], o2: Array[Byte]): Int = { 41 | Bytes.compareTo(o1, o2) 42 | } 43 | } 44 | 45 | val rowKey:Array[Byte] = 46 | key match { 47 | case qualifier: KeyFamilyQualifier => 48 | qualifier.rowKey 49 | case wrapper: ByteArrayWrapper => 50 | wrapper.value 51 | case _ => 52 | key.asInstanceOf[Array[Byte]] 53 | } 54 | val partition = util.Arrays.binarySearch(startKeys, rowKey, comparator) 55 | if (partition < 0) partition * -1 + -2 56 | else partition 57 | } 58 | } 59 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/hadoop/hbase/spark/FamiliesQualifiersValues.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | package org.apache.hadoop.hbase.spark 18 | 19 | import java.util 20 | 21 | /** 22 | * This object is a clean way to store and sort all cells that will be bulk 23 | * loaded into a single row 24 | */ 25 | class FamiliesQualifiersValues extends Serializable { 26 | //Tree maps are used because we need the results to 27 | // be sorted when we read them 28 | val familyMap = new util.TreeMap[ByteArrayWrapper, 29 | util.TreeMap[ByteArrayWrapper, Array[Byte]]]() 30 | 31 | //normally in a row there are more columns then 32 | //column families this wrapper is reused for column 33 | //family look ups 34 | val reusableWrapper = new ByteArrayWrapper(null) 35 | 36 | /** 37 | * Adds a new cell to an existing row 38 | * @param family HBase column family 39 | * @param qualifier HBase column qualifier 40 | * @param value HBase cell value 41 | */ 42 | def += (family: Array[Byte], qualifier: Array[Byte], value: Array[Byte]): Unit = { 43 | 44 | reusableWrapper.value = family 45 | 46 | var qualifierValues = familyMap.get(reusableWrapper) 47 | 48 | if (qualifierValues == null) { 49 | qualifierValues = new util.TreeMap[ByteArrayWrapper, Array[Byte]]() 50 | familyMap.put(new ByteArrayWrapper(family), qualifierValues) 51 | } 52 | 53 | qualifierValues.put(new ByteArrayWrapper(qualifier), value) 54 | } 55 | } 56 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/hadoop/hbase/spark/example/hbasecontext/HBaseDistributedScanExample.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | package org.apache.hadoop.hbase.spark.example.hbasecontext 18 | 19 | import org.apache.hadoop.hbase.spark.HBaseContext 20 | import org.apache.spark.SparkContext 21 | import org.apache.hadoop.hbase.{TableName, HBaseConfiguration} 22 | import org.apache.hadoop.hbase.util.Bytes 23 | import org.apache.hadoop.hbase.client.Scan 24 | import org.apache.spark.SparkConf 25 | /** 26 | * This is a simple example of scanning records from HBase 27 | * with the hbaseRDD function. 28 | */ 29 | object HBaseDistributedScanExample { 30 | def main(args: Array[String]) { 31 | if (args.length < 1) { 32 | println("GenerateGraphs {tableName}") 33 | return 34 | } 35 | 36 | val tableName = args(0) 37 | 38 | val sparkConf = new SparkConf().setAppName("HBaseDistributedScanExample " + tableName ) 39 | val sc = new SparkContext(sparkConf) 40 | 41 | try { 42 | val conf = HBaseConfiguration.create() 43 | 44 | val hbaseContext = new HBaseContext(sc, conf) 45 | 46 | val scan = new Scan() 47 | scan.setCaching(100) 48 | 49 | val getRdd = hbaseContext.hbaseRDD(TableName.valueOf(tableName), scan) 50 | 51 | getRdd.foreach(v => println(Bytes.toString(v._1.get()))) 52 | 53 | println("Length: " + getRdd.map(r => r._1.copyBytes()).collect().length); 54 | 55 | //.collect().foreach(v => println(Bytes.toString(v._1.get()))) 56 | } finally { 57 | sc.stop() 58 | } 59 | } 60 | 61 | } -------------------------------------------------------------------------------- /src/main/scala/org/apache/hadoop/hbase/spark/example/hbasecontext/HBaseBulkDeleteExample.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.hadoop.hbase.spark.example.hbasecontext 19 | 20 | import org.apache.hadoop.hbase.spark.HBaseContext 21 | import org.apache.spark.SparkContext 22 | import org.apache.hadoop.hbase.{TableName, HBaseConfiguration} 23 | import org.apache.hadoop.hbase.util.Bytes 24 | import org.apache.hadoop.hbase.client.Delete 25 | import org.apache.spark.SparkConf 26 | 27 | /** 28 | * This is a simple example of deleting records in HBase 29 | * with the bulkDelete function. 30 | */ 31 | object HBaseBulkDeleteExample { 32 | def main(args: Array[String]) { 33 | if (args.length < 1) { 34 | println("HBaseBulkDeletesExample {tableName} ") 35 | return 36 | } 37 | 38 | val tableName = args(0) 39 | 40 | val sparkConf = new SparkConf().setAppName("HBaseBulkDeleteExample " + tableName) 41 | val sc = new SparkContext(sparkConf) 42 | try { 43 | //[Array[Byte]] 44 | val rdd = sc.parallelize(Array( 45 | Bytes.toBytes("1"), 46 | Bytes.toBytes("2"), 47 | Bytes.toBytes("3"), 48 | Bytes.toBytes("4"), 49 | Bytes.toBytes("5") 50 | )) 51 | 52 | val conf = HBaseConfiguration.create() 53 | 54 | val hbaseContext = new HBaseContext(sc, conf) 55 | hbaseContext.bulkDelete[Array[Byte]](rdd, 56 | TableName.valueOf(tableName), 57 | putRecord => new Delete(putRecord), 58 | 4) 59 | } finally { 60 | sc.stop() 61 | } 62 | } 63 | } -------------------------------------------------------------------------------- /src/main/scala/org/apache/hadoop/hbase/spark/example/rdd/HBaseBulkDeleteExample.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | package org.apache.hadoop.hbase.spark.example.rdd 18 | 19 | import org.apache.hadoop.hbase.client.Delete 20 | import org.apache.hadoop.hbase.{TableName, HBaseConfiguration} 21 | import org.apache.hadoop.hbase.spark.HBaseContext 22 | import org.apache.hadoop.hbase.spark.HBaseRDDFunctions._ 23 | import org.apache.hadoop.hbase.util.Bytes 24 | 25 | import org.apache.spark.{SparkContext, SparkConf} 26 | 27 | /** 28 | * This is a simple example of deleting records in HBase 29 | * with the bulkDelete function. 30 | */ 31 | object HBaseBulkDeleteExample { 32 | def main(args: Array[String]) { 33 | if (args.length < 1) { 34 | println("HBaseBulkDeletesExample {tableName} ") 35 | return 36 | } 37 | 38 | val tableName = args(0) 39 | 40 | val sparkConf = new SparkConf().setAppName("HBaseBulkDeleteExample " + tableName) 41 | val sc = new SparkContext(sparkConf) 42 | try { 43 | //[Array[Byte]] 44 | val rdd = sc.parallelize(Array( 45 | Bytes.toBytes("1"), 46 | Bytes.toBytes("2"), 47 | Bytes.toBytes("3"), 48 | Bytes.toBytes("4"), 49 | Bytes.toBytes("5") 50 | )) 51 | 52 | val conf = HBaseConfiguration.create() 53 | 54 | val hbaseContext = new HBaseContext(sc, conf) 55 | 56 | rdd.hbaseBulkDelete(hbaseContext, TableName.valueOf(tableName), 57 | putRecord => new Delete(putRecord), 58 | 4) 59 | 60 | } finally { 61 | sc.stop() 62 | } 63 | } 64 | } -------------------------------------------------------------------------------- /src/main/scala/org/apache/hadoop/hbase/spark/example/hbasecontext/HBaseStreamingBulkPutExample.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.hadoop.hbase.spark.example.hbasecontext 19 | 20 | import org.apache.hadoop.hbase.spark.HBaseContext 21 | import org.apache.spark.SparkContext 22 | import org.apache.hadoop.hbase.{TableName, HBaseConfiguration} 23 | import org.apache.hadoop.hbase.util.Bytes 24 | import org.apache.hadoop.hbase.client.Put 25 | import org.apache.spark.streaming.StreamingContext 26 | import org.apache.spark.streaming.Seconds 27 | import org.apache.spark.SparkConf 28 | 29 | /** 30 | * This is a simple example of BulkPut with Spark Streaming 31 | */ 32 | object HBaseStreamingBulkPutExample { 33 | def main(args: Array[String]) { 34 | if (args.length < 4) { 35 | println("HBaseStreamingBulkPutExample " + 36 | "{host} {port} {tableName} {columnFamily}") 37 | return 38 | } 39 | 40 | val host = args(0) 41 | val port = args(1) 42 | val tableName = args(2) 43 | val columnFamily = args(3) 44 | 45 | val sparkConf = new SparkConf().setAppName("HBaseBulkPutTimestampExample " + 46 | tableName + " " + columnFamily) 47 | val sc = new SparkContext(sparkConf) 48 | try { 49 | val ssc = new StreamingContext(sc, Seconds(1)) 50 | 51 | val lines = ssc.socketTextStream(host, port.toInt) 52 | 53 | val conf = HBaseConfiguration.create() 54 | 55 | val hbaseContext = new HBaseContext(sc, conf) 56 | 57 | hbaseContext.streamBulkPut[String](lines, 58 | TableName.valueOf(tableName), 59 | (putRecord) => { 60 | if (putRecord.length() > 0) { 61 | val put = new Put(Bytes.toBytes(putRecord)) 62 | put.addColumn(Bytes.toBytes("c"), Bytes.toBytes("foo"), Bytes.toBytes("bar")) 63 | put 64 | } else { 65 | null 66 | } 67 | }) 68 | ssc.start() 69 | ssc.awaitTerminationOrTimeout(60000) 70 | } finally { 71 | sc.stop() 72 | } 73 | } 74 | } -------------------------------------------------------------------------------- /src/main/scala/org/apache/hadoop/hbase/spark/example/hbasecontext/HBaseBulkPutExampleFromFile.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.hadoop.hbase.spark.example.hbasecontext 19 | 20 | import org.apache.hadoop.hbase.spark.HBaseContext 21 | import org.apache.spark.SparkContext 22 | import org.apache.hadoop.hbase.{TableName, HBaseConfiguration} 23 | import org.apache.hadoop.hbase.util.Bytes 24 | import org.apache.hadoop.hbase.client.Put 25 | import org.apache.hadoop.mapred.TextInputFormat 26 | import org.apache.hadoop.io.LongWritable 27 | import org.apache.hadoop.io.Text 28 | import org.apache.spark.SparkConf 29 | 30 | /** 31 | * This is a simple example of putting records in HBase 32 | * with the bulkPut function. In this example we are 33 | * getting the put information from a file 34 | */ 35 | object HBaseBulkPutExampleFromFile { 36 | def main(args: Array[String]) { 37 | if (args.length < 3) { 38 | println("HBaseBulkPutExampleFromFile {tableName} {columnFamily} {inputFile}") 39 | return 40 | } 41 | 42 | val tableName = args(0) 43 | val columnFamily = args(1) 44 | val inputFile = args(2) 45 | 46 | val sparkConf = new SparkConf().setAppName("HBaseBulkPutExampleFromFile " + 47 | tableName + " " + columnFamily + " " + inputFile) 48 | val sc = new SparkContext(sparkConf) 49 | 50 | try { 51 | var rdd = sc.hadoopFile( 52 | inputFile, 53 | classOf[TextInputFormat], 54 | classOf[LongWritable], 55 | classOf[Text]).map(v => { 56 | System.out.println("reading-" + v._2.toString) 57 | v._2.toString 58 | }) 59 | 60 | val conf = HBaseConfiguration.create() 61 | 62 | val hbaseContext = new HBaseContext(sc, conf) 63 | hbaseContext.bulkPut[String](rdd, 64 | TableName.valueOf(tableName), 65 | (putRecord) => { 66 | System.out.println("hbase-" + putRecord) 67 | val put = new Put(Bytes.toBytes("Value- " + putRecord)) 68 | put.addColumn(Bytes.toBytes("c"), Bytes.toBytes("1"), 69 | Bytes.toBytes(putRecord.length())) 70 | put 71 | }); 72 | } finally { 73 | sc.stop() 74 | } 75 | } 76 | } 77 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/hadoop/hbase/spark/example/rdd/HBaseBulkPutExample.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.hadoop.hbase.spark.example.rdd 19 | 20 | import org.apache.hadoop.hbase.client.Put 21 | import org.apache.hadoop.hbase.spark.HBaseContext 22 | import org.apache.hadoop.hbase.spark.HBaseRDDFunctions._ 23 | import org.apache.hadoop.hbase.util.Bytes 24 | import org.apache.hadoop.hbase.{HBaseConfiguration, TableName} 25 | import org.apache.spark.{SparkConf, SparkContext} 26 | 27 | /** 28 | * This is a simple example of putting records in HBase 29 | * with the bulkPut function. 30 | */ 31 | object HBaseBulkPutExample { 32 | def main(args: Array[String]) { 33 | if (args.length < 2) { 34 | println("HBaseBulkPutExample {tableName} {columnFamily}") 35 | return 36 | } 37 | 38 | val tableName = args(0) 39 | val columnFamily = args(1) 40 | 41 | val sparkConf = new SparkConf().setAppName("HBaseBulkPutExample " + 42 | tableName + " " + columnFamily) 43 | val sc = new SparkContext(sparkConf) 44 | 45 | try { 46 | //[(Array[Byte], Array[(Array[Byte], Array[Byte], Array[Byte])])] 47 | val rdd = sc.parallelize(Array( 48 | (Bytes.toBytes("1"), 49 | Array((Bytes.toBytes(columnFamily), Bytes.toBytes("1"), Bytes.toBytes("1")))), 50 | (Bytes.toBytes("2"), 51 | Array((Bytes.toBytes(columnFamily), Bytes.toBytes("1"), Bytes.toBytes("2")))), 52 | (Bytes.toBytes("3"), 53 | Array((Bytes.toBytes(columnFamily), Bytes.toBytes("1"), Bytes.toBytes("3")))), 54 | (Bytes.toBytes("4"), 55 | Array((Bytes.toBytes(columnFamily), Bytes.toBytes("1"), Bytes.toBytes("4")))), 56 | (Bytes.toBytes("5"), 57 | Array((Bytes.toBytes(columnFamily), Bytes.toBytes("1"), Bytes.toBytes("5")))) 58 | )) 59 | 60 | val conf = HBaseConfiguration.create() 61 | 62 | val hbaseContext = new HBaseContext(sc, conf) 63 | 64 | rdd.hbaseBulkPut(hbaseContext, TableName.valueOf(tableName), 65 | (putRecord) => { 66 | val put = new Put(putRecord._1) 67 | putRecord._2.foreach((putValue) => put.addColumn(putValue._1, putValue._2, 68 | putValue._3)) 69 | put 70 | }) 71 | 72 | } finally { 73 | sc.stop() 74 | } 75 | } 76 | } -------------------------------------------------------------------------------- /src/main/scala/org/apache/hadoop/hbase/spark/example/hbasecontext/HBaseBulkPutExample.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.hadoop.hbase.spark.example.hbasecontext 19 | 20 | import org.apache.hadoop.hbase.spark.HBaseContext 21 | import org.apache.spark.SparkContext 22 | import org.apache.hadoop.hbase.{TableName, HBaseConfiguration} 23 | import org.apache.hadoop.hbase.util.Bytes 24 | import org.apache.hadoop.hbase.client.Put 25 | import org.apache.spark.SparkConf 26 | 27 | /** 28 | * This is a simple example of putting records in HBase 29 | * with the bulkPut function. 30 | */ 31 | object HBaseBulkPutExample { 32 | def main(args: Array[String]) { 33 | if (args.length < 2) { 34 | println("HBaseBulkPutExample {tableName} {columnFamily}") 35 | return 36 | } 37 | 38 | val tableName = args(0) 39 | val columnFamily = args(1) 40 | 41 | val sparkConf = new SparkConf().setAppName("HBaseBulkPutExample " + 42 | tableName + " " + columnFamily) 43 | val sc = new SparkContext(sparkConf) 44 | 45 | try { 46 | //[(Array[Byte], Array[(Array[Byte], Array[Byte], Array[Byte])])] 47 | val rdd = sc.parallelize(Array( 48 | (Bytes.toBytes("1"), 49 | Array((Bytes.toBytes(columnFamily), Bytes.toBytes("1"), Bytes.toBytes("1")))), 50 | (Bytes.toBytes("2"), 51 | Array((Bytes.toBytes(columnFamily), Bytes.toBytes("1"), Bytes.toBytes("2")))), 52 | (Bytes.toBytes("3"), 53 | Array((Bytes.toBytes(columnFamily), Bytes.toBytes("1"), Bytes.toBytes("3")))), 54 | (Bytes.toBytes("4"), 55 | Array((Bytes.toBytes(columnFamily), Bytes.toBytes("1"), Bytes.toBytes("4")))), 56 | (Bytes.toBytes("5"), 57 | Array((Bytes.toBytes(columnFamily), Bytes.toBytes("1"), Bytes.toBytes("5")))) 58 | )) 59 | 60 | val conf = HBaseConfiguration.create() 61 | 62 | val hbaseContext = new HBaseContext(sc, conf) 63 | hbaseContext.bulkPut[(Array[Byte], Array[(Array[Byte], Array[Byte], Array[Byte])])](rdd, 64 | TableName.valueOf(tableName), 65 | (putRecord) => { 66 | val put = new Put(putRecord._1) 67 | putRecord._2.foreach((putValue) => 68 | put.addColumn(putValue._1, putValue._2, putValue._3)) 69 | put 70 | }); 71 | } finally { 72 | sc.stop() 73 | } 74 | } 75 | } -------------------------------------------------------------------------------- /src/main/scala/org/apache/hadoop/hbase/spark/example/rdd/HBaseBulkGetExample.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | package org.apache.hadoop.hbase.spark.example.rdd 18 | 19 | import org.apache.hadoop.hbase.client.{Result, Get} 20 | import org.apache.hadoop.hbase.{CellUtil, TableName, HBaseConfiguration} 21 | import org.apache.hadoop.hbase.spark.HBaseContext 22 | import org.apache.hadoop.hbase.util.Bytes 23 | import org.apache.hadoop.hbase.spark.HBaseRDDFunctions._ 24 | import org.apache.spark.{SparkContext, SparkConf} 25 | 26 | /** 27 | * This is a simple example of getting records in HBase 28 | * with the bulkGet function. 29 | */ 30 | object HBaseBulkGetExample { 31 | def main(args: Array[String]) { 32 | if (args.length < 1) { 33 | println("HBaseBulkGetExample {tableName}") 34 | return 35 | } 36 | 37 | val tableName = args(0) 38 | 39 | val sparkConf = new SparkConf().setAppName("HBaseBulkGetExample " + tableName) 40 | val sc = new SparkContext(sparkConf) 41 | 42 | try { 43 | 44 | //[(Array[Byte])] 45 | val rdd = sc.parallelize(Array( 46 | Bytes.toBytes("1"), 47 | Bytes.toBytes("2"), 48 | Bytes.toBytes("3"), 49 | Bytes.toBytes("4"), 50 | Bytes.toBytes("5"), 51 | Bytes.toBytes("6"), 52 | Bytes.toBytes("7"))) 53 | 54 | val conf = HBaseConfiguration.create() 55 | 56 | val hbaseContext = new HBaseContext(sc, conf) 57 | 58 | val getRdd = rdd.hbaseBulkGet[String](hbaseContext, TableName.valueOf(tableName), 2, 59 | record => { 60 | System.out.println("making Get") 61 | new Get(record) 62 | }, 63 | (result: Result) => { 64 | 65 | val it = result.listCells().iterator() 66 | val b = new StringBuilder 67 | 68 | b.append(Bytes.toString(result.getRow) + ":") 69 | 70 | while (it.hasNext) { 71 | val cell = it.next() 72 | val q = Bytes.toString(CellUtil.cloneQualifier(cell)) 73 | if (q.equals("counter")) { 74 | b.append("(" + q + "," + Bytes.toLong(CellUtil.cloneValue(cell)) + ")") 75 | } else { 76 | b.append("(" + q + "," + Bytes.toString(CellUtil.cloneValue(cell)) + ")") 77 | } 78 | } 79 | b.toString() 80 | }) 81 | 82 | getRdd.collect().foreach(v => println(v)) 83 | 84 | } finally { 85 | sc.stop() 86 | } 87 | } 88 | } 89 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/hadoop/hbase/spark/example/hbasecontext/HBaseBulkPutTimestampExample.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.hadoop.hbase.spark.example.hbasecontext 19 | 20 | import org.apache.hadoop.hbase.spark.HBaseContext 21 | import org.apache.spark.SparkContext 22 | import org.apache.hadoop.hbase.{TableName, HBaseConfiguration} 23 | import org.apache.hadoop.hbase.util.Bytes 24 | import org.apache.hadoop.hbase.client.Put 25 | import org.apache.spark.SparkConf 26 | 27 | /** 28 | * This is a simple example of putting records in HBase 29 | * with the bulkPut function. In this example we are 30 | * also setting the timestamp in the put 31 | */ 32 | object HBaseBulkPutTimestampExample { 33 | def main(args: Array[String]) { 34 | if (args.length < 2) { 35 | System.out.println("HBaseBulkPutTimestampExample {tableName} {columnFamily}") 36 | return 37 | } 38 | 39 | val tableName = args(0) 40 | val columnFamily = args(1) 41 | 42 | val sparkConf = new SparkConf().setAppName("HBaseBulkPutTimestampExample " + 43 | tableName + " " + columnFamily) 44 | val sc = new SparkContext(sparkConf) 45 | 46 | try { 47 | 48 | val rdd = sc.parallelize(Array( 49 | (Bytes.toBytes("6"), 50 | Array((Bytes.toBytes(columnFamily), Bytes.toBytes("1"), Bytes.toBytes("1")))), 51 | (Bytes.toBytes("7"), 52 | Array((Bytes.toBytes(columnFamily), Bytes.toBytes("1"), Bytes.toBytes("2")))), 53 | (Bytes.toBytes("8"), 54 | Array((Bytes.toBytes(columnFamily), Bytes.toBytes("1"), Bytes.toBytes("3")))), 55 | (Bytes.toBytes("9"), 56 | Array((Bytes.toBytes(columnFamily), Bytes.toBytes("1"), Bytes.toBytes("4")))), 57 | (Bytes.toBytes("10"), 58 | Array((Bytes.toBytes(columnFamily), Bytes.toBytes("1"), Bytes.toBytes("5")))))) 59 | 60 | val conf = HBaseConfiguration.create() 61 | 62 | val timeStamp = System.currentTimeMillis() 63 | 64 | val hbaseContext = new HBaseContext(sc, conf) 65 | hbaseContext.bulkPut[(Array[Byte], Array[(Array[Byte], Array[Byte], Array[Byte])])](rdd, 66 | TableName.valueOf(tableName), 67 | (putRecord) => { 68 | val put = new Put(putRecord._1) 69 | putRecord._2.foreach((putValue) => put.addColumn(putValue._1, putValue._2, 70 | timeStamp, putValue._3)) 71 | put 72 | }) 73 | } finally { 74 | sc.stop() 75 | } 76 | } 77 | } 78 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/hadoop/hbase/spark/example/rdd/HBaseMapPartitionExample.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.hadoop.hbase.spark.example.rdd 19 | 20 | import org.apache.hadoop.hbase.client.Get 21 | import org.apache.hadoop.hbase.{TableName, HBaseConfiguration} 22 | import org.apache.hadoop.hbase.spark.HBaseContext 23 | import org.apache.hadoop.hbase.spark.HBaseRDDFunctions._ 24 | import org.apache.hadoop.hbase.util.Bytes 25 | import org.apache.spark.{SparkContext, SparkConf} 26 | 27 | /** 28 | * This is a simple example of using the mapPartitions 29 | * method with a HBase connection 30 | */ 31 | object HBaseMapPartitionExample { 32 | def main(args: Array[String]) { 33 | if (args.length < 1) { 34 | println("HBaseBulkGetExample {tableName}") 35 | return 36 | } 37 | 38 | val tableName = args(0) 39 | 40 | val sparkConf = new SparkConf().setAppName("HBaseBulkGetExample " + tableName) 41 | val sc = new SparkContext(sparkConf) 42 | 43 | try { 44 | 45 | //[(Array[Byte])] 46 | val rdd = sc.parallelize(Array( 47 | Bytes.toBytes("1"), 48 | Bytes.toBytes("2"), 49 | Bytes.toBytes("3"), 50 | Bytes.toBytes("4"), 51 | Bytes.toBytes("5"), 52 | Bytes.toBytes("6"), 53 | Bytes.toBytes("7"))) 54 | 55 | val conf = HBaseConfiguration.create() 56 | 57 | val hbaseContext = new HBaseContext(sc, conf) 58 | 59 | val getRdd = rdd.hbaseMapPartitions[String](hbaseContext, (it, connection) => { 60 | val table = connection.getTable(TableName.valueOf(tableName)) 61 | it.map{r => 62 | //batching would be faster. This is just an example 63 | val result = table.get(new Get(r)) 64 | val it = result.listCells().iterator() 65 | val b = new StringBuilder 66 | b.append(Bytes.toString(result.getRow) + ":") 67 | while (it.hasNext) { 68 | val cell = it.next() 69 | val q = Bytes.toString(cell.getQualifierArray) 70 | if (q.equals("counter")) { 71 | b.append("(" + q + "," + Bytes.toLong(cell.getValueArray) + ")") 72 | } else { 73 | b.append("(" + q + "," + Bytes.toString(cell.getValueArray) + ")") 74 | } 75 | } 76 | b.toString() 77 | } 78 | }) 79 | 80 | getRdd.collect().foreach(v => println(v)) 81 | 82 | } finally { 83 | sc.stop() 84 | } 85 | } 86 | } 87 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/hadoop/hbase/spark/example/hbasecontext/HBaseBulkGetExample.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.hadoop.hbase.spark.example.hbasecontext 19 | 20 | import org.apache.hadoop.hbase.spark.HBaseContext 21 | import org.apache.spark.SparkContext 22 | import org.apache.hadoop.hbase.{CellUtil, TableName, HBaseConfiguration} 23 | import org.apache.hadoop.hbase.util.Bytes 24 | import org.apache.hadoop.hbase.client.Get 25 | import org.apache.hadoop.hbase.client.Result 26 | import org.apache.spark.SparkConf 27 | 28 | /** 29 | * This is a simple example of getting records in HBase 30 | * with the bulkGet function. 31 | */ 32 | object HBaseBulkGetExample { 33 | def main(args: Array[String]) { 34 | if (args.length < 1) { 35 | println("HBaseBulkGetExample {tableName}") 36 | return 37 | } 38 | 39 | val tableName = args(0) 40 | 41 | val sparkConf = new SparkConf().setAppName("HBaseBulkGetExample " + tableName) 42 | val sc = new SparkContext(sparkConf) 43 | 44 | try { 45 | 46 | //[(Array[Byte])] 47 | val rdd = sc.parallelize(Array( 48 | Bytes.toBytes("1"), 49 | Bytes.toBytes("2"), 50 | Bytes.toBytes("3"), 51 | Bytes.toBytes("4"), 52 | Bytes.toBytes("5"), 53 | Bytes.toBytes("6"), 54 | Bytes.toBytes("7"))) 55 | 56 | val conf = HBaseConfiguration.create() 57 | 58 | val hbaseContext = new HBaseContext(sc, conf) 59 | 60 | val getRdd = hbaseContext.bulkGet[Array[Byte], String]( 61 | TableName.valueOf(tableName), 62 | 2, 63 | rdd, 64 | record => { 65 | System.out.println("making Get") 66 | new Get(record) 67 | }, 68 | (result: Result) => { 69 | 70 | val it = result.listCells().iterator() 71 | val b = new StringBuilder 72 | 73 | b.append(Bytes.toString(result.getRow) + ":") 74 | 75 | while (it.hasNext) { 76 | val cell = it.next() 77 | val q = Bytes.toString(CellUtil.cloneQualifier(cell)) 78 | if (q.equals("counter")) { 79 | b.append("(" + q + "," + Bytes.toLong(CellUtil.cloneValue(cell)) + ")") 80 | } else { 81 | b.append("(" + q + "," + Bytes.toString(CellUtil.cloneValue(cell)) + ")") 82 | } 83 | } 84 | b.toString() 85 | }) 86 | 87 | getRdd.collect().foreach(v => println(v)) 88 | 89 | } finally { 90 | sc.stop() 91 | } 92 | } 93 | } 94 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/hadoop/hbase/spark/example/rdd/HBaseForeachPartitionExample.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.hadoop.hbase.spark.example.rdd 19 | 20 | import org.apache.hadoop.hbase.client.Put 21 | import org.apache.hadoop.hbase.{TableName, HBaseConfiguration} 22 | import org.apache.hadoop.hbase.spark.HBaseContext 23 | import org.apache.hadoop.hbase.spark.HBaseRDDFunctions._ 24 | import org.apache.hadoop.hbase.util.Bytes 25 | import org.apache.spark.{SparkContext, SparkConf} 26 | 27 | /** 28 | * This is a simple example of using the foreachPartition 29 | * method with a HBase connection 30 | */ 31 | object HBaseForeachPartitionExample { 32 | def main(args: Array[String]) { 33 | if (args.length < 2) { 34 | println("HBaseBulkPutExample {tableName} {columnFamily}") 35 | return 36 | } 37 | 38 | val tableName = args(0) 39 | val columnFamily = args(1) 40 | 41 | val sparkConf = new SparkConf().setAppName("HBaseBulkPutExample " + 42 | tableName + " " + columnFamily) 43 | val sc = new SparkContext(sparkConf) 44 | 45 | try { 46 | //[(Array[Byte], Array[(Array[Byte], Array[Byte], Array[Byte])])] 47 | val rdd = sc.parallelize(Array( 48 | (Bytes.toBytes("1"), 49 | Array((Bytes.toBytes(columnFamily), Bytes.toBytes("1"), Bytes.toBytes("1")))), 50 | (Bytes.toBytes("2"), 51 | Array((Bytes.toBytes(columnFamily), Bytes.toBytes("1"), Bytes.toBytes("2")))), 52 | (Bytes.toBytes("3"), 53 | Array((Bytes.toBytes(columnFamily), Bytes.toBytes("1"), Bytes.toBytes("3")))), 54 | (Bytes.toBytes("4"), 55 | Array((Bytes.toBytes(columnFamily), Bytes.toBytes("1"), Bytes.toBytes("4")))), 56 | (Bytes.toBytes("5"), 57 | Array((Bytes.toBytes(columnFamily), Bytes.toBytes("1"), Bytes.toBytes("5")))) 58 | )) 59 | 60 | val conf = HBaseConfiguration.create() 61 | 62 | val hbaseContext = new HBaseContext(sc, conf) 63 | 64 | 65 | rdd.hbaseForeachPartition(hbaseContext, 66 | (it, connection) => { 67 | val m = connection.getBufferedMutator(TableName.valueOf(tableName)) 68 | 69 | it.foreach(r => { 70 | val put = new Put(r._1) 71 | r._2.foreach((putValue) => 72 | put.addColumn(putValue._1, putValue._2, putValue._3)) 73 | m.mutate(put) 74 | }) 75 | m.flush() 76 | m.close() 77 | }) 78 | 79 | } finally { 80 | sc.stop() 81 | } 82 | } 83 | } 84 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/hadoop/hbase/spark/ColumnFamilyQualifierMapKeyWrapper.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.hadoop.hbase.spark 19 | 20 | import org.apache.hadoop.hbase.util.Bytes 21 | 22 | /** 23 | * A wrapper class that will allow both columnFamily and qualifier to 24 | * be the key of a hashMap. Also allow for finding the value in a hashmap 25 | * with out cloning the HBase value from the HBase Cell object 26 | * @param columnFamily ColumnFamily byte array 27 | * @param columnFamilyOffSet Offset of columnFamily value in the array 28 | * @param columnFamilyLength Length of the columnFamily value in the columnFamily array 29 | * @param qualifier Qualifier byte array 30 | * @param qualifierOffSet Offset of qualifier value in the array 31 | * @param qualifierLength Length of the qualifier value with in the array 32 | */ 33 | class ColumnFamilyQualifierMapKeyWrapper(val columnFamily:Array[Byte], 34 | val columnFamilyOffSet:Int, 35 | val columnFamilyLength:Int, 36 | val qualifier:Array[Byte], 37 | val qualifierOffSet:Int, 38 | val qualifierLength:Int) 39 | extends Serializable{ 40 | 41 | override def equals(other:Any): Boolean = { 42 | val otherWrapper = other.asInstanceOf[ColumnFamilyQualifierMapKeyWrapper] 43 | 44 | Bytes.compareTo(columnFamily, 45 | columnFamilyOffSet, 46 | columnFamilyLength, 47 | otherWrapper.columnFamily, 48 | otherWrapper.columnFamilyOffSet, 49 | otherWrapper.columnFamilyLength) == 0 && Bytes.compareTo(qualifier, 50 | qualifierOffSet, 51 | qualifierLength, 52 | otherWrapper.qualifier, 53 | otherWrapper.qualifierOffSet, 54 | otherWrapper.qualifierLength) == 0 55 | } 56 | 57 | override def hashCode():Int = { 58 | Bytes.hashCode(columnFamily, columnFamilyOffSet, columnFamilyLength) + 59 | Bytes.hashCode(qualifier, qualifierOffSet, qualifierLength) 60 | } 61 | 62 | def cloneColumnFamily():Array[Byte] = { 63 | val resultArray = new Array[Byte](columnFamilyLength) 64 | System.arraycopy(columnFamily, columnFamilyOffSet, resultArray, 0, columnFamilyLength) 65 | resultArray 66 | } 67 | 68 | def cloneQualifier():Array[Byte] = { 69 | val resultArray = new Array[Byte](qualifierLength) 70 | System.arraycopy(qualifier, qualifierOffSet, resultArray, 0, qualifierLength) 71 | resultArray 72 | } 73 | } 74 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/hadoop/hbase/spark/HBaseScanRDD.scala: -------------------------------------------------------------------------------- 1 | package org.apache.hadoop.hbase.spark 2 | 3 | import org.apache.hadoop.hbase.io.ImmutableBytesWritable 4 | import org.apache.spark.deploy.SparkHadoopUtil 5 | import org.apache.spark.{ SparkContext, TaskContext } 6 | import org.apache.spark.broadcast.Broadcast 7 | import org.apache.spark.SerializableWritable 8 | import org.apache.hadoop.conf.Configuration 9 | import org.apache.hadoop.security.Credentials 10 | import org.apache.spark.rdd.RDD 11 | import org.apache.spark.Partition 12 | import org.apache.spark.InterruptibleIterator 13 | import org.apache.hadoop.hbase.mapreduce.TableInputFormat 14 | import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil 15 | import org.apache.hadoop.hbase.client.{Result, Scan} 16 | import org.apache.hadoop.mapreduce.Job 17 | import org.apache.spark.SparkHadoopMapReduceUtilExtended 18 | import org.apache.spark.Logging 19 | import org.apache.hadoop.mapreduce.JobID 20 | import org.apache.hadoop.io.Writable 21 | import org.apache.hadoop.mapreduce.InputSplit 22 | import java.text.SimpleDateFormat 23 | import java.util.Date 24 | import java.util.ArrayList 25 | import org.apache.hadoop.security.UserGroupInformation 26 | import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod 27 | import org.apache.hadoop.hbase.mapreduce.IdentityTableMapper 28 | 29 | class HBaseScanRDD (@transient sc: SparkContext, 30 | @transient tableName: String, 31 | @transient scan: Scan, 32 | val configBroadcast: Broadcast[SerializableWritable[Configuration]], 33 | val credentialsConf: Broadcast[SerializableWritable[Credentials]]) 34 | extends RDD[(ImmutableBytesWritable, Result)](sc, Nil) 35 | with SparkHadoopMapReduceUtilExtended 36 | with Logging { 37 | 38 | @transient var appliedCredentials = false 39 | 40 | /// 41 | @transient val jobTransient = new Job(configBroadcast.value.value, "ExampleRead"); 42 | TableMapReduceUtil.initTableMapperJob( 43 | tableName, // input HBase table name 44 | scan, // Scan instance to control CF and attribute selection 45 | classOf[IdentityTableMapper], // mapper 46 | null, // mapper output key 47 | null, // mapper output value 48 | jobTransient); 49 | 50 | @transient val jobConfigurationTrans = jobTransient.getConfiguration() 51 | jobConfigurationTrans.set(TableInputFormat.INPUT_TABLE, tableName) 52 | val jobConfigBroadcast = sc.broadcast(new SerializableWritable(jobConfigurationTrans)) 53 | //// 54 | 55 | private val jobTrackerId: String = { 56 | val formatter = new SimpleDateFormat("yyyyMMddHHmm") 57 | formatter.format(new Date()) 58 | } 59 | 60 | @transient protected val jobId = new JobID(jobTrackerId, id) 61 | 62 | override def getPartitions: Array[Partition] = { 63 | 64 | addCreds 65 | 66 | val tableInputFormat = new TableInputFormat 67 | tableInputFormat.setConf(jobConfigBroadcast.value.value) 68 | 69 | val jobContext = newJobContext(jobConfigBroadcast.value.value, jobId) 70 | val rawSplits = tableInputFormat.getSplits(jobContext).toArray 71 | val result = new Array[Partition](rawSplits.size) 72 | for (i <- 0 until rawSplits.size) { 73 | result(i) = new NewHadoopPartition(id, i, rawSplits(i).asInstanceOf[InputSplit with Writable]) 74 | } 75 | 76 | result 77 | } 78 | 79 | override def compute(theSplit: Partition, context: TaskContext): InterruptibleIterator[(ImmutableBytesWritable, Result)] = { 80 | 81 | addCreds 82 | applyCreds 83 | 84 | val iter = new Iterator[(ImmutableBytesWritable, Result)] { 85 | 86 | addCreds 87 | applyCreds 88 | 89 | val split = theSplit.asInstanceOf[NewHadoopPartition] 90 | logInfo("Input split: " + split.serializableHadoopSplit) 91 | val conf = jobConfigBroadcast.value.value 92 | 93 | val attemptId = newTaskAttemptID(jobTrackerId, id, isMap = true, split.index, 0) 94 | val hadoopAttemptContext = newTaskAttemptContext(conf, attemptId) 95 | val format = new TableInputFormat 96 | format.setConf(conf) 97 | 98 | val reader = format.createRecordReader( 99 | split.serializableHadoopSplit.value, hadoopAttemptContext) 100 | reader.initialize(split.serializableHadoopSplit.value, hadoopAttemptContext) 101 | 102 | // Register an on-task-completion callback to close the input stream. 103 | context.addOnCompleteCallback(() => close()) 104 | var havePair = false 105 | var finished = false 106 | 107 | override def hasNext: Boolean = { 108 | if (!finished && !havePair) { 109 | finished = !reader.nextKeyValue 110 | havePair = !finished 111 | } 112 | !finished 113 | } 114 | 115 | override def next(): (ImmutableBytesWritable, Result) = { 116 | if (!hasNext) { 117 | throw new java.util.NoSuchElementException("End of stream") 118 | } 119 | havePair = false 120 | 121 | val writableKey = new ImmutableBytesWritable(reader.getCurrentKey.copyBytes()) 122 | 123 | (writableKey, reader.getCurrentValue) 124 | } 125 | 126 | private def close() { 127 | try { 128 | reader.close() 129 | } catch { 130 | case e: Exception => logWarning("Exception in RecordReader.close()", e) 131 | } 132 | } 133 | } 134 | new InterruptibleIterator(context, iter) 135 | } 136 | 137 | def addCreds { 138 | val creds = SparkHadoopUtil.get.getCurrentUserCredentials() 139 | 140 | val ugi = UserGroupInformation.getCurrentUser() 141 | ugi.addCredentials(creds) 142 | // specify that this is a proxy user 143 | ugi.setAuthenticationMethod(AuthenticationMethod.PROXY) 144 | } 145 | 146 | def applyCreds[T]{ 147 | val credentials = SparkHadoopUtil.get.getCurrentUserCredentials() 148 | 149 | if (!appliedCredentials && credentials != null) { 150 | appliedCredentials = true 151 | 152 | @transient val ugi = UserGroupInformation.getCurrentUser 153 | ugi.addCredentials(credentials) 154 | // specify that this is a proxy user 155 | ugi.setAuthenticationMethod(AuthenticationMethod.PROXY) 156 | 157 | ugi.addCredentials(credentialsConf.value.value) 158 | } 159 | } 160 | 161 | private[spark] class NewHadoopPartition( 162 | rddId: Int, 163 | val index: Int, 164 | @transient rawSplit: InputSplit with Writable) 165 | extends Partition { 166 | 167 | val serializableHadoopSplit = new SerializableWritable(rawSplit) 168 | 169 | override def hashCode(): Int = 41 * (41 + rddId) + index 170 | } 171 | } 172 | -------------------------------------------------------------------------------- /pom.unittest.but.no.cluster.xml: -------------------------------------------------------------------------------- 1 | 3 | 4.0.0 4 | 5 | com.cloudera.sa 6 | spark.hbase 7 | 0.0.1-SNAPSHOT 8 | jar 9 | 10 | spark.hbase 11 | http://maven.apache.org 12 | 13 | 14 | UTF-8 15 | 16 | 17 | 18 | 19 | org.scala-lang 20 | scala-compiler 21 | 2.10.4 22 | 23 | 24 | org.scalatest 25 | scalatest_2.10 26 | 2.1.5 27 | 28 | 29 | org.apache.spark 30 | spark-core_2.10 31 | 1.0.0-cdh5.1.0 32 | 33 | 34 | org.apache.spark 35 | spark-streaming_2.10 36 | 1.0.0-cdh5.1.0 37 | test-jar 38 | tests 39 | test 40 | 41 | 42 | org.apache.spark 43 | spark-streaming_2.10 44 | 1.0.0-cdh5.1.0 45 | 46 | 47 | org.apache.hbase 48 | hbase-client 49 | 0.98.1-cdh5.1.0 50 | 51 | 52 | org.apache.hbase 53 | hbase-client 54 | 0.98.1-cdh5.1.0 55 | test-jar 56 | tests 57 | test 58 | 59 | 60 | org.apache.hbase 61 | hbase-server 62 | 0.98.1-cdh5.1.0 63 | 64 | 65 | org.apache.hbase 66 | hbase-server 67 | 0.98.1-cdh5.1.0 68 | test-jar 69 | tests 70 | 71 | 72 | org.apache.hbase 73 | hbase-hadoop2-compat 74 | 0.98.1-cdh5.1.0 75 | runtime 76 | 77 | 78 | org.apache.hbase 79 | hbase-hadoop2-compat 80 | 0.98.1-cdh5.1.0 81 | test-jar 82 | tests 83 | test 84 | 85 | 86 | org.apache.hbase 87 | hbase-common 88 | 0.98.1-cdh5.1.0 89 | 90 | 91 | org.apache.hbase 92 | hbase-common 93 | 0.98.1-cdh5.1.0 94 | test-jar 95 | tests 96 | test 97 | 98 | 99 | org.apache.hbase 100 | hbase-hadoop-compat 101 | 0.98.1-cdh5.1.0 102 | test 103 | 104 | 105 | org.apache.hbase 106 | hbase-hadoop-compat 107 | 0.98.1-cdh5.1.0 108 | test-jar 109 | tests 110 | test 111 | 112 | 113 | 114 | 115 | maven-hadoop 116 | Hadoop Releases 117 | https://repository.cloudera.com/content/repositories/releases/ 118 | 119 | 120 | 121 | 122 | target/scala/classes 123 | target/scala/test-classes 124 | 125 | 126 | org.apache.maven.plugins 127 | maven-surefire-plugin 128 | 2.17 129 | 130 | 131 | org.apache.maven.surefire 132 | surefire-junit47 133 | 2.17 134 | 135 | 136 | 137 | 138 | org.scalatest 139 | scalatest-maven-plugin 140 | 1.0 141 | 142 | . 143 | 144 | 145 | 146 | test 147 | 148 | test 149 | 150 | 151 | 152 | 153 | 154 | 155 | org.scala-tools 156 | maven-scala-plugin 157 | 158 | 159 | compile 160 | 161 | compile 162 | 163 | compile 164 | 165 | 166 | test-compile 167 | 168 | testCompile 169 | 170 | test-compile 171 | 172 | 173 | process-resources 174 | 175 | compile 176 | 177 | 178 | 179 | 180 | 181 | org.apache.maven.plugins 182 | maven-shade-plugin 183 | 2.2 184 | 185 | false 186 | target/SparkHBase.jar 187 | 188 | 189 | *:* 190 | 191 | 192 | 193 | 194 | *:* 195 | 196 | META-INF/*.SF 197 | META-INF/*.DSA 198 | META-INF/*.RSA 199 | 200 | 201 | 202 | 203 | 204 | 205 | package 206 | 207 | shade 208 | 209 | 210 | 211 | 213 | 215 | reference.conf 216 | 217 | 218 | 219 | 220 | 221 | 222 | 223 | 224 | 225 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/hadoop/hbase/spark/HBaseDStreamFunctions.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | package org.apache.hadoop.hbase.spark 18 | 19 | import org.apache.hadoop.hbase.TableName 20 | import org.apache.hadoop.hbase.client._ 21 | import org.apache.hadoop.hbase.io.ImmutableBytesWritable 22 | import org.apache.spark.streaming.dstream.DStream 23 | 24 | import scala.reflect.ClassTag 25 | 26 | /** 27 | * HBaseDStreamFunctions contains a set of implicit functions that can be 28 | * applied to a Spark DStream so that we can easily interact with HBase 29 | */ 30 | object HBaseDStreamFunctions { 31 | 32 | /** 33 | * These are implicit methods for a DStream that contains any type of 34 | * data. 35 | * 36 | * @param dStream This is for dStreams of any type 37 | * @tparam T Type T 38 | */ 39 | implicit class GenericHBaseDStreamFunctions[T](val dStream: DStream[T]) { 40 | 41 | /** 42 | * Implicit method that gives easy access to HBaseContext's bulk 43 | * put. This will not return a new Stream. Think of it like a foreach 44 | * 45 | * @param hc The hbaseContext object to identify which 46 | * HBase cluster connection to use 47 | * @param tableName The tableName that the put will be sent to 48 | * @param f The function that will turn the DStream values 49 | * into HBase Put objects. 50 | */ 51 | def hbaseBulkPut(hc: HBaseContext, 52 | tableName: TableName, 53 | f: (T) => Put): Unit = { 54 | hc.streamBulkPut(dStream, tableName, f) 55 | } 56 | 57 | /** 58 | * Implicit method that gives easy access to HBaseContext's bulk 59 | * get. This will return a new DStream. Think about it as a DStream map 60 | * function. In that every DStream value will get a new value out of 61 | * HBase. That new value will populate the newly generated DStream. 62 | * 63 | * @param hc The hbaseContext object to identify which 64 | * HBase cluster connection to use 65 | * @param tableName The tableName that the put will be sent to 66 | * @param batchSize How many gets to execute in a single batch 67 | * @param f The function that will turn the RDD values 68 | * in HBase Get objects 69 | * @param convertResult The function that will convert a HBase 70 | * Result object into a value that will go 71 | * into the resulting DStream 72 | * @tparam R The type of Object that will be coming 73 | * out of the resulting DStream 74 | * @return A resulting DStream with type R objects 75 | */ 76 | def hbaseBulkGet[R: ClassTag](hc: HBaseContext, 77 | tableName: TableName, 78 | batchSize:Int, f: (T) => Get, convertResult: (Result) => R): 79 | DStream[R] = { 80 | hc.streamBulkGet[T, R](tableName, batchSize, dStream, f, convertResult) 81 | } 82 | 83 | /** 84 | * Implicit method that gives easy access to HBaseContext's bulk 85 | * get. This will return a new DStream. Think about it as a DStream map 86 | * function. In that every DStream value will get a new value out of 87 | * HBase. That new value will populate the newly generated DStream. 88 | * 89 | * @param hc The hbaseContext object to identify which 90 | * HBase cluster connection to use 91 | * @param tableName The tableName that the put will be sent to 92 | * @param batchSize How many gets to execute in a single batch 93 | * @param f The function that will turn the RDD values 94 | * in HBase Get objects 95 | * @return A resulting DStream with type R objects 96 | */ 97 | def hbaseBulkGet(hc: HBaseContext, 98 | tableName: TableName, batchSize:Int, 99 | f: (T) => Get): DStream[(ImmutableBytesWritable, Result)] = { 100 | hc.streamBulkGet[T, (ImmutableBytesWritable, Result)]( 101 | tableName, batchSize, dStream, f, 102 | result => (new ImmutableBytesWritable(result.getRow), result)) 103 | } 104 | 105 | /** 106 | * Implicit method that gives easy access to HBaseContext's bulk 107 | * Delete. This will not return a new DStream. 108 | * 109 | * @param hc The hbaseContext object to identify which HBase 110 | * cluster connection to use 111 | * @param tableName The tableName that the deletes will be sent to 112 | * @param f The function that will convert the DStream value into 113 | * a HBase Delete Object 114 | * @param batchSize The number of Deletes to be sent in a single batch 115 | */ 116 | def hbaseBulkDelete(hc: HBaseContext, 117 | tableName: TableName, 118 | f:(T) => Delete, batchSize:Int): Unit = { 119 | hc.streamBulkDelete(dStream, tableName, f, batchSize) 120 | } 121 | 122 | /** 123 | * Implicit method that gives easy access to HBaseContext's 124 | * foreachPartition method. This will ack very much like a normal DStream 125 | * foreach method but for the fact that you will now have a HBase connection 126 | * while iterating through the values. 127 | * 128 | * @param hc The hbaseContext object to identify which HBase 129 | * cluster connection to use 130 | * @param f This function will get an iterator for a Partition of an 131 | * DStream along with a connection object to HBase 132 | */ 133 | def hbaseForeachPartition(hc: HBaseContext, 134 | f: (Iterator[T], Connection) => Unit): Unit = { 135 | hc.streamForeachPartition(dStream, f) 136 | } 137 | 138 | /** 139 | * Implicit method that gives easy access to HBaseContext's 140 | * mapPartitions method. This will ask very much like a normal DStream 141 | * map partitions method but for the fact that you will now have a 142 | * HBase connection while iterating through the values 143 | * 144 | * @param hc The hbaseContext object to identify which HBase 145 | * cluster connection to use 146 | * @param f This function will get an iterator for a Partition of an 147 | * DStream along with a connection object to HBase 148 | * @tparam R This is the type of objects that will go into the resulting 149 | * DStream 150 | * @return A resulting DStream of type R 151 | */ 152 | def hbaseMapPartitions[R: ClassTag](hc: HBaseContext, 153 | f: (Iterator[T], Connection) => Iterator[R]): 154 | DStream[R] = { 155 | hc.streamMapPartitions(dStream, f) 156 | } 157 | } 158 | } 159 | -------------------------------------------------------------------------------- /LICENSE.txt: -------------------------------------------------------------------------------- 1 | Apache License 2 | 3 | Version 2.0, January 2004 4 | 5 | http://www.apache.org/licenses/ 6 | 7 | TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION 8 | 9 | 1. Definitions. 10 | 11 | "License" shall mean the terms and conditions for use, reproduction, and distribution as defined by Sections 1 through 9 of this document. 12 | 13 | "Licensor" shall mean the copyright owner or entity authorized by the copyright owner that is granting the License. 14 | 15 | "Legal Entity" shall mean the union of the acting entity and all other entities that control, are controlled by, or are under common control with that entity. For the purposes of this definition, "control" means (i) the power, direct or indirect, to cause the direction or management of such entity, whether by contract or otherwise, or (ii) ownership of fifty percent (50%) or more of the outstanding shares, or (iii) beneficial ownership of such entity. 16 | 17 | "You" (or "Your") shall mean an individual or Legal Entity exercising permissions granted by this License. 18 | 19 | "Source" form shall mean the preferred form for making modifications, including but not limited to software source code, documentation source, and configuration files. 20 | 21 | "Object" form shall mean any form resulting from mechanical transformation or translation of a Source form, including but not limited to compiled object code, generated documentation, and conversions to other media types. 22 | 23 | "Work" shall mean the work of authorship, whether in Source or Object form, made available under the License, as indicated by a copyright notice that is included in or attached to the work (an example is provided in the Appendix below). 24 | 25 | "Derivative Works" shall mean any work, whether in Source or Object form, that is based on (or derived from) the Work and for which the editorial revisions, annotations, elaborations, or other modifications represent, as a whole, an original work of authorship. For the purposes of this License, Derivative Works shall not include works that remain separable from, or merely link (or bind by name) to the interfaces of, the Work and Derivative Works thereof. 26 | 27 | "Contribution" shall mean any work of authorship, including the original version of the Work and any modifications or additions to that Work or Derivative Works thereof, that is intentionally submitted to Licensor for inclusion in the Work by the copyright owner or by an individual or Legal Entity authorized to submit on behalf of the copyright owner. For the purposes of this definition, "submitted" means any form of electronic, verbal, or written communication sent to the Licensor or its representatives, including but not limited to communication on electronic mailing lists, source code control systems, and issue tracking systems that are managed by, or on behalf of, the Licensor for the purpose of discussing and improving the Work, but excluding communication that is conspicuously marked or otherwise designated in writing by the copyright owner as "Not a Contribution." 28 | 29 | "Contributor" shall mean Licensor and any individual or Legal Entity on behalf of whom a Contribution has been received by Licensor and subsequently incorporated within the Work. 30 | 31 | 2. Grant of Copyright License. Subject to the terms and conditions of this License, each Contributor hereby grants to You a perpetual, worldwide, non-exclusive, no-charge, royalty-free, irrevocable copyright license to reproduce, prepare Derivative Works of, publicly display, publicly perform, sublicense, and distribute the Work and such Derivative Works in Source or Object form. 32 | 33 | 3. Grant of Patent License. Subject to the terms and conditions of this License, each Contributor hereby grants to You a perpetual, worldwide, non-exclusive, no-charge, royalty-free, irrevocable (except as stated in this section) patent license to make, have made, use, offer to sell, sell, import, and otherwise transfer the Work, where such license applies only to those patent claims licensable by such Contributor that are necessarily infringed by their Contribution(s) alone or by combination of their Contribution(s) with the Work to which such Contribution(s) was submitted. If You institute patent litigation against any entity (including a cross-claim or counterclaim in a lawsuit) alleging that the Work or a Contribution incorporated within the Work constitutes direct or contributory patent infringement, then any patent licenses granted to You under this License for that Work shall terminate as of the date such litigation is filed. 34 | 35 | 4. Redistribution. You may reproduce and distribute copies of the Work or Derivative Works thereof in any medium, with or without modifications, and in Source or Object form, provided that You meet the following conditions: 36 | 37 | You must give any other recipients of the Work or Derivative Works a copy of this License; and 38 | 39 | You must cause any modified files to carry prominent notices stating that You changed the files; and 40 | 41 | You must retain, in the Source form of any Derivative Works that You distribute, all copyright, patent, trademark, and attribution notices from the Source form of the Work, excluding those notices that do not pertain to any part of the Derivative Works; and 42 | 43 | If the Work includes a "NOTICE" text file as part of its distribution, then any Derivative Works that You distribute must include a readable copy of the attribution notices contained within such NOTICE file, excluding those notices that do not pertain to any part of the Derivative Works, in at least one of the following places: within a NOTICE text file distributed as part of the Derivative Works; within the Source form or documentation, if provided along with the Derivative Works; or, within a display generated by the Derivative Works, if and wherever such third-party notices normally appear. The contents of the NOTICE file are for informational purposes only and do not modify the License. You may add Your own attribution notices within Derivative Works that You distribute, alongside or as an addendum to the NOTICE text from the Work, provided that such additional attribution notices cannot be construed as modifying the License. You may add Your own copyright statement to Your modifications and may provide additional or different license terms and conditions for use, reproduction, or distribution of Your modifications, or for any such Derivative Works as a whole, provided Your use, reproduction, and distribution of the Work otherwise complies with the conditions stated in this License. 44 | 45 | 5. Submission of Contributions. Unless You explicitly state otherwise, any Contribution intentionally submitted for inclusion in the Work by You to the Licensor shall be under the terms and conditions of this License, without any additional terms or conditions. Notwithstanding the above, nothing herein shall supersede or modify the terms of any separate license agreement you may have executed with Licensor regarding such Contributions. 46 | 47 | 6. Trademarks. This License does not grant permission to use the trade names, trademarks, service marks, or product names of the Licensor, except as required for reasonable and customary use in describing the origin of the Work and reproducing the content of the NOTICE file. 48 | 49 | 7. Disclaimer of Warranty. Unless required by applicable law or agreed to in writing, Licensor provides the Work (and each Contributor provides its Contributions) on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied, including, without limitation, any warranties or conditions of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A PARTICULAR PURPOSE. You are solely responsible for determining the appropriateness of using or redistributing the Work and assume any risks associated with Your exercise of permissions under this License. 50 | 51 | 8. Limitation of Liability. In no event and under no legal theory, whether in tort (including negligence), contract, or otherwise, unless required by applicable law (such as deliberate and grossly negligent acts) or agreed to in writing, shall any Contributor be liable to You for damages, including any direct, indirect, special, incidental, or consequential damages of any character arising as a result of this License or out of the use or inability to use the Work (including but not limited to damages for loss of goodwill, work stoppage, computer failure or malfunction, or any and all other commercial damages or losses), even if such Contributor has been advised of the possibility of such damages. 52 | 53 | 9. Accepting Warranty or Additional Liability. While redistributing the Work or Derivative Works thereof, You may choose to offer, and charge a fee for, acceptance of support, warranty, indemnity, or other liability obligations and/or rights consistent with this License. However, in accepting such obligations, You may act only on Your own behalf and on Your sole responsibility, not on behalf of any other Contributor, and only if You agree to indemnify, defend, and hold each Contributor harmless for any liability incurred by, or claims asserted against, such Contributor by reason of your accepting any such warranty or additional liability. 54 | 55 | END OF TERMS AND CONDITIONS 56 | 57 | APPENDIX: How to apply the Apache License to your work 58 | To apply the Apache License to your work, attach the following boilerplate notice, with the fields enclosed by brackets "[]" replaced with your own identifying information. (Don't include the brackets!) The text should be enclosed in the appropriate comment syntax for the file format. We also recommend that a file or class name and description of purpose be included on the same "printed page" as the copyright notice for easier identification within third-party archives. 59 | 60 |    Copyright [yyyy] [name of copyright owner] 61 | 62 |    Licensed under the Apache License, Version 2.0 (the "License"); 63 |    you may not use this file except in compliance with the License. 64 |    You may obtain a copy of the License at 65 | 66 |        http://www.apache.org/licenses/LICENSE-2.0 67 | 68 |    Unless required by applicable law or agreed to in writing, software 69 |    distributed under the License is distributed on an "AS IS" BASIS, 70 |    WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 71 |    See the License for the specific language governing permissions and 72 |    limitations under the License. 73 | -------------------------------------------------------------------------------- /src/main/java/org/apache/hadoop/hbase/spark/SparkSQLPushDownFilter.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.hadoop.hbase.spark; 19 | 20 | import com.google.protobuf.ByteString; 21 | import com.google.protobuf.InvalidProtocolBufferException; 22 | import org.apache.commons.logging.Log; 23 | import org.apache.commons.logging.LogFactory; 24 | import org.apache.hadoop.hbase.Cell; 25 | import org.apache.hadoop.hbase.exceptions.DeserializationException; 26 | import org.apache.hadoop.hbase.filter.FilterBase; 27 | import org.apache.hadoop.hbase.spark.protobuf.generated.FilterProtos; 28 | import org.apache.hadoop.hbase.util.ByteStringer; 29 | import org.apache.hadoop.hbase.util.Bytes; 30 | import scala.collection.mutable.MutableList; 31 | 32 | import java.io.IOException; 33 | import java.util.HashMap; 34 | import java.util.List; 35 | import java.util.Map; 36 | 37 | /** 38 | * This filter will push down all qualifier logic given to us 39 | * by SparkSQL so that we have make the filters at the region server level 40 | * and avoid sending the data back to the client to be filtered. 41 | */ 42 | public class SparkSQLPushDownFilter extends FilterBase{ 43 | protected static final Log log = LogFactory.getLog(SparkSQLPushDownFilter.class); 44 | 45 | //The following values are populated with protobuffer 46 | DynamicLogicExpression dynamicLogicExpression; 47 | byte[][] valueFromQueryArray; 48 | HashMap> 49 | currentCellToColumnIndexMap; 50 | 51 | //The following values are transient 52 | HashMap columnToCurrentRowValueMap = null; 53 | 54 | static final byte[] rowKeyFamily = new byte[0]; 55 | static final byte[] rowKeyQualifier = Bytes.toBytes("key"); 56 | 57 | public SparkSQLPushDownFilter(DynamicLogicExpression dynamicLogicExpression, 58 | byte[][] valueFromQueryArray, 59 | HashMap> 61 | currentCellToColumnIndexMap) { 62 | this.dynamicLogicExpression = dynamicLogicExpression; 63 | this.valueFromQueryArray = valueFromQueryArray; 64 | this.currentCellToColumnIndexMap = currentCellToColumnIndexMap; 65 | } 66 | 67 | public SparkSQLPushDownFilter(DynamicLogicExpression dynamicLogicExpression, 68 | byte[][] valueFromQueryArray, 69 | MutableList columnDefinitions) { 70 | this.dynamicLogicExpression = dynamicLogicExpression; 71 | this.valueFromQueryArray = valueFromQueryArray; 72 | 73 | //generate family qualifier to index mapping 74 | this.currentCellToColumnIndexMap = 75 | new HashMap<>(); 76 | 77 | for (int i = 0; i < columnDefinitions.size(); i++) { 78 | SchemaQualifierDefinition definition = columnDefinitions.get(i).get(); 79 | 80 | ByteArrayComparable familyByteComparable = 81 | new ByteArrayComparable(definition.columnFamilyBytes(), 82 | 0, definition.columnFamilyBytes().length); 83 | 84 | HashMap qualifierIndexMap = 85 | currentCellToColumnIndexMap.get(familyByteComparable); 86 | 87 | if (qualifierIndexMap == null) { 88 | qualifierIndexMap = new HashMap<>(); 89 | currentCellToColumnIndexMap.put(familyByteComparable, qualifierIndexMap); 90 | } 91 | ByteArrayComparable qualifierByteComparable = 92 | new ByteArrayComparable(definition.qualifierBytes(), 0, 93 | definition.qualifierBytes().length); 94 | 95 | qualifierIndexMap.put(qualifierByteComparable, definition.columnName()); 96 | } 97 | } 98 | 99 | @Override 100 | public ReturnCode filterKeyValue(Cell c) throws IOException { 101 | 102 | //If the map RowValueMap is empty then we need to populate 103 | // the row key 104 | if (columnToCurrentRowValueMap == null) { 105 | columnToCurrentRowValueMap = new HashMap<>(); 106 | HashMap qualifierColumnMap = 107 | currentCellToColumnIndexMap.get( 108 | new ByteArrayComparable(rowKeyFamily, 0, rowKeyFamily.length)); 109 | 110 | if (qualifierColumnMap != null) { 111 | String rowKeyColumnName = 112 | qualifierColumnMap.get( 113 | new ByteArrayComparable(rowKeyQualifier, 0, 114 | rowKeyQualifier.length)); 115 | //Make sure that the rowKey is part of the where clause 116 | if (rowKeyColumnName != null) { 117 | columnToCurrentRowValueMap.put(rowKeyColumnName, 118 | new ByteArrayComparable(c.getRowArray(), 119 | c.getRowOffset(), c.getRowLength())); 120 | } 121 | } 122 | } 123 | 124 | //Always populate the column value into the RowValueMap 125 | ByteArrayComparable currentFamilyByteComparable = 126 | new ByteArrayComparable(c.getFamilyArray(), 127 | c.getFamilyOffset(), 128 | c.getFamilyLength()); 129 | 130 | HashMap qualifierColumnMap = 131 | currentCellToColumnIndexMap.get( 132 | currentFamilyByteComparable); 133 | 134 | if (qualifierColumnMap != null) { 135 | 136 | String columnName = 137 | qualifierColumnMap.get( 138 | new ByteArrayComparable(c.getQualifierArray(), 139 | c.getQualifierOffset(), 140 | c.getQualifierLength())); 141 | 142 | if (columnName != null) { 143 | columnToCurrentRowValueMap.put(columnName, 144 | new ByteArrayComparable(c.getValueArray(), 145 | c.getValueOffset(), c.getValueLength())); 146 | } 147 | } 148 | 149 | return ReturnCode.INCLUDE; 150 | } 151 | 152 | 153 | @Override 154 | public boolean filterRow() throws IOException { 155 | 156 | try { 157 | boolean result = 158 | dynamicLogicExpression.execute(columnToCurrentRowValueMap, 159 | valueFromQueryArray); 160 | columnToCurrentRowValueMap = null; 161 | return !result; 162 | } catch (Throwable e) { 163 | log.error("Error running dynamic logic on row", e); 164 | } 165 | return false; 166 | } 167 | 168 | 169 | /** 170 | * @param pbBytes A pb serialized instance 171 | * @return An instance of SparkSQLPushDownFilter 172 | * @throws org.apache.hadoop.hbase.exceptions.DeserializationException 173 | */ 174 | @SuppressWarnings("unused") 175 | public static SparkSQLPushDownFilter parseFrom(final byte[] pbBytes) 176 | throws DeserializationException { 177 | 178 | FilterProtos.SQLPredicatePushDownFilter proto; 179 | try { 180 | proto = FilterProtos.SQLPredicatePushDownFilter.parseFrom(pbBytes); 181 | } catch (InvalidProtocolBufferException e) { 182 | throw new DeserializationException(e); 183 | } 184 | 185 | //Load DynamicLogicExpression 186 | DynamicLogicExpression dynamicLogicExpression = 187 | DynamicLogicExpressionBuilder.build(proto.getDynamicLogicExpression()); 188 | 189 | //Load valuesFromQuery 190 | final List valueFromQueryArrayList = proto.getValueFromQueryArrayList(); 191 | byte[][] valueFromQueryArray = new byte[valueFromQueryArrayList.size()][]; 192 | for (int i = 0; i < valueFromQueryArrayList.size(); i++) { 193 | valueFromQueryArray[i] = valueFromQueryArrayList.get(i).toByteArray(); 194 | } 195 | 196 | //Load mapping from HBase family/qualifier to Spark SQL columnName 197 | HashMap> 198 | currentCellToColumnIndexMap = new HashMap<>(); 199 | 200 | for (FilterProtos.SQLPredicatePushDownCellToColumnMapping 201 | sqlPredicatePushDownCellToColumnMapping : 202 | proto.getCellToColumnMappingList()) { 203 | 204 | byte[] familyArray = 205 | sqlPredicatePushDownCellToColumnMapping.getColumnFamily().toByteArray(); 206 | ByteArrayComparable familyByteComparable = 207 | new ByteArrayComparable(familyArray, 0, familyArray.length); 208 | HashMap qualifierMap = 209 | currentCellToColumnIndexMap.get(familyByteComparable); 210 | 211 | if (qualifierMap == null) { 212 | qualifierMap = new HashMap<>(); 213 | currentCellToColumnIndexMap.put(familyByteComparable, qualifierMap); 214 | } 215 | byte[] qualifierArray = 216 | sqlPredicatePushDownCellToColumnMapping.getQualifier().toByteArray(); 217 | 218 | ByteArrayComparable qualifierByteComparable = 219 | new ByteArrayComparable(qualifierArray, 0 ,qualifierArray.length); 220 | 221 | qualifierMap.put(qualifierByteComparable, 222 | sqlPredicatePushDownCellToColumnMapping.getColumnName()); 223 | } 224 | 225 | return new SparkSQLPushDownFilter(dynamicLogicExpression, 226 | valueFromQueryArray, currentCellToColumnIndexMap); 227 | } 228 | 229 | /** 230 | * @return The filter serialized using pb 231 | */ 232 | public byte[] toByteArray() { 233 | 234 | FilterProtos.SQLPredicatePushDownFilter.Builder builder = 235 | FilterProtos.SQLPredicatePushDownFilter.newBuilder(); 236 | 237 | FilterProtos.SQLPredicatePushDownCellToColumnMapping.Builder columnMappingBuilder = 238 | FilterProtos.SQLPredicatePushDownCellToColumnMapping.newBuilder(); 239 | 240 | builder.setDynamicLogicExpression(dynamicLogicExpression.toExpressionString()); 241 | for (byte[] valueFromQuery: valueFromQueryArray) { 242 | builder.addValueFromQueryArray(ByteStringer.wrap(valueFromQuery)); 243 | } 244 | 245 | for (Map.Entry> 246 | familyEntry : currentCellToColumnIndexMap.entrySet()) { 247 | for (Map.Entry qualifierEntry : 248 | familyEntry.getValue().entrySet()) { 249 | columnMappingBuilder.setColumnFamily( 250 | ByteStringer.wrap(familyEntry.getKey().bytes())); 251 | columnMappingBuilder.setQualifier( 252 | ByteStringer.wrap(qualifierEntry.getKey().bytes())); 253 | columnMappingBuilder.setColumnName(qualifierEntry.getValue()); 254 | builder.addCellToColumnMapping(columnMappingBuilder.build()); 255 | } 256 | } 257 | 258 | return builder.build().toByteArray(); 259 | } 260 | } 261 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/hadoop/hbase/spark/DynamicLogicExpression.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.hadoop.hbase.spark 19 | 20 | import java.util 21 | 22 | import org.apache.hadoop.hbase.util.Bytes 23 | 24 | /** 25 | * Dynamic logic for SQL push down logic there is an instance for most 26 | * common operations and a pass through for other operations not covered here 27 | * 28 | * Logic can be nested with And or Or operators. 29 | * 30 | * A logic tree can be written out as a string and reconstructed from that string 31 | * 32 | */ 33 | trait DynamicLogicExpression { 34 | def execute(columnToCurrentRowValueMap: util.HashMap[String, ByteArrayComparable], 35 | valueFromQueryValueArray:Array[Array[Byte]]): Boolean 36 | def toExpressionString: String = { 37 | val strBuilder = new StringBuilder 38 | appendToExpression(strBuilder) 39 | strBuilder.toString() 40 | } 41 | def appendToExpression(strBuilder:StringBuilder) 42 | } 43 | 44 | class AndLogicExpression (val leftExpression:DynamicLogicExpression, 45 | val rightExpression:DynamicLogicExpression) 46 | extends DynamicLogicExpression{ 47 | override def execute(columnToCurrentRowValueMap: 48 | util.HashMap[String, ByteArrayComparable], 49 | valueFromQueryValueArray:Array[Array[Byte]]): Boolean = { 50 | leftExpression.execute(columnToCurrentRowValueMap, valueFromQueryValueArray) && 51 | rightExpression.execute(columnToCurrentRowValueMap, valueFromQueryValueArray) 52 | } 53 | 54 | override def appendToExpression(strBuilder: StringBuilder): Unit = { 55 | strBuilder.append("( ") 56 | strBuilder.append(leftExpression.toExpressionString) 57 | strBuilder.append(" AND ") 58 | strBuilder.append(rightExpression.toExpressionString) 59 | strBuilder.append(" )") 60 | } 61 | } 62 | 63 | class OrLogicExpression (val leftExpression:DynamicLogicExpression, 64 | val rightExpression:DynamicLogicExpression) 65 | extends DynamicLogicExpression{ 66 | override def execute(columnToCurrentRowValueMap: 67 | util.HashMap[String, ByteArrayComparable], 68 | valueFromQueryValueArray:Array[Array[Byte]]): Boolean = { 69 | leftExpression.execute(columnToCurrentRowValueMap, valueFromQueryValueArray) || 70 | rightExpression.execute(columnToCurrentRowValueMap, valueFromQueryValueArray) 71 | } 72 | override def appendToExpression(strBuilder: StringBuilder): Unit = { 73 | strBuilder.append("( ") 74 | strBuilder.append(leftExpression.toExpressionString) 75 | strBuilder.append(" OR ") 76 | strBuilder.append(rightExpression.toExpressionString) 77 | strBuilder.append(" )") 78 | } 79 | } 80 | 81 | class EqualLogicExpression (val columnName:String, 82 | val valueFromQueryIndex:Int, 83 | val isNot:Boolean) extends DynamicLogicExpression{ 84 | override def execute(columnToCurrentRowValueMap: 85 | util.HashMap[String, ByteArrayComparable], 86 | valueFromQueryValueArray:Array[Array[Byte]]): Boolean = { 87 | val currentRowValue = columnToCurrentRowValueMap.get(columnName) 88 | val valueFromQuery = valueFromQueryValueArray(valueFromQueryIndex) 89 | 90 | currentRowValue != null && 91 | Bytes.equals(valueFromQuery, 92 | 0, valueFromQuery.length, currentRowValue.bytes, 93 | currentRowValue.offset, currentRowValue.length) != isNot 94 | } 95 | override def appendToExpression(strBuilder: StringBuilder): Unit = { 96 | val command = if (isNot) "!=" else "==" 97 | strBuilder.append(columnName + " " + command + " " + valueFromQueryIndex) 98 | } 99 | } 100 | 101 | class IsNullLogicExpression (val columnName:String, 102 | val isNot:Boolean) extends DynamicLogicExpression{ 103 | override def execute(columnToCurrentRowValueMap: 104 | util.HashMap[String, ByteArrayComparable], 105 | valueFromQueryValueArray:Array[Array[Byte]]): Boolean = { 106 | val currentRowValue = columnToCurrentRowValueMap.get(columnName) 107 | 108 | (currentRowValue == null) != isNot 109 | } 110 | override def appendToExpression(strBuilder: StringBuilder): Unit = { 111 | val command = if (isNot) "isNotNull" else "isNull" 112 | strBuilder.append(columnName + " " + command) 113 | } 114 | } 115 | 116 | class GreaterThanLogicExpression (val columnName:String, 117 | val valueFromQueryIndex:Int) 118 | extends DynamicLogicExpression{ 119 | override def execute(columnToCurrentRowValueMap: 120 | util.HashMap[String, ByteArrayComparable], 121 | valueFromQueryValueArray:Array[Array[Byte]]): Boolean = { 122 | val currentRowValue = columnToCurrentRowValueMap.get(columnName) 123 | val valueFromQuery = valueFromQueryValueArray(valueFromQueryIndex) 124 | 125 | currentRowValue != null && 126 | Bytes.compareTo(currentRowValue.bytes, 127 | currentRowValue.offset, currentRowValue.length, valueFromQuery, 128 | 0, valueFromQuery.length) > 0 129 | } 130 | override def appendToExpression(strBuilder: StringBuilder): Unit = { 131 | strBuilder.append(columnName + " > " + valueFromQueryIndex) 132 | } 133 | } 134 | 135 | class GreaterThanOrEqualLogicExpression (val columnName:String, 136 | val valueFromQueryIndex:Int) 137 | extends DynamicLogicExpression{ 138 | override def execute(columnToCurrentRowValueMap: 139 | util.HashMap[String, ByteArrayComparable], 140 | valueFromQueryValueArray:Array[Array[Byte]]): Boolean = { 141 | val currentRowValue = columnToCurrentRowValueMap.get(columnName) 142 | val valueFromQuery = valueFromQueryValueArray(valueFromQueryIndex) 143 | 144 | currentRowValue != null && 145 | Bytes.compareTo(currentRowValue.bytes, 146 | currentRowValue.offset, currentRowValue.length, valueFromQuery, 147 | 0, valueFromQuery.length) >= 0 148 | } 149 | override def appendToExpression(strBuilder: StringBuilder): Unit = { 150 | strBuilder.append(columnName + " >= " + valueFromQueryIndex) 151 | } 152 | } 153 | 154 | class LessThanLogicExpression (val columnName:String, 155 | val valueFromQueryIndex:Int) 156 | extends DynamicLogicExpression{ 157 | override def execute(columnToCurrentRowValueMap: 158 | util.HashMap[String, ByteArrayComparable], 159 | valueFromQueryValueArray:Array[Array[Byte]]): Boolean = { 160 | val currentRowValue = columnToCurrentRowValueMap.get(columnName) 161 | val valueFromQuery = valueFromQueryValueArray(valueFromQueryIndex) 162 | 163 | currentRowValue != null && 164 | Bytes.compareTo(currentRowValue.bytes, 165 | currentRowValue.offset, currentRowValue.length, valueFromQuery, 166 | 0, valueFromQuery.length) < 0 167 | } 168 | 169 | override def appendToExpression(strBuilder: StringBuilder): Unit = { 170 | strBuilder.append(columnName + " < " + valueFromQueryIndex) 171 | } 172 | } 173 | 174 | class LessThanOrEqualLogicExpression (val columnName:String, 175 | val valueFromQueryIndex:Int) 176 | extends DynamicLogicExpression{ 177 | override def execute(columnToCurrentRowValueMap: 178 | util.HashMap[String, ByteArrayComparable], 179 | valueFromQueryValueArray:Array[Array[Byte]]): Boolean = { 180 | val currentRowValue = columnToCurrentRowValueMap.get(columnName) 181 | val valueFromQuery = valueFromQueryValueArray(valueFromQueryIndex) 182 | 183 | currentRowValue != null && 184 | Bytes.compareTo(currentRowValue.bytes, 185 | currentRowValue.offset, currentRowValue.length, valueFromQuery, 186 | 0, valueFromQuery.length) <= 0 187 | } 188 | 189 | override def appendToExpression(strBuilder: StringBuilder): Unit = { 190 | strBuilder.append(columnName + " <= " + valueFromQueryIndex) 191 | } 192 | } 193 | 194 | class PassThroughLogicExpression() extends DynamicLogicExpression { 195 | override def execute(columnToCurrentRowValueMap: 196 | util.HashMap[String, ByteArrayComparable], 197 | valueFromQueryValueArray: Array[Array[Byte]]): Boolean = true 198 | 199 | override def appendToExpression(strBuilder: StringBuilder): Unit = { 200 | strBuilder.append("Pass") 201 | } 202 | } 203 | 204 | object DynamicLogicExpressionBuilder { 205 | def build(expressionString:String): DynamicLogicExpression = { 206 | 207 | val expressionAndOffset = build(expressionString.split(' '), 0) 208 | expressionAndOffset._1 209 | } 210 | 211 | private def build(expressionArray:Array[String], 212 | offSet:Int): (DynamicLogicExpression, Int) = { 213 | if (expressionArray(offSet).equals("(")) { 214 | val left = build(expressionArray, offSet + 1) 215 | val right = build(expressionArray, left._2 + 1) 216 | if (expressionArray(left._2).equals("AND")) { 217 | (new AndLogicExpression(left._1, right._1), right._2 + 1) 218 | } else if (expressionArray(left._2).equals("OR")) { 219 | (new OrLogicExpression(left._1, right._1), right._2 + 1) 220 | } else { 221 | throw new Throwable("Unknown gate:" + expressionArray(left._2)) 222 | } 223 | } else { 224 | val command = expressionArray(offSet + 1) 225 | if (command.equals("<")) { 226 | (new LessThanLogicExpression(expressionArray(offSet), 227 | expressionArray(offSet + 2).toInt), offSet + 3) 228 | } else if (command.equals("<=")) { 229 | (new LessThanOrEqualLogicExpression(expressionArray(offSet), 230 | expressionArray(offSet + 2).toInt), offSet + 3) 231 | } else if (command.equals(">")) { 232 | (new GreaterThanLogicExpression(expressionArray(offSet), 233 | expressionArray(offSet + 2).toInt), offSet + 3) 234 | } else if (command.equals(">=")) { 235 | (new GreaterThanOrEqualLogicExpression(expressionArray(offSet), 236 | expressionArray(offSet + 2).toInt), offSet + 3) 237 | } else if (command.equals("==")) { 238 | (new EqualLogicExpression(expressionArray(offSet), 239 | expressionArray(offSet + 2).toInt, false), offSet + 3) 240 | } else if (command.equals("!=")) { 241 | (new EqualLogicExpression(expressionArray(offSet), 242 | expressionArray(offSet + 2).toInt, true), offSet + 3) 243 | } else if (command.equals("isNull")) { 244 | (new IsNullLogicExpression(expressionArray(offSet), false), offSet + 2) 245 | } else if (command.equals("isNotNull")) { 246 | (new IsNullLogicExpression(expressionArray(offSet), true), offSet + 2) 247 | } else if (command.equals("Pass")) { 248 | (new PassThroughLogicExpression, offSet + 2) 249 | } else { 250 | throw new Throwable("Unknown logic command:" + command) 251 | } 252 | } 253 | } 254 | } -------------------------------------------------------------------------------- /src/main/scala/org/apache/hadoop/hbase/spark/HBaseRDDFunctions.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.hadoop.hbase.spark 19 | 20 | import java.util 21 | 22 | import org.apache.hadoop.hbase.{HConstants, TableName} 23 | import org.apache.hadoop.hbase.client._ 24 | import org.apache.hadoop.hbase.io.ImmutableBytesWritable 25 | import org.apache.spark.rdd.RDD 26 | 27 | import scala.reflect.ClassTag 28 | 29 | /** 30 | * HBaseRDDFunctions contains a set of implicit functions that can be 31 | * applied to a Spark RDD so that we can easily interact with HBase 32 | */ 33 | object HBaseRDDFunctions 34 | { 35 | 36 | /** 37 | * These are implicit methods for a RDD that contains any type of 38 | * data. 39 | * 40 | * @param rdd This is for rdd of any type 41 | * @tparam T This is any type 42 | */ 43 | implicit class GenericHBaseRDDFunctions[T](val rdd: RDD[T]) { 44 | 45 | /** 46 | * Implicit method that gives easy access to HBaseContext's bulk 47 | * put. This will not return a new RDD. Think of it like a foreach 48 | * 49 | * @param hc The hbaseContext object to identify which 50 | * HBase cluster connection to use 51 | * @param tableName The tableName that the put will be sent to 52 | * @param f The function that will turn the RDD values 53 | * into HBase Put objects. 54 | */ 55 | def hbaseBulkPut(hc: HBaseContext, 56 | tableName: TableName, 57 | f: (T) => Put): Unit = { 58 | hc.bulkPut(rdd, tableName, f) 59 | } 60 | 61 | /** 62 | * Implicit method that gives easy access to HBaseContext's bulk 63 | * get. This will return a new RDD. Think about it as a RDD map 64 | * function. In that every RDD value will get a new value out of 65 | * HBase. That new value will populate the newly generated RDD. 66 | * 67 | * @param hc The hbaseContext object to identify which 68 | * HBase cluster connection to use 69 | * @param tableName The tableName that the put will be sent to 70 | * @param batchSize How many gets to execute in a single batch 71 | * @param f The function that will turn the RDD values 72 | * in HBase Get objects 73 | * @param convertResult The function that will convert a HBase 74 | * Result object into a value that will go 75 | * into the resulting RDD 76 | * @tparam R The type of Object that will be coming 77 | * out of the resulting RDD 78 | * @return A resulting RDD with type R objects 79 | */ 80 | def hbaseBulkGet[R: ClassTag](hc: HBaseContext, 81 | tableName: TableName, batchSize:Int, 82 | f: (T) => Get, convertResult: (Result) => R): RDD[R] = { 83 | hc.bulkGet[T, R](tableName, batchSize, rdd, f, convertResult) 84 | } 85 | 86 | /** 87 | * Implicit method that gives easy access to HBaseContext's bulk 88 | * get. This will return a new RDD. Think about it as a RDD map 89 | * function. In that every RDD value will get a new value out of 90 | * HBase. That new value will populate the newly generated RDD. 91 | * 92 | * @param hc The hbaseContext object to identify which 93 | * HBase cluster connection to use 94 | * @param tableName The tableName that the put will be sent to 95 | * @param batchSize How many gets to execute in a single batch 96 | * @param f The function that will turn the RDD values 97 | * in HBase Get objects 98 | * @return A resulting RDD with type R objects 99 | */ 100 | def hbaseBulkGet(hc: HBaseContext, 101 | tableName: TableName, batchSize:Int, 102 | f: (T) => Get): RDD[(ImmutableBytesWritable, Result)] = { 103 | hc.bulkGet[T, (ImmutableBytesWritable, Result)](tableName, 104 | batchSize, rdd, f, 105 | result => if (result != null && result.getRow != null) { 106 | (new ImmutableBytesWritable(result.getRow), result) 107 | } else { 108 | null 109 | }) 110 | } 111 | 112 | /** 113 | * Implicit method that gives easy access to HBaseContext's bulk 114 | * Delete. This will not return a new RDD. 115 | * 116 | * @param hc The hbaseContext object to identify which HBase 117 | * cluster connection to use 118 | * @param tableName The tableName that the deletes will be sent to 119 | * @param f The function that will convert the RDD value into 120 | * a HBase Delete Object 121 | * @param batchSize The number of Deletes to be sent in a single batch 122 | */ 123 | def hbaseBulkDelete(hc: HBaseContext, 124 | tableName: TableName, f:(T) => Delete, batchSize:Int): Unit = { 125 | hc.bulkDelete(rdd, tableName, f, batchSize) 126 | } 127 | 128 | /** 129 | * Implicit method that gives easy access to HBaseContext's 130 | * foreachPartition method. This will ack very much like a normal RDD 131 | * foreach method but for the fact that you will now have a HBase connection 132 | * while iterating through the values. 133 | * 134 | * @param hc The hbaseContext object to identify which HBase 135 | * cluster connection to use 136 | * @param f This function will get an iterator for a Partition of an 137 | * RDD along with a connection object to HBase 138 | */ 139 | def hbaseForeachPartition(hc: HBaseContext, 140 | f: (Iterator[T], Connection) => Unit): Unit = { 141 | hc.foreachPartition(rdd, f) 142 | } 143 | 144 | /** 145 | * Implicit method that gives easy access to HBaseContext's 146 | * mapPartitions method. This will ask very much like a normal RDD 147 | * map partitions method but for the fact that you will now have a 148 | * HBase connection while iterating through the values 149 | * 150 | * @param hc The hbaseContext object to identify which HBase 151 | * cluster connection to use 152 | * @param f This function will get an iterator for a Partition of an 153 | * RDD along with a connection object to HBase 154 | * @tparam R This is the type of objects that will go into the resulting 155 | * RDD 156 | * @return A resulting RDD of type R 157 | */ 158 | def hbaseMapPartitions[R: ClassTag](hc: HBaseContext, 159 | f: (Iterator[T], Connection) => Iterator[R]): 160 | RDD[R] = { 161 | hc.mapPartitions[T,R](rdd, f) 162 | } 163 | 164 | /** 165 | * Spark Implementation of HBase Bulk load for wide rows or when 166 | * values are not already combined at the time of the map process 167 | * 168 | * A Spark Implementation of HBase Bulk load 169 | * 170 | * This will take the content from an existing RDD then sort and shuffle 171 | * it with respect to region splits. The result of that sort and shuffle 172 | * will be written to HFiles. 173 | * 174 | * After this function is executed the user will have to call 175 | * LoadIncrementalHFiles.doBulkLoad(...) to move the files into HBase 176 | * 177 | * Also note this version of bulk load is different from past versions in 178 | * that it includes the qualifier as part of the sort process. The 179 | * reason for this is to be able to support rows will very large number 180 | * of columns. 181 | * 182 | * @param tableName The HBase table we are loading into 183 | * @param flatMap A flapMap function that will make every row in the RDD 184 | * into N cells for the bulk load 185 | * @param stagingDir The location on the FileSystem to bulk load into 186 | * @param familyHFileWriteOptionsMap Options that will define how the HFile for a 187 | * column family is written 188 | * @param compactionExclude Compaction excluded for the HFiles 189 | * @param maxSize Max size for the HFiles before they roll 190 | */ 191 | def hbaseBulkLoad(hc: HBaseContext, 192 | tableName: TableName, 193 | flatMap: (T) => Iterator[(KeyFamilyQualifier, Array[Byte])], 194 | stagingDir:String, 195 | familyHFileWriteOptionsMap: 196 | util.Map[Array[Byte], FamilyHFileWriteOptions] = 197 | new util.HashMap[Array[Byte], FamilyHFileWriteOptions](), 198 | compactionExclude: Boolean = false, 199 | maxSize:Long = HConstants.DEFAULT_MAX_FILE_SIZE):Unit = { 200 | hc.bulkLoad(rdd, tableName, 201 | flatMap, stagingDir, familyHFileWriteOptionsMap, 202 | compactionExclude, maxSize) 203 | } 204 | 205 | /** 206 | * Implicit method that gives easy access to HBaseContext's 207 | * bulkLoadThinRows method. 208 | * 209 | * Spark Implementation of HBase Bulk load for short rows some where less then 210 | * a 1000 columns. This bulk load should be faster for tables will thinner 211 | * rows then the other spark implementation of bulk load that puts only one 212 | * value into a record going into a shuffle 213 | * 214 | * This will take the content from an existing RDD then sort and shuffle 215 | * it with respect to region splits. The result of that sort and shuffle 216 | * will be written to HFiles. 217 | * 218 | * After this function is executed the user will have to call 219 | * LoadIncrementalHFiles.doBulkLoad(...) to move the files into HBase 220 | * 221 | * In this implementation only the rowKey is given to the shuffle as the key 222 | * and all the columns are already linked to the RowKey before the shuffle 223 | * stage. The sorting of the qualifier is done in memory out side of the 224 | * shuffle stage 225 | * 226 | * @param tableName The HBase table we are loading into 227 | * @param mapFunction A function that will convert the RDD records to 228 | * the key value format used for the shuffle to prep 229 | * for writing to the bulk loaded HFiles 230 | * @param stagingDir The location on the FileSystem to bulk load into 231 | * @param familyHFileWriteOptionsMap Options that will define how the HFile for a 232 | * column family is written 233 | * @param compactionExclude Compaction excluded for the HFiles 234 | * @param maxSize Max size for the HFiles before they roll 235 | */ 236 | def hbaseBulkLoadThinRows(hc: HBaseContext, 237 | tableName: TableName, 238 | mapFunction: (T) => 239 | (ByteArrayWrapper, FamiliesQualifiersValues), 240 | stagingDir:String, 241 | familyHFileWriteOptionsMap: 242 | util.Map[Array[Byte], FamilyHFileWriteOptions] = 243 | new util.HashMap[Array[Byte], FamilyHFileWriteOptions](), 244 | compactionExclude: Boolean = false, 245 | maxSize:Long = HConstants.DEFAULT_MAX_FILE_SIZE):Unit = { 246 | hc.bulkLoadThinRows(rdd, tableName, 247 | mapFunction, stagingDir, familyHFileWriteOptionsMap, 248 | compactionExclude, maxSize) 249 | } 250 | } 251 | } 252 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/hadoop/hbase/spark/JavaHBaseContext.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.hadoop.hbase.spark 19 | 20 | import org.apache.hadoop.conf.Configuration 21 | import org.apache.hadoop.hbase.TableName 22 | import org.apache.hadoop.hbase.client.{Connection, Delete, Get, Put, Result, Scan} 23 | import org.apache.hadoop.hbase.io.ImmutableBytesWritable 24 | import org.apache.spark.api.java.{JavaRDD, JavaSparkContext} 25 | import org.apache.spark.api.java.function.{FlatMapFunction, Function, VoidFunction} 26 | import org.apache.spark.streaming.api.java.JavaDStream 27 | 28 | import scala.collection.JavaConversions._ 29 | import scala.reflect.ClassTag 30 | 31 | /** 32 | * This is the Java Wrapper over HBaseContext which is written in 33 | * Scala. This class will be used by developers that want to 34 | * work with Spark or Spark Streaming in Java 35 | * 36 | * @param jsc This is the JavaSparkContext that we will wrap 37 | * @param config This is the config information to out HBase cluster 38 | */ 39 | class JavaHBaseContext(@transient jsc: JavaSparkContext, 40 | @transient config: Configuration) extends Serializable { 41 | val hbaseContext = new HBaseContext(jsc.sc, config) 42 | 43 | /** 44 | * A simple enrichment of the traditional Spark javaRdd foreachPartition. 45 | * This function differs from the original in that it offers the 46 | * developer access to a already connected HConnection object 47 | * 48 | * Note: Do not close the HConnection object. All HConnection 49 | * management is handled outside this method 50 | * 51 | * @param javaRdd Original javaRdd with data to iterate over 52 | * @param f Function to be given a iterator to iterate through 53 | * the RDD values and a HConnection object to interact 54 | * with HBase 55 | */ 56 | def foreachPartition[T](javaRdd: JavaRDD[T], 57 | f: VoidFunction[(java.util.Iterator[T], Connection)]) = { 58 | 59 | hbaseContext.foreachPartition(javaRdd.rdd, 60 | (it: Iterator[T], conn: Connection) => { 61 | f.call((it, conn)) 62 | }) 63 | } 64 | 65 | /** 66 | * A simple enrichment of the traditional Spark Streaming dStream foreach 67 | * This function differs from the original in that it offers the 68 | * developer access to a already connected HConnection object 69 | * 70 | * Note: Do not close the HConnection object. All HConnection 71 | * management is handled outside this method 72 | * 73 | * @param javaDstream Original DStream with data to iterate over 74 | * @param f Function to be given a iterator to iterate through 75 | * the JavaDStream values and a HConnection object to 76 | * interact with HBase 77 | */ 78 | def foreachPartition[T](javaDstream: JavaDStream[T], 79 | f: VoidFunction[(Iterator[T], Connection)]) = { 80 | hbaseContext.foreachPartition(javaDstream.dstream, 81 | (it: Iterator[T], conn: Connection) => f.call(it, conn)) 82 | } 83 | 84 | /** 85 | * A simple enrichment of the traditional Spark JavaRDD mapPartition. 86 | * This function differs from the original in that it offers the 87 | * developer access to a already connected HConnection object 88 | * 89 | * Note: Do not close the HConnection object. All HConnection 90 | * management is handled outside this method 91 | * 92 | * Note: Make sure to partition correctly to avoid memory issue when 93 | * getting data from HBase 94 | * 95 | * @param javaRdd Original JavaRdd with data to iterate over 96 | * @param f Function to be given a iterator to iterate through 97 | * the RDD values and a HConnection object to interact 98 | * with HBase 99 | * @return Returns a new RDD generated by the user definition 100 | * function just like normal mapPartition 101 | */ 102 | def mapPartitions[T, R](javaRdd: JavaRDD[T], 103 | f: FlatMapFunction[(java.util.Iterator[T], 104 | Connection), R]): JavaRDD[R] = { 105 | 106 | def fn = (it: Iterator[T], conn: Connection) => 107 | asScalaIterator( 108 | f.call((asJavaIterator(it), conn)).iterator() 109 | ) 110 | 111 | JavaRDD.fromRDD(hbaseContext.mapPartitions(javaRdd.rdd, 112 | (iterator: Iterator[T], connection: Connection) => 113 | fn(iterator, connection))(fakeClassTag[R]))(fakeClassTag[R]) 114 | } 115 | 116 | /** 117 | * A simple enrichment of the traditional Spark Streaming JavaDStream 118 | * mapPartition. 119 | * 120 | * This function differs from the original in that it offers the 121 | * developer access to a already connected HConnection object 122 | * 123 | * Note: Do not close the HConnection object. All HConnection 124 | * management is handled outside this method 125 | * 126 | * Note: Make sure to partition correctly to avoid memory issue when 127 | * getting data from HBase 128 | * 129 | * @param javaDstream Original JavaDStream with data to iterate over 130 | * @param mp Function to be given a iterator to iterate through 131 | * the JavaDStream values and a HConnection object to 132 | * interact with HBase 133 | * @return Returns a new JavaDStream generated by the user 134 | * definition function just like normal mapPartition 135 | */ 136 | def streamMap[T, U](javaDstream: JavaDStream[T], 137 | mp: Function[(Iterator[T], Connection), Iterator[U]]): 138 | JavaDStream[U] = { 139 | JavaDStream.fromDStream(hbaseContext.streamMapPartitions(javaDstream.dstream, 140 | (it: Iterator[T], conn: Connection) => 141 | mp.call(it, conn))(fakeClassTag[U]))(fakeClassTag[U]) 142 | } 143 | 144 | /** 145 | * A simple abstraction over the HBaseContext.foreachPartition method. 146 | * 147 | * It allow addition support for a user to take JavaRDD 148 | * and generate puts and send them to HBase. 149 | * The complexity of managing the HConnection is 150 | * removed from the developer 151 | * 152 | * @param javaRdd Original JavaRDD with data to iterate over 153 | * @param tableName The name of the table to put into 154 | * @param f Function to convert a value in the JavaRDD 155 | * to a HBase Put 156 | */ 157 | def bulkPut[T](javaRdd: JavaRDD[T], 158 | tableName: TableName, 159 | f: Function[(T), Put]) { 160 | 161 | hbaseContext.bulkPut(javaRdd.rdd, tableName, (t: T) => f.call(t)) 162 | } 163 | 164 | /** 165 | * A simple abstraction over the HBaseContext.streamMapPartition method. 166 | * 167 | * It allow addition support for a user to take a JavaDStream and 168 | * generate puts and send them to HBase. 169 | * 170 | * The complexity of managing the HConnection is 171 | * removed from the developer 172 | * 173 | * @param javaDstream Original DStream with data to iterate over 174 | * @param tableName The name of the table to put into 175 | * @param f Function to convert a value in 176 | * the JavaDStream to a HBase Put 177 | */ 178 | def streamBulkPut[T](javaDstream: JavaDStream[T], 179 | tableName: TableName, 180 | f: Function[T, Put]) = { 181 | hbaseContext.streamBulkPut(javaDstream.dstream, 182 | tableName, 183 | (t: T) => f.call(t)) 184 | } 185 | 186 | /** 187 | * A simple abstraction over the HBaseContext.foreachPartition method. 188 | * 189 | * It allow addition support for a user to take a JavaRDD and 190 | * generate delete and send them to HBase. 191 | * 192 | * The complexity of managing the HConnection is 193 | * removed from the developer 194 | * 195 | * @param javaRdd Original JavaRDD with data to iterate over 196 | * @param tableName The name of the table to delete from 197 | * @param f Function to convert a value in the JavaRDD to a 198 | * HBase Deletes 199 | * @param batchSize The number of deletes to batch before sending to HBase 200 | */ 201 | def bulkDelete[T](javaRdd: JavaRDD[T], tableName: TableName, 202 | f: Function[T, Delete], batchSize: Integer) { 203 | hbaseContext.bulkDelete(javaRdd.rdd, tableName, (t: T) => f.call(t), batchSize) 204 | } 205 | 206 | /** 207 | * A simple abstraction over the HBaseContext.streamBulkMutation method. 208 | * 209 | * It allow addition support for a user to take a JavaDStream and 210 | * generate Delete and send them to HBase. 211 | * 212 | * The complexity of managing the HConnection is 213 | * removed from the developer 214 | * 215 | * @param javaDStream Original DStream with data to iterate over 216 | * @param tableName The name of the table to delete from 217 | * @param f Function to convert a value in the JavaDStream to a 218 | * HBase Delete 219 | * @param batchSize The number of deletes to be sent at once 220 | */ 221 | def streamBulkDelete[T](javaDStream: JavaDStream[T], 222 | tableName: TableName, 223 | f: Function[T, Delete], 224 | batchSize: Integer) = { 225 | hbaseContext.streamBulkDelete(javaDStream.dstream, tableName, 226 | (t: T) => f.call(t), 227 | batchSize) 228 | } 229 | 230 | /** 231 | * A simple abstraction over the HBaseContext.mapPartition method. 232 | * 233 | * It allow addition support for a user to take a JavaRDD and generates a 234 | * new RDD based on Gets and the results they bring back from HBase 235 | * 236 | * @param tableName The name of the table to get from 237 | * @param batchSize batch size of how many gets to retrieve in a single fetch 238 | * @param javaRdd Original JavaRDD with data to iterate over 239 | * @param makeGet Function to convert a value in the JavaRDD to a 240 | * HBase Get 241 | * @param convertResult This will convert the HBase Result object to 242 | * what ever the user wants to put in the resulting 243 | * JavaRDD 244 | * @return New JavaRDD that is created by the Get to HBase 245 | */ 246 | def bulkGet[T, U](tableName: TableName, 247 | batchSize: Integer, 248 | javaRdd: JavaRDD[T], 249 | makeGet: Function[T, Get], 250 | convertResult: Function[Result, U]): JavaRDD[U] = { 251 | 252 | JavaRDD.fromRDD(hbaseContext.bulkGet[T, U](tableName, 253 | batchSize, 254 | javaRdd.rdd, 255 | (t: T) => makeGet.call(t), 256 | (r: Result) => { 257 | convertResult.call(r) 258 | })(fakeClassTag[U]))(fakeClassTag[U]) 259 | 260 | } 261 | 262 | /** 263 | * A simple abstraction over the HBaseContext.streamMap method. 264 | * 265 | * It allow addition support for a user to take a DStream and 266 | * generates a new DStream based on Gets and the results 267 | * they bring back from HBase 268 | * 269 | 270 | * @param tableName The name of the table to get from 271 | * @param batchSize The number of gets to be batched together 272 | * @param javaDStream Original DStream with data to iterate over 273 | * @param makeGet Function to convert a value in the JavaDStream to a 274 | * HBase Get 275 | * @param convertResult This will convert the HBase Result object to 276 | * what ever the user wants to put in the resulting 277 | * JavaDStream 278 | * @return New JavaDStream that is created by the Get to HBase 279 | */ 280 | def streamBulkGet[T, U](tableName: TableName, 281 | batchSize: Integer, 282 | javaDStream: JavaDStream[T], 283 | makeGet: Function[T, Get], 284 | convertResult: Function[Result, U]) { 285 | JavaDStream.fromDStream(hbaseContext.streamBulkGet(tableName, 286 | batchSize, 287 | javaDStream.dstream, 288 | (t: T) => makeGet.call(t), 289 | (r: Result) => convertResult.call(r))(fakeClassTag[U]))(fakeClassTag[U]) 290 | } 291 | 292 | /** 293 | * This function will use the native HBase TableInputFormat with the 294 | * given scan object to generate a new JavaRDD 295 | * 296 | * @param tableName The name of the table to scan 297 | * @param scans The HBase scan object to use to read data from HBase 298 | * @param f Function to convert a Result object from HBase into 299 | * What the user wants in the final generated JavaRDD 300 | * @return New JavaRDD with results from scan 301 | */ 302 | def hbaseRDD[U](tableName: TableName, 303 | scans: Scan, 304 | f: Function[(ImmutableBytesWritable, Result), U]): 305 | JavaRDD[U] = { 306 | JavaRDD.fromRDD( 307 | hbaseContext.hbaseRDD[U](tableName, 308 | scans, 309 | (v: (ImmutableBytesWritable, Result)) => 310 | f.call(v._1, v._2))(fakeClassTag[U]))(fakeClassTag[U]) 311 | } 312 | 313 | /** 314 | * A overloaded version of HBaseContext hbaseRDD that define the 315 | * type of the resulting JavaRDD 316 | * 317 | * @param tableName The name of the table to scan 318 | * @param scans The HBase scan object to use to read data from HBase 319 | * @return New JavaRDD with results from scan 320 | */ 321 | def hbaseRDD(tableName: TableName, 322 | scans: Scan): 323 | JavaRDD[(ImmutableBytesWritable, Result)] = { 324 | JavaRDD.fromRDD(hbaseContext.hbaseRDD(tableName, scans)) 325 | } 326 | 327 | /** 328 | * Produces a ClassTag[T], which is actually just a casted ClassTag[AnyRef]. 329 | * 330 | * This method is used to keep ClassTags out of the external Java API, as the Java compiler 331 | * cannot produce them automatically. While this ClassTag-faking does please the compiler, 332 | * it can cause problems at runtime if the Scala API relies on ClassTags for correctness. 333 | * 334 | * Often, though, a ClassTag[AnyRef] will not lead to incorrect behavior, 335 | * just worse performance or security issues. 336 | * For instance, an Array[AnyRef] can hold any type T, 337 | * but may lose primitive 338 | * specialization. 339 | */ 340 | private[spark] 341 | def fakeClassTag[T]: ClassTag[T] = ClassTag.AnyRef.asInstanceOf[ClassTag[T]] 342 | } 343 | -------------------------------------------------------------------------------- /.settings/org.eclipse.jdt.core.prefs: -------------------------------------------------------------------------------- 1 | eclipse.preferences.version=1 2 | org.eclipse.jdt.core.compiler.codegen.targetPlatform=1.5 3 | org.eclipse.jdt.core.compiler.compliance=1.5 4 | org.eclipse.jdt.core.compiler.problem.forbiddenReference=warning 5 | org.eclipse.jdt.core.compiler.source=1.5 6 | org.eclipse.jdt.core.formatter.align_type_members_on_columns=false 7 | org.eclipse.jdt.core.formatter.alignment_for_arguments_in_allocation_expression=16 8 | org.eclipse.jdt.core.formatter.alignment_for_arguments_in_annotation=0 9 | org.eclipse.jdt.core.formatter.alignment_for_arguments_in_enum_constant=16 10 | org.eclipse.jdt.core.formatter.alignment_for_arguments_in_explicit_constructor_call=16 11 | org.eclipse.jdt.core.formatter.alignment_for_arguments_in_method_invocation=16 12 | org.eclipse.jdt.core.formatter.alignment_for_arguments_in_qualified_allocation_expression=16 13 | org.eclipse.jdt.core.formatter.alignment_for_assignment=0 14 | org.eclipse.jdt.core.formatter.alignment_for_binary_expression=16 15 | org.eclipse.jdt.core.formatter.alignment_for_compact_if=16 16 | org.eclipse.jdt.core.formatter.alignment_for_conditional_expression=80 17 | org.eclipse.jdt.core.formatter.alignment_for_enum_constants=0 18 | org.eclipse.jdt.core.formatter.alignment_for_expressions_in_array_initializer=16 19 | org.eclipse.jdt.core.formatter.alignment_for_method_declaration=0 20 | org.eclipse.jdt.core.formatter.alignment_for_multiple_fields=16 21 | org.eclipse.jdt.core.formatter.alignment_for_parameters_in_constructor_declaration=16 22 | org.eclipse.jdt.core.formatter.alignment_for_parameters_in_method_declaration=16 23 | org.eclipse.jdt.core.formatter.alignment_for_resources_in_try=80 24 | org.eclipse.jdt.core.formatter.alignment_for_selector_in_method_invocation=16 25 | org.eclipse.jdt.core.formatter.alignment_for_superclass_in_type_declaration=16 26 | org.eclipse.jdt.core.formatter.alignment_for_superinterfaces_in_enum_declaration=16 27 | org.eclipse.jdt.core.formatter.alignment_for_superinterfaces_in_type_declaration=16 28 | org.eclipse.jdt.core.formatter.alignment_for_throws_clause_in_constructor_declaration=16 29 | org.eclipse.jdt.core.formatter.alignment_for_throws_clause_in_method_declaration=16 30 | org.eclipse.jdt.core.formatter.alignment_for_union_type_in_multicatch=16 31 | org.eclipse.jdt.core.formatter.blank_lines_after_imports=1 32 | org.eclipse.jdt.core.formatter.blank_lines_after_package=1 33 | org.eclipse.jdt.core.formatter.blank_lines_before_field=0 34 | org.eclipse.jdt.core.formatter.blank_lines_before_first_class_body_declaration=0 35 | org.eclipse.jdt.core.formatter.blank_lines_before_imports=1 36 | org.eclipse.jdt.core.formatter.blank_lines_before_member_type=1 37 | org.eclipse.jdt.core.formatter.blank_lines_before_method=1 38 | org.eclipse.jdt.core.formatter.blank_lines_before_new_chunk=1 39 | org.eclipse.jdt.core.formatter.blank_lines_before_package=0 40 | org.eclipse.jdt.core.formatter.blank_lines_between_import_groups=1 41 | org.eclipse.jdt.core.formatter.blank_lines_between_type_declarations=1 42 | org.eclipse.jdt.core.formatter.brace_position_for_annotation_type_declaration=end_of_line 43 | org.eclipse.jdt.core.formatter.brace_position_for_anonymous_type_declaration=end_of_line 44 | org.eclipse.jdt.core.formatter.brace_position_for_array_initializer=end_of_line 45 | org.eclipse.jdt.core.formatter.brace_position_for_block=end_of_line 46 | org.eclipse.jdt.core.formatter.brace_position_for_block_in_case=end_of_line 47 | org.eclipse.jdt.core.formatter.brace_position_for_constructor_declaration=end_of_line 48 | org.eclipse.jdt.core.formatter.brace_position_for_enum_constant=end_of_line 49 | org.eclipse.jdt.core.formatter.brace_position_for_enum_declaration=end_of_line 50 | org.eclipse.jdt.core.formatter.brace_position_for_method_declaration=end_of_line 51 | org.eclipse.jdt.core.formatter.brace_position_for_switch=end_of_line 52 | org.eclipse.jdt.core.formatter.brace_position_for_type_declaration=end_of_line 53 | org.eclipse.jdt.core.formatter.comment.clear_blank_lines_in_block_comment=false 54 | org.eclipse.jdt.core.formatter.comment.clear_blank_lines_in_javadoc_comment=false 55 | org.eclipse.jdt.core.formatter.comment.format_block_comments=true 56 | org.eclipse.jdt.core.formatter.comment.format_header=false 57 | org.eclipse.jdt.core.formatter.comment.format_html=true 58 | org.eclipse.jdt.core.formatter.comment.format_javadoc_comments=true 59 | org.eclipse.jdt.core.formatter.comment.format_line_comments=true 60 | org.eclipse.jdt.core.formatter.comment.format_source_code=true 61 | org.eclipse.jdt.core.formatter.comment.indent_parameter_description=true 62 | org.eclipse.jdt.core.formatter.comment.indent_root_tags=true 63 | org.eclipse.jdt.core.formatter.comment.insert_new_line_before_root_tags=insert 64 | org.eclipse.jdt.core.formatter.comment.insert_new_line_for_parameter=insert 65 | org.eclipse.jdt.core.formatter.comment.line_length=80 66 | org.eclipse.jdt.core.formatter.comment.new_lines_at_block_boundaries=true 67 | org.eclipse.jdt.core.formatter.comment.new_lines_at_javadoc_boundaries=true 68 | org.eclipse.jdt.core.formatter.comment.preserve_white_space_between_code_and_line_comments=false 69 | org.eclipse.jdt.core.formatter.compact_else_if=true 70 | org.eclipse.jdt.core.formatter.continuation_indentation=2 71 | org.eclipse.jdt.core.formatter.continuation_indentation_for_array_initializer=2 72 | org.eclipse.jdt.core.formatter.disabling_tag=@formatter\:off 73 | org.eclipse.jdt.core.formatter.enabling_tag=@formatter\:on 74 | org.eclipse.jdt.core.formatter.format_guardian_clause_on_one_line=false 75 | org.eclipse.jdt.core.formatter.format_line_comment_starting_on_first_column=true 76 | org.eclipse.jdt.core.formatter.indent_body_declarations_compare_to_annotation_declaration_header=true 77 | org.eclipse.jdt.core.formatter.indent_body_declarations_compare_to_enum_constant_header=true 78 | org.eclipse.jdt.core.formatter.indent_body_declarations_compare_to_enum_declaration_header=true 79 | org.eclipse.jdt.core.formatter.indent_body_declarations_compare_to_type_header=true 80 | org.eclipse.jdt.core.formatter.indent_breaks_compare_to_cases=true 81 | org.eclipse.jdt.core.formatter.indent_empty_lines=false 82 | org.eclipse.jdt.core.formatter.indent_statements_compare_to_block=true 83 | org.eclipse.jdt.core.formatter.indent_statements_compare_to_body=true 84 | org.eclipse.jdt.core.formatter.indent_switchstatements_compare_to_cases=true 85 | org.eclipse.jdt.core.formatter.indent_switchstatements_compare_to_switch=false 86 | org.eclipse.jdt.core.formatter.indentation.size=2 87 | org.eclipse.jdt.core.formatter.insert_new_line_after_annotation_on_field=insert 88 | org.eclipse.jdt.core.formatter.insert_new_line_after_annotation_on_local_variable=insert 89 | org.eclipse.jdt.core.formatter.insert_new_line_after_annotation_on_method=insert 90 | org.eclipse.jdt.core.formatter.insert_new_line_after_annotation_on_package=insert 91 | org.eclipse.jdt.core.formatter.insert_new_line_after_annotation_on_parameter=do not insert 92 | org.eclipse.jdt.core.formatter.insert_new_line_after_annotation_on_type=insert 93 | org.eclipse.jdt.core.formatter.insert_new_line_after_label=do not insert 94 | org.eclipse.jdt.core.formatter.insert_new_line_after_opening_brace_in_array_initializer=do not insert 95 | org.eclipse.jdt.core.formatter.insert_new_line_at_end_of_file_if_missing=do not insert 96 | org.eclipse.jdt.core.formatter.insert_new_line_before_catch_in_try_statement=do not insert 97 | org.eclipse.jdt.core.formatter.insert_new_line_before_closing_brace_in_array_initializer=do not insert 98 | org.eclipse.jdt.core.formatter.insert_new_line_before_else_in_if_statement=do not insert 99 | org.eclipse.jdt.core.formatter.insert_new_line_before_finally_in_try_statement=do not insert 100 | org.eclipse.jdt.core.formatter.insert_new_line_before_while_in_do_statement=do not insert 101 | org.eclipse.jdt.core.formatter.insert_new_line_in_empty_annotation_declaration=insert 102 | org.eclipse.jdt.core.formatter.insert_new_line_in_empty_anonymous_type_declaration=insert 103 | org.eclipse.jdt.core.formatter.insert_new_line_in_empty_block=insert 104 | org.eclipse.jdt.core.formatter.insert_new_line_in_empty_enum_constant=insert 105 | org.eclipse.jdt.core.formatter.insert_new_line_in_empty_enum_declaration=insert 106 | org.eclipse.jdt.core.formatter.insert_new_line_in_empty_method_body=insert 107 | org.eclipse.jdt.core.formatter.insert_new_line_in_empty_type_declaration=insert 108 | org.eclipse.jdt.core.formatter.insert_space_after_and_in_type_parameter=insert 109 | org.eclipse.jdt.core.formatter.insert_space_after_assignment_operator=insert 110 | org.eclipse.jdt.core.formatter.insert_space_after_at_in_annotation=do not insert 111 | org.eclipse.jdt.core.formatter.insert_space_after_at_in_annotation_type_declaration=do not insert 112 | org.eclipse.jdt.core.formatter.insert_space_after_binary_operator=insert 113 | org.eclipse.jdt.core.formatter.insert_space_after_closing_angle_bracket_in_type_arguments=insert 114 | org.eclipse.jdt.core.formatter.insert_space_after_closing_angle_bracket_in_type_parameters=insert 115 | org.eclipse.jdt.core.formatter.insert_space_after_closing_brace_in_block=insert 116 | org.eclipse.jdt.core.formatter.insert_space_after_closing_paren_in_cast=insert 117 | org.eclipse.jdt.core.formatter.insert_space_after_colon_in_assert=insert 118 | org.eclipse.jdt.core.formatter.insert_space_after_colon_in_case=insert 119 | org.eclipse.jdt.core.formatter.insert_space_after_colon_in_conditional=insert 120 | org.eclipse.jdt.core.formatter.insert_space_after_colon_in_for=insert 121 | org.eclipse.jdt.core.formatter.insert_space_after_colon_in_labeled_statement=insert 122 | org.eclipse.jdt.core.formatter.insert_space_after_comma_in_allocation_expression=insert 123 | org.eclipse.jdt.core.formatter.insert_space_after_comma_in_annotation=insert 124 | org.eclipse.jdt.core.formatter.insert_space_after_comma_in_array_initializer=insert 125 | org.eclipse.jdt.core.formatter.insert_space_after_comma_in_constructor_declaration_parameters=insert 126 | org.eclipse.jdt.core.formatter.insert_space_after_comma_in_constructor_declaration_throws=insert 127 | org.eclipse.jdt.core.formatter.insert_space_after_comma_in_enum_constant_arguments=insert 128 | org.eclipse.jdt.core.formatter.insert_space_after_comma_in_enum_declarations=insert 129 | org.eclipse.jdt.core.formatter.insert_space_after_comma_in_explicitconstructorcall_arguments=insert 130 | org.eclipse.jdt.core.formatter.insert_space_after_comma_in_for_increments=insert 131 | org.eclipse.jdt.core.formatter.insert_space_after_comma_in_for_inits=insert 132 | org.eclipse.jdt.core.formatter.insert_space_after_comma_in_method_declaration_parameters=insert 133 | org.eclipse.jdt.core.formatter.insert_space_after_comma_in_method_declaration_throws=insert 134 | org.eclipse.jdt.core.formatter.insert_space_after_comma_in_method_invocation_arguments=insert 135 | org.eclipse.jdt.core.formatter.insert_space_after_comma_in_multiple_field_declarations=insert 136 | org.eclipse.jdt.core.formatter.insert_space_after_comma_in_multiple_local_declarations=insert 137 | org.eclipse.jdt.core.formatter.insert_space_after_comma_in_parameterized_type_reference=insert 138 | org.eclipse.jdt.core.formatter.insert_space_after_comma_in_superinterfaces=insert 139 | org.eclipse.jdt.core.formatter.insert_space_after_comma_in_type_arguments=insert 140 | org.eclipse.jdt.core.formatter.insert_space_after_comma_in_type_parameters=insert 141 | org.eclipse.jdt.core.formatter.insert_space_after_ellipsis=insert 142 | org.eclipse.jdt.core.formatter.insert_space_after_opening_angle_bracket_in_parameterized_type_reference=do not insert 143 | org.eclipse.jdt.core.formatter.insert_space_after_opening_angle_bracket_in_type_arguments=do not insert 144 | org.eclipse.jdt.core.formatter.insert_space_after_opening_angle_bracket_in_type_parameters=do not insert 145 | org.eclipse.jdt.core.formatter.insert_space_after_opening_brace_in_array_initializer=insert 146 | org.eclipse.jdt.core.formatter.insert_space_after_opening_bracket_in_array_allocation_expression=do not insert 147 | org.eclipse.jdt.core.formatter.insert_space_after_opening_bracket_in_array_reference=do not insert 148 | org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_annotation=do not insert 149 | org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_cast=do not insert 150 | org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_catch=do not insert 151 | org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_constructor_declaration=do not insert 152 | org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_enum_constant=do not insert 153 | org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_for=do not insert 154 | org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_if=do not insert 155 | org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_method_declaration=do not insert 156 | org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_method_invocation=do not insert 157 | org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_parenthesized_expression=do not insert 158 | org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_switch=do not insert 159 | org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_synchronized=do not insert 160 | org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_try=do not insert 161 | org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_while=do not insert 162 | org.eclipse.jdt.core.formatter.insert_space_after_postfix_operator=do not insert 163 | org.eclipse.jdt.core.formatter.insert_space_after_prefix_operator=do not insert 164 | org.eclipse.jdt.core.formatter.insert_space_after_question_in_conditional=insert 165 | org.eclipse.jdt.core.formatter.insert_space_after_question_in_wildcard=do not insert 166 | org.eclipse.jdt.core.formatter.insert_space_after_semicolon_in_for=insert 167 | org.eclipse.jdt.core.formatter.insert_space_after_semicolon_in_try_resources=insert 168 | org.eclipse.jdt.core.formatter.insert_space_after_unary_operator=do not insert 169 | org.eclipse.jdt.core.formatter.insert_space_before_and_in_type_parameter=insert 170 | org.eclipse.jdt.core.formatter.insert_space_before_assignment_operator=insert 171 | org.eclipse.jdt.core.formatter.insert_space_before_at_in_annotation_type_declaration=insert 172 | org.eclipse.jdt.core.formatter.insert_space_before_binary_operator=insert 173 | org.eclipse.jdt.core.formatter.insert_space_before_closing_angle_bracket_in_parameterized_type_reference=do not insert 174 | org.eclipse.jdt.core.formatter.insert_space_before_closing_angle_bracket_in_type_arguments=do not insert 175 | org.eclipse.jdt.core.formatter.insert_space_before_closing_angle_bracket_in_type_parameters=do not insert 176 | org.eclipse.jdt.core.formatter.insert_space_before_closing_brace_in_array_initializer=insert 177 | org.eclipse.jdt.core.formatter.insert_space_before_closing_bracket_in_array_allocation_expression=do not insert 178 | org.eclipse.jdt.core.formatter.insert_space_before_closing_bracket_in_array_reference=do not insert 179 | org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_annotation=do not insert 180 | org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_cast=do not insert 181 | org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_catch=do not insert 182 | org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_constructor_declaration=do not insert 183 | org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_enum_constant=do not insert 184 | org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_for=do not insert 185 | org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_if=do not insert 186 | org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_method_declaration=do not insert 187 | org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_method_invocation=do not insert 188 | org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_parenthesized_expression=do not insert 189 | org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_switch=do not insert 190 | org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_synchronized=do not insert 191 | org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_try=do not insert 192 | org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_while=do not insert 193 | org.eclipse.jdt.core.formatter.insert_space_before_colon_in_assert=insert 194 | org.eclipse.jdt.core.formatter.insert_space_before_colon_in_case=do not insert 195 | org.eclipse.jdt.core.formatter.insert_space_before_colon_in_conditional=insert 196 | org.eclipse.jdt.core.formatter.insert_space_before_colon_in_default=do not insert 197 | org.eclipse.jdt.core.formatter.insert_space_before_colon_in_for=insert 198 | org.eclipse.jdt.core.formatter.insert_space_before_colon_in_labeled_statement=do not insert 199 | org.eclipse.jdt.core.formatter.insert_space_before_comma_in_allocation_expression=do not insert 200 | org.eclipse.jdt.core.formatter.insert_space_before_comma_in_annotation=do not insert 201 | org.eclipse.jdt.core.formatter.insert_space_before_comma_in_array_initializer=do not insert 202 | org.eclipse.jdt.core.formatter.insert_space_before_comma_in_constructor_declaration_parameters=do not insert 203 | org.eclipse.jdt.core.formatter.insert_space_before_comma_in_constructor_declaration_throws=do not insert 204 | org.eclipse.jdt.core.formatter.insert_space_before_comma_in_enum_constant_arguments=do not insert 205 | org.eclipse.jdt.core.formatter.insert_space_before_comma_in_enum_declarations=do not insert 206 | org.eclipse.jdt.core.formatter.insert_space_before_comma_in_explicitconstructorcall_arguments=do not insert 207 | org.eclipse.jdt.core.formatter.insert_space_before_comma_in_for_increments=do not insert 208 | org.eclipse.jdt.core.formatter.insert_space_before_comma_in_for_inits=do not insert 209 | org.eclipse.jdt.core.formatter.insert_space_before_comma_in_method_declaration_parameters=do not insert 210 | org.eclipse.jdt.core.formatter.insert_space_before_comma_in_method_declaration_throws=do not insert 211 | org.eclipse.jdt.core.formatter.insert_space_before_comma_in_method_invocation_arguments=do not insert 212 | org.eclipse.jdt.core.formatter.insert_space_before_comma_in_multiple_field_declarations=do not insert 213 | org.eclipse.jdt.core.formatter.insert_space_before_comma_in_multiple_local_declarations=do not insert 214 | org.eclipse.jdt.core.formatter.insert_space_before_comma_in_parameterized_type_reference=do not insert 215 | org.eclipse.jdt.core.formatter.insert_space_before_comma_in_superinterfaces=do not insert 216 | org.eclipse.jdt.core.formatter.insert_space_before_comma_in_type_arguments=do not insert 217 | org.eclipse.jdt.core.formatter.insert_space_before_comma_in_type_parameters=do not insert 218 | org.eclipse.jdt.core.formatter.insert_space_before_ellipsis=do not insert 219 | org.eclipse.jdt.core.formatter.insert_space_before_opening_angle_bracket_in_parameterized_type_reference=do not insert 220 | org.eclipse.jdt.core.formatter.insert_space_before_opening_angle_bracket_in_type_arguments=do not insert 221 | org.eclipse.jdt.core.formatter.insert_space_before_opening_angle_bracket_in_type_parameters=do not insert 222 | org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_annotation_type_declaration=insert 223 | org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_anonymous_type_declaration=insert 224 | org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_array_initializer=insert 225 | org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_block=insert 226 | org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_constructor_declaration=insert 227 | org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_enum_constant=insert 228 | org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_enum_declaration=insert 229 | org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_method_declaration=insert 230 | org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_switch=insert 231 | org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_type_declaration=insert 232 | org.eclipse.jdt.core.formatter.insert_space_before_opening_bracket_in_array_allocation_expression=do not insert 233 | org.eclipse.jdt.core.formatter.insert_space_before_opening_bracket_in_array_reference=do not insert 234 | org.eclipse.jdt.core.formatter.insert_space_before_opening_bracket_in_array_type_reference=do not insert 235 | org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_annotation=do not insert 236 | org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_annotation_type_member_declaration=do not insert 237 | org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_catch=insert 238 | org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_constructor_declaration=do not insert 239 | org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_enum_constant=do not insert 240 | org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_for=insert 241 | org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_if=insert 242 | org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_method_declaration=do not insert 243 | org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_method_invocation=do not insert 244 | org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_parenthesized_expression=do not insert 245 | org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_switch=insert 246 | org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_synchronized=insert 247 | org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_try=insert 248 | org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_while=insert 249 | org.eclipse.jdt.core.formatter.insert_space_before_parenthesized_expression_in_return=insert 250 | org.eclipse.jdt.core.formatter.insert_space_before_parenthesized_expression_in_throw=insert 251 | org.eclipse.jdt.core.formatter.insert_space_before_postfix_operator=do not insert 252 | org.eclipse.jdt.core.formatter.insert_space_before_prefix_operator=do not insert 253 | org.eclipse.jdt.core.formatter.insert_space_before_question_in_conditional=insert 254 | org.eclipse.jdt.core.formatter.insert_space_before_question_in_wildcard=do not insert 255 | org.eclipse.jdt.core.formatter.insert_space_before_semicolon=do not insert 256 | org.eclipse.jdt.core.formatter.insert_space_before_semicolon_in_for=do not insert 257 | org.eclipse.jdt.core.formatter.insert_space_before_semicolon_in_try_resources=do not insert 258 | org.eclipse.jdt.core.formatter.insert_space_before_unary_operator=do not insert 259 | org.eclipse.jdt.core.formatter.insert_space_between_brackets_in_array_type_reference=do not insert 260 | org.eclipse.jdt.core.formatter.insert_space_between_empty_braces_in_array_initializer=do not insert 261 | org.eclipse.jdt.core.formatter.insert_space_between_empty_brackets_in_array_allocation_expression=do not insert 262 | org.eclipse.jdt.core.formatter.insert_space_between_empty_parens_in_annotation_type_member_declaration=do not insert 263 | org.eclipse.jdt.core.formatter.insert_space_between_empty_parens_in_constructor_declaration=do not insert 264 | org.eclipse.jdt.core.formatter.insert_space_between_empty_parens_in_enum_constant=do not insert 265 | org.eclipse.jdt.core.formatter.insert_space_between_empty_parens_in_method_declaration=do not insert 266 | org.eclipse.jdt.core.formatter.insert_space_between_empty_parens_in_method_invocation=do not insert 267 | org.eclipse.jdt.core.formatter.join_lines_in_comments=true 268 | org.eclipse.jdt.core.formatter.join_wrapped_lines=true 269 | org.eclipse.jdt.core.formatter.keep_else_statement_on_same_line=false 270 | org.eclipse.jdt.core.formatter.keep_empty_array_initializer_on_one_line=false 271 | org.eclipse.jdt.core.formatter.keep_imple_if_on_one_line=false 272 | org.eclipse.jdt.core.formatter.keep_then_statement_on_same_line=false 273 | org.eclipse.jdt.core.formatter.lineSplit=80 274 | org.eclipse.jdt.core.formatter.never_indent_block_comments_on_first_column=false 275 | org.eclipse.jdt.core.formatter.never_indent_line_comments_on_first_column=false 276 | org.eclipse.jdt.core.formatter.number_of_blank_lines_at_beginning_of_method_body=0 277 | org.eclipse.jdt.core.formatter.number_of_empty_lines_to_preserve=1 278 | org.eclipse.jdt.core.formatter.put_empty_statement_on_new_line=true 279 | org.eclipse.jdt.core.formatter.tabulation.char=space 280 | org.eclipse.jdt.core.formatter.tabulation.size=2 281 | org.eclipse.jdt.core.formatter.use_on_off_tags=false 282 | org.eclipse.jdt.core.formatter.use_tabs_only_for_leading_indentations=false 283 | org.eclipse.jdt.core.formatter.wrap_before_binary_operator=true 284 | org.eclipse.jdt.core.formatter.wrap_before_or_operator_multicatch=true 285 | org.eclipse.jdt.core.formatter.wrap_outer_expressions_when_nested=true 286 | -------------------------------------------------------------------------------- /pom.xml: -------------------------------------------------------------------------------- 1 | 2 | 5 | 4.0.0 6 | 7 | com.cloudera.sa.sparkonhbase 8 | patientproject 9 | 1.0-SNAPSHOT 10 | 11 | 12 | 1.7 13 | 1.7 14 | 1.5.0-cdh5.5.0 15 | 1.0.0-cdh5.5.0 16 | 2.6.0-cdh5.5.0 17 | 0.8.2.0-kafka-1.4.0 18 | 1.6.0-cdh5.5.0 19 | 1.5.0-cdh5.5.0 20 | 21 | 22 | 23 | 24 | org.joda 25 | joda-convert 26 | 1.8 27 | 28 | 29 | junit 30 | junit 31 | 4.11 32 | test 33 | 34 | 35 | org.scala-lang 36 | scala-compiler 37 | 2.10.4 38 | 39 | 40 | org.scalatest 41 | scalatest_2.10 42 | 2.1.5 43 | 44 | 45 | org.apache.spark 46 | spark-core_2.10 47 | ${spark.version} 48 | 49 | 50 | org.apache.spark 51 | spark-sql_2.10 52 | ${spark.version} 53 | 54 | 55 | org.apache.spark 56 | spark-hive_2.10 57 | ${spark.version} 58 | 59 | 60 | org.apache.spark 61 | spark-streaming_2.10 62 | ${spark.version} 63 | 64 | 65 | org.apache.spark 66 | spark-streaming_2.10 67 | ${spark.version} 68 | test-jar 69 | tests 70 | test 71 | 72 | 73 | org.apache.hbase 74 | hbase-server 75 | ${hbase.version} 76 | 77 | 78 | org.apache.hbase 79 | hbase-server 80 | ${hbase.version} 81 | test-jar 82 | 83 | 84 | org.apache.hadoop 85 | hadoop-client 86 | ${hadoop.version} 87 | 88 | 89 | log4j 90 | log4j 91 | 92 | 93 | javax.servlet 94 | servlet-api 95 | 96 | 97 | javax.servlet.jsp 98 | jsp-api 99 | 100 | 101 | org.jruby 102 | jruby-complete 103 | 104 | 105 | org.jboss.netty 106 | netty 107 | 108 | 109 | io.netty 110 | netty 111 | 112 | 113 | 114 | 115 | org.apache.hadoop 116 | hadoop-common 117 | ${hadoop.version} 118 | 119 | 120 | log4j 121 | log4j 122 | 123 | 124 | javax.servlet 125 | servlet-api 126 | 127 | 128 | javax.servlet.jsp 129 | jsp-api 130 | 131 | 132 | org.jruby 133 | jruby-complete 134 | 135 | 136 | org.jboss.netty 137 | netty 138 | 139 | 140 | io.netty 141 | netty 142 | 143 | 144 | 145 | 146 | 147 | org.apache.hadoop 148 | hadoop-common 149 | ${hadoop.version} 150 | test-jar 151 | test 152 | 153 | 154 | log4j 155 | log4j 156 | 157 | 158 | javax.servlet 159 | servlet-api 160 | 161 | 162 | javax.servlet.jsp 163 | jsp-api 164 | 165 | 166 | org.jruby 167 | jruby-complete 168 | 169 | 170 | org.jboss.netty 171 | netty 172 | 173 | 174 | io.netty 175 | netty 176 | 177 | 178 | 179 | 180 | 181 | org.apache.hadoop 182 | hadoop-hdfs 183 | ${hadoop.version} 184 | test-jar 185 | test 186 | 187 | 188 | log4j 189 | log4j 190 | 191 | 192 | javax.servlet 193 | servlet-api 194 | 195 | 196 | javax.servlet.jsp 197 | jsp-api 198 | 199 | 200 | org.jruby 201 | jruby-complete 202 | 203 | 204 | org.jboss.netty 205 | netty 206 | 207 | 208 | io.netty 209 | netty 210 | 211 | 212 | 213 | 214 | 215 | org.apache.hbase 216 | hbase-client 217 | ${hbase.version} 218 | 219 | 220 | log4j 221 | log4j 222 | 223 | 224 | org.apache.thrift 225 | thrift 226 | 227 | 228 | org.jruby 229 | jruby-complete 230 | 231 | 232 | org.slf4j 233 | slf4j-log4j12 234 | 235 | 236 | org.mortbay.jetty 237 | jsp-2.1 238 | 239 | 240 | org.mortbay.jetty 241 | jsp-api-2.1 242 | 243 | 244 | org.mortbay.jetty 245 | servlet-api-2.5 246 | 247 | 248 | com.sun.jersey 249 | jersey-core 250 | 251 | 252 | com.sun.jersey 253 | jersey-json 254 | 255 | 256 | com.sun.jersey 257 | jersey-server 258 | 259 | 260 | org.mortbay.jetty 261 | jetty 262 | 263 | 264 | org.mortbay.jetty 265 | jetty-util 266 | 267 | 268 | tomcat 269 | jasper-runtime 270 | 271 | 272 | tomcat 273 | jasper-compiler 274 | 275 | 276 | org.jruby 277 | jruby-complete 278 | 279 | 280 | org.jboss.netty 281 | netty 282 | 283 | 284 | io.netty 285 | netty 286 | 287 | 288 | 289 | 290 | 291 | org.apache.hbase 292 | hbase-protocol 293 | ${hbase.version} 294 | 295 | 296 | 297 | org.apache.hbase 298 | hbase-annotations 299 | ${hbase.version} 300 | test-jar 301 | test 302 | 303 | 304 | 305 | org.apache.hbase 306 | hbase-hadoop-compat 307 | ${hbase.version} 308 | test 309 | test-jar 310 | 311 | 312 | log4j 313 | log4j 314 | 315 | 316 | org.apache.thrift 317 | thrift 318 | 319 | 320 | org.jruby 321 | jruby-complete 322 | 323 | 324 | org.slf4j 325 | slf4j-log4j12 326 | 327 | 328 | org.mortbay.jetty 329 | jsp-2.1 330 | 331 | 332 | org.mortbay.jetty 333 | jsp-api-2.1 334 | 335 | 336 | org.mortbay.jetty 337 | servlet-api-2.5 338 | 339 | 340 | com.sun.jersey 341 | jersey-core 342 | 343 | 344 | com.sun.jersey 345 | jersey-json 346 | 347 | 348 | com.sun.jersey 349 | jersey-server 350 | 351 | 352 | org.mortbay.jetty 353 | jetty 354 | 355 | 356 | org.mortbay.jetty 357 | jetty-util 358 | 359 | 360 | tomcat 361 | jasper-runtime 362 | 363 | 364 | tomcat 365 | jasper-compiler 366 | 367 | 368 | org.jruby 369 | jruby-complete 370 | 371 | 372 | org.jboss.netty 373 | netty 374 | 375 | 376 | io.netty 377 | netty 378 | 379 | 380 | 381 | 382 | 383 | org.apache.hbase 384 | hbase-hadoop2-compat 385 | ${hbase.version} 386 | test 387 | test-jar 388 | 389 | 390 | log4j 391 | log4j 392 | 393 | 394 | org.apache.thrift 395 | thrift 396 | 397 | 398 | org.jruby 399 | jruby-complete 400 | 401 | 402 | org.slf4j 403 | slf4j-log4j12 404 | 405 | 406 | org.mortbay.jetty 407 | jsp-2.1 408 | 409 | 410 | org.mortbay.jetty 411 | jsp-api-2.1 412 | 413 | 414 | org.mortbay.jetty 415 | servlet-api-2.5 416 | 417 | 418 | com.sun.jersey 419 | jersey-core 420 | 421 | 422 | com.sun.jersey 423 | jersey-json 424 | 425 | 426 | com.sun.jersey 427 | jersey-server 428 | 429 | 430 | org.mortbay.jetty 431 | jetty 432 | 433 | 434 | org.mortbay.jetty 435 | jetty-util 436 | 437 | 438 | tomcat 439 | jasper-runtime 440 | 441 | 442 | tomcat 443 | jasper-compiler 444 | 445 | 446 | org.jruby 447 | jruby-complete 448 | 449 | 450 | org.jboss.netty 451 | netty 452 | 453 | 454 | io.netty 455 | netty 456 | 457 | 458 | 459 | 460 | com.google.protobuf 461 | protobuf-java 462 | 2.5.0 463 | 464 | 465 | commons-logging 466 | commons-logging 467 | 1.2 468 | 469 | 470 | org.kohsuke 471 | wordnet-random-name 472 | 1.3 473 | 474 | 475 | org.apache.kafka 476 | kafka-clients 477 | ${kafka.version} 478 | 479 | 480 | org.apache.flume.flume-ng-sinks 481 | flume-ng-hbase-sink 482 | ${flume.version} 483 | 484 | 485 | 486 | com.twitter 487 | parquet-common 488 | ${parquet.version} 489 | 490 | 491 | com.twitter 492 | parquet-encoding 493 | ${parquet.version} 494 | 495 | 496 | com.twitter 497 | parquet-column 498 | ${parquet.version} 499 | 500 | 501 | com.twitter 502 | parquet-hadoop 503 | ${parquet.version} 504 | 505 | 506 | org.reflections 507 | reflections 508 | 0.9.10 509 | 510 | 511 | 512 | 513 | 514 | cloudera 515 | https://repository.cloudera.com/artifactory/cloudera-repos/ 516 | 517 | 518 | 519 | 520 | 521 | cloudera 522 | https://repository.cloudera.com/artifactory/cloudera-repos/ 523 | 524 | 525 | 526 | 527 | target/scala/classes 528 | target/scala/test-classes 529 | 530 | 531 | org.apache.maven.plugins 532 | maven-compiler-plugin 533 | 534 | 1.7 535 | 1.7 536 | 537 | 538 | 539 | org.apache.maven.plugins 540 | maven-surefire-plugin 541 | 2.17 542 | 543 | 544 | org.apache.maven.surefire 545 | surefire-junit47 546 | 2.17 547 | 548 | 549 | 550 | 551 | org.scalatest 552 | scalatest-maven-plugin 553 | 1.0 554 | 555 | . 556 | 557 | 558 | 559 | test 560 | 561 | test 562 | 563 | 564 | 565 | 566 | 567 | org.scala-tools 568 | maven-scala-plugin 569 | 570 | 571 | compile 572 | 573 | compile 574 | 575 | compile 576 | 577 | 578 | test-compile 579 | 580 | testCompile 581 | 582 | test-compile 583 | 584 | 585 | process-resources 586 | 587 | compile 588 | 589 | 590 | 591 | 592 | 593 | org.apache.maven.plugins 594 | maven-shade-plugin 595 | 2.2 596 | 597 | false 598 | target/SparkOnHBase.jar 599 | 600 | 601 | *:* 602 | 603 | 604 | 605 | 606 | *:* 607 | 608 | META-INF/*.SF 609 | META-INF/*.DSA 610 | META-INF/*.RSA 611 | 612 | 613 | 614 | 615 | 616 | 617 | package 618 | 619 | shade 620 | 621 | 622 | 623 | 625 | 627 | reference.conf 628 | 629 | 630 | 631 | 632 | 633 | 634 | 635 | 636 | --------------------------------------------------------------------------------