└── spark-hbase ├── src ├── main │ ├── resources │ │ ├── show │ │ │ ├── INSERT_QUERY.png │ │ │ └── SELECT_QUERY.png │ │ ├── spark_hbase.properties │ │ ├── test.yml │ │ ├── hbase-site.xml │ │ └── log4j.properties │ ├── scala │ │ └── org │ │ │ └── apache │ │ │ └── spark │ │ │ └── sql │ │ │ └── hbase │ │ │ ├── execution │ │ │ ├── RowKeyGenerator.scala │ │ │ ├── HBaseSqlParser.scala │ │ │ ├── CreateHBaseTableAsSelectCommand.scala │ │ │ ├── HBaseTableScanExec.scala │ │ │ ├── InsertIntoHBaseTable.scala │ │ │ ├── HBaseTableFormat.scala │ │ │ └── HBaseFileFormat.scala │ │ │ ├── utils │ │ │ ├── HBaseSparkFormatUtils.scala │ │ │ ├── SerializableRegionInfo.scala │ │ │ ├── StructFieldConverters.scala │ │ │ ├── HBaseSparkDataUtils.scala │ │ │ └── HBaseSparkFilterUtils.scala │ │ │ ├── types │ │ │ └── RegionInfoUDT.scala │ │ │ ├── HBasePlan.scala │ │ │ ├── client │ │ │ ├── package.scala │ │ │ ├── HBaseClient.scala │ │ │ ├── IsolatedClientLoader.scala │ │ │ └── HBaseClientImpl.scala │ │ │ ├── NewHBaseRDD.scala │ │ │ ├── HBaseSQLClient.scala │ │ │ ├── catalog │ │ │ └── HBaseTableRelation.scala │ │ │ ├── HBaseSessionStateBuilder.scala │ │ │ ├── HBaseSessionCatalog.scala │ │ │ ├── HBaseExternalCatalog.scala │ │ │ ├── HBaseSQLContext.scala │ │ │ └── HBaseStrategies.scala │ └── java │ │ └── org │ │ └── apache │ │ └── spark │ │ ├── sql │ │ └── hbase │ │ │ ├── execution │ │ │ └── DefaultRowKeyGenerator.java │ │ │ └── SparkHBaseConstants.java │ │ └── SparkSQLPushDownFilter.java └── test │ └── scala │ └── org │ └── apache │ └── spark │ └── sql │ └── hbase │ ├── client │ ├── TestHBaseTableProperties.scala │ ├── package.scala │ └── TestHBase.scala │ ├── TConstants.scala │ ├── execution │ └── HBaseFileFormatTestSuite.scala │ └── TestSql.scala ├── README.md └── pom.xml /spark-hbase/src/main/resources/show/INSERT_QUERY.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/wangpy1995/Spark-SQL-HBase/HEAD/spark-hbase/src/main/resources/show/INSERT_QUERY.png -------------------------------------------------------------------------------- /spark-hbase/src/main/resources/show/SELECT_QUERY.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/wangpy1995/Spark-SQL-HBase/HEAD/spark-hbase/src/main/resources/show/SELECT_QUERY.png -------------------------------------------------------------------------------- /spark-hbase/src/main/resources/spark_hbase.properties: -------------------------------------------------------------------------------- 1 | # HBaseClient??????HBase Namespace ?? Table ????? 2 | spark.hbase.client.impl=org.apache.spark.sql.hbase.client.HBaseClientImpl 3 | schema.file.url = /home/pw/IdeaProjects/Spark-SQL-HBase/src/main/resources/test.yml 4 | -------------------------------------------------------------------------------- /spark-hbase/src/main/scala/org/apache/spark/sql/hbase/execution/RowKeyGenerator.scala: -------------------------------------------------------------------------------- 1 | package org.apache.spark.sql.hbase.execution 2 | 3 | import org.apache.spark.sql.Row 4 | 5 | //用于插入HBase时生成rowKey 6 | trait RowKeyGenerator extends Serializable { 7 | 8 | def genRowKey(row: Row): Array[Byte] 9 | 10 | } -------------------------------------------------------------------------------- /spark-hbase/src/main/java/org/apache/spark/sql/hbase/execution/DefaultRowKeyGenerator.java: -------------------------------------------------------------------------------- 1 | package org.apache.spark.sql.hbase.execution; 2 | 3 | import org.apache.hadoop.hbase.util.Bytes; 4 | import org.apache.spark.sql.Row; 5 | 6 | import java.util.concurrent.atomic.AtomicInteger; 7 | 8 | /** 9 | * 每次插入rowkey + 1, 分布式环境下不可用 10 | */ 11 | public class DefaultRowKeyGenerator implements RowKeyGenerator { 12 | 13 | private static final AtomicInteger atomicInteger = new AtomicInteger(0); 14 | 15 | public byte[] genRowKey(Row row) { 16 | String rowKey = String.format("%04d", atomicInteger.addAndGet(1)); 17 | return Bytes.toBytes(rowKey); 18 | } 19 | } 20 | 21 | -------------------------------------------------------------------------------- /spark-hbase/src/main/java/org/apache/spark/sql/hbase/SparkHBaseConstants.java: -------------------------------------------------------------------------------- 1 | package org.apache.spark.sql.hbase; 2 | 3 | public interface SparkHBaseConstants { 4 | enum NAMESPACE implements SparkHBaseConstants {} 5 | 6 | enum TABLE_CONSTANTS implements SparkHBaseConstants { 7 | COLUMN_QUALIFIER_SPLITTER(":"), 8 | ROW_KEY("row_key"); 9 | 10 | private final String value; 11 | 12 | TABLE_CONSTANTS(String value) { 13 | this.value = value; 14 | } 15 | 16 | public String getValue() { 17 | return value; 18 | } 19 | } 20 | 21 | enum COLUMN_FAMILY implements SparkHBaseConstants {} 22 | 23 | enum QUALIFIER implements SparkHBaseConstants {} 24 | } 25 | -------------------------------------------------------------------------------- /spark-hbase/src/main/scala/org/apache/spark/sql/hbase/utils/HBaseSparkFormatUtils.scala: -------------------------------------------------------------------------------- 1 | package org.apache.spark.sql.hbase.utils 2 | 3 | import org.apache.spark.sql.hbase.SparkHBaseConstants.TABLE_CONSTANTS 4 | 5 | object HBaseSparkFormatUtils { 6 | case class SeparateName(familyName: String, qualifierName: String) 7 | 8 | def splitColumnAndQualifierName(columnQualifierNameString: String): SeparateName = { 9 | val results = columnQualifierNameString.split(TABLE_CONSTANTS.COLUMN_QUALIFIER_SPLITTER.getValue, 2) 10 | SeparateName(results.head, results.last) 11 | } 12 | 13 | def combineColumnAndQualifierName(familyName: String, qualifierName: String): String = { 14 | familyName + TABLE_CONSTANTS.COLUMN_QUALIFIER_SPLITTER.getValue + qualifierName 15 | } 16 | } 17 | -------------------------------------------------------------------------------- /spark-hbase/src/main/scala/org/apache/spark/sql/hbase/types/RegionInfoUDT.scala: -------------------------------------------------------------------------------- 1 | package org.apache.spark.sql.hbase.types 2 | 3 | import org.apache.hadoop.hbase.client.RegionInfo 4 | import org.apache.spark.sql.types.{BinaryType, DataType, ObjectType, UserDefinedType} 5 | 6 | class RegionInfoUDT extends UserDefinedType[RegionInfo] { 7 | 8 | override def sqlType: DataType = BinaryType 9 | 10 | override def serialize(obj: RegionInfo): Array[Byte] = RegionInfo.toDelimitedByteArray(obj) 11 | 12 | override def deserialize(datum: Any): RegionInfo = datum match { 13 | case pb: Array[Byte] => RegionInfo.parseFrom(pb) 14 | case _ => null 15 | } 16 | 17 | override def userClass: Class[RegionInfo] = classOf[RegionInfo] 18 | } 19 | 20 | case object RegionInfoUDT extends RegionInfoUDT 21 | -------------------------------------------------------------------------------- /spark-hbase/src/main/scala/org/apache/spark/sql/hbase/HBasePlan.scala: -------------------------------------------------------------------------------- 1 | package org.apache.spark.sql.hbase 2 | 3 | import org.apache.spark.sql.catalyst.catalog.CatalogTable 4 | import org.apache.spark.sql.catalyst.expressions.AttributeReference 5 | import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, LogicalPlanIntegrity} 6 | 7 | /** 8 | * Created by wpy on 17-5-26. 9 | */ 10 | case class HBasePlan( 11 | tableMeta: CatalogTable, 12 | dataCols: Seq[AttributeReference], 13 | output: Seq[AttributeReference], 14 | partitionCols: Seq[AttributeReference] 15 | ) extends LogicalPlan { 16 | override def children = Nil 17 | 18 | override protected def withNewChildrenInternal(newChildren: IndexedSeq[LogicalPlan]): LogicalPlan = { 19 | assert(newChildren.size == 1, "Incorrect number of children") 20 | this 21 | } 22 | } -------------------------------------------------------------------------------- /spark-hbase/src/main/scala/org/apache/spark/sql/hbase/utils/SerializableRegionInfo.scala: -------------------------------------------------------------------------------- 1 | package org.apache.spark.sql.hbase.utils 2 | 3 | import org.apache.hadoop.classification.InterfaceAudience 4 | import org.apache.hadoop.hbase.client.RegionInfo 5 | import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil 6 | import org.apache.spark.annotation.{DeveloperApi, Unstable} 7 | import org.apache.spark.util.Utils 8 | 9 | import java.io.{ObjectInputStream, ObjectOutputStream} 10 | 11 | 12 | @DeveloperApi @Unstable 13 | @InterfaceAudience.Private 14 | class SerializableRegionInfo(@transient var regionInfo: RegionInfo) extends Serializable { 15 | private def writeObject(out: ObjectOutputStream): Unit = Utils.tryOrIOException { 16 | out.defaultWriteObject() 17 | // 序列化region info 18 | val regionInfoProtoBytes = RegionInfo.toDelimitedByteArray(regionInfo) 19 | out.writeInt(regionInfoProtoBytes.length) 20 | out.write(regionInfoProtoBytes) 21 | } 22 | 23 | private def readObject(in: ObjectInputStream): Unit = Utils.tryOrIOException { 24 | in.defaultReadObject() 25 | //读取RegionInfo信息 26 | val len = in.readInt() 27 | val regionInfoProtoBytes = in.readNBytes(len) 28 | regionInfo = RegionInfo.parseFrom(regionInfoProtoBytes) 29 | } 30 | 31 | } 32 | -------------------------------------------------------------------------------- /spark-hbase/src/main/resources/test.yml: -------------------------------------------------------------------------------- 1 | pw:test: 2 | row: 3 | key: string 4 | generator: 5 | name: org.apache.spark.sql.hbase.execution.DefaultRowKeyGenerator 6 | A: 7 | A_00: string 8 | A_01: string 9 | A_02: string 10 | A_03: string 11 | A_04: string 12 | A_05: string 13 | A_06: string 14 | A_07: string 15 | A_08: string 16 | A_09: string 17 | B: 18 | B_00: string 19 | B_01: string 20 | B_02: string 21 | B_03: string 22 | B_04: string 23 | B_05: string 24 | B_06: string 25 | B_07: string 26 | B_08: string 27 | B_09: string 28 | 29 | pw:test_insert: 30 | row: 31 | key: string 32 | generator: 33 | name: org.apache.spark.sql.hbase.execution.DefaultRowKeyGenerator 34 | T: 35 | T_00: string 36 | 37 | hbase:meta: 38 | row: 39 | key: string 40 | generator: 41 | name: org.apache.spark.sql.hbase.execution.DefaultRowKeyGenerator 42 | ns: 43 | d: string 44 | table: 45 | state: string 46 | info: 47 | regioninfo: RegionInfo 48 | seqnumDuringOpen: long 49 | server: string 50 | serverstartcode: timestamp 51 | sn: string 52 | state: string 53 | 54 | hbase:namespace: 55 | row: 56 | key: string 57 | generator: 58 | name: org.apache.spark.sql.hbase.execution.DefaultRowKeyGenerator 59 | info: 60 | d: string -------------------------------------------------------------------------------- /spark-hbase/src/main/scala/org/apache/spark/sql/hbase/utils/StructFieldConverters.scala: -------------------------------------------------------------------------------- 1 | package org.apache.spark.sql.hbase.utils 2 | 3 | import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} 4 | import org.apache.spark.sql.hbase.utils.StructFieldConverters.{fromAttribute, toAttribute} 5 | import org.apache.spark.sql.types.{StructField, StructType} 6 | 7 | /** 8 | * 用于转换StructType到Attribute 9 | */ 10 | object StructFieldConverters { 11 | 12 | implicit def toAttribute(field: StructField): AsAttribute = new AsAttribute(field) 13 | 14 | implicit def toAttributes(structType: StructType): AsAttributes =new AsAttributes(structType) 15 | 16 | 17 | 18 | implicit def fromAttribute(attribute: AttributeReference): AsStructField = new AsStructField(attribute) 19 | 20 | 21 | 22 | implicit def fromAttributes(attributes: Seq[AttributeReference]): AsStructType = new AsStructType(attributes) 23 | } 24 | 25 | final class AsAttribute(value:StructField){ 26 | def toAttribute: AttributeReference = AttributeReference(value.name, value.dataType, value.nullable, value.metadata)() 27 | } 28 | 29 | final class AsAttributes(value:StructType){ 30 | def toAttributes: Seq[AttributeReference] = value.map(_.toAttribute) 31 | } 32 | 33 | final class AsStructField(reference: AttributeReference){ 34 | def fromAttribute:StructField = StructField(reference.name,reference.dataType,reference.nullable,reference.metadata) 35 | } 36 | 37 | final class AsStructType(attributes:Seq[AttributeReference]){ 38 | def fromAttributes:StructType = StructType(attributes.map(attr=>attr.fromAttribute)) 39 | } -------------------------------------------------------------------------------- /spark-hbase/src/test/scala/org/apache/spark/sql/hbase/client/TestHBaseTableProperties.scala: -------------------------------------------------------------------------------- 1 | package org.apache.spark.sql.hbase.client 2 | 3 | import org.apache.spark.sql.hbase.TConstants._ 4 | import org.scalatest.funsuite.AnyFunSuite 5 | 6 | /** 7 | * Created by wpy on 17-5-13. 8 | */ 9 | class TestHBaseTableProperties extends AnyFunSuite { 10 | def testPropertiesAsString(dbName: String, tableName: String): String = { 11 | val cli = IsolatedClientLoader.forVersion("3.0.0", "3.2.4", sparkConf, conf, extraConfig).createClient() 12 | cli.getTableOption(dbName, tableName).get.properties.mkString("\n") 13 | } 14 | 15 | def testExecutor(): Unit = { 16 | val pattern = "\\{([0-9a-zA-Z]*):\\((([0-9a-zA-Z]*([,])?)*)\\)}".r 17 | val str = "{CF1:(Q1, Q2, Q3, Qn)}".replaceAll("\t", " ").replaceAll(" ", "") 18 | /* val p = pattern.pattern matcher str 19 | var x = List.empty[String] 20 | if (p.matches()) 21 | x :::= 22 | Some((1 to p.groupCount).toList map p.group).get 23 | else None 24 | x*/ 25 | str match { 26 | case pattern(columnFamily, qualifiers, _*) => qualifiers.split(",").foreach(q => println(columnFamily + ":" + q)) 27 | case _ => println("none") 28 | } 29 | } 30 | 31 | def testReduce(): Unit = { 32 | val bloom = "CF1: type; CF2: type; CF3: type; CFn: type".replaceAll("\t", "").replaceAll(" ", "") 33 | val r = bloom.split(";").map { b => 34 | val r = b.split(":") 35 | Map(r(0) -> r(1)) 36 | }.reduce(_ ++ _) 37 | println(r) 38 | } 39 | 40 | test("print table properties") { 41 | println(testPropertiesAsString(TEST_NAMESPACE, TEST_TABLE_NAME)) 42 | } 43 | 44 | test("reduce string") { 45 | testReduce() 46 | } 47 | } 48 | -------------------------------------------------------------------------------- /spark-hbase/src/test/scala/org/apache/spark/sql/hbase/client/package.scala: -------------------------------------------------------------------------------- 1 | package org.apache.spark.sql.hbase 2 | 3 | /** 4 | * Created by wpy on 2017/5/11. 5 | */ 6 | package object client { 7 | 8 | // val jruby = new ScriptingContainer() 9 | 10 | /** 兼容多版本 */ 11 | private[hbase] sealed abstract class HBaseVersion( 12 | val fullVersion: String, 13 | val extraDeps: Seq[String] = Nil, 14 | val exclusions: Seq[String] = Nil) 15 | 16 | private[hbase] object hbase { 17 | 18 | case object v1_0 extends HBaseVersion("1.0.0", 19 | exclusions = Seq("jdk.tools:jdk.tools", 20 | "org.glassfish.hk2:*", 21 | "org.glassfish.jersey.bundles.repackaged:jersey-guava", 22 | "org.glassfish.hk2.external:javax.inject")) 23 | 24 | case object v1_1 extends HBaseVersion("1.1.0", 25 | exclusions = Seq("jdk.tools:jdk.tools", 26 | "org.glassfish.hk2:*", 27 | "org.glassfish.jersey.bundles.repackaged:jersey-guava", 28 | "org.glassfish.hk2.external:javax.inject")) 29 | 30 | case object v1_2 extends HBaseVersion("1.2.0", 31 | exclusions = Seq("jdk.tools:jdk.tools", 32 | "org.glassfish.hk2:*", 33 | "org.glassfish.jersey.bundles.repackaged:jersey-guava", 34 | "org.glassfish.hk2.external:javax.inject")) 35 | 36 | case object v2_0 extends HBaseVersion("2.0.0-SNAPSHOT", 37 | exclusions = Seq("jdk.tools:jdk.tools" /*, 38 | "org.glassfish.hk2:*", 39 | "org.glassfish.jersey.bundles.repackaged:jersey-guava", 40 | "org.glassfish.hk2.external:javax.inject"*/)) 41 | 42 | val allSupportedHiveVersions = Set(v1_0, v1_1, v1_2, v2_0) 43 | } 44 | 45 | } 46 | -------------------------------------------------------------------------------- /spark-hbase/src/test/scala/org/apache/spark/sql/hbase/TConstants.scala: -------------------------------------------------------------------------------- 1 | package org.apache.spark.sql.hbase 2 | 3 | import org.apache.hadoop.conf.Configuration 4 | import org.apache.hadoop.hbase.HBaseConfiguration 5 | import org.apache.hadoop.hbase.client.ConnectionFactory 6 | import org.apache.spark.SparkConf 7 | import org.apache.spark.sql.SparkSession 8 | 9 | /** 10 | * Created by wpy on 17-5-13. 11 | */ 12 | private[hbase] object TConstants extends Serializable { 13 | private[hbase] lazy val conf = new Configuration() 14 | private[hbase] lazy val hConf = HBaseConfiguration.create(conf) 15 | private[hbase] lazy val conn = ConnectionFactory.createConnection(hConf) 16 | private[hbase] lazy val admin = conn.getAdmin 17 | 18 | private[hbase] lazy val sparkConf = new SparkConf() 19 | .setMaster("local[*]") 20 | .setAppName("test") 21 | .set("spark.hadoopRDD.ignoreEmptySplits", "false") 22 | .set("spark.driver.port", "4042") 23 | 24 | private[hbase] lazy val ss = SparkSession.builder() 25 | .config(sparkConf) 26 | .getOrCreate() 27 | 28 | private[hbase] lazy val sc = ss.sparkContext 29 | private[hbase] lazy val extraConfig = Map( 30 | "schema.file.url" -> "/home/pw/IdeaProjects/Spark-SQL-HBase/src/main/resources/test.yml", 31 | "spark.hbase.client.impl" -> "org.apache.spark.sql.hbase.client.HBaseClientImpl") 32 | private[hbase] lazy val hs = new HBaseSession( 33 | TConstants.sc, 34 | conf, 35 | extraConfig) 36 | 37 | //namespace and table's name 38 | private[hbase] lazy val TEST_NAMESPACE = "pw" 39 | private[hbase] lazy val TEST_TABLE_NAME = "test" 40 | //column family 41 | private[hbase] lazy val TEST_COL_A = "A" 42 | private[hbase] lazy val TEST_COL_B = "B" 43 | //max count of rows 44 | private[hbase] lazy val MAX_ROW_CNT = 1000 45 | //max count of qualifiers 46 | private[hbase] lazy val MAX_QUALIFIER_CNT = 10 47 | 48 | } 49 | -------------------------------------------------------------------------------- /spark-hbase/src/main/scala/org/apache/spark/sql/hbase/client/package.scala: -------------------------------------------------------------------------------- 1 | package org.apache.spark.sql.hbase 2 | 3 | /** 4 | * Created by wpy on 2017/5/11. 5 | */ 6 | //TODO add shim loader in the future 7 | package object client { 8 | 9 | // val jruby = new ScriptingContainer() 10 | 11 | /** 兼容多版本 */ 12 | private[client] sealed abstract class HBaseVersion( 13 | val fullVersion: String, 14 | val extraDeps: Seq[String] = Nil, 15 | val exclusions: Seq[String] = Nil) 16 | 17 | private[client] object hbase { 18 | 19 | case object v1_0 extends HBaseVersion("1.0.0", 20 | exclusions = Seq("jdk.tools:jdk.tools", 21 | "org.glassfish.hk2:*", 22 | "org.glassfish.jersey.bundles.repackaged:jersey-guava", 23 | "org.glassfish.hk2.external:javax.inject")) 24 | 25 | case object v1_1 extends HBaseVersion("1.1.0", 26 | exclusions = Seq("jdk.tools:jdk.tools", 27 | "org.glassfish.hk2:*", 28 | "org.glassfish.jersey.bundles.repackaged:jersey-guava", 29 | "org.glassfish.hk2.external:javax.inject")) 30 | 31 | case object v1_2 extends HBaseVersion("1.2.0", 32 | exclusions = Seq("jdk.tools:jdk.tools", 33 | "org.glassfish.hk2:*", 34 | "org.glassfish.jersey.bundles.repackaged:jersey-guava", 35 | "org.glassfish.hk2.external:javax.inject")) 36 | 37 | case object v2_0 extends HBaseVersion("2.0.0-SNAPSHOT", 38 | exclusions = Seq("jdk.tools:jdk.tools", 39 | "org.glassfish.hk2:*", 40 | "org.glassfish.jersey.bundles.repackaged:jersey-guava", 41 | "org.glassfish.hk2.external:javax.inject")) 42 | 43 | case object v3_0 extends HBaseVersion("3.0.0-alpha-4-SNAPSHOT", 44 | exclusions = Seq("jdk.tools:jdk.tools", 45 | "org.glassfish.hk2:*", 46 | "org.glassfish.jersey.bundles.repackaged:jersey-guava", 47 | "org.glassfish.hk2.external:javax.inject")) 48 | 49 | val allSupportedHBaseVersions: Set[HBaseVersion] = Set(v1_0, v1_1, v1_2, v2_0, v3_0) 50 | } 51 | 52 | } 53 | -------------------------------------------------------------------------------- /spark-hbase/src/main/scala/org/apache/spark/sql/hbase/NewHBaseRDD.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.spark.sql.hbase 19 | 20 | import org.apache.hadoop.classification.InterfaceAudience 21 | import org.apache.hadoop.conf.Configuration 22 | import org.apache.hadoop.mapred 23 | import org.apache.hadoop.mapred.JobConf 24 | import org.apache.hadoop.mapreduce.InputFormat 25 | import org.apache.spark.deploy.SparkHadoopUtil 26 | import org.apache.spark.rdd.NewHadoopRDD 27 | import org.apache.spark.util.SerializableConfiguration 28 | import org.apache.spark.{InterruptibleIterator, Partition, SparkContext, TaskContext} 29 | 30 | import scala.annotation.meta.param 31 | 32 | @InterfaceAudience.Public 33 | class NewHBaseRDD[K, V](@(transient@param) sc: SparkContext, 34 | @(transient@param) inputFormatClass: Class[_ <: InputFormat[K, V]], 35 | @(transient@param) keyClass: Class[K], 36 | @(transient@param) valueClass: Class[V], 37 | @transient private val _conf: Configuration, 38 | val hBaseContext: HBaseSQLContext) extends NewHadoopRDD(sc, inputFormatClass, keyClass, valueClass, _conf) { 39 | override def compute(theSplit: Partition, context: TaskContext): InterruptibleIterator[(K, V)] = { 40 | val jconf = new JobConf(getConf) 41 | SparkHadoopUtil.get.addCredentials(jconf) 42 | super.compute(theSplit, context) 43 | } 44 | } 45 | 46 | object NewHBaseRDD { 47 | val CONFIGURATION_INSTANTIATION_LOCK = new Object() 48 | } 49 | -------------------------------------------------------------------------------- /spark-hbase/README.md: -------------------------------------------------------------------------------- 1 | # SparkSQLOnHBase 2 | 3 |     该项目基于`Spark SQL`开发,实现了在`HBase`上进行sql查询的功能,支持标准sql查询操作. 4 | 物理逻辑层通过Spark SQL本身的谓词下推,剪枝等对HBase Scan的范围进行了优化, 5 | 后续有空闲时间会增加支持插入、删除、建表相关功能的实现. 6 | 7 | [最新代码中已经实现了通过sql语句插入数据的功能,`rowkey`生成策略可以通过配置文件手动指定]。 8 | 9 |     注:项目的spark hbase由2022.11.15于github上的master分支手动编译得到, 10 | 理论上来讲能兼容Spark3.x版本 11 | 12 |         1. HBase的`Namespace`与SQL中的`Database`对应, 13 | 在未指定`Database`的情况下, 可以使用`namespace.table`进行表查询, 14 | 例如`select * from namespace.table` 15 | 16 |         2. 列名部分采用了HBase的`cf:qualifier`形式, 17 | `ColumnFamily`与`Qualifier`之间以`:`分隔, 18 | 因此想要查询HBase表`A`中的列`cf:1`时,需要在`cf:1`的首尾添加反引号\`cf:1\` 19 | 20 |         3. HBase表以及Namespace的操作默认通过`org.apache.spark.sql.hbase.client.HBaseClientImpl`实现, 21 | 其中包含了从文件中读取hbase schema信息的功能,可以根据自身需求 22 | 修改spark_hbase.properties中`spark.hbase.client.impl`值 23 | 24 |         4. yaml文件中为每个表增加了一个generator选项, 25 | generator的值需要继承trait `org.apache.spark.hbase.execution.RowKeyGenerator`, 26 | 且必须有一个无参数的构造方法,通过实现`genRowKey`方法可以为插入的数据生成RowKey 27 | 28 | 29 | # GetStarted 30 |     1. 替换resource目录下hbase-site.xml,修改spark_hbase.properties的各属性值 31 | 32 |     2. 根据需要,运行项目下test/scala中的 `org.apache.spark.sql.hbase.client.TestHBase`的`createUserNamespaceAndTable()`, 33 | `insertData()`以及 34 | `scan()`方法新建表以及生成数据 35 | 36 |     3. 执行`org.apache.spark.sql.hbase.HBaseSQLClient`的main方法启动程序, 37 | 输入sql语句检查程序是否正常运行 38 | 39 | # 部分截图如下(字太小看不清楚可以手动放大查看) 40 | ## 程序查询语句: 41 |     show databases; 42 | 43 |     show tables; 44 | 45 |     select * from hbase.meta; 46 | 47 |     select * from pw.test where \`A:A_00\` like "%24%"; 48 | 49 | ![Image text](https://github.com/wangpy1995/Spark-SQL-HBase/blob/master/src/main/resources/show/SELECT_QUERY.png) 50 | ## 插入语句截图如下: 51 | 52 |     use pw; 53 | 54 |     insert into test_insert values('0000', 'TestSql'); 55 | 56 |     select * from test_insert; 57 | 58 |     insert into test_insert values('0000', 'TestSql'); 59 | 60 |     insert into test_insert values('0000', 'TestSql'); 61 | 62 |     select * from test_insert; 63 | 64 | ![Image text](https://github.com/wangpy1995/Spark-SQL-HBase/blob/master/src/main/resources/show/INSERT_QUERY.png) 65 | 66 |    注:默认的RowKeyGenerator的rowKey规则是每次插入增加1,因此values中的'0000'不生效符合预期 67 | 68 | ####由于环境限制,程序还有许多部分未来得及测试,诸多不完善之处,还请大家多多提出宝贵意见 69 | -------------------------------------------------------------------------------- /spark-hbase/src/test/scala/org/apache/spark/sql/hbase/execution/HBaseFileFormatTestSuite.scala: -------------------------------------------------------------------------------- 1 | package org.apache.spark.sql.hbase.execution 2 | 3 | import org.apache.hadoop.conf.Configuration 4 | import org.apache.hadoop.fs.{FileSystem, Path} 5 | import org.apache.hadoop.hbase.io.hfile.HFile 6 | import org.apache.hadoop.hbase.util.Bytes 7 | import org.apache.spark.sql.hbase.TConstants._ 8 | import org.scalatest.funsuite.AnyFunSuite 9 | 10 | class HBaseFileFormatTestSuite extends AnyFunSuite { 11 | val hadoopConf = new Configuration() 12 | val hfilePath = "/home/pw/IdeaProjects/hbase/tmp/hbase/data/pw/test/2debdc5e3e71d4e9693a4f10ac74b521/A/a99e95dd0637431cbe07cde90adc28b1" 13 | 14 | test("read hfile") { 15 | var lastKey: String = null 16 | val filePath = new Path(hfilePath) 17 | val fs = FileSystem.get(hadoopConf) 18 | val hfileReader = HFile.createReader(fs, filePath, hadoopConf) 19 | 20 | val scanner = hfileReader.getScanner(hConf, false, false) 21 | scanner.seekTo() 22 | 0 until 10 foreach { _ => 23 | if (scanner.next()) { 24 | val cell = scanner.getCell 25 | val curKey = Bytes.toString(cell.getRowArray, cell.getRowOffset, cell.getRowLength) 26 | if (null != curKey && lastKey != curKey) { 27 | println("====================================================================") 28 | println(s"key: $curKey") 29 | lastKey = curKey 30 | } 31 | println(s"column family: ${Bytes.toString(cell.getFamilyArray, cell.getFamilyOffset, cell.getFamilyLength)}") 32 | println(s"qualifier: ${Bytes.toString(cell.getQualifierArray, cell.getQualifierOffset, cell.getQualifierLength)}") 33 | println(s"value: ${Bytes.toString(cell.getValueArray, cell.getValueOffset, cell.getValueLength)}") 34 | } 35 | } 36 | } 37 | 38 | test("read hfile by sql") { 39 | ss.sql( 40 | s""" 41 | |CREATE TABLE test( 42 | | row_key string, 43 | | A_00 string, 44 | | A_01 string, 45 | | A_02 string, 46 | | A_03 string, 47 | | A_04 string, 48 | | A_05 string, 49 | | A_06 string, 50 | | A_07 string, 51 | | A_08 string, 52 | | A_09 string 53 | |) USING ${classOf[HBaseFileFormat].getCanonicalName} 54 | |OPTIONS( 55 | |path '$hfilePath') 56 | |""".stripMargin).show() 57 | ss.sql("select * from test").show() 58 | ss.sql("select A_00 from test").show() 59 | ss.sql("select row_key from test").show() 60 | assert(ss.sql("select * from test").count() == 1000) 61 | } 62 | } 63 | -------------------------------------------------------------------------------- /spark-hbase/src/main/resources/hbase-site.xml: -------------------------------------------------------------------------------- 1 | 2 | 3 | 22 | 23 | 42 | 46 | 47 | hbase.cluster.distributed 48 | false 49 | 50 | 51 | hbase.tmp.dir 52 | /home/pw/IdeaProjects/hbase/tmp 53 | 54 | 55 | hbase.unsafe.stream.capability.enforce 56 | false 57 | 58 | 59 | -------------------------------------------------------------------------------- /spark-hbase/src/main/resources/log4j.properties: -------------------------------------------------------------------------------- 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 | # Set everything to be logged to the file core/target/unit-tests.log 18 | log4j.rootLogger=DEBUG, CA, FA 19 | #Console Appender 20 | log4j.appender.CA=org.apache.log4j.ConsoleAppender 21 | log4j.appender.CA.layout=org.apache.log4j.PatternLayout 22 | log4j.appender.CA.layout.ConversionPattern=%d{HH:mm:ss.SSS} %p %c: %m%n 23 | log4j.appender.CA.Threshold=WARN 24 | #File Appender 25 | log4j.appender.FA=org.apache.log4j.FileAppender 26 | log4j.appender.FA.append=false 27 | log4j.appender.FA.file=target/unit-tests.log 28 | log4j.appender.FA.layout=org.apache.log4j.PatternLayout 29 | log4j.appender.FA.layout.ConversionPattern=%d{HH:mm:ss.SSS} %t %p %c{1}: %m%n 30 | # Set the logger level of File Appender to WARN 31 | log4j.appender.FA.Threshold=DEBUG 32 | # Some packages are noisy for no good reason. 33 | log4j.additivity.org.apache.hadoop.hive.serde2.lazy.LazyStruct=false 34 | log4j.logger.org.apache.hadoop.hive.serde2.lazy.LazyStruct=OFF 35 | log4j.additivity.org.apache.hadoop.hive.metastore.RetryingHMSHandler=false 36 | log4j.logger.org.apache.hadoop.hive.metastore.RetryingHMSHandler=OFF 37 | log4j.additivity.hive.log=false 38 | log4j.logger.hive.log=OFF 39 | log4j.additivity.parquet.hadoop.ParquetRecordReader=false 40 | log4j.logger.parquet.hadoop.ParquetRecordReader=OFF 41 | log4j.additivity.org.apache.parquet.hadoop.ParquetRecordReader=false 42 | log4j.logger.org.apache.parquet.hadoop.ParquetRecordReader=OFF 43 | log4j.additivity.org.apache.parquet.hadoop.ParquetOutputCommitter=false 44 | log4j.logger.org.apache.parquet.hadoop.ParquetOutputCommitter=OFF 45 | log4j.additivity.hive.ql.metadata.Hive=false 46 | log4j.logger.hive.ql.metadata.Hive=OFF 47 | log4j.additivity.org.apache.hadoop.hive.ql.io.RCFile=false 48 | log4j.logger.org.apache.hadoop.hive.ql.io.RCFile=ERROR 49 | # Parquet related logging 50 | log4j.logger.org.apache.parquet.CorruptStatistics=ERROR 51 | log4j.logger.parquet.CorruptStatistics=ERROR 52 | -------------------------------------------------------------------------------- /spark-hbase/src/test/scala/org/apache/spark/sql/hbase/TestSql.scala: -------------------------------------------------------------------------------- 1 | package org.apache.spark.sql.hbase 2 | 3 | import org.apache.hadoop.hbase.client.{Result, Scan} 4 | import org.apache.hadoop.hbase.filter.{FilterList, SingleColumnValueFilter, SubstringComparator} 5 | import org.apache.hadoop.hbase.io.ImmutableBytesWritable 6 | import org.apache.hadoop.hbase.mapreduce.{IdentityTableMapper, TableInputFormat, TableMapReduceUtil} 7 | import org.apache.hadoop.hbase.util.Bytes 8 | import org.apache.hadoop.hbase.{CompareOperator, TableName} 9 | import org.apache.hadoop.mapred.JobConf 10 | import org.apache.hadoop.mapreduce.Job 11 | import org.apache.spark.deploy.SparkHadoopUtil 12 | import org.apache.spark.sql.hbase.TConstants._ 13 | import org.scalatest.funsuite.AnyFunSuite 14 | 15 | /** 16 | * Created by wpy on 17-5-17. 17 | */ 18 | class TestSql extends AnyFunSuite { 19 | 20 | /** 21 | * 通过查询HBase数据来测试spark newAPIHadoopRDD是否正常 22 | * 23 | */ 24 | test("new newAPIHadoopRDD") { 25 | val job: Job = Job.getInstance(hs.config) 26 | val scan = new Scan() 27 | val table = TableName.valueOf(TEST_NAMESPACE + ":" + TEST_TABLE_NAME) 28 | val bytesColA = Bytes.toBytes(TEST_COL_A) 29 | val bytesColB = Bytes.toBytes(TEST_COL_B) 30 | val bytesAQua0 = Bytes.toBytes(TEST_COL_A + "_00") 31 | val bytesBQua0 = Bytes.toBytes(TEST_COL_B + "_00") 32 | scan.addColumn(bytesColA, bytesAQua0) 33 | scan.addColumn(bytesColB, bytesBQua0) 34 | val filter = new SingleColumnValueFilter(bytesColA, bytesAQua0, CompareOperator.EQUAL, new SubstringComparator("04")) 35 | filter.setFilterIfMissing(true) 36 | scan.setFilter(new FilterList(filter)) 37 | 38 | TableMapReduceUtil.initCredentials(job) 39 | TableMapReduceUtil.initTableMapperJob(table, scan, 40 | classOf[IdentityTableMapper], null, null, job) 41 | 42 | val jconf = new JobConf() 43 | SparkHadoopUtil.get.addCredentials(jconf) 44 | val rdd = hs.sparkContext.newAPIHadoopRDD(job.getConfiguration, classOf[TableInputFormat], classOf[ImmutableBytesWritable], classOf[Result]) 45 | val count = rdd.map { a => 46 | println(a) 47 | a 48 | }.count() 49 | println(count) 50 | } 51 | 52 | test("select * ") { 53 | hs.sql("select * from hbase.meta").show() 54 | hs.sql(s"select * from $TEST_NAMESPACE.$TEST_TABLE_NAME").show() 55 | hs.sql("select * from pw.test_insert").show() 56 | } 57 | 58 | test("select one col"){ 59 | val regionInfo = hs.sql("select `info:regioninfo` from hbase.meta").cache() 60 | regionInfo.show() 61 | println(regionInfo.collect().mkString("Array(", ", ", ")")) 62 | } 63 | 64 | test("insert one col"){ 65 | hs.sql("use pw").show() 66 | hs.sql("insert into test_insert values('0000', 'TestSql')").show() 67 | hs.sql("select * from test_insert").show() 68 | } 69 | } 70 | -------------------------------------------------------------------------------- /spark-hbase/src/main/scala/org/apache/spark/sql/hbase/execution/HBaseSqlParser.scala: -------------------------------------------------------------------------------- 1 | package org.apache.spark.sql.hbase.execution 2 | 3 | import org.apache.spark.sql.catalyst.parser.ParserUtils.withOrigin 4 | import org.apache.spark.sql.catalyst.parser.SqlBaseParser.PrimitiveDataTypeContext 5 | import org.apache.spark.sql.catalyst.parser.{AbstractSqlParser, SqlBaseParser} 6 | import org.apache.spark.sql.errors.QueryParsingErrors 7 | import org.apache.spark.sql.execution.SparkSqlAstBuilder 8 | import org.apache.spark.sql.hbase.types.RegionInfoUDT 9 | import org.apache.spark.sql.internal.{SQLConf, VariableSubstitution} 10 | import org.apache.spark.sql.types._ 11 | 12 | import java.util.Locale 13 | import scala.collection.JavaConverters._ 14 | 15 | class HBaseSqlParser extends AbstractSqlParser { 16 | val astBuilder = new SparkHBaseSqlAstBuilder() 17 | 18 | private val substitutor = new VariableSubstitution() 19 | 20 | protected override def parse[T](command: String)(toResult: SqlBaseParser => T): T = { 21 | super.parse(substitutor.substitute(command))(toResult) 22 | } 23 | } 24 | 25 | object HBaseSqlParser extends HBaseSqlParser 26 | 27 | class SparkHBaseSqlAstBuilder extends SparkSqlAstBuilder { 28 | override def visitPrimitiveDataType(ctx: PrimitiveDataTypeContext): DataType = withOrigin(ctx) { 29 | 30 | val dataType = ctx.`type`().identifier.getText.toLowerCase(Locale.ROOT) 31 | (dataType, ctx.INTEGER_VALUE().asScala.toList) match { 32 | case ("boolean", Nil) => BooleanType 33 | case ("tinyint" | "byte", Nil) => ByteType 34 | case ("smallint" | "short", Nil) => ShortType 35 | case ("int" | "integer", Nil) => IntegerType 36 | case ("bigint" | "long", Nil) => LongType 37 | case ("float" | "real", Nil) => FloatType 38 | case ("double", Nil) => DoubleType 39 | case ("date", Nil) => DateType 40 | case ("timestamp", Nil) => SQLConf.get.timestampType 41 | case ("timestamp_ntz", Nil) => TimestampNTZType 42 | case ("timestamp_ltz", Nil) => TimestampType 43 | case ("string", Nil) => StringType 44 | case ("character" | "char", length :: Nil) => CharType(length.getText.toInt) 45 | case ("varchar", length :: Nil) => VarcharType(length.getText.toInt) 46 | case ("binary", Nil) => BinaryType 47 | case ("decimal" | "dec" | "numeric", Nil) => DecimalType.USER_DEFAULT 48 | case ("decimal" | "dec" | "numeric", precision :: Nil) => 49 | DecimalType(precision.getText.toInt, 0) 50 | case ("decimal" | "dec" | "numeric", precision :: scale :: Nil) => 51 | DecimalType(precision.getText.toInt, scale.getText.toInt) 52 | case ("void", Nil) => NullType 53 | case ("interval", Nil) => CalendarIntervalType 54 | case ("regioninfo", Nil) => RegionInfoUDT 55 | case (dt@("character" | "char" | "varchar"), Nil) => 56 | throw QueryParsingErrors.charTypeMissingLengthError(dt, ctx) 57 | case (dt, params) => 58 | val dtStr = if (params.nonEmpty) s"$dt(${params.mkString(",")})" else dt 59 | throw QueryParsingErrors.dataTypeUnsupportedError(dtStr, ctx) 60 | } 61 | } 62 | } -------------------------------------------------------------------------------- /spark-hbase/src/main/scala/org/apache/spark/sql/hbase/HBaseSQLClient.scala: -------------------------------------------------------------------------------- 1 | package org.apache.spark.sql.hbase 2 | 3 | import jline.console.ConsoleReader 4 | import jline.console.history.FileHistory 5 | import org.apache.hadoop.conf.Configuration 6 | import org.apache.spark.SparkConf 7 | import org.apache.spark.sql.SparkSession 8 | 9 | import java.io.File 10 | import java.util.Properties 11 | 12 | /** 13 | * Created by wpy on 17-5-18. 14 | */ 15 | object HBaseSQLClient { 16 | val prompt = "HBaseSQL " 17 | import scala.collection.JavaConverters._ 18 | private val extraConfigs = { 19 | val in = getClass.getResourceAsStream("/spark_hbase.properties") 20 | val props = new Properties() 21 | props.load(in) 22 | in.close() 23 | props.asScala.toMap 24 | } 25 | private val continuedPrompt = "".padTo(prompt.length, ' ') 26 | private val sparkConf = new SparkConf().setMaster("local[*]").setAppName("test").set("spark.hadoopRDD.ignoreEmptySplits","false") 27 | private val ss = SparkSession.builder().config(sparkConf).getOrCreate() 28 | private val sc = ss.sparkContext 29 | val hs = new HBaseSession(sc, new Configuration(),extraConfigs) 30 | 31 | def main(args: Array[String]): Unit = { 32 | 33 | val reader = new ConsoleReader() 34 | reader.setBellEnabled(false) 35 | val historyDirectory = System.getProperty("user.home") 36 | 37 | try { 38 | if (new File(historyDirectory).exists()) { 39 | val historyFile = historyDirectory + File.separator + ".hbaseqlhistory" 40 | reader.setHistory(new FileHistory(new File(historyFile))) 41 | } 42 | 43 | } catch { 44 | case e: Exception => 45 | System.err.println(e.getMessage) 46 | } 47 | 48 | println("Spark4HBase CLI") 49 | var prefix = "" 50 | 51 | def promptPrefix = { 52 | s"$prompt" 53 | } 54 | 55 | var currentPrompt = promptPrefix 56 | var line = reader.readLine(currentPrompt + ": $ ") 57 | 58 | while (line != null) { 59 | if (prefix.nonEmpty) { 60 | prefix += '\n' 61 | } 62 | 63 | if (line.trim.endsWith(";") && !line.trim.endsWith("\\;")) { 64 | line = prefix + line 65 | processLine(line, allowInterrupting = true) 66 | prefix = "" 67 | currentPrompt = promptPrefix 68 | } 69 | else { 70 | prefix = prefix + line 71 | currentPrompt = continuedPrompt 72 | } 73 | 74 | line = reader.readLine(currentPrompt + " $ ") 75 | } 76 | 77 | System.exit(0) 78 | } 79 | 80 | 81 | private def processLine(line: String, allowInterrupting: Boolean): Unit = { 82 | 83 | val input = line.substring(0, line.length - 1) 84 | try { 85 | process(input.trim()) 86 | } catch { 87 | case e: Exception => 88 | println(e.getMessage) 89 | } 90 | } 91 | 92 | private def process(input: String): Unit = { 93 | val token = input.split("\\s") 94 | token(0).toUpperCase match { 95 | case "EXIT" => ss.close() 96 | System.exit(0) 97 | 98 | case _ => hs.sql(input).show(30) 99 | } 100 | } 101 | 102 | } 103 | -------------------------------------------------------------------------------- /spark-hbase/src/main/scala/org/apache/spark/sql/hbase/execution/CreateHBaseTableAsSelectCommand.scala: -------------------------------------------------------------------------------- 1 | package org.apache.spark.sql.hbase.execution 2 | 3 | import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation 4 | import org.apache.spark.sql.catalyst.catalog.CatalogTable 5 | import org.apache.spark.sql.catalyst.plans.logical.{InsertIntoStatement, LogicalPlan} 6 | import org.apache.spark.sql.execution.command.RunnableCommand 7 | import org.apache.spark.sql.{AnalysisException, Row, SaveMode, SparkSession} 8 | 9 | import scala.util.control.NonFatal 10 | 11 | /** 12 | * Create table and insert the query result into it. 13 | * 14 | * @param tableDesc the Table Describe, which may contains serde, storage handler etc. 15 | * @param query the query whose result will be insert into the new relation 16 | * @param mode SaveMode 17 | */ 18 | case class CreateHBaseTableAsSelectCommand( 19 | tableDesc: CatalogTable, 20 | query: LogicalPlan, 21 | mode: SaveMode) 22 | extends RunnableCommand { 23 | 24 | private val tableIdentifier = tableDesc.identifier 25 | 26 | override def innerChildren: Seq[LogicalPlan] = Seq(query) 27 | 28 | override def run(sparkSession: SparkSession): Seq[Row] = { 29 | if (sparkSession.sessionState.catalog.tableExists(tableIdentifier)) { 30 | assert(mode != SaveMode.Overwrite, 31 | s"Expect the table $tableIdentifier has been dropped when the save mode is Overwrite") 32 | 33 | if (mode == SaveMode.ErrorIfExists) { 34 | throw new AnalysisException(s"$tableIdentifier already exists.") 35 | } 36 | if (mode == SaveMode.Ignore) { 37 | // Since the table already exists and the save mode is Ignore, we will just return. 38 | return Seq.empty 39 | } 40 | 41 | sparkSession.sessionState.executePlan( 42 | InsertIntoStatement( 43 | UnresolvedRelation(tableIdentifier), 44 | Map(), 45 | Seq.empty, 46 | query, 47 | overwrite = false, 48 | ifPartitionNotExists = false)).toRdd 49 | } else { 50 | // TODO ideally, we should get the output data ready first and then 51 | // add the relation into catalog, just in case of failure occurs while data 52 | // processing. 53 | assert(tableDesc.schema.isEmpty) 54 | sparkSession.sessionState.catalog.createTable( 55 | tableDesc.copy(schema = query.schema), ignoreIfExists = false) 56 | 57 | try { 58 | sparkSession.sessionState.executePlan( 59 | InsertIntoStatement( 60 | UnresolvedRelation(tableIdentifier), 61 | Map(), 62 | Seq.empty, 63 | query, 64 | overwrite = true, 65 | ifPartitionNotExists = false)).toRdd 66 | } catch { 67 | case NonFatal(e) => 68 | // drop the created table. 69 | sparkSession.sessionState.catalog.dropTable(tableIdentifier, ignoreIfNotExists = true, 70 | purge = false) 71 | throw e 72 | } 73 | } 74 | 75 | Seq.empty[Row] 76 | } 77 | 78 | def argString: String = { 79 | s"[Database:${tableDesc.database}}, " + 80 | s"TableName: ${tableDesc.identifier.table}, " + 81 | s"InsertIntoHBaseTable]" 82 | } 83 | 84 | override protected def withNewChildrenInternal(newChildren: IndexedSeq[LogicalPlan]): LogicalPlan = { 85 | assert(newChildren.size == 1, "Incorrect number of children") 86 | copy(query=newChildren.head) 87 | } 88 | } 89 | -------------------------------------------------------------------------------- /spark-hbase/src/main/scala/org/apache/spark/sql/hbase/catalog/HBaseTableRelation.scala: -------------------------------------------------------------------------------- 1 | package org.apache.spark.sql.hbase.catalog 2 | 3 | import org.apache.commons.lang3.StringUtils 4 | import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation 5 | import org.apache.spark.sql.catalyst.catalog.{CatalogTable, CatalogTablePartition} 6 | import org.apache.spark.sql.catalyst.expressions.{AttributeReference, ExprId} 7 | import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, Statistics} 8 | import org.apache.spark.sql.catalyst.util.truncatedString 9 | import org.apache.spark.sql.errors.QueryExecutionErrors 10 | import org.apache.spark.sql.internal.SQLConf 11 | 12 | case class HBaseTableRelation( 13 | tableMeta: CatalogTable, 14 | dataCols: Seq[AttributeReference], 15 | partitionCols: Seq[AttributeReference], 16 | tableStats: Option[Statistics] = None, 17 | @transient prunedPartitions: Option[Seq[CatalogTablePartition]] = None) 18 | extends LeafNode with MultiInstanceRelation { 19 | assert(tableMeta.identifier.database.isDefined) 20 | assert(tableMeta.partitionSchema.sameType(partitionCols.toStructType)) 21 | assert(tableMeta.dataSchema.sameType(dataCols.toStructType)) 22 | 23 | // The partition column should always appear after data columns. 24 | override def output: Seq[AttributeReference] = dataCols ++ partitionCols 25 | 26 | def isPartitioned: Boolean = partitionCols.nonEmpty 27 | 28 | override def doCanonicalize(): HBaseTableRelation = copy( 29 | tableMeta = CatalogTable.normalize(tableMeta), 30 | dataCols = dataCols.zipWithIndex.map { 31 | case (attr, index) => attr.withExprId(ExprId(index)) 32 | }, 33 | partitionCols = partitionCols.zipWithIndex.map { 34 | case (attr, index) => attr.withExprId(ExprId(index + dataCols.length)) 35 | }, 36 | tableStats = None 37 | ) 38 | 39 | override def computeStats(): Statistics = { 40 | tableMeta.stats.map(_.toPlanStats(output, conf.cboEnabled || conf.planStatsEnabled)) 41 | .orElse(tableStats) 42 | .getOrElse { 43 | throw new IllegalStateException("Table stats must be specified.") 44 | } 45 | } 46 | 47 | override def newInstance(): HBaseTableRelation = copy( 48 | dataCols = dataCols.map(_.newInstance()), 49 | partitionCols = partitionCols.map(_.newInstance())) 50 | 51 | override def simpleString(maxFields: Int): String = { 52 | val catalogTable = tableMeta.storage.serde match { 53 | case Some(serde) => tableMeta.identifier :: serde :: Nil 54 | case _ => tableMeta.identifier :: Nil 55 | } 56 | 57 | var metadata = Map( 58 | "CatalogTable" -> catalogTable.mkString(", "), 59 | "Data Cols" -> truncatedString(dataCols, "[", ", ", "]", maxFields), 60 | "Partition Cols" -> truncatedString(partitionCols, "[", ", ", "]", maxFields) 61 | ) 62 | 63 | if (prunedPartitions.nonEmpty) { 64 | metadata += ("Pruned Partitions" -> { 65 | val parts = prunedPartitions.get.map { part => 66 | val spec = part.spec.map { case (k, v) => s"$k=$v" }.mkString(", ") 67 | if (part.storage.serde.nonEmpty && part.storage.serde != tableMeta.storage.serde) { 68 | s"($spec, ${part.storage.serde.get})" 69 | } else { 70 | s"($spec)" 71 | } 72 | } 73 | truncatedString(parts, "[", ", ", "]", maxFields) 74 | }) 75 | } 76 | 77 | val metadataEntries = metadata.toSeq.map { 78 | case (key, value) if key == "CatalogTable" => value 79 | case (key, value) => 80 | key + ": " + StringUtils.abbreviate(value, SQLConf.get.maxMetadataStringLength) 81 | } 82 | 83 | val metadataStr = truncatedString(metadataEntries, "[", ", ", "]", maxFields) 84 | s"$nodeName $metadataStr" 85 | } 86 | } 87 | -------------------------------------------------------------------------------- /spark-hbase/pom.xml: -------------------------------------------------------------------------------- 1 | 2 | 4 | 4.0.0 5 | wpy.hbase.spark 6 | spark-hbase_2.12 7 | 0.3.3-SNAPSHOT 8 | jar 9 | Spark Project HBase 10 | 11 | spark-hbase 12 | 2.6.0 13 | 3.5.1 14 | 2.12.17 15 | 2.12 16 | 3.0.2 17 | 18 | 19 | 20 | 21 | org.apache.spark 22 | spark-sql_${scala.binary.version} 23 | ${spark.version} 24 | 25 | 26 | org.apache.spark 27 | spark-streaming_${scala.binary.version} 28 | ${spark.version} 29 | 30 | 31 | org.apache.spark 32 | spark-hive_${scala.binary.version} 33 | ${spark.version} 34 | 35 | 36 | com.google.code.findbugs 37 | jsr305 38 | ${jsr305.version} 39 | 40 | 41 | org.apache.hbase 42 | hbase-server 43 | ${hbase.version} 44 | 45 | 46 | org.apache.hbase 47 | hbase-backup 48 | ${hbase.version} 49 | 50 | 51 | jline 52 | jline 53 | 2.14.6 54 | 55 | 56 | org.scala-lang 57 | scala-library 58 | ${scala.version} 59 | 60 | 61 | org.scala-lang 62 | scala-compiler 63 | ${scala.version} 64 | 65 | 66 | 67 | 68 | org.yaml 69 | snakeyaml 70 | 1.33 71 | 72 | 73 | 74 | 75 | net.minidev 76 | accessors-smart 77 | 2.4.7 78 | 79 | 80 | 81 | 82 | net.minidev 83 | json-smart 84 | 2.4.7 85 | 86 | 87 | 88 | org.scalatest 89 | scalatest_${scala.binary.version} 90 | 3.2.14 91 | 92 | 93 | org.scala-lang 94 | scala-library 95 | 96 | 97 | 98 | 99 | 100 | 101 | 102 | 103 | 104 | org.apache.maven.plugins 105 | maven-compiler-plugin 106 | 107 | UTF-8 108 | 11 109 | 11 110 | 111 | 112 | 113 | 114 | org.scala-tools 115 | maven-scala-plugin 116 | 2.15.2 117 | 118 | 119 | 120 | -------------------------------------------------------------------------------- /spark-hbase/src/main/scala/org/apache/spark/sql/hbase/HBaseSessionStateBuilder.scala: -------------------------------------------------------------------------------- 1 | package org.apache.spark.sql.hbase 2 | 3 | import org.apache.spark.annotation.{Experimental, Stable} 4 | import org.apache.spark.sql.catalyst.analysis.{Analyzer, ResolveSessionCatalog} 5 | import org.apache.spark.sql.catalyst.catalog.ExternalCatalogWithListener 6 | import org.apache.spark.sql.catalyst.parser.ParserInterface 7 | import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan 8 | import org.apache.spark.sql.catalyst.rules.Rule 9 | import org.apache.spark.sql.execution.SparkPlanner 10 | import org.apache.spark.sql.execution.adaptive.LogicalQueryStageStrategy 11 | import org.apache.spark.sql.execution.aggregate.ResolveEncodersInScalaAgg 12 | import org.apache.spark.sql.execution.analysis.DetectAmbiguousSelfJoin 13 | import org.apache.spark.sql.execution.command.CommandCheck 14 | import org.apache.spark.sql.execution.datasources._ 15 | import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Strategy, TableCapabilityCheck} 16 | import org.apache.spark.sql.execution.streaming.ResolveWriteToStream 17 | import org.apache.spark.sql.hbase.execution.HBaseSqlParser 18 | import org.apache.spark.sql.internal.{BaseSessionStateBuilder, SessionState, SparkUDFExpressionBuilder} 19 | import org.apache.spark.sql.{SparkSession, Strategy} 20 | 21 | /** 22 | * Created by wpy on 17-5-17. 23 | */ 24 | @Experimental 25 | @Stable 26 | class HBaseSessionStateBuilder( 27 | hbaseSession: HBaseSession, 28 | parentState: Option[SessionState] = None) 29 | extends BaseSessionStateBuilder(hbaseSession.asInstanceOf[SparkSession], parentState) { 30 | 31 | // override protected lazy val sqlParser: ParserInterface = extensions.buildParser(session, new HBaseSqlParser()) 32 | 33 | override protected lazy val catalog: HBaseSessionCatalog = { 34 | val catalog = new HBaseSessionCatalog( 35 | () => externalCatalog, 36 | () => hbaseSession.sharedState.globalTempViewManager, 37 | functionRegistry, 38 | tableFunctionRegistry, 39 | SessionState.newHadoopConf(hbaseSession.sparkContext.hadoopConfiguration, conf), 40 | sqlParser, 41 | resourceLoader, 42 | new SparkUDFExpressionBuilder) 43 | parentState.foreach(_.catalog.copyStateTo(catalog)) 44 | catalog 45 | } 46 | 47 | private def externalCatalog: ExternalCatalogWithListener = 48 | hbaseSession.sharedState.externalCatalog 49 | 50 | 51 | /** 52 | * A logical query plan `Analyzer` with rules specific to Hive. 53 | */ 54 | override protected def analyzer: Analyzer = new Analyzer(catalogManager) { 55 | override val extendedResolutionRules: Seq[Rule[LogicalPlan]] = { 56 | new ResolveHBaseTable(hbaseSession) +: 57 | new FindDataSourceTable(hbaseSession) +: 58 | new ResolveSQLOnFile(hbaseSession) +: 59 | new FallBackFileSourceV2(hbaseSession) +: 60 | ResolveEncodersInScalaAgg +: 61 | new ResolveSessionCatalog(catalogManager) +: 62 | ResolveWriteToStream +: 63 | customResolutionRules 64 | } 65 | 66 | override val postHocResolutionRules: Seq[Rule[LogicalPlan]] = 67 | DetectAmbiguousSelfJoin +: 68 | PreprocessTableCreation(hbaseSession.sessionState.catalog) +: 69 | PreprocessTableInsertion +: 70 | HBaseAnalysis +: 71 | DataSourceAnalysis +: 72 | customPostHocResolutionRules 73 | 74 | override val extendedCheckRules: Seq[LogicalPlan => Unit] = 75 | PreWriteCheck +: 76 | PreReadCheck +: 77 | HiveOnlyCheck +: 78 | TableCapabilityCheck +: 79 | CommandCheck +: 80 | customCheckRules 81 | } 82 | 83 | override protected def planner: SparkPlanner = { 84 | new SparkPlanner(hbaseSession, experimentalMethods) with HBaseStrategies { 85 | override val sparkSession: SparkSession = session 86 | 87 | override def extraPlanningStrategies: Seq[Strategy] = 88 | super.extraPlanningStrategies ++ customPlanningStrategies 89 | 90 | override def strategies: Seq[Strategy] = { 91 | experimentalMethods.extraStrategies ++ 92 | extraPlanningStrategies ++ ( 93 | LogicalQueryStageStrategy :: 94 | PythonEvals :: 95 | HBaseTableScans :: 96 | HBaseDataSource :: 97 | new DataSourceV2Strategy(session) :: 98 | FileSourceStrategy :: 99 | DataSourceStrategy :: 100 | SpecialLimits :: 101 | Aggregation :: 102 | Window :: 103 | JoinSelection :: 104 | InMemoryScans :: 105 | SparkScripts :: 106 | BasicOperators :: Nil) 107 | } 108 | } 109 | } 110 | 111 | override protected def newBuilder: NewBuilder = (session, _) => { 112 | val hs = new HBaseSession(session.sparkContext, session.sparkContext.hadoopConfiguration, Map.empty) 113 | new HBaseSessionStateBuilder(hs) 114 | } 115 | } 116 | -------------------------------------------------------------------------------- /spark-hbase/src/main/scala/org/apache/spark/sql/hbase/execution/HBaseTableScanExec.scala: -------------------------------------------------------------------------------- 1 | package org.apache.spark.sql.hbase.execution 2 | 3 | import org.apache.hadoop.hbase.TableName 4 | import org.apache.hadoop.hbase.client.Scan 5 | import org.apache.hadoop.hbase.filter._ 6 | import org.apache.hadoop.hbase.util.Bytes 7 | import org.apache.spark.rdd.RDD 8 | import org.apache.spark.sql.catalyst.InternalRow 9 | import org.apache.spark.sql.catalyst.catalog.CatalogTable 10 | import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap, AttributeSet, Expression, UnsafeProjection} 11 | import org.apache.spark.sql.execution.LeafExecNode 12 | import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} 13 | import org.apache.spark.sql.hbase.SparkHBaseConstants.TABLE_CONSTANTS 14 | import org.apache.spark.sql.hbase._ 15 | import org.apache.spark.sql.hbase.utils.{HBaseSparkDataUtils, HBaseSparkFilterUtils, HBaseSparkFormatUtils} 16 | 17 | /** 18 | * Created by wpy on 17-5-16. 19 | */ 20 | private[hbase] 21 | case class HBaseTableScanExec( 22 | requestedAttributes: Seq[Attribute], 23 | plan: HBasePlan, 24 | filter: Seq[Expression])( 25 | @transient private val hbaseSession: HBaseSession) 26 | extends LeafExecNode { 27 | val meta: CatalogTable = plan.tableMeta 28 | val parameters: Map[String, String] = meta.properties 29 | val tableName: String = meta.identifier.database.get + ":" + meta.identifier.table 30 | 31 | override lazy val metrics: Map[String, SQLMetric] = Map( 32 | "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows")) 33 | 34 | override def producedAttributes: AttributeSet = outputSet ++ 35 | AttributeSet(filter.flatMap(_.references)) 36 | 37 | private val originalAttributes = AttributeMap(plan.output.map(a => a -> a)) 38 | 39 | override val output: Seq[Attribute] = { 40 | // Retrieve the original attributes based on expression ID so that capitalization matches. 41 | requestedAttributes.map(originalAttributes) 42 | } 43 | 44 | 45 | override protected def doExecute(): RDD[InternalRow] = { 46 | // show num results in spark web ui 47 | val numOutputRows = longMetric("numOutputRows") 48 | 49 | val filterLists = filter.map(f => HBaseSparkFilterUtils.buildHBaseFilterList4Where(plan.dataCols, requestedAttributes, Some(f))) 50 | val hbaseFilter = HBaseSparkFilterUtils.combineHBaseFilterLists(filterLists) 51 | 52 | val scan = new Scan() 53 | addColumnFamiliesToScan(scan, hbaseFilter, filter) 54 | 55 | //read data from hbase 56 | hbaseSession.sqlContext.hbaseRDD(TableName.valueOf(tableName), scan) 57 | .mapPartitionsWithIndexInternal { (index, iter) => 58 | //判断schema中是否包含row_key信息 59 | val zippedSchema = schema.zipWithIndex 60 | val proj = UnsafeProjection.create(schema) 61 | val columnFamily = zippedSchema.map { case (field, idx) => 62 | val separateName = HBaseSparkFormatUtils.splitColumnAndQualifierName(field.name) 63 | (Bytes.toBytes(separateName.familyName), 64 | Bytes.toBytes(separateName.qualifierName), 65 | idx, 66 | HBaseSparkDataUtils.genHBaseToInternalRowConverter(field.dataType)) 67 | } 68 | proj.initialize(index) 69 | val size = schema.length 70 | val rowKey = zippedSchema.find(_._1.name == TABLE_CONSTANTS.ROW_KEY.getValue) 71 | if (rowKey.isDefined) { 72 | //需要rowKey数据 73 | val rowIdx = rowKey.get._2 74 | val filteredColumnFamily = columnFamily.filter(_._3 != rowIdx) 75 | iter.map { result => 76 | val r = HBaseSparkDataUtils.hbaseResult2InternalRowWithRowKey(result._2, size, filteredColumnFamily, columnFamily(rowIdx)) 77 | numOutputRows += 1 78 | proj(r) 79 | } 80 | } else { 81 | //不需要rowKey数据 82 | iter.map { result => 83 | val r = HBaseSparkDataUtils.hbaseResult2InternalRowWithoutRowKey(result._2, size, columnFamily) 84 | numOutputRows += 1 85 | proj(r) 86 | } 87 | } 88 | } 89 | } 90 | 91 | 92 | override def otherCopyArgs: Seq[AnyRef] = Seq(hbaseSession) 93 | 94 | //columnFamily_QualifierName <=== requestAttribute 95 | def addColumnFamiliesToScan(scan: Scan, filters: Option[Filter], predicate: Seq[Expression]): Scan = { 96 | // 添加需要显示结果的列 97 | requestedAttributes.foreach { attribute => 98 | val separateName = HBaseSparkFormatUtils.splitColumnAndQualifierName(attribute.name) 99 | if (attribute.name != TABLE_CONSTANTS.ROW_KEY.getValue) 100 | scan.addColumn(Bytes.toBytes(separateName.familyName), Bytes.toBytes(separateName.qualifierName)) 101 | } 102 | // 添加需要过滤的列 103 | predicate.foreach { expression => 104 | val attributes = expression.references.map(_.toAttribute).toSet 105 | attributes.filter(_.name != TABLE_CONSTANTS.ROW_KEY.getValue).foreach { attribute => 106 | val separateName = HBaseSparkFormatUtils.splitColumnAndQualifierName(attribute.name) 107 | scan.addColumn(Bytes.toBytes(separateName.familyName), Bytes.toBytes(separateName.qualifierName)) 108 | } 109 | } 110 | // 111 | scan.setCaching(1000) 112 | scan.readAllVersions() 113 | if (filters.isDefined) { 114 | scan.setFilter(filters.get) 115 | } 116 | scan 117 | } 118 | } 119 | -------------------------------------------------------------------------------- /spark-hbase/src/main/scala/org/apache/spark/sql/hbase/execution/InsertIntoHBaseTable.scala: -------------------------------------------------------------------------------- 1 | package org.apache.spark.sql.hbase.execution 2 | 3 | import org.apache.hadoop.hbase.TableName 4 | import org.apache.hadoop.hbase.client.{ConnectionFactory, Put} 5 | import org.apache.hadoop.hbase.util.Bytes 6 | import org.apache.spark.sql.catalyst.InternalRow 7 | import org.apache.spark.sql.catalyst.catalog.CatalogTable 8 | import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan 9 | import org.apache.spark.sql.execution.command.RunnableCommand 10 | import org.apache.spark.sql.hbase.SparkHBaseConstants.TABLE_CONSTANTS 11 | import org.apache.spark.sql.hbase.utils.{HBaseSparkDataUtils, HBaseSparkFormatUtils} 12 | import org.apache.spark.sql.{Dataset, Row, SparkSession} 13 | import org.apache.spark.util.Utils 14 | 15 | 16 | /** 17 | * Command for writing data out to a HBase table. 18 | * 19 | * This class is mostly a mess, for legacy reasons (since it evolved in organic ways and had to 20 | * follow Hive's internal implementations closely, which itself was a mess too). Please don't 21 | * blame Reynold for this! He was just moving code around! 22 | * 23 | * In the future we should converge the write path for Hive with the normal data source write path, 24 | * as defined in `org.apache.spark.sql.execution.datasources.FileFormatWriter`. 25 | * 26 | * @param table the metadata of the table. 27 | * @param query the logical plan representing data to write to. 28 | * @param overwrite overwrite existing table or partitions. 29 | * @param ifNotExists If true, only write if the table or partition does not exist. 30 | */ 31 | case class InsertIntoHBaseTable( 32 | table: CatalogTable, 33 | query: LogicalPlan, 34 | overwrite: Boolean, 35 | ifNotExists: Boolean) extends RunnableCommand { 36 | 37 | private def findRowKeyGenerator(table: CatalogTable): RowKeyGenerator = { 38 | val generatorClsName = table.properties("generator") 39 | val classLoader = Utils.getContextOrSparkClassLoader 40 | 41 | val constructor = classLoader.loadClass(generatorClsName).getConstructor() 42 | constructor.newInstance() match { 43 | case generator: RowKeyGenerator => generator 44 | case _ => null 45 | } 46 | } 47 | 48 | override def innerChildren: Seq[LogicalPlan] = query :: Nil 49 | 50 | /** 51 | * Inserts all the rows in the table into HBase. Row objects are properly serialized with the 52 | * `org.apache.hadoop.mapred.OutputFormat` provided by the table definition. 53 | */ 54 | override def run(sparkSession: SparkSession): Seq[Row] = { 55 | 56 | //put data into hbase here 57 | //反射表schema yml文件中自定义的rowKey生成器 58 | val generator = findRowKeyGenerator(table) 59 | //插入语句中是否指定了列名 60 | val querySchema = if (query.schema.length == table.schema.length) { 61 | table.schema 62 | } else { 63 | query.schema 64 | } 65 | //将查询时输入的列名转换为hbase的cf和qualifier的Bytes形式 66 | val bytesColNames = querySchema.fieldNames 67 | .map(HBaseSparkFormatUtils.splitColumnAndQualifierName) 68 | .map { separatedName => 69 | Bytes.toBytes(separatedName.familyName) -> Bytes.toBytes(separatedName.qualifierName) 70 | } 71 | //尝试获取rowKey字段在插入语句中的位置 72 | val rowKeyIdx: Int = querySchema.getFieldIndex(TABLE_CONSTANTS.ROW_KEY.getValue).getOrElse(-1) 73 | val queryContainsRowKey = rowKeyIdx != -1 74 | //生成对应的转换方法.用于将插入语句中的数据转换为HBase存储时的Bytes 75 | val rowConverters = HBaseSparkDataUtils.genRowConverters(querySchema) 76 | Dataset.ofRows(sparkSession, query).rdd.foreachPartition { rowIterator => 77 | val conn = ConnectionFactory.createConnection() 78 | val tableConn = conn.getTable(TableName.valueOf(table.identifier.database.get + ":" + table.identifier.table)) 79 | val putList = new java.util.ArrayList[Put]() 80 | rowIterator.foreach { row => 81 | //处理rowKey 82 | val rowKey = generator.genRowKey(row) 83 | val put = new Put(rowKey) 84 | if (!queryContainsRowKey) { 85 | for (i <- bytesColNames.indices) { 86 | val colName = bytesColNames(i) 87 | put.addColumn(colName._1, colName._2, rowConverters(i)(row, i)) 88 | } 89 | } else { 90 | for (i <- bytesColNames.indices) { 91 | if (i != rowKeyIdx) { 92 | val colName = bytesColNames(i) 93 | put.addColumn(colName._1, colName._2, rowConverters(i)(row, i)) 94 | } 95 | } 96 | } 97 | putList.add(put) 98 | //TODO 改到配置文件里 99 | if (putList.size() == 1000) { 100 | tableConn.put(putList) 101 | putList.clear() 102 | } 103 | } 104 | tableConn.put(putList) 105 | tableConn.close() 106 | } 107 | //TODO here need some tests and should load at last 108 | // un-cache this table. 109 | // sparkSession.catalog.uncacheTable(table.identifier.quotedString) 110 | sparkSession.sessionState.catalog.refreshTable(table.identifier) 111 | 112 | // It would be nice to just return the childRdd unchanged so insert operations could be chained, 113 | // however for now we return an empty list to simplify compatibility checks with hive, which 114 | // does not return anything for insert operations. 115 | // TODO: implement hbase compatibility as rules. 116 | Seq.empty[Row] 117 | } 118 | 119 | override protected def withNewChildrenInternal(newChildren: IndexedSeq[LogicalPlan]): LogicalPlan = { 120 | assert(newChildren.size == 1, "Incorrect number of children") 121 | copy(query = newChildren.head) 122 | } 123 | } 124 | -------------------------------------------------------------------------------- /spark-hbase/src/test/scala/org/apache/spark/sql/hbase/client/TestHBase.scala: -------------------------------------------------------------------------------- 1 | package org.apache.spark.sql.hbase.client 2 | 3 | import org.apache.hadoop.hbase.client.{Admin, ColumnFamilyDescriptorBuilder, Connection, Put, Scan, TableDescriptorBuilder} 4 | import org.apache.hadoop.hbase.util.Bytes 5 | import org.apache.hadoop.hbase.{NamespaceDescriptor, TableName} 6 | import org.apache.spark.internal.Logging 7 | import org.apache.spark.sql.hbase.TConstants 8 | import org.scalatest.funsuite.AnyFunSuite 9 | 10 | /** 11 | * Created by wpy on 17-5-18. 12 | */ 13 | class TestHBase extends AnyFunSuite with Logging { 14 | 15 | private lazy val table = TConstants.TEST_NAMESPACE + ":" + TConstants.TEST_TABLE_NAME 16 | 17 | def dbExists(admin: Admin, dbName: String): Boolean = { 18 | admin.listNamespaceDescriptors.exists(_.getName == dbName) 19 | } 20 | 21 | def tableExists(admin: Admin, table: String): Boolean = { 22 | admin.tableExists(TableName.valueOf(table)) 23 | } 24 | 25 | def createUserNamespaceAndTable(admin: Admin, namespaceName: String, 26 | table: String, cols: String*): Unit = { 27 | if (!admin.listNamespaces().contains(namespaceName)) { 28 | val namespace = NamespaceDescriptor.create(namespaceName).build() 29 | admin.createNamespace(namespace) 30 | } 31 | if (!admin.tableExists(TableName.valueOf(table))) { 32 | val tableDesc = TableDescriptorBuilder.newBuilder(TableName.valueOf(table)) 33 | cols.foreach { colName => 34 | val col = ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes(colName)).build() 35 | tableDesc.setColumnFamily(col) 36 | } 37 | admin.createTable(tableDesc.build()) 38 | admin.close() 39 | logInfo(s"Succeed in creating table $table") 40 | } 41 | } 42 | 43 | def insertData(conn: Connection, tableName: String, 44 | maxRowCnt: Int, maxQualifierCnt: Int, 45 | splitChar: String, columnNames: String*): Unit = { 46 | val bytesColumnNames = columnNames.map(Bytes.toBytes) 47 | 48 | //前面凑0对齐 49 | val rowPattern = s"%0${math.log10(maxRowCnt).toInt + 1}d" 50 | logInfo(s"Format Row with: $rowPattern") 51 | val qualifierPattern = s"%0${math.log10(maxQualifierCnt).toInt + 1}d" 52 | logInfo(s"Format qualifier with: $qualifierPattern") 53 | val splitterBytes = Bytes.toBytes(splitChar) 54 | val tableConn = conn.getTable(TableName.valueOf(tableName)) 55 | 56 | def insertIntoPut(put: Put, bytesRowKey: Array[Byte], bytesColumnName: Array[Byte], 57 | formattedQualifier: String, splitterBytes: Array[Byte]) = { 58 | //pattern: col+${splitter}+qualifier, A_01 59 | val qualifierBytes = Bytes.add(bytesColumnName, splitterBytes, Bytes.toBytes(formattedQualifier)) 60 | //pattern: col+${splitter}+qualifier+${splitter}+rowKey, A_01_0001 61 | val value = Bytes.add(qualifierBytes, splitterBytes, bytesRowKey) 62 | put.addColumn(bytesColumnName, qualifierBytes, value) 63 | put 64 | } 65 | 66 | for (i <- 0 until TConstants.MAX_ROW_CNT) { 67 | val rowKey = rowPattern.format(i) 68 | val bytesRowKey = Bytes.toBytes(rowKey) 69 | val put = new Put(Bytes.toBytes(rowKey)) 70 | 71 | for (j <- 0 until TConstants.MAX_QUALIFIER_CNT) { 72 | val formattedQualifier = qualifierPattern.format(j) 73 | //col value pattern: qualifier_rowKey 74 | bytesColumnNames.foreach { bytesColumnName => 75 | insertIntoPut(put, bytesRowKey, bytesColumnName, formattedQualifier, splitterBytes) 76 | } 77 | tableConn.put(put) 78 | } 79 | } 80 | logInfo(s"Succeed in inserting ${TConstants.MAX_ROW_CNT} rows") 81 | tableConn.close() 82 | conn.close() 83 | } 84 | 85 | def scan(conn: Connection, tableName: String, cols: String*): Unit = { 86 | 87 | import scala.collection.JavaConverters._ 88 | 89 | val scan = new Scan() 90 | // scan.withStartRow(Bytes.toBytes("")).withStopRow(Bytes.toBytes("")) 91 | scan.addFamily(Bytes.toBytes(TConstants.TEST_COL_A)) 92 | scan.addFamily(Bytes.toBytes(TConstants.TEST_COL_B)) 93 | val cnt = conn.getTable(TableName.valueOf(tableName)).getScanner(scan).iterator().asScala.count(_ => true) 94 | logInfo(s"max row numbers: $cnt") 95 | conn.close() 96 | } 97 | 98 | def dropTable(admin: Admin, table: String): Unit = { 99 | val tableName = TableName.valueOf(table) 100 | while (admin.tableExists(tableName)) { 101 | while (!admin.isTableDisabled(tableName)) { 102 | admin.disableTable(tableName) 103 | } 104 | admin.deleteTable(tableName) 105 | } 106 | logInfo(s"Succeed in dropping table: $table") 107 | admin.close() 108 | } 109 | 110 | 111 | test("create namespace and table") { 112 | createUserNamespaceAndTable( 113 | TConstants.admin, 114 | TConstants.TEST_NAMESPACE, 115 | // "pw:test_insert", 116 | // "T", 117 | table, 118 | TConstants.TEST_COL_A, 119 | TConstants.TEST_COL_B 120 | ) 121 | } 122 | 123 | test("insert some data") { 124 | insertData(TConstants.conn, table, 125 | 1000, 126 | 10, 127 | "_", 128 | TConstants.TEST_COL_A, 129 | TConstants.TEST_COL_B) 130 | } 131 | 132 | test("scan") { 133 | scan(TConstants.conn, 134 | table, 135 | TConstants.TEST_COL_A, 136 | TConstants.TEST_COL_B) 137 | } 138 | 139 | test("delete") { 140 | dropTable(TConstants.admin, table) 141 | } 142 | 143 | 144 | } 145 | 146 | object TestHBase { 147 | val testHBase = new TestHBase() 148 | 149 | def main(args: Array[String]): Unit = { 150 | testHBase.createUserNamespaceAndTable(TConstants.admin, TConstants.TEST_NAMESPACE, testHBase.table) 151 | testHBase.insertData(TConstants.conn, testHBase.table, 152 | 1000, 10, "_", 153 | TConstants.TEST_COL_A, 154 | TConstants.TEST_COL_B) 155 | testHBase.scan(TConstants.conn, testHBase.table, TConstants.TEST_COL_A, TConstants.TEST_COL_B) 156 | } 157 | } 158 | -------------------------------------------------------------------------------- /spark-hbase/src/main/scala/org/apache/spark/sql/hbase/execution/HBaseTableFormat.scala: -------------------------------------------------------------------------------- 1 | package org.apache.spark.sql.hbase.execution 2 | 3 | import org.apache.hadoop.conf.Configuration 4 | import org.apache.hadoop.fs.{FileStatus, Path} 5 | import org.apache.hadoop.hbase.HBaseConfiguration 6 | import org.apache.hadoop.hbase.client.{Mutation, Put} 7 | import org.apache.hadoop.hbase.io.ImmutableBytesWritable 8 | import org.apache.hadoop.hbase.mapreduce.TableOutputFormat 9 | import org.apache.hadoop.hbase.util.Bytes 10 | import org.apache.hadoop.mapreduce.{Job, RecordWriter, TaskAttemptContext} 11 | import org.apache.spark.internal.Logging 12 | import org.apache.spark.sql.SparkSession 13 | import org.apache.spark.sql.catalyst.InternalRow 14 | import org.apache.spark.sql.execution.datasources.{FileFormat, OutputWriter, OutputWriterFactory} 15 | import org.apache.spark.sql.hbase.SparkHBaseConstants.TABLE_CONSTANTS 16 | import org.apache.spark.sql.hbase.utils.{HBaseSparkDataUtils, HBaseSparkFormatUtils} 17 | import org.apache.spark.sql.sources.DataSourceRegister 18 | import org.apache.spark.sql.types.{StructField, StructType} 19 | 20 | import java.text.SimpleDateFormat 21 | import java.util.{Date, Locale} 22 | 23 | class HBaseTableFormat 24 | extends FileFormat 25 | with DataSourceRegister 26 | with Serializable 27 | with Logging { 28 | 29 | private val jobTrackerId: String = { 30 | val formatter = new SimpleDateFormat("yyyyMMddHHmmss", Locale.US) 31 | formatter.format(new Date()) 32 | } 33 | 34 | override def inferSchema(sparkSession: SparkSession, options: Map[String, String], files: Seq[FileStatus]): Option[StructType] = { 35 | throw new UnsupportedOperationException("inferSchema is not supported for hbase data source.") 36 | } 37 | 38 | /* override def buildReader(sparkSession: SparkSession, 39 | dataSchema: StructType, 40 | partitionSchema: StructType, 41 | requiredSchema: StructType, 42 | filters: Seq[Filter], 43 | options: Map[String, String], 44 | hadoopConf: Configuration): PartitionedFile => Iterator[InternalRow] = { 45 | val conf = HBaseConfiguration.create(hadoopConf) 46 | val inputFormat = new TableInputFormat() 47 | inputFormat.setConf(conf) 48 | val requiredEmpty = requiredSchema.isEmpty 49 | if (requiredEmpty) { 50 | _ => Iterator.empty 51 | } else { 52 | val len = requiredSchema.length 53 | val rowKeyIdx = requiredSchema.getFieldIndex(TABLE_CONSTANTS.ROW_KEY.getValue).getOrElse(-1) 54 | val dataColumns = requiredSchema.map { field => 55 | val separateName = HBaseSparkFormatUtils.splitColumnAndQualifierName(field.name) 56 | val family = Bytes.toBytes(separateName.familyName) 57 | val qualifier = Bytes.toBytes(separateName.qualifierName) 58 | val converter = HBaseSparkDataUtils.genHBaseToInternalRowConverter(field.dataType) 59 | val idx = requiredSchema.getFieldIndex(field.name).get 60 | (family, qualifier, idx, converter) 61 | } 62 | val id = sparkSession.sparkContext.newRddId() 63 | //return serializable function 64 | hTable => { 65 | new Iterator[InternalRow] { 66 | //all fields must be serialized 67 | private val attemptId = new TaskAttemptID(jobTrackerId, id, TaskType.MAP, 0, 0) 68 | private val hadoopAttemptContext = new TaskAttemptContextImpl(conf, attemptId) 69 | val hTableReader: RecordReader[ImmutableBytesWritable, Result] = inputFormat.createRecordReader(null, hadoopAttemptContext) 70 | val scanner = hTableReader 71 | var hasNextValue: Boolean = false 72 | 73 | override def hasNext: Boolean = { 74 | hasNextValue = scanner.nextKeyValue() 75 | hasNextValue 76 | } 77 | 78 | override def next(): InternalRow = { 79 | if (requiredEmpty) { 80 | new GenericInternalRow(0) 81 | } else { 82 | val row = new GenericInternalRow(len) 83 | val result = scanner.getCurrentValue 84 | if (rowKeyIdx != -1) { 85 | HBaseSparkDataUtils.hbaseResult2InternalRowWithRowKey(result,len,dataColumns,dataColumns(rowKeyIdx)) 86 | }else{ 87 | HBaseSparkDataUtils.hbaseResult2InternalRowWithoutRowKey(result,len,dataColumns) 88 | } 89 | row 90 | } 91 | } 92 | } 93 | } 94 | } 95 | }*/ 96 | 97 | override def prepareWrite(sparkSession: SparkSession, 98 | job: Job, 99 | options: Map[String, String], 100 | dataSchema: StructType): OutputWriterFactory = { 101 | val dirPath = new Path(options("path")) 102 | val tableName = dirPath.getName 103 | val namespace = dirPath.getParent.getName 104 | new OutputWriterFactory { 105 | override def getFileExtension(context: TaskAttemptContext): String = null 106 | 107 | override def newInstance( 108 | path: String, 109 | dataSchema: StructType, 110 | context: TaskAttemptContext): OutputWriter = { 111 | val conf = HBaseConfiguration.create() 112 | conf.set(TableOutputFormat.OUTPUT_TABLE, s"$namespace:$tableName") 113 | new HBaseTableOutputWriter(conf, context, dataSchema) 114 | } 115 | } 116 | 117 | } 118 | 119 | override def shortName(): String = "hbase" 120 | } 121 | 122 | class HBaseTableOutputWriter(conf: Configuration, context: TaskAttemptContext, dataSchema: StructType) extends OutputWriter { 123 | val tableOutputFormat = new TableOutputFormat[ImmutableBytesWritable]() 124 | tableOutputFormat.setConf(conf) 125 | val tableRecordWriter: RecordWriter[ImmutableBytesWritable, Mutation] = tableOutputFormat.getRecordWriter(context) 126 | 127 | val rowKeyIdx: Int = dataSchema.getFieldIndex(TABLE_CONSTANTS.ROW_KEY.getValue).get 128 | val filteredDataSchema: Seq[StructField] = dataSchema.filter(_.name != TABLE_CONSTANTS.ROW_KEY.getValue) 129 | val rowKeyConverter: (InternalRow, Int) => Array[Byte] = HBaseSparkDataUtils.interRowToHBaseFunc(dataSchema(rowKeyIdx).dataType) 130 | 131 | val schemaMap: Map[String, (Array[Byte], Array[Byte], Int, (InternalRow, Int) => Array[Byte])] = filteredDataSchema.map { field => 132 | val separateName = HBaseSparkFormatUtils.splitColumnAndQualifierName(field.name) 133 | field.name -> ( 134 | Bytes.toBytes(separateName.familyName), 135 | Bytes.toBytes(separateName.qualifierName), 136 | dataSchema.getFieldIndex(field.name).get, 137 | HBaseSparkDataUtils.interRowToHBaseFunc(field.dataType)) 138 | }.toMap 139 | 140 | 141 | override def write(row: InternalRow): Unit = { 142 | if (row.numFields > 0) { 143 | val rowKey = rowKeyConverter(row, rowKeyIdx) 144 | val put = new Put(rowKey) 145 | filteredDataSchema.foreach { field => 146 | val (family, qualifier, idx, converter) = schemaMap(field.name) 147 | put.addColumn(family, qualifier, converter(row, idx)) 148 | } 149 | tableRecordWriter.write(null, put) 150 | } 151 | } 152 | 153 | override def close(): Unit = tableRecordWriter.close(context) 154 | 155 | override def path(): String = { 156 | val name = context.getConfiguration.get("mapreduce.output.fileoutputformat.outputdir") 157 | if (name == null) { 158 | null 159 | } else { 160 | name 161 | } 162 | } 163 | } -------------------------------------------------------------------------------- /spark-hbase/src/main/scala/org/apache/spark/sql/hbase/HBaseSessionCatalog.scala: -------------------------------------------------------------------------------- 1 | package org.apache.spark.sql.hbase 2 | 3 | import org.apache.hadoop.conf.Configuration 4 | import org.apache.spark.sql.catalyst.{CatalystIdentifier, TableIdentifier} 5 | import org.apache.spark.sql.catalyst.analysis.{FunctionRegistry, GetViewColumnByNameAndOrdinal, NoSuchTableException, TableFunctionRegistry} 6 | import org.apache.spark.sql.catalyst.catalog._ 7 | import org.apache.spark.sql.catalyst.expressions.{Alias, UpCast} 8 | import org.apache.spark.sql.catalyst.parser.ParserInterface 9 | import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project, SubqueryAlias, View} 10 | import org.apache.spark.sql.internal.SQLConf 11 | 12 | import java.util.Locale 13 | import org.apache.spark.sql.hbase.utils.StructFieldConverters._ 14 | 15 | /** 16 | * Created by wpy on 17-5-17. 17 | */ 18 | private[sql] class HBaseSessionCatalog( 19 | externalCatalogBuilder: () => ExternalCatalog, 20 | globalTempViewManagerBuilder: () => GlobalTempViewManager, 21 | functionRegistry: FunctionRegistry, 22 | tableFunctionRegistry: TableFunctionRegistry, 23 | hadoopConf: Configuration, 24 | parser: ParserInterface, 25 | functionResourceLoader: FunctionResourceLoader, 26 | functionExpressionBuilder: FunctionExpressionBuilder) 27 | extends SessionCatalog( 28 | externalCatalogBuilder, 29 | globalTempViewManagerBuilder, 30 | functionRegistry, 31 | tableFunctionRegistry, 32 | hadoopConf, 33 | parser, 34 | functionResourceLoader, 35 | functionExpressionBuilder) { 36 | 37 | 38 | override def refreshTable(name: TableIdentifier): Unit = { 39 | super.refreshTable(name) 40 | } 41 | 42 | private def getDatabase(ident: CatalystIdentifier): Option[String] = { 43 | Some(format(ident.database.getOrElse(getCurrentDatabase))) 44 | } 45 | 46 | private def getCatalog(ident: TableIdentifier)={ 47 | if (conf.getConf(SQLConf.LEGACY_NON_IDENTIFIER_OUTPUT_CATALOG_NAME)) { 48 | ident.catalog 49 | } else { 50 | Some(format(ident.catalog.getOrElse("hbase_catalog"))) 51 | } 52 | } 53 | override def qualifyIdentifier(ident: TableIdentifier): TableIdentifier = { 54 | TableIdentifier( 55 | table = format(ident.table), 56 | database = getDatabase(ident), 57 | catalog = getCatalog(ident)) 58 | } 59 | 60 | /** 61 | * 通过table名返回对应table的logical plan 62 | */ 63 | override def lookupRelation(name: TableIdentifier): LogicalPlan = { 64 | synchronized { 65 | val db = format(name.database.getOrElse(currentDb)) 66 | val table = format(name.table) 67 | if (db == globalTempViewManager.database) { 68 | globalTempViewManager.get(table).map { viewDef => 69 | SubqueryAlias(table, viewDef) 70 | }.getOrElse(throw new NoSuchTableException(db, table)) 71 | } else if (name.database.isDefined || !tempViews.contains(table)) { 72 | val metadata = externalCatalog.getTable(db, table) 73 | if (metadata.tableType == CatalogTableType.VIEW) { 74 | val viewText = metadata.viewText.getOrElse(sys.error("Invalid view without text.")) 75 | // The relation is a view, so we wrap the relation by: 76 | // 1. Add a [[View]] operator over the relation to keep track of the view desc; 77 | // 2. Wrap the logical plan in a [[SubqueryAlias]] which tracks the name of the view. 78 | val child = View( 79 | desc = metadata, 80 | isTempView = true, 81 | child = parser.parsePlan(viewText)) 82 | SubqueryAlias(table, child) 83 | } else { 84 | if (metadata.provider.isDefined && metadata.provider.get.toLowerCase(Locale.ROOT).equals("hbase")) { 85 | // 数据源是HBase, 生成HBasePlan 86 | val tablePlan = HBasePlan( 87 | metadata, 88 | // we assume all the columns are nullable. 89 | metadata.schema.toAttributes, 90 | metadata.dataSchema.asNullable.toAttributes, 91 | metadata.partitionSchema.asNullable.toAttributes) 92 | SubqueryAlias(table, tablePlan) 93 | } else { 94 | // 数据源不是HBase, 生成其他外部数据源的Table 95 | getRelation(metadata) 96 | } 97 | } 98 | } else { 99 | SubqueryAlias(table, getTempViewPlan(tempViews(table))) 100 | } 101 | } 102 | } 103 | 104 | private def getTempViewPlan(viewInfo: TemporaryViewRelation): View = viewInfo.plan match { 105 | case Some(p) => View(desc = viewInfo.tableMeta, isTempView = true, child = p) 106 | case None => fromCatalogTable(viewInfo.tableMeta, isTempView = true) 107 | } 108 | 109 | private def fromCatalogTable(metadata: CatalogTable, isTempView: Boolean): View = { 110 | val viewText = metadata.viewText.getOrElse { 111 | throw new IllegalStateException("Invalid view without text.") 112 | } 113 | val viewConfigs = metadata.viewSQLConfigs 114 | val parsedPlan = SQLConf.withExistingConf(View.effectiveSQLConf(viewConfigs, isTempView)) { 115 | parser.parsePlan(viewText) 116 | } 117 | val viewColumnNames = if (metadata.viewQueryColumnNames.isEmpty) { 118 | // For view created before Spark 2.2.0, the view text is already fully qualified, the plan 119 | // output is the same with the view output. 120 | metadata.schema.fieldNames.toSeq 121 | } else { 122 | assert(metadata.viewQueryColumnNames.length == metadata.schema.length) 123 | metadata.viewQueryColumnNames 124 | } 125 | 126 | // For view queries like `SELECT * FROM t`, the schema of the referenced table/view may 127 | // change after the view has been created. We need to add an extra SELECT to pick the columns 128 | // according to the recorded column names (to get the correct view column ordering and omit 129 | // the extra columns that we don't require), with UpCast (to make sure the type change is 130 | // safe) and Alias (to respect user-specified view column names) according to the view schema 131 | // in the catalog. 132 | // Note that, the column names may have duplication, e.g. `CREATE VIEW v(x, y) AS 133 | // SELECT 1 col, 2 col`. We need to make sure that the matching attributes have the same 134 | // number of duplications, and pick the corresponding attribute by ordinal. 135 | val viewConf = View.effectiveSQLConf(metadata.viewSQLConfigs, isTempView) 136 | val normalizeColName: String => String = if (viewConf.caseSensitiveAnalysis) { 137 | identity 138 | } else { 139 | _.toLowerCase(Locale.ROOT) 140 | } 141 | val nameToCounts = viewColumnNames.groupBy(normalizeColName).mapValues(_.length) 142 | val nameToCurrentOrdinal = scala.collection.mutable.HashMap.empty[String, Int] 143 | val viewDDL = buildViewDDL(metadata, isTempView) 144 | 145 | val projectList = viewColumnNames.zip(metadata.schema).map { case (name, field) => 146 | val normalizedName = normalizeColName(name) 147 | val count = nameToCounts(normalizedName) 148 | val ordinal = nameToCurrentOrdinal.getOrElse(normalizedName, 0) 149 | nameToCurrentOrdinal(normalizedName) = ordinal + 1 150 | val col = GetViewColumnByNameAndOrdinal( 151 | metadata.identifier.toString, name, ordinal, count, viewDDL) 152 | Alias(UpCast(col, field.dataType), field.name)(explicitMetadata = Some(field.metadata)) 153 | } 154 | View(desc = metadata, isTempView = isTempView, child = Project(projectList, parsedPlan)) 155 | } 156 | 157 | private def buildViewDDL(metadata: CatalogTable, isTempView: Boolean): Option[String] = { 158 | if (isTempView) { 159 | None 160 | } else { 161 | val viewName = metadata.identifier.unquotedString 162 | val viewText = metadata.viewText.get 163 | val userSpecifiedColumns = 164 | if (metadata.schema.fieldNames.toSeq == metadata.viewQueryColumnNames) { 165 | "" 166 | } else { 167 | s"(${metadata.schema.fieldNames.mkString(", ")})" 168 | } 169 | Some(s"CREATE OR REPLACE VIEW $viewName $userSpecifiedColumns AS $viewText") 170 | } 171 | } 172 | } 173 | -------------------------------------------------------------------------------- /spark-hbase/src/main/scala/org/apache/spark/sql/hbase/HBaseExternalCatalog.scala: -------------------------------------------------------------------------------- 1 | package org.apache.spark.sql.hbase 2 | 3 | import org.apache.hadoop.conf.Configuration 4 | import org.apache.spark.SparkConf 5 | import org.apache.spark.internal.Logging 6 | import org.apache.spark.sql.catalyst.catalog._ 7 | import org.apache.spark.sql.catalyst.expressions.Expression 8 | import org.apache.spark.sql.hbase.client.{HBaseClient, HBaseClientImpl, IsolatedClientLoader} 9 | import org.apache.spark.sql.types.StructType 10 | 11 | /** 12 | * Created by wpy on 17-5-16. 13 | */ 14 | class HBaseExternalCatalog( 15 | conf: SparkConf, 16 | hadoopConf: Configuration, 17 | extraConfig: Map[String, String]) 18 | extends ExternalCatalog 19 | with Logging { 20 | 21 | import CatalogTypes.TablePartitionSpec 22 | 23 | val client: HBaseClient = IsolatedClientLoader.forVersion("3.0.0", "3.2.4", conf, hadoopConf, extraConfig).createClient() 24 | // val client = new HBaseClientImpl(IsolatedClientLoader.hbaseVersion("3.0.0"), conf, hadoopConf, Map.empty, this.getClass.getClassLoader, null) 25 | 26 | override def createDatabase(dbDefinition: CatalogDatabase, ignoreIfExists: Boolean): Unit = { 27 | client.createDatabase(dbDefinition, ignoreIfExists) 28 | } 29 | 30 | override def dropDatabase(db: String, ignoreIfNotExists: Boolean, cascade: Boolean): Unit = { 31 | client.dropDatabase(db, ignoreIfNotExists, cascade) 32 | } 33 | 34 | override def alterDatabase(dbDefinition: CatalogDatabase): Unit = { 35 | client.alterDatabase(dbDefinition) 36 | } 37 | 38 | override def getDatabase(db: String): CatalogDatabase = { 39 | client.getDatabase(db) 40 | } 41 | 42 | override def databaseExists(db: String): Boolean = { 43 | client.databaseExists(db) 44 | } 45 | 46 | override def listDatabases(): Seq[String] = { 47 | client.listDatabases(".*") 48 | } 49 | 50 | override def listDatabases(pattern: String): Seq[String] = { 51 | client.listDatabases(pattern) 52 | } 53 | 54 | override def setCurrentDatabase(db: String): Unit = { 55 | client.setCurrentDatabase(db) 56 | } 57 | 58 | override def createTable(tableDefinition: CatalogTable, ignoreIfExists: Boolean): Unit = { 59 | client.createTable(tableDefinition, ignoreIfExists) 60 | } 61 | 62 | override def dropTable(db: String, table: String, ignoreIfNotExists: Boolean, purge: Boolean): Unit = { 63 | client.dropTable(db, table, ignoreIfNotExists, purge) 64 | } 65 | 66 | override def renameTable(db: String, oldName: String, newName: String): Unit = { 67 | } 68 | 69 | override def alterTable(tableDefinition: CatalogTable): Unit = { 70 | } 71 | 72 | override def alterTableDataSchema(db: String, table: String, schema: StructType): Unit = { 73 | } 74 | 75 | override def getTable(db: String, table: String): CatalogTable = { 76 | client.getTable(db, table) 77 | } 78 | 79 | override def getTablesByName(db: String, tables: Seq[String]): Seq[CatalogTable] = { 80 | tables.map(getTable(db, _)) 81 | } 82 | 83 | override def tableExists(db: String, table: String): Boolean = { 84 | client.tableExists(db, table) 85 | } 86 | 87 | override def listTables(db: String): Seq[String] = { 88 | client.listTables(db) 89 | } 90 | 91 | override def listTables(db: String, pattern: String = ".*"): Seq[String] = { 92 | if (pattern == "*") 93 | listTables(db).filter(_.matches("." + pattern)) 94 | else listTables(db).filter(_.matches(pattern)) 95 | } 96 | 97 | override def listViews(db: String, pattern: String): Seq[String] = { 98 | throw new UnsupportedOperationException("listViews") 99 | } 100 | 101 | override def loadTable(db: String, table: String, loadPath: String, isOverwrite: Boolean, isSrcLocal: Boolean): Unit = { 102 | client.loadTable(loadPath, db + ":" + table, isOverwrite, isSrcLocal) 103 | } 104 | 105 | import scala.collection.JavaConverters._ 106 | 107 | override def loadPartition(db: String, table: String, loadPath: String, partition: TablePartitionSpec, isOverwrite: Boolean, inheritTableSpecs: Boolean, isSrcLocal: Boolean): Unit = { 108 | client.loadPartition(db, table, loadPath, partition.seq.asJava.asInstanceOf[java.util.LinkedHashMap[String, String]], isOverwrite, inheritTableSpecs, isSrcLocal) 109 | } 110 | 111 | override def loadDynamicPartitions(db: String, table: String, loadPath: String, partition: TablePartitionSpec, replace: Boolean, numDP: Int): Unit = { 112 | client.loadDynamicPartitions(db, table, loadPath, partition.seq.asJava.asInstanceOf[java.util.LinkedHashMap[String, String]], replace, numDP) 113 | } 114 | 115 | override def createPartitions(db: String, table: String, parts: Seq[CatalogTablePartition], ignoreIfExists: Boolean): Unit = { 116 | client.createPartitions(db, table, parts, ignoreIfExists) 117 | } 118 | 119 | override def dropPartitions(db: String, table: String, parts: Seq[TablePartitionSpec], ignoreIfNotExists: Boolean, purge: Boolean, retainData: Boolean): Unit = { 120 | client.dropPartitions(db, table, parts, ignoreIfNotExists, purge, retainData) 121 | } 122 | 123 | override def renamePartitions(db: String, table: String, specs: Seq[TablePartitionSpec], newSpecs: Seq[TablePartitionSpec]): Unit = { 124 | client.renamePartitions(db, table, specs, newSpecs) 125 | } 126 | 127 | override def alterPartitions(db: String, table: String, parts: Seq[CatalogTablePartition]): Unit = { 128 | client.alterPartitions(db, table, parts) 129 | } 130 | 131 | override def getPartition(db: String, table: String, spec: TablePartitionSpec): CatalogTablePartition = { 132 | client.getPartition(db, table, spec) 133 | } 134 | 135 | override def getPartitionOption(db: String, table: String, spec: TablePartitionSpec): Option[CatalogTablePartition] = { 136 | client.getPartitionOption(db, table, spec) 137 | } 138 | 139 | override def listPartitionNames(db: String, table: String, partialSpec: Option[TablePartitionSpec]): Seq[String] = { 140 | throw new UnsupportedOperationException("listPartitionNames") 141 | } 142 | 143 | override def listPartitions(db: String, table: String, partialSpec: Option[TablePartitionSpec]): Seq[CatalogTablePartition] = { 144 | throw new UnsupportedOperationException("listPartitions") 145 | } 146 | 147 | override def listPartitionsByFilter(db: String, table: String, predicates: Seq[Expression], defaultTimeZoneId: String): Seq[CatalogTablePartition] = { 148 | throw new UnsupportedOperationException("listPartitionsByFilter") 149 | } 150 | 151 | override def createFunction(db: String, funcDefinition: CatalogFunction): Unit = { 152 | throw new UnsupportedOperationException("doCreateFunction") 153 | } 154 | 155 | override def dropFunction(db: String, funcName: String): Unit = { 156 | throw new UnsupportedOperationException("doDropFunction") 157 | } 158 | 159 | override def renameFunction(db: String, oldName: String, newName: String): Unit = { 160 | throw new UnsupportedOperationException("doRenameFunction") 161 | } 162 | 163 | override def getFunction(db: String, funcName: String): CatalogFunction = { 164 | throw new UnsupportedOperationException("getFunction") 165 | } 166 | 167 | override def functionExists(db: String, funcName: String): Boolean = { 168 | throw new UnsupportedOperationException("functionExists") 169 | } 170 | 171 | override def listFunctions(db: String, pattern: String): Seq[String] = { 172 | throw new UnsupportedOperationException("listFunctions") 173 | } 174 | 175 | override def alterTableStats(db: String, table: String, stats: Option[CatalogStatistics]): Unit = stats match { 176 | case None => logInfo("alter table is ignored") 177 | case _ => throw new UnsupportedOperationException("alterTableStats") 178 | } 179 | 180 | override def alterFunction(db: String, funcDefinition: CatalogFunction): Unit = { 181 | throw new UnsupportedOperationException("doAlterFunction") 182 | } 183 | } 184 | 185 | object HBaseExternalCatalog { 186 | // If defined and larger than 3, a new table will be created with the nubmer of region specified. 187 | val newTable = "newtable" 188 | // The json string specifying hbase catalog information 189 | val regionStart = "regionStart" 190 | val defaultRegionStart = "aaaaaaa" 191 | val regionEnd = "regionEnd" 192 | val defaultRegionEnd = "zzzzzzz" 193 | val tableCatalog = "catalog" 194 | // The row key with format key1:key2 specifying table row key 195 | val rowKey = "rowkey" 196 | // The key for hbase table whose value specify namespace and table name 197 | val table = "table" 198 | // The namespace of hbase table 199 | val nameSpace = "namespace" 200 | // The name of hbase table 201 | val tableName = "name" 202 | // The name of columns in hbase catalog 203 | val columns = "columns" 204 | val cf = "cf" 205 | val col = "col" 206 | val `type` = "type" 207 | // the name of avro schema json string 208 | val avro = "avro" 209 | val delimiter: Byte = 0 210 | val serdes = "serdes" 211 | val length = "length" 212 | 213 | val SPARK_SQL_PREFIX = "spark.sql." 214 | 215 | val TABLE_KEY: String = "hbase.table" 216 | val SCHEMA_COLUMNS_MAPPING_KEY: String = "hbase.columns.mapping" 217 | } -------------------------------------------------------------------------------- /spark-hbase/src/main/scala/org/apache/spark/sql/hbase/client/HBaseClient.scala: -------------------------------------------------------------------------------- 1 | package org.apache.spark.sql.hbase.client 2 | 3 | import java.io.PrintStream 4 | 5 | import org.apache.hadoop.hbase.client.{Admin, Connection} 6 | import org.apache.spark.sql.catalyst.analysis.{NoSuchPartitionException, NoSuchTableException} 7 | import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec 8 | import org.apache.spark.sql.catalyst.catalog.{CatalogDatabase, CatalogTable, CatalogTablePartition} 9 | import org.apache.spark.sql.catalyst.expressions.Expression 10 | 11 | /** 12 | * Created by wpy on 2017/5/11. 13 | * 14 | * we will add more function in the future 15 | * LIKE: 16 | * {compact & split Table} 17 | * {} 18 | */ 19 | //TODO 20 | private[hbase] trait HBaseClient { 21 | 22 | def connection: Connection 23 | 24 | def admin: Admin 25 | 26 | /** Returns the Hive Version of this client. */ 27 | def version: HBaseVersion 28 | 29 | /** Returns the configuration for the given key in the current session. */ 30 | def getConf(key: String, defaultValue: String): String 31 | 32 | def setOut(stream: PrintStream): Unit 33 | 34 | def setInfo(stream: PrintStream): Unit 35 | 36 | def setError(stream: PrintStream): Unit 37 | 38 | /** Returns the names of all tables in the given database. */ 39 | def listTables(dbName: String): Seq[String] 40 | 41 | /** Returns the names of tables in the given database that matches the given pattern. */ 42 | def listTables(dbName: String, pattern: String): Seq[String] 43 | 44 | /** Sets the name of current database. */ 45 | def setCurrentDatabase(databaseName: String): Unit 46 | 47 | /** Returns the metadata for specified database, throwing an exception if it doesn't exist */ 48 | def getDatabase(name: String): CatalogDatabase 49 | 50 | /** Return whether a table/view with the specified name exists. */ 51 | def databaseExists(dbName: String): Boolean 52 | 53 | /** List the names of all the databases that match the specified pattern. */ 54 | def listDatabases(pattern: String): Seq[String] 55 | 56 | /** Return whether a table/view with the specified name exists. */ 57 | def tableExists(dbName: String, tableName: String): Boolean 58 | 59 | /** Returns the specified table, or throws [[NoSuchTableException]]. */ 60 | final def getTable(dbName: String, tableName: String): CatalogTable = { 61 | getTableOption(dbName, tableName).getOrElse(throw new NoSuchTableException(dbName, tableName)) 62 | } 63 | 64 | /** Returns the metadata for the specified table or None if it doesn't exist. */ 65 | def getTableOption(dbName: String, tableName: String): Option[CatalogTable] 66 | 67 | /** Creates a table with the given metadata. */ 68 | def createTable(table: CatalogTable, ignoreIfExists: Boolean): Unit 69 | 70 | /** Drop the specified table. */ 71 | def dropTable(dbName: String, tableName: String, ignoreIfNotExists: Boolean, purge: Boolean): Unit 72 | 73 | /** Alter a table whose name matches the one specified in `table`, assuming it exists. */ 74 | final def alterTable(table: CatalogTable): Unit = alterTable(table.identifier.table, table) 75 | 76 | /** Updates the given table with new metadata, optionally renaming the table. */ 77 | def alterTable(tableName: String, table: CatalogTable): Unit 78 | 79 | /** Creates a new database with the given name. */ 80 | def createDatabase(database: CatalogDatabase, ignoreIfExists: Boolean): Unit 81 | 82 | /** 83 | * Drop the specified database, if it exists. 84 | * 85 | * @param name database to drop 86 | * @param ignoreIfNotExists if true, do not throw error if the database does not exist 87 | * @param cascade whether to remove all associated objects such as tables and functions 88 | */ 89 | def dropDatabase(name: String, ignoreIfNotExists: Boolean, cascade: Boolean): Unit 90 | 91 | /** 92 | * Alter a database whose name matches the one specified in `database`, assuming it exists. 93 | */ 94 | def alterDatabase(database: CatalogDatabase): Unit 95 | 96 | /** 97 | * Create one or many partitions in the given table. 98 | */ 99 | def createPartitions( 100 | db: String, 101 | table: String, 102 | parts: Seq[CatalogTablePartition], 103 | ignoreIfExists: Boolean): Unit 104 | 105 | /** 106 | * Drop one or many partitions in the given table, assuming they exist. 107 | */ 108 | def dropPartitions( 109 | db: String, 110 | table: String, 111 | specs: Seq[TablePartitionSpec], 112 | ignoreIfNotExists: Boolean, 113 | purge: Boolean, 114 | retainData: Boolean): Unit 115 | 116 | /** 117 | * Rename one or many existing table partitions, assuming they exist. 118 | */ 119 | def renamePartitions( 120 | db: String, 121 | table: String, 122 | specs: Seq[TablePartitionSpec], 123 | newSpecs: Seq[TablePartitionSpec]): Unit 124 | 125 | /** 126 | * Alter one or more table partitions whose specs match the ones specified in `newParts`, 127 | * assuming the partitions exist. 128 | */ 129 | def alterPartitions( 130 | db: String, 131 | table: String, 132 | newParts: Seq[CatalogTablePartition]): Unit 133 | 134 | /** Returns the specified partition, or throws [[NoSuchPartitionException]]. */ 135 | final def getPartition( 136 | dbName: String, 137 | tableName: String, 138 | spec: TablePartitionSpec): CatalogTablePartition = { 139 | getPartitionOption(dbName, tableName, spec).getOrElse { 140 | throw new NoSuchPartitionException(dbName, tableName, spec) 141 | } 142 | } 143 | 144 | /** 145 | * Returns the partition names for the given table that match the supplied partition spec. 146 | * If no partition spec is specified, all partitions are returned. 147 | * 148 | * The returned sequence is sorted as strings. 149 | */ 150 | def getPartitionNames( 151 | table: CatalogTable, 152 | partialSpec: Option[TablePartitionSpec] = None): Seq[String] 153 | 154 | /** Returns the specified partition or None if it does not exist. */ 155 | final def getPartitionOption( 156 | db: String, 157 | table: String, 158 | spec: TablePartitionSpec): Option[CatalogTablePartition] = { 159 | getPartitionOption(getTable(db, table), spec) 160 | } 161 | 162 | /** Returns the specified partition or None if it does not exist. */ 163 | def getPartitionOption( 164 | table: CatalogTable, 165 | spec: TablePartitionSpec): Option[CatalogTablePartition] 166 | 167 | /** 168 | * Returns the partitions for the given table that match the supplied partition spec. 169 | * If no partition spec is specified, all partitions are returned. 170 | */ 171 | final def getPartitions( 172 | db: String, 173 | table: String, 174 | partialSpec: Option[TablePartitionSpec]): Seq[CatalogTablePartition] = { 175 | getPartitions(getTable(db, table), partialSpec) 176 | } 177 | 178 | /** 179 | * Returns the partitions for the given table that match the supplied partition spec. 180 | * If no partition spec is specified, all partitions are returned. 181 | */ 182 | def getPartitions( 183 | catalogTable: CatalogTable, 184 | partialSpec: Option[TablePartitionSpec] = None): Seq[CatalogTablePartition] 185 | 186 | /** Returns partitions filtered by predicates for the given table. */ 187 | def getPartitionsByFilter( 188 | catalogTable: CatalogTable, 189 | predicates: Seq[Expression]): Seq[CatalogTablePartition] 190 | 191 | /** Loads a static partition into an existing table. */ 192 | def loadPartition( 193 | loadPath: String, 194 | dbName: String, 195 | tableName: String, 196 | partSpec: java.util.LinkedHashMap[String, String], // Hive relies on LinkedHashMap ordering 197 | replace: Boolean, 198 | inheritTableSpecs: Boolean, 199 | isSrcLocal: Boolean): Unit 200 | 201 | /** Loads data into an existing table. */ 202 | def loadTable( 203 | loadPath: String, // TODO URI 204 | tableName: String, 205 | replace: Boolean, 206 | isSrcLocal: Boolean): Unit 207 | 208 | /** Loads new dynamic partitions into an existing table. */ 209 | def loadDynamicPartitions( 210 | loadPath: String, 211 | dbName: String, 212 | tableName: String, 213 | partSpec: java.util.LinkedHashMap[String, String], // Hive relies on LinkedHashMap ordering 214 | replace: Boolean, 215 | numDP: Int): Unit 216 | 217 | /** Add a jar into class loader */ 218 | def addJar(path: String): Unit 219 | 220 | /** Return a [[HBaseClient]] as new session, that will share the class loader and Hive client */ 221 | def newSession(): HBaseClient 222 | 223 | /** Used for testing only. Removes all metadata from this instance of Hive. */ 224 | def reset(): Unit 225 | 226 | } 227 | -------------------------------------------------------------------------------- /spark-hbase/src/main/scala/org/apache/spark/sql/hbase/utils/HBaseSparkDataUtils.scala: -------------------------------------------------------------------------------- 1 | package org.apache.spark.sql.hbase.utils 2 | 3 | import org.apache.hadoop.hbase.client.{RegionInfo, Result} 4 | import org.apache.hadoop.hbase.util.Bytes 5 | import org.apache.spark.sql.Row 6 | import org.apache.spark.sql.catalyst.InternalRow 7 | import org.apache.spark.sql.catalyst.expressions.GenericInternalRow 8 | import org.apache.spark.sql.hbase.types.RegionInfoUDT 9 | import org.apache.spark.sql.types.{BooleanType, ByteType, DataType, DoubleType, FloatType, IntegerType, LongType, ObjectType, ShortType, StringType, StructType, TimestampType, UserDefinedType} 10 | import org.apache.spark.unsafe.types.UTF8String 11 | 12 | object HBaseSparkDataUtils extends Serializable { 13 | // column family, qualifier, a function that could transform bytes data and set into InternalRow 14 | type CF_QUALIFIER_CONVERTER = (Array[Byte], Array[Byte], Int, (InternalRow, Int, Array[Byte]) => Unit) 15 | 16 | private val HBASE_ROW_BYTES = Bytes.toBytes("row_key") 17 | 18 | /** 19 | * generate a data converter, this converter could get data from 20 | * spark [InternalRow] and transform data to HBase Bytes Value with dataType 21 | * 22 | * @param dataType 数据类型 23 | * @return 24 | */ 25 | def interRowToHBaseFunc(dataType: DataType): (InternalRow, Int) => Array[Byte] = dataType match { 26 | case ByteType => 27 | (internalRow, i) => Array(internalRow.getByte(i)) 28 | 29 | case StringType => 30 | (internalRow, i) => Bytes.toBytes(internalRow.getUTF8String(i).toString) 31 | 32 | //convert to milli seconds 33 | case TimestampType => 34 | (internalRow, i) => Bytes.toBytes(internalRow.getLong(i) / 1000) 35 | case LongType => 36 | (internalRow, i) => Bytes.toBytes(internalRow.getLong(i)) 37 | case IntegerType => 38 | (internalRow, i) => Bytes.toBytes(internalRow.getInt(i)) 39 | case ShortType => 40 | (internalRow, i) => Bytes.toBytes(internalRow.getShort(i)) 41 | 42 | case BooleanType => 43 | (internalRow, i) => Bytes.toBytes(internalRow.getBoolean(i)) 44 | 45 | case DoubleType => 46 | (internalRow, i) => Bytes.toBytes(internalRow.getDouble(i)) 47 | case FloatType => 48 | (internalRow, i) => Bytes.toBytes(internalRow.getFloat(i)) 49 | 50 | case _ => 51 | (internalRow, i) => internalRow.getBinary(i) 52 | } 53 | 54 | def rowToHBaseFunc(dataType: DataType):(Row,Int)=>Array[Byte] = dataType match { 55 | case ByteType => 56 | (row, i) => Array(row.getByte(i)) 57 | 58 | case StringType => 59 | (row, i) => Bytes.toBytes(row.getString(i)) 60 | 61 | //convert to milli seconds 62 | case TimestampType => 63 | (row, i) => Bytes.toBytes(row.getLong(i) / 1000) 64 | case LongType => 65 | (row, i) => Bytes.toBytes(row.getLong(i)) 66 | case IntegerType => 67 | (row, i) => Bytes.toBytes(row.getInt(i)) 68 | case ShortType => 69 | (row, i) => Bytes.toBytes(row.getShort(i)) 70 | 71 | case BooleanType => 72 | (row, i) => Bytes.toBytes(row.getBoolean(i)) 73 | 74 | case DoubleType => 75 | (row, i) => Bytes.toBytes(row.getDouble(i)) 76 | case FloatType => 77 | (row, i) => Bytes.toBytes(row.getFloat(i)) 78 | 79 | case _ => 80 | (row, i) => Bytes.toBytes(row.getString(i)) 81 | } 82 | 83 | def genInternalRowConverters(structType: StructType): Seq[(InternalRow, Int) => Array[Byte]] ={ 84 | structType.map(_.dataType).map(interRowToHBaseFunc) 85 | } 86 | 87 | def genRowConverters(structType: StructType): Seq[(Row, Int) => Array[Byte]] ={ 88 | structType.map(_.dataType).map(rowToHBaseFunc) 89 | } 90 | 91 | /** 92 | * generate a data converter that could transform HBase Bytes Value to Spark InternalRow 93 | * 94 | * @param dataType 数据类型 95 | * @return 96 | */ 97 | def genHBaseToInternalRowConverter(dataType: DataType): (InternalRow, Int, Array[Byte]) => Unit = dataType match { 98 | case ByteType => 99 | (internalRow, i, v) => internalRow.update(i, v.head) 100 | 101 | case StringType => 102 | (internalRow, i, v) => internalRow.update(i, UTF8String.fromBytes(v)) 103 | 104 | //convert to milli seconds 105 | case TimestampType => 106 | (internalRow, i, v) => internalRow.update(i, Bytes.toLong(v) * 1000) 107 | case LongType => 108 | (internalRow, i, v) => internalRow.update(i, Bytes.toLong(v)) 109 | case IntegerType => 110 | (internalRow, i, v) => internalRow.update(i, Bytes.toInt(v)) 111 | case ShortType => 112 | (internalRow, i, v) => internalRow.update(i, Bytes.toShort(v)) 113 | 114 | case BooleanType => 115 | (internalRow, i, v) => internalRow.update(i, Bytes.toBoolean(v)) 116 | 117 | case DoubleType => 118 | (internalRow, i, v) => internalRow.update(i, Bytes.toDouble(v)) 119 | case FloatType => 120 | (internalRow, i, v) => internalRow.update(i, Bytes.toFloat(v)) 121 | 122 | case RegionInfoUDT => 123 | (internalRow, i, v) => internalRow.update(i, v) 124 | 125 | case _ => 126 | (internalRow, i, v) => internalRow.update(i, v) 127 | } 128 | 129 | def genHBaseToInternalRowConverterWithOffset(dataType: DataType): (InternalRow, Int, Array[Byte], Int, Int) => Unit = dataType match { 130 | case ByteType => 131 | (internalRow, i, v, offset, len) => internalRow.update(i, v(offset)) 132 | 133 | case StringType => 134 | (internalRow, i, v, offset, len) => internalRow.update(i, UTF8String.fromBytes(v, offset, len)) 135 | 136 | //convert to milli seconds 137 | case TimestampType => 138 | (internalRow, i, v, offset, len) => internalRow.update(i, Bytes.toLong(v, offset) * 1000) 139 | case LongType => 140 | (internalRow, i, v, offset, len) => internalRow.update(i, Bytes.toLong(v, offset)) 141 | case IntegerType => 142 | (internalRow, i, v, offset, len) => internalRow.update(i, Bytes.toInt(v, offset)) 143 | case ShortType => 144 | (internalRow, i, v, offset, len) => internalRow.update(i, Bytes.toShort(v, offset)) 145 | 146 | case BooleanType => 147 | (internalRow, i, v, offset, len) => internalRow.update(i, Bytes.toBoolean(Array(v(offset)))) 148 | 149 | case DoubleType => 150 | (internalRow, i, v, offset, len) => internalRow.update(i, Bytes.toDouble(v, offset)) 151 | case FloatType => 152 | (internalRow, i, v, offset, len) => internalRow.update(i, Bytes.toFloat(v, offset)) 153 | 154 | case _ => 155 | (internalRow, i, v, offset, len) => internalRow.update(i, Bytes.copy(v, offset, len)) 156 | } 157 | 158 | 159 | /** 160 | * 对HBase中qualifier的数据进行转换 161 | * 不支持row_key 162 | * 163 | * @param result HBaseResult 164 | * @param size 查询列的个数 165 | * @param cols 查询的列集合(不含row_key) 166 | * @return 167 | */ 168 | def hbaseResult2InternalRowWithoutRowKey(result: Result, size: Int, cols: Seq[CF_QUALIFIER_CONVERTER]): InternalRow = { 169 | val internalRow = new GenericInternalRow(size) 170 | cols.foreach { case (family, qualifier, idx, convert) => 171 | val v = result.getValue(family, qualifier) 172 | if (v == null) { 173 | internalRow.update(idx, null) 174 | } else { 175 | convert(internalRow, idx, v) 176 | } 177 | } 178 | internalRow 179 | } 180 | 181 | def hbaseResult2InternalRowWithRowKey(result: Result, 182 | size: Int, 183 | cols: Seq[CF_QUALIFIER_CONVERTER], 184 | row: CF_QUALIFIER_CONVERTER): InternalRow = { 185 | val internalRow = hbaseResult2InternalRowWithoutRowKey(result, size, cols) 186 | row._4(internalRow, row._3, result.getRow) 187 | internalRow 188 | } 189 | 190 | 191 | def toBytes(data: Any, dataType: DataType): Array[Byte] = dataType match { 192 | case ByteType if data.isInstanceOf[Byte] => 193 | Array(data.asInstanceOf[Byte]) 194 | 195 | case ByteType if data.isInstanceOf[java.lang.Byte] => 196 | Array(data.asInstanceOf[java.lang.Byte]) 197 | 198 | case StringType if data.isInstanceOf[String] => 199 | Bytes.toBytes(data.asInstanceOf[String]) 200 | 201 | //convert to seconds 202 | case TimestampType if data.isInstanceOf[Long] => 203 | Bytes.toBytes(data.asInstanceOf[Long] / 1000) 204 | 205 | case TimestampType if data.isInstanceOf[java.lang.Long] => 206 | Bytes.toBytes(data.asInstanceOf[java.lang.Long] / 1000) 207 | 208 | case LongType if data.isInstanceOf[Long] => 209 | Bytes.toBytes(data.asInstanceOf[Long]) 210 | 211 | case TimestampType if data.isInstanceOf[Long] => 212 | Bytes.toBytes(data.asInstanceOf[Long]) 213 | 214 | case IntegerType if data.isInstanceOf[Int] => 215 | Bytes.toBytes(data.asInstanceOf[Int]) 216 | 217 | case TimestampType if data.isInstanceOf[java.lang.Integer] => 218 | Bytes.toBytes(data.asInstanceOf[java.lang.Integer]) 219 | 220 | case ShortType if data.isInstanceOf[Short] => 221 | Bytes.toBytes(data.asInstanceOf[Short]) 222 | 223 | case ShortType if data.isInstanceOf[java.lang.Short] => 224 | Bytes.toBytes(data.asInstanceOf[java.lang.Short]) 225 | 226 | case BooleanType if data.isInstanceOf[Boolean] => 227 | Bytes.toBytes(data.asInstanceOf[Boolean]) 228 | 229 | case BooleanType if data.isInstanceOf[java.lang.Boolean] => 230 | Bytes.toBytes(data.asInstanceOf[java.lang.Boolean]) 231 | 232 | case DoubleType if data.isInstanceOf[Double] => 233 | Bytes.toBytes(data.asInstanceOf[Double]) 234 | 235 | case DoubleType if data.isInstanceOf[java.lang.Double] => 236 | Bytes.toBytes(data.asInstanceOf[java.lang.Double]) 237 | 238 | case FloatType if data.isInstanceOf[Float] => 239 | Bytes.toBytes(data.asInstanceOf[Float]) 240 | 241 | case FloatType if data.isInstanceOf[java.lang.Float] => 242 | Bytes.toBytes(data.asInstanceOf[java.lang.Float]) 243 | 244 | case _ => 245 | Bytes.toBytes(data.toString) 246 | } 247 | 248 | 249 | } 250 | -------------------------------------------------------------------------------- /spark-hbase/src/main/scala/org/apache/spark/sql/hbase/HBaseSQLContext.scala: -------------------------------------------------------------------------------- 1 | package org.apache.spark.sql.hbase 2 | 3 | import org.apache.hadoop.conf.Configuration 4 | import org.apache.hadoop.fs.{FileSystem, Path} 5 | import org.apache.hadoop.hbase._ 6 | import org.apache.hadoop.hbase.client._ 7 | import org.apache.hadoop.hbase.io.ImmutableBytesWritable 8 | import org.apache.hadoop.hbase.mapreduce.{IdentityTableMapper, TableInputFormat, TableMapReduceUtil} 9 | import org.apache.hadoop.mapred.JobConf 10 | import org.apache.hadoop.mapreduce.Job 11 | import org.apache.hadoop.security.{Credentials, UserGroupInformation} 12 | import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod 13 | import org.apache.spark.api.java.JavaSparkContext 14 | import org.apache.spark.broadcast.Broadcast 15 | import org.apache.spark.deploy.SparkHadoopUtil 16 | import org.apache.spark.internal.Logging 17 | import org.apache.spark.rdd.RDD 18 | import org.apache.spark.sql.catalog.Catalog 19 | import org.apache.spark.sql.catalyst.catalog.{ExternalCatalogEvent, ExternalCatalogWithListener} 20 | import org.apache.spark.sql.hbase.types.RegionInfoUDT 21 | import org.apache.spark.sql.internal.{SessionState, SharedState, StaticSQLConf} 22 | import org.apache.spark.sql.types.UDTRegistration 23 | import org.apache.spark.sql.{SQLContext, SparkSession} 24 | import org.apache.spark.{SerializableWritable, SparkContext} 25 | 26 | import scala.annotation.meta.param 27 | import scala.reflect.ClassTag 28 | 29 | /** 30 | * Created by wpy on 17-5-16. 31 | */ 32 | 33 | /** 34 | * 扩展SparkSQLContext功能, 提供对HBase的支持 35 | */ 36 | class HBaseSQLContext private[hbase]( 37 | @(transient@param) _hbaseSession: HBaseSession, 38 | @transient val config: Configuration, 39 | @transient extraConfig: Map[String, String], 40 | val tmpHdfsConfigFile: String = null) 41 | extends SQLContext(_hbaseSession) 42 | with Logging { 43 | self => 44 | @transient private var tmpHdfsConfiguration: Configuration = config 45 | @transient private var appliedCredentials = false 46 | @transient private val job: Job = Job.getInstance(config) 47 | TableMapReduceUtil.initCredentials(job) 48 | @transient private var credentials = job.getCredentials 49 | private val broadcastedConf = _hbaseSession.sparkContext.broadcast(new SerializableWritable(config)) 50 | private val credentialsConf = _hbaseSession.sparkContext.broadcast(new SerializableWritable(job.getCredentials)) 51 | 52 | 53 | if (tmpHdfsConfigFile != null && config != null) { 54 | val fs = FileSystem.newInstance(config) 55 | val tmpPath = new Path(tmpHdfsConfigFile) 56 | if (!fs.exists(tmpPath)) { 57 | val outputStream = fs.create(tmpPath) 58 | config.write(outputStream) 59 | outputStream.close() 60 | } else { 61 | logWarning("tmpHdfsConfigDir " + tmpHdfsConfigFile + " exist!!") 62 | } 63 | } 64 | LatestHBaseContextCache.latest = this 65 | 66 | def this(sc: SparkContext, extraConfig: Map[String, String]) = { 67 | this( 68 | new HBaseSession( 69 | LatestHBaseContextCache.withHBaseExternalCatalog(sc), 70 | new Configuration(), 71 | extraConfig), 72 | new Configuration(), 73 | extraConfig, 74 | null) 75 | } 76 | 77 | def this(sc: SparkContext) = this(sc, Map.empty[String, String]) 78 | 79 | def this(sc: JavaSparkContext, extraConfig: Map[String, String]) = this(sc.sc, extraConfig) 80 | 81 | def this(sc: JavaSparkContext) = this(sc.sc, Map.empty[String, String]) 82 | 83 | /** 84 | * Returns a new HBaseContext as new session, which will have separated SQLConf, UDF/UDAF, 85 | * temporary tables and SessionState, but sharing the same CacheManager, IsolatedClientLoader 86 | * and HBase client (both of execution and metadata) with existing HBaseContext. 87 | */ 88 | override def newSession(): HBaseSQLContext = { 89 | new HBaseSQLContext(_hbaseSession.newSession(), self.config, extraConfig) 90 | } 91 | 92 | /** 93 | * Invalidate and refresh all the cached the metadata of the given table. For performance reasons, 94 | * Spark SQL or the external data source library it uses might cache certain metadata about a 95 | * table, such as the location of blocks. When those change outside of Spark SQL, users should 96 | * call this function to invalidate the cache. 97 | * 98 | * @since 1.3.0 99 | */ 100 | def refreshTable(tableName: String): Unit = { 101 | sparkSession.catalog.refreshTable(tableName) 102 | } 103 | 104 | /** 105 | * This function will use the native HBase TableInputFormat with the 106 | * given scan object to generate a new RDD 107 | * 108 | * @param tableName the name of the table to scan 109 | * @param scan the HBase scan object to use to read data from HBase 110 | * @param f function to convert a Result object from HBase into 111 | * what the user wants in the final generated RDD 112 | * @return new RDD with results from scan 113 | */ 114 | def hbaseRDD[U: ClassTag](tableName: TableName, scan: Scan, 115 | f: ((ImmutableBytesWritable, Result)) => U): RDD[U] = { 116 | 117 | val job: Job = Job.getInstance(getConf(broadcastedConf)) 118 | 119 | 120 | TableMapReduceUtil.initCredentials(job) 121 | TableMapReduceUtil.initTableMapperJob(tableName, scan, 122 | classOf[IdentityTableMapper], null, null, job) 123 | 124 | val jconf = new JobConf(job.getConfiguration) 125 | SparkHadoopUtil.get.addCredentials(jconf) 126 | val rdd = new NewHBaseRDD(_hbaseSession.sparkContext, 127 | classOf[TableInputFormat], 128 | classOf[ImmutableBytesWritable], 129 | classOf[Result], 130 | job.getConfiguration, 131 | this) 132 | rdd.map(f) 133 | } 134 | 135 | /** 136 | * A overloaded version of HBaseContext hbaseRDD that defines the 137 | * type of the resulting RDD 138 | * 139 | * @param tableName the name of the table to scan 140 | * @param scans the HBase scan object to use to read data from HBase 141 | * @return New RDD with results from scan 142 | * 143 | */ 144 | def hbaseRDD(tableName: TableName, scans: Scan): RDD[(ImmutableBytesWritable, Result)] = { 145 | hbaseRDD[(ImmutableBytesWritable, Result)]( 146 | tableName, 147 | scans, 148 | (r: (ImmutableBytesWritable, Result)) => r) 149 | } 150 | 151 | 152 | private def getConf(configBroadcast: Broadcast[SerializableWritable[Configuration]]): Configuration = { 153 | 154 | if (tmpHdfsConfiguration == null && tmpHdfsConfigFile != null) { 155 | val fs = FileSystem.newInstance(SparkHadoopUtil.get.conf) 156 | val inputStream = fs.open(new Path(tmpHdfsConfigFile)) 157 | tmpHdfsConfiguration = new Configuration(false) 158 | tmpHdfsConfiguration.readFields(inputStream) 159 | inputStream.close() 160 | } 161 | 162 | if (tmpHdfsConfiguration == null) { 163 | try { 164 | tmpHdfsConfiguration = configBroadcast.value.value 165 | } catch { 166 | case ex: Exception => logError("Unable to getConfig from broadcast", ex) 167 | } 168 | } 169 | tmpHdfsConfiguration 170 | } 171 | 172 | /** 173 | * underlining wrapper all get mapPartition functions in HBaseContext 174 | */ 175 | private class GetMapPartition[T, U](tableName: TableName, 176 | batchSize: Integer, 177 | makeGet: T => Get, 178 | convertResult: Result => U) 179 | extends Serializable { 180 | 181 | val tName: Array[Byte] = tableName.getName 182 | 183 | def run(iterator: Iterator[T], connection: Connection): Iterator[U] = { 184 | val table = connection.getTable(TableName.valueOf(tName)) 185 | 186 | val gets = new java.util.ArrayList[Get]() 187 | var res = List[U]() 188 | 189 | while (iterator.hasNext) { 190 | gets.add(makeGet(iterator.next())) 191 | 192 | if (gets.size() == batchSize) { 193 | val results = table.get(gets) 194 | res = res ++ results.map(convertResult) 195 | gets.clear() 196 | } 197 | } 198 | if (gets.size() > 0) { 199 | val results = table.get(gets) 200 | res = res ++ results.map(convertResult) 201 | gets.clear() 202 | } 203 | table.close() 204 | res.iterator 205 | } 206 | } 207 | 208 | /** 209 | * Produces a ClassTag[T], which is actually just a casted ClassTag[AnyRef]. 210 | * 211 | * This method is used to keep ClassTags out of the external Java API, as 212 | * the Java compiler cannot produce them automatically. While this 213 | * ClassTag-faking does please the compiler, it can cause problems at runtime 214 | * if the Scala API relies on ClassTags for correctness. 215 | * 216 | * Often, though, a ClassTag[AnyRef] will not lead to incorrect behavior, 217 | * just worse performance or security issues. 218 | * For instance, an Array of AnyRef can hold any type T, but may lose primitive 219 | * specialization. 220 | */ 221 | private[spark] 222 | def fakeClassTag[T]: ClassTag[T] = ClassTag.AnyRef.asInstanceOf[ClassTag[T]] 223 | 224 | } 225 | 226 | object LatestHBaseContextCache { 227 | def withHBaseExternalCatalog(sc: SparkContext): SparkContext = { 228 | sc.conf.set(StaticSQLConf.CATALOG_IMPLEMENTATION.key, "hbase") 229 | sc 230 | } 231 | 232 | var latest: HBaseSQLContext = _ 233 | } 234 | 235 | /** 236 | * 237 | * @param sc spark context 实例 238 | * @param config hadoop相关设置 239 | * @param extraConfig 用户自定义设置: 240 | * {spark.hbase.client.impl -> HBaseClient接口具体实现类 241 | * schema.file.url -> 默认HBaseClient实现中需要用到的hbase table schema文件路径} 242 | */ 243 | class HBaseSession( 244 | @transient val sc: SparkContext, 245 | @transient val config: Configuration, 246 | @transient extraConfig: Map[String, String]) extends SparkSession(sc) { 247 | self => 248 | UDTRegistration.register(classOf[RegionInfo].getCanonicalName, classOf[RegionInfoUDT].getCanonicalName) 249 | @transient 250 | override lazy val sessionState: SessionState = { 251 | new HBaseSessionStateBuilder(this, None).build() 252 | } 253 | 254 | override def newSession(): HBaseSession = { 255 | new HBaseSession(sc, config, extraConfig) 256 | } 257 | 258 | @transient override lazy val catalog: Catalog = new HBaseCatalogImpl(self) 259 | 260 | @transient override lazy val sharedState: SharedState = new HBaseSharedState(sc, initialSessionOptions, extraConfig) 261 | 262 | override val sqlContext: HBaseSQLContext = new HBaseSQLContext(this, config, extraConfig) 263 | 264 | } 265 | 266 | private[hbase] class HBaseSharedState( 267 | val sc: SparkContext, 268 | initialConfigs: scala.collection.Map[String, String], 269 | extraConfig: Map[String, String]) 270 | extends SharedState(sc, initialConfigs) { 271 | 272 | override lazy val externalCatalog: ExternalCatalogWithListener = { 273 | val externalCatalog = new HBaseExternalCatalog( 274 | sc.conf, 275 | HBaseConfiguration.create(), 276 | extraConfig) 277 | val wrapped = new ExternalCatalogWithListener(externalCatalog) 278 | wrapped.addListener((event: ExternalCatalogEvent) => sparkContext.listenerBus.post(event)) 279 | wrapped 280 | } 281 | 282 | } -------------------------------------------------------------------------------- /spark-hbase/src/main/scala/org/apache/spark/sql/hbase/execution/HBaseFileFormat.scala: -------------------------------------------------------------------------------- 1 | package org.apache.spark.sql.hbase.execution 2 | 3 | import org.apache.hadoop.conf.Configuration 4 | import org.apache.hadoop.fs.{FileStatus, FileSystem, Path} 5 | import org.apache.hadoop.hbase.{Cell, CellBuilderFactory, CellBuilderType, HBaseConfiguration} 6 | import org.apache.hadoop.hbase.client.{Mutation, Put} 7 | import org.apache.hadoop.hbase.io.ImmutableBytesWritable 8 | import org.apache.hadoop.hbase.io.hfile.{HFile, HFileScanner} 9 | import org.apache.hadoop.hbase.mapreduce.{HFileOutputFormat2, TableOutputFormat} 10 | import org.apache.hadoop.hbase.util.Bytes 11 | import org.apache.hadoop.mapreduce.{Job, JobID, RecordWriter, TaskAttemptContext} 12 | import org.apache.spark.internal.Logging 13 | import org.apache.spark.sql.SparkSession 14 | import org.apache.spark.sql.catalyst.InternalRow 15 | import org.apache.spark.sql.catalyst.expressions.GenericInternalRow 16 | import org.apache.spark.sql.execution.datasources.{FileFormat, OutputWriter, OutputWriterFactory, PartitionedFile} 17 | import org.apache.spark.sql.hbase.SparkHBaseConstants.TABLE_CONSTANTS 18 | import org.apache.spark.sql.hbase.utils.{HBaseSparkDataUtils, HBaseSparkFormatUtils} 19 | import org.apache.spark.sql.sources.{DataSourceRegister, Filter} 20 | import org.apache.spark.sql.types._ 21 | import org.apache.spark.util.SerializableConfiguration 22 | 23 | import java.text.SimpleDateFormat 24 | import java.util.{Date, Locale} 25 | 26 | /** 27 | * 以Table为单位存取HFile文件 28 | */ 29 | class HBaseFileFormat 30 | extends FileFormat 31 | with DataSourceRegister 32 | with Serializable 33 | with Logging { 34 | 35 | private class HBaseRowArrayByteBuff( 36 | val bytes: Array[Byte], 37 | val offset: Int, 38 | val len: Int) 39 | extends Comparable[HBaseRowArrayByteBuff] 40 | with Serializable { 41 | def this(bytes: Array[Byte]) = this(bytes, 0, bytes.length) 42 | 43 | override def compareTo(t: HBaseRowArrayByteBuff): Int = Bytes.compareTo( 44 | this.bytes, this.offset, this.len, 45 | t.bytes, t.offset, t.len) 46 | 47 | override def toString: String = Bytes.toString(bytes, offset, len) 48 | } 49 | 50 | 51 | override def inferSchema( 52 | sparkSession: SparkSession, 53 | options: Map[String, String], 54 | files: Seq[FileStatus]): Option[StructType] = { 55 | throw new UnsupportedOperationException("inferSchema is not supported for hbase data source.") 56 | } 57 | 58 | /** 59 | * read HFile 60 | * 61 | * @param sparkSession spark session 62 | * @param dataSchema data: HBase Qualifier 63 | * @param partitionSchema partition is not supported in HBase ,so empty partition here 64 | * @param requiredSchema required HBase Qualifier 65 | * @param filters Hbase filter 66 | * @param options not use 67 | * @param hadoopConf hadoop configuration 68 | * @return 69 | */ 70 | override def buildReader( 71 | sparkSession: SparkSession, 72 | dataSchema: StructType, 73 | partitionSchema: StructType, 74 | requiredSchema: StructType, 75 | filters: Seq[Filter], 76 | options: Map[String, String], 77 | hadoopConf: Configuration): PartitionedFile => Iterator[InternalRow] = { 78 | val requiredQualifierNameMap = new java.util.TreeMap[HBaseRowArrayByteBuff, ((InternalRow, Int, Array[Byte], Int, Int) => Unit, Int)]() 79 | val rowKeyField = dataSchema.find(_.name == TABLE_CONSTANTS.ROW_KEY.getValue) 80 | assert(rowKeyField.isDefined) 81 | val rowKeyConverter = HBaseSparkDataUtils.genHBaseToInternalRowConverterWithOffset(rowKeyField.get.dataType) 82 | val requiredEmpty = requiredSchema.isEmpty 83 | val requiredSchemaContainsRowKey = requiredSchema.exists(_.name == TABLE_CONSTANTS.ROW_KEY.getValue) 84 | val requiredRowKeyOnly = requiredSchema.length == 1 && requiredSchemaContainsRowKey 85 | val len = requiredSchema.length 86 | val rowKeyIdx = if (requiredSchemaContainsRowKey) requiredSchema.getFieldIndex(TABLE_CONSTANTS.ROW_KEY.getValue).get else len 87 | 88 | requiredSchema.filter(_.name != TABLE_CONSTANTS.ROW_KEY.getValue).foreach { field => 89 | val qualifier = new HBaseRowArrayByteBuff(Bytes.toBytes(field.name)) 90 | val converter = HBaseSparkDataUtils.genHBaseToInternalRowConverterWithOffset(field.dataType) 91 | val idx = requiredSchema.getFieldIndex(field.name).get 92 | requiredQualifierNameMap.put(qualifier, (converter, idx)) 93 | } 94 | val broadcastedHadoopConf = { 95 | sparkSession.sparkContext.broadcast(new SerializableConfiguration(hadoopConf)) 96 | } 97 | //return serializable function 98 | hfile => { 99 | new Iterator[InternalRow] { 100 | //all fields must be serialized 101 | var seeked = false 102 | val broadcastConfValue: Configuration = broadcastedHadoopConf.value.value 103 | val fs: FileSystem = FileSystem.get(broadcastConfValue) 104 | val hFileReader: HFile.Reader = HFile.createReader(fs, new Path(hfile.filePath.toUri), broadcastedHadoopConf.value.value) 105 | val scanner: HFileScanner = hFileReader.getScanner(broadcastConfValue, false, false) 106 | var hashNextValue: Boolean = false 107 | 108 | override def hasNext: Boolean = { 109 | if (hashNextValue) { 110 | true 111 | } else { 112 | var hasNext: Boolean = false 113 | if (!seeked) { 114 | hasNext = scanner.seekTo() 115 | seeked = true 116 | } else { 117 | if (scanner.isSeeked) hasNext = scanner.next() 118 | } 119 | if (!hasNext) { 120 | hFileReader.close() 121 | false 122 | } else { 123 | hashNextValue = true 124 | true 125 | } 126 | } 127 | } 128 | 129 | override def next(): InternalRow = { 130 | val row = if (requiredEmpty) { 131 | new GenericInternalRow(0) 132 | } else { 133 | new GenericInternalRow(len) 134 | } 135 | var cell = scanner.getCell 136 | var lastKey: HBaseRowArrayByteBuff = new HBaseRowArrayByteBuff(cell.getRowArray, cell.getRowOffset, cell.getRowLength) 137 | var curKey = lastKey 138 | if (requiredSchemaContainsRowKey) { 139 | // append row_key to internalRow 140 | rowKeyConverter(row, rowKeyIdx, cell.getRowArray, cell.getRowOffset, cell.getRowLength) 141 | } 142 | while (scanner.isSeeked && lastKey.compareTo(curKey) == 0) { 143 | lastKey = curKey 144 | if (!requiredEmpty && !requiredRowKeyOnly) { 145 | // this hfile already under column family folder, so here family name is unnecessary 146 | var curQualifierName = new HBaseRowArrayByteBuff( 147 | cell.getQualifierArray, 148 | cell.getQualifierOffset, 149 | cell.getQualifierLength) 150 | while (!requiredQualifierNameMap.containsKey(curQualifierName) && hashNextValue) { 151 | if (scanner.next()) { 152 | cell = scanner.getCell 153 | curQualifierName = new HBaseRowArrayByteBuff( 154 | cell.getQualifierArray, 155 | cell.getQualifierOffset, 156 | cell.getQualifierLength) 157 | } else { 158 | hashNextValue = false 159 | } 160 | } 161 | val (converter, idx) = requiredQualifierNameMap.get(curQualifierName) 162 | val value = cell.getValueArray 163 | converter(row, idx, value, cell.getValueOffset, cell.getValueLength) 164 | // only put needed value into row 165 | } 166 | if (scanner.next()) { 167 | cell = scanner.getCell 168 | curKey = new HBaseRowArrayByteBuff(cell.getRowArray, cell.getRowOffset, cell.getRowLength) 169 | hashNextValue = true 170 | } else { 171 | hashNextValue = false 172 | } 173 | } 174 | row 175 | } 176 | } 177 | } 178 | } 179 | 180 | override def prepareWrite(sparkSession: SparkSession, 181 | job: Job, 182 | options: Map[String, String], 183 | dataSchema: StructType): OutputWriterFactory = { 184 | val dirPath = new Path(options("path")) 185 | val tableName = dirPath.getName 186 | val namespace = dirPath.getParent.getName 187 | new OutputWriterFactory { 188 | override def getFileExtension(context: TaskAttemptContext): String = "hfile" 189 | 190 | override def newInstance( 191 | path: String, 192 | dataSchema: StructType, 193 | context: TaskAttemptContext): OutputWriter = { 194 | context.getConfiguration.set("hbase.mapreduce.hfileoutputformat.table.name", s"$namespace:$tableName") 195 | new HBaseOutputWriter(context, dataSchema) 196 | } 197 | } 198 | 199 | } 200 | 201 | override def shortName(): String = "hbase" 202 | } 203 | 204 | class HBaseOutputWriter(context: TaskAttemptContext, dataSchema: StructType) extends OutputWriter { 205 | val hFileOutputFormat = new HFileOutputFormat2() 206 | val hfileRecordWriter: RecordWriter[ImmutableBytesWritable, Cell] = hFileOutputFormat.getRecordWriter(context) 207 | 208 | val rowKeyIdx: Int = dataSchema.getFieldIndex(TABLE_CONSTANTS.ROW_KEY.getValue).get 209 | val filteredDataSchema: Seq[StructField] = dataSchema.filter(_.name != TABLE_CONSTANTS.ROW_KEY.getValue) 210 | val rowKeyConverter: (InternalRow, Int) => Array[Byte] = HBaseSparkDataUtils.interRowToHBaseFunc(dataSchema(rowKeyIdx).dataType) 211 | 212 | val schemaMap: Map[String, (Array[Byte], Array[Byte], Int, (InternalRow, Int) => Array[Byte])] = filteredDataSchema.map { field => 213 | val separateName = HBaseSparkFormatUtils.splitColumnAndQualifierName(field.name) 214 | field.name -> ( 215 | Bytes.toBytes(separateName.familyName), 216 | Bytes.toBytes(separateName.qualifierName), 217 | dataSchema.getFieldIndex(field.name).get, 218 | HBaseSparkDataUtils.interRowToHBaseFunc(field.dataType)) 219 | }.toMap 220 | 221 | 222 | override def write(row: InternalRow): Unit = { 223 | if (row.numFields > 0) { 224 | val rowKey = rowKeyConverter(row, rowKeyIdx) 225 | val put = new Put(rowKey) 226 | filteredDataSchema.foreach { field => 227 | val cellBuilder = CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY) 228 | val (family, qualifier, idx, converter) = schemaMap(field.name) 229 | cellBuilder.setFamily(family) 230 | cellBuilder.setQualifier(qualifier) 231 | cellBuilder.setValue(converter(row, idx)) 232 | val cell = cellBuilder.build() 233 | hfileRecordWriter.write(null,cell) 234 | } 235 | } 236 | } 237 | 238 | override def close(): Unit = hfileRecordWriter.close(context) 239 | 240 | override def path(): String = { 241 | val name = context.getConfiguration.get("mapreduce.output.fileoutputformat.outputdir") 242 | if (name == null) { 243 | null 244 | } else { 245 | name 246 | } 247 | } 248 | } -------------------------------------------------------------------------------- /spark-hbase/src/main/scala/org/apache/spark/sql/hbase/utils/HBaseSparkFilterUtils.scala: -------------------------------------------------------------------------------- 1 | package org.apache.spark.sql.hbase.utils 2 | 3 | import org.apache.hadoop.hbase.CompareOperator 4 | import org.apache.hadoop.hbase.filter.{BinaryComparator, BinaryPrefixComparator, ByteArrayComparable, Filter, FilterList, NullComparator, RegexStringComparator, RowFilter, SingleColumnValueFilter, SubstringComparator} 5 | import org.apache.hadoop.hbase.util.Bytes 6 | import org.apache.spark.sql.catalyst.expressions.{And, Attribute, AttributeReference, Cast, Contains, EndsWith, EqualTo, Expression, GreaterThan, GreaterThanOrEqual, InSet, IsNotNull, IsNull, LessThan, LessThanOrEqual, Literal, Or, StartsWith} 7 | import org.apache.spark.sql.hbase.SparkHBaseConstants.TABLE_CONSTANTS 8 | import org.apache.spark.sql.types.{BooleanType, ByteType, DoubleType, FloatType, IntegerType, LongType, ShortType, StringType} 9 | import org.apache.spark.unsafe.types.UTF8String 10 | 11 | object HBaseSparkFilterUtils { 12 | 13 | def combineHBaseFilterLists(filterLists: Seq[Option[FilterList]]): Option[Filter] = { 14 | val filterList = new FilterList() 15 | filterLists.foreach { case Some(filter) => filterList.addFilter(filter) } 16 | if (filterList.getFilters.isEmpty) { 17 | None 18 | } else { 19 | Some(filterList) 20 | } 21 | } 22 | 23 | def buildHBaseFilterList4Where( 24 | dataCols: Seq[Attribute], 25 | requestedAttributes: Seq[Attribute], 26 | filter: Option[Expression]): Option[FilterList] = { 27 | if (filter.isEmpty) { 28 | None 29 | } else { 30 | val expression = filter.get 31 | expression match { 32 | case And(left, right) => 33 | val filters = new java.util.ArrayList[Filter] 34 | 35 | if (left != null) { 36 | val leftFilterList = buildHBaseFilterList4Where(dataCols, requestedAttributes, Some(left)) 37 | add2FilterList(filters, leftFilterList, FilterList.Operator.MUST_PASS_ALL) 38 | } 39 | if (right != null) { 40 | val rightFilterList = buildHBaseFilterList4Where(dataCols, requestedAttributes, Some(right)) 41 | add2FilterList(filters, rightFilterList, FilterList.Operator.MUST_PASS_ALL) 42 | } 43 | Some(new FilterList(FilterList.Operator.MUST_PASS_ALL, filters)) 44 | 45 | case Or(left, right) => 46 | val filters = new java.util.ArrayList[Filter] 47 | if (left != null) { 48 | val leftFilterList = buildHBaseFilterList4Where(dataCols, requestedAttributes, Some(left)) 49 | add2FilterList(filters, leftFilterList, FilterList.Operator.MUST_PASS_ONE) 50 | } 51 | if (right != null) { 52 | val rightFilterList = buildHBaseFilterList4Where(dataCols, requestedAttributes, Some(right)) 53 | add2FilterList(filters, rightFilterList, FilterList.Operator.MUST_PASS_ONE) 54 | } 55 | Some(new FilterList(FilterList.Operator.MUST_PASS_ONE, filters)) 56 | 57 | case InSet(value@AttributeReference(name, dataType, _, _), hset) => 58 | dataCols.find(_.name == name) match { 59 | case Some(attribute) => 60 | val separateName = HBaseSparkFormatUtils.splitColumnAndQualifierName(attribute.name) 61 | val filterList = new FilterList(FilterList.Operator.MUST_PASS_ONE) 62 | hset.foreach { item => 63 | val filter = new SingleColumnValueFilter( 64 | Bytes.toBytes(separateName.familyName), Bytes.toBytes(separateName.qualifierName), CompareOperator.EQUAL, 65 | new BinaryComparator(HBaseSparkDataUtils.toBytes(item, dataType))) 66 | filterList.addFilter(filter) 67 | } 68 | Some(filterList) 69 | case _ => None 70 | } 71 | 72 | case IsNull(left: AttributeReference) => 73 | createNullFilter(requestedAttributes, left) 74 | 75 | case IsNotNull(left: AttributeReference) => 76 | createNotNullFilter(requestedAttributes, left) 77 | 78 | case GreaterThan(left: AttributeReference, right: Literal) => 79 | createSingleColumnValueFilter(dataCols, left, right, CompareOperator.GREATER) 80 | 81 | case GreaterThan(left: Literal, right: AttributeReference) => 82 | createSingleColumnValueFilter(dataCols, right, left, CompareOperator.GREATER) 83 | 84 | case GreaterThanOrEqual(left: AttributeReference, right: Literal) => 85 | createSingleColumnValueFilter(dataCols, left, right, CompareOperator.GREATER_OR_EQUAL) 86 | 87 | case GreaterThanOrEqual(left: Literal, right: AttributeReference) => 88 | createSingleColumnValueFilter(dataCols, right, left, CompareOperator.GREATER_OR_EQUAL) 89 | 90 | case EqualTo(left: AttributeReference, right: Literal) => 91 | createSingleColumnValueFilter(dataCols, left, right, CompareOperator.EQUAL) 92 | 93 | case EqualTo(left: Literal, right: AttributeReference) => 94 | createSingleColumnValueFilter(dataCols, right, left, CompareOperator.EQUAL) 95 | 96 | case EqualTo(left: Cast, right: Literal) => 97 | val leftValue: AttributeReference = left.child.asInstanceOf[AttributeReference] 98 | val rightDecimal = BigDecimal(right.value.toString).bigDecimal 99 | val rightValue: Literal = Literal(rightDecimal.stripTrailingZeros().toPlainString) 100 | createSingleColumnValueFilter(dataCols, leftValue, rightValue, CompareOperator.EQUAL) 101 | 102 | case LessThan(left: AttributeReference, right: Literal) => 103 | createSingleColumnValueFilter(dataCols, left, right, CompareOperator.LESS) 104 | 105 | case LessThan(left: Literal, right: AttributeReference) => 106 | createSingleColumnValueFilter(dataCols, right, left, CompareOperator.LESS) 107 | 108 | case LessThanOrEqual(left: AttributeReference, right: Literal) => 109 | createSingleColumnValueFilter(dataCols, left, right, CompareOperator.LESS_OR_EQUAL) 110 | 111 | case LessThanOrEqual(left: Literal, right: AttributeReference) => 112 | createSingleColumnValueFilter(dataCols, right, left, CompareOperator.LESS_OR_EQUAL) 113 | 114 | case StartsWith(left: AttributeReference, right: Literal) => 115 | val regexStringComparator = new RegexStringComparator(".*" + right.value + "$") 116 | createSingleColumnValueFilter(dataCols, left, right, CompareOperator.EQUAL, regexStringComparator) 117 | 118 | case EndsWith(left: AttributeReference, right: Literal) => 119 | val binaryPrefixComparator = new BinaryPrefixComparator(Bytes.toBytes(right.value.toString)) 120 | createSingleColumnValueFilter(dataCols, left, right, CompareOperator.EQUAL, binaryPrefixComparator) 121 | 122 | case Contains(left: AttributeReference, right: Literal) => 123 | val substringComparator = new SubstringComparator(right.value.toString) 124 | createSingleColumnValueFilter(dataCols, left, right, CompareOperator.EQUAL, substringComparator) 125 | 126 | case _ => None 127 | } 128 | } 129 | } 130 | 131 | def createNullFilter( 132 | requestedAttributes: Seq[Attribute], 133 | left: AttributeReference): Option[FilterList] = { 134 | requestedAttributes.find(_.name == left.name) match { 135 | case Some(attribute) => 136 | val separateName = HBaseSparkFormatUtils.splitColumnAndQualifierName(attribute.name) 137 | val filter = new SingleColumnValueFilter( 138 | Bytes.toBytes(separateName.familyName), 139 | Bytes.toBytes(separateName.qualifierName), 140 | CompareOperator.EQUAL, 141 | new NullComparator()) 142 | filter.setFilterIfMissing(true) 143 | Some(new FilterList(filter)) 144 | case _ => None 145 | } 146 | } 147 | 148 | def createNotNullFilter( 149 | requestedAttributes: Seq[Attribute], 150 | left: AttributeReference): Option[FilterList] = { 151 | requestedAttributes.find(_.name == left.name) match { 152 | case Some(attribute) => 153 | val separateName = HBaseSparkFormatUtils.splitColumnAndQualifierName(attribute.name) 154 | val filter = new SingleColumnValueFilter( 155 | Bytes.toBytes(separateName.familyName), 156 | Bytes.toBytes(separateName.qualifierName), 157 | CompareOperator.NOT_EQUAL, 158 | new NullComparator()) 159 | filter.setFilterIfMissing(true) 160 | Some(new FilterList(filter)) 161 | 162 | case _ => None 163 | } 164 | } 165 | 166 | def createSingleColumnValueFilter( 167 | dataCols: Seq[Attribute], 168 | left: AttributeReference, 169 | right: Literal, 170 | compareOp: CompareOperator, 171 | comparable: ByteArrayComparable = null): Option[FilterList] = { 172 | dataCols.find(_.name == left.name) match { 173 | case Some(attribute) => 174 | val hbaseFilter = if (attribute.name == TABLE_CONSTANTS.ROW_KEY.getValue) { 175 | // 当使用row_key作为过滤条件时使用RowFilter加快查询速度 176 | if (comparable != null) { 177 | new RowFilter(compareOp, comparable) 178 | } else { 179 | new RowFilter(compareOp, new BinaryComparator(getBinaryValue(right))) 180 | } 181 | } else { 182 | val separateName = HBaseSparkFormatUtils.splitColumnAndQualifierName(attribute.name) 183 | val kvFilter = if (comparable != null) { 184 | new SingleColumnValueFilter( 185 | Bytes.toBytes(separateName.familyName), 186 | Bytes.toBytes(separateName.qualifierName), 187 | compareOp, 188 | comparable) 189 | } else { 190 | new SingleColumnValueFilter( 191 | Bytes.toBytes(separateName.familyName), 192 | Bytes.toBytes(separateName.qualifierName), 193 | compareOp, 194 | new BinaryComparator(getBinaryValue(right))) 195 | } 196 | // 这里设置为false会导致过滤条件为cf2但返回结果不包含cf2时,hbase不进行过滤 197 | kvFilter.setFilterIfMissing(true) 198 | kvFilter 199 | } 200 | Some(new FilterList(hbaseFilter)) 201 | 202 | case _ => 203 | None 204 | } 205 | } 206 | 207 | private def getBinaryValue(literal: Literal): Array[Byte] = { 208 | literal.dataType match { 209 | case BooleanType => Bytes.toBytes(literal.value.asInstanceOf[Boolean]) 210 | case ByteType => Bytes.toBytes(literal.value.asInstanceOf[Byte]) 211 | case ShortType => Bytes.toBytes(literal.value.asInstanceOf[Short]) 212 | case IntegerType => Bytes.toBytes(literal.value.asInstanceOf[Int]) 213 | case LongType => Bytes.toBytes(literal.value.asInstanceOf[Long]) 214 | case FloatType => Bytes.toBytes(literal.value.asInstanceOf[Float]) 215 | case DoubleType => Bytes.toBytes(literal.value.asInstanceOf[Double]) 216 | case StringType => UTF8String.fromString(literal.value.toString).getBytes 217 | } 218 | } 219 | 220 | private def add2FilterList( 221 | filters: java.util.ArrayList[Filter], 222 | filtersToBeAdded: Option[FilterList], 223 | operator: FilterList.Operator): Unit = { 224 | import collection.JavaConverters._ 225 | if (filtersToBeAdded.isDefined) { 226 | val filterList = filtersToBeAdded.get 227 | val size = filterList.getFilters.size 228 | if (size == 1 || filterList.getOperator == operator) { 229 | filterList.getFilters.asScala.foreach(filters.add) 230 | } else { 231 | filters.add(filterList) 232 | } 233 | } 234 | } 235 | } 236 | -------------------------------------------------------------------------------- /spark-hbase/src/main/java/org/apache/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 | 19 | package org.apache.spark; 20 | 21 | import com.google.protobuf.ByteString; 22 | import com.google.protobuf.InvalidProtocolBufferException; 23 | import org.apache.commons.logging.Log; 24 | import org.apache.commons.logging.LogFactory; 25 | import org.apache.hadoop.hbase.Cell; 26 | import org.apache.hadoop.hbase.classification.InterfaceAudience; 27 | import org.apache.hadoop.hbase.exceptions.DeserializationException; 28 | import org.apache.hadoop.hbase.filter.FilterBase; 29 | import org.apache.hadoop.hbase.util.ByteStringer; 30 | import org.apache.hadoop.hbase.util.Bytes; 31 | import org.apache.spark.datasources.BytesEncoder; 32 | import org.apache.spark.datasources.JavaBytesEncoder; 33 | import org.apache.spark.protobuf.generated.SparkFilterProtos; 34 | import org.apache.spark.sql.datasource.hbase.Field; 35 | import org.apache.spark.sql.hbase.ByteArrayComparable; 36 | import org.apache.spark.sql.hbase.DynamicLogicExpression; 37 | import org.apache.spark.sql.hbase.DynamicLogicExpressionBuilder; 38 | import scala.collection.mutable.MutableList; 39 | 40 | import java.io.IOException; 41 | import java.util.HashMap; 42 | import java.util.List; 43 | import java.util.Map; 44 | 45 | */ 46 | /** 47 | * This filter will push down all qualifier logic given to us 48 | * by SparkSQL so that we have make the filters at the region server level 49 | * and avoid sending the data back to the client to be filtered. 50 | * 51 | * @param pbBytes A pb serialized instance 52 | * @return An instance of SparkSQLPushDownFilter 53 | * @throws org.apache.hadoop.hbase.exceptions.DeserializationException 54 | * @return The filter serialized using pb 55 | *//* 56 | 57 | @InterfaceAudience.Private 58 | public class SparkSQLPushDownFilter extends FilterBase { 59 | protected static final Log log = LogFactory.getLog(SparkSQLPushDownFilter.class); 60 | 61 | //The following values are populated with protobuffer 62 | DynamicLogicExpression dynamicLogicExpression; 63 | byte[][] valueFromQueryArray; 64 | HashMap> 65 | currentCellToColumnIndexMap; 66 | 67 | //The following values are transient 68 | HashMap columnToCurrentRowValueMap = null; 69 | 70 | static final byte[] rowKeyFamily = new byte[0]; 71 | static final byte[] rowKeyQualifier = Bytes.toBytes("key"); 72 | 73 | String encoderClassName; 74 | 75 | public SparkSQLPushDownFilter(DynamicLogicExpression dynamicLogicExpression, 76 | byte[][] valueFromQueryArray, 77 | HashMap> 79 | currentCellToColumnIndexMap, String encoderClassName) { 80 | this.dynamicLogicExpression = dynamicLogicExpression; 81 | this.valueFromQueryArray = valueFromQueryArray; 82 | this.currentCellToColumnIndexMap = currentCellToColumnIndexMap; 83 | this.encoderClassName = encoderClassName; 84 | } 85 | 86 | public SparkSQLPushDownFilter(DynamicLogicExpression dynamicLogicExpression, 87 | byte[][] valueFromQueryArray, 88 | MutableList fields, String encoderClassName) { 89 | this.dynamicLogicExpression = dynamicLogicExpression; 90 | this.valueFromQueryArray = valueFromQueryArray; 91 | this.encoderClassName = encoderClassName; 92 | 93 | //generate family qualifier to index mapping 94 | this.currentCellToColumnIndexMap = 95 | new HashMap<>(); 96 | 97 | for (int i = 0; i < fields.len(); i++) { 98 | Field field = fields.apply(i); 99 | 100 | byte[] cfBytes = field.cfBytes(); 101 | ByteArrayComparable familyByteComparable = 102 | new ByteArrayComparable(cfBytes, 0, cfBytes.length); 103 | 104 | HashMap qualifierIndexMap = 105 | currentCellToColumnIndexMap.computeIfAbsent(familyByteComparable, k -> new HashMap<>()); 106 | 107 | byte[] qBytes = field.colBytes(); 108 | ByteArrayComparable qualifierByteComparable = 109 | new ByteArrayComparable(qBytes, 0, qBytes.length); 110 | 111 | qualifierIndexMap.put(qualifierByteComparable, field.colName()); 112 | } 113 | } 114 | 115 | @Override 116 | public ReturnCode filterKeyValue(Cell c) throws IOException { 117 | 118 | //If the map RowValueMap is empty then we need to populate 119 | // the row key 120 | if (columnToCurrentRowValueMap == null) { 121 | columnToCurrentRowValueMap = new HashMap<>(); 122 | HashMap qualifierColumnMap = 123 | currentCellToColumnIndexMap.get( 124 | new ByteArrayComparable(rowKeyFamily, 0, rowKeyFamily.length)); 125 | 126 | if (qualifierColumnMap != null) { 127 | String rowKeyColumnName = 128 | qualifierColumnMap.get( 129 | new ByteArrayComparable(rowKeyQualifier, 0, 130 | rowKeyQualifier.length)); 131 | //Make sure that the rowKey is part of the where clause 132 | if (rowKeyColumnName != null) { 133 | columnToCurrentRowValueMap.put(rowKeyColumnName, 134 | new ByteArrayComparable(c.getRowArray(), 135 | c.getRowOffset(), c.getRowLength())); 136 | } 137 | } 138 | } 139 | 140 | //Always populate the column value into the RowValueMap 141 | ByteArrayComparable currentFamilyByteComparable = 142 | new ByteArrayComparable(c.getFamilyArray(), 143 | c.getFamilyOffset(), 144 | c.getFamilyLength()); 145 | 146 | HashMap qualifierColumnMap = 147 | currentCellToColumnIndexMap.get( 148 | currentFamilyByteComparable); 149 | 150 | if (qualifierColumnMap != null) { 151 | 152 | String columnName = 153 | qualifierColumnMap.get( 154 | new ByteArrayComparable(c.getQualifierArray(), 155 | c.getQualifierOffset(), 156 | c.getQualifierLength())); 157 | 158 | if (columnName != null) { 159 | columnToCurrentRowValueMap.put(columnName, 160 | new ByteArrayComparable(c.getValueArray(), 161 | c.getValueOffset(), c.getValueLength())); 162 | } 163 | } 164 | 165 | return ReturnCode.INCLUDE; 166 | } 167 | 168 | 169 | @Override 170 | public boolean filterRow() throws IOException { 171 | 172 | try { 173 | boolean result = 174 | dynamicLogicExpression.execute(columnToCurrentRowValueMap, 175 | valueFromQueryArray); 176 | columnToCurrentRowValueMap = null; 177 | return !result; 178 | } catch (Throwable e) { 179 | log.error("Error running dynamic logic on row", e); 180 | } 181 | return false; 182 | } 183 | 184 | 185 | */ 186 | /** 187 | * @param pbBytes A pb serialized instance 188 | * @return An instance of SparkSQLPushDownFilter 189 | * @throws org.apache.hadoop.hbase.exceptions.DeserializationException 190 | *//* 191 | 192 | @SuppressWarnings("unused") 193 | public static SparkSQLPushDownFilter parseFrom(final byte[] pbBytes) 194 | throws DeserializationException { 195 | 196 | SparkFilterProtos.SQLPredicatePushDownFilter proto; 197 | try { 198 | proto = SparkFilterProtos.SQLPredicatePushDownFilter.parseFrom(pbBytes); 199 | } catch (InvalidProtocolBufferException e) { 200 | throw new DeserializationException(e); 201 | } 202 | 203 | String encoder = proto.getEncoderClassName(); 204 | BytesEncoder enc = JavaBytesEncoder.create(encoder); 205 | 206 | //Load DynamicLogicExpression 207 | DynamicLogicExpression dynamicLogicExpression = 208 | DynamicLogicExpressionBuilder.build(proto.getDynamicLogicExpression(), enc); 209 | 210 | //Load valuesFromQuery 211 | final List valueFromQueryArrayList = proto.getValueFromQueryArrayList(); 212 | byte[][] valueFromQueryArray = new byte[valueFromQueryArrayList.size()][]; 213 | for (int i = 0; i < valueFromQueryArrayList.size(); i++) { 214 | valueFromQueryArray[i] = valueFromQueryArrayList.get(i).toByteArray(); 215 | } 216 | 217 | //Load mapping from HBase family/qualifier to Spark SQL columnName 218 | HashMap> 219 | currentCellToColumnIndexMap = new HashMap<>(); 220 | 221 | for (SparkFilterProtos.SQLPredicatePushDownCellToColumnMapping 222 | sqlPredicatePushDownCellToColumnMapping : 223 | proto.getCellToColumnMappingList()) { 224 | 225 | byte[] familyArray = 226 | sqlPredicatePushDownCellToColumnMapping.getColumnFamily().toByteArray(); 227 | ByteArrayComparable familyByteComparable = 228 | new ByteArrayComparable(familyArray, 0, familyArray.length); 229 | HashMap qualifierMap = 230 | currentCellToColumnIndexMap.computeIfAbsent(familyByteComparable, k -> new HashMap<>()); 231 | 232 | byte[] qualifierArray = 233 | sqlPredicatePushDownCellToColumnMapping.getQualifier().toByteArray(); 234 | 235 | ByteArrayComparable qualifierByteComparable = 236 | new ByteArrayComparable(qualifierArray, 0, qualifierArray.length); 237 | 238 | qualifierMap.put(qualifierByteComparable, 239 | sqlPredicatePushDownCellToColumnMapping.getColumnName()); 240 | } 241 | 242 | return new SparkSQLPushDownFilter(dynamicLogicExpression, 243 | valueFromQueryArray, currentCellToColumnIndexMap, encoder); 244 | } 245 | 246 | */ 247 | /** 248 | * @return The filter serialized using pb 249 | *//* 250 | 251 | public byte[] toByteArray() { 252 | 253 | SparkFilterProtos.SQLPredicatePushDownFilter.Builder builder = 254 | SparkFilterProtos.SQLPredicatePushDownFilter.newBuilder(); 255 | 256 | SparkFilterProtos.SQLPredicatePushDownCellToColumnMapping.Builder columnMappingBuilder = 257 | SparkFilterProtos.SQLPredicatePushDownCellToColumnMapping.newBuilder(); 258 | 259 | builder.setDynamicLogicExpression(dynamicLogicExpression.toExpressionString()); 260 | for (byte[] valueFromQuery : valueFromQueryArray) { 261 | builder.addValueFromQueryArray(ByteStringer.wrap(valueFromQuery)); 262 | } 263 | 264 | for (Map.Entry> 265 | familyEntry : currentCellToColumnIndexMap.entrySet()) { 266 | for (Map.Entry qualifierEntry : 267 | familyEntry.getValue().entrySet()) { 268 | columnMappingBuilder.setColumnFamily( 269 | ByteStringer.wrap(familyEntry.getKey().bytes())); 270 | columnMappingBuilder.setQualifier( 271 | ByteStringer.wrap(qualifierEntry.getKey().bytes())); 272 | columnMappingBuilder.setColumnName(qualifierEntry.getValue()); 273 | builder.addCellToColumnMapping(columnMappingBuilder.build()); 274 | } 275 | } 276 | builder.setEncoderClassName(encoderClassName); 277 | 278 | 279 | return builder.build().toByteArray(); 280 | } 281 | } 282 | */ 283 | -------------------------------------------------------------------------------- /spark-hbase/src/main/scala/org/apache/spark/sql/hbase/client/IsolatedClientLoader.scala: -------------------------------------------------------------------------------- 1 | package org.apache.spark.sql.hbase.client 2 | 3 | import java.io.File 4 | import java.lang.reflect.InvocationTargetException 5 | import java.net.{URL, URLClassLoader} 6 | import java.util 7 | import org.apache.commons.io.{FileUtils, IOUtils} 8 | import org.apache.hadoop.conf.Configuration 9 | import org.apache.hadoop.hbase.HConstants 10 | import org.apache.spark.{SparkConf, SparkContext} 11 | import org.apache.spark.deploy.SparkSubmitUtils 12 | import org.apache.spark.internal.Logging 13 | import org.apache.spark.sql.catalyst.util.quietly 14 | import org.apache.spark.sql.internal.{NonClosableMutableURLClassLoader, SQLConf} 15 | import org.apache.spark.util.{MutableURLClassLoader, Utils, VersionUtils} 16 | 17 | /** 18 | * Created by wpy on 2017/5/12. 19 | */ 20 | private[hbase] object IsolatedClientLoader extends Logging { 21 | /** 22 | * Creates isolated HBase client loaders by downloading the requested version from maven. 23 | */ 24 | def forVersion( 25 | version: String, 26 | hadoopVersion: String, 27 | sparkConf: SparkConf, 28 | hadoopConf: Configuration, 29 | config: Map[String, String] = Map.empty, 30 | ivyPath: Option[String] = None, 31 | sharedPrefixes: Seq[String] = Seq.empty, 32 | barrierPrefixes: Seq[String] = Seq.empty): IsolatedClientLoader = synchronized { 33 | val resolvedVersion = hbaseVersion(version) 34 | // We will first try to share Hadoop classes. If we cannot resolve the Hadoop artifact 35 | // with the given version, we will use Hadoop 2.6 and then will not share Hadoop classes. 36 | var sharesHadoopClasses = true 37 | val files = if (resolvedVersions.contains((resolvedVersion, hadoopVersion))) { 38 | resolvedVersions((resolvedVersion, hadoopVersion)) 39 | } else { 40 | val remoteRepos = sparkConf.get(SQLConf.ADDITIONAL_REMOTE_REPOSITORIES) 41 | log.info("downloading jars from maven") 42 | val (downloadedFiles, actualHadoopVersion) = 43 | try { 44 | (downloadVersion(resolvedVersion, hadoopVersion, ivyPath, remoteRepos), hadoopVersion) 45 | } catch { 46 | case e: RuntimeException if e.getMessage.contains("hadoop") => 47 | // If the error message contains hadoop, it is probably because the hadoop 48 | // version cannot be resolved. 49 | val fallbackVersion = if (VersionUtils.majorVersion(hadoopVersion)==3) { 50 | "3.3.4" 51 | } else { 52 | "2.7.4" 53 | } 54 | logWarning(s"Failed to resolve Hadoop artifacts for the version $hadoopVersion. We " + 55 | s"will change the hadoop version from $hadoopVersion to $fallbackVersion and try " + 56 | "again. It is recommended to set jars used by Hive metastore client through " + 57 | "spark.sql.hive.metastore.jars in the production environment.") 58 | (downloadVersion( 59 | resolvedVersion, fallbackVersion, ivyPath, remoteRepos), fallbackVersion) 60 | } 61 | resolvedVersions.put((resolvedVersion, actualHadoopVersion), downloadedFiles) 62 | resolvedVersions((resolvedVersion, actualHadoopVersion)) 63 | } 64 | 65 | new IsolatedClientLoader( 66 | hbaseVersion(version), 67 | sparkConf, 68 | execJars = files, 69 | hadoopConf = hadoopConf, 70 | config = config, 71 | sharesHadoopClasses = sharesHadoopClasses, 72 | sharedPrefixes = sharedPrefixes, 73 | barrierPrefixes = barrierPrefixes) 74 | } 75 | 76 | def hbaseVersion(version: String): HBaseVersion = version match { 77 | case "1.0" | "1.0.0" => hbase.v1_0 78 | case "1.1" | "1.1.0" => hbase.v1_1 79 | case "1.2" | "1.2.0" | "1.2.1" => hbase.v1_2 80 | case "2.0.0-SNAPSHOT" | "2.0" | "2.0.0" | "2.0.1" => hbase.v2_0 81 | case "3.0.0-SNAPSHOT" | "3.0.0-alpha-1-SNAPSHOT" | "3.0.0-alpha-2-SNAPSHOT" | "3.0" | "3.0.0" => hbase.v3_0 82 | } 83 | 84 | def supportsHadoopShadedClient(hadoopVersion: String): Boolean = { 85 | VersionUtils.majorMinorPatchVersion(hadoopVersion).exists { 86 | case (3, 2, v) if v >= 2 => true 87 | case (3, 3, v) if v >= 1 => true 88 | case _ => false 89 | } 90 | } 91 | 92 | private def downloadVersion( 93 | version: HBaseVersion, 94 | hadoopVersion: String, 95 | ivyPath: Option[String], 96 | remoteRepos: String): Seq[URL] = { 97 | val hadoopJarNames = if (supportsHadoopShadedClient(hadoopVersion)) { 98 | Seq(s"org.apache.hadoop:hadoop-client-api:$hadoopVersion", 99 | s"org.apache.hadoop:hadoop-client-runtime:$hadoopVersion") 100 | } else { 101 | Seq(s"org.apache.hadoop:hadoop-client:$hadoopVersion") 102 | } 103 | val hbaseArtifacts = version.extraDeps ++ 104 | Seq("hbase-client", "hbase-common", "hbase-server", 105 | "hbase-hadoop-compat", 106 | "hbase-metrics", "hbase-metrics-api") 107 | .map(a => s"org.apache.hbase:$a:${version.fullVersion}") ++ 108 | Seq("com.google.guava:guava:14.0.1") ++ hadoopJarNames 109 | 110 | val classpath = quietly { 111 | SparkSubmitUtils.resolveMavenCoordinates( 112 | hbaseArtifacts.mkString(","), 113 | SparkSubmitUtils.buildIvySettings( 114 | Some(remoteRepos), 115 | ivyPath), 116 | transitive = true, 117 | exclusions = version.exclusions) 118 | } 119 | val allFiles = classpath.map(new File(_)).toSet 120 | 121 | // TODO: Remove copy logic. 122 | val tempDir = Utils.createTempDir(namePrefix = s"hbase-$version") 123 | allFiles.foreach(f => FileUtils.copyFileToDirectory(f, tempDir)) 124 | logInfo(s"Downloaded metastore jars to ${tempDir.getCanonicalPath}") 125 | tempDir.listFiles().map(_.toURI.toURL) 126 | } 127 | 128 | // A map from a given pair of HBaseVersion and Hadoop version to jar files. 129 | // It is only used by forVersion. 130 | private val resolvedVersions = 131 | new scala.collection.mutable.HashMap[(HBaseVersion, String), Seq[URL]] 132 | } 133 | 134 | private[hbase] class IsolatedClientLoader( 135 | val version: HBaseVersion, 136 | val sparkConf: SparkConf, 137 | val hadoopConf: Configuration, 138 | val execJars: Seq[URL] = Seq.empty, 139 | val config: Map[String, String] = Map.empty, 140 | val isolationOn: Boolean = true, 141 | val sharesHadoopClasses: Boolean = true, 142 | val rootClassLoader: ClassLoader = ClassLoader.getSystemClassLoader.getParent.getParent, 143 | val baseClassLoader: ClassLoader = Thread.currentThread().getContextClassLoader, 144 | val sharedPrefixes: Seq[String] = Seq.empty, 145 | val barrierPrefixes: Seq[String] = Seq.empty) 146 | extends Logging { 147 | 148 | // Check to make sure that the root classloader does not know about HBase. 149 | // assert(Try(rootClassLoader.loadClass("org.apache.hadoop.HBase.conf.HBaseConf")).isFailure) 150 | 151 | /** All jars used by the HBase specific classloader. */ 152 | protected def allJars: Array[URL] = execJars.toArray 153 | 154 | protected def isSharedClass(name: String): Boolean = { 155 | val isHadoopClass = 156 | name.startsWith("org.apache.hadoop.") && !name.startsWith("org.apache.hadoop.hbase.") 157 | 158 | name.contains("slf4j") || 159 | name.contains("log4j") || 160 | name.startsWith("org.apache.spark.") || 161 | (sharesHadoopClasses && isHadoopClass) || 162 | name.startsWith("scala.") || 163 | (name.startsWith("com.google") && !name.startsWith("com.google.cloud")) || 164 | name.startsWith("java.lang.") || 165 | name.startsWith("java.net") || 166 | name.startsWith("org.yaml.snakeyaml") || 167 | sharedPrefixes.exists(name.startsWith) 168 | } 169 | 170 | /** True if `name` refers to a spark class that must see specific version of HBase. */ 171 | protected def isBarrierClass(name: String): Boolean = 172 | name.startsWith(classOf[HBaseClientImpl].getName) || 173 | barrierPrefixes.exists(name.startsWith) 174 | 175 | protected def classToPath(name: String): String = 176 | name.replaceAll("\\.", "/") + ".class" 177 | 178 | /** 179 | * The classloader that is used to load an isolated version of HBase. 180 | * This classloader is a special URLClassLoader that exposes the addURL method. 181 | * So, when we add jar, we can add this new jar directly through the addURL method 182 | * instead of stacking a new URLClassLoader on top of it. 183 | */ 184 | private[hbase] val classLoader: MutableURLClassLoader = { 185 | // val isolatedClassLoader = { 186 | // if (isolationOn) { 187 | // new URLClassLoader(allJars, rootClassLoader) { 188 | // override def loadClass(name: String, resolve: Boolean): Class[_] = { 189 | // val loaded = findLoadedClass(name) 190 | // if (loaded == null) doLoadClass(name, resolve) else loaded 191 | // } 192 | // 193 | // def doLoadClass(name: String, resolve: Boolean): Class[_] = { 194 | // val classFileName = name.replaceAll("\\.", "/") + ".class" 195 | // if (isBarrierClass(name)) { 196 | // // For barrier classes, we construct a new copy of the class. 197 | // val bytes = IOUtils.toByteArray(baseClassLoader.getResourceAsStream(classFileName)) 198 | // logDebug(s"custom defining: $name - ${util.Arrays.hashCode(bytes)}") 199 | // defineClass(name, bytes, 0, bytes.length) 200 | // } else if (!isSharedClass(name)) { 201 | // logDebug(s"hbase class: $name - ${getResource(classToPath(name))}") 202 | // super.loadClass(name, resolve) 203 | // } else { 204 | // // For shared classes, we delegate to baseClassLoader, but fall back in case the 205 | // // class is not found. 206 | // logDebug(s"shared class: $name") 207 | // try { 208 | // baseClassLoader.loadClass(name) 209 | // } catch { 210 | // case _: ClassNotFoundException => 211 | // super.loadClass(name, resolve) 212 | // } 213 | // } 214 | // } 215 | // } 216 | // } else { 217 | // baseClassLoader 218 | // } 219 | // } 220 | // Right now, we create a URLClassLoader that gives preference to isolatedClassLoader 221 | // over its own URLs when it loads classes and resources. 222 | // We may want to use ChildFirstURLClassLoader based on 223 | // the configuration of spark.executor.userClassPathFirst, which gives preference 224 | // to its own URLs over the parent class loader (see Executor's createClassLoader method). 225 | 226 | //这里使用spark默认的classLoader防止出现不同classLoader对同一个class重复load的错误 227 | val isolatedClassLoader = Utils.getContextOrSparkClassLoader 228 | new NonClosableMutableURLClassLoader(isolatedClassLoader) 229 | } 230 | 231 | private[hbase] def addJar(path: URL): Unit = synchronized { 232 | classLoader.addURL(path) 233 | } 234 | 235 | /** The isolated client interface to HBase. */ 236 | private[hbase] def createClient(): HBaseClient = { 237 | if (!isolationOn) { 238 | return new HBaseClientImpl(version, sparkConf, hadoopConf, config, baseClassLoader, this) 239 | } 240 | // Pre-reflective instantiation setup. 241 | logDebug("Initializing the logger to avoid disaster...") 242 | val origLoader = Thread.currentThread().getContextClassLoader 243 | Thread.currentThread.setContextClassLoader(classLoader) 244 | 245 | try { 246 | val clsLoader = classLoader 247 | .loadClass(config.getOrElse("spark.hbase.client.impl", classOf[HBaseClientImpl].getName)) 248 | val constructor = clsLoader.getConstructors.head 249 | constructor.newInstance(version, sparkConf, hadoopConf, config, classLoader, this) 250 | .asInstanceOf[HBaseClient] 251 | } catch { 252 | case e: InvocationTargetException => 253 | e.getCause match { 254 | case cnf: NoClassDefFoundError => 255 | throw new ClassNotFoundException( 256 | s"$cnf when creating HBase client using classpath: ${execJars.mkString(", ")}\n" + 257 | "Please make sure that jars for your version of HBase and hadoop are included in the " + 258 | s"paths passed to ${HConstants.VERSION_FILE_NAME}.", e) 259 | case _ => 260 | throw e 261 | } 262 | } finally { 263 | Thread.currentThread.setContextClassLoader(origLoader) 264 | } 265 | } 266 | 267 | /** 268 | * The place holder for shared HBase client for all the HBaseContext sessions (they share an 269 | * IsolatedClientLoader). 270 | */ 271 | private[hbase] var cachedConnection: Any = _ 272 | 273 | private[hbase] var cachedAdmin: Any = _ 274 | } 275 | -------------------------------------------------------------------------------- /spark-hbase/src/main/scala/org/apache/spark/sql/hbase/HBaseStrategies.scala: -------------------------------------------------------------------------------- 1 | package org.apache.spark.sql.hbase 2 | 3 | import org.apache.spark.rdd.RDD 4 | import org.apache.spark.sql._ 5 | import org.apache.spark.sql.catalyst.catalog.{CatalogTable, UnresolvedCatalogRelation} 6 | import org.apache.spark.sql.catalyst.expressions.{And, Attribute, AttributeMap, AttributeReference, AttributeSet, Expression, GenericInternalRow, NamedExpression} 7 | import org.apache.spark.sql.catalyst.planning.PhysicalOperation 8 | import org.apache.spark.sql.catalyst.plans.logical.{InsertIntoStatement, LogicalPlan} 9 | import org.apache.spark.sql.catalyst.rules.Rule 10 | import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow, expressions} 11 | import org.apache.spark.sql.execution._ 12 | import org.apache.spark.sql.execution.command.{CreateTableCommand, ExecutedCommandExec} 13 | import org.apache.spark.sql.execution.datasources.DataSourceStrategy.selectFilters 14 | import org.apache.spark.sql.execution.datasources.v2.PushedDownOperators 15 | import org.apache.spark.sql.execution.datasources.{CreateTable, LogicalRelation} 16 | import org.apache.spark.sql.hbase.catalog.HBaseTableRelation 17 | import org.apache.spark.sql.hbase.execution.{CreateHBaseTableAsSelectCommand, HBaseTableScanExec, InsertIntoHBaseTable} 18 | import org.apache.spark.sql.hbase.utils.StructFieldConverters._ 19 | import org.apache.spark.sql.sources.{Filter, PrunedFilteredScan} 20 | import org.apache.spark.sql.types.{DataType, StructType} 21 | import org.apache.spark.sql.util.CaseInsensitiveStringMap 22 | 23 | import java.util.Locale 24 | import scala.collection.mutable.ArrayBuffer 25 | 26 | /** 27 | * Created by wpy on 17-5-17. 28 | */ 29 | private[hbase] trait HBaseStrategies { 30 | self: SparkPlanner => 31 | 32 | val sparkSession: SparkSession 33 | 34 | object HBaseDataSource extends Strategy { 35 | def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { 36 | case PhysicalOperation(projects, filters, l@LogicalRelation(t: PrunedFilteredScan, _, _, _)) 37 | if l.catalogTable.isDefined && HBaseAnalysis.isHBaseTable(l.catalogTable.get) => 38 | pruneFilterProject( 39 | l, 40 | projects, 41 | filters, 42 | (a, f) => toCatalystRDD(l, a, t.buildScan(a.map(_.name).toArray, f))) :: Nil 43 | case CreateHBaseTableAsSelectCommand(tableDesc, query, mode) 44 | if HBaseAnalysis.isHBaseTable(tableDesc) => 45 | val cmd = CreateHBaseTableAsSelectCommand(tableDesc, query, mode) 46 | ExecutedCommandExec(cmd) :: Nil 47 | case _ => Nil 48 | } 49 | 50 | /** 51 | * Convert RDD of Row into RDD of InternalRow with objects in catalyst types 52 | */ 53 | private[this] def toCatalystRDD( 54 | relation: LogicalRelation, 55 | output: Seq[Attribute], 56 | rdd: RDD[Row]): RDD[InternalRow] = { 57 | if (relation.relation.needConversion) { 58 | rowToRowRdd(rdd, output.map(_.dataType)) 59 | } else { 60 | rdd.asInstanceOf[RDD[InternalRow]] 61 | } 62 | } 63 | 64 | /** 65 | * Convert the objects inside Row into the types Catalyst expected. 66 | */ 67 | private def rowToRowRdd(data: RDD[Row], outputTypes: Seq[DataType]): RDD[InternalRow] = { 68 | data.mapPartitions { iterator => 69 | val numColumns = outputTypes.length 70 | val mutableRow = new GenericInternalRow(numColumns) 71 | val converters = outputTypes.map(CatalystTypeConverters.createToCatalystConverter) 72 | iterator.map { r => 73 | var i = 0 74 | while (i < numColumns) { 75 | mutableRow(i) = converters(i)(r(i)) 76 | i += 1 77 | } 78 | 79 | mutableRow 80 | } 81 | } 82 | } 83 | 84 | // Based on Public API. 85 | private def pruneFilterProject( 86 | relation: LogicalRelation, 87 | projects: Seq[NamedExpression], 88 | filterPredicates: Seq[Expression], 89 | scanBuilder: (Seq[Attribute], Array[Filter]) => RDD[InternalRow]) = { 90 | pruneFilterProjectRaw( 91 | relation, 92 | projects, 93 | filterPredicates, 94 | (requestedColumns, _, pushedFilters) => { 95 | scanBuilder(requestedColumns, pushedFilters.toArray) 96 | }) 97 | } 98 | 99 | private def pruneFilterProjectRaw( 100 | relation: LogicalRelation, 101 | projects: Seq[NamedExpression], 102 | filterPredicates: Seq[Expression], 103 | scanBuilder: (Seq[Attribute], Seq[Expression], Seq[Filter]) => RDD[InternalRow]): SparkPlan = { 104 | 105 | val projectSet = AttributeSet(projects.flatMap(_.references)) 106 | val filterSet = AttributeSet(filterPredicates.flatMap(_.references)) 107 | 108 | val candidatePredicates = filterPredicates.map { 109 | _ transform { 110 | case a: AttributeReference => relation.attributeMap(a) // Match original case of attributes. 111 | } 112 | } 113 | 114 | val (unhandledPredicates, pushedFilters, handledFilters) = 115 | selectFilters(relation.relation, candidatePredicates) 116 | 117 | // A set of column attributes that are only referenced by pushed down filters. We can eliminate 118 | // them from requested columns. 119 | val handledSet = { 120 | val handledPredicates = filterPredicates.filterNot(unhandledPredicates.contains) 121 | val unhandledSet = AttributeSet(unhandledPredicates.flatMap(_.references)) 122 | AttributeSet(handledPredicates.flatMap(_.references)) -- 123 | (projectSet ++ unhandledSet).map(relation.attributeMap) 124 | } 125 | 126 | // Combines all Catalyst filter `Expression`s that are either not convertible to data source 127 | // `Filter`s or cannot be handled by `relation`. 128 | val filterCondition = unhandledPredicates.reduceLeftOption(expressions.And) 129 | 130 | // These metadata values make scan plans uniquely identifiable for equality checking. 131 | // TODO(SPARK-17701) using strings for equality checking is brittle 132 | val metadata: Map[String, String] = { 133 | val pairs = ArrayBuffer.empty[(String, String)] 134 | 135 | // Mark filters which are handled by the underlying DataSource with an Astrisk 136 | if (pushedFilters.nonEmpty) { 137 | val markedFilters = for (filter <- pushedFilters) yield { 138 | if (handledFilters.contains(filter)) s"*$filter" else s"$filter" 139 | } 140 | pairs += ("PushedFilters" -> markedFilters.mkString("[", ", ", "]")) 141 | } 142 | // TODO 需要支持不带meta信息的普通Attribute 143 | pairs += ("ReadSchema" -> 144 | StructType(projects.map(_.toAttribute.asInstanceOf[AttributeReference].fromAttribute)).catalogString) 145 | pairs.toMap 146 | } 147 | 148 | if (projects.map(_.toAttribute) == projects && 149 | projectSet.size == projects.size && 150 | filterSet.subsetOf(projectSet)) { 151 | // When it is possible to just use column pruning to get the right projection and 152 | // when the columns of this projection are enough to evaluate all filter conditions, 153 | // just do a scan followed by a filter, with no extra project. 154 | val requestedColumns = projects 155 | // Safe due to if above. 156 | .asInstanceOf[Seq[Attribute]] 157 | // Match original case of attributes. 158 | .map(relation.attributeMap) 159 | // Don't request columns that are only referenced by pushed filters. 160 | .filterNot(handledSet.contains) 161 | 162 | val scan = RowDataSourceScanExec( 163 | projects.map(_.toAttribute), 164 | StructType(projects.map(_.toAttribute.asInstanceOf[AttributeReference].fromAttribute)), 165 | pushedFilters.toSet, 166 | pushedFilters.toSet, 167 | PushedDownOperators(None, None, None, None, Seq.empty, Seq.empty), 168 | scanBuilder(requestedColumns, candidatePredicates, pushedFilters), 169 | relation.relation, 170 | relation.catalogTable.map(_.identifier)) 171 | filterCondition.map(FilterExec(_, scan)).getOrElse(scan) 172 | } else { 173 | // Don't request columns that are only referenced by pushed filters. 174 | val requestedColumns = 175 | (projectSet ++ filterSet -- handledSet).map(relation.attributeMap).toSeq 176 | 177 | val scan = RowDataSourceScanExec( 178 | requestedColumns, 179 | StructType(requestedColumns.fromAttributes), 180 | pushedFilters.toSet, 181 | pushedFilters.toSet, 182 | PushedDownOperators(None, None, None, None, Seq.empty, Seq.empty), 183 | scanBuilder(requestedColumns, candidatePredicates, pushedFilters), 184 | relation.relation, 185 | relation.catalogTable.map(_.identifier)) 186 | ProjectExec( 187 | projects, filterCondition.map(FilterExec(_, scan)).getOrElse(scan)) 188 | } 189 | } 190 | } 191 | 192 | object HBaseTableScans extends Strategy { 193 | override def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { 194 | case PhysicalOperation(projectList, filter, h@HBaseTableRelation(tableMeta, dataCols, _, _, _)) 195 | if HBaseAnalysis.isHBaseTable(tableMeta) => 196 | val plan = HBasePlan(tableMeta, dataCols, h.output, h.partitionCols) 197 | filterProject4HBase(plan, projectList, filter) :: Nil 198 | case PhysicalOperation(projectList, filter, plan: HBasePlan) 199 | if HBaseAnalysis.isHBaseTable(plan.tableMeta) => 200 | /* pruneFilterProject( 201 | projectList, 202 | filter, 203 | identity[Seq[Expression]], 204 | HBaseTableScanExec(_, relation, filter)(sparkSession.asInstanceOf[HBaseSession])) :: Nil*/ 205 | filterProject4HBase(plan, projectList, filter) :: Nil 206 | case _ => 207 | Nil 208 | } 209 | } 210 | 211 | protected def filterProject4HBase(plan: HBasePlan, projectList: Seq[NamedExpression], filterPredicates: Seq[Expression]): SparkPlan = { 212 | val attributeMap: AttributeMap[AttributeReference] = AttributeMap(plan.output.map(o => (o, o))) 213 | val projectSet = AttributeSet(projectList.flatMap(_.references)) 214 | 215 | val filterSet = AttributeSet(filterPredicates.flatMap(_.references)) 216 | val filters = if (filterPredicates.nonEmpty) { 217 | Seq( 218 | filterPredicates.map { 219 | _ transform { case a: AttributeReference => attributeMap(a) } 220 | }.reduceLeft(And) 221 | ) 222 | } else { 223 | filterPredicates 224 | } 225 | if (projectList.map(_.toAttribute) == projectList && projectSet.size == projectList.size && filterSet.subsetOf(projectSet)) { 226 | val requestedColumns = projectList.asInstanceOf[Seq[Attribute]].map(attributeMap) 227 | HBaseTableScanExec(requestedColumns, plan, filters)(sparkSession.asInstanceOf[HBaseSession]) 228 | } else { 229 | val requestedColumns = projectSet.toSeq 230 | // val requestedColumns = attributeMap.keySet.toSeq 231 | val scan = HBaseTableScanExec(requestedColumns, plan, filters)(sparkSession.asInstanceOf[HBaseSession]) 232 | ProjectExec(projectList, scan) 233 | } 234 | } 235 | } 236 | 237 | /** 238 | * Replaces generic operations with specific variants that are designed to work with Hive. 239 | * 240 | * Note that, this rule must be run after `PreprocessTableCreation` and 241 | * `PreprocessTableInsertion`. 242 | */ 243 | object HBaseAnalysis extends Rule[LogicalPlan] { 244 | override def apply(plan: LogicalPlan): LogicalPlan = plan transformUp { 245 | case InsertIntoStatement(plan: HBasePlan, _, _, query, overwrite, ifNotExists, _) 246 | if isHBaseTable(plan.tableMeta) => 247 | InsertIntoHBaseTable(plan.tableMeta, query, overwrite, ifNotExists) 248 | 249 | case CreateTable(tableDesc, mode, None) if isHBaseTable(tableDesc) => 250 | CreateTableCommand(tableDesc, ignoreIfExists = mode == SaveMode.Ignore) 251 | 252 | case CreateTable(tableDesc, mode, Some(query)) if isHBaseTable(tableDesc) => 253 | CreateHBaseTableAsSelectCommand(tableDesc, query, mode) 254 | } 255 | 256 | def isHBaseTable(table: CatalogTable): Boolean = { 257 | //TODO 特殊处理hbase表 258 | table.provider.isDefined && table.provider.get.toLowerCase(Locale.ROOT).contains("hbase") 259 | } 260 | } 261 | 262 | class ResolveHBaseTable(sparkSession: SparkSession) extends Rule[LogicalPlan] { 263 | 264 | def readHBaseTable(table: CatalogTable, extraOptions: CaseInsensitiveStringMap): LogicalPlan = { 265 | HBasePlan( 266 | table, 267 | table.dataSchema.asNullable.toAttributes, 268 | table.dataSchema.asNullable.toAttributes, 269 | table.partitionSchema.asNullable.toAttributes) 270 | } 271 | 272 | override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { 273 | case UnresolvedCatalogRelation(tableMeta, options, false) 274 | if HBaseAnalysis.isHBaseTable(tableMeta) => 275 | readHBaseTable(tableMeta, options) 276 | case i@InsertIntoStatement(UnresolvedCatalogRelation(tableMeta, options, false), 277 | _, _, _, _, _, _) if HBaseAnalysis.isHBaseTable(tableMeta) => 278 | i.copy(table = readHBaseTable(tableMeta, options)) 279 | } 280 | } -------------------------------------------------------------------------------- /spark-hbase/src/main/scala/org/apache/spark/sql/hbase/client/HBaseClientImpl.scala: -------------------------------------------------------------------------------- 1 | package org.apache.spark.sql.hbase.client 2 | 3 | import org.apache.hadoop.conf.Configuration 4 | import org.apache.hadoop.fs.{FileSystem, Path} 5 | import org.apache.hadoop.hbase.client._ 6 | import org.apache.hadoop.hbase.io.ImmutableBytesWritable 7 | import org.apache.hadoop.hbase.io.compress.Compression 8 | import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding 9 | import org.apache.hadoop.hbase.mapreduce.TableInputFormat 10 | import org.apache.hadoop.hbase.regionserver.BloomType 11 | import org.apache.hadoop.hbase.security.User 12 | import org.apache.hadoop.hbase.util.Bytes 13 | import org.apache.hadoop.hbase.{HBaseConfiguration, HConstants, NamespaceDescriptor, TableName} 14 | import org.apache.spark.SparkConf 15 | import org.apache.spark.internal.Logging 16 | import org.apache.spark.sql.catalyst.TableIdentifier 17 | import org.apache.spark.sql.catalyst.analysis.NoSuchDatabaseException 18 | import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec 19 | import org.apache.spark.sql.catalyst.catalog._ 20 | import org.apache.spark.sql.catalyst.expressions.Expression 21 | import org.apache.spark.sql.catalyst.parser.CatalystSqlParser 22 | import org.apache.spark.sql.hbase.SparkHBaseConstants.TABLE_CONSTANTS 23 | import org.apache.spark.sql.hbase.execution.{HBaseFileFormat, HBaseSqlParser, HBaseTableFormat} 24 | import org.apache.spark.sql.hbase.utils.HBaseSparkFormatUtils 25 | import org.apache.spark.sql.types.{StructField, StructType} 26 | import org.apache.spark.util.{CircularBuffer, SerializableConfiguration} 27 | import org.yaml.snakeyaml.Yaml 28 | 29 | import java.io.{FileInputStream, ObjectInputStream, PrintStream} 30 | import java.net.URI 31 | import java.{util => ju} 32 | import scala.annotation.meta.param 33 | import scala.collection.JavaConverters._ 34 | 35 | /** 36 | * Created by wpy on 2017/5/12. 37 | */ 38 | class HBaseClientImpl( 39 | @(transient@param) override val version: HBaseVersion, 40 | @(transient@param) sparkConf: SparkConf, 41 | @(transient@param) hadoopConf: Configuration, 42 | extraConfig: Map[String, String], 43 | @(transient@param) initClassLoader: ClassLoader, 44 | @(transient@param) val clientLoader: IsolatedClientLoader) 45 | extends HBaseClient 46 | with Logging { 47 | 48 | private val schemaPath = extraConfig("schema.file.url") 49 | 50 | // Circular buffer to hold what hbase prints to STDOUT and ERR. Only printed when failures occur. 51 | @transient private var outputBuffer = new CircularBuffer() 52 | 53 | private val serializedConf = new SerializableConfiguration(hadoopConf) 54 | 55 | @transient private var conf = HBaseConfiguration.create(serializedConf.value) 56 | 57 | @transient private var userName = User.getCurrent 58 | 59 | @transient var conn: Connection = ConnectionFactory.createConnection(conf, userName) 60 | 61 | lazy val adm: Admin = connection.getAdmin 62 | 63 | /* logInfo( 64 | s"Root directory location for HBase client " + 65 | s"(version ${version.fullVersion}) is ${Option(new Path(conf.get("hbase.rootdir")).toUri)}")*/ 66 | private val retryLimit = conf.getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 31) 67 | private val retryDelayMillis = conf.getLong(HConstants.HBASE_CLIENT_OPERATION_TIMEOUT, 12000) 68 | 69 | /** 70 | * Runs `f` with multiple retries in case the hbase metastore is temporarily unreachable. 71 | */ 72 | private def retryLocked[A](f: => A): A = synchronized { 73 | // HBase sometimes retries internally, so set a deadline to avoid compounding delays. 74 | val deadline = System.nanoTime + (retryLimit * retryDelayMillis * 1e6).toLong 75 | var numTries = 0 76 | var caughtException: Exception = null 77 | do { 78 | numTries += 1 79 | try { 80 | return f 81 | } catch { 82 | case e: Exception if causedByThrift(e) => 83 | caughtException = e 84 | logWarning( 85 | "HBaseClient got thrift exception, destroying client and retrying " + 86 | s"(${retryLimit - numTries} tries remaining)", e) 87 | clientLoader.cachedConnection = null 88 | Thread.sleep(retryDelayMillis) 89 | } 90 | } while (numTries <= retryLimit && System.nanoTime < deadline) 91 | if (System.nanoTime > deadline) { 92 | logWarning("Deadline exceeded") 93 | } 94 | throw caughtException 95 | } 96 | 97 | private def causedByThrift(e: Throwable): Boolean = { 98 | var target = e 99 | while (target != null) { 100 | val msg = target.getMessage 101 | if (msg != null && msg.matches("(?s).*(TApplication|TProtocol|TTransport)Exception.*")) { 102 | return true 103 | } 104 | target = target.getCause 105 | } 106 | false 107 | } 108 | 109 | /** Returns the configuration for the given key in the current session. */ 110 | override def getConf(key: String, defaultValue: String): String = conf.get(key, defaultValue) 111 | 112 | /** 113 | * 读取yml文件, 每次getTable时都会重新读取 114 | * 115 | * @return 116 | */ 117 | private def getSchemaProp = { 118 | // Map[ table, Map[ family, Map[qua_name, type ] ] 119 | val yaml = new Yaml() 120 | val inputStream = new FileInputStream(schemaPath) 121 | val y = yaml.load[ju.Map[String, ju.Map[String, ju.Map[String, String]]]](inputStream) 122 | inputStream.close() 123 | y 124 | } 125 | 126 | private def getColumnNames(schemaMap: ju.Map[String, ju.Map[String, ju.Map[String, String]]], tableName: String) = { 127 | "{+\n" + schemaMap.get(tableName).asScala.map { cf => 128 | cf._1 + "-> (" + cf._2.asScala.keys.mkString(", ") + 129 | ")" 130 | }.mkString(";\n") + "\n}" 131 | } 132 | 133 | /** 134 | * 获取各个qualifier对应的数据类型 135 | * 136 | * @param schemaMap 表结构 137 | * @param tableName 表名 138 | * @return 139 | */ 140 | private def getSchema(schemaMap: ju.Map[String, ju.Map[String, ju.Map[String, String]]], tableName: String) = StructType { 141 | val tableSchema = schemaMap.get(tableName) 142 | val rowKeyField = StructField(TABLE_CONSTANTS.ROW_KEY.getValue, CatalystSqlParser.parseDataType(tableSchema.remove("row").get("key"))) 143 | val columnQualifierFields = tableSchema.asScala.flatMap { case (familyName, qualifier) => 144 | qualifier.asScala.map { case (qualifierName, dataType) => 145 | //family和qualifier字段的名字用“:”组合 146 | StructField( 147 | HBaseSparkFormatUtils.combineColumnAndQualifierName(familyName, qualifierName), 148 | HBaseSqlParser.parseDataType(dataType)) 149 | } 150 | }.toList 151 | rowKeyField :: columnQualifierFields 152 | } 153 | 154 | def connection: Connection = { 155 | if (clientLoader != null) { 156 | if (clientLoader.cachedConnection != null) { 157 | clientLoader.cachedConnection.asInstanceOf[Connection] 158 | } else { 159 | val c = conn 160 | clientLoader.cachedConnection = c 161 | c 162 | } 163 | } else { 164 | conn 165 | } 166 | } 167 | 168 | def admin: Admin = { 169 | if (clientLoader != null) { 170 | if (clientLoader.cachedAdmin != null) { 171 | clientLoader.cachedAdmin.asInstanceOf[Admin] 172 | } else { 173 | val a = adm 174 | clientLoader.cachedAdmin = a 175 | a 176 | } 177 | } else adm 178 | } 179 | 180 | override def setOut(stream: PrintStream): Unit = { 181 | new PrintStream(outputBuffer, true, "UTF-8") 182 | } 183 | 184 | override def setInfo(stream: PrintStream): Unit = { 185 | new PrintStream(outputBuffer, true, "UTF-8") 186 | } 187 | 188 | override def setError(stream: PrintStream): Unit = { 189 | new PrintStream(outputBuffer, true, "UTF-8") 190 | } 191 | 192 | def getQualifiedTableName(tableName: String): String = { 193 | val name = tableName.split(":") 194 | if (name.length > 1) name(1) 195 | else tableName 196 | } 197 | 198 | /** Returns the names of all tables in the given database. */ 199 | override def listTables(dbName: String): Seq[String] = { 200 | admin.listTableNamesByNamespace(dbName).map(name => getQualifiedTableName(name.getNameAsString)) 201 | } 202 | 203 | /** Returns the names of tables in the given database that matches the given pattern. */ 204 | override def listTables(dbName: String, pattern: String): Seq[String] = { 205 | listTables(dbName).filter(_.matches(pattern)) 206 | } 207 | 208 | /** Sets the name of current database. */ 209 | override def setCurrentDatabase(databaseName: String): Unit = { 210 | throw new UnsupportedOperationException(s"setCurrentDatabase($databaseName)") 211 | } 212 | 213 | /** Returns the metadata for specified database, throwing an exception if it doesn't exist */ 214 | override def getDatabase(name: String): CatalogDatabase = { 215 | if (databaseExists(name)) 216 | CatalogDatabase(name, name + " description", new URI(conf.get("hbase.rootdit") + s"/data/$name"), Map.empty) 217 | else 218 | throw new NoSuchElementException(name) 219 | } 220 | 221 | /** Return whether a table/view with the specified name exists. */ 222 | override def databaseExists(dbName: String): Boolean = { 223 | admin.listNamespaceDescriptors.exists(_.getName == dbName) 224 | } 225 | 226 | /** List the names of all the databases that match the specified pattern. */ 227 | override def listDatabases(pattern: String): Seq[String] = { 228 | admin.listNamespaceDescriptors().map(_.getName).filter(_.matches(pattern)) 229 | } 230 | 231 | /** Return whether a table/view with the specified name exists. */ 232 | override def tableExists(dbName: String, tableName: String): Boolean = { 233 | admin.tableExists(TableName.valueOf(s"$dbName:$tableName")) 234 | } 235 | 236 | /** Returns the metadata for the specified table or None if it doesn't exist. */ 237 | override def getTableOption(dbName: String, tableName: String): Option[CatalogTable] = { 238 | val name = s"$dbName:$tableName" 239 | logDebug(s"Looking up $dbName:$tableName") 240 | Option(admin.getDescriptor(TableName.valueOf(name))).map { t => 241 | 242 | //TODO need reality schemas 243 | val schemaMap = getSchemaProp 244 | //获取并remove该表的rowKey生成器 245 | val rowKeyGeneratorName = schemaMap.get(name).remove("generator").get("name") 246 | val schema = getSchema(schemaMap, name) 247 | 248 | //TODO should add more properties in future 249 | val props = { 250 | val regions = admin.getRegions(TableName.valueOf(name)) 251 | val tableDesc = admin.getDescriptor(TableName.valueOf(name)) 252 | /* 253 | {cf1->{q1,q2,...,}} 254 | {cf2->{q1,q2,...,}} 255 | */ 256 | val cols = tableDesc.getColumnFamilies 257 | //{CF1: (Q1, Q2, Q3, ... ,Qn)}; {CF2: (Q1, Q2, Q3, ... ,Qn)}; ... ;{CF3: (Q1, Q2, Q3, ... ,Qn)} 258 | val qualifiers = getColumnNames(schemaMap, name) 259 | // .map(cf=>s"{${cf._1}:(${cf._2.mkString(",")})}").mkString(";") 260 | val encoding = cols.map(family => family.getNameAsString -> Bytes.toString(family.getDataBlockEncoding.getNameInBytes)).mkString(";") 261 | //"K1, K2, K3, ... ,Kn" 262 | val splitKeys = regions.iterator().asScala.map(region => Bytes.toString(region.getEndKey)).mkString(",") 263 | val bloom = cols.map(family => family.getNameAsString + ":" + family.getBloomFilterType.name()).mkString(";") 264 | val zip = cols.map(family => family.getNameAsString + ":" + family.getCompressionType.getName).mkString(";") 265 | val columns = cols.map(_.getNameAsString).mkString(",") 266 | val table = s"$tableName" 267 | val db = s"$dbName" 268 | Map("db" -> db, 269 | "table" -> table, 270 | "qualifiers" -> qualifiers, 271 | "columns" -> columns, 272 | "encoding" -> encoding, 273 | "split" -> splitKeys, 274 | "bloom" -> bloom, 275 | "zip" -> zip, 276 | "generator" -> rowKeyGeneratorName) 277 | } 278 | val path = new Path(conf.get("hbase.rootdir") + s"/data/$dbName/$tableName") 279 | //TODO should add properties like {{cf:{Q1,Q2,...,Qn}}}, {splitKey:{S1,S2,...,Sn}}, {{DataEncoding:{prefix,diff,...}}}, {BloomType:{BloomType}} 280 | CatalogTable( 281 | identifier = TableIdentifier(tableName, Option(dbName)), 282 | //TODO tableType && table schema 283 | tableType = CatalogTableType.EXTERNAL, 284 | storage = CatalogStorageFormat( 285 | locationUri = Option(path.toUri), 286 | // To avoid ClassNotFound exception, we try our best to not get the format class, but get 287 | // the class name directly. However, for non-native tables, there is no interface to get 288 | // the format class name, so we may still throw ClassNotFound in this case. 289 | inputFormat = Some(classOf[TableInputFormat].getTypeName), 290 | outputFormat = Some(classOf[(ImmutableBytesWritable, Result)].getTypeName), 291 | serde = None, 292 | compressed = false, 293 | properties = Map.empty), 294 | schema = schema, 295 | //TODO 可以用来关联HBase外部数据源 296 | provider = Some(classOf[HBaseTableFormat].getCanonicalName), 297 | properties = props, 298 | stats = Some(CatalogStatistics(FileSystem.get(conf).getContentSummary(path).getLength))) 299 | } 300 | } 301 | 302 | /** Creates a table with the given metadata. */ 303 | override def createTable(table: CatalogTable, ignoreIfExists: Boolean): Unit = { 304 | def propStringToMap(propStr: String)(f: String => Map[String, String]): Map[String, String] = { 305 | f(propStr) 306 | } 307 | 308 | def toMapFunc: String => Map[String, String] = { a => 309 | a.replaceAll("\t", "").replaceAll(" ", "").split(";").map { b => 310 | val r = b.split(":") 311 | Map(r(0) -> r(1)) 312 | }.reduce(_ ++ _) 313 | } 314 | 315 | val dbName = table.identifier.database.get 316 | // dbName对应hbase的namespace 317 | if (!databaseExists(dbName)) { 318 | admin.createNamespace(NamespaceDescriptor.create(dbName).build()) 319 | } 320 | val props = table.properties 321 | val tableDesc = TableDescriptorBuilder.newBuilder(TableName.valueOf(table.identifier.table)) 322 | //TODO may be used in "INSERT EXPRESSION" 323 | // "qualifiers" "{CF1:(Q1, Q2, Q3, ... ,Qn)}; {CF2:(Q1, Q2, Q3, ... ,Qn)}; ... ;{CFn:(Q1, Q2, Q3, ... ,Qn)}" 324 | // val cols = props("qualifiers").replaceAll("\t", "").replaceAll(" ", "") 325 | 326 | //"column" "CF1, CF2, CF3, ...,CFn" 327 | val cols = props("column").replaceAll("\t", "").replaceAll(" ", "") 328 | //"encoding" "CF1:DataBlockEncoding; CF2:DataBlockEncoding; ... ;CFn:DataBlockEncoding" 329 | val encoding = propStringToMap(props("encoding"))(toMapFunc) 330 | //"split" "K1, K2, K3, ... ,Kn" 331 | val splitKeys = props("split").replaceAll("\t", "").replaceAll(" ", "") 332 | //"bloom" "CF1: type; CF2: type; CF3: type; ...;CFn: type" 333 | val bloomType = propStringToMap(props("bloom"))(toMapFunc) 334 | //"zip" "{CF1, algorithm}; {CF2, algorithm}; {CF3, algorithm}; ... ;{CFn, algorithm}" 335 | val zip = propStringToMap(props("zip"))(toMapFunc) 336 | 337 | //start get properties & create new table 338 | 339 | //get column_family properties 340 | val pattern = "\\{([0-9a-zA-Z]*):\\((([0-9a-zA-Z]*([,])?)*)\\)}".r 341 | 342 | //unused 343 | /*val colQualifier = cols.split(";").map{ 344 | case pattern(cf, qualifiers, _*) => { 345 | val qualifier = qualifiers.split(",").map(Bytes.toBytes) 346 | qualifier.map(q=>Map(cf->q)).reduce(_++_) 347 | } 348 | }*/ 349 | 350 | cols.split(";").foreach { 351 | case pattern(cf, _, _*) => 352 | val bloom = bloomType(cf) 353 | val compression = zip(cf) 354 | val en = encoding(cf) 355 | val columnFamily = ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes(cf)) 356 | columnFamily.setBlockCacheEnabled(true) 357 | columnFamily.setDataBlockEncoding(DataBlockEncoding.valueOf(en)) 358 | columnFamily.setCompressionType(Compression.Algorithm.valueOf(compression)) 359 | columnFamily.setBloomFilterType(BloomType.valueOf(bloom)) 360 | tableDesc.setColumnFamily(columnFamily.build()) 361 | } 362 | val split = splitKeys.split(",").filter(_.nonEmpty).map(Bytes.toBytes) 363 | //create table 364 | admin.createTable(tableDesc.build(), split) 365 | } 366 | 367 | /** Drop the specified table. */ 368 | override def dropTable(dbName: String, tableName: String, ignoreIfNotExists: Boolean, purge: Boolean): Unit = { 369 | val name = TableName.valueOf(dbName + ":" + tableName) 370 | admin.disableTable(name) 371 | admin.deleteTable(name) 372 | } 373 | 374 | /** Updates the given table with new metadata, optionally renaming the table. */ 375 | override def alterTable(tableName: String, table: CatalogTable): Unit = { 376 | throw new UnsupportedOperationException("alterTable") 377 | } 378 | 379 | /** Creates a new database with the given name. */ 380 | override def createDatabase(database: CatalogDatabase, ignoreIfExists: Boolean): Unit = { 381 | if (!databaseExists(database.name)) { 382 | admin.createNamespace(NamespaceDescriptor.create(database.name).build()) 383 | } else if (!ignoreIfExists) { 384 | admin.deleteNamespace(database.name) 385 | createDatabase(database, ignoreIfExists) 386 | } 387 | } 388 | 389 | /** 390 | * Drop the specified database, if it exists. 391 | * 392 | * @param name database to drop 393 | * @param ignoreIfNotExists if true, do not throw error if the database does not exist 394 | * @param cascade whether to remove all associated objects such as tables and functions 395 | */ 396 | override def dropDatabase(name: String, ignoreIfNotExists: Boolean, cascade: Boolean): Unit = { 397 | admin.deleteNamespace(name) 398 | } 399 | 400 | /** 401 | * Alter a database whose name matches the one specified in `database`, assuming it exists. 402 | */ 403 | override def alterDatabase(database: CatalogDatabase): Unit = { 404 | admin.modifyNamespace(NamespaceDescriptor.create(database.name).build()) 405 | } 406 | 407 | /** 408 | * Create one or many partitions in the given table. 409 | */ 410 | override def createPartitions(db: String, table: String, parts: Seq[CatalogTablePartition], ignoreIfExists: Boolean): Unit = { 411 | throw new UnsupportedOperationException("createPartitions") 412 | } 413 | 414 | /** 415 | * Drop one or many partitions in the given table, assuming they exist. 416 | */ 417 | override def dropPartitions(db: String, table: String, specs: Seq[TablePartitionSpec], ignoreIfNotExists: Boolean, purge: Boolean, retainData: Boolean): Unit = { 418 | throw new UnsupportedOperationException("dropPartitions") 419 | } 420 | 421 | /** 422 | * Rename one or many existing table partitions, assuming they exist. 423 | */ 424 | override def renamePartitions(db: String, table: String, specs: Seq[TablePartitionSpec], newSpecs: Seq[TablePartitionSpec]): Unit = { 425 | throw new UnsupportedOperationException("renamePartitions") 426 | } 427 | 428 | /** 429 | * Alter one or more table partitions whose specs match the ones specified in `newParts`, 430 | * assuming the partitions exist. 431 | */ 432 | override def alterPartitions(db: String, table: String, newParts: Seq[CatalogTablePartition]): Unit = { 433 | throw new UnsupportedOperationException("alterPartitions") 434 | } 435 | 436 | /** 437 | * Returns the partition names for the given table that match the supplied partition spec. 438 | * If no partition spec is specified, all partitions are returned. 439 | * 440 | * The returned sequence is sorted as strings. 441 | */ 442 | override def getPartitionNames(table: CatalogTable, partialSpec: Option[TablePartitionSpec]): Seq[String] = { 443 | throw new UnsupportedOperationException("getPartitionNames") 444 | } 445 | 446 | /** Returns the specified partition or None if it does not exist. */ 447 | override def getPartitionOption(table: CatalogTable, spec: TablePartitionSpec): Option[CatalogTablePartition] = { 448 | throw new UnsupportedOperationException("getPartitionOption") 449 | } 450 | 451 | /** 452 | * Returns the partitions for the given table that match the supplied partition spec. 453 | * If no partition spec is specified, all partitions are returned. 454 | */ 455 | override def getPartitions(catalogTable: CatalogTable, partialSpec: Option[TablePartitionSpec]): Seq[CatalogTablePartition] = { 456 | throw new UnsupportedOperationException("getPartition") 457 | } 458 | 459 | /** Returns partitions filtered by predicates for the given table. */ 460 | override def getPartitionsByFilter(catalogTable: CatalogTable, predicates: Seq[Expression]): Seq[CatalogTablePartition] = { 461 | throw new UnsupportedOperationException("getPartitionsByFilter") 462 | } 463 | 464 | /** Loads a static partition into an existing table. */ 465 | override def loadPartition(loadPath: String, dbName: String, tableName: String, partSpec: ju.LinkedHashMap[String, String], replace: Boolean, inheritTableSpecs: Boolean, isSrcLocal: Boolean): Unit = { 466 | throw new UnsupportedOperationException("loadPartition") 467 | } 468 | 469 | /** Loads data into an existing table. */ 470 | override def loadTable(loadPath: String, tableName: String, replace: Boolean, isSrcLocal: Boolean): Unit = { 471 | //TODO may supported in the future 472 | throw new UnsupportedOperationException("loadTable") 473 | } 474 | 475 | /** Loads new dynamic partitions into an existing table. */ 476 | override def loadDynamicPartitions(loadPath: String, dbName: String, tableName: String, partSpec: ju.LinkedHashMap[String, String], replace: Boolean, numDP: Int): Unit = { 477 | throw new UnsupportedOperationException("loadDynamicPartitions") 478 | } 479 | 480 | /** Add a jar into class loader */ 481 | override def addJar(path: String): Unit = { 482 | throw new UnsupportedOperationException("addJar") 483 | } 484 | 485 | /** Return a [[HBaseClient]] as new session, that will share the class loader and HBase client */ 486 | override def newSession(): HBaseClient = { 487 | clientLoader.createClient() 488 | } 489 | 490 | /** Used for testing only. Removes all data from this instance of HBase. */ 491 | override def reset(): Unit = { 492 | admin.listTableNames().foreach(tableName => admin.truncateTable(tableName, true)) 493 | } 494 | 495 | def readObject(input: ObjectInputStream): Unit = { 496 | outputBuffer = new CircularBuffer() 497 | userName = User.getCurrent 498 | conf = serializedConf.value 499 | conn = ConnectionFactory.createConnection(conf, userName) 500 | } 501 | } 502 | 503 | /** 504 | * Converts the native table metadata representation format CatalogTable to HBase's Table. 505 | */ 506 | object HBaseClientImpl { 507 | } --------------------------------------------------------------------------------