From 8e425e7978639f54eb430cc0a511801811d4e188 Mon Sep 17 00:00:00 2001 From: Mario Molina Date: Sat, 25 Jul 2020 20:18:12 -0500 Subject: [PATCH 01/21] Prepare for next version --- docker-compose.yml | 2 +- docs/source/conf.py | 4 ++-- pom.xml | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/docker-compose.yml b/docker-compose.yml index d03f109..22c70b9 100644 --- a/docker-compose.yml +++ b/docker-compose.yml @@ -45,7 +45,7 @@ services: SCHEMA_REGISTRY_KAFKASTORE_CONNECTION_URL: 'zookeeper:2181' connect-fs: - image: mmolimar/kafka-connect-fs:1.1.0 + image: mmolimar/kafka-connect-fs:1.2.0-SNAPSHOT container_name: connect depends_on: - cp-kafka diff --git a/docs/source/conf.py b/docs/source/conf.py index 724450a..7c79907 100644 --- a/docs/source/conf.py +++ b/docs/source/conf.py @@ -55,9 +55,9 @@ # built documents. # # The short X.Y version. -version = '1.1' +version = '1.2' # The full version, including alpha/beta/rc tags. -release = '1.1' +release = '1.2' # The language for content autogenerated by Sphinx. Refer to documentation # for a list of supported languages. diff --git a/pom.xml b/pom.xml index 8d1c2dc..a0af373 100644 --- a/pom.xml +++ b/pom.xml @@ -4,7 +4,7 @@ com.github.mmolimar.kafka.connect kafka-connect-fs - 1.1.0 + 1.2.0-SNAPSHOT jar kafka-connect-fs From e141293e53630dc7a373014c1b8736e3a0170a48 Mon Sep 17 00:00:00 2001 From: Kyle Brodie Date: Sat, 22 Aug 2020 12:10:49 -0700 Subject: [PATCH 02/21] Use header_names instead of column_N if available (#72) * Use header_names instead of column_N if available * Move header check into buildSchema and remove hasHeader parameter * move .forEach to next line to match the formatting of the previous version of this method * Become less functional since ifPresentOrElse is not available in Java 8 * Add test case for custom headers and header extraction off * Format method to look similar to before * Format checkDataWithHeaders to match checkData * Improve readability of buildSchema And rearrange so the case of extracted or user provided headers is handled first and using default headers is handled second This stems from the potentially incorrect assumption that wanting to use provided or extracted headers is more common than wanting to use the default headers * Apply suggestion to buildSchema --- .../fs/file/reader/UnivocityFileReader.java | 26 ++++++++------ .../file/reader/UnivocityFileReaderTest.java | 36 +++++++++++++++++++ 2 files changed, 52 insertions(+), 10 deletions(-) diff --git a/src/main/java/com/github/mmolimar/kafka/connect/fs/file/reader/UnivocityFileReader.java b/src/main/java/com/github/mmolimar/kafka/connect/fs/file/reader/UnivocityFileReader.java index 25a685d..6022e43 100644 --- a/src/main/java/com/github/mmolimar/kafka/connect/fs/file/reader/UnivocityFileReader.java +++ b/src/main/java/com/github/mmolimar/kafka/connect/fs/file/reader/UnivocityFileReader.java @@ -84,19 +84,25 @@ public UnivocityFileReader(FileSystem fs, Path filePath, Map con super(fs, filePath, new UnivocityToStruct(), config); this.iterator = iterateRecords(); - this.schema = buildSchema(this.iterator, settings.isHeaderExtractionEnabled(), config); + this.schema = buildSchema(this.iterator, config); } - private Schema buildSchema(ResultIterator it, boolean hasHeader, Map config) { + private Schema buildSchema(ResultIterator it, Map config) { SchemaBuilder builder = SchemaBuilder.struct(); - if (it.hasNext() && !hasHeader) { - Record first = it.next(); - List dataTypes = getDataTypes(config, first.getValues()); - IntStream.range(0, first.getValues().length) - .forEach(index -> builder.field(DEFAULT_COLUMN_NAME + (index + 1), dataTypes.get(index))); - seek(0); - } else if (hasHeader) { - Optional.ofNullable(it.getContext().headers()).ifPresent(headers -> { + if (iterator.hasNext() && !settings.isHeaderExtractionEnabled()) { + String[] headers; + if (settings.getHeaders() == null || settings.getHeaders().length == 0) { + Record first = iterator.next(); + headers = new String[first.getValues().length]; + IntStream.range(0, headers.length).forEach(index -> headers[index] = DEFAULT_COLUMN_NAME + (index + 1)); + seek(0); + } else { + headers = settings.getHeaders(); + } + List dataTypes = getDataTypes(config, headers); + IntStream.range(0, headers.length).forEach(index -> builder.field(headers[index], dataTypes.get(index))); + } else if (settings.isHeaderExtractionEnabled()) { + Optional.ofNullable(iterator.getContext().headers()).ifPresent(headers -> { List dataTypes = getDataTypes(config, headers); IntStream.range(0, headers.length) .forEach(index -> builder.field(headers[index], dataTypes.get(index))); diff --git a/src/test/java/com/github/mmolimar/kafka/connect/fs/file/reader/UnivocityFileReaderTest.java b/src/test/java/com/github/mmolimar/kafka/connect/fs/file/reader/UnivocityFileReaderTest.java index 27c0edf..49449ea 100644 --- a/src/test/java/com/github/mmolimar/kafka/connect/fs/file/reader/UnivocityFileReaderTest.java +++ b/src/test/java/com/github/mmolimar/kafka/connect/fs/file/reader/UnivocityFileReaderTest.java @@ -94,6 +94,29 @@ public void readAllDataWithoutHeader(ReaderFsTestConfig fsConfig) throws IOExcep assertEquals(NUM_RECORDS, recordCount, "The number of records in the file does not match"); } + @ParameterizedTest + @MethodSource("fileSystemConfigProvider") + public void readAllDataWithCustomHeaders(ReaderFsTestConfig fsConfig) throws IOException { + Path file = createDataFile(fsConfig, false); + Map readerConfig = getReaderConfig(); + readerConfig.put(T.FILE_READER_DELIMITED_SETTINGS_HEADER, "false"); + // NOTE: 9 custom header names to match the quantity of static fields + // FIELD_COLUMN1, ... , FIELD_COLUMN9 + String[] headers = new String[] { "a", "b", "c", "d", "e", "f", "g", "h", "i" }; + readerConfig.put(T.FILE_READER_DELIMITED_SETTINGS_HEADER_NAMES, String.join(",", headers)); + FileReader reader = getReader(fsConfig.getFs(), file, readerConfig); + + assertTrue(reader.hasNext()); + + int recordCount = 0; + while (reader.hasNext()) { + Struct record = reader.next(); + checkDataWithHeaders(record, recordCount, headers); + recordCount++; + } + assertEquals(NUM_RECORDS, recordCount, "The number of records in the file does not match"); + } + @ParameterizedTest @MethodSource("fileSystemConfigProvider") public void readAllDataWithoutSchema(ReaderFsTestConfig fsConfig) throws IOException { @@ -261,6 +284,19 @@ protected void checkData(Struct record, long index) { ); } + protected void checkDataWithHeaders(Struct record, long index, String[] headers) { + assertAll(() -> assertEquals((byte) 2, record.get(headers[0])), + () -> assertEquals((short) 4, record.get(headers[1])), + () -> assertEquals(8, record.get(headers[2])), + () -> assertEquals(16L, record.get(headers[3])), + () -> assertEquals(32.32f, record.get(headers[4])), + () -> assertEquals(64.64d, record.get(headers[5])), + () -> assertEquals(true, record.get(headers[6])), + () -> assertEquals("test bytes", new String((byte[]) record.get(headers[7]))), + () -> assertEquals("test string", record.get(headers[8])) + ); + } + protected void checkDataString(Struct record) { assertAll( () -> assertEquals("2", record.get(FIELD_COLUMN1)), From 7fd5c4989f928fa0dceeace79bf5671acf84e156 Mon Sep 17 00:00:00 2001 From: Kyle Brodie Date: Wed, 26 Aug 2020 19:06:14 -0700 Subject: [PATCH 03/21] Limit log message to non-empty file readers (#74) This reduces the verbosity of INFO level logs in FsSourceTask from files which are being skipped by the policy or are empty. If users still want to see that they didn't process anything for a file or that they tried to process an empty file they can drop the level down to DEBUG and they'll see the log message on line 105 --- .../java/com/github/mmolimar/kafka/connect/fs/FsSourceTask.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/java/com/github/mmolimar/kafka/connect/fs/FsSourceTask.java b/src/main/java/com/github/mmolimar/kafka/connect/fs/FsSourceTask.java index bb74d5e..5a1ce50 100644 --- a/src/main/java/com/github/mmolimar/kafka/connect/fs/FsSourceTask.java +++ b/src/main/java/com/github/mmolimar/kafka/connect/fs/FsSourceTask.java @@ -89,7 +89,7 @@ public List poll() { Map partitionKey = makePartitionKey.apply(metadata); Map offset = Optional.ofNullable(offsets.get(partitionKey)).orElse(new HashMap<>()); try (FileReader reader = policy.offer(metadata, offset)) { - log.info("{} Processing records for file {}...", this, metadata); + if (reader.hasNext()) log.info("{} Processing records for file {}...", this, metadata); while (reader.hasNext()) { Struct record = reader.next(); // TODO change FileReader interface in the next major version From e8db7c96a2d36313ce90ce1467b95094a8e19f59 Mon Sep 17 00:00:00 2001 From: Mario Molina Date: Sun, 30 Aug 2020 11:56:01 -0500 Subject: [PATCH 04/21] Cobrix dependency --- pom.xml | 24 ++++++++++++++++++++++++ 1 file changed, 24 insertions(+) diff --git a/pom.xml b/pom.xml index a0af373..9aee61a 100644 --- a/pom.xml +++ b/pom.xml @@ -53,6 +53,8 @@ 1.11.0 1.6.3 2.8.4 + 2.1.1 + 2.11.12 9.0.2 0.1.54 5.6.2 @@ -62,6 +64,7 @@ ${maven-compiler.source} 3.2.0 3.8.1 + 4.4.0 3.3.0 0.8.5 4.3.0 @@ -121,6 +124,11 @@ univocity-parsers ${univocity.version} + + za.co.absa.cobrix + cobol-parser_2.12 + ${cobrix.version} + com.cronutils cron-utils @@ -184,6 +192,22 @@ ${maven-compiler.target} + + net.alchim31.maven + scala-maven-plugin + ${maven-scala-plugin.version} + + ${scala.version} + + + + + compile + testCompile + + + + org.apache.maven.plugins maven-surefire-plugin From 2139d2361534a774971d05dc819506d4875a08a6 Mon Sep 17 00:00:00 2001 From: Mario Molina Date: Sun, 30 Aug 2020 21:43:31 -0500 Subject: [PATCH 05/21] New Cobol file reader --- .../fs/file/reader/CobolFileReader.java | 330 ++++++++++++++++++ .../connect/fs/file/reader/CobolReader.scala | 15 + .../fs/file/reader/CobolFileReaderTest.java | 103 ++++++ .../file/reader/data/cobol/companies.cpy | 11 + .../file/reader/data/cobol/companies.dat | Bin 0 -> 6800 bytes 5 files changed, 459 insertions(+) create mode 100644 src/main/java/com/github/mmolimar/kafka/connect/fs/file/reader/CobolFileReader.java create mode 100644 src/main/scala/com/github/mmolimar/kafka/connect/fs/file/reader/CobolReader.scala create mode 100644 src/test/java/com/github/mmolimar/kafka/connect/fs/file/reader/CobolFileReaderTest.java create mode 100644 src/test/resources/file/reader/data/cobol/companies.cpy create mode 100644 src/test/resources/file/reader/data/cobol/companies.dat diff --git a/src/main/java/com/github/mmolimar/kafka/connect/fs/file/reader/CobolFileReader.java b/src/main/java/com/github/mmolimar/kafka/connect/fs/file/reader/CobolFileReader.java new file mode 100644 index 0000000..c9655ee --- /dev/null +++ b/src/main/java/com/github/mmolimar/kafka/connect/fs/file/reader/CobolFileReader.java @@ -0,0 +1,330 @@ +package com.github.mmolimar.kafka.connect.fs.file.reader; + +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.SchemaBuilder; +import org.apache.kafka.connect.data.Struct; +import org.apache.kafka.connect.errors.ConnectException; +import scala.collection.Seq; +import za.co.absa.cobrix.cobol.parser.ast.Group; +import za.co.absa.cobrix.cobol.parser.ast.Primitive; +import za.co.absa.cobrix.cobol.parser.ast.Statement; +import za.co.absa.cobrix.cobol.parser.ast.datatype.AlphaNumeric; +import za.co.absa.cobrix.cobol.parser.ast.datatype.COMP1; +import za.co.absa.cobrix.cobol.parser.ast.datatype.Decimal; +import za.co.absa.cobrix.cobol.parser.ast.datatype.Integral; +import za.co.absa.cobrix.cobol.parser.common.Constants; +import za.co.absa.cobrix.cobol.parser.decoders.FloatingPointFormat$; +import za.co.absa.cobrix.cobol.parser.encoding.RAW$; +import za.co.absa.cobrix.cobol.parser.policies.CommentPolicy; +import za.co.absa.cobrix.cobol.parser.policies.DebugFieldsPolicy$; +import za.co.absa.cobrix.cobol.parser.policies.StringTrimmingPolicy$; +import za.co.absa.cobrix.cobol.reader.VarLenReader; +import za.co.absa.cobrix.cobol.reader.extractors.record.RecordHandler; +import za.co.absa.cobrix.cobol.reader.parameters.ReaderParameters; +import za.co.absa.cobrix.cobol.reader.policies.SchemaRetentionPolicy$; +import za.co.absa.cobrix.cobol.reader.schema.CobolSchema; +import za.co.absa.cobrix.cobol.reader.stream.SimpleStream; + +import java.io.BufferedReader; +import java.io.IOException; +import java.io.InputStream; +import java.io.InputStreamReader; +import java.util.*; +import java.util.stream.IntStream; + +import static com.github.mmolimar.kafka.connect.fs.FsSourceTaskConfig.FILE_READER_PREFIX; +import static scala.collection.JavaConverters.*; + +public class CobolFileReader extends AbstractFileReader { + + private static final String FILE_READER_COBOL = FILE_READER_PREFIX + "cobol."; + private static final String FILE_READER_COPYBOOK_PREFIX = FILE_READER_COBOL + "copybook."; + + public static final String FILE_READER_COPYBOOK_CONTENT = FILE_READER_COPYBOOK_PREFIX + "content"; + public static final String FILE_READER_COPYBOOK_PATH = FILE_READER_COPYBOOK_PREFIX + "path"; + + private final VarLenReader reader; + private final SimpleStream stream; + private final Schema schema; + private String copybook; + private Iterator> iterator; + private ReaderParameters params; + private boolean closed; + + public CobolFileReader(FileSystem fs, Path filePath, Map config) throws Exception { + super(fs, filePath, new CobolToStruct(), config); + + this.reader = CobolReader.varLenReader(copybook, params); + this.stream = new FSStream(getFs(), getFilePath()); + this.schema = extractSchema(reader.getCobolSchema()); + this.iterator = initIterator(); + this.closed = false; + } + + private Iterator> initIterator() throws Exception { + return asJavaIterator(reader.getRecordIterator(stream, 0, 0, 0) + .map(it -> seqAsJavaList(it.seq()))); + } + + private Schema extractSchema(CobolSchema cobolSchema) { + SchemaBuilder builder = SchemaBuilder.struct(); + seqAsJavaList(cobolSchema.getCobolSchema().ast().children()) + .forEach(child -> builder.field(child.name(), schemaForField(child))); + + return builder.build(); + } + + private Schema schemaForField(Statement statement) { + if (statement instanceof Group) { + Group group = (Group) statement; + SchemaBuilder builder = SchemaBuilder.struct(); + seqAsJavaList(group.children()).forEach(child -> builder.field(child.name(), schemaForField(child))); + + return builder.build(); + } + Primitive primitive = (Primitive) statement; + if (primitive.dataType() instanceof Integral) { + Integral dt = (Integral) primitive.dataType(); + if (dt.precision() > Constants.maxLongPrecision()) { + return Schema.OPTIONAL_FLOAT64_SCHEMA; + } else if (dt.precision() > Constants.maxIntegerPrecision()) { + return Schema.OPTIONAL_INT64_SCHEMA; + } else { + return Schema.OPTIONAL_INT32_SCHEMA; + } + } else if (primitive.dataType() instanceof Decimal) { + Decimal dt = (Decimal) primitive.dataType(); + if (dt.compact().exists(c -> c instanceof COMP1)) { + return Schema.OPTIONAL_FLOAT32_SCHEMA; + } + return Schema.OPTIONAL_FLOAT64_SCHEMA; + } else { + AlphaNumeric dt = (AlphaNumeric) primitive.dataType(); + if (dt.enc().exists(enc -> enc instanceof RAW$)) { + return Schema.OPTIONAL_BYTES_SCHEMA; + } + return Schema.OPTIONAL_STRING_SCHEMA; + } + } + + @Override + protected void configure(Map config) { + copybook = copybookContent(config); + params = getReaderParameters(config); + } + + private String copybookContent(Map config) { + String content = Optional.ofNullable(config.get(FILE_READER_COPYBOOK_PATH)) + .map(Path::new) + .map(path -> { + StringBuilder sb = new StringBuilder(); + try (InputStream is = getFs().open(path); + BufferedReader br = new BufferedReader(new InputStreamReader(is))) { + br.lines().forEach(sb::append); + } catch (IOException ioe) { + throw new ConnectException("Cannot read Copybook file: " + path, ioe); + } + return sb.toString(); + }) + .orElseGet(() -> config.get(FILE_READER_COPYBOOK_CONTENT)); + + if (content == null || content.trim().isEmpty()) { + throw new ConnectException("Copybook is not specified."); + } + return content; + } + + private ReaderParameters getReaderParameters(Map config) { + return new ReaderParameters( + true, + "common", + scala.Option.apply(null), + "", + true, + FloatingPointFormat$.MODULE$.IBM(), + false, // variableSizeOccurs + scala.Option.apply(null), + true, // isRecordSequence + false, + false, + 0, + false, + scala.Option.apply(null), + scala.Option.apply(null), + scala.Option.apply(null), + 0, + 0, + 0, + 0, + false, + SchemaRetentionPolicy$.MODULE$.KeepOriginal(), + StringTrimmingPolicy$.MODULE$.TrimBoth(), + scala.Option.apply(null), + new CommentPolicy(true, 6, 72), + false, + true, + scala.collection.immutable.Nil$.empty(), + null, + DebugFieldsPolicy$.MODULE$.NoDebug(), + scala.Option.apply(null), + scala.Option.apply(null), + "" + ); + } + + @Override + protected boolean hasNextRecord() { + return iterator.hasNext(); + } + + @Override + protected CobolRecord nextRecord() { + incrementOffset(); + return new CobolRecord(schema, iterator.next()); + } + + @Override + protected void seekFile(long offset) { + if (currentOffset() > offset) { + try { + iterator = initIterator(); + } catch (Exception e) { + throw new ConnectException("Error seeking file: " + getFilePath(), e); + } + closed = false; + setOffset(0); + } + while (hasNext() && currentOffset() < offset) { + nextRecord(); + } + } + + @Override + protected boolean isClosed() { + return closed; + } + + @Override + public void close() { + try { + stream.close(); + } catch (Exception e) { + log.warn("{} An error has occurred while closing file stream.", this, e); + } + closed = true; + } + + private static class FSStream implements SimpleStream { + + private final FileSystem fs; + private final Path file; + private final FSDataInputStream stream; + + FSStream(FileSystem fs, Path file) throws IOException { + this.fs = fs; + this.file = file; + this.stream = this.fs.open(file); + } + + @Override + public long size() { + try { + return fs.getContentSummary(file).getLength(); + } catch (IOException ioe) { + throw new ConnectException("Cannot retrieve length for file: " + file, ioe); + } + } + + @Override + public long offset() { + try { + return stream.getPos(); + } catch (IOException ioe) { + throw new ConnectException("Cannot get current position for file: " + file, ioe); + } + } + + @Override + public String inputFileName() { + return file.toString(); + } + + @Override + public byte[] next(int numberOfBytes) throws IOException { + byte[] bytes = new byte[(int) Math.min(numberOfBytes, size() - offset())]; + stream.readFully(bytes); + return bytes; + } + + @Override + public void close() throws IOException { + stream.close(); + } + } + + static class CobolToStruct implements ReaderAdapter { + + public Struct apply(CobolRecord record) { + Struct struct = new Struct(record.schema); + record.row.forEach(col -> { + Map column = (Map) col; + column.forEach((k, v) -> struct.put(k, mapValue(record.schema.field(k).schema(), v))); + }); + return struct; + } + + private Object mapValue(Schema schema, Object value) { + if (value == null) return null; + + if (value instanceof String || value instanceof Integer || value instanceof Double) { + return value; + } + Struct struct = new Struct(schema); + Map map = (Map) value; + map.forEach((k, v) -> struct.put(k, mapValue(schema.field(k).schema(), v))); + return struct; + } + } + + static class CobolRecord { + + final Schema schema; + final List row; + + CobolRecord(Schema schema, List row) { + this.schema = schema; + this.row = row; + } + + } + + static class StructHandler implements RecordHandler> { + + @Override + public Map create(Object[] values, Group group) { + return Collections.singletonMap(group.name(), mapValues(group, values)); + } + + @Override + public Seq toSeq(Map record) { + return asScalaBuffer(new ArrayList<>(record.values())).toSeq(); + } + + private Map mapValues(Group group, Object[] values) { + List statements = seqAsJavaList(group.children().toSeq()); + return IntStream.range(0, values.length) + .mapToObj(index -> new AbstractMap.SimpleEntry<>(statements.get(index), values[index])) + .map(entry -> transform(entry.getKey(), entry.getValue())) + .collect(HashMap::new, (m, e) -> m.put(e.getKey(), e.getValue()), HashMap::putAll); + } + + private Map.Entry transform(Statement child, Object value) { + Object childValue = child instanceof Group ? ((Map) value).get(child.name()) : value; + return new AbstractMap.SimpleEntry<>(child.name(), childValue); + } + + } +} diff --git a/src/main/scala/com/github/mmolimar/kafka/connect/fs/file/reader/CobolReader.scala b/src/main/scala/com/github/mmolimar/kafka/connect/fs/file/reader/CobolReader.scala new file mode 100644 index 0000000..e01f1d9 --- /dev/null +++ b/src/main/scala/com/github/mmolimar/kafka/connect/fs/file/reader/CobolReader.scala @@ -0,0 +1,15 @@ +package com.github.mmolimar.kafka.connect.fs.file.reader + +import com.github.mmolimar.kafka.connect.fs.file.reader.CobolFileReader.StructHandler +import za.co.absa.cobrix.cobol.reader.parameters.ReaderParameters +import za.co.absa.cobrix.cobol.reader.{VarLenNestedReader, VarLenReader} + +import scala.collection.Seq + +protected object CobolReader { + + def varLenReader(copybookContent: String, params: ReaderParameters): VarLenReader = { + new VarLenNestedReader[java.util.Map[String, AnyRef]](Seq(copybookContent), params, new StructHandler()) + } + +} diff --git a/src/test/java/com/github/mmolimar/kafka/connect/fs/file/reader/CobolFileReaderTest.java b/src/test/java/com/github/mmolimar/kafka/connect/fs/file/reader/CobolFileReaderTest.java new file mode 100644 index 0000000..4e79874 --- /dev/null +++ b/src/test/java/com/github/mmolimar/kafka/connect/fs/file/reader/CobolFileReaderTest.java @@ -0,0 +1,103 @@ +package com.github.mmolimar.kafka.connect.fs.file.reader; + +import org.apache.hadoop.fs.Path; +import org.apache.kafka.connect.data.Struct; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.MethodSource; + +import java.io.*; +import java.nio.file.Files; +import java.nio.file.StandardCopyOption; +import java.util.HashMap; +import java.util.Map; +import java.util.stream.IntStream; + +import static org.junit.jupiter.api.Assertions.*; + +public class CobolFileReaderTest extends FileReaderTestBase { + + private static final String FILE_EXTENSION = "dt"; + + @Override + protected Class getReaderClass() { + return CobolFileReader.class; + } + + @Override + protected Path createDataFile(ReaderFsTestConfig fsConfig, Object... args) throws IOException { + File cobolFile = File.createTempFile("test-", "." + getFileExtension()); + try (InputStream is = CobolFileReaderTest.class.getResourceAsStream("/file/reader/data/cobol/companies.dat")) { + Files.copy(is, cobolFile.toPath(), StandardCopyOption.REPLACE_EXISTING); + } + IntStream.range(0, NUM_RECORDS).forEach(index -> fsConfig.offsetsByIndex().put(index, (long) index)); + Path path = new Path(new Path(fsConfig.getFsUri()), cobolFile.getName()); + fsConfig.getFs().copyFromLocalFile(new Path(cobolFile.getAbsolutePath()), path); + + return path; + } + + @ParameterizedTest + @MethodSource("fileSystemConfigProvider") + public void invalidFileFormat(ReaderFsTestConfig fsConfig) throws IOException { + File tmp = File.createTempFile("test-", "." + getFileExtension()); + try (BufferedWriter writer = new BufferedWriter(new FileWriter(tmp))) { + writer.write("test"); + } + Path path = new Path(new Path(fsConfig.getFsUri()), tmp.getName()); + fsConfig.getFs().moveFromLocalFile(new Path(tmp.getAbsolutePath()), path); + getReader(fsConfig.getFs(), path, getReaderConfig()); + } + + @ParameterizedTest + @MethodSource("fileSystemConfigProvider") + public void emptyFile(ReaderFsTestConfig fsConfig) throws IOException { + File tmp = File.createTempFile("test-", "." + getFileExtension()); + Path path = new Path(new Path(fsConfig.getFsUri()), tmp.getName()); + fsConfig.getFs().moveFromLocalFile(new Path(tmp.getAbsolutePath()), path); + getReader(fsConfig.getFs(), path, getReaderConfig()); + } + + @Override + protected Map getReaderConfig() { + return new HashMap() {{ + put(CobolFileReader.FILE_READER_COPYBOOK_CONTENT, copybookContent()); + }}; + } + + private String copybookContent() { + return " 01 COMPANY-DETAILS.\n" + + " 05 SEGMENT-ID PIC X(5).\n" + + " 05 COMPANY-ID PIC X(10).\n" + + " 05 STATIC-DETAILS.\n" + + " 10 COMPANY-NAME PIC X(15).\n" + + " 10 ADDRESS PIC X(25).\n" + + " 10 TAXPAYER.\n" + + " 15 TAXPAYER-TYPE PIC X(1).\n" + + " 15 TAXPAYER-STR PIC X(8).\n" + + " 15 TAXPAYER-NUM REDEFINES TAXPAYER-STR\n" + + " PIC 9(8) COMP."; + } + + @Override + protected String getFileExtension() { + return FILE_EXTENSION; + } + + @Override + protected void checkData(Struct record, long index) { + Struct companyDetails = record.getStruct("COMPANY_DETAILS"); + Struct staticDetails = companyDetails.getStruct("STATIC_DETAILS"); + Struct taxpayer = staticDetails.getStruct("TAXPAYER"); + assertAll( + () -> assertEquals("C", companyDetails.getString("SEGMENT_ID")), + () -> assertEquals(String.format("%010d", index), companyDetails.getString("COMPANY_ID")), + + () -> assertEquals("Sample Q&A Ltd.", staticDetails.getString("COMPANY_NAME")), + () -> assertEquals("223344 AK ave, Wonderland", staticDetails.getString("ADDRESS")), + + () -> assertEquals("A", taxpayer.getString("TAXPAYER_TYPE")), + () -> assertEquals("88888888", taxpayer.getString("TAXPAYER_STR")), + () -> assertNull(taxpayer.getInt32("TAXPAYER_NUM")) + ); + } +} diff --git a/src/test/resources/file/reader/data/cobol/companies.cpy b/src/test/resources/file/reader/data/cobol/companies.cpy new file mode 100644 index 0000000..bf237d0 --- /dev/null +++ b/src/test/resources/file/reader/data/cobol/companies.cpy @@ -0,0 +1,11 @@ + 01 COMPANY-DETAILS. + 05 SEGMENT-ID PIC X(5). + 05 COMPANY-ID PIC X(10). + 05 STATIC-DETAILS. + 10 COMPANY-NAME PIC X(15). + 10 ADDRESS PIC X(25). + 10 TAXPAYER. + 15 TAXPAYER-TYPE PIC X(1). + 15 TAXPAYER-STR PIC X(8). + 15 TAXPAYER-NUM REDEFINES TAXPAYER-STR + PIC 9(8) COMP. diff --git a/src/test/resources/file/reader/data/cobol/companies.dat b/src/test/resources/file/reader/data/cobol/companies.dat new file mode 100644 index 0000000000000000000000000000000000000000..c31ef0e8a4b3601ab22781eff4dc7aeb904a8460 GIT binary patch literal 6800 zcmb`_txW_`7(me}Z9IlPsA7`Y3YLYS359Iv01O&|-uJzK2ZCTwgmVkH>MYB;JTG5e zetO?-?~jM|?fSgFKJHf$IhV4YU)JsEaI=2go%X}SaXanj@!j(4k-#IsBZEf)j|v_Q zJUVy`@Ra2mjA0H*<* z25_2z(+r$u;4}lL892?rX$DR+aGHVB44f9=v;e0CI4!_w0Zt2WT7c66oEG4;0H+l= zt-xsoPAhO)fzt|{R^YS(rxiG@z-a?c8*tiy(*~S2;Isj!4LEJUX#-9haN2>>4xD!2 zv;(IdIPJh`2TnV1+JVy!oDSf00H*^u9l+@TP6u#0fYSk-4&ZbErxQ4x!07}|CvZA} S(+QkT;B*3~6F8lJclrqy7UW?7 literal 0 HcmV?d00001 From 11fec9ccdaa5f8b144bb0636679eeed71ec7f93b Mon Sep 17 00:00:00 2001 From: Mario Molina Date: Mon, 31 Aug 2020 19:09:51 -0500 Subject: [PATCH 06/21] Enable cleanup options to manage processed files in policies --- .../kafka/connect/fs/FsSourceTaskConfig.java | 42 +++++++++++++++++++ .../connect/fs/file/reader/FileReader.java | 2 - .../connect/fs/policy/AbstractPolicy.java | 42 +++++++++++++++---- 3 files changed, 77 insertions(+), 9 deletions(-) diff --git a/src/main/java/com/github/mmolimar/kafka/connect/fs/FsSourceTaskConfig.java b/src/main/java/com/github/mmolimar/kafka/connect/fs/FsSourceTaskConfig.java index f3b56ed..b5c434b 100644 --- a/src/main/java/com/github/mmolimar/kafka/connect/fs/FsSourceTaskConfig.java +++ b/src/main/java/com/github/mmolimar/kafka/connect/fs/FsSourceTaskConfig.java @@ -25,6 +25,18 @@ public class FsSourceTaskConfig extends FsSourceConnectorConfig { private static final String POLICY_BATCH_SIZE_DOC = "Number of files to process at a time. Non-positive values disable batching."; private static final String POLICY_BATCH_SIZE_DISPLAY = "Files per batch"; + public static final String POLICY_CLEANUP = POLICY_PREFIX + "cleanup"; + private static final String POLICY_CLEANUP_DOC = "Cleanup strategy to use when skipping files."; + private static final String POLICY_CLEANUP_DISPLAY = "Cleanup strategy"; + + public static final String POLICY_CLEANUP_MOVE_DIR = POLICY_CLEANUP + ".move"; + private static final String POLICY_CLEANUP_MOVE_DIR_DOC = "Target directory to move files for MOVE cleanup strategy."; + private static final String POLICY_CLEANUP_MOVE_DIR_DISPLAY = "Target directory"; + + public static final String POLICY_CLEANUP_MOVE_DIR_PREFIX = POLICY_CLEANUP_MOVE_DIR + ".prefix"; + private static final String POLICY_CLEANUP_MOVE_DIR_PREFIX_DOC = "Prefix to set to moved files."; + private static final String POLICY_CLEANUP_MOVE_DIR_PREFIX_DISPLAY = "File prefix"; + public static final String POLICY_PREFIX_FS = POLICY_PREFIX + "fs."; public static final String FILE_READER_CLASS = FILE_READER_PREFIX + "class"; @@ -95,6 +107,36 @@ public static ConfigDef conf() { ++order, ConfigDef.Width.MEDIUM, POLICY_BATCH_SIZE_DISPLAY + ).define( + POLICY_CLEANUP, + ConfigDef.Type.STRING, + null, + ConfigDef.Importance.MEDIUM, + POLICY_CLEANUP_DOC, + POLICY_GROUP, + ++order, + ConfigDef.Width.MEDIUM, + POLICY_CLEANUP_DISPLAY + ).define( + POLICY_CLEANUP_MOVE_DIR, + ConfigDef.Type.STRING, + null, + ConfigDef.Importance.MEDIUM, + POLICY_CLEANUP_MOVE_DIR_DOC, + POLICY_GROUP, + ++order, + ConfigDef.Width.MEDIUM, + POLICY_CLEANUP_MOVE_DIR_DISPLAY + ).define( + POLICY_CLEANUP_MOVE_DIR_PREFIX, + ConfigDef.Type.STRING, + null, + ConfigDef.Importance.LOW, + POLICY_CLEANUP_MOVE_DIR_PREFIX_DOC, + POLICY_GROUP, + ++order, + ConfigDef.Width.MEDIUM, + POLICY_CLEANUP_MOVE_DIR_PREFIX_DISPLAY ).define( FILE_READER_CLASS, ConfigDef.Type.CLASS, diff --git a/src/main/java/com/github/mmolimar/kafka/connect/fs/file/reader/FileReader.java b/src/main/java/com/github/mmolimar/kafka/connect/fs/file/reader/FileReader.java index a5fe758..1fecfe1 100644 --- a/src/main/java/com/github/mmolimar/kafka/connect/fs/file/reader/FileReader.java +++ b/src/main/java/com/github/mmolimar/kafka/connect/fs/file/reader/FileReader.java @@ -11,8 +11,6 @@ public interface FileReader extends Iterator, Closeable { Path getFilePath(); - Struct next(); - void seek(long offset); long currentOffset(); diff --git a/src/main/java/com/github/mmolimar/kafka/connect/fs/policy/AbstractPolicy.java b/src/main/java/com/github/mmolimar/kafka/connect/fs/policy/AbstractPolicy.java index 9d23c04..5e35da4 100644 --- a/src/main/java/com/github/mmolimar/kafka/connect/fs/policy/AbstractPolicy.java +++ b/src/main/java/com/github/mmolimar/kafka/connect/fs/policy/AbstractPolicy.java @@ -7,10 +7,7 @@ import com.github.mmolimar.kafka.connect.fs.util.ReflectionUtils; import com.github.mmolimar.kafka.connect.fs.util.TailCall; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.LocatedFileStatus; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.RemoteIterator; +import org.apache.hadoop.fs.*; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.connect.data.Struct; import org.apache.kafka.connect.errors.ConnectException; @@ -39,9 +36,16 @@ abstract class AbstractPolicy implements Policy { private final AtomicLong executions; private final boolean recursive; private final int batchSize; + private final Cleanup cleanup; + private final String cleanupDir; + private final String prefixCleanup; private Iterator> partitions; private boolean interrupted; + enum Cleanup { + NONE, MOVE, DELETE + } + public AbstractPolicy(FsSourceTaskConfig conf) throws IOException { this.fileSystems = new ArrayList<>(); this.conf = conf; @@ -49,6 +53,11 @@ public AbstractPolicy(FsSourceTaskConfig conf) throws IOException { this.recursive = conf.getBoolean(FsSourceTaskConfig.POLICY_RECURSIVE); this.fileRegexp = Pattern.compile(conf.getString(FsSourceTaskConfig.POLICY_REGEXP)); this.batchSize = conf.getInt(FsSourceTaskConfig.POLICY_BATCH_SIZE); + this.cleanup = Optional.ofNullable(conf.getString(FsSourceTaskConfig.POLICY_CLEANUP)) + .map(c -> Cleanup.valueOf(c.toUpperCase())).orElse(Cleanup.NONE); + this.prefixCleanup = Optional.ofNullable(conf.getString(FsSourceTaskConfig.POLICY_CLEANUP_MOVE_DIR_PREFIX)) + .orElse(""); + this.cleanupDir = conf.getString(FsSourceTaskConfig.POLICY_CLEANUP_MOVE_DIR); this.interrupted = false; this.partitions = Collections.emptyIterator(); @@ -227,8 +236,7 @@ current, new Path(metadata.getPath()), conf.originals() long fileSize = Long.parseLong(offsetMap.getOrDefault("file-size", "0").toString()); boolean eof = Boolean.parseBoolean(offsetMap.getOrDefault("eof", "false").toString()); if (metadata.getLen() == fileSize && eof) { - log.info("{} Skipping file [{}] due to it was already processed.", this, metadata.getPath()); - return emptyFileReader(new Path(metadata.getPath())); + return cleanupAndReturn(current, new Path(metadata.getPath())); } else { log.info("{} Seeking to offset [{}] for file [{}].", this, offsetMap.get("offset"), metadata.getPath()); @@ -269,7 +277,27 @@ public FileMetadata next() { }; } - private FileReader emptyFileReader(Path filePath) { + private FileReader cleanupAndReturn(FileSystem srcFs, Path filePath) { + try { + switch (cleanup) { + case NONE: + log.info("{} Skipping file [{}] due to it was already processed.", this, filePath); + break; + case MOVE: + Path target = new Path(cleanupDir, prefixCleanup + filePath.getName()); + FileSystem dstFs = FileSystem.newInstance(target.toUri(), srcFs.getConf()); + log.info("{} Moving file [{}] to [{}] due to it was already processed.", this, filePath, target); + FileUtil.copy(srcFs, filePath, srcFs, target, true, true, dstFs.getConf()); + break; + case DELETE: + log.info("{} Deleting file [{}] due to it was already processed.", this, filePath); + srcFs.delete(filePath, false); + break; + } + } catch (IOException ioe) { + log.warn("{} Cannot apply cleanup of type {} in file [{}]. Error message: {}", this, cleanup, filePath, ioe.getMessage()); + } + return new FileReader() { @Override public Path getFilePath() { From 75516b746c2351ac09a5e199165af2353ef90e2c Mon Sep 17 00:00:00 2001 From: Mario Molina Date: Mon, 31 Aug 2020 19:18:15 -0500 Subject: [PATCH 07/21] Updating docs and tests for the cleanup strategies --- config/kafka-connect-fs.properties | 1 + docs/source/config_options.rst | 20 +++++ docs/source/connector.rst | 2 + .../connect/fs/policy/PolicyTestBase.java | 84 +++++++++++++++++++ 4 files changed, 107 insertions(+) diff --git a/config/kafka-connect-fs.properties b/config/kafka-connect-fs.properties index 27b9eb9..8596475 100644 --- a/config/kafka-connect-fs.properties +++ b/config/kafka-connect-fs.properties @@ -7,5 +7,6 @@ policy.class=com.github.mmolimar.kafka.connect.fs.policy.SimplePolicy policy.recursive=true policy.regexp=^.*\.txt$ policy.batch_size=0 +policy.cleanup=none file_reader.class=com.github.mmolimar.kafka.connect.fs.file.reader.TextFileReader file_reader.batch_size=0 diff --git a/docs/source/config_options.rst b/docs/source/config_options.rst index 0abb591..112fb43 100644 --- a/docs/source/config_options.rst +++ b/docs/source/config_options.rst @@ -98,6 +98,26 @@ General config properties for this connector. * Default: ``0`` * Importance: medium +``policy.cleanup`` + Cleanup strategy to use when skipping files. + + * Type: enum (available values ``none``, ``move`` and ``delete``) + * Default: ``none`` + * Importance: medium + +``policy.cleanup.move`` + Target directory to move files for the ``move`` cleanup strategy. Mandatory just in case of using this strategy. + + * Type: string + * Importance: medium + +``policy.cleanup.move.prefix`` + Prefix to set to the filename in moved files. + + * Type: string + * Default: ```` + * Importance: low + ``policy..`` This represents custom properties you can include based on the policy class specified. diff --git a/docs/source/connector.rst b/docs/source/connector.rst index 948dcb7..37059da 100644 --- a/docs/source/connector.rst +++ b/docs/source/connector.rst @@ -53,6 +53,7 @@ The ``kafka-connect-fs.properties`` file defines the following properties as req policy.recursive=true policy.regexp=.* policy.batch_size=0 + policy.cleanup=none file_reader.class= file_reader.batch_size=0 @@ -68,6 +69,7 @@ The ``kafka-connect-fs.properties`` file defines the following properties as req #. Flag to activate traversed recursion in subdirectories when listing files. #. Regular expression to filter files from the FS. #. Number of files that should be handled at a time. Non-positive values disable batching. +#. Cleanup strategy to manage processed files. #. File reader class to read files from the FS (must implement ``com.github.mmolimar.kafka.connect.fs.file.reader.FileReader`` interface). #. Number of records to process at a time. Non-positive values disable batching. diff --git a/src/test/java/com/github/mmolimar/kafka/connect/fs/policy/PolicyTestBase.java b/src/test/java/com/github/mmolimar/kafka/connect/fs/policy/PolicyTestBase.java index 9fbb429..b9ac7dc 100644 --- a/src/test/java/com/github/mmolimar/kafka/connect/fs/policy/PolicyTestBase.java +++ b/src/test/java/com/github/mmolimar/kafka/connect/fs/policy/PolicyTestBase.java @@ -90,6 +90,17 @@ public void invalidConfig(PolicyFsTestConfig fsConfig) { new FsSourceTaskConfig(new HashMap<>()))); } + @ParameterizedTest + @MethodSource("fileSystemConfigProvider") + public void invalidConfigCleanup(PolicyFsTestConfig fsConfig) { + Map originals = fsConfig.getSourceTaskConfig().originalsStrings(); + originals.put(FsSourceTaskConfig.POLICY_CLEANUP, "invalid"); + assertThrows(ConnectException.class, () -> + ReflectionUtils.makePolicy((Class) fsConfig.getSourceTaskConfig() + .getClass(FsSourceTaskConfig.POLICY_CLASS), + new FsSourceTaskConfig(originals))); + } + @ParameterizedTest @MethodSource("fileSystemConfigProvider") public void interruptPolicy(PolicyFsTestConfig fsConfig) throws IOException { @@ -142,6 +153,79 @@ public void oneFilePerFs(PolicyFsTestConfig fsConfig) throws IOException, Interr assertFalse(it.hasNext()); } + @ParameterizedTest + @MethodSource("fileSystemConfigProvider") + public void oneFilePerFsWithMoveCleanup(PolicyFsTestConfig fsConfig) throws IOException { + FileSystem fs = fsConfig.getFs(); + + Path source = new Path(fsConfig.getFsUri().toString(), "source"); + Path target = new Path(fsConfig.getFsUri().toString(), "target"); + fs.mkdirs(source); + fs.mkdirs(target); + Map originals = fsConfig.getSourceTaskConfig().originalsStrings(); + originals.put(FsSourceTaskConfig.FS_URIS, source.toString()); + originals.put(FsSourceTaskConfig.POLICY_CLEANUP, AbstractPolicy.Cleanup.MOVE.toString()); + originals.put(FsSourceTaskConfig.POLICY_CLEANUP_MOVE_DIR, target.toString()); + originals.put(FsSourceTaskConfig.POLICY_CLEANUP_MOVE_DIR_PREFIX, "processed_"); + + FsSourceTaskConfig cfg = new FsSourceTaskConfig(originals); + try (Policy policy = ReflectionUtils.makePolicy((Class) fsConfig.getSourceTaskConfig() + .getClass(FsSourceTaskConfig.POLICY_CLASS), cfg)) { + Path tmpDir = new Path(source, String.valueOf(System.nanoTime())); + fs.mkdirs(tmpDir); + String filename = System.nanoTime() + ".txt"; + Path filePath = new Path(tmpDir, filename); + fs.createNewFile(filePath); + + FileMetadata metadata = new FileMetadata(filePath.toString(), 0L, Collections.emptyList()); + Map offset = new HashMap<>(); + offset.put("offset", 1); + offset.put("eof", true); + FileReader reader = policy.offer(metadata, offset); + assertFalse(reader.hasNext()); + + assertFalse(fs.exists(new Path(source, filename))); + assertTrue(fs.exists(new Path(target, "processed_" + filename))); + + metadata = new FileMetadata(System.nanoTime() + ".txt", 0L, Collections.emptyList()); + reader = policy.offer(metadata, offset); + assertFalse(reader.hasNext()); + } + } + + @ParameterizedTest + @MethodSource("fileSystemConfigProvider") + public void oneFilePerFsWithDeleteCleanup(PolicyFsTestConfig fsConfig) throws IOException { + FileSystem fs = fsConfig.getFs(); + + Map originals = fsConfig.getSourceTaskConfig().originalsStrings(); + originals.put(FsSourceTaskConfig.FS_URIS, fsConfig.getFsUri().toString()); + originals.put(FsSourceTaskConfig.POLICY_CLEANUP, AbstractPolicy.Cleanup.DELETE.toString()); + + FsSourceTaskConfig cfg = new FsSourceTaskConfig(originals); + try (Policy policy = ReflectionUtils.makePolicy((Class) fsConfig.getSourceTaskConfig() + .getClass(FsSourceTaskConfig.POLICY_CLASS), cfg)) { + Path tmpDir = new Path(fsConfig.getFsUri().toString(), String.valueOf(System.nanoTime())); + fs.mkdirs(tmpDir); + String filename = System.nanoTime() + ".txt"; + Path filePath = new Path(tmpDir, filename); + fs.createNewFile(filePath); + + FileMetadata metadata = new FileMetadata(filePath.toString(), 0L, Collections.emptyList()); + Map offset = new HashMap<>(); + offset.put("offset", 1); + offset.put("eof", true); + FileReader reader = policy.offer(metadata, offset); + assertFalse(reader.hasNext()); + + assertFalse(fs.exists(new Path(tmpDir, filename))); + + metadata = new FileMetadata(System.nanoTime() + ".txt", 0L, Collections.emptyList()); + reader = policy.offer(metadata, offset); + assertFalse(reader.hasNext()); + } + } + @ParameterizedTest @MethodSource("fileSystemConfigProvider") public void recursiveDirectory(PolicyFsTestConfig fsConfig) throws IOException, InterruptedException { From f483a117a905bd7928258281b0e81c63262b2bc1 Mon Sep 17 00:00:00 2001 From: Mario Molina Date: Fri, 4 Sep 2020 17:28:46 -0500 Subject: [PATCH 08/21] Configurable params for Cobol file reader --- pom.xml | 29 ++- .../fs/file/reader/CobolFileReader.java | 178 ++++++++------ .../{CobolReader.scala => CobrixReader.scala} | 2 +- .../fs/file/reader/CobolFileReaderTest.java | 181 +++++++++++++-- .../file/reader/data/cobol/code-pages.cpy | 7 + .../file/reader/data/cobol/code-pages.dt | Bin 0 -> 4500 bytes .../cobol/{companies.dat => companies.dt} | Bin .../file/reader/data/cobol/type-variety.cpy | 217 ++++++++++++++++++ .../file/reader/data/cobol/type-variety.dt | Bin 0 -> 149300 bytes 9 files changed, 525 insertions(+), 89 deletions(-) rename src/main/scala/com/github/mmolimar/kafka/connect/fs/file/reader/{CobolReader.scala => CobrixReader.scala} (94%) create mode 100644 src/test/resources/file/reader/data/cobol/code-pages.cpy create mode 100644 src/test/resources/file/reader/data/cobol/code-pages.dt rename src/test/resources/file/reader/data/cobol/{companies.dat => companies.dt} (100%) create mode 100644 src/test/resources/file/reader/data/cobol/type-variety.cpy create mode 100644 src/test/resources/file/reader/data/cobol/type-variety.dt diff --git a/pom.xml b/pom.xml index 9aee61a..3dd30ee 100644 --- a/pom.xml +++ b/pom.xml @@ -169,6 +169,24 @@ + + net.alchim31.maven + scala-maven-plugin + ${maven-scala-plugin.version} + + ${scala.version} + + + + scala-compile-first + process-resources + + add-source + compile + + + + org.apache.maven.plugins maven-jar-plugin @@ -191,19 +209,11 @@ ${maven-compiler.source} ${maven-compiler.target} - - - net.alchim31.maven - scala-maven-plugin - ${maven-scala-plugin.version} - - ${scala.version} - + compile compile - testCompile @@ -309,6 +319,7 @@ parquet orc sequence + cobol diff --git a/src/main/java/com/github/mmolimar/kafka/connect/fs/file/reader/CobolFileReader.java b/src/main/java/com/github/mmolimar/kafka/connect/fs/file/reader/CobolFileReader.java index c9655ee..a32cde3 100644 --- a/src/main/java/com/github/mmolimar/kafka/connect/fs/file/reader/CobolFileReader.java +++ b/src/main/java/com/github/mmolimar/kafka/connect/fs/file/reader/CobolFileReader.java @@ -8,6 +8,8 @@ import org.apache.kafka.connect.data.Struct; import org.apache.kafka.connect.errors.ConnectException; import scala.collection.Seq; +import scala.math.BigDecimal; +import scala.math.ScalaNumber; import za.co.absa.cobrix.cobol.parser.ast.Group; import za.co.absa.cobrix.cobol.parser.ast.Primitive; import za.co.absa.cobrix.cobol.parser.ast.Statement; @@ -41,14 +43,46 @@ public class CobolFileReader extends AbstractFileReader { private static final String FILE_READER_COBOL = FILE_READER_PREFIX + "cobol."; - private static final String FILE_READER_COPYBOOK_PREFIX = FILE_READER_COBOL + "copybook."; + private static final String FILE_READER_COBOL_READER = FILE_READER_COBOL + "reader."; + private static final String FILE_READER_COBOL_COPYBOOK_PREFIX = FILE_READER_COBOL + "copybook."; + + public static final String FILE_READER_COBOL_COPYBOOK_CONTENT = FILE_READER_COBOL_COPYBOOK_PREFIX + "content"; + public static final String FILE_READER_COBOL_COPYBOOK_PATH = FILE_READER_COBOL_COPYBOOK_PREFIX + "path"; + + public static final String FILE_READER_COBOL_READER_IS_EBCDIC = FILE_READER_COBOL_READER + "is_ebcdic"; + public static final String FILE_READER_COBOL_READER_EBCDIC_CODE_PAGE = FILE_READER_COBOL_READER + "ebcdic_code_page"; + public static final String FILE_READER_COBOL_READER_EBCDIC_CODE_PAGE_CLASS = FILE_READER_COBOL_READER + "ebcdic_code_page_class"; + public static final String FILE_READER_COBOL_READER_ASCII_CHARSET = FILE_READER_COBOL_READER + "ascii_charset"; + public static final String FILE_READER_COBOL_READER_IS_UFT16_BIG_ENDIAN = FILE_READER_COBOL_READER + "is_uft16_big_endian"; + public static final String FILE_READER_COBOL_READER_FLOATING_POINT_FORMAT = FILE_READER_COBOL_READER + "floating_point_format"; + public static final String FILE_READER_COBOL_READER_VARIABLE_SIZE_OCCURS = FILE_READER_COBOL_READER + "variable_size_occurs"; + public static final String FILE_READER_COBOL_READER_LENGTH_FIELD_NAME = FILE_READER_COBOL_READER + "length_field_name"; + public static final String FILE_READER_COBOL_READER_IS_RECORD_SEQUENCE = FILE_READER_COBOL_READER + "is_record_sequence"; + public static final String FILE_READER_COBOL_READER_IS_RDW_BIG_ENDIAN = FILE_READER_COBOL_READER + "is_rdw_big_endian"; + public static final String FILE_READER_COBOL_READER_IS_RDW_PART_REC_LENGTH = FILE_READER_COBOL_READER + "is_rdw_part_rec_length"; + public static final String FILE_READER_COBOL_READER_RDW_ADJUSTMENT = FILE_READER_COBOL_READER + "rdw_adjustment"; + public static final String FILE_READER_COBOL_READER_IS_INDEX_GENERATION_NEEDED = FILE_READER_COBOL_READER + "is_index_generation_needed"; + public static final String FILE_READER_COBOL_READER_INPUT_SPLIT_RECORDS = FILE_READER_COBOL_READER + "input_split_records"; + public static final String FILE_READER_COBOL_READER_INPUT_SPLIT_SIZE_MB = FILE_READER_COBOL_READER + "input_split_size_mb"; + public static final String FILE_READER_COBOL_READER_HDFS_DEFAULT_BLOCK_SIZE = FILE_READER_COBOL_READER + "hdfs_default_block_size"; + public static final String FILE_READER_COBOL_READER_START_OFFSET = FILE_READER_COBOL_READER + "start_offset"; + public static final String FILE_READER_COBOL_READER_END_OFFSET = FILE_READER_COBOL_READER + "end_offset"; + public static final String FILE_READER_COBOL_READER_FILE_START_OFFSET = FILE_READER_COBOL_READER + "file_start_offset"; + public static final String FILE_READER_COBOL_READER_FILE_END_OFFSET = FILE_READER_COBOL_READER + "file_end_offset"; + public static final String FILE_READER_COBOL_READER_SCHEMA_POLICY = FILE_READER_COBOL_READER + "schema_policy"; + public static final String FILE_READER_COBOL_READER_STRING_TRIMMING_POLICY = FILE_READER_COBOL_READER + "string_trimming_policy"; + public static final String FILE_READER_COBOL_READER_DROP_GROUP_FILLERS = FILE_READER_COBOL_READER + "drop_group_fillers"; + public static final String FILE_READER_COBOL_READER_DROP_VALUE_FILLERS = FILE_READER_COBOL_READER + "drop_value_fillers"; + public static final String FILE_READER_COBOL_READER_NON_TERMINALS = FILE_READER_COBOL_READER + "non_terminals"; + public static final String FILE_READER_COBOL_READER_DEBUG_FIELDS_POLICY = FILE_READER_COBOL_READER + "debug_fields_policy"; + public static final String FILE_READER_COBOL_READER_RECORD_HEADER_PARSER = FILE_READER_COBOL_READER + "record_header_parser"; + public static final String FILE_READER_COBOL_READER_RHP_ADDITIONAL_INFO = FILE_READER_COBOL_READER + "rhp_additional_info"; + public static final String FILE_READER_COBOL_READER_INPUT_FILE_NAME_COLUMN = FILE_READER_COBOL_READER + "input_file_name_column"; - public static final String FILE_READER_COPYBOOK_CONTENT = FILE_READER_COPYBOOK_PREFIX + "content"; - public static final String FILE_READER_COPYBOOK_PATH = FILE_READER_COPYBOOK_PREFIX + "path"; - private final VarLenReader reader; - private final SimpleStream stream; private final Schema schema; + private final VarLenReader reader; + private SimpleStream stream; private String copybook; private Iterator> iterator; private ReaderParameters params; @@ -57,21 +91,29 @@ public class CobolFileReader extends AbstractFileReader config) throws Exception { super(fs, filePath, new CobolToStruct(), config); - this.reader = CobolReader.varLenReader(copybook, params); - this.stream = new FSStream(getFs(), getFilePath()); + this.reader = CobrixReader.varLenReader(copybook, params); this.schema = extractSchema(reader.getCobolSchema()); this.iterator = initIterator(); this.closed = false; } private Iterator> initIterator() throws Exception { - return asJavaIterator(reader.getRecordIterator(stream, 0, 0, 0) - .map(it -> seqAsJavaList(it.seq()))); + if (stream != null) { + stream.close(); + } + stream = new FSStream(getFs(), getFilePath()); + return asJavaIterator(reader.getRecordIterator(stream, 0, 0, 0).map(it -> seqAsJavaList(it.seq()))); } private Schema extractSchema(CobolSchema cobolSchema) { SchemaBuilder builder = SchemaBuilder.struct(); - seqAsJavaList(cobolSchema.getCobolSchema().ast().children()) + Group group; + if (params.schemaPolicy().id() == SchemaRetentionPolicy$.MODULE$.CollapseRoot().id()) { + group = (Group) cobolSchema.getCobolSchema().ast().children().head(); + } else { + group = cobolSchema.getCobolSchema().ast(); + } + seqAsJavaList(group.children()) .forEach(child -> builder.field(child.name(), schemaForField(child))); return builder.build(); @@ -117,19 +159,19 @@ protected void configure(Map config) { } private String copybookContent(Map config) { - String content = Optional.ofNullable(config.get(FILE_READER_COPYBOOK_PATH)) + String content = Optional.ofNullable(config.get(FILE_READER_COBOL_COPYBOOK_PATH)) .map(Path::new) .map(path -> { StringBuilder sb = new StringBuilder(); try (InputStream is = getFs().open(path); BufferedReader br = new BufferedReader(new InputStreamReader(is))) { - br.lines().forEach(sb::append); + br.lines().forEach(line -> sb.append(line).append("\n")); } catch (IOException ioe) { throw new ConnectException("Cannot read Copybook file: " + path, ioe); } return sb.toString(); }) - .orElseGet(() -> config.get(FILE_READER_COPYBOOK_CONTENT)); + .orElseGet(() -> config.get(FILE_READER_COBOL_COPYBOOK_CONTENT)); if (content == null || content.trim().isEmpty()) { throw new ConnectException("Copybook is not specified."); @@ -139,39 +181,39 @@ private String copybookContent(Map config) { private ReaderParameters getReaderParameters(Map config) { return new ReaderParameters( - true, - "common", - scala.Option.apply(null), - "", - true, - FloatingPointFormat$.MODULE$.IBM(), - false, // variableSizeOccurs - scala.Option.apply(null), - true, // isRecordSequence - false, - false, - 0, - false, - scala.Option.apply(null), - scala.Option.apply(null), - scala.Option.apply(null), - 0, - 0, - 0, - 0, - false, - SchemaRetentionPolicy$.MODULE$.KeepOriginal(), - StringTrimmingPolicy$.MODULE$.TrimBoth(), - scala.Option.apply(null), - new CommentPolicy(true, 6, 72), - false, - true, - scala.collection.immutable.Nil$.empty(), - null, - DebugFieldsPolicy$.MODULE$.NoDebug(), - scala.Option.apply(null), - scala.Option.apply(null), - "" + Boolean.parseBoolean(config.getOrDefault(FILE_READER_COBOL_READER_IS_EBCDIC, "true")), // isEbcdic + config.getOrDefault(FILE_READER_COBOL_READER_EBCDIC_CODE_PAGE, "common"), // ebcdicCodePage + scala.Option.apply(config.get(FILE_READER_COBOL_READER_EBCDIC_CODE_PAGE_CLASS)), // ebcdicCodePageClass + config.getOrDefault(FILE_READER_COBOL_READER_ASCII_CHARSET, ""), // asciiCharset + Boolean.parseBoolean(config.getOrDefault(FILE_READER_COBOL_READER_IS_UFT16_BIG_ENDIAN, "true")), // isUtf16BigEndian + FloatingPointFormat$.MODULE$.withNameOpt(config.getOrDefault(FILE_READER_COBOL_READER_FLOATING_POINT_FORMAT, "ibm")).get(), // floatingPointFormat + Boolean.parseBoolean(config.getOrDefault(FILE_READER_COBOL_READER_VARIABLE_SIZE_OCCURS, "false")), // variableSizeOccurs + scala.Option.apply(config.get(FILE_READER_COBOL_READER_LENGTH_FIELD_NAME)), // lengthFieldName + Boolean.parseBoolean(config.getOrDefault(FILE_READER_COBOL_READER_IS_RECORD_SEQUENCE, "false")), // isRecordSequence + Boolean.parseBoolean(config.getOrDefault(FILE_READER_COBOL_READER_IS_RDW_BIG_ENDIAN, "false")), // isRdwBigEndian + Boolean.parseBoolean(config.getOrDefault(FILE_READER_COBOL_READER_IS_RDW_PART_REC_LENGTH, "false")), // isRdwPartRecLength + Integer.parseInt(config.getOrDefault(FILE_READER_COBOL_READER_RDW_ADJUSTMENT, "0")), // rdwAdjustment + Boolean.parseBoolean(config.getOrDefault(FILE_READER_COBOL_READER_IS_INDEX_GENERATION_NEEDED, "false")), // isIndexGenerationNeeded + scala.Option.apply(config.get(FILE_READER_COBOL_READER_INPUT_SPLIT_RECORDS)), // inputSplitRecords + scala.Option.apply(config.get(FILE_READER_COBOL_READER_INPUT_SPLIT_SIZE_MB)), // inputSplitSizeMB + scala.Option.apply(config.get(FILE_READER_COBOL_READER_HDFS_DEFAULT_BLOCK_SIZE)), // hdfsDefaultBlockSize + Integer.parseInt(config.getOrDefault(FILE_READER_COBOL_READER_START_OFFSET, "0")), // startOffset + Integer.parseInt(config.getOrDefault(FILE_READER_COBOL_READER_END_OFFSET, "0")), // endOffset + Integer.parseInt(config.getOrDefault(FILE_READER_COBOL_READER_FILE_START_OFFSET, "0")), // fileStartOffset + Integer.parseInt(config.getOrDefault(FILE_READER_COBOL_READER_FILE_END_OFFSET, "0")), // fileEndOffset + true, // generateRecordId + SchemaRetentionPolicy$.MODULE$.withNameOpt(config.getOrDefault(FILE_READER_COBOL_READER_SCHEMA_POLICY, "keep_original")).get(), // schemaPolicy + StringTrimmingPolicy$.MODULE$.withNameOpt(config.getOrDefault(FILE_READER_COBOL_READER_STRING_TRIMMING_POLICY, "both")).get(), // stringTrimmingPolicy + scala.Option.apply(null), // multisegment + new CommentPolicy(true, 6, 72), // commentPolicy + Boolean.parseBoolean(config.getOrDefault(FILE_READER_COBOL_READER_DROP_GROUP_FILLERS, "false")), // dropGroupFillers + Boolean.parseBoolean(config.getOrDefault(FILE_READER_COBOL_READER_DROP_VALUE_FILLERS, "true")), // dropValueFillers + asScalaBuffer(Arrays.asList(config.getOrDefault(FILE_READER_COBOL_READER_NON_TERMINALS, "").split(","))), // nonTerminals + scala.collection.immutable.Map$.MODULE$.empty(), // occursMappings + DebugFieldsPolicy$.MODULE$.withNameOpt(config.getOrDefault(FILE_READER_COBOL_READER_DEBUG_FIELDS_POLICY, "none")).get(), // debugFieldsPolicy + scala.Option.apply(config.get(FILE_READER_COBOL_READER_RECORD_HEADER_PARSER)), // recordHeaderParser + scala.Option.apply(config.get(FILE_READER_COBOL_READER_RHP_ADDITIONAL_INFO)), // rhpAdditionalInfo + config.getOrDefault(FILE_READER_COBOL_READER_INPUT_FILE_NAME_COLUMN, "") // inputFileNameColumn ); } @@ -222,29 +264,25 @@ private static class FSStream implements SimpleStream { private final FileSystem fs; private final Path file; private final FSDataInputStream stream; + private final long size; + private long offset; FSStream(FileSystem fs, Path file) throws IOException { this.fs = fs; this.file = file; this.stream = this.fs.open(file); + this.size = fs.getContentSummary(file).getLength(); + this.offset = stream.getPos(); } @Override public long size() { - try { - return fs.getContentSummary(file).getLength(); - } catch (IOException ioe) { - throw new ConnectException("Cannot retrieve length for file: " + file, ioe); - } + return size; } @Override public long offset() { - try { - return stream.getPos(); - } catch (IOException ioe) { - throw new ConnectException("Cannot get current position for file: " + file, ioe); - } + return offset; } @Override @@ -254,8 +292,10 @@ public String inputFileName() { @Override public byte[] next(int numberOfBytes) throws IOException { - byte[] bytes = new byte[(int) Math.min(numberOfBytes, size() - offset())]; + int bytesToRead = (int) Math.min(numberOfBytes, size() - offset()); + byte[] bytes = new byte[bytesToRead]; stream.readFully(bytes); + offset += bytesToRead; return bytes; } @@ -269,17 +309,17 @@ static class CobolToStruct implements ReaderAdapter { public Struct apply(CobolRecord record) { Struct struct = new Struct(record.schema); - record.row.forEach(col -> { - Map column = (Map) col; - column.forEach((k, v) -> struct.put(k, mapValue(record.schema.field(k).schema(), v))); - }); + record.row.stream() + .filter(col -> col instanceof Map) + .forEach(col -> { + Map column = (Map) col; + column.forEach((k, v) -> struct.put(k, mapValue(record.schema.field(k).schema(), v))); + }); return struct; } private Object mapValue(Schema schema, Object value) { - if (value == null) return null; - - if (value instanceof String || value instanceof Integer || value instanceof Double) { + if (schema.type() != Schema.Type.STRUCT || value == null) { return value; } Struct struct = new Struct(schema); @@ -322,7 +362,15 @@ private Map mapValues(Group group, Object[] values) { } private Map.Entry transform(Statement child, Object value) { - Object childValue = child instanceof Group ? ((Map) value).get(child.name()) : value; + Object childValue; + if (child instanceof Group) { + childValue = ((Map) value).get(child.name()); + } else if (value instanceof ScalaNumber) { + childValue = value instanceof scala.math.BigDecimal ? + ((BigDecimal) value).doubleValue() : ((ScalaNumber) value).longValue(); + } else { + childValue = value; + } return new AbstractMap.SimpleEntry<>(child.name(), childValue); } diff --git a/src/main/scala/com/github/mmolimar/kafka/connect/fs/file/reader/CobolReader.scala b/src/main/scala/com/github/mmolimar/kafka/connect/fs/file/reader/CobrixReader.scala similarity index 94% rename from src/main/scala/com/github/mmolimar/kafka/connect/fs/file/reader/CobolReader.scala rename to src/main/scala/com/github/mmolimar/kafka/connect/fs/file/reader/CobrixReader.scala index e01f1d9..abfb9bc 100644 --- a/src/main/scala/com/github/mmolimar/kafka/connect/fs/file/reader/CobolReader.scala +++ b/src/main/scala/com/github/mmolimar/kafka/connect/fs/file/reader/CobrixReader.scala @@ -6,7 +6,7 @@ import za.co.absa.cobrix.cobol.reader.{VarLenNestedReader, VarLenReader} import scala.collection.Seq -protected object CobolReader { +protected object CobrixReader { def varLenReader(copybookContent: String, params: ReaderParameters): VarLenReader = { new VarLenNestedReader[java.util.Map[String, AnyRef]](Seq(copybookContent), params, new StructHandler()) diff --git a/src/test/java/com/github/mmolimar/kafka/connect/fs/file/reader/CobolFileReaderTest.java b/src/test/java/com/github/mmolimar/kafka/connect/fs/file/reader/CobolFileReaderTest.java index 4e79874..cb93495 100644 --- a/src/test/java/com/github/mmolimar/kafka/connect/fs/file/reader/CobolFileReaderTest.java +++ b/src/test/java/com/github/mmolimar/kafka/connect/fs/file/reader/CobolFileReaderTest.java @@ -1,15 +1,20 @@ package com.github.mmolimar.kafka.connect.fs.file.reader; import org.apache.hadoop.fs.Path; +import org.apache.kafka.connect.data.Schema; import org.apache.kafka.connect.data.Struct; +import org.apache.kafka.connect.errors.ConnectException; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.MethodSource; import java.io.*; +import java.net.URL; import java.nio.file.Files; +import java.nio.file.Paths; import java.nio.file.StandardCopyOption; import java.util.HashMap; import java.util.Map; +import java.util.UUID; import java.util.stream.IntStream; import static org.junit.jupiter.api.Assertions.*; @@ -17,6 +22,9 @@ public class CobolFileReaderTest extends FileReaderTestBase { private static final String FILE_EXTENSION = "dt"; + private static final String DATA_FILENAME_1 = "companies"; + private static final String DATA_FILENAME_2 = "type-variety"; + private static final String DATA_FILENAME_3 = "code-pages"; @Override protected Class getReaderClass() { @@ -25,8 +33,9 @@ protected Class getReaderClass() { @Override protected Path createDataFile(ReaderFsTestConfig fsConfig, Object... args) throws IOException { + String filename = args.length < 1 ? DATA_FILENAME_1 : args[0].toString(); File cobolFile = File.createTempFile("test-", "." + getFileExtension()); - try (InputStream is = CobolFileReaderTest.class.getResourceAsStream("/file/reader/data/cobol/companies.dat")) { + try (InputStream is = CobolFileReaderTest.class.getResourceAsStream("/file/reader/data/cobol/" + filename + "." + getFileExtension())) { Files.copy(is, cobolFile.toPath(), StandardCopyOption.REPLACE_EXISTING); } IntStream.range(0, NUM_RECORDS).forEach(index -> fsConfig.offsetsByIndex().put(index, (long) index)); @@ -57,25 +66,169 @@ public void emptyFile(ReaderFsTestConfig fsConfig) throws IOException { getReader(fsConfig.getFs(), path, getReaderConfig()); } + @ParameterizedTest + @MethodSource("fileSystemConfigProvider") + public void emptyCopybook(ReaderFsTestConfig fsConfig) throws IOException { + Path file = createDataFile(fsConfig); + Map readerConfig = getReaderConfig(); + readerConfig.put(CobolFileReader.FILE_READER_COBOL_COPYBOOK_CONTENT, ""); + assertThrows(ConnectException.class, () -> getReader(fsConfig.getFs(), file, readerConfig)); + } + + @ParameterizedTest + @MethodSource("fileSystemConfigProvider") + public void nonExistentCopybook(ReaderFsTestConfig fsConfig) throws IOException { + Path file = createDataFile(fsConfig); + Map readerConfig = getReaderConfig(); + Path copybook = new Path(fsConfig.getFs().getWorkingDirectory(), UUID.randomUUID().toString()); + readerConfig.put(CobolFileReader.FILE_READER_COBOL_COPYBOOK_PATH, copybook.toString()); + assertThrows(ConnectException.class, () -> getReader(fsConfig.getFs(), file, readerConfig)); + } + + @ParameterizedTest + @MethodSource("fileSystemConfigProvider") + public void readAllDataWithCopybookInPath(ReaderFsTestConfig fsConfig) throws IOException { + String dataFilename = DATA_FILENAME_1; + Path file = createDataFile(fsConfig, dataFilename); + Map readerConfig = getReaderConfig(); + readerConfig.put(CobolFileReader.FILE_READER_COBOL_COPYBOOK_PATH, ""); + assertThrows(ConnectException.class, () -> getReader(fsConfig.getFs(), file, readerConfig)); + + File cobolFile = File.createTempFile("copybook-", "." + getFileExtension()); + try (InputStream is = CobolFileReaderTest.class.getResourceAsStream("/file/reader/data/cobol/" + dataFilename + ".cpy")) { + Files.copy(is, cobolFile.toPath(), StandardCopyOption.REPLACE_EXISTING); + } + Path path = new Path(new Path(fsConfig.getFsUri()), cobolFile.getName()); + fsConfig.getFs().copyFromLocalFile(new Path(cobolFile.getAbsolutePath()), path); + readerConfig.put(CobolFileReader.FILE_READER_COBOL_COPYBOOK_PATH, path.toString()); + FileReader reader = getReader(fsConfig.getFs(), file, readerConfig); + + assertTrue(reader.hasNext()); + + int recordCount = 0; + while (reader.hasNext()) { + Struct record = reader.next(); + checkData(record, recordCount); + recordCount++; + } + assertEquals(NUM_RECORDS, recordCount, "The number of records in the file does not match"); + } + + @ParameterizedTest + @MethodSource("fileSystemConfigProvider") + public void readAllDataWithMultipleDataTypes(ReaderFsTestConfig fsConfig) throws IOException { + String dataFilename = DATA_FILENAME_2; + Path file = createDataFile(fsConfig, dataFilename); + Map readerConfig = getReaderConfig(); + readerConfig.put(CobolFileReader.FILE_READER_COBOL_COPYBOOK_CONTENT, copybookContent(dataFilename)); + readerConfig.put(CobolFileReader.FILE_READER_COBOL_READER_SCHEMA_POLICY, "collapse_root"); + readerConfig.put(CobolFileReader.FILE_READER_COBOL_READER_FLOATING_POINT_FORMAT, "ieee754"); + readerConfig.put(CobolFileReader.FILE_READER_COBOL_READER_IS_RECORD_SEQUENCE, "false"); + + FileReader reader = getReader(fsConfig.getFs(), file, readerConfig); + + assertTrue(reader.hasNext()); + + int recordCount = 0; + while (reader.hasNext()) { + Struct record = reader.next(); + recordCount++; + assertEquals(recordCount, record.get("ID")); + assertEquals("Sample", record.get("STRING_VAL")); + } + assertEquals(NUM_RECORDS, recordCount, "The number of records in the file does not match"); + } + + @ParameterizedTest + @MethodSource("fileSystemConfigProvider") + public void readAllDataWithBinaryData(ReaderFsTestConfig fsConfig) throws IOException { + String dataFilename = DATA_FILENAME_3; + Path file = createDataFile(fsConfig, dataFilename); + Map readerConfig = getReaderConfig(); + readerConfig.put(CobolFileReader.FILE_READER_COBOL_COPYBOOK_CONTENT, copybookContent(dataFilename)); + readerConfig.put(CobolFileReader.FILE_READER_COBOL_READER_SCHEMA_POLICY, "collapse_root"); + readerConfig.put(CobolFileReader.FILE_READER_COBOL_READER_IS_RECORD_SEQUENCE, "false"); + FileReader reader = getReader(fsConfig.getFs(), file, readerConfig); + + assertTrue(reader.hasNext()); + + int recordCount = 0; + while (reader.hasNext()) { + Struct record = reader.next(); + assertEquals(Schema.Type.STRING, record.schema().field("CURRENCY").schema().type()); + assertEquals(Schema.Type.STRING, record.schema().field("SIGNATURE").schema().type()); + assertEquals(Schema.Type.STRING, record.schema().field("COMPANY_NAME_NP").schema().type()); + assertEquals(Schema.Type.STRING, record.schema().field("COMPANY_ID").schema().type()); + assertEquals(Schema.Type.INT32, record.schema().field("WEALTH_QFY").schema().type()); + assertEquals(Schema.Type.FLOAT64, record.schema().field("AMOUNT").schema().type()); + assertNotNull(record.get("CURRENCY")); + assertNotNull(record.get("SIGNATURE")); + assertNotNull(record.get("COMPANY_NAME_NP")); + assertNotNull(record.get("COMPANY_ID")); + assertNotNull(record.get("WEALTH_QFY")); + assertNotNull(record.get("AMOUNT")); + assertEquals(6, record.schema().fields().size()); + recordCount++; + } + assertEquals(NUM_RECORDS, recordCount, "The number of records in the file does not match"); + } + + @ParameterizedTest + @MethodSource("fileSystemConfigProvider") + public void readAllDataWithBinaryRawData(ReaderFsTestConfig fsConfig) throws IOException { + String dataFilename = DATA_FILENAME_3; + Path file = createDataFile(fsConfig, dataFilename); + Map readerConfig = getReaderConfig(); + readerConfig.put(CobolFileReader.FILE_READER_COBOL_COPYBOOK_CONTENT, copybookContent(dataFilename)); + readerConfig.put(CobolFileReader.FILE_READER_COBOL_READER_SCHEMA_POLICY, "collapse_root"); + readerConfig.put(CobolFileReader.FILE_READER_COBOL_READER_DEBUG_FIELDS_POLICY, "raw"); + readerConfig.put(CobolFileReader.FILE_READER_COBOL_READER_IS_RECORD_SEQUENCE, "false"); + FileReader reader = getReader(fsConfig.getFs(), file, readerConfig); + + assertTrue(reader.hasNext()); + + int recordCount = 0; + while (reader.hasNext()) { + Struct record = reader.next(); + assertEquals(Schema.Type.STRING, record.schema().field("CURRENCY").schema().type()); + assertEquals(Schema.Type.STRING, record.schema().field("SIGNATURE").schema().type()); + assertEquals(Schema.Type.STRING, record.schema().field("COMPANY_NAME_NP").schema().type()); + assertEquals(Schema.Type.STRING, record.schema().field("COMPANY_ID").schema().type()); + assertEquals(Schema.Type.INT32, record.schema().field("WEALTH_QFY").schema().type()); + assertEquals(Schema.Type.FLOAT64, record.schema().field("AMOUNT").schema().type()); + assertNotNull(record.get("CURRENCY")); + assertNotNull(record.get("CURRENCY_debug")); + assertNotNull(record.get("SIGNATURE")); + assertNotNull(record.get("SIGNATURE_debug")); + assertNotNull(record.get("COMPANY_NAME_NP")); + assertNotNull(record.get("COMPANY_NAME_NP_debug")); + assertNotNull(record.get("COMPANY_ID")); + assertNotNull(record.get("COMPANY_ID_debug")); + assertNotNull(record.get("WEALTH_QFY")); + assertNotNull(record.get("WEALTH_QFY_debug")); + assertNotNull(record.get("AMOUNT")); + assertNotNull(record.get("AMOUNT_debug")); + assertEquals(12, record.schema().fields().size()); + recordCount++; + } + assertEquals(NUM_RECORDS, recordCount, "The number of records in the file does not match"); + } + @Override protected Map getReaderConfig() { return new HashMap() {{ - put(CobolFileReader.FILE_READER_COPYBOOK_CONTENT, copybookContent()); + put(CobolFileReader.FILE_READER_COBOL_COPYBOOK_CONTENT, copybookContent(DATA_FILENAME_1)); + put(CobolFileReader.FILE_READER_COBOL_READER_IS_RECORD_SEQUENCE, "true"); }}; } - private String copybookContent() { - return " 01 COMPANY-DETAILS.\n" + - " 05 SEGMENT-ID PIC X(5).\n" + - " 05 COMPANY-ID PIC X(10).\n" + - " 05 STATIC-DETAILS.\n" + - " 10 COMPANY-NAME PIC X(15).\n" + - " 10 ADDRESS PIC X(25).\n" + - " 10 TAXPAYER.\n" + - " 15 TAXPAYER-TYPE PIC X(1).\n" + - " 15 TAXPAYER-STR PIC X(8).\n" + - " 15 TAXPAYER-NUM REDEFINES TAXPAYER-STR\n" + - " PIC 9(8) COMP."; + private String copybookContent(String filename) { + URL cpy = CobolFileReaderTest.class.getResource("/file/reader/data/cobol/" + filename + ".cpy"); + try { + return String.join("\n", Files.readAllLines(Paths.get(cpy.toURI()))); + } catch (Exception e) { + throw new RuntimeException(e); + } } @Override diff --git a/src/test/resources/file/reader/data/cobol/code-pages.cpy b/src/test/resources/file/reader/data/cobol/code-pages.cpy new file mode 100644 index 0000000..dbea4d5 --- /dev/null +++ b/src/test/resources/file/reader/data/cobol/code-pages.cpy @@ -0,0 +1,7 @@ + 01 TRANSDATA. + 05 CURRENCY PIC X(3). + 05 SIGNATURE PIC X(8). + 05 COMPANY-NAME-NP PIC X(15). + 05 COMPANY-ID PIC X(10). + 05 WEALTH-QFY PIC 9(1). + 05 AMOUNT PIC S9(09)V99 BINARY. diff --git a/src/test/resources/file/reader/data/cobol/code-pages.dt b/src/test/resources/file/reader/data/cobol/code-pages.dt new file mode 100644 index 0000000000000000000000000000000000000000..3e8ca49585c85f9ac7d6f2ae24d1836210854772 GIT binary patch literal 4500 zcmZwKc{r4N8wc>`VJ4loj3p=0q9m0iYaAk5mP!isMktXb`=sJ9kuuZR289TdBve98 zlqFdP>7|--UI`_YvLs8k4rku$I-bsZUw-EA&-K0T-+lj{`+26ZwykG+baG;RWaNH_ zc%|uBpW5*c+aGvLVZ+1^!$z>be-zKUzL!7oti}1O(m#p}3~SQ$Y$E|4p2L!kAA7hz zag|u;&d^%jaGel$uctc@j4M{uJ!$xf``lE5hi@7R>eK5I-)f-vfGWUKDJ}y$zKY63 z7jU+PTL4VdEONfDru}E~?Nvqvh~ph^ZQHo&t5q_H0UJ^Fwu>N(vAP`vrK7R?Tcrur46Cc?VhtJ4)6zM+>y3T zx0z7a^@;biqih3s?tl(uhM><|584bS`luCEc9$;F_>5reWg{pf)mcGnk8w!cP#W6jq>{Ey6L08I2b!vb_e_N=C|+a8OZQEo;4jV1sS-EdwC zE|Tp@?exL}vh*E3VFsRY029gCcR?93ZM7cW`W8uT{Qm5)G=PcZmtG)QyEV|=kE@+K zW;Xt%Vl#p_a)UwgXL?{ji&LMmiD^dO#$E(tr41nY7slhE$^Two-(L3KUXlUuT+Mjo z7)TDO5UNgn`n#x%T^U!HJRfUF)cgjL*&bQGTx+L>*Ed7`UC#qNcfiG$7@*J3ek>1j z5v7oY0)EMk+KFIXSO}EyVUoR`c{jzdP<&Sh>k7a`pUaPci*&PO#hSLxO9Jd3J150a z6wAS(m__mEDuZ7>JtDoAFWU|Rxy3gp*lAt*pW?Y@2%umMfph%EhnEs!HvNJAQN^r>9Y@w)wZ3alZ`+#^j~}#c>STAb z`c9H*Aiu}*UkJu7w1MO0UNGA9CvddaStuq(DBgy#b zm*9atWICv~GvR4mc~q_ra~#FC-XPgdhuZQsNPKn4Dw0Jix)`x*o&aa7J=f^H%w8F- zaxCt(`26HGv3K(U7N#W0?lm6n>ag2Y^SKCM;(;YagUheioMwLg5bkhrZ1b~;EdUdH zqo5Zg^Bb8gnJr(FOh79%ANV9gD7qE<`+*Ott%4_71+jFU|P)_ELkaDQO1BjeY`alyv`&mAzR z$`ZVeliS3bUNBbDO?Y>o7mEQ*v`$Y6jC=97vE2`QqZ5x#^c$F+M6pplC?oA9Q;5l_ zXRRrHv?SFXV4}~3wV;fkYe}ZLd9QPvm-=H?oe0LabHK_KU^BuWozZ+Ol426>RyZF^ zst=Js%|2qxI}@tnAC3zRTM1pBk1^~N4ZH{K15$EFGpDOB;pDv^;0F~klp;ZJz*8Gr z+r^}sW*Ax9 z7pQp^U2c8;V1J?}&L+MYpIZiiGc^}hBepj-<(`~&3B&RBKxQ*KF z*EVd@E9zE2lQCU@n^*gH3FnwQE@@=oO zd@VsY{JP8YXPGV=SW5A%+?x3q!?OFp0p%yv67~(AEDlaGRy$jVVEk+yn4o&UYI1X6 z$*sE~CB`pu0VZ1Ky<_c|F#nUPU=J`c6oqb(oVogdyWxYH0Q(X%`80HI z;3^Ezjn)mYDmURcRSo~O5#89C25=sYR@BVxtWvc)EaFiG fl8KsQB>;AP%a0T$iwZ#P$Ei)~1Nv*dpF1+uYnLBfNy!RmEtM0vX<~Qe@`0WA@$nt zudaJz@MJ}|=2wQS&U)`J<;3rL8`n-t|NO0U##{c!-K}@6hwp!RV}R8fkZ^D1vzfmP zO!>rWwbs75EYJ;7ZFW$|AZw4y#i`HVo@%;&{EbkLAx7&SzktlS>0g!HJF98`<-)SG zmUm10eD9IF?TukicJ=VR+QaOWr!#&n&!&HO{p|Ank6aAw@05-+8n?T>(Wh;;-tQZz2YWog;CZbdFjw=K#X*KO4b$ZuOTG6==sSaOXVK8Aw6lQ&PyVNe%uqKH_Q z1iDUL-nsJvkHsB}GF;Y;>+Z6&bX>P}sF;CvkO$v^#6=8qL_zeE&7dw`l5Yv*zSlKK zx^cQwr^m;qG)NA7G(rn~BxUiAMkL!@)){ehAjvt~T-Bs6kCYoe|1o0o+1K#-#Ag6g zJRu52VkDqN))7&RR+5O5Xa+}$Sx7)@Ejg&L4ylmw>Kyb07kPp_0eb>Hc`qQnf!Bet z&-y0_RaTh|SGSs7=s6nOcTSws*W{Q(Z*0g$Z2?0@WjyO zK*>-pcnBGaot@VUvg|!Bn-)g(`+jFEK7QS6H!Os-^>>;tcFPjC_UxFjq0ICCwVUzf zUq%XrH@$njygs3mi|?abull~fKg;c*JC8J5bMXEJudx@#_&>4azwzpfr&E3%V!wRm zWUHCU+W&11Zjy1CN9@KNkI0~n6EZy_TW=`aIAMG@B5ej}e76xrQsC_u^BPN6DrICngY=y^%tYXf{{prB4!~0Et>l>x>P&E$@g#!DV4)dEH3TJsB-H^mYPDK#04HEev_!yO zMPsyAGCIc08nez~v}iRttyvBK3DF+Z>Z-u{2zo#P??p0r&vNd!f=gNn)KW%u%0!BG zDq789$pNc}(Gh0U)}we74AGX@Gbb#q6AK9v{o4U`lsN@^B?ug2@Su^QUjKDC*p0Eq zD%uz--Wo8xi_L+Orb3VlZH3z&PY6Q#P1!T%OcEipLbI#E4^~||3_Y( z@pQ(|ZO^-Re{8is=d*4bUvpZ?e_|XVkL^?-?^o1m5nNY|hjjk7Ae0ItP;41Qt}TQi z5d;htlp{G5oWrp;IZQ8jYM5WtjNo3+`;~>uqdVov`>pM?2(GKfLppz305J(C+5yZa zocrNW@GOqCnZ@)Z-z5!_Zk+DK!ev{^8YBlk8li;xO2(@ycxL6oZTJ^@ z0zG*zAiaUtfw52iCkRzmS%A49D5XK&WHOixc&PEzkc(aeC}c2|1WAH1m6e)xCcRp( zvjZF;7JDU&))59XE*8C6uh(f|>IEifOr}bc0S?0f)GilIs+cJkOp;_e!b(jhO7UQQ zKxdEWj@l{dEENtgZGDP(XVfFQ+<-@k3%p2VMvNOXeBfY47KM zy!_B}<0s#J)a)ZnugcdPKl97X0t4|Hi8` zo=*AsO-cLo+BdBR4_F?T*CgWtx6JvOZlk^Drv|x=mc@SyZ{VO<2K6rESX2ZDH*D}2Hn>;EmE#KX+|uW#yN&jopBjLQd8m>{Oo+H}EwhQ{ zexe!FyM$v=;T+_?^BN@GINfd?H!V+TkR14EgckZpOy*sNNVd7;DzWpCq}(<)bmM|g z;_h|K8XY*L7N1XSMojU93I~amhL#B+q8Ke2x@Geqs@O1M78201Igkn&ugc+>b(8Dl zI_wGLlzIW_4N@IMy2<|pp;c9BOgN~!NQ1f_#$@=htyUZHG?0tdpkb+n3tCB{s-R`j z;7s_IdmT*jYt>r)3B4K)^;y(DBbuCK zrstVpImIO{QLXFP4lvfy7Hgs9K6%Boni|O|Q?sXtzEF?oG6R-6j$so6$Aa|0k)f6N z*WqBTgP~H<#!x|=Y2;MUlC(KcvYZl>LYS=B`)xreDU?qd_j}vRi>|@rC;au}K6gkw z(tIBqBfK;=Wb&2hMJ`3t#;X5J-z)Un;gK_=>njF%xbo>I+pq1rU7>hYGiRIIzE{8V zxuBS)-)p^RJy$zh{iM}8so;LeJAv0btn!DsbgNI#1T!SY_ik10vd-E0%ggN#uD&yS z+fMJ+ZzNlfOfeR3IX9u%{p!1~y>%t`*tC21GMm3N&76?g=dVupwlzz3%F`J?Pi==! z|M)m@&Z;LOTRWxWjE2tZ@)PY6KYnS^r$=w5D7^=ID}|hau9-?W*plb#y2BJzS zG3idUp@90%=6+}a`VQn+a6Z$Me0OhlgR)IG3CRb@tHyRB_10Fg&;fFC?OllKP zB|(y?Dr&*d0)JD5l$mkB0pbACI>I#Y5BxmW>GcMaMx!?yjAx7nI1JUOeNHr~VWwx9 zV3Z^yQ6&`-8PzEhIol(39k7ZSPjUBB-L&;7B0JP0y4-*Vg$ulh`{G6XhW3cPm60=e z;KGC{7m{?Nv*EoK~o z$4|EM&EVaTddJU5JuY0YX`io}rS$Nd*7AzB|D(c_de^Tt3woYXtXn;3!n38?NabIT z9scn-kGU=_w%>=twKZg)`+G|?4Cb|qpSm~_QuY2VdwN>i zhFw|u$;NxeYuj$O_o_N&eXE&eQ1dlEK6-EN!mnRhKhf*hU7yORo-2&&?)*Kq%^M@T zEpy7#8Nb%uqdRnYb#uV(s1BbxrQ?jopQjF9KA-u_?;{RhR^_;6PRVo~9Xn-Ik?ZLG zvkRt-TIH65(t;cmr$PZ-dyHO1!AUOT!~%e(O4!s0rZ;t>kI$;y!DD@fWoEl(Ov!K^ z9W!OrI`o*2D%r$$HqnLx>YIY-n--vN37guT=}Epb8YJB~-C!TTg|Q8i10NCvhqTZ~ zOcw7SM6%6gou&PWq9<%~RnN;KO8F+x(2p8S= z2*Q}3Wuu1Q_8R#6FnIig<*P59fyAZzxFh3)WNU;;@y-ZEYtyf9cV5$7=8Vu%tT0=}Dtk!o!Yv2FOSNmeU zh>!?*NNh-_OnFG(iDe<3#&pX;X-g)GQ*y|4!s_|41RUeHY82=~0YJSatak(ld&zo- zb;}yOeN$|AdH0a+@{pL2P8n!iiYnQ}O$O1MurXofwRKnu=LSKr5XbRLH=al0gZ3f?T)=|3XioC+`KMH}E=8o?1)RKS8K6 z{Gl`pdXu{}sO$Acy-^E?0Z%iz=rn4Uk`gljS{;XUbw!=opcie4mW~3F106#gL8dxR zMMGF6qtTl5;1&K{tA`9MZak}p$r7XX3~HYfO-?XVEHMy@)m+lm5$XMjEv<-1h&#}7 ziVh%kjpUTq#Ev3zKs}<%3|LA^>|)r&z_B1baAcYo4pw!nv5Gc^3hG89r-DZS3k+tI zwAJuGosY_XZ0REipX`;FpIg~}!~DnL@ss&}u_*`AhbsGiKVEoq?1!OG-#FJSnqeNPwh9sZymQ5Qwyzs;?XRY7<@NLhjw|1r9 z|DVUgzg!-FXZ^huez#l)yS`ApPIGO>&iJ34@^r?pxpvB_fqRR(mQP81)F~ZjG&Ej2 zx__HA``qI@K0cMA%t?q)PRvXQEmBS#*V8*8bXg0?kLxLqMX?jT_FfU6Nx4s74)N{9 zF(KdHclQQKH%@nP%Xw)L4U%h|7W(L(#XB02Y;#$s>F7YwleW34K6_KjH3QZp7@jG> z=M$d+O!0&Y2Z@zVf#}tZ7R~ftkwnYIEF^HOB?lGO;Z4bSb&gEG{3!bodjdUqFCe{v z*MaiXsA2kr5?8a-0YZyZpbfprLmJd|IyL+Y2TTaS>;btL^d^?-xxkC`2BK~NEfC>P ztFTg~jur*#YKw>?NMmmZ^Z8~C{6JT0jT)`NV$zrmx&u1(LC`#m+LfY76*E1=1U)MP zUZ@WLb0SeBwsisSNUDXQjsvk`219v$PvF*$A{wF|(PaiKbsWPc295>kfg?jJ_OHXi zs$K!LQ_&czAgr`CR1k3594P541%=Q-7SZg2AS@`A9UnWhrT;U%;PDeasSU+->TW)_ zm+Tj2UQRMRHDmXHbWxRj*vU_OA1mH}cE_j@ z-+uCL`G4cpDNkqoJdRgAwWUd3kn>?@wSVC__$|+e16mQ1W`Q5>3eTvLC){#*>Xs>3mqlikVM|7D1OY4M-L(IP& z@J|EBg7mjHzhEWTF%7^I%1h8Kni2UZOF!|z&&+loGKYb~Auf~rsR75QH-4wkFk*GJwM_EnP7 z_Z`EK*1w*+`RAnU0ZX5*`ft2C<>`!{&(cN0#;-MrAFt3IZ<2AQOG;XhOI%T!UztnX znz<2aep^Ezzh-VQKwCrng2>f((u4>C7Dh(4Okq%GViJtl@K_FRTs64oh#Aj!@rzA& zNlXiHiCdfISAxdjr~-NL4G{PaU^e00cOrv26BFt?nuFYTbc3WDr`vztpw>YRk^>)& z&_W*p$-EO0$u^f33ZR*S1&nn{B?Ov`9- z3LR(BEijXf7qXg=rW}CP;$jhu{B=Y#yd-_K0Y5keM<-XGR^u-fXFqM0&IG`hezu9ipzXhfH;xO0?@! zBua2|V$Yng)Nu@(7&yk@fg{tza6sFU*1-U)Xk)10fuXITf+vSI2TJCnWKR7LcfBBt zFP3Fo7~r*Q#~670gw?y;ras?k7I6B+s1cEgQOd$yLtKK zF_{gL10RjhLLceLyrU7xHkVvM#{`NVvdtxY-0^bU<<%fkU8}~gg>w+(3AH9(i?ak7<*W!E)Hfab%4+!m0pK~ zx~DX#!?H;D7Y-esT5{3A@@dRKs^@}6lInnZtqvzk>XcCcol*xdCr(sg4YxgH?@0}- z9AWvT1{O2w)bQ6fvl;#)a=@Uj1nYz7p+a<1#oQfY0{oiicaT8w{zs)Ukq|hoPm%l7 zORViYSsYMA@92)WWPVv%r;;3E{_TK&8aNiD2aXJ_{l5+eYaMJ46>SU^JSa4BDtKaO zyin4Y!|#(`GK=qiL8zRpP zM`GU=5;~Rz{rzFyGWq_5ezECSehE?h5cz)m_0ev-JuZfH{rOLi|Hi9Rp3eBS7r*aJx5y ziVHaA#`TKlxsCQ;(PQGa)n(}}o~h|BF43u@!cj2~?GlMe0CADX9F=fCo(w9^JAlWtA5Al z6PpoJJRu6uA`t%g{?H8JL=>X{(IhwxVZ4}y1hi}pyeS#4D)5)O%p!wH$I8j{M`NA~Vz@y3By3j$_!wz%d3792s1(s&-KMe;5u{bqtG& zHiilw7}^>zlWwNXfs*DBymB#Iej#9oAnV|w%oy_iBX54O4+r&6a{A#POnp4Zh2_YU zAtezmLjFVDp7cN4OL;?>Qo2H(q3;=_Fn8_M_PsSXqg|HvYWMxUZ*ELl`Qd`8rNy5A zjaO$po$_nD^P8_9(k3kF5IW~V52f6R?21z$gcS6)Mx10vafsaOLp^sU~yrU7xHkVvQyEH`)+2)28 z$t&aTCg^gn_Uww!CpIIdctV9~`1{ZHhUk&{i_u~c@}xl#vycSTJMgBYKhT$tF32xn zPavn%3rKI^bztn_Q5BsU;;pe*kyxvj+ns}PdiY6?D z)&1<|pvEXxOCk)nK=EXK02XX;ZK|SC+gWMUu#Uz~O=m~(6eyt{(PaiKbsWPc295>k zfg{tzaImUlja9TURPeyi)_~b{u{lsOoQJ>p_mpXT=LkZ@Y`JMoYT$brp*X0Aee}i> z{H4A{J@KUQVa*=}XZGc|UVqkDJO1%8!jM*@-}*Z*CsV%nyKzgNyfA*B;+yy8B)w*N z{l>dfrfqs^+iqC>a=h02R*}^jQ+(fj!H2)4z5X+| zRLn-Z3}RABT*NR(f!vQ5gIdIJOu_Xe-`yJ|-8kKz@^OnI8YBlk8li&KXHr|xc+6CffE@#aAcYo4pwyxm5Mfo3LXU-CD<=cOPd2F(?P*q7_Z0+xg^Na zD_urTzR%2QINJedcJmmET*r#Xw@qLo|9Ypzns{;ov`Hkr$P(MK`&)Z z+m^z+<*zJv{d{xG;nCyQtgIUOP1YCRjrni9I^*e-U%>fwiiB zM7VD#S`(J(zG20Rh&5s3Mi!vdH5SF(AaZTpDk_J7Jx2`MLUeKgnpnoMZW6d*)wa3* zS!;%-MXYr%SrhKQVeOi*bW{vSS9!!Fns^OhzGrbiM2Gk;;aE2b@pioM+LE3}BYp7s#Ad`4PY6t=7#wJZ z^t#bvL5rs4ViuD0dI#Q=j8~QL%(^S`6nWSa$SL&#(i?ak82jXZf>336qZ{YcTS$Yt z0snzXY}hIlP93?x&s~;s0AEDJ4NLGfakO9}2O$;1k@W7FQKUF|`{p)bBs$+YoXk);fo{6@G3LX#I94P5dz)WI4dH2@G z1mXNO#i*WH!Lhdw!s91PEEtY+>z?<_mwF0^BRpTYKITog{b%#DeJ9QklAg)B8ee{{ zgTl4zk=mCx%;~P|_0+(eig70&egE*8)c!}yVC&tZ_pASmv|79P-+TYZS1%rD8H2yn zKYsqRiID0PZb8SbQ73fs+rFD_j_!V-*1em{{P#cJ^|yPLbNytKvtO#d`@@^x z7RS8v=*dx05AUgpoZytFGkz^yN6kn~U)lD0%RMhUrQ?joZOi*xK7M2I^|rTDr)0Wh zjxKN+TQqu6tjpMC3v)&<>bHPMle|&vkx8zjTtXwTc#=zxkt2f`6r94bRuUK&x?qUg znhCSl2W7ZqjLvr%yKeNN7*xzgJ0eZ;A}%tRqwd^KAcKOFIMzx6x$nRRNjFZn<=Dxo z1r3q|AC1sLANg6lqY=qAmvx2?1Qe~Y%~g%RR#~pE(r>?>_ZL2&_zYl*Cq$t@j6?)P zkBDNlXy`_{bf@KF781~kCI=N_@TR0cRO6hw9Nq-OoHwid z;xBa{X;6n%3b4&B9I!2y5jHTv3(Tmql$0P~G;yd-Dc}b9lbEE=0Qmh*9l@+QRng#f zy2N)Kh~Li!!)bKbx)mmWVJ!2aXJ_#J>&)t2)+LMH@rKTLU%> zAPoiE!r&uQ1OLlKqrx9RU z+)XLR{J}we-=z(bZk(=iY}XBC4U%h|7Wyd7;vJ1hwz;g+babF-jcu;#;nZKt)sMHE zH@_wqpHF-SFvSxpOv4{J0vdqUUyN3U7!2DiprGYR4k~!zP3fRTre8jheS$rKp1c>3 z-oWcXd75-E*&+^WEOmg;A{A&uZ)zzG>hP8iEa!kjho^yDG_afyGmsj&pqC_V9TAnx z;6zl1w@V3tPN@TqXdFR~Fs)h*8;ijP<*-x^w#b5s>Jt`q4QN)Q_8HNHrG~TI$7wF< zPe~$WbiO`-PU}-VX%A`a4b`xbSv`t)XS5~u%n3^!$FPZk;|v}&GEEEzt2#y@-J^P0VqbzBDGXb(`t%_{jrKU1|ks>yFU}j>+Pm z>Z+^=%l6z_v@&Azrg$O$=F2fZ*BsldyxFJ4Gh4E6{Onr&zia2d`qCi0dgH30dg(E% z$@=!a`4@_;wWaItY|TG-^v@gHU@*V`d-tGQkpB7W!Pv;!{coPT8&k7UH+}Zv6VJ^2 z#=5lCpFPIL)OeT{tt!cM>^x_lwaV8oQvx+kC{Ij zxulgdI!l0R5}tIWe2wncY~EzZsKaO>M9vp9FelpK`Cq@XxCf?Pw} zj4LBx{D>)Qyctv+#4%YJ2mL#`42;}1c(PkcsfSl_zK2^(ac%}GW}r$oF-aoY-I+}( z_v6K&;y{kcN;$}VCpAdAak?9XG(}E>R;|$qP{YAx# z*b~Sp^#alxcpWHDqao{`AXHfe+R&SPr9mB+scT_(5+f`lH=1AtI$U5k7nbU|0IRKO zk=|Ta)ahXxMOdKC%P3&40*JkmbRV;s85(pZ*q{-13DD?`W_+I%UTx884}&!Ibo^e7&b9* zEJzO=nI?vVwGPHpMH>U=)WueT-{)d;prpsuxow2s++nwiaT{e%%HHr=-eMv=e!{A- zH{dN(!DZ-{sjmvDMO*gF>*%EjefhVa)(m_>2wK$U!kY9IQ)Q2Rc%?L>G*Tw}8 zl^HT^-24=j(uz*g`Y3uhEV==Vy2BDmj1>V{qAIc_H>*=765Gs3waCd3H0Q> zfb<4l2gV*&rHjKFOC2DzNPMHTwKS+(EE-r3PL_njvT!{LFylc0l5zkP#Ufb7B8jk} zqQ+Q9-An<=5v(~UDth>nCwIUgM{4lVQGzY=@0D3qix}!za)B4ITiOAR zB4V#3F?is}(2D)*aImVQ0~HO93L-}R!6u+O+6X9Fj>7+RKBZXSX_p|Iywhb)k2R4q z|Cj`Wxx8bf0rypJ9UuB~bJ^gm4I}!^QFy*r`FZ%m&x8p7`*dH|eLF+blsf+b>jti$ zpXhoqx2E&gUl*+l9CGH@Q*r(O8?VlII^`Gm{Qu0ij~t18dHKR-O)@U?2wETO;p@G= zM~a89(k*s$$#1A}U6sT?sKMAuiT3N1!D7PGnFYPmT?gK<+!SLDG%W9iBG1V{C)uz(*sr z&_{GK?;k|6%_SeDosXhbwz;8u(>&s3vd{NCzH16TpV*9;;t5eG5F-%;(IcW5EfU4; z%X$USaxn`DXxSWiQ!>mPBGWJH74Tul$?!=#IHg`ddV^F4ajbyP*|AZ>IYL!nP91-# zw~+>Q_(26L@!&AwNiVRGImtkZ%k*IRH3+a9E|cJsW?a<4ULv^aupR(^;;fH2g6cY@ zp42dyYYgytQuuT#{83I1Ux?Fd&uPtYm}}4}xd5Xx><;IdV5#Pk?yMvliR07t0sRR( zMC|}l*KtZc!Orw{6wy1nBQBX=mO74M69dPB^uUp6VmMgUG3qMX7%JWxusMK%HU~EB9*egY94=a zQ-BcWn>Kw;?omHkad~2^XIK2BlZS`>;CgA#DXm-EcVGE-&kL~i?v>h0dibMgQSaKJ z8B6{;`gSGErCUc`TDAugOG0Jm6V~n11K-y^bnI@ePgwc9nOCeex8B?O(%Q)DwK)i->QCN|DbwSvw0{QBNgEPZc%^OmN()d^uH|CJ*B;P411dQw%ndZ%)f!-Vw zs5a;gR1}Wh64hhKuudu6)9v4S{G_r5$$^hXXrT|-dY6qgM6%6gouY*lJ#Cvie0j&^@;{dS{X(DKCHQ>e zGeA<1AqpvCB%lExfY1_Rv;sxkHVY_d<&lF5`FK-0XyNpWtfQGc3dhy$fM?nO_qlp8jCQN#be<6jbC4hBBfY%Vuzs=&GeJ%CK^b%?n;$^;9CEc(M7 z)KkX?>jOG_M0dna2{tfsfZ0(*U#KVQ(qM2yF4T=-69dPB^uUp6VmMgUE1^>GMk!WJ z1y2nPy~?c7(|Dm|IRvY7M<{v*-4bN6QOb*!vpqhom;#TV?3({pBc#5r*1unug^Ygb zaC~iWv$110ep7loNtiQZX64*RhCHNvwnN{~=6|?1+cossJHL4T;XD7A`JugfEROqc zygK9QlwZ4P7r$B2G1slP&vsdpjEmfZ3JcuZMikBsa&H@4m{K^`uT>dJ!-=>l2Z2pl zD+k3AFu{Ft?^p))F5+1KG7hd8-Z4J3&8Q9jv1{D}3iI6Ch8NBaK*dBWBp4w$bCmQNV;*ln>HvzG8-fZJ{qBgKGKtU|00rYF8K%@E+|@Vn;TkI zay_ng&h~;CUsvPviOq;9o>1W+vC_~C0Yns|wN}Jw2(!d2B%oz;;7v&vDJSdHdn$Tj zPavn%3rKI^bztmaojN7X#|#GuRe{e*;xBc-x`Vn7R;R;Z#1myByymH9DNGYk0KOW^ zCEPfXXjxzvGTc(2j+O{GK(uyN@QG4K80>G3lj>Lq`!vG#zUFGP?hI(2MQxDD%}+4X z^GvXu=92!TBvM9_7UOJ>SWeZqvRJe=4lwx0v@Jy>g?dDn8L+fYEI7nS*a3BvStuO_ zjxl)P$TTq=tm+sl6>SU^JTSC1RPf}`u%cu+4Uh6D#h{LF3bKTm%DcY5_5AVhFF2@c zeGkDW?gZa?mAmt0VVBlTsQUhmX4@)Pocy77nviSy{IOom)b7evAA}9jFU%V2Iz01a zWyO(R`7?fb`atKPx2$IC2lw{>nr^M#xB0H}R)-&ZMs z_g*%pPjMMa9w{hp3L;m(VYw8HTfQo)j6vHYh$3Pe3OLxcQ&gw9i=v(%61&blW?6=N z-|S^$Vo=cwRZ59T644H1HWWlZr3~8MUGkmJLGC-ILDG%Wo!fguzla9OfsaOLp^xra zyb}@0HkWmd4i^+XZ=0)nqwlxn*5K$TCh9f#eBv{JDV|Va7G(%*45_~uEgCw%VPa6k zEF_>6O%5u=;7!SRRZRxbs zYQWF+z~3X`JI~~zflu0F22x4Ta|o8Uvr@gDY<&f*RYY|Lz+%Drhyx{!LrklMxgHp_ zv?lnq4!ip5b!T*jQ{eCfIz1*6FJw|Atrkzlg1H-yHeUv z#M_}e;*$AgDJdbr{C^A_WAMO{X<|57)vKX)D%u$Et6W1HK?P5YBxz5>F4CQ3SzZSO zVa;l}|9}r$-AjYN{|my&pHt64;@)e0*gPS>_12sh#2CV zeYEqQFi1>4JX^EYy7bTK#q)RGxEmPO`ps?IeXYkk6`#vC+}U5-cGAvys+Yf;aDV)t zJNEr#`e}Ku^sO=BdoDTU>5N~qtZ|2LJu-3e3rBW&IHlu^hQ92bDBshiE9dtcZHiT7 z_Ki@C^zPfSNHNmgC!%l1!j_PC_X$Zsu`|66Zq+G>fP>@WdlrxafZ9sfoFESRglr7y z-8HmjT1t!}qi=V`NUy#f*P(HDRDnGBP9oa$3XTG~?|kk?!sY~Wko!(*kaXj8TXyT! zDYij!jnhIOF`}RZWKpik`I1Re7uqDF1W7GwquP>G1i)X8=f?F$)Q3ttAH)*5OUbcy*5Kr=BIt!k$1+-U~=?;B}xpHQKCyf>31@XhUym zCk^UmeAOPl!h;{gfpL^wSc-)NHK-+ts~X?~8}K(n@*xXRO9bG{qRfsVxbI>;B@4WR zX^+9s2>Yn3jYbo^PpZT9>Xl~0LD0lgS0S2^4Dy9S(Z>NUsd*`*ST2=`gbvvuFm=FO zv5M&}RSqy*kz+#<4N;Hjl6ta?;}|wEaE!qNM~2q^Ux$NL{V>!Hc4@>qs-P8UykTr~G?zBM^Lt&W*b0xI>|WXStC0F$jFwwv5#6f#e4*Xbtmd^N z-cir}BrKRU;>5Pczh9?(bwICBZSMUwt}}CURa;I6qz>`Sefib*EdPyHXFQ$qYrkRr z{DU3e@fy_i=jKf^E^^OY5$irGc17<9_ffr8KpHkG6Q#c1D5er1wa4b6K?GEIq|Heo z1pvW34JslwynusS6{%fZr_J6NS+>?aeMPkUsOS~F!%-2^uu;Y_j85bbaxiqR?&aWPV278201Iq;^Wi&T(b>d+UkCy-O>1*A9d zIxzMy*&^18r4A5Uq%x2{l(v@!b-f-YQ1Ej$lpL-(jJH z9YDQ-Bd(%>JJquS$Ps3Q4^Eh1I~fg3%)=}Xyo!2GZ$1f{cxq}y6PB7zaUYVT1q<7o zRo4gLjnaB5I?ZW&Nbd-P)j75l&x0Y_5_{%^rFA05p(Y28F?is}&=&jG;b2wA8mnky z;0G~iYrx_vu{luEo`U!2ddM!f{7w+QGRQCXy&15(eGeSeJN)2{zu24gk-u_5*riRs z_fkTf>jw*8`{ku?UKHXFzd!PwjdSM8U#|G-u@@_zU8H#Bli1d`n(s?~(S1notldAu z8>J^|gKqb=S_e0~AAGF!?<209fM4jfmwT!47rXwu`x81_M|-`Yn6vL&%gEJFhs=qa zZ`~g?>Vt8!UV7>NRx$n{jNjFY6A#UvSvIfb4k49*r4|t=Lbz~yh=CV%HLW-WU%~cIB z^e@-kI#K)Rz!ZEw@fpArPpB{r{~9-Mh#slG7%eX`7-A|Vpp{1sD&*r$NzcG{tCh0L zvdh>L=*fEl=?%ONl&97N?~jVLVyOd!7O6lRdQ*TjsKY)k2KXQaOb6p>As5(1i>0t( z3!u2I7GO-OD{71|s8d^_WuibGJBD!IdJ~6WoeOs=8d@|a_&f&e6=pWW`#i9S0ydU8 zV=%zxB1YKz#b`b+ngB*kPcuOe`;jnjbwqkIVoNI`-d(cee3);jEvE?Ex-h34v!jS) zP><*`1D4i_1&5e_JD`p-zn#636r=}^4E6f2!@;VK4pg);R1j9$8t`E)0}U%m`WjfB z+fN?e`T*>9u|_eXToK&!g%EiBWU9ZXJO!zzX}n>xPh@*c>E3hThkVko zOHasiu6ihx=PIuFJ$>&fWset9n^)ia>5uJUhcv&u{sGzh<@BF0{nBOOz1@%Bzxd<) zi#Vu%(qhM3km}suerTrku`Zu)n)hZq^N?kaeE-r6!MgYJpil1?`PR)FGAdYv=Pt%!gN+ss~+LJ9y{PE67e8y>{L z_4C(v?Y}Z)@W@Qu`z70D%(~IjOVKz3Rg#FCe4GNzm->(t>*FzgBR` zF7^=aKTGceyR8#X@U>|e%waEen0qmr;V-vFn1(4gY2eUQq4ojM>+c71I&)%VN`;H=rRMAI*wrz1IL2&z>%SK{nz1ORYwOZ+88Q` zGi?nOJTbI6P|{Vx&gVnrRqcNeWNqZi7}LFnXYPTwO9k1^J{RG=I-$je79Uj#_s&&C zub;totFH5-E>G^6esA7icYgo9O-PK#$z%VG zSEoFk@oQuCOnG6_JY9_On59X^1#Ureg4}!x=Ct*8^O>zonbTGn9E;MROceV<0bFzY z4=fSN5NpljjyNC&0}oZ%$iJR7^ybXkwC1w4s3d z&f|Vc7_@jT$F_xYko%t1AnC^GhE1H&wxB_B;G+>*=p!$gcQhi|=8~)E2t(0Bwz;7% z%3R{^EK9mEVEadU#de za1^X7(8CeYO%-!@kO{iOT!I}1>fOu^Fde9`Vp7*}0RMf%imB~A0cJ<>5c-0HxMY4= z>Nti?3>*v614o8d>|ck2RUM;;pRf)1c%jqXZ7=@V=HVT(Y1M`W zp1|_d=0($~Ic`qQnf!BethfM(}aXv&kK&XoN<)a{JP=|S7 znA#>2-Y~%pn|$M+IL%s?k`e^;ph%1K8e<(%t%LX7VVf3SMgg@0!~s<2KCmYMdQxj( zdq(&Z0sM6lKWPnL$gmhRr!{&w^ykpS8PUxt<_=2;XZ=YoY0gR_b$q@)U^!ujsOvc3 zbohOs-cBhZDRf6%GQTV(C3Z1vV&GVi9yl^h3 zpSz^r-7g$shPI}CG^N)(o=ig8cvPN1q&wl>thf8X2 zxU9Wfz2{r+Uoy5X{!tm{l&3R(%H6%M&06woR8GgqC!Nx9M&s^?@BKC|Uw^px*Ow!L zWGSsuWP^)Z4a<}bUeQ0I)v!@ZA-|%3=O7g0a>zBS-{vv`woxn|@6Dj%Adbn1VI1fc zx7;?W?}|wiCJxUGlqI!Fk_}$hYFGv;c1JtNgYQ7%BAYoX<$k;vR2;}L88Ni=ozWoa z#_4uj(R*BGgXF+RBec**Mi%dAM6%6gous1!MbFvh4o@7tqWsL}uRBhC!v&vD@(f@K zCq$u0j6@kkkBDNlVnn=@reYS7j8?UkP3ob%4QWuGVYd3w`j)ydEZC4BB$71`bUHY9A6!4l~nA zCKymhq;XUdDa9l80W)_18@Y(2585dib?kn)>4~VOgU@2yQY0MU=G0zf!JToTZVa0k zIL6?CBh$oiu&QH?RkSf+pBFt1rV1Vb+8ihu%HcP4YoXeGpdgG~E{jjw;<+^m{!}Cg z`8OVdKb{K8o9$+r1eNEc8-Gu1BmZm3wNB$t|0V>c*0@brnpz_Z4Vyarc-ydNWLwT| zTyNR8F!hCPohJM`&F4Sz>Wrr|ey-I|ji}kY%oI3Mb;D^T|A}#gJSaFw?h_H*GDYsw zxpPi%%lL#$l%@osIHHVPH_q!@L_q(3gT@h^5df`><=CVYrnj=sm>zL1eby)R&kUCb z1P91{!h>5TqT(!6Nhc-&M7xC9tmS@)4)Gn$u}O(cPx77KAnC^G4w;xO2&^v@XX4EYWNp=0zG*zAiaUtfw70*<@JUU2MASB>5Vx35-bhsum^w% z-XDS=ndBrF_=p@!VHYFM%Yi$2wzo_$|yj+5j#;u<3Pz`?+HG51YfbU zz~_73^!`VsGLaDRMytjs+AC=+ zcAj9f96O50j@$q_u{*J}P9-@cvT_g;I~B))<3W1R$TTq=tm+tk6>SU^1d+Ce3L2co z3nk-0m`Qv>ek%~(q3ax{od4QO;oDL+z~d*|uWSD~q&_nXT8GO#eq46xwCDJTCi-+a zGHl5bVcO~UmYq6%sz%v7`Qt9psnN4sURD2bdUJnWmzfJ|XLR}Ong7PCGoDWQ`GtQs z`RS-DcjiAg;L9c%7r3R&D{>naG_SD0ZCKllDf0>!_sKyiAr{3+IpjJiZE_g_Bl{){ z2VE!tXuX8_bG`Lj+H8;SGkf~9!lFF4#CdDoh6T(k%tPZuw3|iTL=zXYn4=Qz8+1kA z66VkKB;RWrB;7dO2{SfzN@d?#q=h~blX;h+5*v$1F1duXB z?fKPn4{yik6PpoJJRu4+{FBlkuz7W(l`cj|42qbA1hi}pyeS#4s>q!BEjhef-8qh& zQZFFAf!Bf61!mC2da%?1LW|(#KOEFUq(NN|U!hRLmq2wEa*~Ty%s^^l0L-PL4nnXp zT@vA2a5_UBE%>`60(G?+VrON6C5`nx0R~I3ahF>)ZY&A^g8{P_C-m_7P_^znYM&NO zNCx@Th=9e!A(Q0{2Q8G5q$L7%wHayV8|%neJHo)ZEya_7kUL^^*sWPwCpsKr{_TK& z8aT$_fg{tzZ~$Rab!ZZuiZ%xPQm0L$SMlaR$%Nl14U=v3t`daOWAf=4RzKhO?!)6J zs1JR96B3VUdm?%ZMN7w>x3({JdAaSPWu{h-2>qM?`Dxd;kM5QC*!+m^+GUr_3J>F* zWs5$td|dnSn|}?sIUJTWo~zAPM_H{CBkym0U3TqOngXX^yaxWLq$v-98S zB3zktXY%z`dG0sQSgSOxpEx%@GWq`TXHMVB%?onBXY_E%Tb1mZC`)|G=TNK9o$_?X z&&z9c;%BMR?VAt#LhqE0Ga6d$u8)WBIe10$`B2XkrT4&CrJ`(LREkousS~8}Q3WV% zCF0^t1Qw6&*_VQwbA89s0)Q41ldXttN@0472P?)c8yGUJ)!?8crPsh1rJ{6TR1$iG zG(IYy_$K1w3}!U5Lu>#`F&esPlb}AM<_X%}m6wlZh0`2+~ zYd{J0sIGWxZpekYF>GSs7=s6n46WF|4hO3`hDAjiLj?~CjT{W+M%o-G!RPGcf}f!A zm@5cR&y&qIdN!X?vIHOJ;*J47LgMmTT*nr|=xg8IeKB#r;wdjv^=#`@A^4>icgN4T zBbTMDpZDXoPKW$uCqFu5I21bky25%bzVnBhVMDL$R&V1=R_kkPYl}Senk|#?gVWZW zUw`if$(^Ua`BiP*<=00!@8d!D-qgB>7YRG87w60yv44w8ZS9B8zSwixg#p)UA59+c z(14#lYkN=d+y8UQOs71Z@pC)TdFHP1mkw`xujj{3={Tcdj`(@g=(mHSrltLu6d_OP z93+n`>g?w&k9)pTPG>)PWGqUPyix3tLaxI%CX^9SxxLQ}4sFWeSVAV#+cd>>OkwL5 z!-As8tb@uZ@#cWiGAtqi#JBitpaz7l}l+Ce(45laf&To)(<8)J}O!vub zkR14Egckb9$m0ErNVd7GQ?xfywAwaz`0MKn%70TTwjaMU7@tpk1~A1FDjX|T8lr6z ztx^#eBPC`b0WGkDg9=`FQ!*%FPml{5_!oKtJ$Wx6y@A((^3=iS?3lz-2M8@vfj0D} z&eEU`zl32eI2^bYBb<1Fh2m1m0SSUH5-}|p`AzT!#^_5^Z6Ho1y zKePR*bwa02-z%PcZguAl!>4Wx_;XOG@N&2R#;a4F&iFN-)+sh@dU>ne>Rx{|$v8!s zGB{H?q+sya6y=b4^UDU09Xg&!n^I65Ttu$VcU8t>@up$&*@L_p)FXmpatZ9XJ+94) zO&(dwphRWj;B@7Xyuo7=QLzN=h_oq@xLC^^MRPx%4C)ciF}VbC-vJGhZk(>iCZFk< z4Uz*NjnG0L>B+oTQHk|7l1n~Bvs9zl7y*_5=7x+uvcjDG?NQ~v@uYG2nYs%?&=8xa*?KfJYI$bV?;02j6iqfcT#g@{j8TqIPY3>@xgYP7w z-JRJ`5dF{s^j#|Xj$wL|?~DdXH%_;Y|Ii)<4Uz*NjnG0L`B}Wn5Xm-|RZV*%MbFvh zs@nTZD8Ie&gLbPwI)u+BJ_DHI2^B69D_sE5Bcd3sKoO^4S1}6-0D0t~LO$Me z@0Yc*wb&Er$$J6m4ZIGNrx~VSn8Z>C2rW{9HuR=0(x6WEQ^yPJo`74I;{}FOmP!I_ zqE3qpMok@21DlKEe(Jo80<1wd{6k@cNNn3vV3hVK-ZN;|V6{VXY^8Y0j)Y zh}wrl6ZnP+HRXcgFoO)na!J$>M$8Oe!lHn_s=kWeG_pB=*ScO zMvl4o!l=ESr$$F-{5M{m@^r?}BWRUZ&eXqZruek!)g)uwPd!4uG&ZbLihOCG#hGE9 zhHlP8X><;XV}i)lN3ntwkDkz?e_8>9h87S-#1@uuaPiXksq@;dn9~ zSnt4_l7WY;ev!44wZoo3Pu>eiZ{T%c>|w2oIKHve0YZyZ7V8(4S&_s z!=CQqhF-XBmIXJ+L5jE^7okc}(nd-&Qs|U!HvZ`uRb^g2HvJ{r&RZ zQNHHA^wb^w#kH=pcK97_|MvZ7#_n`2o=~$!+|cX)*(+@3>PqKb$u&f(ye z{!@qdZS9gfm~H4)vLXsL^x7~X5mjKHF7WM1w8NRrTJAfKL93`D@jZ)!+&9_K%PHx` z=}uZXd1EGS=;f3g_-KR{`hfk^Sq~+WZ4Ni|!tP7aa@*X{w~r*pS&zT}MyJxf_C zN9?U&md1e^Y)t?^2Q4NePF=%`r3Q=Tti@OZnvj@`)uPF1W_pGR@F9CnqmD=!p*^V? z(5Dnn*&$jxE0{>)_u+;(xm1thdGLkqh)d>|rH*6R#K18I4;+~$hJ#fdLk0V(V;$k` zQ5}sENP5~FC>hSeN4&BWRiWPtvc-2@4&@(?djCc_JkIig<-zz^w^{XK%Gs0BLy>IGAE>k`cRt|q${%2oHbC<_2%nIz%{>x&wq1#%1zIN=An_j2Z><*6j*=n-B zdN1myKGxcmE_VhjeeCe4HnA}6a{t@MFV2Sa#-e#2K3Ti-vG`SgTo|C;kgz@Z&W=Ru z;?K(Ow?5{oy;I(4<8SA$44G~IV%qItxjUD)(so}k;^K)=XFQ$q3mWX3a_HUH-rw)p zNpMETDUC~C-TH9An5Rd@>wXc+JaV=acudRLvLM!D+N9noTNd>0lZnz0fO_{CL$4D% zF9Rh2r-lsc7eNXDnw!J1F2ryQf|L7g>$afd_RcP4r5@Q^@;#Bi{}TD){?PJ`qcr-eSUvv@}%l5H;QJni=st+vfo zeSLpdh1}J{bNQVQ@cG1N08>063bA4&iXeJ*qs1b$j+TpANI)x^98`$Go02Y4O+IH= zrGRY$7T+bO)C)*&;B}xpb^5G-f>32x(wGIkshc#YlP@2^r)xAI;P(^o0#np1B_#;J zl#5sd%RK6c1`WLZX|MzARq%aF>KhzTD_H#J5HkSX zBh$oiu&QH?RkShS}mu2?|umBZGXk-3$?#)(0;bp z|5N=}`=!zXP8#&XtiwN~%(l#R#?vXk zb^~Ugwye+oy?tQN4rg?n()i<0(Ic+UZV0&Y+oh@a<)a{XpQ1v)6nCHHGYSg*61GBq z`HVRcD7H)?*CD=rGYQyvMDNiYN+Tw75F6~x^wI{dT0XtS>ebu)z1O)176!WetSj_O zLgRc?fjs!`PPCJlO$PVPp)_JbeS0xI$+uU7q#LK}z9^y4yFqf`qY+x@!z+t-G$PsN zvd+=bfuiSab5*Ytk1MZzJZp$s|CRWB;xm9Lo>1X3vC^>+JtB&s2^8@dnu=LSpgfTr zR7k>`k}(B87vrvYMDYmr1aeBffb<4l2g=g|yEHP1r4A5UqylZ|O%F?hy2%6slo1XC zImrbUwD6P*@F615!6Nt-Er~Ft4%5kY0QE?)};X{7fEvqZB`k zcf@2q47aGn+f<47I>7G90JW)tK_>10(RLkhQ5@fY?~bD=1W{=bf*=+W2o^*%5X1(0 z6v2iG3U*B(pxBaN5W5Nfv9~}lz2}$2m>5kF6#>&!#FA!Wj7HOr~};b2wA=&5L9z|y<0Y!Ox#76Db$=0MJH7(Tg3Rdl}ZQ$aQ$ z!Rh^I{eu^KzKnzV>4v3v8+IN$ac`Z>cfnNSw~t=#aBa!H{hmAILSCPD{rf(DxvSIg z?T2q%_PJTBymoQ)@EM!B3>^1zb?oV^DR;-JBR?Jb=l@tvu}h=-{T;TL{Mw~l>{7Ng z-(^kq(pkkWYbN#fTsmt(0pVOCkzG(pt{Y=kL}Io}m||T>K7)FBax8}^E&#b+!q})` zKEnql#RL_(lrGJ6S(CMNRuKxiqaNX0!ikA;W+9)g*I|id6%Ia>vklST0@gmrp7vzxNFbL3Aa)vt+{8B0q;+2Mx=N`6pF=2 zltJ_eD@H3;#7ij^lR%1E99Ssn59H$`MQ254>5kB;RO+m>8e)wb*GhtC#7-MnWMUk@r8})#}H=j}-1#r1XCM_bk+?Wky z4}+gZad`q0EU0cEtI)7j8gVT|7}=>ATs@JP!}S}fU0iojEI|TI)J0r?(Ykb8KFu0Y z<646)AT2IO*b;+%ovd9Xixh$!k!1>uT7-*3qF4VtaLfa`O5+_wrk&wnRmWJXXk)10 z{-CX)f*XZK9=XOk7=Ce;MtJ?=eak8C^6dK`GvmicvNs~k`0-JvO<$QKg`20}Yu@Jd zgxfEv4UMPXOBSXd>H6q`A>ZCq{P4n8n-?d0Ur;`>D|+Eb#ZrirUG=Fa!YLcwqp2_P!vL^^=!z>18bZFdGO zPv)3Y5eK>LWL%}uq0?=UnH0rU8ZAjKXRq zfCA76OSdqNh*MCn#udRV^C(~ku?NAv(%h)12%lscaMM#gto~w9H>&mU8S6<+;~9`V zjnXGXl~YU=GDREpoP&>H7}!Kzw8#^EwndukKuzacT7oyoHZG!PC`V+O0;Be0*v`PQ zAl-3f+8GX3b&R@-Hiimr1=sothvEA0JWtsn4KCWqXbf zxwZMy(t?p6Cx7=?t)r%HqAiW5;mZM$w{_TT6MZ@cefVqj1FqmIj|wQVAn0Q5tzd}8sV%{ zIj7h_fP=(Y>(sF~y&9{233}gmZ)`N)pLh=-#S*&dQ0qG6A4$OP_)JSxkQ9B6Dg1?k=pf^QGgSt*Ghxm+CUFS{$^~jVjeNgzS{DJo-Y2I_u`hWCY{!KA~5K)w3q5|wfyoy>HSxgDB;w%s|l(VfXjwzFDwx@~tA8A+>x zT6GR=wBZ)o2uS4}jd0ef)M`5FQ1pm(Dlz*)aDr}f#RlIux8VJW&4?6FsBpYk=}3qk zVZ~^big-AsViE{QSshp?9kk${Rm$V!ao7{+$$J6m4ZIG_`_$V(sIaQ35uel_v>epo zTgXN@V25W|^_E;<)(N8$13TG3K4h5;maM)JreUKAmPY|w5d$rN8W4qGZlsMOVDKHO z5sJYz8g;-LbVqgCDv&&k(siN=qbAOnY9wnGtbr5;CR(^?Bce5s0>nBp7i(=4VZwb& z7tt2V5m~0dsQnmH`&`CA-NX%pTKMT8aAJ_|I5O=F2dg^PSVbEHb~6_n0iM|{I8zPW z%F*&b?@&RgpRX9)_*M7e^9?wt4|}N)*Hw4low=$`IFuUPH)765E-O7$zaI1O6jb-E z(x3cn#ZLLZ{rbJqZ;HRssq-?gF}v&je*I*Odj7YcPr5r+9s23WKd)sAM>QRs+Pr@M z+E?49oUKe7AEZo(9N)Q8nGohyIllA8U4+|JjO-42UF_W_4YPMGnU=ANLy3_bTTn)% zK`yb+r`qr?N1b~vrZ+Bo9~;Ty}$R6A0xZr{fW(p6i=uy zX^9<+y`g1D;C6_i2@rz;QHADNEGB`Jw>Yp+GCbkqjZz*c55%58Pu>eiZ{T%c-ov0S z4g`$aL1>oBM7}7Elm>O2U;^_?aPm2vI^(l7y3%Nd%F5G<+9a^WEzRvCv;t|b6F zF^LA42oBUXkR3&sAr3!~Ho+?D@M|S*wE)v*&Kpf9K>HM0Xb{cRGjpdIqb2#oYYk{v z7iN(s`h2FPh(?cv&}vIw+bZg9T|^&Hm*_JijM|T3I|IjpbjOjQwYRO+ZH9xj&Pk}9 ziZ%v(rmUu|0mFL}Z4TtLIQwO?;;E2hg6z>MWolmjz)qtcfx%qf`K2${0PT9<(Wghs zVw1Md`b4|V{gH+XrpOHwg{Mm&`SRcotFxTypMGxH#4Q7NJAb%k$DS{{6{lw&N-JzU zD!}ZQ*Ke-*X^6S`mH6u)Y~FPI-%nqI*DkmIo-*_)pnr~hW!1svBcIIMoz!t@)2sJ) zcRsP;E%U0vSBH)Wed?7P;j0#|cr*9-Y*R|^-0Y#3mcRFU{QIT%#0rl5bm(8eQTK+m zahH@IT>L28kvk6E`0b&l`c0Dyv^ZK4BfWUSIGZew?X$dvW+EOJn0 z3KmMns|GUrL!@$B929j+CgZR3Z$Vo^_2#7n6(Z248Z|A z6TyT9a%qB}@h|}tHzA@7?4QOsodpZqn>E4WO;R2O(5R>fPx2@N3vAm$CVLpZ57NTc zr|=RC|95FL=uhc2XF>7=N+W}oPcqfhjM1LsoUuV-Y5DanfhI88(nZM>EaEPDeco0K zmKtv1B07a~M3yNqYEf|xG5a>a?FNo}aHo-JXE<2Zu`N`zF<_Yhu{B`5xJDX7`lwre}CHUUR@i$_8XxzUpF7Vwc*MNv$>L`1c0s;#XfLPedFxco!ls#3hOCFfYZvvz`xEa0 zq3-oWcX z{WLY^+zvv8RUi$$sh>2c!$x?p_BkGS!t_C0)J)gJD9J%U%qJX-vs4OJX@SM~VDVlX zKyQnv?LfRdZV4Xf8?{Y(z22xc!M4;eyu&2;nnuG>(5^xYwW66L%pB$rpJ_gIwvL0M zi;@)c`B+O4t^SZLq-N#eq9?Snc3dPv;0DoWMi{k-js&$kaAJ_|I5M>0{~Qk1Iv6Sy zZ44EJoVJDvZVzf1IW0Vs2wuW_&ff^ahz>Hhnvor*)qV`OpD^d-S@=U#aJr}eoCk!s zU-xYPJU3bX*ol$9%^uP$_=SfY__aCZFQM_%N0aOS?Ko7Hw&{4WC{H4YSWX!SUAsolUcPBU;mEQ8S&?tDY+-gQxbq;K_;TGC(PvyOeaMr0**fs%2 z2Z~l(r$(*odT+vc?V_5e|C)sNCpIHeJfXtbVx`j{dW03D#UhkU^Ti~P(iR66N`{77 zxMyX;d+-l>0zG*zAiaUtfq9>LI|vnqoyYK~};b2wA_E6ErfISi$X_QpFIgrzxhS@Lv!cDjHf-rrVZ23V? zPnXo^aZukrx^rhB%AddgXumLd%V+Q3`{PVSarM++-JicIgxuQq(eY<|Lu6B~`g{KI z*Odvf-*@%*|98@=pbnp`js2f~3e0}FVs_H^Hk&I}HW!b8uhc%61yAVac_07%50LA@ zJyl8O*K&ua{obSe=JFvg-q$qK*Zh-wjA~%XA3rz0^nk>C0*=@cwV23^2WVOu$NVI)Bq4Lo^ufcT$5uwjr}T`F zX9V^PjzdA9gS@5?l@wx|Ucpcvw?kxz?F}6BlR$2}qE)BccDJkdn#GZ=I@h*aXd@zr zcNxN2r*clw(Sf2Tty9O2-pH-GSQj!Ztu7GnPrL_^;tAnl0fh#Lvy2u^j2<8cPfP-V zBbFSLiNiw4c!l%8W z;oUW!Msk7m$Qi}q5|$ZhQp55IoTH)yY%-}~=OeC$>opR8i#H$7AF<@KAu`vU?~a*JDck>6(WAfj?SAPlS#{*6BmbNV zp7`$f_&4US>X&!UVI^-zIY<`YrC2s7zDr@5Y*5S$&n|^)`bHur&quZwWWaT>(@c^r zpRsXNQY3>`WD`NedJ;vHTQPRfl9ijg#%y1n79fl5QY0G`+oiA+1>I3EoT%gy6Och| zlWbx;oIxwHB-=#M+IDiQPPgr@V#(6Mk*zuhHrj9tZG@-t{y{kFRB9C+btrnoIyGwI zTdNaJFTD`3^X6E*KS?tpg%hH{0uRklY8fpSj9eNiF$tu!#es#Afrmb+uNT&no`AhU z>II}X@H#N>;Ypo3&WA`l2o+H^k&lm}q(PlbX@nQF2AGa#fMsv+0((<1$^rbWiDm(3 z!QxwQc$}8!;0@u`qGp|Nl`%Q9Ua zn+nvUU`+21gm3P7eO1q1nXdB9&xI8h&lkq}f13ZoitFenq_twy4nnhF<7s^TGSG5RH^P(`IN;qiOpG8G zBTPTR1W<#-NF0|kz)FfR0T0$a=i(w@p~n!lt&o})O3LA4T9ZZtn=9#IH3GF!56c7? zb+tNU9cUj#3ss_-Bg|Y4V+_@t(=jQi05#0T8PFu@v04}oNd~w(q)-|Pj@h^f4&HGL z+ZQ+%q&tpGJHx@Mei&-3qK%<~=+N-0pe1Qokuy}otvp1Y;_EC3m*W&Gr>*Z1{M|Gh z)Uz}Ezr#WO(}8kd;pCjrz7Hfm>H47S_gCIOUL~Y|ow>pN%Ae0F`rTNxqQrB+T&Mhq zcgM|Io%r&{U;Qy-(BW6^j#Y<#I`Xee#|^(aeYoOA`M~j)+ohbXEX%A^u20L{8l+sG zurfY#tKWcP0$X1;L8*R9S zHo{YRuO*yyDz%ypO%y$1ol2Y?cRE3*3oh3@o{sk?HX~9zp~9pkAfS~HJyL%$S}bVg zG$;}ZoYjGal98y24C*QJ6zmD~M+10 z5Ci^Mt%i<>7d`x}AW|b|;C}_01rs+cSX>$x_DYiS7+{0g(bABDu{>I1e>Xl{D)V~x!!PDG|6hmqYi|_z zH*OIgyK?^2G)4X)r-*x23_2b%Wx4XJiXp+jsviqEJ~c1ZWoQ4pW7UzL4*m1H)Wv1- z@!r|}KKXocyOfJvikEo06vQu4#Jdy>m=&}{5e+K>0OyyD?CMBz?J;6U8UZt+MlUR8 zP+}y<7I<>7dsN7TUb6@K^@NS5i%6vQr3#G+sTij)$SEFvAwbdtH9A_gUfb8LY- z2f6LiR-JC!UB$4T8TqX`2R7Pp3vIyKFRX_W&N`J^Lo+E_ZJkQozOX*w=7-B?b~$t} z-k;cvNb!UU)A08gQ3=r_^%tX+EC!>TQZWexq^u4slnz?BOn^eJkYi6Er_>8bZ{T%c z-or8hVyzgpgU~FMj(mJHNE+11lt!3}2Rp~X3G)`=qJ!BdB8AC{6o5&FoP(!m2Et>y zNegQvwgiYpG=$UIiHKUVDM7qE3o&@+g(Z14xcQL@#&mekuRpBU9D*Y1u$Wr0tYfUq zBaCS}$~i4)Si5MECpv@{NnNF~xlIYuv+^u!V&h^J*g|u}B(uw?{TQ}0a4bl792r{s z{~Qih^%|(1iZ+G{+L=ZUeuvl7=0L9LD4Eij>jwk4X0u{%>D-=*qAob7F9Fqb}$&7hm%*!ucXqPfhY4mjNUOC;Z%(?r{arx8TR!7Gp zHzN(%s7P`Ro1BzJz+v0AuJUBi(0qW9CI$^Ah#XRs?%+|yME&6(Bf8|0~>9)g*J*(c}F9hbt?6k z*vknyVx3A{uz6jAAta(S^fNi$pV*8@@q`M;irG(;D=l^O)w!1zWxD9 zatI?qY6H)OiS6)I53g>RI0G!DsfZmdm~L*X2$%-e+=VUcVYg415~SC`J`lR5I*_bJ z=_*m>5L3k*>MMs#>KYD;9vy87Xf340TG(o7)v!JRGtcX0>mshbj|Y)u3XIy1VS587 z2I-C?L+kpV!@*hyYpkM;p@RE^1{1cMR@3G{4mQb93IpZcykQ_e^`&Cakml}vQgk?| zCv52mx3=J3_C?`A;mda(J-NW=luOaU?GG-z6eWzgFz%-l8?SlF|N3;)TRq0bR~-DQ^-cJN-BIPTuNW7VObj{Ng_;`2kt-~H|V|9!E{y%>ZaK*o{R@Cz;!!8ECtRV=l2wPO@Fvs?%+|JFw5X%(7OU0~>9) zg*Hl4c}F9hbt<)*c3+Afu}&pseECp<*1zwk;c@5j{={ZPiYHW<1r!<}fYe`%R+bnH z>m;B7EebRIH9OT5G zE1O_8i-}QOeuV%qLydDOR-p-gLg1ws%)( zPPs7hhd{-R$B%WBA1DjyeQMz6-io>f9y5-7H?P~AlBa)P7uU&r!~AWt+h3E+=7(}_ zn*a9w)jaHXn9A5(Z!B*Fs!jYjc(ys^fgxwT^^;7B_mgM;P&w0lAoSu*r>%p3zcsSp zr*jd)qV4?=EJ5VBrrkxpSc72Hk)gEoUAv%QRi z+;(28PPgsulLIG&WVh-Z*l5Eov@s=zcOt@Br*azTKt$1V)~T8id!DY+99rDhkhU1_ zPrL_^;t3Tl6DtkPkVjZCS~M|o9u470F$n~YTyjt*4+|yZ)fs%JE;Gnrk&L5z=*fEl z=?%ON)KA=_lyQvOL1>n^N9ho0P=~=9e)og}rfwKvxfQ$^V6z92!WvZ+fK?MYX941? z1)Ov!PGTX35B=`D|tr&a)-@--o4CRO{Q()90TpVKdZGhVi9P{9g zBSS0ppTohbUIm`3Xk);Nj7HiTD!6lKq<@u)m1tYJ}1`n!UU+e;I-4(R)p@MF%TX?5l+|H%3^%&s~+ z2&fD|tK0L9%S`GFfJ^_u5KX=95dy zyw~nn7mwV^d}O2J$u+wB@-za@nC|M7&7hG)WhG*hJvq2B#%+CiXu`nY;s9Becd0D1 z$a`%$Dwm>2EHRTyq+^**GPj+@ppis{+IHt4w_Vh#(`~z}SiW;xP^->?jW*mu8v&`j z6A{ijl?p#p;xIwcTIauSa#4)|#3HD`6n0SrmNm2yf(@(fU|QIA z&H(F)m|(sPyn=xZQx6$o(^0hsh(=u{svKpiwTv;cEGSe0#qy7FDdrGb#0p!u$ZcA9 zZAVNCOCDIeh>B=VG-bd`H9?;hkTmn~0Q-rdis47sErWJ}7(wWK5`pMZXTKF*O0nx4k7 zPzfA1YQUn68#5Osq?Nk5FAsN>yD#q^fP%@WS4va@h>22WD3{v_XVCOyj)h7fx1HRo z(`~yOI%Cje&sLoS8*R9SHr!LiUZ*t@&N`J^OM5v*4_T)Y%U`*iaO;`^Lm zjM_nHmP)O~K|NL))O9*o2@(Dzhlg!AjpPD5$T8X^F%oByiwm?E;ROlmf*QPkpnx63 zU@HWh&)X=%Ukk7*ZIcnENoruJMtp6gJEqeg1<5Lut`$`dG1ViCX{zR&j^{Fuz(fld zZ4rHqjga0@X9u%!5lconBFhvQwI9QF295>kjw3@G$hPvfh~03os$-3Do;p-o1uakG zt%6&CHV1M|IHfU09_8aE2tQ6&WL@~b;5`S+a8MtcI1axo^$4jOagR`2&~fwr*?+m5 z&M7XORUKVD)-|e7agXlZf!#CQ*E&7CrT6+>uN+E!$8%nQ&#oDF$Erg=9r@RJ z$TzPY*f%8ni&5`?)h^|1W&Xr8<;pu0?}k>D}Ck$;z1l6B9~N&>cl$iAolcCIH&b<#uwp znIewODd!-!oz<$-ZM(Z^>6Vq*tvc7XTWBLIm3K74S*KE~X}_mvopmZPYkI!~oztSh zA2|2K`xBcHDV|Va7SJ+?o@KO>#b8j-Sxf=}!0Nz4Nf)8>)T884*c0fsgo?o8c{r%YSq|#(t*HqPSVGqX`%>Vg5$2DJ6z1_!00vpk!M;Qc)Wi2d zF!|61kPN_^bs_>gNVn92{}1e8@bX7(Y$7k}4X`$^TCY2-gB2__rYe*^B&r-`sN82pfH?IK!7bHpUG%cw;qImB$)0Jj@B?!lc# zrk&wnRmb*F(Z*204MT&ef;)%C3pwKv7=EMVUR~c7gm2$d^v?7PT3=wqLH(hr^Y#LD zPk%qyNjPX)bmZH|hPh1oV%KY(dhHZO=PeM5UMx||e;&2a>F2aAa}_t_y1+>@ZhRT^ zNm1HwC12bfs}B8iw=&&4Pot4&Q?ZdR4T)QGB$ZC!#Z_J%h)7yDnl+Q z9@&-oa@$0P*d{8}b}5sSZ0EM>bldK(D{<{#*{XA3qYbyvMtLglM1-?W zrB>5kPSHcwsl?Rx?oDXgclr4tnHTZ?#AZZ_Csf!|taKzqkFa92L^3Cw=1ZtM5tOc2 zC>c|5%79YtCHKOfKu_KaNN?bEVBW(JE)GwO+CgZR_@w@@<)97^)$qWK59cO$#f=x( znSxW!Xe17QYok&cd^k11C#Ed{$q*B1B!v*$5@c(s5x3UStDDp?pHXih8&Jcx^g8`{ zDB=Vbb5bnp94oVdF}hQn)0~!A>Y^mYN@s>(%oS_Vh=o{JtkGm+NoU7J6hd91&y6r+ z9I%XGI|IjpbjOitXE<2vV5n5IFGJz?%AYFh3WZd|6GPsJ`+2Rx`>oV5n;-jN zj8o#T3lIG={e|`QLwXl}@>&x7P+EJd=ISxCIr!ks=MOdi?7jX9{!seP*%?kiHHYT? z9BLkTu6W*=$A2*1yYPBQPV@%z2dH4q#?$~R|cz@zOfD})NLa`W$N{Aj|#VAY>@d`@C zB#^*~8w1EenLsR*3^O=oKq((AAB;VLp1c>3-XPUM95mo(Q8s)yMyLq_{L?~81h^qCqJw8|W{L^Hk9Kx2 zm?R0CP?$8h@meGP-Jq${n2v%Zo?7_7mY5+4boCgw!67}Df`w^lVUiT{nHl0DdfgFj ziV49CY&#fEN3?SB5D1|hk!1>uT2zul%)Sk9yMbdK+;L>u84gzU!{E7!hFJxzKtr$M z&4HZ$C@f+WDC^a6xgZoOWmT$LuTtF^xc!9xmF&&|;`~D=mug{ddGI5fe*MSE$3JEL zhxO?~pC50z= zv2poKZdP>qqV1b=yu4(d;qvmZWck+Qun}1(2z2bsT%tnQnORJSf@p_BvtlLNxlB&7 z?cS=>ZM!?mck;Z*R-FSIZMcOt!c%!CBAj(9^)MYSD0<8~HR^JQpoHJm1Cl%aT#xrB zHY1_HLxovDJt2Ce{$gl~#9+8nDkgzIzSV(+l3}JA7xa?#lJ&x#Ku_KaNN?bEVBV+R z4nl=t_8Go@87d9xIvq@Dgh_YsAPpx>F@y^&RLLmJaijoz-N`u%5Y}zd!*@a!c@(e} zft4I31ELVVVqxY*OE3b$(g3g(gBGUS;PL}H*tP`y9c>L*aJP|$6*+x9JfHNJ>AJxC^P}GKTbhRg|6Gi- zUStpSuT%l;_`mBhTH(V9O=mZpOmsFpy5{j`!@m|rzPV-SfLUK&m+!AUv_fVo{zh@^ z@aEDTTUN};-Eq2q^nqqFrLp5*urlLAf8LUNRA216{vCW$KmAS4KA_sM${Kfb%3JPz z7CiW=$@P^dw=SxWG9Q?j`{mcWhdSR%%!yyt?7V!y&B1Ybg>!S4?c6f=fp6+Y${qUY z$iGf81Fk*zhKsyn`v> zo{yYQJhC(L$#sTvD9QGT?mIn^LyIGcAY#+O07!$};;^~>dyMiPxPELTPH7BO$|}bC zFGImV6v-nhaYQ5yz!1rR|7Too?IR?JGl~@>_KdY_#DP+Q@^+e_~%J zoOKGOG?Gk;Hdv=>MtFNvY4(O`5BR9@{=|C#DV|W_Y_Zaj5Iw?*q45y$2uj5y5RfV& z2W6IFp`@ps#V2){P6l6T9QCFr?**hc@H$XGb+Ae!;~2Gr&@A|(GzWT9f;6a`;F(wt zpF*l}hc+Fg;G%)Ycads2BS99qxJruw7Mzu&M8Hy-ioma)%qG*qKN9RZ0lq0UoiXW8fMf$opBGh5G1b$I(Q-(uv0$l3L>FrYYY~-3b7}|-?#qzH!qwPR;T$+ zfA*y>(v<^!uAYDF(fn|iA+GDt61iQuIV26aP{IzQ*Y1g(vk4gDG`}Z@GKfkjVqp@v zdC{c#{T3{ovm(C2EpNMrTh5g2$_iADK-($A3}J`SE7+dGZF4Ats8HJy$Zf~9>U7)g z_MWh$OHixMwe1$#2+ZLfjdE;AA*r16G)ZMnS*L0aR+Jx>U0U>y*ULlj{=|C#DV`81 z7SJGw9tMs@D}^MwCk9d~CV_xdIXNg(frXMWbPy9L9Q-J|n7}XQ~7xMM)f&;^m)Z z;@ly$Nb1TF<7rz+d&&+5Bb&90sEBezmMJhw(Uvi6XW*CzcN`hoV*fcDtm+L=JNWnr z>!^a@)7HQ}N@*h?*HkY!34LXMbb&8Qzc?zt`k*4P+tHa*UfFtHL}khU;Wef!ti5WSp$Z=wNC!(XYcM7#=M!Kc&D^1J}-RBj@8MRGM+g6 zi}|McM6=&tf1Aw-vA29q-nw{czy%o0n}59KycekU#n53!b4GFf|D9JnZi;ChzV)e) zUgpmR|MT+K)f1N9dSrOUEc3pJ@7)|%Sa4`_eeT?+_I7)BQ(Sk4eme5cvu4kp>qcLv zq!Y)kI&{a88|s+sY^BQ|H$I6R?3?CPmQ?1nE$~+XTK#@420ub2F zV>-*Yoed0Hn!>SF5gg>UQ(ASpZFhTT&L5TCs&inY4Y$z7lpNmC2xpzjIZb;xMbBHO zY92o7Tczojz4iQ!D|mn6J%AKXs4!`X0n>*DhXr+k35Q8Enf#fBv%q0h9@~}|S zMb1D^aFYEY`vZFdJ$Wx6y@A((`e}lXj~K_O9fW4V9;G;_54RlDVL7fwIN%4$Mw5|T zj9S?E0un$;4q<3MLa>oC%Y{#b)EbxwVgsO2YJiEr$Zrc-m=`Uj&V@8yc7^ z2aCWoY0qj6Cqer>S~w?~sb}UehZaThsk03n6w7C7=sxusjJ9;q4nzuWSrZ!<(HS&H zOftKST2zul%n2L7UX*Nwq#)gKWN7XGb2wPl&wx`Z+8FThgO;`iEW+DFn*+JV^KdIi z%FcGq7li#&6z#x)K4~t{k_R-PJHQBM#aqq zH$GeYOJJ9z%^Pa^T=;sQLq8q)=Q*_hwAXk1xwT7n{3?g;IC4Y%hWlreYr4D>$?mtUQ?P3>gVq)kLBy5> zad3WtOVr|!l3k(kDNf501D)nqB>Hba!8jCwG}s0RY)@f2B%9b?#-O#!B-?=;1pT4png_%7JC9cc`qQnf!Bfhsf9gC z8ONv{gl4Hg8hX*aR~gwK_vn6K;|LSHlT|`Xp%M*$6x->nt-@&zPn&oYPAn?2n2ro|5uJ zpSb~ezb!g_&PK>!w1>fez}7Beai~l5nGr@U!o?w8f=`r)Ad;%~33fp*HcR5n8}bv@_) z=h3Nd2Tl}}419a9FfBQBmb>3RXT{g1$2Ch0oku7~jl4SJ$Ty`|&N{z2t15JZ*))pNV$;4hQwA0hyhE{`+pXf(o))SU|MY-Qd2IhezdojkyC43c(4n7>{PTG|$1r@%N8N9Duc~tB zjw3g&W>&g>woi4z>Dtrr#V(!&K`yeO0xwS&Stq~Z0x#Lxc;pK6k&TWg*Zv{XBMCSo zq*n-$i3X^58ON5EF}dFBR=P$_8W6L6K>RWnkAgrKSzv*e2MR7jkvO7~N2KGJP6W3@ zWQgqz99z19$w{{3T6MZ@ch~h@IkLD_=fFlAZlR53IlNa9&N`KIR_tMfoU=~Vtozru z>iT^{ij}<&;{A#D08%`m!YrWB3?7!ziV%ZAL)c$T0`VyDawi96JkS*~UY)>I8s(qM zKgXUxPN^4=-oWd?ywABEgbJ&S1{~B!N`pE)Wf`>wIGV^wE{$q@_QqEq@KHEps0=~| zOBOaGZPeluJ{30@C;(ecv_$L(!LM$%V){lvVU8wzNebVt>R}NP<58ow1|*N5bgigz zh^Z2k_=+;38x}0}s7}fgU97S}8f}FdO~-7-Og1i}B9TJ}m;$5rW7y8Xu^`=XWZD@H zR&|UX?0kfEgpZHZv^8LMx|%iza!obx9o=|ETCl$$Q^}PcLR9}1@{TyDo4R&g1hmV6 z+OK!XBHX_(Y?9yVczF4`wa*0pCM?>eoG|9^`>#0dxIS?F-^1T^cg{BZjtxBUqf7KV zD>jZ_;(vFnI`Y$@f8C~KcYeOk^}X?1ODo%@T;`g#s@OF#epO+aYhp|%&sBvbLOgPc zVq^!=>tf%ol>}TeZT{#Wk^#^r3G>hA;C7!?VO~joYmzEUU6WT8xhBT0DlA21cNB>w zW{QY3y@H`~ZaaXRkud*U4szS&tvcPdyVK?l+Z5EQb8WkYHUd(4N244Y1V}2imL{pp zA?sA)t0f)@=1mESS&@I?{fW(p6i=uyX^BCLeIeSVmeFDn5~)N$%Id&ENf)UlUzDaP z(y%9xQ|bkzH}E>Jy1@6NVm%nOgU~FMntXgTN*dJR{e}q^h=f(l@oXd)cu;4Q1C2Bv zvVh~c@VTcNzAA#>Rza8MQNU79L|~F)OCh~f4zmQ4)8Rh}SbA5jf&J`YuTuE6)O6US zIRwZNv`{6QsS&Ax0ft7BPu-|vx!^4+zq%!$lk|Am$83Ze^}HNr38r<}xOfy4QJ3g5 zBaB+cqGbrU4EMnnu^TuRq&tpGJHx@M4ow2wS-@N&LdcxVN0~zg7 zu5^ps?CI7$ezPLTt$V+<#hVp=O9(d~AipJDD#>-5lQJ8#=MNq`bV3k=((^gyLlnD! zT>6}@TQ|#tljihJD|ZXu?C#b*cC#V?1&dIRaPt8I+vUtq7Pk|?p!8gh`4B~G+sUmu z-L|{)N006v*{XA3qYbyvMtCakGK8~ErPk3dN6{MVRN~$*{~aa^%@sU%KaTe&HX~9z zAqp&@o)A6DXt|3K5~)N$%Id&ENf)UiKa}oN?8KfxPN^4=-oWd?yidIygbJ$+2AtBE zC=KdH*mg`~!Y}g4NiMi96#_^((Zvm2M+qYUDPhdgxBH zv;^Oc+PH{{L=FSP6U=9295>kjw92~aImVMfl9$Ub*vf;;9}%da0}4pK+aeX z&nFp*xbBY&vJufvnnefuE!{N{Za>*S`)*|e?J}<0@1Mwe{d_6!pFdZ3T=mgkstE%I z2!)YeLmI~m3a8>T@1*1<3HLcaTs(YAYL`F0^>{|xO?`g8*=T<6=8+5A%*_Xn-|%>L z{aH-Plx{9d-9oZpLaTFZ+xe?-;p~G-T3{%lKQLT zbRC`g-aJ$08o8p{{zhxXgHDR|b)5XJ0u8?VO+4FSsC#Lz}WW z=IzPkHjVX}yQ9*aK&0aSwzg6eJMjLLSjl3M*zbMDL9ZBV!qe&`r(mGZ1QnyE|n*YphXw($p z{fYMgQaqu;EXokr4VKYj5fXzUCV_xdDLE*!0ShJl;T(BVk5j~9Pavn%3rKH}>L7Zb zb2|vFl&Vpe1HCCp8q{?#C%q95_~{NNbdn2f49Y2Ihz#t+30YbRgg+5r?HAag2y|&4 z1uUgS#2#d{RcwT*=`d3UmS~49l6896(^>8*N=gXHbsFN(!Pk5+HHy4IB&79Y>~};b5(U@l?^qfF}j` zm={B@f?I$#2XZ?6j&6X=+{G*iHMbNVgJSMm)n_0M>ixnt!;BcAuTP)-3J3LX&*Z_37_+u(j_P6am~Pj`d%m*P_`7fKr1HFx=KY#0 z>&^s>_~6#yWgRo8-22|en>`IXrtW;RW7z}4PwzUuFV>--j{NKN?DefHU-@`Y-6v;e zIdsR78~-j+K3??b=-TL{u;;UtLFqwCugY|or|z|5QDnORl8kKRdIlkzo<^>Teb>el zuxp`fA=1C?GC=`c?n1S3#n3Q-9p(kV^=7#CrfKo>1XzvC`MGV^E&ancbXl>iQRj1o_cT%Sk-}qLY0~>9) zg*IYSd9Nazbt?53?d23bYMn}4GkJf4X-`-4+V{HR{fW(p6iVuVC0 z5sifXNoIFePfi3_v?#u-g$V`=ZBb zgfQ0`b_q2#oo>>e1j#cfT`#JfVyciSx_Fjz2A&K0#3QB#6%k86FG(|hBvVYtq_r^0 z#Bf6xr#Q13hoaYZ`j06Lt}`X z_N<@~LSz#>G6i9Eq1;WC+%-#$YrY6yy*Tdz5EpG~$bR9;>#amA0~7 z(W=uR+J?Ie{e9+@wd!2kZlR40IlL25jtvJSl~YfXRHoiKRii9iS9Nt^#6y3C$Km~n z_W)8nAqw$gB+?*ymeHCb;whAhNgyB{1;SDdI@G2WF@Qo>>I?fOo%+kT|B}xGcORVCO z=Pdzlh>O5H78@a0vC9qyGhbj54ZNR+6#?O;KFt0;tkqP3WDQE!iYmvLYBghYC_}v0 zH6D>z>Y_!S=rc2f6{58;a+?yQCxwdj7RBJ#+ZHa?L2+0jDU}t?sQnnWGjPm!i_oG#sP6V z)v+N#*mgeUo~r9%%I}xSw`{3?Nf@&3qW_WV{g22;XEw-Q{dB-{itq_8&qiNeTdckO z%J40TNA8YQhkiQp&ts4JWxexTSL(f{4{4Wjnp5PkAgAEsVZLQf!As}m5A$8KJsY`M zLCB`ZlWV_?BR~kiF8)plWeoDm=hzxB0Ma1mHz&+@ac0cA%#zAvr|@9`PQgXPd`nR< z7e%s&N&u0LWjY`v+Ad{~UoOYifPt2_%UgB2ZFd)JS~9DwRp-D)8*ZVE(p28j2xpy2 zJxa#}iq=`D5vD#^e2o&4?6FsBooN=`@HQVZ~^bi+D1nViE{QSshp? z8D?t9JoQ&)uVPQ2C+`KMH}Ei8r=B^_7*b0 z_B!zAjHm(_HFuaX8i~`u#yr-A!B#)kfR;{f`aL19M<^1Pxu&QH#;q?o8tAZfX)=)uJX(J$~IZXB_ zs&Kv^7JGhMP+$7aZLZo62lc7`IT|2}cW%`!5@LL2JRSdUlDzMQizkXR&k5dteDPzb zX=#D*gYUv!^S}6LuB@;74d-Fm9hUvLtIG@V!C807sv|!g`KR>%ZS;T*3uLJkv!)2` zQO2E*ie;0^yqCgtZE&{t(gog;$Yn$#8xlmWUXu#a2pH_W#0g{}1E3^NjxCo!*I@%D zlqe^zS4QH_M@6zprQS>7x;8iq^}>ltI582x3?*|rAS>E-=h$)yB#IDy3VMl2AOKh$SSaZtHE_?$gbMhFJ%OCO7m(h->%hFneU50Y7`21YEcl@m zU%#YCgF3vM&}m`Sa-#uH_*fJ!u!JC^O`Jh7k_FG{7A&l!0*~r8fZ7(Z)lzFL%)|FT zXbCX*uLZt6(!et}tO^VZ2{h_X>$K;<;8`?!UbK9Ql~B)^CJvL(qy0h6k-m1*P4lu51P z82}B9qjbB@Vf)OYZM^sXXGy%|d3b*6I%|via z9>+m$JFr!!+jci*epE(rtIoCU7TQ>r!#fe-tW!A+bWEUVgLSHAVs>GbcK*lHdTbbr z_b1*1Nb!UUXN#4FXjc$cj1~);7%2$_sd92qrUDBkA#UDyjr9r)^3DzzrpC^)!B*_I3Mopa2NgVuk37>p|41VkmTiNMg<-Zny ztq8nqwh_|f#29N!FbUiQ8@<818ZGRdqsQNIO-+ZI)U_aa1f`FPDu7YdI>wl)Ij5_V zSnA@jmVn+yMBivD1oPBw6z#Z3d;oulEK^|Aehk|gI2NQkj!Zkl!CD8KK}8!w1-Ali z4HeuIv^kJ7Rl}Ra`(#n>@WIj21+p{Cp6Hx&nY@1aq}DtM26dN+^QJjM-g7fXo^IUf zlsNyF1L^sV!u`h=)FeKbT_Ah=kEd&EzB@2So_==4iu)#KeD|W)$*+!mz3=W=b?B!f z|J?NJgip>N@)Q#CcD747Ug6odOyL^ecV(KwH3||dhxV;R&Mh0+Z9(MPX|6J#fa%W5 zCvs?-g!vRRxoLBD`pgX77(IJUWvs%zZ>hpHw(rVhRCY&^a$+WnNCz;TTyC2~(ykz~S(KIDoxiRQLWViHJlivtTKAjlqL=8FdYr%F~R{e!eOo=x#(eW7fb+!%|b znWa}(J&**;1nj%zb@4TG^Y=?`te!fie&Bn!=1cQXzthz~|58EwC8%oytJbikEMT+-7PbCYKwzV&0Z*F@;WHJ@c2jOj#7@lAf}t>l73OI&pj) zQJF%lMljnc+z!n^+Z#BR$>k*5d96C#w!6J{4IWq4s&inY4Y$z7h8*6}2xpzjIZKBN ziZ)oMYBnFOue#B^VDH}3;dp=IJ%AKX2n>sIB}C6MS`}i@L@E)G$|474reL9Dn8BXl zBL7$ZFZKj-O1*&e23`m1r%nsY1W?E6m;&9&4njp#kO#eKtmUAtH|X)8CuLwlApsg2 zHJq{F)Do!Hz<2yw_+1nAK^#5=C4gwr5`js7ErsBNQ9Brv1TSG4^%?`L{Gx3#stsoh zdN|0bKP#%7WU8ka1M9nRYbPX@dc@Qu7tx~L+Uaw)kkO8q5nQu&5sRa`K$#InL6{XJ zaqSHp^Wcsn)6Q_Ps$*DGv@ui=MA{lExCLkH8ldJ}ljc zgLiluiB`2KA3_$uHgmE2yu&@kGy_r&0b(+t-)a80 zGWuXa%zRx|OplD|Q8UfkcK4e$=uF37Zk$v8{q=)ciw|6XDE}G%p|j+#f1N6n6ucJY z$WMp<`Bi^)GUk6%=RZAqZ>b}99J+C3NYsNhV_*EgRUaoWEOyCXobNI_fAQ>Mm)Ucs z1udSv*eepb&XLH5c#^Ap<|>lixqGjqM399HfEJe#LB!@t;F{h!J$!m^kBaeJ=90HK z&t-Pr;@QhkFc9@3h)M)8;lT`%Y+@T^McW%B+Y-oad$j6w+wM+ZEbo`!s&inY4Y$xn zUJmb7A}97XlFB(tJ0F!fX`QOs_W8F}H_LW>-#s@3?@zo3km3nZAT0q=E`#Vcil_0$>1s2zl6sX!WfQ@S*$ z!|EzVeG?oyJdual;qaxWNGS>c{Bp@4NeX_+gk9~mYLi7C1tbGZ1U?Lv42VJ&90p3*Bg1GsX4D@6?P|0DJaO?5Ge=PLStoH^3V&=Rh1D@}2AH@3j;JjR zh!s=o?1<@XT&xA_SO6)6mBgq;#W}>PYXjVF;8>9EI5O=F2dg@^hl++7mfjVk1Y2W> zF+@&R4d2mCRAdJq6lCe+l|A3j7_js=*w9Om%lH2h543Byn8QV~{$EdRsC=_u#~U|3 z%8Y&fny}<-L&pVEeZ!pIdm>}Ol6S7`b}sHS;MuZ`_x?VlVnSnI_e*!jsv|!g`q%AB z)be{LuX=Eg^O3lADVMqCuZee^k-cVMzUz#MO3yU|bH-*PrvxZx?1X%BojGG@B>^Xn zigF_|69C#s1rei)1RN_HwRXU;IkV(trLMVaVqIrstr?h$%I;_?iz|io|oRJ36vy2uCS~3lum;}=Dz(UD*b&Pybnytvjo%hE+m0qahd_b^+P!ZU941XxikOp=5-3%tSkOK((eg_vd?0X?n17~pl3(10KehXF& zZ;Z7@8vyoA;L=nCHfFYk?1`CRa)}m}@`CSUnaq$G&!UKI`b{VxF!*&La1?i3>)6Q_P zs@Fs9RJ1YRow|X>TLpIxZ4TrN4X}xqkMNW0D}pdFSC+oLw@2hp8{qa6R{s3@S|G|^ zA3m2Tr0)9kr+v@GC_2lAz3E#pSh(*a_uad%O^=riaDAqONAEMmvbX&Fp6IgYr^ajJ zYh)WPJ!@_@Uo-bGWSPxxZf$;V-g(W5kV1GuHx~tGJ_+Rd`ILp9nm?F5DrC&U+M9nj ze?IY#vKsTRs-{`9yeq1jzc!TUH0Q!fnkQVjvgxnvkbAt}j%i-dZLvc?9r@?d%X5Cx z>jQo{wkf}xLw6jxVLY$?zfUs14jTN##@{35ks(3y;K-29Y4YGum!Oc&L+2MG=bMe} zA_6Sxw8g26faz|1lPXCDKW%S(3B4#4|!35fUg5jtJ?Tf`Wmlw~VMv zAtnfbwl{D)72Ng|j%|}bZrh_(r`vXSzk;o|F7yQs_4^EJ=xgr8E_3G};OoEzC1T_}sOniw#gRDI8n>bIN!7Ai~EGE$3g>!K5$-9-gDl)B_FR4Lj4zabMp%Jmd$OhxaX0#$JEQTH#+oM zmM%|=HT-m8==o_ds9$crsXJ>n@1EIQ=dt{PyMP z-Zy7_zy0@TMn7Zz;qv-R`hR}@vw4SXdgH6(_eC^!bkdz)@!qZXkGO_(|FT!5Lq8q) z=Q`&@<<#ds^B(#7Y@JnagTE;k@Earp7NZaUb0AePUoROLuEn8 zd6gmS7fG(YyRM?(iup+iGy@<%2`eFreh7L+_gOs0b@}+|*%9)*ULNwCz+SQlve;NlXHP{BUwm zCISm3Bhe`w)MZXGC+rFISwtq815+fqvlQz`6Vu#vUF2rEH0!7}JZtzK;~ zo-yj{K@!iVbE3*=rb4^R|C1q>BMOz;ti;*)zNK;>b} zOG8bASW#38LrlbOB8m^p3*svuiII6Lty?$qW@cvK12N494_}p8YW=U8u`k$FVE@10 z%$eC;b{8Z*)t)oI`Ofe6JKy=8Gvnc$Gc&vtU$!Dela>f*tW;nz4wD0F1p`{I*}!1H zl_f;(7=E$@&9IWbC86*^7d(D>d(|OG-Fgl8=_d(1x^K?0y8O`Mnu&)gAx zPyWd;)tNUmk};0Us~oP*dH4F<=k^_aWpGUhOz8Dt!=4+F#->k>J-ohU$hkMZ!HK*Y z8)q$h7Seln&R@UT^xcH%yTY7a)5`axcU}MDA>+ECv18wxchLRO8L(1S!Z<{>r^6P!4eE&?hU#l-ZIvQ_}j$InJdj(BBnBoQ% z&J`;Sfh{zTmcNLLK@qc%fK~zdQK1kIO8P50p%;DpG9FulSt}r|f!BfZtcTa4Ok$}G zgciXx9ZK*w)zqoOI3M_q8h*97;fLd}&@+t8XDJs8_92t7XtSfJg*bE1&I;tz^$xy%rL(c|$ zF>9bPL`hQ#A96hiIDtNo0| ztV4~r&i!=vR{@u?Gx{|+Hb7Eeb~U`pm>0AyZl3F;M?FG*>g8BkV?2Lj`4{pD>*AW+ zKGpaR+524A#%X?CZrrNc*6-eT<5#^{;%}FyJ${`RE;!ubkE|W<-}q5smySIe+M>;o z`mXEe&G7F3d5A1|RGciqWmIT@EI~S@WK^in;1HCy1fUq0O!iZqRwfhBCw_-Jho(|R z#GcDwdQ(UD+_ZA)cGrAmNU$tvRE#X4%cxL)G~SIW!NiR}(N1DEN!&Mwrcy=XdjZok z`3`Q8bnA3y?eu_pv_uYkv_cDgBq@3OB9diJSw&kRMJp|Hr#n5nrtGdVx8J6B40wIw zHGnB@P+=PWsZK=@J@aTmG~u*Nbt|mB{PLWkxdPkbg-zJB}Hb2A5qeyo9q};&wX;GdNr)uX6XqA*jiCUW~fJWnE^}9 zV!2R>BW2ne-b{d#^7nbKH9O3d()3Blh&(v`_yKCK7 z3Bs4Y(yHLCo}u%|a(2B|KhYaH_0CR%3SESKb8@BgF8<)S<$j&V6N5^Gu+w41wT0Wp zN;g$(^iIvcFR_6(09s;wVT6c zOm-{581|qm=@3? zIq=a6E%f0J3%`jilt`92Sk8_VQuLZ-?sSmvvt_D%p+A2%+<@07UIUoo1{H=gLZS?( zp^%7Tv{;0aXt|h$1hfjsj|#Ayo!DQ|SW&!6e6=Zt?F<|XQkx^w&Tz1*V^|dQWGEoyG;#{KVoZ{%8anl%l1tr( z3c}}=((>Ts-p5kEfXh$fo?(Eg1B6bOlfU{vcrR&Nc<&EXj#(evIOtaNvJkps!)Hz3 zy%iu`qubo5?dWh#wqjAu*Z2Gn4|?|>qiz=eCl5OH%BFzdD~-lyhc*Ts-*Y?ctx3?C zH{IRnco|aN;CCj@FwT58?a1xtT@34=_l~}|@J-`4^D<*3fvJ}o|IulIv~&H#fepEZ zL&m#RMqWu;^R3tH_~mwa+T-WaY0RFb#^1t!O?l@JyL9Z)sMS@xm@jx7>A5?(Tbw*2 zB3Ygq7m<)5PmLJp6_JpV>4j1^fKoEuLdbsjk|%QsIN`}z*+j<;AW;#qZd@Pf`>kUl5TbPmPI4Sb)ZT&{ZGe1|aYq%zWo?-$aM_ri#Qj*E9JpY>{;9bVu#j zvN)tga^RyCTIeHK$s00}EOW|g+R7<<#WHt#RAqRXYMOELdv3S!`owDhQ`{ghxngi2 z+6&C1l_TO}q{J*Fpp{O3R9JuqCH)m$c>0p$61D`o@m4@u1Fr++i7!f-#8MjwEfQap zK5y#O)$q47e9oeS7mBdKLlxMm;LWF)QWOCAz=cCx3%Xnw+FuW!l$**Z0F$6oM_dI~ z1+s>$J;5aE23RQ-{(h*##Zqkq}uVz}nn^L@N zDib+ZSs|Kh){w!&FC z)s$P|THAQ>5On6ow7Ua)LGth^pM1IT{q<*)(+_QJSoyN*NQn0-0$LuRk;8#*uU9Vh_AzmySIe^*{QQ zee4}RV#I;iZ2{6`A1~?TA|DSI>Esn|UOpa@up*Q^0#M9zA$vidnL)sQLS&B+2F)(v zSP<8ny>a8h$cU~U(+9!VFG)Uqq>~GMJi4G_AG9kZCjP`l7v^XI_Y=&Z*}FLw#Pv+R zyR=BUb-GR+yqbKsVkBNNeDApggts zqEzhPSZV{IMGDY{)-=P^sly{y{0JRBGa(zUKECh9kgsTxTv9=DLf5X@0 z%>Z<2sZwj^3g%WvD|q8-Lk(WL!DAr!Hx3rc16dCvQ7`IhYe2IawNXINmzXI*2{8_7 zRaaQ47FK4$gI#V8)Pd9H6iu!~E1-DQS`B7&XhsnaNA%DoGhnG%#W}?MTLF(7IL4sO zk!fc*Sk*C9FrpOs!-zKVY$)LJpfN;Ai;H9gN(`>Y1mRSsOg5oQ-+pheh09OkA2B=_ z(#|KpzMm!>+mSgWH&uRl{sg4jzmIp_Zj4?q(wO#3 zl)hnNw}<^NUo?Jsec!=TiE}&~$FJY+@cFL)b!`aDuhvJ+o?GS8>CMn>Bg^daw8yXO zRQZE~Fz>hDS$=DST{`w?{PE$VZm%A@F)Mkc=b~f>m-u7{hq(9z*oTkzicd)Jjzh_f zh+(-1ge{!4GJ}9mI(qbQVUUt4A|@~5U{v;why;1Z>~*Tc?}<+UnG}7RiB+R%oG*7$t8+RASAV%#``ztd&d*+8i0`^-qU`RsAH8QqYqD^Sr<+vKVg#TqrcGDCtkZ%8W`` zxz8y<^4ez(CzMx%_Eao|%TF3Qx%zWRd#nnKbCpbAHFsYBTa&xI++XOB_;aAJHXuQm z^v>COhu~{b(wmX>A9aWrmYJ~r(l1xm|7Yp$T{`t&GZiOeM`-|{%4Q0 z%ecfTV_SyP+>&i+E>3ed1_f+OlWi_SX;3nXYrM$5=k_UH1f0Ax%e#m{9w8iCMYTcC zW7M4IXSn%vUD+?7*eQKmy3^d^ZE4P^=!bR%#3Yfp@MMlWxt{_Cd5q)ODynVa+pk5^ zt<#+rzGhQgi{!vZE40u@bS!TlM6%4qo}nE9MbB8~B9C4R950n_cyIrX2E0D;G-8Sy zL;=nSgumx@2&^*^#b_0axEK^M3khgh{P3WpzbYkbl$OiNu_chr)Cx#z;B{c^VU1Es zoOi`G5UK*t)N%093#LxJ9{8}2l)q??#(xIlHF2tRpbr8p~ZtpVN-HV4eL zsfym*l?k#kgqc#{H6VNu56fi0dVTfS37xL5uYg0G#KVA0D6+Kv1V1?H5cV29F9o*b zV8j+{#sICEwpa_j$rY=ZUSs7+3pHs@@eCZ992^NS7A!TdI49WNz%d4Gj!Zkl!Kz*k zwNucOp@6W`C_z^)o(_~W6)-qDM%JnCd_gkoN%`LIQ-{voq=U;(a#3>dEl9if>n86k zaocouuePhY({~57^VaQOCZu=N6m*{U_V@C!Ggd9lOd0v6!}pV7QU{M2Gx*>8?goB2 z;KESbuW3>}Um zZVHHR0-*0`?wb~%Z>mUqb3K#q;ucA_PB+VU@_>vM$+b=keWb_ohDIgU#z-#qw3sEy z8OvPcj?SNjH443hK6+yeUY~dxF~tqSQ$j+ts4Jwj{$dn7MZB1%ViuBR&3;IXw2ad@ zO@OSEtP{2bvYA=|X$`y%tS+&SgHUCKfh;_oXzJ9-Gj;eD1(uJ74IT{R4j-V?6dr_& z0^B$=37*1pQKN>3DCT7pFtdX-a8SiS4uZa6rd9^Q!}w;N4D~p9iW&xO!Wuvtn9~bX z)u+_D)1X<3+GV22ab|jk35Jtg(t?GhW3w{x=q5YP2M>S?teMlT0-Cev=cp+|R=-(m+$TmHBJ}=_yOyQ{yA9U++Vs55n z&d1Ro-g;M9D0$G>`$*ueS+mC}z$pES{DnRL3d&q7aB^h1?mViHZX6PZmk_fyQENnpr)r*n||c5ab$>vZQW@!43? zB02EU3N7?e9Lt*wkt}nu$Hmr2k_yXQM8&s7VYhs}7hiJp!Rr%GBS}Gq3Kxl$hST6l zL@`<nKhr8)Y>I(AeYLFEmOpQ4vxlm3)^Gub#{|DJFh6Gz$!w zZ(?ZayP!qVt*UZ9c&47!5Uh;HL6~$) z5AQRr0Qhi)*@-H)l=Noqxtdz1hZmvnk4>Ec*8VWS)Gw+E6)Yg4uRn!OPl%pbsyWUN z#UZukBnNR`FrWx7nh`3sLTo72>P_xNPw)}F6-87+cf=+0%TlvSa)|k|0vY@g9>|zm4*{w5v_C)kEW@Zg#@%Let1yQ zGO#5`gsfJ5{GE1W)z7b(!h-wu+(-8+Zi~< zpv{qKXE<2ZF~ACXGT=LP@oXsI@}Q>!CEamYLp@BozUNDV&@@DLv-*Z_Mf?G{{3NTY zeD6c*;v2H&8R56VFCM!$`nFSt-3POV4Y(`Jj2 z9}*)yQ8`?*j?(qg_1F^V##;et4ZIGlF0qe;P-R6eS)){G>eO+fFR~P=0TSqFaEFV^ zi7C8RqX0bT=90MxQ@X%I8?!PBSgX{*k(etONQzcoPO%CM z(U#aVCoHuc!*&LaF=%sS+8GY`z=xb2tg(Wg4EQEpOV5S^!bDF8O7)lFNus-Or6Vk7 zmoQy2DQlzijqM*mXD+NbaK8W&2S-itav}Qjm@cjZ4@-^i(!6g9M+<%GDtq>ReRW^S z)BhUzsC(7mQIe9&4#mHHJ7Zt(XYOxk`0pk7PW_&7a6K$%_tE~QL+cl6yywq=PZ^E5 zv960Dd1!Dxo@YGx>}TrxSECw*yNx-ai@r7f;^hD9?bWaU(p3GzS^4jNm(MmS2S1$c zG3x#4x652JWr_g1Jnivwd?>7cFy_KS_fIdsW|xjV8U_brR=0{_MR(%{=Lbl=ypyHw zCEhctzT107A5`puDmlbt0nw&=u!-S*I27T}u@rF7+;?z`q+6%EbIzKqkQT|c zP78g&ERC!aA(CZISxE~iT5Xvt-}lMSWjA{SZ`yj~G+v*04Pc5JR5(tobO1!pJX$#- z?oU%O3khg>k{=cN;6X`$bqSxT3s>NOXbE)Vt$?%!UI)rkTMJ*Mi?w2@4TKiKI^4AvlO+-4vux4nN~1Ce~L>wu&|_v_LiK>t&qAhYe;274Tv?P zh=yoO?ZqQ;XIwCkVLJoI7_>Pu^o;%KaImVQ0|h-9aHk-i4S2e)rl$iX_@vQM2$vq} z?I}oHJmmfLy#^;8aE8lIvU}g%97wwk+BxW+P*?PN$9>9&PJ7-uKW)pIZv|z%v&*bg zKi`p^=(lv|r6ZMkhq1e6SF0{Kcos}LTzdH3#=pj^U7q&%^+^BZ_qv#(5RLy=P3ifYoutYWiB$mASTRkeZvPoeX#^=93O4TP#F zYRNP86jP^;LuE_@XYl2u7UuCIBaJYH+hq#C0&^zGOjN62JSjH=Eee=xi-^f5SAv!8 zO-`AbN(Fto1||u>H8Qk1_+Zjv@KLpBLPi=P#VYRO8kY>@lYdeH@nFnM#VML;Gt%5A zS2CzH+#pNA!dV*Ha0Yr?fvY}e`EA3(Q}_2vP0hx`>5&8{@;vWe(!!otKaL;IDW(Rj7^7L9NrLE z_)T$Gbk)o~|LjpbZ`WmU@R2o7d;Gd4+*>nd&hJHUFWqxh9DHO=#~zJ8K63j<#jfrP zXXU>*$Tau}MiNcz4w_d!>|#AG+|n#6pQV&a-^9?;cZ@jr$TsQL>2}-V zI=V<4d}Ny(_-KU|`Y0sBbVysL19n)KLULsA5fswKSXo00#oX!eg=1mxQTXY@(-z?M zNe3T2lY}w^1_mEN_@{Ksh);+yj}{9C3^&4ZF$)Q3MUx*DV(>7eiQp3KZ1B-S7<>d9 zZv~_^@H$XtZ154*ss)4=iGz=3n>uy!q7+_q;yX({%%O!l_^_I#@am5OI&;Ykgg#df z3t5=7C}47c!)#E+%%=&0=@_|tre=VF{V?IK8kcc~yL-3=R$W!;;is=e?P}5F3Nx)? zg3cuAE^$yi7}}GXK}G11xC4&V+kon?SsB7$&gK*^gA(cyU1q>i^T;(P!F3p`#|<1~ z(B{a{+W+Zru&Sd2SR@1aE8v2mkyF5xLr(%qx*GCI^Js^e@M>tC#B1D_oyR8Qibul2 z5r?r;m!~+4ydaFf`H`yEH|M0q`Mo2<7uU^%j{AXeqW*oO@zBzynCMF-_#N;W`>|$b`1yZ2y)W3~X^)?y zd!K{DTo3sTdFJJFcInuoVYvHC^o9GAs%px%j>H$GMUr&^9(i$+bpyuadgM*WPDUv! z8O5nyWZ!dZ9w-4gd1urhqB9ks{9KM@moUBj*ZRqogGNZwwr2E^6nYd&*7}|N zXp^Jp70X=tmaboy{owqd@44f8ygu<7z!W#AFpF{#M9(~0EEt6}C}I{8(26EMD#YMH zNfW6iFG`OJN3kW)jkf~Q8h9Nj&-!}h;~-QSrny!^Yno%~)XAMX+)Khk6xiSf8+Y*3 zR7~OD2@0rqkqW*+=OB!Cs;#RBCrnEOtX1$mX>%pG5x0TCzmI9Sy&EDCxu6mY@NvjGzZ=;`S|Np*(Ysn@&i7lf00WkTN{`btLbgv(Fj z=lp~W($0sM7W5N7yHYrRc_%&D$KaI5Z`LW4>`rZ^|6yldp}GEli!~lIY+ZKh8lmE`CHY zDi)xMCow4|+J4L?k^AA$lyr{yMl(H=Z|4?Cw@x=glD#FaMRMSy64^x+x-W+-KCA>cIG-8SyRG0-c1EOaht#mOMVroi2%i@OzC7lws f1P4jIq#j!W-FPb?t%28pv5$Qmgeohv`q=*m+FCJY literal 0 HcmV?d00001 From 02cf009b03e8d8a9354b96998f538423976a9531 Mon Sep 17 00:00:00 2001 From: Mario Molina Date: Fri, 4 Sep 2020 18:50:36 -0500 Subject: [PATCH 09/21] Updating documentation --- docs/source/config_options.rst | 243 +++++++++++++++++++++++++++++++-- docs/source/filereaders.rst | 11 ++ 2 files changed, 245 insertions(+), 9 deletions(-) diff --git a/docs/source/config_options.rst b/docs/source/config_options.rst index 112fb43..5cc8600 100644 --- a/docs/source/config_options.rst +++ b/docs/source/config_options.rst @@ -299,7 +299,7 @@ In order to configure custom properties for this reader, the name you must use i * Default: ``false`` * Importance: medium -.. _config_options-filereaders-json: +.. _config_options-filereaders-sequencefile: SequenceFile -------------------------------------------- @@ -327,6 +327,231 @@ In order to configure custom properties for this reader, the name you must use i * Default: ``4096`` * Importance: low +.. _config_options-filereaders-cobol: + +Cobol +-------------------------------------------- + +In order to configure custom properties for this reader, the name you must use is ``cobol``. + +``file_reader.cobol.copybook.content`` + The content of the copybook. It is mandatory if property ``file_reader.cobol.copybook.path`` is not set. + + * Type: string + * Default: ``null`` + * Importance: high + +``file_reader.cobol.copybook.path`` + Copybook file path in the file system to be used. It is mandatory if property ``file_reader.cobol.copybook.content`` + is not set. + + * Type: string + * Default: ``null`` + * Importance: high + +``file_reader.cobol.reader.is_ebcdic`` + If the input data file encoding is EBCDIC, otherwise it is ASCII. + + * Type: boolean + * Default: ``true`` + * Importance: medium + +``file_reader.cobol.reader.ebcdic_code_page`` + Code page to be used for EBCDIC to ASCII/Unicode conversions. + + * Type: string + * Default: ``common`` + * Importance: medium + +``file_reader.cobol.reader.ebcdic_code_page_class`` + Custom code page conversion class provided. + + * Type: string + * Default: ``null`` + * Importance: low + +``file_reader.cobol.reader.ascii_charset`` + Charset for ASCII data. + + * Type: string + * Default: ```` + * Importance: low + +``file_reader.cobol.reader.is_uft16_big_endian`` + Flag to consider UTF-16 strings as big-endian. + + * Type: boolean + * Default: ``true`` + * Importance: low + +``file_reader.cobol.reader.floating_point_format`` + Format used for the floating-point numbers. + + * Type: enum (available values ``ibm``, ``ibm_little_endian``, ``ieee754``, and ``ieee754_little_endian``) + * Default: ``ibm`` + * Importance: medium + +``file_reader.cobol.reader.variable_size_occurs`` + If true, occurs depending on data size will depend on the number of elements. + + * Type: boolean + * Default: ``false`` + * Importance: low + +``file_reader.cobol.reader.length_field_name`` + The name for a field that contains the record length. If not set, the copybook record length will be used. + + * Type: string + * Default: ``null`` + * Importance: low + +``file_reader.cobol.reader.is_record_sequence`` + If the input file has 4 byte record length headers. + + * Type: boolean + * Default: ``false`` + * Importance: medium + +``file_reader.cobol.reader.is_rdw_big_endian`` + If the RDW is big endian. + + * Type: boolean + * Default: ``false`` + * Importance: low + +``file_reader.cobol.reader.is_rdw_part_rec_length`` + If the RDW count itself as part of record length itself. + + * Type: boolean + * Default: ``false`` + * Importance: low + +``file_reader.cobol.reader.rdw_adjustment`` + Controls a mismatch between RDW and record length. + + * Type: int + * Default: ``0`` + * Importance: low + +``file_reader.cobol.reader.is_index_generation_needed`` + If the indexing input file before processing is requested. + + * Type: boolean + * Default: ``false`` + * Importance: low + +``file_reader.cobol.reader.input_split_records`` + The number of records to include in each partition. + + * Type: int + * Default: ``null`` + * Importance: low + +``file_reader.cobol.reader.input_split_size_mb`` + A partition size to target. + + * Type: int + * Default: ``null`` + * Importance: low + +``file_reader.cobol.reader.hdfs_default_block_size`` + Default HDFS block size for the HDFS filesystem used. + + * Type: int + * Default: ``null`` + * Importance: low + +``file_reader.cobol.reader.start_offset`` + An offset to the start of the record in each binary data block. + + * Type: int + * Default: ``0`` + * Importance: medium + +``file_reader.cobol.reader.end_offset`` + An offset from the end of the record to the end of the binary data block. + + * Type: int + * Default: ``0`` + * Importance: medium + +``file_reader.cobol.reader.file_start_offset`` + A number of bytes to skip at the beginning of each file. + + * Type: int + * Default: ``0`` + * Importance: medium + +``file_reader.cobol.reader.file_end_offset`` + A number of bytes to skip at the end of each file. + + * Type: int + * Default: ``0`` + * Importance: medium + +``file_reader.cobol.reader.schema_policy`` + Specifies a policy to transform the input schema. + + * Type: enum (available values ``keep_original`` and ``collapse_root``) + * Default: ``keep_original`` + * Importance: medium + +``file_reader.cobol.reader.string_trimming_policy`` + The trim to apply for records with string data types. + + * Type: enum (available values ``both``, ``left``, ``right`` and ``none``) + * Default: ``both`` + * Importance: medium + +``file_reader.cobol.reader.drop_group_fillers`` + If true the parser will drop all FILLER fields, even GROUP FILLERS that have non-FILLER nested fields. + + * Type: boolean + * Default: ``false`` + * Importance: low + +``file_reader.cobol.reader.drop_value_fillers`` + If true the parser will drop all value FILLER fields. + + * Type: boolean + * Default: ``true`` + * Importance: low + +``file_reader.cobol.reader.non_terminals`` + A comma-separated list of group-type fields to combine and parse as primitive fields. + + * Type: string[] + * Default: ``null`` + * Importance: low + +``file_reader.cobol.reader.debug_fields_policy`` + Specifies if debugging fields need to be added and what should they contain. + + * Type: enum (available values ``hex``, ``raw`` and ``none``) + * Default: ``none`` + * Importance: low + +``file_reader.cobol.reader.record_header_parser`` + Parser to be used to parse data field record headers. + + * Type: string + * Default: ``null`` + * Importance: low + +``file_reader.cobol.reader.rhp_additional_info`` + Extra option to be passed to a custom record header parser. + + * Type: string + * Default: ``null`` + * Importance: low + +``file_reader.cobol.reader.input_file_name_column`` + A column name to add to each record containing the input file name. + + * Type: string + * Default: ```` + * Importance: low + .. _config_options-filereaders-json: JSON @@ -920,56 +1145,56 @@ To configure custom properties for this reader, the name you must use is ``agnos ``file_reader.agnostic.extensions.parquet`` A comma-separated string list with the accepted extensions for Parquet files. - * Type: string + * Type: string[] * Default: ``parquet`` * Importance: medium ``file_reader.agnostic.extensions.avro`` A comma-separated string list with the accepted extensions for Avro files. - * Type: string + * Type: string[] * Default: ``avro`` * Importance: medium ``file_reader.agnostic.extensions.orc`` A comma-separated string list with the accepted extensions for ORC files. - * Type: string + * Type: string[] * Default: ``orc`` * Importance: medium ``file_reader.agnostic.extensions.sequence`` A comma-separated string list with the accepted extensions for Sequence files. - * Type: string + * Type: string[] * Default: ``seq`` * Importance: medium ``file_reader.agnostic.extensions.json`` A comma-separated string list with the accepted extensions for JSON files. - * Type: string + * Type: string[] * Default: ``json`` * Importance: medium ``file_reader.agnostic.extensions.csv`` A comma-separated string list with the accepted extensions for CSV files. - * Type: string + * Type: string[] * Default: ``csv`` * Importance: medium ``file_reader.agnostic.extensions.tsv`` A comma-separated string list with the accepted extensions for TSV files. - * Type: string + * Type: string[] * Default: ``tsv`` * Importance: medium ``file_reader.agnostic.extensions.fixed`` A comma-separated string list with the accepted extensions for fixed-width files. - * Type: string + * Type: string[] * Default: ``fixed`` * Importance: medium diff --git a/docs/source/filereaders.rst b/docs/source/filereaders.rst index d38a5e9..c5b1153 100644 --- a/docs/source/filereaders.rst +++ b/docs/source/filereaders.rst @@ -54,6 +54,17 @@ by default but you can customize these field names. More information about properties of this file reader :ref:`here`. +Cobol +^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ + +Mainframe files (Cobol/EBCDIC binary files) can be processed with this reader which uses the +`Cobrix `__ parser. + +By means of the corresponding copybook -representing its schema-, it parses each record and +translate it into a Kafka message with the schema. + +More information about properties of this file reader :ref:`here`. + JSON ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ From b34441143dde50e836a32c2c3250c03189805de4 Mon Sep 17 00:00:00 2001 From: Mario Molina Date: Sat, 5 Sep 2020 17:27:31 -0500 Subject: [PATCH 10/21] Agnostic file reader for Cobol files --- docs/source/config_options.rst | 7 +++++++ .../fs/file/reader/AgnosticFileReader.java | 7 ++++++- .../file/reader/AgnosticFileReaderTest.java | 21 +++++++++++++++++++ 3 files changed, 34 insertions(+), 1 deletion(-) diff --git a/docs/source/config_options.rst b/docs/source/config_options.rst index 5cc8600..9ea840f 100644 --- a/docs/source/config_options.rst +++ b/docs/source/config_options.rst @@ -1170,6 +1170,13 @@ To configure custom properties for this reader, the name you must use is ``agnos * Default: ``seq`` * Importance: medium +``file_reader.agnostic.extensions.cobol`` + A comma-separated string list with the accepted extensions for Cobol files. + + * Type: string[] + * Default: ``dat`` + * Importance: medium + ``file_reader.agnostic.extensions.json`` A comma-separated string list with the accepted extensions for JSON files. diff --git a/src/main/java/com/github/mmolimar/kafka/connect/fs/file/reader/AgnosticFileReader.java b/src/main/java/com/github/mmolimar/kafka/connect/fs/file/reader/AgnosticFileReader.java index 2d0dbe5..0158a4a 100644 --- a/src/main/java/com/github/mmolimar/kafka/connect/fs/file/reader/AgnosticFileReader.java +++ b/src/main/java/com/github/mmolimar/kafka/connect/fs/file/reader/AgnosticFileReader.java @@ -23,6 +23,7 @@ public class AgnosticFileReader extends AbstractFileReader reader; - private Set parquetExtensions, avroExtensions, sequenceExtensions, orcExtensions, + private Set parquetExtensions, avroExtensions, sequenceExtensions, orcExtensions, cobolExtensions, jsonExtensions, csvExtensions, tsvExtensions, fixedExtensions; public AgnosticFileReader(FileSystem fs, Path filePath, Map config) throws Exception { @@ -57,6 +58,8 @@ private AbstractFileReader readerByExtension(FileSystem fs, Path filePat clz = SequenceFileReader.class; } else if (orcExtensions.contains(extension)) { clz = OrcFileReader.class; + } else if (cobolExtensions.contains(extension)) { + clz = CobolFileReader.class; } else if (jsonExtensions.contains(extension)) { clz = JsonFileReader.class; } else if (csvExtensions.contains(extension)) { @@ -82,6 +85,8 @@ protected void configure(Map config) { .toLowerCase().split(",")).collect(Collectors.toSet()); this.orcExtensions = Arrays.stream(config.getOrDefault(FILE_READER_AGNOSTIC_EXTENSIONS_ORC, "orc") .toLowerCase().split(",")).collect(Collectors.toSet()); + this.cobolExtensions = Arrays.stream(config.getOrDefault(FILE_READER_AGNOSTIC_EXTENSIONS_COBOL, "dat") + .toLowerCase().split(",")).collect(Collectors.toSet()); this.jsonExtensions = Arrays.stream(config.getOrDefault(FILE_READER_AGNOSTIC_EXTENSIONS_JSON, "json") .toLowerCase().split(",")).collect(Collectors.toSet()); this.csvExtensions = Arrays.stream(config.getOrDefault(FILE_READER_AGNOSTIC_EXTENSIONS_CSV, "csv") diff --git a/src/test/java/com/github/mmolimar/kafka/connect/fs/file/reader/AgnosticFileReaderTest.java b/src/test/java/com/github/mmolimar/kafka/connect/fs/file/reader/AgnosticFileReaderTest.java index 3c01e86..12dbd38 100644 --- a/src/test/java/com/github/mmolimar/kafka/connect/fs/file/reader/AgnosticFileReaderTest.java +++ b/src/test/java/com/github/mmolimar/kafka/connect/fs/file/reader/AgnosticFileReaderTest.java @@ -202,4 +202,25 @@ public String getFileExtension() { } } + @Nested + class AgnosticCobolFileReaderTest extends CobolFileReaderTest { + + @Override + protected Map getReaderConfig() { + Map config = super.getReaderConfig(); + config.put(AgnosticFileReader.FILE_READER_AGNOSTIC_EXTENSIONS_COBOL, getFileExtension()); + return config; + } + + @Override + public Class getReaderClass() { + return AgnosticFileReader.class; + } + + @Override + public String getFileExtension() { + return "dt"; + } + } + } From 57e0bcdfc85cd0a338b8e1a75cf3667c0c5c7578 Mon Sep 17 00:00:00 2001 From: Mario Molina Date: Sun, 6 Sep 2020 12:32:51 -0500 Subject: [PATCH 11/21] Upgrade to AK 2.6.0 and deps --- pom.xml | 20 ++++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/pom.xml b/pom.xml index 3dd30ee..49c97af 100644 --- a/pom.xml +++ b/pom.xml @@ -46,17 +46,17 @@ UTF-8 - 2.5.0 - 5.5.0 - 3.2.1 - hadoop3-2.1.3 - 1.11.0 + 2.6.0 + 5.5.1 + 3.3.0 + hadoop3-2.1.4 + 1.11.1 1.6.3 - 2.8.4 + 2.9.0 2.1.1 - 2.11.12 - 9.0.2 - 0.1.54 + 2.12.12 + 9.1.1 + 0.1.55 5.6.2 4.2 2.0.7 @@ -140,7 +140,7 @@ ${jsch.version} - + org.junit.jupiter junit-jupiter From e74f57b72e1b3b4c395a843aa75b4ecf239c8e46 Mon Sep 17 00:00:00 2001 From: Mario Molina Date: Sun, 6 Sep 2020 21:17:29 -0500 Subject: [PATCH 12/21] Array support in Cobol files --- .../fs/file/reader/CobolFileReader.java | 31 +++++++++++++----- .../fs/file/reader/CobolFileReaderTest.java | 9 ++++- .../file/reader/data/cobol/companies.cpy | 5 +++ .../file/reader/data/cobol/companies.dt | Bin 6800 -> 1606800 bytes 4 files changed, 36 insertions(+), 9 deletions(-) diff --git a/src/main/java/com/github/mmolimar/kafka/connect/fs/file/reader/CobolFileReader.java b/src/main/java/com/github/mmolimar/kafka/connect/fs/file/reader/CobolFileReader.java index a32cde3..069caf6 100644 --- a/src/main/java/com/github/mmolimar/kafka/connect/fs/file/reader/CobolFileReader.java +++ b/src/main/java/com/github/mmolimar/kafka/connect/fs/file/reader/CobolFileReader.java @@ -35,6 +35,7 @@ import java.io.InputStream; import java.io.InputStreamReader; import java.util.*; +import java.util.stream.Collectors; import java.util.stream.IntStream; import static com.github.mmolimar.kafka.connect.fs.FsSourceTaskConfig.FILE_READER_PREFIX; @@ -122,9 +123,14 @@ private Schema extractSchema(CobolSchema cobolSchema) { private Schema schemaForField(Statement statement) { if (statement instanceof Group) { Group group = (Group) statement; - SchemaBuilder builder = SchemaBuilder.struct(); - seqAsJavaList(group.children()).forEach(child -> builder.field(child.name(), schemaForField(child))); - + SchemaBuilder childrenBuilder = SchemaBuilder.struct(); + seqAsJavaList(group.children()).forEach(child -> childrenBuilder.field(child.name(), schemaForField(child))); + SchemaBuilder builder; + if (group.isArray()) { + builder = SchemaBuilder.array(childrenBuilder.build()); + } else { + builder = childrenBuilder; + } return builder.build(); } Primitive primitive = (Primitive) statement; @@ -313,18 +319,25 @@ public Struct apply(CobolRecord record) { .filter(col -> col instanceof Map) .forEach(col -> { Map column = (Map) col; - column.forEach((k, v) -> struct.put(k, mapValue(record.schema.field(k).schema(), v))); + column.forEach((k, v) -> struct.put(k, mapValue(record.schema.field(k).schema(), k, v))); }); return struct; } - private Object mapValue(Schema schema, Object value) { - if (schema.type() != Schema.Type.STRUCT || value == null) { + private Object mapValue(Schema schema, String fieldName, Object value) { + if (value == null) { + return null; + } else if (schema.type() == Schema.Type.ARRAY) { + List items = (List) value; + return items.stream() + .map(item -> mapValue(schema.valueSchema(), fieldName, ((Map) item).get(fieldName))) + .collect(Collectors.toList()); + } else if (schema.type() != Schema.Type.STRUCT) { return value; } Struct struct = new Struct(schema); Map map = (Map) value; - map.forEach((k, v) -> struct.put(k, mapValue(schema.field(k).schema(), v))); + map.forEach((k, v) -> struct.put(k, mapValue(schema.field(k).schema(), k, v))); return struct; } } @@ -363,8 +376,10 @@ private Map mapValues(Group group, Object[] values) { private Map.Entry transform(Statement child, Object value) { Object childValue; - if (child instanceof Group) { + if (child instanceof Group && value instanceof Map) { childValue = ((Map) value).get(child.name()); + } else if (value instanceof Object[]) { + childValue = Arrays.asList((Object[]) value); } else if (value instanceof ScalaNumber) { childValue = value instanceof scala.math.BigDecimal ? ((BigDecimal) value).doubleValue() : ((ScalaNumber) value).longValue(); diff --git a/src/test/java/com/github/mmolimar/kafka/connect/fs/file/reader/CobolFileReaderTest.java b/src/test/java/com/github/mmolimar/kafka/connect/fs/file/reader/CobolFileReaderTest.java index cb93495..ec55a81 100644 --- a/src/test/java/com/github/mmolimar/kafka/connect/fs/file/reader/CobolFileReaderTest.java +++ b/src/test/java/com/github/mmolimar/kafka/connect/fs/file/reader/CobolFileReaderTest.java @@ -13,6 +13,7 @@ import java.nio.file.Paths; import java.nio.file.StandardCopyOption; import java.util.HashMap; +import java.util.List; import java.util.Map; import java.util.UUID; import java.util.stream.IntStream; @@ -241,6 +242,8 @@ protected void checkData(Struct record, long index) { Struct companyDetails = record.getStruct("COMPANY_DETAILS"); Struct staticDetails = companyDetails.getStruct("STATIC_DETAILS"); Struct taxpayer = staticDetails.getStruct("TAXPAYER"); + Struct strategy = staticDetails.getStruct("STRATEGY"); + List strategyDetails = strategy.getArray("STRATEGY_DETAIL"); assertAll( () -> assertEquals("C", companyDetails.getString("SEGMENT_ID")), () -> assertEquals(String.format("%010d", index), companyDetails.getString("COMPANY_ID")), @@ -250,7 +253,11 @@ protected void checkData(Struct record, long index) { () -> assertEquals("A", taxpayer.getString("TAXPAYER_TYPE")), () -> assertEquals("88888888", taxpayer.getString("TAXPAYER_STR")), - () -> assertNull(taxpayer.getInt32("TAXPAYER_NUM")) + () -> assertNull(taxpayer.getInt32("TAXPAYER_NUM")), + + () -> assertEquals(6, strategyDetails.size()), + () -> assertEquals(1111111, (strategyDetails.get(0)).getInt32("NUM1")), + () -> assertEquals(2222222, (strategyDetails.get(0)).getInt32("NUM2")) ); } } diff --git a/src/test/resources/file/reader/data/cobol/companies.cpy b/src/test/resources/file/reader/data/cobol/companies.cpy index bf237d0..9b627f0 100644 --- a/src/test/resources/file/reader/data/cobol/companies.cpy +++ b/src/test/resources/file/reader/data/cobol/companies.cpy @@ -9,3 +9,8 @@ 15 TAXPAYER-STR PIC X(8). 15 TAXPAYER-NUM REDEFINES TAXPAYER-STR PIC 9(8) COMP. + 10 STRATEGY. + 15 STRATEGY_DETAIL OCCURS 6. + 25 NUM1 PIC 9(7) COMP. + 25 NUM2 PIC 9(7) COMP-3. + diff --git a/src/test/resources/file/reader/data/cobol/companies.dt b/src/test/resources/file/reader/data/cobol/companies.dt index c31ef0e8a4b3601ab22781eff4dc7aeb904a8460..ef1397d9bfa4e96db9e189bf8ed653df41c7dde5 100644 GIT binary patch literal 1606800 zcmeI(t*xeI6ouha0yT`L267w%LmyNz$xH>qK+uFfn9u3qIiUO)Z%`u3;y-@N|$ z-N$$DfB5kB1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+ z009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009ES0@<)%HSELJwi*Ef z1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs z0RjXF5FkK+009C7x&qm--!$x9aMKA8AV7cs0RjXF5FkK+009C72oNAZfB*pk1PBly zK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyFf5P_`(496d~K@{AV7cs z0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZ zfB*pk1PBlyK%gs-4f{jG-UT8ul)@=>!N6AV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&U zAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlq7RZMEs$n0#w$%s_AV7cs0RjXF5FkK+ z009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBly z&=tsr{ib2>f}2i&009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C7 z2oNAZfB*pk1PBlyK!5-N0t5&UAV7csfnkAc*zX$l;cHus009C72oNAZfB*pk1PBly zK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0Rmlt zY}g+f_Aa>T1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N z0t5&UAV7cs0RjXF5FkK+009C72oM++$cFu?VIRJ>)d&zEK!5-N0t5&UAV7cs0RjXF z5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNC96^MrY zbkMMO!A&PXfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBly zK!5-N0t5&UAV7cs0RjXF5FkK+z_36z>_-jz@U^W*fB*pk1PBlyK!5-N0t5&UAV7cs z0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+0D-PRHtZ)2 zdl%ev0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF z5FkK+009C72oNAZfB*pk1PBZZWW#>eun%9`Y6J)nAV7cs0RjXF5FkK+009C72oNAZ zfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBo53S`56(Xe;H zO(#Hr009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk z1PBlyK!5-N0t5&UAV7e?us}BKR}K5{wXH^g009C72oNAZfB*pk1PBlyK!5-N0t5&U zAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7csfv!L{>^BX27u<9L z1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs z0RjXF5FkK+009C72n-8k!+zJW4`17A1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C7 z2oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oUHBWW)Z@uy?^tCqRGz z0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZ zfB*pk1PBlyK!CuoKsM}84g2u5tww+V0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N z0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5;&u0S;G=Yxj53vN0A0t5&U zAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C7 z2oNAZfB*pk1cn8&VLxivhp%ll0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBly zK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PF8mvSB}I*t_7S6Cgl<009C7 z2oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N z0t5&UAV6SPARG3xhJE1PBlyK!5-N0t5&UAV7cs z0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009ES0@<*i zHSELJwi*Ef1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N z0t5&UAV7cs0RjXF5FkK+009C7x&qm-Uo`ApaMKA8AV7cs0RjXF5FkK+009C72oNAZ zfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyFf5P_`&Gj} zd~K@{AV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+ z009C72oNAZfB*pk1PBlyK%gs-4f{>Q-UT!N6AV7cs0RjXF5FkK+009C72oNAZfB*pk1PBly zK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlq7RZMEtYII%w$%s_AV7cs z0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZ zfB*pk1PBly&=tsr{i0#-f}2i&009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF z5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7csfnkAc*smJ);cHus009C72oNAZ zfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&U zAV7cs0RmltY}jua_Aa>T1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk z1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oM++$cFu{VIRJ>)d&zEK!5-N0t5&U zAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C7 z2oNC9708DDp<(ZWn@)fL0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs z0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5;&VS#MepBnbzYg>%~0RjXF5FkK+009C7 z2oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N z0$qV<*l!07dl%ev0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&U zAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBZZWW#>cun%9`Y6J)nAV7cs0RjXF5FkK+ z009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBo5 z3S`56(y({IO(#Hr009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C7 z2oNAZfB*pk1PBlyK!5-N0t5&UAV7e?us}BKXAS%CwXH^g009C72oNAZfB*pk1PBly zK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7csfv!L{ z>=zAt7u<9L1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N z0t5&UAV7cs0RjXF5FkK+009C72n-8k!+zDU4`17A1PBlyK!5-N0t5&UAV7cs0RjXF z5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oUHBWW#>b zuy?^tCqRGz0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+ z009C72oNAZfB*pk1PBlyK!CuoKsM}m4g2u5tww+V0RjXF5FkK+009C72oNAZfB*pk z1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5;&u0S^I4-I=4 z+;jp22oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBly zK!5-N0t5&UAV7cs0RjXF3=3q#{?xD!U)yQ~2oNAZfB*pk1PBlyK!5-N0t5&UAV7cs z0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5a1PBlyK!5-N z0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+ z009ES0@<)1HSELJwi*Ef1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk z1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C7x&qm-pET@UaMKA8AV7cs0RjXF5FkK+ z009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBly zFf5P_`&q+2d~K@{AV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs z0RjXF5FkK+009C72oNAZfB*pk1PBlyK%gs-4f{pI-UTc8?us=2I z!`HSN0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C7 z2oNAZfB*pk1PBlyK!5-N0tC7O(Xc-c8ul)@=>!N6AV7cs0RjXF5FkK+009C72oNAZ zfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlq7RZMEs9_(z zw$%s_AV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+ z009C72oNAZfB*pk1PBly&=tsr{iI>T1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C7 z2oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oM++$cFu@VIRJ>)d&zE zK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF z5FkK+009C72oNC9708DDreW`bn@)fL0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N z0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5;&VS#Me?;7^uYg>%~0RjXF z5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk z1PBlyK!5-N0$qV@*dH49F1YCg2oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+ z009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5EvH7hW)8wAHKHL2oNAZfB*pk z1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs N0RjXF5FpSL_zM>#(s=*? literal 6800 zcmb`_txW_`7(me}Z9IlPsA7`Y3YLYS359Iv01O&|-uJzK2ZCTwgmVkH>MYB;JTG5e zetO?-?~jM|?fSgFKJHf$IhV4YU)JsEaI=2go%X}SaXanj@!j(4k-#IsBZEf)j|v_Q zJUVy`@Ra2mjA0H*<* z25_2z(+r$u;4}lL892?rX$DR+aGHVB44f9=v;e0CI4!_w0Zt2WT7c66oEG4;0H+l= zt-xsoPAhO)fzt|{R^YS(rxiG@z-a?c8*tiy(*~S2;Isj!4LEJUX#-9haN2>>4xD!2 zv;(IdIPJh`2TnV1+JVy!oDSf00H*^u9l+@TP6u#0fYSk-4&ZbErxQ4x!07}|CvZA} S(+QkT;B*3~6F8lJclrqy7UW?7 From cdaa1edb528728c5151f49ce96887662eaf7b1d5 Mon Sep 17 00:00:00 2001 From: Mario Molina Date: Tue, 8 Sep 2020 20:01:30 -0500 Subject: [PATCH 13/21] Disable record id generation --- .../java/com/github/mmolimar/kafka/connect/fs/FsSourceTask.java | 2 +- .../mmolimar/kafka/connect/fs/file/reader/CobolFileReader.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/main/java/com/github/mmolimar/kafka/connect/fs/FsSourceTask.java b/src/main/java/com/github/mmolimar/kafka/connect/fs/FsSourceTask.java index 5a1ce50..979695a 100644 --- a/src/main/java/com/github/mmolimar/kafka/connect/fs/FsSourceTask.java +++ b/src/main/java/com/github/mmolimar/kafka/connect/fs/FsSourceTask.java @@ -97,7 +97,7 @@ public List poll() { ((AbstractFileReader) reader).hasNextBatch() || reader.hasNext() : reader.hasNext(); records.add(convert(metadata, reader.currentOffset(), !hasNext, record)); } - } catch (ConnectException | IOException e) { + } catch (IOException | ConnectException e) { // when an exception happens reading a file, the connector continues log.warn("{} Error reading file [{}]: {}. Keep going...", this, metadata.getPath(), e.getMessage(), e); diff --git a/src/main/java/com/github/mmolimar/kafka/connect/fs/file/reader/CobolFileReader.java b/src/main/java/com/github/mmolimar/kafka/connect/fs/file/reader/CobolFileReader.java index 069caf6..0c835df 100644 --- a/src/main/java/com/github/mmolimar/kafka/connect/fs/file/reader/CobolFileReader.java +++ b/src/main/java/com/github/mmolimar/kafka/connect/fs/file/reader/CobolFileReader.java @@ -207,7 +207,7 @@ private ReaderParameters getReaderParameters(Map config) { Integer.parseInt(config.getOrDefault(FILE_READER_COBOL_READER_END_OFFSET, "0")), // endOffset Integer.parseInt(config.getOrDefault(FILE_READER_COBOL_READER_FILE_START_OFFSET, "0")), // fileStartOffset Integer.parseInt(config.getOrDefault(FILE_READER_COBOL_READER_FILE_END_OFFSET, "0")), // fileEndOffset - true, // generateRecordId + false, // generateRecordId SchemaRetentionPolicy$.MODULE$.withNameOpt(config.getOrDefault(FILE_READER_COBOL_READER_SCHEMA_POLICY, "keep_original")).get(), // schemaPolicy StringTrimmingPolicy$.MODULE$.withNameOpt(config.getOrDefault(FILE_READER_COBOL_READER_STRING_TRIMMING_POLICY, "both")).get(), // stringTrimmingPolicy scala.Option.apply(null), // multisegment From aa837c7c68d5ac19024ce0b52f56a8c35d79d2be Mon Sep 17 00:00:00 2001 From: Mario Molina Date: Fri, 11 Sep 2020 20:42:23 -0500 Subject: [PATCH 14/21] New XML and YAML file readers --- pom.xml | 13 + .../fs/file/reader/JacksonFileReader.java | 219 +++++++++++++++++ .../fs/file/reader/JsonFileReader.java | 194 ++------------- .../connect/fs/file/reader/XmlFileReader.java | 58 +++++ .../fs/file/reader/YamlFileReader.java | 57 +++++ .../fs/file/reader/JacksonFileReaderTest.java | 210 ++++++++++++++++ .../fs/file/reader/JsonFileReaderTest.java | 228 +++--------------- .../fs/file/reader/XmlFileReaderTest.java | 93 +++++++ .../fs/file/reader/YamlFileReaderTest.java | 188 +++++++++++++++ 9 files changed, 893 insertions(+), 367 deletions(-) create mode 100644 src/main/java/com/github/mmolimar/kafka/connect/fs/file/reader/JacksonFileReader.java create mode 100644 src/main/java/com/github/mmolimar/kafka/connect/fs/file/reader/XmlFileReader.java create mode 100644 src/main/java/com/github/mmolimar/kafka/connect/fs/file/reader/YamlFileReader.java create mode 100644 src/test/java/com/github/mmolimar/kafka/connect/fs/file/reader/JacksonFileReaderTest.java create mode 100644 src/test/java/com/github/mmolimar/kafka/connect/fs/file/reader/XmlFileReaderTest.java create mode 100644 src/test/java/com/github/mmolimar/kafka/connect/fs/file/reader/YamlFileReaderTest.java diff --git a/pom.xml b/pom.xml index 49c97af..665c837 100644 --- a/pom.xml +++ b/pom.xml @@ -53,6 +53,7 @@ 1.11.1 1.6.3 2.9.0 + 2.10.2 2.1.1 2.12.12 9.1.1 @@ -124,6 +125,16 @@ univocity-parsers ${univocity.version} + + com.fasterxml.jackson.dataformat + jackson-dataformat-xml + ${jackson-dataformat.version} + + + com.fasterxml.jackson.dataformat + jackson-dataformat-yaml + ${jackson-dataformat.version} + za.co.absa.cobrix cobol-parser_2.12 @@ -315,6 +326,8 @@ csv tsv json + xml + yaml avro parquet orc diff --git a/src/main/java/com/github/mmolimar/kafka/connect/fs/file/reader/JacksonFileReader.java b/src/main/java/com/github/mmolimar/kafka/connect/fs/file/reader/JacksonFileReader.java new file mode 100644 index 0000000..8354efe --- /dev/null +++ b/src/main/java/com/github/mmolimar/kafka/connect/fs/file/reader/JacksonFileReader.java @@ -0,0 +1,219 @@ +package com.github.mmolimar.kafka.connect.fs.file.reader; + +import com.fasterxml.jackson.databind.DeserializationFeature; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.SchemaBuilder; +import org.apache.kafka.connect.data.Struct; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.StreamSupport; + +abstract class JacksonFileReader extends AbstractFileReader { + + private final TextFileReader inner; + private final Schema schema; + private ObjectMapper mapper; + + public JacksonFileReader(FileSystem fs, Path filePath, Map config) throws IOException { + super(fs, filePath, new JacksonToStruct(), config); + + config.put(TextFileReader.FILE_READER_TEXT_ENCODING, readerEncodingConfig(config)); + config.put(TextFileReader.FILE_READER_TEXT_RECORD_PER_LINE, recordPerLineConfig(config)); + config.put(TextFileReader.FILE_READER_TEXT_COMPRESSION_TYPE, compressionTypeConfig(config)); + config.put(TextFileReader.FILE_READER_TEXT_COMPRESSION_CONCATENATED, compressionConcatenatedConfig(config)); + + this.inner = new TextFileReader(fs, filePath, config); + + if (hasNext()) { + String line = inner.nextRecord().getValue(); + this.schema = extractSchema(mapper.readTree(line)); + // back to the first line + inner.seek(0); + } else { + this.schema = SchemaBuilder.struct().build(); + } + } + + protected abstract Object readerEncodingConfig(Map config); + + protected abstract Object recordPerLineConfig(Map config); + + protected abstract Object compressionTypeConfig(Map config); + + protected abstract Object compressionConcatenatedConfig(Map config); + + protected abstract String deserializationConfigPrefix(); + + protected abstract ObjectMapper getObjectMapper(); + + @Override + protected void configure(Map config) { + mapper = getObjectMapper(); + Set deserializationFeatures = Arrays.stream(DeserializationFeature.values()) + .map(Enum::name) + .collect(Collectors.toSet()); + config.entrySet().stream() + .filter(entry -> entry.getKey().startsWith(deserializationConfigPrefix())) + .forEach(entry -> { + String feature = entry.getKey().replaceAll(deserializationConfigPrefix(), ""); + if (deserializationFeatures.contains(feature)) { + mapper.configure(DeserializationFeature.valueOf(feature), + Boolean.parseBoolean(entry.getValue())); + } else { + log.warn("{} Ignoring deserialization configuration [{}] due to it does not exist.", + this, feature); + } + }); + } + + @Override + protected JacksonRecord nextRecord() throws IOException { + JsonNode value = mapper.readTree(inner.nextRecord().getValue()); + return new JacksonRecord(schema, value); + } + + @Override + public boolean hasNextRecord() throws IOException { + return inner.hasNextRecord(); + } + + @Override + public void seekFile(long offset) throws IOException { + inner.seekFile(offset); + } + + @Override + public long currentOffset() { + return inner.currentOffset(); + } + + @Override + public void close() throws IOException { + inner.close(); + } + + @Override + public boolean isClosed() { + return inner.isClosed(); + } + + private static Schema extractSchema(JsonNode jsonNode) { + switch (jsonNode.getNodeType()) { + case BOOLEAN: + return Schema.OPTIONAL_BOOLEAN_SCHEMA; + case NUMBER: + if (jsonNode.isShort()) { + return Schema.OPTIONAL_INT8_SCHEMA; + } else if (jsonNode.isInt()) { + return Schema.OPTIONAL_INT32_SCHEMA; + } else if (jsonNode.isLong()) { + return Schema.OPTIONAL_INT64_SCHEMA; + } else if (jsonNode.isBigInteger()) { + return Schema.OPTIONAL_INT64_SCHEMA; + } else { + return Schema.OPTIONAL_FLOAT64_SCHEMA; + } + case STRING: + return Schema.OPTIONAL_STRING_SCHEMA; + case BINARY: + return Schema.OPTIONAL_BYTES_SCHEMA; + case ARRAY: + Iterable elements = jsonNode::elements; + Schema arraySchema = StreamSupport.stream(elements.spliterator(), false) + .findFirst().map(JacksonFileReader::extractSchema) + .orElse(SchemaBuilder.struct().build()); + return SchemaBuilder.array(arraySchema).build(); + case OBJECT: + SchemaBuilder builder = SchemaBuilder.struct(); + jsonNode.fields() + .forEachRemaining(field -> builder.field(field.getKey(), extractSchema(field.getValue()))); + return builder.build(); + default: + return SchemaBuilder.struct().optional().build(); + } + } + + static class JacksonToStruct implements ReaderAdapter { + + @Override + public Struct apply(JacksonRecord record) { + return toStruct(record.schema, record.value); + } + + private Struct toStruct(Schema schema, JsonNode jsonNode) { + if (jsonNode.isNull()) return null; + Struct struct = new Struct(schema); + jsonNode.fields() + .forEachRemaining(field -> struct.put( + field.getKey(), + mapValue(struct.schema().field(field.getKey()).schema(), field.getValue()) + )); + return struct; + } + + private Object mapValue(Schema schema, JsonNode value) { + if (value == null) return null; + + switch (value.getNodeType()) { + case BOOLEAN: + return value.booleanValue(); + case NUMBER: + if (value.isShort()) { + return value.shortValue(); + } else if (value.isInt()) { + return value.intValue(); + } else if (value.isLong()) { + return value.longValue(); + } else if (value.isBigInteger()) { + return value.bigIntegerValue().longValue(); + } else { + return value.numberValue().doubleValue(); + } + case STRING: + return value.asText(); + case BINARY: + try { + return value.binaryValue(); + } catch (IOException ioe) { + throw new RuntimeException(ioe); + } + case OBJECT: + case POJO: + Struct struct = new Struct(schema); + Iterable> fields = value::fields; + StreamSupport.stream(fields.spliterator(), false) + .forEach(field -> struct.put(field.getKey(), + mapValue(extractSchema(field.getValue()), field.getValue())) + ); + return struct; + case ARRAY: + Iterable arrayElements = value::elements; + return StreamSupport.stream(arrayElements.spliterator(), false) + .map(elm -> mapValue(schema.valueSchema(), elm)) + .collect(Collectors.toList()); + case NULL: + case MISSING: + default: + return null; + } + } + } + + static class JacksonRecord { + private final Schema schema; + private final JsonNode value; + + JacksonRecord(Schema schema, JsonNode value) { + this.schema = schema; + this.value = value; + } + } +} diff --git a/src/main/java/com/github/mmolimar/kafka/connect/fs/file/reader/JsonFileReader.java b/src/main/java/com/github/mmolimar/kafka/connect/fs/file/reader/JsonFileReader.java index f1440b0..530a1a4 100644 --- a/src/main/java/com/github/mmolimar/kafka/connect/fs/file/reader/JsonFileReader.java +++ b/src/main/java/com/github/mmolimar/kafka/connect/fs/file/reader/JsonFileReader.java @@ -1,219 +1,57 @@ package com.github.mmolimar.kafka.connect.fs.file.reader; -import com.fasterxml.jackson.databind.DeserializationFeature; -import com.fasterxml.jackson.databind.JsonNode; import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.kafka.connect.data.Schema; -import org.apache.kafka.connect.data.SchemaBuilder; -import org.apache.kafka.connect.data.Struct; import java.io.IOException; -import java.util.Arrays; import java.util.Map; -import java.util.Set; -import java.util.stream.Collectors; -import java.util.stream.StreamSupport; import static com.github.mmolimar.kafka.connect.fs.FsSourceTaskConfig.FILE_READER_PREFIX; -public class JsonFileReader extends AbstractFileReader { +public class JsonFileReader extends JacksonFileReader { private static final String FILE_READER_JSON = FILE_READER_PREFIX + "json."; private static final String FILE_READER_JSON_COMPRESSION = FILE_READER_JSON + "compression."; - public static final String FILE_READER_JSON_RECORD_PER_LINE = FILE_READER_JSON + "record_per_line"; - public static final String FILE_READER_JSON_DESERIALIZATION_CONFIGS = FILE_READER_JSON + "deserialization."; + static final String FILE_READER_JSON_DESERIALIZATION_CONFIGS = FILE_READER_JSON + "deserialization."; + public static final String FILE_READER_JSON_RECORD_PER_LINE = FILE_READER_JSON + "record_per_line"; public static final String FILE_READER_JSON_COMPRESSION_TYPE = FILE_READER_JSON_COMPRESSION + "type"; public static final String FILE_READER_JSON_COMPRESSION_CONCATENATED = FILE_READER_JSON_COMPRESSION + "concatenated"; public static final String FILE_READER_JSON_ENCODING = FILE_READER_JSON + "encoding"; - private final TextFileReader inner; - private final Schema schema; - private ObjectMapper mapper; - public JsonFileReader(FileSystem fs, Path filePath, Map config) throws IOException { - super(fs, filePath, new JsonToStruct(), config); - - config.put(TextFileReader.FILE_READER_TEXT_ENCODING, config.get(FILE_READER_JSON_ENCODING)); - config.put(TextFileReader.FILE_READER_TEXT_RECORD_PER_LINE, config.get(FILE_READER_JSON_RECORD_PER_LINE)); - config.put(TextFileReader.FILE_READER_TEXT_COMPRESSION_TYPE, config.get(FILE_READER_JSON_COMPRESSION_TYPE)); - config.put(TextFileReader.FILE_READER_TEXT_COMPRESSION_CONCATENATED, config.get(FILE_READER_JSON_COMPRESSION_CONCATENATED)); - - this.inner = new TextFileReader(fs, filePath, config); - - if (hasNext()) { - String line = inner.nextRecord().getValue(); - this.schema = extractSchema(mapper.readTree(line)); - // back to the first line - inner.seek(0); - } else { - this.schema = SchemaBuilder.struct().build(); - } + super(fs, filePath, config); } @Override - protected void configure(Map config) { - mapper = new ObjectMapper(); - Set deserializationFeatures = Arrays.stream(DeserializationFeature.values()) - .map(Enum::name) - .collect(Collectors.toSet()); - config.entrySet().stream() - .filter(entry -> entry.getKey().startsWith(FILE_READER_JSON_DESERIALIZATION_CONFIGS)) - .forEach(entry -> { - String feature = entry.getKey().replaceAll(FILE_READER_JSON_DESERIALIZATION_CONFIGS, ""); - if (deserializationFeatures.contains(feature)) { - mapper.configure(DeserializationFeature.valueOf(feature), - Boolean.parseBoolean(entry.getValue())); - } else { - log.warn("{} Ignoring deserialization configuration [{}] due to it does not exist.", - this, feature); - } - }); + protected Object readerEncodingConfig(Map config) { + return config.get(FILE_READER_JSON_ENCODING); } @Override - protected JsonRecord nextRecord() throws IOException { - JsonNode value = mapper.readTree(inner.nextRecord().getValue()); - return new JsonRecord(schema, value); + protected Object recordPerLineConfig(Map config) { + return config.get(FILE_READER_JSON_RECORD_PER_LINE); } @Override - public boolean hasNextRecord() throws IOException { - return inner.hasNextRecord(); + protected Object compressionTypeConfig(Map config) { + return config.get(FILE_READER_JSON_COMPRESSION_TYPE); } @Override - public void seekFile(long offset) throws IOException { - inner.seekFile(offset); + protected Object compressionConcatenatedConfig(Map config) { + return config.get(FILE_READER_JSON_COMPRESSION_CONCATENATED); } @Override - public long currentOffset() { - return inner.currentOffset(); + protected String deserializationConfigPrefix() { + return FILE_READER_JSON_DESERIALIZATION_CONFIGS; } @Override - public void close() throws IOException { - inner.close(); - } - - @Override - public boolean isClosed() { - return inner.isClosed(); - } - - private static Schema extractSchema(JsonNode jsonNode) { - switch (jsonNode.getNodeType()) { - case BOOLEAN: - return Schema.OPTIONAL_BOOLEAN_SCHEMA; - case NUMBER: - if (jsonNode.isShort()) { - return Schema.OPTIONAL_INT8_SCHEMA; - } else if (jsonNode.isInt()) { - return Schema.OPTIONAL_INT32_SCHEMA; - } else if (jsonNode.isLong()) { - return Schema.OPTIONAL_INT64_SCHEMA; - } else if (jsonNode.isBigInteger()) { - return Schema.OPTIONAL_INT64_SCHEMA; - } else { - return Schema.OPTIONAL_FLOAT64_SCHEMA; - } - case STRING: - return Schema.OPTIONAL_STRING_SCHEMA; - case BINARY: - return Schema.OPTIONAL_BYTES_SCHEMA; - case ARRAY: - Iterable elements = jsonNode::elements; - Schema arraySchema = StreamSupport.stream(elements.spliterator(), false) - .findFirst().map(JsonFileReader::extractSchema) - .orElse(SchemaBuilder.struct().build()); - return SchemaBuilder.array(arraySchema).build(); - case OBJECT: - SchemaBuilder builder = SchemaBuilder.struct(); - jsonNode.fields() - .forEachRemaining(field -> builder.field(field.getKey(), extractSchema(field.getValue()))); - return builder.build(); - default: - return SchemaBuilder.struct().optional().build(); - } - } - - static class JsonToStruct implements ReaderAdapter { - - @Override - public Struct apply(JsonRecord record) { - return toStruct(record.schema, record.value); - } - - private Struct toStruct(Schema schema, JsonNode jsonNode) { - if (jsonNode.isNull()) return null; - Struct struct = new Struct(schema); - jsonNode.fields() - .forEachRemaining(field -> struct.put( - field.getKey(), - mapValue(struct.schema().field(field.getKey()).schema(), field.getValue()) - )); - return struct; - } - - private Object mapValue(Schema schema, JsonNode value) { - if (value == null) return null; - - switch (value.getNodeType()) { - case BOOLEAN: - return value.booleanValue(); - case NUMBER: - if (value.isShort()) { - return value.shortValue(); - } else if (value.isInt()) { - return value.intValue(); - } else if (value.isLong()) { - return value.longValue(); - } else if (value.isBigInteger()) { - return value.bigIntegerValue().longValue(); - } else { - return value.numberValue().doubleValue(); - } - case STRING: - return value.asText(); - case BINARY: - try { - return value.binaryValue(); - } catch (IOException ioe) { - throw new RuntimeException(ioe); - } - case OBJECT: - case POJO: - Struct struct = new Struct(schema); - Iterable> fields = value::fields; - StreamSupport.stream(fields.spliterator(), false) - .forEach(field -> struct.put(field.getKey(), - mapValue(extractSchema(field.getValue()), field.getValue())) - ); - return struct; - case ARRAY: - Iterable arrayElements = value::elements; - return StreamSupport.stream(arrayElements.spliterator(), false) - .map(elm -> mapValue(schema.valueSchema(), elm)) - .collect(Collectors.toList()); - case NULL: - case MISSING: - default: - return null; - } - } - } - - static class JsonRecord { - private final Schema schema; - private final JsonNode value; - - JsonRecord(Schema schema, JsonNode value) { - this.schema = schema; - this.value = value; - } + protected ObjectMapper getObjectMapper() { + return new ObjectMapper(); } } diff --git a/src/main/java/com/github/mmolimar/kafka/connect/fs/file/reader/XmlFileReader.java b/src/main/java/com/github/mmolimar/kafka/connect/fs/file/reader/XmlFileReader.java new file mode 100644 index 0000000..5f7765a --- /dev/null +++ b/src/main/java/com/github/mmolimar/kafka/connect/fs/file/reader/XmlFileReader.java @@ -0,0 +1,58 @@ +package com.github.mmolimar.kafka.connect.fs.file.reader; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.dataformat.xml.XmlMapper; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; + +import java.io.IOException; +import java.util.Map; + +import static com.github.mmolimar.kafka.connect.fs.FsSourceTaskConfig.FILE_READER_PREFIX; + +public class XmlFileReader extends JacksonFileReader { + + private static final String FILE_READER_XML = FILE_READER_PREFIX + "xml."; + private static final String FILE_READER_XML_COMPRESSION = FILE_READER_XML + "compression."; + + static final String FILE_READER_XML_DESERIALIZATION_CONFIGS = FILE_READER_XML + "deserialization."; + + public static final String FILE_READER_XML_RECORD_PER_LINE = FILE_READER_XML + "record_per_line"; + public static final String FILE_READER_XML_COMPRESSION_TYPE = FILE_READER_XML_COMPRESSION + "type"; + public static final String FILE_READER_XML_COMPRESSION_CONCATENATED = FILE_READER_XML_COMPRESSION + "concatenated"; + public static final String FILE_READER_XML_ENCODING = FILE_READER_XML + "encoding"; + + public XmlFileReader(FileSystem fs, Path filePath, Map config) throws IOException { + super(fs, filePath, config); + } + + @Override + protected Object readerEncodingConfig(Map config) { + return config.get(FILE_READER_XML_ENCODING); + } + + @Override + protected Object recordPerLineConfig(Map config) { + return config.get(FILE_READER_XML_RECORD_PER_LINE); + } + + @Override + protected Object compressionTypeConfig(Map config) { + return config.get(FILE_READER_XML_COMPRESSION_TYPE); + } + + @Override + protected Object compressionConcatenatedConfig(Map config) { + return config.get(FILE_READER_XML_COMPRESSION_CONCATENATED); + } + + @Override + protected String deserializationConfigPrefix() { + return FILE_READER_XML_DESERIALIZATION_CONFIGS; + } + + @Override + protected ObjectMapper getObjectMapper() { + return new XmlMapper(); + } +} diff --git a/src/main/java/com/github/mmolimar/kafka/connect/fs/file/reader/YamlFileReader.java b/src/main/java/com/github/mmolimar/kafka/connect/fs/file/reader/YamlFileReader.java new file mode 100644 index 0000000..8be5c6b --- /dev/null +++ b/src/main/java/com/github/mmolimar/kafka/connect/fs/file/reader/YamlFileReader.java @@ -0,0 +1,57 @@ +package com.github.mmolimar.kafka.connect.fs.file.reader; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.dataformat.yaml.YAMLMapper; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; + +import java.io.IOException; +import java.util.Map; + +import static com.github.mmolimar.kafka.connect.fs.FsSourceTaskConfig.FILE_READER_PREFIX; + +public class YamlFileReader extends JacksonFileReader { + + private static final String FILE_READER_YAML = FILE_READER_PREFIX + "yaml."; + private static final String FILE_READER_YAML_COMPRESSION = FILE_READER_YAML + "compression."; + + static final String FILE_READER_YAML_DESERIALIZATION_CONFIGS = FILE_READER_YAML + "deserialization."; + + public static final String FILE_READER_YAML_COMPRESSION_TYPE = FILE_READER_YAML_COMPRESSION + "type"; + public static final String FILE_READER_YAML_COMPRESSION_CONCATENATED = FILE_READER_YAML_COMPRESSION + "concatenated"; + public static final String FILE_READER_YAML_ENCODING = FILE_READER_YAML + "encoding"; + + public YamlFileReader(FileSystem fs, Path filePath, Map config) throws IOException { + super(fs, filePath, config); + } + + @Override + protected Object readerEncodingConfig(Map config) { + return config.get(FILE_READER_YAML_ENCODING); + } + + @Override + protected Object recordPerLineConfig(Map config) { + return false; + } + + @Override + protected Object compressionTypeConfig(Map config) { + return config.get(FILE_READER_YAML_COMPRESSION_TYPE); + } + + @Override + protected Object compressionConcatenatedConfig(Map config) { + return config.get(FILE_READER_YAML_COMPRESSION_CONCATENATED); + } + + @Override + protected String deserializationConfigPrefix() { + return FILE_READER_YAML_DESERIALIZATION_CONFIGS; + } + + @Override + protected ObjectMapper getObjectMapper() { + return new YAMLMapper(); + } +} diff --git a/src/test/java/com/github/mmolimar/kafka/connect/fs/file/reader/JacksonFileReaderTest.java b/src/test/java/com/github/mmolimar/kafka/connect/fs/file/reader/JacksonFileReaderTest.java new file mode 100644 index 0000000..2c6bc8a --- /dev/null +++ b/src/test/java/com/github/mmolimar/kafka/connect/fs/file/reader/JacksonFileReaderTest.java @@ -0,0 +1,210 @@ +package com.github.mmolimar.kafka.connect.fs.file.reader; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.DeserializationFeature; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.ObjectWriter; +import com.fasterxml.jackson.databind.node.ArrayNode; +import com.fasterxml.jackson.databind.node.JsonNodeFactory; +import com.fasterxml.jackson.databind.node.ObjectNode; +import org.apache.hadoop.fs.Path; +import org.apache.kafka.connect.data.Struct; +import org.apache.kafka.connect.errors.ConnectException; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.MethodSource; + +import java.io.File; +import java.io.IOException; +import java.io.PrintWriter; +import java.math.BigInteger; +import java.nio.charset.UnsupportedCharsetException; +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; +import java.util.UUID; +import java.util.stream.IntStream; + +import static org.junit.jupiter.api.Assertions.*; + +abstract class JacksonFileReaderTest extends FileReaderTestBase { + + static final String FIELD_INTEGER = "integerField"; + static final String FIELD_BIG_INTEGER = "bigIntegerField"; + static final String FIELD_LONG = "longField"; + static final String FIELD_BOOLEAN = "booleanField"; + static final String FIELD_STRING = "stringField"; + static final String FIELD_DECIMAL = "decimalField"; + static final String FIELD_BINARY = "binaryField"; + static final String FIELD_ARRAY_SIMPLE = "arraySimpleField"; + static final String FIELD_ARRAY_COMPLEX = "arrayComplexField"; + static final String FIELD_STRUCT = "structField"; + static final String FIELD_NULL = "nullField"; + static final CompressionType COMPRESSION_TYPE_DEFAULT = CompressionType.NONE; + + @Override + protected Path createDataFile(ReaderFsTestConfig fsConfig, Object... args) throws IOException { + int numRecords = args.length < 1 ? NUM_RECORDS : (int) args[0]; + boolean recordPerLine = args.length < 2 || (boolean) args[1]; + CompressionType compression = args.length < 3 ? COMPRESSION_TYPE_DEFAULT : (CompressionType) args[2]; + File txtFile = File.createTempFile("test-", "." + getFileExtension()); + try (PrintWriter writer = new PrintWriter(getOutputStream(txtFile, compression))) { + ObjectWriter objectWriter = getObjectMapper().writerWithDefaultPrettyPrinter(); + IntStream.range(0, numRecords).forEach(index -> { + ObjectNode node = JsonNodeFactory.instance.objectNode() + .put(FIELD_INTEGER, index) + .put(FIELD_BIG_INTEGER, new BigInteger("9999999999999999999")) + .put(FIELD_LONG, Long.MAX_VALUE) + .put(FIELD_STRING, String.format("%d_%s", index, UUID.randomUUID())) + .put(FIELD_BOOLEAN, true) + .put(FIELD_DECIMAL, Double.parseDouble(index + "." + index)) + .put(FIELD_BINARY, "test".getBytes()) + .put(FIELD_NULL, (String) null); + node.putArray(FIELD_ARRAY_SIMPLE) + .add("elm[" + index + "]") + .add("elm[" + (index + 1) + "]"); + ArrayNode array = node.putArray(FIELD_ARRAY_COMPLEX); + array.addObject() + .put(FIELD_INTEGER, index) + .put(FIELD_LONG, Long.MAX_VALUE) + .put(FIELD_STRING, String.format("%d_%s", index, UUID.randomUUID())) + .put(FIELD_BOOLEAN, true) + .put(FIELD_DECIMAL, Double.parseDouble(index + "." + index)) + .put(FIELD_NULL, (String) null); + array.addObject() + .put(FIELD_INTEGER, index + 1) + .put(FIELD_LONG, Long.MAX_VALUE) + .put(FIELD_STRING, String.format("%d_%s", index, UUID.randomUUID())) + .put(FIELD_BOOLEAN, true) + .put(FIELD_DECIMAL, Double.parseDouble(index + "." + index)) + .put(FIELD_NULL, (String) null); + node.putObject(FIELD_STRUCT) + .put(FIELD_INTEGER, (short) index) + .put(FIELD_LONG, Long.MAX_VALUE) + .put(FIELD_STRING, String.format("%d_%s", index, UUID.randomUUID())) + .put(FIELD_BOOLEAN, true) + .put(FIELD_DECIMAL, Double.parseDouble(index + "." + index)) + .put(FIELD_NULL, (String) null); + try { + writer.append(recordPerLine ? objectWriter.writeValueAsString(node).replaceAll("\n", "") + "\n" : objectWriter.writeValueAsString(node)); + } catch (JsonProcessingException jpe) { + throw new RuntimeException(jpe); + } + fsConfig.offsetsByIndex().put(index, (long) index); + }); + } + Path path = new Path(new Path(fsConfig.getFsUri()), txtFile.getName()); + fsConfig.getFs().moveFromLocalFile(new Path(txtFile.getAbsolutePath()), path); + return path; + } + + @ParameterizedTest + @MethodSource("fileSystemConfigProvider") + public void emptyFile(ReaderFsTestConfig fsConfig) throws IOException { + File tmp = File.createTempFile("test-", "." + getFileExtension()); + Path path = new Path(new Path(fsConfig.getFsUri()), tmp.getName()); + fsConfig.getFs().moveFromLocalFile(new Path(tmp.getAbsolutePath()), path); + FileReader reader = getReader(fsConfig.getFs(), path, getReaderConfig()); + assertFalse(reader.hasNext()); + } + + @ParameterizedTest + @MethodSource("fileSystemConfigProvider") + public void validFileEncoding(ReaderFsTestConfig fsConfig) { + Map readerConfig = getReaderConfig(); + readerConfig.put(readerEncodingConfig(), "Cp1252"); + fsConfig.setReader(getReader(fsConfig.getFs(), fsConfig.getDataFile(), readerConfig)); + readAllData(fsConfig); + } + + @ParameterizedTest + @MethodSource("fileSystemConfigProvider") + public void invalidDeserializationConfig(ReaderFsTestConfig fsConfig) { + Map readerConfig = getReaderConfig(); + readerConfig.put(deserializationConfigPrefix() + "invalid", "false"); + fsConfig.setReader(getReader(fsConfig.getFs(), fsConfig.getDataFile(), readerConfig)); + readAllData(fsConfig); + } + + @ParameterizedTest + @MethodSource("fileSystemConfigProvider") + public void invalidFileEncoding(ReaderFsTestConfig fsConfig) { + Map readerConfig = getReaderConfig(); + readerConfig.put(readerEncodingConfig(), "invalid_charset"); + assertThrows(ConnectException.class, () -> getReader(fsConfig.getFs(), fsConfig.getDataFile(), readerConfig)); + assertThrows(UnsupportedCharsetException.class, () -> { + try { + getReader(fsConfig.getFs(), fsConfig.getDataFile(), readerConfig); + } catch (Exception e) { + throw e.getCause(); + } + }); + } + + @ParameterizedTest + @MethodSource("fileSystemConfigProvider") + public void readDataWithRecordPerLineDisabled(ReaderFsTestConfig fsConfig) throws IOException { + Path file = createDataFile(fsConfig, 1, false); + Map readerConfig = getReaderConfig(); + readerConfig.put(recordPerLineConfig(), "false"); + FileReader reader = getReader(fsConfig.getFs(), file, readerConfig); + + assertTrue(reader.hasNext()); + + int recordCount = 0; + while (reader.hasNext()) { + Struct record = reader.next(); + checkData(record, recordCount); + recordCount++; + } + reader.close(); + assertEquals(1, recordCount, "The number of records in the file does not match"); + } + + @ParameterizedTest + @MethodSource("fileSystemConfigProvider") + public void readDifferentCompressionTypes(ReaderFsTestConfig fsConfig) { + Arrays.stream(CompressionType.values()).forEach(compressionType -> { + try { + Path file = createDataFile(fsConfig, NUM_RECORDS, true, compressionType); + Map readerConfig = getReaderConfig(); + readerConfig.put(compressionTypeConfig(), compressionType.toString()); + readerConfig.put(compressionConcatenatedConfig(), "true"); + FileReader reader = getReader(fsConfig.getFs(), file, readerConfig); + + assertTrue(reader.hasNext()); + + int recordCount = 0; + while (reader.hasNext()) { + Struct record = reader.next(); + checkData(record, recordCount); + recordCount++; + } + reader.close(); + assertEquals(NUM_RECORDS, recordCount, "The number of records in the file does not match"); + } catch (Exception e) { + throw new RuntimeException(e); + } + }); + } + + @Override + protected Map getReaderConfig() { + return new HashMap() {{ + String deserializationConfig = DeserializationFeature.ACCEPT_EMPTY_ARRAY_AS_NULL_OBJECT.name(); + put(deserializationConfigPrefix() + deserializationConfig, "true"); + }}; + } + + protected abstract String readerEncodingConfig(); + + protected abstract String recordPerLineConfig(); + + protected abstract String compressionTypeConfig(); + + protected abstract String compressionConcatenatedConfig(); + + protected abstract String deserializationConfigPrefix(); + + protected abstract ObjectMapper getObjectMapper(); + +} diff --git a/src/test/java/com/github/mmolimar/kafka/connect/fs/file/reader/JsonFileReaderTest.java b/src/test/java/com/github/mmolimar/kafka/connect/fs/file/reader/JsonFileReaderTest.java index f7f6da0..6602746 100644 --- a/src/test/java/com/github/mmolimar/kafka/connect/fs/file/reader/JsonFileReaderTest.java +++ b/src/test/java/com/github/mmolimar/kafka/connect/fs/file/reader/JsonFileReaderTest.java @@ -1,202 +1,17 @@ package com.github.mmolimar.kafka.connect.fs.file.reader; -import com.fasterxml.jackson.core.JsonProcessingException; -import com.fasterxml.jackson.databind.DeserializationFeature; import com.fasterxml.jackson.databind.ObjectMapper; -import com.fasterxml.jackson.databind.ObjectWriter; -import com.fasterxml.jackson.databind.node.ArrayNode; -import com.fasterxml.jackson.databind.node.JsonNodeFactory; -import com.fasterxml.jackson.databind.node.ObjectNode; -import org.apache.hadoop.fs.Path; import org.apache.kafka.connect.data.Struct; -import org.apache.kafka.connect.errors.ConnectException; -import org.junit.jupiter.params.ParameterizedTest; -import org.junit.jupiter.params.provider.MethodSource; -import java.io.File; -import java.io.IOException; -import java.io.PrintWriter; import java.math.BigInteger; -import java.nio.charset.UnsupportedCharsetException; -import java.util.*; -import java.util.stream.IntStream; +import java.util.Arrays; +import java.util.List; import static org.junit.jupiter.api.Assertions.*; -public class JsonFileReaderTest extends FileReaderTestBase { - - private static final String FIELD_INTEGER = "integerField"; - private static final String FIELD_BIG_INTEGER = "bigIntegerField"; - private static final String FIELD_LONG = "longField"; - private static final String FIELD_BOOLEAN = "booleanField"; - private static final String FIELD_STRING = "stringField"; - private static final String FIELD_DECIMAL = "decimalField"; - private static final String FIELD_BINARY = "binaryField"; - private static final String FIELD_ARRAY_SIMPLE = "arraySimpleField"; - private static final String FIELD_ARRAY_COMPLEX = "arrayComplexField"; - private static final String FIELD_STRUCT = "structField"; - private static final String FIELD_NULL = "nullField"; - private static final String FILE_EXTENSION = "jsn"; - private static final CompressionType COMPRESSION_TYPE_DEFAULT = CompressionType.NONE; - - @Override - protected Path createDataFile(ReaderFsTestConfig fsConfig, Object... args) throws IOException { - int numRecords = args.length < 1 ? NUM_RECORDS : (int) args[0]; - boolean recordPerLine = args.length < 2 || (boolean) args[1]; - CompressionType compression = args.length < 3 ? COMPRESSION_TYPE_DEFAULT : (CompressionType) args[2]; - File txtFile = File.createTempFile("test-", "." + getFileExtension()); - try (PrintWriter writer = new PrintWriter(getOutputStream(txtFile, compression))) { - ObjectWriter jsonWriter = new ObjectMapper().writerWithDefaultPrettyPrinter(); - IntStream.range(0, numRecords).forEach(index -> { - ObjectNode json = JsonNodeFactory.instance.objectNode() - .put(FIELD_INTEGER, index) - .put(FIELD_BIG_INTEGER, new BigInteger("9999999999999999999")) - .put(FIELD_LONG, Long.MAX_VALUE) - .put(FIELD_STRING, String.format("%d_%s", index, UUID.randomUUID())) - .put(FIELD_BOOLEAN, true) - .put(FIELD_DECIMAL, Double.parseDouble(index + "." + index)) - .put(FIELD_BINARY, "test".getBytes()) - .put(FIELD_NULL, (String) null); - json.putArray(FIELD_ARRAY_SIMPLE) - .add("elm[" + index + "]") - .add("elm[" + (index + 1) + "]"); - ArrayNode array = json.putArray(FIELD_ARRAY_COMPLEX); - array.addObject() - .put(FIELD_INTEGER, index) - .put(FIELD_LONG, Long.MAX_VALUE) - .put(FIELD_STRING, String.format("%d_%s", index, UUID.randomUUID())) - .put(FIELD_BOOLEAN, true) - .put(FIELD_DECIMAL, Double.parseDouble(index + "." + index)) - .put(FIELD_NULL, (String) null); - array.addObject() - .put(FIELD_INTEGER, index + 1) - .put(FIELD_LONG, Long.MAX_VALUE) - .put(FIELD_STRING, String.format("%d_%s", index, UUID.randomUUID())) - .put(FIELD_BOOLEAN, true) - .put(FIELD_DECIMAL, Double.parseDouble(index + "." + index)) - .put(FIELD_NULL, (String) null); - json.putObject(FIELD_STRUCT) - .put(FIELD_INTEGER, (short) index) - .put(FIELD_LONG, Long.MAX_VALUE) - .put(FIELD_STRING, String.format("%d_%s", index, UUID.randomUUID())) - .put(FIELD_BOOLEAN, true) - .put(FIELD_DECIMAL, Double.parseDouble(index + "." + index)) - .put(FIELD_NULL, (String) null); - try { - writer.append(recordPerLine ? json.toString() + "\n" : jsonWriter.writeValueAsString(json)); - } catch (JsonProcessingException jpe) { - throw new RuntimeException(jpe); - } - fsConfig.offsetsByIndex().put(index, (long) index); - }); - } - Path path = new Path(new Path(fsConfig.getFsUri()), txtFile.getName()); - fsConfig.getFs().moveFromLocalFile(new Path(txtFile.getAbsolutePath()), path); - return path; - } - - @ParameterizedTest - @MethodSource("fileSystemConfigProvider") - public void emptyFile(ReaderFsTestConfig fsConfig) throws IOException { - File tmp = File.createTempFile("test-", "." + getFileExtension()); - Path path = new Path(new Path(fsConfig.getFsUri()), tmp.getName()); - fsConfig.getFs().moveFromLocalFile(new Path(tmp.getAbsolutePath()), path); - FileReader reader = getReader(fsConfig.getFs(), path, getReaderConfig()); - assertFalse(reader.hasNext()); - } - - @ParameterizedTest - @MethodSource("fileSystemConfigProvider") - public void validFileEncoding(ReaderFsTestConfig fsConfig) { - Map readerConfig = getReaderConfig(); - readerConfig.put(JsonFileReader.FILE_READER_JSON_ENCODING, "Cp1252"); - fsConfig.setReader(getReader(fsConfig.getFs(), fsConfig.getDataFile(), readerConfig)); - readAllData(fsConfig); - } - - @ParameterizedTest - @MethodSource("fileSystemConfigProvider") - public void invalidDeserializationConfig(ReaderFsTestConfig fsConfig) { - Map readerConfig = getReaderConfig(); - readerConfig.put(JsonFileReader.FILE_READER_JSON_DESERIALIZATION_CONFIGS + "invalid", "false"); - fsConfig.setReader(getReader(fsConfig.getFs(), fsConfig.getDataFile(), readerConfig)); - readAllData(fsConfig); - } - - @ParameterizedTest - @MethodSource("fileSystemConfigProvider") - public void invalidFileEncoding(ReaderFsTestConfig fsConfig) { - Map readerConfig = getReaderConfig(); - readerConfig.put(JsonFileReader.FILE_READER_JSON_ENCODING, "invalid_charset"); - assertThrows(ConnectException.class, () -> getReader(fsConfig.getFs(), fsConfig.getDataFile(), readerConfig)); - assertThrows(UnsupportedCharsetException.class, () -> { - try { - getReader(fsConfig.getFs(), fsConfig.getDataFile(), readerConfig); - } catch (Exception e) { - throw e.getCause(); - } - }); - } +public class JsonFileReaderTest extends JacksonFileReaderTest { - @ParameterizedTest - @MethodSource("fileSystemConfigProvider") - public void readDataWithRecordPerLineDisabled(ReaderFsTestConfig fsConfig) throws IOException { - Path file = createDataFile(fsConfig, 1, false); - Map readerConfig = getReaderConfig(); - readerConfig.put(JsonFileReader.FILE_READER_JSON_RECORD_PER_LINE, "false"); - FileReader reader = getReader(fsConfig.getFs(), file, readerConfig); - - assertTrue(reader.hasNext()); - - int recordCount = 0; - while (reader.hasNext()) { - Struct record = reader.next(); - checkData(record, recordCount); - recordCount++; - } - reader.close(); - assertEquals(1, recordCount, "The number of records in the file does not match"); - } - - @ParameterizedTest - @MethodSource("fileSystemConfigProvider") - public void readDifferentCompressionTypes(ReaderFsTestConfig fsConfig) { - Arrays.stream(CompressionType.values()).forEach(compressionType -> { - try { - Path file = createDataFile(fsConfig, NUM_RECORDS, true, compressionType); - Map readerConfig = getReaderConfig(); - readerConfig.put(JsonFileReader.FILE_READER_JSON_COMPRESSION_TYPE, compressionType.toString()); - readerConfig.put(JsonFileReader.FILE_READER_JSON_COMPRESSION_CONCATENATED, "true"); - FileReader reader = getReader(fsConfig.getFs(), file, readerConfig); - - assertTrue(reader.hasNext()); - - int recordCount = 0; - while (reader.hasNext()) { - Struct record = reader.next(); - checkData(record, recordCount); - recordCount++; - } - reader.close(); - assertEquals(NUM_RECORDS, recordCount, "The number of records in the file does not match"); - } catch (Exception e) { - throw new RuntimeException(e); - } - }); - } - - @Override - protected Class getReaderClass() { - return JsonFileReader.class; - } - - @Override - protected Map getReaderConfig() { - return new HashMap() {{ - String deserializationConfig = DeserializationFeature.ACCEPT_EMPTY_ARRAY_AS_NULL_OBJECT.name(); - put(JsonFileReader.FILE_READER_JSON_DESERIALIZATION_CONFIGS + deserializationConfig, "true"); - }}; - } + private static final String FILE_EXTENSION = "jsn"; @Override protected void checkData(Struct record, long index) { @@ -239,8 +54,43 @@ protected void checkData(Struct record, long index) { ); } + @Override + protected Class getReaderClass() { + return JsonFileReader.class; + } + @Override protected String getFileExtension() { return FILE_EXTENSION; } + + @Override + protected String readerEncodingConfig() { + return JsonFileReader.FILE_READER_JSON_ENCODING; + } + + @Override + protected String recordPerLineConfig() { + return JsonFileReader.FILE_READER_JSON_RECORD_PER_LINE; + } + + @Override + protected String compressionTypeConfig() { + return JsonFileReader.FILE_READER_JSON_COMPRESSION_TYPE; + } + + @Override + protected String compressionConcatenatedConfig() { + return JsonFileReader.FILE_READER_JSON_COMPRESSION_CONCATENATED; + } + + @Override + protected String deserializationConfigPrefix() { + return JsonFileReader.FILE_READER_JSON_DESERIALIZATION_CONFIGS; + } + + @Override + protected ObjectMapper getObjectMapper() { + return new ObjectMapper(); + } } diff --git a/src/test/java/com/github/mmolimar/kafka/connect/fs/file/reader/XmlFileReaderTest.java b/src/test/java/com/github/mmolimar/kafka/connect/fs/file/reader/XmlFileReaderTest.java new file mode 100644 index 0000000..8a035ed --- /dev/null +++ b/src/test/java/com/github/mmolimar/kafka/connect/fs/file/reader/XmlFileReaderTest.java @@ -0,0 +1,93 @@ +package com.github.mmolimar.kafka.connect.fs.file.reader; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.dataformat.xml.XmlMapper; +import org.apache.kafka.connect.data.Struct; + +import static org.junit.jupiter.api.Assertions.*; + +public class XmlFileReaderTest extends JacksonFileReaderTest { + + private static final String FILE_EXTENSION = "xl"; + + @Override + protected void checkData(Struct record, long index) { + Struct array = record.getStruct(FIELD_ARRAY_COMPLEX); + Struct subrecord = record.getStruct(FIELD_STRUCT); + assertAll( + () -> assertEquals(index, Integer.parseInt(record.getString(FIELD_INTEGER))), + () -> assertEquals("9999999999999999999", record.get(FIELD_BIG_INTEGER)), + () -> assertEquals(Long.MAX_VALUE, Long.parseLong(record.getString(FIELD_LONG))), + () -> assertTrue(record.get(FIELD_STRING).toString().startsWith(index + "_")), + () -> assertTrue(Boolean.parseBoolean(record.get(FIELD_BOOLEAN).toString())), + () -> assertEquals(Double.parseDouble(index + "." + index), Double.parseDouble(record.getString(FIELD_DECIMAL))), + () -> assertNull(record.get(FIELD_NULL)), + () -> assertNotNull(record.schema().field(FIELD_NULL)), + () -> assertEquals("dGVzdA==", record.get(FIELD_BINARY)), + () -> assertEquals("elm[" + (index + 1) + "]", record.get(FIELD_ARRAY_SIMPLE)), + + () -> assertEquals(index + 1, Integer.parseInt(array.getString(FIELD_INTEGER))), + () -> assertEquals(Long.MAX_VALUE, Long.parseLong(array.getString(FIELD_LONG))), + () -> assertTrue(array.get(FIELD_STRING).toString().startsWith(index + "_")), + () -> assertTrue(Boolean.parseBoolean(array.get(FIELD_BOOLEAN).toString())), + () -> assertEquals(Double.parseDouble(index + "." + index), Double.parseDouble(array.getString(FIELD_DECIMAL))), + () -> assertNull(array.get(FIELD_NULL)), + () -> assertNotNull(array.schema().field(FIELD_NULL)), + () -> assertEquals(index + 1, Integer.parseInt(array.getString(FIELD_INTEGER))), + () -> assertEquals(Long.MAX_VALUE, Long.parseLong(array.getString(FIELD_LONG))), + () -> assertTrue(array.get(FIELD_STRING).toString().startsWith(index + "_")), + () -> assertTrue(Boolean.parseBoolean(array.get(FIELD_BOOLEAN).toString())), + () -> assertEquals(Double.parseDouble(index + "." + index), Double.parseDouble(array.getString(FIELD_DECIMAL))), + () -> assertNull(array.get(FIELD_NULL)), + () -> assertNotNull(array.schema().field(FIELD_NULL)), + + () -> assertEquals(index, Integer.parseInt(subrecord.getString(FIELD_INTEGER))), + () -> assertEquals(Long.MAX_VALUE, Long.parseLong(subrecord.getString(FIELD_LONG))), + () -> assertTrue(subrecord.get(FIELD_STRING).toString().startsWith(index + "_")), + () -> assertTrue(Boolean.parseBoolean(subrecord.get(FIELD_BOOLEAN).toString())), + () -> assertEquals(Double.parseDouble(index + "." + index), Double.parseDouble(subrecord.getString(FIELD_DECIMAL))), + () -> assertNull(subrecord.get(FIELD_NULL)), + () -> assertNotNull(subrecord.schema().field(FIELD_NULL)) + ); + } + + @Override + protected Class getReaderClass() { + return XmlFileReader.class; + } + + @Override + protected String getFileExtension() { + return FILE_EXTENSION; + } + + @Override + protected String readerEncodingConfig() { + return XmlFileReader.FILE_READER_XML_ENCODING; + } + + @Override + protected String recordPerLineConfig() { + return XmlFileReader.FILE_READER_XML_RECORD_PER_LINE; + } + + @Override + protected String compressionTypeConfig() { + return XmlFileReader.FILE_READER_XML_COMPRESSION_TYPE; + } + + @Override + protected String compressionConcatenatedConfig() { + return XmlFileReader.FILE_READER_XML_COMPRESSION_CONCATENATED; + } + + @Override + protected String deserializationConfigPrefix() { + return XmlFileReader.FILE_READER_XML_DESERIALIZATION_CONFIGS; + } + + @Override + protected ObjectMapper getObjectMapper() { + return new XmlMapper(); + } +} diff --git a/src/test/java/com/github/mmolimar/kafka/connect/fs/file/reader/YamlFileReaderTest.java b/src/test/java/com/github/mmolimar/kafka/connect/fs/file/reader/YamlFileReaderTest.java new file mode 100644 index 0000000..121bda3 --- /dev/null +++ b/src/test/java/com/github/mmolimar/kafka/connect/fs/file/reader/YamlFileReaderTest.java @@ -0,0 +1,188 @@ +package com.github.mmolimar.kafka.connect.fs.file.reader; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.dataformat.yaml.YAMLMapper; +import org.apache.hadoop.fs.Path; +import org.apache.kafka.connect.data.Struct; +import org.junit.jupiter.api.Disabled; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.MethodSource; + +import java.io.BufferedWriter; +import java.io.File; +import java.io.FileWriter; +import java.io.IOException; +import java.math.BigInteger; +import java.util.Arrays; +import java.util.List; +import java.util.Map; + +import static org.junit.jupiter.api.Assertions.*; + +public class YamlFileReaderTest extends JacksonFileReaderTest { + + private static final String FILE_EXTENSION = "yl"; + + protected static final int NUM_RECORDS = 1; + + @Override + protected Path createDataFile(ReaderFsTestConfig fsConfig, Object... args) throws IOException { + CompressionType compression = args.length < 3 ? COMPRESSION_TYPE_DEFAULT : (CompressionType) args[2]; + return super.createDataFile(fsConfig, 1, false, compression); + } + + @ParameterizedTest + @MethodSource("fileSystemConfigProvider") + public void invalidFileFormat(ReaderFsTestConfig fsConfig) throws IOException { + File tmp = File.createTempFile("test-", "." + getFileExtension()); + try (BufferedWriter writer = new BufferedWriter(new FileWriter(tmp))) { + writer.write("test"); + } + Path path = new Path(new Path(fsConfig.getFsUri()), tmp.getName()); + fsConfig.getFs().moveFromLocalFile(new Path(tmp.getAbsolutePath()), path); + getReader(fsConfig.getFs(), path, getReaderConfig()); + } + + @ParameterizedTest + @MethodSource("fileSystemConfigProvider") + public void readAllData(ReaderFsTestConfig fsConfig) { + FileReader reader = fsConfig.getReader(); + assertTrue(reader.hasNext()); + + int recordCount = 0; + while (reader.hasNext()) { + Struct record = reader.next(); + checkData(record, recordCount); + recordCount++; + } + assertEquals(NUM_RECORDS, recordCount, "The number of records in the file does not match"); + } + + @ParameterizedTest + @MethodSource("fileSystemConfigProvider") + @Disabled + public void seekFile(ReaderFsTestConfig fsConfig) { + } + + @ParameterizedTest + @MethodSource("fileSystemConfigProvider") + @Disabled + public void exceededSeek(ReaderFsTestConfig fsConfig) { + + } + + @ParameterizedTest + @MethodSource("fileSystemConfigProvider") + @Disabled + public void readAllDataInBatches(ReaderFsTestConfig fsConfig) { + + } + + @ParameterizedTest + @MethodSource("fileSystemConfigProvider") + public void readDifferentCompressionTypes(ReaderFsTestConfig fsConfig) { + Arrays.stream(CompressionType.values()).forEach(compressionType -> { + try { + Path file = createDataFile(fsConfig, NUM_RECORDS, true, compressionType); + Map readerConfig = getReaderConfig(); + readerConfig.put(compressionTypeConfig(), compressionType.toString()); + readerConfig.put(compressionConcatenatedConfig(), "true"); + FileReader reader = getReader(fsConfig.getFs(), file, readerConfig); + + assertTrue(reader.hasNext()); + + int recordCount = 0; + while (reader.hasNext()) { + Struct record = reader.next(); + checkData(record, recordCount); + recordCount++; + } + reader.close(); + assertEquals(NUM_RECORDS, recordCount, "The number of records in the file does not match"); + } catch (Exception e) { + throw new RuntimeException(e); + } + }); + } + + @Override + protected void checkData(Struct record, long index) { + List array = record.getArray(FIELD_ARRAY_COMPLEX); + Struct subrecord = record.getStruct(FIELD_STRUCT); + assertAll( + () -> assertEquals(index, (int) record.get(FIELD_INTEGER)), + () -> assertEquals(new BigInteger("9999999999999999999").longValue(), record.get(FIELD_BIG_INTEGER)), + () -> assertEquals(Long.MAX_VALUE, (long) record.get(FIELD_LONG)), + () -> assertTrue(record.get(FIELD_STRING).toString().startsWith(index + "_")), + () -> assertTrue(Boolean.parseBoolean(record.get(FIELD_BOOLEAN).toString())), + () -> assertEquals(Double.parseDouble(index + "." + index), (Double) record.get(FIELD_DECIMAL), 0), + () -> assertNull(record.get(FIELD_NULL)), + () -> assertNotNull(record.schema().field(FIELD_NULL)), + () -> assertEquals("test", new String((byte[]) record.get(FIELD_BINARY))), + () -> assertEquals(Arrays.asList("elm[" + index + "]", "elm[" + (index + 1) + "]"), record.get(FIELD_ARRAY_SIMPLE)), + + () -> assertEquals(index, (int) array.get(0).get(FIELD_INTEGER)), + () -> assertEquals(Long.MAX_VALUE, (long) array.get(0).get(FIELD_LONG)), + () -> assertTrue(array.get(0).get(FIELD_STRING).toString().startsWith(index + "_")), + () -> assertTrue(Boolean.parseBoolean(array.get(0).get(FIELD_BOOLEAN).toString())), + () -> assertEquals(Double.parseDouble(index + "." + index), (Double) array.get(0).get(FIELD_DECIMAL), 0), + () -> assertNull(array.get(0).get(FIELD_NULL)), + () -> assertNotNull(array.get(0).schema().field(FIELD_NULL)), + () -> assertEquals(index + 1, (int) array.get(1).get(FIELD_INTEGER)), + () -> assertEquals(Long.MAX_VALUE, (long) array.get(1).get(FIELD_LONG)), + () -> assertTrue(array.get(1).get(FIELD_STRING).toString().startsWith(index + "_")), + () -> assertTrue(Boolean.parseBoolean(array.get(1).get(FIELD_BOOLEAN).toString())), + () -> assertEquals(Double.parseDouble(index + "." + index), (Double) array.get(1).get(FIELD_DECIMAL), 0), + () -> assertNull(array.get(1).get(FIELD_NULL)), + () -> assertNotNull(array.get(1).schema().field(FIELD_NULL)), + + () -> assertEquals(index, (int) subrecord.get(FIELD_INTEGER)), + () -> assertEquals(Long.MAX_VALUE, (long) subrecord.get(FIELD_LONG)), + () -> assertTrue(subrecord.get(FIELD_STRING).toString().startsWith(index + "_")), + () -> assertTrue(Boolean.parseBoolean(subrecord.get(FIELD_BOOLEAN).toString())), + () -> assertEquals(Double.parseDouble(index + "." + index), (Double) subrecord.get(FIELD_DECIMAL), 0), + () -> assertNull(subrecord.get(FIELD_NULL)), + () -> assertNotNull(subrecord.schema().field(FIELD_NULL)) + ); + } + + @Override + protected Class getReaderClass() { + return YamlFileReader.class; + } + + @Override + protected String getFileExtension() { + return FILE_EXTENSION; + } + + @Override + protected String readerEncodingConfig() { + return YamlFileReader.FILE_READER_YAML_ENCODING; + } + + @Override + protected String recordPerLineConfig() { + return "UNKNOWN"; + } + + @Override + protected String compressionTypeConfig() { + return YamlFileReader.FILE_READER_YAML_COMPRESSION_TYPE; + } + + @Override + protected String compressionConcatenatedConfig() { + return YamlFileReader.FILE_READER_YAML_COMPRESSION_CONCATENATED; + } + + @Override + protected String deserializationConfigPrefix() { + return YamlFileReader.FILE_READER_YAML_DESERIALIZATION_CONFIGS; + } + + @Override + protected ObjectMapper getObjectMapper() { + return new YAMLMapper(); + } +} From 9fe64ae2602b968cf4819e76d27ccad224392024 Mon Sep 17 00:00:00 2001 From: Mario Molina Date: Fri, 11 Sep 2020 20:51:50 -0500 Subject: [PATCH 15/21] Updating docs --- docs/source/config_options.rst | 80 ++++++++++++++++++++++++++++++++++ docs/source/connector.rst | 19 +++++++- docs/source/filereaders.rst | 18 ++++++++ 3 files changed, 115 insertions(+), 2 deletions(-) diff --git a/docs/source/config_options.rst b/docs/source/config_options.rst index 9ea840f..436e1e2 100644 --- a/docs/source/config_options.rst +++ b/docs/source/config_options.rst @@ -596,6 +596,86 @@ To configure custom properties for this reader, the name you must use is ``json` * Default: ``true`` * Importance: low +.. _config_options-filereaders-xml: + +XML +-------------------------------------------- + +To configure custom properties for this reader, the name you must use is ``xml``. + +``file_reader.xml.record_per_line`` + If enabled, the reader will read each line as a record. Otherwise, the reader will read the full + content of the file as a record. + + * Type: boolean + * Default: ``true`` + * Importance: medium + +``file_reader.xml.deserialization.`` + Deserialization feature to use when reading a XML file. You can add as much as you like + based on the ones defined `here. `__ + + * Type: boolean + * Importance: medium + +``file_reader.xml.encoding`` + Encoding to use for reading a file. If not specified, the reader will use the default encoding. + + * Type: string + * Default: based on the locale and charset of the underlying operating system. + * Importance: medium + +``file_reader.xml.compression.type`` + Compression type to use when reading a file. + + * Type: enum (available values ``bzip2``, ``gzip`` and ``none``) + * Default: ``none`` + * Importance: medium + +``file_reader.xml.compression.concatenated`` + Flag to specify if the decompression of the reader will finish at the end of the file or after + the first compressed stream. + + * Type: boolean + * Default: ``true`` + * Importance: low + +.. _config_options-filereaders-yaml: + +YAML +-------------------------------------------- + +To configure custom properties for this reader, the name you must use is ``yaml``. + +``file_reader.yaml.deserialization.`` + Deserialization feature to use when reading a YAML file. You can add as much as you like + based on the ones defined `here. `__ + + * Type: boolean + * Importance: medium + +``file_reader.yaml.encoding`` + Encoding to use for reading a file. If not specified, the reader will use the default encoding. + + * Type: string + * Default: based on the locale and charset of the underlying operating system. + * Importance: medium + +``file_reader.yaml.compression.type`` + Compression type to use when reading a file. + + * Type: enum (available values ``bzip2``, ``gzip`` and ``none``) + * Default: ``none`` + * Importance: medium + +``file_reader.yaml.compression.concatenated`` + Flag to specify if the decompression of the reader will finish at the end of the file or after + the first compressed stream. + + * Type: boolean + * Default: ``true`` + * Importance: low + .. _config_options-filereaders-csv: CSV diff --git a/docs/source/connector.rst b/docs/source/connector.rst index 37059da..94593f9 100644 --- a/docs/source/connector.rst +++ b/docs/source/connector.rst @@ -26,7 +26,7 @@ Getting started Prerequisites -------------------------------------------- -- Apache Kafka 2.5.0 +- Apache Kafka 2.6.0 - Java 8 - Confluent Schema Registry (recommended). @@ -146,11 +146,26 @@ They read files and process each record from the FS. The **file reader** is need the connector to process each record and includes in the implementation how to seek and iterate over the records in the file. -The file reader to be used when processing files is defined in ``file_reader.class`` connector property. +The file reader to be used when processing files is defined in the ``file_reader.class`` connector property. In the same way as the policies, the connector provides several sort of readers to parse and read records for different file formats. If you don't have a file reader that fits your needs, just implement one with the unique restriction that it must implement the interface ``com.github.mmolimar.kafka.connect.fs.file.reader.FileReader``. +The are several file readers included which can read the following file formats: + +* Parquet. +* Avro. +* Cobol/EBCDIC. +* CSV. +* TSV. +* Fixed-width. +* JSON. +* XML. +* YAML. +* ORC. +* SequenceFile. +* Text. + .. include:: filereaders.rst diff --git a/docs/source/filereaders.rst b/docs/source/filereaders.rst index c5b1153..fd0d6a4 100644 --- a/docs/source/filereaders.rst +++ b/docs/source/filereaders.rst @@ -74,6 +74,24 @@ and marked as optional in the schema all the fields contained. More information about properties of this file reader :ref:`here`. +XML +^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ + +Reads XML files which might contain multiple number of fields with their specified +data types. The schema for this sort of records is inferred reading the first record +and marked as optional in the schema all the fields contained. + +More information about properties of this file reader :ref:`here`. + +YAML +^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ + +Reads YAML files which might contain multiple number of fields with their specified +data types. The schema for this sort of records is inferred reading the first record +and marked as optional in the schema all the fields contained. + +More information about properties of this file reader :ref:`here`. + CSV ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ From e27dda2a6742c290fc6f24855598d9f115da38b5 Mon Sep 17 00:00:00 2001 From: Mario Molina Date: Sat, 12 Sep 2020 12:08:45 -0500 Subject: [PATCH 16/21] Agnostic file reader with support for XML and YAML files --- docs/source/config_options.rst | 126 ++++++++++-------- docs/source/filereaders.rst | 3 + .../fs/file/reader/AgnosticFileReader.java | 12 +- .../file/reader/AgnosticFileReaderTest.java | 43 +++++- 4 files changed, 126 insertions(+), 58 deletions(-) diff --git a/docs/source/config_options.rst b/docs/source/config_options.rst index 436e1e2..8a98739 100644 --- a/docs/source/config_options.rst +++ b/docs/source/config_options.rst @@ -363,6 +363,62 @@ In order to configure custom properties for this reader, the name you must use i * Default: ``common`` * Importance: medium +``file_reader.cobol.reader.is_record_sequence`` + If the input file has 4 byte record length headers. + + * Type: boolean + * Default: ``false`` + * Importance: medium + +``file_reader.cobol.reader.floating_point_format`` + Format used for the floating-point numbers. + + * Type: enum (available values ``ibm``, ``ibm_little_endian``, ``ieee754``, and ``ieee754_little_endian``) + * Default: ``ibm`` + * Importance: medium + +``file_reader.cobol.reader.schema_policy`` + Specifies a policy to transform the input schema. + + * Type: enum (available values ``keep_original`` and ``collapse_root``) + * Default: ``keep_original`` + * Importance: medium + +``file_reader.cobol.reader.string_trimming_policy`` + The trim to apply for records with string data types. + + * Type: enum (available values ``both``, ``left``, ``right`` and ``none``) + * Default: ``both`` + * Importance: medium + +``file_reader.cobol.reader.start_offset`` + An offset to the start of the record in each binary data block. + + * Type: int + * Default: ``0`` + * Importance: medium + +``file_reader.cobol.reader.end_offset`` + An offset from the end of the record to the end of the binary data block. + + * Type: int + * Default: ``0`` + * Importance: medium + +``file_reader.cobol.reader.file_start_offset`` + A number of bytes to skip at the beginning of each file. + + * Type: int + * Default: ``0`` + * Importance: medium + +``file_reader.cobol.reader.file_end_offset`` + A number of bytes to skip at the end of each file. + + * Type: int + * Default: ``0`` + * Importance: medium + ``file_reader.cobol.reader.ebcdic_code_page_class`` Custom code page conversion class provided. @@ -384,13 +440,6 @@ In order to configure custom properties for this reader, the name you must use i * Default: ``true`` * Importance: low -``file_reader.cobol.reader.floating_point_format`` - Format used for the floating-point numbers. - - * Type: enum (available values ``ibm``, ``ibm_little_endian``, ``ieee754``, and ``ieee754_little_endian``) - * Default: ``ibm`` - * Importance: medium - ``file_reader.cobol.reader.variable_size_occurs`` If true, occurs depending on data size will depend on the number of elements. @@ -405,13 +454,6 @@ In order to configure custom properties for this reader, the name you must use i * Default: ``null`` * Importance: low -``file_reader.cobol.reader.is_record_sequence`` - If the input file has 4 byte record length headers. - - * Type: boolean - * Default: ``false`` - * Importance: medium - ``file_reader.cobol.reader.is_rdw_big_endian`` If the RDW is big endian. @@ -461,48 +503,6 @@ In order to configure custom properties for this reader, the name you must use i * Default: ``null`` * Importance: low -``file_reader.cobol.reader.start_offset`` - An offset to the start of the record in each binary data block. - - * Type: int - * Default: ``0`` - * Importance: medium - -``file_reader.cobol.reader.end_offset`` - An offset from the end of the record to the end of the binary data block. - - * Type: int - * Default: ``0`` - * Importance: medium - -``file_reader.cobol.reader.file_start_offset`` - A number of bytes to skip at the beginning of each file. - - * Type: int - * Default: ``0`` - * Importance: medium - -``file_reader.cobol.reader.file_end_offset`` - A number of bytes to skip at the end of each file. - - * Type: int - * Default: ``0`` - * Importance: medium - -``file_reader.cobol.reader.schema_policy`` - Specifies a policy to transform the input schema. - - * Type: enum (available values ``keep_original`` and ``collapse_root``) - * Default: ``keep_original`` - * Importance: medium - -``file_reader.cobol.reader.string_trimming_policy`` - The trim to apply for records with string data types. - - * Type: enum (available values ``both``, ``left``, ``right`` and ``none``) - * Default: ``both`` - * Importance: medium - ``file_reader.cobol.reader.drop_group_fillers`` If true the parser will drop all FILLER fields, even GROUP FILLERS that have non-FILLER nested fields. @@ -1264,6 +1264,20 @@ To configure custom properties for this reader, the name you must use is ``agnos * Default: ``json`` * Importance: medium +``file_reader.agnostic.extensions.xml`` + A comma-separated string list with the accepted extensions for XML files. + + * Type: string[] + * Default: ``xml`` + * Importance: medium + +``file_reader.agnostic.extensions.yaml`` + A comma-separated string list with the accepted extensions for YAML files. + + * Type: string[] + * Default: ``yaml`` + * Importance: medium + ``file_reader.agnostic.extensions.csv`` A comma-separated string list with the accepted extensions for CSV files. diff --git a/docs/source/filereaders.rst b/docs/source/filereaders.rst index fd0d6a4..0519979 100644 --- a/docs/source/filereaders.rst +++ b/docs/source/filereaders.rst @@ -154,7 +154,10 @@ Default extensions for each format (configurable): * Avro: ``.avro`` * ORC: ``.orc`` * SequenceFile: ``.seq`` +* Cobol / EBCDIC: ``.dat`` * JSON: ``.json`` +* XML: ``.xml`` +* YAML: ``.yaml`` * CSV: ``.csv`` * TSV: ``.tsv`` * FixedWidth: ``.fixed`` diff --git a/src/main/java/com/github/mmolimar/kafka/connect/fs/file/reader/AgnosticFileReader.java b/src/main/java/com/github/mmolimar/kafka/connect/fs/file/reader/AgnosticFileReader.java index 0158a4a..e9c3e77 100644 --- a/src/main/java/com/github/mmolimar/kafka/connect/fs/file/reader/AgnosticFileReader.java +++ b/src/main/java/com/github/mmolimar/kafka/connect/fs/file/reader/AgnosticFileReader.java @@ -25,6 +25,8 @@ public class AgnosticFileReader extends AbstractFileReader reader; private Set parquetExtensions, avroExtensions, sequenceExtensions, orcExtensions, cobolExtensions, - jsonExtensions, csvExtensions, tsvExtensions, fixedExtensions; + jsonExtensions, xmlExtensions, yamlExtensions, csvExtensions, tsvExtensions, fixedExtensions; public AgnosticFileReader(FileSystem fs, Path filePath, Map config) throws Exception { super(fs, filePath, new AgnosticAdapter(), config); @@ -62,6 +64,10 @@ private AbstractFileReader readerByExtension(FileSystem fs, Path filePat clz = CobolFileReader.class; } else if (jsonExtensions.contains(extension)) { clz = JsonFileReader.class; + } else if (xmlExtensions.contains(extension)) { + clz = XmlFileReader.class; + } else if (yamlExtensions.contains(extension)) { + clz = YamlFileReader.class; } else if (csvExtensions.contains(extension)) { clz = CsvFileReader.class; } else if (tsvExtensions.contains(extension)) { @@ -89,6 +95,10 @@ protected void configure(Map config) { .toLowerCase().split(",")).collect(Collectors.toSet()); this.jsonExtensions = Arrays.stream(config.getOrDefault(FILE_READER_AGNOSTIC_EXTENSIONS_JSON, "json") .toLowerCase().split(",")).collect(Collectors.toSet()); + this.xmlExtensions = Arrays.stream(config.getOrDefault(FILE_READER_AGNOSTIC_EXTENSIONS_XML, "xml") + .toLowerCase().split(",")).collect(Collectors.toSet()); + this.yamlExtensions = Arrays.stream(config.getOrDefault(FILE_READER_AGNOSTIC_EXTENSIONS_YAML, "yaml") + .toLowerCase().split(",")).collect(Collectors.toSet()); this.csvExtensions = Arrays.stream(config.getOrDefault(FILE_READER_AGNOSTIC_EXTENSIONS_CSV, "csv") .toLowerCase().split(",")).collect(Collectors.toSet()); this.tsvExtensions = Arrays.stream(config.getOrDefault(FILE_READER_AGNOSTIC_EXTENSIONS_TSV, "tsv") diff --git a/src/test/java/com/github/mmolimar/kafka/connect/fs/file/reader/AgnosticFileReaderTest.java b/src/test/java/com/github/mmolimar/kafka/connect/fs/file/reader/AgnosticFileReaderTest.java index 12dbd38..1055b2a 100644 --- a/src/test/java/com/github/mmolimar/kafka/connect/fs/file/reader/AgnosticFileReaderTest.java +++ b/src/test/java/com/github/mmolimar/kafka/connect/fs/file/reader/AgnosticFileReaderTest.java @@ -113,6 +113,48 @@ public String getFileExtension() { } } + @Nested + class AgnosticXmlFileReaderTest extends XmlFileReaderTest { + + @Override + protected Map getReaderConfig() { + Map config = super.getReaderConfig(); + config.put(AgnosticFileReader.FILE_READER_AGNOSTIC_EXTENSIONS_XML, getFileExtension()); + return config; + } + + @Override + public Class getReaderClass() { + return AgnosticFileReader.class; + } + + @Override + public String getFileExtension() { + return FILE_EXTENSION; + } + } + + @Nested + class AgnosticYamlFileReaderTest extends YamlFileReaderTest { + + @Override + protected Map getReaderConfig() { + Map config = super.getReaderConfig(); + config.put(AgnosticFileReader.FILE_READER_AGNOSTIC_EXTENSIONS_YAML, getFileExtension()); + return config; + } + + @Override + public Class getReaderClass() { + return AgnosticFileReader.class; + } + + @Override + public String getFileExtension() { + return FILE_EXTENSION; + } + } + @Nested class AgnosticAvroFileReaderTest extends AvroFileReaderTest { @@ -222,5 +264,4 @@ public String getFileExtension() { return "dt"; } } - } From 4e0bae0362ab3f822c5a810d520a0c0b5a9614b9 Mon Sep 17 00:00:00 2001 From: Mario Molina Date: Sun, 13 Sep 2020 13:22:25 -0500 Subject: [PATCH 17/21] Updating documentation --- docs/source/config_options.rst | 276 ++++++++++++++++----------------- docs/source/connector.rst | 26 ++-- docs/source/filereaders.rst | 104 ++++++------- pom.xml | 23 ++- 4 files changed, 220 insertions(+), 209 deletions(-) diff --git a/docs/source/config_options.rst b/docs/source/config_options.rst index 8a98739..67f2284 100644 --- a/docs/source/config_options.rst +++ b/docs/source/config_options.rst @@ -244,20 +244,6 @@ File readers Some file readers have custom properties to define and others don't. So, depending on the configuration you'll have to take into account their properties. -.. _config_options-filereaders-avro: - -Avro --------------------------------------------- - -In order to configure custom properties for this reader, the name you must use is ``avro``. - -``file_reader.avro.schema`` - Avro schema in JSON format to use when reading a file. - If not specified, the reader will use the schema defined in the file. - - * Type: string - * Importance: medium - .. _config_options-filereaders-parquet: Parquet @@ -277,6 +263,20 @@ In order to configure custom properties for this reader, the name you must use i * Type: string * Importance: medium +.. _config_options-filereaders-avro: + +Avro +-------------------------------------------- + +In order to configure custom properties for this reader, the name you must use is ``avro``. + +``file_reader.avro.schema`` + Avro schema in JSON format to use when reading a file. + If not specified, the reader will use the schema defined in the file. + + * Type: string + * Importance: medium + .. _config_options-filereaders-orc: ORC @@ -552,130 +552,6 @@ In order to configure custom properties for this reader, the name you must use i * Default: ```` * Importance: low -.. _config_options-filereaders-json: - -JSON --------------------------------------------- - -To configure custom properties for this reader, the name you must use is ``json``. - -``file_reader.json.record_per_line`` - If enabled, the reader will read each line as a record. Otherwise, the reader will read the full - content of the file as a record. - - * Type: boolean - * Default: ``true`` - * Importance: medium - -``file_reader.json.deserialization.`` - Deserialization feature to use when reading a JSON file. You can add as much as you like - based on the ones defined `here. `__ - - * Type: boolean - * Importance: medium - -``file_reader.json.encoding`` - Encoding to use for reading a file. If not specified, the reader will use the default encoding. - - * Type: string - * Default: based on the locale and charset of the underlying operating system. - * Importance: medium - -``file_reader.json.compression.type`` - Compression type to use when reading a file. - - * Type: enum (available values ``bzip2``, ``gzip`` and ``none``) - * Default: ``none`` - * Importance: medium - -``file_reader.json.compression.concatenated`` - Flag to specify if the decompression of the reader will finish at the end of the file or after - the first compressed stream. - - * Type: boolean - * Default: ``true`` - * Importance: low - -.. _config_options-filereaders-xml: - -XML --------------------------------------------- - -To configure custom properties for this reader, the name you must use is ``xml``. - -``file_reader.xml.record_per_line`` - If enabled, the reader will read each line as a record. Otherwise, the reader will read the full - content of the file as a record. - - * Type: boolean - * Default: ``true`` - * Importance: medium - -``file_reader.xml.deserialization.`` - Deserialization feature to use when reading a XML file. You can add as much as you like - based on the ones defined `here. `__ - - * Type: boolean - * Importance: medium - -``file_reader.xml.encoding`` - Encoding to use for reading a file. If not specified, the reader will use the default encoding. - - * Type: string - * Default: based on the locale and charset of the underlying operating system. - * Importance: medium - -``file_reader.xml.compression.type`` - Compression type to use when reading a file. - - * Type: enum (available values ``bzip2``, ``gzip`` and ``none``) - * Default: ``none`` - * Importance: medium - -``file_reader.xml.compression.concatenated`` - Flag to specify if the decompression of the reader will finish at the end of the file or after - the first compressed stream. - - * Type: boolean - * Default: ``true`` - * Importance: low - -.. _config_options-filereaders-yaml: - -YAML --------------------------------------------- - -To configure custom properties for this reader, the name you must use is ``yaml``. - -``file_reader.yaml.deserialization.`` - Deserialization feature to use when reading a YAML file. You can add as much as you like - based on the ones defined `here. `__ - - * Type: boolean - * Importance: medium - -``file_reader.yaml.encoding`` - Encoding to use for reading a file. If not specified, the reader will use the default encoding. - - * Type: string - * Default: based on the locale and charset of the underlying operating system. - * Importance: medium - -``file_reader.yaml.compression.type`` - Compression type to use when reading a file. - - * Type: enum (available values ``bzip2``, ``gzip`` and ``none``) - * Default: ``none`` - * Importance: medium - -``file_reader.yaml.compression.concatenated`` - Flag to specify if the decompression of the reader will finish at the end of the file or after - the first compressed stream. - - * Type: boolean - * Default: ``true`` - * Importance: low - .. _config_options-filereaders-csv: CSV @@ -1171,6 +1047,130 @@ To configure custom properties for this reader, the name you must use is ``delim * Default: ``true`` * Importance: low +.. _config_options-filereaders-json: + +JSON +-------------------------------------------- + +To configure custom properties for this reader, the name you must use is ``json``. + +``file_reader.json.record_per_line`` + If enabled, the reader will read each line as a record. Otherwise, the reader will read the full + content of the file as a record. + + * Type: boolean + * Default: ``true`` + * Importance: medium + +``file_reader.json.deserialization.`` + Deserialization feature to use when reading a JSON file. You can add as much as you like + based on the ones defined `here. `__ + + * Type: boolean + * Importance: medium + +``file_reader.json.encoding`` + Encoding to use for reading a file. If not specified, the reader will use the default encoding. + + * Type: string + * Default: based on the locale and charset of the underlying operating system. + * Importance: medium + +``file_reader.json.compression.type`` + Compression type to use when reading a file. + + * Type: enum (available values ``bzip2``, ``gzip`` and ``none``) + * Default: ``none`` + * Importance: medium + +``file_reader.json.compression.concatenated`` + Flag to specify if the decompression of the reader will finish at the end of the file or after + the first compressed stream. + + * Type: boolean + * Default: ``true`` + * Importance: low + +.. _config_options-filereaders-xml: + +XML +-------------------------------------------- + +To configure custom properties for this reader, the name you must use is ``xml``. + +``file_reader.xml.record_per_line`` + If enabled, the reader will read each line as a record. Otherwise, the reader will read the full + content of the file as a record. + + * Type: boolean + * Default: ``true`` + * Importance: medium + +``file_reader.xml.deserialization.`` + Deserialization feature to use when reading a XML file. You can add as much as you like + based on the ones defined `here. `__ + + * Type: boolean + * Importance: medium + +``file_reader.xml.encoding`` + Encoding to use for reading a file. If not specified, the reader will use the default encoding. + + * Type: string + * Default: based on the locale and charset of the underlying operating system. + * Importance: medium + +``file_reader.xml.compression.type`` + Compression type to use when reading a file. + + * Type: enum (available values ``bzip2``, ``gzip`` and ``none``) + * Default: ``none`` + * Importance: medium + +``file_reader.xml.compression.concatenated`` + Flag to specify if the decompression of the reader will finish at the end of the file or after + the first compressed stream. + + * Type: boolean + * Default: ``true`` + * Importance: low + +.. _config_options-filereaders-yaml: + +YAML +-------------------------------------------- + +To configure custom properties for this reader, the name you must use is ``yaml``. + +``file_reader.yaml.deserialization.`` + Deserialization feature to use when reading a YAML file. You can add as much as you like + based on the ones defined `here. `__ + + * Type: boolean + * Importance: medium + +``file_reader.yaml.encoding`` + Encoding to use for reading a file. If not specified, the reader will use the default encoding. + + * Type: string + * Default: based on the locale and charset of the underlying operating system. + * Importance: medium + +``file_reader.yaml.compression.type`` + Compression type to use when reading a file. + + * Type: enum (available values ``bzip2``, ``gzip`` and ``none``) + * Default: ``none`` + * Importance: medium + +``file_reader.yaml.compression.concatenated`` + Flag to specify if the decompression of the reader will finish at the end of the file or after + the first compressed stream. + + * Type: boolean + * Default: ``true`` + * Importance: low + .. _config_options-filereaders-text: Text diff --git a/docs/source/connector.rst b/docs/source/connector.rst index 94593f9..e88b552 100644 --- a/docs/source/connector.rst +++ b/docs/source/connector.rst @@ -4,6 +4,9 @@ Connector ******************************************** +Kafka Connect FileSystem Connector is a source connector for reading records from different sort of file +formats and from different file system types and load them into Kafka. + The connector takes advantage of the abstraction provided from `Hadoop Common `__ using the implementation of the ``org.apache.hadoop.fs.FileSystem`` class. So, it's possible to use a wide variety of FS or if your FS is not included in the Hadoop Common API you can implement an extension @@ -20,14 +23,17 @@ Among others, these are some file systems it supports: * Local File System. * Hadoop Archive File System. +On the other hand, the following file types are supported: Parquet, Avro, ORC, SequenceFile, Cobol / EBCDIC, +CSV, TSV, Fixed-width, JSON, XML, YAML and Text. + Getting started ============================================ Prerequisites -------------------------------------------- -- Apache Kafka 2.6.0 -- Java 8 +- Apache Kafka 2.6.0. +- Java 8. - Confluent Schema Registry (recommended). Building from source @@ -118,11 +124,11 @@ Policies In order to ingest data from the FS(s), the connector needs a **policy** to define the rules to do it. -Basically, the policy tries to connect to each FS included in ``fs.uris`` connector property, lists files +Basically, the policy tries to connect to each FS included in the ``fs.uris`` connector property, lists files (and filter them using the regular expression provided in the ``policy.regexp`` property) and enables -a file reader to read records from them. +a file reader to read records. -The policy to be used by the connector is defined in ``policy.class`` connector property. +The policy to be used by the connector is defined in the ``policy.class`` connector property. .. important:: When delivering records from the connector to Kafka, they contain their own file offset so, if in the next eventual policy execution this file is processed again, @@ -144,11 +150,11 @@ File readers They read files and process each record from the FS. The **file reader** is needed by the policy to enable the connector to process each record and includes in the implementation how to seek and iterate over the -records in the file. +records within the file. The file reader to be used when processing files is defined in the ``file_reader.class`` connector property. -In the same way as the policies, the connector provides several sort of readers to parse and read records +In the same way as policies, the connector provides several sort of readers to parse and read records for different file formats. If you don't have a file reader that fits your needs, just implement one with the unique restriction that it must implement the interface ``com.github.mmolimar.kafka.connect.fs.file.reader.FileReader``. @@ -157,15 +163,15 @@ The are several file readers included which can read the following file formats: * Parquet. * Avro. -* Cobol/EBCDIC. +* ORC. +* SequenceFile. +* Cobol / EBCDIC. * CSV. * TSV. * Fixed-width. * JSON. * XML. * YAML. -* ORC. -* SequenceFile. * Text. .. include:: filereaders.rst diff --git a/docs/source/filereaders.rst b/docs/source/filereaders.rst index 0519979..3301222 100644 --- a/docs/source/filereaders.rst +++ b/docs/source/filereaders.rst @@ -1,15 +1,3 @@ -Avro -^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ - -Files with `Avro `__ format can be read with this reader. - -The Avro schema is not needed due to is read from the file. The message sent -to Kafka is created by transforming the record by means of -`Confluent avro-converter `__ -API. - -More information about properties of this file reader :ref:`here`. - Parquet ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ @@ -19,13 +7,20 @@ The reader takes advantage of the Parquet-Avro API and uses the Parquet file as if it was an Avro file, so the message sent to Kafka is built in the same way as the Avro file reader does. -.. warning:: Seeking Parquet files is a heavy task because the reader has to - iterate over all records. If the policy processes the same file - over and over again and has to seek the file, the performance - can be affected. - More information about properties of this file reader :ref:`here`. +Avro +^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ + +Files with `Avro `__ format can be read with this reader. + +The Avro schema is not needed due to is read from the file. The message sent +to Kafka is created by transforming the record by means of +`Confluent avro-converter `__ +API. + +More information about properties of this file reader :ref:`here`. + ORC ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ @@ -46,10 +41,10 @@ SequenceFile ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ `Sequence files `__ are one kind of -the Hadoop file formats which are serialized in key/value pairs. +the Hadoop file formats which are serialized in key-value pairs. This reader can process this file format and build a Kafka message with the -key/value pair. These two values are named ``key`` and ``value`` in the message +key-value pair. These two values are named ``key`` and ``value`` in the message by default but you can customize these field names. More information about properties of this file reader :ref:`here`. @@ -65,37 +60,10 @@ translate it into a Kafka message with the schema. More information about properties of this file reader :ref:`here`. -JSON -^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ - -Reads JSON files which might contain multiple number of fields with their specified -data types. The schema for this sort of records is inferred reading the first record -and marked as optional in the schema all the fields contained. - -More information about properties of this file reader :ref:`here`. - -XML -^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ - -Reads XML files which might contain multiple number of fields with their specified -data types. The schema for this sort of records is inferred reading the first record -and marked as optional in the schema all the fields contained. - -More information about properties of this file reader :ref:`here`. - -YAML -^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ - -Reads YAML files which might contain multiple number of fields with their specified -data types. The schema for this sort of records is inferred reading the first record -and marked as optional in the schema all the fields contained. - -More information about properties of this file reader :ref:`here`. - CSV ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ -CSV file reader using a custom token to distinguish different columns on each line. +CSV file reader using a custom token to distinguish different columns in each line. It allows to distinguish a header in the files and set the name of their columns in the message sent to Kafka. If there is no header, the value of each column will be in @@ -109,7 +77,7 @@ More information about properties of this file reader :ref:`here`. +JSON +^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ + +Reads JSON files which might contain multiple number of fields with their specified +data types. The schema for this sort of records is inferred reading the first record +and marked as optional in the schema all the fields contained. + +More information about properties of this file reader :ref:`here`. + +XML +^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ + +Reads XML files which might contain multiple number of fields with their specified +data types. The schema for this sort of records is inferred reading the first record +and marked as optional in the schema all the fields contained. + +.. warning:: Take into account the current + `limitations `__. + +More information about properties of this file reader :ref:`here`. + +YAML +^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ + +Reads YAML files which might contain multiple number of fields with their specified +data types. The schema for this sort of records is inferred reading the first record +and marked as optional in the schema all the fields contained. + +More information about properties of this file reader :ref:`here`. + Text ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ @@ -145,8 +143,8 @@ Agnostic Actually, this reader is a wrapper of the readers listing above. It tries to read any kind of file format using an internal reader based on the file extension, -applying the proper one (Parquet, Avro, ORC, SequenceFile, CSV, TSV or Text). In case of no -extension has been matched, the Text file reader will be applied. +applying the proper one (Parquet, Avro, ORC, SequenceFile, Cobol / EBCDIC, CSV, TSV, FixedWidth, JSON, XML, +YAML, or Text). In case of no extension has been matched, the Text file reader will be applied. Default extensions for each format (configurable): @@ -155,12 +153,12 @@ Default extensions for each format (configurable): * ORC: ``.orc`` * SequenceFile: ``.seq`` * Cobol / EBCDIC: ``.dat`` -* JSON: ``.json`` -* XML: ``.xml`` -* YAML: ``.yaml`` * CSV: ``.csv`` * TSV: ``.tsv`` * FixedWidth: ``.fixed`` +* JSON: ``.json`` +* XML: ``.xml`` +* YAML: ``.yaml`` * Text: any other sort of file extension. More information about properties of this file reader :ref:`here`. diff --git a/pom.xml b/pom.xml index 665c837..da382b8 100644 --- a/pom.xml +++ b/pom.xml @@ -286,11 +286,18 @@ Kafka Connect FileSystem https://kafka-connect-fs.readthedocs.io https://github.com/mmolimar/kafka-connect-fs - + + + The following file types are supported: Parquet, Avro, ORC, SequenceFile, + Cobol / EBCDIC, CSV, TSV, Fixed-width, JSON, XML, YAML and Text. + + Also, the connector has built-in support for file systems such as HDFS, S3, + Google Cloud Storage, Azure Blob Storage, Azure Data Lake Store, FTP, SFTP and + local file system, among others. + ]]> https://github.com/mmolimar/kafka-connect-fs Mario Molina @@ -322,17 +329,17 @@ azure sftp ftp - txt + parquet + avro + orc + sequence + cobol csv tsv json xml yaml - avro - parquet - orc - sequence - cobol + txt From e9caa3b667ddced9b1793324f596e825e2996f5e Mon Sep 17 00:00:00 2001 From: Mario Molina Date: Sun, 13 Sep 2020 14:00:24 -0500 Subject: [PATCH 18/21] Upgrade Docker base images to 5.5.1 --- Dockerfile | 2 +- docker-compose.yml | 6 +-- docs/source/config_options.rst | 43 ++++++++++--------- docs/source/connector.rst | 8 +--- docs/source/filereaders.rst | 8 ++-- docs/source/index.rst | 2 +- docs/source/policies.rst | 2 +- pom.xml | 1 + .../fs/file/reader/AgnosticFileReader.java | 26 +++++------ 9 files changed, 47 insertions(+), 51 deletions(-) diff --git a/Dockerfile b/Dockerfile index 7ba7a9d..387153d 100644 --- a/Dockerfile +++ b/Dockerfile @@ -1,4 +1,4 @@ -FROM confluentinc/cp-kafka-connect-base:5.5.0 +FROM confluentinc/cp-kafka-connect-base:5.5.1 ARG PROJECT_VERSION ENV CONNECT_PLUGIN_PATH="/usr/share/java,/usr/share/confluent-hub-components" diff --git a/docker-compose.yml b/docker-compose.yml index 22c70b9..a55f262 100644 --- a/docker-compose.yml +++ b/docker-compose.yml @@ -1,7 +1,7 @@ version: '3' services: cp-zookeeper: - image: confluentinc/cp-zookeeper:5.5.0 + image: confluentinc/cp-zookeeper:5.5.1 hostname: zookeeper container_name: zookeeper ports: @@ -11,7 +11,7 @@ services: ZOOKEEPER_TICK_TIME: 2000 cp-kafka: - image: confluentinc/cp-kafka:5.5.0 + image: confluentinc/cp-kafka:5.5.1 hostname: kafka container_name: kafka depends_on: @@ -32,7 +32,7 @@ services: CONFLUENT_METRICS_ENABLE: 'false' cp-schema-registry: - image: confluentinc/cp-schema-registry:5.5.0 + image: confluentinc/cp-schema-registry:5.5.1 hostname: schema-registry container_name: schema-registry depends_on: diff --git a/docs/source/config_options.rst b/docs/source/config_options.rst index 67f2284..d82f238 100644 --- a/docs/source/config_options.rst +++ b/docs/source/config_options.rst @@ -39,7 +39,7 @@ General config properties for this connector. Comma-separated URIs of the FS(s). They can be URIs pointing directly to a file in the FS and also can be dynamic using expressions for modifying the URIs in runtime. These expressions have the form ``${XXX}`` where XXX represents a pattern from ``java.time.format.DateTimeFormatter`` - Java class. + `Java class `__. * Type: string * Importance: high @@ -99,7 +99,8 @@ General config properties for this connector. * Importance: medium ``policy.cleanup`` - Cleanup strategy to use when skipping files. + Cleanup strategy to use when skipping files. It's possible to move these files to another folder, remove them + or do nothing. * Type: enum (available values ``none``, ``move`` and ``delete``) * Default: ``none`` @@ -357,7 +358,7 @@ In order to configure custom properties for this reader, the name you must use i * Importance: medium ``file_reader.cobol.reader.ebcdic_code_page`` - Code page to be used for EBCDIC to ASCII/Unicode conversions. + Code page to be used for EBCDIC to ASCII / Unicode conversions. * Type: string * Default: ``common`` @@ -1257,46 +1258,46 @@ To configure custom properties for this reader, the name you must use is ``agnos * Default: ``dat`` * Importance: medium -``file_reader.agnostic.extensions.json`` - A comma-separated string list with the accepted extensions for JSON files. +``file_reader.agnostic.extensions.csv`` + A comma-separated string list with the accepted extensions for CSV files. * Type: string[] - * Default: ``json`` + * Default: ``csv`` * Importance: medium -``file_reader.agnostic.extensions.xml`` - A comma-separated string list with the accepted extensions for XML files. +``file_reader.agnostic.extensions.tsv`` + A comma-separated string list with the accepted extensions for TSV files. * Type: string[] - * Default: ``xml`` + * Default: ``tsv`` * Importance: medium -``file_reader.agnostic.extensions.yaml`` - A comma-separated string list with the accepted extensions for YAML files. +``file_reader.agnostic.extensions.fixed`` + A comma-separated string list with the accepted extensions for fixed-width files. * Type: string[] - * Default: ``yaml`` + * Default: ``fixed`` * Importance: medium -``file_reader.agnostic.extensions.csv`` - A comma-separated string list with the accepted extensions for CSV files. +``file_reader.agnostic.extensions.json`` + A comma-separated string list with the accepted extensions for JSON files. * Type: string[] - * Default: ``csv`` + * Default: ``json`` * Importance: medium -``file_reader.agnostic.extensions.tsv`` - A comma-separated string list with the accepted extensions for TSV files. +``file_reader.agnostic.extensions.xml`` + A comma-separated string list with the accepted extensions for XML files. * Type: string[] - * Default: ``tsv`` + * Default: ``xml`` * Importance: medium -``file_reader.agnostic.extensions.fixed`` - A comma-separated string list with the accepted extensions for fixed-width files. +``file_reader.agnostic.extensions.yaml`` + A comma-separated string list with the accepted extensions for YAML files. * Type: string[] - * Default: ``fixed`` + * Default: ``yaml`` * Importance: medium .. note:: The Agnostic reader uses the previous ones as inner readers. So, in case of using this diff --git a/docs/source/connector.rst b/docs/source/connector.rst index e88b552..f5e7891 100644 --- a/docs/source/connector.rst +++ b/docs/source/connector.rst @@ -4,10 +4,7 @@ Connector ******************************************** -Kafka Connect FileSystem Connector is a source connector for reading records from different sort of file -formats and from different file system types and load them into Kafka. - -The connector takes advantage of the abstraction provided from `Hadoop Common `__ +The connector takes advantage of the abstraction provided from `Hadoop Common `__ using the implementation of the ``org.apache.hadoop.fs.FileSystem`` class. So, it's possible to use a wide variety of FS or if your FS is not included in the Hadoop Common API you can implement an extension of this abstraction and using it in a transparent way. @@ -23,9 +20,6 @@ Among others, these are some file systems it supports: * Local File System. * Hadoop Archive File System. -On the other hand, the following file types are supported: Parquet, Avro, ORC, SequenceFile, Cobol / EBCDIC, -CSV, TSV, Fixed-width, JSON, XML, YAML and Text. - Getting started ============================================ diff --git a/docs/source/filereaders.rst b/docs/source/filereaders.rst index 3301222..d1297a2 100644 --- a/docs/source/filereaders.rst +++ b/docs/source/filereaders.rst @@ -52,7 +52,7 @@ More information about properties of this file reader :ref:`here`__ parser. By means of the corresponding copybook -representing its schema-, it parses each record and @@ -77,7 +77,7 @@ More information about properties of this file reader :ref:`here`__. @@ -113,7 +113,7 @@ data types. The schema for this sort of records is inferred reading the first re and marked as optional in the schema all the fields contained. .. warning:: Take into account the current - `limitations `__. + `limitations `__. More information about properties of this file reader :ref:`here`. diff --git a/docs/source/index.rst b/docs/source/index.rst index cd5bdb9..2ed199c 100644 --- a/docs/source/index.rst +++ b/docs/source/index.rst @@ -11,7 +11,7 @@ The connector supports: * Several sort of File Systems (FS) to use. * Dynamic and static URIs to ingest data from. -* Policies to define rules about how to look for files. +* Policies to define rules about how to look for files and clean them up after processing. * File readers to parse and read different kind of file formats. To learn more about the connector you can read :ref:`this section` and for more detailed diff --git a/docs/source/policies.rst b/docs/source/policies.rst index 1a5f654..3225de6 100644 --- a/docs/source/policies.rst +++ b/docs/source/policies.rst @@ -17,7 +17,7 @@ Cron ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ This policy is scheduled based on cron expressions and their format to put in the configuration -are based on the library `Quartz Scheduler `__ +are based on the library `Quartz Scheduler `__. After finishing each execution, the policy gets slept until the next one is scheduled, if applicable. diff --git a/pom.xml b/pom.xml index da382b8..6cc4834 100644 --- a/pom.xml +++ b/pom.xml @@ -336,6 +336,7 @@ cobol csv tsv + fixed json xml yaml diff --git a/src/main/java/com/github/mmolimar/kafka/connect/fs/file/reader/AgnosticFileReader.java b/src/main/java/com/github/mmolimar/kafka/connect/fs/file/reader/AgnosticFileReader.java index e9c3e77..8761a9d 100644 --- a/src/main/java/com/github/mmolimar/kafka/connect/fs/file/reader/AgnosticFileReader.java +++ b/src/main/java/com/github/mmolimar/kafka/connect/fs/file/reader/AgnosticFileReader.java @@ -34,7 +34,7 @@ public class AgnosticFileReader extends AbstractFileReader reader; private Set parquetExtensions, avroExtensions, sequenceExtensions, orcExtensions, cobolExtensions, - jsonExtensions, xmlExtensions, yamlExtensions, csvExtensions, tsvExtensions, fixedExtensions; + csvExtensions, tsvExtensions, fixedExtensions, jsonExtensions, xmlExtensions, yamlExtensions; public AgnosticFileReader(FileSystem fs, Path filePath, Map config) throws Exception { super(fs, filePath, new AgnosticAdapter(), config); @@ -62,18 +62,18 @@ private AbstractFileReader readerByExtension(FileSystem fs, Path filePat clz = OrcFileReader.class; } else if (cobolExtensions.contains(extension)) { clz = CobolFileReader.class; - } else if (jsonExtensions.contains(extension)) { - clz = JsonFileReader.class; - } else if (xmlExtensions.contains(extension)) { - clz = XmlFileReader.class; - } else if (yamlExtensions.contains(extension)) { - clz = YamlFileReader.class; } else if (csvExtensions.contains(extension)) { clz = CsvFileReader.class; } else if (tsvExtensions.contains(extension)) { clz = TsvFileReader.class; } else if (fixedExtensions.contains(extension)) { clz = FixedWidthFileReader.class; + } else if (jsonExtensions.contains(extension)) { + clz = JsonFileReader.class; + } else if (xmlExtensions.contains(extension)) { + clz = XmlFileReader.class; + } else if (yamlExtensions.contains(extension)) { + clz = YamlFileReader.class; } else { clz = TextFileReader.class; } @@ -93,18 +93,18 @@ protected void configure(Map config) { .toLowerCase().split(",")).collect(Collectors.toSet()); this.cobolExtensions = Arrays.stream(config.getOrDefault(FILE_READER_AGNOSTIC_EXTENSIONS_COBOL, "dat") .toLowerCase().split(",")).collect(Collectors.toSet()); - this.jsonExtensions = Arrays.stream(config.getOrDefault(FILE_READER_AGNOSTIC_EXTENSIONS_JSON, "json") - .toLowerCase().split(",")).collect(Collectors.toSet()); - this.xmlExtensions = Arrays.stream(config.getOrDefault(FILE_READER_AGNOSTIC_EXTENSIONS_XML, "xml") - .toLowerCase().split(",")).collect(Collectors.toSet()); - this.yamlExtensions = Arrays.stream(config.getOrDefault(FILE_READER_AGNOSTIC_EXTENSIONS_YAML, "yaml") - .toLowerCase().split(",")).collect(Collectors.toSet()); this.csvExtensions = Arrays.stream(config.getOrDefault(FILE_READER_AGNOSTIC_EXTENSIONS_CSV, "csv") .toLowerCase().split(",")).collect(Collectors.toSet()); this.tsvExtensions = Arrays.stream(config.getOrDefault(FILE_READER_AGNOSTIC_EXTENSIONS_TSV, "tsv") .toLowerCase().split(",")).collect(Collectors.toSet()); this.fixedExtensions = Arrays.stream(config.getOrDefault(FILE_READER_AGNOSTIC_EXTENSIONS_FIXED, "fixed") .toLowerCase().split(",")).collect(Collectors.toSet()); + this.jsonExtensions = Arrays.stream(config.getOrDefault(FILE_READER_AGNOSTIC_EXTENSIONS_JSON, "json") + .toLowerCase().split(",")).collect(Collectors.toSet()); + this.xmlExtensions = Arrays.stream(config.getOrDefault(FILE_READER_AGNOSTIC_EXTENSIONS_XML, "xml") + .toLowerCase().split(",")).collect(Collectors.toSet()); + this.yamlExtensions = Arrays.stream(config.getOrDefault(FILE_READER_AGNOSTIC_EXTENSIONS_YAML, "yaml") + .toLowerCase().split(",")).collect(Collectors.toSet()); } @Override From 9e9ee5aa91545ad947ba273437b7c889d60de4f9 Mon Sep 17 00:00:00 2001 From: Mario Molina Date: Mon, 14 Sep 2020 18:59:18 -0500 Subject: [PATCH 19/21] Upgrade deps --- pom.xml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pom.xml b/pom.xml index 6cc4834..f5c64de 100644 --- a/pom.xml +++ b/pom.xml @@ -49,7 +49,7 @@ 2.6.0 5.5.1 3.3.0 - hadoop3-2.1.4 + hadoop3-2.1.5 1.11.1 1.6.3 2.9.0 @@ -58,7 +58,7 @@ 2.12.12 9.1.1 0.1.55 - 5.6.2 + 5.7.0 4.2 2.0.7 1.8 From c690a11e03d438967bd1150e407c2a1e714c0ffb Mon Sep 17 00:00:00 2001 From: Mario Molina Date: Mon, 14 Sep 2020 20:38:47 -0500 Subject: [PATCH 20/21] Source directories for Coveralls plugin --- pom.xml | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/pom.xml b/pom.xml index f5c64de..4ae7391 100644 --- a/pom.xml +++ b/pom.xml @@ -271,6 +271,12 @@ org.eluder.coveralls coveralls-maven-plugin ${maven-coveralls-plugin.version} + + + ${basedir}/src/main/java + ${basedir}/src/main/scala + + io.confluent From 1655831e4054e5737ab557cda80036499a9ea67b Mon Sep 17 00:00:00 2001 From: Mario Molina Date: Mon, 14 Sep 2020 21:00:05 -0500 Subject: [PATCH 21/21] Release version 1.2.0 --- docker-compose.yml | 2 +- pom.xml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docker-compose.yml b/docker-compose.yml index a55f262..cbbc7bd 100644 --- a/docker-compose.yml +++ b/docker-compose.yml @@ -45,7 +45,7 @@ services: SCHEMA_REGISTRY_KAFKASTORE_CONNECTION_URL: 'zookeeper:2181' connect-fs: - image: mmolimar/kafka-connect-fs:1.2.0-SNAPSHOT + image: mmolimar/kafka-connect-fs:1.2.0 container_name: connect depends_on: - cp-kafka diff --git a/pom.xml b/pom.xml index 4ae7391..0afe9f9 100644 --- a/pom.xml +++ b/pom.xml @@ -4,7 +4,7 @@ com.github.mmolimar.kafka.connect kafka-connect-fs - 1.2.0-SNAPSHOT + 1.2.0 jar kafka-connect-fs