Skip to content

Commit

Permalink
Hbase datasource
Browse files Browse the repository at this point in the history
  • Loading branch information
耿一斐 committed Jan 8, 2019
1 parent 8460ebd commit 0b6ac42
Show file tree
Hide file tree
Showing 3 changed files with 257 additions and 0 deletions.
Original file line number Diff line number Diff line change
@@ -0,0 +1,133 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package streaming.core.datasource.impl

import org.apache.spark.sql.{DataFrame, DataFrameReader, DataFrameWriter, Row}
import streaming.core.datasource._
import streaming.dsl.{ConnectMeta, DBMappingKey}

/**
* Created by latincross on 12/29/2018.
*/
class MLSQLHbase extends MLSQLSource with MLSQLSink with MLSQLSourceInfo with MLSQLRegistry {


override def fullFormat: String = "org.apache.spark.sql.execution.datasources.hbase"

override def shortFormat: String = "hbase"

override def dbSplitter: String = "/"

override def load(reader: DataFrameReader, config: DataSourceConfig): DataFrame = {
val Array(_dbname, _dbtable) = if (config.path.contains(dbSplitter)) {
config.path.split(dbSplitter, 2)
}else{
Array("" ,config.path)
}

var namespace = ""

val format = config.config.getOrElse("implClass", fullFormat)
if (_dbname != "") {
ConnectMeta.presentThenCall(DBMappingKey(format, _dbname), options => {
if(options.contains("namespace")){
namespace = options("namespace")
}
reader.options(options)
})
}

if (config.config.contains("namespace")){
namespace = config.config("namespace")
}

val inputTableName = if (namespace == "") _dbtable else s"${namespace}:${_dbtable}"

reader.option("inputTableName" ,inputTableName)

//load configs should overwrite connect configs
reader.options(config.config)
reader.format(format).load()
}

override def save(writer: DataFrameWriter[Row], config: DataSinkConfig): Unit = {
val Array(_dbname, _dbtable) = if (config.path.contains(dbSplitter)) {
config.path.split(dbSplitter, 2)
}else{
Array("" ,config.path)
}

var namespace = ""

val format = config.config.getOrElse("implClass", fullFormat)
if (_dbname != "") {
ConnectMeta.presentThenCall(DBMappingKey(format, _dbname), options => {
if(options.contains("namespace")){
namespace = options.get("namespace").get
}
writer.options(options)
})
}

if (config.config.contains("namespace")){
namespace = config.config.get("namespace").get
}

val outputTableName = if (namespace == "") _dbtable else s"${namespace}:${_dbtable}"

writer.mode(config.mode)
writer.option("outputTableName", outputTableName)
//load configs should overwrite connect configs
writer.options(config.config)
config.config.get("partitionByCol").map { item =>
writer.partitionBy(item.split(","): _*)
}
writer.format(config.config.getOrElse("implClass", fullFormat)).save()
}

override def register(): Unit = {
DataSourceRegistry.register(MLSQLDataSourceKey(fullFormat, MLSQLSparkDataSourceType), this)
DataSourceRegistry.register(MLSQLDataSourceKey(shortFormat, MLSQLSparkDataSourceType), this)
}

override def sourceInfo(config: DataAuthConfig): SourceInfo = {
val Array(_dbname, _dbtable) = if (config.path.contains(dbSplitter)) {
config.path.split(dbSplitter, 2)
}else{
Array("" ,config.path)
}

var namespace = _dbname

if (config.config.contains("namespace")){
namespace = config.config.get("namespace").get
}else{
if (_dbname != "") {
val format = config.config.getOrElse("implClass", fullFormat)
ConnectMeta.presentThenCall(DBMappingKey(format, _dbname), options => {
if(options.contains("namespace")){
namespace = options.get("namespace").get
}
})
}
}

SourceInfo(shortFormat ,namespace ,_dbtable)
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,72 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package streaming.test.datasource

import org.apache.spark.streaming.BasicSparkOperation
import org.scalatest.BeforeAndAfterAll
import streaming.core.strategy.platform.SparkRuntime
import streaming.core.{BasicMLSQLConfig, SpecFunctions}
import streaming.dsl.ScriptSQLExec
import streaming.log.Logging

/**
* Created by latincross on 12/27/2018.
*/
class HbaseSpec extends BasicSparkOperation with SpecFunctions with BasicMLSQLConfig with BeforeAndAfterAll with Logging {
"load hbase" should "work fine" in {

withBatchContext(setupBatchContext(batchParams, "classpath:///test/empty.json")) { runtime: SparkRuntime =>
//执行sql
implicit val spark = runtime.sparkSession

var sq = createSSEL
ScriptSQLExec.parse(
s"""
|
|connect hbase where `zk`="127.0.0.1:2181"
|and `family`="cf" as hbase1;
|
|load hbase.`hbase1/mlsql_example`
|as mlsql_example;
|
|select * from mlsql_example as show_data;
|
|
|select '2' as rowkey, 'insert test data' as name as insert_table;
|
|save insert_table as hbase.`hbase1/mlsql_example`;
|
""".stripMargin, sq)
assume(spark.sql("select * from mlsql_example where rowkey='2' ").collect().last.get(0) == "2")
}
}

val server = new streaming.test.servers.HbaseServer("1.2")

override protected def beforeAll(): Unit = {
server.startServer
server.exec("hbase", s"""echo \"create 'mlsql_example','cf'\"| hbase shell""")
server.exec("hbase", s"""echo \"put 'mlsql_example','1','cf:name','this is a test data'\"| hbase shell""")

}

override protected def afterAll(): Unit = {
server.stopServer
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,52 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package streaming.test.servers

import java.net.InetAddress

/**
* Created by latincross on 12/28/2018.
*/
class HbaseServer(version: String) extends WowBaseTestServer {

override def composeYaml: String =
s"""
|version: '2.1'
|services:
| hbase:
| image: harisekhon/hbase:${version}
| ports:
| - "2181:2181"
| - "8080:8080"
| - "8085:8085"
| - "9090:9090"
| - "9095:9095"
| - "16000:16000"
| - "16010:16010"
| - "16201:16201"
| - "16301:16301"
| hostname: ${InetAddress.getLocalHost.getHostName}
""".stripMargin

override def waitToServiceReady: Boolean = {
// wait mongo to ready, runs on host server
val shellCommand = s"""echo 'list'| hbase shell"""
readyCheck("hbase", shellCommand)
}
}

0 comments on commit 0b6ac42

Please sign in to comment.