forked from byzer-org/byzer-lang
-
Notifications
You must be signed in to change notification settings - Fork 0
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
- Loading branch information
耿一斐
committed
Jan 8, 2019
1 parent
8460ebd
commit 0b6ac42
Showing
3 changed files
with
257 additions
and
0 deletions.
There are no files selected for viewing
133 changes: 133 additions & 0 deletions
133
streamingpro-mlsql/src/main/java/streaming/core/datasource/impl/MLSQLHbase.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
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) | ||
} | ||
} |
72 changes: 72 additions & 0 deletions
72
streamingpro-mlsql/src/test/scala/streaming/test/datasource/HbaseSpec.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
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 | ||
} | ||
} |
52 changes: 52 additions & 0 deletions
52
streamingpro-mlsql/src/test/scala/streaming/test/servers/HbaseServer.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
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) | ||
} | ||
} |