└── 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 | 
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 | 
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 | }
--------------------------------------------------------------------------------