@@ -20,12 +20,14 @@ import org.apache.hadoop.fs.Path
2020import org .apache .spark .rdd .RDD
2121import org .apache .spark .sql .SparkSession
2222import org .slf4j .LoggerFactory
23+ import za .co .absa .cobrix .cobol .parser .Copybook
2324import za .co .absa .cobrix .cobol .processor .impl .CobolProcessorBase
2425import za .co .absa .cobrix .cobol .processor .{CobolProcessingStrategy , CobolProcessor , SerializableRawRecordProcessor }
2526import za .co .absa .cobrix .cobol .reader .common .Constants
2627import za .co .absa .cobrix .cobol .reader .index .entry .SparseIndexEntry
2728import za .co .absa .cobrix .cobol .reader .parameters .CobolParametersParser .PARAM_GENERATE_RECORD_ID
2829import za .co .absa .cobrix .cobol .reader .parameters .{CobolParameters , CobolParametersParser , Parameters }
30+ import za .co .absa .cobrix .cobol .reader .schema .CobolSchema
2931import za .co .absa .cobrix .spark .cobol .reader .VarLenReader
3032import za .co .absa .cobrix .spark .cobol .source .index .IndexBuilder
3133import za .co .absa .cobrix .spark .cobol .source .parameters .LocalityParameters
@@ -73,10 +75,6 @@ object SparkCobolProcessor {
7375 throw new IllegalArgumentException (" Copybook contents must be provided." )
7476 }
7577
76- if (rawRecordProcessorOpt.isEmpty) {
77- throw new IllegalArgumentException (" A RawRecordProcessor must be provided." )
78- }
79-
8078 if (numberOfThreads < 1 ) {
8179 throw new IllegalArgumentException (" Number of threads must be at least 1." )
8280 }
@@ -85,23 +83,7 @@ object SparkCobolProcessor {
8583 throw new IllegalArgumentException (" At least one input file must be provided." )
8684 }
8785
88- new SparkCobolProcessorLoader (filePaths, copybookContentsOpt.get, rawRecordProcessorOpt.get, cobolProcessingStrategy, numberOfThreads, caseInsensitiveOptions.toMap)
89- }
90-
91- def toRDD (path : String ): RDD [Array [Byte ]] = {
92- val filePaths = FileUtils
93- .getFiles(path, spark.sparkContext.hadoopConfiguration)
94-
95- toRDD(filePaths)
96- }
97-
98- def toRDD (filePaths : Seq [String ]): RDD [Array [Byte ]] = {
99- if (copybookContentsOpt.isEmpty) {
100- throw new IllegalArgumentException (" Copybook contents must be provided." )
101- }
102-
103- val sconf = new SerializableConfiguration (spark.sparkContext.hadoopConfiguration)
104- getRecordRdd(filePaths, copybookContentsOpt.get, caseInsensitiveOptions.toMap, sconf)
86+ new SparkCobolProcessorLoader (filePaths, copybookContentsOpt.get, rawRecordProcessorOpt, cobolProcessingStrategy, numberOfThreads, caseInsensitiveOptions.toMap)
10587 }
10688
10789 def withCopybookContents (copybookContents : String ): SparkCobolProcessorBuilder = {
@@ -154,12 +136,16 @@ object SparkCobolProcessor {
154136
155137 class SparkCobolProcessorLoader (filesToRead : Seq [String ],
156138 copybookContents : String ,
157- rawRecordProcessor : SerializableRawRecordProcessor ,
139+ rawRecordProcessorOpt : Option [ SerializableRawRecordProcessor ] ,
158140 cobolProcessingStrategy : CobolProcessingStrategy ,
159141 numberOfThreads : Int ,
160142 options : Map [String , String ])
161143 (implicit spark : SparkSession ) {
162144 def save (outputPath : String ): Long = {
145+ if (rawRecordProcessorOpt.isEmpty) {
146+ throw new IllegalArgumentException (" A RawRecordProcessor must be provided." )
147+ }
148+
163149 val cobolProcessor = CobolProcessor .builder
164150 .withCopybookContents(copybookContents)
165151 .withProcessingStrategy(cobolProcessingStrategy)
@@ -170,14 +156,25 @@ object SparkCobolProcessor {
170156 private val sconf = new SerializableConfiguration (spark.sparkContext.hadoopConfiguration)
171157
172158 override def process (listOfFiles : Seq [String ], outputPath : String ): Long = {
173- getFileProcessorRdd(listOfFiles, outputPath, copybookContents, cobolProcessor, rawRecordProcessor , sconf, numberOfThreads)
159+ getFileProcessorRdd(listOfFiles, outputPath, cobolProcessor, rawRecordProcessorOpt.get , sconf, numberOfThreads)
174160 .reduce(_ + _)
175161 }
176162 }
177163
178164 log.info(s " Writing to $outputPath... " )
179165 processor.process(filesToRead, outputPath)
180166 }
167+
168+ def getParsedCopybook : Copybook = {
169+ val cobolParameters = getCobolParameters(filesToRead, copybookContents, options, ignoreRedundantOptions = true )
170+ val readerParameters = CobolParametersParser .getReaderProperties(cobolParameters, None )
171+ CobolSchema .fromReaderParameters(Seq (copybookContents), readerParameters).copybook
172+ }
173+
174+ def toRDD : RDD [Array [Byte ]] = {
175+ val sconf = new SerializableConfiguration (spark.sparkContext.hadoopConfiguration)
176+ getRecordRdd(filesToRead, copybookContents, options, sconf)
177+ }
181178 }
182179
183180 def builder (implicit spark : SparkSession ): SparkCobolProcessorBuilder = {
@@ -186,7 +183,6 @@ object SparkCobolProcessor {
186183
187184 private def getFileProcessorRdd (listOfFiles : Seq [String ],
188185 outputPath : String ,
189- copybookContents : String ,
190186 cobolProcessor : CobolProcessor ,
191187 rawRecordProcessor : SerializableRawRecordProcessor ,
192188 sconf : SerializableConfiguration ,
@@ -195,19 +191,22 @@ object SparkCobolProcessor {
195191 val groupedFiles = listOfFiles.grouped(numberOfThreads).toSeq
196192 val rdd = spark.sparkContext.parallelize(groupedFiles)
197193 rdd.map(group => {
198- processListOfFiles(group, outputPath, copybookContents, cobolProcessor, rawRecordProcessor, sconf, numberOfThreads)
194+ processListOfFiles(group, outputPath, cobolProcessor, rawRecordProcessor, sconf, numberOfThreads)
199195 })
200196 }
201197
198+ private def getCobolParameters (listOfFiles : Seq [String ], copybookContents : String , options : Map [String , String ], ignoreRedundantOptions : Boolean ): CobolParameters = {
199+ val varLenOptions = options + (PARAM_GENERATE_RECORD_ID -> " true" )
200+
201+ CobolParametersParser .parse(new Parameters (varLenOptions), ! ignoreRedundantOptions)
202+ .copy(sourcePaths = listOfFiles, copybookContent = Option (copybookContents))
203+ }
204+
202205 private def getRecordRdd (listOfFiles : Seq [String ],
203206 copybookContents : String ,
204207 options : Map [String , String ],
205208 sconf : SerializableConfiguration )(implicit spark : SparkSession ): RDD [Array [Byte ]] = {
206-
207- val varLenOptions = options + (PARAM_GENERATE_RECORD_ID -> " true" )
208-
209- val cobolParameters : CobolParameters = CobolParametersParser .parse(new Parameters (varLenOptions))
210- .copy(sourcePaths = listOfFiles, copybookContent = Option (copybookContents))
209+ val cobolParameters = getCobolParameters(listOfFiles, copybookContents, options, ignoreRedundantOptions = false )
211210
212211 val readerParameters = CobolParametersParser .getReaderProperties(cobolParameters, None )
213212 val cobolReader = DefaultSource .createVariableLengthReader(cobolParameters, spark)
@@ -248,7 +247,6 @@ object SparkCobolProcessor {
248247
249248 private def processListOfFiles (listOfFiles : Seq [String ],
250249 outputPath : String ,
251- copybookContents : String ,
252250 cobolProcessor : CobolProcessor ,
253251 rawRecordProcessor : SerializableRawRecordProcessor ,
254252 sconf : SerializableConfiguration ,
0 commit comments