Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
40 changes: 22 additions & 18 deletions README.md
Original file line number Diff line number Diff line change
Expand Up @@ -15,28 +15,28 @@ Add mainframe as a source to your data engineering strategy.

Among the motivations for this project, it is possible to highlight:

- Lack of expertise in the Cobol ecosystem, which makes it hard to integrate mainframes into data engineering strategies
- Lack of expertise in the Cobol ecosystem, which makes it hard to integrate mainframes into data engineering strategies.

- Lack of support from the open-source community to initiatives in this field
- Lack of support from the open-source community to initiatives in this field.

- The overwhelming majority (if not all) of tools to cope with this domain are proprietary
- The overwhelming majority (if not all) of tools to cope with this domain are proprietary.

- Several institutions struggle daily to maintain their legacy mainframes, which prevents them from evolving to more modern approaches to data management
- Several institutions struggle daily to maintain their legacy mainframes, which prevents them from evolving to more modern approaches to data management.

- Mainframe data can only take part in data science activities through very expensive investments
- Mainframe data can only take part in data science activities through very expensive investments.


## Features

- Supports primitive types (although some are "Cobol compiler specific")
- Supports primitive types (although some are "Cobol compiler specific").

- Supports REDEFINES, OCCURS and DEPENDING ON fields (e.g. unchecked unions and variable-size arrays)
- Supports REDEFINES, OCCURS and DEPENDING ON fields (e.g. unchecked unions and variable-size arrays).

- Supports nested structures and arrays
- Supports nested structures and arrays.

- Supports HDFS as well as local file systems
- Supports Hadoop (HDFS, S3, ...) as well as local file system.

- The COBOL copybooks parser doesn't have a Spark dependency and can be reused for integrating into other data processing engines
- The COBOL copybooks parser doesn't have a Spark dependency and can be reused for integrating into other data processing engines.

## Videos

Expand Down Expand Up @@ -135,18 +135,20 @@ Code coverage will be generated on path:
{project-root}/cobrix/{module}/target/scala-{scala_version}/jacoco/report/html
```

### Reading Cobol binary files from HDFS/local and querying them
### Reading Cobol binary files from Hadoop/local and querying them

1. Create a Spark ```SQLContext```

2. Start a ```sqlContext.read``` operation specifying ```za.co.absa.cobrix.spark.cobol.source``` as the format

3. Inform the path to the copybook describing the files through ```... .option("copybook", "path_to_copybook_file")```. By default the copybook
is expected to be in HDFS. You can specify that a copybook is located in the local file system by adding `file://` prefix. For example, you
can specify a local file like this `.option("copybook", "file:///home/user/data/compybook.cpy")`. Alternatively, instead of providing a path
to a copybook file you can provide the contents of the copybook itself by using `.option("copybook_contents", "...copybook contents...")`.
3. Inform the path to the copybook describing the files through ```... .option("copybook", "path_to_copybook_file")```.
- By default the copybook is expected to be in the default Hadoop filesystem (HDFS, S3, etc).
- You can specify that a copybook is located in the local file system by adding `file://` prefix.
- For example, you can specify a local file like this `.option("copybook", "file:///home/user/data/copybook.cpy")`.
- Alternatively, instead of providing a path to a copybook file you can provide the contents of the copybook itself by using `.option("copybook_contents", "...copybook contents...")`.
- You can store the copybook in the JAR itself at resources section in this case use `jar://` prefix, e.g.: `.option("copybook", "jar:///copybooks/copybook.cpy")`.

4. Inform the path to the HDFS directory containing the files: ```... .load("path_to_directory_containing_the_binary_files")```
4. Inform the path to the Hadoop directory containing the files: ```... .load("s3a://path_to_directory_containing_the_binary_files")```

5. Inform the query you would like to run on the Cobol Dataframe

Expand Down Expand Up @@ -208,7 +210,7 @@ val spark = SparkSession
.master("local[2]")
.config("duration", 2)
.config("copybook", "path_to_the_copybook")
.config("path", "path_to_source_directory") // could be both, local or HDFS
.config("path", "path_to_source_directory") // could be both, local or Hadoop (s3://, hdfs://, etc)
.getOrCreate()

val streamingContext = new StreamingContext(spark.sparkContext, Seconds(3))
Expand Down Expand Up @@ -607,7 +609,7 @@ records parsing.

However effective, this strategy may also suffer from excessive shuffling, since indexes may be sent to executors far from the actual data.

The latter issue is overcome by extracting the preferred locations for each index directly from HDFS, and then passing those locations to
The latter issue is overcome by extracting the preferred locations for each index directly from HDFS/S3/..., and then passing those locations to
Spark during the creation of the RDD that distributes the indexes.

