├── .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 | } --------------------------------------------------------------------------------