Skip to content

Commit

Permalink
#545 Add support for 'string' debug columns for ASCII file format (D,…
Browse files Browse the repository at this point in the history
… D2).
  • Loading branch information
yruslan committed Dec 19, 2022
1 parent b680aaa commit 566ef0c
Show file tree
Hide file tree
Showing 5 changed files with 56 additions and 16 deletions.
19 changes: 10 additions & 9 deletions README.md
Original file line number Diff line number Diff line change
Expand Up @@ -1320,15 +1320,15 @@ You can have decimals when using COMP-3 as well.

##### Modifier options

| Option (usage example) | Description |
| ------------------------------------------ |:--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|
| .option("schema_retention_policy", "collapse_root") | When `collapse_root` (default) the root level record will be removed from the Spark schema. When `keep_original`, the root level GROUP will be present in the Spark schema |
| .option("drop_group_fillers", "false") | If `true`, all GROUP FILLERs will be dropped from the output schema. If `false` (default), such fields will be retained. |
| .option("drop_value_fillers", "false") | If `true` (default), all non-GROUP FILLERs will be dropped from the output schema. If `false`, such fields will be retained. |
| .option("non_terminals", "GROUP1,GROUP2") | Specifies groups to also be added to the schema as string fields. When this option is specified, the reader will add one extra data field after each matching group containing the string data for the group. |
| .option("generate_record_id", false) | Generate autoincremental 'File_Id', 'Record_Id' and 'Record_Byte_Length' fields. This is used for processing record order dependent data. |
| .option("with_input_file_name_col", "file_name") | Generates a column containing input file name for each record (Similar to Spark SQL `input_file_name()` function). The column name is specified by the value of the option. This option only works for variable record length files. For fixed record length files use `input_file_name()`. |
| .option("debug", "hex") | If specified, each primitive field will be accompanied by a debug field containing raw bytes from the source file. Possible values: `none` (default), `hex`, `binary`. The legacy value `true` is supported and will generate debug fields in HEX. |
| Option (usage example) | Description |
|-----------------------------------------------------|:---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|
| .option("schema_retention_policy", "collapse_root") | When `collapse_root` (default) the root level record will be removed from the Spark schema. When `keep_original`, the root level GROUP will be present in the Spark schema |
| .option("drop_group_fillers", "false") | If `true`, all GROUP FILLERs will be dropped from the output schema. If `false` (default), such fields will be retained. |
| .option("drop_value_fillers", "false") | If `true` (default), all non-GROUP FILLERs will be dropped from the output schema. If `false`, such fields will be retained. |
| .option("non_terminals", "GROUP1,GROUP2") | Specifies groups to also be added to the schema as string fields. When this option is specified, the reader will add one extra data field after each matching group containing the string data for the group. |
| .option("generate_record_id", false) | Generate autoincremental 'File_Id', 'Record_Id' and 'Record_Byte_Length' fields. This is used for processing record order dependent data. |
| .option("with_input_file_name_col", "file_name") | Generates a column containing input file name for each record (Similar to Spark SQL `input_file_name()` function). The column name is specified by the value of the option. This option only works for variable record length files. For fixed record length files use `input_file_name()`. |
| .option("debug", "hex") | If specified, each primitive field will be accompanied by a debug field containing raw bytes from the source file. Possible values: `none` (default), `hex`, `binary`, `string` (ASCII only). The legacy value `true` is supported and will generate debug fields in HEX. |