When processing large collections, the overhead of collecting the locations is offset by the benefits of locality, thus, this feature is
Expand All @@ -618,6 +620,8 @@ enabled by default, but can be disabled by the configuration below:

### Workload optimization for variable-length records parsing

This feature works only for HDFS, not for any other of Hadoop filesystems.

When dealing with variable-length records, Cobrix strives to maximize locality by identifying the preferred locations in the cluster to parse
each record, i.e. the nodes where the record resides.

Expand Down
4 changes: 2 additions & 2 deletions pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -108,9 +108,9 @@
<maven.gpg.plugin.version>1.6</maven.gpg.plugin.version>

<!-- Frameworks and libraries -->
<scala.version>2.12.17</scala.version>
<scala.version>2.12.20</scala.version>
<scala.compat.version>2.12</scala.compat.version>
<spark.version>3.2.3</spark.version>
<spark.version>3.4.4</spark.version>
<scalatest.version>3.2.14</scalatest.version>
<specs.version>2.4.16</specs.version>
<guava.version>15.0</guava.version>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,8 @@ package za.co.absa.cobrix.spark.cobol.source.copybook

import org.apache.hadoop.conf.Configuration
import za.co.absa.cobrix.cobol.reader.parameters.CobolParameters
import za.co.absa.cobrix.spark.cobol.utils.{FileNameUtils, HDFSUtils}
import za.co.absa.cobrix.spark.cobol.utils.FsType.LocalFs
import za.co.absa.cobrix.spark.cobol.utils.{FileNameUtils, FsType, HDFSUtils, ResourceUtils}

import java.nio.charset.StandardCharsets
import java.nio.file.{Files, Paths}
Expand All @@ -33,28 +34,27 @@ object CopybookContentLoader {
(copyBookContents, copyBookPathFileName) match {
case (Some(contents), _) => Seq(contents)
case (None, Some(_)) =>
val (isLocalFS, copyBookFileName) = FileNameUtils.getCopyBookFileName(copyBookPathFileName.get)
Seq(
if (isLocalFS) {
loadCopybookFromLocalFS(copyBookFileName)
} else {
HDFSUtils.loadTextFileFromHadoop(hadoopConf, copyBookFileName)
}
)
case (None, None) => parameters.multiCopybookPath.map(
fileName => {
val (isLocalFS, copyBookFileName) = FileNameUtils.getCopyBookFileName(fileName)
if (isLocalFS) {
loadCopybookFromLocalFS(copyBookFileName)
} else {
HDFSUtils.loadTextFileFromHadoop(hadoopConf, copyBookFileName)
}
}
)
val copybookContent = loadCopybook(copyBookPathFileName.get, hadoopConf)
Seq(copybookContent)
case (None, None) =>
parameters.multiCopybookPath.map(fileName => loadCopybook(fileName, hadoopConf))
}
}

private def loadCopybook(pathToCopybook: String, hadoopConf: Configuration): String = {
val (fsType, copyBookFileName) = FileNameUtils.getCopyBookFileName(pathToCopybook)
fsType match {
case FsType.LocalFs => loadCopybookFromLocalFS(copyBookFileName)
case FsType.JarFs => loadCopybookFromJarResources(copyBookFileName)
case FsType.HadoopFs => HDFSUtils.loadTextFileFromHadoop(hadoopConf, copyBookFileName)
}
}

private def loadCopybookFromLocalFS(copyBookLocalPath: String): String = {
Files.readAllLines(Paths.get(copyBookLocalPath), StandardCharsets.ISO_8859_1).toArray.mkString("\n")
}

