├── .gitignore
├── .github
└── PULL_REQUEST_TEMPLATE
├── .travis.yml
├── core
├── src
│ ├── test
│ │ ├── resources
│ │ │ └── log4j.properties
│ │ └── scala
│ │ │ └── org
│ │ │ └── apache
│ │ │ └── spark
│ │ │ └── sql
│ │ │ ├── TestUtils.scala
│ │ │ ├── CatalogSuite.scala
│ │ │ ├── HBaseTestSuite.scala
│ │ │ ├── Test.scala
│ │ │ ├── DataTypeConverter.scala
│ │ │ ├── MaxVersionsSuite.scala
│ │ │ ├── PhoenixSuite.scala
│ │ │ ├── DataTypeSuite.scala
│ │ │ ├── SHC.scala
│ │ │ ├── AvroSourceKeySuite.scala
│ │ │ ├── CompositeKeySuite.scala
│ │ │ ├── PhoenixCompositeKeySuite.scala
│ │ │ ├── AvroSourceSuite.scala
│ │ │ ├── AvroKeySourceSuite.java
│ │ │ ├── DynamicColumnSuite.scala
│ │ │ └── AvroRecordSuite.scala
│ └── main
│ │ └── scala
│ │ └── org
│ │ └── apache
│ │ └── spark
│ │ └── sql
│ │ └── execution
│ │ └── datasources
│ │ └── hbase
│ │ ├── package.scala
│ │ ├── SparkHBaseConf.scala
│ │ ├── types
│ │ ├── SHCDataType.scala
│ │ ├── PrimitiveType.scala
│ │ └── Phoenix.scala
│ │ ├── Logging.scala
│ │ ├── HBaseResources.scala
│ │ ├── SHCCredentialsManager.scala
│ │ └── HBaseConnectionCache.scala
└── pom.xml
├── examples
├── pom.xml
└── src
│ └── main
│ └── scala
│ └── org
│ └── apache
│ └── spark
│ └── sql
│ └── execution
│ └── datasources
│ └── hbase
│ ├── Catalog.scala
│ ├── AvroRecord.scala
│ ├── DataCoder.scala
│ ├── LRJobForDataSources.scala
│ ├── HBaseSource.scala
│ ├── DataType.scala
│ ├── CompositeKey.scala
│ ├── JoinTablesFrom2Clusters.scala
│ ├── LRJobAccessing2Clusters.scala
│ └── AvroSource.scala
├── pom.xml
└── LICENSE
/.gitignore:
--------------------------------------------------------------------------------
1 | #IDE
2 | .idea/
3 | *.iml
4 |
5 | # Maven
6 | target/
7 |
8 |
--------------------------------------------------------------------------------
/.github/PULL_REQUEST_TEMPLATE:
--------------------------------------------------------------------------------
1 | ## What changes were proposed in this pull request?
2 |
3 | (Please fill in changes proposed in this fix)
4 |
5 | ## How was this patch tested?
6 |
7 | (Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests)
8 |
--------------------------------------------------------------------------------
/.travis.yml:
--------------------------------------------------------------------------------
1 | # Licensed to the Apache Software Foundation (ASF) under one or more
2 | # contributor license agreements. See the NOTICE file distributed with
3 | # this work for additional information regarding copyright ownership.
4 | # The ASF licenses this file to You under the Apache License, Version 2.0
5 | # (the "License"); you may not use this file except in compliance with
6 | # the License. You may obtain a copy of the License at
7 | #
8 | # http://www.apache.org/licenses/LICENSE-2.0
9 | #
10 | # Unless required by applicable law or agreed to in writing, software
11 | # distributed under the License is distributed on an "AS IS" BASIS,
12 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | # See the License for the specific language governing permissions and
14 | # limitations under the License.
15 |
16 | sudo: required
17 | dist: trusty
18 |
19 | language: java
20 | jdk:
21 | - oraclejdk8
22 |
23 | cache:
24 | directories:
25 | - $HOME/.sbt
26 | - $HOME/.m2
27 |
28 | notifications:
29 | email: false
30 |
31 | install:
32 | - mvn -q package install
33 |
--------------------------------------------------------------------------------
/core/src/test/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 |
18 | # Set everything to be logged to the file target/unit-tests.log
19 | log4j.rootCategory=DEBUG, file
20 | log4j.appender.file=org.apache.log4j.FileAppender
21 | log4j.appender.file.append=true
22 | log4j.appender.file.file=target/unit-tests.log
23 | log4j.appender.file.layout=org.apache.log4j.PatternLayout
24 | log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n
25 |
26 | # Ignore messages below warning level from Jetty, because it's a bit verbose
27 | log4j.logger.org.spark-project.jetty=WARN
28 |
--------------------------------------------------------------------------------
/core/src/main/scala/org/apache/spark/sql/execution/datasources/hbase/package.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * (C) 2017 Hortonworks, Inc. All rights reserved. See the NOTICE file
3 | * distributed with this work for additional information regarding copyright
4 | * ownership. This file is licensed to You under the Apache License, Version 2.0
5 | * (the "License"); you may not use this file except in compliance with
6 | * the License. You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 |
17 | package org.apache.spark.sql.execution.datasources
18 |
19 | import scala.math.Ordering
20 |
21 | import org.apache.hadoop.hbase.util.Bytes
22 |
23 | package object hbase {
24 | type HBaseType = Array[Byte]
25 | //Do not use BinaryType.ordering
26 | implicit val order: Ordering[HBaseType] = ord
27 |
28 |
29 | val ord: Ordering[HBaseType] = new Ordering[HBaseType] {
30 |
31 | def compare(x: Array[Byte], y: Array[Byte]): Int = {
32 | return Bytes.compareTo(x, y)
33 | }
34 | }
35 |
36 | val ByteMax = -1.asInstanceOf[Byte]
37 | val ByteMin = 0.asInstanceOf[Byte]
38 | val MaxLength = 256
39 | }
40 |
--------------------------------------------------------------------------------
/core/src/test/scala/org/apache/spark/sql/TestUtils.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * (C) 2017 Hortonworks, Inc. All rights reserved. See the NOTICE file
3 | * distributed with this work for additional information regarding copyright
4 | * ownership. This file is licensed to You under the Apache License, Version 2.0
5 | * (the "License"); you may not use this file except in compliance with
6 | * the License. You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 |
17 | package org.apache.spark.sql
18 |
19 | import java.nio.ByteBuffer
20 | import java.util.{ArrayList, HashMap}
21 |
22 | import scala.util.Random
23 |
24 | object TestUtils {
25 |
26 | def generateRandomByteBuffer(rand: Random, size: Int): ByteBuffer = {
27 | val bb = ByteBuffer.allocate(size)
28 | val arrayOfBytes = new Array[Byte](size)
29 | rand.nextBytes(arrayOfBytes)
30 | bb.put(arrayOfBytes)
31 | }
32 |
33 | def generateRandomMap(rand: Random, size: Int): java.util.Map[String, Int] = {
34 | val jMap = new HashMap[String, Int]()
35 | for (i <- 0 until size) {
36 | jMap.put(rand.nextString(5), i)
37 | }
38 | jMap
39 | }
40 |
41 | def generateRandomArray(rand: Random, size: Int): ArrayList[Boolean] = {
42 | val vec = new ArrayList[Boolean]()
43 | for (i <- 0 until size) {
44 | vec.add(rand.nextBoolean())
45 | }
46 | vec
47 | }
48 | }
49 |
--------------------------------------------------------------------------------
/examples/pom.xml:
--------------------------------------------------------------------------------
1 |
2 |
18 |
19 |
20 |
21 |
23 | 4.0.0
24 |
25 | com.hortonworks
26 | shc
27 | 1.1.3-2.4-s_2.11
28 | ../pom.xml
29 |
30 |
31 | shc-examples
32 | 1.1.3-2.4-s_2.11
33 | jar
34 | HBase Spark Connector Project Examples
35 |
36 |
37 | true
38 | true
39 | true
40 |
41 |
42 |
43 |
44 | com.hortonworks
45 | shc-core
46 | ${project.version}
47 |
48 |
49 |
50 |
51 |
--------------------------------------------------------------------------------
/core/src/main/scala/org/apache/spark/sql/execution/datasources/hbase/SparkHBaseConf.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 | * File modified by Hortonworks, Inc. Modifications are also licensed under
18 | * the Apache Software License, Version 2.0.
19 | */
20 |
21 | package org.apache.spark.sql.execution.datasources.hbase
22 |
23 | import org.apache.hadoop.conf.Configuration
24 | import org.apache.spark.sql.execution.datasources.hbase.types._
25 |
26 |
27 | object SparkHBaseConf {
28 | val testConf = "spark.hbase.connector.test"
29 | val credentialsManagerEnabled = "spark.hbase.connector.security.credentials.enabled"
30 | val expireTimeFraction = "spark.hbase.connector.security.credentials.expireTimeFraction"
31 | val refreshTimeFraction = "spark.hbase.connector.security.credentials.refreshTimeFraction"
32 | val refreshDurationMins = "spark.hbase.connector.security.credentials.refreshDurationMins"
33 | val principal = "spark.hbase.connector.security.credentials"
34 | val keytab = "spark.hbase.connector.security.keytab"
35 |
36 | var conf: Configuration = _
37 | var BulkGetSize = "spark.hbase.connector.bulkGetSize"
38 | var defaultBulkGetSize = 100
39 | var CachingSize = "spark.hbase.connector.cacheSize"
40 | var defaultCachingSize = 100
41 | // in milliseconds
42 | val connectionCloseDelay = 10 * 60 * 1000
43 |
44 | // for SHC DataType
45 | val Avro = classOf[Avro].getSimpleName
46 | val Phoenix = classOf[Phoenix].getSimpleName
47 | val PrimitiveType = classOf[PrimitiveType].getSimpleName
48 | }
49 |
--------------------------------------------------------------------------------
/examples/src/main/scala/org/apache/spark/sql/execution/datasources/hbase/Catalog.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * (C) 2017 Hortonworks, Inc. All rights reserved. See the NOTICE file
3 | * distributed with this work for additional information regarding copyright
4 | * ownership. This file is licensed to You under the Apache License, Version 2.0
5 | * (the "License"); you may not use this file except in compliance with
6 | * the License. You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 |
17 | package org.apache.spark.sql.execution.datasources.hbase.examples
18 |
19 | import org.apache.spark.sql.execution.datasources.hbase._
20 |
21 | object Catalog {
22 | def main(args: Array[String]){
23 | def catalog = s"""{
24 | |"table":{"namespace":"default", "name":"shcExampleTable", "tableCoder":"PrimitiveType"},
25 | |"rowkey":"key1:key2",
26 | |"columns":{
27 | |"col00":{"cf":"rowkey", "col":"key1", "type":"string", "length":"6"},
28 | |"col01":{"cf":"rowkey", "col":"key2", "type":"int"},
29 | |"col1":{"cf":"cf1", "col":"col1", "type":"boolean"},
30 | |"col2":{"cf":"cf2", "col":"col2", "type":"double"},
31 | |"col3":{"cf":"cf3", "col":"col3", "type":"float"},
32 | |"col4":{"cf":"cf4", "col":"col4", "type":"int"},
33 | |"col5":{"cf":"cf5", "col":"col5", "type":"bigint"},
34 | |"col6":{"cf":"cf6", "col":"col6", "type":"smallint"},
35 | |"col7":{"cf":"cf7", "col":"col7", "type":"string"},
36 | |"col8":{"cf":"cf8", "col":"col8", "type":"tinyint"}
37 | |}
38 | |}""".stripMargin
39 |
40 |
41 | val m = HBaseTableCatalog(Map(HBaseTableCatalog.tableCatalog->catalog))
42 | if(m.row.varLength != false){
43 | throw new Exception("value invalid")
44 | }
45 | if(m.row.length != 10){
46 | throw new Exception("value invalid")
47 | }
48 | }
49 | }
50 |
--------------------------------------------------------------------------------
/examples/src/main/scala/org/apache/spark/sql/execution/datasources/hbase/AvroRecord.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * (C) 2017 Hortonworks, Inc. All rights reserved. See the NOTICE file
3 | * distributed with this work for additional information regarding copyright
4 | * ownership. This file is licensed to You under the Apache License, Version 2.0
5 | * (the "License"); you may not use this file except in compliance with
6 | * the License. You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 |
17 | package org.apache.spark.sql.execution.datasources.hbase.examples
18 |
19 | import org.apache.avro.Schema
20 | import org.apache.avro.generic.GenericData
21 | import org.apache.spark.sql.execution.datasources.hbase.types.{AvroSerde, SchemaConverters}
22 |
23 | object AvroRecord {
24 | def main(args: Array[String]) {
25 | //Test avro to schema converterBasic setup
26 | val schemaString =
27 | """{"namespace": "example.avro",
28 | | "type": "record", "name": "User",
29 | | "fields": [ {"name": "name", "type": "string"},
30 | | {"name": "favorite_number", "type": ["int", "null"]},
31 | | {"name": "favorite_color", "type": ["string", "null"]} ] }""".stripMargin
32 |
33 | val avroSchema: Schema = {
34 | val p = new Schema.Parser
35 | p.parse(schemaString)
36 | }
37 | val user1 = new GenericData.Record(avroSchema)
38 | user1.put("name", "Alyssa")
39 | user1.put("favorite_number", 256)
40 |
41 | val user2 = new GenericData.Record(avroSchema)
42 | user2.put("name", "Ben")
43 | user2.put("favorite_number", 7)
44 | user2.put("favorite_color", "red")
45 |
46 | val sqlUser1 = SchemaConverters.createConverterToSQL(avroSchema)(user1)
47 | println(sqlUser1)
48 | val schema = SchemaConverters.toSqlType(avroSchema)
49 | println(s"\nSqlschema: $schema")
50 | val avroUser1 = SchemaConverters.createConverterToAvro(schema.dataType, "avro", "example.avro")(sqlUser1)
51 | val avroByte = AvroSerde.serialize(avroUser1, avroSchema)
52 | val avroUser11 = AvroSerde.deserialize(avroByte, avroSchema)
53 | println(s"$avroUser1")
54 | }
55 | }
56 |
--------------------------------------------------------------------------------
/core/src/test/scala/org/apache/spark/sql/CatalogSuite.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 | * File modified by Hortonworks, Inc. Modifications are also licensed under
18 | * the Apache Software License, Version 2.0.
19 | */
20 |
21 | package org.apache.spark.sql
22 |
23 | import org.apache.spark.sql.execution.datasources.hbase.Logging
24 | import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, FunSuite}
25 | import org.apache.spark.sql.execution.datasources.hbase.HBaseTableCatalog
26 |
27 | class CatalogSuite extends FunSuite with BeforeAndAfterEach with BeforeAndAfterAll with Logging{
28 | def catalog = s"""{
29 | |"table":{"namespace":"default", "name":"table1", "tableCoder":"PrimitiveType"},
30 | |"rowkey":"key1:key2",
31 | |"columns":{
32 | |"col00":{"cf":"rowkey", "col":"key1", "type":"string", "length":"6"},
33 | |"col01":{"cf":"rowkey", "col":"key2", "type":"int"},
34 | |"col1":{"cf":"cf1", "col":"col1", "type":"boolean"},
35 | |"col2":{"cf":"cf2", "col":"col2", "type":"double"},
36 | |"col3":{"cf":"cf3", "col":"col3", "type":"float"},
37 | |"col4":{"cf":"cf4", "col":"col4", "type":"int"},
38 | |"col5":{"cf":"cf5", "col":"col5", "type":"bigint"},
39 | |"col6":{"cf":"cf6", "col":"col6", "type":"smallint"},
40 | |"col8":{"cf":"cf8", "col":"col8", "type":"tinyint"},
41 | |"col7":{"cf":"cf7", "col":"col7", "type":"string"}
42 | |}
43 | |}""".stripMargin
44 |
45 | test("Catalog meta data check") {
46 | val m = HBaseTableCatalog(Map(HBaseTableCatalog.tableCatalog->catalog))
47 | assert(m.row.fields.filter(_.length == -1).isEmpty)
48 | assert(m.row.length == 10)
49 | }
50 |
51 | test("Catalog should preserve the columns order") {
52 | val m = HBaseTableCatalog(Map(HBaseTableCatalog.tableCatalog->catalog))
53 | assert(m.toDataType.fields.map(_.name).sameElements(
54 | Array("col00", "col01", "col1", "col2", "col3", "col4", "col5", "col6", "col8", "col7")))
55 | }
56 | }
57 |
--------------------------------------------------------------------------------
/core/src/test/scala/org/apache/spark/sql/HBaseTestSuite.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 | * File modified by Hortonworks, Inc. Modifications are also licensed under
18 | * the Apache Software License, Version 2.0.
19 | */
20 |
21 | package org.apache.spark.sql
22 |
23 | import java.io.File
24 |
25 | import scala.collection.JavaConverters._
26 |
27 | import com.google.common.io.Files
28 | import org.apache.hadoop.hbase.client._
29 | import org.apache.hadoop.hbase.util.Bytes
30 | import org.apache.hadoop.hbase.{TableName, HBaseTestingUtility}
31 | import org.apache.spark.sql.execution.datasources.hbase.Logging
32 | import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, FunSuite}
33 |
34 | class HBaseTestSuite extends FunSuite with BeforeAndAfterEach with BeforeAndAfterAll with Logging {
35 | private[spark] var htu = HBaseTestingUtility.createLocalHTU()
36 | private[spark] var tableName: Array[Byte] = Bytes.toBytes("t1")
37 | private[spark] var columnFamily: Array[Byte] = Bytes.toBytes("cf0")
38 | private[spark] var columnFamilies: Array[Array[Byte]] =
39 | Array(Bytes.toBytes("cf0"), Bytes.toBytes("cf1"), Bytes.toBytes("cf2"), Bytes.toBytes("cf3"), Bytes.toBytes("cf4"))
40 | var table: Table = null
41 | // private[spark] var columnFamilyStr = Bytes.toString(columnFamily)
42 |
43 | override def beforeAll() {
44 | val tempDir: File = Files.createTempDir
45 | tempDir.deleteOnExit
46 | htu.cleanupTestDir
47 | htu.startMiniZKCluster
48 | htu.startMiniHBaseCluster(1, 4)
49 | logInfo(" - minicluster started")
50 | println(" - minicluster started")
51 | try {
52 | htu.deleteTable(TableName.valueOf(tableName))
53 |
54 | //htu.createTable(TableName.valueOf(tableName), columnFamily, 2, Bytes.toBytes("abc"), Bytes.toBytes("xyz"), 2)
55 | } catch {
56 | case _ : Throwable =>
57 | logInfo(" - no table " + Bytes.toString(tableName) + " found")
58 | }
59 | setupTable()
60 | }
61 |
62 |
63 |
64 | override def afterAll() {
65 | try {
66 | table.close()
67 | println("shutdown")
68 | htu.deleteTable(TableName.valueOf(tableName))
69 | logInfo("shuting down minicluster")
70 | htu.shutdownMiniHBaseCluster
71 | htu.shutdownMiniZKCluster
72 | logInfo(" - minicluster shut down")
73 | htu.cleanupTestDir
74 | } catch {
75 | case _ : Throwable => logError("teardown error")
76 | }
77 | }
78 |
79 | def setupTable() {
80 | val config = htu.getConfiguration
81 | htu.createMultiRegionTable(TableName.valueOf(tableName), columnFamilies)
82 | println("create htable t1")
83 | val connection = ConnectionFactory.createConnection(config)
84 | val r = connection.getRegionLocator(TableName.valueOf("t1"))
85 | table = connection.getTable(TableName.valueOf("t1"))
86 |
87 | val regionLocations = r.getAllRegionLocations.asScala.toSeq
88 | println(s"$regionLocations size: ${regionLocations.size}")
89 | (0 until 100).foreach { x =>
90 | var put = new Put(Bytes.toBytes(s"row$x"))
91 | (0 until 5).foreach { y =>
92 | put.addColumn(columnFamilies(y), Bytes.toBytes(s"c$y"), Bytes.toBytes(s"value $x $y"))
93 | }
94 | table.put(put)
95 | }
96 | }
97 | }
98 |
--------------------------------------------------------------------------------
/core/src/test/scala/org/apache/spark/sql/Test.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * (C) 2017 Hortonworks, Inc. All rights reserved. See the NOTICE file
3 | * distributed with this work for additional information regarding copyright
4 | * ownership. This file is licensed to You under the Apache License, Version 2.0
5 | * (the "License"); you may not use this file except in compliance with
6 | * the License. You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 |
17 | package org.apache.spark.sql
18 |
19 | import org.apache.hadoop.hbase.util.Bytes
20 | import org.apache.spark.sql.execution.datasources.hbase.HBaseTableCatalog
21 | import org.apache.spark.sql.types.BinaryType
22 |
23 | object Test {
24 | def main(args: Array[String]) {
25 | val a: Array[Byte] = Array.fill(10)(Byte.MinValue)
26 | val b = Bytes.toBytes ("row003")
27 | System.arraycopy(b, 0, a, 0, b.length)
28 | val c = Bytes.toBytes(Int.MinValue)
29 | System.arraycopy(c, 0, a, b.length, c.length)
30 | val len = a.indexOf(HBaseTableCatalog.delimiter, 0)
31 | val s1 = Bytes.toString(a, 0, 6)
32 | val s2 = Bytes.toString(a, 0, len)
33 |
34 | /*val l = Bytes.toBytes(Float.MinValue)
35 | val m = Bytes.toBytes(-20.0.asInstanceOf[Double])
36 | val n = Bytes.toBytes(0.0.asInstanceOf[Double])
37 | val o = Bytes.toBytes(20.0.asInstanceOf[Double])
38 | val p = Bytes.toBytes(Float.MaxValue)*/
39 | val l = Array.fill(8)(Byte.MaxValue)
40 | Bytes.putDouble(l, 0, Double.MinValue)
41 | val m = Array.fill(8)(Byte.MaxValue)
42 | Bytes.putDouble(m, 0, -20.0)
43 | val n = Array.fill(8)(Byte.MaxValue)
44 | Bytes.putDouble(n, 0, 0.0)
45 | val o = Array.fill(8)(Byte.MaxValue)
46 | Bytes.putDouble(o, 0, 20.0)
47 | val p = Array.fill(8)(Byte.MaxValue)
48 | Bytes.putDouble(p, 0, Double.MaxValue)
49 |
50 | val c1 = BinaryType.ordering.compare(l, m)
51 | val c2 = BinaryType.ordering.compare(m, n)
52 | val c3 = BinaryType.ordering.compare(n, o)
53 | val c4 = BinaryType.ordering.compare(o, p)
54 |
55 | val p1 = Array.fill(10)(0: Byte)
56 | Bytes.putBytes(p1, 0, Bytes.toBytes("row010"), 0, 6)
57 |
58 | val p2 = Array.fill(10)(-1: Byte)
59 | Bytes.putBytes(p2, 0, Bytes.toBytes("row010"), 0, 6)
60 |
61 | val p3 = Array.fill(10)(Byte.MaxValue)
62 | Bytes.putBytes(p3, 0, Bytes.toBytes("row010"), 0, 6)
63 | Bytes.putInt(p3, 6, 10)
64 |
65 | val p4 = Bytes.compareTo(p1, p3)
66 | val p5 = Bytes.compareTo(p2, p3)
67 |
68 | val z = Array.fill(4)(Byte.MinValue)
69 | Bytes.putInt(z, 0, -1)
70 | val z1 = Array.fill(4)(Byte.MinValue)
71 | Bytes.putInt(z1, 0, -2147483648)
72 |
73 | val z2 = Bytes.compareTo(z, z1)
74 |
75 | val t = Array.fill(4)(-1: Byte)
76 | println(Bytes.toInt(t))
77 |
78 | val s = Bytes.toBytes(1.4.asInstanceOf[Float])
79 | println(Bytes.toInt(s))
80 | println(Bytes.toFloat(s))
81 | val w = Bytes.toBytes(-1.4.asInstanceOf[Float])
82 | println(Bytes.toInt(w))
83 | println(Bytes.toFloat(w))
84 |
85 | val buffer1 = Bytes.toBytes(-1.0f)
86 | val b1 = Bytes.toInt(buffer1)
87 | var buffer = Array.fill(4)(-1: Byte)
88 | var buffer2 = Bytes.toBytes(-1.0f)
89 |
90 | var buffer3 = java.lang.Float.floatToIntBits(-1.0f)
91 | val b3 = Bytes.toBytes(buffer3)
92 | val out = Bytes.toInt(buffer1) ^ Integer.MIN_VALUE
93 | buffer2 = Bytes.toBytes(out)
94 | var i: Int = java.lang.Float.floatToIntBits(-1.0f)
95 | i = (i ^ ((i >> Integer.SIZE - 1) | Integer.MIN_VALUE)) + 1
96 | Bytes.putInt(buffer, 0, i)
97 |
98 | val mn = Bytes.toBytes(-0.0f)
99 | println(Bytes.toFloat(mn))
100 | println(Float.MinPositiveValue)
101 |
102 | println(s"a")
103 | }
104 | }
105 |
--------------------------------------------------------------------------------
/examples/src/main/scala/org/apache/spark/sql/execution/datasources/hbase/DataCoder.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * (C) 2017 Hortonworks, Inc. All rights reserved. See the NOTICE file
3 | * distributed with this work for additional information regarding copyright
4 | * ownership. This file is licensed to You under the Apache License, Version 2.0
5 | * (the "License"); you may not use this file except in compliance with
6 | * the License. You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 |
17 | package org.apache.spark.sql.execution.datasources.hbase.examples
18 |
19 | import org.apache.spark.sql.execution.datasources.hbase._
20 | import org.apache.spark.sql.{DataFrame, SparkSession}
21 |
22 | case class DCRecord(
23 | col00: String,
24 | col01: Int,
25 | col1: Boolean,
26 | col2: Double,
27 | col3: Float,
28 | col4: Int,
29 | col5: Long,
30 | col6: Short,
31 | col7: String,
32 | col8: Byte)
33 |
34 | object DCRecord {
35 | def apply(i: Int): DCRecord = {
36 | DCRecord(s"row${"%03d".format(i)}",
37 | if (i % 2 == 0) {
38 | i
39 | } else {
40 | -i
41 | },
42 | i % 2 == 0,
43 | i.toDouble,
44 | i.toFloat,
45 | i,
46 | i.toLong,
47 | i.toShort,
48 | s"String$i extra",
49 | i.toByte)
50 | }
51 | }
52 |
53 | object DataCoder {
54 | def cat = s"""{
55 | |"table":{"namespace":"default", "name":"shcExampleDCTable", "tableCoder":"Phoenix", "version":"2.0"},
56 | |"rowkey":"key1:key2",
57 | |"columns":{
58 | |"col00":{"cf":"rowkey", "col":"key1", "type":"string"},
59 | |"col01":{"cf":"rowkey", "col":"key2", "type":"int"},
60 | |"col1":{"cf":"CF1", "col":"COL1", "type":"boolean"},
61 | |"col2":{"cf":"CF1", "col":"COL2", "type":"double"},
62 | |"col3":{"cf":"CF2", "col":"COL3", "type":"float"},
63 | |"col4":{"cf":"CF2", "col":"COL4", "type":"int"},
64 | |"col5":{"cf":"CF3", "col":"COL5", "type":"bigint"},
65 | |"col6":{"cf":"CF3", "col":"COL6", "type":"smallint"},
66 | |"col7":{"cf":"CF4", "col":"COL7", "type":"string"},
67 | |"col8":{"cf":"CF4", "col":"COL8", "type":"tinyint"}
68 | |}
69 | |}""".stripMargin
70 |
71 | def main(args: Array[String]){
72 | val spark = SparkSession.builder()
73 | .appName("DataCoderExample")
74 | .getOrCreate()
75 |
76 | val sc = spark.sparkContext
77 | val sqlContext = spark.sqlContext
78 |
79 | import sqlContext.implicits._
80 |
81 | def withCatalog(cat: String): DataFrame = {
82 | sqlContext
83 | .read
84 | .options(Map(HBaseTableCatalog.tableCatalog->cat))
85 | .format("org.apache.spark.sql.execution.datasources.hbase")
86 | .load()
87 | }
88 |
89 | // populate table with composite key
90 | val data = (0 to 255).map { i =>
91 | DCRecord(i)
92 | }
93 |
94 | sc.parallelize(data).toDF.write
95 | .options(Map(HBaseTableCatalog.tableCatalog -> cat, HBaseTableCatalog.newTable -> "5"))
96 | .format("org.apache.spark.sql.execution.datasources.hbase")
97 | .save()
98 |
99 | val df = withCatalog(cat)
100 | df.show
101 | df.filter($"col0" <= "row005")
102 | .select($"col0", $"col1").show
103 | df.filter($"col0" === "row005" || $"col0" <= "row005")
104 | .select($"col0", $"col1").show
105 | df.filter($"col0" > "row250")
106 | .select($"col0", $"col1").show
107 | df.registerTempTable("table1")
108 | val c = sqlContext.sql("select count(col1) from table1 where col0 < 'row050'")
109 | c.show()
110 | }
111 | }
112 |
--------------------------------------------------------------------------------
/core/src/test/scala/org/apache/spark/sql/DataTypeConverter.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 | * File modified by Hortonworks, Inc. Modifications are also licensed under
18 | * the Apache Software License, Version 2.0.
19 | */
20 |
21 | package org.apache.spark.sql
22 |
23 | import org.apache.spark.SparkContext
24 | import org.apache.spark.sql.execution.datasources.hbase.Logging
25 | import org.apache.spark.sql.execution.datasources.hbase.HBaseTableCatalog
26 |
27 | class DataTypeConverter extends SHC with Logging{
28 | ignore("Basic setup") {
29 | val spark = SparkSession.builder()
30 | .master("local")
31 | .appName("HBaseTest")
32 | .config(conf)
33 | .getOrCreate()
34 | val sqlContext = spark.sqlContext
35 |
36 | val complex = s"""MAP>"""
37 | val schema =
38 | s"""{"namespace": "example.avro",
39 | | "type": "record", "name": "User",
40 | | "fields": [ {"name": "name", "type": "string"},
41 | | {"name": "favorite_number", "type": ["int", "null"]},
42 | | {"name": "favorite_color", "type": ["string", "null"]} ]}""".stripMargin
43 |
44 | val catalog = s"""{
45 | |"table":{"namespace":"default", "name":"htable", "tableCoder":"PrimitiveType"},
46 | |"rowkey":"key1:key2",
47 | |"columns":{
48 | |"col1":{"cf":"rowkey", "col":"key1", "type":"binary"},
49 | |"col2":{"cf":"rowkey", "col":"key2", "type":"double"},
50 | |"col3":{"cf":"cf1", "col":"col1", "avro":"schema1"},
51 | |"col4":{"cf":"cf1", "col":"col2", "type":"string"},
52 | |"col5":{"cf":"cf1", "col":"col3", "type":"double"},
53 | |"col6":{"cf":"cf1", "col":"col4", "type":"$complex"}
54 | |}
55 | |}""".stripMargin
56 | val df =
57 | sqlContext.read.options(
58 | Map("schema1"->schema, HBaseTableCatalog.tableCatalog->catalog))
59 | .format("org.apache.spark.sql.execution.datasources.hbase")
60 | .load()
61 | df.write.options(
62 | Map("schema1"->schema, HBaseTableCatalog.tableCatalog->catalog))
63 | .format("org.apache.spark.sql.execution.datasources.hbase")
64 | .save()
65 |
66 | //val s = df.filter((($"col1" < Array(10.toByte)) and ($"col1" > Array(1.toByte))) or ($"col1" === Array(11.toByte))).select("col1")
67 | //val s = df.filter(Column("col1").<(Array(10.toByte)).and(Column("col1").>(Array(1.toByte))).or(Column("col1") === Array(11.toByte))).select("col1")
68 | // val s = df.filter((($"col1" < Array(10.toByte)) && ($"col1" > Array(1.toByte))) || ($"col1" === Array(11.toByte))).select("col1")
69 | //val s = df.filter(($"col1" < Array(10.toByte) && $"col1" > Array(1.toByte)) || $"col1" === Array(11.toByte) || $"col2" === 2.3).select("col1") // range should be (None, None)
70 | val s = df.filter(($"col1" < Array(10.toByte) &&
71 | $"col1" > Array(1.toByte)) ||
72 | $"col1" === Array(11.toByte) &&
73 | $"col2" === 2.3)
74 | .select("col1")
75 | s.count()
76 | df.createOrReplaceTempView("table")
77 | val c = sqlContext.sql("select count(col1) from table")
78 | // c.queryExecution
79 | c.show
80 | val se = df.filter($"col2" > 12).filter($"col4" < Array(10.toByte)).select("col1")
81 |
82 | val se1 = df.filter($"col2" > 12).filter($"col4" < Array(10.toByte)).select("col1")
83 | se.count()
84 | se1.collect.foreach(println(_))
85 | println(df)
86 | }
87 | }
88 |
--------------------------------------------------------------------------------
/core/src/test/scala/org/apache/spark/sql/MaxVersionsSuite.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * (C) 2017 Hortonworks, Inc. All rights reserved. See the NOTICE file
3 | * distributed with this work for additional information regarding copyright
4 | * ownership. This file is licensed to You under the Apache License, Version 2.0
5 | * (the "License"); you may not use this file except in compliance with
6 | * the License. You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 |
17 | package org.apache.spark.sql
18 |
19 | import org.apache.spark.sql.execution.datasources.hbase.Logging
20 | import org.apache.spark.sql.execution.datasources.hbase.{HBaseRelation, HBaseTableCatalog}
21 |
22 | class MaxVersionsSuite extends SHC with Logging {
23 |
24 | def withCatalog(cat: String, options: Map[String,String]): DataFrame = {
25 | sqlContext.read
26 | .options(options ++ Map(HBaseTableCatalog.tableCatalog -> catalog))
27 | .format("org.apache.spark.sql.execution.datasources.hbase")
28 | .load()
29 | }
30 |
31 | def persistDataInHBase(cat: String, data: Seq[HBaseRecord], timestamp: Long): Unit = {
32 | val sql = sqlContext
33 | import sql.implicits._
34 | sc.parallelize(data).toDF.write
35 | .options(Map(
36 | HBaseTableCatalog.tableCatalog -> cat,
37 | HBaseTableCatalog.newTable -> "5",
38 | HBaseRelation.MAX_VERSIONS -> "3",
39 | HBaseRelation.TIMESTAMP -> timestamp.toString
40 | ))
41 | .format("org.apache.spark.sql.execution.datasources.hbase")
42 | .save()
43 | }
44 |
45 | test("Max Versions semantics") {
46 |
47 | val oldestMs = 754869600000L
48 | val oldMs = 754869611111L
49 | val newMs = 754869622222L
50 | val newestMs = 754869633333L
51 |
52 | val oldestData = (0 to 2).map(HBaseRecord(_, "ancient"))
53 | val oldData = (0 to 2).map(HBaseRecord(_, "old"))
54 | val newData = (0 to 2).map(HBaseRecord(_, "new"))
55 | val newestData = (0 to 1).map(HBaseRecord(_, "latest"))
56 |
57 | persistDataInHBase(catalog, oldestData, oldestMs)
58 | persistDataInHBase(catalog, oldData, oldMs)
59 | persistDataInHBase(catalog, newData, newMs)
60 | persistDataInHBase(catalog, newestData, newestMs)
61 |
62 | // Test specific last two versions
63 | val twoVersions: DataFrame = withCatalog(catalog, Map(
64 | HBaseRelation.MAX_VERSIONS -> "2",
65 | HBaseRelation.MERGE_TO_LATEST -> "false"
66 | ))
67 |
68 | //count is made on HBase directly and return number of unique rows
69 | assert(twoVersions.count() == 3)
70 |
71 | val rows = twoVersions.take(10)
72 | assert(rows.size == 6)
73 | assert(rows.count(_.getString(7).contains("ancient")) == 0)
74 | assert(rows.count(_.getString(7).contains("old")) == 1)
75 | assert(rows.count(_.getString(7).contains("new")) == 3)
76 | assert(rows.count(_.getString(7).contains("latest")) == 2)
77 |
78 | //we cannot take more then three because we create table with that size
79 | val threeVersions: DataFrame = withCatalog(catalog, Map(
80 | HBaseRelation.MAX_VERSIONS -> "4",
81 | HBaseRelation.MERGE_TO_LATEST -> "false"
82 | ))
83 |
84 | val threeRows = threeVersions.take(10)
85 | assert(threeRows.size == 9)
86 | assert(threeRows.count(_.getString(7).contains("ancient")) == 1)
87 | assert(threeRows.count(_.getString(7).contains("old")) == 3)
88 | assert(threeRows.count(_.getString(7).contains("new")) == 3)
89 | assert(threeRows.count(_.getString(7).contains("latest")) == 2)
90 |
91 | // Test specific only last versions
92 | val lastVersions: DataFrame = withCatalog(catalog, Map.empty)
93 |
94 | val lastRows = lastVersions.take(10)
95 | assert(lastRows.size == 3)
96 | assert(lastRows.count(_.getString(7).contains("new")) == 1)
97 | assert(lastRows.count(_.getString(7).contains("latest")) == 2)
98 | }
99 | }
100 |
--------------------------------------------------------------------------------
/core/src/main/scala/org/apache/spark/sql/execution/datasources/hbase/types/SHCDataType.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * (C) 2017 Hortonworks, Inc. All rights reserved. See the NOTICE file
3 | * distributed with this work for additional information regarding copyright
4 | * ownership. This file is licensed to You under the Apache License, Version 2.0
5 | * (the "License"); you may not use this file except in compliance with
6 | * the License. You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 |
17 | package org.apache.spark.sql.execution.datasources.hbase.types
18 |
19 | import org.apache.spark.sql.Row
20 | import org.apache.spark.sql.execution.datasources.hbase._
21 |
22 | trait SHCDataType extends Serializable {
23 | // Parse the hbase Field to it's corresponding Scala type which can then be put into
24 | // a Spark GenericRow which is then automatically converted by Spark.
25 | def fromBytes(src: HBaseType): Any
26 |
27 | // Convert input to Byte Array (HBaseType)
28 | def toBytes(input: Any): Array[Byte]
29 |
30 | // If lexicographic sort order is maintained, then return true.
31 | // If return false, the data type can not be the table coder.
32 | def isRowKeySupported(): Boolean = false
33 |
34 | def isCompositeKeySupported(): Boolean = false
35 |
36 | /**
37 | * Takes a HBase Row object and parses all of the fields from it.
38 | * This is independent of which fields were requested from the key
39 | * Because we have all the data it's less complex to parse everything.
40 | *
41 | * @param keyFields all of the fields in the row key, ORDERED by their order in the row key.
42 | */
43 | def decodeCompositeRowKey(row: Array[Byte], keyFields: Seq[Field]): Map[Field, Any] = {
44 | throw new UnsupportedOperationException("Composite key is not supported")
45 | }
46 |
47 | def encodeCompositeRowKey(rkIdxedFields:Seq[(Int, Field)], row: Row): Seq[Array[Byte]] = {
48 | throw new UnsupportedOperationException("Composite key is not supported")
49 | }
50 | }
51 |
52 | /**
53 | * Currently, SHC supports three data types which can be used as serdes: Avro, Phoenix, PrimitiveType.
54 | * Adding New SHC data type needs to implement the trait 'SHCDataType'.
55 | */
56 | object SHCDataTypeFactory {
57 |
58 | def create(f: Field): SHCDataType = {
59 | if (f == null) {
60 | throw new NullPointerException(
61 | "SHCDataTypeFactory: the 'f' parameter used to create SHCDataType " +
62 | "can not be null.")
63 | }
64 |
65 | if (f.fCoder == SparkHBaseConf.Avro) {
66 | new Avro(Some(f))
67 | } else if (f.fCoder == SparkHBaseConf.Phoenix) {
68 | new Phoenix(Some(f))
69 | } else if (f.fCoder == SparkHBaseConf.PrimitiveType) {
70 | new PrimitiveType(Some(f))
71 | } else {
72 | // Data type implemented by user
73 | Class.forName(f.fCoder)
74 | .getConstructor(classOf[Option[Field]])
75 | .newInstance(Some(f))
76 | .asInstanceOf[SHCDataType]
77 | }
78 | }
79 |
80 | // Currently, the function below is only used for creating the table coder.
81 | // One catalog/HBase table can only use one table coder, so the function is
82 | // only called once in 'HBaseTableCatalog' class.
83 | def create(coder: String): SHCDataType = {
84 | if (coder == null || coder.isEmpty) {
85 | throw new NullPointerException(
86 | "SHCDataTypeFactory: the 'coder' parameter used to create SHCDataType " +
87 | "can not be null or empty.")
88 | }
89 |
90 | if (coder == SparkHBaseConf.Avro) {
91 | new Avro()
92 | } else if (coder == SparkHBaseConf.Phoenix) {
93 | new Phoenix()
94 | } else if (coder == SparkHBaseConf.PrimitiveType) {
95 | new PrimitiveType()
96 | } else {
97 | // Data type implemented by user
98 | Class.forName(coder)
99 | .getConstructor(classOf[Option[Field]])
100 | .newInstance(None)
101 | .asInstanceOf[SHCDataType]
102 | }
103 | }
104 | }
105 |
--------------------------------------------------------------------------------
/core/src/test/scala/org/apache/spark/sql/PhoenixSuite.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * (C) 2017 Hortonworks, Inc. All rights reserved. See the NOTICE file
3 | * distributed with this work for additional information regarding copyright
4 | * ownership. This file is licensed to You under the Apache License, Version 2.0
5 | * (the "License"); you may not use this file except in compliance with
6 | * the License. You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 |
17 | package org.apache.spark.sql
18 |
19 | import org.apache.spark.sql.execution.datasources.hbase.Logging
20 | import org.apache.spark.sql.execution.datasources.hbase.HBaseTableCatalog
21 |
22 | case class PRecord(
23 | col0: String,
24 | col1: Boolean,
25 | col2: Double,
26 | col3: Float,
27 | col4: Int,
28 | col5: Long,
29 | col6: Short,
30 | col7: String,
31 | col8: Byte)
32 |
33 | object PRecord {
34 | def apply(i: Int, t: String): PRecord = {
35 | val s = s"""row${"%03d".format(i)}"""
36 | PRecord(s,
37 | i % 2 == 0,
38 | i.toDouble,
39 | i.toFloat,
40 | i,
41 | i.toLong,
42 | i.toShort,
43 | s"String$i: $t",
44 | i.toByte)
45 | }
46 |
47 | def unpadded(i: Int, t: String): PRecord = {
48 | val s = s"""row${i}"""
49 | PRecord(s,
50 | i % 2 == 0,
51 | i.toDouble,
52 | i.toFloat,
53 | i,
54 | i.toLong,
55 | i.toShort,
56 | s"String$i: $t",
57 | i.toByte)
58 | }
59 | }
60 |
61 | class PhoenixSuite extends SHC with Logging {
62 | override def catalog = s"""{
63 | |"table":{"namespace":"default", "name":"phoenixTable",
64 | | "tableCoder":"Phoenix", "version":"2.0"},
65 | |"rowkey":"key",
66 | |"columns":{
67 | |"col0":{"cf":"rowkey", "col":"key", "type":"string"},
68 | |"col1":{"cf":"cf1", "col":"col1", "type":"boolean"},
69 | |"col2":{"cf":"cf2", "col":"col2", "type":"double"},
70 | |"col3":{"cf":"cf3", "col":"col3", "type":"float"},
71 | |"col4":{"cf":"cf4", "col":"col4", "type":"int"},
72 | |"col5":{"cf":"cf5", "col":"col5", "type":"bigint"},
73 | |"col6":{"cf":"cf6", "col":"col6", "type":"smallint"},
74 | |"col7":{"cf":"cf7", "col":"col7", "type":"string"},
75 | |"col8":{"cf":"cf8", "col":"col8", "type":"tinyint"}
76 | |}
77 | |}""".stripMargin
78 |
79 |
80 | def withCatalog(cat: String): DataFrame = {
81 | sqlContext
82 | .read
83 | .options(Map(HBaseTableCatalog.tableCatalog->cat))
84 | .format("org.apache.spark.sql.execution.datasources.hbase")
85 | .load()
86 | }
87 |
88 | test("populate table") {
89 | val sql = sqlContext
90 | import sql.implicits._
91 |
92 | val data = (0 to 255).map { i =>
93 | PRecord(i, "extra")
94 | }
95 | sc.parallelize(data).toDF.write.options(
96 | Map(HBaseTableCatalog.tableCatalog -> catalog, HBaseTableCatalog.newTable -> "5"))
97 | .format("org.apache.spark.sql.execution.datasources.hbase")
98 | .save()
99 | }
100 |
101 | test("full query") {
102 | val df = withCatalog(catalog)
103 | df.show
104 | assert(df.count() == 256)
105 | assert(df.first().getByte(8) == 0)
106 | }
107 |
108 | test("empty column") {
109 | val df = withCatalog(catalog)
110 | df.registerTempTable("table0")
111 | val c = sqlContext.sql("select count(1) from table0").rdd.collect()(0)(0).asInstanceOf[Long]
112 | assert(c == 256)
113 | }
114 |
115 | test("IN and Not IN filter1") {
116 | val df = withCatalog(catalog)
117 | val s = df.filter(($"col0" isin ("row005", "row001", "row002")) and !($"col0" isin ("row001", "row002")))
118 | .select("col0", "col8")
119 | s.explain(true)
120 | s.show
121 | assert(s.count() == 1)
122 | assert(s.first().getByte(1) == 5)
123 | }
124 | }
125 |
--------------------------------------------------------------------------------
/examples/src/main/scala/org/apache/spark/sql/execution/datasources/hbase/LRJobForDataSources.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * (C) 2017 Hortonworks, Inc. All rights reserved. See the NOTICE file
3 | * distributed with this work for additional information regarding copyright
4 | * ownership. This file is licensed to You under the Apache License, Version 2.0
5 | * (the "License"); you may not use this file except in compliance with
6 | * the License. You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 |
17 | package org.apache.spark.sql.execution.datasources.hbase.examples
18 |
19 | import org.apache.spark.sql.{DataFrame, SparkSession}
20 | import org.apache.spark.sql.execution.datasources.hbase.{HBaseRelation, HBaseTableCatalog}
21 |
22 | case class LRRecord(
23 | key: Int,
24 | col1: Boolean,
25 | col2: Double,
26 | col3: Float)
27 |
28 | object LRRecord {
29 | def apply(i: Int): LRRecord = {
30 | LRRecord(i,
31 | i % 2 == 0,
32 | i.toDouble,
33 | i.toFloat)
34 | }
35 | }
36 |
37 | // long running job for different data sources
38 | object LRJobForDataSources {
39 | val cat = s"""{
40 | |"table":{"namespace":"default", "name":"shcExampleTable", "tableCoder":"PrimitiveType"},
41 | |"rowkey":"key",
42 | |"columns":{
43 | |"key":{"cf":"rowkey", "col":"key", "type":"int"},
44 | |"col1":{"cf":"cf1", "col":"col1", "type":"boolean"},
45 | |"col2":{"cf":"cf2", "col":"col2", "type":"double"},
46 | |"col3":{"cf":"cf3", "col":"col3", "type":"float"}
47 | |}
48 | |}""".stripMargin
49 |
50 | def main(args: Array[String]) {
51 | if (args.length < 1) {
52 | System.err.println("Usage: LRJobAccessing2Clusters [sleepTime]")
53 | System.exit(1)
54 | }
55 |
56 | val hiveTableName = args(0)
57 | val sleepTime = if (args.length > 1) args(1).toLong else 2 * 60 * 1000 // sleep 2 min by default
58 |
59 | val spark = SparkSession.builder()
60 | .appName("LRJobForDataSources")
61 | .enableHiveSupport()
62 | .getOrCreate()
63 |
64 | val sc = spark.sparkContext
65 | val sqlContext = spark.sqlContext
66 |
67 | import sqlContext.implicits._
68 | import spark.sql
69 |
70 | def withCatalog(cat: String): DataFrame = {
71 | sqlContext
72 | .read
73 | .options(Map(HBaseTableCatalog.tableCatalog->cat))
74 | .format("org.apache.spark.sql.execution.datasources.hbase")
75 | .load()
76 | }
77 |
78 | val timeEnd = System.currentTimeMillis() + (25 * 60 * 60 * 1000) // 25h later
79 | while (System.currentTimeMillis() < timeEnd) {
80 | // Part 1: write data into Hive table and read data from it, which accesses HDFS
81 | sql(s"DROP TABLE IF EXISTS $hiveTableName")
82 | sql(s"CREATE TABLE $hiveTableName(key INT, col1 BOOLEAN, col2 DOUBLE, col3 FLOAT)")
83 | for (i <- 1 to 3) {
84 | sql(s"INSERT INTO $hiveTableName VALUES ($i, ${i % 2 == 0}, ${i.toDouble}, ${i.toFloat})")
85 | }
86 | val df1 = sql(s"SELECT * FROM $hiveTableName")
87 | df1.show()
88 |
89 | // Part 2: create HBase table, write data into it, read data from it
90 | val data = (0 to 40).map { i =>
91 | LRRecord(i)
92 | }
93 | sc.parallelize(data).toDF.write.options(
94 | Map(HBaseTableCatalog.tableCatalog -> cat, HBaseTableCatalog.newTable -> "5"))
95 | .format("org.apache.spark.sql.execution.datasources.hbase")
96 | .save()
97 | val df2 = withCatalog(cat)
98 | df2.show
99 | df2.filter($"key" <= "5").select($"key", $"col1").show
100 |
101 | // Part 3: join the two dataframes
102 | val s1 = df1.filter($"key" <= "40").select("key", "col1")
103 | val s2 = df2.filter($"key" <= "20" && $"key" >= "1").select("key", "col2")
104 | val result = s1.join(s2, Seq("key"))
105 | result.show()
106 |
107 | Thread.sleep(sleepTime)
108 | }
109 |
110 | spark.stop()
111 | }
112 | }
113 |
--------------------------------------------------------------------------------
/core/src/main/scala/org/apache/spark/sql/execution/datasources/hbase/Logging.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 | * File modified by Hortonworks, Inc. Modifications are also licensed under
18 | * the Apache Software License, Version 2.0.
19 | */
20 |
21 | package org.apache.spark.sql.execution.datasources.hbase
22 |
23 | import org.apache.log4j.LogManager
24 | import org.slf4j.impl.StaticLoggerBinder
25 | import org.slf4j.{Logger, LoggerFactory}
26 |
27 | trait Logging {
28 |
29 | // Make the log field transient so that objects with Logging can
30 | // be serialized and used on another machine
31 | @transient private var log_ : Logger = null
32 |
33 | // Method to get the logger name for this object
34 | protected def logName = {
35 | // Ignore trailing $'s in the class names for Scala objects
36 | this.getClass.getName.stripSuffix("$")
37 | }
38 |
39 | // Method to get or create the logger for this object
40 | protected def log: Logger = {
41 | if (log_ == null) {
42 | initializeLogIfNecessary(false)
43 | log_ = LoggerFactory.getLogger(logName)
44 | }
45 | log_
46 | }
47 |
48 | // Log methods that take only a String
49 | protected def logInfo(msg: => String) {
50 | if (log.isInfoEnabled) log.info(msg)
51 | }
52 |
53 | protected def logDebug(msg: => String) {
54 | if (log.isDebugEnabled) log.debug(msg)
55 | }
56 |
57 | protected def logTrace(msg: => String) {
58 | if (log.isTraceEnabled) log.trace(msg)
59 | }
60 |
61 | protected def logWarning(msg: => String) {
62 | if (log.isWarnEnabled) log.warn(msg)
63 | }
64 |
65 | protected def logError(msg: => String) {
66 | if (log.isErrorEnabled) log.error(msg)
67 | }
68 |
69 | // Log methods that take Throwables (Exceptions/Errors) too
70 | protected def logInfo(msg: => String, throwable: Throwable) {
71 | if (log.isInfoEnabled) log.info(msg, throwable)
72 | }
73 |
74 | protected def logDebug(msg: => String, throwable: Throwable) {
75 | if (log.isDebugEnabled) log.debug(msg, throwable)
76 | }
77 |
78 | protected def logTrace(msg: => String, throwable: Throwable) {
79 | if (log.isTraceEnabled) log.trace(msg, throwable)
80 | }
81 |
82 | protected def logWarning(msg: => String, throwable: Throwable) {
83 | if (log.isWarnEnabled) log.warn(msg, throwable)
84 | }
85 |
86 | protected def logError(msg: => String, throwable: Throwable) {
87 | if (log.isErrorEnabled) log.error(msg, throwable)
88 | }
89 |
90 | protected def isTraceEnabled(): Boolean = {
91 | log.isTraceEnabled
92 | }
93 |
94 | protected def initializeLogIfNecessary(isInterpreter: Boolean): Unit = {
95 | if (!Logging.initialized) {
96 | Logging.initLock.synchronized {
97 | if (!Logging.initialized) {
98 | initializeLogging(isInterpreter)
99 | }
100 | }
101 | }
102 | }
103 |
104 | private def initializeLogging(isInterpreter: Boolean): Unit = {
105 | // Don't use a logger in here, as this is itself occurring during initialization of a logger
106 | // If Log4j 1.2 is being used, but is not initialized, load a default properties file
107 | val binderClass = StaticLoggerBinder.getSingleton.getLoggerFactoryClassStr
108 | // This distinguishes the log4j 1.2 binding, currently
109 | // org.slf4j.impl.Log4jLoggerFactory, from the log4j 2.0 binding, currently
110 | // org.apache.logging.slf4j.Log4jLoggerFactory
111 | val usingLog4j12 = "org.slf4j.impl.Log4jLoggerFactory".equals(binderClass)
112 | if (usingLog4j12) {
113 | val log4j12Initialized = LogManager.getRootLogger.getAllAppenders.hasMoreElements
114 | }
115 | Logging.initialized = true
116 |
117 | // Force a call into slf4j to initialize it. Avoids this happening from multiple threads
118 | // and triggering this: http://mailman.qos.ch/pipermail/slf4j-dev/2010-April/002956.html
119 | log
120 | }
121 | }
122 |
123 | private object Logging {
124 | @volatile private var initialized = false
125 | val initLock = new Object()
126 | }
127 |
--------------------------------------------------------------------------------
/core/src/test/scala/org/apache/spark/sql/DataTypeSuite.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 | * File modified by Hortonworks, Inc. Modifications are also licensed under
18 | * the Apache Software License, Version 2.0.
19 | */
20 |
21 | package org.apache.spark.sql
22 |
23 | import org.apache.spark.sql.execution.datasources.hbase.{HBaseTableCatalog, Logging}
24 |
25 | case class IntKeyRecord(
26 | col0: Integer,
27 | col1: Boolean,
28 | col2: Double,
29 | col3: Float,
30 | col4: Int,
31 | col5: Long,
32 | col6: Short,
33 | col7: String,
34 | col8: Byte)
35 |
36 | object IntKeyRecord {
37 | def apply(i: Int): IntKeyRecord = {
38 | IntKeyRecord(if (i % 2 == 0) i else -i,
39 | i % 2 == 0,
40 | i.toDouble,
41 | i.toFloat,
42 | i,
43 | i.toLong,
44 | i.toShort,
45 | s"String$i extra",
46 | i.toByte)
47 | }
48 | }
49 |
50 | class DataTypeSuite extends SHC with Logging {
51 |
52 | override def catalog = s"""{
53 | |"table":{"namespace":"default", "name":"table1", "tableCoder":"PrimitiveType"},
54 | |"rowkey":"key",
55 | |"columns":{
56 | |"col0":{"cf":"rowkey", "col":"key", "type":"int"},
57 | |"col1":{"cf":"cf1", "col":"col1", "type":"boolean"},
58 | |"col2":{"cf":"cf1", "col":"col2", "type":"double"},
59 | |"col3":{"cf":"cf3", "col":"col3", "type":"float"},
60 | |"col4":{"cf":"cf4", "col":"col4", "type":"int"},
61 | |"col5":{"cf":"cf5", "col":"col5", "type":"bigint"},
62 | |"col6":{"cf":"cf6", "col":"col6", "type":"smallint"},
63 | |"col7":{"cf":"cf7", "col":"col7", "type":"string"},
64 | |"col8":{"cf":"cf7", "col":"col8", "type":"tinyint"}
65 | |}
66 | |}""".stripMargin
67 |
68 | def withCatalog(cat: String): DataFrame = {
69 | sqlContext
70 | .read
71 | .options(Map(HBaseTableCatalog.tableCatalog->cat))
72 | .format("org.apache.spark.sql.execution.datasources.hbase")
73 | .load()
74 | }
75 |
76 | test("populate table") {
77 | //createTable(tableName, columnFamilies)
78 | val sql = sqlContext
79 | import sql.implicits._
80 |
81 | val data = (0 until 32).map { i =>
82 | IntKeyRecord(i)
83 | }
84 | sc.parallelize(data).toDF.write.options(
85 | Map(HBaseTableCatalog.tableCatalog -> catalog, HBaseTableCatalog.newTable -> "5"))
86 | .format("org.apache.spark.sql.execution.datasources.hbase")
87 | .save()
88 | }
89 |
90 | test("less than 0") {
91 | val df = withCatalog(catalog)
92 | val s = df.filter($"col0" < 0)
93 | s.show
94 | assert(s.count() == 16)
95 | }
96 |
97 | test("lessequal than -10") {
98 | val df = withCatalog(catalog)
99 | val s = df.filter($"col0" <= -10)
100 | s.show
101 | assert(s.count() == 11)
102 | }
103 |
104 | test("lessequal than -9") {
105 | val df = withCatalog(catalog)
106 | val s = df.filter($"col0" <= -9)
107 | s.show
108 | assert(s.count() == 12)
109 | }
110 |
111 | test("greaterequal than -9") {
112 | val df = withCatalog(catalog)
113 | val s = df.filter($"col0" >= -9)
114 | s.show
115 | assert(s.count() == 21)
116 | }
117 |
118 | test("greaterequal than 0") {
119 | val df = withCatalog(catalog)
120 | val s = df.filter($"col0" >= 0)
121 | s.show
122 | assert(s.count() == 16)
123 | }
124 |
125 | test("greater than 10") {
126 | val df = withCatalog(catalog)
127 | val s = df.filter($"col0" > 10)
128 | s.show
129 | assert(s.count() == 10)
130 | }
131 |
132 | test("and") {
133 | val df = withCatalog(catalog)
134 | val s = df.filter($"col0" > -10 && $"col0" <= 10)
135 | s.show
136 | assert(s.count() == 11)
137 | }
138 |
139 | test("or") {
140 | val df = withCatalog(catalog)
141 | val s = df.filter($"col0" <= -10 || $"col0" > 10)
142 | s.show
143 | assert(s.count() == 21)
144 | }
145 |
146 | test("all") {
147 | val df = withCatalog(catalog)
148 | val s = df.filter($"col0" >= -100)
149 | s.show
150 | assert(s.count() == 32)
151 | }
152 |
153 | test("full query") {
154 | val df = withCatalog(catalog)
155 | df.show
156 | assert(df.count() == 32)
157 | }
158 | }
159 |
--------------------------------------------------------------------------------
/core/src/test/scala/org/apache/spark/sql/SHC.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 | * File modified by Hortonworks, Inc. Modifications are also licensed under
18 | * the Apache Software License, Version 2.0.
19 | */
20 |
21 | package org.apache.spark.sql
22 |
23 | import org.apache.spark.sql.execution.datasources.hbase.Logging
24 |
25 | import java.io.File
26 |
27 | import com.google.common.io.Files
28 | import org.apache.hadoop.hbase.client.Table
29 | import org.apache.hadoop.hbase.util.Bytes
30 | import org.apache.hadoop.hbase.{HBaseTestingUtility, TableName}
31 | import org.apache.spark.sql.execution.datasources.hbase.SparkHBaseConf
32 | import org.apache.spark.{SparkContext, SparkConf}
33 | import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, FunSuite}
34 |
35 | class SHC extends FunSuite with BeforeAndAfterEach with BeforeAndAfterAll with Logging {
36 | implicit class StringToColumn(val sc: StringContext) {
37 | def $(args: Any*): ColumnName = {
38 | new ColumnName(sc.s(args: _*))
39 | }
40 | }
41 |
42 | var spark: SparkSession = null
43 | var sc: SparkContext = null
44 | var sqlContext: SQLContext = null
45 | var df: DataFrame = null
46 |
47 | private[spark] var htu = new HBaseTestingUtility
48 | private[spark] def tableName = "table1"
49 |
50 | private[spark] def columnFamilies: Array[String] = Array.tabulate(9){ x=> s"cf$x"}
51 | var table: Table = null
52 | val conf = new SparkConf
53 | conf.set(SparkHBaseConf.testConf, "true")
54 | // private[spark] var columnFamilyStr = Bytes.toString(columnFamily)
55 |
56 | def defineCatalog(tName: String) = s"""{
57 | |"table":{"namespace":"default", "name":"$tName"},
58 | |"rowkey":"key",
59 | |"columns":{
60 | |"col0":{"cf":"rowkey", "col":"key", "type":"string"},
61 | |"col1":{"cf":"cf1", "col":"col1", "type":"boolean"},
62 | |"col2":{"cf":"cf2", "col":"col2", "type":"double"},
63 | |"col3":{"cf":"cf3", "col":"col3", "type":"float"},
64 | |"col4":{"cf":"cf4", "col":"col4", "type":"int"},
65 | |"col5":{"cf":"cf5", "col":"col5", "type":"bigint"},
66 | |"col6":{"cf":"cf6", "col":"col6", "type":"smallint"},
67 | |"col7":{"cf":"cf7", "col":"col7", "type":"string"},
68 | |"col8":{"cf":"cf8", "col":"col8", "type":"tinyint"}
69 | |}
70 | |}""".stripMargin
71 |
72 | @deprecated(since = "04.12.2017(dd/mm/year)", message = "use `defineCatalog` instead")
73 | def catalog = defineCatalog(tableName)
74 |
75 | override def beforeAll() {
76 | val tempDir: File = Files.createTempDir
77 | tempDir.deleteOnExit
78 | htu.startMiniCluster
79 | SparkHBaseConf.conf = htu.getConfiguration
80 | logInfo(" - minicluster started")
81 | println(" - minicluster started")
82 |
83 | spark = SparkSession.builder()
84 | .master("local")
85 | .appName("HBaseTest")
86 | .config(conf)
87 | .getOrCreate()
88 |
89 | sqlContext = spark.sqlContext
90 | sc = spark.sparkContext
91 | }
92 |
93 | override def afterAll() {
94 | htu.shutdownMiniCluster()
95 | spark.stop()
96 | }
97 |
98 | def createTable(name: String, cfs: Array[String]) {
99 | val tName = Bytes.toBytes(name)
100 | val bcfs = cfs.map(Bytes.toBytes(_))
101 | try {
102 | htu.deleteTable(TableName.valueOf(tName))
103 | } catch {
104 | case _ : Throwable =>
105 | logInfo(" - no table " + name + " found")
106 | }
107 | htu.createMultiRegionTable(TableName.valueOf(tName), bcfs)
108 | }
109 |
110 |
111 | def createTable(name: Array[Byte], cfs: Array[Array[Byte]]) {
112 | try {
113 | htu.deleteTable(TableName.valueOf(name))
114 | } catch {
115 | case _ : Throwable =>
116 | logInfo(" - no table " + Bytes.toString(name) + " found")
117 | }
118 | htu.createMultiRegionTable(TableName.valueOf(name), cfs)
119 | }
120 | }
121 |
--------------------------------------------------------------------------------
/core/src/main/scala/org/apache/spark/sql/execution/datasources/hbase/HBaseResources.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 | * File modified by Hortonworks, Inc. Modifications are also licensed under
18 | * the Apache Software License, Version 2.0.
19 | */
20 |
21 | package org.apache.spark.sql.execution.datasources.hbase
22 |
23 | import scala.language.implicitConversions
24 |
25 | import org.apache.hadoop.hbase.TableName
26 | import org.apache.hadoop.hbase.client._
27 |
28 | // Resource and ReferencedResources are defined for extensibility,
29 | // e.g., consolidate scan and bulkGet in the future work.
30 |
31 | // User has to invoke release explicitly to release the resource,
32 | // and potentially parent resources
33 | trait Resource {
34 | def release(): Unit
35 | }
36 |
37 | case class ScanResource(tbr: TableResource, rs: ResultScanner) extends Resource {
38 | def release() {
39 | rs.close()
40 | tbr.release()
41 | }
42 | }
43 |
44 | case class GetResource(tbr: TableResource, rs: Array[Result]) extends Resource {
45 | def release() {
46 | tbr.release()
47 | }
48 | }
49 |
50 | // Multiple child resource may use this one, which is reference counted.
51 | // It will not be released until the counter reaches 0
52 | trait ReferencedResource {
53 | var count: Int = 0
54 | def init(): Unit
55 | def destroy(): Unit
56 | def acquire() = synchronized {
57 | try {
58 | count += 1
59 | if (count == 1) {
60 | init()
61 | }
62 | } catch {
63 | case e: Throwable =>
64 | release()
65 | throw e
66 | }
67 | }
68 |
69 | def release() = synchronized {
70 | count -= 1
71 | if (count == 0) {
72 | destroy()
73 | }
74 | }
75 |
76 | def releaseOnException[T](func: => T): T = {
77 | acquire()
78 | val ret = {
79 | try {
80 | func
81 | } catch {
82 | case e: Throwable =>
83 | release()
84 | throw e
85 | }
86 | }
87 | ret
88 | }
89 | }
90 |
91 | case class RegionResource(relation: HBaseRelation) extends ReferencedResource {
92 | var connection: SmartConnection = _
93 | var rl: RegionLocator = _
94 |
95 | override def init(): Unit = {
96 | connection = HBaseConnectionCache.getConnection(relation.hbaseConf)
97 | rl = connection.getRegionLocator(TableName.valueOf(relation.catalog.namespace, relation.catalog.name))
98 | }
99 |
100 | override def destroy(): Unit = {
101 | if (rl != null) {
102 | rl.close()
103 | rl = null
104 | }
105 | if (connection != null) {
106 | connection.close()
107 | connection = null
108 | }
109 | }
110 |
111 | val regions = releaseOnException {
112 | val keys = rl.getStartEndKeys
113 | keys.getFirst.zip(keys.getSecond)
114 | .zipWithIndex
115 | .map(x =>
116 | HBaseRegion(x._2,
117 | Some(x._1._1),
118 | Some(x._1._2),
119 | Some(rl.getRegionLocation(x._1._1).getHostname)))
120 | }
121 | }
122 |
123 | case class TableResource(relation: HBaseRelation) extends ReferencedResource {
124 | var connection: SmartConnection = _
125 | var table: Table = _
126 |
127 | override def init(): Unit = {
128 | connection = HBaseConnectionCache.getConnection(relation.hbaseConf)
129 | table = connection.getTable(TableName.valueOf(relation.catalog.namespace, relation.catalog.name))
130 | }
131 |
132 | override def destroy(): Unit = {
133 | if (table != null) {
134 | table.close()
135 | table = null
136 | }
137 | if (connection != null) {
138 | connection.close()
139 | connection = null
140 | }
141 | }
142 |
143 | def get(list: java.util.List[org.apache.hadoop.hbase.client.Get]) = releaseOnException {
144 | GetResource(this, table.get(list))
145 | }
146 |
147 | def getScanner(scan: Scan): ScanResource = releaseOnException {
148 | ScanResource(this, table.getScanner(scan))
149 | }
150 | }
151 |
152 | object HBaseResources{
153 | implicit def ScanResToScan(sr: ScanResource): ResultScanner = {
154 | sr.rs
155 | }
156 |
157 | implicit def GetResToResult(gr: GetResource): Array[Result] = {
158 | gr.rs
159 | }
160 |
161 | implicit def TableResToTable(tr: TableResource): Table = {
162 | tr.table
163 | }
164 |
165 | implicit def RegionResToRegions(rr: RegionResource): Seq[HBaseRegion] = {
166 | rr.regions
167 | }
168 | }
169 |
--------------------------------------------------------------------------------
/core/src/main/scala/org/apache/spark/sql/execution/datasources/hbase/types/PrimitiveType.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * (C) 2017 Hortonworks, Inc. All rights reserved. See the NOTICE file
3 | * distributed with this work for additional information regarding copyright
4 | * ownership. This file is licensed to You under the Apache License, Version 2.0
5 | * (the "License"); you may not use this file except in compliance with
6 | * the License. You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 |
17 | package org.apache.spark.sql.execution.datasources.hbase.types
18 |
19 | import org.apache.hadoop.hbase.util.Bytes
20 | import org.apache.spark.sql.Row
21 | import org.apache.spark.sql.types._
22 | import org.apache.spark.unsafe.types.UTF8String
23 | import org.apache.spark.sql.execution.datasources.hbase._
24 |
25 | class PrimitiveType(f:Option[Field] = None) extends SHCDataType {
26 |
27 | private def fromBytes(src: HBaseType, dt: DataType): Any = dt match {
28 | case BooleanType => toBoolean(src)
29 | case ByteType => src(0)
30 | case DoubleType => Bytes.toDouble(src)
31 | case FloatType => Bytes.toFloat(src)
32 | case IntegerType => Bytes.toInt(src)
33 | case LongType => Bytes.toLong(src)
34 | case ShortType => Bytes.toShort(src)
35 | case StringType => toUTF8String(src, src.length)
36 | case BinaryType => src
37 | // this block MapType in future if connector want to support it
38 | case m: MapType => fromBytes(src, m.valueType)
39 | case _ => throw new UnsupportedOperationException(s"unsupported data type ${f.get.dt}")
40 | }
41 |
42 | def fromBytes(src: HBaseType): Any = {
43 | if (f.isDefined) {
44 | fromBytes(src, f.get.dt)
45 | } else {
46 | throw new UnsupportedOperationException(
47 | "PrimitiveType coder: without field metadata, " +
48 | "'fromBytes' conversion can not be supported")
49 | }
50 | }
51 |
52 |
53 | def toBytes(input: Any): Array[Byte] = {
54 | input match {
55 | case data: Boolean => Bytes.toBytes(data)
56 | case data: Byte => Array(data)
57 | case data: Array[Byte] => data
58 | case data: Double => Bytes.toBytes(data)
59 | case data: Float => Bytes.toBytes(data)
60 | case data: Int => Bytes.toBytes(data)
61 | case data: Long => Bytes.toBytes(data)
62 | case data: Short => Bytes.toBytes(data)
63 | case data: UTF8String => data.getBytes
64 | case data: String => Bytes.toBytes(data)
65 | case _ => throw new
66 | UnsupportedOperationException(s"PrimitiveType coder: unsupported data type $input")
67 | }
68 | }
69 |
70 | override def isRowKeySupported(): Boolean = true
71 |
72 | override def isCompositeKeySupported(): Boolean = true
73 |
74 | override def decodeCompositeRowKey(row: Array[Byte], keyFields: Seq[Field]): Map[Field, Any] = {
75 | keyFields.foldLeft((0, Seq[(Field, Any)]()))((state, field) => {
76 | val idx = state._1
77 | val parsed = state._2
78 | if (field.length != -1) {
79 | val value = fromBytes(field, row, idx, field.length)
80 | // Return the new index and appended value
81 | (idx + field.length, parsed ++ Seq((field, value)))
82 | } else {
83 | // This is the last dimension.
84 | val value = fromBytes(field, row, idx, row.length - idx)
85 | (row.length + 1, parsed ++ Seq((field, value)))
86 | }
87 | })._2.toMap
88 | }
89 |
90 | private def fromBytes(field: Field, src: HBaseType, offset: Int, length: Int): Any = {
91 | field.dt match {
92 | case BooleanType => toBoolean(src, offset)
93 | case ByteType => src(offset)
94 | case DoubleType => Bytes.toDouble(src, offset)
95 | case FloatType => Bytes.toFloat(src, offset)
96 | case IntegerType => Bytes.toInt(src, offset)
97 | case LongType => Bytes.toLong(src, offset)
98 | case ShortType => Bytes.toShort(src, offset)
99 | case StringType => toUTF8String(src, length, offset)
100 | case BinaryType =>
101 | val newArray = new Array[Byte](length)
102 | System.arraycopy(src, offset, newArray, 0, length)
103 | newArray
104 | case _ => throw new
105 | UnsupportedOperationException(s"PrimitiveType coder: unsupported data type ${field.dt}")
106 | }
107 | }
108 |
109 | override def encodeCompositeRowKey(rkIdxedFields:Seq[(Int, Field)], row: Row): Seq[Array[Byte]] = {
110 | rkIdxedFields.map { case (x, y) =>
111 | SHCDataTypeFactory.create(y).toBytes(row(x))
112 | }
113 | }
114 |
115 | private def toBoolean(input: HBaseType, offset: Int = 0): Boolean = {
116 | input(offset) != 0
117 | }
118 |
119 | private def toUTF8String(input: HBaseType, length: Int, offset: Int = 0): UTF8String = {
120 | UTF8String.fromBytes(input.slice(offset, offset + length))
121 | }
122 | }
123 |
--------------------------------------------------------------------------------
/core/src/main/scala/org/apache/spark/sql/execution/datasources/hbase/types/Phoenix.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 | * File modified by Hortonworks, Inc. Modifications are also licensed under
18 | * the Apache Software License, Version 2.0.
19 | */
20 |
21 | package org.apache.spark.sql.execution.datasources.hbase.types
22 |
23 | import org.apache.hadoop.hbase.io.ImmutableBytesWritable
24 | import org.apache.phoenix.query.QueryConstants
25 | import org.apache.phoenix.schema._
26 | import org.apache.phoenix.schema.RowKeySchema.RowKeySchemaBuilder
27 | import org.apache.phoenix.schema.types._
28 | import org.apache.spark.sql.Row
29 | import org.apache.spark.sql.execution.datasources.hbase._
30 | import org.apache.spark.sql.types._
31 |
32 | class Phoenix(f:Option[Field] = None) extends SHCDataType {
33 | private var schema: RowKeySchema = null
34 |
35 | def fromBytes(src: HBaseType): Any = {
36 | if (f.isDefined) {
37 | mapToPhoenixTypeInstance(f.get.dt).toObject(src)
38 | } else {
39 | throw new UnsupportedOperationException(
40 | "Phoenix coder: without field metadata, 'fromBytes' conversion can not be supported")
41 | }
42 | }
43 |
44 | def toBytes(input: Any): Array[Byte] = {
45 | input match {
46 | case data: Boolean => PBoolean.INSTANCE.toBytes(data)
47 | case data: Byte => PTinyint.INSTANCE.toBytes(data)
48 | case data: Array[Byte] => PVarbinary.INSTANCE.toBytes(data)
49 | case data: Double => PDouble.INSTANCE.toBytes(data)
50 | case data: Float => PFloat.INSTANCE.toBytes(data)
51 | case data: Int => PInteger.INSTANCE.toBytes(data)
52 | case data: Long => PLong.INSTANCE.toBytes(data)
53 | case data: Short => PSmallint.INSTANCE.toBytes(data)
54 | case data: String => PVarchar.INSTANCE.toBytes(data)
55 | case _ => throw new UnsupportedOperationException(s"unsupported data type $input")
56 | }
57 | }
58 |
59 | override def isRowKeySupported(): Boolean = true
60 |
61 | override def isCompositeKeySupported(): Boolean = true
62 |
63 | override def decodeCompositeRowKey(row: Array[Byte], keyFields: Seq[Field]): Map[Field, Any] = {
64 | if (schema == null) schema = buildSchema(keyFields)
65 | val ptr: ImmutableBytesWritable = new ImmutableBytesWritable
66 | val maxOffest = schema.iterator(row, 0, row.length, ptr)
67 | var ret = Map.empty[Field, Any]
68 | for (i <- 0 until schema.getFieldCount) {
69 | if (schema.next(ptr, i, maxOffest) != null) {
70 | val value = mapToPhoenixTypeInstance(keyFields(i).dt)
71 | .toObject(ptr, schema.getField(i).getDataType, SortOrder.getDefault)
72 | ret += ((keyFields(i), value))
73 | }
74 | }
75 | ret
76 | }
77 |
78 | override def encodeCompositeRowKey(rkIdxedFields: Seq[(Int, Field)], row: Row): Seq[Array[Byte]] = {
79 | rkIdxedFields.map { case (x, y) =>
80 | var ret = toBytes(row(x))
81 | // the last dimension of composite key does not need SEPARATOR
82 | if (y.length == -1 && x < rkIdxedFields.size - 1)
83 | ret ++= QueryConstants.SEPARATOR_BYTE_ARRAY
84 | ret
85 | }
86 | }
87 |
88 | private def buildSchema(keyFields: Seq[Field]): RowKeySchema = {
89 | val builder: RowKeySchemaBuilder = new RowKeySchemaBuilder(keyFields.length)
90 | keyFields.foreach{ x =>
91 | builder.addField(buildPDatum(x.dt), false, SortOrder.getDefault)
92 | }
93 | builder.build
94 | }
95 |
96 | private def mapToPhoenixTypeInstance(input: DataType): PDataType[_] = {
97 | input match {
98 | case BooleanType => PBoolean.INSTANCE
99 | case ByteType => PTinyint.INSTANCE
100 | case DoubleType => PDouble.INSTANCE
101 | case IntegerType => PInteger.INSTANCE
102 | case FloatType => PFloat.INSTANCE
103 | case LongType => PLong.INSTANCE
104 | case ShortType => PSmallint.INSTANCE
105 | case StringType => PVarchar.INSTANCE
106 | case BinaryType => PVarbinary.INSTANCE
107 | case _ => throw new UnsupportedOperationException(s"unsupported data type $input")
108 | }
109 | }
110 |
111 | private def buildPDatum(input: DataType): PDatum = new PDatum {
112 | override def getScale: Integer = null
113 | override def isNullable: Boolean = false
114 | override def getDataType: PDataType[_] = mapToPhoenixTypeInstance(input)
115 | override def getMaxLength: Integer = null
116 | override def getSortOrder: SortOrder = SortOrder.getDefault
117 | }
118 | }
119 |
--------------------------------------------------------------------------------
/examples/src/main/scala/org/apache/spark/sql/execution/datasources/hbase/HBaseSource.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 | * File modified by Hortonworks, Inc. Modifications are also licensed under
18 | * the Apache Software License, Version 2.0.
19 | */
20 |
21 | package org.apache.spark.sql.execution.datasources.hbase.examples
22 |
23 | import org.apache.spark.sql.execution.datasources.hbase._
24 | import org.apache.spark.sql.{DataFrame, SparkSession}
25 |
26 | case class HBaseRecord(
27 | col0: String,
28 | col1: Boolean,
29 | col2: Double,
30 | col3: Float,
31 | col4: Int,
32 | col5: Long,
33 | col6: Short,
34 | col7: String,
35 | col8: Byte)
36 |
37 | object HBaseRecord {
38 | def apply(i: Int): HBaseRecord = {
39 | val s = s"""row${"%03d".format(i)}"""
40 | HBaseRecord(s,
41 | i % 2 == 0,
42 | i.toDouble,
43 | i.toFloat,
44 | i,
45 | i.toLong,
46 | i.toShort,
47 | s"String$i extra",
48 | i.toByte)
49 | }
50 | }
51 |
52 | object HBaseSource {
53 | val cat = s"""{
54 | |"table":{"namespace":"default", "name":"shcExampleTable", "tableCoder":"PrimitiveType"},
55 | |"rowkey":"key",
56 | |"columns":{
57 | |"col0":{"cf":"rowkey", "col":"key", "type":"string"},
58 | |"col1":{"cf":"cf1", "col":"col1", "type":"boolean"},
59 | |"col2":{"cf":"cf2", "col":"col2", "type":"double"},
60 | |"col3":{"cf":"cf3", "col":"col3", "type":"float"},
61 | |"col4":{"cf":"cf4", "col":"col4", "type":"int"},
62 | |"col5":{"cf":"cf5", "col":"col5", "type":"bigint"},
63 | |"col6":{"cf":"cf6", "col":"col6", "type":"smallint"},
64 | |"col7":{"cf":"cf7", "col":"col7", "type":"string"},
65 | |"col8":{"cf":"cf8", "col":"col8", "type":"tinyint"}
66 | |}
67 | |}""".stripMargin
68 |
69 | val cat1 = s"""{
70 | |"table":{"namespace":"default", "name":"shcExampleTable1", "tableCoder":"PrimitiveType"},
71 | |"rowkey":"key",
72 | |"columns":{
73 | |"col0":{"cf":"rowkey", "col":"key", "type":"string"},
74 | |"col1":{"cf":"cf1", "col":"col1", "type":"boolean"},
75 | |"col2":{"cf":"cf2", "col":"col2", "type":"double"},
76 | |"col3":{"cf":"cf3", "col":"col3", "type":"float"},
77 | |"col4":{"cf":"cf4", "col":"col4", "type":"int"},
78 | |"col5":{"cf":"cf5", "col":"col5", "type":"bigint"},
79 | |"col6":{"cf":"cf6", "col":"col6", "type":"smallint"},
80 | |"col7":{"cf":"cf7", "col":"col7", "type":"string"},
81 | |"col8":{"cf":"cf8", "col":"col8", "type":"tinyint"}
82 | |}
83 | |}""".stripMargin
84 |
85 | def main(args: Array[String]) {
86 | val spark = SparkSession.builder()
87 | .appName("HBaseSourceExample")
88 | .getOrCreate()
89 |
90 | val sc = spark.sparkContext
91 | val sqlContext = spark.sqlContext
92 |
93 | import sqlContext.implicits._
94 |
95 | def withCatalog(cat: String): DataFrame = {
96 | sqlContext
97 | .read
98 | .options(Map(HBaseTableCatalog.tableCatalog->cat))
99 | .format("org.apache.spark.sql.execution.datasources.hbase")
100 | .load()
101 | }
102 |
103 | val data = (0 to 255).map { i =>
104 | HBaseRecord(i)
105 | }
106 |
107 | sc.parallelize(data).toDF.write.options(
108 | Map(HBaseTableCatalog.tableCatalog -> cat, HBaseTableCatalog.newTable -> "5"))
109 | .format("org.apache.spark.sql.execution.datasources.hbase")
110 | .save()
111 |
112 | // for testing connection sharing only
113 | sc.parallelize(data).toDF.write.options(
114 | Map(HBaseTableCatalog.tableCatalog -> cat1, HBaseTableCatalog.newTable -> "5"))
115 | .format("org.apache.spark.sql.execution.datasources.hbase")
116 | .save()
117 |
118 | val df = withCatalog(cat)
119 | df.show
120 | df.filter($"col0" <= "row005")
121 | .select($"col0", $"col1").show
122 | df.filter($"col0" === "row005" || $"col0" <= "row005")
123 | .select($"col0", $"col1").show
124 | df.filter($"col0" > "row250")
125 | .select($"col0", $"col1").show
126 | df.createOrReplaceTempView("table1")
127 | val c = sqlContext.sql("select count(col1) from table1 where col0 < 'row050'")
128 | c.show()
129 |
130 | // for testing connection sharing only
131 | val df1 = withCatalog(cat1)
132 | df1.show
133 | df1.filter($"col0" <= "row005")
134 | .select($"col0", $"col1").show
135 | df1.filter($"col0" === "row005" || $"col0" <= "row005")
136 | .select($"col0", $"col1").show
137 | df1.filter($"col0" > "row250")
138 | .select($"col0", $"col1").show
139 | df1.createOrReplaceTempView("table1")
140 | val c1 = sqlContext.sql("select count(col1) from table1 where col0 < 'row050'")
141 | c1.show()
142 |
143 | spark.stop()
144 | }
145 | }
146 |
--------------------------------------------------------------------------------
/examples/src/main/scala/org/apache/spark/sql/execution/datasources/hbase/DataType.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 | * File modified by Hortonworks, Inc. Modifications are also licensed under
18 | * the Apache Software License, Version 2.0.
19 | */
20 |
21 | package org.apache.spark.sql.execution.datasources.hbase.examples
22 |
23 | import org.apache.spark.sql.{SparkSession, DataFrame}
24 | import org.apache.spark.sql.execution.datasources.hbase._
25 |
26 | case class IntKeyRecord(
27 | col0: Integer,
28 | col1: Boolean,
29 | col2: Double,
30 | col3: Float,
31 | col4: Int,
32 | col5: Long,
33 | col6: Short,
34 | col7: String,
35 | col8: Byte)
36 |
37 | object IntKeyRecord {
38 | def apply(i: Int): IntKeyRecord = {
39 | IntKeyRecord(if (i % 2 == 0) i else -i,
40 | i % 2 == 0,
41 | i.toDouble,
42 | i.toFloat,
43 | i,
44 | i.toLong,
45 | i.toShort,
46 | s"String$i extra",
47 | i.toByte)
48 | }
49 | }
50 |
51 | object DataType {
52 | val cat = s"""{
53 | |"table":{"namespace":"default", "name":"shcExampleTable", "tableCoder":"PrimitiveType"},
54 | |"rowkey":"key",
55 | |"columns":{
56 | |"col0":{"cf":"rowkey", "col":"key", "type":"int"},
57 | |"col1":{"cf":"cf1", "col":"col1", "type":"boolean"},
58 | |"col2":{"cf":"cf2", "col":"col2", "type":"double"},
59 | |"col3":{"cf":"cf3", "col":"col3", "type":"float"},
60 | |"col4":{"cf":"cf4", "col":"col4", "type":"int"},
61 | |"col5":{"cf":"cf5", "col":"col5", "type":"bigint"},
62 | |"col6":{"cf":"cf6", "col":"col6", "type":"smallint"},
63 | |"col7":{"cf":"cf7", "col":"col7", "type":"string"},
64 | |"col8":{"cf":"cf8", "col":"col8", "type":"tinyint"}
65 | |}
66 | |}""".stripMargin
67 |
68 | def main(args: Array[String]){
69 | val spark = SparkSession.builder()
70 | .appName("DataTypeExample")
71 | .getOrCreate()
72 |
73 | val sc = spark.sparkContext
74 | val sqlContext = spark.sqlContext
75 |
76 | import sqlContext.implicits._
77 |
78 | def withCatalog(cat: String): DataFrame = {
79 | sqlContext
80 | .read
81 | .options(Map(HBaseTableCatalog.tableCatalog->cat))
82 | .format("org.apache.spark.sql.execution.datasources.hbase")
83 | .load()
84 | }
85 |
86 | // test populate table
87 | val data = (0 until 32).map { i =>
88 | IntKeyRecord(i)
89 | }
90 | sc.parallelize(data).toDF.write.options(
91 | Map(HBaseTableCatalog.tableCatalog -> cat, HBaseTableCatalog.newTable -> "5"))
92 | .format("org.apache.spark.sql.execution.datasources.hbase")
93 | .save()
94 |
95 | // test less than 0
96 | val df = withCatalog(cat)
97 | val s = df.filter($"col0" < 0)
98 | s.show
99 | if(s.count() != 16){
100 | throw new Exception("value invalid")
101 | }
102 |
103 | //test less or equal than -10. The number of results is 11
104 | val num1 = df.filter($"col0" <= -10)
105 | num1.show
106 | val c1 = num1.count()
107 | println(s"test result count should be 11: $c1")
108 |
109 | //test less or equal than -9. The number of results is 12
110 | val num2 = df.filter($"col0" <= -9)
111 | num2.show
112 | val c2 = num2.count()
113 | println(s"test result count should be 12: $c2")
114 |
115 | //test greater or equal than -9". The number of results is 21
116 | val num3 = df.filter($"col0" >= -9)
117 | num3.show
118 | val c3 = num3.count()
119 | println(s"test result count should be 21: $c3")
120 |
121 | //test greater or equal than 0. The number of results is 16
122 | val num4 = df.filter($"col0" >= 0)
123 | num4.show
124 | val c4 = num4.count()
125 | println(s"test result count should be 16: $c4")
126 |
127 | //test greater than 10. The number of results is 10
128 | val num5 = df.filter($"col0" > 10)
129 | num5.show
130 | val c5 = num5.count()
131 | println(s"test result count should be 10: $c5")
132 |
133 | // test "and". The number of results is 11
134 | val num6 = df.filter($"col0" > -10 && $"col0" <= 10)
135 | num6.show
136 | val c6 = num6.count()
137 | println(s"test result count should be 11: $c6")
138 |
139 | //test "or". The number of results is 21
140 | val num7 = df.filter($"col0" <= -10 || $"col0" > 10)
141 | num7.show
142 | val c7 = num7.count()
143 | println(s"test result count should be 21: $c7")
144 |
145 | //test "all". The number of results is 32
146 | val num8 = df.filter($"col0" >= -100)
147 | num8.show
148 | val c8 = num8.count()
149 | println(s"test result count should be 32: $c8")
150 |
151 | //test "full query"
152 | val df1 = withCatalog(cat)
153 | df1.show()
154 | val c_df = df1.count()
155 | println(s"df count should be 32: $c_df")
156 | if(c_df != 32){
157 | throw new Exception("value invalid")
158 | }
159 |
160 | spark.stop()
161 | }
162 | }
163 |
--------------------------------------------------------------------------------
/core/src/test/scala/org/apache/spark/sql/AvroSourceKeySuite.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 | * File modified by Hortonworks, Inc. Modifications are also licensed under
18 | * the Apache Software License, Version 2.0.
19 | */
20 |
21 | package org.apache.spark.sql
22 |
23 | import org.apache.avro.Schema
24 | import org.apache.avro.generic.GenericData
25 | import org.apache.spark.sql.execution.datasources.hbase.Logging
26 | import org.apache.spark.sql.execution.datasources.hbase.types.AvroSerde
27 | import org.apache.spark.sql.execution.datasources.hbase.HBaseTableCatalog
28 |
29 | case class AvroHBaseKeyRecord(col0: Array[Byte], col1: Array[Byte])
30 |
31 | object AvroHBaseKeyRecord {
32 | val schemaString =
33 | s"""{"namespace": "example.avro",
34 | | "type": "record", "name": "User",
35 | | "fields": [ {"name": "name", "type": "string"},
36 | | {"name": "favorite_number", "type": ["int", "null"]},
37 | | {"name": "favorite_color", "type": ["string", "null"]} ] }""".stripMargin
38 |
39 | val avroSchema: Schema = {
40 | val p = new Schema.Parser
41 | p.parse(schemaString)
42 | }
43 |
44 | def apply(i: Int): AvroHBaseKeyRecord = {
45 | val user = new GenericData.Record(avroSchema);
46 | user.put("name", s"name${"%03d".format(i)}")
47 | user.put("favorite_number", i)
48 | user.put("favorite_color", s"color${"%03d".format(i)}")
49 | val avroByte = AvroSerde.serialize(user, avroSchema)
50 | AvroHBaseKeyRecord(avroByte, avroByte)
51 | }
52 | }
53 |
54 | class AvroSourceKeySuite extends SHC with Logging{
55 |
56 | // 'catalog' is used when saving data to HBase
57 | override def catalog = s"""{
58 | |"table":{"namespace":"default", "name":"avrotable", "tableCoder":"PrimitiveType"},
59 | |"rowkey":"key",
60 | |"columns":{
61 | |"col0":{"cf":"rowkey", "col":"key", "type":"binary"},
62 | |"col1":{"cf":"cf1", "col":"col1", "type":"binary"}
63 | |}
64 | |}""".stripMargin
65 |
66 | def avroCatalog = s"""{
67 | |"table":{"namespace":"default", "name":"avrotable", "tableCoder":"PrimitiveType"},
68 | |"rowkey":"key",
69 | |"columns":{
70 | |"col0":{"cf":"rowkey", "col":"key", "avro":"avroSchema"},
71 | |"col1":{"cf":"cf1", "col":"col1", "avro":"avroSchema"}
72 | |}
73 | |}""".stripMargin
74 |
75 | def avroCatalogInsert = s"""{
76 | |"table":{"namespace":"default", "name":"avrotableInsert", "tableCoder":"PrimitiveType"},
77 | |"rowkey":"key",
78 | |"columns":{
79 | |"col0":{"cf":"rowkey", "col":"key", "avro":"avroSchema"},
80 | |"col1":{"cf":"cf1", "col":"col1", "avro":"avroSchema"}
81 | |}
82 | |}""".stripMargin
83 |
84 |
85 | def withCatalog(cat: String): DataFrame = {
86 | sqlContext
87 | .read
88 | .options(Map("avroSchema" -> AvroHBaseKeyRecord.schemaString, HBaseTableCatalog.tableCatalog -> cat))
89 | .format("org.apache.spark.sql.execution.datasources.hbase")
90 | .load()
91 | }
92 |
93 | test("populate table") {
94 | //createTable(tableName, columnFamilies)
95 | val sql = sqlContext
96 | import sql.implicits._
97 |
98 | val data = (0 to 255).map { i =>
99 | AvroHBaseKeyRecord(i)
100 | }
101 | sc.parallelize(data).toDF.write.options(
102 | Map(HBaseTableCatalog.tableCatalog -> catalog, HBaseTableCatalog.newTable -> "5"))
103 | .format("org.apache.spark.sql.execution.datasources.hbase")
104 | .save()
105 | }
106 |
107 | test("empty column") {
108 | val df = withCatalog(avroCatalog)
109 | df.createOrReplaceTempView("avrotable")
110 | val c = sqlContext.sql("select count(1) from avrotable").rdd.collect()(0)(0).asInstanceOf[Long]
111 | assert(c == 256)
112 | }
113 |
114 | test("full query") {
115 | val df = withCatalog(avroCatalog)
116 | df.show
117 | df.printSchema()
118 | assert(df.count() == 256)
119 | }
120 |
121 | test("serialization and deserialization query") {
122 | val df = withCatalog(avroCatalog)
123 | df.write.options(
124 | Map("avroSchema"->AvroHBaseKeyRecord.schemaString, HBaseTableCatalog.tableCatalog->avroCatalogInsert,
125 | HBaseTableCatalog.newTable -> "5"))
126 | .format("org.apache.spark.sql.execution.datasources.hbase")
127 | .save()
128 | val newDF = withCatalog(avroCatalogInsert)
129 | newDF.show
130 | newDF.printSchema()
131 | assert(newDF.count() == 256)
132 | }
133 |
134 | test("filtered query") {
135 | val df = withCatalog(avroCatalog)
136 | val r = df.filter($"col1.name" === "name005" || $"col1.name" <= "name005")
137 | .select("col0", "col1.favorite_color", "col1.favorite_number")
138 | r.show
139 | assert(r.count() == 6)
140 | }
141 |
142 | test("Or filter") {
143 | val df = withCatalog(avroCatalog)
144 | val s = df.filter($"col1.name" <= "name005" || $"col1.name".contains("name007"))
145 | .select("col0", "col1.favorite_color", "col1.favorite_number")
146 | s.show
147 | assert(s.count() == 7)
148 | }
149 | }
150 |
--------------------------------------------------------------------------------
/examples/src/main/scala/org/apache/spark/sql/execution/datasources/hbase/CompositeKey.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * (C) 2017 Hortonworks, Inc. All rights reserved. See the NOTICE file
3 | * distributed with this work for additional information regarding copyright
4 | * ownership. This file is licensed to You under the Apache License, Version 2.0
5 | * (the "License"); you may not use this file except in compliance with
6 | * the License. You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 |
17 | package org.apache.spark.sql.execution.datasources.hbase.examples
18 |
19 | import org.apache.spark.sql.execution.datasources.hbase._
20 | import org.apache.spark.sql.{DataFrame, SparkSession}
21 |
22 | case class HBaseCompositeRecord(
23 | col00: String,
24 | col01: Int,
25 | col1: Boolean,
26 | col2: Double,
27 | col3: Float,
28 | col4: Int,
29 | col5: Long,
30 | col6: Short,
31 | col7: String,
32 | col8: Byte)
33 |
34 | object HBaseCompositeRecord {
35 | def apply(i: Int): HBaseCompositeRecord = {
36 | HBaseCompositeRecord(s"row${"%03d".format(i)}",
37 | if (i % 2 == 0) {
38 | i
39 | } else {
40 | -i
41 | },
42 | i % 2 == 0,
43 | i.toDouble,
44 | i.toFloat,
45 | i,
46 | i.toLong,
47 | i.toShort,
48 | s"String$i extra",
49 | i.toByte)
50 | }
51 | }
52 |
53 | object CompositeKey {
54 | def cat = s"""{
55 | |"table":{"namespace":"default", "name":"shcExampleTable", "tableCoder":"PrimitiveType"},
56 | |"rowkey":"key1:key2",
57 | |"columns":{
58 | |"col00":{"cf":"rowkey", "col":"key1", "type":"string", "length":"6"},
59 | |"col01":{"cf":"rowkey", "col":"key2", "type":"int"},
60 | |"col1":{"cf":"cf1", "col":"col1", "type":"boolean"},
61 | |"col2":{"cf":"cf2", "col":"col2", "type":"double"},
62 | |"col3":{"cf":"cf3", "col":"col3", "type":"float"},
63 | |"col4":{"cf":"cf4", "col":"col4", "type":"int"},
64 | |"col5":{"cf":"cf5", "col":"col5", "type":"bigint"},
65 | |"col6":{"cf":"cf6", "col":"col6", "type":"smallint"},
66 | |"col7":{"cf":"cf7", "col":"col7", "type":"string"},
67 | |"col8":{"cf":"cf8", "col":"col8", "type":"tinyint"}
68 | |}
69 | |}""".stripMargin
70 |
71 | def main(args: Array[String]){
72 | val spark = SparkSession.builder()
73 | .appName("CompositeKeyExample")
74 | .getOrCreate()
75 |
76 | val sc = spark.sparkContext
77 | val sqlContext = spark.sqlContext
78 |
79 | import sqlContext.implicits._
80 |
81 | def withCatalog(cat: String): DataFrame = {
82 | sqlContext
83 | .read
84 | .options(Map(HBaseTableCatalog.tableCatalog->cat))
85 | .format("org.apache.spark.sql.execution.datasources.hbase")
86 | .load()
87 | }
88 |
89 | //populate table with composite key
90 | val data = (0 to 255).map { i =>
91 | HBaseCompositeRecord(i)
92 | }
93 | sc.parallelize(data).toDF.write.options(
94 | Map(HBaseTableCatalog.tableCatalog -> cat, HBaseTableCatalog.newTable -> "5"))
95 | .format("org.apache.spark.sql.execution.datasources.hbase")
96 | .save()
97 |
98 | //full query
99 | val df = withCatalog(cat)
100 | df.show
101 | if(df.count() != 256){
102 | throw new Exception("value invalid")
103 | }
104 |
105 | // filtered query1
106 | val s = df.filter($"col00" <= "row050" && $"col01" > 40)
107 | .select("col00", "col01","col1")
108 | s.show
109 | if(s.count() != 5){
110 | throw new Exception("value invalid")
111 | }
112 |
113 | //filtered query2. The number of results is 6
114 | df.filter($"col00" <= "row050" && $"col01" >= 40)
115 | .select("col00", "col01","col1").show
116 |
117 | //filtered query3". The number of results is 3
118 | df.filter($"col00" >= "row250" && $"col01" < 50)
119 | .select("col00", "col01","col1").show
120 |
121 | //filtered query4. The number of results is 11
122 | df.filter($"col00" <= "row010") // row005 not included
123 | .select("col00", "col01","col1").show
124 |
125 | //filtered query5. The number of results is 1
126 | df.filter($"col00" === "row010") // row005 not included
127 | .select("col00", "col01","col1").show
128 |
129 | //filtered query51. The number of results is 1
130 | df.filter($"col00" === "row011") // row005 not included
131 | .select("col00", "col01","col1").show
132 |
133 | //filtered query52. The number of results is 1
134 | df.filter($"col00" === "row005") // row005 not included
135 | .select("col00", "col01","col1")
136 | .show
137 |
138 | //filtered query6. The number of results is 22
139 | df.filter(($"col00" <= "row050" && $"col00" > "row040") ||
140 | $"col00" === "row010" || // no included, since it is composite key
141 | $"col00" === "row020" || // not inlcuded
142 | $"col00" === "r20" || // not included
143 | $"col00" <= "row010") // row005 not included
144 | .select("col00", "col01","col1")
145 | .show
146 |
147 | //filtered query7. The number of results is 17
148 | df.filter(($"col00" <= "row050" && $"col00" > "row040") ||
149 | $"col00" === "row005" || // no included, since it is composite key
150 | $"col00" === "row020" || // not inlcuded
151 | $"col00" === "r20" || // not included
152 | $"col00" <= "row005") // row005 not included
153 | .select("col00", "col01","col1")
154 | .show
155 |
156 | spark.stop()
157 | }
158 | }
159 |
--------------------------------------------------------------------------------
/examples/src/main/scala/org/apache/spark/sql/execution/datasources/hbase/JoinTablesFrom2Clusters.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * (C) 2017 Hortonworks, Inc. All rights reserved. See the NOTICE file
3 | * distributed with this work for additional information regarding copyright
4 | * ownership. This file is licensed to You under the Apache License, Version 2.0
5 | * (the "License"); you may not use this file except in compliance with
6 | * the License. You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 |
17 | package org.apache.spark.sql.execution.datasources.hbase.examples
18 |
19 | import org.apache.spark.sql.execution.datasources.hbase.{HBaseRelation, HBaseTableCatalog}
20 | import org.apache.spark.sql.{DataFrame, SparkSession}
21 |
22 | case class JRecord(
23 | col0: String,
24 | col1: Boolean,
25 | col2: Double,
26 | col3: Float,
27 | col4: Int,
28 | col5: Long,
29 | col6: Short,
30 | col7: String,
31 | col8: Byte)
32 |
33 | object JRecord {
34 | def apply(i: Int): JRecord = {
35 | val s = s"""row${"%03d".format(i)}"""
36 | JRecord(s,
37 | i % 2 == 0,
38 | i.toDouble,
39 | i.toFloat,
40 | i,
41 | i.toLong,
42 | i.toShort,
43 | s"String$i extra",
44 | i.toByte)
45 | }
46 | }
47 |
48 | object JoinTablesFrom2Clusters {
49 | val cat1 = s"""{
50 | |"table":{"namespace":"default", "name":"shcExampleTable1", "tableCoder":"PrimitiveType"},
51 | |"rowkey":"key",
52 | |"columns":{
53 | |"col0":{"cf":"rowkey", "col":"key", "type":"string"},
54 | |"col1":{"cf":"cf1", "col":"col1", "type":"boolean"},
55 | |"col2":{"cf":"cf2", "col":"col2", "type":"double"},
56 | |"col3":{"cf":"cf3", "col":"col3", "type":"float"},
57 | |"col4":{"cf":"cf4", "col":"col4", "type":"int"},
58 | |"col5":{"cf":"cf5", "col":"col5", "type":"bigint"},
59 | |"col6":{"cf":"cf6", "col":"col6", "type":"smallint"},
60 | |"col7":{"cf":"cf7", "col":"col7", "type":"string"},
61 | |"col8":{"cf":"cf8", "col":"col8", "type":"tinyint"}
62 | |}
63 | |}""".stripMargin
64 |
65 | val cat2 = s"""{
66 | |"table":{"namespace":"default", "name":"shcExampleTable2", "tableCoder":"PrimitiveType"},
67 | |"rowkey":"key",
68 | |"columns":{
69 | |"col0":{"cf":"rowkey", "col":"key", "type":"string"},
70 | |"col1":{"cf":"cf1", "col":"col1", "type":"boolean"},
71 | |"col2":{"cf":"cf2", "col":"col2", "type":"double"},
72 | |"col3":{"cf":"cf3", "col":"col3", "type":"float"},
73 | |"col4":{"cf":"cf4", "col":"col4", "type":"int"},
74 | |"col5":{"cf":"cf5", "col":"col5", "type":"bigint"},
75 | |"col6":{"cf":"cf6", "col":"col6", "type":"smallint"},
76 | |"col7":{"cf":"cf7", "col":"col7", "type":"string"},
77 | |"col8":{"cf":"cf8", "col":"col8", "type":"tinyint"}
78 | |}
79 | |}""".stripMargin
80 |
81 |
82 | def main(args: Array[String]): Unit = {
83 | if (args.length < 2) {
84 | System.err.println("Usage: JoinTablesFrom2Clusters ")
85 | System.exit(1)
86 | }
87 |
88 | // configuration file of HBase cluster
89 | val conf1 = args(0)
90 | val conf2 = args(1)
91 |
92 | val spark = SparkSession.builder()
93 | .appName("JoinTablesFrom2Clusters")
94 | .getOrCreate()
95 |
96 | val sc = spark.sparkContext
97 | val sqlContext = spark.sqlContext
98 |
99 | import sqlContext.implicits._
100 |
101 | def withCatalog(cat: String, conf: String): DataFrame = {
102 | sqlContext
103 | .read
104 | .options(Map(HBaseTableCatalog.tableCatalog->cat, HBaseRelation.HBASE_CONFIGFILE -> conf))
105 | .format("org.apache.spark.sql.execution.datasources.hbase")
106 | .load()
107 | }
108 |
109 | def saveData(cat: String, conf: String, data: Seq[JRecord]) = {
110 | sc.parallelize(data).toDF.write
111 | .options(Map(HBaseTableCatalog.tableCatalog -> cat,
112 | HBaseRelation.HBASE_CONFIGFILE -> conf, HBaseTableCatalog.newTable -> "5"))
113 | .format("org.apache.spark.sql.execution.datasources.hbase")
114 | .save()
115 | }
116 |
117 | // data saved into cluster 1
118 | val data1 = (0 to 120).map { i =>
119 | JRecord(i)
120 | }
121 | saveData(cat1, conf1, data1)
122 |
123 | // data saved into cluster 2
124 | val data2 = (100 to 200).map { i =>
125 | JRecord(i)
126 | }
127 | saveData(cat2, conf2, data2)
128 |
129 | val df1 = withCatalog(cat1, conf1)
130 | val df2 = withCatalog(cat2, conf2)
131 | val s1 = df1.filter($"col0" <= "row120" && $"col0" > "row090").select("col0", "col2")
132 | val s2 = df2.filter($"col0" <= "row150" && $"col0" > "row100").select("col0", "col5")
133 | val result = s1.join(s2, Seq("col0"))
134 |
135 | result.sort($"col0".asc, $"col2", $"col5").show() // should be row101 to row120, as following:
136 | /*+------+-----+----+
137 | | col0| col2|col5|
138 | +------+-----+----+
139 | |row101|101.0| 101|
140 | |row102|102.0| 102|
141 | |row103|103.0| 103|
142 | |row104|104.0| 104|
143 | |row105|105.0| 105|
144 | |row106|106.0| 106|
145 | |row107|107.0| 107|
146 | |row108|108.0| 108|
147 | |row109|109.0| 109|
148 | |row110|110.0| 110|
149 | |row111|111.0| 111|
150 | |row112|112.0| 112|
151 | |row113|113.0| 113|
152 | |row114|114.0| 114|
153 | |row115|115.0| 115|
154 | |row116|116.0| 116|
155 | |row117|117.0| 117|
156 | |row118|118.0| 118|
157 | |row119|119.0| 119|
158 | |row120|120.0| 120|
159 | +------+-----+----+ */
160 |
161 | sc.stop()
162 | }
163 | }
164 |
--------------------------------------------------------------------------------
/core/src/test/scala/org/apache/spark/sql/CompositeKeySuite.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * (C) 2017 Hortonworks, Inc. All rights reserved. See the NOTICE file
3 | * distributed with this work for additional information regarding copyright
4 | * ownership. This file is licensed to You under the Apache License, Version 2.0
5 | * (the "License"); you may not use this file except in compliance with
6 | * the License. You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 |
17 | package org.apache.spark.sql
18 |
19 | import org.apache.spark.sql.execution.datasources.hbase.Logging
20 | import org.apache.spark.sql.execution.datasources.hbase.HBaseTableCatalog
21 |
22 | case class HBaseCompositeRecord(
23 | col00: String,
24 | col01: Int,
25 | col1: Boolean,
26 | col2: Double,
27 | col3: Float,
28 | col4: Int,
29 | col5: Long,
30 | col6: Short,
31 | col7: String,
32 | col8: Byte)
33 |
34 | object HBaseCompositeRecord {
35 | def apply(i: Int): HBaseCompositeRecord = {
36 | HBaseCompositeRecord(s"row${"%03d".format(i)}",
37 | if (i % 2 == 0) {
38 | i
39 | } else {
40 | -i
41 | },
42 | i % 2 == 0,
43 | i.toDouble,
44 | i.toFloat,
45 | i,
46 | i.toLong,
47 | i.toShort,
48 | s"String$i extra",
49 | i.toByte)
50 | }
51 | }
52 |
53 | class CompositeKeySuite extends SHC with Logging {
54 |
55 | override def catalog = s"""{
56 | |"table":{"namespace":"default", "name":"table1", "tableCoder":"PrimitiveType"},
57 | |"rowkey":"key1:key2",
58 | |"columns":{
59 | |"col00":{"cf":"rowkey", "col":"key1", "type":"string", "length":"6"},
60 | |"col01":{"cf":"rowkey", "col":"key2", "type":"int"},
61 | |"col1":{"cf":"cf1", "col":"col1", "type":"boolean"},
62 | |"col2":{"cf":"cf2", "col":"col2", "type":"double"},
63 | |"col3":{"cf":"cf3", "col":"col3", "type":"float"},
64 | |"col4":{"cf":"cf4", "col":"col4", "type":"int"},
65 | |"col5":{"cf":"cf5", "col":"col5", "type":"bigint"},
66 | |"col6":{"cf":"cf6", "col":"col6", "type":"smallint"},
67 | |"col7":{"cf":"cf7", "col":"col7", "type":"string"},
68 | |"col8":{"cf":"cf8", "col":"col8", "type":"tinyint"}
69 | |}
70 | |}""".stripMargin
71 |
72 | def withCatalog(cat: String): DataFrame = {
73 | sqlContext
74 | .read
75 | .options(Map(HBaseTableCatalog.tableCatalog->cat))
76 | .format("org.apache.spark.sql.execution.datasources.hbase")
77 | .load()
78 | }
79 |
80 | test("populate table with composite key") {
81 | //createTable(tableName, columnFamilies)
82 | val sql = sqlContext
83 | import sql.implicits._
84 |
85 | val data = (0 to 255).map { i =>
86 | HBaseCompositeRecord(i)
87 | }
88 | sc.parallelize(data).toDF.write.options(
89 | Map(HBaseTableCatalog.tableCatalog -> catalog, HBaseTableCatalog.newTable -> "5"))
90 | .format("org.apache.spark.sql.execution.datasources.hbase")
91 | .save()
92 | }
93 |
94 | test("full query") {
95 | val df = withCatalog(catalog)
96 | df.show
97 | assert(df.count() == 256)
98 | }
99 |
100 | test("filtered query1") {
101 | val df = withCatalog(catalog)
102 | val s = df.filter($"col00" <= "row050" && $"col01" > 40)
103 | .select("col00", "col01","col1")
104 | s.show
105 | assert(s.count() == 5)
106 | }
107 |
108 | test("filtered query2") {
109 | val df = withCatalog(catalog)
110 | val s = df.filter($"col00" <= "row050" && $"col01" >= 40)
111 | .select("col00", "col01","col1")
112 | s.show
113 | assert(s.count() == 6)
114 | }
115 |
116 | test("filtered query3") {
117 | val df = withCatalog(catalog)
118 | val s = df.filter($"col00" >= "row250" && $"col01" < 50)
119 | .select("col00", "col01","col1")
120 | s.show
121 | assert(s.count() == 3)
122 | }
123 |
124 | test("filtered query4") {
125 | val df = withCatalog(catalog)
126 | val s = df.filter($"col00" <= "row010") // row005 not included
127 | .select("col00", "col01","col1")
128 | s.show
129 | assert(s.count() == 11)
130 | }
131 |
132 | test("filtered query5") {
133 | val df = withCatalog(catalog)
134 | val s = df.filter($"col00" === "row010") // row005 not included
135 | .select("col00", "col01","col1")
136 | s.show
137 | assert(s.count() == 1)
138 | }
139 | test("filtered query51") {
140 | val df = withCatalog(catalog)
141 | val s = df.filter($"col00" === "row011") // row005 not included
142 | .select("col00", "col01","col1")
143 | s.show
144 | assert(s.count() == 1)
145 | }
146 |
147 | test("filtered query52") {
148 | val df = withCatalog(catalog)
149 | val s = df.filter($"col00" === "row005") // row005 not included
150 | .select("col00", "col01","col1")
151 | s.show
152 | assert(s.count() == 1)
153 | }
154 |
155 |
156 | test("filtered query6") {
157 | val df = withCatalog(catalog)
158 | val s = df.filter(($"col00" <= "row050" && $"col00" > "row040") ||
159 | $"col00" === "row010" || // no included, since it is composite key
160 | $"col00" === "row020" || // not inlcuded
161 | $"col00" === "r20" || // not included
162 | $"col00" <= "row010") // row005 not included
163 | .select("col00", "col01","col1")
164 | s.show
165 | assert(s.count() == 22)
166 | }
167 |
168 |
169 | test("filtered query7") {
170 | val df = withCatalog(catalog)
171 | val s = df.filter(($"col00" <= "row050" && $"col00" > "row040") ||
172 | $"col00" === "row005" || // no included, since it is composite key
173 | $"col00" === "row020" || // not inlcuded
174 | $"col00" === "r20" || // not included
175 | $"col00" <= "row005") // row005 not included
176 | .select("col00", "col01","col1")
177 | s.show
178 | assert(s.count() == 17)
179 | }
180 | }
181 |
--------------------------------------------------------------------------------
/core/src/test/scala/org/apache/spark/sql/PhoenixCompositeKeySuite.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * (C) 2017 Hortonworks, Inc. All rights reserved. See the NOTICE file
3 | * distributed with this work for additional information regarding copyright
4 | * ownership. This file is licensed to You under the Apache License, Version 2.0
5 | * (the "License"); you may not use this file except in compliance with
6 | * the License. You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 |
17 | package org.apache.spark.sql
18 |
19 | import org.apache.spark.sql.execution.datasources.hbase.Logging
20 | import org.apache.spark.sql.execution.datasources.hbase.HBaseTableCatalog
21 |
22 | case class PCompositeRecord(
23 | col00: String,
24 | col01: Int,
25 | col1: Boolean,
26 | col2: Double,
27 | col3: Float,
28 | col4: Int,
29 | col5: Long,
30 | col6: Short,
31 | col7: String,
32 | col8: Byte)
33 |
34 | object PCompositeRecord {
35 | def apply(i: Int): PCompositeRecord = {
36 | PCompositeRecord(s"row${"%03d".format(i)}",
37 | if (i % 2 == 0) {
38 | i
39 | } else {
40 | -i
41 | },
42 | i % 2 == 0,
43 | i.toDouble,
44 | i.toFloat,
45 | i,
46 | i.toLong,
47 | i.toShort,
48 | s"String$i extra",
49 | i.toByte)
50 | }
51 | }
52 |
53 | class PhoenixCompositeKeySuite extends SHC with Logging {
54 | override def catalog = s"""{
55 | |"table":{"namespace":"default", "name":"table1", "tableCoder":"Phoenix"},
56 | |"rowkey":"key1:key2",
57 | |"columns":{
58 | |"col00":{"cf":"rowkey", "col":"key1", "type":"string"},
59 | |"col01":{"cf":"rowkey", "col":"key2", "type":"int"},
60 | |"col1":{"cf":"cf1", "col":"col1", "type":"boolean"},
61 | |"col2":{"cf":"cf2", "col":"col2", "type":"double"},
62 | |"col3":{"cf":"cf3", "col":"col3", "type":"float"},
63 | |"col4":{"cf":"cf4", "col":"col4", "type":"int"},
64 | |"col5":{"cf":"cf5", "col":"col5", "type":"bigint"},
65 | |"col6":{"cf":"cf6", "col":"col6", "type":"smallint"},
66 | |"col7":{"cf":"cf7", "col":"col7", "type":"string"},
67 | |"col8":{"cf":"cf8", "col":"col8", "type":"tinyint"}
68 | |}
69 | |}""".stripMargin
70 |
71 | def withCatalog(cat: String): DataFrame = {
72 | sqlContext
73 | .read
74 | .options(Map(HBaseTableCatalog.tableCatalog->cat))
75 | .format("org.apache.spark.sql.execution.datasources.hbase")
76 | .load()
77 | }
78 |
79 | test("populate table with composite key") {
80 | //createTable(tableName, columnFamilies)
81 | val sql = sqlContext
82 | import sql.implicits._
83 |
84 | val data = (0 to 255).map { i =>
85 | HBaseCompositeRecord(i)
86 | }
87 | sc.parallelize(data).toDF.write.options(
88 | Map(HBaseTableCatalog.tableCatalog -> catalog, HBaseTableCatalog.newTable -> "5"))
89 | .format("org.apache.spark.sql.execution.datasources.hbase")
90 | .save()
91 | }
92 |
93 | test("full query") {
94 | val df = withCatalog(catalog)
95 | df.show
96 | assert(df.count() == 256)
97 | }
98 |
99 | test("filtered query1") {
100 | val df = withCatalog(catalog)
101 | val s = df.filter($"col00" <= "row050" && $"col01" > 40)
102 | .select("col00", "col01","col1")
103 | s.show
104 | assert(s.count() == 5)
105 | }
106 |
107 | test("filtered query2") {
108 | val df = withCatalog(catalog)
109 | val s = df.filter($"col00" <= "row050" && $"col01" >= 40)
110 | .select("col00", "col01","col1")
111 | s.show
112 | assert(s.count() == 6)
113 | }
114 |
115 | test("filtered query3") {
116 | val df = withCatalog(catalog)
117 | val s = df.filter($"col00" >= "row250" && $"col01" < 50)
118 | .select("col00", "col01","col1")
119 | s.show
120 | assert(s.count() == 3)
121 | }
122 |
123 | test("filtered query4") {
124 | val df = withCatalog(catalog)
125 | val s = df.filter($"col00" <= "row010") // row005 not included
126 | .select("col00", "col01","col1")
127 | s.show
128 | assert(s.count() == 11)
129 | }
130 |
131 | test("filtered query5") {
132 | val df = withCatalog(catalog)
133 | val s = df.filter($"col00" === "row010") // row005 not included
134 | .select("col00", "col01","col1")
135 | s.show
136 | assert(s.count() == 1)
137 | }
138 | test("filtered query51") {
139 | val df = withCatalog(catalog)
140 | val s = df.filter($"col00" === "row011") // row005 not included
141 | .select("col00", "col01","col1")
142 | s.show
143 | assert(s.count() == 1)
144 | }
145 |
146 | test("filtered query52") {
147 | val df = withCatalog(catalog)
148 | val s = df.filter($"col00" === "row005") // row005 not included
149 | .select("col00", "col01","col1")
150 | s.show
151 | assert(s.count() == 1)
152 | }
153 |
154 |
155 | test("filtered query6") {
156 | val df = withCatalog(catalog)
157 | val s = df.filter(($"col00" <= "row050" && $"col00" > "row040") ||
158 | $"col00" === "row010" || // no included, since it is composite key
159 | $"col00" === "row020" || // not inlcuded
160 | $"col00" === "r20" || // not included
161 | $"col00" <= "row010") // row005 not included
162 | .select("col00", "col01","col1")
163 | s.show
164 | assert(s.count() == 22)
165 | }
166 |
167 |
168 | test("filtered query7") {
169 | val df = withCatalog(catalog)
170 | val s = df.filter(($"col00" <= "row050" && $"col00" > "row040") ||
171 | $"col00" === "row005" || // no included, since it is composite key
172 | $"col00" === "row020" || // not inlcuded
173 | $"col00" === "r20" || // not included
174 | $"col00" <= "row005") // row005 not included
175 | .select("col00", "col01","col1")
176 | s.show
177 | assert(s.count() == 17)
178 | }
179 | }
180 |
--------------------------------------------------------------------------------
/examples/src/main/scala/org/apache/spark/sql/execution/datasources/hbase/LRJobAccessing2Clusters.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * (C) 2017 Hortonworks, Inc. All rights reserved. See the NOTICE file
3 | * distributed with this work for additional information regarding copyright
4 | * ownership. This file is licensed to You under the Apache License, Version 2.0
5 | * (the "License"); you may not use this file except in compliance with
6 | * the License. You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 |
17 | package org.apache.spark.sql.execution.datasources.hbase.examples
18 |
19 | import org.apache.spark.sql.{DataFrame, SparkSession}
20 | import org.apache.spark.sql.execution.datasources.hbase.{HBaseRelation, HBaseTableCatalog}
21 |
22 | case class LRecord(
23 | col0: String,
24 | col1: Boolean,
25 | col2: Double,
26 | col3: Float,
27 | col4: Int,
28 | col5: Long,
29 | col6: Short,
30 | col7: String,
31 | col8: Byte)
32 |
33 | object LRecord {
34 | def apply(i: Int): LRecord = {
35 | val s = s"""row${"%03d".format(i)}"""
36 | LRecord(s,
37 | i % 2 == 0,
38 | i.toDouble,
39 | i.toFloat,
40 | i,
41 | i.toLong,
42 | i.toShort,
43 | s"String$i extra",
44 | i.toByte)
45 | }
46 | }
47 |
48 | // long running job to access 2 HBase clusters
49 | object LRJobAccessing2Clusters {
50 | val cat1 = s"""{
51 | |"table":{"namespace":"default", "name":"shcExampleTable1", "tableCoder":"PrimitiveType"},
52 | |"rowkey":"key",
53 | |"columns":{
54 | |"col0":{"cf":"rowkey", "col":"key", "type":"string"},
55 | |"col1":{"cf":"cf1", "col":"col1", "type":"boolean"},
56 | |"col2":{"cf":"cf2", "col":"col2", "type":"double"},
57 | |"col3":{"cf":"cf3", "col":"col3", "type":"float"},
58 | |"col4":{"cf":"cf4", "col":"col4", "type":"int"},
59 | |"col5":{"cf":"cf5", "col":"col5", "type":"bigint"},
60 | |"col6":{"cf":"cf6", "col":"col6", "type":"smallint"},
61 | |"col7":{"cf":"cf7", "col":"col7", "type":"string"},
62 | |"col8":{"cf":"cf8", "col":"col8", "type":"tinyint"}
63 | |}
64 | |}""".stripMargin
65 |
66 | val cat2 = s"""{
67 | |"table":{"namespace":"default", "name":"shcExampleTable2", "tableCoder":"PrimitiveType"},
68 | |"rowkey":"key",
69 | |"columns":{
70 | |"col0":{"cf":"rowkey", "col":"key", "type":"string"},
71 | |"col1":{"cf":"cf1", "col":"col1", "type":"boolean"},
72 | |"col2":{"cf":"cf2", "col":"col2", "type":"double"},
73 | |"col3":{"cf":"cf3", "col":"col3", "type":"float"},
74 | |"col4":{"cf":"cf4", "col":"col4", "type":"int"},
75 | |"col5":{"cf":"cf5", "col":"col5", "type":"bigint"},
76 | |"col6":{"cf":"cf6", "col":"col6", "type":"smallint"},
77 | |"col7":{"cf":"cf7", "col":"col7", "type":"string"},
78 | |"col8":{"cf":"cf8", "col":"col8", "type":"tinyint"}
79 | |}
80 | |}""".stripMargin
81 |
82 |
83 | def main(args: Array[String]): Unit = {
84 | if (args.length < 2) {
85 | System.err.println("Usage: LRJobAccessing2Clusters [sleepTime]")
86 | System.exit(1)
87 | }
88 |
89 | // configuration file of HBase cluster
90 | val conf1 = args(0)
91 | val conf2 = args(1)
92 | val sleepTime = if (args.length > 2) args(2).toLong else 2 * 60 * 1000 // sleep 2 min by default
93 |
94 | val spark = SparkSession.builder()
95 | .appName("LRJobAccessing2Clusters")
96 | .getOrCreate()
97 |
98 | val sc = spark.sparkContext
99 | val sqlContext = spark.sqlContext
100 |
101 | import sqlContext.implicits._
102 |
103 | def withCatalog(cat: String, conf: String): DataFrame = {
104 | sqlContext
105 | .read
106 | .options(Map(HBaseTableCatalog.tableCatalog->cat, HBaseRelation.HBASE_CONFIGFILE -> conf))
107 | .format("org.apache.spark.sql.execution.datasources.hbase")
108 | .load()
109 | }
110 |
111 | def saveData(cat: String, conf: String, data: Seq[LRecord]) = {
112 | sc.parallelize(data).toDF.write
113 | .options(Map(HBaseTableCatalog.tableCatalog -> cat,
114 | HBaseRelation.HBASE_CONFIGFILE -> conf, HBaseTableCatalog.newTable -> "5"))
115 | .format("org.apache.spark.sql.execution.datasources.hbase")
116 | .save()
117 | }
118 |
119 | val timeEnd = System.currentTimeMillis() + (25 * 60 * 60 * 1000) // 25h later
120 | while (System.currentTimeMillis() < timeEnd) {
121 | // data saved into cluster 1
122 | val data1 = (0 to 120).map { i =>
123 | LRecord(i)
124 | }
125 | saveData(cat1, conf1, data1)
126 |
127 | // data saved into cluster 2
128 | val data2 = (100 to 200).map { i =>
129 | LRecord(i)
130 | }
131 | saveData(cat2, conf2, data2)
132 |
133 | val df1 = withCatalog(cat1, conf1)
134 | val df2 = withCatalog(cat2, conf2)
135 | val s1 = df1.filter($"col0" <= "row120" && $"col0" > "row090").select("col0", "col2")
136 | val s2 = df2.filter($"col0" <= "row150" && $"col0" > "row100").select("col0", "col5")
137 | val result = s1.join(s2, Seq("col0"))
138 |
139 | result.sort($"col0".asc, $"col2", $"col5").show() // should be row101 to row120, as following:
140 | /*+------+-----+----+
141 | | col0| col2|col5|
142 | +------+-----+----+
143 | |row101|101.0| 101|
144 | |row102|102.0| 102|
145 | |row103|103.0| 103|
146 | |row104|104.0| 104|
147 | |row105|105.0| 105|
148 | |row106|106.0| 106|
149 | |row107|107.0| 107|
150 | |row108|108.0| 108|
151 | |row109|109.0| 109|
152 | |row110|110.0| 110|
153 | |row111|111.0| 111|
154 | |row112|112.0| 112|
155 | |row113|113.0| 113|
156 | |row114|114.0| 114|
157 | |row115|115.0| 115|
158 | |row116|116.0| 116|
159 | |row117|117.0| 117|
160 | |row118|118.0| 118|
161 | |row119|119.0| 119|
162 | |row120|120.0| 120|
163 | +------+-----+----+ */
164 |
165 | Thread.sleep(sleepTime)
166 | }
167 | sc.stop()
168 | }
169 | }
170 |
--------------------------------------------------------------------------------
/examples/src/main/scala/org/apache/spark/sql/execution/datasources/hbase/AvroSource.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 | * File modified by Hortonworks, Inc. Modifications are also licensed under
18 | * the Apache Software License, Version 2.0.
19 | */
20 |
21 | package org.apache.spark.sql.execution.datasources.hbase.examples
22 |
23 | import org.apache.avro.Schema
24 | import org.apache.avro.generic.GenericData
25 | import org.apache.spark.sql._
26 | import org.apache.spark.sql.execution.datasources.hbase.HBaseTableCatalog
27 | import org.apache.spark.sql.execution.datasources.hbase.types.AvroSerde
28 |
29 | case class AvroHBaseRecord(col0: String,
30 | col1: Array[Byte])
31 |
32 | object AvroHBaseRecord {
33 | val schemaString =
34 | s"""{"namespace": "example.avro",
35 | | "type": "record", "name": "User",
36 | | "fields": [
37 | | {"name": "name", "type": "string"},
38 | | {"name": "favorite_number", "type": ["int", "null"]},
39 | | {"name": "favorite_color", "type": ["string", "null"]},
40 | | {"name": "favorite_array", "type": {"type": "array", "items": "string"}},
41 | | {"name": "favorite_map", "type": {"type": "map", "values": "int"}}
42 | | ] }""".stripMargin
43 |
44 | val avroSchema: Schema = {
45 | val p = new Schema.Parser
46 | p.parse(schemaString)
47 | }
48 |
49 | def apply(i: Int): AvroHBaseRecord = {
50 |
51 | val user = new GenericData.Record(avroSchema)
52 | user.put("name", s"name${"%03d".format(i)}")
53 | user.put("favorite_number", i)
54 | user.put("favorite_color", s"color${"%03d".format(i)}")
55 | val favoriteArray = new GenericData.Array[String](2, avroSchema.getField("favorite_array").schema())
56 | favoriteArray.add(s"number${i}")
57 | favoriteArray.add(s"number${i+1}")
58 | user.put("favorite_array", favoriteArray)
59 | import collection.JavaConverters._
60 | val favoriteMap = Map[String, Int](("key1" -> i), ("key2" -> (i+1))).asJava
61 | user.put("favorite_map", favoriteMap)
62 | val avroByte = AvroSerde.serialize(user, avroSchema)
63 | AvroHBaseRecord(s"name${"%03d".format(i)}", avroByte)
64 | }
65 | }
66 |
67 | object AvroSource {
68 | def catalog = s"""{
69 | |"table":{"namespace":"default", "name":"shcExampleAvrotable", "tableCoder":"PrimitiveType"},
70 | |"rowkey":"key",
71 | |"columns":{
72 | |"col0":{"cf":"rowkey", "col":"key", "type":"string"},
73 | |"col1":{"cf":"cf1", "col":"col1", "type":"binary"}
74 | |}
75 | |}""".stripMargin
76 |
77 | def avroCatalog = s"""{
78 | |"table":{"namespace":"default", "name":"shcExampleAvrotable", "tableCoder":"PrimitiveType"},
79 | |"rowkey":"key",
80 | |"columns":{
81 | |"col0":{"cf":"rowkey", "col":"key", "type":"string"},
82 | |"col1":{"cf":"cf1", "col":"col1", "avro":"avroSchema"}
83 | |}
84 | |}""".stripMargin
85 |
86 | def avroCatalogInsert = s"""{
87 | |"table":{"namespace":"default", "name":"shcExampleAvrotableInsert", "tableCoder":"PrimitiveType"},
88 | |"rowkey":"key",
89 | |"columns":{
90 | |"col0":{"cf":"rowkey", "col":"key", "type":"string"},
91 | |"col1":{"cf":"cf1", "col":"col1", "avro":"avroSchema"}
92 | |}
93 | |}""".stripMargin
94 |
95 | def main(args: Array[String]) {
96 | val spark = SparkSession.builder()
97 | .appName("AvroExample")
98 | .getOrCreate()
99 |
100 | val sc = spark.sparkContext
101 | val sqlContext = spark.sqlContext
102 |
103 | import sqlContext.implicits._
104 |
105 | def withCatalog(cat: String): DataFrame = {
106 | sqlContext
107 | .read
108 | .options(Map("avroSchema" -> AvroHBaseRecord.schemaString, HBaseTableCatalog.tableCatalog -> avroCatalog))
109 | .format("org.apache.spark.sql.execution.datasources.hbase")
110 | .load()
111 | }
112 |
113 | val data = (0 to 255).map { i =>
114 | AvroHBaseRecord(i)
115 | }
116 |
117 | sc.parallelize(data).toDF.write.options(
118 | Map(HBaseTableCatalog.tableCatalog -> catalog, HBaseTableCatalog.newTable -> "5"))
119 | .format("org.apache.spark.sql.execution.datasources.hbase")
120 | .save()
121 |
122 | val df = withCatalog(catalog)
123 | df.show
124 | df.printSchema()
125 | df.createOrReplaceTempView("shcExampleAvrotable")
126 | val c = sqlContext.sql("select count(1) from shcExampleAvrotable")
127 | c.show
128 |
129 | val filtered = df.select($"col0", $"col1.favorite_array").where($"col0" === "name001")
130 | filtered.show
131 | val collected = filtered.collect()
132 | if (collected(0).getSeq[String](1)(0) != "number1") {
133 | throw new Exception("value invalid")
134 | }
135 | if (collected(0).getSeq[String](1)(1) != "number2") {
136 | throw new Exception("value invalid")
137 | }
138 |
139 | df.write.options(
140 | Map("avroSchema"->AvroHBaseRecord.schemaString, HBaseTableCatalog.tableCatalog->avroCatalogInsert,
141 | HBaseTableCatalog.newTable -> "5"))
142 | .format("org.apache.spark.sql.execution.datasources.hbase")
143 | .save()
144 | val newDF = withCatalog(avroCatalogInsert)
145 | newDF.show
146 | newDF.printSchema()
147 | if(newDF.count() != 256) {
148 | throw new Exception("value invalid")
149 | }
150 |
151 | df.filter($"col1.name" === "name005" || $"col1.name" <= "name005")
152 | .select("col0", "col1.favorite_color", "col1.favorite_number")
153 | .show
154 |
155 | df.filter($"col1.name" <= "name005" || $"col1.name".contains("name007"))
156 | .select("col0", "col1.favorite_color", "col1.favorite_number")
157 | .show
158 |
159 | spark.stop()
160 | }
161 | }
162 |
--------------------------------------------------------------------------------
/pom.xml:
--------------------------------------------------------------------------------
1 |
2 |
18 |
19 |
20 |
21 |
23 | 4.0.0
24 |
25 | org.apache
26 | apache
27 | 14
28 |
29 | com.hortonworks
30 | shc
31 | 1.1.3-2.4-s_2.11
32 | pom
33 | HBase Spark Connector Project Parent POM
34 | https://github.com/hortonworks-spark/shc
35 |
36 |
37 |
38 | ${repoid}
39 | ${reponame}
40 | ${repourl}
41 |
42 |
43 |
44 |
45 | 3.5.4
46 | 2.4.0
47 | 2.0.4
48 | 5.0.0-HBase-2.0
49 | ${project.build.directory}/spark-test-classpath.txt
50 | 1.8
51 | 2.11.12
52 | 2.1.0
53 | 2.11
54 | 1.8.2
55 | 3.0.0
56 | 1.8.8
57 |
58 | 64m
59 | 512m
60 | 512m
61 |
62 |
63 |
64 | core
65 | examples
66 |
67 |
68 |
69 |
70 | central
71 |
72 | Maven Repository
73 | https://repo1.maven.org/maven2
74 |
75 | true
76 |
77 |
78 | false
79 |
80 |
81 |
82 | apache-repo
83 | Apache Repository
84 | https://repository.apache.org/content/repositories/releases
85 |
86 | true
87 |
88 |
89 | false
90 |
91 |
92 |
93 |
94 |
95 |
96 | org.scala-lang
97 | scala-library
98 | ${scala.version}
99 |
100 |
101 |
102 |
103 | target/scala-${scala.binary.version}/classes
104 | target/scala-${scala.binary.version}/test-classes
105 |
106 |
107 | ${basedir}/conf
108 | false
109 |
110 | hbase-site.xml
111 |
112 |
113 |
114 |
115 |
116 | net.alchim31.maven
117 | scala-maven-plugin
118 | 3.2.2
119 |
120 |
121 | scala-compile-first
122 | process-resources
123 |
124 | compile
125 |
126 |
127 |
128 | scala-test-compile-first
129 | process-test-resources
130 |
131 | testCompile
132 |
133 |
134 |
135 | attach-scaladocs
136 | verify
137 |
138 | doc-jar
139 |
140 |
141 |
142 |
143 | ${scala.binary.version}
144 | ${scala.version}
145 | incremental
146 | false
147 |
148 | -unchecked
149 | -deprecation
150 | -feature
151 |
152 |
153 | -XX:PermSize=${PermGen}
154 | -XX:MaxPermSize=${MaxPermGen}
155 | -XX:ReservedCodeCacheSize=${CodeCacheSize}
156 |
157 |
158 | -source
159 | ${java.version}
160 | -target
161 | ${java.version}
162 |
163 |
165 |
166 |
167 | org.scalamacros
168 | paradise_${scala.version}
169 | ${scala.macros.version}
170 |
171 |
172 |
173 |
174 |
175 | org.apache.maven.plugins
176 | maven-compiler-plugin
177 | 3.7.0
178 |
179 | 1.8
180 | 1.8
181 |
182 |
183 |
184 |
185 |
186 |
187 |
188 | scala-2.11
189 |
190 | true
191 |
192 |
193 |
194 |
195 |
196 |
--------------------------------------------------------------------------------
/core/pom.xml:
--------------------------------------------------------------------------------
1 |
2 |
18 |
19 |
20 |
21 |
23 | 4.0.0
24 |
25 | com.hortonworks
26 | shc
27 | 1.1.3-2.4-s_2.11
28 | ../pom.xml
29 |
30 |
31 | shc-core
32 | 1.1.3-2.4-s_2.11
33 | jar
34 | HBase Spark Connector Project Core
35 |
36 |
37 |
38 | org.apache.spark
39 | spark-core_${scala.binary.version}
40 | ${spark.version}
41 |
42 |
43 | org.apache.spark
44 | spark-catalyst_${scala.binary.version}
45 | ${spark.version}
46 |
47 |
48 | org.apache.spark
49 | spark-sql_${scala.binary.version}
50 | ${spark.version}
51 |
52 |
53 | org.apache.hbase
54 | hbase-server
55 | ${hbase.version}
56 |
57 |
58 | asm
59 | asm
60 |
61 |
62 | org.jboss.netty
63 | netty
64 |
65 |
66 | io.netty
67 | netty
68 |
69 |
70 | commons-logging
71 | commons-logging
72 |
73 |
74 | org.jruby
75 | jruby-complete
76 |
77 |
78 |
79 |
80 | org.apache.hbase
81 | hbase-common
82 | ${hbase.version}
83 |
84 |
85 | asm
86 | asm
87 |
88 |
89 | org.jboss.netty
90 | netty
91 |
92 |
93 | io.netty
94 | netty
95 |
96 |
97 | commons-logging
98 | commons-logging
99 |
100 |
101 | org.jruby
102 | jruby-complete
103 |
104 |
105 |
106 |
107 | org.apache.hbase
108 | hbase-mapreduce
109 | ${hbase.version}
110 |
111 |
112 | org.apache.phoenix
113 | phoenix-core
114 | ${phoenix.version}
115 |
116 |
117 | org.apache.hbase
118 | hbase-testing-util
119 | test-jar
120 | ${hbase.version}
121 | test
122 |
123 |
124 | org.apache.avro
125 | avro
126 | ${avro.version}
127 |
128 |
129 | org.scalatest
130 | scalatest_${scala.binary.version}
131 | 2.2.1
132 | test
133 |
134 |
135 |
136 |
137 |
138 |
139 |
140 | org.scalatest
141 | scalatest-maven-plugin
142 | 1.0
143 |
144 |
145 | ${project.build.directory}/surefire-reports
146 | .
147 | SparkTestSuite.txt
148 | -ea -Xmx3g -XX:MaxPermSize=${MaxPermGen} -XX:ReservedCodeCacheSize=${CodeCacheSize}
149 |
150 |
151 |
152 | true
153 |
156 | ${basedir}
157 | 1
158 | false
159 | false
160 | true
161 |
162 | false
163 |
164 |
165 |
166 | test
167 |
168 | test
169 |
170 |
171 |
172 |
173 |
174 | org.apache.maven.plugins
175 | maven-clean-plugin
176 | 2.5
177 |
178 |
179 |
180 | work
181 |
182 |
183 | checkpoint
184 |
185 |
186 | lib_managed
187 |
188 |
189 |
190 |
191 |
192 |
193 |
194 |
195 |
--------------------------------------------------------------------------------
/core/src/test/scala/org/apache/spark/sql/AvroSourceSuite.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 | * File modified by Hortonworks, Inc. Modifications are also licensed under
18 | * the Apache Software License, Version 2.0.
19 | */
20 |
21 | package org.apache.spark.sql
22 |
23 | import org.apache.avro.Schema
24 | import org.apache.avro.generic.GenericData
25 | import org.apache.spark.sql.execution.datasources.hbase.HBaseTableCatalog
26 | import org.apache.spark.sql.execution.datasources.hbase.Logging
27 | import org.apache.spark.sql.execution.datasources.hbase.types.AvroSerde
28 |
29 | case class AvroHBaseRecord(col0: String,
30 | col1: Array[Byte])
31 |
32 | object AvroHBaseRecord {
33 | val schemaString =
34 | s"""{"namespace": "example.avro",
35 | | "type": "record", "name": "User",
36 | | "fields": [
37 | | {"name": "name", "type": "string"},
38 | | {"name": "favorite_number", "type": ["int", "null"]},
39 | | {"name": "favorite_color", "type": ["string", "null"]},
40 | | {"name": "favorite_array", "type": {"type": "array", "items": "string"}},
41 | | {"name": "favorite_map", "type": {"type": "map", "values": "int"}}
42 | | ] }""".stripMargin
43 |
44 | val avroSchema: Schema = {
45 | val p = new Schema.Parser
46 | p.parse(schemaString)
47 | }
48 |
49 | def apply(i: Int): AvroHBaseRecord = {
50 |
51 | val user = new GenericData.Record(avroSchema);
52 | user.put("name", s"name${"%03d".format(i)}")
53 | user.put("favorite_number", i)
54 | user.put("favorite_color", s"color${"%03d".format(i)}")
55 | val favoriteArray = new GenericData.Array[String](2, avroSchema.getField("favorite_array").schema())
56 | favoriteArray.add(s"number${i}")
57 | favoriteArray.add(s"number${i+1}")
58 | user.put("favorite_array", favoriteArray)
59 | import collection.JavaConverters._
60 | val favoriteMap = Map[String, Int](("key1" -> i), ("key2" -> (i+1))).asJava
61 | user.put("favorite_map", favoriteMap)
62 | val avroByte = AvroSerde.serialize(user, avroSchema)
63 | AvroHBaseRecord(s"name${"%03d".format(i)}", avroByte)
64 | }
65 | }
66 |
67 | class AvroSourceSuite extends SHC with Logging{
68 |
69 | // 'catalog' is used when saving data to HBase
70 | override def catalog = s"""{
71 | |"table":{"namespace":"default", "name":"avrotable", "tableCoder":"PrimitiveType"},
72 | |"rowkey":"key",
73 | |"columns":{
74 | |"col0":{"cf":"rowkey", "col":"key", "type":"string"},
75 | |"col1":{"cf":"cf1", "col":"col1", "type":"binary"}
76 | |}
77 | |}""".stripMargin
78 |
79 | def avroCatalog = s"""{
80 | |"table":{"namespace":"default", "name":"avrotable", "tableCoder":"PrimitiveType"},
81 | |"rowkey":"key",
82 | |"columns":{
83 | |"col0":{"cf":"rowkey", "col":"key", "type":"string"},
84 | |"col1":{"cf":"cf1", "col":"col1", "avro":"avroSchema"}
85 | |}
86 | |}""".stripMargin
87 |
88 | def avroCatalogInsert = s"""{
89 | |"table":{"namespace":"default", "name":"avrotableInsert", "tableCoder":"PrimitiveType"},
90 | |"rowkey":"key",
91 | |"columns":{
92 | |"col0":{"cf":"rowkey", "col":"key", "type":"string"},
93 | |"col1":{"cf":"cf1", "col":"col1", "avro":"avroSchema"}
94 | |}
95 | |}""".stripMargin
96 |
97 | def withCatalog(cat: String): DataFrame = {
98 | sqlContext
99 | .read
100 | .options(Map("avroSchema" -> AvroHBaseRecord.schemaString, HBaseTableCatalog.tableCatalog -> cat))
101 | .format("org.apache.spark.sql.execution.datasources.hbase")
102 | .load()
103 | }
104 |
105 | test("populate table") {
106 | //createTable(tableName, columnFamilies)
107 | val sql = sqlContext
108 | import sql.implicits._
109 |
110 | val data = (0 to 255).map { i =>
111 | AvroHBaseRecord(i)
112 | }
113 | sc.parallelize(data).toDF.write.options(
114 | Map(HBaseTableCatalog.tableCatalog -> catalog, HBaseTableCatalog.newTable -> "5"))
115 | .format("org.apache.spark.sql.execution.datasources.hbase")
116 | .save()
117 | }
118 |
119 | test("empty column") {
120 | val df = withCatalog(avroCatalog)
121 | df.createOrReplaceTempView("avrotable")
122 | val c = sqlContext.sql("select count(1) from avrotable").rdd.collect()(0)(0).asInstanceOf[Long]
123 | assert(c == 256)
124 | }
125 |
126 | test("full query") {
127 | val df = withCatalog(avroCatalog)
128 | df.show
129 | df.printSchema()
130 | assert(df.count() == 256)
131 | }
132 |
133 | test("array field") {
134 | val df = withCatalog(avroCatalog)
135 | val filtered = df.select($"col0", $"col1.favorite_array").where($"col0" === "name001")
136 | assert(filtered.count() == 1)
137 | val collected = filtered.collect()
138 | assert(collected(0).getSeq[String](1)(0) === "number1")
139 | assert(collected(0).getSeq[String](1)(1) === "number2")
140 | }
141 |
142 | test("map field") {
143 | val df = withCatalog(avroCatalog)
144 | val filtered = df.select(
145 | $"col0",
146 | $"col1.favorite_map".getItem("key1").as("key1"),
147 | $"col1.favorite_map".getItem("key2").as("key2")
148 | )
149 | .where($"col0" === "name001")
150 | assert(filtered.count() == 1)
151 | val collected = filtered.collect()
152 | assert(collected(0)(1) === 1)
153 | assert(collected(0)(2) === 2)
154 | }
155 |
156 | test("serialization and deserialization query") {
157 | val df = withCatalog(avroCatalog)
158 | df.write.options(
159 | Map("avroSchema"->AvroHBaseRecord.schemaString, HBaseTableCatalog.tableCatalog->avroCatalogInsert,
160 | HBaseTableCatalog.newTable -> "5"))
161 | .format("org.apache.spark.sql.execution.datasources.hbase")
162 | .save()
163 | val newDF = withCatalog(avroCatalogInsert)
164 | newDF.show
165 | newDF.printSchema()
166 | assert(newDF.count() == 256)
167 | }
168 |
169 | test("filtered query") {
170 | val df = withCatalog(avroCatalog)
171 | val r = df.filter($"col1.name" === "name005" || $"col1.name" <= "name005").select("col0", "col1.favorite_color", "col1.favorite_number")
172 | r.show
173 | assert(r.count() == 6)
174 | }
175 |
176 | test("Or filter") {
177 | val df = withCatalog(avroCatalog)
178 | val s = df.filter($"col1.name" <= "name005" || $"col1.name".contains("name007"))
179 | .select("col0", "col1.favorite_color", "col1.favorite_number")
180 | s.show
181 | assert(s.count() == 7)
182 | }
183 |
184 | test("IN and Not IN filter") {
185 | val df = withCatalog(avroCatalog)
186 | val s = df.filter(($"col0" isin ("name000", "name001", "name002", "name003", "name004")) and !($"col0" isin ("name001", "name002", "name003")))
187 | .select("col0", "col1.favorite_number", "col1.favorite_color")
188 | s.explain(true)
189 | s.show
190 | assert(s.count() == 2)
191 | }
192 |
193 | }
194 |
--------------------------------------------------------------------------------
/core/src/test/scala/org/apache/spark/sql/AvroKeySourceSuite.java:
--------------------------------------------------------------------------------
1 | /*
2 | * (C) 2017 Hortonworks, Inc. All rights reserved. See the NOTICE file
3 | * distributed with this work for additional information regarding copyright
4 | * ownership. This file is licensed to You under the Apache License, Version 2.0
5 | * (the "License"); you may not use this file except in compliance with
6 | * the License. You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 |
17 | package org.apache.spark.sql;
18 |
19 | import org.apache.avro.Schema;
20 | import org.apache.avro.SchemaBuilder;
21 | import org.apache.avro.generic.GenericData;
22 | import org.apache.avro.generic.GenericDatumWriter;
23 | import org.apache.avro.generic.GenericRecord;
24 | import org.apache.avro.io.DatumWriter;
25 | import org.apache.avro.io.Encoder;
26 | import org.apache.avro.io.EncoderFactory;
27 | import org.apache.hadoop.hbase.HBaseCluster;
28 | import org.apache.hadoop.hbase.HBaseTestingUtility;
29 | import org.apache.hadoop.hbase.TableName;
30 | import org.apache.hadoop.hbase.client.HTable;
31 | import org.apache.hadoop.hbase.client.Put;
32 | import org.apache.hadoop.hbase.util.Bytes;
33 | import org.apache.spark.sql.execution.datasources.hbase.HBaseTableCatalog;
34 | import org.apache.spark.sql.execution.datasources.hbase.SparkHBaseConf;
35 | import org.junit.After;
36 | import org.junit.Before;
37 | import org.junit.Ignore;
38 | import org.junit.Test;
39 |
40 | import java.io.ByteArrayOutputStream;
41 | import java.io.IOException;
42 | import java.util.Arrays;
43 | import java.util.Comparator;
44 | import java.util.HashMap;
45 | import java.util.Map;
46 |
47 | import static org.junit.Assert.assertEquals;
48 |
49 | @Ignore
50 | public class AvroKeySourceSuite {
51 |
52 | private static final TableName TABLE_NAME = TableName.valueOf("TEST_TABLE");
53 | private static final String COLUMN_FAMILY = "COL_FAMILY";
54 | private static final String COLUMN_QUALIFIER = "COL_QUALIFIER";
55 |
56 | private static final String KEY1 = "KEY1";
57 | private static final String KEY2 = "KEY2";
58 |
59 | private final HBaseTestingUtility hBaseTestingUtility = new HBaseTestingUtility();
60 |
61 | private HBaseCluster hbase;
62 | private SparkSession sparkSession;
63 |
64 | @Before
65 | public void setUp() throws Exception {
66 | hbase = hBaseTestingUtility.startMiniCluster();
67 | SparkHBaseConf.conf_$eq(hbase.getConf());
68 | sparkSession = SparkSession.builder()
69 | .master("local[*]")
70 | .appName("TestHBaseAvroKey")
71 | .config(SparkHBaseConf.testConf(), "true")
72 | .getOrCreate();
73 | }
74 |
75 | @After
76 | public void tearDown() throws Exception {
77 | hBaseTestingUtility.shutdownMiniCluster();
78 | sparkSession.stop();
79 | }
80 |
81 | public static Comparator rowComparator = new Comparator() {
82 | @Override
83 | public int compare(Row a, Row b) {
84 | return a.getStruct(1).getStruct(1).getInt(0) - b.getStruct(1).getStruct(1).getInt(0);
85 | }
86 | };
87 |
88 | @Test
89 | public void testAvroKey() throws Exception {
90 | hBaseTestingUtility.createTable(TABLE_NAME, COLUMN_FAMILY);
91 | writeDataToHBase(hbase);
92 |
93 | // Assert contents look as expected.
94 | Dataset df = sparkSession.sqlContext().read()
95 | .format("org.apache.spark.sql.execution.datasources.hbase")
96 | .options(getHBaseSourceOptions()).load();
97 | assertEquals(2, df.count());
98 | df.show();
99 | Row[] rows = (Row[])df.collect();
100 |
101 | // Arrays.sort(rows, (a, b) -> {
102 | // return a.getStruct(1).getStruct(1).getInt(0) - b.getStruct(1).getStruct(1).getInt(0);
103 | // });
104 |
105 | Arrays.sort(rows, rowComparator);
106 |
107 | // Note that the key is duplicated in the hbase key and in the object
108 | assertEquals(KEY1, rows[0].getStruct(0).getString(0));
109 | assertEquals(KEY1, rows[0].getStruct(1).getStruct(0).getString(0));
110 | assertEquals(5, rows[0].getStruct(1).getStruct(1).getInt(0));
111 |
112 | assertEquals(KEY2, rows[1].getStruct(0).getString(0));
113 | assertEquals(KEY2, rows[1].getStruct(1).getStruct(0).getString(0));
114 | assertEquals(7, rows[1].getStruct(1).getStruct(1).getInt(0));
115 | }
116 |
117 | private static void putRecord(HTable table, GenericRecord object) throws Exception {
118 | // The rowKey doesn't actually matter too much.
119 | byte[] keyBytes = avroEncoderFunc((GenericRecord) object.get("key"));
120 | byte[] recordBytes = avroEncoderFunc(object);
121 | Put p = new Put(keyBytes);
122 | p.addColumn(Bytes.toBytes(COLUMN_FAMILY), Bytes.toBytes(COLUMN_QUALIFIER), recordBytes);
123 | table.put(p);
124 | }
125 |
126 | private static void writeDataToHBase(HBaseCluster hbase) throws Exception {
127 | // Write some data directly to it
128 | GenericRecord record1 = getRecord(KEY1, 5);
129 | GenericRecord record2 = getRecord(KEY2, 7);
130 | //HTable testTable = new HTable(hbase.getConf(), TABLE_NAME);
131 | HTable testTable = null;
132 | putRecord(testTable, record1);
133 | putRecord(testTable, record2);
134 | }
135 |
136 | private static Map getHBaseSourceOptions() {
137 | String hbaseCatalog = "{\"table\": {\"namespace\": \"default\", \"name\": \"TEST_TABLE\", \"tableCoder\":\"PrimitiveType\"}," +
138 | "\"rowkey\": \"key\", \"columns\": {"
139 | + "\"key\": {\"cf\": \"rowkey\", \"col\": \"key\", \"avro\": \"keySchema\"},"
140 | + "\"value\": {\"cf\": \"" + COLUMN_FAMILY + "\", \"col\": \"" + COLUMN_QUALIFIER + "\", \"avro\": \"avroSchema\"}"
141 | + "}}";
142 | Map hbaseOptions = new HashMap<>();
143 | hbaseOptions.put(HBaseTableCatalog.tableCatalog(), hbaseCatalog);
144 | System.out.println("keySchema: " + getSchema().getField("key").schema().toString());
145 | System.out.println("avroSchema: " + getSchema().toString());
146 | hbaseOptions.put("keySchema", getSchema().getField("key").schema().toString());
147 | hbaseOptions.put("avroSchema", getSchema().toString());
148 | return hbaseOptions;
149 | }
150 |
151 | private static Schema getSchema() {
152 | return SchemaBuilder.record("TestKeyedRecord").namespace("test")
153 | .fields()
154 | .name("key").type().record("Key").namespace("test")
155 | .fields().name("keyValue").type().stringType().noDefault()
156 | .endRecord().noDefault()
157 | .name("data").type().record("Data").namespace("test")
158 | .fields().name("dataValue").type().intType().noDefault()
159 | .endRecord().noDefault()
160 | .endRecord();
161 | }
162 |
163 | private static GenericRecord getRecord(String key, int value) {
164 | Schema keySchema = getSchema().getField("key").schema();
165 | GenericRecord keyRecord = new GenericData.Record(keySchema);
166 | keyRecord.put("keyValue", key);
167 |
168 | Schema dataSchema = getSchema().getField("data").schema();
169 | GenericRecord dataRecord = new GenericData.Record(dataSchema);
170 | dataRecord.put("dataValue", value);
171 |
172 | GenericRecord record = new GenericData.Record(getSchema());
173 | record.put("key", keyRecord);
174 | record.put("data", dataRecord);
175 | return record;
176 | }
177 |
178 | public static byte[] avroEncoderFunc(GenericRecord record) {
179 | Schema schema = record.getSchema();
180 | ByteArrayOutputStream baos = new ByteArrayOutputStream();
181 | DatumWriter datumWriter = new GenericDatumWriter<>(schema);
182 | Encoder encoder = EncoderFactory.get().directBinaryEncoder(baos, null);
183 | try {
184 | datumWriter.write(record, encoder);
185 | } catch (IOException e) {
186 | throw new RuntimeException("Problem serializing " + record.toString(), e);
187 | }
188 | return baos.toByteArray();
189 | }
190 |
191 | }
192 |
--------------------------------------------------------------------------------
/core/src/test/scala/org/apache/spark/sql/DynamicColumnSuite.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 | * File modified by Hortonworks, Inc. Modifications are also licensed under
18 | * the Apache Software License, Version 2.0.
19 | */
20 |
21 | package org.apache.spark.sql
22 |
23 | import org.apache.spark.rdd.RDD
24 | import org.apache.spark.sql.execution.datasources.hbase.Logging
25 | import org.apache.spark.sql.execution.datasources.hbase.{HBaseRelation, HBaseTableCatalog}
26 |
27 | import scala.reflect.ClassTag
28 |
29 | case class HBaseRecordExtended(
30 | col0: String,
31 | col1: Boolean,
32 | col2: Double,
33 | col3: Float,
34 | col4: Int,
35 | col5: Long,
36 | col6: Short,
37 | col7: String,
38 | col8: Byte,
39 | col9: String)
40 |
41 | object HBaseRecordExtended {
42 | def apply(i: Int, t: String): HBaseRecordExtended = {
43 | val s = s"""row${"%03d".format(i)}"""
44 | HBaseRecordExtended(s,
45 | i % 2 == 0,
46 | i.toDouble,
47 | i.toFloat,
48 | i,
49 | i.toLong,
50 | i.toShort,
51 | s"String$i: $t",
52 | i.toByte,
53 | s"StringExtended$i: $t")
54 | }
55 | val catalog = s"""{
56 | |"table":{"namespace":"default", "name":"table1"},
57 | |"rowkey":"key",
58 | |"columns":{
59 | |"col0":{"cf":"rowkey", "col":"key", "type":"string"},
60 | |"col1":{"cf":"cf1", "col":"col1", "type":"boolean"},
61 | |"col2":{"cf":"cf2", "col":"col2", "type":"double"},
62 | |"col3":{"cf":"cf3", "col":"col3", "type":"float"},
63 | |"col4":{"cf":"cf4", "col":"col4", "type":"int"},
64 | |"col5":{"cf":"cf5", "col":"col5", "type":"bigint"},
65 | |"col6":{"cf":"cf6", "col":"col6", "type":"smallint"},
66 | |"col7":{"cf":"cf7", "col":"col7_1", "type":"string"},
67 | |"col8":{"cf":"cf8", "col":"col8", "type":"tinyint"},
68 | |"col9":{"cf":"cf7", "col":"col7_2", "type":"string"}
69 | |}
70 | |}""".stripMargin
71 |
72 | val catalogcf7 = s"""{
73 | |"table":{"namespace":"default", "name":"table1"},
74 | |"rowkey":"key",
75 | |"columns":{
76 | |"col0":{"cf":"rowkey", "col":"key", "type":"string"},
77 | |"col1":{"cf":"cf7", "col":"col7_1", "type":"string"},
78 | |"col2":{"cf":"cf7", "col":"col7_2", "type":"string"}
79 | |}
80 | |}""".stripMargin
81 |
82 | }
83 |
84 | case class HBaseRecordDynamic(
85 | col0: String,
86 | col1: Boolean,
87 | col2: Map[Long, Double],
88 | col3: Float,
89 | col4: Map[String, Int],
90 | col5: Long,
91 | col6: Short,
92 | col7: Map[String, Map[Long, String]],
93 | col8: Byte)
94 |
95 | object HBaseRecordDynamic {
96 | def apply(i: Int, t: String): HBaseRecordDynamic = {
97 | val s = s"""row${"%03d".format(i)}"""
98 | val ts = System.currentTimeMillis() - i * 1000
99 | HBaseRecordDynamic(s,
100 | i % 2 == 0,
101 | Map(ts -> i.toDouble),
102 | i.toFloat,
103 | Map("col4" -> i),
104 | i.toLong,
105 | i.toShort,
106 | Map("col7" -> Map(ts -> s"String$i: $t")),
107 | i.toByte)
108 | }
109 | val catalog = s"""{
110 | |"table":{"namespace":"default", "name":"table1"},
111 | |"rowkey":"key",
112 | |"columns":{
113 | |"col0":{"cf":"rowkey", "col":"key", "type":"string"},
114 | |"col1":{"cf":"cf1", "col":"col1", "type":"boolean"},
115 | |"col2":{"cf":"cf2", "col":"col2", "type":"map"},
116 | |"col3":{"cf":"cf3", "col":"col3", "type":"float"},
117 | |"col4":{"cf":"cf4", "col":"", "type":"map"},
118 | |"col5":{"cf":"cf5", "col":"col5", "type":"bigint"},
119 | |"col6":{"cf":"cf6", "col":"col6", "type":"smallint"},
120 | |"col7":{"cf":"cf7", "col":"", "type":"map>"},
121 | |"col8":{"cf":"cf8", "col":"col8", "type":"tinyint"}
122 | |}
123 | |}""".stripMargin
124 |
125 | }
126 |
127 |
128 | class DynamicColumnSuite extends SHC with Logging {
129 |
130 | def withCatalog(cat: String, opt: Map[String, String]): DataFrame = {
131 | sqlContext
132 | .read
133 | .options(Map(
134 | HBaseTableCatalog.tableCatalog -> cat
135 | ) ++ opt)
136 | .format("org.apache.spark.sql.execution.datasources.hbase")
137 | .load()
138 | }
139 |
140 | def writeData[T: ClassTag](data: Seq[T], catalog: String)(implicit newProductEncoder: Encoder[T], rddToDatasetHolder: RDD[T] => DatasetHolder[T]) = {
141 | sc.parallelize(data).toDF.write
142 | .options(Map(
143 | HBaseTableCatalog.tableCatalog -> catalog,
144 | HBaseTableCatalog.newTable -> "5",
145 | HBaseRelation.MAX_VERSIONS -> "3"
146 | ))
147 | .format("org.apache.spark.sql.execution.datasources.hbase")
148 | .save()
149 | }
150 |
151 | test("retrieve rows without schema with default type") {
152 | val sql = sqlContext
153 | import sql.implicits._
154 |
155 | sc.parallelize(data).toDF.write
156 |
157 |
158 | def data = (0 to 2).map { i =>
159 | HBaseRecordExtended(i, "schema less")
160 | }
161 |
162 | writeData(data, HBaseRecordExtended.catalog)
163 | writeData(data.map(_.copy(col1 = false, col2 = 1.1, col4 = 1, col7 = "")), HBaseRecordExtended.catalog)
164 |
165 | // Test
166 |
167 | val result = withCatalog(HBaseRecordDynamic.catalog, Map(
168 | HBaseRelation.RESTRICTIVE -> HBaseRelation.Restrictive.none,
169 | HBaseRelation.MAX_VERSIONS -> "3"
170 | ))
171 |
172 |
173 | val rows = result.take(10)
174 |
175 |
176 | assert(rows.size == 3)
177 | println(rows.mkString(" | "))
178 | assert(rows(0).size == 9)
179 | assert(rows(0).getBoolean(1) == false)
180 | assert(rows(0).getMap[Long, Double](2).size == 2)
181 | assert(rows(0).getMap[String, Int](4).head._2 == 1)
182 | assert(rows(0).getMap[String,Map[Long, String]](7).size == 2)
183 |
184 | }
185 |
186 | test("persist rows without schema with default type") {
187 | val sql = sqlContext
188 | import sql.implicits._
189 |
190 | sc.parallelize(data).toDF.write
191 |
192 |
193 | def data = (3 to 5).map { i =>
194 | HBaseRecordDynamic(i, "schema less")
195 | }
196 |
197 | writeData(data, HBaseRecordDynamic.catalog)
198 |
199 | // Test
200 |
201 | val result = withCatalog(HBaseRecordExtended.catalog, Map(
202 | HBaseRelation.RESTRICTIVE -> HBaseRelation.Restrictive.none,
203 | HBaseRelation.MAX_VERSIONS -> "3"
204 | ))
205 |
206 |
207 | val rows = result.take(10)
208 |
209 |
210 | assert(rows.size == 6)
211 |
212 | println(rows.mkString(" | "))
213 |
214 | assert(rows(0).size == 10)
215 |
216 | }
217 |
218 | test("read rows for the families declared in catalog") {
219 | val sql = sqlContext
220 | import sql.implicits._
221 |
222 | sc.parallelize(data).toDF.write
223 |
224 |
225 | def data = (3 to 5).map { i =>
226 | HBaseRecordDynamic(i, "schema less")
227 | }
228 |
229 | writeData(data, HBaseRecordDynamic.catalog)
230 |
231 | // Test
232 |
233 | val result = withCatalog(HBaseRecordExtended.catalogcf7, Map(
234 | HBaseRelation.RESTRICTIVE -> HBaseRelation.Restrictive.family,
235 | HBaseRelation.MAX_VERSIONS -> "3"
236 | ))
237 |
238 |
239 | val rows = result.take(10)
240 |
241 |
242 | assert(rows.size == 6)
243 |
244 | println(rows.mkString(" | "))
245 |
246 | assert(rows(0).size == 3)
247 |
248 | }
249 |
250 | }
251 |
--------------------------------------------------------------------------------
/core/src/test/scala/org/apache/spark/sql/AvroRecordSuite.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright 2014 Databricks
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 |
17 | package org.apache.spark.sql
18 |
19 | import scala.util.Random
20 |
21 | import java.nio.ByteBuffer
22 |
23 | import org.apache.avro.Schema
24 | import org.apache.avro.generic.GenericData
25 | import org.apache.spark.sql.execution.datasources.hbase.Logging
26 | import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, FunSuite}
27 | import org.apache.spark.sql.execution.datasources.hbase.types._
28 |
29 | class AvroRecordSuite extends FunSuite with BeforeAndAfterEach with BeforeAndAfterAll with Logging {
30 |
31 | test("avro to schema converterBasic setup") {
32 | val schemaString =
33 | s"""{"namespace": "example.avro",
34 | | "type": "record", "name": "User",
35 | | "fields": [ {"name": "name", "type": "string"},
36 | | {"name": "favorite_number", "type": ["int", "null"]},
37 | | {"name": "favorite_color", "type": ["string", "null"]} ] }""".stripMargin
38 | val avroSchema: Schema = {
39 | val p = new Schema.Parser
40 | p.parse(schemaString)
41 | }
42 |
43 | val user1 = new GenericData.Record(avroSchema)
44 | user1.put("name", "Alyssa")
45 | user1.put("favorite_number", 256)
46 | // Leave favorite color null
47 |
48 | val user2 = new GenericData.Record(avroSchema)
49 | user2.put("name", "Ben")
50 | user2.put("favorite_number", 7)
51 | user2.put("favorite_color", "red")
52 |
53 | val sqlUser1 = SchemaConverters.createConverterToSQL(avroSchema)(user1)
54 | println(sqlUser1)
55 | val schema = SchemaConverters.toSqlType(avroSchema)
56 | println(s"\nSqlschema: $schema")
57 | val avroUser1 = SchemaConverters.createConverterToAvro(schema.dataType, "avro", "example.avro")(sqlUser1)
58 | val avroByte = AvroSerde.serialize(avroUser1, avroSchema)
59 | val avroUser11 = AvroSerde.deserialize(avroByte, avroSchema)
60 | println(s"$avroUser1")
61 | }
62 |
63 | test("avro array type schema serialize/deserialize") {
64 | val schemaString = s"""{"type": "array", "items": ["int","null"]}""".stripMargin
65 | val avroSchema: Schema = {
66 | val p = new Schema.Parser
67 | p.parse(schemaString)
68 | }
69 |
70 | val data = new java.util.ArrayList[Any]
71 | data.add(1)
72 | data.add(2)
73 | data.add(null)
74 | data.add(-3)
75 |
76 | val sqlConv = SchemaConverters.createConverterToSQL(avroSchema)(data)
77 | println(sqlConv)
78 | val sqlSchema = SchemaConverters.toSqlType(avroSchema)
79 | println(s"\nSqlschema: $sqlSchema")
80 | val avroData = SchemaConverters.createConverterToAvro(sqlSchema.dataType, "avro", "example.avro")(sqlConv)
81 | val avroBytes = AvroSerde.serialize(avroData, avroSchema)
82 | val desData = AvroSerde.deserialize(avroBytes, avroSchema)
83 | println(s"$desData")
84 | }
85 |
86 | test("avro primitive data types union schemas serialize/deserialize") {
87 | val unionNullValue: String = null
88 | val byteArray: Array[Byte] = Array(10.toByte)
89 | val bytes = ByteBuffer.wrap(byteArray)
90 |
91 | val datatypeSchemas = Map("Test string" -> "\"string\"",
92 | unionNullValue -> """["string","null"]""",
93 | true -> """["boolean","null"]""",
94 | 9223372036854775807L -> """["long","null"]""",
95 | -1234.93f -> """["float","null"]""",
96 | 123 -> """["int","null"]""",
97 | 1.7e10d -> """["double","null"]""",
98 | bytes -> """["bytes","null"]""")
99 |
100 | datatypeSchemas.keys.foreach{ data =>
101 | val avroSchema: Schema = {
102 | val p = new Schema.Parser
103 | p.parse(datatypeSchemas(data))
104 | }
105 |
106 | val sqlConv = SchemaConverters.createConverterToSQL(avroSchema)(data)
107 | println(sqlConv)
108 | val sqlSchema = SchemaConverters.toSqlType(avroSchema)
109 | println(s"\nSqlschema: $sqlSchema")
110 | val avroData = SchemaConverters.createConverterToAvro(sqlSchema.dataType, "avro", "example.avro")(sqlConv)
111 | val avroBytes = AvroSerde.serialize(avroData, avroSchema)
112 | val desData = AvroSerde.deserialize(avroBytes, avroSchema)
113 | println(s"$desData")
114 | }
115 | }
116 |
117 | test("test schema complicated") {
118 | val schemaString =
119 | s"""{
120 | "type" : "record",
121 | "name" : "test_schema",
122 | "fields" : [{
123 | "name" : "string",
124 | "type" : "string",
125 | "doc" : "Meaningless string of characters"
126 | }, {
127 | "name" : "simple_map",
128 | "type" : {"type": "map", "values": "int"}
129 | }, {
130 | "name" : "complex_map",
131 | "type" : {"type": "map", "values": {"type": "map", "values": "string"}}
132 | }, {
133 | "name" : "union_string_null",
134 | "type" : ["null", "string"]
135 | }, {
136 | "name" : "union_int_long_null",
137 | "type" : ["int", "long", "null"]
138 | }, {
139 | "name" : "union_float_double",
140 | "type" : ["float", "double"]
141 | }, {
142 | "name": "fixed3",
143 | "type": {"type": "fixed", "size": 3, "name": "fixed3"}
144 | }, {
145 | "name": "fixed2",
146 | "type": {"type": "fixed", "size": 2, "name": "fixed2"}
147 | }, {
148 | "name": "enum",
149 | "type": { "type": "enum",
150 | "name": "Suit",
151 | "symbols" : ["SPADES", "HEARTS", "DIAMONDS", "CLUBS"]
152 | }
153 | }, {
154 | "name": "record",
155 | "type": {
156 | "type": "record",
157 | "name": "record",
158 | "aliases": ["RecordAlias"],
159 | "fields" : [{
160 | "name": "value_field",
161 | "type": "string"
162 | }]
163 | }
164 | }, {
165 | "name": "array_of_boolean",
166 | "type": {"type": "array", "items": "boolean"}
167 | }, {
168 | "name": "bytes",
169 | "type": "bytes"
170 | }]
171 | }""".stripMargin
172 | val avroSchema: Schema = {
173 | val p = new Schema.Parser
174 | p.parse(schemaString)
175 | }
176 | val schema = SchemaConverters.toSqlType(avroSchema)
177 | println(s"\nSqlschema: $schema")
178 | }
179 |
180 | test("complicated") {
181 | val schemaComplex =
182 | s"""{"type" : "record",
183 | | "name" : "test_schema",
184 | | "fields" : [{
185 | | "name" : "string",
186 | | "type" : "string",
187 | | "doc" : "Meaningless string of characters"
188 | | }, {
189 | | "name" : "simple_map",
190 | | "type" : {"type": "map", "values": "int"}
191 | | }, {
192 | | "name" : "union_int_long_null",
193 | | "type" : ["int", "long", "null"]
194 | | }, {
195 | | "name" : "union_float_double",
196 | | "type" : ["float", "double"]
197 | | }, {
198 | | "name": "inner_record",
199 | | "type": {
200 | | "type": "record",
201 | | "name": "inner_record",
202 | | "aliases": ["RecordAlias"],
203 | | "fields" : [{
204 | | "name": "value_field",
205 | | "type": "string"
206 | | }]
207 | | }
208 | | }, {
209 | | "name": "array_of_boolean",
210 | | "type": {"type": "array", "items": "boolean"}
211 | | }, {
212 | | "name": "bytes",
213 | | "type": "bytes"
214 | | }]
215 | | }""".stripMargin
216 |
217 | val avroComplex: Schema = {
218 | val p = new Schema.Parser
219 | p.parse(schemaComplex)
220 | }
221 | val objectSize = 10 // Maps, arrays and strings in our generated file have this size
222 | val schema = SchemaConverters.toSqlType(avroComplex)
223 | println(s"\nSqlschema: $schema")
224 | // Create data that we will put into the avro file
225 | val avroRec = new GenericData.Record(avroComplex)
226 | val innerRec = new GenericData.Record(avroComplex.getField("inner_record").schema())
227 | innerRec.put("value_field", "Inner string")
228 | val rand = new Random()
229 |
230 | avroRec.put("string", rand.nextString(objectSize))
231 | avroRec.put("simple_map", TestUtils.generateRandomMap(rand, objectSize))
232 | avroRec.put("union_int_long_null", rand.nextInt())
233 | avroRec.put("union_float_double", rand.nextDouble())
234 | avroRec.put("inner_record", innerRec)
235 | avroRec.put("array_of_boolean", TestUtils.generateRandomArray(rand, objectSize))
236 | avroRec.put("bytes", TestUtils.generateRandomByteBuffer(rand, objectSize))
237 | println(s"\navroRec: $avroRec")
238 | val sqlRec = SchemaConverters.createConverterToSQL(avroComplex)(avroRec)
239 | println(s"\nsqlRec: $sqlRec")
240 |
241 | val avroRec1 = SchemaConverters.createConverterToAvro(schema.dataType, "test_schema", "example.avro")(sqlRec)
242 | println(s"\navroRec1: $avroRec1")
243 | val avroByte = AvroSerde.serialize(avroRec1, avroComplex)
244 | println("\nserialize")
245 | val avroRec11 = AvroSerde.deserialize(avroByte, avroComplex)
246 | println(s"\navroRec11: $avroRec11")
247 | val sqlRec1 = SchemaConverters.createConverterToSQL(avroComplex)(avroRec11)
248 | println(s"sqlRec1: $sqlRec1")
249 | }
250 | }
251 |
--------------------------------------------------------------------------------
/core/src/main/scala/org/apache/spark/sql/execution/datasources/hbase/SHCCredentialsManager.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * (C) 2017 Hortonworks, Inc. All rights reserved. See the NOTICE file
3 | * distributed with this work for additional information regarding copyright
4 | * ownership. This file is licensed to You under the Apache License, Version 2.0
5 | * (the "License"); you may not use this file except in compliance with
6 | * the License. You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 |
17 | package org.apache.spark.sql.execution.datasources.hbase
18 |
19 | import java.io.{ByteArrayInputStream, DataInputStream}
20 | import java.security.PrivilegedExceptionAction
21 | import java.util.concurrent.{Executors, TimeUnit}
22 | import java.util.Date
23 |
24 | import scala.collection.mutable
25 | import scala.language.existentials
26 | import scala.util.control.NonFatal
27 |
28 | import org.apache.hadoop.conf.Configuration
29 | import org.apache.hadoop.hbase.security.token.{AuthenticationTokenIdentifier, TokenUtil}
30 | import org.apache.hadoop.io.DataOutputBuffer
31 | import org.apache.hadoop.security.{Credentials, UserGroupInformation}
32 | import org.apache.hadoop.security.token.{Token, TokenIdentifier}
33 |
34 | import org.apache.spark.util.{ThreadUtils, Utils}
35 | import org.apache.spark.SparkEnv
36 | import org.apache.spark.sql.execution.datasources.hbase.SHCCredentialsManager._
37 |
38 | final class SHCCredentialsManager private() extends Logging {
39 | private class TokenInfo(
40 | val expireTime: Long,
41 | val issueTime: Long,
42 | val refreshTime: Long,
43 | val conf: Configuration,
44 | val token: Token[_ <: TokenIdentifier],
45 | val serializedToken: Array[Byte])
46 |
47 | private def sparkConf = SparkEnv.get.conf
48 | private val expireTimeFraction = sparkConf.getDouble(SparkHBaseConf.expireTimeFraction, 0.95)
49 | private val refreshTimeFraction = sparkConf.getDouble(SparkHBaseConf.refreshTimeFraction, 0.6)
50 | private val refreshDurationMins = sparkConf.getInt(SparkHBaseConf.refreshDurationMins, 10)
51 |
52 | private val tokensMap = new mutable.HashMap[String, TokenInfo]
53 |
54 | // We assume token expiration time should be no less than 10 minutes by default.
55 | private val nextRefresh = TimeUnit.MINUTES.toMillis(refreshDurationMins)
56 |
57 | private val credentialsManagerEnabled = {
58 | val isEnabled = sparkConf.getBoolean(SparkHBaseConf.credentialsManagerEnabled, false) &&
59 | UserGroupInformation.isSecurityEnabled
60 | logInfo(s"SHCCredentialsManager was${if (isEnabled) "" else " not"} enabled.")
61 | isEnabled
62 | }
63 |
64 | val tokenUpdateExecutor = Executors.newSingleThreadScheduledExecutor(
65 | ThreadUtils.namedThreadFactory("HBase Tokens Refresh Thread"))
66 |
67 | // If SHCCredentialsManager is enabled, start an executor to update tokens
68 | if (credentialsManagerEnabled) {
69 | val tokenUpdateRunnable = new Runnable {
70 | override def run(): Unit = Utils.logUncaughtExceptions(updateTokensIfRequired())
71 | }
72 | tokenUpdateExecutor.scheduleAtFixedRate(
73 | tokenUpdateRunnable, nextRefresh, nextRefresh, TimeUnit.MILLISECONDS)
74 | }
75 |
76 | private val (principal, keytab) = if (credentialsManagerEnabled) {
77 | val p = sparkConf.get(SparkHBaseConf.principal, sparkConf.get("spark.yarn.principal", null))
78 | val k = sparkConf.get(SparkHBaseConf.keytab, sparkConf.get("spark.yarn.keytab", null))
79 | require(p != null, s"neither ${SparkHBaseConf.principal} nor spark.yarn.principal " +
80 | s"is configured, this should be configured to make token renewal work")
81 | require(k != null, s"neither ${SparkHBaseConf.keytab} nor spark.yarn.keytab " +
82 | s"is configured, this should be configured to make token renewal work")
83 | (p, k)
84 | } else {
85 | (null, null)
86 | }
87 |
88 | /**
89 | * Get HBase Token from specified cluster name.
90 | */
91 | def getTokenForCluster(conf: Configuration): Array[Byte] = {
92 | if (!isCredentialsRequired(conf))
93 | return null
94 |
95 | // var token: Token[_ <: TokenIdentifier] = null
96 | var serializedToken: Array[Byte] = null
97 | val identifier = clusterIdentifier(conf)
98 |
99 | val tokenInfoOpt = this.synchronized {
100 | tokensMap.get(identifier)
101 | }
102 |
103 | val needNewToken = if (tokenInfoOpt.isDefined) {
104 | if (isTokenInfoExpired(tokenInfoOpt.get)) {
105 | // Should not happen if refresh thread works as expected
106 | logWarning(s"getTokenForCluster: refresh thread may not be working for cluster $identifier")
107 | true
108 | } else {
109 | // token = tokenInfoOpt.get.token
110 | serializedToken = tokenInfoOpt.get.serializedToken
111 | logDebug(s"getTokenForCluster: Use existing token for cluster $identifier")
112 | false
113 | }
114 | } else {
115 | true
116 | }
117 |
118 | if (needNewToken) {
119 | logInfo(s"getTokenForCluster: Obtaining new token for cluster $identifier")
120 |
121 | val tokenInfo = getNewToken(conf)
122 | this.synchronized {
123 | tokensMap.put(identifier, tokenInfo)
124 | }
125 |
126 | // token = tokenInfo.token
127 | serializedToken = tokenInfo.serializedToken
128 |
129 | logInfo(s"getTokenForCluster: Obtained new token with expiration time" +
130 | s" ${new Date(tokenInfo.expireTime)} and refresh time ${new Date(tokenInfo.refreshTime)} " +
131 | s"for cluster $identifier")
132 | }
133 |
134 | serializedToken
135 | }
136 |
137 | private def isTokenInfoExpired(tokenInfo: TokenInfo): Boolean = {
138 | System.currentTimeMillis() >=
139 | ((tokenInfo.expireTime - tokenInfo.issueTime) * expireTimeFraction + tokenInfo.issueTime).toLong
140 | }
141 |
142 | private def getRefreshTime(issueTime: Long, expireTime: Long): Long = {
143 | require(expireTime > issueTime,
144 | s"Token expire time $expireTime is smaller than issue time $issueTime")
145 |
146 | // the expected expire time would be 60% of real expire time, to avoid long running task
147 | // failure.
148 | ((expireTime - issueTime) * refreshTimeFraction + issueTime).toLong
149 | }
150 |
151 | private def isCredentialsRequired(conf: Configuration): Boolean =
152 | credentialsManagerEnabled && conf.get("hbase.security.authentication") == "kerberos"
153 |
154 | private def updateTokensIfRequired(): Unit = {
155 | val currTime = System.currentTimeMillis()
156 |
157 | // Filter out all the tokens should be re-issued.
158 | val tokensToUpdate = this.synchronized {
159 | tokensMap.filter { case (_, tokenInfo) => tokenInfo.refreshTime <= currTime }
160 | }
161 |
162 | if (tokensToUpdate.isEmpty) {
163 | logInfo("Refresh Thread: No tokens require update")
164 | } else {
165 | // Update all the expect to be expired tokens
166 | val updatedTokens = tokensToUpdate.map { case (cluster, tokenInfo) =>
167 | val token = {
168 | try {
169 | val tok = getNewToken(tokenInfo.conf)
170 | logInfo(s"Refresh Thread: Successfully obtained token for cluster $cluster")
171 | tok
172 | } catch {
173 | case NonFatal(ex) =>
174 | logWarning(s"Refresh Thread: Unable to fetch tokens from HBase cluster $cluster", ex)
175 | null
176 | }
177 | }
178 | (cluster, token)
179 | }.filter(null != _._2)
180 |
181 | this.synchronized {
182 | updatedTokens.foreach { kv => tokensMap.put(kv._1, kv._2) }
183 | }
184 | }
185 | }
186 |
187 | private def getNewToken(conf: Configuration): TokenInfo = {
188 | val kerberosUgi = UserGroupInformation.loginUserFromKeytabAndReturnUGI(principal, keytab)
189 | val tokenInfo = kerberosUgi.doAs(new PrivilegedExceptionAction[TokenInfo] {
190 | override def run(): TokenInfo = {
191 | val token = TokenUtil.obtainToken(HBaseConnectionCache.getConnection(conf).connection)
192 | val tokenIdentifier = token.decodeIdentifier()
193 | val expireTime = tokenIdentifier.getExpirationDate
194 | val issueTime = tokenIdentifier.getIssueDate
195 | val refreshTime = getRefreshTime(issueTime, expireTime)
196 | new TokenInfo(expireTime, issueTime, refreshTime, conf, token, serializeToken(token))
197 | }
198 | })
199 |
200 | UserGroupInformation.getCurrentUser.addToken(tokenInfo.token.getService, tokenInfo.token)
201 | tokenInfo
202 | }
203 |
204 | private def clusterIdentifier(conf: Configuration): String = {
205 | require(conf.get("zookeeper.znode.parent") != null &&
206 | conf.get("hbase.zookeeper.quorum") != null &&
207 | conf.get("hbase.zookeeper.property.clientPort") != null)
208 |
209 | conf.get("zookeeper.znode.parent") + "#" +
210 | conf.get("hbase.zookeeper.quorum") + "#" +
211 | conf.get("hbase.zookeeper.property.clientPort")
212 | }
213 | }
214 |
215 | object SHCCredentialsManager extends Logging {
216 | lazy val manager = new SHCCredentialsManager
217 |
218 | def processShcToken(serializedToken: Array[Byte]): Unit = {
219 | if (null != serializedToken) {
220 | val tok = deserializeToken(serializedToken)
221 | val credentials = new Credentials()
222 | credentials.addToken(tok.getService, tok)
223 |
224 | logInfo(s"Obtained token with expiration date ${new Date(tok.decodeIdentifier()
225 | .asInstanceOf[AuthenticationTokenIdentifier].getExpirationDate)}")
226 |
227 | UserGroupInformation.getCurrentUser.addCredentials(credentials)
228 | }
229 | }
230 |
231 | private def serializeToken(token: Token[_ <: TokenIdentifier]): Array[Byte] = {
232 | val dob: DataOutputBuffer = new DataOutputBuffer()
233 | token.write(dob)
234 | val dobCopy = new Array[Byte](dob.getLength)
235 | System.arraycopy(dob.getData, 0, dobCopy, 0, dobCopy.length)
236 | dobCopy
237 | }
238 |
239 | private def deserializeToken(tokenBytes: Array[Byte]): Token[_ <: TokenIdentifier] = {
240 | val byteStream = new ByteArrayInputStream(tokenBytes)
241 | val dataStream = new DataInputStream(byteStream)
242 | val token = new Token
243 | token.readFields(dataStream)
244 | token
245 | }
246 | }
247 |
--------------------------------------------------------------------------------
/LICENSE:
--------------------------------------------------------------------------------
1 |
2 | Apache License
3 | Version 2.0, January 2004
4 | http://www.apache.org/licenses/
5 |
6 | TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
7 |
8 | 1. Definitions.
9 |
10 | "License" shall mean the terms and conditions for use, reproduction,
11 | and distribution as defined by Sections 1 through 9 of this document.
12 |
13 | "Licensor" shall mean the copyright owner or entity authorized by
14 | the copyright owner that is granting the License.
15 |
16 | "Legal Entity" shall mean the union of the acting entity and all
17 | other entities that control, are controlled by, or are under common
18 | control with that entity. For the purposes of this definition,
19 | "control" means (i) the power, direct or indirect, to cause the
20 | direction or management of such entity, whether by contract or
21 | otherwise, or (ii) ownership of fifty percent (50%) or more of the
22 | outstanding shares, or (iii) beneficial ownership of such entity.
23 |
24 | "You" (or "Your") shall mean an individual or Legal Entity
25 | exercising permissions granted by this License.
26 |
27 | "Source" form shall mean the preferred form for making modifications,
28 | including but not limited to software source code, documentation
29 | source, and configuration files.
30 |
31 | "Object" form shall mean any form resulting from mechanical
32 | transformation or translation of a Source form, including but
33 | not limited to compiled object code, generated documentation,
34 | and conversions to other media types.
35 |
36 | "Work" shall mean the work of authorship, whether in Source or
37 | Object form, made available under the License, as indicated by a
38 | copyright notice that is included in or attached to the work
39 | (an example is provided in the Appendix below).
40 |
41 | "Derivative Works" shall mean any work, whether in Source or Object
42 | form, that is based on (or derived from) the Work and for which the
43 | editorial revisions, annotations, elaborations, or other modifications
44 | represent, as a whole, an original work of authorship. For the purposes
45 | of this License, Derivative Works shall not include works that remain
46 | separable from, or merely link (or bind by name) to the interfaces of,
47 | the Work and Derivative Works thereof.
48 |
49 | "Contribution" shall mean any work of authorship, including
50 | the original version of the Work and any modifications or additions
51 | to that Work or Derivative Works thereof, that is intentionally
52 | submitted to Licensor for inclusion in the Work by the copyright owner
53 | or by an individual or Legal Entity authorized to submit on behalf of
54 | the copyright owner. For the purposes of this definition, "submitted"
55 | means any form of electronic, verbal, or written communication sent
56 | to the Licensor or its representatives, including but not limited to
57 | communication on electronic mailing lists, source code control systems,
58 | and issue tracking systems that are managed by, or on behalf of, the
59 | Licensor for the purpose of discussing and improving the Work, but
60 | excluding communication that is conspicuously marked or otherwise
61 | designated in writing by the copyright owner as "Not a Contribution."
62 |
63 | "Contributor" shall mean Licensor and any individual or Legal Entity
64 | on behalf of whom a Contribution has been received by Licensor and
65 | subsequently incorporated within the Work.
66 |
67 | 2. Grant of Copyright License. Subject to the terms and conditions of
68 | this License, each Contributor hereby grants to You a perpetual,
69 | worldwide, non-exclusive, no-charge, royalty-free, irrevocable
70 | copyright license to reproduce, prepare Derivative Works of,
71 | publicly display, publicly perform, sublicense, and distribute the
72 | Work and such Derivative Works in Source or Object form.
73 |
74 | 3. Grant of Patent License. Subject to the terms and conditions of
75 | this License, each Contributor hereby grants to You a perpetual,
76 | worldwide, non-exclusive, no-charge, royalty-free, irrevocable
77 | (except as stated in this section) patent license to make, have made,
78 | use, offer to sell, sell, import, and otherwise transfer the Work,
79 | where such license applies only to those patent claims licensable
80 | by such Contributor that are necessarily infringed by their
81 | Contribution(s) alone or by combination of their Contribution(s)
82 | with the Work to which such Contribution(s) was submitted. If You
83 | institute patent litigation against any entity (including a
84 | cross-claim or counterclaim in a lawsuit) alleging that the Work
85 | or a Contribution incorporated within the Work constitutes direct
86 | or contributory patent infringement, then any patent licenses
87 | granted to You under this License for that Work shall terminate
88 | as of the date such litigation is filed.
89 |
90 | 4. Redistribution. You may reproduce and distribute copies of the
91 | Work or Derivative Works thereof in any medium, with or without
92 | modifications, and in Source or Object form, provided that You
93 | meet the following conditions:
94 |
95 | (a) You must give any other recipients of the Work or
96 | Derivative Works a copy of this License; and
97 |
98 | (b) You must cause any modified files to carry prominent notices
99 | stating that You changed the files; and
100 |
101 | (c) You must retain, in the Source form of any Derivative Works
102 | that You distribute, all copyright, patent, trademark, and
103 | attribution notices from the Source form of the Work,
104 | excluding those notices that do not pertain to any part of
105 | the Derivative Works; and
106 |
107 | (d) If the Work includes a "NOTICE" text file as part of its
108 | distribution, then any Derivative Works that You distribute must
109 | include a readable copy of the attribution notices contained
110 | within such NOTICE file, excluding those notices that do not
111 | pertain to any part of the Derivative Works, in at least one
112 | of the following places: within a NOTICE text file distributed
113 | as part of the Derivative Works; within the Source form or
114 | documentation, if provided along with the Derivative Works; or,
115 | within a display generated by the Derivative Works, if and
116 | wherever such third-party notices normally appear. The contents
117 | of the NOTICE file are for informational purposes only and
118 | do not modify the License. You may add Your own attribution
119 | notices within Derivative Works that You distribute, alongside
120 | or as an addendum to the NOTICE text from the Work, provided
121 | that such additional attribution notices cannot be construed
122 | as modifying the License.
123 |
124 | You may add Your own copyright statement to Your modifications and
125 | may provide additional or different license terms and conditions
126 | for use, reproduction, or distribution of Your modifications, or
127 | for any such Derivative Works as a whole, provided Your use,
128 | reproduction, and distribution of the Work otherwise complies with
129 | the conditions stated in this License.
130 |
131 | 5. Submission of Contributions. Unless You explicitly state otherwise,
132 | any Contribution intentionally submitted for inclusion in the Work
133 | by You to the Licensor shall be under the terms and conditions of
134 | this License, without any additional terms or conditions.
135 | Notwithstanding the above, nothing herein shall supersede or modify
136 | the terms of any separate license agreement you may have executed
137 | with Licensor regarding such Contributions.
138 |
139 | 6. Trademarks. This License does not grant permission to use the trade
140 | names, trademarks, service marks, or product names of the Licensor,
141 | except as required for reasonable and customary use in describing the
142 | origin of the Work and reproducing the content of the NOTICE file.
143 |
144 | 7. Disclaimer of Warranty. Unless required by applicable law or
145 | agreed to in writing, Licensor provides the Work (and each
146 | Contributor provides its Contributions) on an "AS IS" BASIS,
147 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
148 | implied, including, without limitation, any warranties or conditions
149 | of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
150 | PARTICULAR PURPOSE. You are solely responsible for determining the
151 | appropriateness of using or redistributing the Work and assume any
152 | risks associated with Your exercise of permissions under this License.
153 |
154 | 8. Limitation of Liability. In no event and under no legal theory,
155 | whether in tort (including negligence), contract, or otherwise,
156 | unless required by applicable law (such as deliberate and grossly
157 | negligent acts) or agreed to in writing, shall any Contributor be
158 | liable to You for damages, including any direct, indirect, special,
159 | incidental, or consequential damages of any character arising as a
160 | result of this License or out of the use or inability to use the
161 | Work (including but not limited to damages for loss of goodwill,
162 | work stoppage, computer failure or malfunction, or any and all
163 | other commercial damages or losses), even if such Contributor
164 | has been advised of the possibility of such damages.
165 |
166 | 9. Accepting Warranty or Additional Liability. While redistributing
167 | the Work or Derivative Works thereof, You may choose to offer,
168 | and charge a fee for, acceptance of support, warranty, indemnity,
169 | or other liability obligations and/or rights consistent with this
170 | License. However, in accepting such obligations, You may act only
171 | on Your own behalf and on Your sole responsibility, not on behalf
172 | of any other Contributor, and only if You agree to indemnify,
173 | defend, and hold each Contributor harmless for any liability
174 | incurred by, or claims asserted against, such Contributor by reason
175 | of your accepting any such warranty or additional liability.
176 |
177 | END OF TERMS AND CONDITIONS
178 |
--------------------------------------------------------------------------------
/core/src/main/scala/org/apache/spark/sql/execution/datasources/hbase/HBaseConnectionCache.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 | * File modified by Hortonworks, Inc. Modifications are also licensed under
18 | * the Apache Software License, Version 2.0.
19 | */
20 |
21 | package org.apache.spark.sql.execution.datasources.hbase
22 |
23 | import java.io.{Closeable, IOException}
24 | import java.util.concurrent.atomic.{AtomicBoolean, AtomicLong}
25 |
26 | import scala.collection.mutable
27 |
28 | import org.apache.commons.io.IOUtils
29 | import org.apache.hadoop.conf.Configuration
30 | import org.apache.hadoop.hbase.{HConstants, TableName}
31 | import org.apache.hadoop.hbase.client._
32 | import org.apache.hadoop.hbase.ipc.RpcControllerFactory
33 | import org.apache.hadoop.hbase.security.{User, UserProvider}
34 |
35 | private[spark] object HBaseConnectionCache extends Logging {
36 |
37 | // A hashmap of Spark-HBase connections. Key is HBaseConnectionKey.
38 | private[hbase] val connectionMap = new mutable.HashMap[HBaseConnectionKey, SmartConnection]()
39 |
40 | private val cacheStat = HBaseConnectionCacheStat(0, 0, 0)
41 |
42 | // in milliseconds
43 | private final val DEFAULT_TIME_OUT: Long = SparkHBaseConf.connectionCloseDelay
44 | private val timeout = new AtomicLong(DEFAULT_TIME_OUT)
45 | private val closed = new AtomicBoolean(false)
46 |
47 | private var housekeepingThread = new Thread(new Runnable {
48 | override def run() {
49 | while (true) {
50 | sleep(timeout.get(), allowInterrupt = true, allowClosed = true)
51 | if (closed.get()) return
52 | performHousekeeping(false)
53 | }
54 | }
55 | })
56 | housekeepingThread.setDaemon(true)
57 | housekeepingThread.start()
58 |
59 | // Thread.sleep can be spuriously woken up, this ensure we sleep for atleast the
60 | // 'duration' specified
61 | private[hbase] def sleep(duration: Long, allowInterrupt: Boolean = false, allowClosed: Boolean = false): Unit = {
62 | val startTime = System.currentTimeMillis()
63 | var remaining = duration
64 | while (remaining > 0) {
65 | try {
66 | Thread.sleep(remaining)
67 | } catch {
68 | case ex: InterruptedException if allowInterrupt => return
69 | case ex: Exception => // ignore
70 | }
71 | if (allowClosed && closed.get()) return
72 | val now = System.currentTimeMillis()
73 | remaining = duration - (now - startTime)
74 | }
75 | }
76 |
77 |
78 | def getStat: HBaseConnectionCacheStat = {
79 | connectionMap.synchronized {
80 | cacheStat.setActiveConnections(connectionMap.size)
81 | cacheStat.copy()
82 | }
83 | }
84 |
85 | // resetStats == true for testing, otherwise, it is not modified.
86 | private[hbase] def resetCache(resetStats: Boolean = false): Unit = {
87 | connectionMap.synchronized {
88 | if (closed.get()) return
89 | connectionMap.values.foreach(conn => IOUtils.closeQuietly(conn) )
90 | connectionMap.clear()
91 | if (resetStats) cacheStat.reset()
92 | }
93 | }
94 |
95 | def close(): Unit = {
96 | connectionMap.synchronized {
97 | if (closed.get()) return
98 | try {
99 | housekeepingThread.interrupt()
100 | resetCache()
101 | } finally {
102 | housekeepingThread = null
103 | closed.set(true)
104 | }
105 | }
106 | }
107 |
108 | private[hbase] def performHousekeeping(forceClean: Boolean) = {
109 | val tsNow: Long = System.currentTimeMillis()
110 | val connTimeout = timeout.get()
111 | connectionMap.synchronized {
112 | connectionMap.retain {
113 | (key, conn) => {
114 | if(conn.refCount < 0) {
115 | logError("Bug to be fixed: negative refCount")
116 | }
117 |
118 | if(forceClean || ((conn.refCount <= 0) && (tsNow - conn.timestamp > connTimeout))) {
119 | IOUtils.closeQuietly(conn.connection)
120 | false
121 | } else {
122 | true
123 | }
124 | }
125 | }
126 | }
127 | }
128 |
129 | // For testing purpose only
130 | def getConnection(key: HBaseConnectionKey, conn: => Connection): SmartConnection = {
131 | connectionMap.synchronized {
132 | if (closed.get()) return null
133 | val sc = connectionMap.getOrElseUpdate(key, {
134 | cacheStat.incrementActualConnectionsCreated(1)
135 | new SmartConnection(conn)
136 | })
137 | cacheStat.incrementTotalRequests(1)
138 | sc.refCount += 1
139 | sc
140 | }
141 | }
142 |
143 | def getConnection(conf: Configuration): SmartConnection =
144 | getConnection(new HBaseConnectionKey(conf), ConnectionFactory.createConnection(conf))
145 |
146 | // For testing purpose only
147 | def setTimeout(to: Long) : Unit = {
148 | connectionMap.synchronized {
149 | if (closed.get()) return
150 | timeout.set(to)
151 | housekeepingThread.interrupt()
152 | }
153 | }
154 | }
155 |
156 | private[hbase] class SmartConnection (
157 | val connection: Connection, var refCount: Int = 0, var timestamp: Long = 0) extends Closeable {
158 | def getTable(tableName: TableName): Table = connection.getTable(tableName)
159 | def getRegionLocator(tableName: TableName): RegionLocator = connection.getRegionLocator(tableName)
160 | def isClosed: Boolean = connection.isClosed
161 | def getAdmin: Admin = connection.getAdmin
162 | def close() = {
163 | HBaseConnectionCache.connectionMap.synchronized {
164 | refCount -= 1
165 | if(refCount <= 0)
166 | timestamp = System.currentTimeMillis()
167 | }
168 | }
169 | }
170 |
171 | /**
172 | * Denotes a unique key to an HBase Connection instance.
173 | * Please refer to 'org.apache.hadoop.hbase.client.HConnectionKey'.
174 | *
175 | * In essence, this class captures the properties in Configuration
176 | * that may be used in the process of establishing a connection.
177 | *
178 | */
179 | class HBaseConnectionKey(c: Configuration) extends Logging {
180 | import HBaseConnectionKey.CONNECTION_PROPERTIES
181 |
182 | val (username, properties) = {
183 | var user: String = null
184 |
185 | val confMap = {
186 | if (c != null) {
187 | try {
188 | val provider: UserProvider = UserProvider.instantiate(c)
189 | val currentUser: User = provider.getCurrent
190 | if (currentUser != null) {
191 | user = currentUser.getName
192 | }
193 | }
194 | catch {
195 | case e: IOException =>
196 | logWarning("Error obtaining current user, skipping username in HBaseConnectionKey", e)
197 | }
198 |
199 | CONNECTION_PROPERTIES.flatMap(key =>
200 | Option(c.get(key)).map(value => key -> value)
201 | ).toMap
202 | } else {
203 | Map[String, String]()
204 | }
205 | }
206 | (user, confMap)
207 | }
208 |
209 | override def toString: String = {
210 | s"HBaseConnectionKey{username='$username, properties=$properties}"
211 | }
212 |
213 | override def equals(other: Any): Boolean = other match {
214 | case that: HBaseConnectionKey =>
215 | username == that.username &&
216 | CONNECTION_PROPERTIES.forall(key => this.properties.get(key) == that.properties.get(key))
217 | case _ => false
218 | }
219 |
220 | override def hashCode(): Int = {
221 | val userHashCode = if (null != username) username.hashCode else 0
222 | CONNECTION_PROPERTIES.flatMap(k => properties.get(k)).
223 | foldLeft(userHashCode)((a, b) => 31 * a + (if (null != b) b.hashCode else 0))
224 | }
225 | }
226 |
227 | private[hbase] object HBaseConnectionKey {
228 | private val CONNECTION_PROPERTIES: Array[String] = Array[String](
229 | HConstants.ZOOKEEPER_QUORUM,
230 | HConstants.ZOOKEEPER_ZNODE_PARENT,
231 | HConstants.ZOOKEEPER_CLIENT_PORT,
232 | HConstants.ZOOKEEPER_RECOVERABLE_WAITTIME,
233 | HConstants.HBASE_CLIENT_PAUSE,
234 | HConstants.HBASE_CLIENT_RETRIES_NUMBER,
235 | HConstants.HBASE_RPC_TIMEOUT_KEY,
236 | HConstants.HBASE_META_SCANNER_CACHING,
237 | HConstants.HBASE_CLIENT_INSTANCE_ID,
238 | HConstants.RPC_CODEC_CONF_KEY,
239 | HConstants.USE_META_REPLICAS,
240 | RpcControllerFactory.CUSTOM_CONTROLLER_CONF_KEY)
241 |
242 | }
243 |
244 |
245 | /**
246 | * To log the state of [[HBaseConnectionCache]]
247 | *
248 | * numTotalRequests: number of total connection requests to the cache
249 | * numActualConnectionsCreated: number of actual HBase connections the cache ever created
250 | * numActiveConnections: number of current alive HBase connections the cache is holding
251 | */
252 | class HBaseConnectionCacheStat(private var _numTotalRequests: Long,
253 | private var _numActualConnectionsCreated: Long,
254 | private var _numActiveConnections: Long) {
255 |
256 | def numTotalRequests: Long = _numTotalRequests
257 | def numActualConnectionsCreated: Long = _numActualConnectionsCreated
258 | def numActiveConnections: Long = _numActiveConnections
259 |
260 |
261 | private[hbase] def incrementActualConnectionsCreated(incr: Long) = {
262 | _numActualConnectionsCreated += incr
263 | }
264 |
265 | private[hbase] def incrementTotalRequests(incr: Long) = {
266 | _numTotalRequests += incr
267 | }
268 |
269 | private[hbase] def setActiveConnections(numActiveConnections: Long) = {
270 | this._numActiveConnections = numActiveConnections
271 | }
272 |
273 | private[hbase] def copy(): HBaseConnectionCacheStat =
274 | HBaseConnectionCacheStat(numTotalRequests, numActualConnectionsCreated, numActiveConnections)
275 |
276 | // inplace update to reset - for tests
277 | private[hbase] def reset(): Unit = {
278 | _numTotalRequests = 0
279 | _numActualConnectionsCreated = 0
280 | _numActiveConnections = 0
281 | }
282 | }
283 |
284 | object HBaseConnectionCacheStat {
285 | def apply(numTotalRequests: Long,
286 | numActualConnectionsCreated: Long,
287 | numActiveConnections: Long): HBaseConnectionCacheStat =
288 | new HBaseConnectionCacheStat(numTotalRequests, numActualConnectionsCreated, numActiveConnections)
289 |
290 | def unapply(stat: HBaseConnectionCacheStat): Option[(Long, Long, Long)] =
291 | Some((stat.numTotalRequests, stat.numActualConnectionsCreated, stat.numActiveConnections))
292 |
293 | }
--------------------------------------------------------------------------------