-
Notifications
You must be signed in to change notification settings - Fork 98
/
DefaultSource.scala
286 lines (253 loc) · 11.5 KB
/
DefaultSource.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
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
/*
* Copyright 2015 springml
*
* Licensed 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 com.springml.spark.sftp
import java.io.File
import java.util.UUID
import com.springml.sftp.client.SFTPClient
import com.springml.spark.sftp.util.Utils.ImplicitDataFrameWriter
import org.apache.commons.io.FilenameUtils
import org.apache.hadoop.fs.Path
import org.apache.log4j.Logger
import org.apache.spark.sql.sources.{BaseRelation, CreatableRelationProvider, RelationProvider, SchemaRelationProvider}
import org.apache.spark.sql.types.StructType
import org.apache.spark.sql.{DataFrame, SQLContext, SaveMode}
/**
* Datasource to construct dataframe from a sftp url
*/
class DefaultSource extends RelationProvider with SchemaRelationProvider with CreatableRelationProvider {
@transient val logger = Logger.getLogger(classOf[DefaultSource])
/**
* Copy the file from SFTP to local location and then create dataframe using local file
*/
override def createRelation(sqlContext: SQLContext, parameters: Map[String, String]):BaseRelation = {
createRelation(sqlContext, parameters, null)
}
/**
* Copy the file from SFTP to local location and then create dataframe using local file
*/
override def createRelation(sqlContext: SQLContext, parameters: Map[String, String], schema: StructType) = {
val username = parameters.get("username")
val password = parameters.get("password")
val pemFileLocation = parameters.get("pem")
val pemPassphrase = parameters.get("pemPassphrase")
val host = parameters.getOrElse("host", sys.error("SFTP Host has to be provided using 'host' option"))
val port = parameters.get("port")
val path = parameters.getOrElse("path", sys.error("'path' must be specified"))
val fileType = parameters.getOrElse("fileType", sys.error("File type has to be provided using 'fileType' option"))
val inferSchema = parameters.get("inferSchema")
val header = parameters.getOrElse("header", "true")
val delimiter = parameters.getOrElse("delimiter", ",")
val quote = parameters.getOrElse("quote", "\"")
val escape = parameters.getOrElse("escape", "\\")
val multiLine = parameters.getOrElse("multiLine", "false")
val createDF = parameters.getOrElse("createDF", "true")
val copyLatest = parameters.getOrElse("copyLatest", "false")
val tempFolder = parameters.getOrElse("tempLocation", System.getProperty("java.io.tmpdir"))
val hdfsTemp = parameters.getOrElse("hdfsTempLocation", tempFolder)
val cryptoKey = parameters.getOrElse("cryptoKey", null)
val cryptoAlgorithm = parameters.getOrElse("cryptoAlgorithm", "AES")
val rowTag = parameters.getOrElse(constants.xmlRowTag, null)
val supportedFileTypes = List("csv", "json", "avro", "parquet", "txt", "xml","orc")
if (!supportedFileTypes.contains(fileType)) {
sys.error("fileType " + fileType + " not supported. Supported file types are " + supportedFileTypes)
}
val inferSchemaFlag = if (inferSchema != null && inferSchema.isDefined) {
inferSchema.get
} else {
"false"
}
val sftpClient = getSFTPClient(username, password, pemFileLocation, pemPassphrase, host, port,
cryptoKey, cryptoAlgorithm)
val copiedFileLocation = copy(sftpClient, path, tempFolder, copyLatest.toBoolean)
val fileLocation = copyToHdfs(sqlContext, copiedFileLocation, hdfsTemp)
if (!createDF.toBoolean) {
logger.info("Returning an empty dataframe after copying files...")
createReturnRelation(sqlContext, schema)
} else {
DatasetRelation(fileLocation, fileType, inferSchemaFlag, header, delimiter, quote, escape, multiLine, rowTag, schema,
sqlContext)
}
}
override def createRelation(
sqlContext: SQLContext,
mode: SaveMode,
parameters: Map[String, String],
data: DataFrame): BaseRelation = {
val username = parameters.get("username")
val password = parameters.get("password")
val pemFileLocation = parameters.get("pem")
val pemPassphrase = parameters.get("pemPassphrase")
val host = parameters.getOrElse("host", sys.error("SFTP Host has to be provided using 'host' option"))
val port = parameters.get("port")
val path = parameters.getOrElse("path", sys.error("'path' must be specified"))
val fileType = parameters.getOrElse("fileType", sys.error("File type has to be provided using 'fileType' option"))
val header = parameters.getOrElse("header", "true")
val copyLatest = parameters.getOrElse("copyLatest", "false")
val tmpFolder = parameters.getOrElse("tempLocation", System.getProperty("java.io.tmpdir"))
val hdfsTemp = parameters.getOrElse("hdfsTempLocation", tmpFolder)
val cryptoKey = parameters.getOrElse("cryptoKey", null)
val cryptoAlgorithm = parameters.getOrElse("cryptoAlgorithm", "AES")
val delimiter = parameters.getOrElse("delimiter", ",")
val quote = parameters.getOrElse("quote", "\"")
val escape = parameters.getOrElse("escape", "\\")
val multiLine = parameters.getOrElse("multiLine", "false")
val codec = parameters.getOrElse("codec", null)
val rowTag = parameters.getOrElse(constants.xmlRowTag, null)
val rootTag = parameters.getOrElse(constants.xmlRootTag, null)
val supportedFileTypes = List("csv", "json", "avro", "parquet", "txt", "xml","orc")
if (!supportedFileTypes.contains(fileType)) {
sys.error("fileType " + fileType + " not supported. Supported file types are " + supportedFileTypes)
}
val sftpClient = getSFTPClient(username, password, pemFileLocation, pemPassphrase, host, port,
cryptoKey, cryptoAlgorithm)
val tempFile = writeToTemp(sqlContext, data, hdfsTemp, tmpFolder, fileType, header, delimiter, quote, escape, multiLine, codec, rowTag, rootTag)
upload(tempFile, path, sftpClient)
return createReturnRelation(data)
}
private def copyToHdfs(sqlContext: SQLContext, fileLocation : String,
hdfsTemp : String): String = {
val hadoopConf = sqlContext.sparkContext.hadoopConfiguration
val hdfsPath = new Path(fileLocation)
val fs = hdfsPath.getFileSystem(hadoopConf)
if ("hdfs".equalsIgnoreCase(fs.getScheme)) {
fs.copyFromLocalFile(new Path(fileLocation), new Path(hdfsTemp))
val filePath = hdfsTemp + "/" + hdfsPath.getName
fs.deleteOnExit(new Path(filePath))
return filePath
} else {
return fileLocation
}
}
private def copyFromHdfs(sqlContext: SQLContext, hdfsTemp : String,
fileLocation : String): String = {
val hadoopConf = sqlContext.sparkContext.hadoopConfiguration
val hdfsPath = new Path(hdfsTemp)
val fs = hdfsPath.getFileSystem(hadoopConf)
if ("hdfs".equalsIgnoreCase(fs.getScheme)) {
fs.copyToLocalFile(new Path(hdfsTemp), new Path(fileLocation))
fs.deleteOnExit(new Path(hdfsTemp))
return fileLocation
} else {
return hdfsTemp
}
}
private def upload(source: String, target: String, sftpClient: SFTPClient) {
logger.info("Copying " + source + " to " + target)
sftpClient.copyToFTP(source, target)
}
private def getSFTPClient(
username: Option[String],
password: Option[String],
pemFileLocation: Option[String],
pemPassphrase: Option[String],
host: String,
port: Option[String],
cryptoKey : String,
cryptoAlgorithm : String) : SFTPClient = {
val sftpPort = if (port != null && port.isDefined) {
port.get.toInt
} else {
22
}
val cryptoEnabled = cryptoKey != null
if (cryptoEnabled) {
new SFTPClient(getValue(pemFileLocation), getValue(pemPassphrase), getValue(username),
getValue(password),
host, sftpPort, cryptoEnabled, cryptoKey, cryptoAlgorithm)
} else {
new SFTPClient(getValue(pemFileLocation), getValue(pemPassphrase), getValue(username),
getValue(password), host, sftpPort)
}
}
private def createReturnRelation(data: DataFrame): BaseRelation = {
createReturnRelation(data.sqlContext, data.schema)
}
private def createReturnRelation(sqlContextVar: SQLContext, schemaVar: StructType): BaseRelation = {
new BaseRelation {
override def sqlContext: SQLContext = sqlContextVar
override def schema: StructType = schemaVar
}
}
private def copy(sftpClient: SFTPClient, source: String,
tempFolder: String, latest: Boolean): String = {
var copiedFilePath: String = null
try {
val target = tempFolder + File.separator + FilenameUtils.getName(source)
copiedFilePath = target
if (latest) {
copiedFilePath = sftpClient.copyLatest(source, tempFolder)
} else {
logger.info("Copying " + source + " to " + target)
copiedFilePath = sftpClient.copy(source, target)
}
copiedFilePath
} finally {
addShutdownHook(copiedFilePath)
}
}
private def getValue(param: Option[String]): String = {
if (param != null && param.isDefined) {
param.get
} else {
null
}
}
private def writeToTemp(sqlContext: SQLContext, df: DataFrame,
hdfsTemp: String, tempFolder: String, fileType: String, header: String,
delimiter: String, quote: String, escape: String, multiLine: String, codec: String, rowTag: String, rootTag: String) : String = {
val randomSuffix = "spark_sftp_connection_temp_" + UUID.randomUUID
val hdfsTempLocation = hdfsTemp + File.separator + randomSuffix
val localTempLocation = tempFolder + File.separator + randomSuffix
addShutdownHook(localTempLocation)
fileType match {
case "xml" => df.coalesce(1).write.format(constants.xmlClass)
.option(constants.xmlRowTag, rowTag)
.option(constants.xmlRootTag, rootTag).save(hdfsTempLocation)
case "csv" => df.coalesce(1).
write.
option("header", header).
option("delimiter", delimiter).
option("quote", quote).
option("escape", escape).
option("multiLine", multiLine).
optionNoNull("codec", Option(codec)).
csv(hdfsTempLocation)
case "txt" => df.coalesce(1).write.text(hdfsTempLocation)
case "avro" => df.coalesce(1).write.format("com.databricks.spark.avro").save(hdfsTempLocation)
case _ => df.coalesce(1).write.format(fileType).save(hdfsTempLocation)
}
copyFromHdfs(sqlContext, hdfsTempLocation, localTempLocation)
copiedFile(localTempLocation)
}
private def addShutdownHook(tempLocation: String) {
logger.debug("Adding hook for file " + tempLocation)
val hook = new DeleteTempFileShutdownHook(tempLocation)
Runtime.getRuntime.addShutdownHook(hook)
}
private def copiedFile(tempFileLocation: String) : String = {
val baseTemp = new File(tempFileLocation)
val files = baseTemp.listFiles().filter { x =>
(!x.isDirectory()
&& !x.getName.contains("SUCCESS")
&& !x.isHidden()
&& !x.getName.contains(".crc")
&& !x.getName.contains("_committed_")
&& !x.getName.contains("_started_")
)
}
files(0).getAbsolutePath
}
}