private def loadCopybookFromJarResources(copyBookJarPath: String): String = {
ResourceUtils.readResourceAsString(copyBookJarPath)
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -18,13 +18,13 @@ package za.co.absa.cobrix.spark.cobol.source.parameters

import java.io.FileNotFoundException
import java.nio.file.{Files, Paths}

import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.Path
import org.apache.spark.SparkConf
import za.co.absa.cobrix.cobol.reader.parameters.CobolParameters
import za.co.absa.cobrix.spark.cobol.parameters.CobolParametersParser._
import za.co.absa.cobrix.spark.cobol.utils.FileNameUtils
import za.co.absa.cobrix.spark.cobol.utils.ResourceUtils.getClass
import za.co.absa.cobrix.spark.cobol.utils.{FileNameUtils, FsType}

/**
* This class provides methods for checking the Spark job options after parsed.
Expand Down Expand Up @@ -66,8 +66,8 @@ object CobolParametersValidator {
}

def validatePath(fileName: String): Unit = {
val (isLocalFS, copyBookFileName) = FileNameUtils.getCopyBookFileName(fileName)
if (isLocalFS) {
val (fsType, copyBookFileName) = FileNameUtils.getCopyBookFileName(fileName)
if (fsType == FsType.LocalFs) {
if (!Files.exists(Paths.get(copyBookFileName))) {
throw new FileNotFoundException(s"Copybook not found at $copyBookFileName")
}
Expand All @@ -77,6 +77,12 @@ object CobolParametersValidator {
if (!Files.isReadable(Paths.get(copyBookFileName))) {
throw new IllegalArgumentException(s"The copybook path '$copyBookFileName' is not readable.")
}
} else if (fsType == FsType.JarFs) {
if (getClass.getResourceAsStream(copyBookFileName) == null) {
if (!Files.exists(Paths.get(copyBookFileName))) {
throw new FileNotFoundException(s"Copybook not found at the JAR resource path: $copyBookFileName")
}
}
} else {
val fs = new Path(fileName).getFileSystem(hadoopConf)
if (!fs.exists(new Path(copyBookFileName))) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,14 +18,20 @@ package za.co.absa.cobrix.spark.cobol.utils

object FileNameUtils {
private val LOCALFS_PREFIX = "file://"
private val JAR_PREFIX = "jar://"

def getCopyBookFileName(fileNameURI: String):(Boolean, String) = {
def getCopyBookFileName(fileNameURI: String): (FsType, String) = {
val isLocalFS = fileNameURI.toLowerCase.startsWith(LOCALFS_PREFIX)
val copyBookFileName = if (isLocalFS)
fileNameURI.drop(LOCALFS_PREFIX.length)
val isJar = fileNameURI.toLowerCase.startsWith(JAR_PREFIX)
if (isLocalFS)
(FsType.LocalFs, fileNameURI.drop(LOCALFS_PREFIX.length))
else if (isJar) {
val fileCandidate = fileNameURI.drop(JAR_PREFIX.length)
val filePath = if (fileCandidate.startsWith("/")) fileCandidate else s"/$fileCandidate"
(FsType.JarFs, filePath)
}
else
fileNameURI
(isLocalFS, copyBookFileName)
(FsType.HadoopFs, fileNameURI)
}

}
Original file line number Diff line number Diff line change
@@ -0,0 +1,25 @@
/*
* 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.utils

sealed trait FsType

object FsType {
case object LocalFs extends FsType
case object JarFs extends FsType
case object HadoopFs extends FsType
}
21 changes: 21 additions & 0 deletions spark-cobol/src/test/resources/test/copybook.cpy
Original file line number Diff line number Diff line change
@@ -0,0 +1,21 @@
****************************************************************************
* *
* 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. *
* *
****************************************************************************

01 R.
03 A PIC X(1).
03 B PIC X(2).
Original file line number Diff line number Diff line change
Expand Up @@ -17,12 +17,13 @@
package za.co.absa.cobrix.spark.cobol.source.regression

import java.nio.file.Paths

import org.scalatest.funsuite.AnyFunSuite
import org.slf4j.{Logger, LoggerFactory}
import za.co.absa.cobrix.spark.cobol.source.base.SparkTestBase
import za.co.absa.cobrix.spark.cobol.source.fixtures.BinaryFileFixture

import java.io.FileNotFoundException

class Test11NoCopybookErrMsg extends AnyFunSuite with SparkTestBase with BinaryFileFixture {

private implicit val logger: Logger = LoggerFactory.getLogger(this.getClass)
Expand Down Expand Up @@ -50,6 +51,32 @@ class Test11NoCopybookErrMsg extends AnyFunSuite with SparkTestBase with BinaryF
}
}

test("Test a file loads normally when the copybook is a JAR resource") {
withTempBinFile("bin_file2", ".dat", binFileContents) { tmpFileName =>
val df = spark
.read
.format("cobol")
.option("copybook", "jar:///test/copybook.cpy")
.option("schema_retention_policy", "collapse_root")
.load(tmpFileName)

assert(df.count == 1)
}
}

test("Test a file loads normally when the copybook is a JAR resource short") {
withTempBinFile("bin_file2", ".dat", binFileContents) { tmpFileName =>
val df = spark
.read
.format("cobol")
.option("copybook", "jar://test/copybook.cpy")
.option("schema_retention_policy", "collapse_root")
.load(tmpFileName)

assert(df.count == 1)
}
}

test("Test the error message logged when no copybook is provided") {
val ex = intercept[IllegalStateException] {
spark
Expand Down Expand Up @@ -132,4 +159,15 @@ class Test11NoCopybookErrMsg extends AnyFunSuite with SparkTestBase with BinaryF
}
}

test("Test the error message of the copybook is not in the JAr resource") {
val ex = intercept[FileNotFoundException] {
spark
.read
.format("cobol")
.option("copybook", "jar://test/copybook_non_existent.cpy")
.load("/tmp/doesnt/matter")
}

assert(ex.getMessage == "Copybook not found at the JAR resource path: /test/copybook_non_existent.cpy")
}
}
Loading