/
MLSQLHbase.scala
148 lines (118 loc) · 5.05 KB
/
MLSQLHbase.scala
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
/*
* 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 _root_.streaming.core.datasource._
import _root_.streaming.dsl.mmlib.algs.param.{BaseParams, WowParams}
import _root_.streaming.dsl.{ConnectMeta, DBMappingKey}
import org.apache.spark.ml.param.Param
import org.apache.spark.sql._
/**
* Created by latincross on 12/29/2018.
*/
class MLSQLHbase(override val uid: String) extends MLSQLSource with MLSQLSink with MLSQLSourceInfo with MLSQLRegistry with WowParams {
def this() = this(BaseParams.randomUID())
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 format = config.config.getOrElse("implClass", fullFormat)
val Array(connect, namespace, table) = if (config.path.contains(dbSplitter)) {
config.path.split(dbSplitter) match {
case Array(connect, namespace, table) => Array(connect, namespace, table)
case Array(connectOrNameSpace, table) =>
ConnectMeta.presentThenCall(DBMappingKey(format, connectOrNameSpace), (op) => {}) match {
case Some(i) => Array(connectOrNameSpace, "", table)
case None => Array("", connectOrNameSpace, table)
}
case Array(connect, namespace, table, _*) => Array(connect, namespace, table)
}
} else {
Array("", "", config.path)
}
var finalNameSpace = config.config.getOrElse("namespace", namespace)
ConnectMeta.presentThenCall(DBMappingKey(format, connect), (options) => {
if (options.contains("namespace")) {
finalNameSpace = options.get("namespace").get
}
})
SourceInfo(shortFormat, finalNameSpace, table)
}
override def explainParams(spark: SparkSession) = {
_explainParams(spark)
}
final val zk: Param[String] = new Param[String](this, "zk", "zk address")
final val family: Param[String] = new Param[String](this, "family", "default cf")
}