##### Fixed length record format options (for record_format = F or FB)
| .option("record_format", "F") | Record format from the [spec](https://www.ibm.com/docs/en/zos/2.3.0?topic=files-selecting-record-formats-non-vsam-data-sets). One of `F` (fixed length, default), `FB` (fixed block), V` (variable length RDW), `VB` (variable block BDW+RDW), `D` (ASCII text). |
Expand Down Expand Up @@ -1513,6 +1513,7 @@ A: Update hadoop dll to version 3.2.2 or newer.
## Changelog
- #### 2.6.2 will be released soon.
- [#516](https://github.com/AbsaOSS/cobrix/issues/516) Added support for unsigned packed numbers via a Cobrix extension (COMP-3U).
- [#545](https://github.com/AbsaOSS/cobrix/issues/545) Added support for `string` debug columns for ASCII (D/D2/T) files (`.option("debug", "string")`).
- [#543](https://github.com/AbsaOSS/cobrix/issues/543) Improved performance of processing ASCII text (D/D2/T) files with variable OCCURS.

- #### 2.6.1 released 2 December 2022.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@ import za.co.absa.cobrix.cobol.parser.CopybookParser.CopybookAST
import za.co.absa.cobrix.cobol.parser.ast.datatype.AlphaNumeric
import za.co.absa.cobrix.cobol.parser.ast.{Group, Primitive, Statement}
import za.co.absa.cobrix.cobol.parser.decoders.StringDecoders
import za.co.absa.cobrix.cobol.parser.decoders.StringDecoders.KeepAll
import za.co.absa.cobrix.cobol.parser.encoding._
import za.co.absa.cobrix.cobol.parser.policies.DebugFieldsPolicy
import za.co.absa.cobrix.cobol.parser.policies.DebugFieldsPolicy.DebugFieldsPolicy
Expand All @@ -41,18 +42,21 @@ class DebugFieldsAdder(debugFieldsPolicy: DebugFieldsPolicy) extends AstTransfor
val debugEncoding = debugFieldsPolicy match {
case DebugFieldsPolicy.HexValue => HEX
case DebugFieldsPolicy.RawValue => RAW
case DebugFieldsPolicy.StringValue => ASCII
case _ => throw new IllegalStateException(s"Unexpected debug fields policy: $debugFieldsPolicy.")
}

val debugDecoder = debugFieldsPolicy match {
case DebugFieldsPolicy.HexValue => StringDecoders.decodeHex _
case DebugFieldsPolicy.RawValue => StringDecoders.decodeRaw _
case DebugFieldsPolicy.StringValue => (a: Array[Byte]) => new String(a)
case _ => throw new IllegalStateException(s"Unexpected debug fields policy: $debugFieldsPolicy.")
}

val size = debugFieldsPolicy match {
case DebugFieldsPolicy.HexValue => field.binaryProperties.dataSize * 2
case DebugFieldsPolicy.RawValue => field.binaryProperties.dataSize
case DebugFieldsPolicy.StringValue => field.binaryProperties.dataSize
case _ => throw new IllegalStateException(s"Unexpected debug fields policy: $debugFieldsPolicy.")
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,7 @@ package za.co.absa.cobrix.cobol.parser.policies
object DebugFieldsPolicy extends Enumeration {
type DebugFieldsPolicy = Value

val NoDebug, HexValue, RawValue = Value
val NoDebug, HexValue, RawValue, StringValue = Value

def withNameOpt(s: String): Option[Value] = {
val exactNames = values.find(_.toString == s)
Expand All @@ -31,6 +31,8 @@ object DebugFieldsPolicy extends Enumeration {
Some(HexValue)
} else if (sLowerCase == "binary" || sLowerCase == "raw") {
Some(RawValue)
} else if (sLowerCase == "string") {
Some(StringValue)
} else {
None
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -185,17 +185,23 @@ object CobolParametersParser extends Logging {
}
}

private def getDebuggingFieldsPolicy(params: Parameters): DebugFieldsPolicy = {
private def getDebuggingFieldsPolicy(recordFormat: RecordFormat, params: Parameters): DebugFieldsPolicy = {
val debugFieldsPolicyName = params.getOrElse(PARAM_DEBUG, "false")
val debugFieldsPolicy = DebugFieldsPolicy.withNameOpt(debugFieldsPolicyName)

debugFieldsPolicy match {
val policy = debugFieldsPolicy match {
case Some(p) =>
p
case None =>
throw new IllegalArgumentException(s"Invalid value '$debugFieldsPolicyName' for '$PARAM_DEBUG' option. " +
"Allowed one of: 'true' = 'hex', 'raw', 'false' = 'none'. ")
"Allowed one of: 'true' = 'hex', 'raw', 'binary', 'string' (ASCII only), 'false' = 'none'. ")
}
if (policy == DebugFieldsPolicy.StringValue && recordFormat != RecordFormat.AsciiText && recordFormat != RecordFormat.BasicAsciiText) {
throw new IllegalArgumentException(s"Invalid value '$debugFieldsPolicyName' for '$PARAM_DEBUG' option. " +
"Allowed only for record_format = 'D' or 'D2'.")
}

policy
}

def parse(params: Parameters): CobolParameters = {
Expand Down Expand Up @@ -269,7 +275,7 @@ object CobolParametersParser extends Logging {
params.getOrElse(PARAM_VALUE_FILLERS, "true").toBoolean,
params.getOrElse(PARAM_GROUP_NOT_TERMINALS, "").split(','),
getOccursMappings(params.getOrElse(PARAM_OCCURS_MAPPINGS, "{}")),
getDebuggingFieldsPolicy(params),
getDebuggingFieldsPolicy(recordFormat, params),
params.getOrElse(PARAM_DEBUG_IGNORE_FILE_SIZE, "false").toBoolean
)
validateSparkCobolOptions(params, recordFormat)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,17 +18,17 @@ package za.co.absa.cobrix.spark.cobol.source.integration

import java.nio.charset.StandardCharsets
import java.nio.file.{Files, Paths}

import org.scalatest.FunSuite
import org.slf4j.{Logger, LoggerFactory}
import za.co.absa.cobrix.cobol.parser.CopybookParser
import za.co.absa.cobrix.cobol.parser.policies.DebugFieldsPolicy
import za.co.absa.cobrix.spark.cobol.source.base.{SimpleComparisonBase, SparkTestBase}
import za.co.absa.cobrix.spark.cobol.source.fixtures.BinaryFileFixture
import za.co.absa.cobrix.spark.cobol.utils.{FileUtils, SparkUtils}

import scala.collection.JavaConverters._

class Test24DebugModeSpec extends FunSuite with SparkTestBase with SimpleComparisonBase {
class Test24DebugModeSpec extends FunSuite with SparkTestBase with BinaryFileFixture with SimpleComparisonBase {
private implicit val logger: Logger = LoggerFactory.getLogger(this.getClass)

private val exampleName = "Test24 (debug mode)"
Expand Down Expand Up @@ -147,5 +147,32 @@ class Test24DebugModeSpec extends FunSuite with SparkTestBase with SimpleCompari
Files.delete(Paths.get(actualResultsPath))
}

test("Test debugging of an OCCURS of primitives") {
val copybook =
""" 01 RECORD.
05 A PIC X(2).
05 B PIC 9(3).
"""

val inputText = "1 345\n6 9 "
withTempTextFile("debug_mode", ".dat", StandardCharsets.UTF_8, inputText) { tmpFileName =>
val df = spark
.read
.format("cobol")
.option("copybook_contents", copybook)
.option("pedantic", "true")
.option("record_format", "D")
.option("schema_retention_policy", "collapse_root")
.option("debug", "string")
.load(tmpFileName)

val expected = """[{"A":"1","A_debug":"1 ","B":345,"B_debug":"345"},{"A":"6","A_debug":"6 ","B":9,"B_debug":" 9 "}]"""

val actual = df.toJSON.collect().mkString("[", ",", "]")

assertEqualsMultiline(actual, expected)
}
}


}

0 comments on commit 566ef0c

Please sign in to comment.