diff --git a/README.md b/README.md index 2c6ddeb07..ef64b2cf0 100644 --- a/README.md +++ b/README.md @@ -39,6 +39,9 @@ Among the motivations for this project, it is possible to highlight: - The COBOL copybooks parser doesn't have a Spark dependency and can be reused for integrating into other data processing engines. +- Supports reading files compressed in Hadoop-compatible way (gzip, bzip2, etc), but with limited parallelism. + Uncompressed files are preferred for performance. + ## Videos We have presented Cobrix at DataWorks Summit 2019 and Spark Summit 2019 conferences. The screencasts are available here: @@ -1602,7 +1605,7 @@ The output looks like this: | .option("redefine-segment-id-map:0", "REDEFINED_FIELD1 => SegmentId1,SegmentId2,...") | Specifies a mapping between redefined field names and segment id values. Each option specifies a mapping for a single segment. The numeric value for each mapping option must be incremented so the option keys are unique. | | .option("segment-children:0", "COMPANY => EMPLOYEE,DEPARTMENT") | Specifies a mapping between segment redefined fields and their children. Each option specifies a mapping for a single parent field. The numeric value for each mapping option must be incremented so the option keys are unique. If such mapping is specified hierarchical record structure will be automatically reconstructed. This require `redefine-segment-id-map` to be set. | | .option("enable_indexes", "true") | Turns on indexing of multisegment variable length files (on by default). | -| .option("enable_index_cache", "false") | When true, calculated indexes are cached in memory for later use. This improves performance of processing when same files are processed more than once. | +| .option("enable_index_cache", "true") | When true (default), calculated indexes are cached in memory for later use. This improves performance of processing when same files are processed more than once. | | .option("input_split_records", 50000) | Specifies how many records will be allocated to each split/partition. It will be processed by Spark tasks. (The default is not set and the split will happen according to size, see the next option) | | .option("input_split_size_mb", 100) | Specify how many megabytes to allocate to each partition/split. (The default is 100 MB) | diff --git a/cobol-parser/src/main/scala/za/co/absa/cobrix/cobol/reader/VarLenNestedReader.scala b/cobol-parser/src/main/scala/za/co/absa/cobrix/cobol/reader/VarLenNestedReader.scala index caf568364..5ac664de3 100644 --- a/cobol-parser/src/main/scala/za/co/absa/cobrix/cobol/reader/VarLenNestedReader.scala +++ b/cobol-parser/src/main/scala/za/co/absa/cobrix/cobol/reader/VarLenNestedReader.scala @@ -42,6 +42,8 @@ import scala.reflect.ClassTag class VarLenNestedReader[T: ClassTag](copybookContents: Seq[String], readerProperties: ReaderParameters, handler: RecordHandler[T]) extends VarLenReader with Logging with Serializable { + private val DEFAULT_INDEX_SIZE_COMPRESSED_FILES_MB = 1024 + private val DEFAULT_FS_INDEX_SIZE_MULTIPLIER = 4 protected val cobolSchema: CobolSchema = loadCopyBook(copybookContents) @@ -144,7 +146,7 @@ class VarLenNestedReader[T: ClassTag](copybookContents: Seq[String], fileNumber: Int, isRdwBigEndian: Boolean): ArrayBuffer[SparseIndexEntry] = { val inputSplitSizeRecords: Option[Int] = readerProperties.inputSplitRecords - val inputSplitSizeMB: Option[Int] = getSplitSizeMB + val inputSplitSizeMB: Option[Int] = getSplitSizeMB(dataStream.isCompressed) if (inputSplitSizeRecords.isDefined) { if (inputSplitSizeRecords.get < 1 || inputSplitSizeRecords.get > 1000000000) { @@ -153,8 +155,9 @@ class VarLenNestedReader[T: ClassTag](copybookContents: Seq[String], logger.info(s"Input split size = ${inputSplitSizeRecords.get} records") } else { if (inputSplitSizeMB.nonEmpty) { - if (inputSplitSizeMB.get < 1 || inputSplitSizeMB.get > 2000) { - throw new IllegalArgumentException(s"Invalid input split size of ${inputSplitSizeMB.get} MB.") + val maxSplitSizeMB = if (dataStream.isCompressed) 200000 else 2000 + if (inputSplitSizeMB.get < 1 || inputSplitSizeMB.get > maxSplitSizeMB) { + throw new IllegalArgumentException(s"Invalid input split size of ${inputSplitSizeMB.get} MB (max allowed: $maxSplitSizeMB MB).") } logger.info(s"Input split size = ${inputSplitSizeMB.get} MB") } @@ -214,11 +217,11 @@ class VarLenNestedReader[T: ClassTag](copybookContents: Seq[String], } } - private def getSplitSizeMB: Option[Int] = { - if (readerProperties.inputSplitSizeMB.isDefined) { - readerProperties.inputSplitSizeMB + private def getSplitSizeMB(isCompressed: Boolean): Option[Int] = { + if (isCompressed) { + readerProperties.inputSplitSizeCompressedMB.orElse(Some(DEFAULT_INDEX_SIZE_COMPRESSED_FILES_MB)) } else { - readerProperties.hdfsDefaultBlockSize + readerProperties.inputSplitSizeMB.orElse(readerProperties.fsDefaultBlockSize).map(_ * DEFAULT_FS_INDEX_SIZE_MULTIPLIER) } } diff --git a/cobol-parser/src/main/scala/za/co/absa/cobrix/cobol/reader/parameters/CobolParametersParser.scala b/cobol-parser/src/main/scala/za/co/absa/cobrix/cobol/reader/parameters/CobolParametersParser.scala index 3a4a9ad6f..e563d6650 100644 --- a/cobol-parser/src/main/scala/za/co/absa/cobrix/cobol/reader/parameters/CobolParametersParser.scala +++ b/cobol-parser/src/main/scala/za/co/absa/cobrix/cobol/reader/parameters/CobolParametersParser.scala @@ -118,13 +118,14 @@ object CobolParametersParser extends Logging { val PARAM_SEGMENT_REDEFINE_PREFIX_ALT = "redefine-segment-id-map" // Indexed multisegment file processing - val PARAM_ENABLE_INDEXES = "enable_indexes" - val PARAM_ENABLE_INDEX_CACHE = "enable_index_cache" - val PARAM_INPUT_SPLIT_RECORDS = "input_split_records" - val PARAM_INPUT_SPLIT_SIZE_MB = "input_split_size_mb" - val PARAM_SEGMENT_ID_PREFIX = "segment_id_prefix" - val PARAM_OPTIMIZE_ALLOCATION = "optimize_allocation" - val PARAM_IMPROVE_LOCALITY = "improve_locality" + val PARAM_ENABLE_INDEXES = "enable_indexes" + val PARAM_ENABLE_INDEX_CACHE = "enable_index_cache" + val PARAM_INPUT_SPLIT_RECORDS = "input_split_records" + val PARAM_INPUT_SPLIT_SIZE_MB = "input_split_size_mb" + val PARAM_INPUT_SPLIT_SIZE_COMPRESSED_MB = "input_split_size_compressed_mb" + val PARAM_SEGMENT_ID_PREFIX = "segment_id_prefix" + val PARAM_OPTIMIZE_ALLOCATION = "optimize_allocation" + val PARAM_IMPROVE_LOCALITY = "improve_locality" // Parameters for debugging val PARAM_DEBUG_LAYOUT_POSITIONS = "debug_layout_positions" @@ -385,6 +386,7 @@ object CobolParametersParser extends Logging { isIndexCachingAllowed = false, inputSplitRecords = None, inputSplitSizeMB = None, + inputSplitSizeCompressedMB = None, improveLocality = false, optimizeAllocation = false, inputFileNameColumn = "", @@ -421,7 +423,8 @@ object CobolParametersParser extends Logging { isIndexCachingAllowed = varLenParams.isIndexCachingAllowed, inputSplitRecords = varLenParams.inputSplitRecords, inputSplitSizeMB = varLenParams.inputSplitSizeMB, - hdfsDefaultBlockSize = defaultBlockSize, + inputSplitSizeCompressedMB = varLenParams.inputSplitSizeCompressedMB, + fsDefaultBlockSize = defaultBlockSize, startOffset = parameters.recordStartOffset, endOffset = parameters.recordEndOffset, fileStartOffset = varLenParams.fileStartOffset, @@ -505,9 +508,10 @@ object CobolParametersParser extends Logging { fileEndOffset, isRecordIdGenerationEnabled, params.getOrElse(PARAM_ENABLE_INDEXES, "true").toBoolean, - params.getOrElse(PARAM_ENABLE_INDEX_CACHE, "false").toBoolean, + params.getOrElse(PARAM_ENABLE_INDEX_CACHE, "true").toBoolean, params.get(PARAM_INPUT_SPLIT_RECORDS).map(v => v.toInt), params.get(PARAM_INPUT_SPLIT_SIZE_MB).map(v => v.toInt), + params.get(PARAM_INPUT_SPLIT_SIZE_COMPRESSED_MB).map(v => v.toInt), params.getOrElse(PARAM_IMPROVE_LOCALITY, "true").toBoolean, params.getOrElse(PARAM_OPTIMIZE_ALLOCATION, "false").toBoolean, params.getOrElse(PARAM_INPUT_FILE_COLUMN, ""), diff --git a/cobol-parser/src/main/scala/za/co/absa/cobrix/cobol/reader/parameters/ReaderParameters.scala b/cobol-parser/src/main/scala/za/co/absa/cobrix/cobol/reader/parameters/ReaderParameters.scala index ac679e1a9..01148faec 100644 --- a/cobol-parser/src/main/scala/za/co/absa/cobrix/cobol/reader/parameters/ReaderParameters.scala +++ b/cobol-parser/src/main/scala/za/co/absa/cobrix/cobol/reader/parameters/ReaderParameters.scala @@ -20,7 +20,7 @@ import za.co.absa.cobrix.cobol.parser.decoders.FloatingPointFormat import za.co.absa.cobrix.cobol.parser.decoders.FloatingPointFormat.FloatingPointFormat import za.co.absa.cobrix.cobol.parser.policies.DebugFieldsPolicy.DebugFieldsPolicy import za.co.absa.cobrix.cobol.parser.policies.StringTrimmingPolicy.StringTrimmingPolicy -import za.co.absa.cobrix.cobol.parser.policies.{CommentPolicy, DebugFieldsPolicy, FillerNamingPolicy, MetadataPolicy, StringTrimmingPolicy} +import za.co.absa.cobrix.cobol.parser.policies._ import za.co.absa.cobrix.cobol.parser.recordformats.RecordFormat import za.co.absa.cobrix.cobol.parser.recordformats.RecordFormat.FixedLength import za.co.absa.cobrix.cobol.reader.policies.SchemaRetentionPolicy @@ -50,7 +50,8 @@ import za.co.absa.cobrix.cobol.reader.policies.SchemaRetentionPolicy.SchemaReten * @param isIndexGenerationNeeded Is indexing input file before processing is requested * @param inputSplitRecords The number of records to include in each partition. Notice mainframe records may have variable size, inputSplitMB is the recommended option * @param inputSplitSizeMB A partition size to target. In certain circumstances this size may not be exactly that, but the library will do the best effort to target that size - * @param hdfsDefaultBlockSize Default HDFS block size for the HDFS filesystem used. This value is used as the default split size if inputSplitSizeMB is not specified + * @param inputSplitSizeCompressedMB A partition size to target for compressed files. + * @param fsDefaultBlockSize Default HDFS block size for the HDFS filesystem used. This value is used as the default split size if inputSplitSizeMB is not specified * @param startOffset An offset to the start of the record in each binary data block. * @param endOffset An offset from the end of the record to the end of the binary data block. * @param fileStartOffset A number of bytes to skip at the beginning of each file @@ -102,7 +103,8 @@ case class ReaderParameters( isIndexCachingAllowed: Boolean = false, inputSplitRecords: Option[Int] = None, inputSplitSizeMB: Option[Int] = None, - hdfsDefaultBlockSize: Option[Int] = None, + inputSplitSizeCompressedMB: Option[Int] = None, + fsDefaultBlockSize: Option[Int] = None, startOffset: Int = 0, endOffset: Int = 0, fileStartOffset: Int = 0, diff --git a/cobol-parser/src/main/scala/za/co/absa/cobrix/cobol/reader/parameters/VariableLengthParameters.scala b/cobol-parser/src/main/scala/za/co/absa/cobrix/cobol/reader/parameters/VariableLengthParameters.scala index 153dd8fa2..a89d68e86 100644 --- a/cobol-parser/src/main/scala/za/co/absa/cobrix/cobol/reader/parameters/VariableLengthParameters.scala +++ b/cobol-parser/src/main/scala/za/co/absa/cobrix/cobol/reader/parameters/VariableLengthParameters.scala @@ -17,51 +17,53 @@ package za.co.absa.cobrix.cobol.reader.parameters /** - * This class holds the parameters currently used for parsing variable-length records. + * This class is used to hold the parameters currently used for parsing variable-length records. * - * @param isRecordSequence Does input files have 4 byte record length headers - * @param bdw Block descriptor word (if specified), for FB and VB record formats - * @param isRdwBigEndian Is RDW big endian? It may depend on flavor of mainframe and/or mainframe to PC transfer method - * @param isRdwPartRecLength Does RDW count itself as part of record length itself - * @param rdwAdjustment Controls a mismatch between RDW and record length - * @param recordHeaderParser An optional custom record header parser for non-standard RDWs - * @param recordExtractor An optional custom raw record parser class non-standard record types - * @param rhpAdditionalInfo An optional additional option string passed to a custom record header parser - * @param reAdditionalInfo An optional additional option string passed to a custom record extractor - * @param recordLengthField A field that stores record length - * @param recordLengthMap A mapping between field value and record size. - * @param fileStartOffset A number of bytes to skip at the beginning of each file - * @param fileEndOffset A number of bytes to skip at the end of each file - * @param generateRecordId Generate a sequential record number for each record to be able to retain the order of the original data - * @param isUsingIndex Is indexing input file before processing is requested - * @param isIndexCachingAllowed Is caching of generated index allowed - * @param inputSplitSizeMB A partition size to target. In certain circumstances this size may not be exactly that, but the library will do the best effort to target that size - * @param inputSplitRecords The number of records to include in each partition. Notice mainframe records may have variable size, inputSplitMB is the recommended option - * @param improveLocality Tries to improve locality by extracting preferred locations for variable-length records - * @param optimizeAllocation Optimizes cluster usage in case of optimization for locality in the presence of new nodes (nodes that do not contain any blocks of the files being processed) - * @param inputFileNameColumn A column name to add to the dataframe. The column will contain input file name for each record similar to 'input_file_name()' function + * @param isRecordSequence Do input files have 4 byte record length headers + * @param bdw Block descriptor word (if specified), for FB and VB record formats + * @param isRdwBigEndian Is RDW big endian? It may depend on flavor of mainframe and/or mainframe to PC transfer method + * @param isRdwPartRecLength Does RDW count itself as part of record length itself + * @param rdwAdjustment Controls a mismatch between RDW and record length + * @param recordHeaderParser An optional custom record header parser for non-standard RDWs + * @param recordExtractor An optional custom raw record parser class for non-standard record types + * @param rhpAdditionalInfo An optional additional option string passed to a custom record header parser + * @param reAdditionalInfo An optional additional option string passed to a custom record extractor + * @param recordLengthField A field that stores record length + * @param recordLengthMap A mapping between field value and record size. + * @param fileStartOffset A number of bytes to skip at the beginning of each file + * @param fileEndOffset A number of bytes to skip at the end of each file + * @param generateRecordId Generate a sequential record number for each record to be able to retain the order of the original data + * @param isUsingIndex Is indexing input file before processing is requested + * @param isIndexCachingAllowed Is caching of generated index allowed + * @param inputSplitSizeMB A partition size to target. In certain circumstances this size may not be exactly that, but the library will do the best effort to target that size + * @param inputSplitSizeCompressedMB A partition size to target for compressed files. + * @param inputSplitRecords The number of records to include in each partition. Notice mainframe records may have variable size, inputSplitMB is the recommended option + * @param improveLocality Tries to improve locality by extracting preferred locations for variable-length records + * @param optimizeAllocation Optimizes cluster usage in case of optimization for locality in the presence of new nodes (nodes that do not contain any blocks of the files being processed) + * @param inputFileNameColumn A column name to add to the dataframe. The column will contain input file name for each record similar to 'input_file_name()' function */ case class VariableLengthParameters( - isRecordSequence: Boolean, // [deprecated by recordFormat] - bdw: Option[Bdw], - isRdwBigEndian: Boolean, - isRdwPartRecLength: Boolean, - rdwAdjustment: Int, - recordHeaderParser: Option[String], - recordExtractor: Option[String], - rhpAdditionalInfo: Option[String], - reAdditionalInfo: String, - recordLengthField: String, - recordLengthMap: Map[String, Int], - fileStartOffset: Int, - fileEndOffset: Int, - generateRecordId: Boolean, - isUsingIndex: Boolean, - isIndexCachingAllowed: Boolean, - inputSplitRecords: Option[Int], - inputSplitSizeMB: Option[Int], - improveLocality: Boolean, - optimizeAllocation: Boolean, - inputFileNameColumn: String, - occursMappings: Map[String, Map[String, Int]] + isRecordSequence: Boolean, // [deprecated by recordFormat] + bdw: Option[Bdw], + isRdwBigEndian: Boolean, + isRdwPartRecLength: Boolean, + rdwAdjustment: Int, + recordHeaderParser: Option[String], + recordExtractor: Option[String], + rhpAdditionalInfo: Option[String], + reAdditionalInfo: String, + recordLengthField: String, + recordLengthMap: Map[String, Int], + fileStartOffset: Int, + fileEndOffset: Int, + generateRecordId: Boolean, + isUsingIndex: Boolean, + isIndexCachingAllowed: Boolean, + inputSplitRecords: Option[Int], + inputSplitSizeMB: Option[Int], + inputSplitSizeCompressedMB: Option[Int], + improveLocality: Boolean, + optimizeAllocation: Boolean, + inputFileNameColumn: String, + occursMappings: Map[String, Map[String, Int]] ) diff --git a/cobol-parser/src/main/scala/za/co/absa/cobrix/cobol/reader/stream/SimpleStream.scala b/cobol-parser/src/main/scala/za/co/absa/cobrix/cobol/reader/stream/SimpleStream.scala index 561116121..50aae1959 100644 --- a/cobol-parser/src/main/scala/za/co/absa/cobrix/cobol/reader/stream/SimpleStream.scala +++ b/cobol-parser/src/main/scala/za/co/absa/cobrix/cobol/reader/stream/SimpleStream.scala @@ -29,6 +29,8 @@ trait SimpleStream { def isEndOfStream: Boolean = offset >= size + def isCompressed: Boolean = false + @throws(classOf[Exception]) def copyStream(): SimpleStream diff --git a/data/test40_copybook.cob b/data/test40_copybook.cob new file mode 100644 index 000000000..669fb2f45 --- /dev/null +++ b/data/test40_copybook.cob @@ -0,0 +1,260 @@ + **************************************************************************** + * * + * 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 RECORD. + +*********************************************************************** +******* EDGE CASES +*********************************************************************** + 10 ID PIC 9(7) BINARY. + +********** String + 10 STRING-VAL PIC X(10). + +********** Integral unsigned numbers formatted as strings + 10 NUM-STR-INT01 PIC 9(1). + 10 NUM-STR-INT02 PIC 9(2). + 10 NUM-STR-INT03 PIC 9(3). + 10 NUM-STR-INT04 PIC 9(4). + 10 NUM-STR-INT05 PIC 9(5). + 10 NUM-STR-INT06 PIC 9(8). + 10 NUM-STR-INT07 PIC 9(9). + 10 NUM-STR-INT08 PIC 9(10). + 10 NUM-STR-INT09 PIC 9(11). + 10 NUM-STR-INT10 PIC 9(17). + 10 NUM-STR-INT11 PIC 9(18). + 10 NUM-STR-INT12 PIC 9(19). + 10 NUM-STR-INT13 PIC 9(20). + 10 NUM-STR-INT14 PIC 9(37). + +********** Integral signed numbers formatted as strings + 10 NUM-STR-SINT02 PIC S9(2). + 10 NUM-STR-SINT03 PIC S9(3). + 10 NUM-STR-SINT04 PIC S9(4). + 10 NUM-STR-SINT05 PIC S9(5). + 10 NUM-STR-SINT06 PIC S9(8). + 10 NUM-STR-SINT07 PIC S9(9). + 10 NUM-STR-SINT08 PIC S9(10). + 10 NUM-STR-SINT09 PIC S9(11). + 10 NUM-STR-SINT10 PIC S9(17). + 10 NUM-STR-SINT11 PIC S9(18). + 10 NUM-STR-SINT12 PIC S9(19). + 10 NUM-STR-SINT13 PIC S9(20). + 10 NUM-STR-SINT14 PIC S9(37). + +********** Decimal numbers formatted as strings + 10 NUM-STR-DEC01 PIC 99V9. + 10 NUM-STR-DEC02 PIC 99V99. + 10 NUM-STR-DEC03 PIC 9(3)V99. + 10 NUM-STR-DEC04 PIC 9(4)V9(4). + 10 NUM-STR-DEC05 PIC 9(5)V9(4). + 10 NUM-STR-DEC06 PIC 9(5)V9(5). + 10 NUM-STR-DEC07 PIC 9(15)V99. + 10 NUM-STR-DEC08 PIC 9(16)V99. + 10 NUM-STR-DEC09 PIC 9(17)V99. + 10 NUM-STR-DEC10 PIC 9(18)V9(10). + 10 NUM-STR-SDEC01 PIC S99V9. + 10 NUM-STR-SDEC02 PIC S99V99. + 10 NUM-STR-SDEC03 PIC S9(3)V99. + 10 NUM-STR-SDEC04 PIC S9(4)V9(4). + 10 NUM-STR-SDEC05 PIC S9(5)V9(4). + 10 NUM-STR-SDEC06 PIC S9(5)V9(5). + 10 NUM-STR-SDEC07 PIC S9(15)V99. + 10 NUM-STR-SDEC08 PIC S9(16)V99. + 10 NUM-STR-SDEC09 PIC S9(17)V99. + 10 NUM-STR-SDEC10 PIC S9(18)V9(10). +********** These types are currently not supported, added for the future + 10 NUM-STR-EDEC03 PIC S9(3).99. + 10 NUM-STR-EDEC04 PIC S9(4).9(4). + 10 NUM-STR-EDEC05 PIC S9(5).9(4). + 10 NUM-STR-EDEC06 PIC S9(5).9(5). + +********** Binary formatted integral numbers + 10 NUM-BIN-INT01 PIC 9(1) COMP. + 10 NUM-BIN-INT02 PIC 9(2) COMP. + 10 NUM-BIN-INT03 PIC 9(3) COMP-0. + 10 NUM-BIN-INT04 PIC 9(4) COMP-4. + 10 NUM-BIN-INT05 PIC 9(5) COMP-5. + 10 NUM-BIN-INT06 PIC 9(8) BINARY. + 10 NUM-BIN-INT07 PIC 9(9) BINARY. + 10 NUM-BIN-INT08 PIC 9(10) BINARY. + 10 NUM-BIN-INT09 PIC 9(11) BINARY. + 10 NUM-BIN-INT10 PIC 9(17) BINARY. + 10 NUM-BIN-INT11 PIC 9(18) BINARY. + 10 NUM-BIN-INT12 PIC 9(19) BINARY. + 10 NUM-BIN-INT13 PIC 9(20) BINARY. + 10 NUM-BIN-INT14 PIC 9(37) BINARY. + 10 NUM-SBIN-SINT01 PIC S9(1) COMP. + 10 NUM-SBIN-SINT02 PIC S9(2) COMP. + 10 NUM-SBIN-SINT03 PIC S9(3) COMP. + 10 NUM-SBIN-SINT04 PIC S9(4) COMP. + 10 NUM-SBIN-SINT05 PIC S9(5) COMP. + 10 NUM-SBIN-SINT06 PIC S9(8) BINARY. + 10 NUM-SBIN-SINT07 PIC S9(9) BINARY. + 10 NUM-SBIN-SINT08 PIC S9(10) BINARY. + 10 NUM-SBIN-SINT09 PIC S9(11) BINARY. + 10 NUM-SBIN-SINT10 PIC S9(17) BINARY. + 10 NUM-SBIN-SINT11 PIC S9(18) BINARY. + 10 NUM-SBIN-SINT12 PIC S9(19) BINARY. + 10 NUM-SBIN-SINT13 PIC S9(20) BINARY. + 10 NUM-SBIN-SINT14 PIC S9(37) BINARY. + +********** Binary formatted decimal numbers + 10 NUM-BIN-DEC01 PIC 99V9 COMP. + 10 NUM-BIN-DEC02 PIC 99V99 COMP. + 10 NUM-BIN-DEC03 PIC 9(3)V99 COMP. + 10 NUM-BIN-DEC04 PIC 9(4)V9(4) COMP. + 10 NUM-BIN-DEC05 PIC 9(5)V9(4) COMP. + 10 NUM-BIN-DEC06 PIC 9(5)V9(5) COMP. + 10 NUM-BIN-DEC07 PIC 9(15)V99 COMP. + 10 NUM-BIN-DEC08 PIC 9(16)V99 COMP. + 10 NUM-BIN-DEC09 PIC 9(17)V99 COMP. + 10 NUM-BIN-DEC10 PIC 9(18)V9(10) COMP. + 10 NUM-SBIN-DEC01 PIC S99V9 COMP. + 10 NUM-SBIN-DEC02 PIC S99V99 COMP. + 10 NUM-SBIN-DEC03 PIC S9(3)V99 COMP. + 10 NUM-SBIN-DEC04 PIC S9(4)V9(4) COMP. + 10 NUM-SBIN-DEC05 PIC S9(5)V9(4) COMP. + 10 NUM-SBIN-DEC06 PIC S9(5)V9(5) COMP. + 10 NUM-SBIN-DEC07 PIC S9(15)V99 COMP. + 10 NUM-SBIN-DEC08 PIC S9(16)V99 COMP. + 10 NUM-SBIN-DEC09 PIC S9(17)V99 COMP. + 10 NUM-SBIN-DEC10 PIC S9(18)V9(10) COMP. + +********** BCD formatted integral numbers + 10 NUM-BCD-INT01 PIC 9(1) COMP-3. + 10 NUM-BCD-INT02 PIC 9(2) COMP-3. + 10 NUM-BCD-INT03 PIC 9(3) COMP-3. + 10 NUM-BCD-INT04 PIC 9(4) COMP-3. + 10 NUM-BCD-INT05 PIC 9(5) COMP-3. + 10 NUM-BCD-INT06 PIC 9(8) COMP-3. + 10 NUM-BCD-INT07 PIC 9(9) COMP-3. + 10 NUM-BCD-INT08 PIC 9(10) COMP-3. + 10 NUM-BCD-INT09 PIC 9(11) COMP-3. + 10 NUM-BCD-INT10 PIC 9(17) COMP-3. + 10 NUM-BCD-INT11 PIC 9(18) COMP-3. + 10 NUM-BCD-INT12 PIC 9(19) COMP-3. + 10 NUM-BCD-INT13 PIC 9(20) COMP-3. + 10 NUM-BCD-INT14 PIC 9(37) COMP-3. + + 10 NUM-BCD-SINT01 PIC S9(1) COMP-3. + 10 NUM-BCD-SINT02 PIC S9(2) COMP-3. + 10 NUM-BCD-SINT03 PIC S9(3) COMP-3. + 10 NUM-BCD-SINT04 PIC S9(4) COMP-3. + 10 NUM-BCD-SINT05 PIC S9(5) COMP-3. + 10 NUM-BCD-SINT06 PIC S9(8) COMP-3. + 10 NUM-BCD-SINT07 PIC S9(9) COMP-3. + 10 NUM-BCD-SINT08 PIC S9(10) COMP-3. + 10 NUM-BCD-SINT09 PIC S9(11) COMP-3. + 10 NUM-BCD-SINT10 PIC S9(17) COMP-3. + 10 NUM-BCD-SINT11 PIC S9(18) COMP-3. + 10 NUM-BCD-SINT12 PIC S9(19) COMP-3. + 10 NUM-BCD-SINT13 PIC S9(20) COMP-3. + 10 NUM-BCD-SINT14 PIC S9(37) COMP-3. + +********** BCD formatted decimal numbers + 10 NUM-BCD-DEC01 PIC 99V9 COMP-3. + 10 NUM-BCD-DEC02 PIC 99V99 COMP-3. + 10 NUM-BCD-DEC03 PIC 9(3)V99 COMP-3. + 10 NUM-BCD-DEC04 PIC 9(4)V9(4) COMP-3. + 10 NUM-BCD-DEC05 PIC 9(5)V9(4) COMP-3. + 10 NUM-BCD-DEC06 PIC 9(5)V9(5) COMP-3. + 10 NUM-BCD-DEC07 PIC 9(15)V99 COMP-3. + 10 NUM-BCD-DEC08 PIC 9(16)V99 COMP-3. + 10 NUM-BCD-DEC09 PIC 9(17)V99 COMP-3. + 10 NUM-BCD-DEC10 PIC 9(18)V9(10) COMP-3. + 10 NUM-BCD-SDEC01 PIC S99V9 COMP-3. + 10 NUM-BCD-SDEC02 PIC S99V99 COMP-3. + 10 NUM-BCD-SDEC03 PIC S9(3)V99 COMP-3. + 10 NUM-BCD-SDEC04 PIC S9(4)V9(4) COMP-3. + 10 NUM-BCD-SDEC05 PIC S9(5)V9(4) COMP-3. + 10 NUM-BCD-SDEC06 PIC S9(5)V9(5) COMP-3. + 10 NUM-BCD-SDEC07 PIC S9(15)V99 COMP-3. + 10 NUM-BCD-SDEC08 PIC S9(16)V99 COMP-3. + 10 NUM-BCD-SDEC09 PIC S9(17)V99 COMP-3. + 10 NUM-BCD-SDEC10 PIC S9(18)V9(10) COMP-3. + +********** sign trailing numbers + 10 NUM-SL-STR-INT01 PIC S9(9) SIGN IS + LEADING SEPARATE. + 10 NUM-SL-STR-DEC01 PIC 99V99 SIGN IS + LEADING SEPARATE CHARACTER. + 10 NUM-ST-STR-INT01 PIC S9(9) SIGN IS + TRAILING SEPARATE. + 10 NUM-ST-STR-DEC01 PIC 99V99 SIGN + TRAILING SEPARATE. + 10 NUM-SLI-STR-DEC01 PIC SV9(7) SIGN LEADING. + 10 NUM-STI-STR-DEC01 PIC SV9(7) SIGN TRAILING. + 10 NUM-SLI-DEBUG PIC X(7). + 10 NUM-STI-DEBUG PIC X(7). + +*********************************************************************** +******* FLOATING POINT TYPES +*********************************************************************** + + 10 FLOAT-01 COMP-1. + 10 DOUBLE-01 COMP-2. + +*********************************************************************** +******* COMMON TYPES +*********************************************************************** + 10 COMMON-8-BIN PIC 9(8) BINARY. + 10 COMMON-S3-BIN PIC S9(3) BINARY. + 10 COMMON-S94COMP PIC S9(04) COMP. + 10 COMMON-S8-BIN PIC S9(8) BINARY. + 10 COMMON-DDC97-BIN PIC S9V9(7) BINARY. + 10 COMMON-97COMP3 PIC 9(07) COMP-3. + 10 COMMON-915COMP3 PIC 9(15) COMP-3. + 10 COMMON-S95COMP3 PIC S9(5) COMP-3. + 10 COMMON-S999DCCOMP3 PIC S9(09)V99 COMP-3. + 10 COMMON-S913COMP3 PIC S9(13) COMP-3. + 10 COMMON-S913DCCOMP3 PIC S9(13)V99 COMP-3. + 10 COMMON-S911DCC2 PIC S9(11)V99 COMP-3. + 10 COMMON-S910DCC3 PIC S9(10)V999 COMP-3. + 10 COMMON-S03DDC PIC SV9(5) COMP-3. + 10 COMMON-U03DDC PIC V9(5) COMP-3. + + 10 COMMON-UPC5DDC PIC PPP9(5) COMP-3. + 10 COMMON-SPC5DDC PIC SPP99999 COMP-3. + 10 COMMON-UPI5DDC PIC 9(5)PPP COMP-3. + 10 COMMON-SPI5DDC PIC S99999PPP COMP-3. + + 10 COMMON-UPC5DISP PIC SPPP9(5). + 10 COMMON-UPI5DISP PIC S9(5)PPP. + + 10 COMMON-UPC1BIN PIC SPPP9 COMP. + 10 COMMON-UPI1BIN PIC S9PPPV COMP. + 10 COMMON-UPC3BIN PIC SPPP9(3) COMP. + 10 COMMON-UPI3BIN PIC S9(3)PPP COMP. + 10 COMMON-UPC5BIN PIC SPPP9(5) COMP. + 10 COMMON-UPI5BIN PIC S9(5)PPP COMP. + 10 COMMON-UPC10BIN PIC SPPP9(10) COMP. + 10 COMMON-UPI10BIN PIC S9(10)PPP COMP. + +*********************************************************************** +******* EXOTIC AND COMPILER SPECIFIC +*********************************************************************** + 10 EX-NUM-INT01 PIC +9(8). + 10 EX-NUM-INT02 PIC 9(8)+. + 10 EX-NUM-INT03 PIC -9(8). + 10 EX-NUM-INT04 PIC Z(8)-. + 10 EX-NUM-DEC01 PIC +9(6)V99. + 10 EX-NUM-DEC02 PIC Z(6)VZZ-. + 10 EX-NUM-DEC03 PIC 9(6).99-. + + diff --git a/data/test40_data/example.dat b/data/test40_data/example.dat new file mode 100644 index 000000000..ad2684283 Binary files /dev/null and b/data/test40_data/example.dat differ diff --git a/data/test40_data/example.dat.bz2 b/data/test40_data/example.dat.bz2 new file mode 100644 index 000000000..0390c0263 Binary files /dev/null and b/data/test40_data/example.dat.bz2 differ diff --git a/data/test40_data/example.dat.gz b/data/test40_data/example.dat.gz new file mode 100644 index 000000000..48ae14512 Binary files /dev/null and b/data/test40_data/example.dat.gz differ diff --git a/data/test40_data_ascii/ascii.txt b/data/test40_data_ascii/ascii.txt new file mode 100644 index 000000000..d7b80d288 --- /dev/null +++ b/data/test40_data_ascii/ascii.txt @@ -0,0 +1,3 @@ +12345 +67890 +A1234 \ No newline at end of file diff --git a/data/test40_data_ascii/ascii.txt.gz b/data/test40_data_ascii/ascii.txt.gz new file mode 100644 index 000000000..1cf90242b Binary files /dev/null and b/data/test40_data_ascii/ascii.txt.gz differ diff --git a/data/test40_expected/test40.txt b/data/test40_expected/test40.txt new file mode 100644 index 000000000..308731a94 --- /dev/null +++ b/data/test40_expected/test40.txt @@ -0,0 +1,20 @@ +{"ID":1,"STRING_VAL":"Timika","NUM_STR_INT01":3,"NUM_STR_INT02":30,"NUM_STR_INT03":305,"NUM_STR_INT04":3050,"NUM_STR_INT05":30503,"NUM_STR_INT06":30503932,"NUM_STR_INT07":305039325,"NUM_STR_INT08":3050393257,"NUM_STR_INT09":30503932576,"NUM_STR_INT10":30503932576762676,"NUM_STR_INT11":305039325767626768,"NUM_STR_INT12":3050393257676267687,"NUM_STR_INT13":30503932576762676870,"NUM_STR_INT14":3050393257676267687078781717600592714,"NUM_STR_SINT02":-30,"NUM_STR_SINT03":-305,"NUM_STR_SINT04":-3050,"NUM_STR_SINT05":-30503,"NUM_STR_SINT06":-30503932,"NUM_STR_SINT07":-305039325,"NUM_STR_SINT08":-3050393257,"NUM_STR_SINT09":-30503932576,"NUM_STR_SINT10":-30503932576762676,"NUM_STR_SINT11":-305039325767626768,"NUM_STR_SINT12":-3050393257676267687,"NUM_STR_SINT13":-30503932576762676870,"NUM_STR_SINT14":-3050393257676267687078781717600592714,"NUM_STR_DEC01":30.5,"NUM_STR_DEC02":30.50,"NUM_STR_DEC03":305.03,"NUM_STR_DEC04":3050.3932,"NUM_STR_DEC05":30503.9325,"NUM_STR_DEC06":30503.93257,"NUM_STR_DEC07":305039325767626.76,"NUM_STR_DEC08":3050393257676267.68,"NUM_STR_DEC09":30503932576762676.87,"NUM_STR_DEC10":305039325767626768.7078781717,"NUM_STR_SDEC01":-30.5,"NUM_STR_SDEC02":-30.50,"NUM_STR_SDEC03":-305.03,"NUM_STR_SDEC04":-3050.3932,"NUM_STR_SDEC05":-30503.9325,"NUM_STR_SDEC06":-30503.93257,"NUM_STR_SDEC07":-305039325767626.76,"NUM_STR_SDEC08":-3050393257676267.68,"NUM_STR_SDEC09":-30503932576762676.87,"NUM_STR_SDEC10":-305039325767626768.7078781717,"NUM_STR_EDEC03":-305.03,"NUM_STR_EDEC04":-3050.3932,"NUM_STR_EDEC05":-30503.9325,"NUM_STR_EDEC06":-30503.93257,"NUM_BIN_INT01":3,"NUM_BIN_INT02":30,"NUM_BIN_INT03":305,"NUM_BIN_INT04":3050,"NUM_BIN_INT05":30503,"NUM_BIN_INT06":30503932,"NUM_BIN_INT07":305039325,"NUM_BIN_INT08":3050393257,"NUM_BIN_INT09":30503932576,"NUM_BIN_INT10":30503932576762676,"NUM_BIN_INT11":305039325767626768,"NUM_BIN_INT12":3050393257676267687,"NUM_BIN_INT13":30503932576762676870,"NUM_BIN_INT14":3050393257676267687078781717600592714,"NUM_SBIN_SINT01":-3,"NUM_SBIN_SINT02":-30,"NUM_SBIN_SINT03":-305,"NUM_SBIN_SINT04":-3050,"NUM_SBIN_SINT05":-30503,"NUM_SBIN_SINT06":-30503932,"NUM_SBIN_SINT07":-305039325,"NUM_SBIN_SINT08":-3050393257,"NUM_SBIN_SINT09":-30503932576,"NUM_SBIN_SINT10":-30503932576762676,"NUM_SBIN_SINT11":-305039325767626768,"NUM_SBIN_SINT12":-3050393257676267687,"NUM_SBIN_SINT13":-30503932576762676870,"NUM_SBIN_SINT14":-3050393257676267687078781717600592714,"NUM_BIN_DEC01":30.5,"NUM_BIN_DEC02":30.50,"NUM_BIN_DEC03":305.03,"NUM_BIN_DEC04":3050.3932,"NUM_BIN_DEC05":30503.9325,"NUM_BIN_DEC06":30503.93257,"NUM_BIN_DEC07":305039325767626.76,"NUM_BIN_DEC08":3050393257676267.68,"NUM_BIN_DEC09":30503932576762676.87,"NUM_BIN_DEC10":305039325767626768.7078781717,"NUM_SBIN_DEC01":-30.5,"NUM_SBIN_DEC02":-30.50,"NUM_SBIN_DEC03":-305.03,"NUM_SBIN_DEC04":-3050.3932,"NUM_SBIN_DEC05":-30503.9325,"NUM_SBIN_DEC06":-30503.93257,"NUM_SBIN_DEC07":-305039325767626.76,"NUM_SBIN_DEC08":-3050393257676267.68,"NUM_SBIN_DEC09":-30503932576762676.87,"NUM_SBIN_DEC10":-305039325767626768.7078781717,"NUM_BCD_INT01":3,"NUM_BCD_INT02":30,"NUM_BCD_INT03":305,"NUM_BCD_INT04":3050,"NUM_BCD_INT05":30503,"NUM_BCD_INT06":30503932,"NUM_BCD_INT07":305039325,"NUM_BCD_INT08":3050393257,"NUM_BCD_INT09":30503932576,"NUM_BCD_INT10":30503932576762676,"NUM_BCD_INT11":305039325767626768,"NUM_BCD_INT12":3050393257676267687,"NUM_BCD_INT13":30503932576762676870,"NUM_BCD_INT14":3050393257676267687078781717600592714,"NUM_BCD_SINT01":-3,"NUM_BCD_SINT02":-30,"NUM_BCD_SINT03":-305,"NUM_BCD_SINT04":-3050,"NUM_BCD_SINT05":-30503,"NUM_BCD_SINT06":-30503932,"NUM_BCD_SINT07":-305039325,"NUM_BCD_SINT08":-3050393257,"NUM_BCD_SINT09":-30503932576,"NUM_BCD_SINT10":-30503932576762676,"NUM_BCD_SINT11":-305039325767626768,"NUM_BCD_SINT12":-3050393257676267687,"NUM_BCD_SINT13":-30503932576762676870,"NUM_BCD_SINT14":-3050393257676267687078781717600592714,"NUM_BCD_DEC01":30.5,"NUM_BCD_DEC02":30.50,"NUM_BCD_DEC03":305.03,"NUM_BCD_DEC04":3050.3932,"NUM_BCD_DEC05":30503.9325,"NUM_BCD_DEC06":30503.93257,"NUM_BCD_DEC07":305039325767626.76,"NUM_BCD_DEC08":3050393257676267.68,"NUM_BCD_DEC09":30503932576762676.87,"NUM_BCD_DEC10":305039325767626768.7078781717,"NUM_BCD_SDEC01":-30.5,"NUM_BCD_SDEC02":-30.50,"NUM_BCD_SDEC03":-305.03,"NUM_BCD_SDEC04":-3050.3932,"NUM_BCD_SDEC05":-30503.9325,"NUM_BCD_SDEC06":-30503.93257,"NUM_BCD_SDEC07":-305039325767626.76,"NUM_BCD_SDEC08":-3050393257676267.68,"NUM_BCD_SDEC09":-30503932576762676.87,"NUM_BCD_SDEC10":-305039325767626768.7078781717,"NUM_SL_STR_INT01":-305039325,"NUM_SL_STR_DEC01":-30.50,"NUM_ST_STR_INT01":-305039325,"NUM_ST_STR_DEC01":-30.50,"NUM_SLI_STR_DEC01":-0.3050393,"NUM_STI_STR_DEC01":-0.3050393,"NUM_SLI_DEBUG":"L050393","NUM_STI_DEBUG":"305039L","FLOAT_01":-30503.93,"DOUBLE_01":-3.0503932576762E9,"COMMON_8_BIN":30503932,"COMMON_S3_BIN":305,"COMMON_S94COMP":3050,"COMMON_S8_BIN":30503932,"COMMON_DDC97_BIN":3.0503932,"COMMON_97COMP3":3050393,"COMMON_915COMP3":305039325767626,"COMMON_S95COMP3":-30503,"COMMON_S999DCCOMP3":-305039325.76,"COMMON_S913COMP3":-3050393257676,"COMMON_S913DCCOMP3":-3050393257676.26,"COMMON_S911DCC2":-30503932576.76,"COMMON_S910DCC3":-3050393257.676,"COMMON_S03DDC":-0.30503,"COMMON_U03DDC":0.30503,"COMMON_UPC5DDC":0.00030503,"COMMON_SPC5DDC":-0.0030503,"COMMON_UPI5DDC":30503000,"COMMON_SPI5DDC":-30503000,"COMMON_UPC5DISP":-0.00030503,"COMMON_UPI5DISP":-30503000,"COMMON_UPC1BIN":0.0003,"COMMON_UPI1BIN":3000,"COMMON_UPC3BIN":0.000305,"COMMON_UPI3BIN":305000,"COMMON_UPC5BIN":0.00030503,"COMMON_UPI5BIN":30503000,"COMMON_UPC10BIN":0.0003050393257,"COMMON_UPI10BIN":3050393257000,"EX_NUM_INT01":-30503932,"EX_NUM_INT02":-30503932,"EX_NUM_INT03":-30503932,"EX_NUM_INT04":-30503932,"EX_NUM_DEC01":-305039.32,"EX_NUM_DEC02":-305039.32,"EX_NUM_DEC03":-305039.32} +{"ID":2,"STRING_VAL":"Doretha","NUM_STR_INT01":7,"NUM_STR_INT02":78,"NUM_STR_INT03":784,"NUM_STR_INT04":7844,"NUM_STR_INT05":78449,"NUM_STR_INT06":78449737,"NUM_STR_INT07":784497377,"NUM_STR_INT08":7844973777,"NUM_STR_INT09":78449737776,"NUM_STR_INT10":78449737776077298,"NUM_STR_INT11":784497377760772988,"NUM_STR_INT12":7844973777607729880,"NUM_STR_INT13":78449737776077298809,"NUM_STR_INT14":7844973777607729880906369424872268420,"NUM_STR_SINT02":78,"NUM_STR_SINT03":784,"NUM_STR_SINT04":7844,"NUM_STR_SINT05":78449,"NUM_STR_SINT06":78449737,"NUM_STR_SINT07":784497377,"NUM_STR_SINT08":7844973777,"NUM_STR_SINT09":78449737776,"NUM_STR_SINT10":78449737776077298,"NUM_STR_SINT11":784497377760772988,"NUM_STR_SINT12":7844973777607729880,"NUM_STR_SINT13":78449737776077298809,"NUM_STR_SINT14":7844973777607729880906369424872268420,"NUM_STR_DEC01":78.4,"NUM_STR_DEC02":78.44,"NUM_STR_DEC03":784.49,"NUM_STR_DEC04":7844.9737,"NUM_STR_DEC05":78449.7377,"NUM_STR_DEC06":78449.73777,"NUM_STR_DEC07":784497377760772.98,"NUM_STR_DEC08":7844973777607729.88,"NUM_STR_DEC09":78449737776077298.80,"NUM_STR_DEC10":784497377760772988.0906369424,"NUM_STR_SDEC01":78.4,"NUM_STR_SDEC02":78.44,"NUM_STR_SDEC03":784.49,"NUM_STR_SDEC04":7844.9737,"NUM_STR_SDEC05":78449.7377,"NUM_STR_SDEC06":78449.73777,"NUM_STR_SDEC07":784497377760772.98,"NUM_STR_SDEC08":7844973777607729.88,"NUM_STR_SDEC09":78449737776077298.80,"NUM_STR_SDEC10":784497377760772988.0906369424,"NUM_STR_EDEC03":784.49,"NUM_STR_EDEC04":7844.9737,"NUM_STR_EDEC05":78449.7377,"NUM_STR_EDEC06":78449.73777,"NUM_BIN_INT01":7,"NUM_BIN_INT02":78,"NUM_BIN_INT03":784,"NUM_BIN_INT04":7844,"NUM_BIN_INT05":78449,"NUM_BIN_INT06":78449737,"NUM_BIN_INT07":784497377,"NUM_BIN_INT08":7844973777,"NUM_BIN_INT09":78449737776,"NUM_BIN_INT10":78449737776077298,"NUM_BIN_INT11":784497377760772988,"NUM_BIN_INT12":7844973777607729880,"NUM_BIN_INT13":78449737776077298809,"NUM_BIN_INT14":7844973777607729880906369424872268420,"NUM_SBIN_SINT01":7,"NUM_SBIN_SINT02":78,"NUM_SBIN_SINT03":784,"NUM_SBIN_SINT04":7844,"NUM_SBIN_SINT05":78449,"NUM_SBIN_SINT06":78449737,"NUM_SBIN_SINT07":784497377,"NUM_SBIN_SINT08":7844973777,"NUM_SBIN_SINT09":78449737776,"NUM_SBIN_SINT10":78449737776077298,"NUM_SBIN_SINT11":784497377760772988,"NUM_SBIN_SINT12":7844973777607729880,"NUM_SBIN_SINT13":78449737776077298809,"NUM_SBIN_SINT14":7844973777607729880906369424872268420,"NUM_BIN_DEC01":78.4,"NUM_BIN_DEC02":78.44,"NUM_BIN_DEC03":784.49,"NUM_BIN_DEC04":7844.9737,"NUM_BIN_DEC05":78449.7377,"NUM_BIN_DEC06":78449.73777,"NUM_BIN_DEC07":784497377760772.98,"NUM_BIN_DEC08":7844973777607729.88,"NUM_BIN_DEC09":78449737776077298.80,"NUM_BIN_DEC10":784497377760772988.0906369424,"NUM_SBIN_DEC01":78.4,"NUM_SBIN_DEC02":78.44,"NUM_SBIN_DEC03":784.49,"NUM_SBIN_DEC04":7844.9737,"NUM_SBIN_DEC05":78449.7377,"NUM_SBIN_DEC06":78449.73777,"NUM_SBIN_DEC07":784497377760772.98,"NUM_SBIN_DEC08":7844973777607729.88,"NUM_SBIN_DEC09":78449737776077298.80,"NUM_SBIN_DEC10":784497377760772988.0906369424,"NUM_BCD_INT01":7,"NUM_BCD_INT02":78,"NUM_BCD_INT03":784,"NUM_BCD_INT04":7844,"NUM_BCD_INT05":78449,"NUM_BCD_INT06":78449737,"NUM_BCD_INT07":784497377,"NUM_BCD_INT08":7844973777,"NUM_BCD_INT09":78449737776,"NUM_BCD_INT10":78449737776077298,"NUM_BCD_INT11":784497377760772988,"NUM_BCD_INT12":7844973777607729880,"NUM_BCD_INT13":78449737776077298809,"NUM_BCD_INT14":7844973777607729880906369424872268420,"NUM_BCD_SINT01":7,"NUM_BCD_SINT02":78,"NUM_BCD_SINT03":784,"NUM_BCD_SINT04":7844,"NUM_BCD_SINT05":78449,"NUM_BCD_SINT06":78449737,"NUM_BCD_SINT07":784497377,"NUM_BCD_SINT08":7844973777,"NUM_BCD_SINT09":78449737776,"NUM_BCD_SINT10":78449737776077298,"NUM_BCD_SINT11":784497377760772988,"NUM_BCD_SINT12":7844973777607729880,"NUM_BCD_SINT13":78449737776077298809,"NUM_BCD_SINT14":7844973777607729880906369424872268420,"NUM_BCD_DEC01":78.4,"NUM_BCD_DEC02":78.44,"NUM_BCD_DEC03":784.49,"NUM_BCD_DEC04":7844.9737,"NUM_BCD_DEC05":78449.7377,"NUM_BCD_DEC06":78449.73777,"NUM_BCD_DEC07":784497377760772.98,"NUM_BCD_DEC08":7844973777607729.88,"NUM_BCD_DEC09":78449737776077298.80,"NUM_BCD_DEC10":784497377760772988.0906369424,"NUM_BCD_SDEC01":78.4,"NUM_BCD_SDEC02":78.44,"NUM_BCD_SDEC03":784.49,"NUM_BCD_SDEC04":7844.9737,"NUM_BCD_SDEC05":78449.7377,"NUM_BCD_SDEC06":78449.73777,"NUM_BCD_SDEC07":784497377760772.98,"NUM_BCD_SDEC08":7844973777607729.88,"NUM_BCD_SDEC09":78449737776077298.80,"NUM_BCD_SDEC10":784497377760772988.0906369424,"NUM_SL_STR_INT01":784497377,"NUM_SL_STR_DEC01":78.44,"NUM_ST_STR_INT01":784497377,"NUM_ST_STR_DEC01":78.44,"NUM_SLI_STR_DEC01":0.7844973,"NUM_STI_STR_DEC01":0.7844973,"NUM_SLI_DEBUG":"G844973","NUM_STI_DEBUG":"784497C","FLOAT_01":78449.73,"DOUBLE_01":7.8449737776077E9,"COMMON_8_BIN":78449737,"COMMON_S3_BIN":784,"COMMON_S94COMP":7844,"COMMON_S8_BIN":78449737,"COMMON_DDC97_BIN":7.8449737,"COMMON_97COMP3":7844973,"COMMON_915COMP3":784497377760772,"COMMON_S95COMP3":78449,"COMMON_S999DCCOMP3":784497377.76,"COMMON_S913COMP3":7844973777607,"COMMON_S913DCCOMP3":7844973777607.72,"COMMON_S911DCC2":78449737776.07,"COMMON_S910DCC3":7844973777.607,"COMMON_S03DDC":0.78449,"COMMON_U03DDC":0.78449,"COMMON_UPC5DDC":0.00078449,"COMMON_SPC5DDC":0.0078449,"COMMON_UPI5DDC":78449000,"COMMON_SPI5DDC":78449000,"COMMON_UPC5DISP":0.00078449,"COMMON_UPI5DISP":78449000,"COMMON_UPC1BIN":0.0007,"COMMON_UPI1BIN":7000,"COMMON_UPC3BIN":0.000784,"COMMON_UPI3BIN":784000,"COMMON_UPC5BIN":0.00078449,"COMMON_UPI5BIN":78449000,"COMMON_UPC10BIN":0.0007844973777,"COMMON_UPI10BIN":7844973777000,"EX_NUM_INT01":78449737,"EX_NUM_INT02":78449737,"EX_NUM_INT03":78449737,"EX_NUM_INT04":78449737,"EX_NUM_DEC01":784497.37,"EX_NUM_DEC02":784497.37,"EX_NUM_DEC03":784497.37} +{"ID":3,"STRING_VAL":"Edyth","NUM_STR_INT01":3,"NUM_STR_INT02":39,"NUM_STR_INT03":393,"NUM_STR_INT04":3934,"NUM_STR_INT05":39345,"NUM_STR_INT06":39345497,"NUM_STR_INT07":393454975,"NUM_STR_INT08":3934549756,"NUM_STR_INT09":39345497564,"NUM_STR_INT10":39345497564527858,"NUM_STR_INT11":393454975645278586,"NUM_STR_INT12":3934549756452785864,"NUM_STR_INT13":39345497564527858640,"NUM_STR_INT14":3934549756452785864011258727866492550,"NUM_STR_SINT02":39,"NUM_STR_SINT03":393,"NUM_STR_SINT04":3934,"NUM_STR_SINT05":39345,"NUM_STR_SINT06":39345497,"NUM_STR_SINT07":393454975,"NUM_STR_SINT08":3934549756,"NUM_STR_SINT09":39345497564,"NUM_STR_SINT10":39345497564527858,"NUM_STR_SINT11":393454975645278586,"NUM_STR_SINT12":3934549756452785864,"NUM_STR_SINT13":39345497564527858640,"NUM_STR_SINT14":3934549756452785864011258727866492550,"NUM_STR_DEC01":39.3,"NUM_STR_DEC02":39.34,"NUM_STR_DEC03":393.45,"NUM_STR_DEC04":3934.5497,"NUM_STR_DEC05":39345.4975,"NUM_STR_DEC06":39345.49756,"NUM_STR_DEC07":393454975645278.58,"NUM_STR_DEC08":3934549756452785.86,"NUM_STR_DEC09":39345497564527858.64,"NUM_STR_DEC10":393454975645278586.4011258727,"NUM_STR_SDEC01":39.3,"NUM_STR_SDEC02":39.34,"NUM_STR_SDEC03":393.45,"NUM_STR_SDEC04":3934.5497,"NUM_STR_SDEC05":39345.4975,"NUM_STR_SDEC06":39345.49756,"NUM_STR_SDEC07":393454975645278.58,"NUM_STR_SDEC08":3934549756452785.86,"NUM_STR_SDEC09":39345497564527858.64,"NUM_STR_SDEC10":393454975645278586.4011258727,"NUM_STR_EDEC03":393.45,"NUM_STR_EDEC04":3934.5497,"NUM_STR_EDEC05":39345.4975,"NUM_STR_EDEC06":39345.49756,"NUM_BIN_INT01":3,"NUM_BIN_INT02":39,"NUM_BIN_INT03":393,"NUM_BIN_INT04":3934,"NUM_BIN_INT05":39345,"NUM_BIN_INT06":39345497,"NUM_BIN_INT07":393454975,"NUM_BIN_INT08":3934549756,"NUM_BIN_INT09":39345497564,"NUM_BIN_INT10":39345497564527858,"NUM_BIN_INT11":393454975645278586,"NUM_BIN_INT12":3934549756452785864,"NUM_BIN_INT13":39345497564527858640,"NUM_BIN_INT14":3934549756452785864011258727866492550,"NUM_SBIN_SINT01":3,"NUM_SBIN_SINT02":39,"NUM_SBIN_SINT03":393,"NUM_SBIN_SINT04":3934,"NUM_SBIN_SINT05":39345,"NUM_SBIN_SINT06":39345497,"NUM_SBIN_SINT07":393454975,"NUM_SBIN_SINT08":3934549756,"NUM_SBIN_SINT09":39345497564,"NUM_SBIN_SINT10":39345497564527858,"NUM_SBIN_SINT11":393454975645278586,"NUM_SBIN_SINT12":3934549756452785864,"NUM_SBIN_SINT13":39345497564527858640,"NUM_SBIN_SINT14":3934549756452785864011258727866492550,"NUM_BIN_DEC01":39.3,"NUM_BIN_DEC02":39.34,"NUM_BIN_DEC03":393.45,"NUM_BIN_DEC04":3934.5497,"NUM_BIN_DEC05":39345.4975,"NUM_BIN_DEC06":39345.49756,"NUM_BIN_DEC07":393454975645278.58,"NUM_BIN_DEC08":3934549756452785.86,"NUM_BIN_DEC09":39345497564527858.64,"NUM_BIN_DEC10":393454975645278586.4011258727,"NUM_SBIN_DEC01":39.3,"NUM_SBIN_DEC02":39.34,"NUM_SBIN_DEC03":393.45,"NUM_SBIN_DEC04":3934.5497,"NUM_SBIN_DEC05":39345.4975,"NUM_SBIN_DEC06":39345.49756,"NUM_SBIN_DEC07":393454975645278.58,"NUM_SBIN_DEC08":3934549756452785.86,"NUM_SBIN_DEC09":39345497564527858.64,"NUM_SBIN_DEC10":393454975645278586.4011258727,"NUM_BCD_INT01":3,"NUM_BCD_INT02":39,"NUM_BCD_INT03":393,"NUM_BCD_INT04":3934,"NUM_BCD_INT05":39345,"NUM_BCD_INT06":39345497,"NUM_BCD_INT07":393454975,"NUM_BCD_INT08":3934549756,"NUM_BCD_INT09":39345497564,"NUM_BCD_INT10":39345497564527858,"NUM_BCD_INT11":393454975645278586,"NUM_BCD_INT12":3934549756452785864,"NUM_BCD_INT13":39345497564527858640,"NUM_BCD_INT14":3934549756452785864011258727866492550,"NUM_BCD_SINT01":3,"NUM_BCD_SINT02":39,"NUM_BCD_SINT03":393,"NUM_BCD_SINT04":3934,"NUM_BCD_SINT05":39345,"NUM_BCD_SINT06":39345497,"NUM_BCD_SINT07":393454975,"NUM_BCD_SINT08":3934549756,"NUM_BCD_SINT09":39345497564,"NUM_BCD_SINT10":39345497564527858,"NUM_BCD_SINT11":393454975645278586,"NUM_BCD_SINT12":3934549756452785864,"NUM_BCD_SINT13":39345497564527858640,"NUM_BCD_SINT14":3934549756452785864011258727866492550,"NUM_BCD_DEC01":39.3,"NUM_BCD_DEC02":39.34,"NUM_BCD_DEC03":393.45,"NUM_BCD_DEC04":3934.5497,"NUM_BCD_DEC05":39345.4975,"NUM_BCD_DEC06":39345.49756,"NUM_BCD_DEC07":393454975645278.58,"NUM_BCD_DEC08":3934549756452785.86,"NUM_BCD_DEC09":39345497564527858.64,"NUM_BCD_DEC10":393454975645278586.4011258727,"NUM_BCD_SDEC01":39.3,"NUM_BCD_SDEC02":39.34,"NUM_BCD_SDEC03":393.45,"NUM_BCD_SDEC04":3934.5497,"NUM_BCD_SDEC05":39345.4975,"NUM_BCD_SDEC06":39345.49756,"NUM_BCD_SDEC07":393454975645278.58,"NUM_BCD_SDEC08":3934549756452785.86,"NUM_BCD_SDEC09":39345497564527858.64,"NUM_BCD_SDEC10":393454975645278586.4011258727,"NUM_SL_STR_INT01":393454975,"NUM_SL_STR_DEC01":39.34,"NUM_ST_STR_INT01":393454975,"NUM_ST_STR_DEC01":39.34,"NUM_SLI_STR_DEC01":0.3934549,"NUM_STI_STR_DEC01":0.3934549,"NUM_SLI_DEBUG":"C934549","NUM_STI_DEBUG":"393454I","FLOAT_01":39345.49,"DOUBLE_01":3.9345497564527E9,"COMMON_8_BIN":39345497,"COMMON_S3_BIN":393,"COMMON_S94COMP":3934,"COMMON_S8_BIN":39345497,"COMMON_DDC97_BIN":3.9345497,"COMMON_97COMP3":3934549,"COMMON_915COMP3":393454975645278,"COMMON_S95COMP3":39345,"COMMON_S999DCCOMP3":393454975.64,"COMMON_S913COMP3":3934549756452,"COMMON_S913DCCOMP3":3934549756452.78,"COMMON_S911DCC2":39345497564.52,"COMMON_S910DCC3":3934549756.452,"COMMON_S03DDC":0.39345,"COMMON_U03DDC":0.39345,"COMMON_UPC5DDC":0.00039345,"COMMON_SPC5DDC":0.0039345,"COMMON_UPI5DDC":39345000,"COMMON_SPI5DDC":39345000,"COMMON_UPC5DISP":0.00039345,"COMMON_UPI5DISP":39345000,"COMMON_UPC1BIN":0.0003,"COMMON_UPI1BIN":3000,"COMMON_UPC3BIN":0.000393,"COMMON_UPI3BIN":393000,"COMMON_UPC5BIN":0.00039345,"COMMON_UPI5BIN":39345000,"COMMON_UPC10BIN":0.0003934549756,"COMMON_UPI10BIN":3934549756000,"EX_NUM_INT01":39345497,"EX_NUM_INT02":39345497,"EX_NUM_INT03":39345497,"EX_NUM_INT04":39345497,"EX_NUM_DEC01":393454.97,"EX_NUM_DEC02":393454.97,"EX_NUM_DEC03":393454.97} +{"ID":4,"STRING_VAL":"Mabelle","NUM_STR_INT01":4,"NUM_STR_INT02":43,"NUM_STR_INT03":436,"NUM_STR_INT04":4361,"NUM_STR_INT05":43611,"NUM_STR_INT06":43611299,"NUM_STR_INT07":436112992,"NUM_STR_INT08":4361129928,"NUM_STR_INT09":43611299281,"NUM_STR_INT10":43611299281861113,"NUM_STR_INT11":436112992818611137,"NUM_STR_INT12":4361129928186111372,"NUM_STR_INT13":43611299281861113728,"NUM_STR_INT14":4361129928186111372847286275498745718,"NUM_STR_SINT02":-43,"NUM_STR_SINT03":-436,"NUM_STR_SINT04":-4361,"NUM_STR_SINT05":-43611,"NUM_STR_SINT06":-43611299,"NUM_STR_SINT07":-436112992,"NUM_STR_SINT08":-4361129928,"NUM_STR_SINT09":-43611299281,"NUM_STR_SINT10":-43611299281861113,"NUM_STR_SINT11":-436112992818611137,"NUM_STR_SINT12":-4361129928186111372,"NUM_STR_SINT13":-43611299281861113728,"NUM_STR_SINT14":-4361129928186111372847286275498745718,"NUM_STR_DEC01":43.6,"NUM_STR_DEC02":43.61,"NUM_STR_DEC03":436.11,"NUM_STR_DEC04":4361.1299,"NUM_STR_DEC05":43611.2992,"NUM_STR_DEC06":43611.29928,"NUM_STR_DEC07":436112992818611.13,"NUM_STR_DEC08":4361129928186111.37,"NUM_STR_DEC09":43611299281861113.72,"NUM_STR_DEC10":436112992818611137.2847286275,"NUM_STR_SDEC01":-43.6,"NUM_STR_SDEC02":-43.61,"NUM_STR_SDEC03":-436.11,"NUM_STR_SDEC04":-4361.1299,"NUM_STR_SDEC05":-43611.2992,"NUM_STR_SDEC06":-43611.29928,"NUM_STR_SDEC07":-436112992818611.13,"NUM_STR_SDEC08":-4361129928186111.37,"NUM_STR_SDEC09":-43611299281861113.72,"NUM_STR_SDEC10":-436112992818611137.2847286275,"NUM_STR_EDEC03":-436.11,"NUM_STR_EDEC04":-4361.1299,"NUM_STR_EDEC05":-43611.2992,"NUM_STR_EDEC06":-43611.29928,"NUM_BIN_INT01":4,"NUM_BIN_INT02":43,"NUM_BIN_INT03":436,"NUM_BIN_INT04":4361,"NUM_BIN_INT05":43611,"NUM_BIN_INT06":43611299,"NUM_BIN_INT07":436112992,"NUM_BIN_INT08":4361129928,"NUM_BIN_INT09":43611299281,"NUM_BIN_INT10":43611299281861113,"NUM_BIN_INT11":436112992818611137,"NUM_BIN_INT12":4361129928186111372,"NUM_BIN_INT13":43611299281861113728,"NUM_BIN_INT14":4361129928186111372847286275498745718,"NUM_SBIN_SINT01":-4,"NUM_SBIN_SINT02":-43,"NUM_SBIN_SINT03":-436,"NUM_SBIN_SINT04":-4361,"NUM_SBIN_SINT05":-43611,"NUM_SBIN_SINT06":-43611299,"NUM_SBIN_SINT07":-436112992,"NUM_SBIN_SINT08":-4361129928,"NUM_SBIN_SINT09":-43611299281,"NUM_SBIN_SINT10":-43611299281861113,"NUM_SBIN_SINT11":-436112992818611137,"NUM_SBIN_SINT12":-4361129928186111372,"NUM_SBIN_SINT13":-43611299281861113728,"NUM_SBIN_SINT14":-4361129928186111372847286275498745718,"NUM_BIN_DEC01":43.6,"NUM_BIN_DEC02":43.61,"NUM_BIN_DEC03":436.11,"NUM_BIN_DEC04":4361.1299,"NUM_BIN_DEC05":43611.2992,"NUM_BIN_DEC06":43611.29928,"NUM_BIN_DEC07":436112992818611.13,"NUM_BIN_DEC08":4361129928186111.37,"NUM_BIN_DEC09":43611299281861113.72,"NUM_BIN_DEC10":436112992818611137.2847286275,"NUM_SBIN_DEC01":-43.6,"NUM_SBIN_DEC02":-43.61,"NUM_SBIN_DEC03":-436.11,"NUM_SBIN_DEC04":-4361.1299,"NUM_SBIN_DEC05":-43611.2992,"NUM_SBIN_DEC06":-43611.29928,"NUM_SBIN_DEC07":-436112992818611.13,"NUM_SBIN_DEC08":-4361129928186111.37,"NUM_SBIN_DEC09":-43611299281861113.72,"NUM_SBIN_DEC10":-436112992818611137.2847286275,"NUM_BCD_INT01":4,"NUM_BCD_INT02":43,"NUM_BCD_INT03":436,"NUM_BCD_INT04":4361,"NUM_BCD_INT05":43611,"NUM_BCD_INT06":43611299,"NUM_BCD_INT07":436112992,"NUM_BCD_INT08":4361129928,"NUM_BCD_INT09":43611299281,"NUM_BCD_INT10":43611299281861113,"NUM_BCD_INT11":436112992818611137,"NUM_BCD_INT12":4361129928186111372,"NUM_BCD_INT13":43611299281861113728,"NUM_BCD_INT14":4361129928186111372847286275498745718,"NUM_BCD_SINT01":-4,"NUM_BCD_SINT02":-43,"NUM_BCD_SINT03":-436,"NUM_BCD_SINT04":-4361,"NUM_BCD_SINT05":-43611,"NUM_BCD_SINT06":-43611299,"NUM_BCD_SINT07":-436112992,"NUM_BCD_SINT08":-4361129928,"NUM_BCD_SINT09":-43611299281,"NUM_BCD_SINT10":-43611299281861113,"NUM_BCD_SINT11":-436112992818611137,"NUM_BCD_SINT12":-4361129928186111372,"NUM_BCD_SINT13":-43611299281861113728,"NUM_BCD_SINT14":-4361129928186111372847286275498745718,"NUM_BCD_DEC01":43.6,"NUM_BCD_DEC02":43.61,"NUM_BCD_DEC03":436.11,"NUM_BCD_DEC04":4361.1299,"NUM_BCD_DEC05":43611.2992,"NUM_BCD_DEC06":43611.29928,"NUM_BCD_DEC07":436112992818611.13,"NUM_BCD_DEC08":4361129928186111.37,"NUM_BCD_DEC09":43611299281861113.72,"NUM_BCD_DEC10":436112992818611137.2847286275,"NUM_BCD_SDEC01":-43.6,"NUM_BCD_SDEC02":-43.61,"NUM_BCD_SDEC03":-436.11,"NUM_BCD_SDEC04":-4361.1299,"NUM_BCD_SDEC05":-43611.2992,"NUM_BCD_SDEC06":-43611.29928,"NUM_BCD_SDEC07":-436112992818611.13,"NUM_BCD_SDEC08":-4361129928186111.37,"NUM_BCD_SDEC09":-43611299281861113.72,"NUM_BCD_SDEC10":-436112992818611137.2847286275,"NUM_SL_STR_INT01":-436112992,"NUM_SL_STR_DEC01":-43.61,"NUM_ST_STR_INT01":-436112992,"NUM_ST_STR_DEC01":-43.61,"NUM_SLI_STR_DEC01":-0.4361129,"NUM_STI_STR_DEC01":-0.4361129,"NUM_SLI_DEBUG":"M361129","NUM_STI_DEBUG":"436112R","FLOAT_01":-43611.29,"DOUBLE_01":-4.3611299281861E9,"COMMON_8_BIN":43611299,"COMMON_S3_BIN":436,"COMMON_S94COMP":4361,"COMMON_S8_BIN":43611299,"COMMON_DDC97_BIN":4.3611299,"COMMON_97COMP3":4361129,"COMMON_915COMP3":436112992818611,"COMMON_S95COMP3":-43611,"COMMON_S999DCCOMP3":-436112992.81,"COMMON_S913COMP3":-4361129928186,"COMMON_S913DCCOMP3":-4361129928186.11,"COMMON_S911DCC2":-43611299281.86,"COMMON_S910DCC3":-4361129928.186,"COMMON_S03DDC":-0.43611,"COMMON_U03DDC":0.43611,"COMMON_UPC5DDC":0.00043611,"COMMON_SPC5DDC":-0.0043611,"COMMON_UPI5DDC":43611000,"COMMON_SPI5DDC":-43611000,"COMMON_UPC5DISP":-0.00043611,"COMMON_UPI5DISP":-43611000,"COMMON_UPC1BIN":0.0004,"COMMON_UPI1BIN":4000,"COMMON_UPC3BIN":0.000436,"COMMON_UPI3BIN":436000,"COMMON_UPC5BIN":0.00043611,"COMMON_UPI5BIN":43611000,"COMMON_UPC10BIN":0.0004361129928,"COMMON_UPI10BIN":4361129928000,"EX_NUM_INT01":-43611299,"EX_NUM_INT02":-43611299,"EX_NUM_INT03":-43611299,"EX_NUM_INT04":-43611299,"EX_NUM_DEC01":-436112.99,"EX_NUM_DEC02":-436112.99,"EX_NUM_DEC03":-436112.99} +{"ID":5,"STRING_VAL":"Maya","NUM_STR_INT01":6,"NUM_STR_INT02":65,"NUM_STR_INT03":650,"NUM_STR_INT04":6508,"NUM_STR_INT05":65085,"NUM_STR_INT06":65085490,"NUM_STR_INT07":650854909,"NUM_STR_INT08":6508549094,"NUM_STR_INT09":65085490945,"NUM_STR_INT10":65085490945534899,"NUM_STR_INT11":650854909455348990,"NUM_STR_INT12":6508549094553489906,"NUM_STR_INT13":65085490945534899060,"NUM_STR_INT14":6508549094553489906015086081543487432,"NUM_STR_SINT02":-65,"NUM_STR_SINT03":-650,"NUM_STR_SINT04":-6508,"NUM_STR_SINT05":-65085,"NUM_STR_SINT06":-65085490,"NUM_STR_SINT07":-650854909,"NUM_STR_SINT08":-6508549094,"NUM_STR_SINT09":-65085490945,"NUM_STR_SINT10":-65085490945534899,"NUM_STR_SINT11":-650854909455348990,"NUM_STR_SINT12":-6508549094553489906,"NUM_STR_SINT13":-65085490945534899060,"NUM_STR_SINT14":-6508549094553489906015086081543487432,"NUM_STR_DEC01":65.0,"NUM_STR_DEC02":65.08,"NUM_STR_DEC03":650.85,"NUM_STR_DEC04":6508.5490,"NUM_STR_DEC05":65085.4909,"NUM_STR_DEC06":65085.49094,"NUM_STR_DEC07":650854909455348.99,"NUM_STR_DEC08":6508549094553489.90,"NUM_STR_DEC09":65085490945534899.06,"NUM_STR_DEC10":650854909455348990.6015086081,"NUM_STR_SDEC01":-65.0,"NUM_STR_SDEC02":-65.08,"NUM_STR_SDEC03":-650.85,"NUM_STR_SDEC04":-6508.5490,"NUM_STR_SDEC05":-65085.4909,"NUM_STR_SDEC06":-65085.49094,"NUM_STR_SDEC07":-650854909455348.99,"NUM_STR_SDEC08":-6508549094553489.90,"NUM_STR_SDEC09":-65085490945534899.06,"NUM_STR_SDEC10":-650854909455348990.6015086081,"NUM_STR_EDEC03":-650.85,"NUM_STR_EDEC04":-6508.5490,"NUM_STR_EDEC05":-65085.4909,"NUM_STR_EDEC06":-65085.49094,"NUM_BIN_INT01":6,"NUM_BIN_INT02":65,"NUM_BIN_INT03":650,"NUM_BIN_INT04":6508,"NUM_BIN_INT05":65085,"NUM_BIN_INT06":65085490,"NUM_BIN_INT07":650854909,"NUM_BIN_INT08":6508549094,"NUM_BIN_INT09":65085490945,"NUM_BIN_INT10":65085490945534899,"NUM_BIN_INT11":650854909455348990,"NUM_BIN_INT12":6508549094553489906,"NUM_BIN_INT13":65085490945534899060,"NUM_BIN_INT14":6508549094553489906015086081543487432,"NUM_SBIN_SINT01":-6,"NUM_SBIN_SINT02":-65,"NUM_SBIN_SINT03":-650,"NUM_SBIN_SINT04":-6508,"NUM_SBIN_SINT05":-65085,"NUM_SBIN_SINT06":-65085490,"NUM_SBIN_SINT07":-650854909,"NUM_SBIN_SINT08":-6508549094,"NUM_SBIN_SINT09":-65085490945,"NUM_SBIN_SINT10":-65085490945534899,"NUM_SBIN_SINT11":-650854909455348990,"NUM_SBIN_SINT12":-6508549094553489906,"NUM_SBIN_SINT13":-65085490945534899060,"NUM_SBIN_SINT14":-6508549094553489906015086081543487432,"NUM_BIN_DEC01":65.0,"NUM_BIN_DEC02":65.08,"NUM_BIN_DEC03":650.85,"NUM_BIN_DEC04":6508.5490,"NUM_BIN_DEC05":65085.4909,"NUM_BIN_DEC06":65085.49094,"NUM_BIN_DEC07":650854909455348.99,"NUM_BIN_DEC08":6508549094553489.90,"NUM_BIN_DEC09":65085490945534899.06,"NUM_BIN_DEC10":650854909455348990.6015086081,"NUM_SBIN_DEC01":-65.0,"NUM_SBIN_DEC02":-65.08,"NUM_SBIN_DEC03":-650.85,"NUM_SBIN_DEC04":-6508.5490,"NUM_SBIN_DEC05":-65085.4909,"NUM_SBIN_DEC06":-65085.49094,"NUM_SBIN_DEC07":-650854909455348.99,"NUM_SBIN_DEC08":-6508549094553489.90,"NUM_SBIN_DEC09":-65085490945534899.06,"NUM_SBIN_DEC10":-650854909455348990.6015086081,"NUM_BCD_INT01":6,"NUM_BCD_INT02":65,"NUM_BCD_INT03":650,"NUM_BCD_INT04":6508,"NUM_BCD_INT05":65085,"NUM_BCD_INT06":65085490,"NUM_BCD_INT07":650854909,"NUM_BCD_INT08":6508549094,"NUM_BCD_INT09":65085490945,"NUM_BCD_INT10":65085490945534899,"NUM_BCD_INT11":650854909455348990,"NUM_BCD_INT12":6508549094553489906,"NUM_BCD_INT13":65085490945534899060,"NUM_BCD_INT14":6508549094553489906015086081543487432,"NUM_BCD_SINT01":-6,"NUM_BCD_SINT02":-65,"NUM_BCD_SINT03":-650,"NUM_BCD_SINT04":-6508,"NUM_BCD_SINT05":-65085,"NUM_BCD_SINT06":-65085490,"NUM_BCD_SINT07":-650854909,"NUM_BCD_SINT08":-6508549094,"NUM_BCD_SINT09":-65085490945,"NUM_BCD_SINT10":-65085490945534899,"NUM_BCD_SINT11":-650854909455348990,"NUM_BCD_SINT12":-6508549094553489906,"NUM_BCD_SINT13":-65085490945534899060,"NUM_BCD_SINT14":-6508549094553489906015086081543487432,"NUM_BCD_DEC01":65.0,"NUM_BCD_DEC02":65.08,"NUM_BCD_DEC03":650.85,"NUM_BCD_DEC04":6508.5490,"NUM_BCD_DEC05":65085.4909,"NUM_BCD_DEC06":65085.49094,"NUM_BCD_DEC07":650854909455348.99,"NUM_BCD_DEC08":6508549094553489.90,"NUM_BCD_DEC09":65085490945534899.06,"NUM_BCD_DEC10":650854909455348990.6015086081,"NUM_BCD_SDEC01":-65.0,"NUM_BCD_SDEC02":-65.08,"NUM_BCD_SDEC03":-650.85,"NUM_BCD_SDEC04":-6508.5490,"NUM_BCD_SDEC05":-65085.4909,"NUM_BCD_SDEC06":-65085.49094,"NUM_BCD_SDEC07":-650854909455348.99,"NUM_BCD_SDEC08":-6508549094553489.90,"NUM_BCD_SDEC09":-65085490945534899.06,"NUM_BCD_SDEC10":-650854909455348990.6015086081,"NUM_SL_STR_INT01":-650854909,"NUM_SL_STR_DEC01":-65.08,"NUM_ST_STR_INT01":-650854909,"NUM_ST_STR_DEC01":-65.08,"NUM_SLI_STR_DEC01":-0.6508549,"NUM_STI_STR_DEC01":-0.6508549,"NUM_SLI_DEBUG":"O508549","NUM_STI_DEBUG":"650854R","FLOAT_01":-65085.49,"DOUBLE_01":-6.5085490945534E9,"COMMON_8_BIN":65085490,"COMMON_S3_BIN":650,"COMMON_S94COMP":6508,"COMMON_S8_BIN":65085490,"COMMON_DDC97_BIN":6.5085490,"COMMON_97COMP3":6508549,"COMMON_915COMP3":650854909455348,"COMMON_S95COMP3":-65085,"COMMON_S999DCCOMP3":-650854909.45,"COMMON_S913COMP3":-6508549094553,"COMMON_S913DCCOMP3":-6508549094553.48,"COMMON_S911DCC2":-65085490945.53,"COMMON_S910DCC3":-6508549094.553,"COMMON_S03DDC":-0.65085,"COMMON_U03DDC":0.65085,"COMMON_UPC5DDC":0.00065085,"COMMON_SPC5DDC":-0.0065085,"COMMON_UPI5DDC":65085000,"COMMON_SPI5DDC":-65085000,"COMMON_UPC5DISP":-0.00065085,"COMMON_UPI5DISP":-65085000,"COMMON_UPC1BIN":0.0006,"COMMON_UPI1BIN":6000,"COMMON_UPC3BIN":0.000650,"COMMON_UPI3BIN":650000,"COMMON_UPC5BIN":0.00065085,"COMMON_UPI5BIN":65085000,"COMMON_UPC10BIN":0.0006508549094,"COMMON_UPI10BIN":6508549094000,"EX_NUM_INT01":-65085490,"EX_NUM_INT02":-65085490,"EX_NUM_INT03":-65085490,"EX_NUM_INT04":-65085490,"EX_NUM_DEC01":-650854.90,"EX_NUM_DEC02":-650854.90,"EX_NUM_DEC03":-650854.90} +{"ID":6,"STRING_VAL":"Edyth","NUM_STR_INT01":7,"NUM_STR_INT02":70,"NUM_STR_INT03":700,"NUM_STR_INT04":7005,"NUM_STR_INT05":70057,"NUM_STR_INT06":70057604,"NUM_STR_INT07":700576045,"NUM_STR_INT08":7005760450,"NUM_STR_INT09":70057604508,"NUM_STR_INT10":70057604508033027,"NUM_STR_INT11":700576045080330270,"NUM_STR_INT12":7005760450803302707,"NUM_STR_INT13":70057604508033027079,"NUM_STR_INT14":7005760450803302707952366375647360659,"NUM_STR_SINT02":70,"NUM_STR_SINT03":700,"NUM_STR_SINT04":7005,"NUM_STR_SINT05":70057,"NUM_STR_SINT06":70057604,"NUM_STR_SINT07":700576045,"NUM_STR_SINT08":7005760450,"NUM_STR_SINT09":70057604508,"NUM_STR_SINT10":70057604508033027,"NUM_STR_SINT11":700576045080330270,"NUM_STR_SINT12":7005760450803302707,"NUM_STR_SINT13":70057604508033027079,"NUM_STR_SINT14":7005760450803302707952366375647360659,"NUM_STR_DEC01":70.0,"NUM_STR_DEC02":70.05,"NUM_STR_DEC03":700.57,"NUM_STR_DEC04":7005.7604,"NUM_STR_DEC05":70057.6045,"NUM_STR_DEC06":70057.60450,"NUM_STR_DEC07":700576045080330.27,"NUM_STR_DEC08":7005760450803302.70,"NUM_STR_DEC09":70057604508033027.07,"NUM_STR_DEC10":700576045080330270.7952366375,"NUM_STR_SDEC01":70.0,"NUM_STR_SDEC02":70.05,"NUM_STR_SDEC03":700.57,"NUM_STR_SDEC04":7005.7604,"NUM_STR_SDEC05":70057.6045,"NUM_STR_SDEC06":70057.60450,"NUM_STR_SDEC07":700576045080330.27,"NUM_STR_SDEC08":7005760450803302.70,"NUM_STR_SDEC09":70057604508033027.07,"NUM_STR_SDEC10":700576045080330270.7952366375,"NUM_STR_EDEC03":700.57,"NUM_STR_EDEC04":7005.7604,"NUM_STR_EDEC05":70057.6045,"NUM_STR_EDEC06":70057.60450,"NUM_BIN_INT01":7,"NUM_BIN_INT02":70,"NUM_BIN_INT03":700,"NUM_BIN_INT04":7005,"NUM_BIN_INT05":70057,"NUM_BIN_INT06":70057604,"NUM_BIN_INT07":700576045,"NUM_BIN_INT08":7005760450,"NUM_BIN_INT09":70057604508,"NUM_BIN_INT10":70057604508033027,"NUM_BIN_INT11":700576045080330270,"NUM_BIN_INT12":7005760450803302707,"NUM_BIN_INT13":70057604508033027079,"NUM_BIN_INT14":7005760450803302707952366375647360659,"NUM_SBIN_SINT01":7,"NUM_SBIN_SINT02":70,"NUM_SBIN_SINT03":700,"NUM_SBIN_SINT04":7005,"NUM_SBIN_SINT05":70057,"NUM_SBIN_SINT06":70057604,"NUM_SBIN_SINT07":700576045,"NUM_SBIN_SINT08":7005760450,"NUM_SBIN_SINT09":70057604508,"NUM_SBIN_SINT10":70057604508033027,"NUM_SBIN_SINT11":700576045080330270,"NUM_SBIN_SINT12":7005760450803302707,"NUM_SBIN_SINT13":70057604508033027079,"NUM_SBIN_SINT14":7005760450803302707952366375647360659,"NUM_BIN_DEC01":70.0,"NUM_BIN_DEC02":70.05,"NUM_BIN_DEC03":700.57,"NUM_BIN_DEC04":7005.7604,"NUM_BIN_DEC05":70057.6045,"NUM_BIN_DEC06":70057.60450,"NUM_BIN_DEC07":700576045080330.27,"NUM_BIN_DEC08":7005760450803302.70,"NUM_BIN_DEC09":70057604508033027.07,"NUM_BIN_DEC10":700576045080330270.7952366375,"NUM_SBIN_DEC01":70.0,"NUM_SBIN_DEC02":70.05,"NUM_SBIN_DEC03":700.57,"NUM_SBIN_DEC04":7005.7604,"NUM_SBIN_DEC05":70057.6045,"NUM_SBIN_DEC06":70057.60450,"NUM_SBIN_DEC07":700576045080330.27,"NUM_SBIN_DEC08":7005760450803302.70,"NUM_SBIN_DEC09":70057604508033027.07,"NUM_SBIN_DEC10":700576045080330270.7952366375,"NUM_BCD_INT01":7,"NUM_BCD_INT02":70,"NUM_BCD_INT03":700,"NUM_BCD_INT04":7005,"NUM_BCD_INT05":70057,"NUM_BCD_INT06":70057604,"NUM_BCD_INT07":700576045,"NUM_BCD_INT08":7005760450,"NUM_BCD_INT09":70057604508,"NUM_BCD_INT10":70057604508033027,"NUM_BCD_INT11":700576045080330270,"NUM_BCD_INT12":7005760450803302707,"NUM_BCD_INT13":70057604508033027079,"NUM_BCD_INT14":7005760450803302707952366375647360659,"NUM_BCD_SINT01":7,"NUM_BCD_SINT02":70,"NUM_BCD_SINT03":700,"NUM_BCD_SINT04":7005,"NUM_BCD_SINT05":70057,"NUM_BCD_SINT06":70057604,"NUM_BCD_SINT07":700576045,"NUM_BCD_SINT08":7005760450,"NUM_BCD_SINT09":70057604508,"NUM_BCD_SINT10":70057604508033027,"NUM_BCD_SINT11":700576045080330270,"NUM_BCD_SINT12":7005760450803302707,"NUM_BCD_SINT13":70057604508033027079,"NUM_BCD_SINT14":7005760450803302707952366375647360659,"NUM_BCD_DEC01":70.0,"NUM_BCD_DEC02":70.05,"NUM_BCD_DEC03":700.57,"NUM_BCD_DEC04":7005.7604,"NUM_BCD_DEC05":70057.6045,"NUM_BCD_DEC06":70057.60450,"NUM_BCD_DEC07":700576045080330.27,"NUM_BCD_DEC08":7005760450803302.70,"NUM_BCD_DEC09":70057604508033027.07,"NUM_BCD_DEC10":700576045080330270.7952366375,"NUM_BCD_SDEC01":70.0,"NUM_BCD_SDEC02":70.05,"NUM_BCD_SDEC03":700.57,"NUM_BCD_SDEC04":7005.7604,"NUM_BCD_SDEC05":70057.6045,"NUM_BCD_SDEC06":70057.60450,"NUM_BCD_SDEC07":700576045080330.27,"NUM_BCD_SDEC08":7005760450803302.70,"NUM_BCD_SDEC09":70057604508033027.07,"NUM_BCD_SDEC10":700576045080330270.7952366375,"NUM_SL_STR_INT01":700576045,"NUM_SL_STR_DEC01":70.05,"NUM_ST_STR_INT01":700576045,"NUM_ST_STR_DEC01":70.05,"NUM_SLI_STR_DEC01":0.7005760,"NUM_STI_STR_DEC01":0.7005760,"NUM_SLI_DEBUG":"G005760","NUM_STI_DEBUG":"700576{","FLOAT_01":70057.6,"DOUBLE_01":7.0057604508033E9,"COMMON_8_BIN":70057604,"COMMON_S3_BIN":700,"COMMON_S94COMP":7005,"COMMON_S8_BIN":70057604,"COMMON_DDC97_BIN":7.0057604,"COMMON_97COMP3":7005760,"COMMON_915COMP3":700576045080330,"COMMON_S95COMP3":70057,"COMMON_S999DCCOMP3":700576045.08,"COMMON_S913COMP3":7005760450803,"COMMON_S913DCCOMP3":7005760450803.30,"COMMON_S911DCC2":70057604508.03,"COMMON_S910DCC3":7005760450.803,"COMMON_S03DDC":0.70057,"COMMON_U03DDC":0.70057,"COMMON_UPC5DDC":0.00070057,"COMMON_SPC5DDC":0.0070057,"COMMON_UPI5DDC":70057000,"COMMON_SPI5DDC":70057000,"COMMON_UPC5DISP":0.00070057,"COMMON_UPI5DISP":70057000,"COMMON_UPC1BIN":0.0007,"COMMON_UPI1BIN":7000,"COMMON_UPC3BIN":0.000700,"COMMON_UPI3BIN":700000,"COMMON_UPC5BIN":0.00070057,"COMMON_UPI5BIN":70057000,"COMMON_UPC10BIN":0.0007005760450,"COMMON_UPI10BIN":7005760450000,"EX_NUM_INT01":70057604,"EX_NUM_INT02":70057604,"EX_NUM_INT03":70057604,"EX_NUM_INT04":70057604,"EX_NUM_DEC01":700576.04,"EX_NUM_DEC02":700576.04,"EX_NUM_DEC03":700576.04} +{"ID":7,"STRING_VAL":"Willis","NUM_STR_INT01":1,"NUM_STR_INT02":14,"NUM_STR_INT03":147,"NUM_STR_INT04":1476,"NUM_STR_INT05":14767,"NUM_STR_INT06":14767869,"NUM_STR_INT07":147678697,"NUM_STR_INT08":1476786977,"NUM_STR_INT09":14767869778,"NUM_STR_INT10":14767869778395587,"NUM_STR_INT11":147678697783955876,"NUM_STR_INT12":1476786977839558763,"NUM_STR_INT13":14767869778395587632,"NUM_STR_INT14":1476786977839558763280352581936245907,"NUM_STR_SINT02":-14,"NUM_STR_SINT03":-147,"NUM_STR_SINT04":-1476,"NUM_STR_SINT05":-14767,"NUM_STR_SINT06":-14767869,"NUM_STR_SINT07":-147678697,"NUM_STR_SINT08":-1476786977,"NUM_STR_SINT09":-14767869778,"NUM_STR_SINT10":-14767869778395587,"NUM_STR_SINT11":-147678697783955876,"NUM_STR_SINT12":-1476786977839558763,"NUM_STR_SINT13":-14767869778395587632,"NUM_STR_SINT14":-1476786977839558763280352581936245907,"NUM_STR_DEC01":14.7,"NUM_STR_DEC02":14.76,"NUM_STR_DEC03":147.67,"NUM_STR_DEC04":1476.7869,"NUM_STR_DEC05":14767.8697,"NUM_STR_DEC06":14767.86977,"NUM_STR_DEC07":147678697783955.87,"NUM_STR_DEC08":1476786977839558.76,"NUM_STR_DEC09":14767869778395587.63,"NUM_STR_DEC10":147678697783955876.3280352581,"NUM_STR_SDEC01":-14.7,"NUM_STR_SDEC02":-14.76,"NUM_STR_SDEC03":-147.67,"NUM_STR_SDEC04":-1476.7869,"NUM_STR_SDEC05":-14767.8697,"NUM_STR_SDEC06":-14767.86977,"NUM_STR_SDEC07":-147678697783955.87,"NUM_STR_SDEC08":-1476786977839558.76,"NUM_STR_SDEC09":-14767869778395587.63,"NUM_STR_SDEC10":-147678697783955876.3280352581,"NUM_STR_EDEC03":-147.67,"NUM_STR_EDEC04":-1476.7869,"NUM_STR_EDEC05":-14767.8697,"NUM_STR_EDEC06":-14767.86977,"NUM_BIN_INT01":1,"NUM_BIN_INT02":14,"NUM_BIN_INT03":147,"NUM_BIN_INT04":1476,"NUM_BIN_INT05":14767,"NUM_BIN_INT06":14767869,"NUM_BIN_INT07":147678697,"NUM_BIN_INT08":1476786977,"NUM_BIN_INT09":14767869778,"NUM_BIN_INT10":14767869778395587,"NUM_BIN_INT11":147678697783955876,"NUM_BIN_INT12":1476786977839558763,"NUM_BIN_INT13":14767869778395587632,"NUM_BIN_INT14":1476786977839558763280352581936245907,"NUM_SBIN_SINT01":-1,"NUM_SBIN_SINT02":-14,"NUM_SBIN_SINT03":-147,"NUM_SBIN_SINT04":-1476,"NUM_SBIN_SINT05":-14767,"NUM_SBIN_SINT06":-14767869,"NUM_SBIN_SINT07":-147678697,"NUM_SBIN_SINT08":-1476786977,"NUM_SBIN_SINT09":-14767869778,"NUM_SBIN_SINT10":-14767869778395587,"NUM_SBIN_SINT11":-147678697783955876,"NUM_SBIN_SINT12":-1476786977839558763,"NUM_SBIN_SINT13":-14767869778395587632,"NUM_SBIN_SINT14":-1476786977839558763280352581936245907,"NUM_BIN_DEC01":14.7,"NUM_BIN_DEC02":14.76,"NUM_BIN_DEC03":147.67,"NUM_BIN_DEC04":1476.7869,"NUM_BIN_DEC05":14767.8697,"NUM_BIN_DEC06":14767.86977,"NUM_BIN_DEC07":147678697783955.87,"NUM_BIN_DEC08":1476786977839558.76,"NUM_BIN_DEC09":14767869778395587.63,"NUM_BIN_DEC10":147678697783955876.3280352581,"NUM_SBIN_DEC01":-14.7,"NUM_SBIN_DEC02":-14.76,"NUM_SBIN_DEC03":-147.67,"NUM_SBIN_DEC04":-1476.7869,"NUM_SBIN_DEC05":-14767.8697,"NUM_SBIN_DEC06":-14767.86977,"NUM_SBIN_DEC07":-147678697783955.87,"NUM_SBIN_DEC08":-1476786977839558.76,"NUM_SBIN_DEC09":-14767869778395587.63,"NUM_SBIN_DEC10":-147678697783955876.3280352581,"NUM_BCD_INT01":1,"NUM_BCD_INT02":14,"NUM_BCD_INT03":147,"NUM_BCD_INT04":1476,"NUM_BCD_INT05":14767,"NUM_BCD_INT06":14767869,"NUM_BCD_INT07":147678697,"NUM_BCD_INT08":1476786977,"NUM_BCD_INT09":14767869778,"NUM_BCD_INT10":14767869778395587,"NUM_BCD_INT11":147678697783955876,"NUM_BCD_INT12":1476786977839558763,"NUM_BCD_INT13":14767869778395587632,"NUM_BCD_INT14":1476786977839558763280352581936245907,"NUM_BCD_SINT01":-1,"NUM_BCD_SINT02":-14,"NUM_BCD_SINT03":-147,"NUM_BCD_SINT04":-1476,"NUM_BCD_SINT05":-14767,"NUM_BCD_SINT06":-14767869,"NUM_BCD_SINT07":-147678697,"NUM_BCD_SINT08":-1476786977,"NUM_BCD_SINT09":-14767869778,"NUM_BCD_SINT10":-14767869778395587,"NUM_BCD_SINT11":-147678697783955876,"NUM_BCD_SINT12":-1476786977839558763,"NUM_BCD_SINT13":-14767869778395587632,"NUM_BCD_SINT14":-1476786977839558763280352581936245907,"NUM_BCD_DEC01":14.7,"NUM_BCD_DEC02":14.76,"NUM_BCD_DEC03":147.67,"NUM_BCD_DEC04":1476.7869,"NUM_BCD_DEC05":14767.8697,"NUM_BCD_DEC06":14767.86977,"NUM_BCD_DEC07":147678697783955.87,"NUM_BCD_DEC08":1476786977839558.76,"NUM_BCD_DEC09":14767869778395587.63,"NUM_BCD_DEC10":147678697783955876.3280352581,"NUM_BCD_SDEC01":-14.7,"NUM_BCD_SDEC02":-14.76,"NUM_BCD_SDEC03":-147.67,"NUM_BCD_SDEC04":-1476.7869,"NUM_BCD_SDEC05":-14767.8697,"NUM_BCD_SDEC06":-14767.86977,"NUM_BCD_SDEC07":-147678697783955.87,"NUM_BCD_SDEC08":-1476786977839558.76,"NUM_BCD_SDEC09":-14767869778395587.63,"NUM_BCD_SDEC10":-147678697783955876.3280352581,"NUM_SL_STR_INT01":-147678697,"NUM_SL_STR_DEC01":-14.76,"NUM_ST_STR_INT01":-147678697,"NUM_ST_STR_DEC01":-14.76,"NUM_SLI_STR_DEC01":-0.1476786,"NUM_STI_STR_DEC01":-0.1476786,"NUM_SLI_DEBUG":"J476786","NUM_STI_DEBUG":"147678O","FLOAT_01":-14767.86,"DOUBLE_01":-1.4767869778395E9,"COMMON_8_BIN":14767869,"COMMON_S3_BIN":147,"COMMON_S94COMP":1476,"COMMON_S8_BIN":14767869,"COMMON_DDC97_BIN":1.4767869,"COMMON_97COMP3":1476786,"COMMON_915COMP3":147678697783955,"COMMON_S95COMP3":-14767,"COMMON_S999DCCOMP3":-147678697.78,"COMMON_S913COMP3":-1476786977839,"COMMON_S913DCCOMP3":-1476786977839.55,"COMMON_S911DCC2":-14767869778.39,"COMMON_S910DCC3":-1476786977.839,"COMMON_S03DDC":-0.14767,"COMMON_U03DDC":0.14767,"COMMON_UPC5DDC":0.00014767,"COMMON_SPC5DDC":-0.0014767,"COMMON_UPI5DDC":14767000,"COMMON_SPI5DDC":-14767000,"COMMON_UPC5DISP":-0.00014767,"COMMON_UPI5DISP":-14767000,"COMMON_UPC1BIN":0.0001,"COMMON_UPI1BIN":1000,"COMMON_UPC3BIN":0.000147,"COMMON_UPI3BIN":147000,"COMMON_UPC5BIN":0.00014767,"COMMON_UPI5BIN":14767000,"COMMON_UPC10BIN":0.0001476786977,"COMMON_UPI10BIN":1476786977000,"EX_NUM_INT01":-14767869,"EX_NUM_INT02":-14767869,"EX_NUM_INT03":-14767869,"EX_NUM_INT04":-14767869,"EX_NUM_DEC01":-147678.69,"EX_NUM_DEC02":-147678.69,"EX_NUM_DEC03":-147678.69} +{"ID":8,"STRING_VAL":"Mabelle","NUM_STR_INT01":7,"NUM_STR_INT02":72,"NUM_STR_INT03":720,"NUM_STR_INT04":7209,"NUM_STR_INT05":72091,"NUM_STR_INT06":72091694,"NUM_STR_INT07":720916941,"NUM_STR_INT08":7209169417,"NUM_STR_INT09":72091694175,"NUM_STR_INT10":72091694175254756,"NUM_STR_INT11":720916941752547567,"NUM_STR_INT12":7209169417525475672,"NUM_STR_INT13":72091694175254756729,"NUM_STR_INT14":7209169417525475672938784072548529111,"NUM_STR_SINT02":-72,"NUM_STR_SINT03":-720,"NUM_STR_SINT04":-7209,"NUM_STR_SINT05":-72091,"NUM_STR_SINT06":-72091694,"NUM_STR_SINT07":-720916941,"NUM_STR_SINT08":-7209169417,"NUM_STR_SINT09":-72091694175,"NUM_STR_SINT10":-72091694175254756,"NUM_STR_SINT11":-720916941752547567,"NUM_STR_SINT12":-7209169417525475672,"NUM_STR_SINT13":-72091694175254756729,"NUM_STR_SINT14":-7209169417525475672938784072548529111,"NUM_STR_DEC01":72.0,"NUM_STR_DEC02":72.09,"NUM_STR_DEC03":720.91,"NUM_STR_DEC04":7209.1694,"NUM_STR_DEC05":72091.6941,"NUM_STR_DEC06":72091.69417,"NUM_STR_DEC07":720916941752547.56,"NUM_STR_DEC08":7209169417525475.67,"NUM_STR_DEC09":72091694175254756.72,"NUM_STR_DEC10":720916941752547567.2938784072,"NUM_STR_SDEC01":-72.0,"NUM_STR_SDEC02":-72.09,"NUM_STR_SDEC03":-720.91,"NUM_STR_SDEC04":-7209.1694,"NUM_STR_SDEC05":-72091.6941,"NUM_STR_SDEC06":-72091.69417,"NUM_STR_SDEC07":-720916941752547.56,"NUM_STR_SDEC08":-7209169417525475.67,"NUM_STR_SDEC09":-72091694175254756.72,"NUM_STR_SDEC10":-720916941752547567.2938784072,"NUM_STR_EDEC03":-720.91,"NUM_STR_EDEC04":-7209.1694,"NUM_STR_EDEC05":-72091.6941,"NUM_STR_EDEC06":-72091.69417,"NUM_BIN_INT01":7,"NUM_BIN_INT02":72,"NUM_BIN_INT03":720,"NUM_BIN_INT04":7209,"NUM_BIN_INT05":72091,"NUM_BIN_INT06":72091694,"NUM_BIN_INT07":720916941,"NUM_BIN_INT08":7209169417,"NUM_BIN_INT09":72091694175,"NUM_BIN_INT10":72091694175254756,"NUM_BIN_INT11":720916941752547567,"NUM_BIN_INT12":7209169417525475672,"NUM_BIN_INT13":72091694175254756729,"NUM_BIN_INT14":7209169417525475672938784072548529111,"NUM_SBIN_SINT01":-7,"NUM_SBIN_SINT02":-72,"NUM_SBIN_SINT03":-720,"NUM_SBIN_SINT04":-7209,"NUM_SBIN_SINT05":-72091,"NUM_SBIN_SINT06":-72091694,"NUM_SBIN_SINT07":-720916941,"NUM_SBIN_SINT08":-7209169417,"NUM_SBIN_SINT09":-72091694175,"NUM_SBIN_SINT10":-72091694175254756,"NUM_SBIN_SINT11":-720916941752547567,"NUM_SBIN_SINT12":-7209169417525475672,"NUM_SBIN_SINT13":-72091694175254756729,"NUM_SBIN_SINT14":-7209169417525475672938784072548529111,"NUM_BIN_DEC01":72.0,"NUM_BIN_DEC02":72.09,"NUM_BIN_DEC03":720.91,"NUM_BIN_DEC04":7209.1694,"NUM_BIN_DEC05":72091.6941,"NUM_BIN_DEC06":72091.69417,"NUM_BIN_DEC07":720916941752547.56,"NUM_BIN_DEC08":7209169417525475.67,"NUM_BIN_DEC09":72091694175254756.72,"NUM_BIN_DEC10":720916941752547567.2938784072,"NUM_SBIN_DEC01":-72.0,"NUM_SBIN_DEC02":-72.09,"NUM_SBIN_DEC03":-720.91,"NUM_SBIN_DEC04":-7209.1694,"NUM_SBIN_DEC05":-72091.6941,"NUM_SBIN_DEC06":-72091.69417,"NUM_SBIN_DEC07":-720916941752547.56,"NUM_SBIN_DEC08":-7209169417525475.67,"NUM_SBIN_DEC09":-72091694175254756.72,"NUM_SBIN_DEC10":-720916941752547567.2938784072,"NUM_BCD_INT01":7,"NUM_BCD_INT02":72,"NUM_BCD_INT03":720,"NUM_BCD_INT04":7209,"NUM_BCD_INT05":72091,"NUM_BCD_INT06":72091694,"NUM_BCD_INT07":720916941,"NUM_BCD_INT08":7209169417,"NUM_BCD_INT09":72091694175,"NUM_BCD_INT10":72091694175254756,"NUM_BCD_INT11":720916941752547567,"NUM_BCD_INT12":7209169417525475672,"NUM_BCD_INT13":72091694175254756729,"NUM_BCD_INT14":7209169417525475672938784072548529111,"NUM_BCD_SINT01":-7,"NUM_BCD_SINT02":-72,"NUM_BCD_SINT03":-720,"NUM_BCD_SINT04":-7209,"NUM_BCD_SINT05":-72091,"NUM_BCD_SINT06":-72091694,"NUM_BCD_SINT07":-720916941,"NUM_BCD_SINT08":-7209169417,"NUM_BCD_SINT09":-72091694175,"NUM_BCD_SINT10":-72091694175254756,"NUM_BCD_SINT11":-720916941752547567,"NUM_BCD_SINT12":-7209169417525475672,"NUM_BCD_SINT13":-72091694175254756729,"NUM_BCD_SINT14":-7209169417525475672938784072548529111,"NUM_BCD_DEC01":72.0,"NUM_BCD_DEC02":72.09,"NUM_BCD_DEC03":720.91,"NUM_BCD_DEC04":7209.1694,"NUM_BCD_DEC05":72091.6941,"NUM_BCD_DEC06":72091.69417,"NUM_BCD_DEC07":720916941752547.56,"NUM_BCD_DEC08":7209169417525475.67,"NUM_BCD_DEC09":72091694175254756.72,"NUM_BCD_DEC10":720916941752547567.2938784072,"NUM_BCD_SDEC01":-72.0,"NUM_BCD_SDEC02":-72.09,"NUM_BCD_SDEC03":-720.91,"NUM_BCD_SDEC04":-7209.1694,"NUM_BCD_SDEC05":-72091.6941,"NUM_BCD_SDEC06":-72091.69417,"NUM_BCD_SDEC07":-720916941752547.56,"NUM_BCD_SDEC08":-7209169417525475.67,"NUM_BCD_SDEC09":-72091694175254756.72,"NUM_BCD_SDEC10":-720916941752547567.2938784072,"NUM_SL_STR_INT01":-720916941,"NUM_SL_STR_DEC01":-72.09,"NUM_ST_STR_INT01":-720916941,"NUM_ST_STR_DEC01":-72.09,"NUM_SLI_STR_DEC01":-0.7209169,"NUM_STI_STR_DEC01":-0.7209169,"NUM_SLI_DEBUG":"P209169","NUM_STI_DEBUG":"720916R","FLOAT_01":-72091.69,"DOUBLE_01":-7.2091694175254E9,"COMMON_8_BIN":72091694,"COMMON_S3_BIN":720,"COMMON_S94COMP":7209,"COMMON_S8_BIN":72091694,"COMMON_DDC97_BIN":7.2091694,"COMMON_97COMP3":7209169,"COMMON_915COMP3":720916941752547,"COMMON_S95COMP3":-72091,"COMMON_S999DCCOMP3":-720916941.75,"COMMON_S913COMP3":-7209169417525,"COMMON_S913DCCOMP3":-7209169417525.47,"COMMON_S911DCC2":-72091694175.25,"COMMON_S910DCC3":-7209169417.525,"COMMON_S03DDC":-0.72091,"COMMON_U03DDC":0.72091,"COMMON_UPC5DDC":0.00072091,"COMMON_SPC5DDC":-0.0072091,"COMMON_UPI5DDC":72091000,"COMMON_SPI5DDC":-72091000,"COMMON_UPC5DISP":-0.00072091,"COMMON_UPI5DISP":-72091000,"COMMON_UPC1BIN":0.0007,"COMMON_UPI1BIN":7000,"COMMON_UPC3BIN":0.000720,"COMMON_UPI3BIN":720000,"COMMON_UPC5BIN":0.00072091,"COMMON_UPI5BIN":72091000,"COMMON_UPC10BIN":0.0007209169417,"COMMON_UPI10BIN":7209169417000,"EX_NUM_INT01":-72091694,"EX_NUM_INT02":-72091694,"EX_NUM_INT03":-72091694,"EX_NUM_INT04":-72091694,"EX_NUM_DEC01":-720916.94,"EX_NUM_DEC02":-720916.94,"EX_NUM_DEC03":-720916.94} +{"ID":9,"STRING_VAL":"Deshawn","NUM_STR_INT01":3,"NUM_STR_INT02":34,"NUM_STR_INT03":348,"NUM_STR_INT04":3486,"NUM_STR_INT05":34865,"NUM_STR_INT06":34865783,"NUM_STR_INT07":348657838,"NUM_STR_INT08":3486578382,"NUM_STR_INT09":34865783825,"NUM_STR_INT10":34865783825735136,"NUM_STR_INT11":348657838257351360,"NUM_STR_INT12":3486578382573513600,"NUM_STR_INT13":34865783825735136005,"NUM_STR_INT14":3486578382573513600580443869430340852,"NUM_STR_SINT02":-34,"NUM_STR_SINT03":-348,"NUM_STR_SINT04":-3486,"NUM_STR_SINT05":-34865,"NUM_STR_SINT06":-34865783,"NUM_STR_SINT07":-348657838,"NUM_STR_SINT08":-3486578382,"NUM_STR_SINT09":-34865783825,"NUM_STR_SINT10":-34865783825735136,"NUM_STR_SINT11":-348657838257351360,"NUM_STR_SINT12":-3486578382573513600,"NUM_STR_SINT13":-34865783825735136005,"NUM_STR_SINT14":-3486578382573513600580443869430340852,"NUM_STR_DEC01":34.8,"NUM_STR_DEC02":34.86,"NUM_STR_DEC03":348.65,"NUM_STR_DEC04":3486.5783,"NUM_STR_DEC05":34865.7838,"NUM_STR_DEC06":34865.78382,"NUM_STR_DEC07":348657838257351.36,"NUM_STR_DEC08":3486578382573513.60,"NUM_STR_DEC09":34865783825735136.00,"NUM_STR_DEC10":348657838257351360.0580443869,"NUM_STR_SDEC01":-34.8,"NUM_STR_SDEC02":-34.86,"NUM_STR_SDEC03":-348.65,"NUM_STR_SDEC04":-3486.5783,"NUM_STR_SDEC05":-34865.7838,"NUM_STR_SDEC06":-34865.78382,"NUM_STR_SDEC07":-348657838257351.36,"NUM_STR_SDEC08":-3486578382573513.60,"NUM_STR_SDEC09":-34865783825735136.00,"NUM_STR_SDEC10":-348657838257351360.0580443869,"NUM_STR_EDEC03":-348.65,"NUM_STR_EDEC04":-3486.5783,"NUM_STR_EDEC05":-34865.7838,"NUM_STR_EDEC06":-34865.78382,"NUM_BIN_INT01":3,"NUM_BIN_INT02":34,"NUM_BIN_INT03":348,"NUM_BIN_INT04":3486,"NUM_BIN_INT05":34865,"NUM_BIN_INT06":34865783,"NUM_BIN_INT07":348657838,"NUM_BIN_INT08":3486578382,"NUM_BIN_INT09":34865783825,"NUM_BIN_INT10":34865783825735136,"NUM_BIN_INT11":348657838257351360,"NUM_BIN_INT12":3486578382573513600,"NUM_BIN_INT13":34865783825735136005,"NUM_BIN_INT14":3486578382573513600580443869430340852,"NUM_SBIN_SINT01":-3,"NUM_SBIN_SINT02":-34,"NUM_SBIN_SINT03":-348,"NUM_SBIN_SINT04":-3486,"NUM_SBIN_SINT05":-34865,"NUM_SBIN_SINT06":-34865783,"NUM_SBIN_SINT07":-348657838,"NUM_SBIN_SINT08":-3486578382,"NUM_SBIN_SINT09":-34865783825,"NUM_SBIN_SINT10":-34865783825735136,"NUM_SBIN_SINT11":-348657838257351360,"NUM_SBIN_SINT12":-3486578382573513600,"NUM_SBIN_SINT13":-34865783825735136005,"NUM_SBIN_SINT14":-3486578382573513600580443869430340852,"NUM_BIN_DEC01":34.8,"NUM_BIN_DEC02":34.86,"NUM_BIN_DEC03":348.65,"NUM_BIN_DEC04":3486.5783,"NUM_BIN_DEC05":34865.7838,"NUM_BIN_DEC06":34865.78382,"NUM_BIN_DEC07":348657838257351.36,"NUM_BIN_DEC08":3486578382573513.60,"NUM_BIN_DEC09":34865783825735136.00,"NUM_BIN_DEC10":348657838257351360.0580443869,"NUM_SBIN_DEC01":-34.8,"NUM_SBIN_DEC02":-34.86,"NUM_SBIN_DEC03":-348.65,"NUM_SBIN_DEC04":-3486.5783,"NUM_SBIN_DEC05":-34865.7838,"NUM_SBIN_DEC06":-34865.78382,"NUM_SBIN_DEC07":-348657838257351.36,"NUM_SBIN_DEC08":-3486578382573513.60,"NUM_SBIN_DEC09":-34865783825735136.00,"NUM_SBIN_DEC10":-348657838257351360.0580443869,"NUM_BCD_INT01":3,"NUM_BCD_INT02":34,"NUM_BCD_INT03":348,"NUM_BCD_INT04":3486,"NUM_BCD_INT05":34865,"NUM_BCD_INT06":34865783,"NUM_BCD_INT07":348657838,"NUM_BCD_INT08":3486578382,"NUM_BCD_INT09":34865783825,"NUM_BCD_INT10":34865783825735136,"NUM_BCD_INT11":348657838257351360,"NUM_BCD_INT12":3486578382573513600,"NUM_BCD_INT13":34865783825735136005,"NUM_BCD_INT14":3486578382573513600580443869430340852,"NUM_BCD_SINT01":-3,"NUM_BCD_SINT02":-34,"NUM_BCD_SINT03":-348,"NUM_BCD_SINT04":-3486,"NUM_BCD_SINT05":-34865,"NUM_BCD_SINT06":-34865783,"NUM_BCD_SINT07":-348657838,"NUM_BCD_SINT08":-3486578382,"NUM_BCD_SINT09":-34865783825,"NUM_BCD_SINT10":-34865783825735136,"NUM_BCD_SINT11":-348657838257351360,"NUM_BCD_SINT12":-3486578382573513600,"NUM_BCD_SINT13":-34865783825735136005,"NUM_BCD_SINT14":-3486578382573513600580443869430340852,"NUM_BCD_DEC01":34.8,"NUM_BCD_DEC02":34.86,"NUM_BCD_DEC03":348.65,"NUM_BCD_DEC04":3486.5783,"NUM_BCD_DEC05":34865.7838,"NUM_BCD_DEC06":34865.78382,"NUM_BCD_DEC07":348657838257351.36,"NUM_BCD_DEC08":3486578382573513.60,"NUM_BCD_DEC09":34865783825735136.00,"NUM_BCD_DEC10":348657838257351360.0580443869,"NUM_BCD_SDEC01":-34.8,"NUM_BCD_SDEC02":-34.86,"NUM_BCD_SDEC03":-348.65,"NUM_BCD_SDEC04":-3486.5783,"NUM_BCD_SDEC05":-34865.7838,"NUM_BCD_SDEC06":-34865.78382,"NUM_BCD_SDEC07":-348657838257351.36,"NUM_BCD_SDEC08":-3486578382573513.60,"NUM_BCD_SDEC09":-34865783825735136.00,"NUM_BCD_SDEC10":-348657838257351360.0580443869,"NUM_SL_STR_INT01":-348657838,"NUM_SL_STR_DEC01":-34.86,"NUM_ST_STR_INT01":-348657838,"NUM_ST_STR_DEC01":-34.86,"NUM_SLI_STR_DEC01":-0.3486578,"NUM_STI_STR_DEC01":-0.3486578,"NUM_SLI_DEBUG":"L486578","NUM_STI_DEBUG":"348657Q","FLOAT_01":-34865.78,"DOUBLE_01":-3.4865783825735E9,"COMMON_8_BIN":34865783,"COMMON_S3_BIN":348,"COMMON_S94COMP":3486,"COMMON_S8_BIN":34865783,"COMMON_DDC97_BIN":3.4865783,"COMMON_97COMP3":3486578,"COMMON_915COMP3":348657838257351,"COMMON_S95COMP3":-34865,"COMMON_S999DCCOMP3":-348657838.25,"COMMON_S913COMP3":-3486578382573,"COMMON_S913DCCOMP3":-3486578382573.51,"COMMON_S911DCC2":-34865783825.73,"COMMON_S910DCC3":-3486578382.573,"COMMON_S03DDC":-0.34865,"COMMON_U03DDC":0.34865,"COMMON_UPC5DDC":0.00034865,"COMMON_SPC5DDC":-0.0034865,"COMMON_UPI5DDC":34865000,"COMMON_SPI5DDC":-34865000,"COMMON_UPC5DISP":-0.00034865,"COMMON_UPI5DISP":-34865000,"COMMON_UPC1BIN":0.0003,"COMMON_UPI1BIN":3000,"COMMON_UPC3BIN":0.000348,"COMMON_UPI3BIN":348000,"COMMON_UPC5BIN":0.00034865,"COMMON_UPI5BIN":34865000,"COMMON_UPC10BIN":0.0003486578382,"COMMON_UPI10BIN":3486578382000,"EX_NUM_INT01":-34865783,"EX_NUM_INT02":-34865783,"EX_NUM_INT03":-34865783,"EX_NUM_INT04":-34865783,"EX_NUM_DEC01":-348657.83,"EX_NUM_DEC02":-348657.83,"EX_NUM_DEC03":-348657.83} +{"ID":10,"STRING_VAL":"Jene","NUM_STR_INT01":3,"NUM_STR_INT02":31,"NUM_STR_INT03":316,"NUM_STR_INT04":3162,"NUM_STR_INT05":31623,"NUM_STR_INT06":31623742,"NUM_STR_INT07":316237426,"NUM_STR_INT08":3162374261,"NUM_STR_INT09":31623742617,"NUM_STR_INT10":31623742617648777,"NUM_STR_INT11":316237426176487772,"NUM_STR_INT12":3162374261764877724,"NUM_STR_INT13":31623742617648777240,"NUM_STR_INT14":3162374261764877724038963306011969919,"NUM_STR_SINT02":-31,"NUM_STR_SINT03":-316,"NUM_STR_SINT04":-3162,"NUM_STR_SINT05":-31623,"NUM_STR_SINT06":-31623742,"NUM_STR_SINT07":-316237426,"NUM_STR_SINT08":-3162374261,"NUM_STR_SINT09":-31623742617,"NUM_STR_SINT10":-31623742617648777,"NUM_STR_SINT11":-316237426176487772,"NUM_STR_SINT12":-3162374261764877724,"NUM_STR_SINT13":-31623742617648777240,"NUM_STR_SINT14":-3162374261764877724038963306011969919,"NUM_STR_DEC01":31.6,"NUM_STR_DEC02":31.62,"NUM_STR_DEC03":316.23,"NUM_STR_DEC04":3162.3742,"NUM_STR_DEC05":31623.7426,"NUM_STR_DEC06":31623.74261,"NUM_STR_DEC07":316237426176487.77,"NUM_STR_DEC08":3162374261764877.72,"NUM_STR_DEC09":31623742617648777.24,"NUM_STR_DEC10":316237426176487772.4038963306,"NUM_STR_SDEC01":-31.6,"NUM_STR_SDEC02":-31.62,"NUM_STR_SDEC03":-316.23,"NUM_STR_SDEC04":-3162.3742,"NUM_STR_SDEC05":-31623.7426,"NUM_STR_SDEC06":-31623.74261,"NUM_STR_SDEC07":-316237426176487.77,"NUM_STR_SDEC08":-3162374261764877.72,"NUM_STR_SDEC09":-31623742617648777.24,"NUM_STR_SDEC10":-316237426176487772.4038963306,"NUM_STR_EDEC03":-316.23,"NUM_STR_EDEC04":-3162.3742,"NUM_STR_EDEC05":-31623.7426,"NUM_STR_EDEC06":-31623.74261,"NUM_BIN_INT01":3,"NUM_BIN_INT02":31,"NUM_BIN_INT03":316,"NUM_BIN_INT04":3162,"NUM_BIN_INT05":31623,"NUM_BIN_INT06":31623742,"NUM_BIN_INT07":316237426,"NUM_BIN_INT08":3162374261,"NUM_BIN_INT09":31623742617,"NUM_BIN_INT10":31623742617648777,"NUM_BIN_INT11":316237426176487772,"NUM_BIN_INT12":3162374261764877724,"NUM_BIN_INT13":31623742617648777240,"NUM_BIN_INT14":3162374261764877724038963306011969919,"NUM_SBIN_SINT01":-3,"NUM_SBIN_SINT02":-31,"NUM_SBIN_SINT03":-316,"NUM_SBIN_SINT04":-3162,"NUM_SBIN_SINT05":-31623,"NUM_SBIN_SINT06":-31623742,"NUM_SBIN_SINT07":-316237426,"NUM_SBIN_SINT08":-3162374261,"NUM_SBIN_SINT09":-31623742617,"NUM_SBIN_SINT10":-31623742617648777,"NUM_SBIN_SINT11":-316237426176487772,"NUM_SBIN_SINT12":-3162374261764877724,"NUM_SBIN_SINT13":-31623742617648777240,"NUM_SBIN_SINT14":-3162374261764877724038963306011969919,"NUM_BIN_DEC01":31.6,"NUM_BIN_DEC02":31.62,"NUM_BIN_DEC03":316.23,"NUM_BIN_DEC04":3162.3742,"NUM_BIN_DEC05":31623.7426,"NUM_BIN_DEC06":31623.74261,"NUM_BIN_DEC07":316237426176487.77,"NUM_BIN_DEC08":3162374261764877.72,"NUM_BIN_DEC09":31623742617648777.24,"NUM_BIN_DEC10":316237426176487772.4038963306,"NUM_SBIN_DEC01":-31.6,"NUM_SBIN_DEC02":-31.62,"NUM_SBIN_DEC03":-316.23,"NUM_SBIN_DEC04":-3162.3742,"NUM_SBIN_DEC05":-31623.7426,"NUM_SBIN_DEC06":-31623.74261,"NUM_SBIN_DEC07":-316237426176487.77,"NUM_SBIN_DEC08":-3162374261764877.72,"NUM_SBIN_DEC09":-31623742617648777.24,"NUM_SBIN_DEC10":-316237426176487772.4038963306,"NUM_BCD_INT01":3,"NUM_BCD_INT02":31,"NUM_BCD_INT03":316,"NUM_BCD_INT04":3162,"NUM_BCD_INT05":31623,"NUM_BCD_INT06":31623742,"NUM_BCD_INT07":316237426,"NUM_BCD_INT08":3162374261,"NUM_BCD_INT09":31623742617,"NUM_BCD_INT10":31623742617648777,"NUM_BCD_INT11":316237426176487772,"NUM_BCD_INT12":3162374261764877724,"NUM_BCD_INT13":31623742617648777240,"NUM_BCD_INT14":3162374261764877724038963306011969919,"NUM_BCD_SINT01":-3,"NUM_BCD_SINT02":-31,"NUM_BCD_SINT03":-316,"NUM_BCD_SINT04":-3162,"NUM_BCD_SINT05":-31623,"NUM_BCD_SINT06":-31623742,"NUM_BCD_SINT07":-316237426,"NUM_BCD_SINT08":-3162374261,"NUM_BCD_SINT09":-31623742617,"NUM_BCD_SINT10":-31623742617648777,"NUM_BCD_SINT11":-316237426176487772,"NUM_BCD_SINT12":-3162374261764877724,"NUM_BCD_SINT13":-31623742617648777240,"NUM_BCD_SINT14":-3162374261764877724038963306011969919,"NUM_BCD_DEC01":31.6,"NUM_BCD_DEC02":31.62,"NUM_BCD_DEC03":316.23,"NUM_BCD_DEC04":3162.3742,"NUM_BCD_DEC05":31623.7426,"NUM_BCD_DEC06":31623.74261,"NUM_BCD_DEC07":316237426176487.77,"NUM_BCD_DEC08":3162374261764877.72,"NUM_BCD_DEC09":31623742617648777.24,"NUM_BCD_DEC10":316237426176487772.4038963306,"NUM_BCD_SDEC01":-31.6,"NUM_BCD_SDEC02":-31.62,"NUM_BCD_SDEC03":-316.23,"NUM_BCD_SDEC04":-3162.3742,"NUM_BCD_SDEC05":-31623.7426,"NUM_BCD_SDEC06":-31623.74261,"NUM_BCD_SDEC07":-316237426176487.77,"NUM_BCD_SDEC08":-3162374261764877.72,"NUM_BCD_SDEC09":-31623742617648777.24,"NUM_BCD_SDEC10":-316237426176487772.4038963306,"NUM_SL_STR_INT01":-316237426,"NUM_SL_STR_DEC01":-31.62,"NUM_ST_STR_INT01":-316237426,"NUM_ST_STR_DEC01":-31.62,"NUM_SLI_STR_DEC01":-0.3162374,"NUM_STI_STR_DEC01":-0.3162374,"NUM_SLI_DEBUG":"L162374","NUM_STI_DEBUG":"316237M","FLOAT_01":-31623.74,"DOUBLE_01":-3.1623742617648E9,"COMMON_8_BIN":31623742,"COMMON_S3_BIN":316,"COMMON_S94COMP":3162,"COMMON_S8_BIN":31623742,"COMMON_DDC97_BIN":3.1623742,"COMMON_97COMP3":3162374,"COMMON_915COMP3":316237426176487,"COMMON_S95COMP3":-31623,"COMMON_S999DCCOMP3":-316237426.17,"COMMON_S913COMP3":-3162374261764,"COMMON_S913DCCOMP3":-3162374261764.87,"COMMON_S911DCC2":-31623742617.64,"COMMON_S910DCC3":-3162374261.764,"COMMON_S03DDC":-0.31623,"COMMON_U03DDC":0.31623,"COMMON_UPC5DDC":0.00031623,"COMMON_SPC5DDC":-0.0031623,"COMMON_UPI5DDC":31623000,"COMMON_SPI5DDC":-31623000,"COMMON_UPC5DISP":-0.00031623,"COMMON_UPI5DISP":-31623000,"COMMON_UPC1BIN":0.0003,"COMMON_UPI1BIN":3000,"COMMON_UPC3BIN":0.000316,"COMMON_UPI3BIN":316000,"COMMON_UPC5BIN":0.00031623,"COMMON_UPI5BIN":31623000,"COMMON_UPC10BIN":0.0003162374261,"COMMON_UPI10BIN":3162374261000,"EX_NUM_INT01":-31623742,"EX_NUM_INT02":-31623742,"EX_NUM_INT03":-31623742,"EX_NUM_INT04":-31623742,"EX_NUM_DEC01":-316237.42,"EX_NUM_DEC02":-316237.42,"EX_NUM_DEC03":-316237.42} +{"ID":11,"STRING_VAL":"Maya","NUM_STR_INT01":9,"NUM_STR_INT02":99,"NUM_STR_INT03":993,"NUM_STR_INT04":9938,"NUM_STR_INT05":99382,"NUM_STR_INT06":99382555,"NUM_STR_INT07":993825559,"NUM_STR_INT08":9938255596,"NUM_STR_INT09":99382555961,"NUM_STR_INT10":99382555961361957,"NUM_STR_INT11":993825559613619575,"NUM_STR_INT12":9938255596136195754,"NUM_STR_INT13":99382555961361957549,"NUM_STR_INT14":9938255596136195754905557525606044181,"NUM_STR_SINT02":-99,"NUM_STR_SINT03":-993,"NUM_STR_SINT04":-9938,"NUM_STR_SINT05":-99382,"NUM_STR_SINT06":-99382555,"NUM_STR_SINT07":-993825559,"NUM_STR_SINT08":-9938255596,"NUM_STR_SINT09":-99382555961,"NUM_STR_SINT10":-99382555961361957,"NUM_STR_SINT11":-993825559613619575,"NUM_STR_SINT12":-9938255596136195754,"NUM_STR_SINT13":-99382555961361957549,"NUM_STR_SINT14":-9938255596136195754905557525606044181,"NUM_STR_DEC01":99.3,"NUM_STR_DEC02":99.38,"NUM_STR_DEC03":993.82,"NUM_STR_DEC04":9938.2555,"NUM_STR_DEC05":99382.5559,"NUM_STR_DEC06":99382.55596,"NUM_STR_DEC07":993825559613619.57,"NUM_STR_DEC08":9938255596136195.75,"NUM_STR_DEC09":99382555961361957.54,"NUM_STR_DEC10":993825559613619575.4905557525,"NUM_STR_SDEC01":-99.3,"NUM_STR_SDEC02":-99.38,"NUM_STR_SDEC03":-993.82,"NUM_STR_SDEC04":-9938.2555,"NUM_STR_SDEC05":-99382.5559,"NUM_STR_SDEC06":-99382.55596,"NUM_STR_SDEC07":-993825559613619.57,"NUM_STR_SDEC08":-9938255596136195.75,"NUM_STR_SDEC09":-99382555961361957.54,"NUM_STR_SDEC10":-993825559613619575.4905557525,"NUM_STR_EDEC03":-993.82,"NUM_STR_EDEC04":-9938.2555,"NUM_STR_EDEC05":-99382.5559,"NUM_STR_EDEC06":-99382.55596,"NUM_BIN_INT01":9,"NUM_BIN_INT02":99,"NUM_BIN_INT03":993,"NUM_BIN_INT04":9938,"NUM_BIN_INT05":99382,"NUM_BIN_INT06":99382555,"NUM_BIN_INT07":993825559,"NUM_BIN_INT08":9938255596,"NUM_BIN_INT09":99382555961,"NUM_BIN_INT10":99382555961361957,"NUM_BIN_INT11":993825559613619575,"NUM_BIN_INT12":9938255596136195754,"NUM_BIN_INT13":99382555961361957549,"NUM_BIN_INT14":9938255596136195754905557525606044181,"NUM_SBIN_SINT01":-9,"NUM_SBIN_SINT02":-99,"NUM_SBIN_SINT03":-993,"NUM_SBIN_SINT04":-9938,"NUM_SBIN_SINT05":-99382,"NUM_SBIN_SINT06":-99382555,"NUM_SBIN_SINT07":-993825559,"NUM_SBIN_SINT08":-9938255596,"NUM_SBIN_SINT09":-99382555961,"NUM_SBIN_SINT10":-99382555961361957,"NUM_SBIN_SINT11":-993825559613619575,"NUM_SBIN_SINT12":-9938255596136195754,"NUM_SBIN_SINT13":-99382555961361957549,"NUM_SBIN_SINT14":-9938255596136195754905557525606044181,"NUM_BIN_DEC01":99.3,"NUM_BIN_DEC02":99.38,"NUM_BIN_DEC03":993.82,"NUM_BIN_DEC04":9938.2555,"NUM_BIN_DEC05":99382.5559,"NUM_BIN_DEC06":99382.55596,"NUM_BIN_DEC07":993825559613619.57,"NUM_BIN_DEC08":9938255596136195.75,"NUM_BIN_DEC09":99382555961361957.54,"NUM_BIN_DEC10":993825559613619575.4905557525,"NUM_SBIN_DEC01":-99.3,"NUM_SBIN_DEC02":-99.38,"NUM_SBIN_DEC03":-993.82,"NUM_SBIN_DEC04":-9938.2555,"NUM_SBIN_DEC05":-99382.5559,"NUM_SBIN_DEC06":-99382.55596,"NUM_SBIN_DEC07":-993825559613619.57,"NUM_SBIN_DEC08":-9938255596136195.75,"NUM_SBIN_DEC09":-99382555961361957.54,"NUM_SBIN_DEC10":-993825559613619575.4905557525,"NUM_BCD_INT01":9,"NUM_BCD_INT02":99,"NUM_BCD_INT03":993,"NUM_BCD_INT04":9938,"NUM_BCD_INT05":99382,"NUM_BCD_INT06":99382555,"NUM_BCD_INT07":993825559,"NUM_BCD_INT08":9938255596,"NUM_BCD_INT09":99382555961,"NUM_BCD_INT10":99382555961361957,"NUM_BCD_INT11":993825559613619575,"NUM_BCD_INT12":9938255596136195754,"NUM_BCD_INT13":99382555961361957549,"NUM_BCD_INT14":9938255596136195754905557525606044181,"NUM_BCD_SINT01":-9,"NUM_BCD_SINT02":-99,"NUM_BCD_SINT03":-993,"NUM_BCD_SINT04":-9938,"NUM_BCD_SINT05":-99382,"NUM_BCD_SINT06":-99382555,"NUM_BCD_SINT07":-993825559,"NUM_BCD_SINT08":-9938255596,"NUM_BCD_SINT09":-99382555961,"NUM_BCD_SINT10":-99382555961361957,"NUM_BCD_SINT11":-993825559613619575,"NUM_BCD_SINT12":-9938255596136195754,"NUM_BCD_SINT13":-99382555961361957549,"NUM_BCD_SINT14":-9938255596136195754905557525606044181,"NUM_BCD_DEC01":99.3,"NUM_BCD_DEC02":99.38,"NUM_BCD_DEC03":993.82,"NUM_BCD_DEC04":9938.2555,"NUM_BCD_DEC05":99382.5559,"NUM_BCD_DEC06":99382.55596,"NUM_BCD_DEC07":993825559613619.57,"NUM_BCD_DEC08":9938255596136195.75,"NUM_BCD_DEC09":99382555961361957.54,"NUM_BCD_DEC10":993825559613619575.4905557525,"NUM_BCD_SDEC01":-99.3,"NUM_BCD_SDEC02":-99.38,"NUM_BCD_SDEC03":-993.82,"NUM_BCD_SDEC04":-9938.2555,"NUM_BCD_SDEC05":-99382.5559,"NUM_BCD_SDEC06":-99382.55596,"NUM_BCD_SDEC07":-993825559613619.57,"NUM_BCD_SDEC08":-9938255596136195.75,"NUM_BCD_SDEC09":-99382555961361957.54,"NUM_BCD_SDEC10":-993825559613619575.4905557525,"NUM_SL_STR_INT01":-993825559,"NUM_SL_STR_DEC01":-99.38,"NUM_ST_STR_INT01":-993825559,"NUM_ST_STR_DEC01":-99.38,"NUM_SLI_STR_DEC01":-0.9938255,"NUM_STI_STR_DEC01":-0.9938255,"NUM_SLI_DEBUG":"R938255","NUM_STI_DEBUG":"993825N","FLOAT_01":-99382.55,"DOUBLE_01":-9.9382555961361E9,"COMMON_8_BIN":99382555,"COMMON_S3_BIN":993,"COMMON_S94COMP":9938,"COMMON_S8_BIN":99382555,"COMMON_DDC97_BIN":9.9382555,"COMMON_97COMP3":9938255,"COMMON_915COMP3":993825559613619,"COMMON_S95COMP3":-99382,"COMMON_S999DCCOMP3":-993825559.61,"COMMON_S913COMP3":-9938255596136,"COMMON_S913DCCOMP3":-9938255596136.19,"COMMON_S911DCC2":-99382555961.36,"COMMON_S910DCC3":-9938255596.136,"COMMON_S03DDC":-0.99382,"COMMON_U03DDC":0.99382,"COMMON_UPC5DDC":0.00099382,"COMMON_SPC5DDC":-0.0099382,"COMMON_UPI5DDC":99382000,"COMMON_SPI5DDC":-99382000,"COMMON_UPC5DISP":-0.00099382,"COMMON_UPI5DISP":-99382000,"COMMON_UPC1BIN":0.0009,"COMMON_UPI1BIN":9000,"COMMON_UPC3BIN":0.000993,"COMMON_UPI3BIN":993000,"COMMON_UPC5BIN":0.00099382,"COMMON_UPI5BIN":99382000,"COMMON_UPC10BIN":0.0009938255596,"COMMON_UPI10BIN":9938255596000,"EX_NUM_INT01":-99382555,"EX_NUM_INT02":-99382555,"EX_NUM_INT03":-99382555,"EX_NUM_INT04":-99382555,"EX_NUM_DEC01":-993825.55,"EX_NUM_DEC02":-993825.55,"EX_NUM_DEC03":-993825.55} +{"ID":12,"STRING_VAL":"Eliana","NUM_STR_INT01":3,"NUM_STR_INT02":36,"NUM_STR_INT03":369,"NUM_STR_INT04":3693,"NUM_STR_INT05":36932,"NUM_STR_INT06":36932540,"NUM_STR_INT07":369325402,"NUM_STR_INT08":3693254020,"NUM_STR_INT09":36932540205,"NUM_STR_INT10":36932540205777765,"NUM_STR_INT11":369325402057777650,"NUM_STR_INT12":3693254020577776500,"NUM_STR_INT13":36932540205777765007,"NUM_STR_INT14":3693254020577776500728759400026278176,"NUM_STR_SINT02":-36,"NUM_STR_SINT03":-369,"NUM_STR_SINT04":-3693,"NUM_STR_SINT05":-36932,"NUM_STR_SINT06":-36932540,"NUM_STR_SINT07":-369325402,"NUM_STR_SINT08":-3693254020,"NUM_STR_SINT09":-36932540205,"NUM_STR_SINT10":-36932540205777765,"NUM_STR_SINT11":-369325402057777650,"NUM_STR_SINT12":-3693254020577776500,"NUM_STR_SINT13":-36932540205777765007,"NUM_STR_SINT14":-3693254020577776500728759400026278176,"NUM_STR_DEC01":36.9,"NUM_STR_DEC02":36.93,"NUM_STR_DEC03":369.32,"NUM_STR_DEC04":3693.2540,"NUM_STR_DEC05":36932.5402,"NUM_STR_DEC06":36932.54020,"NUM_STR_DEC07":369325402057777.65,"NUM_STR_DEC08":3693254020577776.50,"NUM_STR_DEC09":36932540205777765.00,"NUM_STR_DEC10":369325402057777650.0728759400,"NUM_STR_SDEC01":-36.9,"NUM_STR_SDEC02":-36.93,"NUM_STR_SDEC03":-369.32,"NUM_STR_SDEC04":-3693.2540,"NUM_STR_SDEC05":-36932.5402,"NUM_STR_SDEC06":-36932.54020,"NUM_STR_SDEC07":-369325402057777.65,"NUM_STR_SDEC08":-3693254020577776.50,"NUM_STR_SDEC09":-36932540205777765.00,"NUM_STR_SDEC10":-369325402057777650.0728759400,"NUM_STR_EDEC03":-369.32,"NUM_STR_EDEC04":-3693.2540,"NUM_STR_EDEC05":-36932.5402,"NUM_STR_EDEC06":-36932.54020,"NUM_BIN_INT01":3,"NUM_BIN_INT02":36,"NUM_BIN_INT03":369,"NUM_BIN_INT04":3693,"NUM_BIN_INT05":36932,"NUM_BIN_INT06":36932540,"NUM_BIN_INT07":369325402,"NUM_BIN_INT08":3693254020,"NUM_BIN_INT09":36932540205,"NUM_BIN_INT10":36932540205777765,"NUM_BIN_INT11":369325402057777650,"NUM_BIN_INT12":3693254020577776500,"NUM_BIN_INT13":36932540205777765007,"NUM_BIN_INT14":3693254020577776500728759400026278176,"NUM_SBIN_SINT01":-3,"NUM_SBIN_SINT02":-36,"NUM_SBIN_SINT03":-369,"NUM_SBIN_SINT04":-3693,"NUM_SBIN_SINT05":-36932,"NUM_SBIN_SINT06":-36932540,"NUM_SBIN_SINT07":-369325402,"NUM_SBIN_SINT08":-3693254020,"NUM_SBIN_SINT09":-36932540205,"NUM_SBIN_SINT10":-36932540205777765,"NUM_SBIN_SINT11":-369325402057777650,"NUM_SBIN_SINT12":-3693254020577776500,"NUM_SBIN_SINT13":-36932540205777765007,"NUM_SBIN_SINT14":-3693254020577776500728759400026278176,"NUM_BIN_DEC01":36.9,"NUM_BIN_DEC02":36.93,"NUM_BIN_DEC03":369.32,"NUM_BIN_DEC04":3693.2540,"NUM_BIN_DEC05":36932.5402,"NUM_BIN_DEC06":36932.54020,"NUM_BIN_DEC07":369325402057777.65,"NUM_BIN_DEC08":3693254020577776.50,"NUM_BIN_DEC09":36932540205777765.00,"NUM_BIN_DEC10":369325402057777650.0728759400,"NUM_SBIN_DEC01":-36.9,"NUM_SBIN_DEC02":-36.93,"NUM_SBIN_DEC03":-369.32,"NUM_SBIN_DEC04":-3693.2540,"NUM_SBIN_DEC05":-36932.5402,"NUM_SBIN_DEC06":-36932.54020,"NUM_SBIN_DEC07":-369325402057777.65,"NUM_SBIN_DEC08":-3693254020577776.50,"NUM_SBIN_DEC09":-36932540205777765.00,"NUM_SBIN_DEC10":-369325402057777650.0728759400,"NUM_BCD_INT01":3,"NUM_BCD_INT02":36,"NUM_BCD_INT03":369,"NUM_BCD_INT04":3693,"NUM_BCD_INT05":36932,"NUM_BCD_INT06":36932540,"NUM_BCD_INT07":369325402,"NUM_BCD_INT08":3693254020,"NUM_BCD_INT09":36932540205,"NUM_BCD_INT10":36932540205777765,"NUM_BCD_INT11":369325402057777650,"NUM_BCD_INT12":3693254020577776500,"NUM_BCD_INT13":36932540205777765007,"NUM_BCD_INT14":3693254020577776500728759400026278176,"NUM_BCD_SINT01":-3,"NUM_BCD_SINT02":-36,"NUM_BCD_SINT03":-369,"NUM_BCD_SINT04":-3693,"NUM_BCD_SINT05":-36932,"NUM_BCD_SINT06":-36932540,"NUM_BCD_SINT07":-369325402,"NUM_BCD_SINT08":-3693254020,"NUM_BCD_SINT09":-36932540205,"NUM_BCD_SINT10":-36932540205777765,"NUM_BCD_SINT11":-369325402057777650,"NUM_BCD_SINT12":-3693254020577776500,"NUM_BCD_SINT13":-36932540205777765007,"NUM_BCD_SINT14":-3693254020577776500728759400026278176,"NUM_BCD_DEC01":36.9,"NUM_BCD_DEC02":36.93,"NUM_BCD_DEC03":369.32,"NUM_BCD_DEC04":3693.2540,"NUM_BCD_DEC05":36932.5402,"NUM_BCD_DEC06":36932.54020,"NUM_BCD_DEC07":369325402057777.65,"NUM_BCD_DEC08":3693254020577776.50,"NUM_BCD_DEC09":36932540205777765.00,"NUM_BCD_DEC10":369325402057777650.0728759400,"NUM_BCD_SDEC01":-36.9,"NUM_BCD_SDEC02":-36.93,"NUM_BCD_SDEC03":-369.32,"NUM_BCD_SDEC04":-3693.2540,"NUM_BCD_SDEC05":-36932.5402,"NUM_BCD_SDEC06":-36932.54020,"NUM_BCD_SDEC07":-369325402057777.65,"NUM_BCD_SDEC08":-3693254020577776.50,"NUM_BCD_SDEC09":-36932540205777765.00,"NUM_BCD_SDEC10":-369325402057777650.0728759400,"NUM_SL_STR_INT01":-369325402,"NUM_SL_STR_DEC01":-36.93,"NUM_ST_STR_INT01":-369325402,"NUM_ST_STR_DEC01":-36.93,"NUM_SLI_STR_DEC01":-0.3693254,"NUM_STI_STR_DEC01":-0.3693254,"NUM_SLI_DEBUG":"L693254","NUM_STI_DEBUG":"369325M","FLOAT_01":-36932.54,"DOUBLE_01":-3.6932540205777E9,"COMMON_8_BIN":36932540,"COMMON_S3_BIN":369,"COMMON_S94COMP":3693,"COMMON_S8_BIN":36932540,"COMMON_DDC97_BIN":3.6932540,"COMMON_97COMP3":3693254,"COMMON_915COMP3":369325402057777,"COMMON_S95COMP3":-36932,"COMMON_S999DCCOMP3":-369325402.05,"COMMON_S913COMP3":-3693254020577,"COMMON_S913DCCOMP3":-3693254020577.77,"COMMON_S911DCC2":-36932540205.77,"COMMON_S910DCC3":-3693254020.577,"COMMON_S03DDC":-0.36932,"COMMON_U03DDC":0.36932,"COMMON_UPC5DDC":0.00036932,"COMMON_SPC5DDC":-0.0036932,"COMMON_UPI5DDC":36932000,"COMMON_SPI5DDC":-36932000,"COMMON_UPC5DISP":-0.00036932,"COMMON_UPI5DISP":-36932000,"COMMON_UPC1BIN":0.0003,"COMMON_UPI1BIN":3000,"COMMON_UPC3BIN":0.000369,"COMMON_UPI3BIN":369000,"COMMON_UPC5BIN":0.00036932,"COMMON_UPI5BIN":36932000,"COMMON_UPC10BIN":0.0003693254020,"COMMON_UPI10BIN":3693254020000,"EX_NUM_INT01":-36932540,"EX_NUM_INT02":-36932540,"EX_NUM_INT03":-36932540,"EX_NUM_INT04":-36932540,"EX_NUM_DEC01":-369325.40,"EX_NUM_DEC02":-369325.40,"EX_NUM_DEC03":-369325.40} +{"ID":13,"STRING_VAL":"Mabelle","NUM_STR_INT01":9,"NUM_STR_INT02":99,"NUM_STR_INT03":994,"NUM_STR_INT04":9947,"NUM_STR_INT05":99479,"NUM_STR_INT06":99479683,"NUM_STR_INT07":994796833,"NUM_STR_INT08":9947968337,"NUM_STR_INT09":99479683373,"NUM_STR_INT10":99479683373533154,"NUM_STR_INT11":994796833735331546,"NUM_STR_INT12":9947968337353315468,"NUM_STR_INT13":99479683373533154685,"NUM_STR_INT14":9947968337353315468504174854002260461,"NUM_STR_SINT02":99,"NUM_STR_SINT03":994,"NUM_STR_SINT04":9947,"NUM_STR_SINT05":99479,"NUM_STR_SINT06":99479683,"NUM_STR_SINT07":994796833,"NUM_STR_SINT08":9947968337,"NUM_STR_SINT09":99479683373,"NUM_STR_SINT10":99479683373533154,"NUM_STR_SINT11":994796833735331546,"NUM_STR_SINT12":9947968337353315468,"NUM_STR_SINT13":99479683373533154685,"NUM_STR_SINT14":9947968337353315468504174854002260461,"NUM_STR_DEC01":99.4,"NUM_STR_DEC02":99.47,"NUM_STR_DEC03":994.79,"NUM_STR_DEC04":9947.9683,"NUM_STR_DEC05":99479.6833,"NUM_STR_DEC06":99479.68337,"NUM_STR_DEC07":994796833735331.54,"NUM_STR_DEC08":9947968337353315.46,"NUM_STR_DEC09":99479683373533154.68,"NUM_STR_DEC10":994796833735331546.8504174854,"NUM_STR_SDEC01":99.4,"NUM_STR_SDEC02":99.47,"NUM_STR_SDEC03":994.79,"NUM_STR_SDEC04":9947.9683,"NUM_STR_SDEC05":99479.6833,"NUM_STR_SDEC06":99479.68337,"NUM_STR_SDEC07":994796833735331.54,"NUM_STR_SDEC08":9947968337353315.46,"NUM_STR_SDEC09":99479683373533154.68,"NUM_STR_SDEC10":994796833735331546.8504174854,"NUM_STR_EDEC03":994.79,"NUM_STR_EDEC04":9947.9683,"NUM_STR_EDEC05":99479.6833,"NUM_STR_EDEC06":99479.68337,"NUM_BIN_INT01":9,"NUM_BIN_INT02":99,"NUM_BIN_INT03":994,"NUM_BIN_INT04":9947,"NUM_BIN_INT05":99479,"NUM_BIN_INT06":99479683,"NUM_BIN_INT07":994796833,"NUM_BIN_INT08":9947968337,"NUM_BIN_INT09":99479683373,"NUM_BIN_INT10":99479683373533154,"NUM_BIN_INT11":994796833735331546,"NUM_BIN_INT12":9947968337353315468,"NUM_BIN_INT13":99479683373533154685,"NUM_BIN_INT14":9947968337353315468504174854002260461,"NUM_SBIN_SINT01":9,"NUM_SBIN_SINT02":99,"NUM_SBIN_SINT03":994,"NUM_SBIN_SINT04":9947,"NUM_SBIN_SINT05":99479,"NUM_SBIN_SINT06":99479683,"NUM_SBIN_SINT07":994796833,"NUM_SBIN_SINT08":9947968337,"NUM_SBIN_SINT09":99479683373,"NUM_SBIN_SINT10":99479683373533154,"NUM_SBIN_SINT11":994796833735331546,"NUM_SBIN_SINT12":9947968337353315468,"NUM_SBIN_SINT13":99479683373533154685,"NUM_SBIN_SINT14":9947968337353315468504174854002260461,"NUM_BIN_DEC01":99.4,"NUM_BIN_DEC02":99.47,"NUM_BIN_DEC03":994.79,"NUM_BIN_DEC04":9947.9683,"NUM_BIN_DEC05":99479.6833,"NUM_BIN_DEC06":99479.68337,"NUM_BIN_DEC07":994796833735331.54,"NUM_BIN_DEC08":9947968337353315.46,"NUM_BIN_DEC09":99479683373533154.68,"NUM_BIN_DEC10":994796833735331546.8504174854,"NUM_SBIN_DEC01":99.4,"NUM_SBIN_DEC02":99.47,"NUM_SBIN_DEC03":994.79,"NUM_SBIN_DEC04":9947.9683,"NUM_SBIN_DEC05":99479.6833,"NUM_SBIN_DEC06":99479.68337,"NUM_SBIN_DEC07":994796833735331.54,"NUM_SBIN_DEC08":9947968337353315.46,"NUM_SBIN_DEC09":99479683373533154.68,"NUM_SBIN_DEC10":994796833735331546.8504174854,"NUM_BCD_INT01":9,"NUM_BCD_INT02":99,"NUM_BCD_INT03":994,"NUM_BCD_INT04":9947,"NUM_BCD_INT05":99479,"NUM_BCD_INT06":99479683,"NUM_BCD_INT07":994796833,"NUM_BCD_INT08":9947968337,"NUM_BCD_INT09":99479683373,"NUM_BCD_INT10":99479683373533154,"NUM_BCD_INT11":994796833735331546,"NUM_BCD_INT12":9947968337353315468,"NUM_BCD_INT13":99479683373533154685,"NUM_BCD_INT14":9947968337353315468504174854002260461,"NUM_BCD_SINT01":9,"NUM_BCD_SINT02":99,"NUM_BCD_SINT03":994,"NUM_BCD_SINT04":9947,"NUM_BCD_SINT05":99479,"NUM_BCD_SINT06":99479683,"NUM_BCD_SINT07":994796833,"NUM_BCD_SINT08":9947968337,"NUM_BCD_SINT09":99479683373,"NUM_BCD_SINT10":99479683373533154,"NUM_BCD_SINT11":994796833735331546,"NUM_BCD_SINT12":9947968337353315468,"NUM_BCD_SINT13":99479683373533154685,"NUM_BCD_SINT14":9947968337353315468504174854002260461,"NUM_BCD_DEC01":99.4,"NUM_BCD_DEC02":99.47,"NUM_BCD_DEC03":994.79,"NUM_BCD_DEC04":9947.9683,"NUM_BCD_DEC05":99479.6833,"NUM_BCD_DEC06":99479.68337,"NUM_BCD_DEC07":994796833735331.54,"NUM_BCD_DEC08":9947968337353315.46,"NUM_BCD_DEC09":99479683373533154.68,"NUM_BCD_DEC10":994796833735331546.8504174854,"NUM_BCD_SDEC01":99.4,"NUM_BCD_SDEC02":99.47,"NUM_BCD_SDEC03":994.79,"NUM_BCD_SDEC04":9947.9683,"NUM_BCD_SDEC05":99479.6833,"NUM_BCD_SDEC06":99479.68337,"NUM_BCD_SDEC07":994796833735331.54,"NUM_BCD_SDEC08":9947968337353315.46,"NUM_BCD_SDEC09":99479683373533154.68,"NUM_BCD_SDEC10":994796833735331546.8504174854,"NUM_SL_STR_INT01":994796833,"NUM_SL_STR_DEC01":99.47,"NUM_ST_STR_INT01":994796833,"NUM_ST_STR_DEC01":99.47,"NUM_SLI_STR_DEC01":0.9947968,"NUM_STI_STR_DEC01":0.9947968,"NUM_SLI_DEBUG":"I947968","NUM_STI_DEBUG":"994796H","FLOAT_01":99479.68,"DOUBLE_01":9.9479683373533E9,"COMMON_8_BIN":99479683,"COMMON_S3_BIN":994,"COMMON_S94COMP":9947,"COMMON_S8_BIN":99479683,"COMMON_DDC97_BIN":9.9479683,"COMMON_97COMP3":9947968,"COMMON_915COMP3":994796833735331,"COMMON_S95COMP3":99479,"COMMON_S999DCCOMP3":994796833.73,"COMMON_S913COMP3":9947968337353,"COMMON_S913DCCOMP3":9947968337353.31,"COMMON_S911DCC2":99479683373.53,"COMMON_S910DCC3":9947968337.353,"COMMON_S03DDC":0.99479,"COMMON_U03DDC":0.99479,"COMMON_UPC5DDC":0.00099479,"COMMON_SPC5DDC":0.0099479,"COMMON_UPI5DDC":99479000,"COMMON_SPI5DDC":99479000,"COMMON_UPC5DISP":0.00099479,"COMMON_UPI5DISP":99479000,"COMMON_UPC1BIN":0.0009,"COMMON_UPI1BIN":9000,"COMMON_UPC3BIN":0.000994,"COMMON_UPI3BIN":994000,"COMMON_UPC5BIN":0.00099479,"COMMON_UPI5BIN":99479000,"COMMON_UPC10BIN":0.0009947968337,"COMMON_UPI10BIN":9947968337000,"EX_NUM_INT01":99479683,"EX_NUM_INT02":99479683,"EX_NUM_INT03":99479683,"EX_NUM_INT04":99479683,"EX_NUM_DEC01":994796.83,"EX_NUM_DEC02":994796.83,"EX_NUM_DEC03":994796.83} +{"ID":14,"STRING_VAL":"Mindy","NUM_STR_INT01":2,"NUM_STR_INT02":20,"NUM_STR_INT03":206,"NUM_STR_INT04":2063,"NUM_STR_INT05":20634,"NUM_STR_INT06":20634354,"NUM_STR_INT07":206343544,"NUM_STR_INT08":2063435443,"NUM_STR_INT09":20634354437,"NUM_STR_INT10":20634354437152765,"NUM_STR_INT11":206343544371527653,"NUM_STR_INT12":2063435443715276535,"NUM_STR_INT13":20634354437152765351,"NUM_STR_INT14":2063435443715276535172479456068917785,"NUM_STR_SINT02":-20,"NUM_STR_SINT03":-206,"NUM_STR_SINT04":-2063,"NUM_STR_SINT05":-20634,"NUM_STR_SINT06":-20634354,"NUM_STR_SINT07":-206343544,"NUM_STR_SINT08":-2063435443,"NUM_STR_SINT09":-20634354437,"NUM_STR_SINT10":-20634354437152765,"NUM_STR_SINT11":-206343544371527653,"NUM_STR_SINT12":-2063435443715276535,"NUM_STR_SINT13":-20634354437152765351,"NUM_STR_SINT14":-2063435443715276535172479456068917785,"NUM_STR_DEC01":20.6,"NUM_STR_DEC02":20.63,"NUM_STR_DEC03":206.34,"NUM_STR_DEC04":2063.4354,"NUM_STR_DEC05":20634.3544,"NUM_STR_DEC06":20634.35443,"NUM_STR_DEC07":206343544371527.65,"NUM_STR_DEC08":2063435443715276.53,"NUM_STR_DEC09":20634354437152765.35,"NUM_STR_DEC10":206343544371527653.5172479456,"NUM_STR_SDEC01":-20.6,"NUM_STR_SDEC02":-20.63,"NUM_STR_SDEC03":-206.34,"NUM_STR_SDEC04":-2063.4354,"NUM_STR_SDEC05":-20634.3544,"NUM_STR_SDEC06":-20634.35443,"NUM_STR_SDEC07":-206343544371527.65,"NUM_STR_SDEC08":-2063435443715276.53,"NUM_STR_SDEC09":-20634354437152765.35,"NUM_STR_SDEC10":-206343544371527653.5172479456,"NUM_STR_EDEC03":-206.34,"NUM_STR_EDEC04":-2063.4354,"NUM_STR_EDEC05":-20634.3544,"NUM_STR_EDEC06":-20634.35443,"NUM_BIN_INT01":2,"NUM_BIN_INT02":20,"NUM_BIN_INT03":206,"NUM_BIN_INT04":2063,"NUM_BIN_INT05":20634,"NUM_BIN_INT06":20634354,"NUM_BIN_INT07":206343544,"NUM_BIN_INT08":2063435443,"NUM_BIN_INT09":20634354437,"NUM_BIN_INT10":20634354437152765,"NUM_BIN_INT11":206343544371527653,"NUM_BIN_INT12":2063435443715276535,"NUM_BIN_INT13":20634354437152765351,"NUM_BIN_INT14":2063435443715276535172479456068917785,"NUM_SBIN_SINT01":-2,"NUM_SBIN_SINT02":-20,"NUM_SBIN_SINT03":-206,"NUM_SBIN_SINT04":-2063,"NUM_SBIN_SINT05":-20634,"NUM_SBIN_SINT06":-20634354,"NUM_SBIN_SINT07":-206343544,"NUM_SBIN_SINT08":-2063435443,"NUM_SBIN_SINT09":-20634354437,"NUM_SBIN_SINT10":-20634354437152765,"NUM_SBIN_SINT11":-206343544371527653,"NUM_SBIN_SINT12":-2063435443715276535,"NUM_SBIN_SINT13":-20634354437152765351,"NUM_SBIN_SINT14":-2063435443715276535172479456068917785,"NUM_BIN_DEC01":20.6,"NUM_BIN_DEC02":20.63,"NUM_BIN_DEC03":206.34,"NUM_BIN_DEC04":2063.4354,"NUM_BIN_DEC05":20634.3544,"NUM_BIN_DEC06":20634.35443,"NUM_BIN_DEC07":206343544371527.65,"NUM_BIN_DEC08":2063435443715276.53,"NUM_BIN_DEC09":20634354437152765.35,"NUM_BIN_DEC10":206343544371527653.5172479456,"NUM_SBIN_DEC01":-20.6,"NUM_SBIN_DEC02":-20.63,"NUM_SBIN_DEC03":-206.34,"NUM_SBIN_DEC04":-2063.4354,"NUM_SBIN_DEC05":-20634.3544,"NUM_SBIN_DEC06":-20634.35443,"NUM_SBIN_DEC07":-206343544371527.65,"NUM_SBIN_DEC08":-2063435443715276.53,"NUM_SBIN_DEC09":-20634354437152765.35,"NUM_SBIN_DEC10":-206343544371527653.5172479456,"NUM_BCD_INT01":2,"NUM_BCD_INT02":20,"NUM_BCD_INT03":206,"NUM_BCD_INT04":2063,"NUM_BCD_INT05":20634,"NUM_BCD_INT06":20634354,"NUM_BCD_INT07":206343544,"NUM_BCD_INT08":2063435443,"NUM_BCD_INT09":20634354437,"NUM_BCD_INT10":20634354437152765,"NUM_BCD_INT11":206343544371527653,"NUM_BCD_INT12":2063435443715276535,"NUM_BCD_INT13":20634354437152765351,"NUM_BCD_INT14":2063435443715276535172479456068917785,"NUM_BCD_SINT01":-2,"NUM_BCD_SINT02":-20,"NUM_BCD_SINT03":-206,"NUM_BCD_SINT04":-2063,"NUM_BCD_SINT05":-20634,"NUM_BCD_SINT06":-20634354,"NUM_BCD_SINT07":-206343544,"NUM_BCD_SINT08":-2063435443,"NUM_BCD_SINT09":-20634354437,"NUM_BCD_SINT10":-20634354437152765,"NUM_BCD_SINT11":-206343544371527653,"NUM_BCD_SINT12":-2063435443715276535,"NUM_BCD_SINT13":-20634354437152765351,"NUM_BCD_SINT14":-2063435443715276535172479456068917785,"NUM_BCD_DEC01":20.6,"NUM_BCD_DEC02":20.63,"NUM_BCD_DEC03":206.34,"NUM_BCD_DEC04":2063.4354,"NUM_BCD_DEC05":20634.3544,"NUM_BCD_DEC06":20634.35443,"NUM_BCD_DEC07":206343544371527.65,"NUM_BCD_DEC08":2063435443715276.53,"NUM_BCD_DEC09":20634354437152765.35,"NUM_BCD_DEC10":206343544371527653.5172479456,"NUM_BCD_SDEC01":-20.6,"NUM_BCD_SDEC02":-20.63,"NUM_BCD_SDEC03":-206.34,"NUM_BCD_SDEC04":-2063.4354,"NUM_BCD_SDEC05":-20634.3544,"NUM_BCD_SDEC06":-20634.35443,"NUM_BCD_SDEC07":-206343544371527.65,"NUM_BCD_SDEC08":-2063435443715276.53,"NUM_BCD_SDEC09":-20634354437152765.35,"NUM_BCD_SDEC10":-206343544371527653.5172479456,"NUM_SL_STR_INT01":-206343544,"NUM_SL_STR_DEC01":-20.63,"NUM_ST_STR_INT01":-206343544,"NUM_ST_STR_DEC01":-20.63,"NUM_SLI_STR_DEC01":-0.2063435,"NUM_STI_STR_DEC01":-0.2063435,"NUM_SLI_DEBUG":"K063435","NUM_STI_DEBUG":"206343N","FLOAT_01":-20634.35,"DOUBLE_01":-2.0634354437152E9,"COMMON_8_BIN":20634354,"COMMON_S3_BIN":206,"COMMON_S94COMP":2063,"COMMON_S8_BIN":20634354,"COMMON_DDC97_BIN":2.0634354,"COMMON_97COMP3":2063435,"COMMON_915COMP3":206343544371527,"COMMON_S95COMP3":-20634,"COMMON_S999DCCOMP3":-206343544.37,"COMMON_S913COMP3":-2063435443715,"COMMON_S913DCCOMP3":-2063435443715.27,"COMMON_S911DCC2":-20634354437.15,"COMMON_S910DCC3":-2063435443.715,"COMMON_S03DDC":-0.20634,"COMMON_U03DDC":0.20634,"COMMON_UPC5DDC":0.00020634,"COMMON_SPC5DDC":-0.0020634,"COMMON_UPI5DDC":20634000,"COMMON_SPI5DDC":-20634000,"COMMON_UPC5DISP":-0.00020634,"COMMON_UPI5DISP":-20634000,"COMMON_UPC1BIN":0.0002,"COMMON_UPI1BIN":2000,"COMMON_UPC3BIN":0.000206,"COMMON_UPI3BIN":206000,"COMMON_UPC5BIN":0.00020634,"COMMON_UPI5BIN":20634000,"COMMON_UPC10BIN":0.0002063435443,"COMMON_UPI10BIN":2063435443000,"EX_NUM_INT01":-20634354,"EX_NUM_INT02":-20634354,"EX_NUM_INT03":-20634354,"EX_NUM_INT04":-20634354,"EX_NUM_DEC01":-206343.54,"EX_NUM_DEC02":-206343.54,"EX_NUM_DEC03":-206343.54} +{"ID":15,"STRING_VAL":"Tyesha","NUM_STR_INT01":9,"NUM_STR_INT02":90,"NUM_STR_INT03":906,"NUM_STR_INT04":9063,"NUM_STR_INT05":90635,"NUM_STR_INT06":90635703,"NUM_STR_INT07":906357034,"NUM_STR_INT08":9063570342,"NUM_STR_INT09":90635703427,"NUM_STR_INT10":90635703427622204,"NUM_STR_INT11":906357034276222048,"NUM_STR_INT12":9063570342762220484,"NUM_STR_INT13":90635703427622204843,"NUM_STR_INT14":9063570342762220484314331929266051497,"NUM_STR_SINT02":-90,"NUM_STR_SINT03":-906,"NUM_STR_SINT04":-9063,"NUM_STR_SINT05":-90635,"NUM_STR_SINT06":-90635703,"NUM_STR_SINT07":-906357034,"NUM_STR_SINT08":-9063570342,"NUM_STR_SINT09":-90635703427,"NUM_STR_SINT10":-90635703427622204,"NUM_STR_SINT11":-906357034276222048,"NUM_STR_SINT12":-9063570342762220484,"NUM_STR_SINT13":-90635703427622204843,"NUM_STR_SINT14":-9063570342762220484314331929266051497,"NUM_STR_DEC01":90.6,"NUM_STR_DEC02":90.63,"NUM_STR_DEC03":906.35,"NUM_STR_DEC04":9063.5703,"NUM_STR_DEC05":90635.7034,"NUM_STR_DEC06":90635.70342,"NUM_STR_DEC07":906357034276222.04,"NUM_STR_DEC08":9063570342762220.48,"NUM_STR_DEC09":90635703427622204.84,"NUM_STR_DEC10":906357034276222048.4314331929,"NUM_STR_SDEC01":-90.6,"NUM_STR_SDEC02":-90.63,"NUM_STR_SDEC03":-906.35,"NUM_STR_SDEC04":-9063.5703,"NUM_STR_SDEC05":-90635.7034,"NUM_STR_SDEC06":-90635.70342,"NUM_STR_SDEC07":-906357034276222.04,"NUM_STR_SDEC08":-9063570342762220.48,"NUM_STR_SDEC09":-90635703427622204.84,"NUM_STR_SDEC10":-906357034276222048.4314331929,"NUM_STR_EDEC03":-906.35,"NUM_STR_EDEC04":-9063.5703,"NUM_STR_EDEC05":-90635.7034,"NUM_STR_EDEC06":-90635.70342,"NUM_BIN_INT01":9,"NUM_BIN_INT02":90,"NUM_BIN_INT03":906,"NUM_BIN_INT04":9063,"NUM_BIN_INT05":90635,"NUM_BIN_INT06":90635703,"NUM_BIN_INT07":906357034,"NUM_BIN_INT08":9063570342,"NUM_BIN_INT09":90635703427,"NUM_BIN_INT10":90635703427622204,"NUM_BIN_INT11":906357034276222048,"NUM_BIN_INT12":9063570342762220484,"NUM_BIN_INT13":90635703427622204843,"NUM_BIN_INT14":9063570342762220484314331929266051497,"NUM_SBIN_SINT01":-9,"NUM_SBIN_SINT02":-90,"NUM_SBIN_SINT03":-906,"NUM_SBIN_SINT04":-9063,"NUM_SBIN_SINT05":-90635,"NUM_SBIN_SINT06":-90635703,"NUM_SBIN_SINT07":-906357034,"NUM_SBIN_SINT08":-9063570342,"NUM_SBIN_SINT09":-90635703427,"NUM_SBIN_SINT10":-90635703427622204,"NUM_SBIN_SINT11":-906357034276222048,"NUM_SBIN_SINT12":-9063570342762220484,"NUM_SBIN_SINT13":-90635703427622204843,"NUM_SBIN_SINT14":-9063570342762220484314331929266051497,"NUM_BIN_DEC01":90.6,"NUM_BIN_DEC02":90.63,"NUM_BIN_DEC03":906.35,"NUM_BIN_DEC04":9063.5703,"NUM_BIN_DEC05":90635.7034,"NUM_BIN_DEC06":90635.70342,"NUM_BIN_DEC07":906357034276222.04,"NUM_BIN_DEC08":9063570342762220.48,"NUM_BIN_DEC09":90635703427622204.84,"NUM_BIN_DEC10":906357034276222048.4314331929,"NUM_SBIN_DEC01":-90.6,"NUM_SBIN_DEC02":-90.63,"NUM_SBIN_DEC03":-906.35,"NUM_SBIN_DEC04":-9063.5703,"NUM_SBIN_DEC05":-90635.7034,"NUM_SBIN_DEC06":-90635.70342,"NUM_SBIN_DEC07":-906357034276222.04,"NUM_SBIN_DEC08":-9063570342762220.48,"NUM_SBIN_DEC09":-90635703427622204.84,"NUM_SBIN_DEC10":-906357034276222048.4314331929,"NUM_BCD_INT01":9,"NUM_BCD_INT02":90,"NUM_BCD_INT03":906,"NUM_BCD_INT04":9063,"NUM_BCD_INT05":90635,"NUM_BCD_INT06":90635703,"NUM_BCD_INT07":906357034,"NUM_BCD_INT08":9063570342,"NUM_BCD_INT09":90635703427,"NUM_BCD_INT10":90635703427622204,"NUM_BCD_INT11":906357034276222048,"NUM_BCD_INT12":9063570342762220484,"NUM_BCD_INT13":90635703427622204843,"NUM_BCD_INT14":9063570342762220484314331929266051497,"NUM_BCD_SINT01":-9,"NUM_BCD_SINT02":-90,"NUM_BCD_SINT03":-906,"NUM_BCD_SINT04":-9063,"NUM_BCD_SINT05":-90635,"NUM_BCD_SINT06":-90635703,"NUM_BCD_SINT07":-906357034,"NUM_BCD_SINT08":-9063570342,"NUM_BCD_SINT09":-90635703427,"NUM_BCD_SINT10":-90635703427622204,"NUM_BCD_SINT11":-906357034276222048,"NUM_BCD_SINT12":-9063570342762220484,"NUM_BCD_SINT13":-90635703427622204843,"NUM_BCD_SINT14":-9063570342762220484314331929266051497,"NUM_BCD_DEC01":90.6,"NUM_BCD_DEC02":90.63,"NUM_BCD_DEC03":906.35,"NUM_BCD_DEC04":9063.5703,"NUM_BCD_DEC05":90635.7034,"NUM_BCD_DEC06":90635.70342,"NUM_BCD_DEC07":906357034276222.04,"NUM_BCD_DEC08":9063570342762220.48,"NUM_BCD_DEC09":90635703427622204.84,"NUM_BCD_DEC10":906357034276222048.4314331929,"NUM_BCD_SDEC01":-90.6,"NUM_BCD_SDEC02":-90.63,"NUM_BCD_SDEC03":-906.35,"NUM_BCD_SDEC04":-9063.5703,"NUM_BCD_SDEC05":-90635.7034,"NUM_BCD_SDEC06":-90635.70342,"NUM_BCD_SDEC07":-906357034276222.04,"NUM_BCD_SDEC08":-9063570342762220.48,"NUM_BCD_SDEC09":-90635703427622204.84,"NUM_BCD_SDEC10":-906357034276222048.4314331929,"NUM_SL_STR_INT01":-906357034,"NUM_SL_STR_DEC01":-90.63,"NUM_ST_STR_INT01":-906357034,"NUM_ST_STR_DEC01":-90.63,"NUM_SLI_STR_DEC01":-0.9063570,"NUM_STI_STR_DEC01":-0.9063570,"NUM_SLI_DEBUG":"R063570","NUM_STI_DEBUG":"906357}","FLOAT_01":-90635.7,"DOUBLE_01":-9.0635703427622E9,"COMMON_8_BIN":90635703,"COMMON_S3_BIN":906,"COMMON_S94COMP":9063,"COMMON_S8_BIN":90635703,"COMMON_DDC97_BIN":9.0635703,"COMMON_97COMP3":9063570,"COMMON_915COMP3":906357034276222,"COMMON_S95COMP3":-90635,"COMMON_S999DCCOMP3":-906357034.27,"COMMON_S913COMP3":-9063570342762,"COMMON_S913DCCOMP3":-9063570342762.22,"COMMON_S911DCC2":-90635703427.62,"COMMON_S910DCC3":-9063570342.762,"COMMON_S03DDC":-0.90635,"COMMON_U03DDC":0.90635,"COMMON_UPC5DDC":0.00090635,"COMMON_SPC5DDC":-0.0090635,"COMMON_UPI5DDC":90635000,"COMMON_SPI5DDC":-90635000,"COMMON_UPC5DISP":-0.00090635,"COMMON_UPI5DISP":-90635000,"COMMON_UPC1BIN":0.0009,"COMMON_UPI1BIN":9000,"COMMON_UPC3BIN":0.000906,"COMMON_UPI3BIN":906000,"COMMON_UPC5BIN":0.00090635,"COMMON_UPI5BIN":90635000,"COMMON_UPC10BIN":0.0009063570342,"COMMON_UPI10BIN":9063570342000,"EX_NUM_INT01":-90635703,"EX_NUM_INT02":-90635703,"EX_NUM_INT03":-90635703,"EX_NUM_INT04":-90635703,"EX_NUM_DEC01":-906357.03,"EX_NUM_DEC02":-906357.03,"EX_NUM_DEC03":-906357.03} +{"ID":16,"STRING_VAL":"Inge","NUM_STR_INT01":9,"NUM_STR_INT02":94,"NUM_STR_INT03":944,"NUM_STR_INT04":9446,"NUM_STR_INT05":94469,"NUM_STR_INT06":94469205,"NUM_STR_INT07":944692055,"NUM_STR_INT08":9446920551,"NUM_STR_INT09":94469205511,"NUM_STR_INT10":94469205511591675,"NUM_STR_INT11":944692055115916755,"NUM_STR_INT12":9446920551159167554,"NUM_STR_INT13":94469205511591675542,"NUM_STR_INT14":9446920551159167554225609673598574132,"NUM_STR_SINT02":-94,"NUM_STR_SINT03":-944,"NUM_STR_SINT04":-9446,"NUM_STR_SINT05":-94469,"NUM_STR_SINT06":-94469205,"NUM_STR_SINT07":-944692055,"NUM_STR_SINT08":-9446920551,"NUM_STR_SINT09":-94469205511,"NUM_STR_SINT10":-94469205511591675,"NUM_STR_SINT11":-944692055115916755,"NUM_STR_SINT12":-9446920551159167554,"NUM_STR_SINT13":-94469205511591675542,"NUM_STR_SINT14":-9446920551159167554225609673598574132,"NUM_STR_DEC01":94.4,"NUM_STR_DEC02":94.46,"NUM_STR_DEC03":944.69,"NUM_STR_DEC04":9446.9205,"NUM_STR_DEC05":94469.2055,"NUM_STR_DEC06":94469.20551,"NUM_STR_DEC07":944692055115916.75,"NUM_STR_DEC08":9446920551159167.55,"NUM_STR_DEC09":94469205511591675.54,"NUM_STR_DEC10":944692055115916755.4225609673,"NUM_STR_SDEC01":-94.4,"NUM_STR_SDEC02":-94.46,"NUM_STR_SDEC03":-944.69,"NUM_STR_SDEC04":-9446.9205,"NUM_STR_SDEC05":-94469.2055,"NUM_STR_SDEC06":-94469.20551,"NUM_STR_SDEC07":-944692055115916.75,"NUM_STR_SDEC08":-9446920551159167.55,"NUM_STR_SDEC09":-94469205511591675.54,"NUM_STR_SDEC10":-944692055115916755.4225609673,"NUM_STR_EDEC03":-944.69,"NUM_STR_EDEC04":-9446.9205,"NUM_STR_EDEC05":-94469.2055,"NUM_STR_EDEC06":-94469.20551,"NUM_BIN_INT01":9,"NUM_BIN_INT02":94,"NUM_BIN_INT03":944,"NUM_BIN_INT04":9446,"NUM_BIN_INT05":94469,"NUM_BIN_INT06":94469205,"NUM_BIN_INT07":944692055,"NUM_BIN_INT08":9446920551,"NUM_BIN_INT09":94469205511,"NUM_BIN_INT10":94469205511591675,"NUM_BIN_INT11":944692055115916755,"NUM_BIN_INT12":9446920551159167554,"NUM_BIN_INT13":94469205511591675542,"NUM_BIN_INT14":9446920551159167554225609673598574132,"NUM_SBIN_SINT01":-9,"NUM_SBIN_SINT02":-94,"NUM_SBIN_SINT03":-944,"NUM_SBIN_SINT04":-9446,"NUM_SBIN_SINT05":-94469,"NUM_SBIN_SINT06":-94469205,"NUM_SBIN_SINT07":-944692055,"NUM_SBIN_SINT08":-9446920551,"NUM_SBIN_SINT09":-94469205511,"NUM_SBIN_SINT10":-94469205511591675,"NUM_SBIN_SINT11":-944692055115916755,"NUM_SBIN_SINT12":-9446920551159167554,"NUM_SBIN_SINT13":-94469205511591675542,"NUM_SBIN_SINT14":-9446920551159167554225609673598574132,"NUM_BIN_DEC01":94.4,"NUM_BIN_DEC02":94.46,"NUM_BIN_DEC03":944.69,"NUM_BIN_DEC04":9446.9205,"NUM_BIN_DEC05":94469.2055,"NUM_BIN_DEC06":94469.20551,"NUM_BIN_DEC07":944692055115916.75,"NUM_BIN_DEC08":9446920551159167.55,"NUM_BIN_DEC09":94469205511591675.54,"NUM_BIN_DEC10":944692055115916755.4225609673,"NUM_SBIN_DEC01":-94.4,"NUM_SBIN_DEC02":-94.46,"NUM_SBIN_DEC03":-944.69,"NUM_SBIN_DEC04":-9446.9205,"NUM_SBIN_DEC05":-94469.2055,"NUM_SBIN_DEC06":-94469.20551,"NUM_SBIN_DEC07":-944692055115916.75,"NUM_SBIN_DEC08":-9446920551159167.55,"NUM_SBIN_DEC09":-94469205511591675.54,"NUM_SBIN_DEC10":-944692055115916755.4225609673,"NUM_BCD_INT01":9,"NUM_BCD_INT02":94,"NUM_BCD_INT03":944,"NUM_BCD_INT04":9446,"NUM_BCD_INT05":94469,"NUM_BCD_INT06":94469205,"NUM_BCD_INT07":944692055,"NUM_BCD_INT08":9446920551,"NUM_BCD_INT09":94469205511,"NUM_BCD_INT10":94469205511591675,"NUM_BCD_INT11":944692055115916755,"NUM_BCD_INT12":9446920551159167554,"NUM_BCD_INT13":94469205511591675542,"NUM_BCD_INT14":9446920551159167554225609673598574132,"NUM_BCD_SINT01":-9,"NUM_BCD_SINT02":-94,"NUM_BCD_SINT03":-944,"NUM_BCD_SINT04":-9446,"NUM_BCD_SINT05":-94469,"NUM_BCD_SINT06":-94469205,"NUM_BCD_SINT07":-944692055,"NUM_BCD_SINT08":-9446920551,"NUM_BCD_SINT09":-94469205511,"NUM_BCD_SINT10":-94469205511591675,"NUM_BCD_SINT11":-944692055115916755,"NUM_BCD_SINT12":-9446920551159167554,"NUM_BCD_SINT13":-94469205511591675542,"NUM_BCD_SINT14":-9446920551159167554225609673598574132,"NUM_BCD_DEC01":94.4,"NUM_BCD_DEC02":94.46,"NUM_BCD_DEC03":944.69,"NUM_BCD_DEC04":9446.9205,"NUM_BCD_DEC05":94469.2055,"NUM_BCD_DEC06":94469.20551,"NUM_BCD_DEC07":944692055115916.75,"NUM_BCD_DEC08":9446920551159167.55,"NUM_BCD_DEC09":94469205511591675.54,"NUM_BCD_DEC10":944692055115916755.4225609673,"NUM_BCD_SDEC01":-94.4,"NUM_BCD_SDEC02":-94.46,"NUM_BCD_SDEC03":-944.69,"NUM_BCD_SDEC04":-9446.9205,"NUM_BCD_SDEC05":-94469.2055,"NUM_BCD_SDEC06":-94469.20551,"NUM_BCD_SDEC07":-944692055115916.75,"NUM_BCD_SDEC08":-9446920551159167.55,"NUM_BCD_SDEC09":-94469205511591675.54,"NUM_BCD_SDEC10":-944692055115916755.4225609673,"NUM_SL_STR_INT01":-944692055,"NUM_SL_STR_DEC01":-94.46,"NUM_ST_STR_INT01":-944692055,"NUM_ST_STR_DEC01":-94.46,"NUM_SLI_STR_DEC01":-0.9446920,"NUM_STI_STR_DEC01":-0.9446920,"NUM_SLI_DEBUG":"R446920","NUM_STI_DEBUG":"944692}","FLOAT_01":-94469.2,"DOUBLE_01":-9.4469205511591E9,"COMMON_8_BIN":94469205,"COMMON_S3_BIN":944,"COMMON_S94COMP":9446,"COMMON_S8_BIN":94469205,"COMMON_DDC97_BIN":9.4469205,"COMMON_97COMP3":9446920,"COMMON_915COMP3":944692055115916,"COMMON_S95COMP3":-94469,"COMMON_S999DCCOMP3":-944692055.11,"COMMON_S913COMP3":-9446920551159,"COMMON_S913DCCOMP3":-9446920551159.16,"COMMON_S911DCC2":-94469205511.59,"COMMON_S910DCC3":-9446920551.159,"COMMON_S03DDC":-0.94469,"COMMON_U03DDC":0.94469,"COMMON_UPC5DDC":0.00094469,"COMMON_SPC5DDC":-0.0094469,"COMMON_UPI5DDC":94469000,"COMMON_SPI5DDC":-94469000,"COMMON_UPC5DISP":-0.00094469,"COMMON_UPI5DISP":-94469000,"COMMON_UPC1BIN":0.0009,"COMMON_UPI1BIN":9000,"COMMON_UPC3BIN":0.000944,"COMMON_UPI3BIN":944000,"COMMON_UPC5BIN":0.00094469,"COMMON_UPI5BIN":94469000,"COMMON_UPC10BIN":0.0009446920551,"COMMON_UPI10BIN":9446920551000,"EX_NUM_INT01":-94469205,"EX_NUM_INT02":-94469205,"EX_NUM_INT03":-94469205,"EX_NUM_INT04":-94469205,"EX_NUM_DEC01":-944692.05,"EX_NUM_DEC02":-944692.05,"EX_NUM_DEC03":-944692.05} +{"ID":17,"STRING_VAL":"Carrie","NUM_STR_INT01":8,"NUM_STR_INT02":89,"NUM_STR_INT03":890,"NUM_STR_INT04":8905,"NUM_STR_INT05":89058,"NUM_STR_INT06":89058403,"NUM_STR_INT07":890584037,"NUM_STR_INT08":8905840379,"NUM_STR_INT09":89058403790,"NUM_STR_INT10":89058403790543065,"NUM_STR_INT11":890584037905430657,"NUM_STR_INT12":8905840379054306577,"NUM_STR_INT13":89058403790543065772,"NUM_STR_INT14":8905840379054306577260641738046162765,"NUM_STR_SINT02":-89,"NUM_STR_SINT03":-890,"NUM_STR_SINT04":-8905,"NUM_STR_SINT05":-89058,"NUM_STR_SINT06":-89058403,"NUM_STR_SINT07":-890584037,"NUM_STR_SINT08":-8905840379,"NUM_STR_SINT09":-89058403790,"NUM_STR_SINT10":-89058403790543065,"NUM_STR_SINT11":-890584037905430657,"NUM_STR_SINT12":-8905840379054306577,"NUM_STR_SINT13":-89058403790543065772,"NUM_STR_SINT14":-8905840379054306577260641738046162765,"NUM_STR_DEC01":89.0,"NUM_STR_DEC02":89.05,"NUM_STR_DEC03":890.58,"NUM_STR_DEC04":8905.8403,"NUM_STR_DEC05":89058.4037,"NUM_STR_DEC06":89058.40379,"NUM_STR_DEC07":890584037905430.65,"NUM_STR_DEC08":8905840379054306.57,"NUM_STR_DEC09":89058403790543065.77,"NUM_STR_DEC10":890584037905430657.7260641738,"NUM_STR_SDEC01":-89.0,"NUM_STR_SDEC02":-89.05,"NUM_STR_SDEC03":-890.58,"NUM_STR_SDEC04":-8905.8403,"NUM_STR_SDEC05":-89058.4037,"NUM_STR_SDEC06":-89058.40379,"NUM_STR_SDEC07":-890584037905430.65,"NUM_STR_SDEC08":-8905840379054306.57,"NUM_STR_SDEC09":-89058403790543065.77,"NUM_STR_SDEC10":-890584037905430657.7260641738,"NUM_STR_EDEC03":-890.58,"NUM_STR_EDEC04":-8905.8403,"NUM_STR_EDEC05":-89058.4037,"NUM_STR_EDEC06":-89058.40379,"NUM_BIN_INT01":8,"NUM_BIN_INT02":89,"NUM_BIN_INT03":890,"NUM_BIN_INT04":8905,"NUM_BIN_INT05":89058,"NUM_BIN_INT06":89058403,"NUM_BIN_INT07":890584037,"NUM_BIN_INT08":8905840379,"NUM_BIN_INT09":89058403790,"NUM_BIN_INT10":89058403790543065,"NUM_BIN_INT11":890584037905430657,"NUM_BIN_INT12":8905840379054306577,"NUM_BIN_INT13":89058403790543065772,"NUM_BIN_INT14":8905840379054306577260641738046162765,"NUM_SBIN_SINT01":-8,"NUM_SBIN_SINT02":-89,"NUM_SBIN_SINT03":-890,"NUM_SBIN_SINT04":-8905,"NUM_SBIN_SINT05":-89058,"NUM_SBIN_SINT06":-89058403,"NUM_SBIN_SINT07":-890584037,"NUM_SBIN_SINT08":-8905840379,"NUM_SBIN_SINT09":-89058403790,"NUM_SBIN_SINT10":-89058403790543065,"NUM_SBIN_SINT11":-890584037905430657,"NUM_SBIN_SINT12":-8905840379054306577,"NUM_SBIN_SINT13":-89058403790543065772,"NUM_SBIN_SINT14":-8905840379054306577260641738046162765,"NUM_BIN_DEC01":89.0,"NUM_BIN_DEC02":89.05,"NUM_BIN_DEC03":890.58,"NUM_BIN_DEC04":8905.8403,"NUM_BIN_DEC05":89058.4037,"NUM_BIN_DEC06":89058.40379,"NUM_BIN_DEC07":890584037905430.65,"NUM_BIN_DEC08":8905840379054306.57,"NUM_BIN_DEC09":89058403790543065.77,"NUM_BIN_DEC10":890584037905430657.7260641738,"NUM_SBIN_DEC01":-89.0,"NUM_SBIN_DEC02":-89.05,"NUM_SBIN_DEC03":-890.58,"NUM_SBIN_DEC04":-8905.8403,"NUM_SBIN_DEC05":-89058.4037,"NUM_SBIN_DEC06":-89058.40379,"NUM_SBIN_DEC07":-890584037905430.65,"NUM_SBIN_DEC08":-8905840379054306.57,"NUM_SBIN_DEC09":-89058403790543065.77,"NUM_SBIN_DEC10":-890584037905430657.7260641738,"NUM_BCD_INT01":8,"NUM_BCD_INT02":89,"NUM_BCD_INT03":890,"NUM_BCD_INT04":8905,"NUM_BCD_INT05":89058,"NUM_BCD_INT06":89058403,"NUM_BCD_INT07":890584037,"NUM_BCD_INT08":8905840379,"NUM_BCD_INT09":89058403790,"NUM_BCD_INT10":89058403790543065,"NUM_BCD_INT11":890584037905430657,"NUM_BCD_INT12":8905840379054306577,"NUM_BCD_INT13":89058403790543065772,"NUM_BCD_INT14":8905840379054306577260641738046162765,"NUM_BCD_SINT01":-8,"NUM_BCD_SINT02":-89,"NUM_BCD_SINT03":-890,"NUM_BCD_SINT04":-8905,"NUM_BCD_SINT05":-89058,"NUM_BCD_SINT06":-89058403,"NUM_BCD_SINT07":-890584037,"NUM_BCD_SINT08":-8905840379,"NUM_BCD_SINT09":-89058403790,"NUM_BCD_SINT10":-89058403790543065,"NUM_BCD_SINT11":-890584037905430657,"NUM_BCD_SINT12":-8905840379054306577,"NUM_BCD_SINT13":-89058403790543065772,"NUM_BCD_SINT14":-8905840379054306577260641738046162765,"NUM_BCD_DEC01":89.0,"NUM_BCD_DEC02":89.05,"NUM_BCD_DEC03":890.58,"NUM_BCD_DEC04":8905.8403,"NUM_BCD_DEC05":89058.4037,"NUM_BCD_DEC06":89058.40379,"NUM_BCD_DEC07":890584037905430.65,"NUM_BCD_DEC08":8905840379054306.57,"NUM_BCD_DEC09":89058403790543065.77,"NUM_BCD_DEC10":890584037905430657.7260641738,"NUM_BCD_SDEC01":-89.0,"NUM_BCD_SDEC02":-89.05,"NUM_BCD_SDEC03":-890.58,"NUM_BCD_SDEC04":-8905.8403,"NUM_BCD_SDEC05":-89058.4037,"NUM_BCD_SDEC06":-89058.40379,"NUM_BCD_SDEC07":-890584037905430.65,"NUM_BCD_SDEC08":-8905840379054306.57,"NUM_BCD_SDEC09":-89058403790543065.77,"NUM_BCD_SDEC10":-890584037905430657.7260641738,"NUM_SL_STR_INT01":-890584037,"NUM_SL_STR_DEC01":-89.05,"NUM_ST_STR_INT01":-890584037,"NUM_ST_STR_DEC01":-89.05,"NUM_SLI_STR_DEC01":-0.8905840,"NUM_STI_STR_DEC01":-0.8905840,"NUM_SLI_DEBUG":"Q905840","NUM_STI_DEBUG":"890584}","FLOAT_01":-89058.4,"DOUBLE_01":-8.9058403790543E9,"COMMON_8_BIN":89058403,"COMMON_S3_BIN":890,"COMMON_S94COMP":8905,"COMMON_S8_BIN":89058403,"COMMON_DDC97_BIN":8.9058403,"COMMON_97COMP3":8905840,"COMMON_915COMP3":890584037905430,"COMMON_S95COMP3":-89058,"COMMON_S999DCCOMP3":-890584037.90,"COMMON_S913COMP3":-8905840379054,"COMMON_S913DCCOMP3":-8905840379054.30,"COMMON_S911DCC2":-89058403790.54,"COMMON_S910DCC3":-8905840379.054,"COMMON_S03DDC":-0.89058,"COMMON_U03DDC":0.89058,"COMMON_UPC5DDC":0.00089058,"COMMON_SPC5DDC":-0.0089058,"COMMON_UPI5DDC":89058000,"COMMON_SPI5DDC":-89058000,"COMMON_UPC5DISP":-0.00089058,"COMMON_UPI5DISP":-89058000,"COMMON_UPC1BIN":0.0008,"COMMON_UPI1BIN":8000,"COMMON_UPC3BIN":0.000890,"COMMON_UPI3BIN":890000,"COMMON_UPC5BIN":0.00089058,"COMMON_UPI5BIN":89058000,"COMMON_UPC10BIN":0.0008905840379,"COMMON_UPI10BIN":8905840379000,"EX_NUM_INT01":-89058403,"EX_NUM_INT02":-89058403,"EX_NUM_INT03":-89058403,"EX_NUM_INT04":-89058403,"EX_NUM_DEC01":-890584.03,"EX_NUM_DEC02":-890584.03,"EX_NUM_DEC03":-890584.03} +{"ID":18,"STRING_VAL":"Cassey","NUM_STR_INT01":1,"NUM_STR_INT02":13,"NUM_STR_INT03":131,"NUM_STR_INT04":1314,"NUM_STR_INT05":13140,"NUM_STR_INT06":13140769,"NUM_STR_INT07":131407694,"NUM_STR_INT08":1314076945,"NUM_STR_INT09":13140769450,"NUM_STR_INT10":13140769450114788,"NUM_STR_INT11":131407694501147882,"NUM_STR_INT12":1314076945011478822,"NUM_STR_INT13":13140769450114788224,"NUM_STR_INT14":1314076945011478822446008858659148490,"NUM_STR_SINT02":13,"NUM_STR_SINT03":131,"NUM_STR_SINT04":1314,"NUM_STR_SINT05":13140,"NUM_STR_SINT06":13140769,"NUM_STR_SINT07":131407694,"NUM_STR_SINT08":1314076945,"NUM_STR_SINT09":13140769450,"NUM_STR_SINT10":13140769450114788,"NUM_STR_SINT11":131407694501147882,"NUM_STR_SINT12":1314076945011478822,"NUM_STR_SINT13":13140769450114788224,"NUM_STR_SINT14":1314076945011478822446008858659148490,"NUM_STR_DEC01":13.1,"NUM_STR_DEC02":13.14,"NUM_STR_DEC03":131.40,"NUM_STR_DEC04":1314.0769,"NUM_STR_DEC05":13140.7694,"NUM_STR_DEC06":13140.76945,"NUM_STR_DEC07":131407694501147.88,"NUM_STR_DEC08":1314076945011478.82,"NUM_STR_DEC09":13140769450114788.22,"NUM_STR_DEC10":131407694501147882.2446008858,"NUM_STR_SDEC01":13.1,"NUM_STR_SDEC02":13.14,"NUM_STR_SDEC03":131.40,"NUM_STR_SDEC04":1314.0769,"NUM_STR_SDEC05":13140.7694,"NUM_STR_SDEC06":13140.76945,"NUM_STR_SDEC07":131407694501147.88,"NUM_STR_SDEC08":1314076945011478.82,"NUM_STR_SDEC09":13140769450114788.22,"NUM_STR_SDEC10":131407694501147882.2446008858,"NUM_STR_EDEC03":131.40,"NUM_STR_EDEC04":1314.0769,"NUM_STR_EDEC05":13140.7694,"NUM_STR_EDEC06":13140.76945,"NUM_BIN_INT01":1,"NUM_BIN_INT02":13,"NUM_BIN_INT03":131,"NUM_BIN_INT04":1314,"NUM_BIN_INT05":13140,"NUM_BIN_INT06":13140769,"NUM_BIN_INT07":131407694,"NUM_BIN_INT08":1314076945,"NUM_BIN_INT09":13140769450,"NUM_BIN_INT10":13140769450114788,"NUM_BIN_INT11":131407694501147882,"NUM_BIN_INT12":1314076945011478822,"NUM_BIN_INT13":13140769450114788224,"NUM_BIN_INT14":1314076945011478822446008858659148490,"NUM_SBIN_SINT01":1,"NUM_SBIN_SINT02":13,"NUM_SBIN_SINT03":131,"NUM_SBIN_SINT04":1314,"NUM_SBIN_SINT05":13140,"NUM_SBIN_SINT06":13140769,"NUM_SBIN_SINT07":131407694,"NUM_SBIN_SINT08":1314076945,"NUM_SBIN_SINT09":13140769450,"NUM_SBIN_SINT10":13140769450114788,"NUM_SBIN_SINT11":131407694501147882,"NUM_SBIN_SINT12":1314076945011478822,"NUM_SBIN_SINT13":13140769450114788224,"NUM_SBIN_SINT14":1314076945011478822446008858659148490,"NUM_BIN_DEC01":13.1,"NUM_BIN_DEC02":13.14,"NUM_BIN_DEC03":131.40,"NUM_BIN_DEC04":1314.0769,"NUM_BIN_DEC05":13140.7694,"NUM_BIN_DEC06":13140.76945,"NUM_BIN_DEC07":131407694501147.88,"NUM_BIN_DEC08":1314076945011478.82,"NUM_BIN_DEC09":13140769450114788.22,"NUM_BIN_DEC10":131407694501147882.2446008858,"NUM_SBIN_DEC01":13.1,"NUM_SBIN_DEC02":13.14,"NUM_SBIN_DEC03":131.40,"NUM_SBIN_DEC04":1314.0769,"NUM_SBIN_DEC05":13140.7694,"NUM_SBIN_DEC06":13140.76945,"NUM_SBIN_DEC07":131407694501147.88,"NUM_SBIN_DEC08":1314076945011478.82,"NUM_SBIN_DEC09":13140769450114788.22,"NUM_SBIN_DEC10":131407694501147882.2446008858,"NUM_BCD_INT01":1,"NUM_BCD_INT02":13,"NUM_BCD_INT03":131,"NUM_BCD_INT04":1314,"NUM_BCD_INT05":13140,"NUM_BCD_INT06":13140769,"NUM_BCD_INT07":131407694,"NUM_BCD_INT08":1314076945,"NUM_BCD_INT09":13140769450,"NUM_BCD_INT10":13140769450114788,"NUM_BCD_INT11":131407694501147882,"NUM_BCD_INT12":1314076945011478822,"NUM_BCD_INT13":13140769450114788224,"NUM_BCD_INT14":1314076945011478822446008858659148490,"NUM_BCD_SINT01":1,"NUM_BCD_SINT02":13,"NUM_BCD_SINT03":131,"NUM_BCD_SINT04":1314,"NUM_BCD_SINT05":13140,"NUM_BCD_SINT06":13140769,"NUM_BCD_SINT07":131407694,"NUM_BCD_SINT08":1314076945,"NUM_BCD_SINT09":13140769450,"NUM_BCD_SINT10":13140769450114788,"NUM_BCD_SINT11":131407694501147882,"NUM_BCD_SINT12":1314076945011478822,"NUM_BCD_SINT13":13140769450114788224,"NUM_BCD_SINT14":1314076945011478822446008858659148490,"NUM_BCD_DEC01":13.1,"NUM_BCD_DEC02":13.14,"NUM_BCD_DEC03":131.40,"NUM_BCD_DEC04":1314.0769,"NUM_BCD_DEC05":13140.7694,"NUM_BCD_DEC06":13140.76945,"NUM_BCD_DEC07":131407694501147.88,"NUM_BCD_DEC08":1314076945011478.82,"NUM_BCD_DEC09":13140769450114788.22,"NUM_BCD_DEC10":131407694501147882.2446008858,"NUM_BCD_SDEC01":13.1,"NUM_BCD_SDEC02":13.14,"NUM_BCD_SDEC03":131.40,"NUM_BCD_SDEC04":1314.0769,"NUM_BCD_SDEC05":13140.7694,"NUM_BCD_SDEC06":13140.76945,"NUM_BCD_SDEC07":131407694501147.88,"NUM_BCD_SDEC08":1314076945011478.82,"NUM_BCD_SDEC09":13140769450114788.22,"NUM_BCD_SDEC10":131407694501147882.2446008858,"NUM_SL_STR_INT01":131407694,"NUM_SL_STR_DEC01":13.14,"NUM_ST_STR_INT01":131407694,"NUM_ST_STR_DEC01":13.14,"NUM_SLI_STR_DEC01":0.1314076,"NUM_STI_STR_DEC01":0.1314076,"NUM_SLI_DEBUG":"A314076","NUM_STI_DEBUG":"131407F","FLOAT_01":13140.76,"DOUBLE_01":1.3140769450114E9,"COMMON_8_BIN":13140769,"COMMON_S3_BIN":131,"COMMON_S94COMP":1314,"COMMON_S8_BIN":13140769,"COMMON_DDC97_BIN":1.3140769,"COMMON_97COMP3":1314076,"COMMON_915COMP3":131407694501147,"COMMON_S95COMP3":13140,"COMMON_S999DCCOMP3":131407694.50,"COMMON_S913COMP3":1314076945011,"COMMON_S913DCCOMP3":1314076945011.47,"COMMON_S911DCC2":13140769450.11,"COMMON_S910DCC3":1314076945.011,"COMMON_S03DDC":0.13140,"COMMON_U03DDC":0.13140,"COMMON_UPC5DDC":0.00013140,"COMMON_SPC5DDC":0.0013140,"COMMON_UPI5DDC":13140000,"COMMON_SPI5DDC":13140000,"COMMON_UPC5DISP":0.00013140,"COMMON_UPI5DISP":13140000,"COMMON_UPC1BIN":0.0001,"COMMON_UPI1BIN":1000,"COMMON_UPC3BIN":0.000131,"COMMON_UPI3BIN":131000,"COMMON_UPC5BIN":0.00013140,"COMMON_UPI5BIN":13140000,"COMMON_UPC10BIN":0.0001314076945,"COMMON_UPI10BIN":1314076945000,"EX_NUM_INT01":13140769,"EX_NUM_INT02":13140769,"EX_NUM_INT03":13140769,"EX_NUM_INT04":13140769,"EX_NUM_DEC01":131407.69,"EX_NUM_DEC02":131407.69,"EX_NUM_DEC03":131407.69} +{"ID":19,"STRING_VAL":"Eliana","NUM_STR_INT01":7,"NUM_STR_INT02":70,"NUM_STR_INT03":703,"NUM_STR_INT04":7034,"NUM_STR_INT05":70343,"NUM_STR_INT06":70343709,"NUM_STR_INT07":703437097,"NUM_STR_INT08":7034370970,"NUM_STR_INT09":70343709708,"NUM_STR_INT10":70343709708999957,"NUM_STR_INT11":703437097089999577,"NUM_STR_INT12":7034370970899995775,"NUM_STR_INT13":70343709708999957759,"NUM_STR_INT14":7034370970899995775972741521054695295,"NUM_STR_SINT02":-70,"NUM_STR_SINT03":-703,"NUM_STR_SINT04":-7034,"NUM_STR_SINT05":-70343,"NUM_STR_SINT06":-70343709,"NUM_STR_SINT07":-703437097,"NUM_STR_SINT08":-7034370970,"NUM_STR_SINT09":-70343709708,"NUM_STR_SINT10":-70343709708999957,"NUM_STR_SINT11":-703437097089999577,"NUM_STR_SINT12":-7034370970899995775,"NUM_STR_SINT13":-70343709708999957759,"NUM_STR_SINT14":-7034370970899995775972741521054695295,"NUM_STR_DEC01":70.3,"NUM_STR_DEC02":70.34,"NUM_STR_DEC03":703.43,"NUM_STR_DEC04":7034.3709,"NUM_STR_DEC05":70343.7097,"NUM_STR_DEC06":70343.70970,"NUM_STR_DEC07":703437097089999.57,"NUM_STR_DEC08":7034370970899995.77,"NUM_STR_DEC09":70343709708999957.75,"NUM_STR_DEC10":703437097089999577.5972741521,"NUM_STR_SDEC01":-70.3,"NUM_STR_SDEC02":-70.34,"NUM_STR_SDEC03":-703.43,"NUM_STR_SDEC04":-7034.3709,"NUM_STR_SDEC05":-70343.7097,"NUM_STR_SDEC06":-70343.70970,"NUM_STR_SDEC07":-703437097089999.57,"NUM_STR_SDEC08":-7034370970899995.77,"NUM_STR_SDEC09":-70343709708999957.75,"NUM_STR_SDEC10":-703437097089999577.5972741521,"NUM_STR_EDEC03":-703.43,"NUM_STR_EDEC04":-7034.3709,"NUM_STR_EDEC05":-70343.7097,"NUM_STR_EDEC06":-70343.70970,"NUM_BIN_INT01":7,"NUM_BIN_INT02":70,"NUM_BIN_INT03":703,"NUM_BIN_INT04":7034,"NUM_BIN_INT05":70343,"NUM_BIN_INT06":70343709,"NUM_BIN_INT07":703437097,"NUM_BIN_INT08":7034370970,"NUM_BIN_INT09":70343709708,"NUM_BIN_INT10":70343709708999957,"NUM_BIN_INT11":703437097089999577,"NUM_BIN_INT12":7034370970899995775,"NUM_BIN_INT13":70343709708999957759,"NUM_BIN_INT14":7034370970899995775972741521054695295,"NUM_SBIN_SINT01":-7,"NUM_SBIN_SINT02":-70,"NUM_SBIN_SINT03":-703,"NUM_SBIN_SINT04":-7034,"NUM_SBIN_SINT05":-70343,"NUM_SBIN_SINT06":-70343709,"NUM_SBIN_SINT07":-703437097,"NUM_SBIN_SINT08":-7034370970,"NUM_SBIN_SINT09":-70343709708,"NUM_SBIN_SINT10":-70343709708999957,"NUM_SBIN_SINT11":-703437097089999577,"NUM_SBIN_SINT12":-7034370970899995775,"NUM_SBIN_SINT13":-70343709708999957759,"NUM_SBIN_SINT14":-7034370970899995775972741521054695295,"NUM_BIN_DEC01":70.3,"NUM_BIN_DEC02":70.34,"NUM_BIN_DEC03":703.43,"NUM_BIN_DEC04":7034.3709,"NUM_BIN_DEC05":70343.7097,"NUM_BIN_DEC06":70343.70970,"NUM_BIN_DEC07":703437097089999.57,"NUM_BIN_DEC08":7034370970899995.77,"NUM_BIN_DEC09":70343709708999957.75,"NUM_BIN_DEC10":703437097089999577.5972741521,"NUM_SBIN_DEC01":-70.3,"NUM_SBIN_DEC02":-70.34,"NUM_SBIN_DEC03":-703.43,"NUM_SBIN_DEC04":-7034.3709,"NUM_SBIN_DEC05":-70343.7097,"NUM_SBIN_DEC06":-70343.70970,"NUM_SBIN_DEC07":-703437097089999.57,"NUM_SBIN_DEC08":-7034370970899995.77,"NUM_SBIN_DEC09":-70343709708999957.75,"NUM_SBIN_DEC10":-703437097089999577.5972741521,"NUM_BCD_INT01":7,"NUM_BCD_INT02":70,"NUM_BCD_INT03":703,"NUM_BCD_INT04":7034,"NUM_BCD_INT05":70343,"NUM_BCD_INT06":70343709,"NUM_BCD_INT07":703437097,"NUM_BCD_INT08":7034370970,"NUM_BCD_INT09":70343709708,"NUM_BCD_INT10":70343709708999957,"NUM_BCD_INT11":703437097089999577,"NUM_BCD_INT12":7034370970899995775,"NUM_BCD_INT13":70343709708999957759,"NUM_BCD_INT14":7034370970899995775972741521054695295,"NUM_BCD_SINT01":-7,"NUM_BCD_SINT02":-70,"NUM_BCD_SINT03":-703,"NUM_BCD_SINT04":-7034,"NUM_BCD_SINT05":-70343,"NUM_BCD_SINT06":-70343709,"NUM_BCD_SINT07":-703437097,"NUM_BCD_SINT08":-7034370970,"NUM_BCD_SINT09":-70343709708,"NUM_BCD_SINT10":-70343709708999957,"NUM_BCD_SINT11":-703437097089999577,"NUM_BCD_SINT12":-7034370970899995775,"NUM_BCD_SINT13":-70343709708999957759,"NUM_BCD_SINT14":-7034370970899995775972741521054695295,"NUM_BCD_DEC01":70.3,"NUM_BCD_DEC02":70.34,"NUM_BCD_DEC03":703.43,"NUM_BCD_DEC04":7034.3709,"NUM_BCD_DEC05":70343.7097,"NUM_BCD_DEC06":70343.70970,"NUM_BCD_DEC07":703437097089999.57,"NUM_BCD_DEC08":7034370970899995.77,"NUM_BCD_DEC09":70343709708999957.75,"NUM_BCD_DEC10":703437097089999577.5972741521,"NUM_BCD_SDEC01":-70.3,"NUM_BCD_SDEC02":-70.34,"NUM_BCD_SDEC03":-703.43,"NUM_BCD_SDEC04":-7034.3709,"NUM_BCD_SDEC05":-70343.7097,"NUM_BCD_SDEC06":-70343.70970,"NUM_BCD_SDEC07":-703437097089999.57,"NUM_BCD_SDEC08":-7034370970899995.77,"NUM_BCD_SDEC09":-70343709708999957.75,"NUM_BCD_SDEC10":-703437097089999577.5972741521,"NUM_SL_STR_INT01":-703437097,"NUM_SL_STR_DEC01":-70.34,"NUM_ST_STR_INT01":-703437097,"NUM_ST_STR_DEC01":-70.34,"NUM_SLI_STR_DEC01":-0.7034370,"NUM_STI_STR_DEC01":-0.7034370,"NUM_SLI_DEBUG":"P034370","NUM_STI_DEBUG":"703437}","FLOAT_01":-70343.7,"DOUBLE_01":-7.0343709708999E9,"COMMON_8_BIN":70343709,"COMMON_S3_BIN":703,"COMMON_S94COMP":7034,"COMMON_S8_BIN":70343709,"COMMON_DDC97_BIN":7.0343709,"COMMON_97COMP3":7034370,"COMMON_915COMP3":703437097089999,"COMMON_S95COMP3":-70343,"COMMON_S999DCCOMP3":-703437097.08,"COMMON_S913COMP3":-7034370970899,"COMMON_S913DCCOMP3":-7034370970899.99,"COMMON_S911DCC2":-70343709708.99,"COMMON_S910DCC3":-7034370970.899,"COMMON_S03DDC":-0.70343,"COMMON_U03DDC":0.70343,"COMMON_UPC5DDC":0.00070343,"COMMON_SPC5DDC":-0.0070343,"COMMON_UPI5DDC":70343000,"COMMON_SPI5DDC":-70343000,"COMMON_UPC5DISP":-0.00070343,"COMMON_UPI5DISP":-70343000,"COMMON_UPC1BIN":0.0007,"COMMON_UPI1BIN":7000,"COMMON_UPC3BIN":0.000703,"COMMON_UPI3BIN":703000,"COMMON_UPC5BIN":0.00070343,"COMMON_UPI5BIN":70343000,"COMMON_UPC10BIN":0.0007034370970,"COMMON_UPI10BIN":7034370970000,"EX_NUM_INT01":-70343709,"EX_NUM_INT02":-70343709,"EX_NUM_INT03":-70343709,"EX_NUM_INT04":-70343709,"EX_NUM_DEC01":-703437.09,"EX_NUM_DEC02":-703437.09,"EX_NUM_DEC03":-703437.09} +{"ID":20,"STRING_VAL":"Deandra","NUM_STR_INT01":3,"NUM_STR_INT02":33,"NUM_STR_INT03":334,"NUM_STR_INT04":3346,"NUM_STR_INT05":33461,"NUM_STR_INT06":33461507,"NUM_STR_INT07":334615074,"NUM_STR_INT08":3346150748,"NUM_STR_INT09":33461507485,"NUM_STR_INT10":33461507485699982,"NUM_STR_INT11":334615074856999820,"NUM_STR_INT12":3346150748569998207,"NUM_STR_INT13":33461507485699982075,"NUM_STR_INT14":3346150748569998207570469608763829614,"NUM_STR_SINT02":33,"NUM_STR_SINT03":334,"NUM_STR_SINT04":3346,"NUM_STR_SINT05":33461,"NUM_STR_SINT06":33461507,"NUM_STR_SINT07":334615074,"NUM_STR_SINT08":3346150748,"NUM_STR_SINT09":33461507485,"NUM_STR_SINT10":33461507485699982,"NUM_STR_SINT11":334615074856999820,"NUM_STR_SINT12":3346150748569998207,"NUM_STR_SINT13":33461507485699982075,"NUM_STR_SINT14":3346150748569998207570469608763829614,"NUM_STR_DEC01":33.4,"NUM_STR_DEC02":33.46,"NUM_STR_DEC03":334.61,"NUM_STR_DEC04":3346.1507,"NUM_STR_DEC05":33461.5074,"NUM_STR_DEC06":33461.50748,"NUM_STR_DEC07":334615074856999.82,"NUM_STR_DEC08":3346150748569998.20,"NUM_STR_DEC09":33461507485699982.07,"NUM_STR_DEC10":334615074856999820.7570469608,"NUM_STR_SDEC01":33.4,"NUM_STR_SDEC02":33.46,"NUM_STR_SDEC03":334.61,"NUM_STR_SDEC04":3346.1507,"NUM_STR_SDEC05":33461.5074,"NUM_STR_SDEC06":33461.50748,"NUM_STR_SDEC07":334615074856999.82,"NUM_STR_SDEC08":3346150748569998.20,"NUM_STR_SDEC09":33461507485699982.07,"NUM_STR_SDEC10":334615074856999820.7570469608,"NUM_STR_EDEC03":334.61,"NUM_STR_EDEC04":3346.1507,"NUM_STR_EDEC05":33461.5074,"NUM_STR_EDEC06":33461.50748,"NUM_BIN_INT01":3,"NUM_BIN_INT02":33,"NUM_BIN_INT03":334,"NUM_BIN_INT04":3346,"NUM_BIN_INT05":33461,"NUM_BIN_INT06":33461507,"NUM_BIN_INT07":334615074,"NUM_BIN_INT08":3346150748,"NUM_BIN_INT09":33461507485,"NUM_BIN_INT10":33461507485699982,"NUM_BIN_INT11":334615074856999820,"NUM_BIN_INT12":3346150748569998207,"NUM_BIN_INT13":33461507485699982075,"NUM_BIN_INT14":3346150748569998207570469608763829614,"NUM_SBIN_SINT01":3,"NUM_SBIN_SINT02":33,"NUM_SBIN_SINT03":334,"NUM_SBIN_SINT04":3346,"NUM_SBIN_SINT05":33461,"NUM_SBIN_SINT06":33461507,"NUM_SBIN_SINT07":334615074,"NUM_SBIN_SINT08":3346150748,"NUM_SBIN_SINT09":33461507485,"NUM_SBIN_SINT10":33461507485699982,"NUM_SBIN_SINT11":334615074856999820,"NUM_SBIN_SINT12":3346150748569998207,"NUM_SBIN_SINT13":33461507485699982075,"NUM_SBIN_SINT14":3346150748569998207570469608763829614,"NUM_BIN_DEC01":33.4,"NUM_BIN_DEC02":33.46,"NUM_BIN_DEC03":334.61,"NUM_BIN_DEC04":3346.1507,"NUM_BIN_DEC05":33461.5074,"NUM_BIN_DEC06":33461.50748,"NUM_BIN_DEC07":334615074856999.82,"NUM_BIN_DEC08":3346150748569998.20,"NUM_BIN_DEC09":33461507485699982.07,"NUM_BIN_DEC10":334615074856999820.7570469608,"NUM_SBIN_DEC01":33.4,"NUM_SBIN_DEC02":33.46,"NUM_SBIN_DEC03":334.61,"NUM_SBIN_DEC04":3346.1507,"NUM_SBIN_DEC05":33461.5074,"NUM_SBIN_DEC06":33461.50748,"NUM_SBIN_DEC07":334615074856999.82,"NUM_SBIN_DEC08":3346150748569998.20,"NUM_SBIN_DEC09":33461507485699982.07,"NUM_SBIN_DEC10":334615074856999820.7570469608,"NUM_BCD_INT01":3,"NUM_BCD_INT02":33,"NUM_BCD_INT03":334,"NUM_BCD_INT04":3346,"NUM_BCD_INT05":33461,"NUM_BCD_INT06":33461507,"NUM_BCD_INT07":334615074,"NUM_BCD_INT08":3346150748,"NUM_BCD_INT09":33461507485,"NUM_BCD_INT10":33461507485699982,"NUM_BCD_INT11":334615074856999820,"NUM_BCD_INT12":3346150748569998207,"NUM_BCD_INT13":33461507485699982075,"NUM_BCD_INT14":3346150748569998207570469608763829614,"NUM_BCD_SINT01":3,"NUM_BCD_SINT02":33,"NUM_BCD_SINT03":334,"NUM_BCD_SINT04":3346,"NUM_BCD_SINT05":33461,"NUM_BCD_SINT06":33461507,"NUM_BCD_SINT07":334615074,"NUM_BCD_SINT08":3346150748,"NUM_BCD_SINT09":33461507485,"NUM_BCD_SINT10":33461507485699982,"NUM_BCD_SINT11":334615074856999820,"NUM_BCD_SINT12":3346150748569998207,"NUM_BCD_SINT13":33461507485699982075,"NUM_BCD_SINT14":3346150748569998207570469608763829614,"NUM_BCD_DEC01":33.4,"NUM_BCD_DEC02":33.46,"NUM_BCD_DEC03":334.61,"NUM_BCD_DEC04":3346.1507,"NUM_BCD_DEC05":33461.5074,"NUM_BCD_DEC06":33461.50748,"NUM_BCD_DEC07":334615074856999.82,"NUM_BCD_DEC08":3346150748569998.20,"NUM_BCD_DEC09":33461507485699982.07,"NUM_BCD_DEC10":334615074856999820.7570469608,"NUM_BCD_SDEC01":33.4,"NUM_BCD_SDEC02":33.46,"NUM_BCD_SDEC03":334.61,"NUM_BCD_SDEC04":3346.1507,"NUM_BCD_SDEC05":33461.5074,"NUM_BCD_SDEC06":33461.50748,"NUM_BCD_SDEC07":334615074856999.82,"NUM_BCD_SDEC08":3346150748569998.20,"NUM_BCD_SDEC09":33461507485699982.07,"NUM_BCD_SDEC10":334615074856999820.7570469608,"NUM_SL_STR_INT01":334615074,"NUM_SL_STR_DEC01":33.46,"NUM_ST_STR_INT01":334615074,"NUM_ST_STR_DEC01":33.46,"NUM_SLI_STR_DEC01":0.3346150,"NUM_STI_STR_DEC01":0.3346150,"NUM_SLI_DEBUG":"C346150","NUM_STI_DEBUG":"334615{","FLOAT_01":33461.5,"DOUBLE_01":3.3461507485699E9,"COMMON_8_BIN":33461507,"COMMON_S3_BIN":334,"COMMON_S94COMP":3346,"COMMON_S8_BIN":33461507,"COMMON_DDC97_BIN":3.3461507,"COMMON_97COMP3":3346150,"COMMON_915COMP3":334615074856999,"COMMON_S95COMP3":33461,"COMMON_S999DCCOMP3":334615074.85,"COMMON_S913COMP3":3346150748569,"COMMON_S913DCCOMP3":3346150748569.99,"COMMON_S911DCC2":33461507485.69,"COMMON_S910DCC3":3346150748.569,"COMMON_S03DDC":0.33461,"COMMON_U03DDC":0.33461,"COMMON_UPC5DDC":0.00033461,"COMMON_SPC5DDC":0.0033461,"COMMON_UPI5DDC":33461000,"COMMON_SPI5DDC":33461000,"COMMON_UPC5DISP":0.00033461,"COMMON_UPI5DISP":33461000,"COMMON_UPC1BIN":0.0003,"COMMON_UPI1BIN":3000,"COMMON_UPC3BIN":0.000334,"COMMON_UPI3BIN":334000,"COMMON_UPC5BIN":0.00033461,"COMMON_UPI5BIN":33461000,"COMMON_UPC10BIN":0.0003346150748,"COMMON_UPI10BIN":3346150748000,"EX_NUM_INT01":33461507,"EX_NUM_INT02":33461507,"EX_NUM_INT03":33461507,"EX_NUM_INT04":33461507,"EX_NUM_DEC01":334615.07,"EX_NUM_DEC02":334615.07,"EX_NUM_DEC03":334615.07} diff --git a/data/test40_expected/test40_layout.txt b/data/test40_expected/test40_layout.txt new file mode 100644 index 000000000..fb99eec58 --- /dev/null +++ b/data/test40_expected/test40_layout.txt @@ -0,0 +1,393 @@ +-------- FIELD LEVEL/NAME --------- --ATTRIBS-- FLD START END LENGTH + +1 RECORD 1 1 1493 1493 + 10 ID r 2 1 4 4 + 10 ID_debug R 3 1 4 4 + 10 STRING_VAL r 4 5 14 10 + 10 STRING_VAL_debug R 5 5 14 10 + 10 NUM_STR_INT01 r 6 15 15 1 + 10 NUM_STR_INT01_debug R 7 15 15 1 + 10 NUM_STR_INT02 r 8 16 17 2 + 10 NUM_STR_INT02_debug R 9 16 17 2 + 10 NUM_STR_INT03 r 10 18 20 3 + 10 NUM_STR_INT03_debug R 11 18 20 3 + 10 NUM_STR_INT04 r 12 21 24 4 + 10 NUM_STR_INT04_debug R 13 21 24 4 + 10 NUM_STR_INT05 r 14 25 29 5 + 10 NUM_STR_INT05_debug R 15 25 29 5 + 10 NUM_STR_INT06 r 16 30 37 8 + 10 NUM_STR_INT06_debug R 17 30 37 8 + 10 NUM_STR_INT07 r 18 38 46 9 + 10 NUM_STR_INT07_debug R 19 38 46 9 + 10 NUM_STR_INT08 r 20 47 56 10 + 10 NUM_STR_INT08_debug R 21 47 56 10 + 10 NUM_STR_INT09 r 22 57 67 11 + 10 NUM_STR_INT09_debug R 23 57 67 11 + 10 NUM_STR_INT10 r 24 68 84 17 + 10 NUM_STR_INT10_debug R 25 68 84 17 + 10 NUM_STR_INT11 r 26 85 102 18 + 10 NUM_STR_INT11_debug R 27 85 102 18 + 10 NUM_STR_INT12 r 28 103 121 19 + 10 NUM_STR_INT12_debug R 29 103 121 19 + 10 NUM_STR_INT13 r 30 122 141 20 + 10 NUM_STR_INT13_debug R 31 122 141 20 + 10 NUM_STR_INT14 r 32 142 178 37 + 10 NUM_STR_INT14_debug R 33 142 178 37 + 10 NUM_STR_SINT02 r 34 179 180 2 + 10 NUM_STR_SINT02_debug R 35 179 180 2 + 10 NUM_STR_SINT03 r 36 181 183 3 + 10 NUM_STR_SINT03_debug R 37 181 183 3 + 10 NUM_STR_SINT04 r 38 184 187 4 + 10 NUM_STR_SINT04_debug R 39 184 187 4 + 10 NUM_STR_SINT05 r 40 188 192 5 + 10 NUM_STR_SINT05_debug R 41 188 192 5 + 10 NUM_STR_SINT06 r 42 193 200 8 + 10 NUM_STR_SINT06_debug R 43 193 200 8 + 10 NUM_STR_SINT07 r 44 201 209 9 + 10 NUM_STR_SINT07_debug R 45 201 209 9 + 10 NUM_STR_SINT08 r 46 210 219 10 + 10 NUM_STR_SINT08_debug R 47 210 219 10 + 10 NUM_STR_SINT09 r 48 220 230 11 + 10 NUM_STR_SINT09_debug R 49 220 230 11 + 10 NUM_STR_SINT10 r 50 231 247 17 + 10 NUM_STR_SINT10_debug R 51 231 247 17 + 10 NUM_STR_SINT11 r 52 248 265 18 + 10 NUM_STR_SINT11_debug R 53 248 265 18 + 10 NUM_STR_SINT12 r 54 266 284 19 + 10 NUM_STR_SINT12_debug R 55 266 284 19 + 10 NUM_STR_SINT13 r 56 285 304 20 + 10 NUM_STR_SINT13_debug R 57 285 304 20 + 10 NUM_STR_SINT14 r 58 305 341 37 + 10 NUM_STR_SINT14_debug R 59 305 341 37 + 10 NUM_STR_DEC01 r 60 342 344 3 + 10 NUM_STR_DEC01_debug R 61 342 344 3 + 10 NUM_STR_DEC02 r 62 345 348 4 + 10 NUM_STR_DEC02_debug R 63 345 348 4 + 10 NUM_STR_DEC03 r 64 349 353 5 + 10 NUM_STR_DEC03_debug R 65 349 353 5 + 10 NUM_STR_DEC04 r 66 354 361 8 + 10 NUM_STR_DEC04_debug R 67 354 361 8 + 10 NUM_STR_DEC05 r 68 362 370 9 + 10 NUM_STR_DEC05_debug R 69 362 370 9 + 10 NUM_STR_DEC06 r 70 371 380 10 + 10 NUM_STR_DEC06_debug R 71 371 380 10 + 10 NUM_STR_DEC07 r 72 381 397 17 + 10 NUM_STR_DEC07_debug R 73 381 397 17 + 10 NUM_STR_DEC08 r 74 398 415 18 + 10 NUM_STR_DEC08_debug R 75 398 415 18 + 10 NUM_STR_DEC09 r 76 416 434 19 + 10 NUM_STR_DEC09_debug R 77 416 434 19 + 10 NUM_STR_DEC10 r 78 435 462 28 + 10 NUM_STR_DEC10_debug R 79 435 462 28 + 10 NUM_STR_SDEC01 r 80 463 465 3 + 10 NUM_STR_SDEC01_debug R 81 463 465 3 + 10 NUM_STR_SDEC02 r 82 466 469 4 + 10 NUM_STR_SDEC02_debug R 83 466 469 4 + 10 NUM_STR_SDEC03 r 84 470 474 5 + 10 NUM_STR_SDEC03_debug R 85 470 474 5 + 10 NUM_STR_SDEC04 r 86 475 482 8 + 10 NUM_STR_SDEC04_debug R 87 475 482 8 + 10 NUM_STR_SDEC05 r 88 483 491 9 + 10 NUM_STR_SDEC05_debug R 89 483 491 9 + 10 NUM_STR_SDEC06 r 90 492 501 10 + 10 NUM_STR_SDEC06_debug R 91 492 501 10 + 10 NUM_STR_SDEC07 r 92 502 518 17 + 10 NUM_STR_SDEC07_debug R 93 502 518 17 + 10 NUM_STR_SDEC08 r 94 519 536 18 + 10 NUM_STR_SDEC08_debug R 95 519 536 18 + 10 NUM_STR_SDEC09 r 96 537 555 19 + 10 NUM_STR_SDEC09_debug R 97 537 555 19 + 10 NUM_STR_SDEC10 r 98 556 583 28 + 10 NUM_STR_SDEC10_debug R 99 556 583 28 + 10 NUM_STR_EDEC03 r 100 584 589 6 + 10 NUM_STR_EDEC03_debug R 101 584 589 6 + 10 NUM_STR_EDEC04 r 102 590 598 9 + 10 NUM_STR_EDEC04_debug R 103 590 598 9 + 10 NUM_STR_EDEC05 r 104 599 608 10 + 10 NUM_STR_EDEC05_debug R 105 599 608 10 + 10 NUM_STR_EDEC06 r 106 609 619 11 + 10 NUM_STR_EDEC06_debug R 107 609 619 11 + 10 NUM_BIN_INT01 r 108 620 621 2 + 10 NUM_BIN_INT01_debug R 109 620 621 2 + 10 NUM_BIN_INT02 r 110 622 623 2 + 10 NUM_BIN_INT02_debug R 111 622 623 2 + 10 NUM_BIN_INT03 r 112 624 625 2 + 10 NUM_BIN_INT03_debug R 113 624 625 2 + 10 NUM_BIN_INT04 r 114 626 627 2 + 10 NUM_BIN_INT04_debug R 115 626 627 2 + 10 NUM_BIN_INT05 r 116 628 631 4 + 10 NUM_BIN_INT05_debug R 117 628 631 4 + 10 NUM_BIN_INT06 r 118 632 635 4 + 10 NUM_BIN_INT06_debug R 119 632 635 4 + 10 NUM_BIN_INT07 r 120 636 639 4 + 10 NUM_BIN_INT07_debug R 121 636 639 4 + 10 NUM_BIN_INT08 r 122 640 647 8 + 10 NUM_BIN_INT08_debug R 123 640 647 8 + 10 NUM_BIN_INT09 r 124 648 655 8 + 10 NUM_BIN_INT09_debug R 125 648 655 8 + 10 NUM_BIN_INT10 r 126 656 663 8 + 10 NUM_BIN_INT10_debug R 127 656 663 8 + 10 NUM_BIN_INT11 r 128 664 671 8 + 10 NUM_BIN_INT11_debug R 129 664 671 8 + 10 NUM_BIN_INT12 r 130 672 680 9 + 10 NUM_BIN_INT12_debug R 131 672 680 9 + 10 NUM_BIN_INT13 r 132 681 689 9 + 10 NUM_BIN_INT13_debug R 133 681 689 9 + 10 NUM_BIN_INT14 r 134 690 705 16 + 10 NUM_BIN_INT14_debug R 135 690 705 16 + 10 NUM_SBIN_SINT01 r 136 706 707 2 + 10 NUM_SBIN_SINT01_debug R 137 706 707 2 + 10 NUM_SBIN_SINT02 r 138 708 709 2 + 10 NUM_SBIN_SINT02_debug R 139 708 709 2 + 10 NUM_SBIN_SINT03 r 140 710 711 2 + 10 NUM_SBIN_SINT03_debug R 141 710 711 2 + 10 NUM_SBIN_SINT04 r 142 712 713 2 + 10 NUM_SBIN_SINT04_debug R 143 712 713 2 + 10 NUM_SBIN_SINT05 r 144 714 717 4 + 10 NUM_SBIN_SINT05_debug R 145 714 717 4 + 10 NUM_SBIN_SINT06 r 146 718 721 4 + 10 NUM_SBIN_SINT06_debug R 147 718 721 4 + 10 NUM_SBIN_SINT07 r 148 722 725 4 + 10 NUM_SBIN_SINT07_debug R 149 722 725 4 + 10 NUM_SBIN_SINT08 r 150 726 733 8 + 10 NUM_SBIN_SINT08_debug R 151 726 733 8 + 10 NUM_SBIN_SINT09 r 152 734 741 8 + 10 NUM_SBIN_SINT09_debug R 153 734 741 8 + 10 NUM_SBIN_SINT10 r 154 742 749 8 + 10 NUM_SBIN_SINT10_debug R 155 742 749 8 + 10 NUM_SBIN_SINT11 r 156 750 757 8 + 10 NUM_SBIN_SINT11_debug R 157 750 757 8 + 10 NUM_SBIN_SINT12 r 158 758 766 9 + 10 NUM_SBIN_SINT12_debug R 159 758 766 9 + 10 NUM_SBIN_SINT13 r 160 767 775 9 + 10 NUM_SBIN_SINT13_debug R 161 767 775 9 + 10 NUM_SBIN_SINT14 r 162 776 791 16 + 10 NUM_SBIN_SINT14_debug R 163 776 791 16 + 10 NUM_BIN_DEC01 r 164 792 793 2 + 10 NUM_BIN_DEC01_debug R 165 792 793 2 + 10 NUM_BIN_DEC02 r 166 794 795 2 + 10 NUM_BIN_DEC02_debug R 167 794 795 2 + 10 NUM_BIN_DEC03 r 168 796 799 4 + 10 NUM_BIN_DEC03_debug R 169 796 799 4 + 10 NUM_BIN_DEC04 r 170 800 803 4 + 10 NUM_BIN_DEC04_debug R 171 800 803 4 + 10 NUM_BIN_DEC05 r 172 804 807 4 + 10 NUM_BIN_DEC05_debug R 173 804 807 4 + 10 NUM_BIN_DEC06 r 174 808 815 8 + 10 NUM_BIN_DEC06_debug R 175 808 815 8 + 10 NUM_BIN_DEC07 r 176 816 823 8 + 10 NUM_BIN_DEC07_debug R 177 816 823 8 + 10 NUM_BIN_DEC08 r 178 824 831 8 + 10 NUM_BIN_DEC08_debug R 179 824 831 8 + 10 NUM_BIN_DEC09 r 180 832 840 9 + 10 NUM_BIN_DEC09_debug R 181 832 840 9 + 10 NUM_BIN_DEC10 r 182 841 852 12 + 10 NUM_BIN_DEC10_debug R 183 841 852 12 + 10 NUM_SBIN_DEC01 r 184 853 854 2 + 10 NUM_SBIN_DEC01_debug R 185 853 854 2 + 10 NUM_SBIN_DEC02 r 186 855 856 2 + 10 NUM_SBIN_DEC02_debug R 187 855 856 2 + 10 NUM_SBIN_DEC03 r 188 857 860 4 + 10 NUM_SBIN_DEC03_debug R 189 857 860 4 + 10 NUM_SBIN_DEC04 r 190 861 864 4 + 10 NUM_SBIN_DEC04_debug R 191 861 864 4 + 10 NUM_SBIN_DEC05 r 192 865 868 4 + 10 NUM_SBIN_DEC05_debug R 193 865 868 4 + 10 NUM_SBIN_DEC06 r 194 869 876 8 + 10 NUM_SBIN_DEC06_debug R 195 869 876 8 + 10 NUM_SBIN_DEC07 r 196 877 884 8 + 10 NUM_SBIN_DEC07_debug R 197 877 884 8 + 10 NUM_SBIN_DEC08 r 198 885 892 8 + 10 NUM_SBIN_DEC08_debug R 199 885 892 8 + 10 NUM_SBIN_DEC09 r 200 893 901 9 + 10 NUM_SBIN_DEC09_debug R 201 893 901 9 + 10 NUM_SBIN_DEC10 r 202 902 913 12 + 10 NUM_SBIN_DEC10_debug R 203 902 913 12 + 10 NUM_BCD_INT01 r 204 914 914 1 + 10 NUM_BCD_INT01_debug R 205 914 914 1 + 10 NUM_BCD_INT02 r 206 915 916 2 + 10 NUM_BCD_INT02_debug R 207 915 916 2 + 10 NUM_BCD_INT03 r 208 917 918 2 + 10 NUM_BCD_INT03_debug R 209 917 918 2 + 10 NUM_BCD_INT04 r 210 919 921 3 + 10 NUM_BCD_INT04_debug R 211 919 921 3 + 10 NUM_BCD_INT05 r 212 922 924 3 + 10 NUM_BCD_INT05_debug R 213 922 924 3 + 10 NUM_BCD_INT06 r 214 925 929 5 + 10 NUM_BCD_INT06_debug R 215 925 929 5 + 10 NUM_BCD_INT07 r 216 930 934 5 + 10 NUM_BCD_INT07_debug R 217 930 934 5 + 10 NUM_BCD_INT08 r 218 935 940 6 + 10 NUM_BCD_INT08_debug R 219 935 940 6 + 10 NUM_BCD_INT09 r 220 941 946 6 + 10 NUM_BCD_INT09_debug R 221 941 946 6 + 10 NUM_BCD_INT10 r 222 947 955 9 + 10 NUM_BCD_INT10_debug R 223 947 955 9 + 10 NUM_BCD_INT11 r 224 956 965 10 + 10 NUM_BCD_INT11_debug R 225 956 965 10 + 10 NUM_BCD_INT12 r 226 966 975 10 + 10 NUM_BCD_INT12_debug R 227 966 975 10 + 10 NUM_BCD_INT13 r 228 976 986 11 + 10 NUM_BCD_INT13_debug R 229 976 986 11 + 10 NUM_BCD_INT14 r 230 987 1005 19 + 10 NUM_BCD_INT14_debug R 231 987 1005 19 + 10 NUM_BCD_SINT01 r 232 1006 1006 1 + 10 NUM_BCD_SINT01_debug R 233 1006 1006 1 + 10 NUM_BCD_SINT02 r 234 1007 1008 2 + 10 NUM_BCD_SINT02_debug R 235 1007 1008 2 + 10 NUM_BCD_SINT03 r 236 1009 1010 2 + 10 NUM_BCD_SINT03_debug R 237 1009 1010 2 + 10 NUM_BCD_SINT04 r 238 1011 1013 3 + 10 NUM_BCD_SINT04_debug R 239 1011 1013 3 + 10 NUM_BCD_SINT05 r 240 1014 1016 3 + 10 NUM_BCD_SINT05_debug R 241 1014 1016 3 + 10 NUM_BCD_SINT06 r 242 1017 1021 5 + 10 NUM_BCD_SINT06_debug R 243 1017 1021 5 + 10 NUM_BCD_SINT07 r 244 1022 1026 5 + 10 NUM_BCD_SINT07_debug R 245 1022 1026 5 + 10 NUM_BCD_SINT08 r 246 1027 1032 6 + 10 NUM_BCD_SINT08_debug R 247 1027 1032 6 + 10 NUM_BCD_SINT09 r 248 1033 1038 6 + 10 NUM_BCD_SINT09_debug R 249 1033 1038 6 + 10 NUM_BCD_SINT10 r 250 1039 1047 9 + 10 NUM_BCD_SINT10_debug R 251 1039 1047 9 + 10 NUM_BCD_SINT11 r 252 1048 1057 10 + 10 NUM_BCD_SINT11_debug R 253 1048 1057 10 + 10 NUM_BCD_SINT12 r 254 1058 1067 10 + 10 NUM_BCD_SINT12_debug R 255 1058 1067 10 + 10 NUM_BCD_SINT13 r 256 1068 1078 11 + 10 NUM_BCD_SINT13_debug R 257 1068 1078 11 + 10 NUM_BCD_SINT14 r 258 1079 1097 19 + 10 NUM_BCD_SINT14_debug R 259 1079 1097 19 + 10 NUM_BCD_DEC01 r 260 1098 1099 2 + 10 NUM_BCD_DEC01_debug R 261 1098 1099 2 + 10 NUM_BCD_DEC02 r 262 1100 1102 3 + 10 NUM_BCD_DEC02_debug R 263 1100 1102 3 + 10 NUM_BCD_DEC03 r 264 1103 1105 3 + 10 NUM_BCD_DEC03_debug R 265 1103 1105 3 + 10 NUM_BCD_DEC04 r 266 1106 1110 5 + 10 NUM_BCD_DEC04_debug R 267 1106 1110 5 + 10 NUM_BCD_DEC05 r 268 1111 1115 5 + 10 NUM_BCD_DEC05_debug R 269 1111 1115 5 + 10 NUM_BCD_DEC06 r 270 1116 1121 6 + 10 NUM_BCD_DEC06_debug R 271 1116 1121 6 + 10 NUM_BCD_DEC07 r 272 1122 1130 9 + 10 NUM_BCD_DEC07_debug R 273 1122 1130 9 + 10 NUM_BCD_DEC08 r 274 1131 1140 10 + 10 NUM_BCD_DEC08_debug R 275 1131 1140 10 + 10 NUM_BCD_DEC09 r 276 1141 1150 10 + 10 NUM_BCD_DEC09_debug R 277 1141 1150 10 + 10 NUM_BCD_DEC10 r 278 1151 1165 15 + 10 NUM_BCD_DEC10_debug R 279 1151 1165 15 + 10 NUM_BCD_SDEC01 r 280 1166 1167 2 + 10 NUM_BCD_SDEC01_debug R 281 1166 1167 2 + 10 NUM_BCD_SDEC02 r 282 1168 1170 3 + 10 NUM_BCD_SDEC02_debug R 283 1168 1170 3 + 10 NUM_BCD_SDEC03 r 284 1171 1173 3 + 10 NUM_BCD_SDEC03_debug R 285 1171 1173 3 + 10 NUM_BCD_SDEC04 r 286 1174 1178 5 + 10 NUM_BCD_SDEC04_debug R 287 1174 1178 5 + 10 NUM_BCD_SDEC05 r 288 1179 1183 5 + 10 NUM_BCD_SDEC05_debug R 289 1179 1183 5 + 10 NUM_BCD_SDEC06 r 290 1184 1189 6 + 10 NUM_BCD_SDEC06_debug R 291 1184 1189 6 + 10 NUM_BCD_SDEC07 r 292 1190 1198 9 + 10 NUM_BCD_SDEC07_debug R 293 1190 1198 9 + 10 NUM_BCD_SDEC08 r 294 1199 1208 10 + 10 NUM_BCD_SDEC08_debug R 295 1199 1208 10 + 10 NUM_BCD_SDEC09 r 296 1209 1218 10 + 10 NUM_BCD_SDEC09_debug R 297 1209 1218 10 + 10 NUM_BCD_SDEC10 r 298 1219 1233 15 + 10 NUM_BCD_SDEC10_debug R 299 1219 1233 15 + 10 NUM_SL_STR_INT01 r 300 1234 1243 10 + 10 NUM_SL_STR_INT01_debug R 301 1234 1243 10 + 10 NUM_SL_STR_DEC01 r 302 1244 1248 5 + 10 NUM_SL_STR_DEC01_debug R 303 1244 1248 5 + 10 NUM_ST_STR_INT01 r 304 1249 1258 10 + 10 NUM_ST_STR_INT01_debug R 305 1249 1258 10 + 10 NUM_ST_STR_DEC01 r 306 1259 1263 5 + 10 NUM_ST_STR_DEC01_debug R 307 1259 1263 5 + 10 NUM_SLI_STR_DEC01 r 308 1264 1270 7 + 10 NUM_SLI_STR_DEC01_debug R 309 1264 1270 7 + 10 NUM_STI_STR_DEC01 r 310 1271 1277 7 + 10 NUM_STI_STR_DEC01_debug R 311 1271 1277 7 + 10 NUM_SLI_DEBUG r 312 1278 1284 7 + 10 NUM_SLI_DEBUG_debug R 313 1278 1284 7 + 10 NUM_STI_DEBUG r 314 1285 1291 7 + 10 NUM_STI_DEBUG_debug R 315 1285 1291 7 + 10 FLOAT_01 r 316 1292 1295 4 + 10 FLOAT_01_debug R 317 1292 1295 4 + 10 DOUBLE_01 r 318 1296 1303 8 + 10 DOUBLE_01_debug R 319 1296 1303 8 + 10 COMMON_8_BIN r 320 1304 1307 4 + 10 COMMON_8_BIN_debug R 321 1304 1307 4 + 10 COMMON_S3_BIN r 322 1308 1309 2 + 10 COMMON_S3_BIN_debug R 323 1308 1309 2 + 10 COMMON_S94COMP r 324 1310 1311 2 + 10 COMMON_S94COMP_debug R 325 1310 1311 2 + 10 COMMON_S8_BIN r 326 1312 1315 4 + 10 COMMON_S8_BIN_debug R 327 1312 1315 4 + 10 COMMON_DDC97_BIN r 328 1316 1319 4 + 10 COMMON_DDC97_BIN_debug R 329 1316 1319 4 + 10 COMMON_97COMP3 r 330 1320 1323 4 + 10 COMMON_97COMP3_debug R 331 1320 1323 4 + 10 COMMON_915COMP3 r 332 1324 1331 8 + 10 COMMON_915COMP3_debug R 333 1324 1331 8 + 10 COMMON_S95COMP3 r 334 1332 1334 3 + 10 COMMON_S95COMP3_debug R 335 1332 1334 3 + 10 COMMON_S999DCCOMP3 r 336 1335 1340 6 + 10 COMMON_S999DCCOMP3_debug R 337 1335 1340 6 + 10 COMMON_S913COMP3 r 338 1341 1347 7 + 10 COMMON_S913COMP3_debug R 339 1341 1347 7 + 10 COMMON_S913DCCOMP3 r 340 1348 1355 8 + 10 COMMON_S913DCCOMP3_debug R 341 1348 1355 8 + 10 COMMON_S911DCC2 r 342 1356 1362 7 + 10 COMMON_S911DCC2_debug R 343 1356 1362 7 + 10 COMMON_S910DCC3 r 344 1363 1369 7 + 10 COMMON_S910DCC3_debug R 345 1363 1369 7 + 10 COMMON_S03DDC r 346 1370 1372 3 + 10 COMMON_S03DDC_debug R 347 1370 1372 3 + 10 COMMON_U03DDC r 348 1373 1375 3 + 10 COMMON_U03DDC_debug R 349 1373 1375 3 + 10 COMMON_UPC5DDC r 350 1376 1378 3 + 10 COMMON_UPC5DDC_debug R 351 1376 1378 3 + 10 COMMON_SPC5DDC r 352 1379 1381 3 + 10 COMMON_SPC5DDC_debug R 353 1379 1381 3 + 10 COMMON_UPI5DDC r 354 1382 1384 3 + 10 COMMON_UPI5DDC_debug R 355 1382 1384 3 + 10 COMMON_SPI5DDC r 356 1385 1387 3 + 10 COMMON_SPI5DDC_debug R 357 1385 1387 3 + 10 COMMON_UPC5DISP r 358 1388 1392 5 + 10 COMMON_UPC5DISP_debug R 359 1388 1392 5 + 10 COMMON_UPI5DISP r 360 1393 1397 5 + 10 COMMON_UPI5DISP_debug R 361 1393 1397 5 + 10 COMMON_UPC1BIN r 362 1398 1399 2 + 10 COMMON_UPC1BIN_debug R 363 1398 1399 2 + 10 COMMON_UPI1BIN r 364 1400 1401 2 + 10 COMMON_UPI1BIN_debug R 365 1400 1401 2 + 10 COMMON_UPC3BIN r 366 1402 1403 2 + 10 COMMON_UPC3BIN_debug R 367 1402 1403 2 + 10 COMMON_UPI3BIN r 368 1404 1405 2 + 10 COMMON_UPI3BIN_debug R 369 1404 1405 2 + 10 COMMON_UPC5BIN r 370 1406 1409 4 + 10 COMMON_UPC5BIN_debug R 371 1406 1409 4 + 10 COMMON_UPI5BIN r 372 1410 1413 4 + 10 COMMON_UPI5BIN_debug R 373 1410 1413 4 + 10 COMMON_UPC10BIN r 374 1414 1421 8 + 10 COMMON_UPC10BIN_debug R 375 1414 1421 8 + 10 COMMON_UPI10BIN r 376 1422 1429 8 + 10 COMMON_UPI10BIN_debug R 377 1422 1429 8 + 10 EX_NUM_INT01 r 378 1430 1438 9 + 10 EX_NUM_INT01_debug R 379 1430 1438 9 + 10 EX_NUM_INT02 r 380 1439 1447 9 + 10 EX_NUM_INT02_debug R 381 1439 1447 9 + 10 EX_NUM_INT03 r 382 1448 1456 9 + 10 EX_NUM_INT03_debug R 383 1448 1456 9 + 10 EX_NUM_INT04 r 384 1457 1465 9 + 10 EX_NUM_INT04_debug R 385 1457 1465 9 + 10 EX_NUM_DEC01 r 386 1466 1474 9 + 10 EX_NUM_DEC01_debug R 387 1466 1474 9 + 10 EX_NUM_DEC02 r 388 1475 1483 9 + 10 EX_NUM_DEC02_debug R 389 1475 1483 9 + 10 EX_NUM_DEC03 r 390 1484 1493 10 + 10 EX_NUM_DEC03_debug R 391 1484 1493 10 \ No newline at end of file diff --git a/data/test40_expected/test40_schema.json b/data/test40_expected/test40_schema.json new file mode 100644 index 000000000..ca2d7eb23 --- /dev/null +++ b/data/test40_expected/test40_schema.json @@ -0,0 +1,985 @@ +{ + "type" : "struct", + "fields" : [ { + "name" : "ID", + "type" : "integer", + "nullable" : true, + "metadata" : { } + }, { + "name" : "STRING_VAL", + "type" : "string", + "nullable" : true, + "metadata" : { + "maxLength" : 10 + } + }, { + "name" : "NUM_STR_INT01", + "type" : "integer", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_STR_INT02", + "type" : "integer", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_STR_INT03", + "type" : "integer", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_STR_INT04", + "type" : "integer", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_STR_INT05", + "type" : "integer", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_STR_INT06", + "type" : "integer", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_STR_INT07", + "type" : "integer", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_STR_INT08", + "type" : "long", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_STR_INT09", + "type" : "long", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_STR_INT10", + "type" : "long", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_STR_INT11", + "type" : "long", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_STR_INT12", + "type" : "decimal(19,0)", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_STR_INT13", + "type" : "decimal(20,0)", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_STR_INT14", + "type" : "decimal(37,0)", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_STR_SINT02", + "type" : "integer", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_STR_SINT03", + "type" : "integer", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_STR_SINT04", + "type" : "integer", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_STR_SINT05", + "type" : "integer", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_STR_SINT06", + "type" : "integer", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_STR_SINT07", + "type" : "integer", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_STR_SINT08", + "type" : "long", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_STR_SINT09", + "type" : "long", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_STR_SINT10", + "type" : "long", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_STR_SINT11", + "type" : "long", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_STR_SINT12", + "type" : "decimal(19,0)", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_STR_SINT13", + "type" : "decimal(20,0)", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_STR_SINT14", + "type" : "decimal(37,0)", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_STR_DEC01", + "type" : "decimal(3,1)", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_STR_DEC02", + "type" : "decimal(4,2)", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_STR_DEC03", + "type" : "decimal(5,2)", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_STR_DEC04", + "type" : "decimal(8,4)", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_STR_DEC05", + "type" : "decimal(9,4)", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_STR_DEC06", + "type" : "decimal(10,5)", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_STR_DEC07", + "type" : "decimal(17,2)", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_STR_DEC08", + "type" : "decimal(18,2)", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_STR_DEC09", + "type" : "decimal(19,2)", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_STR_DEC10", + "type" : "decimal(28,10)", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_STR_SDEC01", + "type" : "decimal(3,1)", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_STR_SDEC02", + "type" : "decimal(4,2)", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_STR_SDEC03", + "type" : "decimal(5,2)", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_STR_SDEC04", + "type" : "decimal(8,4)", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_STR_SDEC05", + "type" : "decimal(9,4)", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_STR_SDEC06", + "type" : "decimal(10,5)", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_STR_SDEC07", + "type" : "decimal(17,2)", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_STR_SDEC08", + "type" : "decimal(18,2)", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_STR_SDEC09", + "type" : "decimal(19,2)", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_STR_SDEC10", + "type" : "decimal(28,10)", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_STR_EDEC03", + "type" : "decimal(5,2)", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_STR_EDEC04", + "type" : "decimal(8,4)", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_STR_EDEC05", + "type" : "decimal(9,4)", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_STR_EDEC06", + "type" : "decimal(10,5)", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_BIN_INT01", + "type" : "integer", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_BIN_INT02", + "type" : "integer", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_BIN_INT03", + "type" : "integer", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_BIN_INT04", + "type" : "integer", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_BIN_INT05", + "type" : "integer", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_BIN_INT06", + "type" : "integer", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_BIN_INT07", + "type" : "long", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_BIN_INT08", + "type" : "long", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_BIN_INT09", + "type" : "long", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_BIN_INT10", + "type" : "long", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_BIN_INT11", + "type" : "decimal(20,0)", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_BIN_INT12", + "type" : "decimal(19,0)", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_BIN_INT13", + "type" : "decimal(20,0)", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_BIN_INT14", + "type" : "decimal(37,0)", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_SBIN_SINT01", + "type" : "integer", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_SBIN_SINT02", + "type" : "integer", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_SBIN_SINT03", + "type" : "integer", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_SBIN_SINT04", + "type" : "integer", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_SBIN_SINT05", + "type" : "integer", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_SBIN_SINT06", + "type" : "integer", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_SBIN_SINT07", + "type" : "integer", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_SBIN_SINT08", + "type" : "long", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_SBIN_SINT09", + "type" : "long", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_SBIN_SINT10", + "type" : "long", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_SBIN_SINT11", + "type" : "long", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_SBIN_SINT12", + "type" : "decimal(19,0)", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_SBIN_SINT13", + "type" : "decimal(20,0)", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_SBIN_SINT14", + "type" : "decimal(37,0)", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_BIN_DEC01", + "type" : "decimal(3,1)", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_BIN_DEC02", + "type" : "decimal(4,2)", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_BIN_DEC03", + "type" : "decimal(5,2)", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_BIN_DEC04", + "type" : "decimal(8,4)", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_BIN_DEC05", + "type" : "decimal(9,4)", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_BIN_DEC06", + "type" : "decimal(10,5)", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_BIN_DEC07", + "type" : "decimal(17,2)", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_BIN_DEC08", + "type" : "decimal(18,2)", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_BIN_DEC09", + "type" : "decimal(19,2)", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_BIN_DEC10", + "type" : "decimal(28,10)", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_SBIN_DEC01", + "type" : "decimal(3,1)", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_SBIN_DEC02", + "type" : "decimal(4,2)", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_SBIN_DEC03", + "type" : "decimal(5,2)", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_SBIN_DEC04", + "type" : "decimal(8,4)", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_SBIN_DEC05", + "type" : "decimal(9,4)", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_SBIN_DEC06", + "type" : "decimal(10,5)", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_SBIN_DEC07", + "type" : "decimal(17,2)", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_SBIN_DEC08", + "type" : "decimal(18,2)", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_SBIN_DEC09", + "type" : "decimal(19,2)", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_SBIN_DEC10", + "type" : "decimal(28,10)", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_BCD_INT01", + "type" : "integer", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_BCD_INT02", + "type" : "integer", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_BCD_INT03", + "type" : "integer", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_BCD_INT04", + "type" : "integer", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_BCD_INT05", + "type" : "integer", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_BCD_INT06", + "type" : "integer", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_BCD_INT07", + "type" : "integer", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_BCD_INT08", + "type" : "long", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_BCD_INT09", + "type" : "long", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_BCD_INT10", + "type" : "long", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_BCD_INT11", + "type" : "long", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_BCD_INT12", + "type" : "decimal(19,0)", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_BCD_INT13", + "type" : "decimal(20,0)", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_BCD_INT14", + "type" : "decimal(37,0)", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_BCD_SINT01", + "type" : "integer", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_BCD_SINT02", + "type" : "integer", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_BCD_SINT03", + "type" : "integer", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_BCD_SINT04", + "type" : "integer", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_BCD_SINT05", + "type" : "integer", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_BCD_SINT06", + "type" : "integer", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_BCD_SINT07", + "type" : "integer", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_BCD_SINT08", + "type" : "long", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_BCD_SINT09", + "type" : "long", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_BCD_SINT10", + "type" : "long", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_BCD_SINT11", + "type" : "long", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_BCD_SINT12", + "type" : "decimal(19,0)", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_BCD_SINT13", + "type" : "decimal(20,0)", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_BCD_SINT14", + "type" : "decimal(37,0)", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_BCD_DEC01", + "type" : "decimal(3,1)", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_BCD_DEC02", + "type" : "decimal(4,2)", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_BCD_DEC03", + "type" : "decimal(5,2)", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_BCD_DEC04", + "type" : "decimal(8,4)", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_BCD_DEC05", + "type" : "decimal(9,4)", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_BCD_DEC06", + "type" : "decimal(10,5)", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_BCD_DEC07", + "type" : "decimal(17,2)", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_BCD_DEC08", + "type" : "decimal(18,2)", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_BCD_DEC09", + "type" : "decimal(19,2)", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_BCD_DEC10", + "type" : "decimal(28,10)", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_BCD_SDEC01", + "type" : "decimal(3,1)", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_BCD_SDEC02", + "type" : "decimal(4,2)", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_BCD_SDEC03", + "type" : "decimal(5,2)", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_BCD_SDEC04", + "type" : "decimal(8,4)", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_BCD_SDEC05", + "type" : "decimal(9,4)", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_BCD_SDEC06", + "type" : "decimal(10,5)", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_BCD_SDEC07", + "type" : "decimal(17,2)", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_BCD_SDEC08", + "type" : "decimal(18,2)", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_BCD_SDEC09", + "type" : "decimal(19,2)", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_BCD_SDEC10", + "type" : "decimal(28,10)", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_SL_STR_INT01", + "type" : "integer", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_SL_STR_DEC01", + "type" : "decimal(4,2)", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_ST_STR_INT01", + "type" : "integer", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_ST_STR_DEC01", + "type" : "decimal(4,2)", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_SLI_STR_DEC01", + "type" : "decimal(7,7)", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_STI_STR_DEC01", + "type" : "decimal(7,7)", + "nullable" : true, + "metadata" : { } + }, { + "name" : "NUM_SLI_DEBUG", + "type" : "string", + "nullable" : true, + "metadata" : { + "maxLength" : 7 + } + }, { + "name" : "NUM_STI_DEBUG", + "type" : "string", + "nullable" : true, + "metadata" : { + "maxLength" : 7 + } + }, { + "name" : "FLOAT_01", + "type" : "float", + "nullable" : true, + "metadata" : { } + }, { + "name" : "DOUBLE_01", + "type" : "double", + "nullable" : true, + "metadata" : { } + }, { + "name" : "COMMON_8_BIN", + "type" : "integer", + "nullable" : true, + "metadata" : { } + }, { + "name" : "COMMON_S3_BIN", + "type" : "integer", + "nullable" : true, + "metadata" : { } + }, { + "name" : "COMMON_S94COMP", + "type" : "integer", + "nullable" : true, + "metadata" : { } + }, { + "name" : "COMMON_S8_BIN", + "type" : "integer", + "nullable" : true, + "metadata" : { } + }, { + "name" : "COMMON_DDC97_BIN", + "type" : "decimal(8,7)", + "nullable" : true, + "metadata" : { } + }, { + "name" : "COMMON_97COMP3", + "type" : "integer", + "nullable" : true, + "metadata" : { } + }, { + "name" : "COMMON_915COMP3", + "type" : "long", + "nullable" : true, + "metadata" : { } + }, { + "name" : "COMMON_S95COMP3", + "type" : "integer", + "nullable" : true, + "metadata" : { } + }, { + "name" : "COMMON_S999DCCOMP3", + "type" : "decimal(11,2)", + "nullable" : true, + "metadata" : { } + }, { + "name" : "COMMON_S913COMP3", + "type" : "long", + "nullable" : true, + "metadata" : { } + }, { + "name" : "COMMON_S913DCCOMP3", + "type" : "decimal(15,2)", + "nullable" : true, + "metadata" : { } + }, { + "name" : "COMMON_S911DCC2", + "type" : "decimal(13,2)", + "nullable" : true, + "metadata" : { } + }, { + "name" : "COMMON_S910DCC3", + "type" : "decimal(13,3)", + "nullable" : true, + "metadata" : { } + }, { + "name" : "COMMON_S03DDC", + "type" : "decimal(5,5)", + "nullable" : true, + "metadata" : { } + }, { + "name" : "COMMON_U03DDC", + "type" : "decimal(5,5)", + "nullable" : true, + "metadata" : { } + }, { + "name" : "COMMON_UPC5DDC", + "type" : "decimal(8,8)", + "nullable" : true, + "metadata" : { } + }, { + "name" : "COMMON_SPC5DDC", + "type" : "decimal(7,7)", + "nullable" : true, + "metadata" : { } + }, { + "name" : "COMMON_UPI5DDC", + "type" : "decimal(8,0)", + "nullable" : true, + "metadata" : { } + }, { + "name" : "COMMON_SPI5DDC", + "type" : "decimal(8,0)", + "nullable" : true, + "metadata" : { } + }, { + "name" : "COMMON_UPC5DISP", + "type" : "decimal(8,8)", + "nullable" : true, + "metadata" : { } + }, { + "name" : "COMMON_UPI5DISP", + "type" : "decimal(8,0)", + "nullable" : true, + "metadata" : { } + }, { + "name" : "COMMON_UPC1BIN", + "type" : "decimal(4,4)", + "nullable" : true, + "metadata" : { } + }, { + "name" : "COMMON_UPI1BIN", + "type" : "decimal(4,0)", + "nullable" : true, + "metadata" : { } + }, { + "name" : "COMMON_UPC3BIN", + "type" : "decimal(6,6)", + "nullable" : true, + "metadata" : { } + }, { + "name" : "COMMON_UPI3BIN", + "type" : "decimal(6,0)", + "nullable" : true, + "metadata" : { } + }, { + "name" : "COMMON_UPC5BIN", + "type" : "decimal(8,8)", + "nullable" : true, + "metadata" : { } + }, { + "name" : "COMMON_UPI5BIN", + "type" : "decimal(8,0)", + "nullable" : true, + "metadata" : { } + }, { + "name" : "COMMON_UPC10BIN", + "type" : "decimal(13,13)", + "nullable" : true, + "metadata" : { } + }, { + "name" : "COMMON_UPI10BIN", + "type" : "decimal(13,0)", + "nullable" : true, + "metadata" : { } + }, { + "name" : "EX_NUM_INT01", + "type" : "integer", + "nullable" : true, + "metadata" : { } + }, { + "name" : "EX_NUM_INT02", + "type" : "integer", + "nullable" : true, + "metadata" : { } + }, { + "name" : "EX_NUM_INT03", + "type" : "integer", + "nullable" : true, + "metadata" : { } + }, { + "name" : "EX_NUM_INT04", + "type" : "integer", + "nullable" : true, + "metadata" : { } + }, { + "name" : "EX_NUM_DEC01", + "type" : "decimal(8,2)", + "nullable" : true, + "metadata" : { } + }, { + "name" : "EX_NUM_DEC02", + "type" : "decimal(8,2)", + "nullable" : true, + "metadata" : { } + }, { + "name" : "EX_NUM_DEC03", + "type" : "decimal(8,2)", + "nullable" : true, + "metadata" : { } + } ] +} \ No newline at end of file diff --git a/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/SparkCobolProcessor.scala b/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/SparkCobolProcessor.scala index 8f0763584..9d70cfa68 100644 --- a/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/SparkCobolProcessor.scala +++ b/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/SparkCobolProcessor.scala @@ -229,8 +229,8 @@ object SparkCobolProcessor { val numOfBytesMsg = if (numOfBytes > 0) s"${numOfBytes / Constants.megabyte} MB" else "until the end" log.info(s"Going to process offsets ${indexEntry.offsetFrom}...${indexEntry.offsetTo} ($numOfBytesMsg) of $fileName") - val dataStream = new FileStreamer(filePathName, fileSystem, indexEntry.offsetFrom, numOfBytes) - val headerStream = new FileStreamer(filePathName, fileSystem) + val dataStream = new FileStreamer(filePathName, sconf.value, indexEntry.offsetFrom, numOfBytes) + val headerStream = new FileStreamer(filePathName, sconf.value) CobolProcessorBase.getRecordExtractor(readerParameters, copybookContents, dataStream, Some(headerStream)) }) @@ -240,7 +240,7 @@ object SparkCobolProcessor { val hadoopConfig = sconf.value log.info(s"Going to process data from $inputFile") val inputFs = new Path(inputFile).getFileSystem(hadoopConfig) - val ifs = new FileStreamer(inputFile, inputFs) + val ifs = new FileStreamer(inputFile, sconf.value) CobolProcessorBase.getRecordExtractor(readerParameters, copybookContents, ifs, None) } @@ -266,7 +266,7 @@ object SparkCobolProcessor { Future { val hadoopConfig = sconf.value val inputFs = new Path(inputFIle).getFileSystem(hadoopConfig) - val ifs = new FileStreamer(inputFIle, inputFs) + val ifs = new FileStreamer(inputFIle, sconf.value) val outputFile = new Path(outputPath, fileName) val outputFs = outputFile.getFileSystem(hadoopConfig) val ofs = new BufferedOutputStream(outputFs.create(outputFile, true)) diff --git a/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/source/CobolRelation.scala b/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/source/CobolRelation.scala index ff7e639f5..55339c23b 100644 --- a/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/source/CobolRelation.scala +++ b/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/source/CobolRelation.scala @@ -16,22 +16,22 @@ package za.co.absa.cobrix.spark.cobol.source -import java.io.{IOException, ObjectInputStream, ObjectOutputStream} - import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.mapred.FileInputFormat +import org.apache.hadoop.fs.Path +import org.apache.hadoop.io.compress.CompressionCodecFactory import org.apache.spark.rdd.RDD import org.apache.spark.sql.sources.{BaseRelation, TableScan} import org.apache.spark.sql.types._ import org.apache.spark.sql.{Row, SQLContext} -import za.co.absa.cobrix.spark.cobol.reader.{FixedLenReader, FixedLenTextReader, Reader, VarLenReader} import za.co.absa.cobrix.cobol.reader.index.entry.SparseIndexEntry +import za.co.absa.cobrix.spark.cobol.reader.{FixedLenReader, FixedLenTextReader, Reader, VarLenReader} import za.co.absa.cobrix.spark.cobol.source.index.IndexBuilder import za.co.absa.cobrix.spark.cobol.source.parameters.LocalityParameters import za.co.absa.cobrix.spark.cobol.source.scanners.CobolScanners import za.co.absa.cobrix.spark.cobol.source.types.FileWithOrder import za.co.absa.cobrix.spark.cobol.utils.FileUtils +import java.io.{IOException, ObjectInputStream, ObjectOutputStream} import scala.util.control.NonFatal @@ -63,6 +63,7 @@ class SerializableConfiguration(@transient var value: Configuration) extends Ser * Its constructor is expected to change after the hierarchy of [[za.co.absa.cobrix.spark.cobol.reader.Reader]] is put in place. */ class CobolRelation(sourceDirs: Seq[String], + filesList: Array[FileWithOrder], cobolReader: Reader, localityParams: LocalityParameters, debugIgnoreFileSize: Boolean) @@ -71,8 +72,6 @@ class CobolRelation(sourceDirs: Seq[String], with Serializable with TableScan { - private val filesList = CobolRelation.getListFilesWithOrder(sourceDirs, sqlContext, isRecursiveRetrieval) - private lazy val indexes: RDD[SparseIndexEntry] = IndexBuilder.buildIndex(filesList, cobolReader, sqlContext, cobolReader.getReaderProperties.isIndexCachingAllowed)(localityParams) override def schema: StructType = { @@ -94,15 +93,7 @@ class CobolRelation(sourceDirs: Seq[String], } } - /** - * Checks if the recursive file retrieval flag is set - */ - private def isRecursiveRetrieval: Boolean = { - val hadoopConf = sqlContext.sparkContext.hadoopConfiguration - hadoopConf.getBoolean(FileInputFormat.INPUT_DIR_RECURSIVE, false) - } - - private[source] def parseRecords(reader: FixedLenReader, records: RDD[Array[Byte]]) = { + private[source] def parseRecords(reader: FixedLenReader, records: RDD[Array[Byte]]): RDD[Row] = { records.flatMap(record => { val it = reader.getRowIterator(record) for (parsedRecord <- it) yield { @@ -125,8 +116,16 @@ object CobolRelation { .getFiles(sourceDir, sqlContext.sparkContext.hadoopConfiguration, isRecursiveRetrieval) }).toArray + val hadoopConf = sqlContext.sparkContext.hadoopConfiguration + val factory = new CompressionCodecFactory(hadoopConf) + allFiles .zipWithIndex - .map(file => FileWithOrder(file._1, file._2)) + .map { case (fileName, order) => + val codec = factory.getCodec(new Path(fileName)) + val isCompressed = codec != null + + FileWithOrder(fileName, order, isCompressed) + } } } \ No newline at end of file diff --git a/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/source/DefaultSource.scala b/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/source/DefaultSource.scala index 63619cc9b..cea0b29da 100644 --- a/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/source/DefaultSource.scala +++ b/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/source/DefaultSource.scala @@ -18,6 +18,7 @@ package za.co.absa.cobrix.spark.cobol.source import org.apache.hadoop.fs.Path import org.apache.hadoop.io.{BytesWritable, NullWritable} +import org.apache.hadoop.mapred.FileInputFormat import org.apache.spark.sql.sources._ import org.apache.spark.sql.types.StructType import org.apache.spark.sql.{DataFrame, SQLContext, SaveMode, SparkSession} @@ -59,8 +60,17 @@ class DefaultSource val cobolParameters = CobolParametersParser.parse(new Parameters(parameters)) CobolParametersValidator.checkSanity(cobolParameters) + val filesList = CobolRelation.getListFilesWithOrder(cobolParameters.sourcePaths, sqlContext, isRecursiveRetrieval(sqlContext)) + + val hasCompressedFiles = filesList.exists(_.isCompressed) + + if (hasCompressedFiles) { + logger.info(s"Compressed files found. Binary parallelism and indexes will be adjusted accordingly.") + } + new CobolRelation(cobolParameters.sourcePaths, - buildEitherReader(sqlContext.sparkSession, cobolParameters), + filesList, + buildEitherReader(sqlContext.sparkSession, cobolParameters, hasCompressedFiles), LocalityParameters.extract(cobolParameters), cobolParameters.debugIgnoreFileSize)(sqlContext) } @@ -126,6 +136,14 @@ class DefaultSource //TODO fix with the correct implementation once the correct Reader hierarchy is put in place. override def buildReader(spark: SparkSession, parameters: Map[String, String]): FixedLenReader = null + + /** + * Checks if the recursive file retrieval flag is set + */ + private def isRecursiveRetrieval(sqlContext: SQLContext): Boolean = { + val hadoopConf = sqlContext.sparkContext.hadoopConfiguration + hadoopConf.getBoolean(FileInputFormat.INPUT_DIR_RECURSIVE, false) + } } object DefaultSource { @@ -136,10 +154,10 @@ object DefaultSource { * * This method will probably be removed once the correct hierarchy for [[FixedLenReader]] is put in place. */ - def buildEitherReader(spark: SparkSession, cobolParameters: CobolParameters): Reader = { + def buildEitherReader(spark: SparkSession, cobolParameters: CobolParameters, hasCompressedFiles: Boolean): Reader = { val reader = if (cobolParameters.isText && cobolParameters.variableLengthParams.isEmpty) { createTextReader(cobolParameters, spark) - } else if (cobolParameters.variableLengthParams.isEmpty) { + } else if (cobolParameters.variableLengthParams.isEmpty && !hasCompressedFiles) { createFixedLengthReader(cobolParameters, spark) } else { @@ -156,7 +174,7 @@ object DefaultSource { */ def createTextReader(parameters: CobolParameters, spark: SparkSession): FixedLenReader = { val copybookContent = CopybookContentLoader.load(parameters, spark.sparkContext.hadoopConfiguration) - val defaultHdfsBlockSize = SparkUtils.getDefaultHdfsBlockSize(spark, parameters.sourcePaths.headOption) + val defaultHdfsBlockSize = SparkUtils.getDefaultFsBlockSize(spark, parameters.sourcePaths.headOption) new FixedLenTextReader(copybookContent, getReaderProperties(parameters, defaultHdfsBlockSize) ) } @@ -167,7 +185,7 @@ object DefaultSource { def createFixedLengthReader(parameters: CobolParameters, spark: SparkSession): FixedLenReader = { val copybookContent = CopybookContentLoader.load(parameters, spark.sparkContext.hadoopConfiguration) - val defaultHdfsBlockSize = SparkUtils.getDefaultHdfsBlockSize(spark, parameters.sourcePaths.headOption) + val defaultHdfsBlockSize = SparkUtils.getDefaultFsBlockSize(spark, parameters.sourcePaths.headOption) new FixedLenNestedReader(copybookContent, getReaderProperties(parameters, defaultHdfsBlockSize) ) } @@ -181,7 +199,7 @@ object DefaultSource { val copybookContent = CopybookContentLoader.load(parameters, spark.sparkContext.hadoopConfiguration) - val defaultHdfsBlockSize = SparkUtils.getDefaultHdfsBlockSize(spark, parameters.sourcePaths.headOption) + val defaultHdfsBlockSize = SparkUtils.getDefaultFsBlockSize(spark, parameters.sourcePaths.headOption) new VarLenNestedReader( copybookContent, getReaderProperties(parameters, defaultHdfsBlockSize) ) diff --git a/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/source/index/IndexBuilder.scala b/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/source/index/IndexBuilder.scala index ef334d6e5..4d01f8c0c 100644 --- a/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/source/index/IndexBuilder.scala +++ b/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/source/index/IndexBuilder.scala @@ -32,7 +32,7 @@ import za.co.absa.cobrix.spark.cobol.source.SerializableConfiguration import za.co.absa.cobrix.spark.cobol.source.parameters.LocalityParameters import za.co.absa.cobrix.spark.cobol.source.streaming.FileStreamer import za.co.absa.cobrix.spark.cobol.source.types.FileWithOrder -import za.co.absa.cobrix.spark.cobol.utils.{HDFSUtils, SparkUtils} +import za.co.absa.cobrix.spark.cobol.utils.{FileUtils, HDFSUtils, SparkUtils} import java.util.concurrent.ConcurrentHashMap import scala.collection.mutable.ArrayBuffer @@ -231,15 +231,21 @@ private[cobol] object IndexBuilder extends Logging { val maximumBytes = if (fileEndOffset == 0) { 0 } else { - val bytesToRead = fileSystem.getContentSummary(path).getLength - fileEndOffset - startOffset + val fileSize = if (FileUtils.isCompressed(path, config)) { + FileUtils.getCompressedFileSize(path,config) + } else { + fileSystem.getFileStatus(path).getLen + } + + val bytesToRead = fileSize - fileEndOffset - startOffset if (bytesToRead < 0) 0 else bytesToRead } - val inputStream = new FileStreamer(filePath, fileSystem, startOffset, maximumBytes) - val headerStream = new FileStreamer(filePath, fileSystem) + val inputStream = new FileStreamer(filePath, config, startOffset, maximumBytes) + val headerStream = new FileStreamer(filePath, config) (inputStream, headerStream, maximumBytes) } diff --git a/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/source/scanners/CobolScanners.scala b/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/source/scanners/CobolScanners.scala index 9d7456a9d..8a5893344 100644 --- a/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/source/scanners/CobolScanners.scala +++ b/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/source/scanners/CobolScanners.scala @@ -43,14 +43,13 @@ private[source] object CobolScanners extends Logging { indexes.flatMap(indexEntry => { val filePathName = filesMap(indexEntry.fileId) val path = new Path(filePathName) - val fileSystem = path.getFileSystem(sconf.value) val fileName = path.getName val numOfBytes = if (indexEntry.offsetTo > 0L) indexEntry.offsetTo - indexEntry.offsetFrom else 0L val numOfBytesMsg = if (numOfBytes > 0) s"${numOfBytes / Constants.megabyte} MB" else "until the end" logger.info(s"Going to process offsets ${indexEntry.offsetFrom}...${indexEntry.offsetTo} ($numOfBytesMsg) of $fileName") - val dataStream = new FileStreamer(filePathName, fileSystem, indexEntry.offsetFrom, numOfBytes) - val headerStream = new FileStreamer(filePathName, fileSystem) + val dataStream = new FileStreamer(filePathName, sconf.value, indexEntry.offsetFrom, numOfBytes) + val headerStream = new FileStreamer(filePathName, sconf.value) reader.getRowIterator(dataStream, headerStream, indexEntry.offsetFrom, indexEntry.fileId, indexEntry.recordIndex) }) } @@ -73,10 +72,16 @@ private[source] object CobolScanners extends Logging { val maximumFileBytes = if (reader.getReaderProperties.fileEndOffset == 0) { 0 } else { - fileSystem.getFileStatus(path).getLen - reader.getReaderProperties.fileEndOffset - startFileOffset + val fileSize = if (FileUtils.isCompressed(path, sconf.value)) { + FileUtils.getCompressedFileSize(path, sconf.value) + } else { + fileSystem.getFileStatus(path).getLen + } + + fileSize - reader.getReaderProperties.fileEndOffset - startFileOffset } - val dataStream = new FileStreamer(filePath, fileSystem, startFileOffset, maximumFileBytes) - val headerStream = new FileStreamer(filePath, fileSystem, startFileOffset) + val dataStream = new FileStreamer(filePath, sconf.value, startFileOffset, maximumFileBytes) + val headerStream = new FileStreamer(filePath, sconf.value, startFileOffset) reader.getRowIterator(dataStream, headerStream, startFileOffset, fileOrder, 0L) }) }) diff --git a/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/source/streaming/BufferedFSDataInputStream.scala b/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/source/streaming/BufferedFSDataInputStream.scala index b6d3cf653..7aa39cbb7 100644 --- a/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/source/streaming/BufferedFSDataInputStream.scala +++ b/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/source/streaming/BufferedFSDataInputStream.scala @@ -16,29 +16,29 @@ package za.co.absa.cobrix.spark.cobol.source.streaming -import org.apache.hadoop.fs.{FSDataInputStream, FileSystem, Path} +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{FSDataInputStream, Path} +import za.co.absa.cobrix.spark.cobol.utils.FileUtils -import java.io.IOException +import java.io.{IOException, InputStream} -class BufferedFSDataInputStream(filePath: Path, fileSystem: FileSystem, startOffset: Long, bufferSizeInMegabytes: Int, maximumBytes: Long ) { +class BufferedFSDataInputStream(filePath: Path, hadoopConfig: Configuration, startOffset: Long, bufferSizeInMegabytes: Int, maximumBytes: Long ) { val bytesInMegabyte: Int = 1048576 + private var isCompressedStream = false if (bufferSizeInMegabytes <=0 || bufferSizeInMegabytes > 1000) { throw new IllegalArgumentException(s"Invalid buffer size $bufferSizeInMegabytes MB.") } - var in: FSDataInputStream = fileSystem.open(filePath) - if (startOffset > 0) { - in.seek(startOffset) - } + private var in: InputStream = openStream() private val bufferSizeInBytes = bufferSizeInMegabytes * bytesInMegabyte private var isStreamClosed = in == null private val buffer = new Array[Byte](bufferSizeInBytes) private var bufferPos = 0 - private var bufferConitainBytes = 0 - private var bytesRead = 0 + private var bufferContainBytes = 0 + private var bytesRead = 0L @throws[IOException] def close(): Unit = { @@ -49,46 +49,50 @@ class BufferedFSDataInputStream(filePath: Path, fileSystem: FileSystem, startOff } } - def isClosed: Boolean = isStreamClosed && bufferPos >= bufferConitainBytes + def isClosed: Boolean = isStreamClosed && bufferPos >= bufferContainBytes + + def isCompressed: Boolean = isCompressedStream def readFully(b: Array[Byte], off: Int, len: Int): Int = { if (isClosed) { -1 - } else if (bufferPos + len < bufferConitainBytes) { + } else if (bufferPos + len < bufferContainBytes) { System.arraycopy(buffer, bufferPos, b, off, len) bufferPos += len len } else { var offsetLeft = off var lengthLeft = len - if (bufferPos < bufferConitainBytes) { - val bytesLeft = bufferConitainBytes - bufferPos + if (bufferPos < bufferContainBytes) { + val bytesLeft = bufferContainBytes - bufferPos System.arraycopy(buffer, bufferPos, b, off, bytesLeft) - lengthLeft -= bufferConitainBytes - bufferPos + lengthLeft -= bytesLeft offsetLeft += bytesLeft } bufferPos = 0 - bufferConitainBytes = if ( (maximumBytes>0 && bytesRead >= maximumBytes) || isStreamClosed) { + bufferContainBytes = if ( (maximumBytes>0 && bytesRead >= maximumBytes) || isStreamClosed) { close() 0 } else { - val toRead = if (maximumBytes >0) Math.min(bufferSizeInBytes, maximumBytes - bytesRead) else bufferSizeInBytes - readFullyHelper(buffer, 0, bufferSizeInBytes) + val toRead = if (maximumBytes > 0) Math.min(bufferSizeInBytes, maximumBytes - bytesRead) else bufferSizeInBytes + readFullyHelper(buffer, 0, toRead.toInt) } - bytesRead += bufferConitainBytes - if (bufferConitainBytes > 0) { - if (bufferPos + lengthLeft < bufferConitainBytes) { + bytesRead += bufferContainBytes + if (bufferContainBytes > 0) { + if (bufferPos + lengthLeft < bufferContainBytes) { System.arraycopy(buffer, bufferPos, b, offsetLeft, lengthLeft) bufferPos += lengthLeft offsetLeft += lengthLeft lengthLeft = 0 } else { - if (bufferConitainBytes > 0) { - System.arraycopy(buffer, bufferPos, b, offsetLeft, lengthLeft) - bufferPos += bufferConitainBytes - offsetLeft += bufferConitainBytes - lengthLeft -= bufferConitainBytes + if (bufferContainBytes > 0 && lengthLeft > 0) { + val available = bufferContainBytes - bufferPos + val bytesToCopy = Math.min(lengthLeft, available) + System.arraycopy(buffer, bufferPos, b, offsetLeft, bytesToCopy) + bufferPos += bytesToCopy + offsetLeft += bytesToCopy + lengthLeft -= bytesToCopy } } } @@ -115,4 +119,31 @@ class BufferedFSDataInputStream(filePath: Path, fileSystem: FileSystem, startOff } } + private def openStream(): InputStream = { + val fileSystem = filePath.getFileSystem(hadoopConfig) + val codec = FileUtils.getCompressionCodec(filePath, hadoopConfig) + val fsIn: FSDataInputStream = fileSystem.open(filePath) + + val baseStream = if (codec != null) { + isCompressedStream = true + codec.createInputStream(fsIn) + } else { + // No compression detected + fsIn + } + + if (startOffset > 0) { + if (codec == null) { + fsIn.seek(startOffset) + } else { + var toSkip = startOffset + while (toSkip > 0) { + val skipped = baseStream.skip(toSkip) + if (skipped <= 0) return baseStream + toSkip -= skipped + } + } + } + baseStream + } } diff --git a/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/source/streaming/FileStreamer.scala b/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/source/streaming/FileStreamer.scala index dc61db2fb..acf786d4b 100644 --- a/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/source/streaming/FileStreamer.scala +++ b/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/source/streaming/FileStreamer.scala @@ -16,11 +16,12 @@ package za.co.absa.cobrix.spark.cobol.source.streaming -import org.apache.hadoop.fs.{FSDataInputStream, FileSystem, Path} +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{ContentSummary, Path} import org.apache.log4j.Logger -import za.co.absa.cobrix.cobol.reader.stream.SimpleStream -import org.apache.hadoop.fs.ContentSummary import za.co.absa.cobrix.cobol.reader.common.Constants +import za.co.absa.cobrix.cobol.reader.stream.SimpleStream +import za.co.absa.cobrix.spark.cobol.utils.FileUtils import java.io.IOException @@ -33,13 +34,14 @@ import java.io.IOException * file be consumed. * * @param filePath String containing the fully qualified path to the file. - * @param fileSystem Underlying Hadoop file system. + * @param hadoopConfig Hadoop configuration. * @note This class is not thread-safe and should only be accessed from a single thread */ -class FileStreamer(filePath: String, fileSystem: FileSystem, startOffset: Long = 0L, maximumBytes: Long = 0L) extends SimpleStream { +class FileStreamer(filePath: String, hadoopConfig: Configuration, startOffset: Long = 0L, maximumBytes: Long = 0L) extends SimpleStream { private val logger = Logger.getLogger(FileStreamer.this.getClass) + private val hadoopPath = new Path(filePath) private var byteIndex = startOffset // This ensures that the file is never opened if the stream is never used. This serves two purposes: @@ -49,7 +51,9 @@ class FileStreamer(filePath: String, fileSystem: FileSystem, startOffset: Long = private var wasOpened = false private var bufferedStream: BufferedFSDataInputStream = _ - private lazy val fileSize = getHadoopFileSize(new Path(filePath)) + private lazy val isCompressedStream = FileUtils.isCompressed(hadoopPath, hadoopConfig) + + private lazy val fileSize = getHadoopFileSize(hadoopPath) override def inputFileName: String = filePath @@ -59,6 +63,14 @@ class FileStreamer(filePath: String, fileSystem: FileSystem, startOffset: Long = override def offset: Long = byteIndex + override def isCompressed: Boolean = isCompressedStream + + override def isEndOfStream: Boolean = if (isCompressed) { + wasOpened && (bufferedStream == null || bufferedStream.isClosed) + } else { + offset >= size + } + /** * Retrieves a given number of bytes from the file stream. * @@ -123,18 +135,19 @@ class FileStreamer(filePath: String, fileSystem: FileSystem, startOffset: Long = } override def copyStream(): SimpleStream = { - new FileStreamer(filePath, fileSystem, startOffset, maximumBytes) + new FileStreamer(filePath, hadoopConfig, startOffset, maximumBytes) } @throws[IOException] private def ensureOpened(): Unit = { if (!wasOpened) { - bufferedStream = new BufferedFSDataInputStream(new Path(filePath), fileSystem, startOffset, Constants.defaultStreamBufferInMB, maximumBytes) + bufferedStream = new BufferedFSDataInputStream(new Path(filePath), hadoopConfig, startOffset, Constants.defaultStreamBufferInMB, maximumBytes) wasOpened = true } } private def getHadoopFileSize(hadoopPath: Path): Long = { + val fileSystem = hadoopPath.getFileSystem(hadoopConfig) val cSummary: ContentSummary = fileSystem.getContentSummary(hadoopPath) cSummary.getLength } diff --git a/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/source/types/FileWithOrder.scala b/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/source/types/FileWithOrder.scala index 8efb0db5b..bb9b59897 100644 --- a/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/source/types/FileWithOrder.scala +++ b/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/source/types/FileWithOrder.scala @@ -19,4 +19,4 @@ package za.co.absa.cobrix.spark.cobol.source.types /** * Represents a file attached to an order. */ -private[source] case class FileWithOrder(filePath: String, order: Int) +private[source] case class FileWithOrder(filePath: String, order: Int, isCompressed: Boolean) diff --git a/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/utils/FileUtils.scala b/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/utils/FileUtils.scala index 35ec188fc..4499ff0ea 100644 --- a/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/utils/FileUtils.scala +++ b/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/utils/FileUtils.scala @@ -16,13 +16,14 @@ package za.co.absa.cobrix.spark.cobol.utils -import java.io.{FileOutputStream, OutputStreamWriter, PrintWriter} -import java.nio.charset.StandardCharsets -import java.nio.file.{Files, Paths} import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs._ +import org.apache.hadoop.io.compress.{CompressionCodec, CompressionCodecFactory} import za.co.absa.cobrix.cobol.internal.Logging +import java.io.{FileOutputStream, IOException, OutputStreamWriter, PrintWriter} +import java.nio.charset.StandardCharsets +import java.nio.file.{Files, Paths} import scala.collection.JavaConverters._ /** @@ -33,7 +34,6 @@ import scala.collection.JavaConverters._ * Applies the same filter as Hadoop's FileInputFormat, which excludes files starting with '.' or '_'. */ object FileUtils extends Logging { - val THRESHOLD_DIR_LENGTH_FOR_SINGLE_FILE_CHECK = 50 private val hiddenFileFilter = new PathFilter() { @@ -216,6 +216,43 @@ object FileUtils extends Logging { allNonDivisibleFiles.map(status => (status.getPath.toString, status.getLen)) } + def isCompressed(file: Path, hadoopConfig: Configuration): Boolean = { + getCompressionCodec(file, hadoopConfig) != null + } + + def getCompressionCodec(file: Path, hadoopConfig: Configuration): CompressionCodec = { + val factory = new CompressionCodecFactory(hadoopConfig) + factory.getCodec(file) + } + + def getCompressedFileSize(file: Path, hadoopConfig: Configuration): Long = { + logger.warn(s"Using full scan to determine file size of $file..") + val codec = getCompressionCodec(file, hadoopConfig) + val fileSystem = file.getFileSystem(hadoopConfig) + val fsIn: FSDataInputStream = fileSystem.open(file) + val ifs = codec.createInputStream(fsIn) + + val size = try { + val SKIP_BUFFER_SIZE = 1024*1024*50 + var totalBytesSkipped = 0L + var skippedLast = 1L + while (skippedLast > 0) { + skippedLast = ifs.skip(SKIP_BUFFER_SIZE) + if (skippedLast > 0) + totalBytesSkipped += skippedLast + } + totalBytesSkipped + } catch { + case e: IOException => + throw new IOException(s"Unable to determine compressed file size for $file", e) + } finally { + ifs.close() + fsIn.close() + } + logger.info(s"The size of the uncompressed file $file is $size bytes.") + size + } + private def isNonDivisible(fileStatus: FileStatus, divisor: Long) = fileStatus.getLen % divisor != 0 /** diff --git a/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/utils/SparkUtils.scala b/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/utils/SparkUtils.scala index 80fb49869..72c040228 100644 --- a/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/utils/SparkUtils.scala +++ b/spark-cobol/src/main/scala/za/co/absa/cobrix/spark/cobol/utils/SparkUtils.scala @@ -19,12 +19,12 @@ package za.co.absa.cobrix.spark.cobol.utils import com.fasterxml.jackson.databind.ObjectMapper import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.spark.SparkContext -import org.apache.spark.sql.functions.{array, col, expr, max, struct} -import za.co.absa.cobrix.spark.cobol.utils.impl.HofsWrapper.transform +import org.apache.spark.sql.functions._ import org.apache.spark.sql.types._ import org.apache.spark.sql.{Column, DataFrame, SparkSession} import za.co.absa.cobrix.cobol.internal.Logging import za.co.absa.cobrix.spark.cobol.parameters.MetadataFields.MAX_ELEMENTS +import za.co.absa.cobrix.spark.cobol.utils.impl.HofsWrapper.transform import scala.annotation.tailrec import scala.collection.mutable @@ -489,7 +489,7 @@ object SparkUtils extends Logging { }.getOrElse(None) } - def getDefaultHdfsBlockSize(spark: SparkSession, pathOpt: Option[String]): Option[Int] = { + def getDefaultFsBlockSize(spark: SparkSession, pathOpt: Option[String]): Option[Int] = { val conf = spark.sparkContext.hadoopConfiguration val fileSystem =pathOpt match { diff --git a/spark-cobol/src/test/scala/za/co/absa/cobrix/spark/cobol/source/CobolRelationSpec.scala b/spark-cobol/src/test/scala/za/co/absa/cobrix/spark/cobol/source/CobolRelationSpec.scala index 84aa01788..e233488bf 100644 --- a/spark-cobol/src/test/scala/za/co/absa/cobrix/spark/cobol/source/CobolRelationSpec.scala +++ b/spark-cobol/src/test/scala/za/co/absa/cobrix/spark/cobol/source/CobolRelationSpec.scala @@ -16,8 +16,6 @@ package za.co.absa.cobrix.spark.cobol.source -import java.io.File - import org.apache.commons.io.FileUtils import org.apache.spark.SparkException import org.apache.spark.rdd.RDD @@ -31,6 +29,8 @@ import za.co.absa.cobrix.spark.cobol.source.base.impl.{DummyCobolSchema, DummyFi import za.co.absa.cobrix.spark.cobol.source.parameters.LocalityParameters import za.co.absa.cobrix.spark.cobol.source.utils.SourceTestUtils.{createFileInRandomDirectory, sampleCopybook} +import java.io.File + class CobolRelationSpec extends SparkCobolTestBase with Serializable { private val logger = LoggerFactory.getLogger(this.getClass) @@ -61,7 +61,12 @@ class CobolRelationSpec extends SparkCobolTestBase with Serializable { it should "return an RDD[Row] if data are correct" in { val testReader: FixedLenReader = new DummyFixedLenReader(sparkSchema, cobolSchema, testData)(() => ()) - val relation = new CobolRelation(Seq(copybookFile.getParentFile.getAbsolutePath), + + val sourcePaths = Seq(copybookFile.getParentFile.getAbsolutePath) + val filesList = CobolRelation.getListFilesWithOrder(sourcePaths, sqlContext, isRecursiveRetrieval = true) + + val relation = new CobolRelation(sourcePaths, + filesList, testReader, localityParams = localityParams, debugIgnoreFileSize = false)(sqlContext) @@ -85,7 +90,10 @@ class CobolRelationSpec extends SparkCobolTestBase with Serializable { it should "manage exceptions from Reader" in { val exceptionMessage = "exception expected message" val testReader: FixedLenReader = new DummyFixedLenReader(sparkSchema, cobolSchema, testData)(() => throw new Exception(exceptionMessage)) - val relation = new CobolRelation(Seq(copybookFile.getParentFile.getAbsolutePath), + val sourcePaths = Seq(copybookFile.getParentFile.getAbsolutePath) + val filesList = CobolRelation.getListFilesWithOrder(sourcePaths, sqlContext, isRecursiveRetrieval = true) + val relation = new CobolRelation(sourcePaths, + filesList, testReader, localityParams = localityParams, debugIgnoreFileSize = false)(sqlContext) @@ -100,7 +108,10 @@ class CobolRelationSpec extends SparkCobolTestBase with Serializable { val absentField = "absentField" val modifiedSparkSchema = sparkSchema.add(StructField(absentField, StringType, false)) val testReader: FixedLenReader = new DummyFixedLenReader(modifiedSparkSchema, cobolSchema, testData)(() => ()) - val relation = new CobolRelation(Seq(copybookFile.getParentFile.getAbsolutePath), + val sourcePaths = Seq(copybookFile.getParentFile.getAbsolutePath) + val filesList = CobolRelation.getListFilesWithOrder(sourcePaths, sqlContext, isRecursiveRetrieval = true) + val relation = new CobolRelation(sourcePaths, + filesList, testReader, localityParams = localityParams, debugIgnoreFileSize = false)(sqlContext) diff --git a/spark-cobol/src/test/scala/za/co/absa/cobrix/spark/cobol/source/index/IndexBuilderSpec.scala b/spark-cobol/src/test/scala/za/co/absa/cobrix/spark/cobol/source/index/IndexBuilderSpec.scala index 4affdfefd..6d4884223 100644 --- a/spark-cobol/src/test/scala/za/co/absa/cobrix/spark/cobol/source/index/IndexBuilderSpec.scala +++ b/spark-cobol/src/test/scala/za/co/absa/cobrix/spark/cobol/source/index/IndexBuilderSpec.scala @@ -53,7 +53,7 @@ class IndexBuilderSpec extends AnyWordSpec with BinaryFileFixture with SparkTest val file = createFile(new File(tempDir), "test_file", records) val filesWithOrder = Array( - FileWithOrder(file.getAbsolutePath, 0) + FileWithOrder(file.getAbsolutePath, 0, isCompressed = false) ) val readerParameters = ReaderParameters(isIndexGenerationNeeded = true, @@ -75,7 +75,7 @@ class IndexBuilderSpec extends AnyWordSpec with BinaryFileFixture with SparkTest val file = createFile(new File(tempDir), "test_file", records) val filesWithOrder = Array( - FileWithOrder(file.getAbsolutePath, 0) + FileWithOrder(file.getAbsolutePath, 0, isCompressed = false) ) val readerParameters = ReaderParameters(isIndexGenerationNeeded = true, @@ -97,7 +97,7 @@ class IndexBuilderSpec extends AnyWordSpec with BinaryFileFixture with SparkTest val file = createFile(new File(tempDir), "test_file", records) val filesWithOrder = Array( - FileWithOrder(file.getAbsolutePath, 0) + FileWithOrder(file.getAbsolutePath, 0, isCompressed = false) ) val reader = mock(classOf[Reader]) @@ -117,7 +117,7 @@ class IndexBuilderSpec extends AnyWordSpec with BinaryFileFixture with SparkTest val file = createFile(new File(tempDir), "test_file", records) val filesWithOrder = Array( - FileWithOrder(file.getAbsolutePath, 0) + FileWithOrder(file.getAbsolutePath, 0, isCompressed = false) ) val readerParameters = ReaderParameters(isIndexGenerationNeeded = true, @@ -137,7 +137,7 @@ class IndexBuilderSpec extends AnyWordSpec with BinaryFileFixture with SparkTest val file = createFile(new File(tempDir), "test_file", records) val filesWithOrder = Array( - FileWithOrder(file.getAbsolutePath, 0) + FileWithOrder(file.getAbsolutePath, 0, isCompressed = false) ) val readerParameters = ReaderParameters(isIndexGenerationNeeded = true, @@ -159,7 +159,7 @@ class IndexBuilderSpec extends AnyWordSpec with BinaryFileFixture with SparkTest val file = createFile(new File(tempDir), "test_file", records) val filesWithOrder = Array( - FileWithOrder(file.getAbsolutePath, 0) + FileWithOrder(file.getAbsolutePath, 0, isCompressed = false) ) val readerParameters = ReaderParameters(isIndexGenerationNeeded = true, @@ -178,7 +178,7 @@ class IndexBuilderSpec extends AnyWordSpec with BinaryFileFixture with SparkTest val file = createFile(new File(tempDir), "test_file", records) val filesWithOrder = Array( - FileWithOrder(file.getAbsolutePath, 0) + FileWithOrder(file.getAbsolutePath, 0, isCompressed = false) ) val readerParameters = ReaderParameters(isIndexGenerationNeeded = true, @@ -198,10 +198,10 @@ class IndexBuilderSpec extends AnyWordSpec with BinaryFileFixture with SparkTest "buildIndexForFullFiles()" should { "generate a spare index placeholder for each file" in { val files = Array( - FileWithOrder("dummy_file1", 0), - FileWithOrder("dummy_file2", 1), - FileWithOrder("dummy_file3", 2), - FileWithOrder("dummy_file4", 3) + FileWithOrder("dummy_file1", 0, isCompressed = false), + FileWithOrder("dummy_file2", 1, isCompressed = false), + FileWithOrder("dummy_file3", 2, isCompressed = false), + FileWithOrder("dummy_file4", 3, isCompressed = false) ) val sparseIndexRDD = IndexBuilder.buildIndexForFullFiles(files, spark.sqlContext) @@ -239,8 +239,8 @@ class IndexBuilderSpec extends AnyWordSpec with BinaryFileFixture with SparkTest Files.createFile(Paths.get(tempDir, "text1.txt")) Files.createFile(Paths.get(tempDir, "text2.txt")) - val files = Array(FileWithOrder(Paths.get(tempDir, "text1.txt").toString, 0), - FileWithOrder(Paths.get(tempDir, "text2.txt").toString, 1)) + val files = Array(FileWithOrder(Paths.get(tempDir, "text1.txt").toString, 0, isCompressed = false), + FileWithOrder(Paths.get(tempDir, "text2.txt").toString, 1, isCompressed = false)) val filesWIthPreferredLocations = IndexBuilder.toRDDWithLocality(files, spark.sparkContext.hadoopConfiguration, spark.sqlContext) diff --git a/spark-cobol/src/test/scala/za/co/absa/cobrix/spark/cobol/source/integration/Test37RecordLengthMappingSpec.scala b/spark-cobol/src/test/scala/za/co/absa/cobrix/spark/cobol/source/integration/Test37RecordLengthMappingSpec.scala index 7c8c77fe4..af627b41b 100644 --- a/spark-cobol/src/test/scala/za/co/absa/cobrix/spark/cobol/source/integration/Test37RecordLengthMappingSpec.scala +++ b/spark-cobol/src/test/scala/za/co/absa/cobrix/spark/cobol/source/integration/Test37RecordLengthMappingSpec.scala @@ -16,7 +16,6 @@ package za.co.absa.cobrix.spark.cobol.source.integration -import org.apache.hadoop.fs.Path import org.apache.spark.SparkException import org.scalatest.wordspec.AnyWordSpec import za.co.absa.cobrix.cobol.reader.parameters.CobolParametersParser.{PARAM_ENABLE_INDEXES, PARAM_ENABLE_INDEX_CACHE} @@ -59,6 +58,7 @@ class Test37RecordLengthMappingSpec extends AnyWordSpec with SparkTestBase with .option("record_format", "F") .option("record_length_field", "SEG-ID") .option("input_split_records", "2") + .option("enable_index_cache", "false") .option("pedantic", "true") .option("record_length_map", """{"A":4,"B":7,"C":8}""") .load(tempFile) @@ -79,6 +79,7 @@ class Test37RecordLengthMappingSpec extends AnyWordSpec with SparkTestBase with .option("record_format", "V") .option("record_length_field", "SEG-ID") .option("input_split_records", "2") + .option("enable_index_cache", "false") .option("pedantic", "true") .option("record_length_map", """{"A":4,"B":7,"C":8}""") .load(tempFile) @@ -99,6 +100,7 @@ class Test37RecordLengthMappingSpec extends AnyWordSpec with SparkTestBase with .option("record_format", "F") .option("record_length_field", "SEG-ID") .option("input_split_records", "2") + .option("enable_index_cache", "false") .option("pedantic", "true") .option("record_length_map", """{"1":4,"2":7,"3":8}""") .load(tempFile) @@ -119,6 +121,7 @@ class Test37RecordLengthMappingSpec extends AnyWordSpec with SparkTestBase with .option("record_format", "F") .option("record_length_field", "SEG-ID") .option("input_split_records", "2") + .option("enable_index_cache", "false") .option("pedantic", "true") .option("record_length_map", """{"1":4,"2":7,"3":8}""") .option("strict_integral_precision", "true") @@ -163,6 +166,7 @@ class Test37RecordLengthMappingSpec extends AnyWordSpec with SparkTestBase with .option("file_start_offset", 1) .option("file_end_offset", 2) .option("input_split_records", "2") + .option("enable_index_cache", "false") .option("pedantic", "true") .option("record_length_map", """{"A":4,"B":7,"C":8}""") .load(tempFile) diff --git a/spark-cobol/src/test/scala/za/co/absa/cobrix/spark/cobol/source/integration/Test40CompressesFilesSpec.scala b/spark-cobol/src/test/scala/za/co/absa/cobrix/spark/cobol/source/integration/Test40CompressesFilesSpec.scala new file mode 100644 index 000000000..f5fb54c34 --- /dev/null +++ b/spark-cobol/src/test/scala/za/co/absa/cobrix/spark/cobol/source/integration/Test40CompressesFilesSpec.scala @@ -0,0 +1,247 @@ +/* + * 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.source.integration + +import org.scalatest.Assertion +import org.scalatest.funsuite.AnyFunSuite +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 java.nio.charset.StandardCharsets +import java.nio.file.{Files, Paths} +import scala.collection.JavaConverters._ + +class Test40CompressesFilesSpec extends AnyFunSuite with SparkTestBase with BinaryFileFixture with SimpleComparisonBase { + private val exampleName = "Test40 (compressed files)" + + private val inputCopybookPath = "file://../data/test40_copybook.cob" + private val inputCopybookFSPath = "../data/test40_copybook.cob" + private val expectedSchemaPath = "../data/test40_expected/test40_schema.json" + private val expectedLayoutPath = "../data/test40_expected/test40_layout.txt" + private val actualSchemaPath = "../data/test40_expected/test40_schema_actual.json" + private val actualLayoutPath = "../data/test40_expected/test40_layout_actual.txt" + private val expectedResultsPath = "../data/test40_expected/test40.txt" + private val actualResultsPath = "../data/test40_expected/test40_actual.txt" + + def testCompressedFile(inputDataPath: String, useIndexes: Boolean = false): Assertion = { + // Comparing layout + val copybookContents = Files.readAllLines(Paths.get(inputCopybookFSPath), StandardCharsets.ISO_8859_1).toArray.mkString("\n") + val cobolSchema = CopybookParser.parseTree(copybookContents, debugFieldsPolicy = DebugFieldsPolicy.HexValue) + val actualLayout = cobolSchema.generateRecordLayoutPositions() + val expectedLayout = Files.readAllLines(Paths.get(expectedLayoutPath), StandardCharsets.ISO_8859_1).toArray.mkString("\n") + + if (actualLayout != expectedLayout) { + FileUtils.writeStringToFile(actualLayout, actualLayoutPath) + assert(false, s"The actual layout doesn't match what is expected for $exampleName example. Please compare contents of $expectedLayoutPath to " + + s"$actualLayoutPath for details.") + } + + val options = if (useIndexes) { + Map( + "input_split_records" -> "1", + "enable_index_cache" -> "false", + "generate_record_id" -> "true" + ) + } else { + Map.empty[String, String] + } + + val df = spark + .read + .format("cobol") + .option("copybook", inputCopybookPath) + .option("schema_retention_policy", "collapse_root") + .option("floating_point_format", "IEEE754") + .option("strict_sign_overpunching", "true") + .options(options) + .option("pedantic", "true") + .load(inputDataPath) + .drop("File_Id", "Record_Id", "Record_Byte_Length") + + val expectedSchema = Files.readAllLines(Paths.get(expectedSchemaPath), StandardCharsets.ISO_8859_1).toArray.mkString("\n") + val actualSchema = SparkUtils.prettyJSON(df.schema.json) + + if (actualSchema != expectedSchema) { + FileUtils.writeStringToFile(actualSchema, actualSchemaPath) + assert(false, s"The actual schema doesn't match what is expected for $exampleName example. Please compare contents of $expectedSchemaPath to " + + s"$actualSchemaPath for details.") + } + + // Fill nulls with zeros so by looking at json you can tell a field is missing. Otherwise json won't contain null fields. + val actualDf = df.orderBy("ID").na.fill(0).toJSON.take(20) + FileUtils.writeStringsToFile(actualDf, actualResultsPath) + val actual = Files.readAllLines(Paths.get(actualResultsPath), StandardCharsets.ISO_8859_1).asScala.toArray + + // toList is used to convert the Java list to Scala list. If it is skipped the resulting type will be Array[AnyRef] instead of Array[String] + val expected = Files.readAllLines(Paths.get(expectedResultsPath), StandardCharsets.ISO_8859_1).asScala.toArray + + if (!actual.sameElements(expected)) { + assert(false, s"The actual data doesn't match what is expected for $exampleName example. Please compare contents of $expectedResultsPath to " + + s"$actualResultsPath for details.") + } + Files.delete(Paths.get(actualResultsPath)) + + succeed + } + + def testCompressedAsciiFile(options: Map[String, String]): Assertion = { + val inputDataPath = "../data/test40_data_ascii/ascii.txt.gz" + + val df = spark + .read + .format("cobol") + .option("copybook_contents", + """ + | 01 RECORD. + | 05 DATA PIC X(5). + |""".stripMargin) + .option("record_format", "D") + .option("pedantic", "true") + .options(options) + .load(inputDataPath) + + assert(df.count == 3) + + val actual = df.orderBy("data") + .collect() + .map(a => a.getString(0)) + .mkString(",") + + assert(actual == "12345,67890,A1234") + } + + def testMixedAsciiFiles(options: Map[String, String]): Assertion = { + val inputDataPath = "../data/test40_data_ascii/" + + val df = spark + .read + .format("cobol") + .option("copybook_contents", + """ + | 01 RECORD. + | 05 DATA PIC X(5). + |""".stripMargin) + .option("record_format", "D") + .option("pedantic", "true") + .options(options) + .load(inputDataPath) + + assert(df.count == 6) + + val actual = df.orderBy("data") + .collect() + .map(a => a.getString(0)) + .mkString(",") + + assert(actual == "12345,12345,67890,67890,A1234,A1234") + } + + test("Test compressed EBCDIC gzip file without indexes") { + testCompressedFile("../data/test40_data/example.dat.gz") + } + + test("Test compressed EBCDIC bzip2 file without indexes") { + testCompressedFile("../data/test40_data/example.dat.bz2") + } + + test("Test compressed EBCDIC gzip file with indexes") { + testCompressedFile("../data/test40_data/example.dat.gz", useIndexes = true) + } + + test("Test compressed EBCDIC bzip2 file with indexes") { + testCompressedFile("../data/test40_data/example.dat.bz2", useIndexes = true) + } + + test("read mixed compressed EBCDIC files") { + val inputDataPath = "../data/test40_data" + + val df = spark + .read + .format("cobol") + .option("copybook", inputCopybookPath) + .option("schema_retention_policy", "collapse_root") + .option("floating_point_format", "IEEE754") + .option("strict_sign_overpunching", "true") + .option("generate_record_id", "true") + .option("enable_index_cache", "false") + .option("pedantic", "true") + .load(inputDataPath) + + assert(df.count == 300) + } + + test("read mixed compressed EBCDIC files and file_end_offset") { + val inputDataPath = "../data/test40_data" + + val df = spark + .read + .format("cobol") + .option("copybook", inputCopybookPath) + .option("schema_retention_policy", "collapse_root") + .option("floating_point_format", "IEEE754") + .option("strict_sign_overpunching", "true") + .option("file_end_offset", 1493) + .option("enable_index_cache", "false") + .option("pedantic", "true") + .load(inputDataPath) + + assert(df.count == 297) + } + + test("read a compressed ASCII file 1") { + testCompressedAsciiFile(Map( + "record_format" -> "D" + )) + } + + test("read a compressed ASCII file 2") { + testCompressedAsciiFile(Map( + "record_format" -> "D", + "ascii_charset" -> "ISO-8859-1" + )) + } + + test("read a compressed ASCII file 3") { + testCompressedAsciiFile(Map( + "record_format" -> "D2" + )) + } + + test("read a mixed ASCII files 1") { + testMixedAsciiFiles(Map( + "record_format" -> "D" + )) + } + + test("read a mixed ASCII files 2") { + testMixedAsciiFiles(Map( + "record_format" -> "D", + "ascii_charset" -> "ISO-8859-1" + )) + } + + test("read a mixed ASCII files 3") { + testMixedAsciiFiles(Map( + "record_format" -> "D2" + )) + } +} + + diff --git a/spark-cobol/src/test/scala/za/co/absa/cobrix/spark/cobol/source/integration/Test5MultisegmentSpec.scala b/spark-cobol/src/test/scala/za/co/absa/cobrix/spark/cobol/source/integration/Test5MultisegmentSpec.scala index 9ff23e376..56d8ec0c5 100644 --- a/spark-cobol/src/test/scala/za/co/absa/cobrix/spark/cobol/source/integration/Test5MultisegmentSpec.scala +++ b/spark-cobol/src/test/scala/za/co/absa/cobrix/spark/cobol/source/integration/Test5MultisegmentSpec.scala @@ -16,11 +16,7 @@ package za.co.absa.cobrix.spark.cobol.source.integration -import java.nio.charset.StandardCharsets -import java.nio.file.{Files, Paths} - import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.FileSystem import org.scalatest.funsuite.AnyFunSuite import za.co.absa.cobrix.cobol.parser.CopybookParser import za.co.absa.cobrix.cobol.parser.ast.Primitive @@ -31,6 +27,8 @@ import za.co.absa.cobrix.spark.cobol.source.base.SparkTestBase import za.co.absa.cobrix.spark.cobol.source.streaming.FileStreamer import za.co.absa.cobrix.spark.cobol.utils.{FileUtils, SparkUtils} +import java.nio.charset.StandardCharsets +import java.nio.file.{Files, Paths} import scala.collection.JavaConverters._ //noinspection NameBooleanParameters @@ -210,7 +208,7 @@ class Test5MultisegmentSpec extends AnyFunSuite with SparkTestBase { val segmentIdField = copybook.getFieldByName("SEGMENT_ID").asInstanceOf[Primitive] val segmentIdRootValue = "C" - val stream = new FileStreamer("../data/test5_data/COMP.DETAILS.SEP30.DATA.dat", FileSystem.get(new Configuration())) + val stream = new FileStreamer("../data/test5_data/COMP.DETAILS.SEP30.DATA.dat", new Configuration()) val recordHeaderParser = RecordHeaderParserFactory.createRecordHeaderParser(Constants.RhRdwLittleEndian, 0, 0, 0, 0) val indexes = IndexGenerator.sparseIndexGenerator(0, stream, 0L, diff --git a/spark-cobol/src/test/scala/za/co/absa/cobrix/spark/cobol/source/regression/Test12MultiRootSparseIndex.scala b/spark-cobol/src/test/scala/za/co/absa/cobrix/spark/cobol/source/regression/Test12MultiRootSparseIndex.scala index a087502bf..abdb0c096 100644 --- a/spark-cobol/src/test/scala/za/co/absa/cobrix/spark/cobol/source/regression/Test12MultiRootSparseIndex.scala +++ b/spark-cobol/src/test/scala/za/co/absa/cobrix/spark/cobol/source/regression/Test12MultiRootSparseIndex.scala @@ -17,7 +17,6 @@ package za.co.absa.cobrix.spark.cobol.source.regression import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.FileSystem import org.scalatest.wordspec.AnyWordSpec import org.slf4j.{Logger, LoggerFactory} import za.co.absa.cobrix.cobol.parser.CopybookParser @@ -62,7 +61,7 @@ class Test12MultiRootSparseIndex extends AnyWordSpec with SparkTestBase with Bin val segmentIdField = copybook.getFieldByName("S").asInstanceOf[Primitive] val segmentIdRootValues = "0,1" - val stream = new FileStreamer(tmpFileName, FileSystem.get(new Configuration())) + val stream = new FileStreamer(tmpFileName, new Configuration()) val recordHeaderParser = RecordHeaderParserFactory.createRecordHeaderParser(Constants.RhRdwFixedLength, 3, 0, 0, 0) val indexes = IndexGenerator.sparseIndexGenerator(0, stream, 0L, @@ -78,7 +77,7 @@ class Test12MultiRootSparseIndex extends AnyWordSpec with SparkTestBase with Bin val segmentIdField = copybook.getFieldByName("S").asInstanceOf[Primitive] val segmentIdRootValues = "0,1" - val stream = new FileStreamer(tmpFileName, FileSystem.get(new Configuration())) + val stream = new FileStreamer(tmpFileName, new Configuration()) val recordHeaderParser = RecordHeaderParserFactory.createRecordHeaderParser(Constants.RhRdwFixedLength, 3, 0, 0, 0) val indexes = IndexGenerator.sparseIndexGenerator(0, stream, 0L, diff --git a/spark-cobol/src/test/scala/za/co/absa/cobrix/spark/cobol/source/streaming/FileStreamerSpec.scala b/spark-cobol/src/test/scala/za/co/absa/cobrix/spark/cobol/source/streaming/FileStreamerSpec.scala index 0e476db64..c05281a3d 100644 --- a/spark-cobol/src/test/scala/za/co/absa/cobrix/spark/cobol/source/streaming/FileStreamerSpec.scala +++ b/spark-cobol/src/test/scala/za/co/absa/cobrix/spark/cobol/source/streaming/FileStreamerSpec.scala @@ -16,14 +16,14 @@ package za.co.absa.cobrix.spark.cobol.source.streaming -import java.io.{File, FileNotFoundException} import org.apache.commons.io.FileUtils import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.FileSystem import org.scalatest.BeforeAndAfter import org.scalatest.flatspec.AnyFlatSpec import org.scalatest.matchers.should.Matchers +import java.io.{File, FileNotFoundException} + class FileStreamerSpec extends AnyFlatSpec with BeforeAndAfter with Matchers { private val TEMP_DIR = new File("tmp-"+System.currentTimeMillis()) @@ -44,14 +44,14 @@ class FileStreamerSpec extends AnyFlatSpec with BeforeAndAfter with Matchers { it should "throw if file does not exist" in { assertThrows[FileNotFoundException] { - val stream = new FileStreamer(new File(TEMP_DIR, "inexistent").getAbsolutePath, FileSystem.get(new Configuration())) + val stream = new FileStreamer(new File(TEMP_DIR, "inexistent").getAbsolutePath, new Configuration()) stream.size } } it should "not throw if the stream is never used, even if the file does not exist" in { noException should be thrownBy { - new FileStreamer(new File(TEMP_DIR, "inexistent").getAbsolutePath, FileSystem.get(new Configuration())) + new FileStreamer(new File(TEMP_DIR, "inexistent").getAbsolutePath, new Configuration()) } } it should "return array of same length than expected number of bytes if enough data" in { @@ -59,7 +59,7 @@ class FileStreamerSpec extends AnyFlatSpec with BeforeAndAfter with Matchers { val iterations = 10 val tmpFile = createTempFile(iterations * batchLength) - streamer = new FileStreamer(tmpFile.getAbsolutePath, FileSystem.get(new Configuration())) + streamer = new FileStreamer(tmpFile.getAbsolutePath, new Configuration()) for (i <- 0 until iterations) { assert(streamer.next(batchLength).length == batchLength) } @@ -71,7 +71,7 @@ class FileStreamerSpec extends AnyFlatSpec with BeforeAndAfter with Matchers { val extraBytes = 1 val tmpFile = createTempFile(iterations * batchLength + extraBytes) - streamer = new FileStreamer(tmpFile.getAbsolutePath, FileSystem.get(new Configuration())) + streamer = new FileStreamer(tmpFile.getAbsolutePath, new Configuration()) streamer.next(iterations * batchLength ) // consumes all but the extra bytes assert(streamer.next(batchLength).length == extraBytes) } @@ -79,7 +79,7 @@ class FileStreamerSpec extends AnyFlatSpec with BeforeAndAfter with Matchers { it should "return empty array if end of file was reached" in { val totalBytes = 10 val tmpFile = createTempFile(totalBytes) - streamer = new FileStreamer(tmpFile.getAbsolutePath, FileSystem.get(new Configuration())) + streamer = new FileStreamer(tmpFile.getAbsolutePath, new Configuration()) streamer.next(totalBytes) // consumes all bytes assert(streamer.next(totalBytes).length == 0) } @@ -87,7 +87,7 @@ class FileStreamerSpec extends AnyFlatSpec with BeforeAndAfter with Matchers { it should "return empty array if file is empty" in { val totalBytes = 0 val tmpFile = createTempFile(totalBytes) - streamer = new FileStreamer(tmpFile.getAbsolutePath, FileSystem.get(new Configuration())) + streamer = new FileStreamer(tmpFile.getAbsolutePath, new Configuration()) assert(streamer.next(totalBytes).length == totalBytes) }