From 06385b36d3ad71d17f2a4b169e560d1412895c14 Mon Sep 17 00:00:00 2001 From: Florian Hussonnois Date: Wed, 10 Feb 2021 19:21:40 +0100 Subject: [PATCH] refactor(filesystems): add module filepulse-commons-fs This commit relocates some classes --- checkstyle/suppressions.xml | 13 +- .../connect-file-pulse-quickstart-avro.json | 6 +- config/connect-file-pulse-quickstart-csv.json | 6 +- .../connect-file-pulse-quickstart-log4j.json | 8 +- .../filepulse/internal/StringUtils.java | 6 + .../reader/AbstractFileInputIterator.java | 14 +- .../reader/AbstractFileInputReader.java | 37 +- .../filepulse/reader/FileInputIterator.java | 2 +- .../filepulse/reader/FileInputReader.java | 4 +- .../filepulse}/reader/IteratorManager.java | 8 +- .../filepulse/source/FileInputIterable.java | 14 +- .../filepulse/source/FileObjectMeta.java | 6 +- .../filepulse-commons-fs/pom.xml | 48 +++ .../fs/reader/IndexRecordOffset.java} | 23 +- .../fs/reader/avro/AvroDataFileIterator.java | 159 +++++++ .../reader/avro/AvroDataStreamIterator.java | 123 ++++++ .../fs/reader/avro}/AvroRecordOffset.java | 7 +- .../reader/avro/AvroTypedStructConverter.java | 173 ++++++++ .../fs/reader/text}/BytesRecordOffset.java | 19 +- .../reader/text}/NonBlockingBufferReader.java | 47 +- .../fs/reader/text/RowFileInputIterator.java | 191 ++++++++ .../text/RowFileInputIteratorBuilder.java | 112 +++++ .../text/RowFileInputIteratorDecorator.java | 84 ++++ .../text}/RowFileInputReaderConfig.java | 22 +- .../fs/reader/text}/RowFileRecordOffset.java | 7 +- .../text/RowFileWithFooterInputIterator.java | 109 +++++ .../text/RowFileWithHeadersInputIterator.java | 97 +++++ .../internal/ReversedInputFileReader.java | 4 +- .../fs/reader/text}/internal/TextBlock.java | 7 +- .../fs/reader/xml/XMLFileInputIterator.java} | 224 +++++----- .../reader/xml}/XMLFileInputReaderConfig.java | 13 +- .../reader/avro/AvroDataFileIteratorTest.java | 44 ++ .../avro/AvroDataStreamIteratorTest.java | 44 ++ .../reader/avro/AvroRecordConverterTest.java | 136 ++++++ .../reader/avro/BaseAvroDataIteratorTest.java | 139 ++++++ .../text}/NonBlockingBufferReaderTest.java | 12 +- .../text}/RowFileInputIteratorTest.java | 13 +- .../internal/ReverseInputFileReaderTest.java | 2 +- .../reader/xml/XMLFileInputIteratorTest.java | 282 ++++++++++++ .../filepulse-local-fs/pom.xml | 22 +- .../fs/{local => }/DeleteCleanupPolicy.java | 2 +- .../{local => }/LocalFSDirectoryListing.java | 9 +- .../LocalFSDirectoryListingConfig.java | 2 +- .../fs/{local => }/MoveCleanupPolicy.java | 2 +- .../fs/{local => }/codec/CodecHandler.java | 2 +- .../{local => }/codec/CodecHandlerUtils.java | 2 +- .../fs/{local => }/codec/CodecManager.java | 2 +- .../fs/{local => }/codec/GZipCodec.java | 2 +- .../fs/{local => }/codec/TarballCodec.java | 2 +- .../fs/{local => }/codec/ZipCodec.java | 2 +- .../filter/IgnoreHiddenFileListFilter.java | 2 +- .../fs/local/reader/AvroFileInputReader.java | 328 -------------- .../fs/local/reader/RowFileInputIterator.java | 408 ------------------ .../fs/reader/BaseLocalFileInputReader.java | 47 ++ .../fs/reader/LocalAvroFileInputReader.java | 57 +++ .../LocalBytesArrayInputReader.java} | 37 +- .../LocalRowFileInputReader.java} | 36 +- .../fs/reader/LocalXMLFileInputReader.java | 72 ++++ .../{local => }/DeleteCleanupPolicyTest.java | 2 +- .../LocalFSDirectoryListingTest.java | 3 +- .../fs/{local => }/MoveCleanupPolicyTest.java | 2 +- .../local/reader/AvroFileInputReaderTest.java | 242 ----------- .../local/reader/XMLFileInputReaderTest.java | 283 ------------ .../reader/BytesArrayInputReaderTest.java | 6 +- connect-file-pulse-filesystems/pom.xml | 11 +- ...ickstart-connect-file-pulse-csv.properties | 4 +- ...kstart-connect-file-pulse-log4j.properties | 4 +- .../filepulse/config/CommonConfig.java | 4 +- .../filepulse/config/ConnectorConfig.java | 2 +- .../fs}/reader/FileInputMetadataReader.java | 29 +- .../DefaultFileRecordsPollingConsumer.java | 3 +- pom.xml | 20 +- 72 files changed, 2277 insertions(+), 1639 deletions(-) rename {connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/local => connect-file-pulse-api/src/main/java/io/streamthoughts/kafka/connect/filepulse}/reader/AbstractFileInputIterator.java (84%) rename {connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/local => connect-file-pulse-api/src/main/java/io/streamthoughts/kafka/connect/filepulse}/reader/AbstractFileInputReader.java (69%) rename {connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/local => connect-file-pulse-api/src/main/java/io/streamthoughts/kafka/connect/filepulse}/reader/IteratorManager.java (90%) create mode 100644 connect-file-pulse-filesystems/filepulse-commons-fs/pom.xml rename connect-file-pulse-filesystems/{filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/local/reader/XMLRecordOffset.java => filepulse-commons-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/IndexRecordOffset.java} (68%) create mode 100644 connect-file-pulse-filesystems/filepulse-commons-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/avro/AvroDataFileIterator.java create mode 100644 connect-file-pulse-filesystems/filepulse-commons-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/avro/AvroDataStreamIterator.java rename connect-file-pulse-filesystems/{filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/local/reader => filepulse-commons-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/avro}/AvroRecordOffset.java (95%) create mode 100644 connect-file-pulse-filesystems/filepulse-commons-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/avro/AvroTypedStructConverter.java rename connect-file-pulse-filesystems/{filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/local/reader => filepulse-commons-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/text}/BytesRecordOffset.java (86%) rename connect-file-pulse-filesystems/{filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/local/reader/internal => filepulse-commons-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/text}/NonBlockingBufferReader.java (90%) create mode 100644 connect-file-pulse-filesystems/filepulse-commons-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/text/RowFileInputIterator.java create mode 100644 connect-file-pulse-filesystems/filepulse-commons-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/text/RowFileInputIteratorBuilder.java create mode 100644 connect-file-pulse-filesystems/filepulse-commons-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/text/RowFileInputIteratorDecorator.java rename connect-file-pulse-filesystems/{filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/local/reader => filepulse-commons-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/text}/RowFileInputReaderConfig.java (87%) rename connect-file-pulse-filesystems/{filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/local/reader => filepulse-commons-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/text}/RowFileRecordOffset.java (95%) create mode 100644 connect-file-pulse-filesystems/filepulse-commons-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/text/RowFileWithFooterInputIterator.java create mode 100644 connect-file-pulse-filesystems/filepulse-commons-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/text/RowFileWithHeadersInputIterator.java rename connect-file-pulse-filesystems/{filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/local/reader => filepulse-commons-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/text}/internal/ReversedInputFileReader.java (98%) rename connect-file-pulse-filesystems/{filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/local/reader => filepulse-commons-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/text}/internal/TextBlock.java (95%) rename connect-file-pulse-filesystems/{filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/local/reader/XMLFileInputReader.java => filepulse-commons-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/xml/XMLFileInputIterator.java} (64%) rename connect-file-pulse-filesystems/{filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/local/reader => filepulse-commons-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/xml}/XMLFileInputReaderConfig.java (90%) create mode 100644 connect-file-pulse-filesystems/filepulse-commons-fs/src/test/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/avro/AvroDataFileIteratorTest.java create mode 100644 connect-file-pulse-filesystems/filepulse-commons-fs/src/test/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/avro/AvroDataStreamIteratorTest.java create mode 100644 connect-file-pulse-filesystems/filepulse-commons-fs/src/test/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/avro/AvroRecordConverterTest.java create mode 100644 connect-file-pulse-filesystems/filepulse-commons-fs/src/test/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/avro/BaseAvroDataIteratorTest.java rename connect-file-pulse-filesystems/{filepulse-local-fs/src/test/java/io/streamthoughts/kafka/connect/filepulse/fs/local/reader/internal => filepulse-commons-fs/src/test/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/text}/NonBlockingBufferReaderTest.java (93%) rename connect-file-pulse-filesystems/{filepulse-local-fs/src/test/java/io/streamthoughts/kafka/connect/filepulse/fs/local/reader => filepulse-commons-fs/src/test/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/text}/RowFileInputIteratorTest.java (85%) rename connect-file-pulse-filesystems/{filepulse-local-fs/src/test/java/io/streamthoughts/kafka/connect/filepulse/fs/local/reader => filepulse-commons-fs/src/test/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/text}/internal/ReverseInputFileReaderTest.java (98%) create mode 100644 connect-file-pulse-filesystems/filepulse-commons-fs/src/test/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/xml/XMLFileInputIteratorTest.java rename connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/{local => }/DeleteCleanupPolicy.java (97%) rename connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/{local => }/LocalFSDirectoryListing.java (93%) rename connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/{local => }/LocalFSDirectoryListingConfig.java (97%) rename connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/{local => }/MoveCleanupPolicy.java (99%) rename connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/{local => }/codec/CodecHandler.java (96%) rename connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/{local => }/codec/CodecHandlerUtils.java (97%) rename connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/{local => }/codec/CodecManager.java (96%) rename connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/{local => }/codec/GZipCodec.java (97%) rename connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/{local => }/codec/TarballCodec.java (98%) rename connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/{local => }/codec/ZipCodec.java (97%) rename connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/{local => }/filter/IgnoreHiddenFileListFilter.java (96%) delete mode 100644 connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/local/reader/AvroFileInputReader.java delete mode 100644 connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/local/reader/RowFileInputIterator.java create mode 100644 connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/BaseLocalFileInputReader.java create mode 100644 connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/LocalAvroFileInputReader.java rename connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/{local/reader/BytesArrayInputReader.java => reader/LocalBytesArrayInputReader.java} (73%) rename connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/{local/reader/RowFileInputReader.java => reader/LocalRowFileInputReader.java} (57%) create mode 100644 connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/LocalXMLFileInputReader.java rename connect-file-pulse-filesystems/filepulse-local-fs/src/test/java/io/streamthoughts/kafka/connect/filepulse/fs/{local => }/DeleteCleanupPolicyTest.java (97%) rename connect-file-pulse-filesystems/filepulse-local-fs/src/test/java/io/streamthoughts/kafka/connect/filepulse/fs/{local => }/LocalFSDirectoryListingTest.java (97%) rename connect-file-pulse-filesystems/filepulse-local-fs/src/test/java/io/streamthoughts/kafka/connect/filepulse/fs/{local => }/MoveCleanupPolicyTest.java (98%) delete mode 100644 connect-file-pulse-filesystems/filepulse-local-fs/src/test/java/io/streamthoughts/kafka/connect/filepulse/fs/local/reader/AvroFileInputReaderTest.java delete mode 100644 connect-file-pulse-filesystems/filepulse-local-fs/src/test/java/io/streamthoughts/kafka/connect/filepulse/fs/local/reader/XMLFileInputReaderTest.java rename connect-file-pulse-filesystems/filepulse-local-fs/src/test/java/io/streamthoughts/kafka/connect/filepulse/fs/{local => }/reader/BytesArrayInputReaderTest.java (94%) rename {connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/local => connect-file-pulse-plugin/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs}/reader/FileInputMetadataReader.java (78%) diff --git a/checkstyle/suppressions.xml b/checkstyle/suppressions.xml index 9169ce109..509b225b1 100644 --- a/checkstyle/suppressions.xml +++ b/checkstyle/suppressions.xml @@ -41,21 +41,10 @@ */ --> - - - - - - - - - - - - + diff --git a/config/connect-file-pulse-quickstart-avro.json b/config/connect-file-pulse-quickstart-avro.json index 41f1f7e06..cd6bdd937 100644 --- a/config/connect-file-pulse-quickstart-avro.json +++ b/config/connect-file-pulse-quickstart-avro.json @@ -2,14 +2,14 @@ "config": { "connector.class": "io.streamthoughts.kafka.connect.filepulse.source.FilePulseSourceConnector", "fs.cleanup.policy.class": "io.streamthoughts.kafka.connect.filepulse.clean.LogCleanupPolicy", - "fs.listing.class": "io.streamthoughts.kafka.connect.filepulse.fs.local.LocalFSDirectoryListing", - "fs.scan.directory.path": "/tmp/kafka-connect/examples/", + "fs.listing.class" : "io.streamthoughts.kafka.connect.filepulse.fs.LocalFSDirectoryListing", + "fs.listing.directory.path":"/tmp/kafka-connect/examples/", "fs.scan.interval.ms": "10000", "internal.kafka.reporter.bootstrap.servers": "broker:29092", "internal.kafka.reporter.topic": "connect-file-pulse-status", "offset.strategy": "name", "read.max.wait.ms": "5000", - "task.reader.class": "io.streamthoughts.kafka.connect.filepulse.fs.local.reader.AvroFileInputReader", + "task.reader.class": "io.streamthoughts.kafka.connect.filepulse.fs.reader.LocalAvroFileInputReader", "topic": "connect-file-pulse-quickstart-avro", "tasks.max": 1 }, diff --git a/config/connect-file-pulse-quickstart-csv.json b/config/connect-file-pulse-quickstart-csv.json index 0b8a720f1..0ac89ee6f 100644 --- a/config/connect-file-pulse-quickstart-csv.json +++ b/config/connect-file-pulse-quickstart-csv.json @@ -9,8 +9,10 @@ "filters.ParseDelimitedRow.trimColumn": "true", "filters.ParseDelimitedRow.type": "io.streamthoughts.kafka.connect.filepulse.filter.DelimitedRowFilter", "fs.cleanup.policy.class": "io.streamthoughts.kafka.connect.filepulse.clean.LogCleanupPolicy", - "fs.listing.class": "io.streamthoughts.kafka.connect.filepulse.fs.local.LocalFSDirectoryListing", - "fs.scan.directory.path": "/tmp/kafka-connect/examples/", + "fs.listing.class" : "io.streamthoughts.kafka.connect.filepulse.fs.LocalFSDirectoryListing", + "fs.listing.directory.path":"/tmp/kafka-connect/examples/", + "fs.listing.filters":"io.streamthoughts.kafka.connect.filepulse.fs.filter.RegexFileListFilter", + "file.filter.regex.pattern":".*\\.csv$", "fs.scan.interval.ms": "10000", "internal.kafka.reporter.bootstrap.servers": "broker:29092", "internal.kafka.reporter.topic": "connect-file-pulse-status", diff --git a/config/connect-file-pulse-quickstart-log4j.json b/config/connect-file-pulse-quickstart-log4j.json index a5356f604..1f5af615c 100644 --- a/config/connect-file-pulse-quickstart-log4j.json +++ b/config/connect-file-pulse-quickstart-log4j.json @@ -11,14 +11,16 @@ "filters.ParseLog4jLog.type": "io.streamthoughts.kafka.connect.filepulse.filter.GrokFilter", "filters.ParseLog4jLog.ignoreFailure": "true", "fs.cleanup.policy.class": "io.streamthoughts.kafka.connect.filepulse.clean.LogCleanupPolicy", - "fs.listing.class": "io.streamthoughts.kafka.connect.filepulse.fs.local.LocalFSDirectoryListing", - "fs.scan.directory.path": "/var/log/kafka/", + "fs.listing.class" : "io.streamthoughts.kafka.connect.filepulse.fs.LocalFSDirectoryListing", + "fs.listing.directory.path":"/tmp/kafka-connect/examples/", + "fs.listing.filters":"io.streamthoughts.kafka.connect.filepulse.fs.filter.RegexFileListFilter", + "file.filter.regex.pattern":".*\\.log$", "fs.scan.interval.ms": "10000", "internal.kafka.reporter.bootstrap.servers": "broker:29092", "internal.kafka.reporter.topic": "connect-file-pulse-status", "offset.strategy": "name", "read.max.wait.ms": "5000", - "task.reader.class": "io.streamthoughts.kafka.connect.filepulse.fs.local.reader.RowFileInputReader", + "task.reader.class": "io.streamthoughts.kafka.connect.filepulse.fs.reader.LocalRowFileInputReader", "topic": "connect-file-pulse-quickstart-log4j", "tasks.max": 1 }, diff --git a/connect-file-pulse-api/src/main/java/io/streamthoughts/kafka/connect/filepulse/internal/StringUtils.java b/connect-file-pulse-api/src/main/java/io/streamthoughts/kafka/connect/filepulse/internal/StringUtils.java index 2eb7ae365..10847ada6 100644 --- a/connect-file-pulse-api/src/main/java/io/streamthoughts/kafka/connect/filepulse/internal/StringUtils.java +++ b/connect-file-pulse-api/src/main/java/io/streamthoughts/kafka/connect/filepulse/internal/StringUtils.java @@ -18,6 +18,8 @@ */ package io.streamthoughts.kafka.connect.filepulse.internal; +import java.util.Objects; + public class StringUtils { /** @@ -35,4 +37,8 @@ public static boolean isFastSplit(final String regex) { (ch < Character.MIN_HIGH_SURROGATE || ch > Character.MAX_LOW_SURROGATE); } + + public static boolean isNotBlank(final String string) { + return !(Objects.isNull(string) || string.isBlank()); + } } diff --git a/connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/local/reader/AbstractFileInputIterator.java b/connect-file-pulse-api/src/main/java/io/streamthoughts/kafka/connect/filepulse/reader/AbstractFileInputIterator.java similarity index 84% rename from connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/local/reader/AbstractFileInputIterator.java rename to connect-file-pulse-api/src/main/java/io/streamthoughts/kafka/connect/filepulse/reader/AbstractFileInputIterator.java index 8246ce556..f62ae2a3c 100644 --- a/connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/local/reader/AbstractFileInputIterator.java +++ b/connect-file-pulse-api/src/main/java/io/streamthoughts/kafka/connect/filepulse/reader/AbstractFileInputIterator.java @@ -1,5 +1,5 @@ /* - * Copyright 2019-2020 StreamThoughts. + * Copyright 2019-2021 StreamThoughts. * * Licensed to the Apache Software Foundation (ASF) under one or more * contributor license agreements. See the NOTICE file distributed with @@ -16,9 +16,8 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.streamthoughts.kafka.connect.filepulse.fs.local.reader; +package io.streamthoughts.kafka.connect.filepulse.reader; -import io.streamthoughts.kafka.connect.filepulse.reader.FileInputIterator; import io.streamthoughts.kafka.connect.filepulse.source.FileContext; import io.streamthoughts.kafka.connect.filepulse.source.FileRecord; @@ -47,13 +46,14 @@ public abstract class AbstractFileInputIterator implements FileInputIterator< */ public AbstractFileInputIterator(final IteratorManager iteratorManager, final FileContext context) { - Objects.requireNonNull(iteratorManager, "iteratorManager can't be null"); - Objects.requireNonNull(context, "context can't be null"); - this.iteratorManager = iteratorManager; - this.context = context; + this.iteratorManager = Objects.requireNonNull(iteratorManager, "iteratorManager can't be null"); + this.context = Objects.requireNonNull(context, "context can't be null"); closed = new AtomicBoolean(false); } + /** + * {@inheritDoc} + */ @Override public FileContext context() { return context; diff --git a/connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/local/reader/AbstractFileInputReader.java b/connect-file-pulse-api/src/main/java/io/streamthoughts/kafka/connect/filepulse/reader/AbstractFileInputReader.java similarity index 69% rename from connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/local/reader/AbstractFileInputReader.java rename to connect-file-pulse-api/src/main/java/io/streamthoughts/kafka/connect/filepulse/reader/AbstractFileInputReader.java index ca7645387..0a120a516 100644 --- a/connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/local/reader/AbstractFileInputReader.java +++ b/connect-file-pulse-api/src/main/java/io/streamthoughts/kafka/connect/filepulse/reader/AbstractFileInputReader.java @@ -16,21 +16,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.streamthoughts.kafka.connect.filepulse.fs.local.reader; +package io.streamthoughts.kafka.connect.filepulse.reader; import io.streamthoughts.kafka.connect.filepulse.data.TypedStruct; -import io.streamthoughts.kafka.connect.filepulse.reader.FileInputIterator; -import io.streamthoughts.kafka.connect.filepulse.reader.FileInputReader; import io.streamthoughts.kafka.connect.filepulse.source.FileContext; import io.streamthoughts.kafka.connect.filepulse.source.FileRecord; -import io.streamthoughts.kafka.connect.filepulse.source.LocalFileObjectMeta; -import io.streamthoughts.kafka.connect.filepulse.source.FileObjectMeta; -import java.io.File; -import java.net.URI; -import java.nio.file.Files; -import java.nio.file.Paths; -import java.util.Map; import java.util.concurrent.atomic.AtomicBoolean; public abstract class AbstractFileInputReader implements FileInputReader { @@ -42,35 +33,11 @@ public abstract class AbstractFileInputReader implements FileInputReader { /** * Creates a new {@link AbstractFileInputReader} instance. */ - AbstractFileInputReader() { + public AbstractFileInputReader() { this.isClosed = new AtomicBoolean(false); this.openIterators = new IteratorManager(); } - /** - * {@inheritDoc} - */ - @Override - public void configure(final Map configs) { - - } - - /** - * {@inheritDoc} - */ - @Override - public FileObjectMeta readMetadata(final URI fileURI) { - return new LocalFileObjectMeta(new File(fileURI)); - } - - /** - * {@inheritDoc} - */ - @Override - public boolean isReadable(final URI fileURI) { - return Files.isReadable(Paths.get(fileURI)); - } - /** * {@inheritDoc} */ diff --git a/connect-file-pulse-api/src/main/java/io/streamthoughts/kafka/connect/filepulse/reader/FileInputIterator.java b/connect-file-pulse-api/src/main/java/io/streamthoughts/kafka/connect/filepulse/reader/FileInputIterator.java index ff0749c35..e95ec696e 100644 --- a/connect-file-pulse-api/src/main/java/io/streamthoughts/kafka/connect/filepulse/reader/FileInputIterator.java +++ b/connect-file-pulse-api/src/main/java/io/streamthoughts/kafka/connect/filepulse/reader/FileInputIterator.java @@ -28,7 +28,7 @@ * * @param type of value. */ -public interface FileInputIterator extends Iterator> { +public interface FileInputIterator extends Iterator>, AutoCloseable { /** * Gets the iterator context. diff --git a/connect-file-pulse-api/src/main/java/io/streamthoughts/kafka/connect/filepulse/reader/FileInputReader.java b/connect-file-pulse-api/src/main/java/io/streamthoughts/kafka/connect/filepulse/reader/FileInputReader.java index 5097ffd8a..993337db9 100644 --- a/connect-file-pulse-api/src/main/java/io/streamthoughts/kafka/connect/filepulse/reader/FileInputReader.java +++ b/connect-file-pulse-api/src/main/java/io/streamthoughts/kafka/connect/filepulse/reader/FileInputReader.java @@ -38,7 +38,9 @@ public interface FileInputReader extends Configurable, AutoCloseable { * @param configs the reader configuration. */ @Override - void configure(final Map configs); + default void configure(final Map configs) { + + } /** * Gets the metadata for the source object identified by the given {@link URI}. diff --git a/connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/local/reader/IteratorManager.java b/connect-file-pulse-api/src/main/java/io/streamthoughts/kafka/connect/filepulse/reader/IteratorManager.java similarity index 90% rename from connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/local/reader/IteratorManager.java rename to connect-file-pulse-api/src/main/java/io/streamthoughts/kafka/connect/filepulse/reader/IteratorManager.java index 71f455deb..2bd928e06 100644 --- a/connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/local/reader/IteratorManager.java +++ b/connect-file-pulse-api/src/main/java/io/streamthoughts/kafka/connect/filepulse/reader/IteratorManager.java @@ -16,9 +16,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.streamthoughts.kafka.connect.filepulse.fs.local.reader; - -import io.streamthoughts.kafka.connect.filepulse.reader.FileInputIterator; +package io.streamthoughts.kafka.connect.filepulse.reader; import java.util.Collections; import java.util.HashSet; @@ -27,14 +25,14 @@ /** * Default class to easily close all open {@link FileInputIterator} instances. */ -class IteratorManager { +public class IteratorManager { private final Set> openIterators; /** * Creates a new {@link IteratorManager} instance. */ - IteratorManager() { + public IteratorManager() { this.openIterators = Collections.synchronizedSet(new HashSet<>()); } diff --git a/connect-file-pulse-api/src/main/java/io/streamthoughts/kafka/connect/filepulse/source/FileInputIterable.java b/connect-file-pulse-api/src/main/java/io/streamthoughts/kafka/connect/filepulse/source/FileInputIterable.java index f4a85efe8..e6fdbb571 100644 --- a/connect-file-pulse-api/src/main/java/io/streamthoughts/kafka/connect/filepulse/source/FileInputIterable.java +++ b/connect-file-pulse-api/src/main/java/io/streamthoughts/kafka/connect/filepulse/source/FileInputIterable.java @@ -47,10 +47,8 @@ public class FileInputIterable implements Iterable= metadata.contentLength(); + static boolean isAlreadyCompleted(final FileObjectOffset committedOffset, final FileObjectMeta metadata) { + return committedOffset != null && committedOffset.position() >= metadata.contentLength(); } } \ No newline at end of file diff --git a/connect-file-pulse-api/src/main/java/io/streamthoughts/kafka/connect/filepulse/source/FileObjectMeta.java b/connect-file-pulse-api/src/main/java/io/streamthoughts/kafka/connect/filepulse/source/FileObjectMeta.java index 41dcfc9d3..558c973b2 100644 --- a/connect-file-pulse-api/src/main/java/io/streamthoughts/kafka/connect/filepulse/source/FileObjectMeta.java +++ b/connect-file-pulse-api/src/main/java/io/streamthoughts/kafka/connect/filepulse/source/FileObjectMeta.java @@ -103,7 +103,7 @@ class ContentDigest { public ContentDigest(@JsonProperty("digest") final String digest, @JsonProperty("algorithm") final String algorithm) { this.digest = Objects.requireNonNull(digest, "digest should not be null");; - this.algorithm = Objects.requireNonNull(algorithm, "algorithm should not be null"); + this.algorithm = Objects.requireNonNull(algorithm, "algorithm should not be null").toUpperCase(); } @JsonProperty("digest") @@ -132,10 +132,10 @@ public int hashCode() { @Override public String toString() { - return "ContentDigest{" + + return "[" + "digest=" + digest + ", algorithm='" + algorithm + '\'' + - '}'; + ']'; } } } diff --git a/connect-file-pulse-filesystems/filepulse-commons-fs/pom.xml b/connect-file-pulse-filesystems/filepulse-commons-fs/pom.xml new file mode 100644 index 000000000..0e8cca879 --- /dev/null +++ b/connect-file-pulse-filesystems/filepulse-commons-fs/pom.xml @@ -0,0 +1,48 @@ + + + + + io.streamthoughts + kafka-connect-filepulse-filesystems + 2.0.0-SNAPSHOT + + 4.0.0 + + kafka-connect-filepulse-commons-fs + + + ${project.parent.basedir}/.. + + + + + net.sf.saxon + Saxon-HE + + + io.confluent + kafka-connect-avro-converter + provided + + + + \ No newline at end of file diff --git a/connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/local/reader/XMLRecordOffset.java b/connect-file-pulse-filesystems/filepulse-commons-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/IndexRecordOffset.java similarity index 68% rename from connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/local/reader/XMLRecordOffset.java rename to connect-file-pulse-filesystems/filepulse-commons-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/IndexRecordOffset.java index f66d8a1da..b33fae0c5 100644 --- a/connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/local/reader/XMLRecordOffset.java +++ b/connect-file-pulse-filesystems/filepulse-commons-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/IndexRecordOffset.java @@ -1,5 +1,5 @@ /* - * Copyright 2019-2020 StreamThoughts. + * Copyright 2019-2021 StreamThoughts. * * Licensed to the Apache Software Foundation (ASF) under one or more * contributor license agreements. See the NOTICE file distributed with @@ -16,32 +16,35 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.streamthoughts.kafka.connect.filepulse.fs.local.reader; +package io.streamthoughts.kafka.connect.filepulse.fs.reader; import io.streamthoughts.kafka.connect.filepulse.source.FileObjectOffset; import io.streamthoughts.kafka.connect.filepulse.source.TimestampedRecordOffset; import org.apache.kafka.common.utils.Time; -public class XMLRecordOffset extends TimestampedRecordOffset { +/** + * Represents the position of a record into a file based on an index. + */ +public class IndexRecordOffset extends TimestampedRecordOffset { private final long records; /** - * Creates a new {@link XMLRecordOffset} instance. + * Creates a new {@link IndexRecordOffset} instance. * - * @param records + * @param records the record offset. */ - public XMLRecordOffset(final long records) { + public IndexRecordOffset(final long records) { this(Time.SYSTEM.milliseconds(), records); } /** - * Creates a new {@link XMLRecordOffset} instance. + * Creates a new {@link IndexRecordOffset} instance. * - * @param timestamp - * @param records + * @param timestamp the timestamp attached to this offset. + * @param records the record offset. */ - private XMLRecordOffset(final long timestamp, final long records) { + private IndexRecordOffset(final long timestamp, final long records) { super(timestamp); this.records = records; } diff --git a/connect-file-pulse-filesystems/filepulse-commons-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/avro/AvroDataFileIterator.java b/connect-file-pulse-filesystems/filepulse-commons-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/avro/AvroDataFileIterator.java new file mode 100644 index 000000000..ba012ff6e --- /dev/null +++ b/connect-file-pulse-filesystems/filepulse-commons-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/avro/AvroDataFileIterator.java @@ -0,0 +1,159 @@ +/* + * Copyright 2019-2021 StreamThoughts. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.streamthoughts.kafka.connect.filepulse.fs.reader.avro; + +import io.streamthoughts.kafka.connect.filepulse.data.TypedStruct; +import io.streamthoughts.kafka.connect.filepulse.internal.Silent; +import io.streamthoughts.kafka.connect.filepulse.reader.AbstractFileInputIterator; +import io.streamthoughts.kafka.connect.filepulse.reader.IteratorManager; +import io.streamthoughts.kafka.connect.filepulse.reader.ReaderException; +import io.streamthoughts.kafka.connect.filepulse.reader.RecordsIterable; +import io.streamthoughts.kafka.connect.filepulse.source.FileContext; +import io.streamthoughts.kafka.connect.filepulse.source.FileObjectOffset; +import io.streamthoughts.kafka.connect.filepulse.source.FileRecord; +import io.streamthoughts.kafka.connect.filepulse.source.TypedFileRecord; +import org.apache.avro.file.DataFileReader; +import org.apache.avro.generic.GenericDatumReader; +import org.apache.avro.generic.GenericRecord; +import org.apache.kafka.common.utils.Time; + +import java.io.File; +import java.io.IOException; +import java.util.Objects; + +import static io.streamthoughts.kafka.connect.filepulse.internal.Silent.unchecked; + +public class AvroDataFileIterator extends AbstractFileInputIterator { + + private long recordsReadSinceLastSync = 0L; + + private long lastSync = -1L; + + private final DataFileReader reader; + + /** + * Creates a new {@link AvroDataFileIterator} instance. + * + * @param iteratorManager the {@link IteratorManager} instance. + * @param context the {@link FileContext} instance. + */ + public AvroDataFileIterator(final IteratorManager iteratorManager, + final FileContext context, + final File avroFile) throws IOException { + super(iteratorManager, context); + this.reader = new DataFileReader<>(avroFile, new GenericDatumReader<>());; + } + + /** + * {@inheritDoc} + */ + @Override + public void seekTo(final FileObjectOffset offset) { + Objects.requireNonNull(offset, "offset can't be null"); + if (offset.position() != -1) { + unchecked(() -> reader.seek(offset.position()), ReaderException::new); + recordsReadSinceLastSync = 0L; + lastSync = reader.previousSync(); + skipRecordsUntil(offset.rows()); + } + } + + /** + * Skips records until {@link #recordsReadSinceLastSync} equal the specified records. + * + * @param records the number of records to skip. + */ + private void skipRecordsUntil(long records) { + while (recordsReadSinceLastSync < records) { + nextRecord(); + } + } + + /** + * Updates the current {@link #context}. + */ + private void updateContext() { + final FileObjectOffset offset = new FileObjectOffset( + lastSync, + recordsReadSinceLastSync, + Time.SYSTEM.milliseconds()); + context = context.withOffset(offset); + } + + /** + * {@inheritDoc} + */ + @Override + public RecordsIterable> next() { + try { + final GenericRecord record = nextRecord(); + final TypedStruct struct = AvroTypedStructConverter.fromGenericRecord(record); + + AvroRecordOffset offset = new AvroRecordOffset( + lastSync, + position(), + recordsReadSinceLastSync + ); + + return RecordsIterable.of(new TypedFileRecord(offset, struct)); + } finally { + updateContext(); + } + } + + /** + * Read the next records. + * + * @return a {@link GenericRecord} instance. + */ + private GenericRecord nextRecord() { + // start to read a new block. + if (reader.previousSync() != lastSync) { + lastSync = reader.previousSync(); + recordsReadSinceLastSync = 0; + } + + final GenericRecord record = reader.next(); + recordsReadSinceLastSync++; + return record; + } + + private long position() { + return Silent.unchecked(reader::tell, ReaderException::new); + } + + /** + * {@inheritDoc} + */ + @Override + public boolean hasNext() { + return reader.hasNext(); + } + + /** + * {@inheritDoc} + */ + @Override + public void close() { + if (!isClose()) { + Silent.unchecked(reader::close, ReaderException::new); + super.close(); + } + } +} \ No newline at end of file diff --git a/connect-file-pulse-filesystems/filepulse-commons-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/avro/AvroDataStreamIterator.java b/connect-file-pulse-filesystems/filepulse-commons-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/avro/AvroDataStreamIterator.java new file mode 100644 index 000000000..16225ef8e --- /dev/null +++ b/connect-file-pulse-filesystems/filepulse-commons-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/avro/AvroDataStreamIterator.java @@ -0,0 +1,123 @@ +/* + * Copyright 2019-2021 StreamThoughts. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.streamthoughts.kafka.connect.filepulse.fs.reader.avro; + +import io.streamthoughts.kafka.connect.filepulse.data.TypedStruct; +import io.streamthoughts.kafka.connect.filepulse.fs.reader.IndexRecordOffset; +import io.streamthoughts.kafka.connect.filepulse.internal.Silent; +import io.streamthoughts.kafka.connect.filepulse.reader.AbstractFileInputIterator; +import io.streamthoughts.kafka.connect.filepulse.reader.IteratorManager; +import io.streamthoughts.kafka.connect.filepulse.reader.RecordsIterable; +import io.streamthoughts.kafka.connect.filepulse.source.FileContext; +import io.streamthoughts.kafka.connect.filepulse.source.FileObjectOffset; +import io.streamthoughts.kafka.connect.filepulse.source.FileRecord; +import io.streamthoughts.kafka.connect.filepulse.source.TypedFileRecord; +import org.apache.avro.file.DataFileStream; +import org.apache.avro.generic.GenericDatumReader; +import org.apache.avro.generic.GenericRecord; +import org.apache.kafka.common.utils.Time; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.io.InputStream; +import java.util.Objects; + +public class AvroDataStreamIterator extends AbstractFileInputIterator { + + private static final Logger LOG = LoggerFactory.getLogger(AvroDataStreamIterator.class); + + private final DataFileStream stream; + + private long position = 0; + + /** + * Creates a new {@link AvroDataStreamIterator} instance. + * + * @param iteratorManager the {@link IteratorManager} instance. + * @param context the {@link FileContext} instance. + */ + public AvroDataStreamIterator(final IteratorManager iteratorManager, + final FileContext context, + final InputStream stream) throws IOException { + super(iteratorManager, context); + this.stream = new DataFileStream<>(stream, new GenericDatumReader<>()); + } + + /** + * {@inheritDoc} + */ + @Override + public void seekTo(final FileObjectOffset offset) { + Objects.requireNonNull(offset, "offset can't be null"); + if (offset.position() != -1) { + LOG.info("Seeking to skip to Avro record {}", offset.position() ); + position = (int) offset.position(); + int i = 0; + while (i < position) { + stream.next(); + i++; + } + } + } + + /** + * Updates the current {@link #context}. + */ + private void updateContext() { + context = context.withOffset(new FileObjectOffset(position, position, Time.SYSTEM.milliseconds())); + } + + /** + * {@inheritDoc} + */ + @Override + public RecordsIterable> next() { + try { + final GenericRecord record = stream.next(); + return incrementAndGet(AvroTypedStructConverter.fromGenericRecord(record)); + } finally { + updateContext(); + } + } + + private RecordsIterable> incrementAndGet(final TypedStruct struct) { + position++; + return RecordsIterable.of(new TypedFileRecord(new IndexRecordOffset(position), struct)); + } + + /** + * {@inheritDoc} + */ + @Override + public boolean hasNext() { + return stream.hasNext(); + } + + /** + * {@inheritDoc} + */ + @Override + public void close() { + if (!isClose()) { + Silent.unchecked(stream::close); + super.close(); + } + } +} \ No newline at end of file diff --git a/connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/local/reader/AvroRecordOffset.java b/connect-file-pulse-filesystems/filepulse-commons-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/avro/AvroRecordOffset.java similarity index 95% rename from connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/local/reader/AvroRecordOffset.java rename to connect-file-pulse-filesystems/filepulse-commons-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/avro/AvroRecordOffset.java index 0173a64b8..d185fd21b 100644 --- a/connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/local/reader/AvroRecordOffset.java +++ b/connect-file-pulse-filesystems/filepulse-commons-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/avro/AvroRecordOffset.java @@ -1,5 +1,5 @@ /* - * Copyright 2019-2020 StreamThoughts. + * Copyright 2019-2021 StreamThoughts. * * Licensed to the Apache Software Foundation (ASF) under one or more * contributor license agreements. See the NOTICE file distributed with @@ -16,7 +16,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.streamthoughts.kafka.connect.filepulse.fs.local.reader; +package io.streamthoughts.kafka.connect.filepulse.fs.reader.avro; import io.streamthoughts.kafka.connect.filepulse.source.FileObjectOffset; import io.streamthoughts.kafka.connect.filepulse.source.TimestampedRecordOffset; @@ -24,6 +24,9 @@ import java.util.Objects; +/** + * Represents the position of a record into an Avro container file. + */ public class AvroRecordOffset extends TimestampedRecordOffset { /** diff --git a/connect-file-pulse-filesystems/filepulse-commons-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/avro/AvroTypedStructConverter.java b/connect-file-pulse-filesystems/filepulse-commons-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/avro/AvroTypedStructConverter.java new file mode 100644 index 000000000..660735cfd --- /dev/null +++ b/connect-file-pulse-filesystems/filepulse-commons-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/avro/AvroTypedStructConverter.java @@ -0,0 +1,173 @@ +/* + * Copyright 2019-2021 StreamThoughts. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.streamthoughts.kafka.connect.filepulse.fs.reader.avro; + +import io.streamthoughts.kafka.connect.filepulse.data.Type; +import io.streamthoughts.kafka.connect.filepulse.data.TypedStruct; +import io.streamthoughts.kafka.connect.filepulse.data.TypedValue; +import io.streamthoughts.kafka.connect.filepulse.reader.ReaderException; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.function.BiFunction; + +import static io.streamthoughts.kafka.connect.filepulse.data.TypedValue.bool; +import static io.streamthoughts.kafka.connect.filepulse.data.TypedValue.float32; +import static io.streamthoughts.kafka.connect.filepulse.data.TypedValue.float64; +import static io.streamthoughts.kafka.connect.filepulse.data.TypedValue.int32; +import static io.streamthoughts.kafka.connect.filepulse.data.TypedValue.int64; +import static io.streamthoughts.kafka.connect.filepulse.data.TypedValue.string; +import static io.streamthoughts.kafka.connect.filepulse.data.TypedValue.struct; + +/** + * The {@link AvroTypedStructConverter} can be used to convert an avro object into a {@link TypedStruct}. + */ +public class AvroTypedStructConverter { + + private static final Map> AVRO_TYPES_TO_CONVERTER; + + static { + AVRO_TYPES_TO_CONVERTER = new HashMap<>(); + AVRO_TYPES_TO_CONVERTER.put(Schema.Type.BYTES, AvroTypedStructConverter::convertBytes); + AVRO_TYPES_TO_CONVERTER.put(Schema.Type.ENUM, AvroTypedStructConverter::convertEnum); + AVRO_TYPES_TO_CONVERTER.put(Schema.Type.STRING, AvroTypedStructConverter::convertString); + AVRO_TYPES_TO_CONVERTER.put(Schema.Type.UNION, AvroTypedStructConverter::convertUnion); + AVRO_TYPES_TO_CONVERTER.put(Schema.Type.ARRAY, AvroTypedStructConverter::convertCollection); + AVRO_TYPES_TO_CONVERTER.put(Schema.Type.MAP, AvroTypedStructConverter::convertMap); + AVRO_TYPES_TO_CONVERTER.put(Schema.Type.BOOLEAN, + (schema, value) -> bool((Boolean) value)); + AVRO_TYPES_TO_CONVERTER.put(Schema.Type.INT, + (schema, value) -> int32((Integer) value)); + AVRO_TYPES_TO_CONVERTER.put(Schema.Type.LONG, + (schema, value) -> int64((Long) value)); + AVRO_TYPES_TO_CONVERTER.put(Schema.Type.FLOAT, + (schema, value) -> float32((Float) value)); + AVRO_TYPES_TO_CONVERTER.put(Schema.Type.DOUBLE, + (schema, value) -> float64((Double) value)); + AVRO_TYPES_TO_CONVERTER.put(Schema.Type.RECORD, + (schema, value) -> struct(fromGenericRecord((GenericRecord) value))); + } + + /** + * Converts the specified {@link GenericRecord} instance into a {@link TypedStruct}. + * + * @param record the {@link GenericRecord} instance to convert. + * @return a new {@link TypedStruct} instance. + */ + static TypedStruct fromGenericRecord(final GenericRecord record) { + TypedStruct struct = TypedStruct.create(); + final Schema schema = record.getSchema(); + for (Schema.Field field : schema.getFields()) { + final String name = field.name(); + final Object value = record.get(name); + struct = struct.put(name, fromSchemaAndValue(field.schema(), value)); + } + return struct; + } + + /** + * converts the specified object into {@link TypedValue}. + * + * @param schema the {@link Schema} instance. + * @param value the record value. + * @return a new {@link TypedValue} instance. + */ + private static TypedValue fromSchemaAndValue(final Schema schema, final Object value) { + final Schema.Type fieldType = schema.getType(); + + BiFunction converter = AVRO_TYPES_TO_CONVERTER.get(fieldType); + if (converter == null) { + throw new ReaderException("Unsupported avro type : " + fieldType); + } + return converter.apply(schema, value); + } + + + private static TypedValue convertEnum(final Schema schema, + final Object value) { + final String stringValue = (value != null) ? ((Enum) value).name() : null; + return string(stringValue); + } + + private static TypedValue convertUnion(final Schema schema, + final Object value) { + final List types = schema.getTypes(); + final Optional nonNullSchema = types + .stream() + .filter(s -> s.getType() != Schema.Type.NULL) + .findFirst(); + return fromSchemaAndValue(nonNullSchema.get(), value); + } + + private static TypedValue convertString(final Schema schema, + final Object value) { + // use org.apache.avro.util.Utf8 for string value. + final String stringValue = (value != null) ? value.toString() : null; + return string(stringValue); + } + + private static TypedValue convertBytes(final Schema schema, + final Object value) { + return (value != null) ? + TypedValue.any(value).as(Type.BYTES) : + TypedValue.of(null, Type.BYTES); + } + + @SuppressWarnings("unchecked") + private static TypedValue convertMap(final Schema schema, + final Object value) { + + Map map = (Map) value; + final Schema valueSchema = schema.getValueType(); + Type mapValueType = null; + final Map converted = new HashMap<>(); + for (Map.Entry o : map.entrySet()) { + TypedValue element = fromSchemaAndValue(valueSchema, o.getValue()); + // use org.apache.avro.util.Utf8 for string value. + converted.put(o.getKey().toString(), element.value()); + mapValueType = element.type(); + } + return (mapValueType != null) ? + TypedValue.map(converted, mapValueType) : + TypedValue.of(converted, Type.MAP); + } + + @SuppressWarnings("unchecked") + private static TypedValue convertCollection(final Schema schema, + final Object value) { + final Collection array = (Collection) value; + final Schema elementSchema = schema.getElementType(); + Type arrayType = null; + final Collection converted = new ArrayList<>(array.size()); + for (Object o : array) { + TypedValue element = fromSchemaAndValue(elementSchema, o); + converted.add(element.value()); + arrayType = element.type(); + } + return (arrayType != null) ? + TypedValue.array(converted, arrayType) : + TypedValue.of(converted, Type.ARRAY); + } +} diff --git a/connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/local/reader/BytesRecordOffset.java b/connect-file-pulse-filesystems/filepulse-commons-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/text/BytesRecordOffset.java similarity index 86% rename from connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/local/reader/BytesRecordOffset.java rename to connect-file-pulse-filesystems/filepulse-commons-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/text/BytesRecordOffset.java index 4190db2b1..08d32e693 100644 --- a/connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/local/reader/BytesRecordOffset.java +++ b/connect-file-pulse-filesystems/filepulse-commons-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/text/BytesRecordOffset.java @@ -1,5 +1,5 @@ /* - * Copyright 2019-2020 StreamThoughts. + * Copyright 2019-2021 StreamThoughts. * * Licensed to the Apache Software Foundation (ASF) under one or more * contributor license agreements. See the NOTICE file distributed with @@ -16,7 +16,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.streamthoughts.kafka.connect.filepulse.fs.local.reader; +package io.streamthoughts.kafka.connect.filepulse.fs.reader.text; import io.streamthoughts.kafka.connect.filepulse.source.FileObjectOffset; import io.streamthoughts.kafka.connect.filepulse.source.TimestampedRecordOffset; @@ -30,7 +30,6 @@ public class BytesRecordOffset extends TimestampedRecordOffset { private final long endPosition; - public static BytesRecordOffset empty() { return new BytesRecordOffset( -1, @@ -38,11 +37,15 @@ public static BytesRecordOffset empty() { SystemTime.SYSTEM.milliseconds()); } - public static BytesRecordOffset with(long startPosition, long endPosition) { - return new BytesRecordOffset( - startPosition, - endPosition, - SystemTime.SYSTEM.milliseconds()); + /** + * Creates a new {@link BytesRecordOffset} instance. + * + * @param startPosition the starting position. + * @param endPosition the ending position. + */ + public BytesRecordOffset(long startPosition, + long endPosition) { + this(startPosition, endPosition, SystemTime.SYSTEM.milliseconds()); } /** diff --git a/connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/local/reader/internal/NonBlockingBufferReader.java b/connect-file-pulse-filesystems/filepulse-commons-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/text/NonBlockingBufferReader.java similarity index 90% rename from connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/local/reader/internal/NonBlockingBufferReader.java rename to connect-file-pulse-filesystems/filepulse-commons-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/text/NonBlockingBufferReader.java index 18ea960aa..1b9504806 100644 --- a/connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/local/reader/internal/NonBlockingBufferReader.java +++ b/connect-file-pulse-filesystems/filepulse-commons-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/text/NonBlockingBufferReader.java @@ -1,5 +1,5 @@ /* - * Copyright 2019-2020 StreamThoughts. + * Copyright 2019-2021 StreamThoughts. * * Licensed to the Apache Software Foundation (ASF) under one or more * contributor license agreements. See the NOTICE file distributed with @@ -16,24 +16,21 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.streamthoughts.kafka.connect.filepulse.fs.local.reader.internal; +package io.streamthoughts.kafka.connect.filepulse.fs.reader.text; -import io.streamthoughts.kafka.connect.filepulse.reader.ReaderException; +import io.streamthoughts.kafka.connect.filepulse.fs.reader.text.internal.TextBlock; import org.apache.kafka.connect.errors.ConnectException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.BufferedReader; -import java.io.File; -import java.io.FileInputStream; -import java.io.FileNotFoundException; import java.io.IOException; import java.io.InputStream; import java.io.InputStreamReader; import java.nio.charset.Charset; +import java.nio.charset.StandardCharsets; import java.util.LinkedList; import java.util.List; -import java.util.Objects; /** * A BufferedReader wrapper to read lines in non-blocking way. @@ -50,7 +47,7 @@ public class NonBlockingBufferReader implements AutoCloseable { private final Charset charset; - private BufferedReader reader; + private final BufferedReader reader; // The current bytes position. private Long offset = 0L; @@ -61,42 +58,43 @@ public class NonBlockingBufferReader implements AutoCloseable { // The current buffer positions. private int bufferOffset = 0; - // Number of bytes read during last iteration. - private int nread = -1; - private boolean isAutoFlushOnEOF = true; /** * Creates a new {@link NonBlockingBufferReader} instance. * - * @param file the input file pointer. + * @param stream the input stream. + */ + public NonBlockingBufferReader(final InputStream stream) { + this(stream, StandardCharsets.UTF_8); + } + + + /** + * Creates a new {@link NonBlockingBufferReader} instance. + * + * @param stream the input stream. * @param charset the input file charset. */ - public NonBlockingBufferReader(final File file, + public NonBlockingBufferReader(final InputStream stream, final Charset charset) { - this(file, DEFAULT_INITIAL_CAPACITY, charset); + this(stream, DEFAULT_INITIAL_CAPACITY, charset); } /** * Creates a new {@link NonBlockingBufferReader} instance. * - * @param file the input file pointer. + * @param stream the input stream. * @param initialCapacity the buffer initial capacity. * @param charset the input file charset. */ - public NonBlockingBufferReader(final File file, + public NonBlockingBufferReader(final InputStream stream, final int initialCapacity, final Charset charset) { - Objects.requireNonNull(file, "file can't be null"); this.initialCapacity = initialCapacity; this.buffer = new char[initialCapacity]; this.charset = charset; - try { - LOG.debug("Opening file {}", file); - this.stream = new FileInputStream(file); - } catch (FileNotFoundException e) { - throw new ReaderException("Can't found source file : " + file); - } + this.stream = stream; this.reader = new BufferedReader(new InputStreamReader(stream, charset)); } @@ -129,7 +127,8 @@ public List readLines(final int minRecords, final boolean strict) thr // Instead we have to manage splitting lines ourselves, using simple backoff when no new value // is available. final List records = new LinkedList<>(); - nread = 0; + // Number of bytes read during last iteration. + int nread = 0; boolean maxNumRecordsNotReached = true; while (reader.ready() && (records.isEmpty() || records.size() < minRecords)) { diff --git a/connect-file-pulse-filesystems/filepulse-commons-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/text/RowFileInputIterator.java b/connect-file-pulse-filesystems/filepulse-commons-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/text/RowFileInputIterator.java new file mode 100644 index 000000000..2718e5a10 --- /dev/null +++ b/connect-file-pulse-filesystems/filepulse-commons-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/text/RowFileInputIterator.java @@ -0,0 +1,191 @@ +/* + * Copyright 2019-2021 StreamThoughts. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.streamthoughts.kafka.connect.filepulse.fs.reader.text; + +import io.streamthoughts.kafka.connect.filepulse.data.TypedStruct; +import io.streamthoughts.kafka.connect.filepulse.fs.reader.text.internal.TextBlock; +import io.streamthoughts.kafka.connect.filepulse.reader.AbstractFileInputIterator; +import io.streamthoughts.kafka.connect.filepulse.reader.IteratorManager; +import io.streamthoughts.kafka.connect.filepulse.reader.RecordsIterable; +import io.streamthoughts.kafka.connect.filepulse.source.FileContext; +import io.streamthoughts.kafka.connect.filepulse.source.FileObjectOffset; +import io.streamthoughts.kafka.connect.filepulse.source.FileRecord; +import io.streamthoughts.kafka.connect.filepulse.source.FileRecordOffset; +import io.streamthoughts.kafka.connect.filepulse.source.TypedFileRecord; +import org.apache.kafka.common.utils.Time; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.LinkedList; +import java.util.List; +import java.util.Objects; + +public class RowFileInputIterator extends AbstractFileInputIterator { + + private static final Logger LOG = LoggerFactory.getLogger(RowFileInputIterator.class); + + /** + * The buffer reader. + */ + private final NonBlockingBufferReader reader; + + /** + * The minimum number of lines to read before returning records. + */ + private int minNumReadRecords = 0; + + private long offsetLines = 0L; + + private long maxWaitMs = 0L; + + private long lastObservedRecords; + + /** + /** + * Creates a new {@link RowFileInputIterator} instance. + * + * @param context the {@link FileContext} + * @param reader the {@link NonBlockingBufferReader}. + * @param iteratorManager the {@link IteratorManager}. + */ + public RowFileInputIterator(final FileContext context, + final NonBlockingBufferReader reader, + final IteratorManager iteratorManager) { + super(iteratorManager, context); + this.reader = Objects.requireNonNull(reader, "reader can't be null"); + lastObservedRecords = Time.SYSTEM.milliseconds(); + } + + public RowFileInputIterator setMinNumReadRecords(final int minNumReadRecords) { + this.minNumReadRecords = minNumReadRecords; + return this; + } + + public RowFileInputIterator setMaxWaitMs(final long maxWaitMs) { + this.maxWaitMs = maxWaitMs; + return this; + } + + /** + * {@inheritDoc} + */ + @Override + public void seekTo(final FileObjectOffset offset) { + Objects.requireNonNull(offset, "offset can't be null"); + if (offset.position() != -1) { + offsetLines = offset.rows(); + reader.seekTo(offset.position()); + } + updateContext(); + } + + /** + * {@inheritDoc} + */ + @Override + public RecordsIterable> next() { + try { + mayWaitForLinesToBeAvailable(); + List> records = new LinkedList<>(); + List lines = reader.readLines(minNumReadRecords, false); + if (lines != null) { + for (TextBlock line : lines) { + offsetLines++; + TypedStruct struct = TypedStruct.create(); + struct.put(TypedFileRecord.DEFAULT_MESSAGE_FIELD, line.data()); + final FileRecordOffset offset = RowFileRecordOffset + .with(line.startOffset(), line.endOffset()) + .withSize(line.size()) + .withRowNumber(offsetLines); + records.add(new TypedFileRecord(offset, struct)); + } + } + if (!records.isEmpty() && canWaitForMoreRecords()) { + // Only update lastObservedRecords if no more record is expected to be read, + // otherwise the next iteration will be performed. + lastObservedRecords = Time.SYSTEM.milliseconds(); + } + return new RecordsIterable<>(records); + } catch (IOException e) { + // Underlying stream was killed, probably as a result of calling stop. Allow to return + // null, and driving thread will handle any shutdown if necessary. + } finally { + updateContext(); + } + return null; + } + + private void mayWaitForLinesToBeAvailable() { + if (!reader.hasNext()) { + LOG.debug("Waiting for more bytes from file {} (timeout={}ms)", context.metadata(), maxWaitMs); + final long timeout = lastObservedRecords + maxWaitMs; + while (!reader.hasNext() && canWaitForMoreRecords()) { + Time.SYSTEM.sleep(Math.min(100, Math.abs(timeout - Time.SYSTEM.milliseconds()))); + } + } + + if (!reader.hasNext() && !canWaitForMoreRecords()) { + LOG.info( + "Timeout after waiting for more bytes from file {} after '{}ms'.", + context.metadata(), + maxWaitMs + ); + if (reader.remaining()) { + LOG.info("Remaining buffered bytes detected"); + reader.enableAutoFlush(); + } + } + } + + private void updateContext() { + final FileObjectOffset offset = new FileObjectOffset( + reader.position(), + offsetLines, + Time.SYSTEM.milliseconds()); + context = context.withOffset(offset); + } + + /** + * {@inheritDoc} + */ + @Override + public boolean hasNext() { + return reader.hasNext() || + reader.remaining() || + canWaitForMoreRecords(); + } + + private boolean canWaitForMoreRecords() { + return lastObservedRecords + maxWaitMs > Time.SYSTEM.milliseconds(); + } + + /** + * {@inheritDoc} + */ + @Override + public void close() { + if (!isClose()) { + if (this.reader != null) { + this.reader.close(); + } + super.close(); + } + } +} diff --git a/connect-file-pulse-filesystems/filepulse-commons-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/text/RowFileInputIteratorBuilder.java b/connect-file-pulse-filesystems/filepulse-commons-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/text/RowFileInputIteratorBuilder.java new file mode 100644 index 000000000..bced7d78f --- /dev/null +++ b/connect-file-pulse-filesystems/filepulse-commons-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/text/RowFileInputIteratorBuilder.java @@ -0,0 +1,112 @@ +/* + * Copyright 2019-2021 StreamThoughts. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.streamthoughts.kafka.connect.filepulse.fs.reader.text; + +import io.streamthoughts.kafka.connect.filepulse.data.TypedStruct; +import io.streamthoughts.kafka.connect.filepulse.reader.FileInputIterator; +import io.streamthoughts.kafka.connect.filepulse.reader.IteratorManager; +import io.streamthoughts.kafka.connect.filepulse.source.FileContext; +import io.streamthoughts.kafka.connect.filepulse.source.FileRecord; + +import java.io.File; +import java.nio.charset.Charset; +import java.nio.charset.StandardCharsets; +import java.util.function.Supplier; + +/** + * Simple class to build a new {@link RowFileInputIterator}. + */ +public class RowFileInputIteratorBuilder { + + private Charset charset = StandardCharsets.UTF_8; + private int minNumReadRecords = 1; + private FileContext context; + private long waitMaxMs = 0; + private int skipHeaders = 0; + private int skipFooters = 0; + private IteratorManager iteratorManager; + private Supplier readerSupplier; + + public RowFileInputIteratorBuilder withReaderSupplier(final Supplier readerSupplier) { + this.readerSupplier = readerSupplier; + return this; + } + + public RowFileInputIteratorBuilder withContext(final FileContext context) { + this.context = context; + return this; + } + + public RowFileInputIteratorBuilder withSkipHeaders(final int skipHeaders) { + this.skipHeaders = skipHeaders; + return this; + } + + public RowFileInputIteratorBuilder withSkipFooters(final int skipFooters) { + this.skipFooters = skipFooters; + return this; + } + + public RowFileInputIteratorBuilder withMinNumReadRecords(final int minNumReadRecords) { + this.minNumReadRecords = minNumReadRecords; + return this; + } + + public RowFileInputIteratorBuilder withMaxWaitMs(final long maxWaitMs) { + this.waitMaxMs = maxWaitMs; + return this; + } + + public RowFileInputIteratorBuilder withCharset(final Charset charset) { + this.charset = charset; + return this; + } + + public RowFileInputIteratorBuilder withIteratorManager(final IteratorManager iteratorManager) { + this.iteratorManager = iteratorManager; + return this; + } + + public FileInputIterator> build() { + FileInputIterator> iterator; + + iterator = new RowFileInputIterator(context, readerSupplier.get(), iteratorManager) + .setMinNumReadRecords(minNumReadRecords) + .setMaxWaitMs(waitMaxMs); + + if (skipFooters > 0) { + iterator = new RowFileWithFooterInputIterator( + skipFooters, + new File(context.metadata().uri()), + charset, + iterator + ); + } + + if (skipHeaders > 0) { + iterator = new RowFileWithHeadersInputIterator( + skipHeaders, + readerSupplier, + iterator + ); + } + + return iterator; + } +} diff --git a/connect-file-pulse-filesystems/filepulse-commons-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/text/RowFileInputIteratorDecorator.java b/connect-file-pulse-filesystems/filepulse-commons-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/text/RowFileInputIteratorDecorator.java new file mode 100644 index 000000000..54af29961 --- /dev/null +++ b/connect-file-pulse-filesystems/filepulse-commons-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/text/RowFileInputIteratorDecorator.java @@ -0,0 +1,84 @@ +/* + * Copyright 2019-2021 StreamThoughts. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.streamthoughts.kafka.connect.filepulse.fs.reader.text; + +import io.streamthoughts.kafka.connect.filepulse.data.TypedStruct; +import io.streamthoughts.kafka.connect.filepulse.reader.FileInputIterator; +import io.streamthoughts.kafka.connect.filepulse.source.FileContext; +import io.streamthoughts.kafka.connect.filepulse.source.FileObjectOffset; +import io.streamthoughts.kafka.connect.filepulse.source.FileRecord; + +import java.util.Objects; + +/** + * An abstract class that can be used to decorate a {@link FileInputIterator}. + */ +public abstract class RowFileInputIteratorDecorator implements FileInputIterator> { + + protected final FileInputIterator> iterator; + + /** + * Creates a new {@link RowFileInputIteratorDecorator} instance. + * + * @param iterator the {@link FileInputIterator} to decorate. + */ + public RowFileInputIteratorDecorator(final FileInputIterator> iterator) { + this.iterator = Objects.requireNonNull(iterator, "iterator should not be null"); + } + + /** + * {@inheritDoc} + */ + @Override + public FileContext context() { + return iterator.context(); + } + + /** + * {@inheritDoc} + */ + @Override + public void seekTo(final FileObjectOffset offset) { + iterator.seekTo(offset); + } + + /** + * {@inheritDoc} + */ + @Override + public boolean hasNext() { + return iterator.hasNext(); + } + + /** + * {@inheritDoc} + */ + @Override + public void close() { + iterator.close(); + } + + /** + * {@inheritDoc} + */ + @Override + public boolean isClose() { + return iterator.isClose(); + } +} diff --git a/connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/local/reader/RowFileInputReaderConfig.java b/connect-file-pulse-filesystems/filepulse-commons-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/text/RowFileInputReaderConfig.java similarity index 87% rename from connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/local/reader/RowFileInputReaderConfig.java rename to connect-file-pulse-filesystems/filepulse-commons-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/text/RowFileInputReaderConfig.java index 9b8b6b880..1f51f6101 100644 --- a/connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/local/reader/RowFileInputReaderConfig.java +++ b/connect-file-pulse-filesystems/filepulse-commons-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/text/RowFileInputReaderConfig.java @@ -1,5 +1,5 @@ /* - * Copyright 2019-2020 StreamThoughts. + * Copyright 2019-2021 StreamThoughts. * * Licensed to the Apache Software Foundation (ASF) under one or more * contributor license agreements. See the NOTICE file distributed with @@ -16,7 +16,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.streamthoughts.kafka.connect.filepulse.fs.local.reader; +package io.streamthoughts.kafka.connect.filepulse.fs.reader.text; import org.apache.kafka.common.config.AbstractConfig; import org.apache.kafka.common.config.ConfigDef; @@ -26,9 +26,9 @@ import java.util.Map; /** - * Configuration class for {@link RowFileInputReader}. + * Configuration for reading a file line by line. */ -class RowFileInputReaderConfig extends AbstractConfig { +public class RowFileInputReaderConfig extends AbstractConfig { public static final String FILE_ENCODING_CONFIG = "file.encoding"; public static final String FILE_ENCODING_DOC = "The text file encoding to use (default = UTF_8)"; @@ -57,29 +57,29 @@ class RowFileInputReaderConfig extends AbstractConfig { * Creates a new {@link RowFileInputReaderConfig} instance. * @param originals the reader configuration. */ - RowFileInputReaderConfig(final Map originals) { + public RowFileInputReaderConfig(final Map originals) { super(configDef(), originals); } - int bufferInitialBytesSize() { + public int bufferInitialBytesSize() { return getInt(BUFFER_INIT_BYTES_SIZE_CONFIG); } - int minReadRecords() { + public int minReadRecords() { return getInt(MIN_NUM_READ_RECORDS_CONFIG); } - Charset charset() { + public Charset charset() { return Charset.forName(getString(FILE_ENCODING_CONFIG)); } - int skipHeaders() { + public int skipHeaders() { return getInt(READER_FIELD_HEADER_CONFIG); } - int skipFooters() { return getInt(READER_FIELD_FOOTER_CONFIG); } + public int skipFooters() { return getInt(READER_FIELD_FOOTER_CONFIG); } - long maxWaitMs() { return getLong(READER_WAIT_MAX_MS_CONFIG); } + public long maxWaitMs() { return getLong(READER_WAIT_MAX_MS_CONFIG); } private static ConfigDef configDef() { return new ConfigDef() diff --git a/connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/local/reader/RowFileRecordOffset.java b/connect-file-pulse-filesystems/filepulse-commons-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/text/RowFileRecordOffset.java similarity index 95% rename from connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/local/reader/RowFileRecordOffset.java rename to connect-file-pulse-filesystems/filepulse-commons-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/text/RowFileRecordOffset.java index f40ca473d..5d765875f 100644 --- a/connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/local/reader/RowFileRecordOffset.java +++ b/connect-file-pulse-filesystems/filepulse-commons-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/text/RowFileRecordOffset.java @@ -1,5 +1,5 @@ /* - * Copyright 2019-2020 StreamThoughts. + * Copyright 2019-2021 StreamThoughts. * * Licensed to the Apache Software Foundation (ASF) under one or more * contributor license agreements. See the NOTICE file distributed with @@ -16,13 +16,16 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.streamthoughts.kafka.connect.filepulse.fs.local.reader; +package io.streamthoughts.kafka.connect.filepulse.fs.reader.text; import io.streamthoughts.kafka.connect.filepulse.source.FileObjectOffset; import org.apache.kafka.common.utils.SystemTime; import java.util.Objects; +/** + * Represents the position of a record into a text XML file. + */ public class RowFileRecordOffset extends BytesRecordOffset { private final long rows; diff --git a/connect-file-pulse-filesystems/filepulse-commons-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/text/RowFileWithFooterInputIterator.java b/connect-file-pulse-filesystems/filepulse-commons-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/text/RowFileWithFooterInputIterator.java new file mode 100644 index 000000000..ad8bf5799 --- /dev/null +++ b/connect-file-pulse-filesystems/filepulse-commons-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/text/RowFileWithFooterInputIterator.java @@ -0,0 +1,109 @@ +/* + * Copyright 2019-2021 StreamThoughts. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.streamthoughts.kafka.connect.filepulse.fs.reader.text; + +import io.streamthoughts.kafka.connect.filepulse.data.TypedStruct; +import io.streamthoughts.kafka.connect.filepulse.fs.reader.text.internal.ReversedInputFileReader; +import io.streamthoughts.kafka.connect.filepulse.fs.reader.text.internal.TextBlock; +import io.streamthoughts.kafka.connect.filepulse.reader.FileInputIterator; +import io.streamthoughts.kafka.connect.filepulse.reader.ReaderException; +import io.streamthoughts.kafka.connect.filepulse.reader.RecordsIterable; +import io.streamthoughts.kafka.connect.filepulse.source.FileRecord; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.nio.charset.Charset; +import java.util.Collections; +import java.util.List; +import java.util.function.Predicate; +import java.util.stream.Collectors; + +public class RowFileWithFooterInputIterator extends RowFileInputIteratorDecorator { + + private static final Logger LOG = LoggerFactory.getLogger(RowFileWithFooterInputIterator.class); + + private static final String HEADERS_RECORD_FIELD = "footers"; + + /** + * The number of rows to be skipped at the end of file. + */ + private final int skipFooters; + + /** + * The local file. + */ + private final File file; + + /** + * The file charset. + */ + private final Charset charset; + + private List footers; + + private List footersStrings; + + public RowFileWithFooterInputIterator(final int skipFooters, + final File file, + final Charset charset, + final FileInputIterator> iterator) { + super(iterator); + this.skipFooters = skipFooters; + this.file = file; + this.charset = charset; + } + + /** + * {@inheritDoc} + */ + @Override + public RecordsIterable> next() { + if (footers == null) { + final String path = file.getPath(); + if (skipFooters > 0) { + LOG.info("Starting to read footer lines ({}) from file {}", skipFooters, file.getName()); + + try (final ReversedInputFileReader reversedReader = new ReversedInputFileReader(path, charset)) { + footers = reversedReader.readLines(skipFooters); + } catch (Exception e) { + throw new RuntimeException("", e); + } + if (footers.size() < skipFooters) { + throw new ReaderException("Not enough data for reading footers from file " + + path + + " (available=" + footers.size() + ", expecting=" + skipFooters + ")"); + } + Collections.reverse(footers); + footersStrings = footers.stream().map(TextBlock::data).collect(Collectors.toList()); + } + } + + final RecordsIterable> records = iterator.next(); + + return new RecordsIterable<>(records.stream() + .filter(isNotFooterLine()) + .peek(record -> record.value().put(HEADERS_RECORD_FIELD, footersStrings)) + .collect(Collectors.toList())); + } + + private Predicate> isNotFooterLine() { + return record -> ((RowFileRecordOffset) record.offset()).startPosition() < footers.get(0).startOffset(); + } +} diff --git a/connect-file-pulse-filesystems/filepulse-commons-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/text/RowFileWithHeadersInputIterator.java b/connect-file-pulse-filesystems/filepulse-commons-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/text/RowFileWithHeadersInputIterator.java new file mode 100644 index 000000000..6eb3c51e1 --- /dev/null +++ b/connect-file-pulse-filesystems/filepulse-commons-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/text/RowFileWithHeadersInputIterator.java @@ -0,0 +1,97 @@ +/* + * Copyright 2019-2021 StreamThoughts. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.streamthoughts.kafka.connect.filepulse.fs.reader.text; + +import io.streamthoughts.kafka.connect.filepulse.data.TypedStruct; +import io.streamthoughts.kafka.connect.filepulse.reader.FileInputIterator; +import io.streamthoughts.kafka.connect.filepulse.reader.ReaderException; +import io.streamthoughts.kafka.connect.filepulse.reader.RecordsIterable; +import io.streamthoughts.kafka.connect.filepulse.source.FileRecord; +import io.streamthoughts.kafka.connect.filepulse.fs.reader.text.internal.TextBlock; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.List; +import java.util.function.Predicate; +import java.util.function.Supplier; +import java.util.stream.Collectors; + +public class RowFileWithHeadersInputIterator extends RowFileInputIteratorDecorator { + + private static final Logger LOG = LoggerFactory.getLogger(RowFileWithHeadersInputIterator.class); + + private static final String HEADERS_RECORD_FIELD = "headers"; + + /** + * The number of rows to be skipped in the beginning of file. + */ + private final int skipHeaders; + + private final Supplier reader; + + private List headers; + + private List headerStrings; + + public RowFileWithHeadersInputIterator(final int skipHeaders, + final Supplier reader, + final FileInputIterator> iterator) { + super(iterator); + this.skipHeaders = skipHeaders; + this.reader = reader; + } + + @Override + public RecordsIterable> next() { + if (headers == null) { + LOG.info("Starting to read header lines ({}) from file {}", skipHeaders, context().metadata().uri()); + try (final NonBlockingBufferReader sequentialReader = reader.get()) { + headers = sequentialReader.readLines(skipHeaders, true); + headerStrings = headers + .stream() + .map(TextBlock::data) + .collect(Collectors.toList()); + } catch (Exception e) { + throw new RuntimeException("", e); + } + if (headers.size() < skipHeaders) { + throw new ReaderException( + String.format( + "Not enough data for reading headers from file %s (available=%d, expecting=%d)", + context().metadata().uri(), + headers.size(), + skipHeaders) + ); + } + } + + final RecordsIterable> records = iterator.next(); + + return new RecordsIterable<>(records.stream() + .filter(isNotHeaderLine()).peek(record -> record.value().put(HEADERS_RECORD_FIELD, headerStrings)) + .collect(Collectors.toList())); + } + + private Predicate> isNotHeaderLine() { + return record -> { + final RowFileRecordOffset offset = (RowFileRecordOffset) record.offset(); + return offset.startPosition() > headers.get(skipHeaders - 1).startOffset(); + }; + } +} diff --git a/connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/local/reader/internal/ReversedInputFileReader.java b/connect-file-pulse-filesystems/filepulse-commons-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/text/internal/ReversedInputFileReader.java similarity index 98% rename from connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/local/reader/internal/ReversedInputFileReader.java rename to connect-file-pulse-filesystems/filepulse-commons-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/text/internal/ReversedInputFileReader.java index 12615f865..10b727910 100644 --- a/connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/local/reader/internal/ReversedInputFileReader.java +++ b/connect-file-pulse-filesystems/filepulse-commons-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/text/internal/ReversedInputFileReader.java @@ -1,5 +1,5 @@ /* - * Copyright 2019-2020 StreamThoughts. + * Copyright 2019-2021 StreamThoughts. * * Licensed to the Apache Software Foundation (ASF) under one or more * contributor license agreements. See the NOTICE file distributed with @@ -16,7 +16,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.streamthoughts.kafka.connect.filepulse.fs.local.reader.internal; +package io.streamthoughts.kafka.connect.filepulse.fs.reader.text.internal; import java.io.File; import java.io.IOException; diff --git a/connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/local/reader/internal/TextBlock.java b/connect-file-pulse-filesystems/filepulse-commons-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/text/internal/TextBlock.java similarity index 95% rename from connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/local/reader/internal/TextBlock.java rename to connect-file-pulse-filesystems/filepulse-commons-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/text/internal/TextBlock.java index 02b54152c..8ba0ddd48 100644 --- a/connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/local/reader/internal/TextBlock.java +++ b/connect-file-pulse-filesystems/filepulse-commons-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/text/internal/TextBlock.java @@ -1,5 +1,5 @@ /* - * Copyright 2019-2020 StreamThoughts. + * Copyright 2019-2021 StreamThoughts. * * Licensed to the Apache Software Foundation (ASF) under one or more * contributor license agreements. See the NOTICE file distributed with @@ -16,7 +16,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.streamthoughts.kafka.connect.filepulse.fs.local.reader.internal; +package io.streamthoughts.kafka.connect.filepulse.fs.reader.text.internal; import java.nio.charset.Charset; import java.util.Objects; @@ -38,7 +38,7 @@ public class TextBlock { * @param endOffset the byte ending position of the value. * @param size the size of value in bytes */ - TextBlock(final String data, + public TextBlock(final String data, final Charset charset, final long startOffset, final long endOffset, @@ -57,6 +57,7 @@ public String data() { public long startOffset() { return startOffset; } + public long endOffset() { return endOffset; } diff --git a/connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/local/reader/XMLFileInputReader.java b/connect-file-pulse-filesystems/filepulse-commons-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/xml/XMLFileInputIterator.java similarity index 64% rename from connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/local/reader/XMLFileInputReader.java rename to connect-file-pulse-filesystems/filepulse-commons-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/xml/XMLFileInputIterator.java index d60eb5fd0..caca1a878 100644 --- a/connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/local/reader/XMLFileInputReader.java +++ b/connect-file-pulse-filesystems/filepulse-commons-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/xml/XMLFileInputIterator.java @@ -1,5 +1,5 @@ /* - * Copyright 2019-2020 StreamThoughts. + * Copyright 2019-2021 StreamThoughts. * * Licensed to the Apache Software Foundation (ASF) under one or more * contributor license agreements. See the NOTICE file distributed with @@ -16,7 +16,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.streamthoughts.kafka.connect.filepulse.fs.local.reader; +package io.streamthoughts.kafka.connect.filepulse.fs.reader.xml; import io.streamthoughts.kafka.connect.filepulse.data.ArraySchema; import io.streamthoughts.kafka.connect.filepulse.data.FieldPaths; @@ -25,13 +25,14 @@ import io.streamthoughts.kafka.connect.filepulse.data.TypedField; import io.streamthoughts.kafka.connect.filepulse.data.TypedStruct; import io.streamthoughts.kafka.connect.filepulse.data.TypedValue; -import io.streamthoughts.kafka.connect.filepulse.reader.FileInputIterator; -import io.streamthoughts.kafka.connect.filepulse.reader.FileInputReader; +import io.streamthoughts.kafka.connect.filepulse.fs.reader.IndexRecordOffset; +import io.streamthoughts.kafka.connect.filepulse.reader.AbstractFileInputIterator; +import io.streamthoughts.kafka.connect.filepulse.reader.IteratorManager; import io.streamthoughts.kafka.connect.filepulse.reader.ReaderException; import io.streamthoughts.kafka.connect.filepulse.reader.RecordsIterable; import io.streamthoughts.kafka.connect.filepulse.source.FileContext; -import io.streamthoughts.kafka.connect.filepulse.source.FileRecord; import io.streamthoughts.kafka.connect.filepulse.source.FileObjectOffset; +import io.streamthoughts.kafka.connect.filepulse.source.FileRecord; import io.streamthoughts.kafka.connect.filepulse.source.TypedFileRecord; import net.sf.saxon.lib.NamespaceConstant; import org.w3c.dom.Document; @@ -49,12 +50,10 @@ import javax.xml.xpath.XPathExpression; import javax.xml.xpath.XPathExpressionException; import javax.xml.xpath.XPathFactory; -import java.io.File; -import java.io.FileInputStream; import java.io.IOException; +import java.io.InputStream; import java.util.LinkedList; import java.util.List; -import java.util.Map; import java.util.Objects; import java.util.Optional; import java.util.stream.Collectors; @@ -63,146 +62,119 @@ import static io.streamthoughts.kafka.connect.filepulse.source.TypedFileRecord.DEFAULT_MESSAGE_FIELD; import static java.util.Collections.singletonList; -/** - * This {@link FileInputReader} can be used for reading XML source files. - */ -public class XMLFileInputReader extends AbstractFileInputReader { - - private XMLFileInputReaderConfig configs; - - /** - * {@inheritDoc} - */ - @Override - public void configure(final Map configs) { - this.configs = new XMLFileInputReaderConfig(configs); - } - - /** - * {@inheritDoc} - */ - @Override - protected FileInputIterator> newIterator(final FileContext context, - final IteratorManager iteratorManager) { - return new XMLFileInputIterator(configs, iteratorManager, context); - } - - private static class XMLFileInputIterator extends AbstractFileInputIterator { +public class XMLFileInputIterator extends AbstractFileInputIterator { - private final XMLFileInputReaderConfig config; + private final XMLFileInputReaderConfig config; - private final Object xpathResult; + private final Object xpathResult; - private final int totalRecords; + private final int totalRecords; - private int position = 0; + private int position = 0; - private final ResultType type; + private final ResultType type; - private enum ResultType {NODE_SET, STRING } + private enum ResultType {NODE_SET, STRING} - XMLFileInputIterator(final XMLFileInputReaderConfig config, - final IteratorManager iteratorManager, - final FileContext context) { - super(iteratorManager, context); + public XMLFileInputIterator(final XMLFileInputReaderConfig config, + final IteratorManager iteratorManager, + final FileContext context, + final InputStream stream) { + super(iteratorManager, context); - this.config = config; - System.setProperty( - "javax.xml.xpath.XPathFactory:"+ NamespaceConstant.OBJECT_MODEL_SAXON, + this.config = config; + System.setProperty( + "javax.xml.xpath.XPathFactory:" + NamespaceConstant.OBJECT_MODEL_SAXON, "net.sf.saxon.xpath.XPathFactoryImpl" - ); + ); - final QName qName = new QName("http://www.w3.org/1999/XSL/Transform", config.resultType()); + final QName qName = new QName("http://www.w3.org/1999/XSL/Transform", config.resultType()); - try (FileInputStream is = new FileInputStream(new File(context.metadata().uri()))) { + try (stream) { - DocumentBuilderFactory builderFactory = DocumentBuilderFactory.newInstance(); - builderFactory.setIgnoringElementContentWhitespace(true); - builderFactory.setIgnoringComments(true); - DocumentBuilder builder = builderFactory.newDocumentBuilder(); - Document document = builder.parse(new InputSource(is)); + DocumentBuilderFactory builderFactory = DocumentBuilderFactory.newInstance(); + builderFactory.setIgnoringElementContentWhitespace(true); + builderFactory.setIgnoringComments(true); + DocumentBuilder builder = builderFactory.newDocumentBuilder(); + Document document = builder.parse(new InputSource(stream)); - final XPathFactory xPathFactory = XPathFactory.newInstance(NamespaceConstant.OBJECT_MODEL_SAXON); - XPath expression = xPathFactory.newXPath(); - XPathExpression e = expression.compile(config.xpathQuery()); + final XPathFactory xPathFactory = XPathFactory.newInstance(NamespaceConstant.OBJECT_MODEL_SAXON); + XPath expression = xPathFactory.newXPath(); + XPathExpression e = expression.compile(config.xpathQuery()); - xpathResult = e.evaluate(document, qName); + xpathResult = e.evaluate(document, qName); - } catch (XPathExpressionException e) { - throw new ReaderException( + } catch (XPathExpressionException e) { + throw new ReaderException( "Cannot compile XPath expression '" + config.xpathQuery() + "'", e); - } catch (IOException e) { - throw new ReaderException( + } catch (IOException e) { + throw new ReaderException( "Error happened while reading source file '" + context + "'", e); - } catch (Exception e) { - throw new ReaderException( + } catch (Exception e) { + throw new ReaderException( "Unexpected error happened while initializing 'XMLFileInputIterator'", e); - } - - if (XPathConstants.NODESET.equals(qName)) { - type = ResultType.NODE_SET; - totalRecords = ((NodeList) xpathResult).getLength(); - } - - else if (XPathConstants.STRING.equals(qName)) { - type = ResultType.STRING; - totalRecords = 1; - } - else { - throw new ReaderException("Unsupported result type '" + config.resultType() + "'"); - } } - /** - * {@inheritDoc} - */ - @Override - public void seekTo(final FileObjectOffset offset) { - Objects.requireNonNull(offset, "offset can't be null"); - if (offset.position() != -1) { - this.position = (int)offset.position(); - } + if (XPathConstants.NODESET.equals(qName)) { + type = ResultType.NODE_SET; + totalRecords = ((NodeList) xpathResult).getLength(); + } else if (XPathConstants.STRING.equals(qName)) { + type = ResultType.STRING; + totalRecords = 1; + } else { + throw new ReaderException("Unsupported result type '" + config.resultType() + "'"); } + } - /** - * {@inheritDoc} - */ - @Override - public RecordsIterable> next() { + /** + * {@inheritDoc} + */ + @Override + public void seekTo(final FileObjectOffset offset) { + Objects.requireNonNull(offset, "offset can't be null"); + if (offset.position() != -1) { + this.position = (int) offset.position(); + } + } - if (type == ResultType.NODE_SET) { - final Node item = ((NodeList)xpathResult).item(position); + /** + * {@inheritDoc} + */ + @Override + public RecordsIterable> next() { - if (item == null) return RecordsIterable.empty(); + if (type == ResultType.NODE_SET) { + final Node item = ((NodeList) xpathResult).item(position); - try { - final FieldPaths forceArrayFields = FieldPaths.from(config.forceArrayFields()); - final TypedStruct struct = Node2StructConverter.convertNodeObjectTree(item, forceArrayFields); - return incrementAndGet(struct); - } catch (Exception e) { - throw new ReaderException("Fail to convert XML document to connect struct object: " + context, e); - } - } + if (item == null) return RecordsIterable.empty(); - if (type == ResultType.STRING) { - return incrementAndGet(TypedStruct.create().put(DEFAULT_MESSAGE_FIELD, (String) xpathResult)); + try { + final FieldPaths forceArrayFields = FieldPaths.from(config.forceArrayFields()); + final TypedStruct struct = Node2StructConverter.convertNodeObjectTree(item, forceArrayFields); + return incrementAndGet(struct); + } catch (Exception e) { + throw new ReaderException("Fail to convert XML document to connect struct object: " + context, e); } - - throw new ReaderException("Unsupported result type '" + type + "'"); } - private RecordsIterable> incrementAndGet(final TypedStruct struct) { - position++; - return RecordsIterable.of(new TypedFileRecord(new XMLRecordOffset(position), struct)); + if (type == ResultType.STRING) { + return incrementAndGet(TypedStruct.create().put(DEFAULT_MESSAGE_FIELD, (String) xpathResult)); } - /** - * {@inheritDoc} - */ - @Override - public boolean hasNext() { - return position < totalRecords; - } + throw new ReaderException("Unsupported result type '" + type + "'"); + } + + private RecordsIterable> incrementAndGet(final TypedStruct struct) { + position++; + return RecordsIterable.of(new TypedFileRecord(new IndexRecordOffset(position), struct)); + } + + /** + * {@inheritDoc} + */ + @Override + public boolean hasNext() { + return position < totalRecords; } /** @@ -215,8 +187,8 @@ private static class Node2StructConverter { /** * Converts the given {@link Node} object tree into a new new {@link TypedStruct} instance. * - * @param node the {@link Node} object tree to convert. - * @return the new {@link TypedStruct} instance. + * @param node the {@link Node} object tree to convert. + * @return the new {@link TypedStruct} instance. */ private static TypedStruct convertNodeObjectTree(final Node node, final FieldPaths forceArrayFields) { Objects.requireNonNull(node, "node cannot be null"); @@ -249,14 +221,14 @@ private static TypedStruct enrichStructWithObject(final TypedStruct container, if (field.type() == Type.ARRAY) { List array = container.getArray(nodeName); array.add(nodeValue); - value = TypedValue.array(array, ((ArraySchema)field.schema()).valueSchema()); + value = TypedValue.array(array, ((ArraySchema) field.schema()).valueSchema()); } else { List array = new LinkedList<>(); array.add(container.get(nodeName).value()); array.add(nodeValue); value = TypedValue.array(array, field.schema()); } - } else if(forceArrayField) { + } else if (forceArrayField) { List array = new LinkedList<>(); array.add(nodeValue); value = TypedValue.array(array, SchemaSupplier.lazy(nodeValue).get()); @@ -320,14 +292,14 @@ private static List collectAllNotNewLineNodes(final NodeList nodes) { } return IntStream.range(0, nodes.getLength()) - .mapToObj(nodes::item) - .filter(it -> !isWhitespaceOrNewLineNodeElement(it)) - .collect(Collectors.toList()); + .mapToObj(nodes::item) + .filter(it -> !isWhitespaceOrNewLineNodeElement(it)) + .collect(Collectors.toList()); } private static boolean isTextNode(final Node n) { return isNodeOfType(n, Node.TEXT_NODE) || - isNodeOfType(n, Node.CDATA_SECTION_NODE); + isNodeOfType(n, Node.CDATA_SECTION_NODE); } private static boolean isElementNode(final Node n) { @@ -352,7 +324,7 @@ private static void addAllNodeAttributes(final TypedStruct struct, } private static boolean isNotXmlNamespace(final Node node) { - return !XMLConstants.XMLNS_ATTRIBUTE.equalsIgnoreCase(node.getPrefix()) ; + return !XMLConstants.XMLNS_ATTRIBUTE.equalsIgnoreCase(node.getPrefix()); } private static boolean isWhitespaceOrNewLineNodeElement(final Node node) { diff --git a/connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/local/reader/XMLFileInputReaderConfig.java b/connect-file-pulse-filesystems/filepulse-commons-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/xml/XMLFileInputReaderConfig.java similarity index 90% rename from connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/local/reader/XMLFileInputReaderConfig.java rename to connect-file-pulse-filesystems/filepulse-commons-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/xml/XMLFileInputReaderConfig.java index e8fa9af2d..7d6d4a749 100644 --- a/connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/local/reader/XMLFileInputReaderConfig.java +++ b/connect-file-pulse-filesystems/filepulse-commons-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/xml/XMLFileInputReaderConfig.java @@ -1,5 +1,5 @@ /* - * Copyright 2019-2020 StreamThoughts. + * Copyright 2019-2021 StreamThoughts. * * Licensed to the Apache Software Foundation (ASF) under one or more * contributor license agreements. See the NOTICE file distributed with @@ -16,7 +16,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.streamthoughts.kafka.connect.filepulse.fs.local.reader; +package io.streamthoughts.kafka.connect.filepulse.fs.reader.xml; import org.apache.kafka.common.config.AbstractConfig; import org.apache.kafka.common.config.ConfigDef; @@ -29,7 +29,6 @@ import static org.apache.kafka.common.config.ConfigDef.ValidString.in; /** - * Configuration class for {@link RowFileInputReader}. */ public class XMLFileInputReaderConfig extends AbstractConfig { @@ -50,19 +49,19 @@ public class XMLFileInputReaderConfig extends AbstractConfig { * * @param originals the reader configuration. */ - XMLFileInputReaderConfig(final Map originals) { + public XMLFileInputReaderConfig(final Map originals) { super(configDef(), originals); } - String xpathQuery() { + public String xpathQuery() { return getString(XPATH_QUERY_CONFIG); } - String resultType() { + public String resultType() { return getString(XPATH_RESULT_TYPE_CONFIG); } - List forceArrayFields() { + public List forceArrayFields() { return getList(FORCE_ARRAY_ON_FIELDS_CONFIG); } diff --git a/connect-file-pulse-filesystems/filepulse-commons-fs/src/test/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/avro/AvroDataFileIteratorTest.java b/connect-file-pulse-filesystems/filepulse-commons-fs/src/test/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/avro/AvroDataFileIteratorTest.java new file mode 100644 index 000000000..a4d799dcc --- /dev/null +++ b/connect-file-pulse-filesystems/filepulse-commons-fs/src/test/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/avro/AvroDataFileIteratorTest.java @@ -0,0 +1,44 @@ +/* + * Copyright 2019-2021 StreamThoughts. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.streamthoughts.kafka.connect.filepulse.fs.reader.avro; + +import io.streamthoughts.kafka.connect.filepulse.data.TypedStruct; +import io.streamthoughts.kafka.connect.filepulse.internal.Silent; +import io.streamthoughts.kafka.connect.filepulse.reader.FileInputIterator; +import io.streamthoughts.kafka.connect.filepulse.reader.IteratorManager; +import io.streamthoughts.kafka.connect.filepulse.source.FileContext; +import io.streamthoughts.kafka.connect.filepulse.source.FileRecord; + +import java.io.File; +import java.io.FileInputStream; + +public class AvroDataFileIteratorTest extends BaseAvroDataIteratorTest { + + /** + * {@inheritDoc} + */ + @Override + FileInputIterator> newIterator(final FileContext context) { + return Silent.unchecked(() -> new AvroDataFileIterator( + new IteratorManager(), + context, + new File(context.metadata().uri())) + ); + } +} \ No newline at end of file diff --git a/connect-file-pulse-filesystems/filepulse-commons-fs/src/test/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/avro/AvroDataStreamIteratorTest.java b/connect-file-pulse-filesystems/filepulse-commons-fs/src/test/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/avro/AvroDataStreamIteratorTest.java new file mode 100644 index 000000000..9f859b7bd --- /dev/null +++ b/connect-file-pulse-filesystems/filepulse-commons-fs/src/test/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/avro/AvroDataStreamIteratorTest.java @@ -0,0 +1,44 @@ +/* + * Copyright 2019-2021 StreamThoughts. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.streamthoughts.kafka.connect.filepulse.fs.reader.avro; + +import io.streamthoughts.kafka.connect.filepulse.data.TypedStruct; +import io.streamthoughts.kafka.connect.filepulse.internal.Silent; +import io.streamthoughts.kafka.connect.filepulse.reader.FileInputIterator; +import io.streamthoughts.kafka.connect.filepulse.reader.IteratorManager; +import io.streamthoughts.kafka.connect.filepulse.source.FileContext; +import io.streamthoughts.kafka.connect.filepulse.source.FileRecord; + +import java.io.File; +import java.io.FileInputStream; + +public class AvroDataStreamIteratorTest extends BaseAvroDataIteratorTest { + + /** + * {@inheritDoc} + */ + @Override + FileInputIterator> newIterator(final FileContext context) { + return Silent.unchecked(() -> new AvroDataStreamIterator( + new IteratorManager(), + context, + new FileInputStream(new File(context.metadata().uri()))) + ); + } +} \ No newline at end of file diff --git a/connect-file-pulse-filesystems/filepulse-commons-fs/src/test/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/avro/AvroRecordConverterTest.java b/connect-file-pulse-filesystems/filepulse-commons-fs/src/test/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/avro/AvroRecordConverterTest.java new file mode 100644 index 000000000..3f5295348 --- /dev/null +++ b/connect-file-pulse-filesystems/filepulse-commons-fs/src/test/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/avro/AvroRecordConverterTest.java @@ -0,0 +1,136 @@ +/* + * Copyright 2019-2021 StreamThoughts. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.streamthoughts.kafka.connect.filepulse.fs.reader.avro; + +import io.streamthoughts.kafka.connect.filepulse.data.TypedStruct; +import org.apache.avro.Schema; +import org.apache.avro.SchemaBuilder; +import org.apache.avro.generic.GenericData; +import org.junit.Assert; +import org.junit.Test; + +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; + +public class AvroRecordConverterTest { + + private static final long NOW = System.currentTimeMillis(); + + private static final Schema DEFAULT_TEST_SCHEMA; + private static final GenericData.Record DEFAULT_GENERIC_RECORD; + + static { + DEFAULT_TEST_SCHEMA = SchemaBuilder.record("test") + .fields() + .requiredString("firstName") + .requiredString("lastName") + .requiredInt("age") + .requiredLong("timestamp") + .endRecord(); + + DEFAULT_GENERIC_RECORD = new GenericData.Record(DEFAULT_TEST_SCHEMA); + DEFAULT_GENERIC_RECORD.put("lastName", "foo"); + DEFAULT_GENERIC_RECORD.put("firstName", "bar"); + DEFAULT_GENERIC_RECORD.put("age", 42); + DEFAULT_GENERIC_RECORD.put("timestamp", NOW); + } + + @Test + public void shouldConvertGenericRecordWithPrimitiveTypes() { + + final TypedStruct struct = AvroTypedStructConverter.fromGenericRecord(DEFAULT_GENERIC_RECORD); + + Assert.assertEquals("bar", struct.getString("firstName")); + Assert.assertEquals("foo", struct.getString("lastName")); + Assert.assertEquals(42, struct.getInt("age").intValue()); + Assert.assertEquals(NOW, struct.getLong("timestamp").longValue()); + } + + @Test + public void shouldConvertGenericRecordWithArrayType() { + final Schema schema = SchemaBuilder.record("test") + .fields() + .name("field") + .type(SchemaBuilder.array().items(Schema.create(Schema.Type.STRING))).noDefault() + .endRecord(); + + GenericData.Record avro = new GenericData.Record(schema); + avro.put("field", Arrays.asList("foo", "bar")); + + final TypedStruct struct = AvroTypedStructConverter.fromGenericRecord(avro); + Assert.assertTrue(Arrays.asList("foo", "bar").containsAll(struct.getArray("field"))); + } + + @Test + public void shouldConvertGenericRecordWithMapType() { + final Schema schema = SchemaBuilder.record("test") + .fields() + .name("field") + .type(SchemaBuilder.map().values(Schema.create(Schema.Type.STRING))).noDefault() + .endRecord(); + + Map map = new HashMap<>(); + map.put("field1", "foo"); + map.put("field2", "bar"); + + GenericData.Record avro = new GenericData.Record(schema); + avro.put("field", map); + + final TypedStruct struct = AvroTypedStructConverter.fromGenericRecord(avro); + + Assert.assertEquals(map, struct.getMap("field")); + } + + @Test + public void shouldConvertGenericRecordWithUnionType() { + final Schema schema = SchemaBuilder.record("test") + .fields() + .optionalString("field") + .endRecord(); + + GenericData.Record avro = new GenericData.Record(schema); + avro.put("field", "string-value"); + + final TypedStruct struct = AvroTypedStructConverter.fromGenericRecord(avro); + + Assert.assertEquals( "string-value", struct.getString("field")); + } + + @Test + public void shouldConvertGenericRecordWithRecordType() { + final Schema schema = SchemaBuilder.record("test") + .fields() + .name("field") + .type(DEFAULT_TEST_SCHEMA).noDefault() + .endRecord(); + + GenericData.Record avro = new GenericData.Record(schema); + avro.put("field", DEFAULT_GENERIC_RECORD); + + final TypedStruct struct = AvroTypedStructConverter.fromGenericRecord(avro); + + TypedStruct field = struct.getStruct("field"); + + Assert.assertEquals("bar", field.getString("firstName")); + Assert.assertEquals("foo", field.getString("lastName")); + Assert.assertEquals(42, field.getInt("age").intValue()); + Assert.assertEquals(NOW, field.getLong("timestamp").longValue()); + } +} \ No newline at end of file diff --git a/connect-file-pulse-filesystems/filepulse-commons-fs/src/test/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/avro/BaseAvroDataIteratorTest.java b/connect-file-pulse-filesystems/filepulse-commons-fs/src/test/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/avro/BaseAvroDataIteratorTest.java new file mode 100644 index 000000000..f52c28902 --- /dev/null +++ b/connect-file-pulse-filesystems/filepulse-commons-fs/src/test/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/avro/BaseAvroDataIteratorTest.java @@ -0,0 +1,139 @@ +/* + * Copyright 2019-2021 StreamThoughts. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.streamthoughts.kafka.connect.filepulse.fs.reader.avro; + +import io.streamthoughts.kafka.connect.filepulse.data.TypedStruct; +import io.streamthoughts.kafka.connect.filepulse.internal.Silent; +import io.streamthoughts.kafka.connect.filepulse.reader.FileInputIterator; +import io.streamthoughts.kafka.connect.filepulse.reader.IteratorManager; +import io.streamthoughts.kafka.connect.filepulse.reader.RecordsIterable; +import io.streamthoughts.kafka.connect.filepulse.source.FileContext; +import io.streamthoughts.kafka.connect.filepulse.source.FileRecord; +import io.streamthoughts.kafka.connect.filepulse.source.FileRecordOffset; +import io.streamthoughts.kafka.connect.filepulse.source.LocalFileObjectMeta; +import org.apache.avro.Schema; +import org.apache.avro.SchemaBuilder; +import org.apache.avro.file.DataFileWriter; +import org.apache.avro.generic.GenericData; +import org.apache.avro.generic.GenericDatumWriter; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.io.DatumWriter; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import java.io.File; +import java.io.FileInputStream; +import java.io.IOException; + +public abstract class BaseAvroDataIteratorTest { + + private static final long NOW = System.currentTimeMillis(); + + private static final Schema DEFAULT_TEST_SCHEMA; + private static final GenericData.Record DEFAULT_GENERIC_RECORD; + + static { + DEFAULT_TEST_SCHEMA = SchemaBuilder.record("test") + .fields() + .requiredString("firstName") + .requiredString("lastName") + .requiredInt("age") + .requiredLong("timestamp") + .endRecord(); + DEFAULT_GENERIC_RECORD = new GenericData.Record(DEFAULT_TEST_SCHEMA); + DEFAULT_GENERIC_RECORD.put("lastName", "foo"); + DEFAULT_GENERIC_RECORD.put("firstName", "bar"); + DEFAULT_GENERIC_RECORD.put("age", 42); + DEFAULT_GENERIC_RECORD.put("timestamp", NOW); + } + + @Rule + public TemporaryFolder testFolder = new TemporaryFolder(); + + private File file; + private FileContext context; + + @Before + public void setUp() throws IOException { + file = testFolder.newFile(); + context = new FileContext(new LocalFileObjectMeta(file)); + writeGenericRecords( + DEFAULT_TEST_SCHEMA, + DEFAULT_GENERIC_RECORD, + DEFAULT_GENERIC_RECORD, + DEFAULT_GENERIC_RECORD + ); + } + + @Test + public void should_read_given_multiple_avro_record() { + + try (FileInputIterator> iterator = newIterator(context)) { + Assert.assertTrue(iterator.hasNext()); + int records = 0; + while (iterator.hasNext()) { + iterator.next(); + records++; + } + Assert.assertEquals(3, records); + } + } + + @Test + public void should_seek_to_given_valid_position() { + final FileRecordOffset offset; + try (FileInputIterator> iterator = newIterator(context)) { + Assert.assertTrue(iterator.hasNext()); + RecordsIterable> next = iterator.next(); + offset = next.last().offset(); + } + + try (FileInputIterator> iterator = newIterator(context)) { + iterator.seekTo(offset.toSourceOffset()); + // Attemps to read remaining records. + Assert.assertTrue(iterator.hasNext()); + int records = 0; + while (iterator.hasNext()) { + iterator.next(); + records++; + + } + Assert.assertEquals(2, records); + } + } + + public void writeGenericRecords(final Schema schema, final GenericRecord... records) { + DatumWriter datumWriter = new GenericDatumWriter<>(schema); + try { + DataFileWriter dataFileWriter = new DataFileWriter<>(datumWriter); + dataFileWriter.create(schema, file); + for (GenericRecord record : records) { + dataFileWriter.append(record); + } + dataFileWriter.close(); + } catch (IOException e) { + Assert.fail(e.getMessage()); + } + } + + abstract FileInputIterator> newIterator(final FileContext context); +} \ No newline at end of file diff --git a/connect-file-pulse-filesystems/filepulse-local-fs/src/test/java/io/streamthoughts/kafka/connect/filepulse/fs/local/reader/internal/NonBlockingBufferReaderTest.java b/connect-file-pulse-filesystems/filepulse-commons-fs/src/test/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/text/NonBlockingBufferReaderTest.java similarity index 93% rename from connect-file-pulse-filesystems/filepulse-local-fs/src/test/java/io/streamthoughts/kafka/connect/filepulse/fs/local/reader/internal/NonBlockingBufferReaderTest.java rename to connect-file-pulse-filesystems/filepulse-commons-fs/src/test/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/text/NonBlockingBufferReaderTest.java index 2da3f1a08..bdba4ee6a 100644 --- a/connect-file-pulse-filesystems/filepulse-local-fs/src/test/java/io/streamthoughts/kafka/connect/filepulse/fs/local/reader/internal/NonBlockingBufferReaderTest.java +++ b/connect-file-pulse-filesystems/filepulse-commons-fs/src/test/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/text/NonBlockingBufferReaderTest.java @@ -16,8 +16,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.streamthoughts.kafka.connect.filepulse.fs.local.reader.internal; +package io.streamthoughts.kafka.connect.filepulse.fs.reader.text; +import io.streamthoughts.kafka.connect.filepulse.fs.reader.text.NonBlockingBufferReader; +import io.streamthoughts.kafka.connect.filepulse.fs.reader.text.internal.TextBlock; import org.junit.Assert; import org.junit.Before; import org.junit.Rule; @@ -26,6 +28,8 @@ import java.io.BufferedWriter; import java.io.File; +import java.io.FileInputStream; +import java.io.FileNotFoundException; import java.io.IOException; import java.nio.charset.Charset; import java.nio.charset.StandardCharsets; @@ -33,7 +37,7 @@ import java.util.ArrayList; import java.util.List; -import static io.streamthoughts.kafka.connect.filepulse.fs.local.reader.internal.ReversedInputFileReader.DEFAULT_INITIAL_CAPACITY; +import static io.streamthoughts.kafka.connect.filepulse.fs.reader.text.NonBlockingBufferReader.DEFAULT_INITIAL_CAPACITY; public class NonBlockingBufferReaderTest { @@ -130,9 +134,9 @@ public void shouldReadAllLinesGivenStrictEqualsTrue()throws Exception { } private static NonBlockingBufferReader createReaderWithCapacity(final File file, - final int defaultInitialCapacity) { + final int defaultInitialCapacity) throws FileNotFoundException { return new NonBlockingBufferReader( - file, + new FileInputStream(file), defaultInitialCapacity, Charset.defaultCharset()); } diff --git a/connect-file-pulse-filesystems/filepulse-local-fs/src/test/java/io/streamthoughts/kafka/connect/filepulse/fs/local/reader/RowFileInputIteratorTest.java b/connect-file-pulse-filesystems/filepulse-commons-fs/src/test/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/text/RowFileInputIteratorTest.java similarity index 85% rename from connect-file-pulse-filesystems/filepulse-local-fs/src/test/java/io/streamthoughts/kafka/connect/filepulse/fs/local/reader/RowFileInputIteratorTest.java rename to connect-file-pulse-filesystems/filepulse-commons-fs/src/test/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/text/RowFileInputIteratorTest.java index 4bc530d98..fdf7c56e4 100644 --- a/connect-file-pulse-filesystems/filepulse-local-fs/src/test/java/io/streamthoughts/kafka/connect/filepulse/fs/local/reader/RowFileInputIteratorTest.java +++ b/connect-file-pulse-filesystems/filepulse-commons-fs/src/test/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/text/RowFileInputIteratorTest.java @@ -16,9 +16,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.streamthoughts.kafka.connect.filepulse.fs.local.reader; +package io.streamthoughts.kafka.connect.filepulse.fs.reader.text; import io.streamthoughts.kafka.connect.filepulse.data.TypedStruct; +import io.streamthoughts.kafka.connect.filepulse.fs.reader.text.NonBlockingBufferReader; +import io.streamthoughts.kafka.connect.filepulse.fs.reader.text.RowFileInputIterator; +import io.streamthoughts.kafka.connect.filepulse.reader.IteratorManager; import io.streamthoughts.kafka.connect.filepulse.reader.RecordsIterable; import io.streamthoughts.kafka.connect.filepulse.source.FileContext; import io.streamthoughts.kafka.connect.filepulse.source.FileRecord; @@ -33,6 +36,7 @@ import java.io.BufferedWriter; import java.io.File; +import java.io.FileInputStream; import java.io.IOException; import java.nio.charset.Charset; import java.nio.file.Files; @@ -58,10 +62,9 @@ public void setUp() throws IOException { generateLines(writer); } FileContext context = new FileContext(new LocalFileObjectMeta(file)); - iterator = RowFileInputIterator.newBuilder() - .withContext(context) - .withIteratorManager(new IteratorManager()) - .build(); + + final FileInputStream stream = new FileInputStream(file); + iterator = new RowFileInputIterator(context, new NonBlockingBufferReader(stream), new IteratorManager()); } @After diff --git a/connect-file-pulse-filesystems/filepulse-local-fs/src/test/java/io/streamthoughts/kafka/connect/filepulse/fs/local/reader/internal/ReverseInputFileReaderTest.java b/connect-file-pulse-filesystems/filepulse-commons-fs/src/test/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/text/internal/ReverseInputFileReaderTest.java similarity index 98% rename from connect-file-pulse-filesystems/filepulse-local-fs/src/test/java/io/streamthoughts/kafka/connect/filepulse/fs/local/reader/internal/ReverseInputFileReaderTest.java rename to connect-file-pulse-filesystems/filepulse-commons-fs/src/test/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/text/internal/ReverseInputFileReaderTest.java index 1728a905d..5458ede84 100644 --- a/connect-file-pulse-filesystems/filepulse-local-fs/src/test/java/io/streamthoughts/kafka/connect/filepulse/fs/local/reader/internal/ReverseInputFileReaderTest.java +++ b/connect-file-pulse-filesystems/filepulse-commons-fs/src/test/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/text/internal/ReverseInputFileReaderTest.java @@ -16,7 +16,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.streamthoughts.kafka.connect.filepulse.fs.local.reader.internal; +package io.streamthoughts.kafka.connect.filepulse.fs.reader.text.internal; import org.junit.Assert; import org.junit.Before; diff --git a/connect-file-pulse-filesystems/filepulse-commons-fs/src/test/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/xml/XMLFileInputIteratorTest.java b/connect-file-pulse-filesystems/filepulse-commons-fs/src/test/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/xml/XMLFileInputIteratorTest.java new file mode 100644 index 000000000..5b4a797cf --- /dev/null +++ b/connect-file-pulse-filesystems/filepulse-commons-fs/src/test/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/xml/XMLFileInputIteratorTest.java @@ -0,0 +1,282 @@ +/* + * Copyright 2019-2020 StreamThoughts. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.streamthoughts.kafka.connect.filepulse.fs.reader.xml; + +import io.streamthoughts.kafka.connect.filepulse.data.Type; +import io.streamthoughts.kafka.connect.filepulse.data.TypedStruct; +import io.streamthoughts.kafka.connect.filepulse.data.TypedValue; +import io.streamthoughts.kafka.connect.filepulse.reader.FileInputIterator; +import io.streamthoughts.kafka.connect.filepulse.reader.IteratorManager; +import io.streamthoughts.kafka.connect.filepulse.source.FileContext; +import io.streamthoughts.kafka.connect.filepulse.source.FileRecord; +import io.streamthoughts.kafka.connect.filepulse.source.LocalFileObjectMeta; +import io.streamthoughts.kafka.connect.filepulse.source.TypedFileRecord; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import java.io.BufferedWriter; +import java.io.File; +import java.io.FileInputStream; +import java.io.IOException; +import java.nio.charset.Charset; +import java.nio.file.Files; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static io.streamthoughts.kafka.connect.filepulse.fs.reader.xml.XMLFileInputReaderConfig.FORCE_ARRAY_ON_FIELDS_CONFIG; +import static io.streamthoughts.kafka.connect.filepulse.fs.reader.xml.XMLFileInputReaderConfig.XPATH_QUERY_CONFIG; +import static io.streamthoughts.kafka.connect.filepulse.fs.reader.xml.XMLFileInputReaderConfig.XPATH_RESULT_TYPE_CONFIG; + +public class XMLFileInputIteratorTest { + + @Rule + public TemporaryFolder testFolder = new TemporaryFolder(); + + @Test + public void should_read_all_records_given_valid_xpath_expression() throws IOException { + final XMLFileInputReaderConfig config = new XMLFileInputReaderConfig( + Collections.singletonMap(XPATH_QUERY_CONFIG, "//broker") + ); + + try ( + XMLFileInputIterator iterator = createXMLFileInputIterator(config, DEFAULT_TEST_XML_DOCUMENT) + ) { + List> records = new ArrayList<>(); + iterator.forEachRemaining(r -> records.addAll(r.collect())); + + Assert.assertEquals(3, records.size()); + + assertTopicPartitionObject(records.get(0).value(), "101", "0"); + assertTopicPartitionObject(records.get(1).value(), "102", "1"); + assertTopicPartitionObject(records.get(2).value(), "103", "2"); + } + } + + @Test + public void should_read_all_records_given_root_xpath_expression() throws IOException { + final XMLFileInputReaderConfig config = new XMLFileInputReaderConfig( + Collections.singletonMap(XPATH_QUERY_CONFIG, "/cluster") + ); + + try ( + XMLFileInputIterator iterator = createXMLFileInputIterator(config, DEFAULT_TEST_XML_DOCUMENT) + ) { + List> records = new ArrayList<>(); + iterator.forEachRemaining(r -> records.addAll(r.collect())); + + Assert.assertEquals(1, records.size()); + + FileRecord record = records.get(0); + TypedStruct struct = record.value(); + Assert.assertEquals("my-cluster", struct.getString("id")); + Assert.assertEquals("2.3.0", struct.getString("version")); + List brokers = struct.getArray("broker"); + Assert.assertEquals(3, brokers.size()); + + assertTopicPartitionObject(brokers.get(0), "101", "0"); + assertTopicPartitionObject(brokers.get(1), "102", "1"); + assertTopicPartitionObject(brokers.get(2), "103", "2"); + } + + } + + @Test + public void should_ignore_white_space_and_nl_nodes() throws IOException { + + final XMLFileInputReaderConfig config = new XMLFileInputReaderConfig( + Collections.singletonMap(XPATH_QUERY_CONFIG, "/") + ); + + try ( + XMLFileInputIterator iterator1 = createXMLFileInputIterator(config, "data"); + XMLFileInputIterator iterator2 = createXMLFileInputIterator(config, "\n\tdata\n") + ) { + FileRecord rs1 = iterator1.next().last(); + FileRecord rs2 = iterator2.next().last(); + Assert.assertEquals(rs1.value(), rs2.value()); + } + } + + @Test + public void should_read_record_given_document_with_cdata_node() throws IOException { + + final XMLFileInputReaderConfig config = new XMLFileInputReaderConfig( + Collections.singletonMap(XPATH_QUERY_CONFIG, "/") + ); + + try (var iterator = createXMLFileInputIterator(config, CDATA_TEST_XML_DOCUMENT)) { + List> records = new ArrayList<>(); + iterator.forEachRemaining(r -> records.addAll(r.collect())); + + Assert.assertEquals(1, records.size()); + Assert.assertEquals("dummy text", records.get(0).value().getString("ROOT")); + } + } + + @Test + public void should_read_record_given_document_with_comment_node() throws IOException { + + final XMLFileInputReaderConfig config = new XMLFileInputReaderConfig( + Collections.singletonMap(XPATH_QUERY_CONFIG, "/") + ); + + try (var iterator = createXMLFileInputIterator(config, COMMENT_TEST_XML_DOCUMENT)) { + List> records = new ArrayList<>(); + iterator.forEachRemaining(r -> records.addAll(r.collect())); + + Assert.assertEquals(1, records.size()); + Assert.assertEquals("dummy text", records.get(0).value().getString("ROOT")); + } + } + + @Test + public void should_read_record_given_node_xpath_expression() throws IOException { + final XMLFileInputReaderConfig config = new XMLFileInputReaderConfig(new HashMap() {{ + put(XPATH_QUERY_CONFIG, "(//broker)[1]/topicPartition/logSize/text()"); + put(XPATH_RESULT_TYPE_CONFIG, "STRING"); + }}); + + try (var iterator = createXMLFileInputIterator(config, DEFAULT_TEST_XML_DOCUMENT)) { + List> records = new ArrayList<>(); + iterator.forEachRemaining(r -> records.addAll(r.collect())); + + Assert.assertEquals(1, records.size()); + Assert.assertEquals("1G", records.get(0).value().getString(TypedFileRecord.DEFAULT_MESSAGE_FIELD)); + } + } + + @Test + public void should_read_record_given_valid_force_array_fields() throws IOException { + final XMLFileInputReaderConfig config = new XMLFileInputReaderConfig(new HashMap() {{ + put(XPATH_QUERY_CONFIG, "//broker[1]"); + put(FORCE_ARRAY_ON_FIELDS_CONFIG, "broker.topicPartition, broker.topicPartition.numSegments"); + }}); + + try (var iterator = createXMLFileInputIterator(config, DEFAULT_TEST_XML_DOCUMENT)) { + List> records = new ArrayList<>(); + iterator.forEachRemaining(r -> records.addAll(r.collect())); + + Assert.assertEquals(1, records.size()); + + TypedStruct record = records.get(0).value(); + Assert.assertEquals(Type.ARRAY, record.get("topicPartition").type()); + Assert.assertEquals(1, record.get("topicPartition").getArray().size()); + TypedValue segment = ((TypedStruct) record.get("topicPartition").getArray().iterator().next()).find("numSegments"); + Assert.assertEquals(Type.ARRAY, segment.type()); + Assert.assertEquals(1, segment.getArray().size()); + } + } + + @Test + public void should_read_record_given_valid_force_array_fields_and_default_xpath() throws IOException { + final XMLFileInputReaderConfig config = new XMLFileInputReaderConfig(new HashMap() {{ + put(FORCE_ARRAY_ON_FIELDS_CONFIG, "cluster.broker.topicPartition"); + }}); + + try (var iterator = createXMLFileInputIterator(config, DEFAULT_TEST_XML_DOCUMENT)) { + List> records = new ArrayList<>(); + iterator.forEachRemaining(r -> records.addAll(r.collect())); + + TypedStruct record = records.get(0).value(); + TypedStruct element = (TypedStruct) record.find("cluster.broker").getArray().iterator().next(); + Assert.assertEquals(Type.ARRAY, element.find("topicPartition").type()); + } + } + + @Test + public void should_read_record_given_single_text_node_with_attrs() throws IOException { + final XMLFileInputReaderConfig config = new XMLFileInputReaderConfig(Collections.emptyMap()); + try (var iterator = createXMLFileInputIterator(config, TEXT_NODE_TEST_XML_DOCUMENT)) { + List> records = new ArrayList<>(); + iterator.forEachRemaining(r -> records.addAll(r.collect())); + + Assert.assertEquals(1, records.size()); + Assert.assertEquals("dummy text", records.get(0).value().find("ROOT.value").getString()); + Assert.assertEquals("dummy attr", records.get(0).value().find("ROOT.attr").getString()); + } + } + + private XMLFileInputIterator createXMLFileInputIterator(final XMLFileInputReaderConfig config, + final String xmlDocument) throws IOException { + File file = testFolder.newFile(); + FileContext context; + try (BufferedWriter bw = Files.newBufferedWriter(file.toPath(), Charset.defaultCharset())) { + bw.append(xmlDocument); + bw.flush(); + context = new FileContext(new LocalFileObjectMeta(file)); + } + return new XMLFileInputIterator(config, new IteratorManager(), context, new FileInputStream(file)); + } + + private static void assertTopicPartitionObject(final TypedStruct struct, + final String expectedId, + final String expectedNum) { + Assert.assertEquals(expectedId, struct.getString("id")); + TypedStruct topicPartition = struct.getStruct("topicPartition"); + Assert.assertNotNull(topicPartition); + Assert.assertEquals("topicA", topicPartition.getString("topic")); + Assert.assertEquals(expectedNum, topicPartition.getString("num")); + Assert.assertEquals("true", topicPartition.getString("insync")); + Assert.assertEquals("0", topicPartition.getString("earliestOffset")); + Assert.assertEquals("100", topicPartition.getString("endLogOffset")); + Assert.assertEquals("1G", topicPartition.getString("logSize")); + Assert.assertEquals("1", topicPartition.getString("numSegments")); + } + + private static final String TEXT_NODE_TEST_XML_DOCUMENT = "dummy text"; + + private static final String COMMENT_TEST_XML_DOCUMENT = "dummy text"; + + private static final String CDATA_TEST_XML_DOCUMENT = "\n\t\n"; + + private static final String DEFAULT_TEST_XML_DOCUMENT = "" + + "\n" + + "\t\n" + + "\t\t\n" + + "\t\t\t0\n" + + "\t\t\t100\n" + + "\t\t\t1G\n" + + "\t\t\t1\n" + + "\t\t\n" + + "\t\n" + + "\t\n" + + "\t\t\n" + + "\t\t\t0\n" + + "\t\t\t100\n" + + "\t\t\t1G\n" + + "\t\t\t1\n" + + "\t\t\n" + + "\t\n" + + "\t\n" + + "\t\t\n" + + "\t\t\t0\n" + + "\t\t\t100\n" + + "\t\t\t1G\n" + + "\t\t\t1\n" + + "\t\t\n" + + "\t\t\t\n" + + ""; +} \ No newline at end of file diff --git a/connect-file-pulse-filesystems/filepulse-local-fs/pom.xml b/connect-file-pulse-filesystems/filepulse-local-fs/pom.xml index cc34b84e4..e6b309786 100644 --- a/connect-file-pulse-filesystems/filepulse-local-fs/pom.xml +++ b/connect-file-pulse-filesystems/filepulse-local-fs/pom.xml @@ -35,25 +35,17 @@ - org.apache.commons - commons-compress - - - net.sf.saxon - Saxon-HE + io.streamthoughts + kafka-connect-filepulse-commons-fs + ${project.version} - io.confluent - kafka-connect-avro-converter - provided - - - junit - junit + org.apache.commons + commons-compress - org.mockito - mockito-core + org.apache.avro + avro \ No newline at end of file diff --git a/connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/local/DeleteCleanupPolicy.java b/connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/DeleteCleanupPolicy.java similarity index 97% rename from connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/local/DeleteCleanupPolicy.java rename to connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/DeleteCleanupPolicy.java index ba5a912ea..a838365ba 100644 --- a/connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/local/DeleteCleanupPolicy.java +++ b/connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/DeleteCleanupPolicy.java @@ -16,7 +16,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.streamthoughts.kafka.connect.filepulse.fs.local; +package io.streamthoughts.kafka.connect.filepulse.fs; import io.streamthoughts.kafka.connect.filepulse.clean.FileCleanupPolicy; import io.streamthoughts.kafka.connect.filepulse.source.FileObject; diff --git a/connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/local/LocalFSDirectoryListing.java b/connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/LocalFSDirectoryListing.java similarity index 93% rename from connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/local/LocalFSDirectoryListing.java rename to connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/LocalFSDirectoryListing.java index 122118ebb..d57b47c42 100644 --- a/connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/local/LocalFSDirectoryListing.java +++ b/connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/LocalFSDirectoryListing.java @@ -16,13 +16,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.streamthoughts.kafka.connect.filepulse.fs.local; +package io.streamthoughts.kafka.connect.filepulse.fs; -import io.streamthoughts.kafka.connect.filepulse.fs.CompositeFileListFilter; -import io.streamthoughts.kafka.connect.filepulse.fs.FileSystemListing; -import io.streamthoughts.kafka.connect.filepulse.fs.FileListFilter; -import io.streamthoughts.kafka.connect.filepulse.fs.local.codec.CodecHandler; -import io.streamthoughts.kafka.connect.filepulse.fs.local.codec.CodecManager; +import io.streamthoughts.kafka.connect.filepulse.fs.codec.CodecHandler; +import io.streamthoughts.kafka.connect.filepulse.fs.codec.CodecManager; import io.streamthoughts.kafka.connect.filepulse.source.LocalFileObjectMeta; import io.streamthoughts.kafka.connect.filepulse.source.FileObjectMeta; import org.apache.kafka.connect.errors.ConnectException; diff --git a/connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/local/LocalFSDirectoryListingConfig.java b/connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/LocalFSDirectoryListingConfig.java similarity index 97% rename from connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/local/LocalFSDirectoryListingConfig.java rename to connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/LocalFSDirectoryListingConfig.java index 6b46fe891..63f299156 100644 --- a/connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/local/LocalFSDirectoryListingConfig.java +++ b/connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/LocalFSDirectoryListingConfig.java @@ -17,7 +17,7 @@ * limitations under the License. */ -package io.streamthoughts.kafka.connect.filepulse.fs.local; +package io.streamthoughts.kafka.connect.filepulse.fs; import org.apache.kafka.common.config.AbstractConfig; import org.apache.kafka.common.config.ConfigDef; diff --git a/connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/local/MoveCleanupPolicy.java b/connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/MoveCleanupPolicy.java similarity index 99% rename from connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/local/MoveCleanupPolicy.java rename to connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/MoveCleanupPolicy.java index 4ce58b03a..967c58585 100644 --- a/connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/local/MoveCleanupPolicy.java +++ b/connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/MoveCleanupPolicy.java @@ -16,7 +16,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.streamthoughts.kafka.connect.filepulse.fs.local; +package io.streamthoughts.kafka.connect.filepulse.fs; import io.streamthoughts.kafka.connect.filepulse.clean.FileCleanupPolicy; import io.streamthoughts.kafka.connect.filepulse.internal.IOUtils; diff --git a/connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/local/codec/CodecHandler.java b/connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/codec/CodecHandler.java similarity index 96% rename from connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/local/codec/CodecHandler.java rename to connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/codec/CodecHandler.java index ec1cce701..0b3e4b987 100644 --- a/connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/local/codec/CodecHandler.java +++ b/connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/codec/CodecHandler.java @@ -16,7 +16,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.streamthoughts.kafka.connect.filepulse.fs.local.codec; +package io.streamthoughts.kafka.connect.filepulse.fs.codec; import org.apache.kafka.common.Configurable; diff --git a/connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/local/codec/CodecHandlerUtils.java b/connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/codec/CodecHandlerUtils.java similarity index 97% rename from connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/local/codec/CodecHandlerUtils.java rename to connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/codec/CodecHandlerUtils.java index c6274cd1e..935fde9b8 100644 --- a/connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/local/codec/CodecHandlerUtils.java +++ b/connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/codec/CodecHandlerUtils.java @@ -16,7 +16,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.streamthoughts.kafka.connect.filepulse.fs.local.codec; +package io.streamthoughts.kafka.connect.filepulse.fs.codec; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/local/codec/CodecManager.java b/connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/codec/CodecManager.java similarity index 96% rename from connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/local/codec/CodecManager.java rename to connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/codec/CodecManager.java index 479c340f6..fd42acf75 100644 --- a/connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/local/codec/CodecManager.java +++ b/connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/codec/CodecManager.java @@ -16,7 +16,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.streamthoughts.kafka.connect.filepulse.fs.local.codec; +package io.streamthoughts.kafka.connect.filepulse.fs.codec; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/local/codec/GZipCodec.java b/connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/codec/GZipCodec.java similarity index 97% rename from connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/local/codec/GZipCodec.java rename to connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/codec/GZipCodec.java index cd65a8f6d..07321b1e5 100644 --- a/connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/local/codec/GZipCodec.java +++ b/connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/codec/GZipCodec.java @@ -16,7 +16,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.streamthoughts.kafka.connect.filepulse.fs.local.codec; +package io.streamthoughts.kafka.connect.filepulse.fs.codec; import io.streamthoughts.kafka.connect.filepulse.internal.IOUtils; import org.slf4j.Logger; diff --git a/connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/local/codec/TarballCodec.java b/connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/codec/TarballCodec.java similarity index 98% rename from connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/local/codec/TarballCodec.java rename to connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/codec/TarballCodec.java index f319e5ccd..19cc03c70 100644 --- a/connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/local/codec/TarballCodec.java +++ b/connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/codec/TarballCodec.java @@ -16,7 +16,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.streamthoughts.kafka.connect.filepulse.fs.local.codec; +package io.streamthoughts.kafka.connect.filepulse.fs.codec; import io.streamthoughts.kafka.connect.filepulse.internal.IOUtils; import org.apache.commons.compress.archivers.ArchiveEntry; diff --git a/connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/local/codec/ZipCodec.java b/connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/codec/ZipCodec.java similarity index 97% rename from connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/local/codec/ZipCodec.java rename to connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/codec/ZipCodec.java index abc22fd1c..01aee401c 100644 --- a/connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/local/codec/ZipCodec.java +++ b/connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/codec/ZipCodec.java @@ -16,7 +16,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.streamthoughts.kafka.connect.filepulse.fs.local.codec; +package io.streamthoughts.kafka.connect.filepulse.fs.codec; import io.streamthoughts.kafka.connect.filepulse.internal.IOUtils; import org.slf4j.Logger; diff --git a/connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/local/filter/IgnoreHiddenFileListFilter.java b/connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/filter/IgnoreHiddenFileListFilter.java similarity index 96% rename from connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/local/filter/IgnoreHiddenFileListFilter.java rename to connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/filter/IgnoreHiddenFileListFilter.java index a05d49f26..67f127585 100644 --- a/connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/local/filter/IgnoreHiddenFileListFilter.java +++ b/connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/filter/IgnoreHiddenFileListFilter.java @@ -16,7 +16,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.streamthoughts.kafka.connect.filepulse.fs.local.filter; +package io.streamthoughts.kafka.connect.filepulse.fs.filter; import io.streamthoughts.kafka.connect.filepulse.fs.PredicateFileListFilter; import io.streamthoughts.kafka.connect.filepulse.source.FileObjectMeta; diff --git a/connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/local/reader/AvroFileInputReader.java b/connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/local/reader/AvroFileInputReader.java deleted file mode 100644 index 62271210e..000000000 --- a/connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/local/reader/AvroFileInputReader.java +++ /dev/null @@ -1,328 +0,0 @@ -/* - * Copyright 2019-2020 StreamThoughts. - * - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package io.streamthoughts.kafka.connect.filepulse.fs.local.reader; - -import io.streamthoughts.kafka.connect.filepulse.annotation.VisibleForTesting; -import io.streamthoughts.kafka.connect.filepulse.data.Type; -import io.streamthoughts.kafka.connect.filepulse.data.TypedStruct; -import io.streamthoughts.kafka.connect.filepulse.data.TypedValue; -import io.streamthoughts.kafka.connect.filepulse.internal.Silent; -import io.streamthoughts.kafka.connect.filepulse.reader.FileInputIterator; -import io.streamthoughts.kafka.connect.filepulse.reader.ReaderException; -import io.streamthoughts.kafka.connect.filepulse.reader.RecordsIterable; -import io.streamthoughts.kafka.connect.filepulse.source.FileContext; -import io.streamthoughts.kafka.connect.filepulse.source.FileRecord; -import io.streamthoughts.kafka.connect.filepulse.source.FileObjectOffset; -import io.streamthoughts.kafka.connect.filepulse.source.TypedFileRecord; -import org.apache.avro.Schema; -import org.apache.avro.file.DataFileReader; -import org.apache.avro.generic.GenericDatumReader; -import org.apache.avro.generic.GenericRecord; -import org.apache.kafka.common.utils.Time; - -import java.io.File; -import java.util.ArrayList; -import java.util.Collection; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Objects; -import java.util.Optional; -import java.util.function.BiFunction; - -import static io.streamthoughts.kafka.connect.filepulse.data.TypedValue.bool; -import static io.streamthoughts.kafka.connect.filepulse.data.TypedValue.float32; -import static io.streamthoughts.kafka.connect.filepulse.data.TypedValue.float64; -import static io.streamthoughts.kafka.connect.filepulse.data.TypedValue.int32; -import static io.streamthoughts.kafka.connect.filepulse.data.TypedValue.int64; -import static io.streamthoughts.kafka.connect.filepulse.data.TypedValue.string; -import static io.streamthoughts.kafka.connect.filepulse.data.TypedValue.struct; -import static io.streamthoughts.kafka.connect.filepulse.internal.Silent.unchecked; - -/** - * Read Avro container files. - */ -public class AvroFileInputReader extends AbstractFileInputReader { - - /** - * Creates a new {@link AvroFileInputReader} instance. - */ - public AvroFileInputReader() { - super(); - } - - /** - * {@inheritDoc} - */ - @Override - protected FileInputIterator> newIterator(final FileContext context, - final IteratorManager iteratorManager) { - return new AvroFileIterator(iteratorManager, context); - } - - public static class AvroFileIterator extends AbstractFileInputIterator { - - private long recordsReadSinceLastSync = 0L; - - private long lastSync = -1L; - - private final GenericDatumReader reader; - - private final DataFileReader dataFileReader; - - /** - * Creates a new {@link AvroFileIterator} instance. - * - * @param iteratorManager the {@link IteratorManager} instance. - * @param context the {@link FileContext} instance. - */ - AvroFileIterator(final IteratorManager iteratorManager, final FileContext context) { - super(iteratorManager, context); - reader = new GenericDatumReader(); - dataFileReader = Silent.unchecked(() -> - new DataFileReader(new File(context.metadata().uri()), reader), ReaderException::new); - } - - /** - * {@inheritDoc} - */ - @Override - public void seekTo(final FileObjectOffset offset) { - Objects.requireNonNull(offset, "offset can't be null"); - if (offset.position() != -1) { - unchecked(() -> dataFileReader.seek(offset.position()), ReaderException::new); - recordsReadSinceLastSync = 0L; - lastSync = dataFileReader.previousSync(); - skipRecordsUntil(offset.rows()); - } - } - - /** - * Skips records until {@link #recordsReadSinceLastSync} equal the specified records. - * - * @param records the number of records to skip. - */ - private void skipRecordsUntil(long records) { - while(recordsReadSinceLastSync < records) { - nextRecord(); - } - } - - /** - * Updates the current {@link #context}. - */ - private void updateContext() { - final FileObjectOffset offset = new FileObjectOffset( - lastSync, - recordsReadSinceLastSync, - Time.SYSTEM.milliseconds()); - context = context.withOffset(offset); - } - - /** - * {@inheritDoc} - */ - @Override - public RecordsIterable> next() { - try { - final GenericRecord record = nextRecord(); - final TypedStruct struct = TypedStructConverter.fromGenericRecord(record); - - AvroRecordOffset offset = new AvroRecordOffset( - lastSync, - position(), - recordsReadSinceLastSync - ); - - return RecordsIterable.of(new TypedFileRecord(offset, struct)); - } finally { - updateContext(); - } - } - - /** - * Read the next records. - * - * @return a {@link GenericRecord} instance. - */ - private GenericRecord nextRecord() { - // start to read a new block. - if (dataFileReader.previousSync() != lastSync) { - lastSync = dataFileReader.previousSync(); - recordsReadSinceLastSync = 0; - } - - final GenericRecord record = dataFileReader.next(); - recordsReadSinceLastSync++; - return record; - } - - private long position() { - return Silent.unchecked(() -> dataFileReader.tell(), ReaderException::new); - } - - /** - * {@inheritDoc} - */ - @Override - public boolean hasNext() { - return dataFileReader.hasNext(); - } - - /** - * {@inheritDoc} - */ - @Override - public void close() { - if (!isClose()) { - Silent.unchecked(() -> dataFileReader.close(), ReaderException::new); - super.close(); - } - } - } - - @VisibleForTesting - public static class TypedStructConverter { - - private static final Map> AVRO_TYPES_TO_CONVERTER; - - static { - AVRO_TYPES_TO_CONVERTER = new HashMap<>(); - AVRO_TYPES_TO_CONVERTER.put(Schema.Type.BYTES, TypedStructConverter::convertBytes); - AVRO_TYPES_TO_CONVERTER.put(Schema.Type.ENUM, TypedStructConverter::convertEnum); - AVRO_TYPES_TO_CONVERTER.put(Schema.Type.STRING, TypedStructConverter::convertString); - AVRO_TYPES_TO_CONVERTER.put(Schema.Type.UNION, TypedStructConverter::convertUnion); - AVRO_TYPES_TO_CONVERTER.put(Schema.Type.ARRAY, TypedStructConverter::convertCollection); - AVRO_TYPES_TO_CONVERTER.put(Schema.Type.MAP, TypedStructConverter::convertMap); - AVRO_TYPES_TO_CONVERTER.put(Schema.Type.BOOLEAN, - (schema, value) -> bool((Boolean)value)); - AVRO_TYPES_TO_CONVERTER.put(Schema.Type.INT, - (schema, value) -> int32((Integer)value)); - AVRO_TYPES_TO_CONVERTER.put(Schema.Type.LONG, - (schema, value) -> int64((Long)value)); - AVRO_TYPES_TO_CONVERTER.put(Schema.Type.FLOAT, - (schema, value) -> float32((Float) value)); - AVRO_TYPES_TO_CONVERTER.put(Schema.Type.DOUBLE, - (schema, value) -> float64((Double) value)); - AVRO_TYPES_TO_CONVERTER.put(Schema.Type.RECORD, - (schema, value) -> struct(fromGenericRecord((GenericRecord)value))); - } - - /** - * Converts the specified {@link GenericRecord} instance into a {@link TypedStruct}. - * - * @param record the {@link GenericRecord} instance to convert. - * @return a new {@link TypedStruct} instance. - */ - static TypedStruct fromGenericRecord(final GenericRecord record) { - TypedStruct struct = TypedStruct.create(); - final Schema schema = record.getSchema(); - for (Schema.Field field : schema.getFields()) { - final String name = field.name(); - final Object value = record.get(name); - struct = struct.put(name, fromSchemaAndValue(field.schema(), value)); - } - return struct; - } - - /** - * converts the specified object into {@link TypedValue}. - * - * @param schema the {@link Schema} instance. - * @param value the record value. - * @return a new {@link TypedValue} instance. - */ - private static TypedValue fromSchemaAndValue(final Schema schema, final Object value) { - final Schema.Type fieldType = schema.getType(); - - BiFunction converter = AVRO_TYPES_TO_CONVERTER.get(fieldType); - if (converter == null) { - throw new ReaderException("Unsupported avro type : " + fieldType); - } - return converter.apply(schema, value); - } - - - private static TypedValue convertEnum(final Schema schema, - final Object value) { - final String stringValue = (value != null) ? ((Enum) value).name() : null; - return string(stringValue); - } - - private static TypedValue convertUnion(final Schema schema, - final Object value) { - final List types = schema.getTypes(); - final Optional nonNullSchema = types - .stream() - .filter(s -> s.getType() != Schema.Type.NULL) - .findFirst(); - return fromSchemaAndValue(nonNullSchema.get(), value); - } - - private static TypedValue convertString(final Schema schema, - final Object value) { - // use org.apache.avro.util.Utf8 for string value. - final String stringValue = (value != null) ? value.toString() : null; - return string(stringValue); - } - - private static TypedValue convertBytes(final Schema schema, - final Object value) { - return (value != null) ? - TypedValue.any(value).as(Type.BYTES) : - TypedValue.of(null, Type.BYTES); - } - - @SuppressWarnings("unchecked") - private static TypedValue convertMap(final Schema schema, - final Object value) { - - Map map = (Map)value; - final Schema valueSchema = schema.getValueType(); - Type mapValueType = null; - final Map converted = new HashMap<>(); - for (Map.Entry o : map.entrySet()) { - TypedValue element = fromSchemaAndValue(valueSchema, o.getValue()); - // use org.apache.avro.util.Utf8 for string value. - converted.put(o.getKey().toString(), element.value()); - mapValueType = element.type(); - } - return (mapValueType != null) ? - TypedValue.map(converted, mapValueType) : - TypedValue.of(converted, Type.MAP); - } - - @SuppressWarnings("unchecked") - private static TypedValue convertCollection(final Schema schema, - final Object value) { - final Collection array = (Collection)value; - final Schema elementSchema = schema.getElementType(); - Type arrayType = null; - final Collection converted = new ArrayList<>(array.size()); - for (Object o : array) { - TypedValue element = fromSchemaAndValue(elementSchema, o); - converted.add(element.value()); - arrayType = element.type(); - } - return (arrayType != null) ? - TypedValue.array(converted, arrayType) : - TypedValue.of(converted, Type.ARRAY); - } - } -} \ No newline at end of file diff --git a/connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/local/reader/RowFileInputIterator.java b/connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/local/reader/RowFileInputIterator.java deleted file mode 100644 index a58e10561..000000000 --- a/connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/local/reader/RowFileInputIterator.java +++ /dev/null @@ -1,408 +0,0 @@ -/* - * Copyright 2019-2020 StreamThoughts. - * - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package io.streamthoughts.kafka.connect.filepulse.fs.local.reader; - -import io.streamthoughts.kafka.connect.filepulse.data.TypedStruct; -import io.streamthoughts.kafka.connect.filepulse.fs.local.reader.internal.NonBlockingBufferReader; -import io.streamthoughts.kafka.connect.filepulse.fs.local.reader.internal.ReversedInputFileReader; -import io.streamthoughts.kafka.connect.filepulse.fs.local.reader.internal.TextBlock; -import io.streamthoughts.kafka.connect.filepulse.reader.ReaderException; -import io.streamthoughts.kafka.connect.filepulse.reader.RecordsIterable; -import io.streamthoughts.kafka.connect.filepulse.source.FileContext; -import io.streamthoughts.kafka.connect.filepulse.source.FileRecord; -import io.streamthoughts.kafka.connect.filepulse.source.FileRecordOffset; -import io.streamthoughts.kafka.connect.filepulse.source.FileObjectOffset; -import io.streamthoughts.kafka.connect.filepulse.source.TypedFileRecord; -import org.apache.kafka.common.utils.Time; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.File; -import java.io.IOException; -import java.nio.charset.Charset; -import java.nio.charset.StandardCharsets; -import java.util.Collections; -import java.util.LinkedList; -import java.util.List; -import java.util.Objects; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.stream.Collectors; - -public class RowFileInputIterator extends AbstractFileInputIterator { - - private static final Logger LOG = LoggerFactory.getLogger(RowFileInputIterator.class); - - private static final String HEADERS_RECORD_FIELD = "headers"; - private static final String FOOTERS_RECORD_FIELD = "footers"; - - /** - * The buffer reader. - */ - private final NonBlockingBufferReader reader; - - /** - * The minimum number of lines to read before returning records. - */ - private int minNumReadRecords = 0; - - /** - * The number of rows to be skipped in the beginning of file. - */ - private int skipHeaders = 0; - - /** - * The number of rows to be skipped at the end of file. - */ - private int skipFooters = 0; - - private List headers; - private List headerStrings; - - private List footers; - private List footersStrings; - - private long offsetLines = 0L; - - private final Charset charset; - - private long maxWaitMs = 0L; - - private long lastObservedRecords = -1L; - - private final AtomicBoolean initialized = new AtomicBoolean(false); - - /** - /** - * Creates a new {@link RowFileInputIterator} instance. - * - * @param context the text file context. - * @param reader the buffer reader. - * @param iteratorManager the iterator manager. - */ - private RowFileInputIterator(final FileContext context, - final NonBlockingBufferReader reader, - final IteratorManager iteratorManager, - final Charset charset) { - super(iteratorManager, context); - Objects.requireNonNull(reader, "reader can't be null"); - Objects.requireNonNull(iteratorManager, "iteratorManager can't be null"); - Objects.requireNonNull(charset, "charset can't be null"); - this.reader = reader; - this.charset = charset; - } - - private void setMinNumReadRecords(final int minNumReadRecords) { - this.minNumReadRecords = minNumReadRecords; - } - - private void setSkipHeaders(final int skipHeaders) { - this.skipHeaders = skipHeaders; - } - - private void setSkipFooters(final int skipFooters) { - this.skipFooters = skipFooters; - } - - private void setMaxWaitMs(final long maxWaitMs) { - this.maxWaitMs = maxWaitMs; - } - - /** - * {@inheritDoc} - */ - @Override - public void seekTo(final FileObjectOffset offset) { - Objects.requireNonNull(offset, "offset can't be null"); - if (offset.position() != -1) { - offsetLines = offset.rows(); - reader.seekTo(offset.position()); - } - updateContext(); - } - - /** - * {@inheritDoc} - */ - @Override - public RecordsIterable> next() { - try { - initializeIfNeeded(); - mayWaitForLinesToBeAvailable(); - List> records = new LinkedList<>(); - List lines = reader.readLines(minNumReadRecords, false); - if (lines != null) { - for (TextBlock line : lines) { - offsetLines++; - if (isNotLineFooter(line) && isNotLineHeader(line)) { - records.add(createOutputRecord(line)); - } - } - } - if (!records.isEmpty() && canWaitForMoreRecords()) { - // Only update lastObservedRecords if no more record is expected to be read, - // otherwise the next iteration will be performed. - lastObservedRecords = Time.SYSTEM.milliseconds(); - } - return new RecordsIterable<>(records); - } catch (IOException e) { - // Underlying stream was killed, probably as a result of calling stop. Allow to return - // null, and driving thread will handle any shutdown if necessary. - } finally { - updateContext(); - } - return null; - } - - private void mayWaitForLinesToBeAvailable() { - if (!reader.hasNext()) { - LOG.debug("Waiting for more bytes from file {} (timeout={}ms)", context.metadata(), maxWaitMs); - final long timeout = lastObservedRecords + maxWaitMs; - while (!reader.hasNext() && canWaitForMoreRecords()) { - Time.SYSTEM.sleep(Math.min(100, Math.abs(timeout - Time.SYSTEM.milliseconds()))); - } - } - - if (!reader.hasNext() && !canWaitForMoreRecords()) { - LOG.info( - "Timeout after waiting for more bytes from file {} after '{}ms'.", - context.metadata(), - maxWaitMs - ); - if (reader.remaining()) { - LOG.info("Remaining buffered bytes detected"); - reader.enableAutoFlush(); - } - } - } - - private void updateContext() { - final FileObjectOffset offset = new FileObjectOffset( - reader.position(), - offsetLines, - Time.SYSTEM.milliseconds()); - context = context.withOffset(offset); - } - - private FileRecord createOutputRecord(final TextBlock record) { - - TypedStruct struct = TypedStruct.create(); - struct.put(TypedFileRecord.DEFAULT_MESSAGE_FIELD, record.data()); - if (skipHeaders > 0) { - struct.put(HEADERS_RECORD_FIELD, headerStrings); - } - - if (skipFooters > 0) { - struct.put(FOOTERS_RECORD_FIELD, footersStrings); - } - - final FileRecordOffset offset = RowFileRecordOffset - .with(record.startOffset(), record.endOffset()) - .withSize(record.size()) - .withRowNumber(offsetLines); - - return new TypedFileRecord(offset, struct); - } - - private void initializeIfNeeded() { - if (!initialized.get()) { - mayReadHeaders(); - mayReadFooters(); - lastObservedRecords = Time.SYSTEM.milliseconds(); - initialized.set(true); - } - } - - /** - * {@inheritDoc} - */ - @Override - public boolean hasNext() { - return reader.hasNext() || - reader.remaining() || - canWaitForMoreRecords(); - } - - private boolean canWaitForMoreRecords() { - return lastObservedRecords + maxWaitMs > Time.SYSTEM.milliseconds(); - } - - /** - * {@inheritDoc} - */ - @Override - public void close() { - if (!isClose()) { - if (this.reader != null) { - this.reader.close(); - } - super.close(); - } - } - - private void mayReadFooters() { - final File file = new File(context.metadata().uri()); - final String path = file.getPath(); - if (skipFooters > 0) { - LOG.info("Starting to read footer lines ({}) from file {}", skipFooters, file.getName()); - - try (final ReversedInputFileReader reversedReader = new ReversedInputFileReader(path, charset)) { - footers = reversedReader.readLines(skipFooters); - } catch (Exception e) { - throw new RuntimeException("", e); - } - if (footers.size() < skipFooters) { - throw new ReaderException("Not enough value for reading footer lines from file " - + path - + " (available=" + footers.size() + ", expecting=" + skipFooters + ")"); - } - Collections.reverse(footers); - footersStrings = footers.stream().map(TextBlock::data).collect(Collectors.toList()); - } - } - - private void mayReadHeaders() { - final File file = new File(context.metadata().uri()); - final String path = file.getPath(); - if (skipHeaders > 0) { - LOG.info("Starting to read header lines ({}) from file {}", skipHeaders, file.getName()); - try (final NonBlockingBufferReader sequentialReader = - new NonBlockingBufferReader(new File(path), charset)) { - headers = sequentialReader.readLines(skipHeaders, true); - headerStrings = headers - .stream() - .map(TextBlock::data) - .collect(Collectors.toList()); - } catch (Exception e) { - throw new RuntimeException("", e); - } - if (headers.size() < skipHeaders) { - throw new ReaderException( - String.format( - "Not enough value for reading header lines from file %s (available=%d, expecting=%d)", - path, - headers.size(), - skipHeaders) - ); - } - } - } - - private boolean isNotLineHeader(final TextBlock source) { - return skipHeaders <= 0 || - source.startOffset() > headers.get(skipHeaders - 1).startOffset(); - } - - private boolean isNotLineFooter(final TextBlock source) { - return skipFooters <= 0 || - source.startOffset() < footers.get(0).startOffset(); - } - - /** - * Simple class to build a new {@link RowFileInputIterator}. - */ - public static class Builder { - - private Charset charset; - private int minNumReadRecords; - private FileContext context; - private int initialBufferSize; - private int skipHeaders; - private int skipFooters; - private long waitMaxMs; - private IteratorManager iteratorManager; - - /** - * Creates a new {@link Builder} instance. - */ - private Builder() { - this.charset = StandardCharsets.UTF_8; - this.minNumReadRecords = 1; - this.initialBufferSize = NonBlockingBufferReader.DEFAULT_INITIAL_CAPACITY; - } - - Builder withContext(final FileContext context) { - this.context = context; - return this; - } - - Builder withSkipHeaders(final int skipHeaders) { - this.skipHeaders = skipHeaders; - return this; - } - - Builder withSkipFooters(final int skipFooters) { - this.skipFooters = skipFooters; - return this; - } - - Builder withMinNumReadRecords(final int minNumReadRecords) { - this.minNumReadRecords = minNumReadRecords; - return this; - } - - Builder withMaxWaitMs(final long maxWaitMs) { - this.waitMaxMs = maxWaitMs; - return this; - } - - Builder withInitialBufferSize(final int initialBufferSize) { - this.initialBufferSize = initialBufferSize; - return this; - } - - Builder withCharset(final Charset charset) { - this.charset = charset; - return this; - } - - Builder withIteratorManager(final IteratorManager iteratorManager) { - this.iteratorManager = iteratorManager; - return this; - } - - RowFileInputIterator build() { - validateNotNull(context, "context"); - final File file = new File(context.metadata().uri()); - NonBlockingBufferReader reader = new NonBlockingBufferReader(file, initialBufferSize, charset); - reader.disableAutoFlush(); - RowFileInputIterator iterator = new RowFileInputIterator(context, reader, iteratorManager, charset); - iterator.setSkipFooters(skipFooters); - iterator.setSkipHeaders(skipHeaders); - iterator.setMinNumReadRecords(minNumReadRecords); - iterator.setMaxWaitMs(waitMaxMs); - return iterator; - } - - private void validateNotNull(final Object o, final String property) { - if (o == null) { - throw new IllegalStateException( - "Error while building new RowFileInputIterator. The property " + property +" is null."); - } - } - } - - /** - * Returns a new {@link Builder} instance. - * - * @return a new {@link Builder} instance. - */ - static Builder newBuilder() { - return new Builder(); - } -} diff --git a/connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/BaseLocalFileInputReader.java b/connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/BaseLocalFileInputReader.java new file mode 100644 index 000000000..79abf637c --- /dev/null +++ b/connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/BaseLocalFileInputReader.java @@ -0,0 +1,47 @@ +/* + * Copyright 2019-2021 StreamThoughts. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.streamthoughts.kafka.connect.filepulse.fs.reader; + +import io.streamthoughts.kafka.connect.filepulse.reader.AbstractFileInputReader; +import io.streamthoughts.kafka.connect.filepulse.source.FileObjectMeta; +import io.streamthoughts.kafka.connect.filepulse.source.LocalFileObjectMeta; + +import java.io.File; +import java.net.URI; +import java.nio.file.Files; +import java.nio.file.Paths; + +public abstract class BaseLocalFileInputReader extends AbstractFileInputReader { + + /** + * {@inheritDoc} + */ + @Override + public FileObjectMeta readMetadata(final URI fileURI) { + return new LocalFileObjectMeta(new File(fileURI)); + } + + /** + * {@inheritDoc} + */ + @Override + public boolean isReadable(final URI fileURI) { + return Files.isReadable(Paths.get(fileURI)); + } +} diff --git a/connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/LocalAvroFileInputReader.java b/connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/LocalAvroFileInputReader.java new file mode 100644 index 000000000..f4ba25601 --- /dev/null +++ b/connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/LocalAvroFileInputReader.java @@ -0,0 +1,57 @@ +/* + * Copyright 2019-2020 StreamThoughts. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.streamthoughts.kafka.connect.filepulse.fs.reader; + +import io.streamthoughts.kafka.connect.filepulse.data.TypedStruct; +import io.streamthoughts.kafka.connect.filepulse.fs.reader.avro.AvroDataFileIterator; +import io.streamthoughts.kafka.connect.filepulse.reader.FileInputIterator; +import io.streamthoughts.kafka.connect.filepulse.reader.IteratorManager; +import io.streamthoughts.kafka.connect.filepulse.reader.ReaderException; +import io.streamthoughts.kafka.connect.filepulse.source.FileContext; +import io.streamthoughts.kafka.connect.filepulse.source.FileRecord; + +import java.io.File; +import java.io.IOException; + +/** + * The {@link LocalAvroFileInputReader} can be used for reading data records from an Avro container file. + */ +public class LocalAvroFileInputReader extends BaseLocalFileInputReader { + + /** + * Creates a new {@link LocalAvroFileInputReader} instance. + */ + public LocalAvroFileInputReader() { + super(); + } + + /** + * {@inheritDoc} + */ + @Override + protected FileInputIterator> newIterator(final FileContext context, + final IteratorManager iteratorManager) { + try { + final File file = new File(context.metadata().uri()); + return new AvroDataFileIterator(iteratorManager, context, file); + } catch (ReaderException | IOException e) { + throw new ReaderException("Failed to create a new iterator for Avro file", e); + } + } +} \ No newline at end of file diff --git a/connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/local/reader/BytesArrayInputReader.java b/connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/LocalBytesArrayInputReader.java similarity index 73% rename from connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/local/reader/BytesArrayInputReader.java rename to connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/LocalBytesArrayInputReader.java index 4ad023cb3..3d26e091b 100644 --- a/connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/local/reader/BytesArrayInputReader.java +++ b/connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/LocalBytesArrayInputReader.java @@ -1,5 +1,5 @@ /* - * Copyright 2019-2020 StreamThoughts. + * Copyright 2019-2021 StreamThoughts. * * Licensed to the Apache Software Foundation (ASF) under one or more * contributor license agreements. See the NOTICE file distributed with @@ -16,33 +16,37 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.streamthoughts.kafka.connect.filepulse.fs.local.reader; +package io.streamthoughts.kafka.connect.filepulse.fs.reader; import io.streamthoughts.kafka.connect.filepulse.data.TypedStruct; +import io.streamthoughts.kafka.connect.filepulse.fs.reader.text.BytesRecordOffset; +import io.streamthoughts.kafka.connect.filepulse.reader.AbstractFileInputIterator; import io.streamthoughts.kafka.connect.filepulse.reader.FileInputIterator; +import io.streamthoughts.kafka.connect.filepulse.reader.IteratorManager; import io.streamthoughts.kafka.connect.filepulse.reader.ReaderException; import io.streamthoughts.kafka.connect.filepulse.reader.RecordsIterable; import io.streamthoughts.kafka.connect.filepulse.source.FileContext; -import io.streamthoughts.kafka.connect.filepulse.source.FileRecord; -import io.streamthoughts.kafka.connect.filepulse.source.FileRecordOffset; import io.streamthoughts.kafka.connect.filepulse.source.FileObjectOffset; +import io.streamthoughts.kafka.connect.filepulse.source.FileRecord; import io.streamthoughts.kafka.connect.filepulse.source.TypedFileRecord; import java.io.IOException; +import java.net.URI; import java.nio.file.Files; -import java.nio.file.Path; import java.nio.file.Paths; import java.util.NoSuchElementException; /** - * Reads all bytes from an input files. + * + * The {@code BytesArrayInputReader} creates one record per input file. + * Each record has single field {@code message} containing the content of the file as a byte array. */ -public class BytesArrayInputReader extends AbstractFileInputReader { +public class LocalBytesArrayInputReader extends BaseLocalFileInputReader { /** - * Creates a new {@link BytesArrayInputReader} instance. + * Creates a new {@link LocalBytesArrayInputReader} instance. */ - public BytesArrayInputReader() { + public LocalBytesArrayInputReader() { super(); } @@ -91,18 +95,13 @@ public RecordsIterable> next() { throw new NoSuchElementException(); } + final URI uri = context().metadata().uri(); try { - final Path path = Paths.get(context().metadata().uri()); - byte[] bytes = Files.readAllBytes(path); - - TypedStruct struct = TypedStruct.create(); - struct.put(TypedFileRecord.DEFAULT_MESSAGE_FIELD, bytes); - - final FileRecordOffset offset = BytesRecordOffset.with(0, bytes.length); - - return RecordsIterable.of(new TypedFileRecord(offset, struct)); + byte[] bytes = Files.readAllBytes(Paths.get(uri)); + TypedStruct struct = TypedStruct.create().put(TypedFileRecord.DEFAULT_MESSAGE_FIELD, bytes); + return RecordsIterable.of(new TypedFileRecord(new BytesRecordOffset(0, bytes.length), struct)); } catch (IOException e) { - throw new ReaderException("Error while reading file : " + context().metadata(), e); + throw new ReaderException("Failed to read file: " + uri, e); } finally { hasNext = false; } diff --git a/connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/local/reader/RowFileInputReader.java b/connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/LocalRowFileInputReader.java similarity index 57% rename from connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/local/reader/RowFileInputReader.java rename to connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/LocalRowFileInputReader.java index 44f384d53..0e539f348 100644 --- a/connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/local/reader/RowFileInputReader.java +++ b/connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/LocalRowFileInputReader.java @@ -16,26 +16,34 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.streamthoughts.kafka.connect.filepulse.fs.local.reader; +package io.streamthoughts.kafka.connect.filepulse.fs.reader; import io.streamthoughts.kafka.connect.filepulse.data.TypedStruct; +import io.streamthoughts.kafka.connect.filepulse.fs.reader.text.NonBlockingBufferReader; +import io.streamthoughts.kafka.connect.filepulse.fs.reader.text.RowFileInputIteratorBuilder; +import io.streamthoughts.kafka.connect.filepulse.fs.reader.text.RowFileInputReaderConfig; import io.streamthoughts.kafka.connect.filepulse.reader.FileInputIterator; +import io.streamthoughts.kafka.connect.filepulse.reader.IteratorManager; +import io.streamthoughts.kafka.connect.filepulse.reader.ReaderException; import io.streamthoughts.kafka.connect.filepulse.source.FileContext; import io.streamthoughts.kafka.connect.filepulse.source.FileRecord; +import java.io.File; +import java.io.FileInputStream; +import java.io.FileNotFoundException; import java.util.Map; /** - * Default reader to read flat files row by row + * The {@code RowFileInputReader} reads an input local file line by line. */ -public class RowFileInputReader extends AbstractFileInputReader { +public class LocalRowFileInputReader extends BaseLocalFileInputReader { private RowFileInputReaderConfig configs; /** - * Creates a new {@link RowFileInputReader} instance. + * Creates a new {@link LocalRowFileInputReader} instance. */ - public RowFileInputReader() { + public LocalRowFileInputReader() { super(); } @@ -57,15 +65,29 @@ public RowFileInputReaderConfig config() { @Override protected FileInputIterator> newIterator(final FileContext context, final IteratorManager iteratorManager) { - return RowFileInputIterator.newBuilder() + return new RowFileInputIteratorBuilder() .withContext(context) .withCharset(configs.charset()) - .withInitialBufferSize(configs.bufferInitialBytesSize()) .withMinNumReadRecords(configs.minReadRecords()) .withSkipHeaders(configs.skipHeaders()) .withSkipFooters(configs.skipFooters()) .withMaxWaitMs(configs.maxWaitMs()) .withIteratorManager(iteratorManager) + .withReaderSupplier(() -> { + try { + final File file = new File(context.metadata().uri()); + final FileInputStream stream = new FileInputStream(file); + var br = new NonBlockingBufferReader( + stream, + configs.bufferInitialBytesSize(), + configs.charset() + ); + br.disableAutoFlush(); + return br; + } catch (FileNotFoundException e) { + throw new ReaderException("Failed to open file: " + context.metadata().uri()); + } + }) .build(); } } diff --git a/connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/LocalXMLFileInputReader.java b/connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/LocalXMLFileInputReader.java new file mode 100644 index 000000000..88d7ddfc6 --- /dev/null +++ b/connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/LocalXMLFileInputReader.java @@ -0,0 +1,72 @@ +/* + * Copyright 2019-2020 StreamThoughts. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.streamthoughts.kafka.connect.filepulse.fs.reader; + +import io.streamthoughts.kafka.connect.filepulse.data.TypedStruct; +import io.streamthoughts.kafka.connect.filepulse.fs.reader.xml.XMLFileInputIterator; +import io.streamthoughts.kafka.connect.filepulse.fs.reader.xml.XMLFileInputReaderConfig; +import io.streamthoughts.kafka.connect.filepulse.reader.FileInputIterator; +import io.streamthoughts.kafka.connect.filepulse.reader.FileInputReader; +import io.streamthoughts.kafka.connect.filepulse.reader.IteratorManager; +import io.streamthoughts.kafka.connect.filepulse.reader.ReaderException; +import io.streamthoughts.kafka.connect.filepulse.source.FileContext; +import io.streamthoughts.kafka.connect.filepulse.source.FileRecord; + +import java.io.File; +import java.io.FileInputStream; +import java.io.FileNotFoundException; +import java.net.URI; +import java.util.Map; + +/** + * This {@link FileInputReader} can be used for reading XML source files. + */ +public class LocalXMLFileInputReader extends BaseLocalFileInputReader { + + private XMLFileInputReaderConfig configs; + + /** + * {@inheritDoc} + */ + @Override + public void configure(final Map configs) { + this.configs = new XMLFileInputReaderConfig(configs); + } + + /** + * {@inheritDoc} + */ + @Override + protected FileInputIterator> newIterator(final FileContext context, + final IteratorManager iteratorManager) { + + final URI uri = context.metadata().uri(); + try { + FileInputStream stream = new FileInputStream(new File(uri)); + return new XMLFileInputIterator( + configs, + iteratorManager, + context, + stream + ); + } catch (FileNotFoundException e) { + throw new ReaderException("Failed to open file: " + uri + ", file does not exist or is not accessible."); + } + } +} diff --git a/connect-file-pulse-filesystems/filepulse-local-fs/src/test/java/io/streamthoughts/kafka/connect/filepulse/fs/local/DeleteCleanupPolicyTest.java b/connect-file-pulse-filesystems/filepulse-local-fs/src/test/java/io/streamthoughts/kafka/connect/filepulse/fs/DeleteCleanupPolicyTest.java similarity index 97% rename from connect-file-pulse-filesystems/filepulse-local-fs/src/test/java/io/streamthoughts/kafka/connect/filepulse/fs/local/DeleteCleanupPolicyTest.java rename to connect-file-pulse-filesystems/filepulse-local-fs/src/test/java/io/streamthoughts/kafka/connect/filepulse/fs/DeleteCleanupPolicyTest.java index e4e4047f2..249c3ff2f 100644 --- a/connect-file-pulse-filesystems/filepulse-local-fs/src/test/java/io/streamthoughts/kafka/connect/filepulse/fs/local/DeleteCleanupPolicyTest.java +++ b/connect-file-pulse-filesystems/filepulse-local-fs/src/test/java/io/streamthoughts/kafka/connect/filepulse/fs/DeleteCleanupPolicyTest.java @@ -16,7 +16,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.streamthoughts.kafka.connect.filepulse.fs.local; +package io.streamthoughts.kafka.connect.filepulse.fs; import io.streamthoughts.kafka.connect.filepulse.source.LocalFileObjectMeta; import io.streamthoughts.kafka.connect.filepulse.source.FileObject; diff --git a/connect-file-pulse-filesystems/filepulse-local-fs/src/test/java/io/streamthoughts/kafka/connect/filepulse/fs/local/LocalFSDirectoryListingTest.java b/connect-file-pulse-filesystems/filepulse-local-fs/src/test/java/io/streamthoughts/kafka/connect/filepulse/fs/LocalFSDirectoryListingTest.java similarity index 97% rename from connect-file-pulse-filesystems/filepulse-local-fs/src/test/java/io/streamthoughts/kafka/connect/filepulse/fs/local/LocalFSDirectoryListingTest.java rename to connect-file-pulse-filesystems/filepulse-local-fs/src/test/java/io/streamthoughts/kafka/connect/filepulse/fs/LocalFSDirectoryListingTest.java index 45b24a605..3700f8618 100644 --- a/connect-file-pulse-filesystems/filepulse-local-fs/src/test/java/io/streamthoughts/kafka/connect/filepulse/fs/local/LocalFSDirectoryListingTest.java +++ b/connect-file-pulse-filesystems/filepulse-local-fs/src/test/java/io/streamthoughts/kafka/connect/filepulse/fs/LocalFSDirectoryListingTest.java @@ -16,9 +16,8 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.streamthoughts.kafka.connect.filepulse.fs.local; +package io.streamthoughts.kafka.connect.filepulse.fs; -import io.streamthoughts.kafka.connect.filepulse.fs.FileSystemListing; import io.streamthoughts.kafka.connect.filepulse.source.FileObjectMeta; import org.junit.Assert; import org.junit.Before; diff --git a/connect-file-pulse-filesystems/filepulse-local-fs/src/test/java/io/streamthoughts/kafka/connect/filepulse/fs/local/MoveCleanupPolicyTest.java b/connect-file-pulse-filesystems/filepulse-local-fs/src/test/java/io/streamthoughts/kafka/connect/filepulse/fs/MoveCleanupPolicyTest.java similarity index 98% rename from connect-file-pulse-filesystems/filepulse-local-fs/src/test/java/io/streamthoughts/kafka/connect/filepulse/fs/local/MoveCleanupPolicyTest.java rename to connect-file-pulse-filesystems/filepulse-local-fs/src/test/java/io/streamthoughts/kafka/connect/filepulse/fs/MoveCleanupPolicyTest.java index 8606c5ce8..cfd971e15 100644 --- a/connect-file-pulse-filesystems/filepulse-local-fs/src/test/java/io/streamthoughts/kafka/connect/filepulse/fs/local/MoveCleanupPolicyTest.java +++ b/connect-file-pulse-filesystems/filepulse-local-fs/src/test/java/io/streamthoughts/kafka/connect/filepulse/fs/MoveCleanupPolicyTest.java @@ -16,7 +16,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.streamthoughts.kafka.connect.filepulse.fs.local; +package io.streamthoughts.kafka.connect.filepulse.fs; import io.streamthoughts.kafka.connect.filepulse.source.LocalFileObjectMeta; import io.streamthoughts.kafka.connect.filepulse.source.FileObject; diff --git a/connect-file-pulse-filesystems/filepulse-local-fs/src/test/java/io/streamthoughts/kafka/connect/filepulse/fs/local/reader/AvroFileInputReaderTest.java b/connect-file-pulse-filesystems/filepulse-local-fs/src/test/java/io/streamthoughts/kafka/connect/filepulse/fs/local/reader/AvroFileInputReaderTest.java deleted file mode 100644 index 1ffc33173..000000000 --- a/connect-file-pulse-filesystems/filepulse-local-fs/src/test/java/io/streamthoughts/kafka/connect/filepulse/fs/local/reader/AvroFileInputReaderTest.java +++ /dev/null @@ -1,242 +0,0 @@ -/* - * Copyright 2019-2020 StreamThoughts. - * - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package io.streamthoughts.kafka.connect.filepulse.fs.local.reader; - -import io.streamthoughts.kafka.connect.filepulse.data.TypedStruct; -import io.streamthoughts.kafka.connect.filepulse.reader.FileInputIterator; -import io.streamthoughts.kafka.connect.filepulse.reader.RecordsIterable; -import io.streamthoughts.kafka.connect.filepulse.source.FileContext; -import io.streamthoughts.kafka.connect.filepulse.source.FileRecord; -import io.streamthoughts.kafka.connect.filepulse.source.FileRecordOffset; -import io.streamthoughts.kafka.connect.filepulse.source.LocalFileObjectMeta; -import org.apache.avro.Schema; -import org.apache.avro.SchemaBuilder; -import org.apache.avro.file.DataFileWriter; -import org.apache.avro.generic.GenericData; -import org.apache.avro.generic.GenericDatumWriter; -import org.apache.avro.generic.GenericRecord; -import org.apache.avro.io.DatumWriter; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; - -import java.io.File; -import java.io.IOException; -import java.util.Arrays; -import java.util.HashMap; -import java.util.Map; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; - -public class AvroFileInputReaderTest { - - private static final long NOW = System.currentTimeMillis(); - - private static final Schema DEFAULT_TEST_SCHEMA; - private static final GenericData.Record DEFAULT_GENERIC_RECORD; - - static { - DEFAULT_TEST_SCHEMA = SchemaBuilder.record("test") - .fields() - .requiredString("firstName") - .requiredString("lastName") - .requiredInt("age") - .requiredLong("timestamp") - .endRecord(); - - DEFAULT_GENERIC_RECORD = new GenericData.Record(DEFAULT_TEST_SCHEMA); - DEFAULT_GENERIC_RECORD.put("lastName", "foo"); - DEFAULT_GENERIC_RECORD.put("firstName", "bar"); - DEFAULT_GENERIC_RECORD.put("age", 42); - DEFAULT_GENERIC_RECORD.put("timestamp", NOW); - } - - @Rule - public TemporaryFolder testFolder = new TemporaryFolder(); - - private File file; - private FileContext context; - - private AvroFileInputReader reader; - - @Before - public void setUp() throws IOException { - file = testFolder.newFile(); - context = new FileContext(new LocalFileObjectMeta(file)); - reader = new AvroFileInputReader(); - } - - @After - public void tearDown() { - reader.close(); - } - - @Test - public void shouldConvertGenericRecordWithPrimitiveTypes() { - - final TypedStruct struct = AvroFileInputReader.TypedStructConverter.fromGenericRecord(DEFAULT_GENERIC_RECORD); - - Assert.assertEquals("bar", struct.getString("firstName")); - Assert.assertEquals("foo", struct.getString("lastName")); - Assert.assertEquals(42, struct.getInt("age").intValue()); - Assert.assertEquals(NOW, struct.getLong("timestamp").longValue()); - } - - @Test - public void shouldConvertGenericRecordWithArrayType() { - final Schema schema = SchemaBuilder.record("test") - .fields() - .name("field") - .type(SchemaBuilder.array().items(Schema.create(Schema.Type.STRING))).noDefault() - .endRecord(); - - GenericData.Record avro = new GenericData.Record(schema); - avro.put("field", Arrays.asList("foo", "bar")); - - final TypedStruct struct = AvroFileInputReader.TypedStructConverter.fromGenericRecord(avro); - Assert.assertTrue(Arrays.asList("foo", "bar").containsAll(struct.getArray("field"))); - } - - @Test - public void shouldConvertGenericRecordWithMapType() { - final Schema schema = SchemaBuilder.record("test") - .fields() - .name("field") - .type(SchemaBuilder.map().values(Schema.create(Schema.Type.STRING))).noDefault() - .endRecord(); - - Map map = new HashMap<>(); - map.put("field1", "foo"); - map.put("field2", "bar"); - - GenericData.Record avro = new GenericData.Record(schema); - avro.put("field", map); - - final TypedStruct struct = AvroFileInputReader.TypedStructConverter.fromGenericRecord(avro); - - Assert.assertEquals(map, struct.getMap("field")); - } - - @Test - public void shouldConvertGenericRecordWithUnionType() { - final Schema schema = SchemaBuilder.record("test") - .fields() - .optionalString("field") - .endRecord(); - - GenericData.Record avro = new GenericData.Record(schema); - avro.put("field", "string-value"); - - final TypedStruct struct = AvroFileInputReader.TypedStructConverter.fromGenericRecord(avro); - - Assert.assertEquals( "string-value", struct.getString("field")); - } - - @Test - public void shouldConvertGenericRecordWithRecordType() { - final Schema schema = SchemaBuilder.record("test") - .fields() - .name("field") - .type(DEFAULT_TEST_SCHEMA).noDefault() - .endRecord(); - - GenericData.Record avro = new GenericData.Record(schema); - avro.put("field", DEFAULT_GENERIC_RECORD); - - final TypedStruct struct = AvroFileInputReader.TypedStructConverter.fromGenericRecord(avro); - - TypedStruct field = struct.getStruct("field"); - - Assert.assertEquals("bar", field.getString("firstName")); - Assert.assertEquals("foo", field.getString("lastName")); - Assert.assertEquals(42, field.getInt("age").intValue()); - Assert.assertEquals(NOW, field.getLong("timestamp").longValue()); - } - - @Test - public void shouldReadGivenMultipleAvroRecord() { - - writeGenericRecords( - DEFAULT_TEST_SCHEMA, - DEFAULT_GENERIC_RECORD, - DEFAULT_GENERIC_RECORD, - DEFAULT_GENERIC_RECORD); - - FileInputIterator> iterator = reader.newIterator(context); - - Assert.assertTrue(iterator.hasNext()); - int records = 0; - while (iterator.hasNext()) { - iterator.next(); - records++; - } - Assert.assertEquals(3, records); - } - - @Test - public void shouldSeekToGivenAValidPosition() { - - writeGenericRecords( - DEFAULT_TEST_SCHEMA, - DEFAULT_GENERIC_RECORD, - DEFAULT_GENERIC_RECORD, - DEFAULT_GENERIC_RECORD); - - FileInputIterator> iterator = reader.newIterator(context); - Assert.assertTrue(iterator.hasNext()); - - RecordsIterable> next = iterator.next(); - final FileRecordOffset offset = next.last().offset(); - - // Close the first iterator. - iterator.close(); - - // Get a new iterator. - iterator = reader.newIterator(context); - iterator.seekTo(offset.toSourceOffset()); - - // Attemps to read remaining records. - Assert.assertTrue(iterator.hasNext()); - int records = 0; - while (iterator.hasNext()) { - iterator.next(); - records++; - - } - Assert.assertEquals(2, records); - } - - public void writeGenericRecords(final Schema schema, final GenericRecord...records) { - DatumWriter datumWriter = new GenericDatumWriter<>(schema); - try { - DataFileWriter dataFileWriter = new DataFileWriter<>(datumWriter); - dataFileWriter.create(schema, file); - for (GenericRecord record : records) { - dataFileWriter.append(record); - } - dataFileWriter.close(); - } catch (IOException e) { - Assert.fail(e.getMessage()); - } - } -} \ No newline at end of file diff --git a/connect-file-pulse-filesystems/filepulse-local-fs/src/test/java/io/streamthoughts/kafka/connect/filepulse/fs/local/reader/XMLFileInputReaderTest.java b/connect-file-pulse-filesystems/filepulse-local-fs/src/test/java/io/streamthoughts/kafka/connect/filepulse/fs/local/reader/XMLFileInputReaderTest.java deleted file mode 100644 index 12e9a5c49..000000000 --- a/connect-file-pulse-filesystems/filepulse-local-fs/src/test/java/io/streamthoughts/kafka/connect/filepulse/fs/local/reader/XMLFileInputReaderTest.java +++ /dev/null @@ -1,283 +0,0 @@ -/* - * Copyright 2019-2020 StreamThoughts. - * - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package io.streamthoughts.kafka.connect.filepulse.fs.local.reader; - -import io.streamthoughts.kafka.connect.filepulse.data.Type; -import io.streamthoughts.kafka.connect.filepulse.data.TypedStruct; -import io.streamthoughts.kafka.connect.filepulse.data.TypedValue; -import io.streamthoughts.kafka.connect.filepulse.reader.FileInputIterator; -import io.streamthoughts.kafka.connect.filepulse.source.FileContext; -import io.streamthoughts.kafka.connect.filepulse.source.FileRecord; -import io.streamthoughts.kafka.connect.filepulse.source.LocalFileObjectMeta; -import io.streamthoughts.kafka.connect.filepulse.source.TypedFileRecord; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; - -import java.io.BufferedWriter; -import java.io.File; -import java.io.IOException; -import java.nio.charset.Charset; -import java.nio.file.Files; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -import static io.streamthoughts.kafka.connect.filepulse.fs.local.reader.XMLFileInputReaderConfig.FORCE_ARRAY_ON_FIELDS_CONFIG; -import static io.streamthoughts.kafka.connect.filepulse.fs.local.reader.XMLFileInputReaderConfig.XPATH_QUERY_CONFIG; -import static io.streamthoughts.kafka.connect.filepulse.fs.local.reader.XMLFileInputReaderConfig.XPATH_RESULT_TYPE_CONFIG; - -public class XMLFileInputReaderTest { - - @Rule - public TemporaryFolder testFolder = new TemporaryFolder(); - - private FileContext context; - - private XMLFileInputReader reader; - - @Before - public void setUp() throws IOException { - reader = createNewXMLFileInputReader(DEFAULT_TEST_XML_DOCUMENT); - } - - @After - public void tearDown() { - reader.close(); - } - - @Test - public void should_read_all_records_given_valid_xpath_expression() { - reader.configure(new HashMap(){{ - put(XPATH_QUERY_CONFIG, "//broker"); - }}); - - FileInputIterator> iterator = reader.newIterator(context); - List> records = new ArrayList<>(); - iterator.forEachRemaining(r -> records.addAll(r.collect())); - - Assert.assertEquals(3, records.size()); - - assertTopicPartitionObject(records.get(0).value(), "101", "0"); - assertTopicPartitionObject(records.get(1).value(), "102", "1"); - assertTopicPartitionObject(records.get(2).value(), "103", "2"); - } - - @Test - public void should_read_all_records_given_root_xpath_expression() { - reader.configure(new HashMap(){{ - put(XPATH_QUERY_CONFIG, "/cluster"); - }}); - - FileInputIterator> iterator = reader.newIterator(context); - List> records = new ArrayList<>(); - iterator.forEachRemaining(r -> records.addAll(r.collect())); - - Assert.assertEquals(1, records.size()); - - FileRecord record = records.get(0); - TypedStruct struct = record.value(); - Assert.assertEquals("my-cluster", struct.getString("id")); - Assert.assertEquals("2.3.0", struct.getString("version")); - List brokers = struct.getArray("broker"); - Assert.assertEquals(3, brokers.size()); - - assertTopicPartitionObject(brokers.get(0), "101", "0"); - assertTopicPartitionObject(brokers.get(1), "102", "1"); - assertTopicPartitionObject(brokers.get(2), "103", "2"); - } - - @Test - public void should_ignore_white_space_and_nl_nodes() throws IOException { - try(XMLFileInputReader reader1 = - createNewXMLFileInputReader("data")) { - try(XMLFileInputReader reader2 = - createNewXMLFileInputReader("\n\tdata\n")) - { - Map config = new HashMap() {{ put(XPATH_QUERY_CONFIG, "/"); }}; - - reader1.configure(config); - reader2.configure(config); - - FileRecord rs1 = reader1.newIterator(context).next().last(); - FileRecord rs2 = reader2.newIterator(context).next().last(); - Assert.assertEquals(rs1.value(), rs2.value()); - } - } - } - - @Test - public void should_read_record_given_document_with_cdata_node() throws IOException { - try(XMLFileInputReader reader = createNewXMLFileInputReader(CDATA_TEST_XML_DOCUMENT)) { - reader.configure(new HashMap(){{ - put(XPATH_QUERY_CONFIG, "/"); - }}); - - FileInputIterator> iterator = reader.newIterator(context); - List> records = new ArrayList<>(); - iterator.forEachRemaining(r -> records.addAll(r.collect())); - - Assert.assertEquals(1, records.size()); - Assert.assertEquals("dummy text", records.get(0).value().getString("ROOT")); - } - } - - @Test - public void should_read_record_given_document_with_comment_node() throws IOException { - try(XMLFileInputReader reader = createNewXMLFileInputReader(COMMENT_TEST_XML_DOCUMENT)) { - reader.configure(new HashMap(){{ - put(XPATH_QUERY_CONFIG, "/"); - }}); - - FileInputIterator> iterator = reader.newIterator(context); - List> records = new ArrayList<>(); - iterator.forEachRemaining(r -> records.addAll(r.collect())); - - Assert.assertEquals(1, records.size()); - Assert.assertEquals("dummy text", records.get(0).value().getString("ROOT")); - } - } - - @Test - public void should_read_record_given_node_xpath_expression() { - reader.configure(new HashMap(){{ - put(XPATH_QUERY_CONFIG, "(//broker)[1]/topicPartition/logSize/text()"); - put(XPATH_RESULT_TYPE_CONFIG, "STRING"); - }}); - FileInputIterator> iterator = reader.newIterator(context); - List> records = new ArrayList<>(); - iterator.forEachRemaining(r -> records.addAll(r.collect())); - - Assert.assertEquals(1, records.size()); - Assert.assertEquals("1G", records.get(0).value().getString(TypedFileRecord.DEFAULT_MESSAGE_FIELD)); - } - - @Test - public void should_read_record_given_valid_force_array_fields() { - reader.configure(new HashMap(){{ - put(XPATH_QUERY_CONFIG, "//broker[1]"); - put(FORCE_ARRAY_ON_FIELDS_CONFIG, "broker.topicPartition, broker.topicPartition.numSegments"); - }}); - - FileInputIterator> iterator = reader.newIterator(context); - List> records = new ArrayList<>(); - iterator.forEachRemaining(r -> records.addAll(r.collect())); - - Assert.assertEquals(1, records.size()); - - TypedStruct record = records.get(0).value(); - Assert.assertEquals(Type.ARRAY, record.get("topicPartition").type()); - Assert.assertEquals(1, record.get("topicPartition").getArray().size()); - TypedValue segment = ((TypedStruct) record.get("topicPartition").getArray().iterator().next()).find("numSegments"); - Assert.assertEquals(Type.ARRAY, segment.type()); - Assert.assertEquals(1, segment.getArray().size()); - } - - @Test - public void should_read_record_given_valid_force_array_fields_and_default_xpath() { - reader.configure(new HashMap(){{ - put(FORCE_ARRAY_ON_FIELDS_CONFIG, "cluster.broker.topicPartition"); - }}); - - FileInputIterator> iterator = reader.newIterator(context); - List> records = new ArrayList<>(); - iterator.forEachRemaining(r -> records.addAll(r.collect())); - - TypedStruct record = records.get(0).value(); - TypedStruct element = (TypedStruct)record.find("cluster.broker").getArray().iterator().next(); - Assert.assertEquals(Type.ARRAY, element.find("topicPartition").type()); - } - - @Test - public void should_read_record_given_single_text_node_with_attrs() throws IOException { - try(XMLFileInputReader reader = createNewXMLFileInputReader(TEXT_NODE_TEST_XML_DOCUMENT)) { - reader.configure(new HashMap()); - FileInputIterator> iterator = reader.newIterator(context); - List> records = new ArrayList<>(); - iterator.forEachRemaining(r -> records.addAll(r.collect())); - - Assert.assertEquals(1, records.size()); - Assert.assertEquals("dummy text", records.get(0).value().find("ROOT.value").getString()); - Assert.assertEquals("dummy attr", records.get(0).value().find("ROOT.attr").getString()); - } - } - - private XMLFileInputReader createNewXMLFileInputReader(final String xmlDocument) throws IOException { - File file = testFolder.newFile(); - try (BufferedWriter bw = Files.newBufferedWriter(file.toPath(), Charset.defaultCharset())) { - bw.append(xmlDocument); - bw.flush(); - context = new FileContext(new LocalFileObjectMeta(file)); - } - return new XMLFileInputReader(); - } - - private static void assertTopicPartitionObject(final TypedStruct struct, - final String expectedId, - final String expectedNum) { - Assert.assertEquals(expectedId, struct.getString("id")); - TypedStruct topicPartition = struct.getStruct("topicPartition"); - Assert.assertNotNull(topicPartition); - Assert.assertEquals("topicA", topicPartition.getString("topic")); - Assert.assertEquals(expectedNum, topicPartition.getString("num")); - Assert.assertEquals("true", topicPartition.getString("insync")); - Assert.assertEquals("0", topicPartition.getString("earliestOffset")); - Assert.assertEquals("100", topicPartition.getString("endLogOffset")); - Assert.assertEquals("1G", topicPartition.getString("logSize")); - Assert.assertEquals("1", topicPartition.getString("numSegments")); - } - - private static final String TEXT_NODE_TEST_XML_DOCUMENT = "dummy text"; - - private static final String COMMENT_TEST_XML_DOCUMENT = "dummy text"; - - private static final String CDATA_TEST_XML_DOCUMENT = "\n\t\n"; - - private static final String DEFAULT_TEST_XML_DOCUMENT = "" + - "\n" + - "\t\n" + - "\t\t\n" + - "\t\t\t0\n" + - "\t\t\t100\n" + - "\t\t\t1G\n" + - "\t\t\t1\n" + - "\t\t\n" + - "\t\n" + - "\t\n" + - "\t\t\n" + - "\t\t\t0\n" + - "\t\t\t100\n" + - "\t\t\t1G\n" + - "\t\t\t1\n" + - "\t\t\n" + - "\t\n" + - "\t\n" + - "\t\t\n" + - "\t\t\t0\n" + - "\t\t\t100\n" + - "\t\t\t1G\n" + - "\t\t\t1\n" + - "\t\t\n" + - "\t\t\t\n" + - ""; -} \ No newline at end of file diff --git a/connect-file-pulse-filesystems/filepulse-local-fs/src/test/java/io/streamthoughts/kafka/connect/filepulse/fs/local/reader/BytesArrayInputReaderTest.java b/connect-file-pulse-filesystems/filepulse-local-fs/src/test/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/BytesArrayInputReaderTest.java similarity index 94% rename from connect-file-pulse-filesystems/filepulse-local-fs/src/test/java/io/streamthoughts/kafka/connect/filepulse/fs/local/reader/BytesArrayInputReaderTest.java rename to connect-file-pulse-filesystems/filepulse-local-fs/src/test/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/BytesArrayInputReaderTest.java index 9515bb67f..72f62809d 100644 --- a/connect-file-pulse-filesystems/filepulse-local-fs/src/test/java/io/streamthoughts/kafka/connect/filepulse/fs/local/reader/BytesArrayInputReaderTest.java +++ b/connect-file-pulse-filesystems/filepulse-local-fs/src/test/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/BytesArrayInputReaderTest.java @@ -16,7 +16,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.streamthoughts.kafka.connect.filepulse.fs.local.reader; +package io.streamthoughts.kafka.connect.filepulse.fs.reader; import io.streamthoughts.kafka.connect.filepulse.data.Type; import io.streamthoughts.kafka.connect.filepulse.data.TypedStruct; @@ -49,7 +49,7 @@ public class BytesArrayInputReaderTest { private File file; private FileContext context; - private BytesArrayInputReader reader; + private LocalBytesArrayInputReader reader; @Before public void setUp() throws IOException { @@ -60,7 +60,7 @@ public void setUp() throws IOException { context = new FileContext(new LocalFileObjectMeta(file)); } - reader = new BytesArrayInputReader(); + reader = new LocalBytesArrayInputReader(); } @After diff --git a/connect-file-pulse-filesystems/pom.xml b/connect-file-pulse-filesystems/pom.xml index 135a68810..cfc21555e 100644 --- a/connect-file-pulse-filesystems/pom.xml +++ b/connect-file-pulse-filesystems/pom.xml @@ -32,6 +32,7 @@ pom + filepulse-commons-fs filepulse-local-fs @@ -53,6 +54,14 @@ org.apache.kafka connect-api + + junit + junit + + + org.mockito + mockito-core + - \ No newline at end of file + diff --git a/connect-file-pulse-plugin/config/quickstart-connect-file-pulse-csv.properties b/connect-file-pulse-plugin/config/quickstart-connect-file-pulse-csv.properties index fac6f8023..72318daeb 100644 --- a/connect-file-pulse-plugin/config/quickstart-connect-file-pulse-csv.properties +++ b/connect-file-pulse-plugin/config/quickstart-connect-file-pulse-csv.properties @@ -28,11 +28,11 @@ filters.ParseDelimitedRow.extractColumnName=headers filters.ParseDelimitedRow.trimColumn=true filters.ParseDelimitedRow.type=io.streamthoughts.kafka.connect.filepulse.filter.DelimitedRowFilter skip.headers=1 -task.reader.class=io.streamthoughts.kafka.connect.filepulse.fs.local.reader.RowFileInputReader +task.reader.class=io.streamthoughts.kafka.connect.filepulse.fs.reader.LocalRowFileInputReader # File scanning fs.cleanup.policy.class=io.streamthoughts.kafka.connect.filepulse.clean.LogCleanupPolicy -fs.scanner.class=io.streamthoughts.kafka.connect.filepulse.fs.local.LocalFSDirectoryListing +fs.scanner.class=io.streamthoughts.kafka.connect.filepulse.fs.LocalFSDirectoryListing fs.scan.directory.path=/tmp/kafka-connect/examples/ fs.scan.interval.ms=10000 diff --git a/connect-file-pulse-plugin/config/quickstart-connect-file-pulse-log4j.properties b/connect-file-pulse-plugin/config/quickstart-connect-file-pulse-log4j.properties index 30a2f04e0..3276bd58a 100644 --- a/connect-file-pulse-plugin/config/quickstart-connect-file-pulse-log4j.properties +++ b/connect-file-pulse-plugin/config/quickstart-connect-file-pulse-log4j.properties @@ -36,12 +36,12 @@ filters.ParseLog4jLog.type=io.streamthoughts.kafka.connect.filepulse.filter.Grok filters.ParseLog4jLog.ignoreFailure=true # File scanning -fs.scanner.class=io.streamthoughts.kafka.connect.filepulse.fs.local.LocalFSDirectoryListing +fs.scanner.class=io.streamthoughts.kafka.connect.filepulse.fs.LocalFSDirectoryListing fs.scan.directory.path=/var/log/kafka/ fs.scan.interval.ms=10000 fs.cleanup.policy.class=io.streamthoughts.kafka.connect.filepulse.clean.LogCleanupPolicy -task.reader.class=io.streamthoughts.kafka.connect.filepulse.fs.local.reader.RowFileInputReader +task.reader.class=io.streamthoughts.kafka.connect.filepulse.fs.reader.LocalRowFileInputReader # Internal Reporting internal.kafka.reporter.bootstrap.servers=localhost:9092 diff --git a/connect-file-pulse-plugin/src/main/java/io/streamthoughts/kafka/connect/filepulse/config/CommonConfig.java b/connect-file-pulse-plugin/src/main/java/io/streamthoughts/kafka/connect/filepulse/config/CommonConfig.java index 9498be84a..cfa4b6bb2 100644 --- a/connect-file-pulse-plugin/src/main/java/io/streamthoughts/kafka/connect/filepulse/config/CommonConfig.java +++ b/connect-file-pulse-plugin/src/main/java/io/streamthoughts/kafka/connect/filepulse/config/CommonConfig.java @@ -20,7 +20,7 @@ import io.streamthoughts.kafka.connect.filepulse.offset.DefaultOffsetPolicy; import io.streamthoughts.kafka.connect.filepulse.source.SourceOffsetPolicy; -import io.streamthoughts.kafka.connect.filepulse.fs.local.reader.RowFileInputReader; +import io.streamthoughts.kafka.connect.filepulse.fs.reader.LocalRowFileInputReader; import org.apache.kafka.clients.CommonClientConfigs; import org.apache.kafka.clients.producer.ProducerConfig; import org.apache.kafka.common.config.AbstractConfig; @@ -69,7 +69,7 @@ public class CommonConfig extends AbstractConfig { static ConfigDef getConf() { return new ConfigDef() .define(FILE_READER_CLASS_CONFIG, ConfigDef.Type.CLASS, - RowFileInputReader.class, ConfigDef.Importance.HIGH, FILE_READER_CLASS_CONFIG_DOC) + LocalRowFileInputReader.class, ConfigDef.Importance.HIGH, FILE_READER_CLASS_CONFIG_DOC) .define(OUTPUT_TOPIC_CONFIG, ConfigDef.Type.STRING, ConfigDef.Importance.HIGH, OUTPUT_TOPIC_DOC) diff --git a/connect-file-pulse-plugin/src/main/java/io/streamthoughts/kafka/connect/filepulse/config/ConnectorConfig.java b/connect-file-pulse-plugin/src/main/java/io/streamthoughts/kafka/connect/filepulse/config/ConnectorConfig.java index 9eda60b5d..42950a433 100644 --- a/connect-file-pulse-plugin/src/main/java/io/streamthoughts/kafka/connect/filepulse/config/ConnectorConfig.java +++ b/connect-file-pulse-plugin/src/main/java/io/streamthoughts/kafka/connect/filepulse/config/ConnectorConfig.java @@ -21,7 +21,7 @@ import io.streamthoughts.kafka.connect.filepulse.clean.FileCleanupPolicy; import io.streamthoughts.kafka.connect.filepulse.fs.FileSystemListing; import io.streamthoughts.kafka.connect.filepulse.fs.FileListFilter; -import io.streamthoughts.kafka.connect.filepulse.fs.local.LocalFSDirectoryListing; +import io.streamthoughts.kafka.connect.filepulse.fs.LocalFSDirectoryListing; import org.apache.kafka.common.config.ConfigDef; import java.util.Collections; diff --git a/connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/local/reader/FileInputMetadataReader.java b/connect-file-pulse-plugin/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/FileInputMetadataReader.java similarity index 78% rename from connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/local/reader/FileInputMetadataReader.java rename to connect-file-pulse-plugin/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/FileInputMetadataReader.java index 2d181825c..c3196831e 100644 --- a/connect-file-pulse-filesystems/filepulse-local-fs/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/local/reader/FileInputMetadataReader.java +++ b/connect-file-pulse-plugin/src/main/java/io/streamthoughts/kafka/connect/filepulse/fs/reader/FileInputMetadataReader.java @@ -17,16 +17,18 @@ * limitations under the License. */ -package io.streamthoughts.kafka.connect.filepulse.fs.local.reader; +package io.streamthoughts.kafka.connect.filepulse.fs.reader; import io.streamthoughts.kafka.connect.filepulse.data.TypedStruct; import io.streamthoughts.kafka.connect.filepulse.filter.FilterContext; +import io.streamthoughts.kafka.connect.filepulse.fs.reader.text.BytesRecordOffset; import io.streamthoughts.kafka.connect.filepulse.reader.FileInputIterator; +import io.streamthoughts.kafka.connect.filepulse.reader.IteratorManager; import io.streamthoughts.kafka.connect.filepulse.reader.RecordsIterable; import io.streamthoughts.kafka.connect.filepulse.source.FileContext; -import io.streamthoughts.kafka.connect.filepulse.source.FileRecord; -import io.streamthoughts.kafka.connect.filepulse.source.LocalFileObjectMeta; +import io.streamthoughts.kafka.connect.filepulse.source.FileObjectMeta; import io.streamthoughts.kafka.connect.filepulse.source.FileObjectOffset; +import io.streamthoughts.kafka.connect.filepulse.source.FileRecord; import io.streamthoughts.kafka.connect.filepulse.source.TypedFileRecord; import java.util.Collections; @@ -36,7 +38,9 @@ /** * Send a single record containing file metadata. */ -public class FileInputMetadataReader extends AbstractFileInputReader { +public class FileInputMetadataReader extends BaseLocalFileInputReader { + + public static final String METADATA_RECORD_NAME = "io.streamthoughts.kafka.connect.filepulse.FileMetadata"; /** * {@inheritDoc} @@ -45,18 +49,17 @@ public class FileInputMetadataReader extends AbstractFileInputReader { protected FileInputIterator> newIterator(final FileContext context, final IteratorManager iteratorManager) { - - final LocalFileObjectMeta metadata = (LocalFileObjectMeta)context.metadata(); - TypedFileRecord record = new TypedFileRecord( - BytesRecordOffset.empty(), - TypedStruct.create("kafka.connect.filepulse.FileMetadata") + final FileObjectMeta metadata = context.metadata(); + final TypedStruct struct = TypedStruct.create(METADATA_RECORD_NAME) .put("name", metadata.name()) - .put("path", metadata.path()) - .put("hash", metadata.contentDigest().digest()) + .put("uri", metadata.stringURI()) + .put("contentDigest", metadata.contentDigest().digest()) + .put("contentDigestAlgorithm", metadata.contentDigest().algorithm()) .put("lastModified", metadata.lastModified()) .put("size", metadata.contentLength()) - .put("inode", metadata.inode()) - ); + .put("metadata", metadata.userDefinedMetadata()); + + TypedFileRecord record = new TypedFileRecord(BytesRecordOffset.empty(), struct); return new DelegatingFileInputIterator(context, Collections.singleton(record).iterator()); } diff --git a/connect-file-pulse-plugin/src/main/java/io/streamthoughts/kafka/connect/filepulse/source/DefaultFileRecordsPollingConsumer.java b/connect-file-pulse-plugin/src/main/java/io/streamthoughts/kafka/connect/filepulse/source/DefaultFileRecordsPollingConsumer.java index 6ce1a549e..116233476 100644 --- a/connect-file-pulse-plugin/src/main/java/io/streamthoughts/kafka/connect/filepulse/source/DefaultFileRecordsPollingConsumer.java +++ b/connect-file-pulse-plugin/src/main/java/io/streamthoughts/kafka/connect/filepulse/source/DefaultFileRecordsPollingConsumer.java @@ -302,9 +302,10 @@ private FileInputIterator> getOrCloseIteratorIfNoMoreRec private void closeIterator(final FileInputIterator> iterator, final Exception cause) { try { + LOG.info("Closing iterator for source {} ", iterator.context().metadata()); iterator.close(); } catch (final Exception e) { - LOG.debug("Error while closing file '{}'", iterator.context(), e); + LOG.debug("Error while closing file '{}'", iterator.context().metadata(), e); } finally { deleteFileQueueAndInvokeListener(iterator.context(), cause); } diff --git a/pom.xml b/pom.xml index 78e2e2706..da60fdbce 100644 --- a/pom.xml +++ b/pom.xml @@ -29,7 +29,9 @@ 2.0.0-SNAPSHOT Kafka Connect Source File Pulse Reactor - Connect File Pulse is a multi-purpose source connector for streaming files from a local filesystem to Kafka. + Connect File Pulse is a multi-purpose source connector for streaming files from a local filesystem to + Kafka. + connect-file-pulse-api @@ -265,11 +267,17 @@ 2.5 - - org.apache.commons - commons-compress - 1.20 - + + org.apache.commons + commons-compress + 1.20 + + + + org.apache.avro + avro + 1.10.1 + com.jsoniter