-
Notifications
You must be signed in to change notification settings - Fork 79
/
Copy pathSparkCobolOptionsBuilder.scala
120 lines (101 loc) · 6.02 KB
/
SparkCobolOptionsBuilder.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
/*
* Copyright 2018 ABSA Group Limited
*
* 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 za.co.absa.cobrix.spark.cobol.builder
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.catalyst.expressions.GenericRow
import org.apache.spark.sql.{DataFrame, Row, SparkSession}
import za.co.absa.cobrix.cobol.parser.CopybookParser
import za.co.absa.cobrix.cobol.parser.encoding.codepage.CodePage
import za.co.absa.cobrix.cobol.parser.encoding.{ASCII, EBCDIC}
import za.co.absa.cobrix.cobol.reader.extractors.record.RecordExtractors
import za.co.absa.cobrix.cobol.reader.parameters.{CobolParameters, ReaderParameters}
import za.co.absa.cobrix.spark.cobol.reader.RowHandler
import za.co.absa.cobrix.spark.cobol.schema.CobolSchema
import java.nio.charset.{Charset, StandardCharsets}
import scala.collection.mutable
class SparkCobolOptionsBuilder(copybookContent: String)(implicit spark: SparkSession) {
private val caseInsensitiveOptions = new mutable.HashMap[String, String]()
def option(key: String, value: String): SparkCobolOptionsBuilder = {
caseInsensitiveOptions += (key.toLowerCase -> value)
this
}
def options(options: Map[String, String]): SparkCobolOptionsBuilder = {
caseInsensitiveOptions ++= options.map(kv => (kv._1.toLowerCase(), kv._2))
this
}
def load(rdd: RDD[Array[Byte]]): DataFrame = {
val params = RddReaderParams.forBinary(caseInsensitiveOptions.toMap)
getDataFrame(rdd, params)
}
def loadText(rddText: RDD[String]): DataFrame = {
val params = RddReaderParams.forText(caseInsensitiveOptions.toMap)
getDataFrame(rddText.map(_.getBytes()), params)
}
private[cobol] def getDataFrame(rdd: RDD[Array[Byte]], readerParams: ReaderParameters): DataFrame = {
val encoding = if (readerParams.isEbcdic) EBCDIC else ASCII
val asciiCharset = if (readerParams.asciiCharset.isEmpty) StandardCharsets.UTF_8 else Charset.forName(readerParams.asciiCharset)
val segmentRedefines = readerParams.multisegment.map(r => r.segmentIdRedefineMap.values.toList.distinct).getOrElse(Nil)
val parsedCopybook = CopybookParser.parse(copyBookContents = copybookContent,
dataEncoding = encoding,
dropGroupFillers = readerParams.dropGroupFillers,
dropValueFillers = readerParams.dropValueFillers,
fillerNamingPolicy = readerParams.fillerNamingPolicy,
segmentRedefines = segmentRedefines,
fieldParentMap = readerParams.multisegment.map(_.fieldParentMap).getOrElse(Map.empty),
stringTrimmingPolicy = readerParams.stringTrimmingPolicy,
commentPolicy = readerParams.commentPolicy,
strictSignOverpunch = readerParams.strictSignOverpunch,
improvedNullDetection = readerParams.improvedNullDetection,
ebcdicCodePage = getCodePage(readerParams.ebcdicCodePage, readerParams.ebcdicCodePageClass),
asciiCharset = asciiCharset,
isUtf16BigEndian = readerParams.isUtf16BigEndian,
floatingPointFormat = readerParams.floatingPointFormat,
nonTerminals = readerParams.nonTerminals,
occursHandlers = readerParams.occursMappings,
debugFieldsPolicy = readerParams.debugFieldsPolicy,
fieldCodePageMap = readerParams.fieldCodePage
)
val cobolSchema = new CobolSchema(parsedCopybook,
readerParams.schemaPolicy,
inputFileNameField = "",
generateRecordId = false,
readerParams.generateRecordBytes,
metadataPolicy = readerParams.metadataPolicy)
val sparkSchema = cobolSchema.getSparkSchema
val recordHandler = new RowHandler()
val schemaRetentionPolicy = readerParams.schemaPolicy
val minimumRecordLength = readerParams.minimumRecordLength
val maximumRecordLength = readerParams.maximumRecordLength
val rddRow = rdd
.filter(array => array.nonEmpty && array.length >= minimumRecordLength && array.length <= maximumRecordLength)
.map(array => {
val record = RecordExtractors.extractRecord[GenericRow](parsedCopybook.ast,
array,
0,
schemaRetentionPolicy,
generateRecordBytes = readerParams.generateRecordBytes,
handler = recordHandler)
Row.fromSeq(record)
})
spark.createDataFrame(rddRow, sparkSchema)
}
private def getCodePage(codePageName: String, codePageClass: Option[String]): CodePage = {
codePageClass match {
case Some(c) => CodePage.getCodePageByClass(c)
case None => CodePage.getCodePageByName(codePageName)
}
}
}