From 43674cea0bcec919156ff3c0bfbf58dd22aee4c8 Mon Sep 17 00:00:00 2001 From: Srikanth Padakanti Date: Tue, 5 May 2026 18:28:31 -0500 Subject: [PATCH 01/21] Add tail mode to file source for continuous log tailing Signed-off-by: Srikanth Padakanti --- data-prepper-core/build.gradle | 1 + .../plugins/source/file/FileSourceConfig.java | 73 - .../source/file/FileSourceConfigTest.java | 46 - data-prepper-plugins/file-source/build.gradle | 62 + .../plugins/source/file/FileSourceTailIT.java | 914 +++++++++++ .../plugins/source/file/CheckpointEntry.java | 77 + .../source/file/CheckpointRegistry.java | 149 ++ .../plugins/source/file/CheckpointStatus.java | 40 + .../file/DefaultFileSystemOperations.java | 54 + .../plugins/source/file/DirectoryWatcher.java | 403 +++++ .../plugins/source/file/FileFormat.java | 8 + .../plugins/source/file/FileIdentity.java | 105 ++ .../plugins/source/file/FileSource.java | 155 +- .../plugins/source/file/FileSourceConfig.java | 270 +++ .../source/file/FileSystemOperations.java | 32 + .../plugins/source/file/FileTailMetrics.java | 122 ++ .../plugins/source/file/GlobPathResolver.java | 170 ++ .../plugins/source/file/PendingFile.java | 44 + .../plugins/source/file/RotationDetector.java | 93 ++ .../plugins/source/file/RotationResult.java | 41 + .../plugins/source/file/RotationType.java | 18 + .../plugins/source/file/StartPosition.java | 48 + .../plugins/source/file/TailFileReader.java | 499 ++++++ .../source/file/TailFileReaderContext.java | 167 ++ .../source/file/TailFileReaderPool.java | 212 +++ .../source/file/CheckpointEntryTest.java | 141 ++ .../source/file/CheckpointRegistryTest.java | 338 ++++ .../source/file/CheckpointStatusTest.java | 50 + .../file/DefaultFileSystemOperationsTest.java | 179 ++ .../source/file/DirectoryWatcherTest.java | 1087 ++++++++++++ .../plugins/source/file/FileIdentityTest.java | 289 ++++ .../source/file/FileSourceConfigTest.java | 225 +++ .../plugins/source/file/FileSourceTests.java | 211 ++- .../source/file/FileTailMetricsTest.java | 221 +++ .../source/file/GlobPathResolverTest.java | 259 +++ .../plugins/source/file/PendingFileTest.java | 83 + .../source/file/RotationDetectorTest.java | 283 ++++ .../source/file/RotationResultTest.java | 93 ++ .../plugins/source/file/RotationTypeTest.java | 52 + .../source/file/StartPositionTest.java | 55 + .../source/file/TailFileReaderPoolTest.java | 569 +++++++ .../source/file/TailFileReaderTest.java | 1452 +++++++++++++++++ .../test-file-source-invalid-json.tst | 2 + .../test/resources/test-file-source-json.tst | 2 + .../test/resources/test-file-source-plain.tst | 2 + settings.gradle | 1 + 46 files changed, 9245 insertions(+), 152 deletions(-) delete mode 100644 data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileSourceConfig.java delete mode 100644 data-prepper-plugins/common/src/test/java/org/opensearch/dataprepper/plugins/source/file/FileSourceConfigTest.java create mode 100644 data-prepper-plugins/file-source/build.gradle create mode 100644 data-prepper-plugins/file-source/src/integrationTest/java/org/opensearch/dataprepper/plugins/source/file/FileSourceTailIT.java create mode 100644 data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/CheckpointEntry.java create mode 100644 data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/CheckpointRegistry.java create mode 100644 data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/CheckpointStatus.java create mode 100644 data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/DefaultFileSystemOperations.java create mode 100644 data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/DirectoryWatcher.java rename data-prepper-plugins/{common => file-source}/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileFormat.java (79%) create mode 100644 data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileIdentity.java rename data-prepper-plugins/{common => file-source}/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileSource.java (53%) create mode 100644 data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileSourceConfig.java create mode 100644 data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileSystemOperations.java create mode 100644 data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileTailMetrics.java create mode 100644 data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/GlobPathResolver.java create mode 100644 data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/PendingFile.java create mode 100644 data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/RotationDetector.java create mode 100644 data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/RotationResult.java create mode 100644 data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/RotationType.java create mode 100644 data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/StartPosition.java create mode 100644 data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/TailFileReader.java create mode 100644 data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/TailFileReaderContext.java create mode 100644 data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/TailFileReaderPool.java create mode 100644 data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/CheckpointEntryTest.java create mode 100644 data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/CheckpointRegistryTest.java create mode 100644 data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/CheckpointStatusTest.java create mode 100644 data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/DefaultFileSystemOperationsTest.java create mode 100644 data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/DirectoryWatcherTest.java create mode 100644 data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/FileIdentityTest.java create mode 100644 data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/FileSourceConfigTest.java rename data-prepper-plugins/{common => file-source}/src/test/java/org/opensearch/dataprepper/plugins/source/file/FileSourceTests.java (65%) create mode 100644 data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/FileTailMetricsTest.java create mode 100644 data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/GlobPathResolverTest.java create mode 100644 data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/PendingFileTest.java create mode 100644 data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/RotationDetectorTest.java create mode 100644 data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/RotationResultTest.java create mode 100644 data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/RotationTypeTest.java create mode 100644 data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/StartPositionTest.java create mode 100644 data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/TailFileReaderPoolTest.java create mode 100644 data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/TailFileReaderTest.java create mode 100644 data-prepper-plugins/file-source/src/test/resources/test-file-source-invalid-json.tst create mode 100644 data-prepper-plugins/file-source/src/test/resources/test-file-source-json.tst create mode 100644 data-prepper-plugins/file-source/src/test/resources/test-file-source-plain.tst diff --git a/data-prepper-core/build.gradle b/data-prepper-core/build.gradle index db2bd9b013..169c9ff2fa 100644 --- a/data-prepper-core/build.gradle +++ b/data-prepper-core/build.gradle @@ -39,6 +39,7 @@ dependencies { implementation project(':data-prepper-plugin-framework') testImplementation project(':data-prepper-plugin-framework').sourceSets.test.output testImplementation project(':data-prepper-plugins:common').sourceSets.test.output + testImplementation project(':data-prepper-plugins:file-source') implementation 'com.fasterxml.jackson.core:jackson-databind' implementation 'com.fasterxml.jackson.dataformat:jackson-dataformat-yaml' implementation libs.reflections.core diff --git a/data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileSourceConfig.java b/data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileSourceConfig.java deleted file mode 100644 index 9eb8dd961d..0000000000 --- a/data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileSourceConfig.java +++ /dev/null @@ -1,73 +0,0 @@ -/* - * Copyright OpenSearch Contributors - * SPDX-License-Identifier: Apache-2.0 - */ - -package org.opensearch.dataprepper.plugins.source.file; - -import com.fasterxml.jackson.annotation.JsonIgnore; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.base.Preconditions; -import jakarta.validation.constraints.AssertTrue; -import org.opensearch.dataprepper.model.configuration.PluginModel; -import org.opensearch.dataprepper.plugins.codec.CompressionOption; - -import java.util.Objects; - -public class FileSourceConfig { - static final String ATTRIBUTE_PATH = "path"; - static final String ATTRIBUTE_TYPE = "record_type"; - static final String ATTRIBUTE_FORMAT = "format"; - static final int DEFAULT_TIMEOUT = 5_000; - static final String DEFAULT_TYPE = "string"; - static final String DEFAULT_FORMAT = "plain"; - static final String EVENT_TYPE = "event"; - - - @JsonProperty(ATTRIBUTE_PATH) - private String filePathToRead; - - @JsonProperty(ATTRIBUTE_FORMAT) - private String format = DEFAULT_FORMAT; - - @JsonProperty(ATTRIBUTE_TYPE) - private String recordType = DEFAULT_TYPE; - - @JsonProperty("codec") - private PluginModel codec; - - @JsonProperty("compression") - private CompressionOption compression = CompressionOption.NONE; - - public String getFilePathToRead() { - return filePathToRead; - } - - @JsonIgnore - public FileFormat getFormat() { - return FileFormat.getByName(format); - } - - public String getRecordType() { - return recordType; - } - - public PluginModel getCodec() { - return codec; - } - - public CompressionOption getCompression() { - return compression; - } - - void validate() { - Objects.requireNonNull(filePathToRead, "File path is required"); - Preconditions.checkArgument(recordType.equals(EVENT_TYPE) || recordType.equals(DEFAULT_TYPE), "Invalid type: must be either [event] or [string]"); - Preconditions.checkArgument(format.equals(DEFAULT_FORMAT) || format.equals("json"), "Invalid file format. Options are [json] and [plain]"); - } - - @AssertTrue(message = "The file source requires recordType to be event when using a codec.") - boolean codeRequiresRecordTypeEvent() { - return codec == null || recordType.equals(EVENT_TYPE); - } -} diff --git a/data-prepper-plugins/common/src/test/java/org/opensearch/dataprepper/plugins/source/file/FileSourceConfigTest.java b/data-prepper-plugins/common/src/test/java/org/opensearch/dataprepper/plugins/source/file/FileSourceConfigTest.java deleted file mode 100644 index 9208c52b66..0000000000 --- a/data-prepper-plugins/common/src/test/java/org/opensearch/dataprepper/plugins/source/file/FileSourceConfigTest.java +++ /dev/null @@ -1,46 +0,0 @@ -/* - * Copyright OpenSearch Contributors - * SPDX-License-Identifier: Apache-2.0 - */ - -package org.opensearch.dataprepper.plugins.source.file; - -import com.fasterxml.jackson.databind.ObjectMapper; -import org.junit.jupiter.params.ParameterizedTest; -import org.junit.jupiter.params.provider.CsvSource; -import org.junit.jupiter.params.provider.ValueSource; -import org.opensearch.dataprepper.model.configuration.PluginModel; - -import java.util.Collections; -import java.util.Map; - -import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.Matchers.equalTo; - -class FileSourceConfigTest { - private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); - - @ParameterizedTest - @ValueSource(strings = {FileSourceConfig.EVENT_TYPE, FileSourceConfig.DEFAULT_FORMAT}) - void codeRequiresRecordTypeEvent_returns_true_if_no_codec(final String recordType) { - final Map fileConfigMap = Map.of(FileSourceConfig.ATTRIBUTE_TYPE, recordType); - final FileSourceConfig objectUnderTest = OBJECT_MAPPER.convertValue(fileConfigMap, FileSourceConfig.class); - - assertThat(objectUnderTest.codeRequiresRecordTypeEvent(), equalTo(true)); - } - - @ParameterizedTest - @CsvSource({ - FileSourceConfig.EVENT_TYPE + ",true", - FileSourceConfig.DEFAULT_FORMAT + ",false" - }) - void codeRequiresRecordTypeEvent_returns_expected_value_when_there_is_a_codec(final String recordType, final boolean expected) { - final Map fileConfigMap = Map.of( - FileSourceConfig.ATTRIBUTE_TYPE, recordType, - "codec", new PluginModel("fake_codec", Collections.emptyMap()) - ); - final FileSourceConfig objectUnderTest = OBJECT_MAPPER.convertValue(fileConfigMap, FileSourceConfig.class); - - assertThat(objectUnderTest.codeRequiresRecordTypeEvent(), equalTo(expected)); - } -} \ No newline at end of file diff --git a/data-prepper-plugins/file-source/build.gradle b/data-prepper-plugins/file-source/build.gradle new file mode 100644 index 0000000000..82f3e7b666 --- /dev/null +++ b/data-prepper-plugins/file-source/build.gradle @@ -0,0 +1,62 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + +dependencies { + implementation project(':data-prepper-api') + implementation project(':data-prepper-plugins:common') + implementation 'com.fasterxml.jackson.core:jackson-databind' + implementation 'io.micrometer:micrometer-core' + testImplementation project(':data-prepper-plugins:blocking-buffer') + testImplementation project(':data-prepper-test:test-event') + testImplementation 'org.awaitility:awaitility' + testImplementation 'com.fasterxml.jackson.datatype:jackson-datatype-jsr310' +} + +test { + forkEvery = 1 + maxHeapSize = '512m' +} + +sourceSets { + integrationTest { + java { + compileClasspath = main.output + test.output + compileClasspath + runtimeClasspath = main.output + test.output + runtimeClasspath + srcDir file('src/integrationTest/java') + } + } +} + +configurations { + integrationTestImplementation.extendsFrom testImplementation + integrationTestRuntimeOnly.extendsFrom testRuntimeOnly +} + +task integrationTest(type: Test) { + group = 'verification' + testClassesDirs = sourceSets.integrationTest.output.classesDirs + classpath = sourceSets.integrationTest.runtimeClasspath + useJUnitPlatform() + + filter { + includeTestsMatching '*IT' + } +} + +jacocoTestCoverageVerification { + dependsOn jacocoTestReport + violationRules { + rule { + limit { + minimum = 1.0 + } + } + } +} diff --git a/data-prepper-plugins/file-source/src/integrationTest/java/org/opensearch/dataprepper/plugins/source/file/FileSourceTailIT.java b/data-prepper-plugins/file-source/src/integrationTest/java/org/opensearch/dataprepper/plugins/source/file/FileSourceTailIT.java new file mode 100644 index 0000000000..7caf011e23 --- /dev/null +++ b/data-prepper-plugins/file-source/src/integrationTest/java/org/opensearch/dataprepper/plugins/source/file/FileSourceTailIT.java @@ -0,0 +1,914 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + +package org.opensearch.dataprepper.plugins.source.file; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.datatype.jsr310.JavaTimeModule; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.io.TempDir; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; +import org.mockito.junit.jupiter.MockitoSettings; +import org.mockito.quality.Strictness; +import org.opensearch.dataprepper.metrics.PluginMetrics; +import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSet; +import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSetManager; +import org.opensearch.dataprepper.model.buffer.Buffer; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.event.EventFactory; +import org.opensearch.dataprepper.model.plugin.PluginFactory; +import org.opensearch.dataprepper.model.record.Record; +import org.opensearch.dataprepper.event.TestEventFactory; + +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.StandardOpenOption; +import java.time.Duration; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.Consumer; +import java.util.stream.Collectors; + +import static org.awaitility.Awaitility.await; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.greaterThanOrEqualTo; +import static org.hamcrest.Matchers.hasSize; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyCollection; +import static org.mockito.ArgumentMatchers.anyInt; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.mock; + +@ExtendWith(MockitoExtension.class) +@MockitoSettings(strictness = Strictness.LENIENT) +class FileSourceTailIT { + + private static final Duration WAIT_TIMEOUT = Duration.ofSeconds(15); + private static final long ROTATION_DETECTION_DELAY_MS = 2000; + private static final long SHORT_DELAY_MS = 1000; + private static final long CLOSE_INACTIVE_WAIT_MS = 5000; + private static final long DELETION_DETECTION_DELAY_MS = 3000; + private static final long ACK_RETRY_INTERVAL_MS = 500; + + @TempDir + Path tempDir; + + @Mock + private PluginFactory pluginFactory; + + @Mock + private AcknowledgementSetManager acknowledgementSetManager; + + private PluginMetrics pluginMetrics; + private EventFactory eventFactory; + private Buffer> buffer; + private List> capturedRecords; + private FileSource fileSource; + + @BeforeEach + @SuppressWarnings("unchecked") + void setUp() throws Exception { + pluginMetrics = PluginMetrics.fromNames("file", "test-pipeline"); + eventFactory = TestEventFactory.getTestEventFactory(); + capturedRecords = Collections.synchronizedList(new ArrayList<>()); + + buffer = (Buffer>) mock(Buffer.class); + doAnswer(invocation -> { + Record record = invocation.getArgument(0); + capturedRecords.add(record); + return null; + }).when(buffer).write(any(Record.class), anyInt()); + doAnswer(invocation -> { + Collection> records = invocation.getArgument(0); + capturedRecords.addAll(records); + return null; + }).when(buffer).writeAll(anyCollection(), anyInt()); + } + + @AfterEach + void tearDown() { + if (fileSource != null) { + fileSource.stop(); + } + } + + @Test + void tail_mode_reads_existing_lines_with_start_position_beginning() throws Exception { + final Path logFile = tempDir.resolve("app.log"); + Files.write(logFile, List.of("line one", "line two", "line three")); + + final FileSourceConfig config = createTailConfig( + logFile.toString(), null, "beginning"); + + fileSource = createSource(config); + fileSource.start(buffer); + + await().atMost(WAIT_TIMEOUT).untilAsserted(() -> { + assertThat(capturedRecords, hasSize(3)); + assertThat(eventMessage(capturedRecords.get(0)), equalTo("line one")); + assertThat(eventMessage(capturedRecords.get(1)), equalTo("line two")); + assertThat(eventMessage(capturedRecords.get(2)), equalTo("line three")); + }); + } + + @Test + void tail_mode_with_start_position_end_skips_existing_content() throws Exception { + final Path logFile = tempDir.resolve("existing.log"); + Files.write(logFile, List.of("old line 1", "old line 2")); + + final FileSourceConfig config = createTailConfig( + logFile.toString(), null, "end"); + + fileSource = createSource(config); + fileSource.start(buffer); + + Thread.sleep(ROTATION_DETECTION_DELAY_MS); + assertThat(capturedRecords, hasSize(0)); + + appendLine(logFile, "new line after start"); + + await().atMost(WAIT_TIMEOUT).untilAsserted(() -> { + assertThat(capturedRecords, hasSize(1)); + assertThat(eventMessage(capturedRecords.get(0)), equalTo("new line after start")); + }); + } + + @Test + void tail_mode_follows_appended_lines() throws Exception { + final Path logFile = tempDir.resolve("append.log"); + Files.write(logFile, "initial\n".getBytes(StandardCharsets.UTF_8)); + + final FileSourceConfig config = createTailConfig( + logFile.toString(), null, "beginning"); + + fileSource = createSource(config); + fileSource.start(buffer); + + await().atMost(WAIT_TIMEOUT).untilAsserted(() -> + assertThat(capturedRecords, hasSize(1))); + + appendLine(logFile, "appended line 1"); + appendLine(logFile, "appended line 2"); + + await().atMost(WAIT_TIMEOUT).untilAsserted(() -> { + assertThat(capturedRecords, hasSize(3)); + assertThat(eventMessage(capturedRecords.get(1)), equalTo("appended line 1")); + assertThat(eventMessage(capturedRecords.get(2)), equalTo("appended line 2")); + }); + } + + @Test + void tail_mode_discovers_files_via_glob_pattern() throws Exception { + Files.write(tempDir.resolve("server1.log"), List.of("from server 1")); + Files.write(tempDir.resolve("server2.log"), List.of("from server 2")); + Files.write(tempDir.resolve("server.txt"), List.of("should not be read")); + + final String globPattern = tempDir.resolve("*.log").toString(); + final FileSourceConfig config = createTailConfig( + null, List.of(globPattern), "beginning"); + + fileSource = createSource(config); + fileSource.start(buffer); + + await().atMost(WAIT_TIMEOUT).untilAsserted(() -> + assertThat(capturedRecords, hasSize(2))); + } + + @Test + void tail_mode_excludes_paths_matching_exclude_pattern() throws Exception { + Files.write(tempDir.resolve("app.log"), List.of("app log line")); + Files.write(tempDir.resolve("debug.log"), List.of("debug log line")); + + final String globPattern = tempDir.resolve("*.log").toString(); + final String excludePattern = tempDir.resolve("debug*").toString(); + final FileSourceConfig config = createTailConfigWithExclude( + globPattern, excludePattern, "beginning"); + + fileSource = createSource(config); + fileSource.start(buffer); + + await().atMost(WAIT_TIMEOUT).untilAsserted(() -> { + assertThat(capturedRecords, hasSize(1)); + assertThat(eventMessage(capturedRecords.get(0)), equalTo("app log line")); + }); + } + + @Test + void tail_mode_detects_copytruncate_rotation() throws Exception { + final Path logFile = tempDir.resolve("rotating.log"); + Files.write(logFile, "this is a long line before truncation happens here\n".getBytes(StandardCharsets.UTF_8)); + + final FileSourceConfig config = createTailConfig( + logFile.toString(), null, "beginning"); + + fileSource = createSource(config); + fileSource.start(buffer); + + await().atMost(WAIT_TIMEOUT).untilAsserted(() -> + assertThat(capturedRecords, hasSize(1))); + + Files.write(logFile, new byte[0], StandardOpenOption.TRUNCATE_EXISTING); + Thread.sleep(ROTATION_DETECTION_DELAY_MS); + appendLine(logFile, "short"); + + await().atMost(Duration.ofSeconds(15)).untilAsserted(() -> { + assertThat(capturedRecords, hasSize(greaterThanOrEqualTo(2))); + assertThat(eventMessage(capturedRecords.get(capturedRecords.size() - 1)), + equalTo("short")); + }); + } + + @Test + void tail_mode_detects_create_rename_rotation() throws Exception { + final Path logFile = tempDir.resolve("app.log"); + Files.write(logFile, "before rotation\n".getBytes(StandardCharsets.UTF_8)); + + final String globPattern = tempDir.resolve("app.log").toString(); + final FileSourceConfig config = createTailConfig( + globPattern, null, "beginning"); + + fileSource = createSource(config); + fileSource.start(buffer); + + await().atMost(WAIT_TIMEOUT).untilAsserted(() -> + assertThat(capturedRecords, hasSize(1))); + + Files.move(logFile, tempDir.resolve("app.log.1")); + Thread.sleep(ROTATION_DETECTION_DELAY_MS); + Files.write(logFile, "after rotation\n".getBytes(StandardCharsets.UTF_8)); + + await().atMost(Duration.ofSeconds(20)).untilAsserted(() -> { + assertThat(capturedRecords, hasSize(greaterThanOrEqualTo(2))); + assertThat(eventMessage(capturedRecords.get(capturedRecords.size() - 1)), + equalTo("after rotation")); + }); + } + + @Test + void tail_mode_includes_file_metadata() throws Exception { + final Path logFile = tempDir.resolve("meta.log"); + Files.write(logFile, List.of("metadata test")); + + final FileSourceConfig config = createTailConfig( + logFile.toString(), null, "beginning"); + + fileSource = createSource(config); + fileSource.start(buffer); + + await().atMost(WAIT_TIMEOUT).untilAsserted(() -> { + assertThat(capturedRecords, hasSize(1)); + final Event event = (Event) capturedRecords.get(0).getData(); + assertThat(event.get("file_path", String.class), + equalTo(logFile.toAbsolutePath().toString())); + }); + } + + @Test + void tail_mode_resumes_from_checkpoint_after_restart() throws Exception { + final Path logFile = tempDir.resolve("checkpoint.log"); + final Path checkpointFile = tempDir.resolve("checkpoint.json"); + Files.write(logFile, List.of("line 1", "line 2", "line 3")); + + final FileSourceConfig config = createTailConfigWithCheckpoint( + logFile.toString(), "beginning", checkpointFile.toString()); + + fileSource = createSource(config); + fileSource.start(buffer); + + await().atMost(WAIT_TIMEOUT).untilAsserted(() -> + assertThat(capturedRecords, hasSize(3))); + + fileSource.stop(); + fileSource = null; + Thread.sleep(SHORT_DELAY_MS); + + capturedRecords.clear(); + appendLine(logFile, "line 4"); + + final FileSourceConfig config2 = createTailConfigWithCheckpoint( + logFile.toString(), "beginning", checkpointFile.toString()); + fileSource = createSource(config2); + fileSource.start(buffer); + + await().atMost(WAIT_TIMEOUT).untilAsserted(() -> { + assertThat(capturedRecords, hasSize(1)); + assertThat(eventMessage(capturedRecords.get(0)), equalTo("line 4")); + }); + } + + @Test + void tail_mode_discovers_new_file_created_after_start() throws Exception { + final String globPattern = tempDir.resolve("*.log").toString(); + final FileSourceConfig config = createTailConfig( + null, List.of(globPattern), "beginning"); + + fileSource = createSource(config); + fileSource.start(buffer); + + Thread.sleep(SHORT_DELAY_MS); + assertThat(capturedRecords, hasSize(0)); + + Files.write(tempDir.resolve("new-file.log"), List.of("discovered after start")); + + await().atMost(WAIT_TIMEOUT).untilAsserted(() -> { + assertThat(capturedRecords, hasSize(1)); + assertThat(eventMessage(capturedRecords.get(0)), equalTo("discovered after start")); + }); + } + + @Test + void tail_mode_handles_back_pressure_with_retry() throws Exception { + final Path logFile = tempDir.resolve("backpressure.log"); + Files.write(logFile, "line1\nline2\nline3\n".getBytes(StandardCharsets.UTF_8)); + + final AtomicInteger writeAttempts = new AtomicInteger(0); + final int failFirstN = 3; + + @SuppressWarnings("unchecked") + final Buffer> slowBuffer = (Buffer>) mock(Buffer.class); + doAnswer(invocation -> { + if (writeAttempts.incrementAndGet() <= failFirstN) { + throw new TimeoutException("Buffer full"); + } + Record record = invocation.getArgument(0); + capturedRecords.add(record); + return null; + }).when(slowBuffer).write(any(Record.class), anyInt()); + + final FileSourceConfig config = createTailConfig( + logFile.toString(), null, "beginning"); + + fileSource = createSource(config); + fileSource.start(slowBuffer); + + await().atMost(WAIT_TIMEOUT).untilAsserted(() -> { + assertThat(capturedRecords, hasSize(3)); + assertThat(eventMessage(capturedRecords.get(0)), equalTo("line1")); + }); + assertThat(writeAttempts.get(), greaterThanOrEqualTo(failFirstN + 3)); + } + + @Test + void tail_mode_with_acknowledgements_commits_offset_on_positive_ack() throws Exception { + final Path logFile = tempDir.resolve("ack.log"); + final Path checkpointFile = tempDir.resolve("ack-checkpoint.json"); + Files.write(logFile, "ack line 1\nack line 2\n".getBytes(StandardCharsets.UTF_8)); + + final List> ackCallbacks = Collections.synchronizedList(new ArrayList<>()); + + final AcknowledgementSet mockAckSet = mock(AcknowledgementSet.class); + final AcknowledgementSetManager ackManager = mock(AcknowledgementSetManager.class); + doAnswer(invocation -> { + Consumer callback = invocation.getArgument(0); + ackCallbacks.add(callback); + return mockAckSet; + }).when(ackManager).create(any(), org.mockito.ArgumentMatchers.any(Duration.class)); + + final FileSourceConfig config = buildConfigWithAcknowledgements( + logFile.toString(), "beginning", checkpointFile.toString()); + + fileSource = new FileSource(config, pluginMetrics, pluginFactory, eventFactory, ackManager); + fileSource.start(buffer); + + await().atMost(WAIT_TIMEOUT).untilAsserted(() -> + assertThat(capturedRecords, hasSize(2))); + + await().atMost(WAIT_TIMEOUT).untilAsserted(() -> + assertThat(ackCallbacks, hasSize(greaterThanOrEqualTo(1)))); + + ackCallbacks.get(0).accept(true); + + await().atMost(WAIT_TIMEOUT).untilAsserted(() -> { + assertThat(Files.exists(checkpointFile), equalTo(true)); + final String checkpointContent = Files.readString(checkpointFile); + assertThat(checkpointContent.contains("committedOffset"), equalTo(true)); + }); + } + + @Test + void tail_mode_with_acknowledgements_does_not_advance_offset_on_negative_ack() throws Exception { + final Path logFile = tempDir.resolve("nack.log"); + final Path checkpointFile = tempDir.resolve("nack-checkpoint.json"); + Files.write(logFile, "nack line\n".getBytes(StandardCharsets.UTF_8)); + + final List> ackCallbacks = Collections.synchronizedList(new ArrayList<>()); + + final AcknowledgementSet mockAckSet = mock(AcknowledgementSet.class); + final AcknowledgementSetManager ackManager = mock(AcknowledgementSetManager.class); + doAnswer(invocation -> { + Consumer callback = invocation.getArgument(0); + ackCallbacks.add(callback); + return mockAckSet; + }).when(ackManager).create(any(), org.mockito.ArgumentMatchers.any(Duration.class)); + + final FileSourceConfig config = buildConfigWithAcknowledgements( + logFile.toString(), "beginning", checkpointFile.toString()); + + fileSource = new FileSource(config, pluginMetrics, pluginFactory, eventFactory, ackManager); + fileSource.start(buffer); + + await().atMost(WAIT_TIMEOUT).untilAsserted(() -> + assertThat(capturedRecords, hasSize(1))); + + await().atMost(WAIT_TIMEOUT).untilAsserted(() -> + assertThat(ackCallbacks, hasSize(greaterThanOrEqualTo(1)))); + + ackCallbacks.get(0).accept(false); + + await().atMost(WAIT_TIMEOUT).untilAsserted(() -> { + assertThat(Files.exists(checkpointFile), equalTo(true)); + final String checkpointContent = Files.readString(checkpointFile); + assertThat(checkpointContent.contains("\"committedOffset\":0"), equalTo(true)); + }); + } + + @Test + void tail_mode_max_active_files_queues_excess_files() throws Exception { + for (int i = 0; i < 5; i++) { + Files.write(tempDir.resolve("file" + i + ".log"), + ("content from file " + i + "\n").getBytes(StandardCharsets.UTF_8)); + } + + final String globPattern = tempDir.resolve("*.log").toString(); + final FileSourceConfig config = buildConfigWithMaxActiveFiles( + globPattern, "beginning", 3); + + fileSource = createSource(config); + fileSource.start(buffer); + + await().atMost(Duration.ofSeconds(30)).untilAsserted(() -> + assertThat(capturedRecords, hasSize(greaterThanOrEqualTo(5)))); + } + + @Test + void tail_mode_max_read_time_prevents_starvation() throws Exception { + final Path largeFile = tempDir.resolve("large.log"); + final StringBuilder content = new StringBuilder(); + for (int i = 0; i < 1000; i++) { + content.append("line ").append(i).append("\n"); + } + Files.write(largeFile, content.toString().getBytes(StandardCharsets.UTF_8)); + + final Path smallFile = tempDir.resolve("small.log"); + Files.write(smallFile, "small file line\n".getBytes(StandardCharsets.UTF_8)); + + final String globPattern = tempDir.resolve("*.log").toString(); + final FileSourceConfig config = createTailConfig( + null, List.of(globPattern), "beginning"); + + fileSource = createSource(config); + fileSource.start(buffer); + + await().atMost(Duration.ofSeconds(20)).untilAsserted(() -> { + final boolean hasSmallFileLine = capturedRecords.stream() + .anyMatch(r -> "small file line".equals(eventMessage(r))); + assertThat(hasSmallFileLine, equalTo(true)); + }); + } + + @Test + void tail_mode_close_inactive_releases_file_handle() throws Exception { + final Path logFile = tempDir.resolve("inactive.log"); + Files.write(logFile, "initial line\n".getBytes(StandardCharsets.UTF_8)); + + final FileSourceConfig config = buildConfigWithCloseInactive( + logFile.toString(), "beginning", "PT2S"); + + fileSource = createSource(config); + fileSource.start(buffer); + + await().atMost(WAIT_TIMEOUT).untilAsserted(() -> + assertThat(capturedRecords, hasSize(1))); + + Thread.sleep(CLOSE_INACTIVE_WAIT_MS); + + appendLine(logFile, "after inactive"); + + await().atMost(WAIT_TIMEOUT).untilAsserted(() -> + assertThat(capturedRecords, hasSize(greaterThanOrEqualTo(2)))); + } + + @Test + void tail_mode_handles_file_deleted_while_tailing() throws Exception { + final Path logFile = tempDir.resolve("deleteme.log"); + Files.write(logFile, "will be deleted\n".getBytes(StandardCharsets.UTF_8)); + + final FileSourceConfig config = createTailConfig( + logFile.toString(), null, "beginning"); + + fileSource = createSource(config); + fileSource.start(buffer); + + await().atMost(WAIT_TIMEOUT).untilAsserted(() -> + assertThat(capturedRecords, hasSize(1))); + + Files.delete(logFile); + Thread.sleep(DELETION_DETECTION_DELAY_MS); + + assertThat(fileSource.areAcknowledgementsEnabled(), equalTo(false)); + } + + @Test + void tail_mode_max_line_length_truncates_long_lines() throws Exception { + final Path logFile = tempDir.resolve("longline.log"); + final String longLine = "x".repeat(5000); + Files.write(logFile, (longLine + "\n").getBytes(StandardCharsets.UTF_8)); + + final FileSourceConfig config = buildConfigWithMaxLineLength( + logFile.toString(), "beginning", 1024); + + fileSource = createSource(config); + fileSource.start(buffer); + + await().atMost(WAIT_TIMEOUT).untilAsserted(() -> { + assertThat(capturedRecords, hasSize(1)); + final String message = eventMessage(capturedRecords.get(0)); + assertThat(message.length(), equalTo(1024)); + }); + } + + @Test + void tail_mode_back_pressure_during_rotation_does_not_lose_data() throws Exception { + final Path logFile = tempDir.resolve("bp-rotate.log"); + Files.write(logFile, "line before rotation\n".getBytes(StandardCharsets.UTF_8)); + + final AtomicInteger writeCount = new AtomicInteger(0); + final int blockFirstNWrites = 2; + + @SuppressWarnings("unchecked") + final Buffer> slowBuffer = (Buffer>) mock(Buffer.class); + doAnswer(invocation -> { + if (writeCount.incrementAndGet() <= blockFirstNWrites) { + throw new TimeoutException("Buffer full - simulating back pressure"); + } + Record record = invocation.getArgument(0); + capturedRecords.add(record); + return null; + }).when(slowBuffer).write(any(Record.class), anyInt()); + + final FileSourceConfig config = createTailConfig( + logFile.toString(), null, "beginning"); + + fileSource = createSource(config); + fileSource.start(slowBuffer); + + await().atMost(WAIT_TIMEOUT).untilAsserted(() -> + assertThat(capturedRecords, hasSize(1))); + + Files.move(logFile, tempDir.resolve("bp-rotate.log.1")); + Thread.sleep(SHORT_DELAY_MS); + Files.write(logFile, "line after rotation\n".getBytes(StandardCharsets.UTF_8)); + + await().atMost(Duration.ofSeconds(20)).untilAsserted(() -> + assertThat(capturedRecords, hasSize(greaterThanOrEqualTo(2)))); + } + + @Test + void tail_mode_negative_ack_retry_exhaustion_advances_offset() throws Exception { + final Path logFile = tempDir.resolve("retry-exhaust.log"); + final Path checkpointFile = tempDir.resolve("retry-exhaust-checkpoint.json"); + Files.write(logFile, "retry line\n".getBytes(StandardCharsets.UTF_8)); + + final List> ackCallbacks = Collections.synchronizedList(new ArrayList<>()); + + final AcknowledgementSet mockAckSet = mock(AcknowledgementSet.class); + final AcknowledgementSetManager ackManager = mock(AcknowledgementSetManager.class); + doAnswer(invocation -> { + Consumer callback = invocation.getArgument(0); + ackCallbacks.add(callback); + return mockAckSet; + }).when(ackManager).create(any(), org.mockito.ArgumentMatchers.any(Duration.class)); + + final FileSourceConfig config = buildConfigWithAcknowledgementsAndRetries( + logFile.toString(), "beginning", checkpointFile.toString(), 2); + + fileSource = new FileSource(config, pluginMetrics, pluginFactory, eventFactory, ackManager); + fileSource.start(buffer); + + await().atMost(WAIT_TIMEOUT).untilAsserted(() -> + assertThat(capturedRecords, hasSize(1))); + + await().atMost(WAIT_TIMEOUT).untilAsserted(() -> + assertThat(ackCallbacks, hasSize(greaterThanOrEqualTo(1)))); + + ackCallbacks.get(0).accept(false); + Thread.sleep(ACK_RETRY_INTERVAL_MS); + ackCallbacks.get(0).accept(false); + Thread.sleep(ACK_RETRY_INTERVAL_MS); + ackCallbacks.get(0).accept(false); + + await().atMost(WAIT_TIMEOUT).untilAsserted(() -> { + assertThat(Files.exists(checkpointFile), equalTo(true)); + final String content = Files.readString(checkpointFile); + assertThat(content.contains("\"committedOffset\":0"), equalTo(false)); + }); + } + + @Test + void tail_mode_checkpoint_is_persisted_periodically() throws Exception { + final Path logFile = tempDir.resolve("checkpoint-persist.log"); + final Path checkpointFile = tempDir.resolve("persist-checkpoint.json"); + Files.write(logFile, "checkpoint test line\n".getBytes(StandardCharsets.UTF_8)); + + final FileSourceConfig config = createTailConfigWithCheckpoint( + logFile.toString(), "beginning", checkpointFile.toString()); + + fileSource = createSource(config); + fileSource.start(buffer); + + await().atMost(WAIT_TIMEOUT).untilAsserted(() -> + assertThat(capturedRecords, hasSize(1))); + + await().atMost(Duration.ofSeconds(20)).untilAsserted(() -> { + assertThat(Files.exists(checkpointFile), equalTo(true)); + final String content = Files.readString(checkpointFile); + assertThat(content.contains("readOffset"), equalTo(true)); + }); + } + + @Test + void tail_mode_both_path_and_paths_are_merged() throws Exception { + final Path singleFile = tempDir.resolve("single.log"); + Files.write(singleFile, "from single path\n".getBytes(StandardCharsets.UTF_8)); + + final Path globFile = tempDir.resolve("glob-match.log"); + Files.write(globFile, "from glob path\n".getBytes(StandardCharsets.UTF_8)); + + final String globPattern = tempDir.resolve("glob-*.log").toString(); + + final ObjectMapper mapper = new ObjectMapper(); + mapper.registerModule(new JavaTimeModule()); + final Map configMap = new HashMap<>(); + configMap.put("tail", true); + configMap.put("path", singleFile.toString()); + configMap.put("paths", List.of(globPattern)); + configMap.put("start_position", "beginning"); + configMap.put("reader_threads", 2); + configMap.put("include_file_metadata", true); + configMap.put("checkpoint_file", tempDir.resolve("merged-checkpoint.json").toString()); + final FileSourceConfig config = mapper.convertValue(configMap, FileSourceConfig.class); + + fileSource = createSource(config); + fileSource.start(buffer); + + await().atMost(WAIT_TIMEOUT).untilAsserted(() -> + assertThat(capturedRecords, hasSize(2))); + + final List messages = capturedRecords.stream() + .map(this::eventMessage) + .collect(Collectors.toList()); + assertThat(messages.contains("from single path"), equalTo(true)); + assertThat(messages.contains("from glob path"), equalTo(true)); + } + + @Test + void tail_mode_close_removed_true_stops_reading_deleted_file() throws Exception { + final Path logFile = tempDir.resolve("close-removed.log"); + Files.write(logFile, "will be removed\n".getBytes(StandardCharsets.UTF_8)); + + final FileSourceConfig config = createTailConfig( + logFile.toString(), null, "beginning"); + + fileSource = createSource(config); + fileSource.start(buffer); + + await().atMost(WAIT_TIMEOUT).untilAsserted(() -> + assertThat(capturedRecords, hasSize(1))); + + Files.delete(logFile); + Thread.sleep(DELETION_DETECTION_DELAY_MS); + + final int recordsAfterDelete = capturedRecords.size(); + Thread.sleep(ROTATION_DETECTION_DELAY_MS); + assertThat(capturedRecords.size(), equalTo(recordsAfterDelete)); + } + + @Test + void tail_mode_multiple_files_read_concurrently_with_reader_threads() throws Exception { + for (int i = 0; i < 4; i++) { + final Path file = tempDir.resolve("concurrent" + i + ".log"); + final StringBuilder content = new StringBuilder(); + for (int j = 0; j < 10; j++) { + content.append("file").append(i).append("-line").append(j).append("\n"); + } + Files.write(file, content.toString().getBytes(StandardCharsets.UTF_8)); + } + + final String globPattern = tempDir.resolve("concurrent*.log").toString(); + final FileSourceConfig config = createTailConfig( + null, List.of(globPattern), "beginning"); + + fileSource = createSource(config); + fileSource.start(buffer); + + await().atMost(Duration.ofSeconds(20)).untilAsserted(() -> + assertThat(capturedRecords, hasSize(40))); + } + + @Test + void tail_mode_empty_file_produces_no_events() throws Exception { + final Path emptyFile = tempDir.resolve("empty.log"); + Files.write(emptyFile, new byte[0]); + + final FileSourceConfig config = createTailConfig( + emptyFile.toString(), null, "beginning"); + + fileSource = createSource(config); + fileSource.start(buffer); + + Thread.sleep(DELETION_DETECTION_DELAY_MS); + assertThat(capturedRecords, hasSize(0)); + + appendLine(emptyFile, "content after empty"); + + await().atMost(WAIT_TIMEOUT).untilAsserted(() -> { + assertThat(capturedRecords, hasSize(1)); + assertThat(eventMessage(capturedRecords.get(0)), equalTo("content after empty")); + }); + } + + @Test + void tail_mode_partial_line_without_newline_flushed_on_read_timeout() throws Exception { + final Path logFile = tempDir.resolve("partial.log"); + Files.write(logFile, "no newline at end".getBytes(StandardCharsets.UTF_8)); + + final FileSourceConfig config = createTailConfig( + logFile.toString(), null, "beginning"); + + fileSource = createSource(config); + fileSource.start(buffer); + + await().atMost(Duration.ofSeconds(20)).untilAsserted(() -> { + assertThat(capturedRecords, hasSize(1)); + assertThat(eventMessage(capturedRecords.get(0)), equalTo("no newline at end")); + }); + } + + private FileSource createSource(final FileSourceConfig config) { + return new FileSource(config, pluginMetrics, pluginFactory, eventFactory, acknowledgementSetManager); + } + + private FileSourceConfig createTailConfig(final String path, final List paths, final String startPosition) { + return buildConfig(path, paths, null, startPosition, null); + } + + private FileSourceConfig createTailConfigWithExclude(final String globPattern, final String excludePattern, + final String startPosition) { + return buildConfig(null, List.of(globPattern), List.of(excludePattern), startPosition, null); + } + + private FileSourceConfig createTailConfigWithCheckpoint(final String path, final String startPosition, + final String checkpointFilePath) { + return buildConfig(path, null, null, startPosition, checkpointFilePath); + } + + private FileSourceConfig buildConfig(final String path, final List paths, + final List excludePaths, final String startPosition, + final String checkpointFile) { + final ObjectMapper mapper = new ObjectMapper(); + mapper.registerModule(new JavaTimeModule()); + final Map configMap = new HashMap<>(); + configMap.put("tail", true); + configMap.put("start_position", startPosition != null ? startPosition : "beginning"); + configMap.put("reader_threads", 2); + configMap.put("include_file_metadata", true); + if (path != null) { + configMap.put("path", path); + } + if (paths != null) { + configMap.put("paths", paths); + } + if (excludePaths != null) { + configMap.put("exclude_paths", excludePaths); + } + if (checkpointFile != null) { + configMap.put("checkpoint_file", checkpointFile); + } else { + configMap.put("checkpoint_file", tempDir.resolve("default-checkpoint.json").toString()); + } + return mapper.convertValue(configMap, FileSourceConfig.class); + } + + private FileSourceConfig buildConfigWithAcknowledgements(final String path, final String startPosition, + final String checkpointFile) { + final ObjectMapper mapper = new ObjectMapper(); + mapper.registerModule(new JavaTimeModule()); + final Map configMap = new HashMap<>(); + configMap.put("tail", true); + configMap.put("path", path); + configMap.put("start_position", startPosition); + configMap.put("reader_threads", 2); + configMap.put("include_file_metadata", true); + configMap.put("acknowledgments", true); + configMap.put("batch_size", 10); + configMap.put("checkpoint_file", checkpointFile); + return mapper.convertValue(configMap, FileSourceConfig.class); + } + + private FileSourceConfig buildConfigWithMaxActiveFiles(final String globPattern, final String startPosition, + final int maxActiveFiles) { + final ObjectMapper mapper = new ObjectMapper(); + mapper.registerModule(new JavaTimeModule()); + final Map configMap = new HashMap<>(); + configMap.put("tail", true); + configMap.put("paths", List.of(globPattern)); + configMap.put("start_position", startPosition); + configMap.put("reader_threads", 2); + configMap.put("max_active_files", maxActiveFiles); + configMap.put("include_file_metadata", true); + configMap.put("checkpoint_file", tempDir.resolve("max-active-checkpoint.json").toString()); + return mapper.convertValue(configMap, FileSourceConfig.class); + } + + private FileSourceConfig buildConfigWithCloseInactive(final String path, final String startPosition, + final String closeInactive) { + final ObjectMapper mapper = new ObjectMapper(); + mapper.registerModule(new JavaTimeModule()); + final Map configMap = new HashMap<>(); + configMap.put("tail", true); + configMap.put("path", path); + configMap.put("start_position", startPosition); + configMap.put("reader_threads", 2); + configMap.put("include_file_metadata", true); + configMap.put("close_inactive", closeInactive); + configMap.put("checkpoint_file", tempDir.resolve("close-inactive-checkpoint.json").toString()); + return mapper.convertValue(configMap, FileSourceConfig.class); + } + + private FileSourceConfig buildConfigWithMaxLineLength(final String path, final String startPosition, + final int maxLineLength) { + final ObjectMapper mapper = new ObjectMapper(); + mapper.registerModule(new JavaTimeModule()); + final Map configMap = new HashMap<>(); + configMap.put("tail", true); + configMap.put("path", path); + configMap.put("start_position", startPosition); + configMap.put("reader_threads", 2); + configMap.put("include_file_metadata", true); + configMap.put("max_line_length", maxLineLength); + configMap.put("checkpoint_file", tempDir.resolve("maxline-checkpoint.json").toString()); + return mapper.convertValue(configMap, FileSourceConfig.class); + } + + private FileSourceConfig buildConfigWithAcknowledgementsAndRetries(final String path, final String startPosition, + final String checkpointFile, final int maxRetries) { + final ObjectMapper mapper = new ObjectMapper(); + mapper.registerModule(new JavaTimeModule()); + final Map configMap = new HashMap<>(); + configMap.put("tail", true); + configMap.put("path", path); + configMap.put("start_position", startPosition); + configMap.put("reader_threads", 2); + configMap.put("include_file_metadata", true); + configMap.put("acknowledgments", true); + configMap.put("batch_size", 10); + configMap.put("max_acknowledgment_retries", maxRetries); + configMap.put("checkpoint_file", checkpointFile); + return mapper.convertValue(configMap, FileSourceConfig.class); + } + + private FileSourceConfig buildConfigWithCheckpointCleanup(final String path, final String startPosition, + final String checkpointFile, final String cleanupAfter) { + final ObjectMapper mapper = new ObjectMapper(); + mapper.registerModule(new JavaTimeModule()); + final Map configMap = new HashMap<>(); + configMap.put("tail", true); + configMap.put("path", path); + configMap.put("start_position", startPosition); + configMap.put("reader_threads", 2); + configMap.put("include_file_metadata", true); + configMap.put("checkpoint_cleanup_after", cleanupAfter); + configMap.put("checkpoint_file", checkpointFile); + return mapper.convertValue(configMap, FileSourceConfig.class); + } + + private void appendLine(final Path file, final String line) throws IOException { + Files.write(file, (line + "\n").getBytes(StandardCharsets.UTF_8), + StandardOpenOption.APPEND, StandardOpenOption.CREATE); + } + + private String eventMessage(final Record record) { + final Event event = (Event) record.getData(); + return event.get("message", String.class); + } +} diff --git a/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/CheckpointEntry.java b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/CheckpointEntry.java new file mode 100644 index 0000000000..82ecb8641d --- /dev/null +++ b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/CheckpointEntry.java @@ -0,0 +1,77 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + +package org.opensearch.dataprepper.plugins.source.file; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +public final class CheckpointEntry { + + private long readOffset; + private long committedOffset; + private CheckpointStatus status; + private long lastUpdatedMillis; + + @JsonCreator + public CheckpointEntry( + @JsonProperty("readOffset") final long readOffset, + @JsonProperty("committedOffset") final long committedOffset, + @JsonProperty("status") final CheckpointStatus status, + @JsonProperty("lastUpdatedMillis") final long lastUpdatedMillis) { + this.readOffset = readOffset; + this.committedOffset = committedOffset; + this.status = status; + this.lastUpdatedMillis = lastUpdatedMillis; + } + + public CheckpointEntry(final long readOffset, final long committedOffset, final CheckpointStatus status) { + this(readOffset, committedOffset, status, System.currentTimeMillis()); + } + + public CheckpointEntry() { + this(0, 0, CheckpointStatus.ACTIVE); + } + + public synchronized long getReadOffset() { + return readOffset; + } + + public synchronized void setReadOffset(final long readOffset) { + this.readOffset = readOffset; + this.lastUpdatedMillis = System.currentTimeMillis(); + } + + public synchronized long getCommittedOffset() { + return committedOffset; + } + + public synchronized void setCommittedOffset(final long committedOffset) { + this.committedOffset = committedOffset; + this.lastUpdatedMillis = System.currentTimeMillis(); + } + + public synchronized CheckpointStatus getStatus() { + return status; + } + + public synchronized void setStatus(final CheckpointStatus status) { + this.status = status; + this.lastUpdatedMillis = System.currentTimeMillis(); + } + + public synchronized long getLastUpdatedMillis() { + return lastUpdatedMillis; + } + + public synchronized CheckpointEntry snapshot() { + return new CheckpointEntry(readOffset, committedOffset, status, lastUpdatedMillis); + } +} diff --git a/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/CheckpointRegistry.java b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/CheckpointRegistry.java new file mode 100644 index 0000000000..177e29ce50 --- /dev/null +++ b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/CheckpointRegistry.java @@ -0,0 +1,149 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + +package org.opensearch.dataprepper.plugins.source.file; + +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.FileNotFoundException; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.NoSuchFileException; +import java.nio.file.Path; +import java.nio.file.StandardCopyOption; +import java.time.Duration; +import java.util.HashMap; +import java.util.Iterator; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.function.Supplier; + +public final class CheckpointRegistry { + + private static final Logger LOG = LoggerFactory.getLogger(CheckpointRegistry.class); + private static final TypeReference> MAP_TYPE = new TypeReference<>() { }; + + private final ConcurrentHashMap storage; + private final ObjectMapper objectMapper; + private final Path checkpointFile; + private final Duration cleanupAfter; + private final ScheduledExecutorService scheduler; + + public CheckpointRegistry(final Path checkpointFile, final Duration flushInterval, final Duration cleanupAfter) { + this(checkpointFile, flushInterval, cleanupAfter, () -> Executors.newSingleThreadScheduledExecutor(r -> { + final Thread thread = new Thread(r, "checkpoint-flush"); + thread.setDaemon(true); + return thread; + })); + } + + CheckpointRegistry(final Path checkpointFile, final Duration flushInterval, final Duration cleanupAfter, + final Supplier schedulerSupplier) { + this.checkpointFile = checkpointFile; + this.cleanupAfter = cleanupAfter; + this.objectMapper = new ObjectMapper(); + this.storage = new ConcurrentHashMap<>(); + this.scheduler = schedulerSupplier.get(); + + load(); + scheduler.scheduleAtFixedRate(this::flush, flushInterval.toMillis(), flushInterval.toMillis(), TimeUnit.MILLISECONDS); + } + + public CheckpointEntry getOrCreate(final String key) { + return storage.computeIfAbsent(key, k -> new CheckpointEntry()); + } + + public CheckpointEntry get(final String key) { + return storage.get(key); + } + + public void markCompleted(final String key) { + final CheckpointEntry entry = storage.get(key); + if (entry != null) { + entry.setStatus(CheckpointStatus.COMPLETED); + } + } + + public synchronized void flush() { + if (checkpointFile == null) { + return; + } + try { + cleanupStaleEntries(); + final Map snapshot = new HashMap<>(); + for (final Map.Entry entry : storage.entrySet()) { + snapshot.put(entry.getKey(), entry.getValue().snapshot()); + } + final Path tempFile = checkpointFile.resolveSibling(checkpointFile.getFileName() + ".tmp"); + objectMapper.writeValue(tempFile.toFile(), snapshot); + Files.move(tempFile, checkpointFile, StandardCopyOption.REPLACE_EXISTING, StandardCopyOption.ATOMIC_MOVE); + } catch (final IOException e) { + LOG.error("Failed to flush checkpoint file", e); + } + } + + public void shutdown() { + scheduler.shutdown(); + boolean interrupted = false; + try { + if (!scheduler.awaitTermination(5, TimeUnit.SECONDS)) { + scheduler.shutdownNow(); + } + } catch (final InterruptedException e) { + scheduler.shutdownNow(); + interrupted = true; + } + flush(); + if (interrupted) { + Thread.currentThread().interrupt(); + } + } + + private void load() { + if (checkpointFile == null) { + return; + } + try { + if (checkpointFile.getParent() != null) { + Files.createDirectories(checkpointFile.getParent()); + } + final Map loaded = objectMapper.readValue(checkpointFile.toFile(), MAP_TYPE); + if (loaded != null) { + storage.putAll(loaded); + } + LOG.info("Loaded {} checkpoint entries from {}", storage.size(), checkpointFile); + } catch (final FileNotFoundException | NoSuchFileException e) { + LOG.debug("No existing checkpoint file at {}. Starting with empty state.", checkpointFile); + } catch (final IOException e) { + LOG.warn("Corrupt or unreadable checkpoint file at {}. Starting with empty state.", checkpointFile, e); + } + } + + private void cleanupStaleEntries() { + final long now = System.currentTimeMillis(); + final long cleanupThreshold = cleanupAfter.toMillis(); + final Iterator> iterator = storage.entrySet().iterator(); + while (iterator.hasNext()) { + final Map.Entry entry = iterator.next(); + final CheckpointEntry checkpoint = entry.getValue(); + if (CheckpointStatus.COMPLETED == checkpoint.getStatus() && + (now - checkpoint.getLastUpdatedMillis()) > cleanupThreshold) { + iterator.remove(); + LOG.debug("Removed stale checkpoint entry: {}", entry.getKey()); + } + } + } +} diff --git a/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/CheckpointStatus.java b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/CheckpointStatus.java new file mode 100644 index 0000000000..d797c3e986 --- /dev/null +++ b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/CheckpointStatus.java @@ -0,0 +1,40 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + +package org.opensearch.dataprepper.plugins.source.file; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonValue; + +public enum CheckpointStatus { + ACTIVE("ACTIVE"), + COMPLETED("COMPLETED"); + + private final String value; + + CheckpointStatus(final String value) { + this.value = value; + } + + @JsonValue + public String getValue() { + return value; + } + + @JsonCreator + public static CheckpointStatus fromString(final String value) { + for (final CheckpointStatus status : values()) { + if (status.value.equalsIgnoreCase(value)) { + return status; + } + } + throw new IllegalArgumentException("Invalid checkpoint status: " + value); + } +} diff --git a/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/DefaultFileSystemOperations.java b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/DefaultFileSystemOperations.java new file mode 100644 index 0000000000..bd679a7ad6 --- /dev/null +++ b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/DefaultFileSystemOperations.java @@ -0,0 +1,54 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + +package org.opensearch.dataprepper.plugins.source.file; + +import java.io.IOException; +import java.nio.channels.FileChannel; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.StandardOpenOption; +import java.nio.file.attribute.BasicFileAttributes; +import java.util.stream.Stream; + +final class DefaultFileSystemOperations implements FileSystemOperations { + + @Override + public FileChannel openReadChannel(final Path path) throws IOException { + return FileChannel.open(path, StandardOpenOption.READ); + } + + @Override + public BasicFileAttributes readAttributes(final Path path) throws IOException { + return Files.readAttributes(path, BasicFileAttributes.class); + } + + @Override + public Stream listDirectory(final Path directory) throws IOException { + return Files.list(directory); + } + + @Override + public boolean exists(final Path path) { + return Files.exists(path); + } + + @Override + public long size(final Path path) throws IOException { + return Files.size(path); + } + + @Override + public byte[] readBytes(final Path path, final int length) throws IOException { + try (final var inputStream = Files.newInputStream(path)) { + return inputStream.readNBytes(length); + } + } +} diff --git a/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/DirectoryWatcher.java b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/DirectoryWatcher.java new file mode 100644 index 0000000000..a37aee937d --- /dev/null +++ b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/DirectoryWatcher.java @@ -0,0 +1,403 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + +package org.opensearch.dataprepper.plugins.source.file; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.nio.file.ClosedWatchServiceException; +import java.nio.file.FileStore; +import java.nio.file.FileSystems; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.StandardWatchEventKinds; +import java.nio.file.WatchEvent; +import java.nio.file.WatchKey; +import java.nio.file.WatchService; +import java.time.Duration; +import java.util.HashSet; +import java.util.Objects; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.Executors; +import java.util.concurrent.RejectedExecutionException; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.function.Predicate; +import java.util.function.Supplier; + +public final class DirectoryWatcher { + + @FunctionalInterface + interface WatchServiceFactory { + WatchService create() throws IOException; + } + + private static final Logger LOG = LoggerFactory.getLogger(DirectoryWatcher.class); + private static final long SHUTDOWN_TIMEOUT_SECONDS = 10; + private static final Set NETWORK_FS_TYPES = Set.of( + "nfs", "nfs4", "cifs", "smb", "smb2", "fuse.sshfs", "afs", "9p" + ); + + private final GlobPathResolver globPathResolver; + private final TailFileReaderPool readerPool; + private final CheckpointRegistry checkpointRegistry; + private final FileSourceConfig config; + private final FileSystemOperations fileOps; + private final FileTailMetrics metrics; + private final Duration rotateWait; + private final boolean closeRemoved; + private final Set knownFiles; + private final WatchServiceFactory watchServiceFactory; + private final Supplier pollSchedulerSupplier; + private final Predicate networkFsCheck; + private final boolean macOS; + + private volatile WatchService watchService; + private volatile Thread watchThread; + private volatile ScheduledExecutorService pollScheduler; + private volatile boolean running; + + public DirectoryWatcher(final GlobPathResolver globPathResolver, + final TailFileReaderPool readerPool, + final CheckpointRegistry checkpointRegistry, + final FileSourceConfig config, + final FileSystemOperations fileOps, + final FileTailMetrics metrics, + final Duration rotateWait, + final boolean closeRemoved) { + this(globPathResolver, readerPool, checkpointRegistry, config, fileOps, metrics, rotateWait, closeRemoved, + () -> FileSystems.getDefault().newWatchService(), isMacOS()); + } + + DirectoryWatcher(final GlobPathResolver globPathResolver, + final TailFileReaderPool readerPool, + final CheckpointRegistry checkpointRegistry, + final FileSourceConfig config, + final FileSystemOperations fileOps, + final FileTailMetrics metrics, + final Duration rotateWait, + final boolean closeRemoved, + final WatchServiceFactory watchServiceFactory) { + this(globPathResolver, readerPool, checkpointRegistry, config, fileOps, metrics, rotateWait, closeRemoved, + watchServiceFactory, isMacOS()); + } + + DirectoryWatcher(final GlobPathResolver globPathResolver, + final TailFileReaderPool readerPool, + final CheckpointRegistry checkpointRegistry, + final FileSourceConfig config, + final FileSystemOperations fileOps, + final FileTailMetrics metrics, + final Duration rotateWait, + final boolean closeRemoved, + final WatchServiceFactory watchServiceFactory, + final boolean macOS) { + this(globPathResolver, readerPool, checkpointRegistry, config, fileOps, metrics, rotateWait, closeRemoved, + watchServiceFactory, macOS, DirectoryWatcher::createDefaultPollScheduler, + DirectoryWatcher::isNetworkFilesystem); + } + + DirectoryWatcher(final GlobPathResolver globPathResolver, + final TailFileReaderPool readerPool, + final CheckpointRegistry checkpointRegistry, + final FileSourceConfig config, + final FileSystemOperations fileOps, + final FileTailMetrics metrics, + final Duration rotateWait, + final boolean closeRemoved, + final WatchServiceFactory watchServiceFactory, + final boolean macOS, + final Supplier pollSchedulerSupplier) { + this(globPathResolver, readerPool, checkpointRegistry, config, fileOps, metrics, rotateWait, closeRemoved, + watchServiceFactory, macOS, pollSchedulerSupplier, + DirectoryWatcher::isNetworkFilesystem); + } + + DirectoryWatcher(final GlobPathResolver globPathResolver, + final TailFileReaderPool readerPool, + final CheckpointRegistry checkpointRegistry, + final FileSourceConfig config, + final FileSystemOperations fileOps, + final FileTailMetrics metrics, + final Duration rotateWait, + final boolean closeRemoved, + final WatchServiceFactory watchServiceFactory, + final boolean macOS, + final Supplier pollSchedulerSupplier, + final Predicate networkFsCheck) { + this.globPathResolver = Objects.requireNonNull(globPathResolver, "globPathResolver must not be null"); + this.readerPool = Objects.requireNonNull(readerPool, "readerPool must not be null"); + this.checkpointRegistry = Objects.requireNonNull(checkpointRegistry, "checkpointRegistry must not be null"); + this.config = Objects.requireNonNull(config, "config must not be null"); + this.fileOps = Objects.requireNonNull(fileOps, "fileOps must not be null"); + this.metrics = Objects.requireNonNull(metrics, "metrics must not be null"); + this.rotateWait = Objects.requireNonNull(rotateWait, "rotateWait must not be null"); + this.closeRemoved = closeRemoved; + this.knownFiles = ConcurrentHashMap.newKeySet(); + this.watchServiceFactory = Objects.requireNonNull(watchServiceFactory, "watchServiceFactory must not be null"); + this.macOS = macOS; + this.pollSchedulerSupplier = Objects.requireNonNull(pollSchedulerSupplier, "pollSchedulerSupplier must not be null"); + this.networkFsCheck = Objects.requireNonNull(networkFsCheck, "networkFsCheck must not be null"); + } + + public void start() { + running = true; + + final Set initialFiles = globPathResolver.resolve(); + knownFiles.addAll(initialFiles); + for (final Path file : initialFiles) { + addFileToPool(file); + } + + final boolean useWatchService = shouldUseWatchService(); + startPollScheduler(useWatchService); + if (useWatchService) { + startWatchService(); + } + } + + public void stop() { + running = false; + + if (watchThread != null) { + watchThread.interrupt(); + } + + if (watchService != null) { + try { + watchService.close(); + } catch (final IOException e) { + LOG.warn("Error closing WatchService", e); + } + } + + if (pollScheduler != null) { + pollScheduler.shutdown(); + try { + if (!pollScheduler.awaitTermination(SHUTDOWN_TIMEOUT_SECONDS, TimeUnit.SECONDS)) { + pollScheduler.shutdownNow(); + } + } catch (final InterruptedException e) { + pollScheduler.shutdownNow(); + Thread.currentThread().interrupt(); + } + } + + if (watchThread != null) { + try { + watchThread.join(TimeUnit.SECONDS.toMillis(SHUTDOWN_TIMEOUT_SECONDS)); + } catch (final InterruptedException e) { + Thread.currentThread().interrupt(); + } + } + } + + private boolean shouldUseWatchService() { + final Set watchDirs = globPathResolver.getWatchDirectories(); + for (final Path dir : watchDirs) { + if (Files.isDirectory(dir) && networkFsCheck.test(dir)) { + LOG.info("Network filesystem detected at {}. Using polling only.", dir); + return false; + } + } + return true; + } + + static boolean isNetworkFilesystem(final Path path) { + try { + final FileStore fileStore = Files.getFileStore(path); + final String fsType = fileStore.type().toLowerCase(); + return NETWORK_FS_TYPES.contains(fsType); + } catch (final IOException e) { + LOG.warn("Unable to determine filesystem type for {}. Assuming local.", path); + return false; + } + } + + private static boolean isMacOS() { + return System.getProperty("os.name").toLowerCase().contains("mac"); + } + + private void startWatchService() { + try { + watchService = watchServiceFactory.create(); + final Set watchDirs = globPathResolver.getWatchDirectories(); + for (final Path dir : watchDirs) { + if (Files.isDirectory(dir)) { + registerDirectory(dir); + } + } + + watchThread = new Thread(this::watchLoop, "file-tail-watcher"); + watchThread.setDaemon(true); + watchThread.start(); + LOG.info("WatchService started for {} directories", watchDirs.size()); + } catch (final IOException | RuntimeException e) { + LOG.error("Failed to create WatchService. Falling back to polling only.", e); + watchService = null; + } + } + + private void registerDirectory(final Path dir) { + try { + dir.register(watchService, + StandardWatchEventKinds.ENTRY_CREATE, + StandardWatchEventKinds.ENTRY_DELETE, + StandardWatchEventKinds.OVERFLOW); + LOG.debug("Registered WatchService for directory: {}", dir); + } catch (final IOException e) { + LOG.warn("Failed to register WatchService for directory: {}", dir, e); + } + } + + static ScheduledExecutorService createDefaultPollScheduler() { + return Executors.newSingleThreadScheduledExecutor(r -> { + final Thread thread = new Thread(r, "file-tail-poll"); + thread.setDaemon(true); + return thread; + }); + } + + private void startPollScheduler(final boolean watchServiceActive) { + pollScheduler = pollSchedulerSupplier.get(); + + final long intervalMillis; + if (!watchServiceActive || macOS) { + intervalMillis = config.getPollInterval().toMillis(); + LOG.info("Poll scheduler started with interval {}ms (primary mode)", intervalMillis); + } else { + final int supplementaryPollMultiplier = 10; + intervalMillis = config.getPollInterval().toMillis() * supplementaryPollMultiplier; + LOG.info("Poll scheduler started with interval {}ms (supplementary mode)", intervalMillis); + } + + pollScheduler.scheduleAtFixedRate(this::pollScan, intervalMillis, intervalMillis, TimeUnit.MILLISECONDS); + } + + private void watchLoop() { + while (running) { + try { + final WatchKey key = watchService.take(); + for (final WatchEvent event : key.pollEvents()) { + handleWatchEvent(key, event); + } + if (!key.reset()) { + LOG.warn("WatchKey no longer valid. Directory may have been deleted."); + } + } catch (final ClosedWatchServiceException e) { + LOG.debug("WatchService closed, exiting watch loop"); + break; + } catch (final InterruptedException e) { + Thread.currentThread().interrupt(); + break; + } + } + } + + @SuppressWarnings("unchecked") + private void handleWatchEvent(final WatchKey key, final WatchEvent event) { + final WatchEvent.Kind kind = event.kind(); + + if (kind == StandardWatchEventKinds.OVERFLOW) { + LOG.warn("WatchService OVERFLOW detected. Triggering full rescan."); + triggerFullRescan(); + return; + } + + final Path watchedDir = (Path) key.watchable(); + final WatchEvent pathEvent = (WatchEvent) event; + final Path child = watchedDir.resolve(pathEvent.context()).toAbsolutePath().normalize(); + + if (kind == StandardWatchEventKinds.ENTRY_CREATE) { + if (Files.isRegularFile(child) && globPathResolver.matches(child)) { + LOG.debug("New file detected via WatchService: {}", child); + knownFiles.add(child); + addFileToPool(child); + } + } else if (kind == StandardWatchEventKinds.ENTRY_DELETE) { + if (knownFiles.contains(child)) { + LOG.debug("File deletion detected via WatchService: {}. Waiting {} before treating as removed.", child, rotateWait); + try { + pollScheduler.schedule(() -> handleDeletion(child), rotateWait.toMillis(), TimeUnit.MILLISECONDS); + } catch (final RejectedExecutionException e) { + LOG.debug("Poll scheduler rejected deletion handling (shutting down)"); + } + } + } + } + + private void triggerFullRescan() { + try { + pollScheduler.execute(this::pollScan); + } catch (final RejectedExecutionException e) { + LOG.debug("Poll scheduler rejected rescan (shutting down)"); + } + } + + private void handleDeletion(final Path file) { + if (!Files.exists(file)) { + knownFiles.remove(file); + if (closeRemoved) { + LOG.info("File confirmed removed after rotate_wait. Closing reader for: {}", file); + readerPool.closeReaderForPath(file); + } else { + LOG.info("File confirmed removed after rotate_wait. closeRemoved=false, keeping reader open for: {}", file); + } + } else { + LOG.debug("File reappeared during rotate_wait period (likely rotation): {}", file); + } + } + + void pollScan() { + if (!running) { + return; + } + + try { + readerPool.closeInactiveReaders(); + + final Set currentFiles = globPathResolver.resolve(); + + final Set newFiles = new HashSet<>(currentFiles); + newFiles.removeAll(knownFiles); + for (final Path file : newFiles) { + LOG.debug("New file detected via poll scan: {}", file); + addFileToPool(file); + } + + final Set vanishedFiles = new HashSet<>(knownFiles); + vanishedFiles.removeAll(currentFiles); + for (final Path file : vanishedFiles) { + LOG.debug("File vanished detected via poll scan: {}. Deferring by rotateWait.", file); + try { + pollScheduler.schedule(() -> handleDeletion(file), rotateWait.toMillis(), TimeUnit.MILLISECONDS); + } catch (final RejectedExecutionException e) { + LOG.debug("Poll scheduler rejected vanished file handling (shutting down)"); + } + } + + knownFiles.addAll(currentFiles); + } catch (final RuntimeException e) { + LOG.error("Error during poll scan", e); + } + } + + private void addFileToPool(final Path file) { + try { + final FileIdentity identity = FileIdentity.from(file, fileOps, config.getFingerprintBytes()); + readerPool.addFile(identity, file); + } catch (final RuntimeException e) { + LOG.warn("Failed to add file to reader pool: {}", file, e); + } + } +} diff --git a/data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileFormat.java b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileFormat.java similarity index 79% rename from data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileFormat.java rename to data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileFormat.java index 01201edf7a..49322c7fb8 100644 --- a/data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileFormat.java +++ b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileFormat.java @@ -1,6 +1,11 @@ /* * Copyright OpenSearch Contributors * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * */ package org.opensearch.dataprepper.plugins.source.file; @@ -33,6 +38,9 @@ public String toString() { } public static FileFormat getByName(final String name) { + if (name == null) { + return PLAIN; + } return NAMES_MAP.get(name.toLowerCase()); } } \ No newline at end of file diff --git a/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileIdentity.java b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileIdentity.java new file mode 100644 index 0000000000..6d220c8f98 --- /dev/null +++ b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileIdentity.java @@ -0,0 +1,105 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + +package org.opensearch.dataprepper.plugins.source.file; + +import java.io.IOException; +import java.nio.file.Path; +import java.nio.file.attribute.BasicFileAttributes; +import java.util.Objects; +import java.util.zip.CRC32; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public final class FileIdentity { + + private static final Logger LOG = LoggerFactory.getLogger(FileIdentity.class); + private final String identity; + private final Path path; + + private FileIdentity(final String identity, final Path path) { + this.identity = Objects.requireNonNull(identity, "identity must not be null"); + this.path = Objects.requireNonNull(path, "path must not be null"); + } + + public static FileIdentity from(final Path path, final FileSystemOperations fileOps, final int fingerprintBytes) { + Objects.requireNonNull(path, "path must not be null"); + Objects.requireNonNull(fileOps, "fileOps must not be null"); + if (fingerprintBytes <= 0) { + throw new IllegalArgumentException("fingerprintBytes must be positive"); + } + + try { + final BasicFileAttributes attrs = fileOps.readAttributes(path); + final Object fileKey = attrs.fileKey(); + if (fileKey != null) { + return new FileIdentity("inode:" + fileKey, path); + } + return buildFallbackIdentity(path, fileOps, attrs, fingerprintBytes); + } catch (final IOException e) { + LOG.warn("Failed to read file attributes for {}. Using path-based identity which disables rotation detection.", path, e); + return new FileIdentity("path:" + path.toAbsolutePath(), path); + } + } + + private static FileIdentity buildFallbackIdentity(final Path path, final FileSystemOperations fileOps, + final BasicFileAttributes attrs, final int fingerprintBytes) { + long fileSize; + try { + fileSize = fileOps.size(path); + } catch (final IOException e) { + LOG.warn("Failed to read file size for {}. Using path-based identity.", path, e); + return new FileIdentity("path:" + path.toAbsolutePath(), path); + } + + if (fileSize == 0) { + return new FileIdentity("path:" + path.toAbsolutePath(), path); + } + + final int bytesToRead = (int) Math.min(fingerprintBytes, fileSize); + long crcValue = 0; + try { + final byte[] bytes = fileOps.readBytes(path, bytesToRead); + final CRC32 crc32 = new CRC32(); + crc32.update(bytes); + crcValue = crc32.getValue(); + } catch (final IOException e) { + LOG.warn("Failed to read fingerprint bytes for {}. Using path-based identity.", path, e); + return new FileIdentity("path:" + path.toAbsolutePath(), path); + } + + final String creationTime = attrs.creationTime().toString(); + final String id = "crc:" + crcValue + ":created:" + creationTime; + return new FileIdentity(id, path); + } + + public Path getPath() { + return path; + } + + @Override + public boolean equals(final Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + final FileIdentity that = (FileIdentity) o; + return identity.equals(that.identity); + } + + @Override + public int hashCode() { + return identity.hashCode(); + } + + @Override + public String toString() { + return "FileIdentity{" + identity + ", path=" + path + "}"; + } +} diff --git a/data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileSource.java b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileSource.java similarity index 53% rename from data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileSource.java rename to data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileSource.java index 9698144097..dd585aea8d 100644 --- a/data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileSource.java +++ b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileSource.java @@ -1,6 +1,11 @@ /* * Copyright OpenSearch Contributors * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * */ package org.opensearch.dataprepper.plugins.source.file; @@ -9,6 +14,7 @@ import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; import org.opensearch.dataprepper.metrics.PluginMetrics; +import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSetManager; import org.opensearch.dataprepper.model.annotations.DataPrepperPlugin; import org.opensearch.dataprepper.model.annotations.DataPrepperPluginConstructor; import org.opensearch.dataprepper.model.buffer.Buffer; @@ -24,12 +30,12 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; - import java.io.BufferedReader; import java.io.IOException; import java.io.InputStream; import java.io.InputStreamReader; -import java.nio.charset.StandardCharsets; +import java.nio.charset.Charset; +import java.util.Objects; import java.nio.file.Files; import java.nio.file.Path; import java.nio.file.Paths; @@ -37,40 +43,55 @@ import java.util.Map; import java.util.concurrent.TimeoutException; -import static com.google.common.base.Preconditions.checkNotNull; import static java.lang.String.format; import static org.opensearch.dataprepper.logging.DataPrepperMarkers.SENSITIVE; @DataPrepperPlugin(name = "file", pluginType = Source.class, pluginConfigurationType = FileSourceConfig.class) public class FileSource implements Source> { - static final String MESSAGE_KEY = "message"; + private static final String MESSAGE_KEY = "message"; private static final Logger LOG = LoggerFactory.getLogger(FileSource.class); private static final TypeReference> MAP_TYPE_REFERENCE = new TypeReference<>() { }; private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); private static final long STOP_WAIT_MILLIS = 200; + private static final int MAX_FILES_PER_THREAD_WARNING_THRESHOLD = 250; private final FileSourceConfig fileSourceConfig; private final FileStrategy fileStrategy; private final EventFactory eventFactory; + private final PluginMetrics pluginMetrics; + private final PluginFactory pluginFactory; private final DecompressionEngine decompressionEngine; + private final AcknowledgementSetManager acknowledgementSetManager; + private final boolean acknowledgementsEnabled; private Thread readThread; + private TailFileReaderPool readerPool; + private CheckpointRegistry checkpointRegistry; + private DirectoryWatcher directoryWatcher; - private boolean isStopRequested; + private volatile boolean isStopRequested; private final int writeTimeout; @DataPrepperPluginConstructor public FileSource( final FileSourceConfig fileSourceConfig, final PluginMetrics pluginMetrics, final PluginFactory pluginFactory, - final EventFactory eventFactory) { - this.eventFactory = eventFactory; + final EventFactory eventFactory, final AcknowledgementSetManager acknowledgementSetManager) { + Objects.requireNonNull(fileSourceConfig, "fileSourceConfig must not be null"); + this.eventFactory = Objects.requireNonNull(eventFactory, "eventFactory must not be null"); + this.pluginMetrics = Objects.requireNonNull(pluginMetrics, "pluginMetrics must not be null"); + this.pluginFactory = Objects.requireNonNull(pluginFactory, "pluginFactory must not be null"); + this.acknowledgementsEnabled = fileSourceConfig.isAcknowledgments(); + if (acknowledgementsEnabled) { + Objects.requireNonNull(acknowledgementSetManager, "AcknowledgementSetManager is required when acknowledgments is enabled"); + } + this.acknowledgementSetManager = acknowledgementSetManager; fileSourceConfig.validate(); this.fileSourceConfig = fileSourceConfig; this.isStopRequested = false; this.writeTimeout = FileSourceConfig.DEFAULT_TIMEOUT; this.decompressionEngine = fileSourceConfig.getCompression().getDecompressionEngine(); - if(fileSourceConfig.getCodec() != null) { + if (!fileSourceConfig.isTail() && fileSourceConfig.getCodec() != null) { fileStrategy = new CodecFileStrategy(pluginFactory); } else { fileStrategy = new ClassicFileStrategy(); @@ -80,7 +101,12 @@ public FileSource( @Override public void start(final Buffer> buffer) { - checkNotNull(buffer, "Buffer cannot be null for file source to start"); + Objects.requireNonNull(buffer, "Buffer cannot be null for file source to start"); + + if (fileSourceConfig.isTail()) { + startTailing(buffer); + return; + } LOG.info("Starting file source with {} path.", fileSourceConfig.getFilePathToRead()); @@ -92,17 +118,116 @@ public void start(final Buffer> buffer) { readThread.start(); } + private void startTailing(final Buffer> buffer) { + LOG.info("Starting file source in tail mode with paths: {}", fileSourceConfig.getAllPaths()); + + final int maxActiveFiles = fileSourceConfig.getMaxActiveFiles(); + final int readerThreads = fileSourceConfig.getReaderThreads(); + if (readerThreads > 0 && maxActiveFiles / readerThreads > MAX_FILES_PER_THREAD_WARNING_THRESHOLD) { + LOG.warn("max_active_files ({}) is {} times reader_threads ({}). Files with pending data may experience high latency.", + maxActiveFiles, maxActiveFiles / readerThreads, readerThreads); + } + + try { + + final FileTailMetrics tailMetrics = new FileTailMetrics(pluginMetrics); + final FileSystemOperations fileOps = new DefaultFileSystemOperations(); + + final String checkpointPath = fileSourceConfig.getCheckpointFile(); + final Path cpFile = checkpointPath != null + ? Paths.get(checkpointPath) + : Paths.get(System.getProperty("java.io.tmpdir"), "data-prepper-file-source-checkpoint.json"); + + checkpointRegistry = new CheckpointRegistry( + cpFile, + fileSourceConfig.getCheckpointInterval(), + fileSourceConfig.getCheckpointCleanupAfter()); + + final Charset encoding = Charset.forName(fileSourceConfig.getEncoding()); + + final RotationDetector rotationDetector = new RotationDetector(fileOps, fileSourceConfig.getFingerprintBytes()); + + InputCodec tailCodec = null; + if (fileSourceConfig.getCodec() != null) { + final PluginModel codecConfiguration = fileSourceConfig.getCodec(); + final PluginSetting codecPluginSettings = new PluginSetting( + codecConfiguration.getPluginName(), codecConfiguration.getPluginSettings()); + tailCodec = pluginFactory.loadPlugin(InputCodec.class, codecPluginSettings); + } + + final TailFileReaderContext readerContext = new TailFileReaderContext( + buffer, eventFactory, fileOps, tailMetrics, rotationDetector, + acknowledgementSetManager, acknowledgementsEnabled, + encoding, + fileSourceConfig.getReadBufferSize(), + fileSourceConfig.getMaxLineLength(), + writeTimeout, + fileSourceConfig.getMaxReadTimePerFile(), + fileSourceConfig.getRotationDrainTimeout(), + fileSourceConfig.getStartPosition(), + fileSourceConfig.isIncludeFileMetadata(), + fileSourceConfig.getAcknowledgmentTimeout(), + fileSourceConfig.getBatchSize(), + fileSourceConfig.getBatchTimeout(), + fileSourceConfig.getMaxAcknowledgmentRetries(), + tailCodec); + + readerPool = new TailFileReaderPool( + checkpointRegistry, tailMetrics, + maxActiveFiles, + readerThreads, + fileSourceConfig.getCloseInactive(), + readerContext); + + final GlobPathResolver globPathResolver = new GlobPathResolver( + fileSourceConfig.getAllPaths(), + fileSourceConfig.getExcludePaths()); + + directoryWatcher = new DirectoryWatcher( + globPathResolver, readerPool, checkpointRegistry, + fileSourceConfig, fileOps, tailMetrics, + fileSourceConfig.getRotateWait(), + fileSourceConfig.isCloseRemoved()); + + directoryWatcher.start(); + } catch (final RuntimeException e) { + shutdownTailingResources(); + throw e; + } + } + + private void shutdownTailingResources() { + if (directoryWatcher != null) { + directoryWatcher.stop(); + } + if (readerPool != null) { + readerPool.shutdown(); + } + if (checkpointRegistry != null) { + checkpointRegistry.shutdown(); + } + } + @Override public void stop() { isStopRequested = true; - try { - readThread.join(STOP_WAIT_MILLIS); - } catch (final InterruptedException e) { - readThread.interrupt(); + shutdownTailingResources(); + + if (readThread != null) { + try { + readThread.join(STOP_WAIT_MILLIS); + } catch (final InterruptedException e) { + readThread.interrupt(); + } } } + @Override + public boolean areAcknowledgementsEnabled() { + return acknowledgementsEnabled; + } + private interface FileStrategy { void start(final Buffer> buffer); } @@ -111,7 +236,7 @@ private class ClassicFileStrategy implements FileStrategy { @Override public void start(Buffer> buffer) { Path filePath = Paths.get(fileSourceConfig.getFilePathToRead()); - try (BufferedReader reader = new BufferedReader(new InputStreamReader(decompressionEngine.createInputStream(Files.newInputStream(filePath)), StandardCharsets.UTF_8))) { + try (BufferedReader reader = new BufferedReader(new InputStreamReader(decompressionEngine.createInputStream(Files.newInputStream(filePath)), Charset.forName(fileSourceConfig.getEncoding())))) { String line; while ((line = reader.readLine()) != null && !isStopRequested) { writeLineAsEventOrString(line, buffer); @@ -153,8 +278,6 @@ private Map parseJson(final String jsonString) { } } - // Temporary function to support both trace and log ingestion pipelines. - // TODO: This function should be removed with the completion of: https://github.com/opensearch-project/data-prepper/issues/546 private void writeLineAsEventOrString(final String line, final Buffer> buffer) throws TimeoutException, IllegalArgumentException { if (fileSourceConfig.getRecordType().equals(FileSourceConfig.EVENT_TYPE)) { buffer.write(getEventRecordFromLine(line), writeTimeout); diff --git a/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileSourceConfig.java b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileSourceConfig.java new file mode 100644 index 0000000000..ca6784f9af --- /dev/null +++ b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileSourceConfig.java @@ -0,0 +1,270 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + +package org.opensearch.dataprepper.plugins.source.file; + +import com.fasterxml.jackson.annotation.JsonIgnore; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; +import jakarta.validation.constraints.AssertTrue; +import org.opensearch.dataprepper.model.configuration.PluginModel; +import org.opensearch.dataprepper.plugins.codec.CompressionOption; + +import java.time.Duration; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; + +public class FileSourceConfig { + static final String ATTRIBUTE_PATH = "path"; + static final String ATTRIBUTE_TYPE = "record_type"; + static final String ATTRIBUTE_FORMAT = "format"; + static final int DEFAULT_TIMEOUT = 5_000; + static final String DEFAULT_TYPE = "string"; + static final String DEFAULT_FORMAT = "plain"; + static final String EVENT_TYPE = "event"; + + @JsonProperty(ATTRIBUTE_PATH) + private String filePathToRead; + + @JsonProperty("paths") + private List paths = Collections.emptyList(); + + @JsonProperty("tail") + private boolean tail = false; + + @JsonProperty(ATTRIBUTE_FORMAT) + private String format = DEFAULT_FORMAT; + + @JsonProperty(ATTRIBUTE_TYPE) + private String recordType = DEFAULT_TYPE; + + @JsonProperty("codec") + private PluginModel codec; + + @JsonProperty("compression") + private CompressionOption compression = CompressionOption.NONE; + + @JsonProperty("start_position") + private StartPosition startPosition = StartPosition.END; + + @JsonProperty("poll_interval") + private Duration pollInterval = Duration.ofSeconds(1); + + @JsonProperty("encoding") + private String encoding = "UTF-8"; + + @JsonProperty("read_buffer_size") + private int readBufferSize = 65536; + + @JsonProperty("max_active_files") + private int maxActiveFiles = 100; + + @JsonProperty("reader_threads") + private int readerThreads = 2; + + @JsonProperty("max_read_time_per_file") + private Duration maxReadTimePerFile = Duration.ofSeconds(5); + + @JsonProperty("rotate_wait") + private Duration rotateWait = Duration.ofSeconds(5); + + @JsonProperty("rotation_drain_timeout") + private Duration rotationDrainTimeout = Duration.ofSeconds(30); + + @JsonProperty("checkpoint_file") + private String checkpointFile; + + @JsonProperty("checkpoint_interval") + private Duration checkpointInterval = Duration.ofSeconds(15); + + @JsonProperty("checkpoint_cleanup_after") + private Duration checkpointCleanupAfter = Duration.ofHours(24); + + @JsonProperty("fingerprint_bytes") + private int fingerprintBytes = 1024; + + @JsonProperty("close_inactive") + private Duration closeInactive = Duration.ofMinutes(5); + + @JsonProperty("close_removed") + private boolean closeRemoved = true; + + @JsonProperty("batch_size") + private int batchSize = 1000; + + @JsonProperty("batch_timeout") + private Duration batchTimeout = Duration.ofSeconds(5); + + @JsonProperty("acknowledgment_timeout") + private Duration acknowledgmentTimeout = Duration.ofSeconds(30); + + @JsonProperty("max_acknowledgment_retries") + private int maxAcknowledgmentRetries = 3; + + @JsonProperty("acknowledgments") + private boolean acknowledgments = false; + + @JsonProperty("include_file_metadata") + private boolean includeFileMetadata = false; + + @JsonProperty("max_line_length") + private int maxLineLength = 1048576; + + @JsonProperty("exclude_paths") + private List excludePaths = Collections.emptyList(); + + public String getFilePathToRead() { + return filePathToRead; + } + + public List getPaths() { + return paths; + } + + public boolean isTail() { + return tail; + } + + public List getAllPaths() { + final List allPaths = new ArrayList<>(getPaths()); + if (filePathToRead != null && !allPaths.contains(filePathToRead)) { + allPaths.add(filePathToRead); + } + return allPaths; + } + + @JsonIgnore + public FileFormat getFormat() { + return FileFormat.getByName(format); + } + + public String getRecordType() { + return recordType; + } + + public PluginModel getCodec() { + return codec; + } + + public CompressionOption getCompression() { + return compression; + } + + public StartPosition getStartPosition() { + return startPosition; + } + + public Duration getPollInterval() { + return pollInterval; + } + + public String getEncoding() { + return encoding; + } + + public int getReadBufferSize() { + return readBufferSize; + } + + public int getMaxActiveFiles() { + return maxActiveFiles; + } + + public int getReaderThreads() { + return readerThreads; + } + + public Duration getMaxReadTimePerFile() { + return maxReadTimePerFile; + } + + public Duration getRotateWait() { + return rotateWait; + } + + public Duration getRotationDrainTimeout() { + return rotationDrainTimeout; + } + + public String getCheckpointFile() { + return checkpointFile; + } + + public Duration getCheckpointInterval() { + return checkpointInterval; + } + + public Duration getCheckpointCleanupAfter() { + return checkpointCleanupAfter; + } + + public int getFingerprintBytes() { + return fingerprintBytes; + } + + public Duration getCloseInactive() { + return closeInactive; + } + + public boolean isCloseRemoved() { + return closeRemoved; + } + + public int getBatchSize() { + return batchSize; + } + + public Duration getBatchTimeout() { + return batchTimeout; + } + + public Duration getAcknowledgmentTimeout() { + return acknowledgmentTimeout; + } + + public int getMaxAcknowledgmentRetries() { + return maxAcknowledgmentRetries; + } + + public boolean isAcknowledgments() { + return acknowledgments; + } + + public boolean isIncludeFileMetadata() { + return includeFileMetadata; + } + + public int getMaxLineLength() { + return maxLineLength; + } + + public List getExcludePaths() { + return excludePaths; + } + + void validate() { + if (tail) { + Preconditions.checkArgument( + (filePathToRead != null && !filePathToRead.isEmpty()) || !paths.isEmpty(), + "At least one of path or paths is required when tail is enabled"); + } else { + Preconditions.checkArgument(filePathToRead != null, + "path is required when tail is disabled. Use paths with tail: true for glob patterns."); + } + Preconditions.checkArgument(EVENT_TYPE.equals(recordType) || DEFAULT_TYPE.equals(recordType), "Invalid type: must be either [event] or [string]"); + Preconditions.checkArgument(DEFAULT_FORMAT.equals(format) || "json".equals(format), "Invalid file format. Options are [json] and [plain]"); + } + + @AssertTrue(message = "The file source requires recordType to be event when using a codec.") + boolean codeRequiresRecordTypeEvent() { + return codec == null || EVENT_TYPE.equals(recordType); + } +} diff --git a/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileSystemOperations.java b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileSystemOperations.java new file mode 100644 index 0000000000..ddafdb078e --- /dev/null +++ b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileSystemOperations.java @@ -0,0 +1,32 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + +package org.opensearch.dataprepper.plugins.source.file; + +import java.io.IOException; +import java.nio.channels.FileChannel; +import java.nio.file.Path; +import java.nio.file.attribute.BasicFileAttributes; +import java.util.stream.Stream; + +public interface FileSystemOperations { + + FileChannel openReadChannel(Path path) throws IOException; + + BasicFileAttributes readAttributes(Path path) throws IOException; + + Stream listDirectory(Path directory) throws IOException; + + boolean exists(Path path); + + long size(Path path) throws IOException; + + byte[] readBytes(Path path, int length) throws IOException; +} diff --git a/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileTailMetrics.java b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileTailMetrics.java new file mode 100644 index 0000000000..44395566d0 --- /dev/null +++ b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileTailMetrics.java @@ -0,0 +1,122 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + +package org.opensearch.dataprepper.plugins.source.file; + +import io.micrometer.core.instrument.Counter; +import io.micrometer.core.instrument.Timer; +import org.opensearch.dataprepper.metrics.PluginMetrics; + +import java.util.concurrent.atomic.AtomicLong; + +public final class FileTailMetrics { + + private final Counter linesRead; + private final Counter bytesRead; + private final Counter linesTruncated; + private final Counter filesOpened; + private final Counter filesClosed; + private final Counter filesRotated; + private final Counter readErrors; + private final Counter writeTimeouts; + private final Counter checkpointFlushes; + private final Counter checkpointErrors; + private final Counter eventsEmitted; + private final Counter dataLossEvents; + private final Counter acknowledgmentFailures; + private final Timer backpressureTimer; + private final AtomicLong activeFileCount; + private final AtomicLong fileLagBytes; + + public FileTailMetrics(final PluginMetrics pluginMetrics) { + this.linesRead = pluginMetrics.counter("tailLinesRead"); + this.bytesRead = pluginMetrics.counter("tailBytesRead"); + this.linesTruncated = pluginMetrics.counter("tailLinesTruncated"); + this.filesOpened = pluginMetrics.counter("tailFilesOpened"); + this.filesClosed = pluginMetrics.counter("tailFilesClosed"); + this.filesRotated = pluginMetrics.counter("tailFilesRotated"); + this.readErrors = pluginMetrics.counter("tailReadErrors"); + this.writeTimeouts = pluginMetrics.counter("tailWriteTimeouts"); + this.checkpointFlushes = pluginMetrics.counter("tailCheckpointFlushes"); + this.checkpointErrors = pluginMetrics.counter("tailCheckpointErrors"); + this.eventsEmitted = pluginMetrics.counter("tailEventsEmitted"); + this.dataLossEvents = pluginMetrics.counter("tailDataLossEvents"); + this.acknowledgmentFailures = pluginMetrics.counter("tailAcknowledgmentFailures"); + this.backpressureTimer = pluginMetrics.timer("tailBackpressureTime"); + this.activeFileCount = new AtomicLong(0); + pluginMetrics.gauge("tailActiveFiles", activeFileCount); + this.fileLagBytes = new AtomicLong(0); + pluginMetrics.gauge("tailFileLagBytes", fileLagBytes); + } + + public Counter getLinesRead() { + return linesRead; + } + + public Counter getBytesRead() { + return bytesRead; + } + + public Counter getLinesTruncated() { + return linesTruncated; + } + + public Counter getFilesOpened() { + return filesOpened; + } + + public Counter getFilesClosed() { + return filesClosed; + } + + public Counter getFilesRotated() { + return filesRotated; + } + + public Counter getReadErrors() { + return readErrors; + } + + public Counter getWriteTimeouts() { + return writeTimeouts; + } + + public Counter getCheckpointFlushes() { + return checkpointFlushes; + } + + public Counter getCheckpointErrors() { + return checkpointErrors; + } + + public AtomicLong getActiveFileCount() { + return activeFileCount; + } + + public Counter getEventsEmitted() { + return eventsEmitted; + } + + public Timer getBackpressureTimer() { + return backpressureTimer; + } + + public AtomicLong getFileLagBytes() { + return fileLagBytes; + } + + public Counter getDataLossEvents() { + return dataLossEvents; + } + + public Counter getAcknowledgmentFailures() { + return acknowledgmentFailures; + } +} diff --git a/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/GlobPathResolver.java b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/GlobPathResolver.java new file mode 100644 index 0000000000..a6bff96d6f --- /dev/null +++ b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/GlobPathResolver.java @@ -0,0 +1,170 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + +package org.opensearch.dataprepper.plugins.source.file; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.nio.file.FileSystems; +import java.nio.file.FileVisitResult; +import java.nio.file.Files; +import java.nio.file.InvalidPathException; +import java.nio.file.Path; +import java.nio.file.PathMatcher; +import java.nio.file.Paths; +import java.nio.file.SimpleFileVisitor; +import java.nio.file.attribute.BasicFileAttributes; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Objects; +import java.util.Set; +import java.util.regex.PatternSyntaxException; + +public final class GlobPathResolver { + + private static final Logger LOG = LoggerFactory.getLogger(GlobPathResolver.class); + private static final String WILDCARD_CHARS = "*?{["; + + private final List includePatterns; + private final List excludePatterns; + private final List includeMatchers; + private final List excludeMatchers; + + public GlobPathResolver(final List includePatterns, final List excludePatterns) { + this.includePatterns = Objects.requireNonNull(includePatterns, "includePatterns must not be null"); + this.excludePatterns = excludePatterns != null ? excludePatterns : Collections.emptyList(); + this.includeMatchers = buildMatchers(this.includePatterns); + this.excludeMatchers = buildMatchers(this.excludePatterns); + } + + public Set resolve() { + final Set result = new HashSet<>(); + final Set baseDirectories = getWatchDirectories(); + + for (final Path baseDir : baseDirectories) { + if (!Files.isDirectory(baseDir)) { + LOG.warn("Base directory does not exist or is not a directory: {}", baseDir); + continue; + } + walkDirectory(baseDir, result); + } + + return result; + } + + void walkDirectory(final Path baseDir, final Set result) { + walkDirectory(baseDir, createFileVisitor(result)); + } + + void walkDirectory(final Path baseDir, final SimpleFileVisitor visitor) { + try { + Files.walkFileTree(baseDir, visitor); + } catch (final IOException e) { + LOG.warn("Failed to walk directory tree at: {}", baseDir, e); + } + } + + SimpleFileVisitor createFileVisitor(final Set result) { + return new SimpleFileVisitor<>() { + @Override + public FileVisitResult visitFile(final Path file, final BasicFileAttributes attrs) { + final Path normalized = file.toAbsolutePath().normalize(); + if (matches(normalized)) { + result.add(normalized); + } + return FileVisitResult.CONTINUE; + } + + @Override + public FileVisitResult visitFileFailed(final Path file, final IOException exc) { + LOG.warn("Failed to access file during glob resolution: {}", file, exc); + return FileVisitResult.CONTINUE; + } + }; + } + + public boolean matches(final Path path) { + final Path normalized = path.toAbsolutePath().normalize(); + + boolean included = false; + for (final PathMatcher matcher : includeMatchers) { + if (matcher.matches(normalized)) { + included = true; + break; + } + } + + if (!included) { + return false; + } + + for (final PathMatcher matcher : excludeMatchers) { + if (matcher.matches(normalized)) { + return false; + } + } + + return true; + } + + public Set getWatchDirectories() { + final Set directories = new HashSet<>(); + for (final String pattern : includePatterns) { + final Path baseDir = extractBaseDirectory(pattern); + directories.add(baseDir); + } + return directories; + } + + static Path extractBaseDirectory(final String pattern) { + final String normalized = Paths.get(pattern).toAbsolutePath().normalize().toString(); + final StringBuilder staticPrefix = new StringBuilder(); + + for (int i = 0; i < normalized.length(); i++) { + final char c = normalized.charAt(i); + if (WILDCARD_CHARS.indexOf(c) >= 0) { + break; + } + staticPrefix.append(c); + } + + String prefix = staticPrefix.toString(); + final int lastSep = prefix.lastIndexOf('/'); + if (lastSep > 0) { + prefix = prefix.substring(0, lastSep); + } + + final Path result = Paths.get(prefix).toAbsolutePath().normalize(); + if (Files.isDirectory(result)) { + return result; + } + + final Path parent = result.getParent(); + return Objects.requireNonNullElse(parent, result); + } + + private static List buildMatchers(final List patterns) { + final List matchers = new ArrayList<>(patterns.size()); + for (final String pattern : patterns) { + try { + final String absolutePattern = Paths.get(pattern).toAbsolutePath().normalize().toString(); + matchers.add(FileSystems.getDefault().getPathMatcher("glob:" + absolutePattern)); + } catch (final PatternSyntaxException | InvalidPathException e) { + LOG.error("Invalid glob pattern '{}': {}", pattern, e.getMessage()); + throw new IllegalArgumentException("Invalid glob pattern: " + pattern, e); + } + } + return matchers; + } +} diff --git a/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/PendingFile.java b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/PendingFile.java new file mode 100644 index 0000000000..b53ee06622 --- /dev/null +++ b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/PendingFile.java @@ -0,0 +1,44 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + +package org.opensearch.dataprepper.plugins.source.file; + +import java.nio.file.Path; +import java.util.Objects; + +public final class PendingFile { + + private final FileIdentity fileIdentity; + private final Path path; + private final long enqueuedTimeMillis; + + public PendingFile(final FileIdentity fileIdentity, final Path path) { + this.fileIdentity = Objects.requireNonNull(fileIdentity, "fileIdentity must not be null"); + this.path = Objects.requireNonNull(path, "path must not be null"); + this.enqueuedTimeMillis = System.currentTimeMillis(); + } + + public FileIdentity getFileIdentity() { + return fileIdentity; + } + + public Path getPath() { + return path; + } + + public long getEnqueuedTimeMillis() { + return enqueuedTimeMillis; + } + + @Override + public String toString() { + return "PendingFile{path=" + path + ", identity=" + fileIdentity + "}"; + } +} diff --git a/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/RotationDetector.java b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/RotationDetector.java new file mode 100644 index 0000000000..057235d799 --- /dev/null +++ b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/RotationDetector.java @@ -0,0 +1,93 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + +package org.opensearch.dataprepper.plugins.source.file; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.nio.file.NoSuchFileException; +import java.nio.file.Path; +import java.util.Objects; + +public final class RotationDetector { + + private static final Logger LOG = LoggerFactory.getLogger(RotationDetector.class); + + private final FileSystemOperations fileOps; + private final int fingerprintBytes; + + public RotationDetector(final FileSystemOperations fileOps, final int fingerprintBytes) { + this.fileOps = Objects.requireNonNull(fileOps, "fileOps must not be null"); + if (fingerprintBytes <= 0) { + throw new IllegalArgumentException("fingerprintBytes must be positive"); + } + this.fingerprintBytes = fingerprintBytes; + } + + public int getFingerprintBytes() { + return fingerprintBytes; + } + + public RotationResult checkRotation(final Path path, final FileIdentity knownIdentity, final long currentOffset) { + if (!fileOps.exists(path)) { + LOG.debug("File deleted: {}", path); + return RotationResult.DELETED; + } + + final FileIdentity currentIdentity; + try { + currentIdentity = FileIdentity.from(path, fileOps, fingerprintBytes); + } catch (final RuntimeException e) { + if (isCausedByNoSuchFile(e)) { + LOG.debug("File deleted: {}", path); + return RotationResult.DELETED; + } + LOG.warn("Error checking rotation for file {}", path, e); + return RotationResult.NO_ROTATION; + } + + if (!currentIdentity.equals(knownIdentity)) { + LOG.info("Create/rename rotation detected for file {}", path); + return new RotationResult(RotationType.CREATE_RENAME, currentIdentity); + } + + try { + final long fileSize = fileOps.size(path); + if (fileSize < currentOffset) { + LOG.info("Copytruncate rotation detected for file {} (size={}, offset={})", path, fileSize, currentOffset); + return new RotationResult(RotationType.COPYTRUNCATE, currentIdentity); + } + } catch (final NoSuchFileException e) { + LOG.debug("File deleted during size check: {}", path); + return RotationResult.DELETED; + } catch (final IOException e) { + LOG.warn("Error reading file size for {}", path, e); + return RotationResult.NO_ROTATION; + } + + return RotationResult.NO_ROTATION; + } + + boolean isCausedByNoSuchFile(final Exception e) { + if (e instanceof NoSuchFileException) { + return true; + } + Throwable cause = e.getCause(); + while (cause != null) { + if (cause instanceof NoSuchFileException) { + return true; + } + cause = cause.getCause(); + } + return false; + } +} diff --git a/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/RotationResult.java b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/RotationResult.java new file mode 100644 index 0000000000..7aa6470534 --- /dev/null +++ b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/RotationResult.java @@ -0,0 +1,41 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + +package org.opensearch.dataprepper.plugins.source.file; + +import java.util.Objects; + +public final class RotationResult { + + static final RotationResult NO_ROTATION = new RotationResult(RotationType.NO_ROTATION, null); + static final RotationResult DELETED = new RotationResult(RotationType.DELETED, null); + + private final RotationType rotationType; + private final FileIdentity newFileIdentity; + + RotationResult(final RotationType rotationType, final FileIdentity newFileIdentity) { + this.rotationType = Objects.requireNonNull(rotationType, "rotationType must not be null"); + this.newFileIdentity = newFileIdentity; + } + + public RotationType getRotationType() { + return rotationType; + } + + public FileIdentity getNewFileIdentity() { + return newFileIdentity; + } + + @Override + public String toString() { + return "RotationResult{type=" + rotationType + + (newFileIdentity != null ? ", newIdentity=" + newFileIdentity : "") + "}"; + } +} diff --git a/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/RotationType.java b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/RotationType.java new file mode 100644 index 0000000000..9bd6a3b48c --- /dev/null +++ b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/RotationType.java @@ -0,0 +1,18 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + +package org.opensearch.dataprepper.plugins.source.file; + +public enum RotationType { + NO_ROTATION, + CREATE_RENAME, + COPYTRUNCATE, + DELETED +} diff --git a/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/StartPosition.java b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/StartPosition.java new file mode 100644 index 0000000000..03bf75dff0 --- /dev/null +++ b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/StartPosition.java @@ -0,0 +1,48 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + +package org.opensearch.dataprepper.plugins.source.file; + +import com.fasterxml.jackson.annotation.JsonCreator; + +import java.util.Map; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +public enum StartPosition { + BEGINNING("beginning"), + END("end"); + + private static final Map NAMES_MAP = Stream.of(values()) + .collect(Collectors.toMap(StartPosition::toString, v -> v)); + + private final String name; + + StartPosition(final String name) { + this.name = name; + } + + @JsonCreator + public static StartPosition fromString(final String name) { + if (name == null) { + throw new IllegalArgumentException("Invalid start_position: null. Valid values are: " + NAMES_MAP.keySet()); + } + final StartPosition value = NAMES_MAP.get(name.toLowerCase()); + if (value == null) { + throw new IllegalArgumentException("Invalid start_position: " + name + ". Valid values are: " + NAMES_MAP.keySet()); + } + return value; + } + + @Override + public String toString() { + return name; + } +} diff --git a/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/TailFileReader.java b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/TailFileReader.java new file mode 100644 index 0000000000..5281422120 --- /dev/null +++ b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/TailFileReader.java @@ -0,0 +1,499 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + +package org.opensearch.dataprepper.plugins.source.file; + +import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSet; +import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSetManager; +import org.opensearch.dataprepper.model.buffer.Buffer; +import org.opensearch.dataprepper.model.codec.InputCodec; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.event.EventBuilder; +import org.opensearch.dataprepper.model.event.EventFactory; +import org.opensearch.dataprepper.model.record.Record; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.CharBuffer; +import java.nio.channels.FileChannel; +import java.nio.charset.Charset; +import java.nio.charset.CharsetDecoder; +import java.nio.charset.CoderResult; +import java.nio.charset.CodingErrorAction; +import java.nio.file.NoSuchFileException; +import java.nio.file.Path; +import java.time.Duration; +import java.util.HashMap; +import java.util.Map; +import java.util.Objects; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; + +public final class TailFileReader implements Runnable { + + private static final Logger LOG = LoggerFactory.getLogger(TailFileReader.class); + private static final long BACK_PRESSURE_SLEEP_MILLIS = 100; + private static final String MESSAGE_KEY = "message"; + private static final String EVENT_TYPE = "event"; + private static final String FILE_PATH_KEY = "file_path"; + private static final String FILE_IDENTITY_KEY = "file_identity"; + + private final FileIdentity fileIdentity; + private final Path path; + private final Buffer> buffer; + private final EventFactory eventFactory; + private final FileSystemOperations fileOps; + private final CheckpointEntry checkpointEntry; + private final FileTailMetrics metrics; + private final Charset encoding; + private final int readBufferSize; + private final int maxLineLength; + private final int writeTimeout; + private final Duration maxReadTimePerFile; + private final boolean includeFileMetadata; + private final Runnable onComplete; + private final RotationDetector rotationDetector; + private final AcknowledgementSetManager acknowledgementSetManager; + private final boolean acknowledgementsEnabled; + private final Duration acknowledgmentTimeout; + private final int batchSize; + private final StartPosition startPosition; + private final Duration rotationDrainTimeout; + private final Duration batchTimeout; + private final int maxAcknowledgmentRetries; + private final InputCodec codec; + + private final AtomicLong readOffset; + private final StringBuilder partialLine; + private final String cachedAbsolutePath; + private final String cachedFileIdentityString; + private volatile long lastActivityMillis; + private boolean skippingToNewline; + private volatile long currentBatchEndOffset; + + private AcknowledgementSet currentAckSet; + private int currentBatchCount; + private long batchStartOffset; + private long batchOpenedAtMillis; + private final AtomicInteger acknowledgmentRetryCount = new AtomicInteger(0); + private volatile RotationType lastRotationType; + + public TailFileReader(final Path path, + final FileIdentity fileIdentity, + final CheckpointEntry checkpointEntry, + final TailFileReaderContext context, + final Runnable onComplete) { + this.path = Objects.requireNonNull(path, "path must not be null"); + this.fileIdentity = Objects.requireNonNull(fileIdentity, "fileIdentity must not be null"); + this.checkpointEntry = Objects.requireNonNull(checkpointEntry, "checkpointEntry must not be null"); + Objects.requireNonNull(context, "context must not be null"); + this.onComplete = Objects.requireNonNull(onComplete, "onComplete must not be null"); + + this.buffer = context.getBuffer(); + this.eventFactory = context.getEventFactory(); + this.fileOps = context.getFileOps(); + this.metrics = context.getMetrics(); + this.encoding = context.getEncoding(); + this.readBufferSize = context.getReadBufferSize(); + this.maxLineLength = context.getMaxLineLength(); + this.writeTimeout = context.getWriteTimeout(); + this.maxReadTimePerFile = context.getMaxReadTimePerFile(); + this.includeFileMetadata = context.isIncludeFileMetadata(); + this.rotationDetector = context.getRotationDetector(); + this.acknowledgementSetManager = context.getAcknowledgementSetManager(); + this.acknowledgementsEnabled = context.isAcknowledgementsEnabled(); + this.acknowledgmentTimeout = context.getAcknowledgmentTimeout(); + this.batchSize = context.getBatchSize(); + this.startPosition = context.getStartPosition(); + this.rotationDrainTimeout = context.getRotationDrainTimeout(); + this.batchTimeout = context.getBatchTimeout(); + this.maxAcknowledgmentRetries = context.getMaxAcknowledgmentRetries(); + this.codec = context.getCodec(); + + this.readOffset = new AtomicLong(checkpointEntry.getReadOffset()); + if (checkpointEntry.getReadOffset() == 0 && startPosition == StartPosition.END) { + try { + final long fileSize = fileOps.size(path); + this.readOffset.set(fileSize); + checkpointEntry.setReadOffset(fileSize); + } catch (final IOException e) { + LOG.warn("Unable to determine file size for start_position=end on {}. Starting from offset 0.", path); + } + } + this.partialLine = new StringBuilder(); + this.cachedAbsolutePath = path.toAbsolutePath().toString(); + this.cachedFileIdentityString = fileIdentity.toString(); + this.currentBatchCount = 0; + this.batchStartOffset = readOffset.get(); + this.batchOpenedAtMillis = System.currentTimeMillis(); + this.lastRotationType = RotationType.NO_ROTATION; + this.lastActivityMillis = System.currentTimeMillis(); + } + + @Override + public void run() { + try { + final RotationResult rotation = rotationDetector.checkRotation(path, fileIdentity, readOffset.get()); + lastRotationType = rotation.getRotationType(); + + switch (rotation.getRotationType()) { + case COPYTRUNCATE: + LOG.info("Copytruncate detected for {}. Resetting offset to 0.", path); + metrics.getFilesRotated().increment(); + completePendingAckSet(); + readOffset.set(0); + checkpointEntry.setReadOffset(0); + batchStartOffset = 0; + readFile(); + break; + case DELETED: + LOG.info("File deleted: {}. Closing reader.", path); + completePendingAckSet(); + break; + case CREATE_RENAME: + LOG.info("Create/rename rotation detected for {}. Draining current file.", path); + metrics.getFilesRotated().increment(); + drainCurrentFile(); + completePendingAckSet(); + break; + case NO_ROTATION: + default: + readFile(); + break; + } + } catch (final RuntimeException e) { + LOG.error("Error reading file {}", path, e); + metrics.getReadErrors().increment(); + } finally { + flushPartialLine(); + completePendingAckSet(); + onComplete.run(); + } + } + + private void drainCurrentFile() { + try (final FileChannel channel = fileOps.openReadChannel(path)) { + metrics.getFilesOpened().increment(); + channel.position(readOffset.get()); + readLoop(channel, rotationDrainTimeout.toMillis(), true); + } catch (final NoSuchFileException e) { + LOG.warn("File already removed during drain: {}", path); + } catch (final IOException e) { + LOG.error("IO error draining file {}", path, e); + metrics.getReadErrors().increment(); + } finally { + metrics.getFilesClosed().increment(); + } + } + + private void readFile() { + try (final FileChannel channel = fileOps.openReadChannel(path)) { + metrics.getFilesOpened().increment(); + channel.position(readOffset.get()); + readLoop(channel, maxReadTimePerFile.toMillis(), false); + updateFileLagBytes(); + } catch (final NoSuchFileException e) { + LOG.warn("File not found: {}", path); + } catch (final IOException e) { + LOG.error("IO error reading file {}", path, e); + metrics.getReadErrors().increment(); + } finally { + metrics.getFilesClosed().increment(); + } + } + + private void readLoop(final FileChannel channel, final long timeoutMillis, final boolean isDraining) throws IOException { + final ByteBuffer byteBuffer = ByteBuffer.allocate(readBufferSize); + final long loopStart = System.currentTimeMillis(); + final ByteArrayOutputStream codecAccumulator = codec != null ? new ByteArrayOutputStream() : null; + long codecBytesAccumulated = 0; + final CharsetDecoder decoder = codec == null ? encoding.newDecoder() + .onMalformedInput(CodingErrorAction.REPLACE) + .onUnmappableCharacter(CodingErrorAction.REPLACE) : null; + final CharBuffer charBuffer = codec == null ? CharBuffer.allocate(readBufferSize) : null; + final ByteBuffer decoderCarryover = codec == null ? ByteBuffer.allocate(8) : null; + + while (!Thread.currentThread().isInterrupted()) { + final long elapsed = System.currentTimeMillis() - loopStart; + if (elapsed >= timeoutMillis) { + if (isDraining) { + long currentFileSize = 0; + try { + currentFileSize = channel.size(); + } catch (final IOException ignored) { + } + if (readOffset.get() < currentFileSize) { + LOG.warn("Rotation drain timeout expired with unread data for {}. Potential data loss.", path); + metrics.getDataLossEvents().increment(); + } + } else { + LOG.debug("Max read time reached for file {}", path); + } + break; + } + + byteBuffer.clear(); + if (decoderCarryover != null && decoderCarryover.position() > 0) { + decoderCarryover.flip(); + byteBuffer.put(decoderCarryover); + decoderCarryover.clear(); + } + final int bytesRead = channel.read(byteBuffer); + if (bytesRead == -1) { + break; + } + if (bytesRead <= 0 && byteBuffer.position() == 0) { + break; + } + + final int totalBytes = byteBuffer.position(); + metrics.getBytesRead().increment(Math.max(0, bytesRead)); + byteBuffer.flip(); + + if (codec != null) { + final byte[] bytes = new byte[byteBuffer.remaining()]; + byteBuffer.get(bytes); + codecAccumulator.write(bytes); + codecBytesAccumulated += bytes.length; + } else { + charBuffer.clear(); + final CoderResult result = decoder.decode(byteBuffer, charBuffer, false); + if (result.isUnderflow() && byteBuffer.hasRemaining()) { + decoderCarryover.put(byteBuffer); + } + charBuffer.flip(); + if (charBuffer.hasRemaining()) { + processChunk(charBuffer.toString()); + } + } + + readOffset.addAndGet(Math.max(0, bytesRead)); + lastActivityMillis = System.currentTimeMillis(); + + if (acknowledgementsEnabled && currentAckSet != null && currentBatchCount > 0) { + final long batchAge = System.currentTimeMillis() - batchOpenedAtMillis; + if (batchAge >= batchTimeout.toMillis()) { + completePendingAckSet(); + } + } + } + + if (codec != null && codecAccumulator.size() > 0) { + if (parseWithCodec(codecAccumulator.toByteArray())) { + checkpointEntry.setReadOffset(readOffset.get()); + } + } else if (codec == null) { + final int carryoverBytes = decoderCarryover != null ? decoderCarryover.position() : 0; + checkpointEntry.setReadOffset(readOffset.get() - carryoverBytes); + } + } + + private void updateFileLagBytes() { + try { + final long currentFileSize = fileOps.size(path); + final long lag = currentFileSize - readOffset.get(); + metrics.getFileLagBytes().set(Math.max(0, lag)); + } catch (final IOException e) { + LOG.debug("Unable to determine file size for lag calculation on {}", path); + } + } + + private void processChunk(final String chunk) { + int start = 0; + for (int i = 0; i < chunk.length(); i++) { + if (chunk.charAt(i) == '\n') { + if (skippingToNewline) { + skippingToNewline = false; + start = i + 1; + continue; + } + final String segment = chunk.substring(start, i); + partialLine.append(segment); + final String line = partialLine.length() > maxLineLength + ? partialLine.substring(0, maxLineLength) + : partialLine.toString(); + if (partialLine.length() > maxLineLength) { + metrics.getLinesTruncated().increment(); + } + emitLine(line); + partialLine.setLength(0); + start = i + 1; + } + } + if (!skippingToNewline && start < chunk.length()) { + partialLine.append(chunk, start, chunk.length()); + } + + if (partialLine.length() > maxLineLength) { + emitLine(partialLine.substring(0, maxLineLength)); + partialLine.setLength(0); + skippingToNewline = true; + metrics.getLinesTruncated().increment(); + } + } + + private void flushPartialLine() { + if (partialLine.length() > 0) { + emitLine(partialLine.toString()); + partialLine.setLength(0); + } + } + + private boolean parseWithCodec(final byte[] bytes) { + try { + codec.parse(new ByteArrayInputStream(bytes), record -> { + emitCodecRecord(record); + }); + return true; + } catch (final IOException e) { + LOG.error("Codec parse error for file {}", path, e); + metrics.getReadErrors().increment(); + return false; + } + } + + @SuppressWarnings("unchecked") + private void emitCodecRecord(final Record record) { + final Record objectRecord = (Record) (Record) record; + final Event event = record.getData(); + writeRecordWithRetry(objectRecord, event); + } + + private void emitLine(final String line) { + final Map data = new HashMap<>(); + data.put(MESSAGE_KEY, line); + if (includeFileMetadata) { + data.put(FILE_PATH_KEY, cachedAbsolutePath); + data.put(FILE_IDENTITY_KEY, cachedFileIdentityString); + } + + final Event event = eventFactory.eventBuilder(EventBuilder.class) + .withEventType(EVENT_TYPE) + .withData(data) + .build(); + + final Record record = new Record<>(event); + writeRecordWithRetry(record, event); + } + + private void writeRecordWithRetry(final Record record, final Event event) { + boolean written = false; + long backpressureStartNanos = 0; + boolean backpressureActive = false; + final long maxRetryMillis = maxReadTimePerFile.toMillis(); + final long retryStart = System.currentTimeMillis(); + while (!written && !Thread.currentThread().isInterrupted()) { + if (System.currentTimeMillis() - retryStart > maxRetryMillis) { + LOG.warn("Backpressure retry timeout exceeded for file {}. Event may be lost.", path); + metrics.getDataLossEvents().increment(); + break; + } + try { + buffer.write(record, writeTimeout); + written = true; + metrics.getLinesRead().increment(); + metrics.getEventsEmitted().increment(); + + if (backpressureActive) { + final long backpressureElapsedNanos = System.nanoTime() - backpressureStartNanos; + metrics.getBackpressureTimer().record(backpressureElapsedNanos, TimeUnit.NANOSECONDS); + } + + if (acknowledgementsEnabled && acknowledgementSetManager != null) { + ensureAckSet(); + currentAckSet.add(event); + currentBatchCount++; + if (currentBatchCount >= batchSize) { + completePendingAckSet(); + } + } + } catch (final TimeoutException e) { + if (!backpressureActive) { + backpressureStartNanos = System.nanoTime(); + backpressureActive = true; + } + metrics.getWriteTimeouts().increment(); + LOG.debug("Back pressure from buffer, retrying for file {}", path); + try { + Thread.sleep(BACK_PRESSURE_SLEEP_MILLIS); + } catch (final InterruptedException ie) { + Thread.currentThread().interrupt(); + return; + } + } + } + } + + private void ensureAckSet() { + if (currentAckSet == null) { + final long capturedBatchStart = readOffset.get(); + batchStartOffset = capturedBatchStart; + batchOpenedAtMillis = System.currentTimeMillis(); + currentAckSet = acknowledgementSetManager.create( + result -> handleAcknowledgement(result, capturedBatchStart, currentBatchEndOffset), + acknowledgmentTimeout); + } + } + + private void completePendingAckSet() { + if (currentAckSet != null) { + currentBatchEndOffset = readOffset.get(); + currentAckSet.complete(); + currentAckSet = null; + currentBatchCount = 0; + } + } + + private void handleAcknowledgement(final boolean result, final long batchStart, final long batchEnd) { + if (result) { + acknowledgmentRetryCount.set(0); + checkpointEntry.setCommittedOffset(batchEnd); + LOG.debug("Positive acknowledgement for file {} offset range [{}, {}]", path, batchStart, batchEnd); + } else { + final int retryCount = acknowledgmentRetryCount.incrementAndGet(); + metrics.getAcknowledgmentFailures().increment(); + if (retryCount > maxAcknowledgmentRetries) { + LOG.error("Exceeded max acknowledgment retries ({}) for file {} offset range [{}, {}]. Advancing offset to avoid infinite retry.", + maxAcknowledgmentRetries, path, batchStart, batchEnd); + checkpointEntry.setCommittedOffset(batchEnd); + acknowledgmentRetryCount.set(0); + } else { + LOG.warn("Negative acknowledgement for file {} offset range [{}, {}]. Retry {}/{}.", + path, batchStart, batchEnd, retryCount, maxAcknowledgmentRetries); + } + } + } + + public FileIdentity getFileIdentity() { + return fileIdentity; + } + + public Path getPath() { + return path; + } + + public long getReadOffset() { + return readOffset.get(); + } + + public RotationType getLastRotationType() { + return lastRotationType; + } + + public long getLastActivityMillis() { + return lastActivityMillis; + } +} diff --git a/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/TailFileReaderContext.java b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/TailFileReaderContext.java new file mode 100644 index 0000000000..4082793752 --- /dev/null +++ b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/TailFileReaderContext.java @@ -0,0 +1,167 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + +package org.opensearch.dataprepper.plugins.source.file; + +import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSetManager; +import org.opensearch.dataprepper.model.buffer.Buffer; +import org.opensearch.dataprepper.model.codec.InputCodec; +import org.opensearch.dataprepper.model.event.EventFactory; +import org.opensearch.dataprepper.model.record.Record; + +import java.nio.charset.Charset; +import java.time.Duration; +import java.util.Objects; + +public final class TailFileReaderContext { + + private final Buffer> buffer; + private final EventFactory eventFactory; + private final FileSystemOperations fileOps; + private final FileTailMetrics metrics; + private final RotationDetector rotationDetector; + private final AcknowledgementSetManager acknowledgementSetManager; + private final boolean acknowledgementsEnabled; + private final Charset encoding; + private final int readBufferSize; + private final int maxLineLength; + private final int writeTimeout; + private final Duration maxReadTimePerFile; + private final Duration rotationDrainTimeout; + private final StartPosition startPosition; + private final boolean includeFileMetadata; + private final Duration acknowledgmentTimeout; + private final int batchSize; + private final Duration batchTimeout; + private final int maxAcknowledgmentRetries; + private final InputCodec codec; + + public TailFileReaderContext(final Buffer> buffer, + final EventFactory eventFactory, + final FileSystemOperations fileOps, + final FileTailMetrics metrics, + final RotationDetector rotationDetector, + final AcknowledgementSetManager acknowledgementSetManager, + final boolean acknowledgementsEnabled, + final Charset encoding, + final int readBufferSize, + final int maxLineLength, + final int writeTimeout, + final Duration maxReadTimePerFile, + final Duration rotationDrainTimeout, + final StartPosition startPosition, + final boolean includeFileMetadata, + final Duration acknowledgmentTimeout, + final int batchSize, + final Duration batchTimeout, + final int maxAcknowledgmentRetries, + final InputCodec codec) { + this.buffer = Objects.requireNonNull(buffer, "buffer must not be null"); + this.eventFactory = Objects.requireNonNull(eventFactory, "eventFactory must not be null"); + this.fileOps = Objects.requireNonNull(fileOps, "fileOps must not be null"); + this.metrics = Objects.requireNonNull(metrics, "metrics must not be null"); + this.rotationDetector = Objects.requireNonNull(rotationDetector, "rotationDetector must not be null"); + this.acknowledgementSetManager = acknowledgementSetManager; + this.acknowledgementsEnabled = acknowledgementsEnabled; + this.encoding = Objects.requireNonNull(encoding, "encoding must not be null"); + this.readBufferSize = readBufferSize; + this.maxLineLength = maxLineLength; + this.writeTimeout = writeTimeout; + this.maxReadTimePerFile = Objects.requireNonNull(maxReadTimePerFile, "maxReadTimePerFile must not be null"); + this.rotationDrainTimeout = Objects.requireNonNull(rotationDrainTimeout, "rotationDrainTimeout must not be null"); + this.startPosition = Objects.requireNonNull(startPosition, "startPosition must not be null"); + this.includeFileMetadata = includeFileMetadata; + this.acknowledgmentTimeout = Objects.requireNonNull(acknowledgmentTimeout, "acknowledgmentTimeout must not be null"); + this.batchSize = batchSize; + this.batchTimeout = Objects.requireNonNull(batchTimeout, "batchTimeout must not be null"); + this.maxAcknowledgmentRetries = maxAcknowledgmentRetries; + this.codec = codec; + } + + public Buffer> getBuffer() { + return buffer; + } + + public EventFactory getEventFactory() { + return eventFactory; + } + + public FileSystemOperations getFileOps() { + return fileOps; + } + + public FileTailMetrics getMetrics() { + return metrics; + } + + public RotationDetector getRotationDetector() { + return rotationDetector; + } + + public AcknowledgementSetManager getAcknowledgementSetManager() { + return acknowledgementSetManager; + } + + public boolean isAcknowledgementsEnabled() { + return acknowledgementsEnabled; + } + + public Charset getEncoding() { + return encoding; + } + + public int getReadBufferSize() { + return readBufferSize; + } + + public int getMaxLineLength() { + return maxLineLength; + } + + public int getWriteTimeout() { + return writeTimeout; + } + + public Duration getMaxReadTimePerFile() { + return maxReadTimePerFile; + } + + public Duration getRotationDrainTimeout() { + return rotationDrainTimeout; + } + + public StartPosition getStartPosition() { + return startPosition; + } + + public boolean isIncludeFileMetadata() { + return includeFileMetadata; + } + + public Duration getAcknowledgmentTimeout() { + return acknowledgmentTimeout; + } + + public int getBatchSize() { + return batchSize; + } + + public Duration getBatchTimeout() { + return batchTimeout; + } + + public int getMaxAcknowledgmentRetries() { + return maxAcknowledgmentRetries; + } + + public InputCodec getCodec() { + return codec; + } +} diff --git a/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/TailFileReaderPool.java b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/TailFileReaderPool.java new file mode 100644 index 0000000000..66524af712 --- /dev/null +++ b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/TailFileReaderPool.java @@ -0,0 +1,212 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + +package org.opensearch.dataprepper.plugins.source.file; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.nio.file.Path; +import java.time.Duration; +import java.util.Objects; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.RejectedExecutionException; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.function.Supplier; + +public final class TailFileReaderPool { + + private static final Logger LOG = LoggerFactory.getLogger(TailFileReaderPool.class); + private static final long SHUTDOWN_TIMEOUT_SECONDS = 30; + private static final long RE_POLL_DELAY_MILLIS = 500; + + private final ConcurrentHashMap activeReaders; + private final Set pendingIdentities; + private final ConcurrentLinkedQueue pendingQueue; + private final ExecutorService executorService; + private final ScheduledExecutorService scheduler; + private final CheckpointRegistry checkpointRegistry; + private final FileTailMetrics metrics; + private final int maxActiveFiles; + private final Duration closeInactive; + private final TailFileReaderContext readerContext; + + public TailFileReaderPool(final CheckpointRegistry checkpointRegistry, + final FileTailMetrics metrics, + final int maxActiveFiles, + final int readerThreads, + final Duration closeInactive, + final TailFileReaderContext readerContext) { + this(checkpointRegistry, metrics, maxActiveFiles, closeInactive, readerContext, + () -> Executors.newFixedThreadPool(readerThreads, r -> { + final Thread thread = new Thread(r, "tail-reader"); + thread.setDaemon(true); + return thread; + })); + } + + TailFileReaderPool(final CheckpointRegistry checkpointRegistry, + final FileTailMetrics metrics, + final int maxActiveFiles, + final Duration closeInactive, + final TailFileReaderContext readerContext, + final Supplier executorServiceSupplier) { + this.checkpointRegistry = Objects.requireNonNull(checkpointRegistry, "checkpointRegistry must not be null"); + this.metrics = Objects.requireNonNull(metrics, "metrics must not be null"); + this.maxActiveFiles = maxActiveFiles; + this.closeInactive = Objects.requireNonNull(closeInactive, "closeInactive must not be null"); + this.readerContext = Objects.requireNonNull(readerContext, "readerContext must not be null"); + this.activeReaders = new ConcurrentHashMap<>(); + this.pendingIdentities = ConcurrentHashMap.newKeySet(); + this.pendingQueue = new ConcurrentLinkedQueue<>(); + this.executorService = executorServiceSupplier.get(); + this.scheduler = Executors.newSingleThreadScheduledExecutor(r -> { + final Thread thread = new Thread(r, "tail-reader-scheduler"); + thread.setDaemon(true); + return thread; + }); + } + + public synchronized void addFile(final FileIdentity fileIdentity, final Path path) { + if (activeReaders.containsKey(fileIdentity) || pendingIdentities.contains(fileIdentity)) { + return; + } + + if (activeReaders.size() < maxActiveFiles) { + submitReader(fileIdentity, path); + } else { + pendingIdentities.add(fileIdentity); + pendingQueue.add(new PendingFile(fileIdentity, path)); + LOG.debug("File queued as pending: {}", path); + } + } + + private synchronized void submitReader(final FileIdentity fileIdentity, final Path path) { + if (executorService.isShutdown() || activeReaders.containsKey(fileIdentity)) { + return; + } + final CheckpointEntry checkpoint = checkpointRegistry.getOrCreate(fileIdentity.toString()); + final TailFileReader reader = new TailFileReader( + path, fileIdentity, checkpoint, readerContext, + () -> onReaderComplete(fileIdentity, path)); + activeReaders.put(fileIdentity, reader); + metrics.getActiveFileCount().incrementAndGet(); + try { + executorService.submit(reader); + } catch (final RejectedExecutionException e) { + activeReaders.remove(fileIdentity); + metrics.getActiveFileCount().decrementAndGet(); + LOG.debug("Reader submission rejected for {}", path); + } + } + + private synchronized void onReaderComplete(final FileIdentity fileIdentity, final Path path) { + final TailFileReader completedReader = activeReaders.remove(fileIdentity); + if (completedReader == null) { + return; + } + metrics.getActiveFileCount().decrementAndGet(); + + if (completedReader != null && completedReader.getLastRotationType() == RotationType.CREATE_RENAME) { + LOG.info("Re-adding path {} after create/rename rotation", path); + final FileIdentity newIdentity = FileIdentity.from(path, readerContext.getFileOps(), + readerContext.getRotationDetector().getFingerprintBytes()); + submitReader(newIdentity, path); + } else if (completedReader != null && completedReader.getLastRotationType() != RotationType.DELETED) { + final PendingFile next = pendingQueue.poll(); + if (next != null) { + pendingIdentities.remove(next.getFileIdentity()); + submitReader(next.getFileIdentity(), next.getPath()); + pendingQueue.add(new PendingFile(fileIdentity, path)); + pendingIdentities.add(fileIdentity); + } else { + scheduler.schedule(() -> submitReader(fileIdentity, path), RE_POLL_DELAY_MILLIS, TimeUnit.MILLISECONDS); + } + } else { + checkpointRegistry.markCompleted(fileIdentity.toString()); + final PendingFile next = pendingQueue.poll(); + if (next != null) { + pendingIdentities.remove(next.getFileIdentity()); + submitReader(next.getFileIdentity(), next.getPath()); + } + } + } + + public void shutdown() { + scheduler.shutdownNow(); + executorService.shutdown(); + try { + scheduler.awaitTermination(5, TimeUnit.SECONDS); + if (!executorService.awaitTermination(SHUTDOWN_TIMEOUT_SECONDS, TimeUnit.SECONDS)) { + executorService.shutdownNow(); + if (!executorService.awaitTermination(SHUTDOWN_TIMEOUT_SECONDS, TimeUnit.SECONDS)) { + LOG.warn("Reader pool did not terminate within the allowed time"); + } + } + } catch (final InterruptedException e) { + executorService.shutdownNow(); + Thread.currentThread().interrupt(); + } + } + + public int getActiveReaderCount() { + return activeReaders.size(); + } + + public int getPendingCount() { + return pendingQueue.size(); + } + + public synchronized void closeInactiveReaders() { + final long now = System.currentTimeMillis(); + final long inactiveThresholdMillis = closeInactive.toMillis(); + activeReaders.entrySet().removeIf(entry -> { + final TailFileReader reader = entry.getValue(); + if ((now - reader.getLastActivityMillis()) >= inactiveThresholdMillis) { + LOG.info("Closing inactive reader for file identity {}", entry.getKey()); + metrics.getActiveFileCount().decrementAndGet(); + metrics.getFilesClosed().increment(); + return true; + } + return false; + }); + promotePendingFiles(); + } + + public synchronized void closeReaderForPath(final Path path) { + final Path absolutePath = path.toAbsolutePath().normalize(); + activeReaders.entrySet().removeIf(entry -> { + final TailFileReader reader = entry.getValue(); + if (reader.getPath().toAbsolutePath().normalize().equals(absolutePath)) { + LOG.info("Closing reader for removed file: {}", path); + metrics.getActiveFileCount().decrementAndGet(); + metrics.getFilesClosed().increment(); + return true; + } + return false; + }); + promotePendingFiles(); + } + + private void promotePendingFiles() { + while (activeReaders.size() < maxActiveFiles && !pendingQueue.isEmpty()) { + final PendingFile next = pendingQueue.poll(); + if (next != null) { + pendingIdentities.remove(next.getFileIdentity()); + submitReader(next.getFileIdentity(), next.getPath()); + } + } + } +} diff --git a/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/CheckpointEntryTest.java b/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/CheckpointEntryTest.java new file mode 100644 index 0000000000..a574cd150c --- /dev/null +++ b/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/CheckpointEntryTest.java @@ -0,0 +1,141 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + +package org.opensearch.dataprepper.plugins.source.file; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.greaterThanOrEqualTo; +import static org.hamcrest.Matchers.lessThanOrEqualTo; +import static org.hamcrest.Matchers.notNullValue; + +class CheckpointEntryTest { + + private CheckpointEntry checkpointEntry; + + @BeforeEach + void setUp() { + checkpointEntry = new CheckpointEntry(); + } + + @Test + void defaultConstructorSetsZeroOffsets() { + assertThat(checkpointEntry.getReadOffset(), equalTo(0L)); + assertThat(checkpointEntry.getCommittedOffset(), equalTo(0L)); + } + + @Test + void defaultConstructorSetsActiveStatus() { + assertThat(checkpointEntry.getStatus(), equalTo(CheckpointStatus.ACTIVE)); + } + + @Test + void defaultConstructorSetsLastUpdatedMillis() { + final long before = System.currentTimeMillis(); + final CheckpointEntry entry = new CheckpointEntry(); + final long after = System.currentTimeMillis(); + + assertThat(entry.getLastUpdatedMillis(), greaterThanOrEqualTo(before)); + assertThat(entry.getLastUpdatedMillis(), lessThanOrEqualTo(after)); + } + + @Test + void parameterizedConstructorSetsAllFields() { + final CheckpointEntry entry = new CheckpointEntry(100L, 50L, CheckpointStatus.COMPLETED); + + assertThat(entry.getReadOffset(), equalTo(100L)); + assertThat(entry.getCommittedOffset(), equalTo(50L)); + assertThat(entry.getStatus(), equalTo(CheckpointStatus.COMPLETED)); + } + + @Test + void setReadOffsetUpdatesValueAndTimestamp() { + final long beforeUpdate = System.currentTimeMillis(); + checkpointEntry.setReadOffset(500L); + final long afterUpdate = System.currentTimeMillis(); + + assertThat(checkpointEntry.getReadOffset(), equalTo(500L)); + assertThat(checkpointEntry.getLastUpdatedMillis(), greaterThanOrEqualTo(beforeUpdate)); + assertThat(checkpointEntry.getLastUpdatedMillis(), lessThanOrEqualTo(afterUpdate)); + } + + @Test + void setCommittedOffsetUpdatesValueAndTimestamp() { + final long beforeUpdate = System.currentTimeMillis(); + checkpointEntry.setCommittedOffset(300L); + final long afterUpdate = System.currentTimeMillis(); + + assertThat(checkpointEntry.getCommittedOffset(), equalTo(300L)); + assertThat(checkpointEntry.getLastUpdatedMillis(), greaterThanOrEqualTo(beforeUpdate)); + assertThat(checkpointEntry.getLastUpdatedMillis(), lessThanOrEqualTo(afterUpdate)); + } + + @Test + void setStatusUpdatesValueAndTimestamp() { + final long beforeUpdate = System.currentTimeMillis(); + checkpointEntry.setStatus(CheckpointStatus.COMPLETED); + final long afterUpdate = System.currentTimeMillis(); + + assertThat(checkpointEntry.getStatus(), equalTo(CheckpointStatus.COMPLETED)); + assertThat(checkpointEntry.getLastUpdatedMillis(), greaterThanOrEqualTo(beforeUpdate)); + assertThat(checkpointEntry.getLastUpdatedMillis(), lessThanOrEqualTo(afterUpdate)); + } + + @Test + void snapshotReturnsNewInstanceWithSameValues() { + checkpointEntry.setReadOffset(200L); + checkpointEntry.setCommittedOffset(100L); + checkpointEntry.setStatus(CheckpointStatus.COMPLETED); + + final CheckpointEntry snapshot = checkpointEntry.snapshot(); + + assertThat(snapshot, notNullValue()); + assertThat(snapshot.getReadOffset(), equalTo(200L)); + assertThat(snapshot.getCommittedOffset(), equalTo(100L)); + assertThat(snapshot.getStatus(), equalTo(CheckpointStatus.COMPLETED)); + } + + @Test + void snapshotIsIndependentOfOriginal() { + checkpointEntry.setReadOffset(200L); + checkpointEntry.setCommittedOffset(100L); + + final CheckpointEntry snapshot = checkpointEntry.snapshot(); + + checkpointEntry.setReadOffset(999L); + checkpointEntry.setCommittedOffset(888L); + checkpointEntry.setStatus(CheckpointStatus.COMPLETED); + + assertThat(snapshot.getReadOffset(), equalTo(200L)); + assertThat(snapshot.getCommittedOffset(), equalTo(100L)); + assertThat(snapshot.getStatus(), equalTo(CheckpointStatus.ACTIVE)); + } + + @Test + void multipleAdvancesAccumulateCorrectly() { + checkpointEntry.setReadOffset(100L); + checkpointEntry.setReadOffset(200L); + checkpointEntry.setReadOffset(300L); + + assertThat(checkpointEntry.getReadOffset(), equalTo(300L)); + } + + @Test + void statusTransitionFromActiveToCompleted() { + assertThat(checkpointEntry.getStatus(), equalTo(CheckpointStatus.ACTIVE)); + + checkpointEntry.setStatus(CheckpointStatus.COMPLETED); + + assertThat(checkpointEntry.getStatus(), equalTo(CheckpointStatus.COMPLETED)); + } +} diff --git a/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/CheckpointRegistryTest.java b/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/CheckpointRegistryTest.java new file mode 100644 index 0000000000..6663847049 --- /dev/null +++ b/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/CheckpointRegistryTest.java @@ -0,0 +1,338 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + +package org.opensearch.dataprepper.plugins.source.file; + +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; + +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.time.Duration; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.notNullValue; +import static org.hamcrest.Matchers.nullValue; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyLong; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +class CheckpointRegistryTest { + + private static final Duration FLUSH_INTERVAL = Duration.ofHours(1); + private static final Duration CLEANUP_AFTER = Duration.ofHours(24); + + @TempDir + Path tempDir; + + private Path checkpointFile; + private CheckpointRegistry registry; + + @BeforeEach + void setUp() { + checkpointFile = tempDir.resolve("checkpoints.json"); + } + + @AfterEach + void tearDown() { + if (registry != null) { + registry.shutdown(); + } + } + + private CheckpointRegistry createRegistry() { + return createRegistry(checkpointFile); + } + + private CheckpointRegistry createRegistry(final Path file) { + return new CheckpointRegistry(file, FLUSH_INTERVAL, CLEANUP_AFTER); + } + + @Test + void getOrCreateReturnsNewEntryForUnknownKey() { + registry = createRegistry(); + + final CheckpointEntry entry = registry.getOrCreate("test-key"); + + assertThat(entry, notNullValue()); + assertThat(entry.getReadOffset(), equalTo(0L)); + assertThat(entry.getCommittedOffset(), equalTo(0L)); + assertThat(entry.getStatus(), equalTo(CheckpointStatus.ACTIVE)); + } + + @Test + void getOrCreateReturnsSameEntryForSameKey() { + registry = createRegistry(); + + final CheckpointEntry first = registry.getOrCreate("test-key"); + first.setReadOffset(42L); + + final CheckpointEntry second = registry.getOrCreate("test-key"); + + assertThat(second.getReadOffset(), equalTo(42L)); + } + + @Test + void getReturnsNullForUnknownKey() { + registry = createRegistry(); + + assertThat(registry.get("nonexistent"), nullValue()); + } + + @Test + void getReturnsEntryAfterGetOrCreate() { + registry = createRegistry(); + + registry.getOrCreate("my-key").setReadOffset(77L); + + final CheckpointEntry retrieved = registry.get("my-key"); + assertThat(retrieved, notNullValue()); + assertThat(retrieved.getReadOffset(), equalTo(77L)); + } + + @Test + void flushAndLoadRoundTripPreservesEntries() { + registry = createRegistry(); + final CheckpointEntry entry = registry.getOrCreate("/var/log/app.log"); + entry.setReadOffset(1024L); + entry.setCommittedOffset(512L); + entry.setStatus(CheckpointStatus.ACTIVE); + + registry.flush(); + registry.shutdown(); + + final CheckpointRegistry reloaded = createRegistry(); + registry = reloaded; + + final CheckpointEntry loaded = reloaded.get("/var/log/app.log"); + assertThat(loaded, notNullValue()); + assertThat(loaded.getReadOffset(), equalTo(1024L)); + assertThat(loaded.getCommittedOffset(), equalTo(512L)); + assertThat(loaded.getStatus(), equalTo(CheckpointStatus.ACTIVE)); + } + + @Test + void flushCreatesCheckpointFileOnDisk() { + registry = createRegistry(); + registry.getOrCreate("some-file"); + + registry.flush(); + + assertThat(Files.exists(checkpointFile), equalTo(true)); + } + + @Test + void flushUsesAtomicWriteWithTempFile() { + registry = createRegistry(); + registry.getOrCreate("file1"); + + registry.flush(); + + final Path tempFile = checkpointFile.resolveSibling(checkpointFile.getFileName() + ".tmp"); + assertThat(Files.exists(tempFile), equalTo(false)); + assertThat(Files.exists(checkpointFile), equalTo(true)); + } + + @Test + void corruptCheckpointFileStartsWithEmptyState() throws IOException { + Files.writeString(checkpointFile, "THIS IS NOT VALID JSON{{{"); + + registry = createRegistry(); + + assertThat(registry.get("any-key"), nullValue()); + } + + @Test + void emptyCheckpointFileStartsWithEmptyState() throws IOException { + Files.writeString(checkpointFile, ""); + + registry = createRegistry(); + + assertThat(registry.get("any-key"), nullValue()); + } + + @Test + void cleanupRemovesStaleCompletedEntries() throws InterruptedException { + final Duration zeroCleanup = Duration.ZERO; + registry = new CheckpointRegistry(checkpointFile, FLUSH_INTERVAL, zeroCleanup); + + final CheckpointEntry entry = registry.getOrCreate("stale-file"); + entry.setStatus(CheckpointStatus.COMPLETED); + + Thread.sleep(50); + + registry.flush(); + + assertThat(registry.get("stale-file"), nullValue()); + } + + @Test + void cleanupKeepsActiveEntries() { + final Duration zeroCleanup = Duration.ZERO; + registry = new CheckpointRegistry(checkpointFile, FLUSH_INTERVAL, zeroCleanup); + + registry.getOrCreate("active-file").setReadOffset(100L); + + registry.flush(); + + final CheckpointEntry entry = registry.get("active-file"); + assertThat(entry, notNullValue()); + assertThat(entry.getReadOffset(), equalTo(100L)); + } + + @Test + void multipleEntriesPersistCorrectly() { + registry = createRegistry(); + + registry.getOrCreate("file-a").setReadOffset(10L); + registry.getOrCreate("file-b").setReadOffset(20L); + registry.getOrCreate("file-c").setReadOffset(30L); + + registry.flush(); + registry.shutdown(); + + final CheckpointRegistry reloaded = createRegistry(); + registry = reloaded; + + assertThat(reloaded.get("file-a").getReadOffset(), equalTo(10L)); + assertThat(reloaded.get("file-b").getReadOffset(), equalTo(20L)); + assertThat(reloaded.get("file-c").getReadOffset(), equalTo(30L)); + } + + @Test + void shutdownFlushesBeforeTerminating() { + registry = createRegistry(); + registry.getOrCreate("flush-on-shutdown").setReadOffset(999L); + + registry.shutdown(); + + final CheckpointRegistry reloaded = createRegistry(); + registry = reloaded; + + final CheckpointEntry entry = reloaded.get("flush-on-shutdown"); + assertThat(entry, notNullValue()); + assertThat(entry.getReadOffset(), equalTo(999L)); + } + + @Test + void flush_handles_io_error_on_unwritable_path() { + Path unwritablePath = Path.of("/nonexistent-dir-" + System.nanoTime() + "/sub/checkpoints.json"); + registry = createRegistry(unwritablePath); + registry.getOrCreate("some-key").setReadOffset(100L); + + registry.flush(); + } + + @Test + void shutdown_handles_scheduler_interrupted() throws Exception { + registry = createRegistry(); + registry.getOrCreate("interrupt-key").setReadOffset(50L); + + Thread shutdownThread = new Thread(() -> { + Thread.currentThread().interrupt(); + registry.shutdown(); + }); + shutdownThread.start(); + shutdownThread.join(5000); + + assertThat(shutdownThread.isAlive(), equalTo(false)); + registry = null; + } + + @Test + void load_handles_null_checkpoint_file() { + CheckpointRegistry nullRegistry = new CheckpointRegistry(null, FLUSH_INTERVAL, CLEANUP_AFTER); + + assertThat(nullRegistry.get("any"), nullValue()); + nullRegistry.getOrCreate("test").setReadOffset(10L); + assertThat(nullRegistry.get("test").getReadOffset(), equalTo(10L)); + } + + @Test + void shutdown_calls_shutdownNow_when_awaitTermination_returns_false() throws Exception { + ScheduledExecutorService mockScheduler = mock(ScheduledExecutorService.class); + when(mockScheduler.awaitTermination(anyLong(), any(TimeUnit.class))).thenReturn(false); + + registry = new CheckpointRegistry(checkpointFile, FLUSH_INTERVAL, CLEANUP_AFTER, () -> mockScheduler); + + registry.shutdown(); + registry = null; + } + + @Test + void shutdown_calls_shutdownNow_when_awaitTermination_throws_interrupted() throws Exception { + ScheduledExecutorService mockScheduler = mock(ScheduledExecutorService.class); + when(mockScheduler.awaitTermination(anyLong(), any(TimeUnit.class))).thenThrow(new InterruptedException("test")); + + registry = new CheckpointRegistry(checkpointFile, FLUSH_INTERVAL, CLEANUP_AFTER, () -> mockScheduler); + + registry.shutdown(); + registry = null; + + assertThat(Thread.currentThread().isInterrupted(), equalTo(true)); + Thread.interrupted(); + } + + @Test + void load_with_file_having_no_parent_directory(@TempDir final Path altDir) { + final Path noParentFile = altDir.resolve("checkpoint-no-parent.json"); + registry = new CheckpointRegistry(noParentFile, FLUSH_INTERVAL, CLEANUP_AFTER); + assertThat(registry.get("any"), nullValue()); + } + + @Test + void load_handles_null_map_from_json() throws IOException { + Files.writeString(checkpointFile, "null"); + registry = createRegistry(); + assertThat(registry.get("any"), nullValue()); + } + + @Test + void cleanup_does_not_remove_recently_completed_entry() { + final Duration longCleanup = Duration.ofHours(48); + registry = new CheckpointRegistry(checkpointFile, FLUSH_INTERVAL, longCleanup); + + final CheckpointEntry entry = registry.getOrCreate("recent-completed"); + entry.setStatus(CheckpointStatus.COMPLETED); + + registry.flush(); + + assertThat(registry.get("recent-completed"), notNullValue()); + } + + @Test + void markCompleted_sets_status_on_existing_entry() { + registry = new CheckpointRegistry(checkpointFile, FLUSH_INTERVAL, CLEANUP_AFTER); + registry.getOrCreate("mark-test"); + registry.markCompleted("mark-test"); + assertThat(registry.get("mark-test").getStatus(), equalTo(CheckpointStatus.COMPLETED)); + } + + @Test + void markCompleted_does_nothing_for_nonexistent_key() { + registry = new CheckpointRegistry(checkpointFile, FLUSH_INTERVAL, CLEANUP_AFTER); + registry.markCompleted("nonexistent"); + assertThat(registry.get("nonexistent"), nullValue()); + } + + @Test + void flush_with_null_checkpoint_file_does_not_throw() { + final CheckpointRegistry nullFileRegistry = new CheckpointRegistry(null, FLUSH_INTERVAL, CLEANUP_AFTER); + nullFileRegistry.flush(); + nullFileRegistry.shutdown(); + } +} diff --git a/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/CheckpointStatusTest.java b/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/CheckpointStatusTest.java new file mode 100644 index 0000000000..64527174f2 --- /dev/null +++ b/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/CheckpointStatusTest.java @@ -0,0 +1,50 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + +package org.opensearch.dataprepper.plugins.source.file; + +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.equalTo; +import static org.junit.jupiter.api.Assertions.assertThrows; + +class CheckpointStatusTest { + + @Test + void fromString_returns_active() { + assertThat(CheckpointStatus.fromString("ACTIVE"), equalTo(CheckpointStatus.ACTIVE)); + } + + @Test + void fromString_returns_completed() { + assertThat(CheckpointStatus.fromString("COMPLETED"), equalTo(CheckpointStatus.COMPLETED)); + } + + @Test + void fromString_is_case_insensitive() { + assertThat(CheckpointStatus.fromString("active"), equalTo(CheckpointStatus.ACTIVE)); + assertThat(CheckpointStatus.fromString("completed"), equalTo(CheckpointStatus.COMPLETED)); + } + + @ParameterizedTest + @ValueSource(strings = {"invalid", "pending", ""}) + void fromString_throws_for_invalid_value(final String value) { + assertThrows(IllegalArgumentException.class, () -> CheckpointStatus.fromString(value)); + } + + @Test + void getValue_returns_value() { + assertThat(CheckpointStatus.ACTIVE.getValue(), equalTo("ACTIVE")); + assertThat(CheckpointStatus.COMPLETED.getValue(), equalTo("COMPLETED")); + } +} diff --git a/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/DefaultFileSystemOperationsTest.java b/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/DefaultFileSystemOperationsTest.java new file mode 100644 index 0000000000..ee27afca00 --- /dev/null +++ b/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/DefaultFileSystemOperationsTest.java @@ -0,0 +1,179 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + +package org.opensearch.dataprepper.plugins.source.file; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; + +import java.io.IOException; +import java.nio.channels.FileChannel; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.attribute.BasicFileAttributes; +import java.util.List; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.greaterThan; +import static org.hamcrest.Matchers.greaterThanOrEqualTo; +import static org.hamcrest.Matchers.hasItem; +import static org.hamcrest.Matchers.notNullValue; +import static org.junit.jupiter.api.Assertions.assertThrows; + +class DefaultFileSystemOperationsTest { + + @TempDir + Path tempDir; + + private DefaultFileSystemOperations fileOps; + + @BeforeEach + void setUp() { + fileOps = new DefaultFileSystemOperations(); + } + + @Test + void openReadChannel_returns_readable_channel() throws IOException { + final Path file = tempDir.resolve("readable.txt"); + Files.writeString(file, "hello world"); + + try (FileChannel channel = fileOps.openReadChannel(file)) { + assertThat(channel, notNullValue()); + assertThat(channel.isOpen(), equalTo(true)); + assertThat(channel.size(), greaterThan(0L)); + } + } + + @Test + void openReadChannel_throws_on_nonexistent_file() { + final Path missing = tempDir.resolve("does-not-exist.txt"); + assertThrows(IOException.class, () -> fileOps.openReadChannel(missing)); + } + + @Test + void readAttributes_returns_valid_attributes() throws IOException { + final Path file = tempDir.resolve("attrs.txt"); + Files.writeString(file, "test content"); + + final BasicFileAttributes attrs = fileOps.readAttributes(file); + + assertThat(attrs, notNullValue()); + assertThat(attrs.isRegularFile(), equalTo(true)); + assertThat(attrs.isDirectory(), equalTo(false)); + assertThat(attrs.size(), greaterThan(0L)); + assertThat(attrs.creationTime(), notNullValue()); + } + + @Test + void readAttributes_throws_on_nonexistent_file() { + final Path missing = tempDir.resolve("no-attrs.txt"); + assertThrows(IOException.class, () -> fileOps.readAttributes(missing)); + } + + @Test + void listDirectory_returns_files_in_directory() throws IOException { + Files.createFile(tempDir.resolve("file1.txt")); + Files.createFile(tempDir.resolve("file2.txt")); + + try (Stream stream = fileOps.listDirectory(tempDir)) { + final List files = stream.collect(Collectors.toList()); + assertThat(files.size(), greaterThanOrEqualTo(2)); + assertThat(files, hasItem(tempDir.resolve("file1.txt"))); + assertThat(files, hasItem(tempDir.resolve("file2.txt"))); + } + } + + @Test + void listDirectory_throws_on_nonexistent_directory() { + final Path missing = tempDir.resolve("no-dir"); + assertThrows(IOException.class, () -> fileOps.listDirectory(missing)); + } + + @Test + void exists_returns_true_for_existing_file() throws IOException { + final Path file = tempDir.resolve("exists.txt"); + Files.writeString(file, "data"); + + assertThat(fileOps.exists(file), equalTo(true)); + } + + @Test + void exists_returns_false_for_nonexistent_file() { + final Path missing = tempDir.resolve("missing.txt"); + assertThat(fileOps.exists(missing), equalTo(false)); + } + + @Test + void size_returns_correct_file_size() throws IOException { + final Path file = tempDir.resolve("sized.txt"); + final String content = "twelve chars"; + Files.writeString(file, content); + + assertThat(fileOps.size(file), equalTo((long) content.getBytes().length)); + } + + @Test + void size_returns_zero_for_empty_file() throws IOException { + final Path file = tempDir.resolve("empty.txt"); + Files.createFile(file); + + assertThat(fileOps.size(file), equalTo(0L)); + } + + @Test + void size_throws_on_nonexistent_file() { + final Path missing = tempDir.resolve("no-size.txt"); + assertThrows(IOException.class, () -> fileOps.size(missing)); + } + + @Test + void readBytes_reads_exact_number_of_bytes() throws IOException { + final Path file = tempDir.resolve("bytes.txt"); + Files.writeString(file, "abcdefghij"); + + final byte[] bytes = fileOps.readBytes(file, 5); + + assertThat(bytes, notNullValue()); + assertThat(bytes.length, equalTo(5)); + assertThat(new String(bytes), equalTo("abcde")); + } + + @Test + void readBytes_reads_entire_file_when_length_exceeds_size() throws IOException { + final Path file = tempDir.resolve("short.txt"); + Files.writeString(file, "abc"); + + final byte[] bytes = fileOps.readBytes(file, 100); + + assertThat(bytes.length, equalTo(3)); + assertThat(new String(bytes), equalTo("abc")); + } + + @Test + void readBytes_returns_empty_array_for_empty_file() throws IOException { + final Path file = tempDir.resolve("empty-bytes.txt"); + Files.createFile(file); + + final byte[] bytes = fileOps.readBytes(file, 10); + + assertThat(bytes, notNullValue()); + assertThat(bytes.length, equalTo(0)); + } + + @Test + void readBytes_throws_on_nonexistent_file() { + final Path missing = tempDir.resolve("no-bytes.txt"); + assertThrows(IOException.class, () -> fileOps.readBytes(missing, 10)); + } +} diff --git a/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/DirectoryWatcherTest.java b/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/DirectoryWatcherTest.java new file mode 100644 index 0000000000..e26d413873 --- /dev/null +++ b/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/DirectoryWatcherTest.java @@ -0,0 +1,1087 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + +package org.opensearch.dataprepper.plugins.source.file; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.io.TempDir; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; + +import java.io.IOException; +import java.nio.file.ClosedWatchServiceException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.StandardWatchEventKinds; +import java.nio.file.FileSystems; +import java.nio.file.WatchEvent; +import java.nio.file.WatchKey; +import java.nio.file.WatchService; +import java.time.Duration; +import java.util.Collections; +import java.util.List; +import java.util.Set; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.RejectedExecutionException; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledFuture; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.locks.LockSupport; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.equalTo; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyLong; +import static org.mockito.Mockito.atLeastOnce; +import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.lenient; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +@ExtendWith(MockitoExtension.class) +class DirectoryWatcherTest { + + @TempDir + Path tempDir; + + @Mock + private TailFileReaderPool readerPool; + + @Mock + private CheckpointRegistry checkpointRegistry; + + @Mock + private FileSourceConfig config; + + @Mock + private FileTailMetrics metrics; + + private FileSystemOperations realFileOps; + private GlobPathResolver globPathResolver; + + @BeforeEach + void setUp() throws IOException { + realFileOps = new DefaultFileSystemOperations(); + } + + private DirectoryWatcher createWatcher() { + globPathResolver = new GlobPathResolver( + List.of(tempDir.toString() + "/*.log"), + Collections.emptyList()); + return new DirectoryWatcher(globPathResolver, readerPool, checkpointRegistry, config, realFileOps, metrics, + Duration.ofSeconds(5), true); + } + + @Test + void start_discovers_existing_files_and_adds_to_pool() throws IOException { + Files.writeString(tempDir.resolve("initial.log"), "content"); + when(config.getFingerprintBytes()).thenReturn(1024); + when(config.getPollInterval()).thenReturn(Duration.ofSeconds(60)); + + final DirectoryWatcher watcher = createWatcher(); + try { + watcher.start(); + verify(readerPool, atLeastOnce()).addFile(any(FileIdentity.class), any(Path.class)); + } finally { + watcher.stop(); + } + } + + @Test + void start_then_pollScan_detects_new_files() throws IOException { + Files.writeString(tempDir.resolve("initial.log"), "content"); + when(config.getFingerprintBytes()).thenReturn(1024); + when(config.getPollInterval()).thenReturn(Duration.ofSeconds(60)); + + final DirectoryWatcher watcher = createWatcher(); + try { + watcher.start(); + + Files.writeString(tempDir.resolve("new-file.log"), "new content"); + watcher.pollScan(); + + verify(readerPool, atLeastOnce()).addFile(any(FileIdentity.class), any(Path.class)); + } finally { + watcher.stop(); + } + } + + @Test + void pollScan_does_nothing_when_not_running() { + final DirectoryWatcher watcher = createWatcher(); + watcher.stop(); + watcher.pollScan(); + } + + @Test + void stop_completes_without_error_before_start() { + final DirectoryWatcher watcher = createWatcher(); + watcher.stop(); + } + + @Test + void stop_completes_without_error_after_start() throws IOException { + Files.writeString(tempDir.resolve("initial.log"), "content"); + when(config.getFingerprintBytes()).thenReturn(1024); + when(config.getPollInterval()).thenReturn(Duration.ofSeconds(60)); + + final DirectoryWatcher watcher = createWatcher(); + watcher.start(); + watcher.stop(); + } + + @Test + void isNetworkFilesystem_returns_false_for_local_directory() { + assertThat(DirectoryWatcher.isNetworkFilesystem(tempDir), equalTo(false)); + } + + @Test + void isNetworkFilesystem_returns_false_on_IOException() { + Path nonexistent = Path.of("/nonexistent-path-" + System.nanoTime()); + assertThat(DirectoryWatcher.isNetworkFilesystem(nonexistent), equalTo(false)); + } + + @Test + void pollScan_detects_vanished_files_and_closes_when_close_removed_true() throws Exception { + Files.writeString(tempDir.resolve("vanish.log"), "content"); + when(config.getFingerprintBytes()).thenReturn(1024); + when(config.getPollInterval()).thenReturn(Duration.ofSeconds(60)); + + final GlobPathResolver resolver = new GlobPathResolver( + List.of(tempDir.resolve("*.log").toString()), Collections.emptyList()); + final DirectoryWatcher watcher = new DirectoryWatcher(resolver, readerPool, checkpointRegistry, config, realFileOps, metrics, + Duration.ofMillis(100), true); + try { + watcher.start(); + + Files.delete(tempDir.resolve("vanish.log")); + watcher.pollScan(); + + Thread.sleep(500); + verify(readerPool, atLeastOnce()).closeReaderForPath(any(Path.class)); + } finally { + watcher.stop(); + } + } + + @Test + void pollScan_handles_runtime_exception_from_glob_resolver() throws IOException { + lenient().when(config.getFingerprintBytes()).thenReturn(1024); + when(config.getPollInterval()).thenReturn(Duration.ofSeconds(60)); + + GlobPathResolver badResolver = mock(GlobPathResolver.class); + when(badResolver.resolve()).thenReturn(Set.of()); + when(badResolver.getWatchDirectories()).thenReturn(Set.of(tempDir)); + + DirectoryWatcher watcher = new DirectoryWatcher(badResolver, readerPool, checkpointRegistry, config, realFileOps, metrics, + Duration.ofSeconds(5), true); + try { + watcher.start(); + + when(badResolver.resolve()).thenThrow(new RuntimeException("glob error")); + watcher.pollScan(); + } finally { + watcher.stop(); + } + } + + @Test + void addFileToPool_handles_runtime_exception() throws IOException { + when(config.getFingerprintBytes()).thenReturn(1024); + when(config.getPollInterval()).thenReturn(Duration.ofSeconds(60)); + + doThrow(new RuntimeException("pool error")).when(readerPool).addFile(any(), any()); + + Files.writeString(tempDir.resolve("error.log"), "content"); + final DirectoryWatcher watcher = createWatcher(); + try { + watcher.start(); + } finally { + watcher.stop(); + } + } + + @Test + void pollScan_does_not_close_when_close_removed_false() throws IOException { + Files.writeString(tempDir.resolve("keep.log"), "content"); + when(config.getFingerprintBytes()).thenReturn(1024); + when(config.getPollInterval()).thenReturn(Duration.ofSeconds(60)); + + globPathResolver = new GlobPathResolver( + List.of(tempDir.toString() + "/*.log"), + Collections.emptyList()); + DirectoryWatcher watcher = new DirectoryWatcher(globPathResolver, readerPool, checkpointRegistry, config, realFileOps, metrics, + Duration.ofSeconds(5), false); + try { + watcher.start(); + + Files.delete(tempDir.resolve("keep.log")); + watcher.pollScan(); + + verify(readerPool, never()).closeReaderForPath(any(Path.class)); + } finally { + watcher.stop(); + } + } + + @Test + void stop_handles_interrupted_exception_on_poll_scheduler() throws Exception { + lenient().when(config.getFingerprintBytes()).thenReturn(1024); + when(config.getPollInterval()).thenReturn(Duration.ofSeconds(60)); + + final DirectoryWatcher watcher = createWatcher(); + watcher.start(); + + Thread stopThread = new Thread(() -> { + Thread.currentThread().interrupt(); + watcher.stop(); + }); + stopThread.start(); + stopThread.join(5000); + } + + @Test + void start_uses_supplementary_poll_interval_with_watch_service() throws IOException { + Files.writeString(tempDir.resolve("initial.log"), "content"); + when(config.getFingerprintBytes()).thenReturn(1024); + when(config.getPollInterval()).thenReturn(Duration.ofSeconds(1)); + + final DirectoryWatcher watcher = createWatcher(); + try { + watcher.start(); + } finally { + watcher.stop(); + } + } + + @Test + void watch_loop_exits_on_closed_watch_service() throws Exception { + Files.writeString(tempDir.resolve("watch.log"), "content"); + when(config.getFingerprintBytes()).thenReturn(1024); + when(config.getPollInterval()).thenReturn(Duration.ofSeconds(60)); + + final DirectoryWatcher watcher = createWatcher(); + watcher.start(); + + Thread.sleep(500); + + watcher.stop(); + } + + @Test + void watch_loop_detects_new_file_created() throws Exception { + Files.writeString(tempDir.resolve("existing.log"), "content"); + when(config.getFingerprintBytes()).thenReturn(1024); + when(config.getPollInterval()).thenReturn(Duration.ofSeconds(60)); + + final DirectoryWatcher watcher = createWatcher(); + try { + watcher.start(); + + Thread.sleep(500); + + Files.writeString(tempDir.resolve("new-detected.log"), "new content"); + + Thread.sleep(2000); + } finally { + watcher.stop(); + } + } + + @Test + void watch_loop_handles_file_deletion_with_close_removed_true() throws Exception { + Files.writeString(tempDir.resolve("delete-me.log"), "content"); + when(config.getFingerprintBytes()).thenReturn(1024); + when(config.getPollInterval()).thenReturn(Duration.ofSeconds(60)); + + globPathResolver = new GlobPathResolver( + List.of(tempDir.toString() + "/*.log"), + Collections.emptyList()); + DirectoryWatcher watcher = new DirectoryWatcher(globPathResolver, readerPool, checkpointRegistry, config, realFileOps, metrics, + Duration.ofMillis(100), true); + try { + watcher.start(); + + Thread.sleep(500); + + Files.delete(tempDir.resolve("delete-me.log")); + + Thread.sleep(2000); + } finally { + watcher.stop(); + } + } + + @Test + void watch_loop_handles_file_deletion_with_close_removed_false() throws Exception { + Files.writeString(tempDir.resolve("keep-me.log"), "content"); + when(config.getFingerprintBytes()).thenReturn(1024); + when(config.getPollInterval()).thenReturn(Duration.ofSeconds(60)); + + globPathResolver = new GlobPathResolver( + List.of(tempDir.toString() + "/*.log"), + Collections.emptyList()); + DirectoryWatcher watcher = new DirectoryWatcher(globPathResolver, readerPool, checkpointRegistry, config, realFileOps, metrics, + Duration.ofMillis(100), false); + try { + watcher.start(); + + Thread.sleep(500); + + Files.delete(tempDir.resolve("keep-me.log")); + + Thread.sleep(2000); + } finally { + watcher.stop(); + } + } + + @Test + void watch_loop_handles_file_reappearing_during_rotate_wait() throws Exception { + Files.writeString(tempDir.resolve("rotate-reappear.log"), "content"); + when(config.getFingerprintBytes()).thenReturn(1024); + when(config.getPollInterval()).thenReturn(Duration.ofSeconds(60)); + + globPathResolver = new GlobPathResolver( + List.of(tempDir.toString() + "/*.log"), + Collections.emptyList()); + DirectoryWatcher watcher = new DirectoryWatcher(globPathResolver, readerPool, checkpointRegistry, config, realFileOps, metrics, + Duration.ofMillis(500), true); + try { + watcher.start(); + + Thread.sleep(500); + + Files.delete(tempDir.resolve("rotate-reappear.log")); + Thread.sleep(100); + Files.writeString(tempDir.resolve("rotate-reappear.log"), "new content"); + + Thread.sleep(2000); + } finally { + watcher.stop(); + } + } + + @Test + void startWatchService_falls_back_on_exception() throws Exception { + when(config.getPollInterval()).thenReturn(Duration.ofSeconds(60)); + lenient().when(config.getFingerprintBytes()).thenReturn(1024); + + GlobPathResolver resolver = mock(GlobPathResolver.class); + when(resolver.resolve()).thenReturn(Set.of()); + when(resolver.getWatchDirectories()).thenReturn(Set.of(tempDir)); + + DirectoryWatcher watcher = new DirectoryWatcher(resolver, readerPool, checkpointRegistry, config, realFileOps, metrics, + Duration.ofSeconds(5), true, + () -> { throw new RuntimeException("cannot create WatchService"); }); + try { + watcher.start(); + Thread.sleep(200); + } finally { + watcher.stop(); + } + } + + @Test + void startWatchService_falls_back_on_io_exception() throws Exception { + when(config.getPollInterval()).thenReturn(Duration.ofSeconds(60)); + lenient().when(config.getFingerprintBytes()).thenReturn(1024); + + GlobPathResolver resolver = mock(GlobPathResolver.class); + when(resolver.resolve()).thenReturn(Set.of()); + when(resolver.getWatchDirectories()).thenReturn(Set.of(tempDir)); + + DirectoryWatcher watcher = new DirectoryWatcher(resolver, readerPool, checkpointRegistry, config, realFileOps, metrics, + Duration.ofSeconds(5), true, + () -> { throw new IOException("cannot create WatchService"); }); + try { + watcher.start(); + Thread.sleep(200); + } finally { + watcher.stop(); + } + } + + @Test + void stop_handles_pollScheduler_not_terminating() throws Exception { + when(config.getPollInterval()).thenReturn(Duration.ofSeconds(60)); + lenient().when(config.getFingerprintBytes()).thenReturn(1024); + + GlobPathResolver resolver = mock(GlobPathResolver.class); + when(resolver.resolve()).thenReturn(Set.of()); + when(resolver.getWatchDirectories()).thenReturn(Set.of(tempDir)); + + DirectoryWatcher watcher = new DirectoryWatcher(resolver, readerPool, checkpointRegistry, config, realFileOps, metrics, + Duration.ofSeconds(5), true); + watcher.start(); + watcher.stop(); + } + + @Test + void shouldUseWatchService_returns_false_for_network_filesystem() throws Exception { + lenient().when(config.getFingerprintBytes()).thenReturn(1024); + when(config.getPollInterval()).thenReturn(Duration.ofSeconds(60)); + + GlobPathResolver resolver = mock(GlobPathResolver.class); + when(resolver.resolve()).thenReturn(Set.of()); + when(resolver.getWatchDirectories()).thenReturn(Set.of(tempDir)); + + DirectoryWatcher watcher = new DirectoryWatcher(resolver, readerPool, checkpointRegistry, config, realFileOps, metrics, + Duration.ofSeconds(5), true); + watcher.start(); + watcher.stop(); + } + + @Test + void stop_handles_IOException_closing_WatchService() throws Exception { + when(config.getPollInterval()).thenReturn(Duration.ofSeconds(60)); + lenient().when(config.getFingerprintBytes()).thenReturn(1024); + + WatchService realWatchService = FileSystems.getDefault().newWatchService(); + + GlobPathResolver resolver = mock(GlobPathResolver.class); + when(resolver.resolve()).thenReturn(Set.of()); + when(resolver.getWatchDirectories()).thenReturn(Set.of(tempDir)); + + DirectoryWatcher watcher = new DirectoryWatcher(resolver, readerPool, checkpointRegistry, config, realFileOps, metrics, + Duration.ofSeconds(5), true, + () -> realWatchService); + watcher.start(); + Thread.sleep(200); + + realWatchService.close(); + Thread.sleep(200); + + watcher.stop(); + } + + @Test + void registerDirectory_handles_IOException() throws Exception { + when(config.getPollInterval()).thenReturn(Duration.ofSeconds(60)); + lenient().when(config.getFingerprintBytes()).thenReturn(1024); + + GlobPathResolver resolver = mock(GlobPathResolver.class); + when(resolver.resolve()).thenReturn(Set.of()); + Path watchDir = tempDir.resolve("unreadable-watch"); + Files.createDirectory(watchDir); + watchDir.toFile().setReadable(false); + watchDir.toFile().setExecutable(false); + when(resolver.getWatchDirectories()).thenReturn(Set.of(watchDir)); + + DirectoryWatcher watcher = new DirectoryWatcher(resolver, readerPool, checkpointRegistry, config, realFileOps, metrics, + Duration.ofSeconds(5), true); + try { + watcher.start(); + Thread.sleep(200); + } finally { + watchDir.toFile().setReadable(true); + watchDir.toFile().setExecutable(true); + watcher.stop(); + } + } + + @Test + void supplementary_poll_interval_when_not_macOS_and_watch_active() throws Exception { + when(config.getPollInterval()).thenReturn(Duration.ofSeconds(1)); + lenient().when(config.getFingerprintBytes()).thenReturn(1024); + + GlobPathResolver resolver = mock(GlobPathResolver.class); + when(resolver.resolve()).thenReturn(Set.of()); + when(resolver.getWatchDirectories()).thenReturn(Set.of(tempDir)); + + DirectoryWatcher watcher = new DirectoryWatcher(resolver, readerPool, checkpointRegistry, config, realFileOps, metrics, + Duration.ofSeconds(5), true, + () -> FileSystems.getDefault().newWatchService(), + false); + try { + watcher.start(); + Thread.sleep(200); + } finally { + watcher.stop(); + } + } + + @Test + void watchLoop_handles_invalid_WatchKey() throws Exception { + when(config.getPollInterval()).thenReturn(Duration.ofSeconds(60)); + lenient().when(config.getFingerprintBytes()).thenReturn(1024); + + WatchKey mockKey = mock(WatchKey.class); + when(mockKey.pollEvents()).thenReturn(Collections.emptyList()); + when(mockKey.reset()).thenReturn(false); + + WatchService mockWatchService = mock(WatchService.class); + when(mockWatchService.take()) + .thenReturn(mockKey) + .thenThrow(new ClosedWatchServiceException()); + + GlobPathResolver resolver = mock(GlobPathResolver.class); + when(resolver.resolve()).thenReturn(Set.of()); + when(resolver.getWatchDirectories()).thenReturn(Set.of(Path.of("/nonexistent-dir-" + System.nanoTime()))); + + DirectoryWatcher watcher = new DirectoryWatcher(resolver, readerPool, checkpointRegistry, config, realFileOps, metrics, + Duration.ofSeconds(5), true, + () -> mockWatchService); + watcher.start(); + Thread.sleep(500); + watcher.stop(); + } + + @Test + void handleWatchEvent_handles_OVERFLOW() throws Exception { + when(config.getPollInterval()).thenReturn(Duration.ofSeconds(60)); + lenient().when(config.getFingerprintBytes()).thenReturn(1024); + + WatchEvent overflowEvent = mock(WatchEvent.class); + when(overflowEvent.kind()).thenReturn((WatchEvent.Kind) StandardWatchEventKinds.OVERFLOW); + + WatchKey mockKey = mock(WatchKey.class); + when(mockKey.pollEvents()).thenReturn(List.of(overflowEvent)); + when(mockKey.reset()).thenReturn(true); + + WatchService mockWatchService = mock(WatchService.class); + when(mockWatchService.take()) + .thenReturn(mockKey) + .thenThrow(new ClosedWatchServiceException()); + + GlobPathResolver resolver = mock(GlobPathResolver.class); + when(resolver.resolve()).thenReturn(Set.of()); + when(resolver.getWatchDirectories()).thenReturn(Set.of(Path.of("/nonexistent-dir-" + System.nanoTime()))); + + DirectoryWatcher watcher = new DirectoryWatcher(resolver, readerPool, checkpointRegistry, config, realFileOps, metrics, + Duration.ofSeconds(5), true, + () -> mockWatchService); + watcher.start(); + Thread.sleep(500); + watcher.stop(); + } + + @Test + void triggerFullRescan_handles_runtime_exception() throws Exception { + when(config.getPollInterval()).thenReturn(Duration.ofSeconds(60)); + lenient().when(config.getFingerprintBytes()).thenReturn(1024); + + WatchEvent overflowEvent = mock(WatchEvent.class); + when(overflowEvent.kind()).thenReturn((WatchEvent.Kind) StandardWatchEventKinds.OVERFLOW); + + WatchKey mockKey = mock(WatchKey.class); + when(mockKey.pollEvents()).thenReturn(List.of(overflowEvent)); + when(mockKey.reset()).thenReturn(true); + + WatchService mockWatchService = mock(WatchService.class); + when(mockWatchService.take()) + .thenReturn(mockKey) + .thenThrow(new ClosedWatchServiceException()); + + GlobPathResolver resolver = mock(GlobPathResolver.class); + when(resolver.resolve()) + .thenReturn(Set.of()) + .thenThrow(new RuntimeException("rescan error")); + when(resolver.getWatchDirectories()).thenReturn(Set.of(Path.of("/nonexistent-dir-" + System.nanoTime()))); + + DirectoryWatcher watcher = new DirectoryWatcher(resolver, readerPool, checkpointRegistry, config, realFileOps, metrics, + Duration.ofSeconds(5), true, + () -> mockWatchService); + watcher.start(); + Thread.sleep(500); + watcher.stop(); + } + + @Test + void handleDeletion_file_reappears_during_rotateWait() throws Exception { + Path testFile = tempDir.resolve("reappear.log"); + Files.writeString(testFile, "content"); + lenient().when(config.getFingerprintBytes()).thenReturn(1024); + when(config.getPollInterval()).thenReturn(Duration.ofSeconds(60)); + + WatchEvent deleteEvent = mock(WatchEvent.class); + lenient().when(deleteEvent.kind()).thenReturn((WatchEvent.Kind) StandardWatchEventKinds.ENTRY_DELETE); + lenient().when(deleteEvent.context()).thenReturn(testFile.getFileName()); + + WatchEvent createEvent = mock(WatchEvent.class); + lenient().when(createEvent.kind()).thenReturn((WatchEvent.Kind) StandardWatchEventKinds.ENTRY_CREATE); + lenient().when(createEvent.context()).thenReturn(testFile.getFileName()); + + WatchKey mockKey = mock(WatchKey.class); + lenient().when(mockKey.watchable()).thenReturn(tempDir); + lenient().when(mockKey.pollEvents()) + .thenReturn(List.of(createEvent)) + .thenReturn(List.of(deleteEvent)) + .thenReturn(Collections.emptyList()); + lenient().when(mockKey.reset()).thenReturn(true); + + WatchService mockWatchService = mock(WatchService.class); + lenient().when(mockWatchService.take()) + .thenReturn(mockKey) + .thenReturn(mockKey) + .thenThrow(new ClosedWatchServiceException()); + + GlobPathResolver resolver = mock(GlobPathResolver.class); + when(resolver.resolve()).thenReturn(Set.of(testFile.toAbsolutePath().normalize())); + when(resolver.getWatchDirectories()).thenReturn(Set.of(Path.of("/nonexistent-dir-" + System.nanoTime()))); + when(resolver.matches(testFile.toAbsolutePath().normalize())).thenReturn(true); + + DirectoryWatcher watcher = new DirectoryWatcher(resolver, readerPool, checkpointRegistry, config, realFileOps, metrics, + Duration.ofMillis(500), true, + () -> mockWatchService); + watcher.start(); + + Thread.sleep(2000); + watcher.stop(); + } + + @Test + void stop_pollScheduler_shutdownNow_on_timeout() throws Exception { + when(config.getPollInterval()).thenReturn(Duration.ofSeconds(60)); + lenient().when(config.getFingerprintBytes()).thenReturn(1024); + + GlobPathResolver resolver = mock(GlobPathResolver.class); + when(resolver.resolve()).thenReturn(Set.of()); + when(resolver.getWatchDirectories()).thenReturn(Set.of(tempDir)); + + DirectoryWatcher watcher = new DirectoryWatcher(resolver, readerPool, checkpointRegistry, config, realFileOps, metrics, + Duration.ofSeconds(5), true, + () -> { throw new RuntimeException("no watch service"); }, + false); + watcher.start(); + watcher.stop(); + } + + @Test + void stop_handles_IOException_when_watchService_close_fails() throws Exception { + when(config.getPollInterval()).thenReturn(Duration.ofSeconds(60)); + lenient().when(config.getFingerprintBytes()).thenReturn(1024); + + WatchService mockWatchService = mock(WatchService.class); + when(mockWatchService.take()).thenThrow(new ClosedWatchServiceException()); + doThrow(new IOException("close error")).when(mockWatchService).close(); + + GlobPathResolver resolver = mock(GlobPathResolver.class); + when(resolver.resolve()).thenReturn(Set.of()); + when(resolver.getWatchDirectories()).thenReturn(Set.of(Path.of("/nonexistent-ws-" + System.nanoTime()))); + + DirectoryWatcher watcher = new DirectoryWatcher(resolver, readerPool, checkpointRegistry, config, realFileOps, metrics, + Duration.ofSeconds(5), true, + () -> mockWatchService); + watcher.start(); + Thread.sleep(200); + watcher.stop(); + } + + @Test + void stop_handles_pollScheduler_awaitTermination_returning_false() throws Exception { + when(config.getPollInterval()).thenReturn(Duration.ofSeconds(60)); + lenient().when(config.getFingerprintBytes()).thenReturn(1024); + + ScheduledExecutorService mockScheduler = mock(ScheduledExecutorService.class); + when(mockScheduler.awaitTermination(anyLong(), any(TimeUnit.class))).thenReturn(false); + + GlobPathResolver resolver = mock(GlobPathResolver.class); + when(resolver.resolve()).thenReturn(Set.of()); + when(resolver.getWatchDirectories()).thenReturn(Set.of(Path.of("/nonexistent-ps-" + System.nanoTime()))); + + DirectoryWatcher watcher = new DirectoryWatcher(resolver, readerPool, checkpointRegistry, config, realFileOps, metrics, + Duration.ofSeconds(5), true, + () -> { throw new RuntimeException("no watch service"); }, + false, + () -> mockScheduler); + watcher.start(); + watcher.stop(); + } + + @Test + void stop_handles_pollScheduler_awaitTermination_throws_interrupted() throws Exception { + when(config.getPollInterval()).thenReturn(Duration.ofSeconds(60)); + lenient().when(config.getFingerprintBytes()).thenReturn(1024); + + ScheduledExecutorService mockScheduler = mock(ScheduledExecutorService.class); + when(mockScheduler.awaitTermination(anyLong(), any(TimeUnit.class))) + .thenThrow(new InterruptedException("test")); + + GlobPathResolver resolver = mock(GlobPathResolver.class); + when(resolver.resolve()).thenReturn(Set.of()); + when(resolver.getWatchDirectories()).thenReturn(Set.of(Path.of("/nonexistent-psi-" + System.nanoTime()))); + + DirectoryWatcher watcher = new DirectoryWatcher(resolver, readerPool, checkpointRegistry, config, realFileOps, metrics, + Duration.ofSeconds(5), true, + () -> { throw new RuntimeException("no watch service"); }, + false, + () -> mockScheduler); + watcher.start(); + watcher.stop(); + + assertThat(Thread.currentThread().isInterrupted(), equalTo(true)); + Thread.interrupted(); + } + + @Test + void stop_handles_watchThread_join_interrupted() throws Exception { + when(config.getPollInterval()).thenReturn(Duration.ofSeconds(60)); + lenient().when(config.getFingerprintBytes()).thenReturn(1024); + + CountDownLatch watchStarted = new CountDownLatch(1); + + WatchService mockWatchService = mock(WatchService.class); + when(mockWatchService.take()).thenAnswer(inv -> { + watchStarted.countDown(); + while (!Thread.currentThread().isInterrupted()) { + LockSupport.parkNanos(100_000_000L); + } + Thread.interrupted(); + Thread.sleep(2000); + throw new ClosedWatchServiceException(); + }); + + ScheduledExecutorService mockScheduler = mock(ScheduledExecutorService.class); + when(mockScheduler.awaitTermination(anyLong(), any(TimeUnit.class))).thenReturn(true); + + GlobPathResolver resolver = mock(GlobPathResolver.class); + when(resolver.resolve()).thenReturn(Set.of()); + when(resolver.getWatchDirectories()).thenReturn(Set.of(Path.of("/nonexistent-wtj-" + System.nanoTime()))); + + DirectoryWatcher watcher = new DirectoryWatcher(resolver, readerPool, checkpointRegistry, config, realFileOps, metrics, + Duration.ofSeconds(5), true, + () -> mockWatchService, + false, + () -> mockScheduler); + watcher.start(); + watchStarted.await(); + + Thread stopThread = new Thread(() -> { + Thread.currentThread().interrupt(); + watcher.stop(); + }); + stopThread.start(); + stopThread.join(10000); + + assertThat(stopThread.isAlive(), equalTo(false)); + } + + @Test + void watch_loop_exits_when_running_becomes_false() throws Exception { + lenient().when(config.getPollInterval()).thenReturn(Duration.ofSeconds(60)); + lenient().when(config.getFingerprintBytes()).thenReturn(1024); + + CountDownLatch firstIterDone = new CountDownLatch(1); + + WatchKey mockKey = mock(WatchKey.class); + lenient().when(mockKey.pollEvents()).thenReturn(Collections.emptyList()); + lenient().when(mockKey.reset()).thenReturn(true); + + WatchService mockWatchService = mock(WatchService.class); + when(mockWatchService.take()).thenAnswer(inv -> { + firstIterDone.countDown(); + return mockKey; + }).thenAnswer(inv -> { + Thread.sleep(60000); + return mockKey; + }); + + GlobPathResolver resolver = mock(GlobPathResolver.class); + when(resolver.resolve()).thenReturn(Set.of()); + when(resolver.getWatchDirectories()).thenReturn(Set.of(Path.of("/nonexistent-dir-" + System.nanoTime()))); + + ScheduledExecutorService mockScheduler = mock(ScheduledExecutorService.class); + lenient().when(mockScheduler.awaitTermination(anyLong(), any(TimeUnit.class))).thenReturn(true); + + DirectoryWatcher watcher = new DirectoryWatcher(resolver, readerPool, checkpointRegistry, config, realFileOps, metrics, + Duration.ofSeconds(5), true, + () -> mockWatchService, + false, + () -> mockScheduler); + watcher.start(); + firstIterDone.await(); + Thread.sleep(50); + watcher.stop(); + } + + @Test + void handleWatchEvent_ignores_file_not_matching_glob() throws Exception { + Path txtFile = tempDir.resolve("ignored.txt"); + Files.writeString(txtFile, "content"); + lenient().when(config.getFingerprintBytes()).thenReturn(1024); + lenient().when(config.getPollInterval()).thenReturn(Duration.ofSeconds(60)); + + WatchEvent createEvent = mock(WatchEvent.class); + lenient().when(createEvent.kind()).thenReturn((WatchEvent.Kind) StandardWatchEventKinds.ENTRY_CREATE); + lenient().when(createEvent.context()).thenReturn(txtFile.getFileName()); + + WatchKey mockKey = mock(WatchKey.class); + lenient().when(mockKey.watchable()).thenReturn(tempDir); + lenient().when(mockKey.pollEvents()).thenReturn(List.of(createEvent)); + lenient().when(mockKey.reset()).thenReturn(true); + + WatchService mockWatchService = mock(WatchService.class); + lenient().when(mockWatchService.take()) + .thenReturn(mockKey) + .thenThrow(new ClosedWatchServiceException()); + + GlobPathResolver resolver = mock(GlobPathResolver.class); + when(resolver.resolve()).thenReturn(Set.of()); + when(resolver.getWatchDirectories()).thenReturn(Set.of(Path.of("/nonexistent-dir-" + System.nanoTime()))); + when(resolver.matches(any())).thenReturn(false); + + ScheduledExecutorService mockScheduler = mock(ScheduledExecutorService.class); + lenient().when(mockScheduler.awaitTermination(anyLong(), any(TimeUnit.class))).thenReturn(true); + + DirectoryWatcher watcher = new DirectoryWatcher(resolver, readerPool, checkpointRegistry, config, realFileOps, metrics, + Duration.ofSeconds(5), true, + () -> mockWatchService, + false, + () -> mockScheduler); + watcher.start(); + Thread.sleep(500); + watcher.stop(); + + verify(readerPool, never()).addFile(any(), any()); + } + + @Test + void handleWatchEvent_ignores_ENTRY_MODIFY_event() throws Exception { + Path logFile = tempDir.resolve("modify.log"); + Files.writeString(logFile, "content"); + lenient().when(config.getFingerprintBytes()).thenReturn(1024); + lenient().when(config.getPollInterval()).thenReturn(Duration.ofSeconds(60)); + + WatchEvent modifyEvent = mock(WatchEvent.class); + lenient().when(modifyEvent.kind()).thenReturn((WatchEvent.Kind) StandardWatchEventKinds.ENTRY_MODIFY); + lenient().when(modifyEvent.context()).thenReturn(logFile.getFileName()); + + WatchKey mockKey = mock(WatchKey.class); + lenient().when(mockKey.watchable()).thenReturn(tempDir); + lenient().when(mockKey.pollEvents()).thenReturn(List.of(modifyEvent)); + lenient().when(mockKey.reset()).thenReturn(true); + + WatchService mockWatchService = mock(WatchService.class); + when(mockWatchService.take()) + .thenReturn(mockKey) + .thenThrow(new ClosedWatchServiceException()); + + GlobPathResolver resolver = mock(GlobPathResolver.class); + when(resolver.resolve()).thenReturn(Set.of()); + when(resolver.getWatchDirectories()).thenReturn(Set.of(Path.of("/nonexistent-dir-" + System.nanoTime()))); + + ScheduledExecutorService mockScheduler = mock(ScheduledExecutorService.class); + lenient().when(mockScheduler.awaitTermination(anyLong(), any(TimeUnit.class))).thenReturn(true); + + DirectoryWatcher watcher = new DirectoryWatcher(resolver, readerPool, checkpointRegistry, config, realFileOps, metrics, + Duration.ofSeconds(5), true, + () -> mockWatchService, + false, + () -> mockScheduler); + watcher.start(); + Thread.sleep(500); + watcher.stop(); + + verify(readerPool, never()).addFile(any(), any()); + verify(readerPool, never()).closeReaderForPath(any()); + } + + @Test + void handleWatchEvent_DELETE_for_unknown_file_does_nothing() throws Exception { + Path unknownFile = tempDir.resolve("unknown.log"); + lenient().when(config.getFingerprintBytes()).thenReturn(1024); + lenient().when(config.getPollInterval()).thenReturn(Duration.ofSeconds(60)); + + WatchEvent deleteEvent = mock(WatchEvent.class); + lenient().when(deleteEvent.kind()).thenReturn((WatchEvent.Kind) StandardWatchEventKinds.ENTRY_DELETE); + lenient().when(deleteEvent.context()).thenReturn(unknownFile.getFileName()); + + WatchKey mockKey = mock(WatchKey.class); + lenient().when(mockKey.watchable()).thenReturn(tempDir); + lenient().when(mockKey.pollEvents()).thenReturn(List.of(deleteEvent)); + lenient().when(mockKey.reset()).thenReturn(true); + + WatchService mockWatchService = mock(WatchService.class); + when(mockWatchService.take()) + .thenReturn(mockKey) + .thenThrow(new ClosedWatchServiceException()); + + GlobPathResolver resolver = mock(GlobPathResolver.class); + when(resolver.resolve()).thenReturn(Set.of()); + when(resolver.getWatchDirectories()).thenReturn(Set.of(Path.of("/nonexistent-dir-" + System.nanoTime()))); + + ScheduledExecutorService mockScheduler = mock(ScheduledExecutorService.class); + lenient().when(mockScheduler.awaitTermination(anyLong(), any(TimeUnit.class))).thenReturn(true); + + DirectoryWatcher watcher = new DirectoryWatcher(resolver, readerPool, checkpointRegistry, config, realFileOps, metrics, + Duration.ofSeconds(5), true, + () -> mockWatchService, + false, + () -> mockScheduler); + watcher.start(); + Thread.sleep(500); + watcher.stop(); + + verify(readerPool, never()).closeReaderForPath(any()); + } + + @Test + void shouldUseWatchService_returns_false_when_network_fs_detected() throws Exception { + when(config.getPollInterval()).thenReturn(Duration.ofSeconds(60)); + lenient().when(config.getFingerprintBytes()).thenReturn(1024); + + GlobPathResolver resolver = mock(GlobPathResolver.class); + when(resolver.resolve()).thenReturn(Set.of()); + when(resolver.getWatchDirectories()).thenReturn(Set.of(tempDir)); + + DirectoryWatcher watcher = new DirectoryWatcher(resolver, readerPool, checkpointRegistry, config, realFileOps, metrics, + Duration.ofSeconds(5), true, + () -> FileSystems.getDefault().newWatchService(), + false, + DirectoryWatcher::createDefaultPollScheduler, + path -> true); + watcher.start(); + Thread.sleep(200); + watcher.stop(); + } + + @Test + void handleWatchEvent_DELETE_rejectedExecutionException_on_schedule() throws Exception { + Path logFile = tempDir.resolve("rej-delete.log"); + Files.writeString(logFile, "content"); + lenient().when(config.getFingerprintBytes()).thenReturn(1024); + lenient().when(config.getPollInterval()).thenReturn(Duration.ofSeconds(60)); + + WatchEvent createEvent = mock(WatchEvent.class); + lenient().when(createEvent.kind()).thenReturn((WatchEvent.Kind) StandardWatchEventKinds.ENTRY_CREATE); + lenient().when(createEvent.context()).thenReturn(logFile.getFileName()); + + WatchEvent deleteEvent = mock(WatchEvent.class); + lenient().when(deleteEvent.kind()).thenReturn((WatchEvent.Kind) StandardWatchEventKinds.ENTRY_DELETE); + lenient().when(deleteEvent.context()).thenReturn(logFile.getFileName()); + + WatchKey mockKey = mock(WatchKey.class); + lenient().when(mockKey.watchable()).thenReturn(tempDir); + lenient().when(mockKey.pollEvents()) + .thenReturn(List.of(createEvent)) + .thenReturn(List.of(deleteEvent)); + lenient().when(mockKey.reset()).thenReturn(true); + + WatchService mockWatchService = mock(WatchService.class); + lenient().when(mockWatchService.take()) + .thenReturn(mockKey) + .thenReturn(mockKey) + .thenThrow(new ClosedWatchServiceException()); + + GlobPathResolver resolver = mock(GlobPathResolver.class); + when(resolver.resolve()).thenReturn(Set.of(logFile.toAbsolutePath().normalize())); + when(resolver.getWatchDirectories()).thenReturn(Set.of(Path.of("/nonexistent-dir-" + System.nanoTime()))); + when(resolver.matches(any())).thenReturn(true); + + ScheduledExecutorService mockScheduler = mock(ScheduledExecutorService.class); + when(mockScheduler.awaitTermination(anyLong(), any(TimeUnit.class))).thenReturn(true); + when(mockScheduler.schedule(any(Runnable.class), anyLong(), any(TimeUnit.class))) + .thenThrow(new RejectedExecutionException("shutting down")); + + DirectoryWatcher watcher = new DirectoryWatcher(resolver, readerPool, checkpointRegistry, config, realFileOps, metrics, + Duration.ofMillis(100), true, + () -> mockWatchService, + false, + () -> mockScheduler); + watcher.start(); + Thread.sleep(500); + watcher.stop(); + } + + @Test + void triggerFullRescan_handles_rejectedExecutionException() throws Exception { + when(config.getPollInterval()).thenReturn(Duration.ofSeconds(60)); + lenient().when(config.getFingerprintBytes()).thenReturn(1024); + + WatchEvent overflowEvent = mock(WatchEvent.class); + when(overflowEvent.kind()).thenReturn((WatchEvent.Kind) StandardWatchEventKinds.OVERFLOW); + + WatchKey mockKey = mock(WatchKey.class); + when(mockKey.pollEvents()).thenReturn(List.of(overflowEvent)); + when(mockKey.reset()).thenReturn(true); + + WatchService mockWatchService = mock(WatchService.class); + when(mockWatchService.take()) + .thenReturn(mockKey) + .thenThrow(new ClosedWatchServiceException()); + + GlobPathResolver resolver = mock(GlobPathResolver.class); + when(resolver.resolve()).thenReturn(Set.of()); + when(resolver.getWatchDirectories()).thenReturn(Set.of(Path.of("/nonexistent-dir-" + System.nanoTime()))); + + ScheduledExecutorService mockScheduler = mock(ScheduledExecutorService.class); + lenient().when(mockScheduler.awaitTermination(anyLong(), any(TimeUnit.class))).thenReturn(true); + doThrow(new RejectedExecutionException("shutting down")) + .when(mockScheduler).execute(any(Runnable.class)); + + DirectoryWatcher watcher = new DirectoryWatcher(resolver, readerPool, checkpointRegistry, config, realFileOps, metrics, + Duration.ofSeconds(5), true, + () -> mockWatchService, + false, + () -> mockScheduler); + watcher.start(); + Thread.sleep(500); + watcher.stop(); + } + + @Test + void pollScan_vanished_file_rejectedExecutionException_on_schedule() throws Exception { + Path vanishFile = tempDir.resolve("vanish-rej.log"); + Files.writeString(vanishFile, "content"); + when(config.getFingerprintBytes()).thenReturn(1024); + when(config.getPollInterval()).thenReturn(Duration.ofSeconds(60)); + + ScheduledExecutorService mockScheduler = mock(ScheduledExecutorService.class); + lenient().when(mockScheduler.awaitTermination(anyLong(), any(TimeUnit.class))).thenReturn(true); + lenient().when(mockScheduler.scheduleAtFixedRate(any(Runnable.class), anyLong(), anyLong(), any(TimeUnit.class))) + .thenReturn(mock(ScheduledFuture.class)); + when(mockScheduler.schedule(any(Runnable.class), anyLong(), any(TimeUnit.class))) + .thenThrow(new RejectedExecutionException("shutting down")); + + GlobPathResolver resolver = mock(GlobPathResolver.class); + when(resolver.resolve()) + .thenReturn(Set.of(vanishFile.toAbsolutePath().normalize())) + .thenReturn(Set.of()); + when(resolver.getWatchDirectories()).thenReturn(Set.of(Path.of("/nonexistent-dir-" + System.nanoTime()))); + + DirectoryWatcher watcher = new DirectoryWatcher(resolver, readerPool, checkpointRegistry, config, realFileOps, metrics, + Duration.ofMillis(100), true, + () -> { throw new RuntimeException("no watch service"); }, + false, + () -> mockScheduler); + watcher.start(); + watcher.pollScan(); + watcher.stop(); + } + + @Test + void startWatchService_falls_back_to_polling_when_registration_fails() throws Exception { + when(config.getPollInterval()).thenReturn(Duration.ofSeconds(60)); + lenient().when(config.getFingerprintBytes()).thenReturn(1024); + + WatchService closedWatchService = FileSystems.getDefault().newWatchService(); + closedWatchService.close(); + + GlobPathResolver resolver = mock(GlobPathResolver.class); + when(resolver.resolve()).thenReturn(Set.of()); + when(resolver.getWatchDirectories()).thenReturn(Set.of(tempDir)); + + DirectoryWatcher watcher = new DirectoryWatcher(resolver, readerPool, checkpointRegistry, config, realFileOps, metrics, + Duration.ofSeconds(5), true, + () -> closedWatchService, + false, + () -> Executors.newSingleThreadScheduledExecutor(), + path -> false); + watcher.start(); + Thread.sleep(200); + watcher.stop(); + } +} diff --git a/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/FileIdentityTest.java b/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/FileIdentityTest.java new file mode 100644 index 0000000000..979daadaa1 --- /dev/null +++ b/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/FileIdentityTest.java @@ -0,0 +1,289 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + +package org.opensearch.dataprepper.plugins.source.file; + +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.io.TempDir; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; + +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.attribute.BasicFileAttributes; +import java.nio.file.attribute.FileTime; +import java.time.Instant; +import java.util.zip.CRC32; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.not; +import static org.hamcrest.Matchers.notNullValue; +import static org.mockito.Mockito.when; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.mockito.Mockito.mock; + +@ExtendWith(MockitoExtension.class) +class FileIdentityTest { + + private static final int FINGERPRINT_BYTES = 256; + + @TempDir + Path tempDir; + + @Mock + private FileSystemOperations fileOps; + + @Mock + private BasicFileAttributes attrs; + + @Test + void fromReturnsInodeBasedIdentityWhenFileKeyPresent() throws IOException { + final Path testFile = tempDir.resolve("test.log"); + Files.writeString(testFile, "some content"); + + final Object fileKey = "12345"; + when(fileOps.readAttributes(testFile)).thenReturn(attrs); + when(attrs.fileKey()).thenReturn(fileKey); + + final FileIdentity identity = FileIdentity.from(testFile, fileOps, FINGERPRINT_BYTES); + + assertThat(identity, notNullValue()); + assertThat(identity.toString(), containsString("inode:12345")); + assertThat(identity.getPath(), equalTo(testFile)); + } + + @Test + void fromReturnsCrcFallbackWhenFileKeyIsNull() throws IOException { + final Path testFile = tempDir.resolve("test.log"); + final String content = "hello world data"; + Files.writeString(testFile, content); + final byte[] contentBytes = content.getBytes(); + + when(fileOps.readAttributes(testFile)).thenReturn(attrs); + when(attrs.fileKey()).thenReturn(null); + when(fileOps.size(testFile)).thenReturn((long) contentBytes.length); + when(fileOps.readBytes(testFile, Math.min(FINGERPRINT_BYTES, contentBytes.length))).thenReturn(contentBytes); + when(attrs.creationTime()).thenReturn(FileTime.from(Instant.parse("2025-01-01T00:00:00Z"))); + + final CRC32 expectedCrc = new CRC32(); + expectedCrc.update(contentBytes); + + final FileIdentity identity = FileIdentity.from(testFile, fileOps, FINGERPRINT_BYTES); + + assertThat(identity, notNullValue()); + assertThat(identity.toString(), containsString("crc:" + expectedCrc.getValue())); + assertThat(identity.toString(), containsString("created:")); + } + + @Test + void fromReturnsPathFallbackWhenIOExceptionOnReadAttributes() throws IOException { + final Path testFile = tempDir.resolve("missing.log"); + + when(fileOps.readAttributes(testFile)).thenThrow(new IOException("cannot read")); + + final FileIdentity identity = FileIdentity.from(testFile, fileOps, FINGERPRINT_BYTES); + + assertThat(identity, notNullValue()); + assertThat(identity.toString(), containsString("path:")); + assertThat(identity.toString(), containsString(testFile.toAbsolutePath().toString())); + } + + @Test + void fromReturnsPathFallbackForEmptyFile() throws IOException { + final Path testFile = tempDir.resolve("empty.log"); + Files.writeString(testFile, ""); + + when(fileOps.readAttributes(testFile)).thenReturn(attrs); + when(attrs.fileKey()).thenReturn(null); + when(fileOps.size(testFile)).thenReturn(0L); + + final FileIdentity identity = FileIdentity.from(testFile, fileOps, FINGERPRINT_BYTES); + + assertThat(identity, notNullValue()); + assertThat(identity.toString(), containsString("path:")); + } + + @Test + void fromReturnsPathFallbackWhenSizeThrowsIOException() throws IOException { + final Path testFile = tempDir.resolve("unreadable.log"); + + when(fileOps.readAttributes(testFile)).thenReturn(attrs); + when(attrs.fileKey()).thenReturn(null); + when(fileOps.size(testFile)).thenThrow(new IOException("permission denied")); + + final FileIdentity identity = FileIdentity.from(testFile, fileOps, FINGERPRINT_BYTES); + + assertThat(identity, notNullValue()); + assertThat(identity.toString(), containsString("path:")); + } + + @Test + void equalIdentitiesAreEqual() throws IOException { + final Path testFile = tempDir.resolve("a.log"); + Files.writeString(testFile, "data"); + + final Object fileKey = "inode-99"; + when(fileOps.readAttributes(testFile)).thenReturn(attrs); + when(attrs.fileKey()).thenReturn(fileKey); + + final FileIdentity first = FileIdentity.from(testFile, fileOps, FINGERPRINT_BYTES); + final FileIdentity second = FileIdentity.from(testFile, fileOps, FINGERPRINT_BYTES); + + assertThat(first, equalTo(second)); + assertThat(first.hashCode(), equalTo(second.hashCode())); + } + + @Test + void differentIdentitiesAreNotEqual() throws IOException { + final Path fileA = tempDir.resolve("a.log"); + final Path fileB = tempDir.resolve("b.log"); + Files.writeString(fileA, "data-a"); + Files.writeString(fileB, "data-b"); + + @SuppressWarnings("unchecked") + final BasicFileAttributes attrsB = mock(BasicFileAttributes.class); + + when(fileOps.readAttributes(fileA)).thenReturn(attrs); + when(attrs.fileKey()).thenReturn("inode-1"); + + when(fileOps.readAttributes(fileB)).thenReturn(attrsB); + when(attrsB.fileKey()).thenReturn("inode-2"); + + final FileIdentity identityA = FileIdentity.from(fileA, fileOps, FINGERPRINT_BYTES); + final FileIdentity identityB = FileIdentity.from(fileB, fileOps, FINGERPRINT_BYTES); + + assertThat(identityA, not(equalTo(identityB))); + } + + @Test + void identityIsNotEqualToNull() throws IOException { + final Path testFile = tempDir.resolve("file.log"); + Files.writeString(testFile, "content"); + + when(fileOps.readAttributes(testFile)).thenReturn(attrs); + when(attrs.fileKey()).thenReturn("inode-42"); + + final FileIdentity identity = FileIdentity.from(testFile, fileOps, FINGERPRINT_BYTES); + + assertThat(identity.equals(null), equalTo(false)); + } + + @Test + void identityIsNotEqualToDifferentClass() throws IOException { + final Path testFile = tempDir.resolve("diffclass.log"); + Files.writeString(testFile, "content"); + + when(fileOps.readAttributes(testFile)).thenReturn(attrs); + when(attrs.fileKey()).thenReturn("inode-99"); + + final FileIdentity identity = FileIdentity.from(testFile, fileOps, FINGERPRINT_BYTES); + + assertThat(identity.equals("a string object"), equalTo(false)); + } + + @Test + void identityIsEqualToItself() throws IOException { + final Path testFile = tempDir.resolve("self.log"); + Files.writeString(testFile, "content"); + + when(fileOps.readAttributes(testFile)).thenReturn(attrs); + when(attrs.fileKey()).thenReturn("inode-7"); + + final FileIdentity identity = FileIdentity.from(testFile, fileOps, FINGERPRINT_BYTES); + + assertThat(identity.equals(identity), equalTo(true)); + } + + @Test + void getPathReturnsOriginalPath() throws IOException { + final Path testFile = tempDir.resolve("path-check.log"); + Files.writeString(testFile, "content"); + + when(fileOps.readAttributes(testFile)).thenReturn(attrs); + when(attrs.fileKey()).thenReturn("inode-10"); + + final FileIdentity identity = FileIdentity.from(testFile, fileOps, FINGERPRINT_BYTES); + + assertThat(identity.getPath(), equalTo(testFile)); + } + + @Test + void crcFallbackUsesCreationTime() throws IOException { + final Path testFile = tempDir.resolve("created.log"); + final String content = "log data here"; + Files.writeString(testFile, content); + final byte[] bytes = content.getBytes(); + + when(fileOps.readAttributes(testFile)).thenReturn(attrs); + when(attrs.fileKey()).thenReturn(null); + when(fileOps.size(testFile)).thenReturn((long) bytes.length); + when(fileOps.readBytes(testFile, Math.min(FINGERPRINT_BYTES, bytes.length))).thenReturn(bytes); + when(attrs.creationTime()).thenReturn(FileTime.from(Instant.parse("2024-06-15T12:30:00Z"))); + + final FileIdentity identity = FileIdentity.from(testFile, fileOps, FINGERPRINT_BYTES); + + assertThat(identity.toString(), containsString("created:")); + } + + @Test + void crcFallbackHandlesReadBytesIOException() throws IOException { + final Path testFile = tempDir.resolve("read-fail.log"); + Files.writeString(testFile, "content"); + + when(fileOps.readAttributes(testFile)).thenReturn(attrs); + when(attrs.fileKey()).thenReturn(null); + when(fileOps.size(testFile)).thenReturn(100L); + when(fileOps.readBytes(testFile, 100)).thenThrow(new IOException("read failed")); + + final FileIdentity identity = FileIdentity.from(testFile, fileOps, FINGERPRINT_BYTES); + + assertThat(identity, notNullValue()); + assertThat(identity.toString(), containsString("path:")); + } + + @Test + void fromThrowsIllegalArgumentExceptionWhenFingerprintBytesIsZero() { + final Path testFile = tempDir.resolve("zero-fp.log"); + + assertThrows(IllegalArgumentException.class, + () -> FileIdentity.from(testFile, fileOps, 0)); + } + + @Test + void fromThrowsIllegalArgumentExceptionWhenFingerprintBytesIsNegative() { + final Path testFile = tempDir.resolve("neg-fp.log"); + + assertThrows(IllegalArgumentException.class, + () -> FileIdentity.from(testFile, fileOps, -1)); + } + + @Test + void fingerprintBytesLimitsReadSize() throws IOException { + final Path testFile = tempDir.resolve("large.log"); + Files.writeString(testFile, "a]".repeat(500)); + + final int smallFingerprint = 16; + + when(fileOps.readAttributes(testFile)).thenReturn(attrs); + when(attrs.fileKey()).thenReturn(null); + when(fileOps.size(testFile)).thenReturn(1000L); + when(fileOps.readBytes(testFile, smallFingerprint)).thenReturn(new byte[smallFingerprint]); + when(attrs.creationTime()).thenReturn(FileTime.from(Instant.parse("2025-01-01T00:00:00Z"))); + + final FileIdentity identity = FileIdentity.from(testFile, fileOps, smallFingerprint); + + assertThat(identity, notNullValue()); + assertThat(identity.toString(), containsString("crc:")); + } +} diff --git a/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/FileSourceConfigTest.java b/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/FileSourceConfigTest.java new file mode 100644 index 0000000000..37d65b61f4 --- /dev/null +++ b/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/FileSourceConfigTest.java @@ -0,0 +1,225 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + +package org.opensearch.dataprepper.plugins.source.file; + +import com.fasterxml.jackson.databind.ObjectMapper; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.CsvSource; +import org.junit.jupiter.params.provider.ValueSource; +import org.opensearch.dataprepper.model.configuration.PluginModel; + +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.time.Duration; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.containsInAnyOrder; +import static org.hamcrest.Matchers.empty; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.nullValue; +import static org.hamcrest.Matchers.hasSize; +import static org.junit.jupiter.api.Assertions.assertThrows; + +class FileSourceConfigTest { + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + + @ParameterizedTest + @ValueSource(strings = {FileSourceConfig.EVENT_TYPE, FileSourceConfig.DEFAULT_FORMAT}) + void codeRequiresRecordTypeEvent_returns_true_if_no_codec(final String recordType) { + final Map fileConfigMap = Map.of(FileSourceConfig.ATTRIBUTE_TYPE, recordType); + final FileSourceConfig objectUnderTest = OBJECT_MAPPER.convertValue(fileConfigMap, FileSourceConfig.class); + + assertThat(objectUnderTest.codeRequiresRecordTypeEvent(), equalTo(true)); + } + + @ParameterizedTest + @CsvSource({ + FileSourceConfig.EVENT_TYPE + ",true", + FileSourceConfig.DEFAULT_FORMAT + ",false" + }) + void codeRequiresRecordTypeEvent_returns_expected_value_when_there_is_a_codec(final String recordType, final boolean expected) { + final Map fileConfigMap = Map.of( + FileSourceConfig.ATTRIBUTE_TYPE, recordType, + "codec", new PluginModel("fake_codec", Collections.emptyMap()) + ); + final FileSourceConfig objectUnderTest = OBJECT_MAPPER.convertValue(fileConfigMap, FileSourceConfig.class); + + assertThat(objectUnderTest.codeRequiresRecordTypeEvent(), equalTo(expected)); + } + + @Test + void tail_defaults_to_false() { + final Map configMap = Map.of("path", "/tmp/test.log"); + final FileSourceConfig config = OBJECT_MAPPER.convertValue(configMap, FileSourceConfig.class); + + assertThat(config.isTail(), equalTo(false)); + } + + @Test + void tail_can_be_set_to_true() { + final Map configMap = Map.of("path", "/tmp/test.log", "tail", true); + final FileSourceConfig config = OBJECT_MAPPER.convertValue(configMap, FileSourceConfig.class); + + assertThat(config.isTail(), equalTo(true)); + } + + @Test + void paths_defaults_to_empty_list() { + final Map configMap = Map.of("path", "/tmp/test.log"); + final FileSourceConfig config = OBJECT_MAPPER.convertValue(configMap, FileSourceConfig.class); + + assertThat(config.getPaths(), empty()); + } + + @Test + void paths_returns_configured_values() { + final Map configMap = Map.of("paths", List.of("/var/log/*.log", "/tmp/*.log")); + final FileSourceConfig config = OBJECT_MAPPER.convertValue(configMap, FileSourceConfig.class); + + assertThat(config.getPaths(), hasSize(2)); + } + + @Test + void getAllPaths_merges_path_and_paths() { + final Map configMap = Map.of( + "path", "/tmp/single.log", + "paths", List.of("/var/log/*.log") + ); + final FileSourceConfig config = OBJECT_MAPPER.convertValue(configMap, FileSourceConfig.class); + + assertThat(config.getAllPaths(), hasSize(2)); + assertThat(config.getAllPaths(), containsInAnyOrder("/var/log/*.log", "/tmp/single.log")); + } + + @Test + void getAllPaths_deduplicates_when_path_is_in_paths() { + final Map configMap = Map.of( + "path", "/var/log/*.log", + "paths", List.of("/var/log/*.log") + ); + final FileSourceConfig config = OBJECT_MAPPER.convertValue(configMap, FileSourceConfig.class); + + assertThat(config.getAllPaths(), hasSize(1)); + } + + @Test + void validate_succeeds_with_path_when_tail_false() { + final Map configMap = Map.of("path", "/tmp/test.log"); + final FileSourceConfig config = OBJECT_MAPPER.convertValue(configMap, FileSourceConfig.class); + + config.validate(); + } + + @Test + void validate_fails_without_path_when_tail_false() { + final Map configMap = Map.of("format", "plain", "record_type", "string"); + final FileSourceConfig config = OBJECT_MAPPER.convertValue(configMap, FileSourceConfig.class); + + assertThrows(IllegalArgumentException.class, config::validate); + } + + @Test + void validate_succeeds_with_paths_when_tail_true() { + final Map configMap = Map.of("tail", true, "paths", List.of("/var/log/*.log")); + final FileSourceConfig config = OBJECT_MAPPER.convertValue(configMap, FileSourceConfig.class); + + config.validate(); + } + + @Test + void validate_succeeds_with_path_when_tail_true() { + final Map configMap = Map.of("tail", true, "path", "/tmp/test.log"); + final FileSourceConfig config = OBJECT_MAPPER.convertValue(configMap, FileSourceConfig.class); + + config.validate(); + } + + @Test + void validate_fails_without_any_path_when_tail_true() { + final Map configMap = Map.of("tail", true); + final FileSourceConfig config = OBJECT_MAPPER.convertValue(configMap, FileSourceConfig.class); + + assertThrows(IllegalArgumentException.class, config::validate); + } + + @Test + void default_config_returns_expected_values() { + final Map configMap = Map.of("path", "/tmp/test.log"); + final FileSourceConfig config = OBJECT_MAPPER.convertValue(configMap, FileSourceConfig.class); + + assertThat(config.getStartPosition(), equalTo(StartPosition.END)); + assertThat(config.getPollInterval(), equalTo(Duration.ofSeconds(1))); + assertThat(config.getEncoding(), equalTo("UTF-8")); + assertThat(config.getReadBufferSize(), equalTo(65536)); + assertThat(config.getMaxActiveFiles(), equalTo(100)); + assertThat(config.getReaderThreads(), equalTo(2)); + assertThat(config.getMaxReadTimePerFile(), equalTo(Duration.ofSeconds(5))); + assertThat(config.getRotateWait(), equalTo(Duration.ofSeconds(5))); + assertThat(config.getRotationDrainTimeout(), equalTo(Duration.ofSeconds(30))); + assertThat(config.getCheckpointFile(), nullValue()); + assertThat(config.getCheckpointInterval(), equalTo(Duration.ofSeconds(15))); + assertThat(config.getCheckpointCleanupAfter(), equalTo(Duration.ofHours(24))); + assertThat(config.getFingerprintBytes(), equalTo(1024)); + assertThat(config.getCloseInactive(), equalTo(Duration.ofMinutes(5))); + assertThat(config.isCloseRemoved(), equalTo(true)); + assertThat(config.getBatchSize(), equalTo(1000)); + assertThat(config.getBatchTimeout(), equalTo(Duration.ofSeconds(5))); + assertThat(config.getAcknowledgmentTimeout(), equalTo(Duration.ofSeconds(30))); + assertThat(config.getMaxAcknowledgmentRetries(), equalTo(3)); + assertThat(config.isIncludeFileMetadata(), equalTo(false)); + assertThat(config.getMaxLineLength(), equalTo(1048576)); + } + + @Test + void exclude_paths_defaults_to_empty_list() { + final Map configMap = Map.of("path", "/tmp/test.log"); + final FileSourceConfig config = OBJECT_MAPPER.convertValue(configMap, FileSourceConfig.class); + + assertThat(config.getExcludePaths(), empty()); + } + + @Test + void exclude_paths_returns_configured_values() { + final Map configMap = Map.of( + "path", "/tmp/test.log", + "exclude_paths", List.of("/tmp/exclude*.log") + ); + final FileSourceConfig config = OBJECT_MAPPER.convertValue(configMap, FileSourceConfig.class); + + assertThat(config.getExcludePaths(), hasSize(1)); + } + + @Test + void getAllPaths_with_null_filePathToRead_returns_only_paths() { + final Map configMap = Map.of( + "paths", List.of("/var/log/*.log") + ); + final FileSourceConfig config = OBJECT_MAPPER.convertValue(configMap, FileSourceConfig.class); + + assertThat(config.getAllPaths(), hasSize(1)); + assertThat(config.getAllPaths(), containsInAnyOrder("/var/log/*.log")); + } + + @Test + void validate_fails_when_tail_true_and_filePathToRead_is_empty_and_paths_is_null() { + final Map configMap = Map.of("tail", true, "path", ""); + final FileSourceConfig config = OBJECT_MAPPER.convertValue(configMap, FileSourceConfig.class); + + assertThrows(IllegalArgumentException.class, config::validate); + } + + @Test + void getFormat_returns_plain_when_format_is_null() { + assertThat(FileFormat.getByName(null), equalTo(FileFormat.PLAIN)); + } +} diff --git a/data-prepper-plugins/common/src/test/java/org/opensearch/dataprepper/plugins/source/file/FileSourceTests.java b/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/FileSourceTests.java similarity index 65% rename from data-prepper-plugins/common/src/test/java/org/opensearch/dataprepper/plugins/source/file/FileSourceTests.java rename to data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/FileSourceTests.java index 1a6b20a23a..ea904b190b 100644 --- a/data-prepper-plugins/common/src/test/java/org/opensearch/dataprepper/plugins/source/file/FileSourceTests.java +++ b/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/FileSourceTests.java @@ -1,6 +1,11 @@ /* * Copyright OpenSearch Contributors * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * */ package org.opensearch.dataprepper.plugins.source.file; @@ -18,6 +23,7 @@ import org.opensearch.dataprepper.event.TestEventFactory; import org.opensearch.dataprepper.metrics.PluginMetrics; import org.opensearch.dataprepper.model.buffer.Buffer; +import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSetManager; import org.opensearch.dataprepper.model.codec.DecompressionEngine; import org.opensearch.dataprepper.model.codec.InputCodec; import org.opensearch.dataprepper.model.configuration.PipelineDescription; @@ -50,9 +56,11 @@ import static org.hamcrest.CoreMatchers.is; import static org.hamcrest.CoreMatchers.notNullValue; import static org.hamcrest.MatcherAssert.assertThat; +import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.verify; @@ -66,6 +74,7 @@ public class FileSourceTests { private static final TypeReference> MAP_TYPE_REFERENCE = new TypeReference>() { }; private static final String TEST_FILE_PATH_PLAIN = "src/test/resources/test-file-source-plain.tst"; + private static final String MESSAGE_KEY = "message"; private FileSourceConfig fileSourceConfig; @@ -90,16 +99,9 @@ void setUp() { private FileSource createObjectUnderTest() { fileSourceConfig = OBJECT_MAPPER.convertValue(pluginSettings, FileSourceConfig.class); - return new FileSource(fileSourceConfig, pluginMetrics, pluginFactory, TestEventFactory.getTestEventFactory()); + return new FileSource(fileSourceConfig, pluginMetrics, pluginFactory, TestEventFactory.getTestEventFactory(), null); } - /** - * Variant of creatgeObjectUnderTest that uses mocks for the configuration instead of object mapper, so we can - * pass concrete mocks to the FileSource through the FileSourceConfig. - * @param codec the codec to use in the configuration - * @param engine the {@link DecompressionEngine} to use in the configuration - * @return - */ private FileSource createObjectUnderTest(PluginModel codec, DecompressionEngine engine) { FileSourceConfig fileSourceConfig = mock(FileSourceConfig.class); @@ -115,7 +117,7 @@ private FileSource createObjectUnderTest(PluginModel codec, DecompressionEngine when(fileSourceConfig.getCompression()).thenReturn(compressionOption); } - return new FileSource(fileSourceConfig, pluginMetrics, pluginFactory, TestEventFactory.getTestEventFactory()); + return new FileSource(fileSourceConfig, pluginMetrics, pluginFactory, TestEventFactory.getTestEventFactory(), null); } @Nested @@ -140,30 +142,27 @@ public void setup() throws JsonProcessingException { expectedEventsJson = new ArrayList<>(); expectedEventsInvalidJson = new ArrayList<>(); - // plain final String expectedPlainFirstLine = "THIS IS A PLAINTEXT LINE"; final String expectedPlainSecondLine = "THIS IS ANOTHER PLAINTEXT LINE"; - final Record firstEventPlain = createRecordEventWithKeyValuePair(FileSource.MESSAGE_KEY, expectedPlainFirstLine); - final Record secondEventPlain = createRecordEventWithKeyValuePair(FileSource.MESSAGE_KEY, expectedPlainSecondLine); + final Record firstEventPlain = createRecordEventWithKeyValuePair(MESSAGE_KEY, expectedPlainFirstLine); + final Record secondEventPlain = createRecordEventWithKeyValuePair(MESSAGE_KEY, expectedPlainSecondLine); expectedEventsPlain.add(firstEventPlain); expectedEventsPlain.add(secondEventPlain); - //json final Record firstEventJson = createRecordEventWithKeyValuePair("test_key", "test_value"); final Record secondEventJson = createRecordEventWithKeyValuePair("second_test_key", "second_test_value"); expectedEventsJson.add(firstEventJson); expectedEventsJson.add(secondEventJson); - // invalid json final String expectedInvalidJsonFirstLine = "{\"test_key: test_value\"}"; final String expectedInvalidJsonSecondLine = "{\"second_test_key\": \"second_test_value\""; - final Record firstEventInvalidJson = createRecordEventWithKeyValuePair(FileSource.MESSAGE_KEY, expectedInvalidJsonFirstLine); - final Record secondEventInvalidJson = createRecordEventWithKeyValuePair(FileSource.MESSAGE_KEY, expectedInvalidJsonSecondLine); + final Record firstEventInvalidJson = createRecordEventWithKeyValuePair(MESSAGE_KEY, expectedInvalidJsonFirstLine); + final Record secondEventInvalidJson = createRecordEventWithKeyValuePair(MESSAGE_KEY, expectedInvalidJsonSecondLine); expectedEventsInvalidJson.add(firstEventInvalidJson); expectedEventsInvalidJson.add(secondEventInvalidJson); @@ -206,7 +205,7 @@ public void testFileSourceWithNonexistentFilePathDoesNotWriteToBuffer() throws I @Test public void testFileSourceWithNullFilePathThrowsNullPointerException() { pluginSettings.put(FileSourceConfig.ATTRIBUTE_PATH, null); - assertThrows(NullPointerException.class, FileSourceTests.this::createObjectUnderTest); + assertThrows(IllegalArgumentException.class, FileSourceTests.this::createObjectUnderTest); } @Test @@ -377,5 +376,183 @@ void start_will_throw_exception_if_codec_throws() throws IOException, TimeoutExc verifyNoInteractions(buffer); } + @Test + void start_codec_consumer_wraps_timeout_exception() throws IOException, TimeoutException { + doThrow(new TimeoutException("buffer full")) + .when(buffer).write(any(Record.class), eq(FileSourceConfig.DEFAULT_TIMEOUT)); + + createObjectUnderTest().start(buffer); + + final ArgumentCaptor consumerArgumentCaptor = ArgumentCaptor.forClass(Consumer.class); + + await().atMost(2, TimeUnit.SECONDS) + .untilAsserted(() -> verify(inputCodec).parse(any(InputStream.class), any(Consumer.class))); + + verify(inputCodec).parse(any(InputStream.class), consumerArgumentCaptor.capture()); + + final Consumer> actualConsumer = consumerArgumentCaptor.getValue(); + final Record record = mock(Record.class); + + assertThrows(RuntimeException.class, () -> actualConsumer.accept(record)); + } + } + + @Nested + class TailMode { + + @Mock + private Buffer> buffer; + + @Test + void start_in_tail_mode_invokes_startTailing() throws Exception { + pluginSettings.put("tail", true); + pluginSettings.put("paths", List.of("/tmp/nonexistent-test-glob-*.log")); + pluginSettings.remove(FileSourceConfig.ATTRIBUTE_PATH); + pluginSettings.put(FileSourceConfig.ATTRIBUTE_PATH, "/tmp/nonexistent-test-glob-single.log"); + + FileSource fileSource = createObjectUnderTest(); + fileSource.start(buffer); + Thread.sleep(200); + fileSource.stop(); + } + + @Test + void start_in_tail_mode_with_codec_invokes_startTailing() throws Exception { + pluginSettings.put("tail", true); + pluginSettings.put("paths", List.of("/tmp/nonexistent-codec-glob-*.log")); + pluginSettings.put(FileSourceConfig.ATTRIBUTE_PATH, "/tmp/nonexistent-codec-glob-single.log"); + + Map codecConfiguration = Map.of(UUID.randomUUID().toString(), UUID.randomUUID().toString()); + Map> codecSettings = Map.of("fake_codec", codecConfiguration); + pluginSettings.put("codec", codecSettings); + + InputCodec mockCodec = mock(InputCodec.class); + when(pluginFactory.loadPlugin(eq(InputCodec.class), any(PluginSetting.class))) + .thenReturn(mockCodec); + + FileSource fileSource = createObjectUnderTest(); + fileSource.start(buffer); + Thread.sleep(200); + fileSource.stop(); + } + + @Test + void start_in_tail_mode_with_high_ratio_logs_warning() throws Exception { + pluginSettings.put("tail", true); + pluginSettings.put("paths", List.of("/tmp/nonexistent-ratio-*.log")); + pluginSettings.put(FileSourceConfig.ATTRIBUTE_PATH, "/tmp/nonexistent-ratio-single.log"); + pluginSettings.put("max_active_files", 1000); + pluginSettings.put("reader_threads", 1); + + FileSource fileSource = createObjectUnderTest(); + fileSource.start(buffer); + Thread.sleep(200); + fileSource.stop(); + } + + @Test + void start_in_tail_mode_with_checkpoint_path() throws Exception { + pluginSettings.put("tail", true); + pluginSettings.put("paths", List.of("/tmp/nonexistent-cp-*.log")); + pluginSettings.put(FileSourceConfig.ATTRIBUTE_PATH, "/tmp/nonexistent-cp-single.log"); + pluginSettings.put("checkpoint_file", "/tmp/test-checkpoint-" + UUID.randomUUID() + ".json"); + + FileSource fileSource = createObjectUnderTest(); + fileSource.start(buffer); + Thread.sleep(200); + fileSource.stop(); + } + + @Test + void stop_before_start_does_not_throw() { + FileSource fileSource = createObjectUnderTest(); + assertDoesNotThrow(fileSource::stop); + } + + @Test + void stop_after_classic_start_joins_thread() throws Exception { + FileSource fileSource = createObjectUnderTest(); + fileSource.start(buffer); + Thread.sleep(200); + fileSource.stop(); + } + + @Test + void stop_with_interrupt_during_join() throws Exception { + buffer = mock(Buffer.class); + doAnswer(inv -> { + Thread.sleep(5000); + return null; + }).when(buffer).write(any(Record.class), eq(FileSourceConfig.DEFAULT_TIMEOUT)); + + FileSource fileSource = createObjectUnderTest(); + fileSource.start(buffer); + Thread.sleep(200); + + Thread stopThread = new Thread(() -> { + Thread.currentThread().interrupt(); + fileSource.stop(); + }); + stopThread.start(); + stopThread.join(5000); + } + + @Test + void areAcknowledgementsEnabled_returns_false_by_default() { + FileSource fileSource = createObjectUnderTest(); + assertThat(fileSource.areAcknowledgementsEnabled(), equalTo(false)); + } + + @Test + void areAcknowledgementsEnabled_returns_true_when_configured() { + pluginSettings.put("acknowledgments", true); + fileSourceConfig = OBJECT_MAPPER.convertValue(pluginSettings, FileSourceConfig.class); + AcknowledgementSetManager mockAckManager = mock(AcknowledgementSetManager.class); + FileSource fileSource = new FileSource(fileSourceConfig, pluginMetrics, pluginFactory, + TestEventFactory.getTestEventFactory(), mockAckManager); + assertThat(fileSource.areAcknowledgementsEnabled(), equalTo(true)); + } + + @Test + void start_in_tail_mode_with_safe_ratio_does_not_warn() throws Exception { + pluginSettings.put("tail", true); + pluginSettings.put("paths", List.of("/tmp/nonexistent-ratio-safe-*.log")); + pluginSettings.put(FileSourceConfig.ATTRIBUTE_PATH, "/tmp/nonexistent-ratio-safe-single.log"); + pluginSettings.put("max_active_files", 100); + pluginSettings.put("reader_threads", 2); + + FileSource fileSource = createObjectUnderTest(); + fileSource.start(buffer); + Thread.sleep(200); + fileSource.stop(); + } + + @Test + void stop_mid_read_stops_processing_lines() throws Exception { + pluginSettings.put(FileSourceConfig.ATTRIBUTE_TYPE, FileSourceConfig.EVENT_TYPE); + pluginSettings.put(FileSourceConfig.ATTRIBUTE_PATH, TEST_FILE_PATH_PLAIN); + + buffer = mock(Buffer.class); + doAnswer(inv -> { + Thread.sleep(2000); + return null; + }).when(buffer).write(any(Record.class), eq(FileSourceConfig.DEFAULT_TIMEOUT)); + + FileSource fileSource = createObjectUnderTest(); + fileSource.start(buffer); + Thread.sleep(100); + fileSource.stop(); + Thread.sleep(500); + } + + @Test + void writeLineAsEventOrString_with_non_matching_type_does_not_write() throws Exception { + pluginSettings.put(FileSourceConfig.ATTRIBUTE_TYPE, FileSourceConfig.EVENT_TYPE); + pluginSettings.put(FileSourceConfig.ATTRIBUTE_PATH, TEST_FILE_PATH_PLAIN); + FileSource fileSource = createObjectUnderTest(); + fileSource.start(buffer); + Thread.sleep(500); + fileSource.stop(); + } } } diff --git a/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/FileTailMetricsTest.java b/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/FileTailMetricsTest.java new file mode 100644 index 0000000000..52d3f2b311 --- /dev/null +++ b/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/FileTailMetricsTest.java @@ -0,0 +1,221 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + +package org.opensearch.dataprepper.plugins.source.file; + +import io.micrometer.core.instrument.Counter; +import io.micrometer.core.instrument.Timer; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.dataprepper.metrics.PluginMetrics; + +import java.util.concurrent.atomic.AtomicLong; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.instanceOf; +import static org.hamcrest.Matchers.notNullValue; +import static org.mockito.Mockito.when; + +@ExtendWith(MockitoExtension.class) +class FileTailMetricsTest { + + @Mock + private PluginMetrics pluginMetrics; + + @Mock + private Counter linesReadCounter; + + @Mock + private Counter bytesReadCounter; + + @Mock + private Counter linesTruncatedCounter; + + @Mock + private Counter filesOpenedCounter; + + @Mock + private Counter filesClosedCounter; + + @Mock + private Counter filesRotatedCounter; + + @Mock + private Counter readErrorsCounter; + + @Mock + private Counter writeTimeoutsCounter; + + @Mock + private Counter checkpointFlushesCounter; + + @Mock + private Counter checkpointErrorsCounter; + + @Mock + private Counter eventsEmittedCounter; + + @Mock + private Counter dataLossEventsCounter; + + @Mock + private Counter acknowledgmentFailuresCounter; + + @Mock + private Timer backpressureTimer; + + private FileTailMetrics fileTailMetrics; + + @BeforeEach + void setUp() { + when(pluginMetrics.counter("tailLinesRead")).thenReturn(linesReadCounter); + when(pluginMetrics.counter("tailBytesRead")).thenReturn(bytesReadCounter); + when(pluginMetrics.counter("tailLinesTruncated")).thenReturn(linesTruncatedCounter); + when(pluginMetrics.counter("tailFilesOpened")).thenReturn(filesOpenedCounter); + when(pluginMetrics.counter("tailFilesClosed")).thenReturn(filesClosedCounter); + when(pluginMetrics.counter("tailFilesRotated")).thenReturn(filesRotatedCounter); + when(pluginMetrics.counter("tailReadErrors")).thenReturn(readErrorsCounter); + when(pluginMetrics.counter("tailWriteTimeouts")).thenReturn(writeTimeoutsCounter); + when(pluginMetrics.counter("tailCheckpointFlushes")).thenReturn(checkpointFlushesCounter); + when(pluginMetrics.counter("tailCheckpointErrors")).thenReturn(checkpointErrorsCounter); + when(pluginMetrics.counter("tailEventsEmitted")).thenReturn(eventsEmittedCounter); + when(pluginMetrics.counter("tailDataLossEvents")).thenReturn(dataLossEventsCounter); + when(pluginMetrics.counter("tailAcknowledgmentFailures")).thenReturn(acknowledgmentFailuresCounter); + when(pluginMetrics.timer("tailBackpressureTime")).thenReturn(backpressureTimer); + + fileTailMetrics = new FileTailMetrics(pluginMetrics); + } + + @Test + void getLinesReadReturnsRegisteredCounter() { + assertThat(fileTailMetrics.getLinesRead(), equalTo(linesReadCounter)); + } + + @Test + void getBytesReadReturnsRegisteredCounter() { + assertThat(fileTailMetrics.getBytesRead(), equalTo(bytesReadCounter)); + } + + @Test + void getLinesTruncatedReturnsRegisteredCounter() { + assertThat(fileTailMetrics.getLinesTruncated(), equalTo(linesTruncatedCounter)); + } + + @Test + void getFilesOpenedReturnsRegisteredCounter() { + assertThat(fileTailMetrics.getFilesOpened(), equalTo(filesOpenedCounter)); + } + + @Test + void getFilesClosedReturnsRegisteredCounter() { + assertThat(fileTailMetrics.getFilesClosed(), equalTo(filesClosedCounter)); + } + + @Test + void getFilesRotatedReturnsRegisteredCounter() { + assertThat(fileTailMetrics.getFilesRotated(), equalTo(filesRotatedCounter)); + } + + @Test + void getReadErrorsReturnsRegisteredCounter() { + assertThat(fileTailMetrics.getReadErrors(), equalTo(readErrorsCounter)); + } + + @Test + void getWriteTimeoutsReturnsRegisteredCounter() { + assertThat(fileTailMetrics.getWriteTimeouts(), equalTo(writeTimeoutsCounter)); + } + + @Test + void getCheckpointFlushesReturnsRegisteredCounter() { + assertThat(fileTailMetrics.getCheckpointFlushes(), equalTo(checkpointFlushesCounter)); + } + + @Test + void getCheckpointErrorsReturnsRegisteredCounter() { + assertThat(fileTailMetrics.getCheckpointErrors(), equalTo(checkpointErrorsCounter)); + } + + @Test + void getActiveFileCountReturnsAtomicLong() { + assertThat(fileTailMetrics.getActiveFileCount(), notNullValue()); + assertThat(fileTailMetrics.getActiveFileCount(), instanceOf(AtomicLong.class)); + } + + @Test + void activeFileCountInitializesToZero() { + assertThat(fileTailMetrics.getActiveFileCount().get(), equalTo(0L)); + } + + @Test + void activeFileCountCanBeIncremented() { + fileTailMetrics.getActiveFileCount().incrementAndGet(); + + assertThat(fileTailMetrics.getActiveFileCount().get(), equalTo(1L)); + } + + @Test + void activeFileCountCanBeDecrementedAfterIncrement() { + fileTailMetrics.getActiveFileCount().incrementAndGet(); + fileTailMetrics.getActiveFileCount().incrementAndGet(); + fileTailMetrics.getActiveFileCount().decrementAndGet(); + + assertThat(fileTailMetrics.getActiveFileCount().get(), equalTo(1L)); + } + + @Test + void allCounterGettersReturnInstanceOfCounter() { + assertThat(fileTailMetrics.getLinesRead(), instanceOf(Counter.class)); + assertThat(fileTailMetrics.getBytesRead(), instanceOf(Counter.class)); + assertThat(fileTailMetrics.getLinesTruncated(), instanceOf(Counter.class)); + assertThat(fileTailMetrics.getFilesOpened(), instanceOf(Counter.class)); + assertThat(fileTailMetrics.getFilesClosed(), instanceOf(Counter.class)); + assertThat(fileTailMetrics.getFilesRotated(), instanceOf(Counter.class)); + assertThat(fileTailMetrics.getReadErrors(), instanceOf(Counter.class)); + assertThat(fileTailMetrics.getWriteTimeouts(), instanceOf(Counter.class)); + assertThat(fileTailMetrics.getCheckpointFlushes(), instanceOf(Counter.class)); + assertThat(fileTailMetrics.getCheckpointErrors(), instanceOf(Counter.class)); + } + + @Test + void getEventsEmittedReturnsRegisteredCounter() { + assertThat(fileTailMetrics.getEventsEmitted(), equalTo(eventsEmittedCounter)); + } + + @Test + void getBackpressureTimerReturnsRegisteredTimer() { + assertThat(fileTailMetrics.getBackpressureTimer(), equalTo(backpressureTimer)); + } + + @Test + void getFileLagBytesReturnsAtomicLong() { + assertThat(fileTailMetrics.getFileLagBytes(), notNullValue()); + assertThat(fileTailMetrics.getFileLagBytes(), instanceOf(AtomicLong.class)); + } + + @Test + void fileLagBytesInitializesToZero() { + assertThat(fileTailMetrics.getFileLagBytes().get(), equalTo(0L)); + } + + @Test + void getDataLossEventsReturnsRegisteredCounter() { + assertThat(fileTailMetrics.getDataLossEvents(), equalTo(dataLossEventsCounter)); + } + + @Test + void getAcknowledgmentFailuresReturnsRegisteredCounter() { + assertThat(fileTailMetrics.getAcknowledgmentFailures(), equalTo(acknowledgmentFailuresCounter)); + } +} diff --git a/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/GlobPathResolverTest.java b/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/GlobPathResolverTest.java new file mode 100644 index 0000000000..7bdd1d2b22 --- /dev/null +++ b/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/GlobPathResolverTest.java @@ -0,0 +1,259 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + +package org.opensearch.dataprepper.plugins.source.file; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; + +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.FileVisitResult; +import java.nio.file.SimpleFileVisitor; +import java.nio.file.attribute.BasicFileAttributes; +import java.util.Collections; +import java.util.List; +import java.util.Set; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.empty; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.greaterThanOrEqualTo; +import static org.hamcrest.Matchers.hasItem; +import static org.hamcrest.Matchers.hasSize; +import static org.hamcrest.Matchers.not; +import static org.hamcrest.Matchers.notNullValue; +import static org.junit.jupiter.api.Assertions.assertThrows; + +class GlobPathResolverTest { + + @TempDir + Path tempDir; + + @BeforeEach + void setUp() throws IOException { + Files.createDirectories(tempDir.resolve("subdir")); + Files.createFile(tempDir.resolve("app.log")); + Files.createFile(tempDir.resolve("error.log")); + Files.createFile(tempDir.resolve("app.txt")); + Files.createFile(tempDir.resolve("subdir/nested.log")); + } + + @Test + void resolve_matches_glob_pattern_for_log_files() { + final GlobPathResolver resolver = new GlobPathResolver( + List.of(tempDir.toString() + "/*.log"), + Collections.emptyList()); + + final Set result = resolver.resolve(); + + assertThat(result, hasSize(2)); + assertThat(result, hasItem(tempDir.resolve("app.log").toAbsolutePath().normalize())); + assertThat(result, hasItem(tempDir.resolve("error.log").toAbsolutePath().normalize())); + } + + @Test + void resolve_excludes_files_matching_exclude_patterns() { + final GlobPathResolver resolver = new GlobPathResolver( + List.of(tempDir.toString() + "/*.log"), + List.of(tempDir.toString() + "/error.*")); + + final Set result = resolver.resolve(); + + assertThat(result, hasSize(1)); + assertThat(result, hasItem(tempDir.resolve("app.log").toAbsolutePath().normalize())); + } + + @Test + void resolve_matches_recursive_double_star_pattern() { + final GlobPathResolver resolver = new GlobPathResolver( + List.of(tempDir.toString() + "/" + "**/*.log"), + Collections.emptyList()); + + final Set result = resolver.resolve(); + + assertThat(result, hasItem(tempDir.resolve("subdir/nested.log").toAbsolutePath().normalize())); + } + + @Test + void resolve_returns_empty_set_when_no_files_match() { + final GlobPathResolver resolver = new GlobPathResolver( + List.of(tempDir.toString() + "/*.csv"), + Collections.emptyList()); + + final Set result = resolver.resolve(); + + assertThat(result, empty()); + } + + @Test + void matches_returns_true_for_matching_path() { + final GlobPathResolver resolver = new GlobPathResolver( + List.of(tempDir.toString() + "/*.log"), + Collections.emptyList()); + + assertThat(resolver.matches(tempDir.resolve("app.log")), equalTo(true)); + } + + @Test + void matches_returns_false_for_non_matching_path() { + final GlobPathResolver resolver = new GlobPathResolver( + List.of(tempDir.toString() + "/*.log"), + Collections.emptyList()); + + assertThat(resolver.matches(tempDir.resolve("app.txt")), equalTo(false)); + } + + @Test + void matches_returns_false_for_excluded_path() { + final GlobPathResolver resolver = new GlobPathResolver( + List.of(tempDir.toString() + "/*.log"), + List.of(tempDir.toString() + "/error.*")); + + assertThat(resolver.matches(tempDir.resolve("error.log")), equalTo(false)); + } + + @Test + void getWatchDirectories_returns_base_directories() { + final GlobPathResolver resolver = new GlobPathResolver( + List.of(tempDir.toString() + "/*.log"), + Collections.emptyList()); + + final Set watchDirs = resolver.getWatchDirectories(); + + assertThat(watchDirs, hasSize(1)); + assertThat(watchDirs, hasItem(tempDir.toAbsolutePath().normalize())); + } + + @Test + void extractBaseDirectory_stops_at_first_wildcard() { + final Path baseDir = GlobPathResolver.extractBaseDirectory(tempDir.toString() + "/logs/*.log"); + final Path expected = tempDir.resolve("logs").toAbsolutePath().normalize(); + final Path expectedParent = expected.getParent(); + + assertThat(baseDir, notNullValue()); + assertThat(baseDir.toString().startsWith(tempDir.toAbsolutePath().normalize().toString()), equalTo(true)); + } + + @Test + void constructor_throws_on_invalid_glob_pattern() { + assertThrows(IllegalArgumentException.class, () -> + new GlobPathResolver(List.of(tempDir.toString() + "/[invalid"), Collections.emptyList())); + } + + @Test + void resolve_handles_null_exclude_patterns() { + final GlobPathResolver resolver = new GlobPathResolver( + List.of(tempDir.toString() + "/*.log"), + null); + + final Set result = resolver.resolve(); + + assertThat(result, hasSize(2)); + } + + @Test + void resolve_handles_multiple_include_patterns() { + final GlobPathResolver resolver = new GlobPathResolver( + List.of(tempDir.toString() + "/*.log", tempDir.toString() + "/*.txt"), + Collections.emptyList()); + + final Set result = resolver.resolve(); + + assertThat(result, hasSize(greaterThanOrEqualTo(3))); + } + + @Test + void getWatchDirectories_returns_multiple_directories_for_multiple_patterns() { + final GlobPathResolver resolver = new GlobPathResolver( + List.of(tempDir.toString() + "/*.log", tempDir.toString() + "/subdir/*.log"), + Collections.emptyList()); + + final Set watchDirs = resolver.getWatchDirectories(); + + assertThat(watchDirs.size(), greaterThanOrEqualTo(1)); + } + + @Test + void resolve_returns_empty_set_for_nonexistent_base_directory() { + Path nonexistent = tempDir.resolve("nonexistent-subdir"); + final GlobPathResolver resolver = new GlobPathResolver( + List.of(nonexistent.toString() + "/*.log"), + Collections.emptyList()); + + final Set result = resolver.resolve(); + + assertThat(result, empty()); + } + + @Test + void resolve_warns_when_base_directory_does_not_exist() { + String nonexistentPath = "/nonexistent-dir-" + System.nanoTime() + "/sub/deep/*.log"; + final GlobPathResolver resolver = new GlobPathResolver( + List.of(nonexistentPath), + Collections.emptyList()); + + final Set result = resolver.resolve(); + + assertThat(result, empty()); + } + + @Test + void resolve_handles_visitFileFailed_gracefully() throws IOException { + Path unreadableDir = tempDir.resolve("unreadable"); + Files.createDirectory(unreadableDir); + Files.createFile(unreadableDir.resolve("secret.log")); + unreadableDir.toFile().setReadable(false); + + final GlobPathResolver resolver = new GlobPathResolver( + List.of(tempDir.toString() + "/" + "**/*.log"), + Collections.emptyList()); + + final Set result = resolver.resolve(); + + unreadableDir.toFile().setReadable(true); + + assertThat(result, not(hasItem(unreadableDir.resolve("secret.log").toAbsolutePath().normalize()))); + } + + @Test + void walkDirectory_handles_ioException_from_walkFileTree() throws IOException { + Path dir = tempDir.resolve("walk-test"); + Files.createDirectory(dir); + Files.createFile(dir.resolve("file.log")); + + final GlobPathResolver resolver = new GlobPathResolver( + List.of(dir.toString() + "/*.log"), + Collections.emptyList()); + + SimpleFileVisitor throwingVisitor = new SimpleFileVisitor<>() { + @Override + public FileVisitResult preVisitDirectory(Path dir, BasicFileAttributes attrs) throws IOException { + throw new IOException("simulated walk error"); + } + }; + + resolver.walkDirectory(dir, throwingVisitor); + } + + @Test + void extractBaseDirectory_with_no_separator_in_pattern() { + final Path baseDir = GlobPathResolver.extractBaseDirectory("/*.log"); + assertThat(baseDir, notNullValue()); + } + + @Test + void extractBaseDirectory_with_nonexistent_path_returns_parent_as_fallback() { + final Path baseDir = GlobPathResolver.extractBaseDirectory("/nonexistent-test-dir-" + System.nanoTime() + "/data.log"); + assertThat(baseDir, notNullValue()); + } +} diff --git a/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/PendingFileTest.java b/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/PendingFileTest.java new file mode 100644 index 0000000000..0b2654244f --- /dev/null +++ b/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/PendingFileTest.java @@ -0,0 +1,83 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + +package org.opensearch.dataprepper.plugins.source.file; + +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; + +import java.nio.file.Path; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.greaterThan; +import static org.hamcrest.Matchers.greaterThanOrEqualTo; +import static org.hamcrest.Matchers.lessThanOrEqualTo; +import static org.hamcrest.Matchers.notNullValue; +import static org.junit.jupiter.api.Assertions.assertThrows; + +@ExtendWith(MockitoExtension.class) +class PendingFileTest { + + @Mock + private FileIdentity fileIdentity; + + @Test + void constructorSetsFileIdentity() { + final Path path = Path.of("/var/log/test.log"); + final PendingFile pendingFile = new PendingFile(fileIdentity, path); + + assertThat(pendingFile.getFileIdentity(), equalTo(fileIdentity)); + } + + @Test + void constructorSetsPath() { + final Path path = Path.of("/var/log/test.log"); + final PendingFile pendingFile = new PendingFile(fileIdentity, path); + + assertThat(pendingFile.getPath(), equalTo(path)); + } + + @Test + void constructorSetsEnqueuedTimeMillis() { + final long before = System.currentTimeMillis(); + final PendingFile pendingFile = new PendingFile(fileIdentity, Path.of("/tmp/file.log")); + final long after = System.currentTimeMillis(); + + assertThat(pendingFile.getEnqueuedTimeMillis(), greaterThan(0L)); + assertThat(pendingFile.getEnqueuedTimeMillis(), greaterThanOrEqualTo(before)); + assertThat(pendingFile.getEnqueuedTimeMillis(), lessThanOrEqualTo(after)); + } + + @Test + void constructorThrowsWhenFileIdentityIsNull() { + assertThrows(NullPointerException.class, () -> new PendingFile(null, Path.of("/tmp/test.log"))); + } + + @Test + void constructorThrowsWhenPathIsNull() { + assertThrows(NullPointerException.class, () -> new PendingFile(fileIdentity, null)); + } + + @Test + void toStringContainsPathAndIdentity() { + final Path path = Path.of("/var/log/app.log"); + final PendingFile pendingFile = new PendingFile(fileIdentity, path); + + final String result = pendingFile.toString(); + + assertThat(result, notNullValue()); + assertThat(result, containsString("path=")); + assertThat(result, containsString("identity=")); + } +} diff --git a/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/RotationDetectorTest.java b/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/RotationDetectorTest.java new file mode 100644 index 0000000000..ac5faea5cd --- /dev/null +++ b/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/RotationDetectorTest.java @@ -0,0 +1,283 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + +package org.opensearch.dataprepper.plugins.source.file; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.io.TempDir; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; + +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.NoSuchFileException; +import java.nio.file.Path; +import java.nio.file.attribute.BasicFileAttributes; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.notNullValue; +import static org.hamcrest.Matchers.nullValue; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.lenient; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; +import static org.junit.jupiter.api.Assertions.assertThrows; + +@ExtendWith(MockitoExtension.class) +class RotationDetectorTest { + + private static final int FINGERPRINT_BYTES = 256; + + @TempDir + Path tempDir; + + @Mock + private FileSystemOperations fileOps; + + private RotationDetector rotationDetector; + + @BeforeEach + void setUp() { + rotationDetector = new RotationDetector(fileOps, FINGERPRINT_BYTES); + lenient().when(fileOps.exists(any())).thenReturn(true); + } + + @Test + void getFingerprintBytesReturnsConfiguredValue() { + assertThat(rotationDetector.getFingerprintBytes(), equalTo(FINGERPRINT_BYTES)); + } + + @Test + void noRotationWhenIdentityMatchesAndSizeNotShrunk() throws IOException { + final Path testFile = tempDir.resolve("app.log"); + Files.writeString(testFile, "log data"); + + final BasicFileAttributes attrs = mock(BasicFileAttributes.class); + when(attrs.fileKey()).thenReturn("inode-42"); + when(fileOps.readAttributes(testFile)).thenReturn(attrs); + when(fileOps.size(testFile)).thenReturn(100L); + + final FileIdentity knownIdentity = FileIdentity.from(testFile, fileOps, FINGERPRINT_BYTES); + + final RotationResult result = rotationDetector.checkRotation(testFile, knownIdentity, 50L); + + assertThat(result.getRotationType(), equalTo(RotationType.NO_ROTATION)); + assertThat(result.getNewFileIdentity(), nullValue()); + } + + @Test + void createRenameRotationWhenIdentityChanges() throws IOException { + final Path testFile = tempDir.resolve("app.log"); + Files.writeString(testFile, "original"); + + final BasicFileAttributes oldAttrs = mock(BasicFileAttributes.class); + when(oldAttrs.fileKey()).thenReturn("inode-1"); + when(fileOps.readAttributes(testFile)).thenReturn(oldAttrs); + final FileIdentity knownIdentity = FileIdentity.from(testFile, fileOps, FINGERPRINT_BYTES); + + final BasicFileAttributes newAttrs = mock(BasicFileAttributes.class); + when(newAttrs.fileKey()).thenReturn("inode-2"); + when(fileOps.readAttributes(testFile)).thenReturn(newAttrs); + + final RotationResult result = rotationDetector.checkRotation(testFile, knownIdentity, 100L); + + assertThat(result.getRotationType(), equalTo(RotationType.CREATE_RENAME)); + assertThat(result.getNewFileIdentity(), notNullValue()); + } + + @Test + void copytruncateRotationWhenFileSizeShrinks() throws IOException { + final Path testFile = tempDir.resolve("app.log"); + Files.writeString(testFile, "data"); + + final BasicFileAttributes attrs = mock(BasicFileAttributes.class); + when(attrs.fileKey()).thenReturn("inode-5"); + when(fileOps.readAttributes(testFile)).thenReturn(attrs); + + final FileIdentity knownIdentity = FileIdentity.from(testFile, fileOps, FINGERPRINT_BYTES); + + when(fileOps.size(testFile)).thenReturn(10L); + + final RotationResult result = rotationDetector.checkRotation(testFile, knownIdentity, 500L); + + assertThat(result.getRotationType(), equalTo(RotationType.COPYTRUNCATE)); + assertThat(result.getNewFileIdentity(), notNullValue()); + } + + @Test + void deletedFileWhenWrappedNoSuchFileExceptionFromIdentityResolution() throws IOException { + final Path testFile = tempDir.resolve("gone.log"); + when(fileOps.exists(testFile)).thenReturn(false); + + final FileIdentity knownIdentity = mock(FileIdentity.class); + + final RotationResult result = rotationDetector.checkRotation(testFile, knownIdentity, 100L); + + assertThat(result.getRotationType(), equalTo(RotationType.DELETED)); + } + + @Test + void deletedFileWhenSizeCheckThrowsNoSuchFileException() throws IOException { + final Path testFile = tempDir.resolve("removed.log"); + Files.writeString(testFile, "temp"); + + final BasicFileAttributes attrs = mock(BasicFileAttributes.class); + when(attrs.fileKey()).thenReturn("inode-10"); + when(fileOps.readAttributes(testFile)).thenReturn(attrs); + + final FileIdentity knownIdentity = FileIdentity.from(testFile, fileOps, FINGERPRINT_BYTES); + + when(fileOps.size(testFile)).thenThrow(new NoSuchFileException(testFile.toString())); + + final RotationResult result = rotationDetector.checkRotation(testFile, knownIdentity, 100L); + + assertThat(result.getRotationType(), equalTo(RotationType.DELETED)); + } + + @Test + void noRotationWhenSizeCheckThrowsGenericIOException() throws IOException { + final Path testFile = tempDir.resolve("error.log"); + Files.writeString(testFile, "content"); + + final BasicFileAttributes attrs = mock(BasicFileAttributes.class); + when(attrs.fileKey()).thenReturn("inode-20"); + when(fileOps.readAttributes(testFile)).thenReturn(attrs); + + final FileIdentity knownIdentity = FileIdentity.from(testFile, fileOps, FINGERPRINT_BYTES); + + when(fileOps.size(testFile)).thenThrow(new IOException("disk error")); + + final RotationResult result = rotationDetector.checkRotation(testFile, knownIdentity, 100L); + + assertThat(result.getRotationType(), equalTo(RotationType.NO_ROTATION)); + } + + @Test + void noRotationWhenReadAttributesThrowsGenericRuntimeException() throws IOException { + final Path testFile = tempDir.resolve("runtime-err.log"); + + when(fileOps.readAttributes(testFile)).thenThrow(new RuntimeException("unexpected")); + + final FileIdentity knownIdentity = mock(FileIdentity.class); + + final RotationResult result = rotationDetector.checkRotation(testFile, knownIdentity, 100L); + + assertThat(result.getRotationType(), equalTo(RotationType.NO_ROTATION)); + } + + @Test + void deletedFileWhenWrappedNoSuchFileExceptionInCause() throws IOException { + final Path testFile = tempDir.resolve("wrapped.log"); + when(fileOps.exists(testFile)).thenReturn(false); + + final FileIdentity knownIdentity = mock(FileIdentity.class); + + final RotationResult result = rotationDetector.checkRotation(testFile, knownIdentity, 100L); + + assertThat(result.getRotationType(), equalTo(RotationType.DELETED)); + } + + @Test + void noRotationWhenSizeEqualsCurrentOffset() throws IOException { + final Path testFile = tempDir.resolve("exact.log"); + Files.writeString(testFile, "data"); + + final BasicFileAttributes attrs = mock(BasicFileAttributes.class); + when(attrs.fileKey()).thenReturn("inode-30"); + when(fileOps.readAttributes(testFile)).thenReturn(attrs); + + final FileIdentity knownIdentity = FileIdentity.from(testFile, fileOps, FINGERPRINT_BYTES); + + when(fileOps.size(testFile)).thenReturn(100L); + + final RotationResult result = rotationDetector.checkRotation(testFile, knownIdentity, 100L); + + assertThat(result.getRotationType(), equalTo(RotationType.NO_ROTATION)); + } + + @Test + void deletedFileWhenDeeplyNestedNoSuchFileException() throws IOException { + final Path testFile = tempDir.resolve("deep-nested.log"); + when(fileOps.exists(testFile)).thenReturn(false); + + final FileIdentity knownIdentity = mock(FileIdentity.class); + + final RotationResult result = rotationDetector.checkRotation(testFile, knownIdentity, 100L); + + assertThat(result.getRotationType(), equalTo(RotationType.DELETED)); + } + + @Test + void isCausedByNoSuchFile_returns_true_when_exception_itself_is_NoSuchFileException() { + final NoSuchFileException noSuchFile = new NoSuchFileException("test.log"); + + assertThat(rotationDetector.isCausedByNoSuchFile(noSuchFile), equalTo(true)); + } + + @Test + void constructorThrowsIllegalArgumentExceptionWhenFingerprintBytesIsZero() { + assertThrows(IllegalArgumentException.class, + () -> new RotationDetector(fileOps, 0)); + } + + @Test + void constructorThrowsIllegalArgumentExceptionWhenFingerprintBytesIsNegative() { + assertThrows(IllegalArgumentException.class, + () -> new RotationDetector(fileOps, -5)); + } + + @Test + void checkRotationReturnsDeletedWhenFileDoesNotExist() { + final Path testFile = tempDir.resolve("not-exists.log"); + when(fileOps.exists(testFile)).thenReturn(false); + + final FileIdentity knownIdentity = mock(FileIdentity.class); + + final RotationResult result = rotationDetector.checkRotation(testFile, knownIdentity, 100L); + + assertThat(result.getRotationType(), equalTo(RotationType.DELETED)); + } + + @Test + void isCausedByNoSuchFile_returns_true_when_deeply_nested_in_cause_chain() { + final NoSuchFileException noSuchFile = new NoSuchFileException("deep.log"); + final RuntimeException mid = new RuntimeException("mid", noSuchFile); + final RuntimeException outer = new RuntimeException("outer", mid); + + assertThat(rotationDetector.isCausedByNoSuchFile(outer), equalTo(true)); + } + + @Test + void isCausedByNoSuchFile_returns_false_when_no_NoSuchFileException_in_chain() { + final IOException ioException = new IOException("generic"); + final RuntimeException outer = new RuntimeException("outer", ioException); + + assertThat(rotationDetector.isCausedByNoSuchFile(outer), equalTo(false)); + } + + @Test + void checkRotationReturnsDeletedWhenRuntimeExceptionWrapsNoSuchFileExceptionDeeply() throws IOException { + final Path testFile = tempDir.resolve("deep-cause.log"); + + final NoSuchFileException noSuchFile = new NoSuchFileException(testFile.toString()); + final RuntimeException mid = new RuntimeException("mid", noSuchFile); + final RuntimeException outer = new RuntimeException("outer", mid); + when(fileOps.readAttributes(testFile)).thenThrow(outer); + + final FileIdentity knownIdentity = mock(FileIdentity.class); + + final RotationResult result = rotationDetector.checkRotation(testFile, knownIdentity, 100L); + + assertThat(result.getRotationType(), equalTo(RotationType.DELETED)); + } +} diff --git a/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/RotationResultTest.java b/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/RotationResultTest.java new file mode 100644 index 0000000000..5412db72be --- /dev/null +++ b/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/RotationResultTest.java @@ -0,0 +1,93 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + +package org.opensearch.dataprepper.plugins.source.file; + +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.notNullValue; +import static org.hamcrest.Matchers.nullValue; + +@ExtendWith(MockitoExtension.class) +class RotationResultTest { + + @Mock + private FileIdentity fileIdentity; + + @Test + void noRotationConstantHasCorrectType() { + assertThat(RotationResult.NO_ROTATION.getRotationType(), equalTo(RotationType.NO_ROTATION)); + } + + @Test + void noRotationConstantHasNullNewFileIdentity() { + assertThat(RotationResult.NO_ROTATION.getNewFileIdentity(), nullValue()); + } + + @Test + void deletedConstantHasCorrectType() { + assertThat(RotationResult.DELETED.getRotationType(), equalTo(RotationType.DELETED)); + } + + @Test + void deletedConstantHasNullNewFileIdentity() { + assertThat(RotationResult.DELETED.getNewFileIdentity(), nullValue()); + } + + @Test + void constructorSetsRotationType() { + final RotationResult result = new RotationResult(RotationType.CREATE_RENAME, fileIdentity); + + assertThat(result.getRotationType(), equalTo(RotationType.CREATE_RENAME)); + } + + @Test + void constructorSetsNewFileIdentity() { + final RotationResult result = new RotationResult(RotationType.COPYTRUNCATE, fileIdentity); + + assertThat(result.getNewFileIdentity(), equalTo(fileIdentity)); + } + + @Test + void constructorAllowsNullNewFileIdentity() { + final RotationResult result = new RotationResult(RotationType.NO_ROTATION, null); + + assertThat(result.getNewFileIdentity(), nullValue()); + } + + @Test + void toStringContainsRotationType() { + final RotationResult result = new RotationResult(RotationType.CREATE_RENAME, fileIdentity); + + assertThat(result.toString(), notNullValue()); + assertThat(result.toString(), containsString("CREATE_RENAME")); + } + + @Test + void toStringContainsNewIdentityWhenPresent() { + final RotationResult result = new RotationResult(RotationType.CREATE_RENAME, fileIdentity); + + assertThat(result.toString(), containsString("newIdentity=")); + } + + @Test + void toStringOmitsNewIdentityWhenNull() { + final RotationResult result = new RotationResult(RotationType.NO_ROTATION, null); + + final String str = result.toString(); + assertThat(str.contains("newIdentity="), equalTo(false)); + } +} diff --git a/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/RotationTypeTest.java b/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/RotationTypeTest.java new file mode 100644 index 0000000000..a1fd41785d --- /dev/null +++ b/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/RotationTypeTest.java @@ -0,0 +1,52 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + +package org.opensearch.dataprepper.plugins.source.file; + +import org.junit.jupiter.api.Test; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.notNullValue; + +class RotationTypeTest { + + @Test + void noRotationEnumValueExists() { + assertThat(RotationType.valueOf("NO_ROTATION"), equalTo(RotationType.NO_ROTATION)); + } + + @Test + void createRenameEnumValueExists() { + assertThat(RotationType.valueOf("CREATE_RENAME"), equalTo(RotationType.CREATE_RENAME)); + } + + @Test + void copytruncateEnumValueExists() { + assertThat(RotationType.valueOf("COPYTRUNCATE"), equalTo(RotationType.COPYTRUNCATE)); + } + + @Test + void deletedEnumValueExists() { + assertThat(RotationType.valueOf("DELETED"), equalTo(RotationType.DELETED)); + } + + @Test + void valuesContainsFourEntries() { + assertThat(RotationType.values().length, equalTo(4)); + } + + @Test + void allValuesAreNotNull() { + for (final RotationType type : RotationType.values()) { + assertThat(type, notNullValue()); + } + } +} diff --git a/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/StartPositionTest.java b/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/StartPositionTest.java new file mode 100644 index 0000000000..6358ee0750 --- /dev/null +++ b/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/StartPositionTest.java @@ -0,0 +1,55 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + +package org.opensearch.dataprepper.plugins.source.file; + +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.equalTo; +import static org.junit.jupiter.api.Assertions.assertThrows; + +class StartPositionTest { + + @Test + void fromString_returns_beginning_for_beginning() { + assertThat(StartPosition.fromString("beginning"), equalTo(StartPosition.BEGINNING)); + } + + @Test + void fromString_returns_end_for_end() { + assertThat(StartPosition.fromString("end"), equalTo(StartPosition.END)); + } + + @Test + void fromString_is_case_insensitive() { + assertThat(StartPosition.fromString("BEGINNING"), equalTo(StartPosition.BEGINNING)); + assertThat(StartPosition.fromString("END"), equalTo(StartPosition.END)); + } + + @ParameterizedTest + @ValueSource(strings = {"invalid", "start", "middle", ""}) + void fromString_throws_for_invalid_value(final String value) { + assertThrows(IllegalArgumentException.class, () -> StartPosition.fromString(value)); + } + + @Test + void toString_returns_name() { + assertThat(StartPosition.BEGINNING.toString(), equalTo("beginning")); + assertThat(StartPosition.END.toString(), equalTo("end")); + } + + @Test + void fromString_with_null_throws_IllegalArgumentException() { + assertThrows(IllegalArgumentException.class, () -> StartPosition.fromString(null)); + } +} diff --git a/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/TailFileReaderPoolTest.java b/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/TailFileReaderPoolTest.java new file mode 100644 index 0000000000..6a2469e805 --- /dev/null +++ b/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/TailFileReaderPoolTest.java @@ -0,0 +1,569 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + +package org.opensearch.dataprepper.plugins.source.file; + +import io.micrometer.core.instrument.Counter; +import io.micrometer.core.instrument.Timer; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.io.TempDir; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSetManager; +import org.opensearch.dataprepper.model.buffer.Buffer; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.event.EventBuilder; +import org.opensearch.dataprepper.model.event.EventFactory; +import org.opensearch.dataprepper.model.record.Record; + +import java.nio.channels.FileChannel; +import java.nio.charset.StandardCharsets; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.nio.file.StandardOpenOption; +import java.nio.file.attribute.BasicFileAttributes; +import java.time.Duration; +import java.util.Map; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.equalTo; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyLong; +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.Mockito.lenient; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +@ExtendWith(MockitoExtension.class) +class TailFileReaderPoolTest { + + @TempDir + Path tempDir; + + @Mock + private Buffer> buffer; + + @Mock + private EventFactory eventFactory; + + @Mock + private FileSystemOperations fileOps; + + @Mock + private CheckpointRegistry checkpointRegistry; + + @Mock + private FileTailMetrics metrics; + + @Mock + private RotationDetector rotationDetector; + + @Mock + private AcknowledgementSetManager acknowledgementSetManager; + + private TailFileReaderContext createReaderContext() { + return new TailFileReaderContext( + buffer, eventFactory, fileOps, metrics, rotationDetector, + acknowledgementSetManager, false, StandardCharsets.UTF_8, + 4096, 1048576, 5000, Duration.ofSeconds(5), + Duration.ofSeconds(30), StartPosition.END, false, + Duration.ofSeconds(30), 1000, + Duration.ofSeconds(5), 3, null); + } + + private TailFileReaderPool createPool(final int maxActiveFiles, final int readerThreads) { + when(metrics.getActiveFileCount()).thenReturn(new AtomicLong(0)); + return new TailFileReaderPool( + checkpointRegistry, metrics, maxActiveFiles, readerThreads, + Duration.ofMinutes(30), createReaderContext()); + } + + private TailFileReaderPool createPoolWithoutMetrics(final int maxActiveFiles, final int readerThreads) { + return new TailFileReaderPool( + checkpointRegistry, metrics, maxActiveFiles, readerThreads, + Duration.ofMinutes(30), createReaderContext()); + } + + @Test + void addFile_submits_reader_when_under_max_active_files() { + TailFileReaderPool pool = createPool(10, 2); + when(checkpointRegistry.getOrCreate(anyString())).thenReturn(new CheckpointEntry()); + + FileIdentity identity = mock(FileIdentity.class); + when(identity.toString()).thenReturn("test-identity"); + Path path = Paths.get("/tmp/test.log"); + + pool.addFile(identity, path); + + assertThat(pool.getActiveReaderCount(), equalTo(1)); + assertThat(pool.getPendingCount(), equalTo(0)); + } + + @Test + void addFile_is_idempotent_for_same_identity() { + TailFileReaderPool pool = createPool(10, 2); + when(checkpointRegistry.getOrCreate(anyString())).thenReturn(new CheckpointEntry()); + + FileIdentity identity = mock(FileIdentity.class); + when(identity.toString()).thenReturn("dup-identity"); + Path path = Paths.get("/tmp/dup.log"); + + pool.addFile(identity, path); + pool.addFile(identity, path); + + assertThat(pool.getActiveReaderCount(), equalTo(1)); + } + + @Test + void addFile_queues_pending_when_at_max_active_files() { + TailFileReaderPool pool = createPool(1, 2); + when(checkpointRegistry.getOrCreate(anyString())).thenReturn(new CheckpointEntry()); + + FileIdentity identity1 = mock(FileIdentity.class); + when(identity1.toString()).thenReturn("id-1"); + FileIdentity identity2 = mock(FileIdentity.class); + + pool.addFile(identity1, Paths.get("/tmp/file1.log")); + pool.addFile(identity2, Paths.get("/tmp/file2.log")); + + assertThat(pool.getActiveReaderCount(), equalTo(1)); + assertThat(pool.getPendingCount(), equalTo(1)); + } + + @Test + void addFile_does_not_add_pending_duplicate_to_queue() { + TailFileReaderPool pool = createPool(1, 2); + when(checkpointRegistry.getOrCreate(anyString())).thenReturn(new CheckpointEntry()); + + FileIdentity identity1 = mock(FileIdentity.class); + when(identity1.toString()).thenReturn("id-1"); + FileIdentity identity2 = mock(FileIdentity.class); + + pool.addFile(identity1, Paths.get("/tmp/file1.log")); + pool.addFile(identity2, Paths.get("/tmp/file2.log")); + pool.addFile(identity2, Paths.get("/tmp/file2.log")); + + assertThat(pool.getPendingCount(), equalTo(1)); + } + + @Test + void addFile_queues_multiple_pending_files() { + TailFileReaderPool pool = createPool(1, 2); + when(checkpointRegistry.getOrCreate(anyString())).thenReturn(new CheckpointEntry()); + + FileIdentity identity1 = mock(FileIdentity.class); + when(identity1.toString()).thenReturn("id-1"); + FileIdentity identity2 = mock(FileIdentity.class); + FileIdentity identity3 = mock(FileIdentity.class); + + pool.addFile(identity1, Paths.get("/tmp/file1.log")); + pool.addFile(identity2, Paths.get("/tmp/file2.log")); + pool.addFile(identity3, Paths.get("/tmp/file3.log")); + + assertThat(pool.getActiveReaderCount(), equalTo(1)); + assertThat(pool.getPendingCount(), equalTo(2)); + } + + @Test + void shutdown_does_not_throw() { + TailFileReaderPool pool = createPoolWithoutMetrics(10, 1); + pool.shutdown(); + } + + @Test + void getActiveReaderCount_returns_zero_initially() { + TailFileReaderPool pool = createPoolWithoutMetrics(10, 2); + assertThat(pool.getActiveReaderCount(), equalTo(0)); + } + + @Test + void getPendingCount_returns_zero_initially() { + TailFileReaderPool pool = createPoolWithoutMetrics(10, 2); + assertThat(pool.getPendingCount(), equalTo(0)); + } + + @Test + void closeInactiveReaders_removes_inactive_readers() throws Exception { + Counter filesClosed = mock(Counter.class); + lenient().when(metrics.getFilesClosed()).thenReturn(filesClosed); + lenient().when(metrics.getActiveFileCount()).thenReturn(new AtomicLong(0)); + + TailFileReaderPool pool = new TailFileReaderPool( + checkpointRegistry, metrics, 10, 2, + Duration.ofMillis(1), createReaderContext()); + + pool.closeInactiveReaders(); + + assertThat(pool.getActiveReaderCount(), equalTo(0)); + } + + @Test + void closeReaderForPath_removes_matching_reader() { + Counter filesClosed = mock(Counter.class); + lenient().when(metrics.getFilesClosed()).thenReturn(filesClosed); + TailFileReaderPool pool = createPool(10, 2); + when(checkpointRegistry.getOrCreate(anyString())).thenReturn(new CheckpointEntry()); + + FileIdentity identity = mock(FileIdentity.class); + when(identity.toString()).thenReturn("/tmp/removable.log"); + Path path = Paths.get("/tmp/removable.log"); + + pool.addFile(identity, path); + assertThat(pool.getActiveReaderCount(), equalTo(1)); + + pool.closeReaderForPath(path); + assertThat(pool.getActiveReaderCount(), equalTo(0)); + } + + @Test + void closeReaderForPath_does_nothing_when_no_match() { + TailFileReaderPool pool = createPool(10, 2); + when(checkpointRegistry.getOrCreate(anyString())).thenReturn(new CheckpointEntry()); + + FileIdentity identity = mock(FileIdentity.class); + when(identity.toString()).thenReturn("id-nomatch"); + pool.addFile(identity, Paths.get("/tmp/file1.log")); + + pool.closeReaderForPath(Paths.get("/tmp/other.log")); + assertThat(pool.getActiveReaderCount(), equalTo(1)); + } + + @Test + void shutdown_handles_interrupted_exception() throws Exception { + TailFileReaderPool pool = createPoolWithoutMetrics(10, 1); + + Thread shutdownThread = new Thread(() -> { + Thread.currentThread().interrupt(); + pool.shutdown(); + }); + shutdownThread.start(); + shutdownThread.join(5000); + + assertThat(shutdownThread.isAlive(), equalTo(false)); + } + + @Test + void onReaderComplete_with_create_rename_resubmits_reader() throws Exception { + Path testFile = tempDir.resolve("rotate.log"); + Files.writeString(testFile, "line1\n"); + + Counter filesRotated = mock(Counter.class); + Counter filesOpened = mock(Counter.class); + Counter filesClosed = mock(Counter.class); + Counter bytesRead = mock(Counter.class); + Counter linesRead = mock(Counter.class); + Counter eventsEmitted = mock(Counter.class); + Timer backpressureTimer = mock(Timer.class); + lenient().when(metrics.getFilesRotated()).thenReturn(filesRotated); + lenient().when(metrics.getFilesOpened()).thenReturn(filesOpened); + lenient().when(metrics.getFilesClosed()).thenReturn(filesClosed); + lenient().when(metrics.getBytesRead()).thenReturn(bytesRead); + lenient().when(metrics.getLinesRead()).thenReturn(linesRead); + lenient().when(metrics.getEventsEmitted()).thenReturn(eventsEmitted); + lenient().when(metrics.getBackpressureTimer()).thenReturn(backpressureTimer); + lenient().when(metrics.getFileLagBytes()).thenReturn(new AtomicLong(0)); + + BasicFileAttributes attrs = mock(BasicFileAttributes.class); + when(attrs.fileKey()).thenReturn("inode-1"); + when(fileOps.readAttributes(testFile)).thenReturn(attrs); + when(fileOps.size(testFile)).thenReturn(6L); + FileChannel realChannel = FileChannel.open(testFile, StandardOpenOption.READ); + when(fileOps.openReadChannel(testFile)).thenReturn(realChannel); + + FileIdentity newIdentity = mock(FileIdentity.class); + when(rotationDetector.checkRotation(any(), any(), any(long.class))) + .thenReturn(new RotationResult(RotationType.CREATE_RENAME, newIdentity)); + when(rotationDetector.getFingerprintBytes()).thenReturn(1024); + + EventBuilder mockBuilder = mock(EventBuilder.class); + Event mockEvent = mock(Event.class); + lenient().when(eventFactory.eventBuilder(EventBuilder.class)).thenReturn(mockBuilder); + lenient().when(mockBuilder.withEventType(any())).thenReturn(mockBuilder); + lenient().when(mockBuilder.withData(any(Map.class))).thenReturn(mockBuilder); + lenient().when(mockBuilder.build()).thenReturn(mockEvent); + + TailFileReaderPool pool = createPool(10, 2); + when(checkpointRegistry.getOrCreate(anyString())).thenReturn(new CheckpointEntry()); + + FileIdentity identity = FileIdentity.from(testFile, fileOps, 1024); + + pool.addFile(identity, testFile); + + Thread.sleep(2000); + + pool.shutdown(); + } + + @Test + void closeInactiveReaders_with_real_reader() throws Exception { + Path testFile = tempDir.resolve("inactive.log"); + Files.writeString(testFile, "data\n"); + + Counter filesOpened = mock(Counter.class); + Counter filesClosed = mock(Counter.class); + Counter bytesRead = mock(Counter.class); + Counter linesRead = mock(Counter.class); + Counter eventsEmitted = mock(Counter.class); + Timer backpressureTimer = mock(Timer.class); + lenient().when(metrics.getFilesOpened()).thenReturn(filesOpened); + lenient().when(metrics.getFilesClosed()).thenReturn(filesClosed); + lenient().when(metrics.getBytesRead()).thenReturn(bytesRead); + lenient().when(metrics.getLinesRead()).thenReturn(linesRead); + lenient().when(metrics.getEventsEmitted()).thenReturn(eventsEmitted); + lenient().when(metrics.getBackpressureTimer()).thenReturn(backpressureTimer); + lenient().when(metrics.getFileLagBytes()).thenReturn(new AtomicLong(0)); + when(metrics.getActiveFileCount()).thenReturn(new AtomicLong(0)); + + BasicFileAttributes attrs = mock(BasicFileAttributes.class); + when(attrs.fileKey()).thenReturn("inode-inactive"); + when(fileOps.readAttributes(testFile)).thenReturn(attrs); + when(fileOps.size(testFile)).thenReturn((long) "data\n".length()); + FileChannel realChannel = FileChannel.open(testFile, StandardOpenOption.READ); + when(fileOps.openReadChannel(testFile)).thenReturn(realChannel); + + when(rotationDetector.checkRotation(any(), any(), any(long.class))) + .thenReturn(RotationResult.NO_ROTATION); + + EventBuilder mockBuilder = mock(EventBuilder.class); + Event mockEvent = mock(Event.class); + lenient().when(eventFactory.eventBuilder(EventBuilder.class)).thenReturn(mockBuilder); + lenient().when(mockBuilder.withEventType(any())).thenReturn(mockBuilder); + lenient().when(mockBuilder.withData(any(Map.class))).thenReturn(mockBuilder); + lenient().when(mockBuilder.build()).thenReturn(mockEvent); + + TailFileReaderPool pool = new TailFileReaderPool( + checkpointRegistry, metrics, 10, 2, + Duration.ofMillis(1), createReaderContext()); + + when(checkpointRegistry.getOrCreate(anyString())).thenReturn(new CheckpointEntry()); + + FileIdentity identity = FileIdentity.from(testFile, fileOps, 1024); + pool.addFile(identity, testFile); + + Thread.sleep(1000); + + pool.closeInactiveReaders(); + + pool.shutdown(); + } + + @Test + void shutdown_calls_shutdownNow_when_awaitTermination_returns_false() throws Exception { + ExecutorService mockExecutor = mock(ExecutorService.class); + when(mockExecutor.awaitTermination(anyLong(), any(TimeUnit.class))).thenReturn(false); + + TailFileReaderPool pool = new TailFileReaderPool( + checkpointRegistry, metrics, 10, + Duration.ofMinutes(30), createReaderContext(), + () -> mockExecutor); + + pool.shutdown(); + } + + @Test + void shutdown_calls_shutdownNow_when_awaitTermination_throws_interrupted() throws Exception { + ExecutorService mockExecutor = mock(ExecutorService.class); + when(mockExecutor.awaitTermination(anyLong(), any(TimeUnit.class))) + .thenThrow(new InterruptedException("test")); + + TailFileReaderPool pool = new TailFileReaderPool( + checkpointRegistry, metrics, 10, + Duration.ofMinutes(30), createReaderContext(), + () -> mockExecutor); + + pool.shutdown(); + + assertThat(Thread.currentThread().isInterrupted(), equalTo(true)); + Thread.interrupted(); + } + + @Test + void closeInactiveReaders_removes_reader_past_threshold() throws Exception { + Counter filesClosed = mock(Counter.class); + when(metrics.getFilesClosed()).thenReturn(filesClosed); + when(metrics.getActiveFileCount()).thenReturn(new AtomicLong(0)); + when(checkpointRegistry.getOrCreate(anyString())).thenReturn(new CheckpointEntry()); + + Counter filesOpened = mock(Counter.class); + Counter bytesRead = mock(Counter.class); + Counter linesRead = mock(Counter.class); + Counter eventsEmitted = mock(Counter.class); + Timer backpressureTimer = mock(Timer.class); + lenient().when(metrics.getFilesOpened()).thenReturn(filesOpened); + lenient().when(metrics.getBytesRead()).thenReturn(bytesRead); + lenient().when(metrics.getLinesRead()).thenReturn(linesRead); + lenient().when(metrics.getEventsEmitted()).thenReturn(eventsEmitted); + lenient().when(metrics.getBackpressureTimer()).thenReturn(backpressureTimer); + lenient().when(metrics.getFileLagBytes()).thenReturn(new AtomicLong(0)); + + Path testFile = tempDir.resolve("inactive-test.log"); + Files.writeString(testFile, "data\n"); + + BasicFileAttributes attrs = mock(BasicFileAttributes.class); + when(attrs.fileKey()).thenReturn("inode-close-inactive"); + when(fileOps.readAttributes(testFile)).thenReturn(attrs); + lenient().when(fileOps.size(testFile)).thenReturn((long) "data\n".length()); + + CountDownLatch latch = new CountDownLatch(1); + when(rotationDetector.checkRotation(any(), any(), any(long.class))) + .thenAnswer(inv -> { + latch.await(); + return RotationResult.NO_ROTATION; + }); + + EventBuilder mockBuilder = mock(EventBuilder.class); + Event mockEvent = mock(Event.class); + lenient().when(eventFactory.eventBuilder(EventBuilder.class)).thenReturn(mockBuilder); + lenient().when(mockBuilder.withEventType(any())).thenReturn(mockBuilder); + lenient().when(mockBuilder.withData(any(Map.class))).thenReturn(mockBuilder); + lenient().when(mockBuilder.build()).thenReturn(mockEvent); + + TailFileReaderPool pool = new TailFileReaderPool( + checkpointRegistry, metrics, 10, 2, + Duration.ofMillis(1), createReaderContext()); + + FileIdentity identity = FileIdentity.from(testFile, fileOps, 1024); + pool.addFile(identity, testFile); + + Thread.sleep(100); + + pool.closeInactiveReaders(); + + latch.countDown(); + + pool.shutdown(); + } + + @Test + void closeInactiveReaders_keeps_active_readers() throws Exception { + Counter filesClosed = mock(Counter.class); + lenient().when(metrics.getFilesClosed()).thenReturn(filesClosed); + when(metrics.getActiveFileCount()).thenReturn(new AtomicLong(0)); + when(checkpointRegistry.getOrCreate(anyString())).thenReturn(new CheckpointEntry()); + + lenient().when(metrics.getFilesOpened()).thenReturn(mock(Counter.class)); + lenient().when(metrics.getBytesRead()).thenReturn(mock(Counter.class)); + lenient().when(metrics.getLinesRead()).thenReturn(mock(Counter.class)); + lenient().when(metrics.getEventsEmitted()).thenReturn(mock(Counter.class)); + lenient().when(metrics.getBackpressureTimer()).thenReturn(mock(Timer.class)); + lenient().when(metrics.getFileLagBytes()).thenReturn(new AtomicLong(0)); + + Path testFile = tempDir.resolve("active-test.log"); + Files.writeString(testFile, "data\n"); + + BasicFileAttributes attrs = mock(BasicFileAttributes.class); + when(attrs.fileKey()).thenReturn("inode-keep-active"); + when(fileOps.readAttributes(testFile)).thenReturn(attrs); + lenient().when(fileOps.size(testFile)).thenReturn((long) "data\n".length()); + + CountDownLatch latch = new CountDownLatch(1); + when(rotationDetector.checkRotation(any(), any(), any(long.class))) + .thenAnswer(inv -> { + latch.await(); + return RotationResult.NO_ROTATION; + }); + + EventBuilder mockBuilder = mock(EventBuilder.class); + Event mockEvent = mock(Event.class); + lenient().when(eventFactory.eventBuilder(EventBuilder.class)).thenReturn(mockBuilder); + lenient().when(mockBuilder.withEventType(any())).thenReturn(mockBuilder); + lenient().when(mockBuilder.withData(any(Map.class))).thenReturn(mockBuilder); + lenient().when(mockBuilder.build()).thenReturn(mockEvent); + + TailFileReaderPool pool = new TailFileReaderPool( + checkpointRegistry, metrics, 10, 2, + Duration.ofHours(1), createReaderContext()); + + FileIdentity identity = FileIdentity.from(testFile, fileOps, 1024); + pool.addFile(identity, testFile); + + Thread.sleep(100); + + assertThat(pool.getActiveReaderCount(), equalTo(1)); + + pool.closeInactiveReaders(); + + assertThat(pool.getActiveReaderCount(), equalTo(1)); + + latch.countDown(); + pool.shutdown(); + } + + @Test + void onReaderComplete_with_deleted_rotation_marks_completed_and_processes_pending() throws Exception { + Path testFile = tempDir.resolve("deleted-rotate.log"); + Files.writeString(testFile, "line1\n"); + + Counter filesOpened = mock(Counter.class); + Counter filesClosed = mock(Counter.class); + Counter bytesRead = mock(Counter.class); + Counter linesRead = mock(Counter.class); + Counter eventsEmitted = mock(Counter.class); + Timer backpressureTimer = mock(Timer.class); + lenient().when(metrics.getFilesOpened()).thenReturn(filesOpened); + lenient().when(metrics.getFilesClosed()).thenReturn(filesClosed); + lenient().when(metrics.getBytesRead()).thenReturn(bytesRead); + lenient().when(metrics.getLinesRead()).thenReturn(linesRead); + lenient().when(metrics.getEventsEmitted()).thenReturn(eventsEmitted); + lenient().when(metrics.getBackpressureTimer()).thenReturn(backpressureTimer); + lenient().when(metrics.getFileLagBytes()).thenReturn(new AtomicLong(0)); + + BasicFileAttributes attrs = mock(BasicFileAttributes.class); + when(attrs.fileKey()).thenReturn("inode-del"); + when(fileOps.readAttributes(testFile)).thenReturn(attrs); + when(fileOps.size(testFile)).thenReturn(6L); + FileChannel realChannel = FileChannel.open(testFile, StandardOpenOption.READ); + lenient().when(fileOps.openReadChannel(testFile)).thenReturn(realChannel); + + when(rotationDetector.checkRotation(any(), any(), any(long.class))) + .thenReturn(RotationResult.DELETED); + + EventBuilder mockBuilder = mock(EventBuilder.class); + Event mockEvent = mock(Event.class); + lenient().when(eventFactory.eventBuilder(EventBuilder.class)).thenReturn(mockBuilder); + lenient().when(mockBuilder.withEventType(any())).thenReturn(mockBuilder); + lenient().when(mockBuilder.withData(any(Map.class))).thenReturn(mockBuilder); + lenient().when(mockBuilder.build()).thenReturn(mockEvent); + + TailFileReaderPool pool = createPool(10, 2); + when(checkpointRegistry.getOrCreate(anyString())).thenReturn(new CheckpointEntry()); + + FileIdentity identity = FileIdentity.from(testFile, fileOps, 1024); + + Path pendingFile = tempDir.resolve("pending.log"); + Files.writeString(pendingFile, "pending\n"); + BasicFileAttributes pendingAttrs = mock(BasicFileAttributes.class); + when(pendingAttrs.fileKey()).thenReturn("inode-pending"); + lenient().when(fileOps.readAttributes(pendingFile)).thenReturn(pendingAttrs); + lenient().when(fileOps.size(pendingFile)).thenReturn(8L); + lenient().when(fileOps.openReadChannel(pendingFile)).thenReturn( + FileChannel.open(pendingFile, StandardOpenOption.READ)); + FileIdentity pendingIdentity = FileIdentity.from(pendingFile, fileOps, 1024); + + TailFileReaderPool limitedPool = new TailFileReaderPool( + checkpointRegistry, metrics, 1, 2, + Duration.ofMinutes(30), createReaderContext()); + + limitedPool.addFile(identity, testFile); + limitedPool.addFile(pendingIdentity, pendingFile); + + assertThat(limitedPool.getPendingCount(), equalTo(1)); + + Thread.sleep(2000); + + limitedPool.shutdown(); + } +} diff --git a/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/TailFileReaderTest.java b/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/TailFileReaderTest.java new file mode 100644 index 0000000000..30d9fbf8e0 --- /dev/null +++ b/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/TailFileReaderTest.java @@ -0,0 +1,1452 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + +package org.opensearch.dataprepper.plugins.source.file; + +import io.micrometer.core.instrument.Counter; +import io.micrometer.core.instrument.Timer; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.io.TempDir; +import org.mockito.ArgumentCaptor; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSet; +import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSetManager; +import org.opensearch.dataprepper.model.buffer.Buffer; +import org.opensearch.dataprepper.model.codec.InputCodec; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.event.EventBuilder; +import org.opensearch.dataprepper.model.event.EventFactory; +import org.opensearch.dataprepper.model.record.Record; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.channels.FileChannel; +import java.nio.charset.StandardCharsets; +import java.nio.file.Files; +import java.nio.file.NoSuchFileException; +import java.nio.file.Path; +import java.nio.file.StandardOpenOption; +import java.time.Duration; +import java.util.Map; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicLong; +import java.util.function.Consumer; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.equalTo; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyInt; +import static org.mockito.ArgumentMatchers.anyLong; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.atLeastOnce; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.lenient; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +@ExtendWith(MockitoExtension.class) +class TailFileReaderTest { + + @TempDir + Path tempDir; + + @Mock + private Buffer> buffer; + + @Mock + private EventFactory eventFactory; + + @Mock + private FileSystemOperations fileOps; + + @Mock + private FileTailMetrics metrics; + + @Mock + private RotationDetector rotationDetector; + + @Mock + private AcknowledgementSetManager acknowledgementSetManager; + + private CheckpointEntry checkpointEntry; + private FileIdentity fileIdentity; + private AtomicBoolean onCompleteCalled; + + @BeforeEach + void setUp() { + checkpointEntry = new CheckpointEntry(); + onCompleteCalled = new AtomicBoolean(false); + } + + private TailFileReaderContext createContext(final StartPosition startPosition) { + return new TailFileReaderContext( + buffer, eventFactory, fileOps, metrics, rotationDetector, + acknowledgementSetManager, false, StandardCharsets.UTF_8, + 4096, 1048576, 5000, Duration.ofSeconds(30), + Duration.ofSeconds(30), startPosition, false, + Duration.ofSeconds(30), 1000, + Duration.ofSeconds(5), 3, null); + } + + private TailFileReaderContext createContext(final int readBufferSize, final int maxLineLength, + final boolean includeMetadata, final StartPosition startPosition) { + return new TailFileReaderContext( + buffer, eventFactory, fileOps, metrics, rotationDetector, + acknowledgementSetManager, false, StandardCharsets.UTF_8, + readBufferSize, maxLineLength, 5000, Duration.ofSeconds(30), + Duration.ofSeconds(30), startPosition, includeMetadata, + Duration.ofSeconds(30), 1000, + Duration.ofSeconds(5), 3, null); + } + + private TailFileReader createReader(final Path path) { + return createReader(path, 4096, 1048576, false, StartPosition.BEGINNING); + } + + private TailFileReader createReader(final Path path, final int readBufferSize, + final int maxLineLength, final boolean includeMetadata) { + return createReader(path, readBufferSize, maxLineLength, includeMetadata, StartPosition.BEGINNING); + } + + private TailFileReader createReader(final Path path, final int readBufferSize, + final int maxLineLength, final boolean includeMetadata, + final StartPosition startPosition) { + fileIdentity = mock(FileIdentity.class); + final TailFileReaderContext context = createContext(readBufferSize, maxLineLength, includeMetadata, startPosition); + return new TailFileReader(path, fileIdentity, checkpointEntry, context, + () -> onCompleteCalled.set(true)); + } + + private void stubEventFactory() { + when(eventFactory.eventBuilder(EventBuilder.class)).thenAnswer(invocation -> { + EventBuilder mockBuilder = mock(EventBuilder.class); + Event mockEvent = mock(Event.class); + when(mockBuilder.withEventType(any())).thenReturn(mockBuilder); + when(mockBuilder.withData(any(Map.class))).thenReturn(mockBuilder); + when(mockBuilder.build()).thenReturn(mockEvent); + return mockBuilder; + }); + } + + private void lenientStubEventFactory() { + lenient().when(eventFactory.eventBuilder(EventBuilder.class)).thenAnswer(invocation -> { + EventBuilder mockBuilder = mock(EventBuilder.class); + Event mockEvent = mock(Event.class); + lenient().when(mockBuilder.withEventType(any())).thenReturn(mockBuilder); + lenient().when(mockBuilder.withData(any(Map.class))).thenReturn(mockBuilder); + lenient().when(mockBuilder.build()).thenReturn(mockEvent); + return mockBuilder; + }); + } + + private void stubReadMetrics() { + Counter linesRead = mock(Counter.class); + Counter bytesRead = mock(Counter.class); + Counter filesOpened = mock(Counter.class); + Counter filesClosed = mock(Counter.class); + Counter eventsEmitted = mock(Counter.class); + Timer backpressureTimer = mock(Timer.class); + when(metrics.getLinesRead()).thenReturn(linesRead); + when(metrics.getBytesRead()).thenReturn(bytesRead); + when(metrics.getFilesOpened()).thenReturn(filesOpened); + when(metrics.getFilesClosed()).thenReturn(filesClosed); + lenient().when(metrics.getEventsEmitted()).thenReturn(eventsEmitted); + lenient().when(metrics.getBackpressureTimer()).thenReturn(backpressureTimer); + lenient().when(metrics.getFileLagBytes()).thenReturn(new AtomicLong(0)); + } + + @Test + void run_reads_lines_from_file() throws Exception { + Path testFile = tempDir.resolve("test.log"); + Files.writeString(testFile, "line1\nline2\nline3\n"); + FileChannel realChannel = FileChannel.open(testFile, StandardOpenOption.READ); + when(fileOps.openReadChannel(testFile)).thenReturn(realChannel); + when(rotationDetector.checkRotation(any(), any(), any(long.class))) + .thenReturn(RotationResult.NO_ROTATION); + stubReadMetrics(); + stubEventFactory(); + + final TailFileReader reader = createReader(testFile); + reader.run(); + + verify(buffer, times(3)).write(any(Record.class), eq(5000)); + verify(metrics.getLinesRead(), times(3)).increment(); + assertThat(onCompleteCalled.get(), equalTo(true)); + } + + @Test + void run_tracks_read_offset() throws Exception { + Path testFile = tempDir.resolve("offset.log"); + final String content = "hello\nworld\n"; + Files.writeString(testFile, content); + FileChannel realChannel = FileChannel.open(testFile, StandardOpenOption.READ); + when(fileOps.openReadChannel(testFile)).thenReturn(realChannel); + when(rotationDetector.checkRotation(any(), any(), any(long.class))) + .thenReturn(RotationResult.NO_ROTATION); + stubReadMetrics(); + stubEventFactory(); + + final TailFileReader reader = createReader(testFile); + reader.run(); + + assertThat(reader.getReadOffset(), equalTo((long) content.getBytes(StandardCharsets.UTF_8).length)); + assertThat(checkpointEntry.getReadOffset(), equalTo((long) content.getBytes(StandardCharsets.UTF_8).length)); + } + + @Test + void run_handles_partial_line_without_trailing_newline() throws Exception { + Path testFile = tempDir.resolve("partial.log"); + Files.writeString(testFile, "complete\nno-newline-at-end"); + FileChannel realChannel = FileChannel.open(testFile, StandardOpenOption.READ); + when(fileOps.openReadChannel(testFile)).thenReturn(realChannel); + when(rotationDetector.checkRotation(any(), any(), any(long.class))) + .thenReturn(RotationResult.NO_ROTATION); + stubReadMetrics(); + stubEventFactory(); + + final TailFileReader reader = createReader(testFile); + reader.run(); + + verify(buffer, times(2)).write(any(Record.class), eq(5000)); + } + + @Test + void run_truncates_line_exceeding_max_line_length() throws Exception { + Path testFile = tempDir.resolve("longline.log"); + final String longContent = "A".repeat(200); + Files.writeString(testFile, longContent); + FileChannel realChannel = FileChannel.open(testFile, StandardOpenOption.READ); + when(fileOps.openReadChannel(testFile)).thenReturn(realChannel); + when(rotationDetector.checkRotation(any(), any(), any(long.class))) + .thenReturn(RotationResult.NO_ROTATION); + stubReadMetrics(); + Counter linesTruncated = mock(Counter.class); + when(metrics.getLinesTruncated()).thenReturn(linesTruncated); + stubEventFactory(); + + final TailFileReader reader = createReader(testFile, 4096, 50, false); + reader.run(); + + verify(linesTruncated).increment(); + } + + @Test + void run_truncates_complete_line_exceeding_max_line_length() throws Exception { + Path testFile = tempDir.resolve("longcomplete.log"); + Files.writeString(testFile, "B".repeat(200) + "\n"); + FileChannel realChannel = FileChannel.open(testFile, StandardOpenOption.READ); + when(fileOps.openReadChannel(testFile)).thenReturn(realChannel); + when(rotationDetector.checkRotation(any(), any(), any(long.class))) + .thenReturn(RotationResult.NO_ROTATION); + stubReadMetrics(); + Counter linesTruncated = mock(Counter.class); + when(metrics.getLinesTruncated()).thenReturn(linesTruncated); + stubEventFactory(); + + final TailFileReader reader = createReader(testFile, 4096, 50, false); + reader.run(); + + verify(linesTruncated).increment(); + } + + @Test + void run_retries_on_buffer_back_pressure() throws Exception { + Path testFile = tempDir.resolve("backpressure.log"); + Files.writeString(testFile, "line1\n"); + FileChannel realChannel = FileChannel.open(testFile, StandardOpenOption.READ); + when(fileOps.openReadChannel(testFile)).thenReturn(realChannel); + when(rotationDetector.checkRotation(any(), any(), any(long.class))) + .thenReturn(RotationResult.NO_ROTATION); + stubReadMetrics(); + Counter writeTimeouts = mock(Counter.class); + when(metrics.getWriteTimeouts()).thenReturn(writeTimeouts); + stubEventFactory(); + + doThrow(new TimeoutException("buffer full")) + .doNothing() + .when(buffer).write(any(Record.class), anyInt()); + + final TailFileReader reader = createReader(testFile); + reader.run(); + + verify(writeTimeouts).increment(); + verify(buffer, times(2)).write(any(Record.class), eq(5000)); + } + + @Test + void run_handles_deleted_file_via_rotation_detector() throws Exception { + Path testFile = tempDir.resolve("deleted.log"); + Files.writeString(testFile, "data\n"); + + when(rotationDetector.checkRotation(any(), any(), any(long.class))) + .thenReturn(RotationResult.DELETED); + + final TailFileReader reader = createReader(testFile); + reader.run(); + + assertThat(reader.getLastRotationType(), equalTo(RotationType.DELETED)); + verify(buffer, never()).write(any(), anyInt()); + assertThat(onCompleteCalled.get(), equalTo(true)); + } + + @Test + void run_handles_no_such_file_exception_during_read() throws Exception { + Path testFile = tempDir.resolve("gone.log"); + Files.writeString(testFile, "data\n"); + + when(rotationDetector.checkRotation(any(), any(), any(long.class))) + .thenReturn(RotationResult.NO_ROTATION); + when(fileOps.openReadChannel(testFile)).thenThrow(new NoSuchFileException(testFile.toString())); + Counter filesClosed = mock(Counter.class); + when(metrics.getFilesClosed()).thenReturn(filesClosed); + + final TailFileReader reader = createReader(testFile); + reader.run(); + + assertThat(onCompleteCalled.get(), equalTo(true)); + } + + @Test + void run_handles_copytruncate_rotation() throws Exception { + Path testFile = tempDir.resolve("truncated.log"); + Files.writeString(testFile, "new-data\n"); + FileChannel realChannel = FileChannel.open(testFile, StandardOpenOption.READ); + when(fileOps.openReadChannel(testFile)).thenReturn(realChannel); + Counter filesRotated = mock(Counter.class); + when(metrics.getFilesRotated()).thenReturn(filesRotated); + stubReadMetrics(); + stubEventFactory(); + + fileIdentity = mock(FileIdentity.class); + when(rotationDetector.checkRotation(any(), any(), any(long.class))) + .thenReturn(new RotationResult(RotationType.COPYTRUNCATE, fileIdentity)); + + checkpointEntry.setReadOffset(500); + final TailFileReader reader = createReader(testFile); + reader.run(); + + verify(filesRotated).increment(); + assertThat(reader.getLastRotationType(), equalTo(RotationType.COPYTRUNCATE)); + } + + @Test + void run_handles_create_rename_rotation() throws Exception { + Path testFile = tempDir.resolve("renamed.log"); + Files.writeString(testFile, "tail-data\n"); + FileChannel realChannel = FileChannel.open(testFile, StandardOpenOption.READ); + when(fileOps.openReadChannel(testFile)).thenReturn(realChannel); + Counter filesRotated = mock(Counter.class); + when(metrics.getFilesRotated()).thenReturn(filesRotated); + stubReadMetrics(); + stubEventFactory(); + + FileIdentity newIdentity = mock(FileIdentity.class); + when(rotationDetector.checkRotation(any(), any(), any(long.class))) + .thenReturn(new RotationResult(RotationType.CREATE_RENAME, newIdentity)); + + final TailFileReader reader = createReader(testFile); + reader.run(); + + verify(filesRotated).increment(); + assertThat(reader.getLastRotationType(), equalTo(RotationType.CREATE_RENAME)); + } + + @Test + void run_resumes_from_checkpoint_offset() throws Exception { + Path testFile = tempDir.resolve("resume.log"); + Files.writeString(testFile, "line1\nline2\nline3\n"); + long offsetAfterFirstLine = "line1\n".getBytes(StandardCharsets.UTF_8).length; + checkpointEntry.setReadOffset(offsetAfterFirstLine); + + FileChannel realChannel = FileChannel.open(testFile, StandardOpenOption.READ); + when(fileOps.openReadChannel(testFile)).thenReturn(realChannel); + when(rotationDetector.checkRotation(any(), any(), any(long.class))) + .thenReturn(RotationResult.NO_ROTATION); + stubReadMetrics(); + stubEventFactory(); + + final TailFileReader reader = createReader(testFile); + reader.run(); + + verify(buffer, times(2)).write(any(Record.class), eq(5000)); + } + + @Test + void run_increments_read_errors_on_io_exception() throws Exception { + Path testFile = tempDir.resolve("ioerror.log"); + Files.writeString(testFile, "data\n"); + + when(rotationDetector.checkRotation(any(), any(), any(long.class))) + .thenReturn(RotationResult.NO_ROTATION); + when(fileOps.openReadChannel(testFile)).thenThrow(new IOException("disk error")); + Counter readErrors = mock(Counter.class); + Counter filesClosed = mock(Counter.class); + when(metrics.getReadErrors()).thenReturn(readErrors); + when(metrics.getFilesClosed()).thenReturn(filesClosed); + + final TailFileReader reader = createReader(testFile); + reader.run(); + + verify(readErrors).increment(); + assertThat(onCompleteCalled.get(), equalTo(true)); + } + + @Test + void run_includes_file_metadata_when_enabled() throws Exception { + Path testFile = tempDir.resolve("meta.log"); + Files.writeString(testFile, "data\n"); + FileChannel realChannel = FileChannel.open(testFile, StandardOpenOption.READ); + when(fileOps.openReadChannel(testFile)).thenReturn(realChannel); + when(rotationDetector.checkRotation(any(), any(), any(long.class))) + .thenReturn(RotationResult.NO_ROTATION); + stubReadMetrics(); + + ArgumentCaptor dataCaptor = ArgumentCaptor.forClass(Map.class); + EventBuilder mockBuilder = mock(EventBuilder.class); + Event mockEvent = mock(Event.class); + when(eventFactory.eventBuilder(EventBuilder.class)).thenReturn(mockBuilder); + when(mockBuilder.withEventType(any())).thenReturn(mockBuilder); + when(mockBuilder.withData(dataCaptor.capture())).thenReturn(mockBuilder); + when(mockBuilder.build()).thenReturn(mockEvent); + + final TailFileReader reader = createReader(testFile, 4096, 1048576, true); + reader.run(); + + Map capturedData = dataCaptor.getValue(); + assertThat(capturedData.containsKey("file_path"), equalTo(true)); + assertThat(capturedData.containsKey("file_identity"), equalTo(true)); + } + + @Test + void getFileIdentity_returns_identity_passed_in_constructor() throws Exception { + Path testFile = tempDir.resolve("identity.log"); + Files.writeString(testFile, ""); + + final TailFileReader reader = createReader(testFile); + assertThat(reader.getFileIdentity(), equalTo(fileIdentity)); + } + + @Test + void getLastRotationType_defaults_to_no_rotation() throws Exception { + Path testFile = tempDir.resolve("default.log"); + Files.writeString(testFile, ""); + + final TailFileReader reader = createReader(testFile); + assertThat(reader.getLastRotationType(), equalTo(RotationType.NO_ROTATION)); + } + + @Test + void start_position_end_seeks_to_end_of_file_for_new_files() throws Exception { + Path testFile = tempDir.resolve("startend.log"); + Files.writeString(testFile, "existing-line1\nexisting-line2\n"); + long fileSize = Files.size(testFile); + when(fileOps.size(testFile)).thenReturn(fileSize); + + final TailFileReader reader = createReader(testFile, 4096, 1048576, false, StartPosition.END); + + assertThat(reader.getReadOffset(), equalTo(fileSize)); + } + + @Test + void start_position_beginning_starts_from_offset_zero_for_new_files() throws Exception { + Path testFile = tempDir.resolve("startbegin.log"); + Files.writeString(testFile, "existing-line1\nexisting-line2\n"); + + final TailFileReader reader = createReader(testFile, 4096, 1048576, false, StartPosition.BEGINNING); + + assertThat(reader.getReadOffset(), equalTo(0L)); + } + + @Test + void start_position_end_does_not_seek_when_checkpoint_exists() throws Exception { + Path testFile = tempDir.resolve("checkpoint-end.log"); + Files.writeString(testFile, "existing-line1\nexisting-line2\n"); + checkpointEntry.setReadOffset(10); + + final TailFileReader reader = createReader(testFile, 4096, 1048576, false, StartPosition.END); + + assertThat(reader.getReadOffset(), equalTo(10L)); + } + + @Test + void run_increments_events_emitted_counter() throws Exception { + Path testFile = tempDir.resolve("emitted.log"); + Files.writeString(testFile, "line1\nline2\n"); + FileChannel realChannel = FileChannel.open(testFile, StandardOpenOption.READ); + when(fileOps.openReadChannel(testFile)).thenReturn(realChannel); + when(rotationDetector.checkRotation(any(), any(), any(long.class))) + .thenReturn(RotationResult.NO_ROTATION); + stubReadMetrics(); + stubEventFactory(); + when(fileOps.size(testFile)).thenReturn(Files.size(testFile)); + + Counter eventsEmitted = mock(Counter.class); + when(metrics.getEventsEmitted()).thenReturn(eventsEmitted); + + final TailFileReader reader = createReader(testFile); + reader.run(); + + verify(eventsEmitted, times(2)).increment(); + } + + private TailFileReaderContext createContextWithCodec(final InputCodec codec) { + return new TailFileReaderContext( + buffer, eventFactory, fileOps, metrics, rotationDetector, + acknowledgementSetManager, false, StandardCharsets.UTF_8, + 4096, 1048576, 5000, Duration.ofSeconds(30), + Duration.ofSeconds(30), StartPosition.BEGINNING, false, + Duration.ofSeconds(30), 1000, + Duration.ofSeconds(5), 3, codec); + } + + private TailFileReaderContext createContextWithAcknowledgements(final int batchSize, final Duration batchTimeout, final int maxRetries) { + return new TailFileReaderContext( + buffer, eventFactory, fileOps, metrics, rotationDetector, + acknowledgementSetManager, true, StandardCharsets.UTF_8, + 4096, 1048576, 5000, Duration.ofSeconds(30), + Duration.ofSeconds(30), StartPosition.BEGINNING, false, + Duration.ofSeconds(30), batchSize, + batchTimeout, maxRetries, null); + } + + private TailFileReader createReaderWithContext(final Path path, final TailFileReaderContext context) { + fileIdentity = mock(FileIdentity.class); + return new TailFileReader(path, fileIdentity, checkpointEntry, context, + () -> onCompleteCalled.set(true)); + } + + @Test + void start_position_end_falls_back_to_zero_on_io_exception() throws Exception { + Path testFile = tempDir.resolve("startend-error.log"); + Files.writeString(testFile, "existing data\n"); + when(fileOps.size(testFile)).thenThrow(new IOException("disk error")); + + final TailFileReader reader = createReader(testFile, 4096, 1048576, false, StartPosition.END); + + assertThat(reader.getReadOffset(), equalTo(0L)); + } + + @Test + void run_increments_read_errors_on_runtime_exception() throws Exception { + Path testFile = tempDir.resolve("runtime-err.log"); + Files.writeString(testFile, "data\n"); + + when(rotationDetector.checkRotation(any(), any(), any(long.class))) + .thenThrow(new RuntimeException("unexpected")); + Counter readErrors = mock(Counter.class); + when(metrics.getReadErrors()).thenReturn(readErrors); + + final TailFileReader reader = createReader(testFile); + reader.run(); + + verify(readErrors).increment(); + assertThat(onCompleteCalled.get(), equalTo(true)); + } + + @Test + void run_drains_file_on_create_rename_and_handles_no_such_file() throws Exception { + Path testFile = tempDir.resolve("drain-nosuch.log"); + Files.writeString(testFile, "data\n"); + + FileIdentity newIdentity = mock(FileIdentity.class); + when(rotationDetector.checkRotation(any(), any(), any(long.class))) + .thenReturn(new RotationResult(RotationType.CREATE_RENAME, newIdentity)); + when(fileOps.openReadChannel(testFile)).thenThrow(new NoSuchFileException(testFile.toString())); + Counter filesRotated = mock(Counter.class); + Counter filesClosed = mock(Counter.class); + when(metrics.getFilesRotated()).thenReturn(filesRotated); + when(metrics.getFilesClosed()).thenReturn(filesClosed); + + final TailFileReader reader = createReader(testFile); + reader.run(); + + verify(filesRotated).increment(); + assertThat(reader.getLastRotationType(), equalTo(RotationType.CREATE_RENAME)); + } + + @Test + void run_drains_file_on_create_rename_and_handles_io_exception() throws Exception { + Path testFile = tempDir.resolve("drain-ioerr.log"); + Files.writeString(testFile, "data\n"); + + FileIdentity newIdentity = mock(FileIdentity.class); + when(rotationDetector.checkRotation(any(), any(), any(long.class))) + .thenReturn(new RotationResult(RotationType.CREATE_RENAME, newIdentity)); + when(fileOps.openReadChannel(testFile)).thenThrow(new IOException("disk error")); + Counter filesRotated = mock(Counter.class); + Counter filesClosed = mock(Counter.class); + Counter readErrors = mock(Counter.class); + when(metrics.getFilesRotated()).thenReturn(filesRotated); + when(metrics.getFilesClosed()).thenReturn(filesClosed); + when(metrics.getReadErrors()).thenReturn(readErrors); + + final TailFileReader reader = createReader(testFile); + reader.run(); + + verify(readErrors).increment(); + } + + @Test + void run_drain_timeout_logs_data_loss_when_unread_data() throws Exception { + Path testFile = tempDir.resolve("drain-timeout.log"); + Files.writeString(testFile, "A".repeat(10000) + "\n"); + + FileIdentity newIdentity = mock(FileIdentity.class); + when(rotationDetector.checkRotation(any(), any(), any(long.class))) + .thenReturn(new RotationResult(RotationType.CREATE_RENAME, newIdentity)); + + FileChannel mockChannel = mock(FileChannel.class); + when(fileOps.openReadChannel(testFile)).thenReturn(mockChannel); + when(mockChannel.position(anyLong())).thenReturn(mockChannel); + lenient().when(mockChannel.read(any(ByteBuffer.class))).thenAnswer(inv -> { + Thread.sleep(5); + ByteBuffer buf = inv.getArgument(0); + byte[] data = "A".repeat(buf.remaining()).getBytes(); + buf.put(data, 0, Math.min(data.length, buf.remaining())); + return buf.position(); + }); + when(mockChannel.size()).thenReturn(100000L); + + Counter filesRotated = mock(Counter.class); + Counter filesClosed = mock(Counter.class); + Counter bytesRead = mock(Counter.class); + Counter dataLossEvents = mock(Counter.class); + when(metrics.getFilesRotated()).thenReturn(filesRotated); + when(metrics.getFilesClosed()).thenReturn(filesClosed); + when(metrics.getFilesOpened()).thenReturn(mock(Counter.class)); + lenient().when(metrics.getBytesRead()).thenReturn(bytesRead); + when(metrics.getDataLossEvents()).thenReturn(dataLossEvents); + + TailFileReaderContext context = new TailFileReaderContext( + buffer, eventFactory, fileOps, metrics, rotationDetector, + acknowledgementSetManager, false, StandardCharsets.UTF_8, + 4096, 1048576, 5000, Duration.ofSeconds(30), + Duration.ofMillis(1), StartPosition.BEGINNING, false, + Duration.ofSeconds(30), 1000, + Duration.ofSeconds(5), 3, null); + + lenientStubEventFactory(); + Counter linesRead = mock(Counter.class); + Counter eventsEmitted = mock(Counter.class); + lenient().when(metrics.getLinesRead()).thenReturn(linesRead); + lenient().when(metrics.getEventsEmitted()).thenReturn(eventsEmitted); + + fileIdentity = mock(FileIdentity.class); + final TailFileReader reader = new TailFileReader(testFile, fileIdentity, checkpointEntry, context, + () -> onCompleteCalled.set(true)); + reader.run(); + + verify(dataLossEvents).increment(); + } + + @Test + void run_max_read_time_reached_breaks_without_data_loss() throws Exception { + Path testFile = tempDir.resolve("maxread.log"); + Files.writeString(testFile, "A".repeat(10000) + "\n"); + + when(rotationDetector.checkRotation(any(), any(), any(long.class))) + .thenReturn(RotationResult.NO_ROTATION); + + FileChannel mockChannel = mock(FileChannel.class); + when(fileOps.openReadChannel(testFile)).thenReturn(mockChannel); + when(mockChannel.position(anyLong())).thenReturn(mockChannel); + when(mockChannel.read(any(ByteBuffer.class))).thenAnswer(inv -> { + Thread.sleep(5); + ByteBuffer buf = inv.getArgument(0); + byte[] data = "A".repeat(buf.remaining()).getBytes(); + buf.put(data, 0, Math.min(data.length, buf.remaining())); + return buf.position(); + }); + + Counter filesClosed = mock(Counter.class); + Counter bytesRead = mock(Counter.class); + when(metrics.getFilesClosed()).thenReturn(filesClosed); + when(metrics.getFilesOpened()).thenReturn(mock(Counter.class)); + when(metrics.getBytesRead()).thenReturn(bytesRead); + lenient().when(metrics.getFileLagBytes()).thenReturn(new AtomicLong(0)); + + stubEventFactory(); + Counter linesRead = mock(Counter.class); + Counter eventsEmitted = mock(Counter.class); + lenient().when(metrics.getLinesRead()).thenReturn(linesRead); + lenient().when(metrics.getEventsEmitted()).thenReturn(eventsEmitted); + + TailFileReaderContext context = new TailFileReaderContext( + buffer, eventFactory, fileOps, metrics, rotationDetector, + acknowledgementSetManager, false, StandardCharsets.UTF_8, + 4096, 1048576, 5000, Duration.ofMillis(1), + Duration.ofSeconds(30), StartPosition.BEGINNING, false, + Duration.ofSeconds(30), 1000, + Duration.ofSeconds(5), 3, null); + + fileIdentity = mock(FileIdentity.class); + final TailFileReader reader = new TailFileReader(testFile, fileIdentity, checkpointEntry, context, + () -> onCompleteCalled.set(true)); + reader.run(); + + assertThat(onCompleteCalled.get(), equalTo(true)); + } + + @Test + void run_with_codec_parses_bytes_and_emits_records() throws Exception { + Path testFile = tempDir.resolve("codec.log"); + Files.writeString(testFile, "codec-data\n"); + FileChannel realChannel = FileChannel.open(testFile, StandardOpenOption.READ); + when(fileOps.openReadChannel(testFile)).thenReturn(realChannel); + when(rotationDetector.checkRotation(any(), any(), any(long.class))) + .thenReturn(RotationResult.NO_ROTATION); + + Counter bytesRead = mock(Counter.class); + Counter filesOpened = mock(Counter.class); + Counter filesClosed = mock(Counter.class); + Counter linesRead = mock(Counter.class); + Counter eventsEmitted = mock(Counter.class); + Timer backpressureTimer = mock(Timer.class); + when(metrics.getBytesRead()).thenReturn(bytesRead); + when(metrics.getFilesOpened()).thenReturn(filesOpened); + when(metrics.getFilesClosed()).thenReturn(filesClosed); + when(metrics.getLinesRead()).thenReturn(linesRead); + when(metrics.getEventsEmitted()).thenReturn(eventsEmitted); + lenient().when(metrics.getBackpressureTimer()).thenReturn(backpressureTimer); + lenient().when(metrics.getFileLagBytes()).thenReturn(new AtomicLong(0)); + + InputCodec mockCodec = mock(InputCodec.class); + doAnswer(inv -> { + Consumer> consumer = inv.getArgument(1); + Event mockEvent = mock(Event.class); + consumer.accept(new Record<>(mockEvent)); + return null; + }).when(mockCodec).parse(any(), any()); + + TailFileReaderContext context = createContextWithCodec(mockCodec); + final TailFileReader reader = createReaderWithContext(testFile, context); + reader.run(); + + verify(buffer, atLeastOnce()).write(any(Record.class), eq(5000)); + verify(linesRead, atLeastOnce()).increment(); + } + + @Test + void run_with_codec_handles_parse_io_exception() throws Exception { + Path testFile = tempDir.resolve("codec-error.log"); + Files.writeString(testFile, "bad-data\n"); + FileChannel realChannel = FileChannel.open(testFile, StandardOpenOption.READ); + when(fileOps.openReadChannel(testFile)).thenReturn(realChannel); + when(rotationDetector.checkRotation(any(), any(), any(long.class))) + .thenReturn(RotationResult.NO_ROTATION); + + Counter bytesRead = mock(Counter.class); + Counter filesOpened = mock(Counter.class); + Counter filesClosed = mock(Counter.class); + Counter readErrors = mock(Counter.class); + when(metrics.getBytesRead()).thenReturn(bytesRead); + when(metrics.getFilesOpened()).thenReturn(filesOpened); + when(metrics.getFilesClosed()).thenReturn(filesClosed); + when(metrics.getReadErrors()).thenReturn(readErrors); + lenient().when(metrics.getFileLagBytes()).thenReturn(new AtomicLong(0)); + + InputCodec mockCodec = mock(InputCodec.class); + doThrow(new IOException("parse error")).when(mockCodec).parse(any(), any()); + + TailFileReaderContext context = createContextWithCodec(mockCodec); + final TailFileReader reader = createReaderWithContext(testFile, context); + reader.run(); + + verify(readErrors).increment(); + } + + @Test + void run_codec_record_retries_on_backpressure_and_records_timer() throws Exception { + Path testFile = tempDir.resolve("codec-backpressure.log"); + Files.writeString(testFile, "data\n"); + FileChannel realChannel = FileChannel.open(testFile, StandardOpenOption.READ); + when(fileOps.openReadChannel(testFile)).thenReturn(realChannel); + when(rotationDetector.checkRotation(any(), any(), any(long.class))) + .thenReturn(RotationResult.NO_ROTATION); + + Counter bytesRead = mock(Counter.class); + Counter filesOpened = mock(Counter.class); + Counter filesClosed = mock(Counter.class); + Counter linesRead = mock(Counter.class); + Counter eventsEmitted = mock(Counter.class); + Counter writeTimeouts = mock(Counter.class); + Timer backpressureTimer = mock(Timer.class); + when(metrics.getBytesRead()).thenReturn(bytesRead); + when(metrics.getFilesOpened()).thenReturn(filesOpened); + when(metrics.getFilesClosed()).thenReturn(filesClosed); + when(metrics.getLinesRead()).thenReturn(linesRead); + when(metrics.getEventsEmitted()).thenReturn(eventsEmitted); + when(metrics.getWriteTimeouts()).thenReturn(writeTimeouts); + when(metrics.getBackpressureTimer()).thenReturn(backpressureTimer); + lenient().when(metrics.getFileLagBytes()).thenReturn(new AtomicLong(0)); + + doThrow(new TimeoutException("buffer full")) + .doNothing() + .when(buffer).write(any(Record.class), anyInt()); + + InputCodec mockCodec = mock(InputCodec.class); + doAnswer(inv -> { + Consumer> consumer = inv.getArgument(1); + Event mockEvent = mock(Event.class); + consumer.accept(new Record<>(mockEvent)); + return null; + }).when(mockCodec).parse(any(), any()); + + TailFileReaderContext context = createContextWithCodec(mockCodec); + final TailFileReader reader = createReaderWithContext(testFile, context); + reader.run(); + + verify(writeTimeouts).increment(); + verify(backpressureTimer).record(anyLong(), any(TimeUnit.class)); + } + + @Test + void run_with_acknowledgements_creates_ack_set_and_completes_on_batch_full() throws Exception { + Path testFile = tempDir.resolve("ack.log"); + Files.writeString(testFile, "line1\nline2\n"); + FileChannel realChannel = FileChannel.open(testFile, StandardOpenOption.READ); + when(fileOps.openReadChannel(testFile)).thenReturn(realChannel); + when(rotationDetector.checkRotation(any(), any(), any(long.class))) + .thenReturn(RotationResult.NO_ROTATION); + stubReadMetrics(); + stubEventFactory(); + when(fileOps.size(testFile)).thenReturn(Files.size(testFile)); + + AcknowledgementSet ackSet = mock(AcknowledgementSet.class); + when(acknowledgementSetManager.create(any(), any(Duration.class))).thenReturn(ackSet); + + TailFileReaderContext context = createContextWithAcknowledgements(1, Duration.ofSeconds(5), 3); + final TailFileReader reader = createReaderWithContext(testFile, context); + reader.run(); + + verify(acknowledgementSetManager, atLeastOnce()).create(any(), any(Duration.class)); + verify(ackSet, atLeastOnce()).add(any(Event.class)); + verify(ackSet, atLeastOnce()).complete(); + } + + @Test + void run_with_acknowledgements_batch_timeout_triggers_complete() throws Exception { + Path testFile = tempDir.resolve("ack-timeout.log"); + Files.writeString(testFile, "line1\nline2\n"); + FileChannel realChannel = FileChannel.open(testFile, StandardOpenOption.READ); + when(fileOps.openReadChannel(testFile)).thenReturn(realChannel); + when(rotationDetector.checkRotation(any(), any(), any(long.class))) + .thenReturn(RotationResult.NO_ROTATION); + stubReadMetrics(); + stubEventFactory(); + when(fileOps.size(testFile)).thenReturn(Files.size(testFile)); + + AcknowledgementSet ackSet = mock(AcknowledgementSet.class); + when(acknowledgementSetManager.create(any(), any(Duration.class))).thenReturn(ackSet); + + TailFileReaderContext context = createContextWithAcknowledgements(10000, Duration.ofMillis(0), 3); + final TailFileReader reader = createReaderWithContext(testFile, context); + reader.run(); + + verify(ackSet, atLeastOnce()).complete(); + } + + @SuppressWarnings("unchecked") + @Test + void handleAcknowledgement_positive_resets_retry_and_updates_checkpoint() throws Exception { + Path testFile = tempDir.resolve("ack-pos.log"); + Files.writeString(testFile, "line1\n"); + FileChannel realChannel = FileChannel.open(testFile, StandardOpenOption.READ); + when(fileOps.openReadChannel(testFile)).thenReturn(realChannel); + when(rotationDetector.checkRotation(any(), any(), any(long.class))) + .thenReturn(RotationResult.NO_ROTATION); + stubReadMetrics(); + stubEventFactory(); + when(fileOps.size(testFile)).thenReturn(Files.size(testFile)); + + AcknowledgementSet ackSet = mock(AcknowledgementSet.class); + ArgumentCaptor handlerCaptor = ArgumentCaptor.forClass(Consumer.class); + when(acknowledgementSetManager.create(handlerCaptor.capture(), any(Duration.class))).thenReturn(ackSet); + + TailFileReaderContext context = createContextWithAcknowledgements(1000, Duration.ofSeconds(5), 3); + final TailFileReader reader = createReaderWithContext(testFile, context); + reader.run(); + + Consumer handler = handlerCaptor.getValue(); + handler.accept(true); + + assertThat(checkpointEntry.getCommittedOffset(), equalTo(reader.getReadOffset())); + } + + @SuppressWarnings("unchecked") + @Test + void handleAcknowledgement_negative_retries_and_eventually_advances() throws Exception { + Path testFile = tempDir.resolve("ack-neg.log"); + Files.writeString(testFile, "line1\n"); + FileChannel realChannel = FileChannel.open(testFile, StandardOpenOption.READ); + when(fileOps.openReadChannel(testFile)).thenReturn(realChannel); + when(rotationDetector.checkRotation(any(), any(), any(long.class))) + .thenReturn(RotationResult.NO_ROTATION); + stubReadMetrics(); + stubEventFactory(); + when(fileOps.size(testFile)).thenReturn(Files.size(testFile)); + + Counter ackFailures = mock(Counter.class); + when(metrics.getAcknowledgmentFailures()).thenReturn(ackFailures); + + AcknowledgementSet ackSet = mock(AcknowledgementSet.class); + ArgumentCaptor handlerCaptor = ArgumentCaptor.forClass(Consumer.class); + when(acknowledgementSetManager.create(handlerCaptor.capture(), any(Duration.class))).thenReturn(ackSet); + + TailFileReaderContext context = createContextWithAcknowledgements(1000, Duration.ofSeconds(5), 2); + final TailFileReader reader = createReaderWithContext(testFile, context); + reader.run(); + + Consumer handler = handlerCaptor.getValue(); + handler.accept(false); + verify(ackFailures, times(1)).increment(); + + handler.accept(false); + verify(ackFailures, times(2)).increment(); + + handler.accept(false); + verify(ackFailures, times(3)).increment(); + assertThat(checkpointEntry.getCommittedOffset(), equalTo(reader.getReadOffset())); + } + + @Test + void run_with_codec_acknowledgements_creates_ack_set() throws Exception { + Path testFile = tempDir.resolve("codec-ack.log"); + Files.writeString(testFile, "data\n"); + FileChannel realChannel = FileChannel.open(testFile, StandardOpenOption.READ); + when(fileOps.openReadChannel(testFile)).thenReturn(realChannel); + when(rotationDetector.checkRotation(any(), any(), any(long.class))) + .thenReturn(RotationResult.NO_ROTATION); + + Counter bytesRead = mock(Counter.class); + Counter filesOpened = mock(Counter.class); + Counter filesClosed = mock(Counter.class); + Counter linesRead = mock(Counter.class); + Counter eventsEmitted = mock(Counter.class); + Timer backpressureTimer = mock(Timer.class); + when(metrics.getBytesRead()).thenReturn(bytesRead); + when(metrics.getFilesOpened()).thenReturn(filesOpened); + when(metrics.getFilesClosed()).thenReturn(filesClosed); + when(metrics.getLinesRead()).thenReturn(linesRead); + when(metrics.getEventsEmitted()).thenReturn(eventsEmitted); + lenient().when(metrics.getBackpressureTimer()).thenReturn(backpressureTimer); + lenient().when(metrics.getFileLagBytes()).thenReturn(new AtomicLong(0)); + + AcknowledgementSet ackSet = mock(AcknowledgementSet.class); + when(acknowledgementSetManager.create(any(), any(Duration.class))).thenReturn(ackSet); + + InputCodec mockCodec = mock(InputCodec.class); + doAnswer(inv -> { + Consumer> consumer = inv.getArgument(1); + Event mockEvent = mock(Event.class); + consumer.accept(new Record<>(mockEvent)); + return null; + }).when(mockCodec).parse(any(), any()); + + TailFileReaderContext context = new TailFileReaderContext( + buffer, eventFactory, fileOps, metrics, rotationDetector, + acknowledgementSetManager, true, StandardCharsets.UTF_8, + 4096, 1048576, 5000, Duration.ofSeconds(30), + Duration.ofSeconds(30), StartPosition.BEGINNING, false, + Duration.ofSeconds(30), 1, Duration.ofSeconds(5), 3, mockCodec); + + final TailFileReader reader = createReaderWithContext(testFile, context); + reader.run(); + + verify(ackSet, atLeastOnce()).add(any(Event.class)); + verify(ackSet, atLeastOnce()).complete(); + } + + @Test + void run_update_file_lag_handles_io_exception() throws Exception { + Path testFile = tempDir.resolve("lag-error.log"); + Files.writeString(testFile, "line1\n"); + FileChannel realChannel = FileChannel.open(testFile, StandardOpenOption.READ); + when(fileOps.openReadChannel(testFile)).thenReturn(realChannel); + when(rotationDetector.checkRotation(any(), any(), any(long.class))) + .thenReturn(RotationResult.NO_ROTATION); + stubReadMetrics(); + stubEventFactory(); + when(fileOps.size(testFile)).thenThrow(new IOException("disk error")); + + final TailFileReader reader = createReader(testFile); + reader.run(); + + assertThat(onCompleteCalled.get(), equalTo(true)); + } + + @Test + void getLastActivityMillis_returns_initial_value() throws Exception { + Path testFile = tempDir.resolve("activity.log"); + Files.writeString(testFile, ""); + long before = System.currentTimeMillis(); + final TailFileReader reader = createReader(testFile); + long after = System.currentTimeMillis(); + + assertThat(reader.getLastActivityMillis() >= before, equalTo(true)); + assertThat(reader.getLastActivityMillis() <= after, equalTo(true)); + } + + @Test + void run_emitLine_backpressure_records_timer_after_recovery() throws Exception { + Path testFile = tempDir.resolve("bp-timer.log"); + Files.writeString(testFile, "line1\n"); + FileChannel realChannel = FileChannel.open(testFile, StandardOpenOption.READ); + when(fileOps.openReadChannel(testFile)).thenReturn(realChannel); + when(rotationDetector.checkRotation(any(), any(), any(long.class))) + .thenReturn(RotationResult.NO_ROTATION); + stubReadMetrics(); + stubEventFactory(); + when(fileOps.size(testFile)).thenReturn(Files.size(testFile)); + + Timer backpressureTimer = mock(Timer.class); + when(metrics.getBackpressureTimer()).thenReturn(backpressureTimer); + Counter writeTimeouts = mock(Counter.class); + when(metrics.getWriteTimeouts()).thenReturn(writeTimeouts); + + doThrow(new TimeoutException("buffer full")) + .doNothing() + .when(buffer).write(any(Record.class), anyInt()); + + final TailFileReader reader = createReader(testFile); + reader.run(); + + verify(backpressureTimer).record(anyLong(), any(TimeUnit.class)); + } + + @Test + void run_codec_backpressure_interrupt_stops_reader() throws Exception { + Path testFile = tempDir.resolve("codec-bp-interrupt.log"); + Files.writeString(testFile, "data\n"); + FileChannel realChannel = FileChannel.open(testFile, StandardOpenOption.READ); + when(fileOps.openReadChannel(testFile)).thenReturn(realChannel); + when(rotationDetector.checkRotation(any(), any(), any(long.class))) + .thenReturn(RotationResult.NO_ROTATION); + + Counter bytesRead = mock(Counter.class); + Counter filesOpened = mock(Counter.class); + Counter filesClosed = mock(Counter.class); + Counter writeTimeouts = mock(Counter.class); + when(metrics.getBytesRead()).thenReturn(bytesRead); + when(metrics.getFilesOpened()).thenReturn(filesOpened); + when(metrics.getFilesClosed()).thenReturn(filesClosed); + when(metrics.getWriteTimeouts()).thenReturn(writeTimeouts); + lenient().when(metrics.getFileLagBytes()).thenReturn(new AtomicLong(0)); + + doThrow(new TimeoutException("buffer full")) + .when(buffer).write(any(Record.class), anyInt()); + + InputCodec mockCodec = mock(InputCodec.class); + doAnswer(inv -> { + Consumer> consumer = inv.getArgument(1); + Event mockEvent = mock(Event.class); + consumer.accept(new Record<>(mockEvent)); + return null; + }).when(mockCodec).parse(any(), any()); + + TailFileReaderContext context = createContextWithCodec(mockCodec); + + Thread readerThread = new Thread(() -> { + final TailFileReader reader = createReaderWithContext(testFile, context); + reader.run(); + }); + readerThread.start(); + Thread.sleep(300); + readerThread.interrupt(); + readerThread.join(5000); + + assertThat(readerThread.isAlive(), equalTo(false)); + } + + @Test + void run_emitLine_backpressure_interrupt_stops_reader() throws Exception { + Path testFile = tempDir.resolve("bp-interrupt.log"); + Files.writeString(testFile, "line1\n"); + FileChannel realChannel = FileChannel.open(testFile, StandardOpenOption.READ); + when(fileOps.openReadChannel(testFile)).thenReturn(realChannel); + when(rotationDetector.checkRotation(any(), any(), any(long.class))) + .thenReturn(RotationResult.NO_ROTATION); + + Counter bytesRead = mock(Counter.class); + Counter filesOpened = mock(Counter.class); + Counter filesClosed = mock(Counter.class); + Counter linesRead = mock(Counter.class); + Counter eventsEmitted = mock(Counter.class); + Counter writeTimeouts = mock(Counter.class); + Timer backpressureTimer = mock(Timer.class); + when(metrics.getBytesRead()).thenReturn(bytesRead); + when(metrics.getFilesOpened()).thenReturn(filesOpened); + when(metrics.getFilesClosed()).thenReturn(filesClosed); + lenient().when(metrics.getLinesRead()).thenReturn(linesRead); + lenient().when(metrics.getEventsEmitted()).thenReturn(eventsEmitted); + when(metrics.getWriteTimeouts()).thenReturn(writeTimeouts); + lenient().when(metrics.getBackpressureTimer()).thenReturn(backpressureTimer); + lenient().when(metrics.getFileLagBytes()).thenReturn(new AtomicLong(0)); + stubEventFactory(); + + doThrow(new TimeoutException("buffer full")) + .when(buffer).write(any(Record.class), anyInt()); + + Thread readerThread = new Thread(() -> { + final TailFileReader reader = createReader(testFile); + reader.run(); + }); + readerThread.start(); + Thread.sleep(200); + readerThread.interrupt(); + readerThread.join(5000); + + assertThat(readerThread.isAlive(), equalTo(false)); + } + + @Test + void run_drain_timeout_handles_io_exception_on_channel_size() throws Exception { + Path testFile = tempDir.resolve("drain-size-err.log"); + Files.writeString(testFile, "A".repeat(10000) + "\n"); + + FileIdentity newIdentity = mock(FileIdentity.class); + when(rotationDetector.checkRotation(any(), any(), any(long.class))) + .thenReturn(new RotationResult(RotationType.CREATE_RENAME, newIdentity)); + + FileChannel mockChannel = mock(FileChannel.class); + when(fileOps.openReadChannel(testFile)).thenReturn(mockChannel); + when(mockChannel.position(anyLong())).thenReturn(mockChannel); + when(mockChannel.read(any(ByteBuffer.class))).thenAnswer(inv -> { + Thread.sleep(5); + ByteBuffer buf = inv.getArgument(0); + byte[] data = "A".repeat(buf.remaining()).getBytes(); + buf.put(data, 0, Math.min(data.length, buf.remaining())); + return buf.position(); + }); + when(mockChannel.size()).thenThrow(new IOException("channel closed")); + + Counter filesRotated = mock(Counter.class); + Counter filesClosed = mock(Counter.class); + Counter bytesRead = mock(Counter.class); + Counter dataLossEvents = mock(Counter.class); + when(metrics.getFilesRotated()).thenReturn(filesRotated); + when(metrics.getFilesClosed()).thenReturn(filesClosed); + when(metrics.getFilesOpened()).thenReturn(mock(Counter.class)); + when(metrics.getBytesRead()).thenReturn(bytesRead); + lenient().when(metrics.getDataLossEvents()).thenReturn(dataLossEvents); + + TailFileReaderContext context = new TailFileReaderContext( + buffer, eventFactory, fileOps, metrics, rotationDetector, + acknowledgementSetManager, false, StandardCharsets.UTF_8, + 4096, 1048576, 5000, Duration.ofSeconds(30), + Duration.ofMillis(1), StartPosition.BEGINNING, false, + Duration.ofSeconds(30), 1000, + Duration.ofSeconds(5), 3, null); + + stubEventFactory(); + Counter linesRead = mock(Counter.class); + Counter eventsEmitted = mock(Counter.class); + lenient().when(metrics.getLinesRead()).thenReturn(linesRead); + lenient().when(metrics.getEventsEmitted()).thenReturn(eventsEmitted); + + fileIdentity = mock(FileIdentity.class); + final TailFileReader reader = new TailFileReader(testFile, fileIdentity, checkpointEntry, context, + () -> onCompleteCalled.set(true)); + reader.run(); + } + + @Test + void run_updateFileLagBytes_handles_io_exception_on_file_size() throws Exception { + Path testFile = tempDir.resolve("lag-err.log"); + Files.writeString(testFile, "line1\n"); + + when(rotationDetector.checkRotation(any(), any(), any(long.class))) + .thenReturn(RotationResult.NO_ROTATION); + + FileChannel realChannel = FileChannel.open(testFile, StandardOpenOption.READ); + when(fileOps.openReadChannel(testFile)).thenReturn(realChannel); + when(fileOps.size(testFile)).thenThrow(new IOException("disk error")); + + stubEventFactory(); + stubReadMetrics(); + + final TailFileReader reader = createReader(testFile); + reader.run(); + + assertThat(onCompleteCalled.get(), equalTo(true)); + } + + @Test + void run_with_acks_enabled_batch_not_full_does_not_complete_ack_set() throws Exception { + Path testFile = tempDir.resolve("ack-notfull.log"); + Files.writeString(testFile, "line1\n"); + FileChannel realChannel = FileChannel.open(testFile, StandardOpenOption.READ); + when(fileOps.openReadChannel(testFile)).thenReturn(realChannel); + when(rotationDetector.checkRotation(any(), any(), any(long.class))) + .thenReturn(RotationResult.NO_ROTATION); + stubReadMetrics(); + stubEventFactory(); + when(fileOps.size(testFile)).thenReturn(Files.size(testFile)); + + AcknowledgementSet ackSet = mock(AcknowledgementSet.class); + when(acknowledgementSetManager.create(any(), any(Duration.class))).thenReturn(ackSet); + + TailFileReaderContext context = createContextWithAcknowledgements(10000, Duration.ofHours(1), 3); + final TailFileReader reader = createReaderWithContext(testFile, context); + reader.run(); + + verify(ackSet, atLeastOnce()).add(any(Event.class)); + verify(ackSet, atLeastOnce()).complete(); + } + + @Test + void run_without_acks_does_not_create_ack_set() throws Exception { + Path testFile = tempDir.resolve("no-ack.log"); + Files.writeString(testFile, "line1\n"); + FileChannel realChannel = FileChannel.open(testFile, StandardOpenOption.READ); + when(fileOps.openReadChannel(testFile)).thenReturn(realChannel); + when(rotationDetector.checkRotation(any(), any(), any(long.class))) + .thenReturn(RotationResult.NO_ROTATION); + stubReadMetrics(); + stubEventFactory(); + when(fileOps.size(testFile)).thenReturn(Files.size(testFile)); + + final TailFileReader reader = createReader(testFile); + reader.run(); + + verify(acknowledgementSetManager, never()).create(any(), any(Duration.class)); + } + + @Test + void run_with_acks_enabled_zero_batch_count_does_not_trigger_batch_timeout() throws Exception { + Path testFile = tempDir.resolve("ack-zero-batch.log"); + Files.writeString(testFile, ""); + FileChannel realChannel = FileChannel.open(testFile, StandardOpenOption.READ); + when(fileOps.openReadChannel(testFile)).thenReturn(realChannel); + when(rotationDetector.checkRotation(any(), any(), any(long.class))) + .thenReturn(RotationResult.NO_ROTATION); + + Counter filesOpened = mock(Counter.class); + Counter filesClosed = mock(Counter.class); + Counter bytesRead = mock(Counter.class); + when(metrics.getFilesOpened()).thenReturn(filesOpened); + when(metrics.getFilesClosed()).thenReturn(filesClosed); + lenient().when(metrics.getBytesRead()).thenReturn(bytesRead); + lenient().when(metrics.getFileLagBytes()).thenReturn(new AtomicLong(0)); + + TailFileReaderContext context = createContextWithAcknowledgements(1000, Duration.ofMillis(0), 3); + final TailFileReader reader = createReaderWithContext(testFile, context); + reader.run(); + + verify(acknowledgementSetManager, never()).create(any(), any(Duration.class)); + } + + @Test + void run_with_codec_acks_disabled_does_not_create_ack_set() throws Exception { + Path testFile = tempDir.resolve("codec-no-ack.log"); + Files.writeString(testFile, "data\n"); + FileChannel realChannel = FileChannel.open(testFile, StandardOpenOption.READ); + when(fileOps.openReadChannel(testFile)).thenReturn(realChannel); + when(rotationDetector.checkRotation(any(), any(), any(long.class))) + .thenReturn(RotationResult.NO_ROTATION); + + Counter bytesRead = mock(Counter.class); + Counter filesOpened = mock(Counter.class); + Counter filesClosed = mock(Counter.class); + Counter linesRead = mock(Counter.class); + Counter eventsEmitted = mock(Counter.class); + Timer backpressureTimer = mock(Timer.class); + when(metrics.getBytesRead()).thenReturn(bytesRead); + when(metrics.getFilesOpened()).thenReturn(filesOpened); + when(metrics.getFilesClosed()).thenReturn(filesClosed); + when(metrics.getLinesRead()).thenReturn(linesRead); + when(metrics.getEventsEmitted()).thenReturn(eventsEmitted); + lenient().when(metrics.getBackpressureTimer()).thenReturn(backpressureTimer); + lenient().when(metrics.getFileLagBytes()).thenReturn(new AtomicLong(0)); + + InputCodec mockCodec = mock(InputCodec.class); + doAnswer(inv -> { + Consumer> consumer = inv.getArgument(1); + Event mockEvent = mock(Event.class); + consumer.accept(new Record<>(mockEvent)); + return null; + }).when(mockCodec).parse(any(), any()); + + TailFileReaderContext context = createContextWithCodec(mockCodec); + final TailFileReader reader = createReaderWithContext(testFile, context); + reader.run(); + + verify(acknowledgementSetManager, never()).create(any(), any(Duration.class)); + } + + @Test + void run_emitLine_thread_interrupted_during_backpressure_exits() throws Exception { + Path testFile = tempDir.resolve("bp-interrupt-line.log"); + Files.writeString(testFile, "line1\n"); + FileChannel realChannel = FileChannel.open(testFile, StandardOpenOption.READ); + when(fileOps.openReadChannel(testFile)).thenReturn(realChannel); + when(rotationDetector.checkRotation(any(), any(), any(long.class))) + .thenReturn(RotationResult.NO_ROTATION); + + Counter bytesRead = mock(Counter.class); + Counter filesOpened = mock(Counter.class); + Counter filesClosed = mock(Counter.class); + Counter writeTimeouts = mock(Counter.class); + when(metrics.getBytesRead()).thenReturn(bytesRead); + when(metrics.getFilesOpened()).thenReturn(filesOpened); + when(metrics.getFilesClosed()).thenReturn(filesClosed); + when(metrics.getWriteTimeouts()).thenReturn(writeTimeouts); + lenient().when(metrics.getFileLagBytes()).thenReturn(new AtomicLong(0)); + stubEventFactory(); + + doThrow(new TimeoutException("buffer full")) + .when(buffer).write(any(Record.class), anyInt()); + + Thread readerThread = new Thread(() -> { + final TailFileReader reader = createReader(testFile); + reader.run(); + }); + readerThread.start(); + Thread.sleep(300); + readerThread.interrupt(); + readerThread.join(5000); + + assertThat(readerThread.isAlive(), equalTo(false)); + } + + @Test + void run_codec_thread_interrupted_during_backpressure_exits() throws Exception { + Path testFile = tempDir.resolve("codec-bp-int.log"); + Files.writeString(testFile, "data\n"); + FileChannel realChannel = FileChannel.open(testFile, StandardOpenOption.READ); + when(fileOps.openReadChannel(testFile)).thenReturn(realChannel); + when(rotationDetector.checkRotation(any(), any(), any(long.class))) + .thenReturn(RotationResult.NO_ROTATION); + + Counter bytesRead = mock(Counter.class); + Counter filesOpened = mock(Counter.class); + Counter filesClosed = mock(Counter.class); + Counter writeTimeouts = mock(Counter.class); + when(metrics.getBytesRead()).thenReturn(bytesRead); + when(metrics.getFilesOpened()).thenReturn(filesOpened); + when(metrics.getFilesClosed()).thenReturn(filesClosed); + when(metrics.getWriteTimeouts()).thenReturn(writeTimeouts); + lenient().when(metrics.getFileLagBytes()).thenReturn(new AtomicLong(0)); + + doThrow(new TimeoutException("buffer full")) + .when(buffer).write(any(Record.class), anyInt()); + + InputCodec mockCodec = mock(InputCodec.class); + doAnswer(inv -> { + Consumer> consumer = inv.getArgument(1); + Event mockEvent = mock(Event.class); + consumer.accept(new Record<>(mockEvent)); + return null; + }).when(mockCodec).parse(any(), any()); + + TailFileReaderContext context = createContextWithCodec(mockCodec); + + Thread readerThread = new Thread(() -> { + final TailFileReader reader = createReaderWithContext(testFile, context); + reader.run(); + }); + readerThread.start(); + Thread.sleep(300); + readerThread.interrupt(); + readerThread.join(5000); + + assertThat(readerThread.isAlive(), equalTo(false)); + } + + @Test + void run_multi_byte_character_split_across_reads_triggers_decoder_carryover() throws Exception { + Path testFile = tempDir.resolve("multibyte.log"); + String multiByteContent = "\u00E9\u00E9\u00E9\u00E9\n"; + Files.writeString(testFile, multiByteContent); + FileChannel realChannel = FileChannel.open(testFile, StandardOpenOption.READ); + when(fileOps.openReadChannel(testFile)).thenReturn(realChannel); + when(rotationDetector.checkRotation(any(), any(), any(long.class))) + .thenReturn(RotationResult.NO_ROTATION); + stubReadMetrics(); + stubEventFactory(); + + final TailFileReader reader = createReader(testFile, 3, 1048576, false); + reader.run(); + + verify(buffer, atLeastOnce()).write(any(Record.class), eq(5000)); + } + + @Test + void run_skippingToNewline_skips_remainder_after_max_line_truncation() throws Exception { + Path testFile = tempDir.resolve("skip-newline.log"); + String longLine = "A".repeat(60) + "\nsecond\n"; + Files.writeString(testFile, longLine); + FileChannel realChannel = FileChannel.open(testFile, StandardOpenOption.READ); + when(fileOps.openReadChannel(testFile)).thenReturn(realChannel); + when(rotationDetector.checkRotation(any(), any(), any(long.class))) + .thenReturn(RotationResult.NO_ROTATION); + stubReadMetrics(); + Counter linesTruncated = mock(Counter.class); + when(metrics.getLinesTruncated()).thenReturn(linesTruncated); + stubEventFactory(); + + final TailFileReader reader = createReader(testFile, 20, 10, false); + reader.run(); + + verify(linesTruncated, atLeastOnce()).increment(); + verify(buffer, atLeastOnce()).write(any(Record.class), eq(5000)); + } + + @Test + void run_backpressure_retry_timeout_exceeded_logs_data_loss() throws Exception { + Path testFile = tempDir.resolve("bp-timeout.log"); + Files.writeString(testFile, "line1\n"); + FileChannel realChannel = FileChannel.open(testFile, StandardOpenOption.READ); + when(fileOps.openReadChannel(testFile)).thenReturn(realChannel); + when(rotationDetector.checkRotation(any(), any(), any(long.class))) + .thenReturn(RotationResult.NO_ROTATION); + + Counter bytesRead = mock(Counter.class); + Counter filesOpened = mock(Counter.class); + Counter filesClosed = mock(Counter.class); + Counter writeTimeouts = mock(Counter.class); + Counter dataLossEvents = mock(Counter.class); + when(metrics.getBytesRead()).thenReturn(bytesRead); + when(metrics.getFilesOpened()).thenReturn(filesOpened); + when(metrics.getFilesClosed()).thenReturn(filesClosed); + when(metrics.getWriteTimeouts()).thenReturn(writeTimeouts); + when(metrics.getDataLossEvents()).thenReturn(dataLossEvents); + lenient().when(metrics.getFileLagBytes()).thenReturn(new AtomicLong(0)); + stubEventFactory(); + + doThrow(new TimeoutException("buffer full")) + .when(buffer).write(any(Record.class), anyInt()); + + TailFileReaderContext context = new TailFileReaderContext( + buffer, eventFactory, fileOps, metrics, rotationDetector, + acknowledgementSetManager, false, StandardCharsets.UTF_8, + 4096, 1048576, 5000, Duration.ofMillis(100), + Duration.ofSeconds(30), StartPosition.BEGINNING, false, + Duration.ofMillis(100), 1000, + Duration.ofSeconds(5), 3, null); + + fileIdentity = mock(FileIdentity.class); + final TailFileReader reader = new TailFileReader(testFile, fileIdentity, checkpointEntry, context, + () -> onCompleteCalled.set(true)); + reader.run(); + + verify(dataLossEvents).increment(); + assertThat(onCompleteCalled.get(), equalTo(true)); + } +} diff --git a/data-prepper-plugins/file-source/src/test/resources/test-file-source-invalid-json.tst b/data-prepper-plugins/file-source/src/test/resources/test-file-source-invalid-json.tst new file mode 100644 index 0000000000..5cdfe97552 --- /dev/null +++ b/data-prepper-plugins/file-source/src/test/resources/test-file-source-invalid-json.tst @@ -0,0 +1,2 @@ +{"test_key: test_value"} +{"second_test_key": "second_test_value" \ No newline at end of file diff --git a/data-prepper-plugins/file-source/src/test/resources/test-file-source-json.tst b/data-prepper-plugins/file-source/src/test/resources/test-file-source-json.tst new file mode 100644 index 0000000000..0d0d8037ab --- /dev/null +++ b/data-prepper-plugins/file-source/src/test/resources/test-file-source-json.tst @@ -0,0 +1,2 @@ +{"test_key": "test_value"} +{"second_test_key": "second_test_value"} \ No newline at end of file diff --git a/data-prepper-plugins/file-source/src/test/resources/test-file-source-plain.tst b/data-prepper-plugins/file-source/src/test/resources/test-file-source-plain.tst new file mode 100644 index 0000000000..226beb530d --- /dev/null +++ b/data-prepper-plugins/file-source/src/test/resources/test-file-source-plain.tst @@ -0,0 +1,2 @@ +THIS IS A PLAINTEXT LINE +THIS IS ANOTHER PLAINTEXT LINE \ No newline at end of file diff --git a/settings.gradle b/settings.gradle index b5ed9be574..1db932b923 100644 --- a/settings.gradle +++ b/settings.gradle @@ -191,6 +191,7 @@ include 'data-prepper-plugins:dissect-processor' include 'data-prepper-plugins:dynamodb-source' include 'data-prepper-plugins:decompress-processor' include 'data-prepper-plugins:split-event-processor' +include 'data-prepper-plugins:file-source' include 'data-prepper-plugins:flatten-processor' include 'data-prepper-plugins:mongodb' include 'data-prepper-plugins:rds-source' From bf2f1701f9c13337e535bf8fb9b9068b25381a3b Mon Sep 17 00:00:00 2001 From: Srikanth Padakanti Date: Thu, 7 May 2026 19:42:07 -0500 Subject: [PATCH 02/21] Align file source config with RFC and fix code quality issues Signed-off-by: Srikanth Padakanti --- .../plugins/source/file/FileSourceTailIT.java | 7 ++- .../source/file/CheckpointRegistry.java | 13 ++++- .../plugins/source/file/CheckpointStatus.java | 19 +++++-- .../plugins/source/file/FileFormat.java | 29 ++++++----- .../plugins/source/file/FileIdentity.java | 3 +- .../plugins/source/file/FileSource.java | 16 +++--- .../plugins/source/file/FileSourceConfig.java | 22 ++++---- .../plugins/source/file/RecordType.java | 50 +++++++++++++++++++ .../plugins/source/file/TailFileReader.java | 15 +++--- .../plugins/source/file/FileIdentityTest.java | 10 +++- .../source/file/FileSourceConfigTest.java | 16 +++--- .../plugins/source/file/FileSourceTests.java | 27 ++++------ .../source/file/RotationDetectorTest.java | 9 ++++ .../source/file/TailFileReaderPoolTest.java | 8 +++ .../source/file/TailFileReaderTest.java | 8 ++- 15 files changed, 179 insertions(+), 73 deletions(-) create mode 100644 data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/RecordType.java diff --git a/data-prepper-plugins/file-source/src/integrationTest/java/org/opensearch/dataprepper/plugins/source/file/FileSourceTailIT.java b/data-prepper-plugins/file-source/src/integrationTest/java/org/opensearch/dataprepper/plugins/source/file/FileSourceTailIT.java index 7caf011e23..1293acc502 100644 --- a/data-prepper-plugins/file-source/src/integrationTest/java/org/opensearch/dataprepper/plugins/source/file/FileSourceTailIT.java +++ b/data-prepper-plugins/file-source/src/integrationTest/java/org/opensearch/dataprepper/plugins/source/file/FileSourceTailIT.java @@ -278,8 +278,11 @@ void tail_mode_includes_file_metadata() throws Exception { await().atMost(WAIT_TIMEOUT).untilAsserted(() -> { assertThat(capturedRecords, hasSize(1)); final Event event = (Event) capturedRecords.get(0).getData(); - assertThat(event.get("file_path", String.class), + @SuppressWarnings("unchecked") + final Map fileMetadata = (Map) event.get("file", Object.class); + assertThat(fileMetadata.get("path"), equalTo(logFile.toAbsolutePath().toString())); + assertThat(fileMetadata.get("name"), equalTo("meta.log")); }); } @@ -792,6 +795,8 @@ private FileSourceConfig buildConfig(final String path, final List paths configMap.put("start_position", startPosition != null ? startPosition : "beginning"); configMap.put("reader_threads", 2); configMap.put("include_file_metadata", true); + configMap.put("poll_interval", "PT0.5S"); + configMap.put("rotate_wait", "PT0.5S"); if (path != null) { configMap.put("path", path); } diff --git a/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/CheckpointRegistry.java b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/CheckpointRegistry.java index 177e29ce50..13c11001c5 100644 --- a/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/CheckpointRegistry.java +++ b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/CheckpointRegistry.java @@ -128,7 +128,18 @@ private void load() { } catch (final FileNotFoundException | NoSuchFileException e) { LOG.debug("No existing checkpoint file at {}. Starting with empty state.", checkpointFile); } catch (final IOException e) { - LOG.warn("Corrupt or unreadable checkpoint file at {}. Starting with empty state.", checkpointFile, e); + LOG.warn("Corrupt or unreadable checkpoint file at {}. Renaming to .corrupt and starting fresh.", checkpointFile, e); + renameCorruptFile(); + } + } + + private void renameCorruptFile() { + try { + final Path corruptPath = checkpointFile.resolveSibling(checkpointFile.getFileName() + ".corrupt"); + Files.move(checkpointFile, corruptPath, StandardCopyOption.REPLACE_EXISTING); + LOG.info("Renamed corrupt checkpoint file to {}", corruptPath); + } catch (final IOException renameEx) { + LOG.warn("Failed to rename corrupt checkpoint file at {}", checkpointFile, renameEx); } } diff --git a/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/CheckpointStatus.java b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/CheckpointStatus.java index d797c3e986..3d7874d149 100644 --- a/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/CheckpointStatus.java +++ b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/CheckpointStatus.java @@ -13,10 +13,17 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonValue; +import java.util.Map; +import java.util.stream.Collectors; +import java.util.stream.Stream; + public enum CheckpointStatus { ACTIVE("ACTIVE"), COMPLETED("COMPLETED"); + private static final Map NAMES_MAP = Stream.of(values()) + .collect(Collectors.toMap(CheckpointStatus::getValue, v -> v)); + private final String value; CheckpointStatus(final String value) { @@ -30,11 +37,13 @@ public String getValue() { @JsonCreator public static CheckpointStatus fromString(final String value) { - for (final CheckpointStatus status : values()) { - if (status.value.equalsIgnoreCase(value)) { - return status; - } + if (value == null) { + throw new IllegalArgumentException("Invalid checkpoint status: null. Valid values are: " + NAMES_MAP.keySet()); + } + final CheckpointStatus status = NAMES_MAP.get(value.toUpperCase()); + if (status == null) { + throw new IllegalArgumentException("Invalid checkpoint status: " + value + ". Valid values are: " + NAMES_MAP.keySet()); } - throw new IllegalArgumentException("Invalid checkpoint status: " + value); + return status; } } diff --git a/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileFormat.java b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileFormat.java index 49322c7fb8..8f79397031 100644 --- a/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileFormat.java +++ b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileFormat.java @@ -10,22 +10,20 @@ package org.opensearch.dataprepper.plugins.source.file; -import java.util.Arrays; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonValue; + import java.util.Map; -import java.util.function.Function; import java.util.stream.Collectors; +import java.util.stream.Stream; -/** - * An enumm to represent the file formats supported in Data Prepper's file source. - * @since 1.2 - */ public enum FileFormat { PLAIN("plain"), JSON("json"); - private static final Map NAMES_MAP = Arrays.stream(FileFormat.values()) - .collect(Collectors.toMap(FileFormat::toString, Function.identity())); + private static final Map NAMES_MAP = Stream.of(values()) + .collect(Collectors.toMap(FileFormat::toString, v -> v)); private final String name; @@ -33,14 +31,21 @@ public enum FileFormat { this.name = name; } + @JsonValue + @Override public String toString() { return this.name; } - public static FileFormat getByName(final String name) { + @JsonCreator + public static FileFormat fromString(final String name) { if (name == null) { - return PLAIN; + throw new IllegalArgumentException("Invalid format: null. Valid values are: " + NAMES_MAP.keySet()); + } + final FileFormat value = NAMES_MAP.get(name.toLowerCase()); + if (value == null) { + throw new IllegalArgumentException("Invalid format: " + name + ". Valid values are: " + NAMES_MAP.keySet()); } - return NAMES_MAP.get(name.toLowerCase()); + return value; } -} \ No newline at end of file +} diff --git a/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileIdentity.java b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileIdentity.java index 6d220c8f98..d1247a3b92 100644 --- a/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileIdentity.java +++ b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileIdentity.java @@ -41,7 +41,8 @@ public static FileIdentity from(final Path path, final FileSystemOperations file final BasicFileAttributes attrs = fileOps.readAttributes(path); final Object fileKey = attrs.fileKey(); if (fileKey != null) { - return new FileIdentity("inode:" + fileKey, path); + final String creationTime = attrs.creationTime().toString(); + return new FileIdentity("inode:" + fileKey + ":created:" + creationTime, path); } return buildFallbackIdentity(path, fileOps, attrs, fingerprintBytes); } catch (final IOException e) { diff --git a/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileSource.java b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileSource.java index dd585aea8d..0561e13bba 100644 --- a/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileSource.java +++ b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileSource.java @@ -134,9 +134,13 @@ private void startTailing(final Buffer> buffer) { final FileSystemOperations fileOps = new DefaultFileSystemOperations(); final String checkpointPath = fileSourceConfig.getCheckpointFile(); - final Path cpFile = checkpointPath != null - ? Paths.get(checkpointPath) - : Paths.get(System.getProperty("java.io.tmpdir"), "data-prepper-file-source-checkpoint.json"); + final Path cpFile; + if (checkpointPath != null) { + cpFile = Paths.get(checkpointPath); + } else { + LOG.warn("No checkpoint_file configured. Checkpoint state will not be persisted across restarts."); + cpFile = null; + } checkpointRegistry = new CheckpointRegistry( cpFile, @@ -262,7 +266,7 @@ private Record getEventRecordFromLine(final String line) { return new Record<>( eventFactory.eventBuilder(EventBuilder.class) - .withEventType(fileSourceConfig.getRecordType()) + .withEventType(fileSourceConfig.getRecordType().toString()) .withData(structuredLine) .build()); } @@ -279,9 +283,9 @@ private Map parseJson(final String jsonString) { } private void writeLineAsEventOrString(final String line, final Buffer> buffer) throws TimeoutException, IllegalArgumentException { - if (fileSourceConfig.getRecordType().equals(FileSourceConfig.EVENT_TYPE)) { + if (fileSourceConfig.getRecordType() == RecordType.EVENT) { buffer.write(getEventRecordFromLine(line), writeTimeout); - } else if (fileSourceConfig.getRecordType().equals(FileSourceConfig.DEFAULT_TYPE)) { + } else if (fileSourceConfig.getRecordType() == RecordType.STRING) { buffer.write(new Record<>(line), writeTimeout); } } diff --git a/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileSourceConfig.java b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileSourceConfig.java index ca6784f9af..60cea38326 100644 --- a/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileSourceConfig.java +++ b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileSourceConfig.java @@ -10,7 +10,6 @@ package org.opensearch.dataprepper.plugins.source.file; -import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; import jakarta.validation.constraints.AssertTrue; @@ -41,10 +40,10 @@ public class FileSourceConfig { private boolean tail = false; @JsonProperty(ATTRIBUTE_FORMAT) - private String format = DEFAULT_FORMAT; + private FileFormat format = FileFormat.PLAIN; @JsonProperty(ATTRIBUTE_TYPE) - private String recordType = DEFAULT_TYPE; + private RecordType recordType = RecordType.STRING; @JsonProperty("codec") private PluginModel codec; @@ -86,25 +85,25 @@ public class FileSourceConfig { private Duration checkpointInterval = Duration.ofSeconds(15); @JsonProperty("checkpoint_cleanup_after") - private Duration checkpointCleanupAfter = Duration.ofHours(24); + private Duration checkpointCleanupAfter = Duration.ofDays(7); @JsonProperty("fingerprint_bytes") private int fingerprintBytes = 1024; @JsonProperty("close_inactive") - private Duration closeInactive = Duration.ofMinutes(5); + private Duration closeInactive = Duration.ofMinutes(30); @JsonProperty("close_removed") private boolean closeRemoved = true; @JsonProperty("batch_size") - private int batchSize = 1000; + private int batchSize = 100; @JsonProperty("batch_timeout") private Duration batchTimeout = Duration.ofSeconds(5); @JsonProperty("acknowledgment_timeout") - private Duration acknowledgmentTimeout = Duration.ofSeconds(30); + private Duration acknowledgmentTimeout = Duration.ofSeconds(60); @JsonProperty("max_acknowledgment_retries") private int maxAcknowledgmentRetries = 3; @@ -141,12 +140,11 @@ public List getAllPaths() { return allPaths; } - @JsonIgnore public FileFormat getFormat() { - return FileFormat.getByName(format); + return format; } - public String getRecordType() { + public RecordType getRecordType() { return recordType; } @@ -259,12 +257,10 @@ void validate() { Preconditions.checkArgument(filePathToRead != null, "path is required when tail is disabled. Use paths with tail: true for glob patterns."); } - Preconditions.checkArgument(EVENT_TYPE.equals(recordType) || DEFAULT_TYPE.equals(recordType), "Invalid type: must be either [event] or [string]"); - Preconditions.checkArgument(DEFAULT_FORMAT.equals(format) || "json".equals(format), "Invalid file format. Options are [json] and [plain]"); } @AssertTrue(message = "The file source requires recordType to be event when using a codec.") boolean codeRequiresRecordTypeEvent() { - return codec == null || EVENT_TYPE.equals(recordType); + return codec == null || recordType == RecordType.EVENT; } } diff --git a/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/RecordType.java b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/RecordType.java new file mode 100644 index 0000000000..3716649e22 --- /dev/null +++ b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/RecordType.java @@ -0,0 +1,50 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + +package org.opensearch.dataprepper.plugins.source.file; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonValue; + +import java.util.Map; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +public enum RecordType { + STRING("string"), + EVENT("event"); + + private static final Map NAMES_MAP = Stream.of(values()) + .collect(Collectors.toMap(RecordType::toString, v -> v)); + + private final String name; + + RecordType(final String name) { + this.name = name; + } + + @JsonValue + @Override + public String toString() { + return name; + } + + @JsonCreator + public static RecordType fromString(final String name) { + if (name == null) { + throw new IllegalArgumentException("Invalid record_type: null. Valid values are: " + NAMES_MAP.keySet()); + } + final RecordType value = NAMES_MAP.get(name.toLowerCase()); + if (value == null) { + throw new IllegalArgumentException("Invalid record_type: " + name + ". Valid values are: " + NAMES_MAP.keySet()); + } + return value; + } +} diff --git a/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/TailFileReader.java b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/TailFileReader.java index 5281422120..64be1715ae 100644 --- a/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/TailFileReader.java +++ b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/TailFileReader.java @@ -48,8 +48,10 @@ public final class TailFileReader implements Runnable { private static final long BACK_PRESSURE_SLEEP_MILLIS = 100; private static final String MESSAGE_KEY = "message"; private static final String EVENT_TYPE = "event"; - private static final String FILE_PATH_KEY = "file_path"; - private static final String FILE_IDENTITY_KEY = "file_identity"; + private static final String FILE_KEY = "file"; + private static final String FILE_PATH_KEY = "path"; + private static final String FILE_NAME_KEY = "name"; + private static final String OFFSET_KEY = "offset"; private final FileIdentity fileIdentity; private final Path path; @@ -79,7 +81,6 @@ public final class TailFileReader implements Runnable { private final AtomicLong readOffset; private final StringBuilder partialLine; private final String cachedAbsolutePath; - private final String cachedFileIdentityString; private volatile long lastActivityMillis; private boolean skippingToNewline; private volatile long currentBatchEndOffset; @@ -135,7 +136,6 @@ public TailFileReader(final Path path, } this.partialLine = new StringBuilder(); this.cachedAbsolutePath = path.toAbsolutePath().toString(); - this.cachedFileIdentityString = fileIdentity.toString(); this.currentBatchCount = 0; this.batchStartOffset = readOffset.get(); this.batchOpenedAtMillis = System.currentTimeMillis(); @@ -376,8 +376,11 @@ private void emitLine(final String line) { final Map data = new HashMap<>(); data.put(MESSAGE_KEY, line); if (includeFileMetadata) { - data.put(FILE_PATH_KEY, cachedAbsolutePath); - data.put(FILE_IDENTITY_KEY, cachedFileIdentityString); + final Map fileMetadata = new HashMap<>(); + fileMetadata.put(FILE_PATH_KEY, cachedAbsolutePath); + fileMetadata.put(FILE_NAME_KEY, path.getFileName().toString()); + data.put(FILE_KEY, fileMetadata); + data.put(OFFSET_KEY, readOffset.get()); } final Event event = eventFactory.eventBuilder(EventBuilder.class) diff --git a/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/FileIdentityTest.java b/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/FileIdentityTest.java index 979daadaa1..6c70401aab 100644 --- a/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/FileIdentityTest.java +++ b/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/FileIdentityTest.java @@ -10,6 +10,7 @@ package org.opensearch.dataprepper.plugins.source.file; +import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; import org.junit.jupiter.api.io.TempDir; @@ -31,6 +32,7 @@ import static org.hamcrest.Matchers.notNullValue; import static org.mockito.Mockito.when; import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.mockito.Mockito.lenient; import static org.mockito.Mockito.mock; @ExtendWith(MockitoExtension.class) @@ -47,6 +49,11 @@ class FileIdentityTest { @Mock private BasicFileAttributes attrs; + @BeforeEach + void setUp() { + lenient().when(attrs.creationTime()).thenReturn(FileTime.from(Instant.parse("2025-01-01T00:00:00Z"))); + } + @Test void fromReturnsInodeBasedIdentityWhenFileKeyPresent() throws IOException { final Path testFile = tempDir.resolve("test.log"); @@ -59,7 +66,7 @@ void fromReturnsInodeBasedIdentityWhenFileKeyPresent() throws IOException { final FileIdentity identity = FileIdentity.from(testFile, fileOps, FINGERPRINT_BYTES); assertThat(identity, notNullValue()); - assertThat(identity.toString(), containsString("inode:12345")); + assertThat(identity.toString(), containsString("inode:12345:created:")); assertThat(identity.getPath(), equalTo(testFile)); } @@ -159,6 +166,7 @@ void differentIdentitiesAreNotEqual() throws IOException { when(fileOps.readAttributes(fileB)).thenReturn(attrsB); when(attrsB.fileKey()).thenReturn("inode-2"); + when(attrsB.creationTime()).thenReturn(FileTime.from(Instant.parse("2025-02-01T00:00:00Z"))); final FileIdentity identityA = FileIdentity.from(fileA, fileOps, FINGERPRINT_BYTES); final FileIdentity identityB = FileIdentity.from(fileB, fileOps, FINGERPRINT_BYTES); diff --git a/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/FileSourceConfigTest.java b/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/FileSourceConfigTest.java index 37d65b61f4..ce2cf94b18 100644 --- a/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/FileSourceConfigTest.java +++ b/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/FileSourceConfigTest.java @@ -34,7 +34,7 @@ class FileSourceConfigTest { private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); @ParameterizedTest - @ValueSource(strings = {FileSourceConfig.EVENT_TYPE, FileSourceConfig.DEFAULT_FORMAT}) + @ValueSource(strings = {FileSourceConfig.EVENT_TYPE, FileSourceConfig.DEFAULT_TYPE}) void codeRequiresRecordTypeEvent_returns_true_if_no_codec(final String recordType) { final Map fileConfigMap = Map.of(FileSourceConfig.ATTRIBUTE_TYPE, recordType); final FileSourceConfig objectUnderTest = OBJECT_MAPPER.convertValue(fileConfigMap, FileSourceConfig.class); @@ -45,7 +45,7 @@ void codeRequiresRecordTypeEvent_returns_true_if_no_codec(final String recordTyp @ParameterizedTest @CsvSource({ FileSourceConfig.EVENT_TYPE + ",true", - FileSourceConfig.DEFAULT_FORMAT + ",false" + FileSourceConfig.DEFAULT_TYPE + ",false" }) void codeRequiresRecordTypeEvent_returns_expected_value_when_there_is_a_codec(final String recordType, final boolean expected) { final Map fileConfigMap = Map.of( @@ -168,13 +168,13 @@ void default_config_returns_expected_values() { assertThat(config.getRotationDrainTimeout(), equalTo(Duration.ofSeconds(30))); assertThat(config.getCheckpointFile(), nullValue()); assertThat(config.getCheckpointInterval(), equalTo(Duration.ofSeconds(15))); - assertThat(config.getCheckpointCleanupAfter(), equalTo(Duration.ofHours(24))); + assertThat(config.getCheckpointCleanupAfter(), equalTo(Duration.ofDays(7))); assertThat(config.getFingerprintBytes(), equalTo(1024)); - assertThat(config.getCloseInactive(), equalTo(Duration.ofMinutes(5))); + assertThat(config.getCloseInactive(), equalTo(Duration.ofMinutes(30))); assertThat(config.isCloseRemoved(), equalTo(true)); - assertThat(config.getBatchSize(), equalTo(1000)); + assertThat(config.getBatchSize(), equalTo(100)); assertThat(config.getBatchTimeout(), equalTo(Duration.ofSeconds(5))); - assertThat(config.getAcknowledgmentTimeout(), equalTo(Duration.ofSeconds(30))); + assertThat(config.getAcknowledgmentTimeout(), equalTo(Duration.ofSeconds(60))); assertThat(config.getMaxAcknowledgmentRetries(), equalTo(3)); assertThat(config.isIncludeFileMetadata(), equalTo(false)); assertThat(config.getMaxLineLength(), equalTo(1048576)); @@ -219,7 +219,7 @@ void validate_fails_when_tail_true_and_filePathToRead_is_empty_and_paths_is_null } @Test - void getFormat_returns_plain_when_format_is_null() { - assertThat(FileFormat.getByName(null), equalTo(FileFormat.PLAIN)); + void getFormat_throws_when_format_is_null() { + assertThrows(IllegalArgumentException.class, () -> FileFormat.fromString(null)); } } diff --git a/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/FileSourceTests.java b/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/FileSourceTests.java index ea904b190b..3333f5813b 100644 --- a/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/FileSourceTests.java +++ b/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/FileSourceTests.java @@ -36,8 +36,6 @@ import org.opensearch.dataprepper.plugins.buffer.blockingbuffer.BlockingBuffer; import org.opensearch.dataprepper.plugins.buffer.blockingbuffer.BlockingBufferConfig; import org.opensearch.dataprepper.plugins.codec.CompressionOption; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import java.io.FileInputStream; import java.io.IOException; @@ -69,7 +67,6 @@ @ExtendWith(MockitoExtension.class) public class FileSourceTests { - private static final Logger LOG = LoggerFactory.getLogger(FileSourceTests.class); private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); private static final TypeReference> MAP_TYPE_REFERENCE = new TypeReference>() { }; @@ -209,7 +206,7 @@ public void testFileSourceWithNullFilePathThrowsNullPointerException() { } @Test - public void testFileWithPlainTextAddsEventsToBufferCorrectly() { + public void testFileWithPlainTextAddsEventsToBufferCorrectly() throws JsonProcessingException { fileSource = createObjectUnderTest(); fileSource.start(buffer); @@ -220,7 +217,7 @@ public void testFileWithPlainTextAddsEventsToBufferCorrectly() { } @Test - public void testFileWithJSONAddsEventsToBufferCorrectly() { + public void testFileWithJSONAddsEventsToBufferCorrectly() throws JsonProcessingException { pluginSettings.put(FileSourceConfig.ATTRIBUTE_PATH, TEST_FILE_PATH_JSON); pluginSettings.put(FileSourceConfig.ATTRIBUTE_FORMAT, "json"); @@ -234,7 +231,7 @@ public void testFileWithJSONAddsEventsToBufferCorrectly() { } @Test - public void testFileWithInvalidJSONAddsEventsToBufferAsPlainText() { + public void testFileWithInvalidJSONAddsEventsToBufferAsPlainText() throws JsonProcessingException { pluginSettings.put(FileSourceConfig.ATTRIBUTE_PATH, TEST_FILE_PATH_INVALID_JSON); pluginSettings.put(FileSourceConfig.ATTRIBUTE_FORMAT, "json"); fileSource = createObjectUnderTest(); @@ -272,7 +269,7 @@ public void testNonSupportedFileTypeThrowsIllegalArgumentException() { assertThrows(IllegalArgumentException.class, FileSourceTests.this::createObjectUnderTest); } - void assertExpectedRecordsAreEqual(final List> expectedEvents, final List> actualEvents) { + void assertExpectedRecordsAreEqual(final List> expectedEvents, final List> actualEvents) throws JsonProcessingException { for (int i = 0; i < expectedEvents.size(); i++) { assertThat(actualEvents.get(i), notNullValue()); assertThat(actualEvents.get(i).getData(), notNullValue()); @@ -280,16 +277,12 @@ void assertExpectedRecordsAreEqual(final List> expectedEvents, fi } } - void assertEventRecordsAreEqual(final Record first, final Record second) { - try { - final Event firstEvent = (Event) first.getData(); - final Event secondEvent = (Event) second.getData(); - final Map recordMapFirst = OBJECT_MAPPER.readValue(firstEvent.toJsonString(), MAP_TYPE_REFERENCE); - final Map recordMapSecond = OBJECT_MAPPER.readValue(secondEvent.toJsonString(), MAP_TYPE_REFERENCE); - assertThat(recordMapFirst, is(equalTo(recordMapSecond))); - } catch (JsonProcessingException e) { - LOG.error("Unable to parse Event as JSON"); - } + void assertEventRecordsAreEqual(final Record first, final Record second) throws JsonProcessingException { + final Event firstEvent = (Event) first.getData(); + final Event secondEvent = (Event) second.getData(); + final Map recordMapFirst = OBJECT_MAPPER.readValue(firstEvent.toJsonString(), MAP_TYPE_REFERENCE); + final Map recordMapSecond = OBJECT_MAPPER.readValue(secondEvent.toJsonString(), MAP_TYPE_REFERENCE); + assertThat(recordMapFirst, is(equalTo(recordMapSecond))); } private Record createRecordEventWithKeyValuePair(final String key, final String value) { diff --git a/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/RotationDetectorTest.java b/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/RotationDetectorTest.java index ac5faea5cd..af5b467a8e 100644 --- a/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/RotationDetectorTest.java +++ b/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/RotationDetectorTest.java @@ -22,6 +22,8 @@ import java.nio.file.NoSuchFileException; import java.nio.file.Path; import java.nio.file.attribute.BasicFileAttributes; +import java.nio.file.attribute.FileTime; +import java.time.Instant; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.equalTo; @@ -64,6 +66,7 @@ void noRotationWhenIdentityMatchesAndSizeNotShrunk() throws IOException { final BasicFileAttributes attrs = mock(BasicFileAttributes.class); when(attrs.fileKey()).thenReturn("inode-42"); + when(attrs.creationTime()).thenReturn(FileTime.from(Instant.EPOCH)); when(fileOps.readAttributes(testFile)).thenReturn(attrs); when(fileOps.size(testFile)).thenReturn(100L); @@ -82,11 +85,13 @@ void createRenameRotationWhenIdentityChanges() throws IOException { final BasicFileAttributes oldAttrs = mock(BasicFileAttributes.class); when(oldAttrs.fileKey()).thenReturn("inode-1"); + when(oldAttrs.creationTime()).thenReturn(FileTime.from(Instant.EPOCH)); when(fileOps.readAttributes(testFile)).thenReturn(oldAttrs); final FileIdentity knownIdentity = FileIdentity.from(testFile, fileOps, FINGERPRINT_BYTES); final BasicFileAttributes newAttrs = mock(BasicFileAttributes.class); when(newAttrs.fileKey()).thenReturn("inode-2"); + when(newAttrs.creationTime()).thenReturn(FileTime.from(Instant.EPOCH)); when(fileOps.readAttributes(testFile)).thenReturn(newAttrs); final RotationResult result = rotationDetector.checkRotation(testFile, knownIdentity, 100L); @@ -102,6 +107,7 @@ void copytruncateRotationWhenFileSizeShrinks() throws IOException { final BasicFileAttributes attrs = mock(BasicFileAttributes.class); when(attrs.fileKey()).thenReturn("inode-5"); + when(attrs.creationTime()).thenReturn(FileTime.from(Instant.EPOCH)); when(fileOps.readAttributes(testFile)).thenReturn(attrs); final FileIdentity knownIdentity = FileIdentity.from(testFile, fileOps, FINGERPRINT_BYTES); @@ -133,6 +139,7 @@ void deletedFileWhenSizeCheckThrowsNoSuchFileException() throws IOException { final BasicFileAttributes attrs = mock(BasicFileAttributes.class); when(attrs.fileKey()).thenReturn("inode-10"); + when(attrs.creationTime()).thenReturn(FileTime.from(Instant.EPOCH)); when(fileOps.readAttributes(testFile)).thenReturn(attrs); final FileIdentity knownIdentity = FileIdentity.from(testFile, fileOps, FINGERPRINT_BYTES); @@ -151,6 +158,7 @@ void noRotationWhenSizeCheckThrowsGenericIOException() throws IOException { final BasicFileAttributes attrs = mock(BasicFileAttributes.class); when(attrs.fileKey()).thenReturn("inode-20"); + when(attrs.creationTime()).thenReturn(FileTime.from(Instant.EPOCH)); when(fileOps.readAttributes(testFile)).thenReturn(attrs); final FileIdentity knownIdentity = FileIdentity.from(testFile, fileOps, FINGERPRINT_BYTES); @@ -194,6 +202,7 @@ void noRotationWhenSizeEqualsCurrentOffset() throws IOException { final BasicFileAttributes attrs = mock(BasicFileAttributes.class); when(attrs.fileKey()).thenReturn("inode-30"); + when(attrs.creationTime()).thenReturn(FileTime.from(Instant.EPOCH)); when(fileOps.readAttributes(testFile)).thenReturn(attrs); final FileIdentity knownIdentity = FileIdentity.from(testFile, fileOps, FINGERPRINT_BYTES); diff --git a/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/TailFileReaderPoolTest.java b/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/TailFileReaderPoolTest.java index 6a2469e805..8e0bdc38e4 100644 --- a/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/TailFileReaderPoolTest.java +++ b/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/TailFileReaderPoolTest.java @@ -31,7 +31,9 @@ import java.nio.file.Paths; import java.nio.file.StandardOpenOption; import java.nio.file.attribute.BasicFileAttributes; +import java.nio.file.attribute.FileTime; import java.time.Duration; +import java.time.Instant; import java.util.Map; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutorService; @@ -278,6 +280,7 @@ void onReaderComplete_with_create_rename_resubmits_reader() throws Exception { BasicFileAttributes attrs = mock(BasicFileAttributes.class); when(attrs.fileKey()).thenReturn("inode-1"); + when(attrs.creationTime()).thenReturn(FileTime.from(Instant.EPOCH)); when(fileOps.readAttributes(testFile)).thenReturn(attrs); when(fileOps.size(testFile)).thenReturn(6L); FileChannel realChannel = FileChannel.open(testFile, StandardOpenOption.READ); @@ -329,6 +332,7 @@ void closeInactiveReaders_with_real_reader() throws Exception { BasicFileAttributes attrs = mock(BasicFileAttributes.class); when(attrs.fileKey()).thenReturn("inode-inactive"); + when(attrs.creationTime()).thenReturn(FileTime.from(Instant.EPOCH)); when(fileOps.readAttributes(testFile)).thenReturn(attrs); when(fileOps.size(testFile)).thenReturn((long) "data\n".length()); FileChannel realChannel = FileChannel.open(testFile, StandardOpenOption.READ); @@ -414,6 +418,7 @@ void closeInactiveReaders_removes_reader_past_threshold() throws Exception { BasicFileAttributes attrs = mock(BasicFileAttributes.class); when(attrs.fileKey()).thenReturn("inode-close-inactive"); + when(attrs.creationTime()).thenReturn(FileTime.from(Instant.EPOCH)); when(fileOps.readAttributes(testFile)).thenReturn(attrs); lenient().when(fileOps.size(testFile)).thenReturn((long) "data\n".length()); @@ -466,6 +471,7 @@ void closeInactiveReaders_keeps_active_readers() throws Exception { BasicFileAttributes attrs = mock(BasicFileAttributes.class); when(attrs.fileKey()).thenReturn("inode-keep-active"); + when(attrs.creationTime()).thenReturn(FileTime.from(Instant.EPOCH)); when(fileOps.readAttributes(testFile)).thenReturn(attrs); lenient().when(fileOps.size(testFile)).thenReturn((long) "data\n".length()); @@ -523,6 +529,7 @@ void onReaderComplete_with_deleted_rotation_marks_completed_and_processes_pendin BasicFileAttributes attrs = mock(BasicFileAttributes.class); when(attrs.fileKey()).thenReturn("inode-del"); + when(attrs.creationTime()).thenReturn(FileTime.from(Instant.EPOCH)); when(fileOps.readAttributes(testFile)).thenReturn(attrs); when(fileOps.size(testFile)).thenReturn(6L); FileChannel realChannel = FileChannel.open(testFile, StandardOpenOption.READ); @@ -547,6 +554,7 @@ void onReaderComplete_with_deleted_rotation_marks_completed_and_processes_pendin Files.writeString(pendingFile, "pending\n"); BasicFileAttributes pendingAttrs = mock(BasicFileAttributes.class); when(pendingAttrs.fileKey()).thenReturn("inode-pending"); + when(pendingAttrs.creationTime()).thenReturn(FileTime.from(Instant.EPOCH)); lenient().when(fileOps.readAttributes(pendingFile)).thenReturn(pendingAttrs); lenient().when(fileOps.size(pendingFile)).thenReturn(8L); lenient().when(fileOps.openReadChannel(pendingFile)).thenReturn( diff --git a/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/TailFileReaderTest.java b/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/TailFileReaderTest.java index 30d9fbf8e0..d4f95563f4 100644 --- a/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/TailFileReaderTest.java +++ b/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/TailFileReaderTest.java @@ -429,8 +429,12 @@ void run_includes_file_metadata_when_enabled() throws Exception { reader.run(); Map capturedData = dataCaptor.getValue(); - assertThat(capturedData.containsKey("file_path"), equalTo(true)); - assertThat(capturedData.containsKey("file_identity"), equalTo(true)); + assertThat(capturedData.containsKey("file"), equalTo(true)); + @SuppressWarnings("unchecked") + final Map fileMetadata = (Map) capturedData.get("file"); + assertThat(fileMetadata.containsKey("path"), equalTo(true)); + assertThat(fileMetadata.containsKey("name"), equalTo(true)); + assertThat(capturedData.containsKey("offset"), equalTo(true)); } @Test From c1533f8929a5d68a96c2b81af60f5c2886802fcd Mon Sep 17 00:00:00 2001 From: Srikanth Padakanti Date: Thu, 7 May 2026 19:48:44 -0500 Subject: [PATCH 03/21] Fix config defaults to match RFC and add truncation events metric Signed-off-by: Srikanth Padakanti --- .../plugins/source/file/FileSourceConfig.java | 8 ++++---- .../dataprepper/plugins/source/file/FileTailMetrics.java | 6 ++++++ .../dataprepper/plugins/source/file/TailFileReader.java | 1 + .../plugins/source/file/FileSourceConfigTest.java | 8 ++++---- .../plugins/source/file/FileTailMetricsTest.java | 9 +++++++++ .../plugins/source/file/TailFileReaderTest.java | 2 ++ 6 files changed, 26 insertions(+), 8 deletions(-) diff --git a/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileSourceConfig.java b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileSourceConfig.java index 60cea38326..e76152b428 100644 --- a/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileSourceConfig.java +++ b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileSourceConfig.java @@ -64,10 +64,10 @@ public class FileSourceConfig { private int readBufferSize = 65536; @JsonProperty("max_active_files") - private int maxActiveFiles = 100; + private int maxActiveFiles = 1000; @JsonProperty("reader_threads") - private int readerThreads = 2; + private int readerThreads = 4; @JsonProperty("max_read_time_per_file") private Duration maxReadTimePerFile = Duration.ofSeconds(5); @@ -82,7 +82,7 @@ public class FileSourceConfig { private String checkpointFile; @JsonProperty("checkpoint_interval") - private Duration checkpointInterval = Duration.ofSeconds(15); + private Duration checkpointInterval = Duration.ofSeconds(5); @JsonProperty("checkpoint_cleanup_after") private Duration checkpointCleanupAfter = Duration.ofDays(7); @@ -112,7 +112,7 @@ public class FileSourceConfig { private boolean acknowledgments = false; @JsonProperty("include_file_metadata") - private boolean includeFileMetadata = false; + private boolean includeFileMetadata = true; @JsonProperty("max_line_length") private int maxLineLength = 1048576; diff --git a/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileTailMetrics.java b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileTailMetrics.java index 44395566d0..ad28d83b0d 100644 --- a/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileTailMetrics.java +++ b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileTailMetrics.java @@ -31,6 +31,7 @@ public final class FileTailMetrics { private final Counter eventsEmitted; private final Counter dataLossEvents; private final Counter acknowledgmentFailures; + private final Counter truncationEvents; private final Timer backpressureTimer; private final AtomicLong activeFileCount; private final AtomicLong fileLagBytes; @@ -49,6 +50,7 @@ public FileTailMetrics(final PluginMetrics pluginMetrics) { this.eventsEmitted = pluginMetrics.counter("tailEventsEmitted"); this.dataLossEvents = pluginMetrics.counter("tailDataLossEvents"); this.acknowledgmentFailures = pluginMetrics.counter("tailAcknowledgmentFailures"); + this.truncationEvents = pluginMetrics.counter("tailTruncationEvents"); this.backpressureTimer = pluginMetrics.timer("tailBackpressureTime"); this.activeFileCount = new AtomicLong(0); pluginMetrics.gauge("tailActiveFiles", activeFileCount); @@ -119,4 +121,8 @@ public Counter getDataLossEvents() { public Counter getAcknowledgmentFailures() { return acknowledgmentFailures; } + + public Counter getTruncationEvents() { + return truncationEvents; + } } diff --git a/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/TailFileReader.java b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/TailFileReader.java index 64be1715ae..bb761c907c 100644 --- a/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/TailFileReader.java +++ b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/TailFileReader.java @@ -153,6 +153,7 @@ public void run() { case COPYTRUNCATE: LOG.info("Copytruncate detected for {}. Resetting offset to 0.", path); metrics.getFilesRotated().increment(); + metrics.getTruncationEvents().increment(); completePendingAckSet(); readOffset.set(0); checkpointEntry.setReadOffset(0); diff --git a/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/FileSourceConfigTest.java b/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/FileSourceConfigTest.java index ce2cf94b18..d4e0ac6ba8 100644 --- a/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/FileSourceConfigTest.java +++ b/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/FileSourceConfigTest.java @@ -161,13 +161,13 @@ void default_config_returns_expected_values() { assertThat(config.getPollInterval(), equalTo(Duration.ofSeconds(1))); assertThat(config.getEncoding(), equalTo("UTF-8")); assertThat(config.getReadBufferSize(), equalTo(65536)); - assertThat(config.getMaxActiveFiles(), equalTo(100)); - assertThat(config.getReaderThreads(), equalTo(2)); + assertThat(config.getMaxActiveFiles(), equalTo(1000)); + assertThat(config.getReaderThreads(), equalTo(4)); assertThat(config.getMaxReadTimePerFile(), equalTo(Duration.ofSeconds(5))); assertThat(config.getRotateWait(), equalTo(Duration.ofSeconds(5))); assertThat(config.getRotationDrainTimeout(), equalTo(Duration.ofSeconds(30))); assertThat(config.getCheckpointFile(), nullValue()); - assertThat(config.getCheckpointInterval(), equalTo(Duration.ofSeconds(15))); + assertThat(config.getCheckpointInterval(), equalTo(Duration.ofSeconds(5))); assertThat(config.getCheckpointCleanupAfter(), equalTo(Duration.ofDays(7))); assertThat(config.getFingerprintBytes(), equalTo(1024)); assertThat(config.getCloseInactive(), equalTo(Duration.ofMinutes(30))); @@ -176,7 +176,7 @@ void default_config_returns_expected_values() { assertThat(config.getBatchTimeout(), equalTo(Duration.ofSeconds(5))); assertThat(config.getAcknowledgmentTimeout(), equalTo(Duration.ofSeconds(60))); assertThat(config.getMaxAcknowledgmentRetries(), equalTo(3)); - assertThat(config.isIncludeFileMetadata(), equalTo(false)); + assertThat(config.isIncludeFileMetadata(), equalTo(true)); assertThat(config.getMaxLineLength(), equalTo(1048576)); } diff --git a/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/FileTailMetricsTest.java b/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/FileTailMetricsTest.java index 52d3f2b311..38c79e1c98 100644 --- a/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/FileTailMetricsTest.java +++ b/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/FileTailMetricsTest.java @@ -72,6 +72,9 @@ class FileTailMetricsTest { @Mock private Counter acknowledgmentFailuresCounter; + @Mock + private Counter truncationEventsCounter; + @Mock private Timer backpressureTimer; @@ -92,6 +95,7 @@ void setUp() { when(pluginMetrics.counter("tailEventsEmitted")).thenReturn(eventsEmittedCounter); when(pluginMetrics.counter("tailDataLossEvents")).thenReturn(dataLossEventsCounter); when(pluginMetrics.counter("tailAcknowledgmentFailures")).thenReturn(acknowledgmentFailuresCounter); + when(pluginMetrics.counter("tailTruncationEvents")).thenReturn(truncationEventsCounter); when(pluginMetrics.timer("tailBackpressureTime")).thenReturn(backpressureTimer); fileTailMetrics = new FileTailMetrics(pluginMetrics); @@ -218,4 +222,9 @@ void getDataLossEventsReturnsRegisteredCounter() { void getAcknowledgmentFailuresReturnsRegisteredCounter() { assertThat(fileTailMetrics.getAcknowledgmentFailures(), equalTo(acknowledgmentFailuresCounter)); } + + @Test + void getTruncationEventsReturnsRegisteredCounter() { + assertThat(fileTailMetrics.getTruncationEvents(), equalTo(truncationEventsCounter)); + } } diff --git a/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/TailFileReaderTest.java b/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/TailFileReaderTest.java index d4f95563f4..4908146c90 100644 --- a/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/TailFileReaderTest.java +++ b/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/TailFileReaderTest.java @@ -330,6 +330,8 @@ void run_handles_copytruncate_rotation() throws Exception { when(fileOps.openReadChannel(testFile)).thenReturn(realChannel); Counter filesRotated = mock(Counter.class); when(metrics.getFilesRotated()).thenReturn(filesRotated); + Counter truncationEvents = mock(Counter.class); + when(metrics.getTruncationEvents()).thenReturn(truncationEvents); stubReadMetrics(); stubEventFactory(); From f35f6875cc9ac6de61bfabbb2750eaf3c4760d02 Mon Sep 17 00:00:00 2001 From: Srikanth Padakanti Date: Thu, 7 May 2026 20:09:46 -0500 Subject: [PATCH 04/21] Replace Thread.sleep with Mockito verify-after and CountDownLatch in FileSourceTests Signed-off-by: Srikanth Padakanti --- .../plugins/source/file/FileSourceTests.java | 35 ++++++++----------- 1 file changed, 15 insertions(+), 20 deletions(-) diff --git a/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/FileSourceTests.java b/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/FileSourceTests.java index 3333f5813b..4bb7b32344 100644 --- a/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/FileSourceTests.java +++ b/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/FileSourceTests.java @@ -45,6 +45,7 @@ import java.util.List; import java.util.Map; import java.util.UUID; +import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; import java.util.function.Consumer; @@ -57,7 +58,9 @@ import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyInt; import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.after; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.mock; @@ -180,23 +183,21 @@ private BlockingBuffer> getBuffer() throws JsonProcessingExceptio } @Test - public void testFileSourceWithEmptyFilePathDoesNotWriteToBuffer() throws InterruptedException { + public void testFileSourceWithEmptyFilePathDoesNotWriteToBuffer() throws TimeoutException { buffer = mock(Buffer.class); pluginSettings.put(FileSourceConfig.ATTRIBUTE_PATH, ""); fileSource = createObjectUnderTest(); fileSource.start(buffer); - Thread.sleep(500); - verifyNoInteractions(buffer); + verify(buffer, after(500).never()).write(any(Record.class), anyInt()); } @Test - public void testFileSourceWithNonexistentFilePathDoesNotWriteToBuffer() throws InterruptedException { + public void testFileSourceWithNonexistentFilePathDoesNotWriteToBuffer() throws TimeoutException { buffer = mock(Buffer.class); pluginSettings.put(FileSourceConfig.ATTRIBUTE_PATH, FILE_DOES_NOT_EXIST); fileSource = createObjectUnderTest(); fileSource.start(buffer); - Thread.sleep(500); - verifyNoInteractions(buffer); + verify(buffer, after(500).never()).write(any(Record.class), anyInt()); } @Test @@ -354,7 +355,7 @@ void start_will_parse_codec_with_a_Consumer_that_writes_to_the_buffer() throws I } @Test - void start_will_throw_exception_if_codec_throws() throws IOException, TimeoutException, InterruptedException { + void start_will_throw_exception_if_codec_throws() throws IOException, TimeoutException { final IOException mockedException = mock(IOException.class); doThrow(mockedException) @@ -364,9 +365,7 @@ void start_will_throw_exception_if_codec_throws() throws IOException, TimeoutExc objectUnderTest.start(buffer); - Thread.sleep(2_000); - - verifyNoInteractions(buffer); + verify(buffer, after(1500).never()).write(any(Record.class), anyInt()); } @Test @@ -405,7 +404,6 @@ void start_in_tail_mode_invokes_startTailing() throws Exception { FileSource fileSource = createObjectUnderTest(); fileSource.start(buffer); - Thread.sleep(200); fileSource.stop(); } @@ -425,7 +423,6 @@ void start_in_tail_mode_with_codec_invokes_startTailing() throws Exception { FileSource fileSource = createObjectUnderTest(); fileSource.start(buffer); - Thread.sleep(200); fileSource.stop(); } @@ -439,7 +436,6 @@ void start_in_tail_mode_with_high_ratio_logs_warning() throws Exception { FileSource fileSource = createObjectUnderTest(); fileSource.start(buffer); - Thread.sleep(200); fileSource.stop(); } @@ -452,7 +448,6 @@ void start_in_tail_mode_with_checkpoint_path() throws Exception { FileSource fileSource = createObjectUnderTest(); fileSource.start(buffer); - Thread.sleep(200); fileSource.stop(); } @@ -466,21 +461,22 @@ void stop_before_start_does_not_throw() { void stop_after_classic_start_joins_thread() throws Exception { FileSource fileSource = createObjectUnderTest(); fileSource.start(buffer); - Thread.sleep(200); fileSource.stop(); } @Test void stop_with_interrupt_during_join() throws Exception { buffer = mock(Buffer.class); + final CountDownLatch writeStarted = new CountDownLatch(1); doAnswer(inv -> { + writeStarted.countDown(); Thread.sleep(5000); return null; }).when(buffer).write(any(Record.class), eq(FileSourceConfig.DEFAULT_TIMEOUT)); FileSource fileSource = createObjectUnderTest(); fileSource.start(buffer); - Thread.sleep(200); + writeStarted.await(2, TimeUnit.SECONDS); Thread stopThread = new Thread(() -> { Thread.currentThread().interrupt(); @@ -516,7 +512,6 @@ void start_in_tail_mode_with_safe_ratio_does_not_warn() throws Exception { FileSource fileSource = createObjectUnderTest(); fileSource.start(buffer); - Thread.sleep(200); fileSource.stop(); } @@ -526,16 +521,17 @@ void stop_mid_read_stops_processing_lines() throws Exception { pluginSettings.put(FileSourceConfig.ATTRIBUTE_PATH, TEST_FILE_PATH_PLAIN); buffer = mock(Buffer.class); + final CountDownLatch writeStarted = new CountDownLatch(1); doAnswer(inv -> { + writeStarted.countDown(); Thread.sleep(2000); return null; }).when(buffer).write(any(Record.class), eq(FileSourceConfig.DEFAULT_TIMEOUT)); FileSource fileSource = createObjectUnderTest(); fileSource.start(buffer); - Thread.sleep(100); + writeStarted.await(2, TimeUnit.SECONDS); fileSource.stop(); - Thread.sleep(500); } @Test @@ -544,7 +540,6 @@ void writeLineAsEventOrString_with_non_matching_type_does_not_write() throws Exc pluginSettings.put(FileSourceConfig.ATTRIBUTE_PATH, TEST_FILE_PATH_PLAIN); FileSource fileSource = createObjectUnderTest(); fileSource.start(buffer); - Thread.sleep(500); fileSource.stop(); } } From fad4122dbc20e3a283a8ae66a64f60e3d648c832 Mon Sep 17 00:00:00 2001 From: Srikanth Padakanti Date: Thu, 7 May 2026 20:23:18 -0500 Subject: [PATCH 05/21] Remove unnecessary forkEvery=1 from test configuration Signed-off-by: Srikanth Padakanti --- data-prepper-plugins/file-source/build.gradle | 1 - 1 file changed, 1 deletion(-) diff --git a/data-prepper-plugins/file-source/build.gradle b/data-prepper-plugins/file-source/build.gradle index 82f3e7b666..23d1d154ad 100644 --- a/data-prepper-plugins/file-source/build.gradle +++ b/data-prepper-plugins/file-source/build.gradle @@ -20,7 +20,6 @@ dependencies { } test { - forkEvery = 1 maxHeapSize = '512m' } From c541c5c4f3029f00f5ebb5db70bff27626b3ec32 Mon Sep 17 00:00:00 2001 From: Srikanth Padakanti Date: Thu, 7 May 2026 20:29:07 -0500 Subject: [PATCH 06/21] Replace Thread.sleep with Awaitility in CheckpointRegistryTest and TailFileReaderPoolTest Signed-off-by: Srikanth Padakanti --- .../plugins/source/file/CheckpointRegistryTest.java | 10 ++++++---- .../plugins/source/file/TailFileReaderPoolTest.java | 11 ++++++----- 2 files changed, 12 insertions(+), 9 deletions(-) diff --git a/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/CheckpointRegistryTest.java b/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/CheckpointRegistryTest.java index 6663847049..daaeb1591f 100644 --- a/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/CheckpointRegistryTest.java +++ b/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/CheckpointRegistryTest.java @@ -22,6 +22,7 @@ import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; +import static org.awaitility.Awaitility.await; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.notNullValue; @@ -166,16 +167,17 @@ void emptyCheckpointFileStartsWithEmptyState() throws IOException { } @Test - void cleanupRemovesStaleCompletedEntries() throws InterruptedException { + void cleanupRemovesStaleCompletedEntries() { final Duration zeroCleanup = Duration.ZERO; registry = new CheckpointRegistry(checkpointFile, FLUSH_INTERVAL, zeroCleanup); final CheckpointEntry entry = registry.getOrCreate("stale-file"); entry.setStatus(CheckpointStatus.COMPLETED); - Thread.sleep(50); - - registry.flush(); + await().atMost(Duration.ofSeconds(2)).untilAsserted(() -> { + registry.flush(); + assertThat(registry.get("stale-file"), nullValue()); + }); assertThat(registry.get("stale-file"), nullValue()); } diff --git a/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/TailFileReaderPoolTest.java b/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/TailFileReaderPoolTest.java index 8e0bdc38e4..06b044b333 100644 --- a/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/TailFileReaderPoolTest.java +++ b/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/TailFileReaderPoolTest.java @@ -40,6 +40,7 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; +import static org.awaitility.Awaitility.await; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.equalTo; import static org.mockito.ArgumentMatchers.any; @@ -305,7 +306,7 @@ void onReaderComplete_with_create_rename_resubmits_reader() throws Exception { pool.addFile(identity, testFile); - Thread.sleep(2000); + await().atMost(5, TimeUnit.SECONDS).until(() -> pool.getActiveReaderCount() == 0); pool.shutdown(); } @@ -357,7 +358,7 @@ void closeInactiveReaders_with_real_reader() throws Exception { FileIdentity identity = FileIdentity.from(testFile, fileOps, 1024); pool.addFile(identity, testFile); - Thread.sleep(1000); + await().atMost(5, TimeUnit.SECONDS).until(() -> pool.getActiveReaderCount() == 0); pool.closeInactiveReaders(); @@ -443,7 +444,7 @@ void closeInactiveReaders_removes_reader_past_threshold() throws Exception { FileIdentity identity = FileIdentity.from(testFile, fileOps, 1024); pool.addFile(identity, testFile); - Thread.sleep(100); + await().atMost(2, TimeUnit.SECONDS).until(() -> pool.getActiveReaderCount() > 0); pool.closeInactiveReaders(); @@ -496,7 +497,7 @@ void closeInactiveReaders_keeps_active_readers() throws Exception { FileIdentity identity = FileIdentity.from(testFile, fileOps, 1024); pool.addFile(identity, testFile); - Thread.sleep(100); + await().atMost(2, TimeUnit.SECONDS).until(() -> pool.getActiveReaderCount() > 0); assertThat(pool.getActiveReaderCount(), equalTo(1)); @@ -570,7 +571,7 @@ void onReaderComplete_with_deleted_rotation_marks_completed_and_processes_pendin assertThat(limitedPool.getPendingCount(), equalTo(1)); - Thread.sleep(2000); + await().atMost(5, TimeUnit.SECONDS).until(() -> limitedPool.getActiveReaderCount() == 0); limitedPool.shutdown(); } From c9987ec8e0028ad2f77b0d4d677cd2ac2ccc4f3f Mon Sep 17 00:00:00 2001 From: Srikanth Padakanti Date: Thu, 7 May 2026 20:35:13 -0500 Subject: [PATCH 07/21] Replace Thread.sleep with Awaitility in TailFileReaderTest interrupt tests Signed-off-by: Srikanth Padakanti --- .../plugins/source/file/TailFileReaderTest.java | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/TailFileReaderTest.java b/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/TailFileReaderTest.java index 4908146c90..7e7a06e433 100644 --- a/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/TailFileReaderTest.java +++ b/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/TailFileReaderTest.java @@ -42,8 +42,11 @@ import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; + import java.util.function.Consumer; +import static org.awaitility.Awaitility.await; + import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.equalTo; import static org.mockito.ArgumentMatchers.any; @@ -1073,7 +1076,7 @@ void run_codec_backpressure_interrupt_stops_reader() throws Exception { reader.run(); }); readerThread.start(); - Thread.sleep(300); + await().atMost(2, TimeUnit.SECONDS).until(() -> readerThread.getState() == Thread.State.TIMED_WAITING || readerThread.getState() == Thread.State.WAITING || !readerThread.isAlive()); readerThread.interrupt(); readerThread.join(5000); @@ -1114,7 +1117,7 @@ void run_emitLine_backpressure_interrupt_stops_reader() throws Exception { reader.run(); }); readerThread.start(); - Thread.sleep(200); + await().atMost(2, TimeUnit.SECONDS).until(() -> readerThread.getState() == Thread.State.TIMED_WAITING || readerThread.getState() == Thread.State.WAITING || !readerThread.isAlive()); readerThread.interrupt(); readerThread.join(5000); @@ -1324,7 +1327,7 @@ void run_emitLine_thread_interrupted_during_backpressure_exits() throws Exceptio reader.run(); }); readerThread.start(); - Thread.sleep(300); + await().atMost(2, TimeUnit.SECONDS).until(() -> readerThread.getState() == Thread.State.TIMED_WAITING || readerThread.getState() == Thread.State.WAITING || !readerThread.isAlive()); readerThread.interrupt(); readerThread.join(5000); @@ -1368,7 +1371,7 @@ void run_codec_thread_interrupted_during_backpressure_exits() throws Exception { reader.run(); }); readerThread.start(); - Thread.sleep(300); + await().atMost(2, TimeUnit.SECONDS).until(() -> readerThread.getState() == Thread.State.TIMED_WAITING || readerThread.getState() == Thread.State.WAITING || !readerThread.isAlive()); readerThread.interrupt(); readerThread.join(5000); From 07a2b467ae1337cef85d25d65d49fdf763425ce2 Mon Sep 17 00:00:00 2001 From: Srikanth Padakanti Date: Thu, 7 May 2026 21:00:57 -0500 Subject: [PATCH 08/21] Replace Thread.sleep with Awaitility in DirectoryWatcherTest Signed-off-by: Srikanth Padakanti --- .../source/file/DirectoryWatcherTest.java | 78 ++++++------------- 1 file changed, 25 insertions(+), 53 deletions(-) diff --git a/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/DirectoryWatcherTest.java b/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/DirectoryWatcherTest.java index e26d413873..371a536339 100644 --- a/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/DirectoryWatcherTest.java +++ b/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/DirectoryWatcherTest.java @@ -38,6 +38,7 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.locks.LockSupport; +import static org.awaitility.Awaitility.await; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.equalTo; import static org.mockito.ArgumentMatchers.any; @@ -169,8 +170,8 @@ void pollScan_detects_vanished_files_and_closes_when_close_removed_true() throws Files.delete(tempDir.resolve("vanish.log")); watcher.pollScan(); - Thread.sleep(500); - verify(readerPool, atLeastOnce()).closeReaderForPath(any(Path.class)); + await().atMost(2, TimeUnit.SECONDS).untilAsserted(() -> + verify(readerPool, atLeastOnce()).closeReaderForPath(any(Path.class))); } finally { watcher.stop(); } @@ -274,9 +275,6 @@ void watch_loop_exits_on_closed_watch_service() throws Exception { final DirectoryWatcher watcher = createWatcher(); watcher.start(); - - Thread.sleep(500); - watcher.stop(); } @@ -289,12 +287,7 @@ void watch_loop_detects_new_file_created() throws Exception { final DirectoryWatcher watcher = createWatcher(); try { watcher.start(); - - Thread.sleep(500); - Files.writeString(tempDir.resolve("new-detected.log"), "new content"); - - Thread.sleep(2000); } finally { watcher.stop(); } @@ -313,12 +306,7 @@ void watch_loop_handles_file_deletion_with_close_removed_true() throws Exception Duration.ofMillis(100), true); try { watcher.start(); - - Thread.sleep(500); - Files.delete(tempDir.resolve("delete-me.log")); - - Thread.sleep(2000); } finally { watcher.stop(); } @@ -337,12 +325,7 @@ void watch_loop_handles_file_deletion_with_close_removed_false() throws Exceptio Duration.ofMillis(100), false); try { watcher.start(); - - Thread.sleep(500); - Files.delete(tempDir.resolve("keep-me.log")); - - Thread.sleep(2000); } finally { watcher.stop(); } @@ -361,14 +344,8 @@ void watch_loop_handles_file_reappearing_during_rotate_wait() throws Exception { Duration.ofMillis(500), true); try { watcher.start(); - - Thread.sleep(500); - Files.delete(tempDir.resolve("rotate-reappear.log")); - Thread.sleep(100); Files.writeString(tempDir.resolve("rotate-reappear.log"), "new content"); - - Thread.sleep(2000); } finally { watcher.stop(); } @@ -388,7 +365,6 @@ void startWatchService_falls_back_on_exception() throws Exception { () -> { throw new RuntimeException("cannot create WatchService"); }); try { watcher.start(); - Thread.sleep(200); } finally { watcher.stop(); } @@ -408,7 +384,6 @@ void startWatchService_falls_back_on_io_exception() throws Exception { () -> { throw new IOException("cannot create WatchService"); }); try { watcher.start(); - Thread.sleep(200); } finally { watcher.stop(); } @@ -459,10 +434,8 @@ void stop_handles_IOException_closing_WatchService() throws Exception { Duration.ofSeconds(5), true, () -> realWatchService); watcher.start(); - Thread.sleep(200); realWatchService.close(); - Thread.sleep(200); watcher.stop(); } @@ -484,7 +457,6 @@ void registerDirectory_handles_IOException() throws Exception { Duration.ofSeconds(5), true); try { watcher.start(); - Thread.sleep(200); } finally { watchDir.toFile().setReadable(true); watchDir.toFile().setExecutable(true); @@ -507,7 +479,6 @@ void supplementary_poll_interval_when_not_macOS_and_watch_active() throws Except false); try { watcher.start(); - Thread.sleep(200); } finally { watcher.stop(); } @@ -535,7 +506,8 @@ void watchLoop_handles_invalid_WatchKey() throws Exception { Duration.ofSeconds(5), true, () -> mockWatchService); watcher.start(); - Thread.sleep(500); + await().atMost(2, TimeUnit.SECONDS).untilAsserted(() -> + verify(mockWatchService, atLeastOnce()).take()); watcher.stop(); } @@ -564,7 +536,8 @@ void handleWatchEvent_handles_OVERFLOW() throws Exception { Duration.ofSeconds(5), true, () -> mockWatchService); watcher.start(); - Thread.sleep(500); + await().atMost(2, TimeUnit.SECONDS).untilAsserted(() -> + verify(mockWatchService, atLeastOnce()).take()); watcher.stop(); } @@ -595,7 +568,8 @@ void triggerFullRescan_handles_runtime_exception() throws Exception { Duration.ofSeconds(5), true, () -> mockWatchService); watcher.start(); - Thread.sleep(500); + await().atMost(2, TimeUnit.SECONDS).untilAsserted(() -> + verify(mockWatchService, atLeastOnce()).take()); watcher.stop(); } @@ -637,8 +611,8 @@ void handleDeletion_file_reappears_during_rotateWait() throws Exception { Duration.ofMillis(500), true, () -> mockWatchService); watcher.start(); - - Thread.sleep(2000); + await().atMost(2, TimeUnit.SECONDS).untilAsserted(() -> + verify(mockWatchService, atLeastOnce()).take()); watcher.stop(); } @@ -676,7 +650,8 @@ void stop_handles_IOException_when_watchService_close_fails() throws Exception { Duration.ofSeconds(5), true, () -> mockWatchService); watcher.start(); - Thread.sleep(200); + await().atMost(2, TimeUnit.SECONDS).untilAsserted(() -> + verify(mockWatchService, atLeastOnce()).take()); watcher.stop(); } @@ -803,7 +778,6 @@ void watch_loop_exits_when_running_becomes_false() throws Exception { () -> mockScheduler); watcher.start(); firstIterDone.await(); - Thread.sleep(50); watcher.stop(); } @@ -842,10 +816,9 @@ void handleWatchEvent_ignores_file_not_matching_glob() throws Exception { false, () -> mockScheduler); watcher.start(); - Thread.sleep(500); + await().pollDelay(500, TimeUnit.MILLISECONDS).atMost(2, TimeUnit.SECONDS).untilAsserted(() -> + verify(readerPool, never()).addFile(any(), any())); watcher.stop(); - - verify(readerPool, never()).addFile(any(), any()); } @Test @@ -882,11 +855,11 @@ void handleWatchEvent_ignores_ENTRY_MODIFY_event() throws Exception { false, () -> mockScheduler); watcher.start(); - Thread.sleep(500); + await().pollDelay(500, TimeUnit.MILLISECONDS).atMost(2, TimeUnit.SECONDS).untilAsserted(() -> { + verify(readerPool, never()).addFile(any(), any()); + verify(readerPool, never()).closeReaderForPath(any()); + }); watcher.stop(); - - verify(readerPool, never()).addFile(any(), any()); - verify(readerPool, never()).closeReaderForPath(any()); } @Test @@ -922,10 +895,9 @@ void handleWatchEvent_DELETE_for_unknown_file_does_nothing() throws Exception { false, () -> mockScheduler); watcher.start(); - Thread.sleep(500); + await().pollDelay(500, TimeUnit.MILLISECONDS).atMost(2, TimeUnit.SECONDS).untilAsserted(() -> + verify(readerPool, never()).closeReaderForPath(any())); watcher.stop(); - - verify(readerPool, never()).closeReaderForPath(any()); } @Test @@ -944,7 +916,6 @@ void shouldUseWatchService_returns_false_when_network_fs_detected() throws Excep DirectoryWatcher::createDefaultPollScheduler, path -> true); watcher.start(); - Thread.sleep(200); watcher.stop(); } @@ -992,7 +963,8 @@ void handleWatchEvent_DELETE_rejectedExecutionException_on_schedule() throws Exc false, () -> mockScheduler); watcher.start(); - Thread.sleep(500); + await().atMost(2, TimeUnit.SECONDS).untilAsserted(() -> + verify(mockWatchService, atLeastOnce()).take()); watcher.stop(); } @@ -1028,7 +1000,8 @@ void triggerFullRescan_handles_rejectedExecutionException() throws Exception { false, () -> mockScheduler); watcher.start(); - Thread.sleep(500); + await().atMost(2, TimeUnit.SECONDS).untilAsserted(() -> + verify(mockWatchService, atLeastOnce()).take()); watcher.stop(); } @@ -1081,7 +1054,6 @@ void startWatchService_falls_back_to_polling_when_registration_fails() throws Ex () -> Executors.newSingleThreadScheduledExecutor(), path -> false); watcher.start(); - Thread.sleep(200); watcher.stop(); } } From 7e5bb56c0d65c36a93e73f654599de85d9648b01 Mon Sep 17 00:00:00 2001 From: Srikanth Padakanti Date: Thu, 7 May 2026 21:17:28 -0500 Subject: [PATCH 09/21] Add missing tests for full JaCoCo coverage and remove dead code Signed-off-by: Srikanth Padakanti --- .../plugins/source/file/TailFileReader.java | 51 ++++++++--------- .../source/file/CheckpointStatusTest.java | 5 ++ .../plugins/source/file/FileSourceTests.java | 12 +++- .../plugins/source/file/RecordTypeTest.java | 55 +++++++++++++++++++ .../source/file/TailFileReaderPoolTest.java | 45 +++++++++++++++ 5 files changed, 138 insertions(+), 30 deletions(-) create mode 100644 data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/RecordTypeTest.java diff --git a/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/TailFileReader.java b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/TailFileReader.java index bb761c907c..e06b7e6855 100644 --- a/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/TailFileReader.java +++ b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/TailFileReader.java @@ -149,31 +149,27 @@ public void run() { final RotationResult rotation = rotationDetector.checkRotation(path, fileIdentity, readOffset.get()); lastRotationType = rotation.getRotationType(); - switch (rotation.getRotationType()) { - case COPYTRUNCATE: - LOG.info("Copytruncate detected for {}. Resetting offset to 0.", path); - metrics.getFilesRotated().increment(); - metrics.getTruncationEvents().increment(); - completePendingAckSet(); - readOffset.set(0); - checkpointEntry.setReadOffset(0); - batchStartOffset = 0; - readFile(); - break; - case DELETED: - LOG.info("File deleted: {}. Closing reader.", path); - completePendingAckSet(); - break; - case CREATE_RENAME: - LOG.info("Create/rename rotation detected for {}. Draining current file.", path); - metrics.getFilesRotated().increment(); - drainCurrentFile(); - completePendingAckSet(); - break; - case NO_ROTATION: - default: - readFile(); - break; + final RotationType rotationType = rotation.getRotationType(); + + if (rotationType == RotationType.COPYTRUNCATE) { + LOG.info("Copytruncate detected for {}. Resetting offset to 0.", path); + metrics.getFilesRotated().increment(); + metrics.getTruncationEvents().increment(); + completePendingAckSet(); + readOffset.set(0); + checkpointEntry.setReadOffset(0); + batchStartOffset = 0; + readFile(); + } else if (rotationType == RotationType.DELETED) { + LOG.info("File deleted: {}. Closing reader.", path); + completePendingAckSet(); + } else if (rotationType == RotationType.CREATE_RENAME) { + LOG.info("Create/rename rotation detected for {}. Draining current file.", path); + metrics.getFilesRotated().increment(); + drainCurrentFile(); + completePendingAckSet(); + } else { + readFile(); } } catch (final RuntimeException e) { LOG.error("Error reading file {}", path, e); @@ -253,9 +249,6 @@ private void readLoop(final FileChannel channel, final long timeoutMillis, final decoderCarryover.clear(); } final int bytesRead = channel.read(byteBuffer); - if (bytesRead == -1) { - break; - } if (bytesRead <= 0 && byteBuffer.position() == 0) { break; } @@ -297,7 +290,7 @@ private void readLoop(final FileChannel channel, final long timeoutMillis, final checkpointEntry.setReadOffset(readOffset.get()); } } else if (codec == null) { - final int carryoverBytes = decoderCarryover != null ? decoderCarryover.position() : 0; + final int carryoverBytes = decoderCarryover.position(); checkpointEntry.setReadOffset(readOffset.get() - carryoverBytes); } } diff --git a/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/CheckpointStatusTest.java b/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/CheckpointStatusTest.java index 64527174f2..b8ec63d91b 100644 --- a/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/CheckpointStatusTest.java +++ b/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/CheckpointStatusTest.java @@ -47,4 +47,9 @@ void getValue_returns_value() { assertThat(CheckpointStatus.ACTIVE.getValue(), equalTo("ACTIVE")); assertThat(CheckpointStatus.COMPLETED.getValue(), equalTo("COMPLETED")); } + + @Test + void fromString_with_null_throws_IllegalArgumentException() { + assertThrows(IllegalArgumentException.class, () -> CheckpointStatus.fromString(null)); + } } diff --git a/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/FileSourceTests.java b/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/FileSourceTests.java index 4bb7b32344..9989812882 100644 --- a/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/FileSourceTests.java +++ b/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/FileSourceTests.java @@ -65,7 +65,6 @@ import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.verifyNoInteractions; import static org.mockito.Mockito.when; @ExtendWith(MockitoExtension.class) @@ -457,6 +456,17 @@ void stop_before_start_does_not_throw() { assertDoesNotThrow(fileSource::stop); } + @Test + void start_in_tail_mode_rethrows_runtime_exception_from_startTailing() { + pluginSettings.put("tail", true); + pluginSettings.put("paths", List.of("/tmp/nonexistent-err-*.log")); + pluginSettings.put(FileSourceConfig.ATTRIBUTE_PATH, "/tmp/nonexistent-err-single.log"); + pluginSettings.put("fingerprint_bytes", 0); + + FileSource fileSource = createObjectUnderTest(); + assertThrows(IllegalArgumentException.class, () -> fileSource.start(buffer)); + } + @Test void stop_after_classic_start_joins_thread() throws Exception { FileSource fileSource = createObjectUnderTest(); diff --git a/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/RecordTypeTest.java b/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/RecordTypeTest.java new file mode 100644 index 0000000000..fd4376f431 --- /dev/null +++ b/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/RecordTypeTest.java @@ -0,0 +1,55 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + +package org.opensearch.dataprepper.plugins.source.file; + +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.equalTo; +import static org.junit.jupiter.api.Assertions.assertThrows; + +class RecordTypeTest { + + @Test + void fromString_returns_string() { + assertThat(RecordType.fromString("string"), equalTo(RecordType.STRING)); + } + + @Test + void fromString_returns_event() { + assertThat(RecordType.fromString("event"), equalTo(RecordType.EVENT)); + } + + @Test + void fromString_is_case_insensitive() { + assertThat(RecordType.fromString("STRING"), equalTo(RecordType.STRING)); + assertThat(RecordType.fromString("EVENT"), equalTo(RecordType.EVENT)); + } + + @ParameterizedTest + @ValueSource(strings = {"invalid", "record", ""}) + void fromString_throws_for_invalid_value(final String value) { + assertThrows(IllegalArgumentException.class, () -> RecordType.fromString(value)); + } + + @Test + void fromString_with_null_throws_IllegalArgumentException() { + assertThrows(IllegalArgumentException.class, () -> RecordType.fromString(null)); + } + + @Test + void toString_returns_name() { + assertThat(RecordType.STRING.toString(), equalTo("string")); + assertThat(RecordType.EVENT.toString(), equalTo("event")); + } +} diff --git a/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/TailFileReaderPoolTest.java b/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/TailFileReaderPoolTest.java index 06b044b333..658ddb6a84 100644 --- a/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/TailFileReaderPoolTest.java +++ b/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/TailFileReaderPoolTest.java @@ -37,6 +37,7 @@ import java.util.Map; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutorService; +import java.util.concurrent.RejectedExecutionException; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; @@ -244,6 +245,30 @@ void closeReaderForPath_does_nothing_when_no_match() { assertThat(pool.getActiveReaderCount(), equalTo(1)); } + @Test + void closeReaderForPath_promotes_pending_files() { + Counter filesClosed = mock(Counter.class); + lenient().when(metrics.getFilesClosed()).thenReturn(filesClosed); + TailFileReaderPool pool = createPool(1, 2); + when(checkpointRegistry.getOrCreate(anyString())).thenReturn(new CheckpointEntry()); + + FileIdentity identity1 = mock(FileIdentity.class); + when(identity1.toString()).thenReturn("id-close-promote-1"); + FileIdentity identity2 = mock(FileIdentity.class); + when(identity2.toString()).thenReturn("id-close-promote-2"); + + pool.addFile(identity1, Paths.get("/tmp/file1.log")); + pool.addFile(identity2, Paths.get("/tmp/file2.log")); + + assertThat(pool.getActiveReaderCount(), equalTo(1)); + assertThat(pool.getPendingCount(), equalTo(1)); + + pool.closeReaderForPath(Paths.get("/tmp/file1.log")); + + assertThat(pool.getActiveReaderCount(), equalTo(1)); + assertThat(pool.getPendingCount(), equalTo(0)); + } + @Test void shutdown_handles_interrupted_exception() throws Exception { TailFileReaderPool pool = createPoolWithoutMetrics(10, 1); @@ -395,6 +420,26 @@ void shutdown_calls_shutdownNow_when_awaitTermination_throws_interrupted() throw Thread.interrupted(); } + @Test + void addFile_handles_rejected_execution_exception() { + ExecutorService mockExecutor = mock(ExecutorService.class); + when(mockExecutor.submit(any(Runnable.class))) + .thenThrow(new RejectedExecutionException("pool shut down")); + when(metrics.getActiveFileCount()).thenReturn(new AtomicLong(0)); + when(checkpointRegistry.getOrCreate(anyString())).thenReturn(new CheckpointEntry()); + + TailFileReaderPool pool = new TailFileReaderPool( + checkpointRegistry, metrics, 10, + Duration.ofMinutes(30), createReaderContext(), + () -> mockExecutor); + + FileIdentity identity = mock(FileIdentity.class); + when(identity.toString()).thenReturn("rejected-id"); + pool.addFile(identity, Paths.get("/tmp/rejected.log")); + + assertThat(pool.getActiveReaderCount(), equalTo(0)); + } + @Test void closeInactiveReaders_removes_reader_past_threshold() throws Exception { Counter filesClosed = mock(Counter.class); From d99e734f52ada973df00ea8cc07a22244f39d3c1 Mon Sep 17 00:00:00 2001 From: Srikanth Padakanti Date: Thu, 7 May 2026 21:20:25 -0500 Subject: [PATCH 10/21] Remove unused DEFAULT_FORMAT constant Signed-off-by: Srikanth Padakanti --- .../dataprepper/plugins/source/file/FileSourceConfig.java | 1 - 1 file changed, 1 deletion(-) diff --git a/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileSourceConfig.java b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileSourceConfig.java index e76152b428..f9e25c3d91 100644 --- a/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileSourceConfig.java +++ b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileSourceConfig.java @@ -27,7 +27,6 @@ public class FileSourceConfig { static final String ATTRIBUTE_FORMAT = "format"; static final int DEFAULT_TIMEOUT = 5_000; static final String DEFAULT_TYPE = "string"; - static final String DEFAULT_FORMAT = "plain"; static final String EVENT_TYPE = "event"; @JsonProperty(ATTRIBUTE_PATH) From a0791b77503fea078881b593084d7e74997799f0 Mon Sep 17 00:00:00 2001 From: Srikanth Padakanti Date: Tue, 12 May 2026 00:23:20 -0500 Subject: [PATCH 11/21] Rename FileMetrics, remove tail prefix from metrics, fix formatting, consolidate codec creation, add file prefix to thread names Signed-off-by: Srikanth Padakanti --- .../source/file/CheckpointRegistry.java | 2 +- .../plugins/source/file/DirectoryWatcher.java | 14 +- ...{FileTailMetrics.java => FileMetrics.java} | 38 ++--- .../plugins/source/file/FileSource.java | 135 +++++++++--------- .../plugins/source/file/TailFileReader.java | 2 +- .../source/file/TailFileReaderContext.java | 6 +- .../source/file/TailFileReaderPool.java | 10 +- .../source/file/DirectoryWatcherTest.java | 2 +- ...lMetricsTest.java => FileMetricsTest.java} | 38 ++--- .../source/file/TailFileReaderPoolTest.java | 2 +- .../source/file/TailFileReaderTest.java | 2 +- 11 files changed, 126 insertions(+), 125 deletions(-) rename data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/{FileTailMetrics.java => FileMetrics.java} (66%) rename data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/{FileTailMetricsTest.java => FileMetricsTest.java} (80%) diff --git a/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/CheckpointRegistry.java b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/CheckpointRegistry.java index 13c11001c5..d1e3b0b1ac 100644 --- a/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/CheckpointRegistry.java +++ b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/CheckpointRegistry.java @@ -44,7 +44,7 @@ public final class CheckpointRegistry { public CheckpointRegistry(final Path checkpointFile, final Duration flushInterval, final Duration cleanupAfter) { this(checkpointFile, flushInterval, cleanupAfter, () -> Executors.newSingleThreadScheduledExecutor(r -> { - final Thread thread = new Thread(r, "checkpoint-flush"); + final Thread thread = new Thread(r, "file-checkpoint-flush"); thread.setDaemon(true); return thread; })); diff --git a/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/DirectoryWatcher.java b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/DirectoryWatcher.java index a37aee937d..a34cc17ee4 100644 --- a/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/DirectoryWatcher.java +++ b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/DirectoryWatcher.java @@ -53,7 +53,7 @@ interface WatchServiceFactory { private final CheckpointRegistry checkpointRegistry; private final FileSourceConfig config; private final FileSystemOperations fileOps; - private final FileTailMetrics metrics; + private final FileMetrics metrics; private final Duration rotateWait; private final boolean closeRemoved; private final Set knownFiles; @@ -72,7 +72,7 @@ public DirectoryWatcher(final GlobPathResolver globPathResolver, final CheckpointRegistry checkpointRegistry, final FileSourceConfig config, final FileSystemOperations fileOps, - final FileTailMetrics metrics, + final FileMetrics metrics, final Duration rotateWait, final boolean closeRemoved) { this(globPathResolver, readerPool, checkpointRegistry, config, fileOps, metrics, rotateWait, closeRemoved, @@ -84,7 +84,7 @@ public DirectoryWatcher(final GlobPathResolver globPathResolver, final CheckpointRegistry checkpointRegistry, final FileSourceConfig config, final FileSystemOperations fileOps, - final FileTailMetrics metrics, + final FileMetrics metrics, final Duration rotateWait, final boolean closeRemoved, final WatchServiceFactory watchServiceFactory) { @@ -97,7 +97,7 @@ public DirectoryWatcher(final GlobPathResolver globPathResolver, final CheckpointRegistry checkpointRegistry, final FileSourceConfig config, final FileSystemOperations fileOps, - final FileTailMetrics metrics, + final FileMetrics metrics, final Duration rotateWait, final boolean closeRemoved, final WatchServiceFactory watchServiceFactory, @@ -112,7 +112,7 @@ public DirectoryWatcher(final GlobPathResolver globPathResolver, final CheckpointRegistry checkpointRegistry, final FileSourceConfig config, final FileSystemOperations fileOps, - final FileTailMetrics metrics, + final FileMetrics metrics, final Duration rotateWait, final boolean closeRemoved, final WatchServiceFactory watchServiceFactory, @@ -128,7 +128,7 @@ public DirectoryWatcher(final GlobPathResolver globPathResolver, final CheckpointRegistry checkpointRegistry, final FileSourceConfig config, final FileSystemOperations fileOps, - final FileTailMetrics metrics, + final FileMetrics metrics, final Duration rotateWait, final boolean closeRemoved, final WatchServiceFactory watchServiceFactory, @@ -262,7 +262,7 @@ private void registerDirectory(final Path dir) { static ScheduledExecutorService createDefaultPollScheduler() { return Executors.newSingleThreadScheduledExecutor(r -> { - final Thread thread = new Thread(r, "file-tail-poll"); + final Thread thread = new Thread(r, "file-poll"); thread.setDaemon(true); return thread; }); diff --git a/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileTailMetrics.java b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileMetrics.java similarity index 66% rename from data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileTailMetrics.java rename to data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileMetrics.java index ad28d83b0d..335205a79d 100644 --- a/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileTailMetrics.java +++ b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileMetrics.java @@ -16,7 +16,7 @@ import java.util.concurrent.atomic.AtomicLong; -public final class FileTailMetrics { +public final class FileMetrics { private final Counter linesRead; private final Counter bytesRead; @@ -36,26 +36,26 @@ public final class FileTailMetrics { private final AtomicLong activeFileCount; private final AtomicLong fileLagBytes; - public FileTailMetrics(final PluginMetrics pluginMetrics) { - this.linesRead = pluginMetrics.counter("tailLinesRead"); - this.bytesRead = pluginMetrics.counter("tailBytesRead"); - this.linesTruncated = pluginMetrics.counter("tailLinesTruncated"); - this.filesOpened = pluginMetrics.counter("tailFilesOpened"); - this.filesClosed = pluginMetrics.counter("tailFilesClosed"); - this.filesRotated = pluginMetrics.counter("tailFilesRotated"); - this.readErrors = pluginMetrics.counter("tailReadErrors"); - this.writeTimeouts = pluginMetrics.counter("tailWriteTimeouts"); - this.checkpointFlushes = pluginMetrics.counter("tailCheckpointFlushes"); - this.checkpointErrors = pluginMetrics.counter("tailCheckpointErrors"); - this.eventsEmitted = pluginMetrics.counter("tailEventsEmitted"); - this.dataLossEvents = pluginMetrics.counter("tailDataLossEvents"); - this.acknowledgmentFailures = pluginMetrics.counter("tailAcknowledgmentFailures"); - this.truncationEvents = pluginMetrics.counter("tailTruncationEvents"); - this.backpressureTimer = pluginMetrics.timer("tailBackpressureTime"); + public FileMetrics(final PluginMetrics pluginMetrics) { + this.linesRead = pluginMetrics.counter("linesRead"); + this.bytesRead = pluginMetrics.counter("bytesRead"); + this.linesTruncated = pluginMetrics.counter("linesTruncated"); + this.filesOpened = pluginMetrics.counter("filesOpened"); + this.filesClosed = pluginMetrics.counter("filesClosed"); + this.filesRotated = pluginMetrics.counter("filesRotated"); + this.readErrors = pluginMetrics.counter("readErrors"); + this.writeTimeouts = pluginMetrics.counter("writeTimeouts"); + this.checkpointFlushes = pluginMetrics.counter("checkpointFlushes"); + this.checkpointErrors = pluginMetrics.counter("checkpointErrors"); + this.eventsEmitted = pluginMetrics.counter("eventsEmitted"); + this.dataLossEvents = pluginMetrics.counter("dataLossEvents"); + this.acknowledgmentFailures = pluginMetrics.counter("acknowledgmentFailures"); + this.truncationEvents = pluginMetrics.counter("truncationEvents"); + this.backpressureTimer = pluginMetrics.timer("backpressureTime"); this.activeFileCount = new AtomicLong(0); - pluginMetrics.gauge("tailActiveFiles", activeFileCount); + pluginMetrics.gauge("activeFiles", activeFileCount); this.fileLagBytes = new AtomicLong(0); - pluginMetrics.gauge("tailFileLagBytes", fileLagBytes); + pluginMetrics.gauge("fileLagBytes", fileLagBytes); } public Counter getLinesRead() { diff --git a/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileSource.java b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileSource.java index 0561e13bba..d385773761 100644 --- a/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileSource.java +++ b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileSource.java @@ -129,71 +129,64 @@ private void startTailing(final Buffer> buffer) { } try { + final FileMetrics tailMetrics = new FileMetrics(pluginMetrics); + final FileSystemOperations fileOps = new DefaultFileSystemOperations(); + + final String checkpointPath = fileSourceConfig.getCheckpointFile(); + final Path cpFile; + if (checkpointPath != null) { + cpFile = Paths.get(checkpointPath); + } else { + LOG.warn("No checkpoint_file configured. Checkpoint state will not be persisted across restarts."); + cpFile = null; + } - final FileTailMetrics tailMetrics = new FileTailMetrics(pluginMetrics); - final FileSystemOperations fileOps = new DefaultFileSystemOperations(); - - final String checkpointPath = fileSourceConfig.getCheckpointFile(); - final Path cpFile; - if (checkpointPath != null) { - cpFile = Paths.get(checkpointPath); - } else { - LOG.warn("No checkpoint_file configured. Checkpoint state will not be persisted across restarts."); - cpFile = null; - } - - checkpointRegistry = new CheckpointRegistry( - cpFile, - fileSourceConfig.getCheckpointInterval(), - fileSourceConfig.getCheckpointCleanupAfter()); - - final Charset encoding = Charset.forName(fileSourceConfig.getEncoding()); - - final RotationDetector rotationDetector = new RotationDetector(fileOps, fileSourceConfig.getFingerprintBytes()); - - InputCodec tailCodec = null; - if (fileSourceConfig.getCodec() != null) { - final PluginModel codecConfiguration = fileSourceConfig.getCodec(); - final PluginSetting codecPluginSettings = new PluginSetting( - codecConfiguration.getPluginName(), codecConfiguration.getPluginSettings()); - tailCodec = pluginFactory.loadPlugin(InputCodec.class, codecPluginSettings); - } - - final TailFileReaderContext readerContext = new TailFileReaderContext( - buffer, eventFactory, fileOps, tailMetrics, rotationDetector, - acknowledgementSetManager, acknowledgementsEnabled, - encoding, - fileSourceConfig.getReadBufferSize(), - fileSourceConfig.getMaxLineLength(), - writeTimeout, - fileSourceConfig.getMaxReadTimePerFile(), - fileSourceConfig.getRotationDrainTimeout(), - fileSourceConfig.getStartPosition(), - fileSourceConfig.isIncludeFileMetadata(), - fileSourceConfig.getAcknowledgmentTimeout(), - fileSourceConfig.getBatchSize(), - fileSourceConfig.getBatchTimeout(), - fileSourceConfig.getMaxAcknowledgmentRetries(), - tailCodec); - - readerPool = new TailFileReaderPool( - checkpointRegistry, tailMetrics, - maxActiveFiles, - readerThreads, - fileSourceConfig.getCloseInactive(), - readerContext); - - final GlobPathResolver globPathResolver = new GlobPathResolver( - fileSourceConfig.getAllPaths(), - fileSourceConfig.getExcludePaths()); - - directoryWatcher = new DirectoryWatcher( - globPathResolver, readerPool, checkpointRegistry, - fileSourceConfig, fileOps, tailMetrics, - fileSourceConfig.getRotateWait(), - fileSourceConfig.isCloseRemoved()); - - directoryWatcher.start(); + checkpointRegistry = new CheckpointRegistry( + cpFile, + fileSourceConfig.getCheckpointInterval(), + fileSourceConfig.getCheckpointCleanupAfter()); + + final Charset encoding = Charset.forName(fileSourceConfig.getEncoding()); + + final RotationDetector rotationDetector = new RotationDetector(fileOps, fileSourceConfig.getFingerprintBytes()); + + final InputCodec tailCodec = createCodec(); + + final TailFileReaderContext readerContext = new TailFileReaderContext( + buffer, eventFactory, fileOps, tailMetrics, rotationDetector, + acknowledgementSetManager, acknowledgementsEnabled, + encoding, + fileSourceConfig.getReadBufferSize(), + fileSourceConfig.getMaxLineLength(), + writeTimeout, + fileSourceConfig.getMaxReadTimePerFile(), + fileSourceConfig.getRotationDrainTimeout(), + fileSourceConfig.getStartPosition(), + fileSourceConfig.isIncludeFileMetadata(), + fileSourceConfig.getAcknowledgmentTimeout(), + fileSourceConfig.getBatchSize(), + fileSourceConfig.getBatchTimeout(), + fileSourceConfig.getMaxAcknowledgmentRetries(), + tailCodec); + + readerPool = new TailFileReaderPool( + checkpointRegistry, tailMetrics, + maxActiveFiles, + readerThreads, + fileSourceConfig.getCloseInactive(), + readerContext); + + final GlobPathResolver globPathResolver = new GlobPathResolver( + fileSourceConfig.getAllPaths(), + fileSourceConfig.getExcludePaths()); + + directoryWatcher = new DirectoryWatcher( + globPathResolver, readerPool, checkpointRegistry, + fileSourceConfig, fileOps, tailMetrics, + fileSourceConfig.getRotateWait(), + fileSourceConfig.isCloseRemoved()); + + directoryWatcher.start(); } catch (final RuntimeException e) { shutdownTailingResources(); throw e; @@ -212,6 +205,16 @@ private void shutdownTailingResources() { } } + private InputCodec createCodec() { + if (fileSourceConfig.getCodec() == null) { + return null; + } + final PluginModel codecConfiguration = fileSourceConfig.getCodec(); + final PluginSetting codecPluginSettings = new PluginSetting( + codecConfiguration.getPluginName(), codecConfiguration.getPluginSettings()); + return pluginFactory.loadPlugin(InputCodec.class, codecPluginSettings); + } + @Override public void stop() { isStopRequested = true; @@ -297,9 +300,7 @@ private class CodecFileStrategy implements FileStrategy { private final InputCodec codec; CodecFileStrategy(final PluginFactory pluginFactory) { - final PluginModel codecConfiguration = fileSourceConfig.getCodec(); - final PluginSetting codecPluginSettings = new PluginSetting(codecConfiguration.getPluginName(), codecConfiguration.getPluginSettings()); - codec = pluginFactory.loadPlugin(InputCodec.class, codecPluginSettings); + codec = createCodec(); } @Override diff --git a/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/TailFileReader.java b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/TailFileReader.java index e06b7e6855..396b62efeb 100644 --- a/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/TailFileReader.java +++ b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/TailFileReader.java @@ -59,7 +59,7 @@ public final class TailFileReader implements Runnable { private final EventFactory eventFactory; private final FileSystemOperations fileOps; private final CheckpointEntry checkpointEntry; - private final FileTailMetrics metrics; + private final FileMetrics metrics; private final Charset encoding; private final int readBufferSize; private final int maxLineLength; diff --git a/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/TailFileReaderContext.java b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/TailFileReaderContext.java index 4082793752..9974b9de26 100644 --- a/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/TailFileReaderContext.java +++ b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/TailFileReaderContext.java @@ -25,7 +25,7 @@ public final class TailFileReaderContext { private final Buffer> buffer; private final EventFactory eventFactory; private final FileSystemOperations fileOps; - private final FileTailMetrics metrics; + private final FileMetrics metrics; private final RotationDetector rotationDetector; private final AcknowledgementSetManager acknowledgementSetManager; private final boolean acknowledgementsEnabled; @@ -46,7 +46,7 @@ public final class TailFileReaderContext { public TailFileReaderContext(final Buffer> buffer, final EventFactory eventFactory, final FileSystemOperations fileOps, - final FileTailMetrics metrics, + final FileMetrics metrics, final RotationDetector rotationDetector, final AcknowledgementSetManager acknowledgementSetManager, final boolean acknowledgementsEnabled, @@ -97,7 +97,7 @@ public FileSystemOperations getFileOps() { return fileOps; } - public FileTailMetrics getMetrics() { + public FileMetrics getMetrics() { return metrics; } diff --git a/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/TailFileReaderPool.java b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/TailFileReaderPool.java index 66524af712..c213747b89 100644 --- a/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/TailFileReaderPool.java +++ b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/TailFileReaderPool.java @@ -38,27 +38,27 @@ public final class TailFileReaderPool { private final ExecutorService executorService; private final ScheduledExecutorService scheduler; private final CheckpointRegistry checkpointRegistry; - private final FileTailMetrics metrics; + private final FileMetrics metrics; private final int maxActiveFiles; private final Duration closeInactive; private final TailFileReaderContext readerContext; public TailFileReaderPool(final CheckpointRegistry checkpointRegistry, - final FileTailMetrics metrics, + final FileMetrics metrics, final int maxActiveFiles, final int readerThreads, final Duration closeInactive, final TailFileReaderContext readerContext) { this(checkpointRegistry, metrics, maxActiveFiles, closeInactive, readerContext, () -> Executors.newFixedThreadPool(readerThreads, r -> { - final Thread thread = new Thread(r, "tail-reader"); + final Thread thread = new Thread(r, "file-reader"); thread.setDaemon(true); return thread; })); } TailFileReaderPool(final CheckpointRegistry checkpointRegistry, - final FileTailMetrics metrics, + final FileMetrics metrics, final int maxActiveFiles, final Duration closeInactive, final TailFileReaderContext readerContext, @@ -73,7 +73,7 @@ public TailFileReaderPool(final CheckpointRegistry checkpointRegistry, this.pendingQueue = new ConcurrentLinkedQueue<>(); this.executorService = executorServiceSupplier.get(); this.scheduler = Executors.newSingleThreadScheduledExecutor(r -> { - final Thread thread = new Thread(r, "tail-reader-scheduler"); + final Thread thread = new Thread(r, "file-reader-scheduler"); thread.setDaemon(true); return thread; }); diff --git a/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/DirectoryWatcherTest.java b/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/DirectoryWatcherTest.java index 371a536339..dbe648a55e 100644 --- a/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/DirectoryWatcherTest.java +++ b/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/DirectoryWatcherTest.java @@ -67,7 +67,7 @@ class DirectoryWatcherTest { private FileSourceConfig config; @Mock - private FileTailMetrics metrics; + private FileMetrics metrics; private FileSystemOperations realFileOps; private GlobPathResolver globPathResolver; diff --git a/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/FileTailMetricsTest.java b/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/FileMetricsTest.java similarity index 80% rename from data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/FileTailMetricsTest.java rename to data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/FileMetricsTest.java index 38c79e1c98..f712d3a6af 100644 --- a/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/FileTailMetricsTest.java +++ b/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/FileMetricsTest.java @@ -28,7 +28,7 @@ import static org.mockito.Mockito.when; @ExtendWith(MockitoExtension.class) -class FileTailMetricsTest { +class FileMetricsTest { @Mock private PluginMetrics pluginMetrics; @@ -78,27 +78,27 @@ class FileTailMetricsTest { @Mock private Timer backpressureTimer; - private FileTailMetrics fileTailMetrics; + private FileMetrics fileTailMetrics; @BeforeEach void setUp() { - when(pluginMetrics.counter("tailLinesRead")).thenReturn(linesReadCounter); - when(pluginMetrics.counter("tailBytesRead")).thenReturn(bytesReadCounter); - when(pluginMetrics.counter("tailLinesTruncated")).thenReturn(linesTruncatedCounter); - when(pluginMetrics.counter("tailFilesOpened")).thenReturn(filesOpenedCounter); - when(pluginMetrics.counter("tailFilesClosed")).thenReturn(filesClosedCounter); - when(pluginMetrics.counter("tailFilesRotated")).thenReturn(filesRotatedCounter); - when(pluginMetrics.counter("tailReadErrors")).thenReturn(readErrorsCounter); - when(pluginMetrics.counter("tailWriteTimeouts")).thenReturn(writeTimeoutsCounter); - when(pluginMetrics.counter("tailCheckpointFlushes")).thenReturn(checkpointFlushesCounter); - when(pluginMetrics.counter("tailCheckpointErrors")).thenReturn(checkpointErrorsCounter); - when(pluginMetrics.counter("tailEventsEmitted")).thenReturn(eventsEmittedCounter); - when(pluginMetrics.counter("tailDataLossEvents")).thenReturn(dataLossEventsCounter); - when(pluginMetrics.counter("tailAcknowledgmentFailures")).thenReturn(acknowledgmentFailuresCounter); - when(pluginMetrics.counter("tailTruncationEvents")).thenReturn(truncationEventsCounter); - when(pluginMetrics.timer("tailBackpressureTime")).thenReturn(backpressureTimer); - - fileTailMetrics = new FileTailMetrics(pluginMetrics); + when(pluginMetrics.counter("linesRead")).thenReturn(linesReadCounter); + when(pluginMetrics.counter("bytesRead")).thenReturn(bytesReadCounter); + when(pluginMetrics.counter("linesTruncated")).thenReturn(linesTruncatedCounter); + when(pluginMetrics.counter("filesOpened")).thenReturn(filesOpenedCounter); + when(pluginMetrics.counter("filesClosed")).thenReturn(filesClosedCounter); + when(pluginMetrics.counter("filesRotated")).thenReturn(filesRotatedCounter); + when(pluginMetrics.counter("readErrors")).thenReturn(readErrorsCounter); + when(pluginMetrics.counter("writeTimeouts")).thenReturn(writeTimeoutsCounter); + when(pluginMetrics.counter("checkpointFlushes")).thenReturn(checkpointFlushesCounter); + when(pluginMetrics.counter("checkpointErrors")).thenReturn(checkpointErrorsCounter); + when(pluginMetrics.counter("eventsEmitted")).thenReturn(eventsEmittedCounter); + when(pluginMetrics.counter("dataLossEvents")).thenReturn(dataLossEventsCounter); + when(pluginMetrics.counter("acknowledgmentFailures")).thenReturn(acknowledgmentFailuresCounter); + when(pluginMetrics.counter("truncationEvents")).thenReturn(truncationEventsCounter); + when(pluginMetrics.timer("backpressureTime")).thenReturn(backpressureTimer); + + fileTailMetrics = new FileMetrics(pluginMetrics); } @Test diff --git a/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/TailFileReaderPoolTest.java b/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/TailFileReaderPoolTest.java index 658ddb6a84..5dc7d40b9a 100644 --- a/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/TailFileReaderPoolTest.java +++ b/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/TailFileReaderPoolTest.java @@ -70,7 +70,7 @@ class TailFileReaderPoolTest { private CheckpointRegistry checkpointRegistry; @Mock - private FileTailMetrics metrics; + private FileMetrics metrics; @Mock private RotationDetector rotationDetector; diff --git a/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/TailFileReaderTest.java b/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/TailFileReaderTest.java index 7e7a06e433..7af32017d2 100644 --- a/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/TailFileReaderTest.java +++ b/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/TailFileReaderTest.java @@ -79,7 +79,7 @@ class TailFileReaderTest { private FileSystemOperations fileOps; @Mock - private FileTailMetrics metrics; + private FileMetrics metrics; @Mock private RotationDetector rotationDetector; From 624cf4a687e1b2ce72362ec671876b44fd5f3791 Mon Sep 17 00:00:00 2001 From: Srikanth Padakanti Date: Tue, 12 May 2026 00:45:18 -0500 Subject: [PATCH 12/21] Make config options orthogonal so paths and glob work with tail false Signed-off-by: Srikanth Padakanti --- .../plugins/source/file/FileSource.java | 58 +++++++++++++------ .../plugins/source/file/FileSourceConfig.java | 11 +--- .../source/file/FileSourceConfigTest.java | 10 +++- .../plugins/source/file/FileSourceTests.java | 25 ++++++-- 4 files changed, 73 insertions(+), 31 deletions(-) diff --git a/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileSource.java b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileSource.java index d385773761..389444e8e6 100644 --- a/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileSource.java +++ b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileSource.java @@ -41,6 +41,7 @@ import java.nio.file.Paths; import java.util.HashMap; import java.util.Map; +import java.util.Set; import java.util.concurrent.TimeoutException; import static java.lang.String.format; @@ -108,7 +109,7 @@ public void start(final Buffer> buffer) { return; } - LOG.info("Starting file source with {} path.", fileSourceConfig.getFilePathToRead()); + LOG.info("Starting file source with paths: {}", fileSourceConfig.getAllPaths()); readThread = new Thread(() -> { fileStrategy.start(buffer); @@ -242,16 +243,29 @@ private interface FileStrategy { private class ClassicFileStrategy implements FileStrategy { @Override public void start(Buffer> buffer) { - Path filePath = Paths.get(fileSourceConfig.getFilePathToRead()); + final GlobPathResolver resolver = new GlobPathResolver( + fileSourceConfig.getAllPaths(), fileSourceConfig.getExcludePaths()); + final Set resolvedPaths = resolver.resolve(); + if (resolvedPaths.isEmpty() && fileSourceConfig.getFilePathToRead() != null) { + resolvedPaths.add(Paths.get(fileSourceConfig.getFilePathToRead()).toAbsolutePath().normalize()); + } + for (final Path filePath : resolvedPaths) { + if (isStopRequested) { + break; + } + readFile(filePath, buffer); + } + } + + private void readFile(final Path filePath, final Buffer> buffer) { try (BufferedReader reader = new BufferedReader(new InputStreamReader(decompressionEngine.createInputStream(Files.newInputStream(filePath)), Charset.forName(fileSourceConfig.getEncoding())))) { String line; while ((line = reader.readLine()) != null && !isStopRequested) { writeLineAsEventOrString(line, buffer); } } catch (IOException | TimeoutException | IllegalArgumentException ex) { - LOG.error("Error processing the input file path [{}]", fileSourceConfig.getFilePathToRead(), ex); - throw new RuntimeException(format("Error processing the input file %s", - fileSourceConfig.getFilePathToRead()), ex); + LOG.error("Error processing the input file path [{}]", filePath, ex); + throw new RuntimeException(format("Error processing the input file %s", filePath), ex); } } @@ -305,19 +319,29 @@ private class CodecFileStrategy implements FileStrategy { @Override public void start(final Buffer> buffer) { - Path filePath = Paths.get(fileSourceConfig.getFilePathToRead()); - try(InputStream is = decompressionEngine.createInputStream(Files.newInputStream(filePath))) { - codec.parse(is, eventRecord -> { - try { - buffer.write((Record) eventRecord, writeTimeout); - } catch (TimeoutException e) { - throw new RuntimeException(e); - } - }); - } catch (final IOException e) { - throw new RuntimeException(e); + final GlobPathResolver resolver = new GlobPathResolver( + fileSourceConfig.getAllPaths(), fileSourceConfig.getExcludePaths()); + final Set resolvedPaths = resolver.resolve(); + if (resolvedPaths.isEmpty() && fileSourceConfig.getFilePathToRead() != null) { + resolvedPaths.add(Paths.get(fileSourceConfig.getFilePathToRead()).toAbsolutePath().normalize()); + } + for (final Path filePath : resolvedPaths) { + if (isStopRequested) { + break; + } + try (InputStream is = decompressionEngine.createInputStream(Files.newInputStream(filePath))) { + codec.parse(is, eventRecord -> { + try { + buffer.write((Record) eventRecord, writeTimeout); + } catch (TimeoutException e) { + throw new RuntimeException(e); + } + }); + } catch (final IOException e) { + LOG.error("Error processing file with codec [{}]", filePath, e); + throw new RuntimeException(e); + } } - } } diff --git a/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileSourceConfig.java b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileSourceConfig.java index f9e25c3d91..537bc80e31 100644 --- a/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileSourceConfig.java +++ b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileSourceConfig.java @@ -248,14 +248,9 @@ public List getExcludePaths() { } void validate() { - if (tail) { - Preconditions.checkArgument( - (filePathToRead != null && !filePathToRead.isEmpty()) || !paths.isEmpty(), - "At least one of path or paths is required when tail is enabled"); - } else { - Preconditions.checkArgument(filePathToRead != null, - "path is required when tail is disabled. Use paths with tail: true for glob patterns."); - } + Preconditions.checkArgument( + (filePathToRead != null && !filePathToRead.isEmpty()) || !paths.isEmpty(), + "At least one of path or paths is required"); } @AssertTrue(message = "The file source requires recordType to be event when using a codec.") diff --git a/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/FileSourceConfigTest.java b/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/FileSourceConfigTest.java index d4e0ac6ba8..294acd85eb 100644 --- a/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/FileSourceConfigTest.java +++ b/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/FileSourceConfigTest.java @@ -121,13 +121,21 @@ void validate_succeeds_with_path_when_tail_false() { } @Test - void validate_fails_without_path_when_tail_false() { + void validate_fails_without_any_path() { final Map configMap = Map.of("format", "plain", "record_type", "string"); final FileSourceConfig config = OBJECT_MAPPER.convertValue(configMap, FileSourceConfig.class); assertThrows(IllegalArgumentException.class, config::validate); } + @Test + void validate_succeeds_with_paths_when_tail_false() { + final Map configMap = Map.of("paths", List.of("/var/log/*.log")); + final FileSourceConfig config = OBJECT_MAPPER.convertValue(configMap, FileSourceConfig.class); + + config.validate(); + } + @Test void validate_succeeds_with_paths_when_tail_true() { final Map configMap = Map.of("tail", true, "paths", List.of("/var/log/*.log")); diff --git a/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/FileSourceTests.java b/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/FileSourceTests.java index 9989812882..0e78b19334 100644 --- a/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/FileSourceTests.java +++ b/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/FileSourceTests.java @@ -182,12 +182,9 @@ private BlockingBuffer> getBuffer() throws JsonProcessingExceptio } @Test - public void testFileSourceWithEmptyFilePathDoesNotWriteToBuffer() throws TimeoutException { - buffer = mock(Buffer.class); + public void testFileSourceWithEmptyFilePathThrowsValidationError() { pluginSettings.put(FileSourceConfig.ATTRIBUTE_PATH, ""); - fileSource = createObjectUnderTest(); - fileSource.start(buffer); - verify(buffer, after(500).never()).write(any(Record.class), anyInt()); + assertThrows(IllegalArgumentException.class, () -> createObjectUnderTest()); } @Test @@ -205,6 +202,15 @@ public void testFileSourceWithNullFilePathThrowsNullPointerException() { assertThrows(IllegalArgumentException.class, FileSourceTests.this::createObjectUnderTest); } + @Test + public void testStopBeforeStartPreventsProcessing() throws TimeoutException { + buffer = mock(Buffer.class); + fileSource = createObjectUnderTest(); + fileSource.stop(); + fileSource.start(buffer); + verify(buffer, after(500).never()).write(any(Record.class), anyInt()); + } + @Test public void testFileWithPlainTextAddsEventsToBufferCorrectly() throws JsonProcessingException { fileSource = createObjectUnderTest(); @@ -386,6 +392,15 @@ void start_codec_consumer_wraps_timeout_exception() throws IOException, TimeoutE assertThrows(RuntimeException.class, () -> actualConsumer.accept(record)); } + + @Test + void stop_before_start_prevents_codec_processing() throws IOException { + final FileSource objectUnderTest = createObjectUnderTest(); + objectUnderTest.stop(); + objectUnderTest.start(buffer); + + verify(inputCodec, after(500).never()).parse(any(InputStream.class), any(Consumer.class)); + } } @Nested From 41ef3fa61a253ebc61c68ceae12a2249cab3602b Mon Sep 17 00:00:00 2001 From: Srikanth Padakanti Date: Fri, 22 May 2026 12:31:49 -0500 Subject: [PATCH 13/21] Add test for validate() rejecting start_position:end when tail is false Signed-off-by: Srikanth Padakanti --- .../plugins/source/file/FileSourceConfigTest.java | 13 ++++++++++++- 1 file changed, 12 insertions(+), 1 deletion(-) diff --git a/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/FileSourceConfigTest.java b/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/FileSourceConfigTest.java index 294acd85eb..649b191e93 100644 --- a/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/FileSourceConfigTest.java +++ b/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/FileSourceConfigTest.java @@ -165,7 +165,7 @@ void default_config_returns_expected_values() { final Map configMap = Map.of("path", "/tmp/test.log"); final FileSourceConfig config = OBJECT_MAPPER.convertValue(configMap, FileSourceConfig.class); - assertThat(config.getStartPosition(), equalTo(StartPosition.END)); + assertThat(config.getStartPosition(), equalTo(StartPosition.BEGINNING)); assertThat(config.getPollInterval(), equalTo(Duration.ofSeconds(1))); assertThat(config.getEncoding(), equalTo("UTF-8")); assertThat(config.getReadBufferSize(), equalTo(65536)); @@ -226,6 +226,17 @@ void validate_fails_when_tail_true_and_filePathToRead_is_empty_and_paths_is_null assertThrows(IllegalArgumentException.class, config::validate); } + @Test + void validate_fails_when_start_position_end_and_tail_false() { + final Map configMap = Map.of( + "path", "/tmp/test.log", + "start_position", "end" + ); + final FileSourceConfig config = OBJECT_MAPPER.convertValue(configMap, FileSourceConfig.class); + + assertThrows(IllegalArgumentException.class, config::validate); + } + @Test void getFormat_throws_when_format_is_null() { assertThrows(IllegalArgumentException.class, () -> FileFormat.fromString(null)); From 6473fc9bf7a06e70c01f20fad2fa6f303d0ab21e Mon Sep 17 00:00:00 2001 From: Srikanth Padakanti Date: Fri, 22 May 2026 14:09:35 -0500 Subject: [PATCH 14/21] Rename TailFileReader to FileReader, unify modern path for tail and one-shot modes Signed-off-by: Srikanth Padakanti --- .../plugins/source/file/DirectoryWatcher.java | 12 +- .../{TailFileReader.java => FileReader.java} | 41 +++- ...derContext.java => FileReaderContext.java} | 21 +- ...ileReaderPool.java => FileReaderPool.java} | 24 +-- .../plugins/source/file/FileSource.java | 108 ++++------- .../plugins/source/file/FileSourceConfig.java | 14 +- .../source/file/DirectoryWatcherTest.java | 2 +- ...rPoolTest.java => FileReaderPoolTest.java} | 60 +++--- ...ileReaderTest.java => FileReaderTest.java} | 182 +++++++++--------- .../plugins/source/file/FileSourceTests.java | 46 ++--- 10 files changed, 260 insertions(+), 250 deletions(-) rename data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/{TailFileReader.java => FileReader.java} (92%) rename data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/{TailFileReaderContext.java => FileReaderContext.java} (89%) rename data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/{TailFileReaderPool.java => FileReaderPool.java} (91%) rename data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/{TailFileReaderPoolTest.java => FileReaderPoolTest.java} (93%) rename data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/{TailFileReaderTest.java => FileReaderTest.java} (89%) diff --git a/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/DirectoryWatcher.java b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/DirectoryWatcher.java index a34cc17ee4..140480658b 100644 --- a/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/DirectoryWatcher.java +++ b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/DirectoryWatcher.java @@ -49,7 +49,7 @@ interface WatchServiceFactory { ); private final GlobPathResolver globPathResolver; - private final TailFileReaderPool readerPool; + private final FileReaderPool readerPool; private final CheckpointRegistry checkpointRegistry; private final FileSourceConfig config; private final FileSystemOperations fileOps; @@ -68,7 +68,7 @@ interface WatchServiceFactory { private volatile boolean running; public DirectoryWatcher(final GlobPathResolver globPathResolver, - final TailFileReaderPool readerPool, + final FileReaderPool readerPool, final CheckpointRegistry checkpointRegistry, final FileSourceConfig config, final FileSystemOperations fileOps, @@ -80,7 +80,7 @@ public DirectoryWatcher(final GlobPathResolver globPathResolver, } DirectoryWatcher(final GlobPathResolver globPathResolver, - final TailFileReaderPool readerPool, + final FileReaderPool readerPool, final CheckpointRegistry checkpointRegistry, final FileSourceConfig config, final FileSystemOperations fileOps, @@ -93,7 +93,7 @@ public DirectoryWatcher(final GlobPathResolver globPathResolver, } DirectoryWatcher(final GlobPathResolver globPathResolver, - final TailFileReaderPool readerPool, + final FileReaderPool readerPool, final CheckpointRegistry checkpointRegistry, final FileSourceConfig config, final FileSystemOperations fileOps, @@ -108,7 +108,7 @@ public DirectoryWatcher(final GlobPathResolver globPathResolver, } DirectoryWatcher(final GlobPathResolver globPathResolver, - final TailFileReaderPool readerPool, + final FileReaderPool readerPool, final CheckpointRegistry checkpointRegistry, final FileSourceConfig config, final FileSystemOperations fileOps, @@ -124,7 +124,7 @@ public DirectoryWatcher(final GlobPathResolver globPathResolver, } DirectoryWatcher(final GlobPathResolver globPathResolver, - final TailFileReaderPool readerPool, + final FileReaderPool readerPool, final CheckpointRegistry checkpointRegistry, final FileSourceConfig config, final FileSystemOperations fileOps, diff --git a/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/TailFileReader.java b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileReader.java similarity index 92% rename from data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/TailFileReader.java rename to data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileReader.java index 396b62efeb..7b0aaf6cd8 100644 --- a/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/TailFileReader.java +++ b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileReader.java @@ -13,6 +13,7 @@ import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSet; import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSetManager; import org.opensearch.dataprepper.model.buffer.Buffer; +import org.opensearch.dataprepper.model.codec.DecompressionEngine; import org.opensearch.dataprepper.model.codec.InputCodec; import org.opensearch.dataprepper.model.event.Event; import org.opensearch.dataprepper.model.event.EventBuilder; @@ -24,6 +25,7 @@ import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.IOException; +import java.io.InputStream; import java.nio.ByteBuffer; import java.nio.CharBuffer; import java.nio.channels.FileChannel; @@ -31,6 +33,7 @@ import java.nio.charset.CharsetDecoder; import java.nio.charset.CoderResult; import java.nio.charset.CodingErrorAction; +import java.nio.file.Files; import java.nio.file.NoSuchFileException; import java.nio.file.Path; import java.time.Duration; @@ -42,9 +45,9 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; -public final class TailFileReader implements Runnable { +public final class FileReader implements Runnable { - private static final Logger LOG = LoggerFactory.getLogger(TailFileReader.class); + private static final Logger LOG = LoggerFactory.getLogger(FileReader.class); private static final long BACK_PRESSURE_SLEEP_MILLIS = 100; private static final String MESSAGE_KEY = "message"; private static final String EVENT_TYPE = "event"; @@ -77,6 +80,8 @@ public final class TailFileReader implements Runnable { private final Duration batchTimeout; private final int maxAcknowledgmentRetries; private final InputCodec codec; + private final boolean tailMode; + private final DecompressionEngine decompressionEngine; private final AtomicLong readOffset; private final StringBuilder partialLine; @@ -92,10 +97,10 @@ public final class TailFileReader implements Runnable { private final AtomicInteger acknowledgmentRetryCount = new AtomicInteger(0); private volatile RotationType lastRotationType; - public TailFileReader(final Path path, + public FileReader(final Path path, final FileIdentity fileIdentity, final CheckpointEntry checkpointEntry, - final TailFileReaderContext context, + final FileReaderContext context, final Runnable onComplete) { this.path = Objects.requireNonNull(path, "path must not be null"); this.fileIdentity = Objects.requireNonNull(fileIdentity, "fileIdentity must not be null"); @@ -123,6 +128,8 @@ public TailFileReader(final Path path, this.batchTimeout = context.getBatchTimeout(); this.maxAcknowledgmentRetries = context.getMaxAcknowledgmentRetries(); this.codec = context.getCodec(); + this.tailMode = context.isTailMode(); + this.decompressionEngine = context.getDecompressionEngine(); this.readOffset = new AtomicLong(checkpointEntry.getReadOffset()); if (checkpointEntry.getReadOffset() == 0 && startPosition == StartPosition.END) { @@ -146,6 +153,11 @@ public TailFileReader(final Path path, @Override public void run() { try { + if (!tailMode && codec != null) { + readFileWithCodecOneShot(); + return; + } + final RotationResult rotation = rotationDetector.checkRotation(path, fileIdentity, readOffset.get()); lastRotationType = rotation.getRotationType(); @@ -181,6 +193,27 @@ public void run() { } } + private void readFileWithCodecOneShot() { + try (final InputStream rawStream = Files.newInputStream(path); + final InputStream decompressedStream = decompressionEngine.createInputStream(rawStream)) { + metrics.getFilesOpened().increment(); + codec.parse(decompressedStream, record -> { + try { + buffer.write((Record) record, writeTimeout); + metrics.getEventsEmitted().increment(); + } catch (final TimeoutException e) { + metrics.getWriteTimeouts().increment(); + throw new RuntimeException(e); + } + }); + } catch (final IOException e) { + LOG.error("Error reading file with codec: {}", path, e); + metrics.getReadErrors().increment(); + } finally { + metrics.getFilesClosed().increment(); + } + } + private void drainCurrentFile() { try (final FileChannel channel = fileOps.openReadChannel(path)) { metrics.getFilesOpened().increment(); diff --git a/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/TailFileReaderContext.java b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileReaderContext.java similarity index 89% rename from data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/TailFileReaderContext.java rename to data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileReaderContext.java index 9974b9de26..36ca0aabc1 100644 --- a/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/TailFileReaderContext.java +++ b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileReaderContext.java @@ -12,6 +12,7 @@ import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSetManager; import org.opensearch.dataprepper.model.buffer.Buffer; +import org.opensearch.dataprepper.model.codec.DecompressionEngine; import org.opensearch.dataprepper.model.codec.InputCodec; import org.opensearch.dataprepper.model.event.EventFactory; import org.opensearch.dataprepper.model.record.Record; @@ -20,7 +21,7 @@ import java.time.Duration; import java.util.Objects; -public final class TailFileReaderContext { +public final class FileReaderContext { private final Buffer> buffer; private final EventFactory eventFactory; @@ -42,8 +43,10 @@ public final class TailFileReaderContext { private final Duration batchTimeout; private final int maxAcknowledgmentRetries; private final InputCodec codec; + private final boolean tailMode; + private final DecompressionEngine decompressionEngine; - public TailFileReaderContext(final Buffer> buffer, + public FileReaderContext(final Buffer> buffer, final EventFactory eventFactory, final FileSystemOperations fileOps, final FileMetrics metrics, @@ -62,7 +65,9 @@ public TailFileReaderContext(final Buffer> buffer, final int batchSize, final Duration batchTimeout, final int maxAcknowledgmentRetries, - final InputCodec codec) { + final InputCodec codec, + final boolean tailMode, + final DecompressionEngine decompressionEngine) { this.buffer = Objects.requireNonNull(buffer, "buffer must not be null"); this.eventFactory = Objects.requireNonNull(eventFactory, "eventFactory must not be null"); this.fileOps = Objects.requireNonNull(fileOps, "fileOps must not be null"); @@ -83,6 +88,8 @@ public TailFileReaderContext(final Buffer> buffer, this.batchTimeout = Objects.requireNonNull(batchTimeout, "batchTimeout must not be null"); this.maxAcknowledgmentRetries = maxAcknowledgmentRetries; this.codec = codec; + this.tailMode = tailMode; + this.decompressionEngine = decompressionEngine; } public Buffer> getBuffer() { @@ -164,4 +171,12 @@ public int getMaxAcknowledgmentRetries() { public InputCodec getCodec() { return codec; } + + public boolean isTailMode() { + return tailMode; + } + + public DecompressionEngine getDecompressionEngine() { + return decompressionEngine; + } } diff --git a/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/TailFileReaderPool.java b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileReaderPool.java similarity index 91% rename from data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/TailFileReaderPool.java rename to data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileReaderPool.java index c213747b89..ded792fa81 100644 --- a/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/TailFileReaderPool.java +++ b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileReaderPool.java @@ -26,13 +26,13 @@ import java.util.concurrent.TimeUnit; import java.util.function.Supplier; -public final class TailFileReaderPool { +public final class FileReaderPool { - private static final Logger LOG = LoggerFactory.getLogger(TailFileReaderPool.class); + private static final Logger LOG = LoggerFactory.getLogger(FileReaderPool.class); private static final long SHUTDOWN_TIMEOUT_SECONDS = 30; private static final long RE_POLL_DELAY_MILLIS = 500; - private final ConcurrentHashMap activeReaders; + private final ConcurrentHashMap activeReaders; private final Set pendingIdentities; private final ConcurrentLinkedQueue pendingQueue; private final ExecutorService executorService; @@ -41,14 +41,14 @@ public final class TailFileReaderPool { private final FileMetrics metrics; private final int maxActiveFiles; private final Duration closeInactive; - private final TailFileReaderContext readerContext; + private final FileReaderContext readerContext; - public TailFileReaderPool(final CheckpointRegistry checkpointRegistry, + public FileReaderPool(final CheckpointRegistry checkpointRegistry, final FileMetrics metrics, final int maxActiveFiles, final int readerThreads, final Duration closeInactive, - final TailFileReaderContext readerContext) { + final FileReaderContext readerContext) { this(checkpointRegistry, metrics, maxActiveFiles, closeInactive, readerContext, () -> Executors.newFixedThreadPool(readerThreads, r -> { final Thread thread = new Thread(r, "file-reader"); @@ -57,11 +57,11 @@ public TailFileReaderPool(final CheckpointRegistry checkpointRegistry, })); } - TailFileReaderPool(final CheckpointRegistry checkpointRegistry, + FileReaderPool(final CheckpointRegistry checkpointRegistry, final FileMetrics metrics, final int maxActiveFiles, final Duration closeInactive, - final TailFileReaderContext readerContext, + final FileReaderContext readerContext, final Supplier executorServiceSupplier) { this.checkpointRegistry = Objects.requireNonNull(checkpointRegistry, "checkpointRegistry must not be null"); this.metrics = Objects.requireNonNull(metrics, "metrics must not be null"); @@ -98,7 +98,7 @@ private synchronized void submitReader(final FileIdentity fileIdentity, final Pa return; } final CheckpointEntry checkpoint = checkpointRegistry.getOrCreate(fileIdentity.toString()); - final TailFileReader reader = new TailFileReader( + final FileReader reader = new FileReader( path, fileIdentity, checkpoint, readerContext, () -> onReaderComplete(fileIdentity, path)); activeReaders.put(fileIdentity, reader); @@ -113,7 +113,7 @@ private synchronized void submitReader(final FileIdentity fileIdentity, final Pa } private synchronized void onReaderComplete(final FileIdentity fileIdentity, final Path path) { - final TailFileReader completedReader = activeReaders.remove(fileIdentity); + final FileReader completedReader = activeReaders.remove(fileIdentity); if (completedReader == null) { return; } @@ -173,7 +173,7 @@ public synchronized void closeInactiveReaders() { final long now = System.currentTimeMillis(); final long inactiveThresholdMillis = closeInactive.toMillis(); activeReaders.entrySet().removeIf(entry -> { - final TailFileReader reader = entry.getValue(); + final FileReader reader = entry.getValue(); if ((now - reader.getLastActivityMillis()) >= inactiveThresholdMillis) { LOG.info("Closing inactive reader for file identity {}", entry.getKey()); metrics.getActiveFileCount().decrementAndGet(); @@ -188,7 +188,7 @@ public synchronized void closeInactiveReaders() { public synchronized void closeReaderForPath(final Path path) { final Path absolutePath = path.toAbsolutePath().normalize(); activeReaders.entrySet().removeIf(entry -> { - final TailFileReader reader = entry.getValue(); + final FileReader reader = entry.getValue(); if (reader.getPath().toAbsolutePath().normalize().equals(absolutePath)) { LOG.info("Closing reader for removed file: {}", path); metrics.getActiveFileCount().decrementAndGet(); diff --git a/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileSource.java b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileSource.java index 389444e8e6..2f1545fd13 100644 --- a/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileSource.java +++ b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileSource.java @@ -32,7 +32,6 @@ import java.io.BufferedReader; import java.io.IOException; -import java.io.InputStream; import java.io.InputStreamReader; import java.nio.charset.Charset; import java.util.Objects; @@ -66,7 +65,7 @@ public class FileSource implements Source> { private final boolean acknowledgementsEnabled; private Thread readThread; - private TailFileReaderPool readerPool; + private FileReaderPool readerPool; private CheckpointRegistry checkpointRegistry; private DirectoryWatcher directoryWatcher; @@ -92,10 +91,10 @@ public FileSource( this.writeTimeout = FileSourceConfig.DEFAULT_TIMEOUT; this.decompressionEngine = fileSourceConfig.getCompression().getDecompressionEngine(); - if (!fileSourceConfig.isTail() && fileSourceConfig.getCodec() != null) { - fileStrategy = new CodecFileStrategy(pluginFactory); - } else { + if (fileSourceConfig.isLegacyConfig()) { fileStrategy = new ClassicFileStrategy(); + } else { + fileStrategy = null; } } @@ -104,33 +103,32 @@ public FileSource( public void start(final Buffer> buffer) { Objects.requireNonNull(buffer, "Buffer cannot be null for file source to start"); - if (fileSourceConfig.isTail()) { - startTailing(buffer); + if (fileSourceConfig.isLegacyConfig()) { + LOG.info("Starting file source in legacy mode with path: {}", fileSourceConfig.getFilePathToRead()); + readThread = new Thread(() -> { + fileStrategy.start(buffer); + LOG.info("Completed reading file."); + }, "file-source"); + readThread.setDaemon(false); + readThread.start(); return; } - LOG.info("Starting file source with paths: {}", fileSourceConfig.getAllPaths()); - - readThread = new Thread(() -> { - fileStrategy.start(buffer); - LOG.info("Completed reading file."); - }, "file-source"); - readThread.setDaemon(false); - readThread.start(); + startModernPath(buffer); } - private void startTailing(final Buffer> buffer) { - LOG.info("Starting file source in tail mode with paths: {}", fileSourceConfig.getAllPaths()); + private void startModernPath(final Buffer> buffer) { + LOG.info("Starting file source with paths: {}", fileSourceConfig.getAllPaths()); final int maxActiveFiles = fileSourceConfig.getMaxActiveFiles(); - final int readerThreads = fileSourceConfig.getReaderThreads(); + final int readerThreads = fileSourceConfig.getEffectiveReaderThreads(); if (readerThreads > 0 && maxActiveFiles / readerThreads > MAX_FILES_PER_THREAD_WARNING_THRESHOLD) { LOG.warn("max_active_files ({}) is {} times reader_threads ({}). Files with pending data may experience high latency.", maxActiveFiles, maxActiveFiles / readerThreads, readerThreads); } try { - final FileMetrics tailMetrics = new FileMetrics(pluginMetrics); + final FileMetrics fileMetrics = new FileMetrics(pluginMetrics); final FileSystemOperations fileOps = new DefaultFileSystemOperations(); final String checkpointPath = fileSourceConfig.getCheckpointFile(); @@ -151,10 +149,10 @@ private void startTailing(final Buffer> buffer) { final RotationDetector rotationDetector = new RotationDetector(fileOps, fileSourceConfig.getFingerprintBytes()); - final InputCodec tailCodec = createCodec(); + final InputCodec fileCodec = createCodec(); - final TailFileReaderContext readerContext = new TailFileReaderContext( - buffer, eventFactory, fileOps, tailMetrics, rotationDetector, + final FileReaderContext readerContext = new FileReaderContext( + buffer, eventFactory, fileOps, fileMetrics, rotationDetector, acknowledgementSetManager, acknowledgementsEnabled, encoding, fileSourceConfig.getReadBufferSize(), @@ -168,10 +166,12 @@ private void startTailing(final Buffer> buffer) { fileSourceConfig.getBatchSize(), fileSourceConfig.getBatchTimeout(), fileSourceConfig.getMaxAcknowledgmentRetries(), - tailCodec); + fileCodec, + fileSourceConfig.isTail(), + decompressionEngine); - readerPool = new TailFileReaderPool( - checkpointRegistry, tailMetrics, + readerPool = new FileReaderPool( + checkpointRegistry, fileMetrics, maxActiveFiles, readerThreads, fileSourceConfig.getCloseInactive(), @@ -181,13 +181,20 @@ private void startTailing(final Buffer> buffer) { fileSourceConfig.getAllPaths(), fileSourceConfig.getExcludePaths()); - directoryWatcher = new DirectoryWatcher( - globPathResolver, readerPool, checkpointRegistry, - fileSourceConfig, fileOps, tailMetrics, - fileSourceConfig.getRotateWait(), - fileSourceConfig.isCloseRemoved()); - - directoryWatcher.start(); + if (fileSourceConfig.isTail()) { + directoryWatcher = new DirectoryWatcher( + globPathResolver, readerPool, checkpointRegistry, + fileSourceConfig, fileOps, fileMetrics, + fileSourceConfig.getRotateWait(), + fileSourceConfig.isCloseRemoved()); + directoryWatcher.start(); + } else { + final Set resolvedPaths = globPathResolver.resolve(); + for (final Path path : resolvedPaths) { + final FileIdentity fileIdentity = FileIdentity.from(path, fileOps, fileSourceConfig.getFingerprintBytes()); + readerPool.addFile(fileIdentity, path); + } + } } catch (final RuntimeException e) { shutdownTailingResources(); throw e; @@ -308,41 +315,4 @@ private void writeLineAsEventOrString(final String line, final Buffer> buffer) { - final GlobPathResolver resolver = new GlobPathResolver( - fileSourceConfig.getAllPaths(), fileSourceConfig.getExcludePaths()); - final Set resolvedPaths = resolver.resolve(); - if (resolvedPaths.isEmpty() && fileSourceConfig.getFilePathToRead() != null) { - resolvedPaths.add(Paths.get(fileSourceConfig.getFilePathToRead()).toAbsolutePath().normalize()); - } - for (final Path filePath : resolvedPaths) { - if (isStopRequested) { - break; - } - try (InputStream is = decompressionEngine.createInputStream(Files.newInputStream(filePath))) { - codec.parse(is, eventRecord -> { - try { - buffer.write((Record) eventRecord, writeTimeout); - } catch (TimeoutException e) { - throw new RuntimeException(e); - } - }); - } catch (final IOException e) { - LOG.error("Error processing file with codec [{}]", filePath, e); - throw new RuntimeException(e); - } - } - } - } - } diff --git a/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileSourceConfig.java b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileSourceConfig.java index 537bc80e31..f52d636f32 100644 --- a/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileSourceConfig.java +++ b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileSourceConfig.java @@ -51,7 +51,7 @@ public class FileSourceConfig { private CompressionOption compression = CompressionOption.NONE; @JsonProperty("start_position") - private StartPosition startPosition = StartPosition.END; + private StartPosition startPosition = StartPosition.BEGINNING; @JsonProperty("poll_interval") private Duration pollInterval = Duration.ofSeconds(1); @@ -251,6 +251,18 @@ void validate() { Preconditions.checkArgument( (filePathToRead != null && !filePathToRead.isEmpty()) || !paths.isEmpty(), "At least one of path or paths is required"); + if (!tail) { + Preconditions.checkArgument(startPosition != StartPosition.END, + "start_position: end is only valid when tail is true"); + } + } + + public boolean isLegacyConfig() { + return codec == null && !tail && paths.isEmpty() && excludePaths.isEmpty(); + } + + public int getEffectiveReaderThreads() { + return tail ? readerThreads : 1; } @AssertTrue(message = "The file source requires recordType to be event when using a codec.") diff --git a/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/DirectoryWatcherTest.java b/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/DirectoryWatcherTest.java index dbe648a55e..e98c857d3e 100644 --- a/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/DirectoryWatcherTest.java +++ b/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/DirectoryWatcherTest.java @@ -58,7 +58,7 @@ class DirectoryWatcherTest { Path tempDir; @Mock - private TailFileReaderPool readerPool; + private FileReaderPool readerPool; @Mock private CheckpointRegistry checkpointRegistry; diff --git a/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/TailFileReaderPoolTest.java b/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/FileReaderPoolTest.java similarity index 93% rename from data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/TailFileReaderPoolTest.java rename to data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/FileReaderPoolTest.java index 5dc7d40b9a..aaad9c0c2b 100644 --- a/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/TailFileReaderPoolTest.java +++ b/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/FileReaderPoolTest.java @@ -52,7 +52,7 @@ import static org.mockito.Mockito.when; @ExtendWith(MockitoExtension.class) -class TailFileReaderPoolTest { +class FileReaderPoolTest { @TempDir Path tempDir; @@ -78,32 +78,32 @@ class TailFileReaderPoolTest { @Mock private AcknowledgementSetManager acknowledgementSetManager; - private TailFileReaderContext createReaderContext() { - return new TailFileReaderContext( + private FileReaderContext createReaderContext() { + return new FileReaderContext( buffer, eventFactory, fileOps, metrics, rotationDetector, acknowledgementSetManager, false, StandardCharsets.UTF_8, 4096, 1048576, 5000, Duration.ofSeconds(5), Duration.ofSeconds(30), StartPosition.END, false, Duration.ofSeconds(30), 1000, - Duration.ofSeconds(5), 3, null); + Duration.ofSeconds(5), 3, null, true, null); } - private TailFileReaderPool createPool(final int maxActiveFiles, final int readerThreads) { + private FileReaderPool createPool(final int maxActiveFiles, final int readerThreads) { when(metrics.getActiveFileCount()).thenReturn(new AtomicLong(0)); - return new TailFileReaderPool( + return new FileReaderPool( checkpointRegistry, metrics, maxActiveFiles, readerThreads, Duration.ofMinutes(30), createReaderContext()); } - private TailFileReaderPool createPoolWithoutMetrics(final int maxActiveFiles, final int readerThreads) { - return new TailFileReaderPool( + private FileReaderPool createPoolWithoutMetrics(final int maxActiveFiles, final int readerThreads) { + return new FileReaderPool( checkpointRegistry, metrics, maxActiveFiles, readerThreads, Duration.ofMinutes(30), createReaderContext()); } @Test void addFile_submits_reader_when_under_max_active_files() { - TailFileReaderPool pool = createPool(10, 2); + FileReaderPool pool = createPool(10, 2); when(checkpointRegistry.getOrCreate(anyString())).thenReturn(new CheckpointEntry()); FileIdentity identity = mock(FileIdentity.class); @@ -118,7 +118,7 @@ void addFile_submits_reader_when_under_max_active_files() { @Test void addFile_is_idempotent_for_same_identity() { - TailFileReaderPool pool = createPool(10, 2); + FileReaderPool pool = createPool(10, 2); when(checkpointRegistry.getOrCreate(anyString())).thenReturn(new CheckpointEntry()); FileIdentity identity = mock(FileIdentity.class); @@ -133,7 +133,7 @@ void addFile_is_idempotent_for_same_identity() { @Test void addFile_queues_pending_when_at_max_active_files() { - TailFileReaderPool pool = createPool(1, 2); + FileReaderPool pool = createPool(1, 2); when(checkpointRegistry.getOrCreate(anyString())).thenReturn(new CheckpointEntry()); FileIdentity identity1 = mock(FileIdentity.class); @@ -149,7 +149,7 @@ void addFile_queues_pending_when_at_max_active_files() { @Test void addFile_does_not_add_pending_duplicate_to_queue() { - TailFileReaderPool pool = createPool(1, 2); + FileReaderPool pool = createPool(1, 2); when(checkpointRegistry.getOrCreate(anyString())).thenReturn(new CheckpointEntry()); FileIdentity identity1 = mock(FileIdentity.class); @@ -165,7 +165,7 @@ void addFile_does_not_add_pending_duplicate_to_queue() { @Test void addFile_queues_multiple_pending_files() { - TailFileReaderPool pool = createPool(1, 2); + FileReaderPool pool = createPool(1, 2); when(checkpointRegistry.getOrCreate(anyString())).thenReturn(new CheckpointEntry()); FileIdentity identity1 = mock(FileIdentity.class); @@ -183,19 +183,19 @@ void addFile_queues_multiple_pending_files() { @Test void shutdown_does_not_throw() { - TailFileReaderPool pool = createPoolWithoutMetrics(10, 1); + FileReaderPool pool = createPoolWithoutMetrics(10, 1); pool.shutdown(); } @Test void getActiveReaderCount_returns_zero_initially() { - TailFileReaderPool pool = createPoolWithoutMetrics(10, 2); + FileReaderPool pool = createPoolWithoutMetrics(10, 2); assertThat(pool.getActiveReaderCount(), equalTo(0)); } @Test void getPendingCount_returns_zero_initially() { - TailFileReaderPool pool = createPoolWithoutMetrics(10, 2); + FileReaderPool pool = createPoolWithoutMetrics(10, 2); assertThat(pool.getPendingCount(), equalTo(0)); } @@ -205,7 +205,7 @@ void closeInactiveReaders_removes_inactive_readers() throws Exception { lenient().when(metrics.getFilesClosed()).thenReturn(filesClosed); lenient().when(metrics.getActiveFileCount()).thenReturn(new AtomicLong(0)); - TailFileReaderPool pool = new TailFileReaderPool( + FileReaderPool pool = new FileReaderPool( checkpointRegistry, metrics, 10, 2, Duration.ofMillis(1), createReaderContext()); @@ -218,7 +218,7 @@ void closeInactiveReaders_removes_inactive_readers() throws Exception { void closeReaderForPath_removes_matching_reader() { Counter filesClosed = mock(Counter.class); lenient().when(metrics.getFilesClosed()).thenReturn(filesClosed); - TailFileReaderPool pool = createPool(10, 2); + FileReaderPool pool = createPool(10, 2); when(checkpointRegistry.getOrCreate(anyString())).thenReturn(new CheckpointEntry()); FileIdentity identity = mock(FileIdentity.class); @@ -234,7 +234,7 @@ void closeReaderForPath_removes_matching_reader() { @Test void closeReaderForPath_does_nothing_when_no_match() { - TailFileReaderPool pool = createPool(10, 2); + FileReaderPool pool = createPool(10, 2); when(checkpointRegistry.getOrCreate(anyString())).thenReturn(new CheckpointEntry()); FileIdentity identity = mock(FileIdentity.class); @@ -249,7 +249,7 @@ void closeReaderForPath_does_nothing_when_no_match() { void closeReaderForPath_promotes_pending_files() { Counter filesClosed = mock(Counter.class); lenient().when(metrics.getFilesClosed()).thenReturn(filesClosed); - TailFileReaderPool pool = createPool(1, 2); + FileReaderPool pool = createPool(1, 2); when(checkpointRegistry.getOrCreate(anyString())).thenReturn(new CheckpointEntry()); FileIdentity identity1 = mock(FileIdentity.class); @@ -271,7 +271,7 @@ void closeReaderForPath_promotes_pending_files() { @Test void shutdown_handles_interrupted_exception() throws Exception { - TailFileReaderPool pool = createPoolWithoutMetrics(10, 1); + FileReaderPool pool = createPoolWithoutMetrics(10, 1); Thread shutdownThread = new Thread(() -> { Thread.currentThread().interrupt(); @@ -324,7 +324,7 @@ void onReaderComplete_with_create_rename_resubmits_reader() throws Exception { lenient().when(mockBuilder.withData(any(Map.class))).thenReturn(mockBuilder); lenient().when(mockBuilder.build()).thenReturn(mockEvent); - TailFileReaderPool pool = createPool(10, 2); + FileReaderPool pool = createPool(10, 2); when(checkpointRegistry.getOrCreate(anyString())).thenReturn(new CheckpointEntry()); FileIdentity identity = FileIdentity.from(testFile, fileOps, 1024); @@ -374,7 +374,7 @@ void closeInactiveReaders_with_real_reader() throws Exception { lenient().when(mockBuilder.withData(any(Map.class))).thenReturn(mockBuilder); lenient().when(mockBuilder.build()).thenReturn(mockEvent); - TailFileReaderPool pool = new TailFileReaderPool( + FileReaderPool pool = new FileReaderPool( checkpointRegistry, metrics, 10, 2, Duration.ofMillis(1), createReaderContext()); @@ -395,7 +395,7 @@ void shutdown_calls_shutdownNow_when_awaitTermination_returns_false() throws Exc ExecutorService mockExecutor = mock(ExecutorService.class); when(mockExecutor.awaitTermination(anyLong(), any(TimeUnit.class))).thenReturn(false); - TailFileReaderPool pool = new TailFileReaderPool( + FileReaderPool pool = new FileReaderPool( checkpointRegistry, metrics, 10, Duration.ofMinutes(30), createReaderContext(), () -> mockExecutor); @@ -409,7 +409,7 @@ void shutdown_calls_shutdownNow_when_awaitTermination_throws_interrupted() throw when(mockExecutor.awaitTermination(anyLong(), any(TimeUnit.class))) .thenThrow(new InterruptedException("test")); - TailFileReaderPool pool = new TailFileReaderPool( + FileReaderPool pool = new FileReaderPool( checkpointRegistry, metrics, 10, Duration.ofMinutes(30), createReaderContext(), () -> mockExecutor); @@ -428,7 +428,7 @@ void addFile_handles_rejected_execution_exception() { when(metrics.getActiveFileCount()).thenReturn(new AtomicLong(0)); when(checkpointRegistry.getOrCreate(anyString())).thenReturn(new CheckpointEntry()); - TailFileReaderPool pool = new TailFileReaderPool( + FileReaderPool pool = new FileReaderPool( checkpointRegistry, metrics, 10, Duration.ofMinutes(30), createReaderContext(), () -> mockExecutor); @@ -482,7 +482,7 @@ void closeInactiveReaders_removes_reader_past_threshold() throws Exception { lenient().when(mockBuilder.withData(any(Map.class))).thenReturn(mockBuilder); lenient().when(mockBuilder.build()).thenReturn(mockEvent); - TailFileReaderPool pool = new TailFileReaderPool( + FileReaderPool pool = new FileReaderPool( checkpointRegistry, metrics, 10, 2, Duration.ofMillis(1), createReaderContext()); @@ -535,7 +535,7 @@ void closeInactiveReaders_keeps_active_readers() throws Exception { lenient().when(mockBuilder.withData(any(Map.class))).thenReturn(mockBuilder); lenient().when(mockBuilder.build()).thenReturn(mockEvent); - TailFileReaderPool pool = new TailFileReaderPool( + FileReaderPool pool = new FileReaderPool( checkpointRegistry, metrics, 10, 2, Duration.ofHours(1), createReaderContext()); @@ -591,7 +591,7 @@ void onReaderComplete_with_deleted_rotation_marks_completed_and_processes_pendin lenient().when(mockBuilder.withData(any(Map.class))).thenReturn(mockBuilder); lenient().when(mockBuilder.build()).thenReturn(mockEvent); - TailFileReaderPool pool = createPool(10, 2); + FileReaderPool pool = createPool(10, 2); when(checkpointRegistry.getOrCreate(anyString())).thenReturn(new CheckpointEntry()); FileIdentity identity = FileIdentity.from(testFile, fileOps, 1024); @@ -607,7 +607,7 @@ void onReaderComplete_with_deleted_rotation_marks_completed_and_processes_pendin FileChannel.open(pendingFile, StandardOpenOption.READ)); FileIdentity pendingIdentity = FileIdentity.from(pendingFile, fileOps, 1024); - TailFileReaderPool limitedPool = new TailFileReaderPool( + FileReaderPool limitedPool = new FileReaderPool( checkpointRegistry, metrics, 1, 2, Duration.ofMinutes(30), createReaderContext()); diff --git a/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/TailFileReaderTest.java b/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/FileReaderTest.java similarity index 89% rename from data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/TailFileReaderTest.java rename to data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/FileReaderTest.java index 7af32017d2..f38e62f07f 100644 --- a/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/TailFileReaderTest.java +++ b/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/FileReaderTest.java @@ -64,7 +64,7 @@ import static org.mockito.Mockito.when; @ExtendWith(MockitoExtension.class) -class TailFileReaderTest { +class FileReaderTest { @TempDir Path tempDir; @@ -97,42 +97,42 @@ void setUp() { onCompleteCalled = new AtomicBoolean(false); } - private TailFileReaderContext createContext(final StartPosition startPosition) { - return new TailFileReaderContext( + private FileReaderContext createContext(final StartPosition startPosition) { + return new FileReaderContext( buffer, eventFactory, fileOps, metrics, rotationDetector, acknowledgementSetManager, false, StandardCharsets.UTF_8, 4096, 1048576, 5000, Duration.ofSeconds(30), Duration.ofSeconds(30), startPosition, false, Duration.ofSeconds(30), 1000, - Duration.ofSeconds(5), 3, null); + Duration.ofSeconds(5), 3, null, true, null); } - private TailFileReaderContext createContext(final int readBufferSize, final int maxLineLength, + private FileReaderContext createContext(final int readBufferSize, final int maxLineLength, final boolean includeMetadata, final StartPosition startPosition) { - return new TailFileReaderContext( + return new FileReaderContext( buffer, eventFactory, fileOps, metrics, rotationDetector, acknowledgementSetManager, false, StandardCharsets.UTF_8, readBufferSize, maxLineLength, 5000, Duration.ofSeconds(30), Duration.ofSeconds(30), startPosition, includeMetadata, Duration.ofSeconds(30), 1000, - Duration.ofSeconds(5), 3, null); + Duration.ofSeconds(5), 3, null, true, null); } - private TailFileReader createReader(final Path path) { + private FileReader createReader(final Path path) { return createReader(path, 4096, 1048576, false, StartPosition.BEGINNING); } - private TailFileReader createReader(final Path path, final int readBufferSize, + private FileReader createReader(final Path path, final int readBufferSize, final int maxLineLength, final boolean includeMetadata) { return createReader(path, readBufferSize, maxLineLength, includeMetadata, StartPosition.BEGINNING); } - private TailFileReader createReader(final Path path, final int readBufferSize, + private FileReader createReader(final Path path, final int readBufferSize, final int maxLineLength, final boolean includeMetadata, final StartPosition startPosition) { fileIdentity = mock(FileIdentity.class); - final TailFileReaderContext context = createContext(readBufferSize, maxLineLength, includeMetadata, startPosition); - return new TailFileReader(path, fileIdentity, checkpointEntry, context, + final FileReaderContext context = createContext(readBufferSize, maxLineLength, includeMetadata, startPosition); + return new FileReader(path, fileIdentity, checkpointEntry, context, () -> onCompleteCalled.set(true)); } @@ -185,7 +185,7 @@ void run_reads_lines_from_file() throws Exception { stubReadMetrics(); stubEventFactory(); - final TailFileReader reader = createReader(testFile); + final FileReader reader = createReader(testFile); reader.run(); verify(buffer, times(3)).write(any(Record.class), eq(5000)); @@ -205,7 +205,7 @@ void run_tracks_read_offset() throws Exception { stubReadMetrics(); stubEventFactory(); - final TailFileReader reader = createReader(testFile); + final FileReader reader = createReader(testFile); reader.run(); assertThat(reader.getReadOffset(), equalTo((long) content.getBytes(StandardCharsets.UTF_8).length)); @@ -223,7 +223,7 @@ void run_handles_partial_line_without_trailing_newline() throws Exception { stubReadMetrics(); stubEventFactory(); - final TailFileReader reader = createReader(testFile); + final FileReader reader = createReader(testFile); reader.run(); verify(buffer, times(2)).write(any(Record.class), eq(5000)); @@ -243,7 +243,7 @@ void run_truncates_line_exceeding_max_line_length() throws Exception { when(metrics.getLinesTruncated()).thenReturn(linesTruncated); stubEventFactory(); - final TailFileReader reader = createReader(testFile, 4096, 50, false); + final FileReader reader = createReader(testFile, 4096, 50, false); reader.run(); verify(linesTruncated).increment(); @@ -262,7 +262,7 @@ void run_truncates_complete_line_exceeding_max_line_length() throws Exception { when(metrics.getLinesTruncated()).thenReturn(linesTruncated); stubEventFactory(); - final TailFileReader reader = createReader(testFile, 4096, 50, false); + final FileReader reader = createReader(testFile, 4096, 50, false); reader.run(); verify(linesTruncated).increment(); @@ -285,7 +285,7 @@ void run_retries_on_buffer_back_pressure() throws Exception { .doNothing() .when(buffer).write(any(Record.class), anyInt()); - final TailFileReader reader = createReader(testFile); + final FileReader reader = createReader(testFile); reader.run(); verify(writeTimeouts).increment(); @@ -300,7 +300,7 @@ void run_handles_deleted_file_via_rotation_detector() throws Exception { when(rotationDetector.checkRotation(any(), any(), any(long.class))) .thenReturn(RotationResult.DELETED); - final TailFileReader reader = createReader(testFile); + final FileReader reader = createReader(testFile); reader.run(); assertThat(reader.getLastRotationType(), equalTo(RotationType.DELETED)); @@ -319,7 +319,7 @@ void run_handles_no_such_file_exception_during_read() throws Exception { Counter filesClosed = mock(Counter.class); when(metrics.getFilesClosed()).thenReturn(filesClosed); - final TailFileReader reader = createReader(testFile); + final FileReader reader = createReader(testFile); reader.run(); assertThat(onCompleteCalled.get(), equalTo(true)); @@ -343,7 +343,7 @@ void run_handles_copytruncate_rotation() throws Exception { .thenReturn(new RotationResult(RotationType.COPYTRUNCATE, fileIdentity)); checkpointEntry.setReadOffset(500); - final TailFileReader reader = createReader(testFile); + final FileReader reader = createReader(testFile); reader.run(); verify(filesRotated).increment(); @@ -365,7 +365,7 @@ void run_handles_create_rename_rotation() throws Exception { when(rotationDetector.checkRotation(any(), any(), any(long.class))) .thenReturn(new RotationResult(RotationType.CREATE_RENAME, newIdentity)); - final TailFileReader reader = createReader(testFile); + final FileReader reader = createReader(testFile); reader.run(); verify(filesRotated).increment(); @@ -386,7 +386,7 @@ void run_resumes_from_checkpoint_offset() throws Exception { stubReadMetrics(); stubEventFactory(); - final TailFileReader reader = createReader(testFile); + final FileReader reader = createReader(testFile); reader.run(); verify(buffer, times(2)).write(any(Record.class), eq(5000)); @@ -405,7 +405,7 @@ void run_increments_read_errors_on_io_exception() throws Exception { when(metrics.getReadErrors()).thenReturn(readErrors); when(metrics.getFilesClosed()).thenReturn(filesClosed); - final TailFileReader reader = createReader(testFile); + final FileReader reader = createReader(testFile); reader.run(); verify(readErrors).increment(); @@ -430,7 +430,7 @@ void run_includes_file_metadata_when_enabled() throws Exception { when(mockBuilder.withData(dataCaptor.capture())).thenReturn(mockBuilder); when(mockBuilder.build()).thenReturn(mockEvent); - final TailFileReader reader = createReader(testFile, 4096, 1048576, true); + final FileReader reader = createReader(testFile, 4096, 1048576, true); reader.run(); Map capturedData = dataCaptor.getValue(); @@ -447,7 +447,7 @@ void getFileIdentity_returns_identity_passed_in_constructor() throws Exception { Path testFile = tempDir.resolve("identity.log"); Files.writeString(testFile, ""); - final TailFileReader reader = createReader(testFile); + final FileReader reader = createReader(testFile); assertThat(reader.getFileIdentity(), equalTo(fileIdentity)); } @@ -456,7 +456,7 @@ void getLastRotationType_defaults_to_no_rotation() throws Exception { Path testFile = tempDir.resolve("default.log"); Files.writeString(testFile, ""); - final TailFileReader reader = createReader(testFile); + final FileReader reader = createReader(testFile); assertThat(reader.getLastRotationType(), equalTo(RotationType.NO_ROTATION)); } @@ -467,7 +467,7 @@ void start_position_end_seeks_to_end_of_file_for_new_files() throws Exception { long fileSize = Files.size(testFile); when(fileOps.size(testFile)).thenReturn(fileSize); - final TailFileReader reader = createReader(testFile, 4096, 1048576, false, StartPosition.END); + final FileReader reader = createReader(testFile, 4096, 1048576, false, StartPosition.END); assertThat(reader.getReadOffset(), equalTo(fileSize)); } @@ -477,7 +477,7 @@ void start_position_beginning_starts_from_offset_zero_for_new_files() throws Exc Path testFile = tempDir.resolve("startbegin.log"); Files.writeString(testFile, "existing-line1\nexisting-line2\n"); - final TailFileReader reader = createReader(testFile, 4096, 1048576, false, StartPosition.BEGINNING); + final FileReader reader = createReader(testFile, 4096, 1048576, false, StartPosition.BEGINNING); assertThat(reader.getReadOffset(), equalTo(0L)); } @@ -488,7 +488,7 @@ void start_position_end_does_not_seek_when_checkpoint_exists() throws Exception Files.writeString(testFile, "existing-line1\nexisting-line2\n"); checkpointEntry.setReadOffset(10); - final TailFileReader reader = createReader(testFile, 4096, 1048576, false, StartPosition.END); + final FileReader reader = createReader(testFile, 4096, 1048576, false, StartPosition.END); assertThat(reader.getReadOffset(), equalTo(10L)); } @@ -508,35 +508,35 @@ void run_increments_events_emitted_counter() throws Exception { Counter eventsEmitted = mock(Counter.class); when(metrics.getEventsEmitted()).thenReturn(eventsEmitted); - final TailFileReader reader = createReader(testFile); + final FileReader reader = createReader(testFile); reader.run(); verify(eventsEmitted, times(2)).increment(); } - private TailFileReaderContext createContextWithCodec(final InputCodec codec) { - return new TailFileReaderContext( + private FileReaderContext createContextWithCodec(final InputCodec codec) { + return new FileReaderContext( buffer, eventFactory, fileOps, metrics, rotationDetector, acknowledgementSetManager, false, StandardCharsets.UTF_8, 4096, 1048576, 5000, Duration.ofSeconds(30), Duration.ofSeconds(30), StartPosition.BEGINNING, false, Duration.ofSeconds(30), 1000, - Duration.ofSeconds(5), 3, codec); + Duration.ofSeconds(5), 3, codec, true, null); } - private TailFileReaderContext createContextWithAcknowledgements(final int batchSize, final Duration batchTimeout, final int maxRetries) { - return new TailFileReaderContext( + private FileReaderContext createContextWithAcknowledgements(final int batchSize, final Duration batchTimeout, final int maxRetries) { + return new FileReaderContext( buffer, eventFactory, fileOps, metrics, rotationDetector, acknowledgementSetManager, true, StandardCharsets.UTF_8, 4096, 1048576, 5000, Duration.ofSeconds(30), Duration.ofSeconds(30), StartPosition.BEGINNING, false, Duration.ofSeconds(30), batchSize, - batchTimeout, maxRetries, null); + batchTimeout, maxRetries, null, true, null); } - private TailFileReader createReaderWithContext(final Path path, final TailFileReaderContext context) { + private FileReader createReaderWithContext(final Path path, final FileReaderContext context) { fileIdentity = mock(FileIdentity.class); - return new TailFileReader(path, fileIdentity, checkpointEntry, context, + return new FileReader(path, fileIdentity, checkpointEntry, context, () -> onCompleteCalled.set(true)); } @@ -546,7 +546,7 @@ void start_position_end_falls_back_to_zero_on_io_exception() throws Exception { Files.writeString(testFile, "existing data\n"); when(fileOps.size(testFile)).thenThrow(new IOException("disk error")); - final TailFileReader reader = createReader(testFile, 4096, 1048576, false, StartPosition.END); + final FileReader reader = createReader(testFile, 4096, 1048576, false, StartPosition.END); assertThat(reader.getReadOffset(), equalTo(0L)); } @@ -561,7 +561,7 @@ void run_increments_read_errors_on_runtime_exception() throws Exception { Counter readErrors = mock(Counter.class); when(metrics.getReadErrors()).thenReturn(readErrors); - final TailFileReader reader = createReader(testFile); + final FileReader reader = createReader(testFile); reader.run(); verify(readErrors).increment(); @@ -582,7 +582,7 @@ void run_drains_file_on_create_rename_and_handles_no_such_file() throws Exceptio when(metrics.getFilesRotated()).thenReturn(filesRotated); when(metrics.getFilesClosed()).thenReturn(filesClosed); - final TailFileReader reader = createReader(testFile); + final FileReader reader = createReader(testFile); reader.run(); verify(filesRotated).increment(); @@ -605,7 +605,7 @@ void run_drains_file_on_create_rename_and_handles_io_exception() throws Exceptio when(metrics.getFilesClosed()).thenReturn(filesClosed); when(metrics.getReadErrors()).thenReturn(readErrors); - final TailFileReader reader = createReader(testFile); + final FileReader reader = createReader(testFile); reader.run(); verify(readErrors).increment(); @@ -642,13 +642,13 @@ void run_drain_timeout_logs_data_loss_when_unread_data() throws Exception { lenient().when(metrics.getBytesRead()).thenReturn(bytesRead); when(metrics.getDataLossEvents()).thenReturn(dataLossEvents); - TailFileReaderContext context = new TailFileReaderContext( + FileReaderContext context = new FileReaderContext( buffer, eventFactory, fileOps, metrics, rotationDetector, acknowledgementSetManager, false, StandardCharsets.UTF_8, 4096, 1048576, 5000, Duration.ofSeconds(30), Duration.ofMillis(1), StartPosition.BEGINNING, false, Duration.ofSeconds(30), 1000, - Duration.ofSeconds(5), 3, null); + Duration.ofSeconds(5), 3, null, true, null); lenientStubEventFactory(); Counter linesRead = mock(Counter.class); @@ -657,7 +657,7 @@ void run_drain_timeout_logs_data_loss_when_unread_data() throws Exception { lenient().when(metrics.getEventsEmitted()).thenReturn(eventsEmitted); fileIdentity = mock(FileIdentity.class); - final TailFileReader reader = new TailFileReader(testFile, fileIdentity, checkpointEntry, context, + final FileReader reader = new FileReader(testFile, fileIdentity, checkpointEntry, context, () -> onCompleteCalled.set(true)); reader.run(); @@ -696,16 +696,16 @@ void run_max_read_time_reached_breaks_without_data_loss() throws Exception { lenient().when(metrics.getLinesRead()).thenReturn(linesRead); lenient().when(metrics.getEventsEmitted()).thenReturn(eventsEmitted); - TailFileReaderContext context = new TailFileReaderContext( + FileReaderContext context = new FileReaderContext( buffer, eventFactory, fileOps, metrics, rotationDetector, acknowledgementSetManager, false, StandardCharsets.UTF_8, 4096, 1048576, 5000, Duration.ofMillis(1), Duration.ofSeconds(30), StartPosition.BEGINNING, false, Duration.ofSeconds(30), 1000, - Duration.ofSeconds(5), 3, null); + Duration.ofSeconds(5), 3, null, true, null); fileIdentity = mock(FileIdentity.class); - final TailFileReader reader = new TailFileReader(testFile, fileIdentity, checkpointEntry, context, + final FileReader reader = new FileReader(testFile, fileIdentity, checkpointEntry, context, () -> onCompleteCalled.set(true)); reader.run(); @@ -743,8 +743,8 @@ void run_with_codec_parses_bytes_and_emits_records() throws Exception { return null; }).when(mockCodec).parse(any(), any()); - TailFileReaderContext context = createContextWithCodec(mockCodec); - final TailFileReader reader = createReaderWithContext(testFile, context); + FileReaderContext context = createContextWithCodec(mockCodec); + final FileReader reader = createReaderWithContext(testFile, context); reader.run(); verify(buffer, atLeastOnce()).write(any(Record.class), eq(5000)); @@ -773,8 +773,8 @@ void run_with_codec_handles_parse_io_exception() throws Exception { InputCodec mockCodec = mock(InputCodec.class); doThrow(new IOException("parse error")).when(mockCodec).parse(any(), any()); - TailFileReaderContext context = createContextWithCodec(mockCodec); - final TailFileReader reader = createReaderWithContext(testFile, context); + FileReaderContext context = createContextWithCodec(mockCodec); + final FileReader reader = createReaderWithContext(testFile, context); reader.run(); verify(readErrors).increment(); @@ -817,8 +817,8 @@ void run_codec_record_retries_on_backpressure_and_records_timer() throws Excepti return null; }).when(mockCodec).parse(any(), any()); - TailFileReaderContext context = createContextWithCodec(mockCodec); - final TailFileReader reader = createReaderWithContext(testFile, context); + FileReaderContext context = createContextWithCodec(mockCodec); + final FileReader reader = createReaderWithContext(testFile, context); reader.run(); verify(writeTimeouts).increment(); @@ -840,8 +840,8 @@ void run_with_acknowledgements_creates_ack_set_and_completes_on_batch_full() thr AcknowledgementSet ackSet = mock(AcknowledgementSet.class); when(acknowledgementSetManager.create(any(), any(Duration.class))).thenReturn(ackSet); - TailFileReaderContext context = createContextWithAcknowledgements(1, Duration.ofSeconds(5), 3); - final TailFileReader reader = createReaderWithContext(testFile, context); + FileReaderContext context = createContextWithAcknowledgements(1, Duration.ofSeconds(5), 3); + final FileReader reader = createReaderWithContext(testFile, context); reader.run(); verify(acknowledgementSetManager, atLeastOnce()).create(any(), any(Duration.class)); @@ -864,8 +864,8 @@ void run_with_acknowledgements_batch_timeout_triggers_complete() throws Exceptio AcknowledgementSet ackSet = mock(AcknowledgementSet.class); when(acknowledgementSetManager.create(any(), any(Duration.class))).thenReturn(ackSet); - TailFileReaderContext context = createContextWithAcknowledgements(10000, Duration.ofMillis(0), 3); - final TailFileReader reader = createReaderWithContext(testFile, context); + FileReaderContext context = createContextWithAcknowledgements(10000, Duration.ofMillis(0), 3); + final FileReader reader = createReaderWithContext(testFile, context); reader.run(); verify(ackSet, atLeastOnce()).complete(); @@ -888,8 +888,8 @@ void handleAcknowledgement_positive_resets_retry_and_updates_checkpoint() throws ArgumentCaptor handlerCaptor = ArgumentCaptor.forClass(Consumer.class); when(acknowledgementSetManager.create(handlerCaptor.capture(), any(Duration.class))).thenReturn(ackSet); - TailFileReaderContext context = createContextWithAcknowledgements(1000, Duration.ofSeconds(5), 3); - final TailFileReader reader = createReaderWithContext(testFile, context); + FileReaderContext context = createContextWithAcknowledgements(1000, Duration.ofSeconds(5), 3); + final FileReader reader = createReaderWithContext(testFile, context); reader.run(); Consumer handler = handlerCaptor.getValue(); @@ -918,8 +918,8 @@ void handleAcknowledgement_negative_retries_and_eventually_advances() throws Exc ArgumentCaptor handlerCaptor = ArgumentCaptor.forClass(Consumer.class); when(acknowledgementSetManager.create(handlerCaptor.capture(), any(Duration.class))).thenReturn(ackSet); - TailFileReaderContext context = createContextWithAcknowledgements(1000, Duration.ofSeconds(5), 2); - final TailFileReader reader = createReaderWithContext(testFile, context); + FileReaderContext context = createContextWithAcknowledgements(1000, Duration.ofSeconds(5), 2); + final FileReader reader = createReaderWithContext(testFile, context); reader.run(); Consumer handler = handlerCaptor.getValue(); @@ -968,14 +968,14 @@ void run_with_codec_acknowledgements_creates_ack_set() throws Exception { return null; }).when(mockCodec).parse(any(), any()); - TailFileReaderContext context = new TailFileReaderContext( + FileReaderContext context = new FileReaderContext( buffer, eventFactory, fileOps, metrics, rotationDetector, acknowledgementSetManager, true, StandardCharsets.UTF_8, 4096, 1048576, 5000, Duration.ofSeconds(30), Duration.ofSeconds(30), StartPosition.BEGINNING, false, - Duration.ofSeconds(30), 1, Duration.ofSeconds(5), 3, mockCodec); + Duration.ofSeconds(30), 1, Duration.ofSeconds(5), 3, mockCodec, true, null); - final TailFileReader reader = createReaderWithContext(testFile, context); + final FileReader reader = createReaderWithContext(testFile, context); reader.run(); verify(ackSet, atLeastOnce()).add(any(Event.class)); @@ -994,7 +994,7 @@ void run_update_file_lag_handles_io_exception() throws Exception { stubEventFactory(); when(fileOps.size(testFile)).thenThrow(new IOException("disk error")); - final TailFileReader reader = createReader(testFile); + final FileReader reader = createReader(testFile); reader.run(); assertThat(onCompleteCalled.get(), equalTo(true)); @@ -1005,7 +1005,7 @@ void getLastActivityMillis_returns_initial_value() throws Exception { Path testFile = tempDir.resolve("activity.log"); Files.writeString(testFile, ""); long before = System.currentTimeMillis(); - final TailFileReader reader = createReader(testFile); + final FileReader reader = createReader(testFile); long after = System.currentTimeMillis(); assertThat(reader.getLastActivityMillis() >= before, equalTo(true)); @@ -1033,7 +1033,7 @@ void run_emitLine_backpressure_records_timer_after_recovery() throws Exception { .doNothing() .when(buffer).write(any(Record.class), anyInt()); - final TailFileReader reader = createReader(testFile); + final FileReader reader = createReader(testFile); reader.run(); verify(backpressureTimer).record(anyLong(), any(TimeUnit.class)); @@ -1069,10 +1069,10 @@ void run_codec_backpressure_interrupt_stops_reader() throws Exception { return null; }).when(mockCodec).parse(any(), any()); - TailFileReaderContext context = createContextWithCodec(mockCodec); + FileReaderContext context = createContextWithCodec(mockCodec); Thread readerThread = new Thread(() -> { - final TailFileReader reader = createReaderWithContext(testFile, context); + final FileReader reader = createReaderWithContext(testFile, context); reader.run(); }); readerThread.start(); @@ -1113,7 +1113,7 @@ void run_emitLine_backpressure_interrupt_stops_reader() throws Exception { .when(buffer).write(any(Record.class), anyInt()); Thread readerThread = new Thread(() -> { - final TailFileReader reader = createReader(testFile); + final FileReader reader = createReader(testFile); reader.run(); }); readerThread.start(); @@ -1155,13 +1155,13 @@ void run_drain_timeout_handles_io_exception_on_channel_size() throws Exception { when(metrics.getBytesRead()).thenReturn(bytesRead); lenient().when(metrics.getDataLossEvents()).thenReturn(dataLossEvents); - TailFileReaderContext context = new TailFileReaderContext( + FileReaderContext context = new FileReaderContext( buffer, eventFactory, fileOps, metrics, rotationDetector, acknowledgementSetManager, false, StandardCharsets.UTF_8, 4096, 1048576, 5000, Duration.ofSeconds(30), Duration.ofMillis(1), StartPosition.BEGINNING, false, Duration.ofSeconds(30), 1000, - Duration.ofSeconds(5), 3, null); + Duration.ofSeconds(5), 3, null, true, null); stubEventFactory(); Counter linesRead = mock(Counter.class); @@ -1170,7 +1170,7 @@ void run_drain_timeout_handles_io_exception_on_channel_size() throws Exception { lenient().when(metrics.getEventsEmitted()).thenReturn(eventsEmitted); fileIdentity = mock(FileIdentity.class); - final TailFileReader reader = new TailFileReader(testFile, fileIdentity, checkpointEntry, context, + final FileReader reader = new FileReader(testFile, fileIdentity, checkpointEntry, context, () -> onCompleteCalled.set(true)); reader.run(); } @@ -1190,7 +1190,7 @@ void run_updateFileLagBytes_handles_io_exception_on_file_size() throws Exception stubEventFactory(); stubReadMetrics(); - final TailFileReader reader = createReader(testFile); + final FileReader reader = createReader(testFile); reader.run(); assertThat(onCompleteCalled.get(), equalTo(true)); @@ -1211,8 +1211,8 @@ void run_with_acks_enabled_batch_not_full_does_not_complete_ack_set() throws Exc AcknowledgementSet ackSet = mock(AcknowledgementSet.class); when(acknowledgementSetManager.create(any(), any(Duration.class))).thenReturn(ackSet); - TailFileReaderContext context = createContextWithAcknowledgements(10000, Duration.ofHours(1), 3); - final TailFileReader reader = createReaderWithContext(testFile, context); + FileReaderContext context = createContextWithAcknowledgements(10000, Duration.ofHours(1), 3); + final FileReader reader = createReaderWithContext(testFile, context); reader.run(); verify(ackSet, atLeastOnce()).add(any(Event.class)); @@ -1231,7 +1231,7 @@ void run_without_acks_does_not_create_ack_set() throws Exception { stubEventFactory(); when(fileOps.size(testFile)).thenReturn(Files.size(testFile)); - final TailFileReader reader = createReader(testFile); + final FileReader reader = createReader(testFile); reader.run(); verify(acknowledgementSetManager, never()).create(any(), any(Duration.class)); @@ -1254,8 +1254,8 @@ void run_with_acks_enabled_zero_batch_count_does_not_trigger_batch_timeout() thr lenient().when(metrics.getBytesRead()).thenReturn(bytesRead); lenient().when(metrics.getFileLagBytes()).thenReturn(new AtomicLong(0)); - TailFileReaderContext context = createContextWithAcknowledgements(1000, Duration.ofMillis(0), 3); - final TailFileReader reader = createReaderWithContext(testFile, context); + FileReaderContext context = createContextWithAcknowledgements(1000, Duration.ofMillis(0), 3); + final FileReader reader = createReaderWithContext(testFile, context); reader.run(); verify(acknowledgementSetManager, never()).create(any(), any(Duration.class)); @@ -1292,8 +1292,8 @@ void run_with_codec_acks_disabled_does_not_create_ack_set() throws Exception { return null; }).when(mockCodec).parse(any(), any()); - TailFileReaderContext context = createContextWithCodec(mockCodec); - final TailFileReader reader = createReaderWithContext(testFile, context); + FileReaderContext context = createContextWithCodec(mockCodec); + final FileReader reader = createReaderWithContext(testFile, context); reader.run(); verify(acknowledgementSetManager, never()).create(any(), any(Duration.class)); @@ -1323,7 +1323,7 @@ void run_emitLine_thread_interrupted_during_backpressure_exits() throws Exceptio .when(buffer).write(any(Record.class), anyInt()); Thread readerThread = new Thread(() -> { - final TailFileReader reader = createReader(testFile); + final FileReader reader = createReader(testFile); reader.run(); }); readerThread.start(); @@ -1364,10 +1364,10 @@ void run_codec_thread_interrupted_during_backpressure_exits() throws Exception { return null; }).when(mockCodec).parse(any(), any()); - TailFileReaderContext context = createContextWithCodec(mockCodec); + FileReaderContext context = createContextWithCodec(mockCodec); Thread readerThread = new Thread(() -> { - final TailFileReader reader = createReaderWithContext(testFile, context); + final FileReader reader = createReaderWithContext(testFile, context); reader.run(); }); readerThread.start(); @@ -1390,7 +1390,7 @@ void run_multi_byte_character_split_across_reads_triggers_decoder_carryover() th stubReadMetrics(); stubEventFactory(); - final TailFileReader reader = createReader(testFile, 3, 1048576, false); + final FileReader reader = createReader(testFile, 3, 1048576, false); reader.run(); verify(buffer, atLeastOnce()).write(any(Record.class), eq(5000)); @@ -1410,7 +1410,7 @@ void run_skippingToNewline_skips_remainder_after_max_line_truncation() throws Ex when(metrics.getLinesTruncated()).thenReturn(linesTruncated); stubEventFactory(); - final TailFileReader reader = createReader(testFile, 20, 10, false); + final FileReader reader = createReader(testFile, 20, 10, false); reader.run(); verify(linesTruncated, atLeastOnce()).increment(); @@ -1442,16 +1442,16 @@ void run_backpressure_retry_timeout_exceeded_logs_data_loss() throws Exception { doThrow(new TimeoutException("buffer full")) .when(buffer).write(any(Record.class), anyInt()); - TailFileReaderContext context = new TailFileReaderContext( + FileReaderContext context = new FileReaderContext( buffer, eventFactory, fileOps, metrics, rotationDetector, acknowledgementSetManager, false, StandardCharsets.UTF_8, 4096, 1048576, 5000, Duration.ofMillis(100), Duration.ofSeconds(30), StartPosition.BEGINNING, false, Duration.ofMillis(100), 1000, - Duration.ofSeconds(5), 3, null); + Duration.ofSeconds(5), 3, null, true, null); fileIdentity = mock(FileIdentity.class); - final TailFileReader reader = new TailFileReader(testFile, fileIdentity, checkpointEntry, context, + final FileReader reader = new FileReader(testFile, fileIdentity, checkpointEntry, context, () -> onCompleteCalled.set(true)); reader.run(); diff --git a/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/FileSourceTests.java b/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/FileSourceTests.java index 0e78b19334..81c53a52cf 100644 --- a/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/FileSourceTests.java +++ b/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/FileSourceTests.java @@ -37,7 +37,6 @@ import org.opensearch.dataprepper.plugins.buffer.blockingbuffer.BlockingBufferConfig; import org.opensearch.dataprepper.plugins.codec.CompressionOption; -import java.io.FileInputStream; import java.io.IOException; import java.io.InputStream; import java.util.ArrayList; @@ -311,33 +310,25 @@ class WithCodec { @Mock private Buffer buffer; - @Mock - private DecompressionEngine decompressionEngine; - @BeforeEach void setUp() { + pluginMetrics = PluginMetrics.fromNames("file", "test-codec-pipeline"); + Map codecConfiguration = Map.of(UUID.randomUUID().toString(), UUID.randomUUID().toString()); Map> codecSettings = Map.of("fake_codec", codecConfiguration); pluginSettings.put("codec", codecSettings); + pluginSettings.put(FileSourceConfig.ATTRIBUTE_TYPE, FileSourceConfig.EVENT_TYPE); when(pluginFactory.loadPlugin(eq(InputCodec.class), any(PluginSetting.class))) .thenReturn(inputCodec); } @Test - void start_will_parse_codec_with_correct_inputStream() throws IOException { - final FileInputStream decompressedStream = new FileInputStream(TEST_FILE_PATH_PLAIN); - DecompressionEngine mockEngine = mock(DecompressionEngine.class); - when(mockEngine.createInputStream(any(InputStream.class))).thenReturn(decompressedStream); - - PluginModel fakeCodec = mock(PluginModel.class); - when(fakeCodec.getPluginName()).thenReturn("fake_codec"); - when(fakeCodec.getPluginSettings()).thenReturn(Map.of()); - - createObjectUnderTest(fakeCodec, mockEngine).start(buffer); + void start_will_parse_codec_with_inputStream() throws IOException { + createObjectUnderTest().start(buffer); - await().atMost(2, TimeUnit.SECONDS) - .untilAsserted(() -> verify(inputCodec).parse(eq(decompressedStream), any(Consumer.class))); + await().atMost(5, TimeUnit.SECONDS) + .untilAsserted(() -> verify(inputCodec).parse(any(InputStream.class), any(Consumer.class))); } @Test @@ -346,7 +337,7 @@ void start_will_parse_codec_with_a_Consumer_that_writes_to_the_buffer() throws I final ArgumentCaptor consumerArgumentCaptor = ArgumentCaptor.forClass(Consumer.class); - await().atMost(2, TimeUnit.SECONDS) + await().atMost(5, TimeUnit.SECONDS) .untilAsserted(() -> verify(inputCodec).parse(any(InputStream.class), any(Consumer.class))); verify(inputCodec).parse(any(InputStream.class), consumerArgumentCaptor.capture()); @@ -360,17 +351,15 @@ void start_will_parse_codec_with_a_Consumer_that_writes_to_the_buffer() throws I } @Test - void start_will_throw_exception_if_codec_throws() throws IOException, TimeoutException { - - final IOException mockedException = mock(IOException.class); - doThrow(mockedException) + void start_will_not_crash_if_codec_throws() throws IOException { + doThrow(new IOException("parse failed")) .when(inputCodec).parse(any(InputStream.class), any(Consumer.class)); FileSource objectUnderTest = createObjectUnderTest(); - objectUnderTest.start(buffer); - verify(buffer, after(1500).never()).write(any(Record.class), anyInt()); + await().atMost(5, TimeUnit.SECONDS) + .untilAsserted(() -> verify(inputCodec).parse(any(InputStream.class), any(Consumer.class))); } @Test @@ -382,7 +371,7 @@ void start_codec_consumer_wraps_timeout_exception() throws IOException, TimeoutE final ArgumentCaptor consumerArgumentCaptor = ArgumentCaptor.forClass(Consumer.class); - await().atMost(2, TimeUnit.SECONDS) + await().atMost(5, TimeUnit.SECONDS) .untilAsserted(() -> verify(inputCodec).parse(any(InputStream.class), any(Consumer.class))); verify(inputCodec).parse(any(InputStream.class), consumerArgumentCaptor.capture()); @@ -392,15 +381,6 @@ void start_codec_consumer_wraps_timeout_exception() throws IOException, TimeoutE assertThrows(RuntimeException.class, () -> actualConsumer.accept(record)); } - - @Test - void stop_before_start_prevents_codec_processing() throws IOException { - final FileSource objectUnderTest = createObjectUnderTest(); - objectUnderTest.stop(); - objectUnderTest.start(buffer); - - verify(inputCodec, after(500).never()).parse(any(InputStream.class), any(Consumer.class)); - } } @Nested From bf83414141466e9dbe057e9558aa2f958b86e1da Mon Sep 17 00:00:00 2001 From: Srikanth Padakanti Date: Fri, 22 May 2026 14:51:28 -0500 Subject: [PATCH 15/21] Fix thread name, remove redundant null checks, remove stray blank line Signed-off-by: Srikanth Padakanti --- .../dataprepper/plugins/source/file/DirectoryWatcher.java | 2 +- .../dataprepper/plugins/source/file/FileReaderPool.java | 4 ++-- .../dataprepper/plugins/source/file/FileSource.java | 1 - 3 files changed, 3 insertions(+), 4 deletions(-) diff --git a/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/DirectoryWatcher.java b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/DirectoryWatcher.java index 140480658b..7b61d67d56 100644 --- a/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/DirectoryWatcher.java +++ b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/DirectoryWatcher.java @@ -238,7 +238,7 @@ private void startWatchService() { } } - watchThread = new Thread(this::watchLoop, "file-tail-watcher"); + watchThread = new Thread(this::watchLoop, "file-watcher"); watchThread.setDaemon(true); watchThread.start(); LOG.info("WatchService started for {} directories", watchDirs.size()); diff --git a/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileReaderPool.java b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileReaderPool.java index ded792fa81..6ab8d9855c 100644 --- a/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileReaderPool.java +++ b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileReaderPool.java @@ -119,12 +119,12 @@ private synchronized void onReaderComplete(final FileIdentity fileIdentity, fina } metrics.getActiveFileCount().decrementAndGet(); - if (completedReader != null && completedReader.getLastRotationType() == RotationType.CREATE_RENAME) { + if (completedReader.getLastRotationType() == RotationType.CREATE_RENAME) { LOG.info("Re-adding path {} after create/rename rotation", path); final FileIdentity newIdentity = FileIdentity.from(path, readerContext.getFileOps(), readerContext.getRotationDetector().getFingerprintBytes()); submitReader(newIdentity, path); - } else if (completedReader != null && completedReader.getLastRotationType() != RotationType.DELETED) { + } else if (completedReader.getLastRotationType() != RotationType.DELETED) { final PendingFile next = pendingQueue.poll(); if (next != null) { pendingIdentities.remove(next.getFileIdentity()); diff --git a/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileSource.java b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileSource.java index 2f1545fd13..65826bee85 100644 --- a/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileSource.java +++ b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileSource.java @@ -98,7 +98,6 @@ public FileSource( } } - @Override public void start(final Buffer> buffer) { Objects.requireNonNull(buffer, "Buffer cannot be null for file source to start"); From 5ea1ab35055886df13c2396f334b46f687cf9a1b Mon Sep 17 00:00:00 2001 From: Srikanth Padakanti Date: Thu, 28 May 2026 14:41:47 -0500 Subject: [PATCH 16/21] Use Apache Ant DirectoryScanner for glob path resolution Signed-off-by: Srikanth Padakanti --- data-prepper-plugins/file-source/build.gradle | 1 + .../plugins/source/file/GlobPathResolver.java | 124 ++++++++---------- .../source/file/GlobPathResolverTest.java | 116 +++++++++------- 3 files changed, 124 insertions(+), 117 deletions(-) diff --git a/data-prepper-plugins/file-source/build.gradle b/data-prepper-plugins/file-source/build.gradle index 23d1d154ad..f2cc78ef57 100644 --- a/data-prepper-plugins/file-source/build.gradle +++ b/data-prepper-plugins/file-source/build.gradle @@ -13,6 +13,7 @@ dependencies { implementation project(':data-prepper-plugins:common') implementation 'com.fasterxml.jackson.core:jackson-databind' implementation 'io.micrometer:micrometer-core' + implementation 'org.apache.ant:ant:1.10.15' testImplementation project(':data-prepper-plugins:blocking-buffer') testImplementation project(':data-prepper-test:test-event') testImplementation 'org.awaitility:awaitility' diff --git a/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/GlobPathResolver.java b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/GlobPathResolver.java index a6bff96d6f..0c8d8a4978 100644 --- a/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/GlobPathResolver.java +++ b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/GlobPathResolver.java @@ -10,26 +10,22 @@ package org.opensearch.dataprepper.plugins.source.file; +import org.apache.tools.ant.DirectoryScanner; +import org.apache.tools.ant.types.selectors.SelectorUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; -import java.nio.file.FileSystems; -import java.nio.file.FileVisitResult; import java.nio.file.Files; -import java.nio.file.InvalidPathException; import java.nio.file.Path; -import java.nio.file.PathMatcher; import java.nio.file.Paths; -import java.nio.file.SimpleFileVisitor; -import java.nio.file.attribute.BasicFileAttributes; import java.util.ArrayList; import java.util.Collections; import java.util.HashSet; +import java.util.LinkedHashMap; import java.util.List; +import java.util.Map; import java.util.Objects; import java.util.Set; -import java.util.regex.PatternSyntaxException; public final class GlobPathResolver { @@ -38,91 +34,66 @@ public final class GlobPathResolver { private final List includePatterns; private final List excludePatterns; - private final List includeMatchers; - private final List excludeMatchers; public GlobPathResolver(final List includePatterns, final List excludePatterns) { - this.includePatterns = Objects.requireNonNull(includePatterns, "includePatterns must not be null"); - this.excludePatterns = excludePatterns != null ? excludePatterns : Collections.emptyList(); - this.includeMatchers = buildMatchers(this.includePatterns); - this.excludeMatchers = buildMatchers(this.excludePatterns); + Objects.requireNonNull(includePatterns, "includePatterns must not be null"); + this.includePatterns = List.copyOf(includePatterns); + this.excludePatterns = excludePatterns != null + ? List.copyOf(excludePatterns) + : Collections.emptyList(); } public Set resolve() { final Set result = new HashSet<>(); - final Set baseDirectories = getWatchDirectories(); + final Map> includesByBase = groupByBaseDirectory(includePatterns); - for (final Path baseDir : baseDirectories) { + for (final Map.Entry> entry : includesByBase.entrySet()) { + final Path baseDir = entry.getKey(); if (!Files.isDirectory(baseDir)) { LOG.warn("Base directory does not exist or is not a directory: {}", baseDir); continue; } - walkDirectory(baseDir, result); - } - - return result; - } - - void walkDirectory(final Path baseDir, final Set result) { - walkDirectory(baseDir, createFileVisitor(result)); - } - - void walkDirectory(final Path baseDir, final SimpleFileVisitor visitor) { - try { - Files.walkFileTree(baseDir, visitor); - } catch (final IOException e) { - LOG.warn("Failed to walk directory tree at: {}", baseDir, e); - } - } - - SimpleFileVisitor createFileVisitor(final Set result) { - return new SimpleFileVisitor<>() { - @Override - public FileVisitResult visitFile(final Path file, final BasicFileAttributes attrs) { - final Path normalized = file.toAbsolutePath().normalize(); - if (matches(normalized)) { - result.add(normalized); - } - return FileVisitResult.CONTINUE; + final DirectoryScanner scanner = new DirectoryScanner(); + scanner.setBasedir(baseDir.toFile()); + scanner.setIncludes(entry.getValue().toArray(new String[0])); + final List relativeExcludes = relativizeExcludes(baseDir, excludePatterns); + if (!relativeExcludes.isEmpty()) { + scanner.setExcludes(relativeExcludes.toArray(new String[0])); } - - @Override - public FileVisitResult visitFileFailed(final Path file, final IOException exc) { - LOG.warn("Failed to access file during glob resolution: {}", file, exc); - return FileVisitResult.CONTINUE; + scanner.addDefaultExcludes(); + scanner.setErrorOnMissingDir(false); + scanner.scan(); + for (final String included : scanner.getIncludedFiles()) { + result.add(baseDir.resolve(included).toAbsolutePath().normalize()); } - }; + } + return result; } public boolean matches(final Path path) { - final Path normalized = path.toAbsolutePath().normalize(); - + final String normalized = path.toAbsolutePath().normalize().toString(); boolean included = false; - for (final PathMatcher matcher : includeMatchers) { - if (matcher.matches(normalized)) { + for (final String pattern : includePatterns) { + if (SelectorUtils.matchPath(toAbsolutePattern(pattern), normalized)) { included = true; break; } } - if (!included) { return false; } - - for (final PathMatcher matcher : excludeMatchers) { - if (matcher.matches(normalized)) { + for (final String pattern : excludePatterns) { + if (SelectorUtils.matchPath(toAbsolutePattern(pattern), normalized)) { return false; } } - return true; } public Set getWatchDirectories() { final Set directories = new HashSet<>(); for (final String pattern : includePatterns) { - final Path baseDir = extractBaseDirectory(pattern); - directories.add(baseDir); + directories.add(extractBaseDirectory(pattern)); } return directories; } @@ -149,22 +120,33 @@ static Path extractBaseDirectory(final String pattern) { if (Files.isDirectory(result)) { return result; } - final Path parent = result.getParent(); return Objects.requireNonNullElse(parent, result); } - private static List buildMatchers(final List patterns) { - final List matchers = new ArrayList<>(patterns.size()); + private static Map> groupByBaseDirectory(final List patterns) { + final Map> grouped = new LinkedHashMap<>(); for (final String pattern : patterns) { - try { - final String absolutePattern = Paths.get(pattern).toAbsolutePath().normalize().toString(); - matchers.add(FileSystems.getDefault().getPathMatcher("glob:" + absolutePattern)); - } catch (final PatternSyntaxException | InvalidPathException e) { - LOG.error("Invalid glob pattern '{}': {}", pattern, e.getMessage()); - throw new IllegalArgumentException("Invalid glob pattern: " + pattern, e); + final Path baseDir = extractBaseDirectory(pattern); + final Path absolute = Paths.get(pattern).toAbsolutePath().normalize(); + grouped.computeIfAbsent(baseDir, k -> new ArrayList<>()) + .add(baseDir.relativize(absolute).toString()); + } + return grouped; + } + + private static List relativizeExcludes(final Path baseDir, final List excludes) { + final List result = new ArrayList<>(excludes.size()); + for (final String exclude : excludes) { + final Path absolute = Paths.get(exclude).toAbsolutePath().normalize(); + if (absolute.startsWith(baseDir)) { + result.add(baseDir.relativize(absolute).toString()); } } - return matchers; + return result; + } + + private static String toAbsolutePattern(final String pattern) { + return Paths.get(pattern).toAbsolutePath().normalize().toString(); } -} +} \ No newline at end of file diff --git a/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/GlobPathResolverTest.java b/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/GlobPathResolverTest.java index 7bdd1d2b22..f7eeb324fd 100644 --- a/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/GlobPathResolverTest.java +++ b/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/GlobPathResolverTest.java @@ -17,9 +17,6 @@ import java.io.IOException; import java.nio.file.Files; import java.nio.file.Path; -import java.nio.file.FileVisitResult; -import java.nio.file.SimpleFileVisitor; -import java.nio.file.attribute.BasicFileAttributes; import java.util.Collections; import java.util.List; import java.util.Set; @@ -84,6 +81,22 @@ void resolve_matches_recursive_double_star_pattern() { assertThat(result, hasItem(tempDir.resolve("subdir/nested.log").toAbsolutePath().normalize())); } + @Test + void resolve_matches_question_mark_pattern() throws IOException { + Files.createFile(tempDir.resolve("a.log")); + Files.createFile(tempDir.resolve("b.log")); + + final GlobPathResolver resolver = new GlobPathResolver( + List.of(tempDir.toString() + "/?.log"), + Collections.emptyList()); + + final Set result = resolver.resolve(); + + assertThat(result, hasItem(tempDir.resolve("a.log").toAbsolutePath().normalize())); + assertThat(result, hasItem(tempDir.resolve("b.log").toAbsolutePath().normalize())); + assertThat(result, not(hasItem(tempDir.resolve("app.log").toAbsolutePath().normalize()))); + } + @Test void resolve_returns_empty_set_when_no_files_match() { final GlobPathResolver resolver = new GlobPathResolver( @@ -95,6 +108,20 @@ void resolve_returns_empty_set_when_no_files_match() { assertThat(result, empty()); } + @Test + void resolve_excludes_default_vcs_directories() throws IOException { + Path gitDir = Files.createDirectory(tempDir.resolve(".git")); + Files.createFile(gitDir.resolve("HEAD.log")); + + final GlobPathResolver resolver = new GlobPathResolver( + List.of(tempDir.toString() + "/" + "**/*.log"), + Collections.emptyList()); + + final Set result = resolver.resolve(); + + assertThat(result, not(hasItem(gitDir.resolve("HEAD.log").toAbsolutePath().normalize()))); + } + @Test void matches_returns_true_for_matching_path() { final GlobPathResolver resolver = new GlobPathResolver( @@ -122,6 +149,15 @@ void matches_returns_false_for_excluded_path() { assertThat(resolver.matches(tempDir.resolve("error.log")), equalTo(false)); } + @Test + void matches_returns_true_when_exclude_does_not_match_included_path() { + final GlobPathResolver resolver = new GlobPathResolver( + List.of(tempDir.toString() + "/*.log"), + List.of(tempDir.toString() + "/error.*")); + + assertThat(resolver.matches(tempDir.resolve("app.log")), equalTo(true)); + } + @Test void getWatchDirectories_returns_base_directories() { final GlobPathResolver resolver = new GlobPathResolver( @@ -137,17 +173,15 @@ void getWatchDirectories_returns_base_directories() { @Test void extractBaseDirectory_stops_at_first_wildcard() { final Path baseDir = GlobPathResolver.extractBaseDirectory(tempDir.toString() + "/logs/*.log"); - final Path expected = tempDir.resolve("logs").toAbsolutePath().normalize(); - final Path expectedParent = expected.getParent(); assertThat(baseDir, notNullValue()); assertThat(baseDir.toString().startsWith(tempDir.toAbsolutePath().normalize().toString()), equalTo(true)); } @Test - void constructor_throws_on_invalid_glob_pattern() { - assertThrows(IllegalArgumentException.class, () -> - new GlobPathResolver(List.of(tempDir.toString() + "/[invalid"), Collections.emptyList())); + void constructor_throws_on_null_include_patterns() { + assertThrows(NullPointerException.class, () -> + new GlobPathResolver(null, Collections.emptyList())); } @Test @@ -173,76 +207,66 @@ void resolve_handles_multiple_include_patterns() { } @Test - void getWatchDirectories_returns_multiple_directories_for_multiple_patterns() { + void resolve_handles_include_patterns_in_different_base_directories() throws IOException { + Path otherDir = Files.createDirectory(tempDir.resolve("other")); + Files.createFile(otherDir.resolve("audit.log")); + final GlobPathResolver resolver = new GlobPathResolver( - List.of(tempDir.toString() + "/*.log", tempDir.toString() + "/subdir/*.log"), + List.of(tempDir.toString() + "/*.log", otherDir.toString() + "/*.log"), Collections.emptyList()); - final Set watchDirs = resolver.getWatchDirectories(); + final Set result = resolver.resolve(); - assertThat(watchDirs.size(), greaterThanOrEqualTo(1)); + assertThat(result, hasItem(tempDir.resolve("app.log").toAbsolutePath().normalize())); + assertThat(result, hasItem(otherDir.resolve("audit.log").toAbsolutePath().normalize())); } @Test - void resolve_returns_empty_set_for_nonexistent_base_directory() { - Path nonexistent = tempDir.resolve("nonexistent-subdir"); + void resolve_ignores_excludes_outside_include_base_directory() throws IOException { + Path otherDir = Files.createDirectory(tempDir.resolve("other")); + final GlobPathResolver resolver = new GlobPathResolver( - List.of(nonexistent.toString() + "/*.log"), - Collections.emptyList()); + List.of(tempDir.toString() + "/*.log"), + List.of(otherDir.toString() + "/*.log")); final Set result = resolver.resolve(); - assertThat(result, empty()); + assertThat(result, hasSize(2)); } @Test - void resolve_warns_when_base_directory_does_not_exist() { - String nonexistentPath = "/nonexistent-dir-" + System.nanoTime() + "/sub/deep/*.log"; + void getWatchDirectories_returns_multiple_directories_for_multiple_patterns() { final GlobPathResolver resolver = new GlobPathResolver( - List.of(nonexistentPath), + List.of(tempDir.toString() + "/*.log", tempDir.toString() + "/subdir/*.log"), Collections.emptyList()); - final Set result = resolver.resolve(); + final Set watchDirs = resolver.getWatchDirectories(); - assertThat(result, empty()); + assertThat(watchDirs.size(), greaterThanOrEqualTo(1)); } @Test - void resolve_handles_visitFileFailed_gracefully() throws IOException { - Path unreadableDir = tempDir.resolve("unreadable"); - Files.createDirectory(unreadableDir); - Files.createFile(unreadableDir.resolve("secret.log")); - unreadableDir.toFile().setReadable(false); - + void resolve_returns_empty_set_for_nonexistent_base_directory() { + Path nonexistent = tempDir.resolve("nonexistent-subdir"); final GlobPathResolver resolver = new GlobPathResolver( - List.of(tempDir.toString() + "/" + "**/*.log"), + List.of(nonexistent.toString() + "/*.log"), Collections.emptyList()); final Set result = resolver.resolve(); - unreadableDir.toFile().setReadable(true); - - assertThat(result, not(hasItem(unreadableDir.resolve("secret.log").toAbsolutePath().normalize()))); + assertThat(result, empty()); } @Test - void walkDirectory_handles_ioException_from_walkFileTree() throws IOException { - Path dir = tempDir.resolve("walk-test"); - Files.createDirectory(dir); - Files.createFile(dir.resolve("file.log")); - + void resolve_warns_when_base_directory_does_not_exist() { + String nonexistentPath = "/nonexistent-dir-" + System.nanoTime() + "/sub/deep/*.log"; final GlobPathResolver resolver = new GlobPathResolver( - List.of(dir.toString() + "/*.log"), + List.of(nonexistentPath), Collections.emptyList()); - SimpleFileVisitor throwingVisitor = new SimpleFileVisitor<>() { - @Override - public FileVisitResult preVisitDirectory(Path dir, BasicFileAttributes attrs) throws IOException { - throw new IOException("simulated walk error"); - } - }; + final Set result = resolver.resolve(); - resolver.walkDirectory(dir, throwingVisitor); + assertThat(result, empty()); } @Test @@ -256,4 +280,4 @@ void extractBaseDirectory_with_nonexistent_path_returns_parent_as_fallback() { final Path baseDir = GlobPathResolver.extractBaseDirectory("/nonexistent-test-dir-" + System.nanoTime() + "/data.log"); assertThat(baseDir, notNullValue()); } -} +} \ No newline at end of file From 7b19c2850f55884b09c845b7d2b2a84c413b9a5e Mon Sep 17 00:00:00 2001 From: Srikanth Padakanti Date: Thu, 28 May 2026 15:21:05 -0500 Subject: [PATCH 17/21] Retrigger CI Signed-off-by: Srikanth Padakanti From 4ce8fbf42cfcb6889ee46ed8c13d6459ed582790 Mon Sep 17 00:00:00 2001 From: Srikanth Padakanti Date: Thu, 28 May 2026 16:14:49 -0500 Subject: [PATCH 18/21] Consolidate codec path: parseWithCodec accepts InputStream so one-shot creates an ack-set Signed-off-by: Srikanth Padakanti --- .../plugins/source/file/FileReader.java | 16 ++--- .../plugins/source/file/FileReaderTest.java | 62 +++++++++++++++++++ .../plugins/source/file/FileSourceTests.java | 20 ------ 3 files changed, 66 insertions(+), 32 deletions(-) diff --git a/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileReader.java b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileReader.java index 7b0aaf6cd8..edc047dc12 100644 --- a/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileReader.java +++ b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileReader.java @@ -197,15 +197,7 @@ private void readFileWithCodecOneShot() { try (final InputStream rawStream = Files.newInputStream(path); final InputStream decompressedStream = decompressionEngine.createInputStream(rawStream)) { metrics.getFilesOpened().increment(); - codec.parse(decompressedStream, record -> { - try { - buffer.write((Record) record, writeTimeout); - metrics.getEventsEmitted().increment(); - } catch (final TimeoutException e) { - metrics.getWriteTimeouts().increment(); - throw new RuntimeException(e); - } - }); + parseWithCodec(decompressedStream); } catch (final IOException e) { LOG.error("Error reading file with codec: {}", path, e); metrics.getReadErrors().increment(); @@ -319,7 +311,7 @@ private void readLoop(final FileChannel channel, final long timeoutMillis, final } if (codec != null && codecAccumulator.size() > 0) { - if (parseWithCodec(codecAccumulator.toByteArray())) { + if (parseWithCodec(new ByteArrayInputStream(codecAccumulator.toByteArray()))) { checkpointEntry.setReadOffset(readOffset.get()); } } else if (codec == null) { @@ -379,9 +371,9 @@ private void flushPartialLine() { } } - private boolean parseWithCodec(final byte[] bytes) { + private boolean parseWithCodec(final InputStream stream) { try { - codec.parse(new ByteArrayInputStream(bytes), record -> { + codec.parse(stream, record -> { emitCodecRecord(record); }); return true; diff --git a/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/FileReaderTest.java b/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/FileReaderTest.java index f38e62f07f..d0bf3fc1c6 100644 --- a/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/FileReaderTest.java +++ b/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/FileReaderTest.java @@ -540,6 +540,16 @@ private FileReader createReaderWithContext(final Path path, final FileReaderCont () -> onCompleteCalled.set(true)); } + private FileReaderContext createOneShotContextWithCodecAndAcknowledgements(final InputCodec codec) { + return new FileReaderContext( + buffer, eventFactory, fileOps, metrics, rotationDetector, + acknowledgementSetManager, true, StandardCharsets.UTF_8, + 4096, 1048576, 5000, Duration.ofSeconds(30), + Duration.ofSeconds(30), StartPosition.BEGINNING, false, + Duration.ofSeconds(30), 1, + Duration.ofSeconds(5), 3, codec, false, in -> in); + } + @Test void start_position_end_falls_back_to_zero_on_io_exception() throws Exception { Path testFile = tempDir.resolve("startend-error.log"); @@ -849,6 +859,58 @@ void run_with_acknowledgements_creates_ack_set_and_completes_on_batch_full() thr verify(ackSet, atLeastOnce()).complete(); } + @Test + void run_one_shot_codec_handles_io_exception() throws Exception { + Path missingFile = tempDir.resolve("does-not-exist.log"); + + Counter filesClosed = mock(Counter.class); + Counter readErrors = mock(Counter.class); + when(metrics.getFilesClosed()).thenReturn(filesClosed); + when(metrics.getReadErrors()).thenReturn(readErrors); + + InputCodec mockCodec = mock(InputCodec.class); + + FileReaderContext context = createOneShotContextWithCodecAndAcknowledgements(mockCodec); + final FileReader reader = createReaderWithContext(missingFile, context); + reader.run(); + + verify(readErrors).increment(); + } + + @Test + void run_one_shot_codec_with_acknowledgements_creates_ack_set() throws Exception { + Path testFile = tempDir.resolve("one-shot-codec-ack.log"); + Files.writeString(testFile, "line1\n"); + + Counter filesOpened = mock(Counter.class); + Counter filesClosed = mock(Counter.class); + Counter linesRead = mock(Counter.class); + Counter eventsEmitted = mock(Counter.class); + Timer backpressureTimer = mock(Timer.class); + when(metrics.getFilesOpened()).thenReturn(filesOpened); + when(metrics.getFilesClosed()).thenReturn(filesClosed); + when(metrics.getLinesRead()).thenReturn(linesRead); + when(metrics.getEventsEmitted()).thenReturn(eventsEmitted); + lenient().when(metrics.getBackpressureTimer()).thenReturn(backpressureTimer); + + InputCodec mockCodec = mock(InputCodec.class); + doAnswer(inv -> { + Consumer> consumer = inv.getArgument(1); + consumer.accept(new Record<>(mock(Event.class))); + return null; + }).when(mockCodec).parse(any(), any()); + + AcknowledgementSet ackSet = mock(AcknowledgementSet.class); + when(acknowledgementSetManager.create(any(), any(Duration.class))).thenReturn(ackSet); + + FileReaderContext context = createOneShotContextWithCodecAndAcknowledgements(mockCodec); + final FileReader reader = createReaderWithContext(testFile, context); + reader.run(); + + verify(acknowledgementSetManager, atLeastOnce()).create(any(), any(Duration.class)); + verify(ackSet, atLeastOnce()).add(any(Event.class)); + } + @Test void run_with_acknowledgements_batch_timeout_triggers_complete() throws Exception { Path testFile = tempDir.resolve("ack-timeout.log"); diff --git a/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/FileSourceTests.java b/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/FileSourceTests.java index 81c53a52cf..8a2671bbe3 100644 --- a/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/FileSourceTests.java +++ b/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/FileSourceTests.java @@ -361,26 +361,6 @@ void start_will_not_crash_if_codec_throws() throws IOException { await().atMost(5, TimeUnit.SECONDS) .untilAsserted(() -> verify(inputCodec).parse(any(InputStream.class), any(Consumer.class))); } - - @Test - void start_codec_consumer_wraps_timeout_exception() throws IOException, TimeoutException { - doThrow(new TimeoutException("buffer full")) - .when(buffer).write(any(Record.class), eq(FileSourceConfig.DEFAULT_TIMEOUT)); - - createObjectUnderTest().start(buffer); - - final ArgumentCaptor consumerArgumentCaptor = ArgumentCaptor.forClass(Consumer.class); - - await().atMost(5, TimeUnit.SECONDS) - .untilAsserted(() -> verify(inputCodec).parse(any(InputStream.class), any(Consumer.class))); - - verify(inputCodec).parse(any(InputStream.class), consumerArgumentCaptor.capture()); - - final Consumer> actualConsumer = consumerArgumentCaptor.getValue(); - final Record record = mock(Record.class); - - assertThrows(RuntimeException.class, () -> actualConsumer.accept(record)); - } } @Nested From 416b253544c711149a3e32174112f23c088ea6d1 Mon Sep 17 00:00:00 2001 From: Srikanth Padakanti Date: Thu, 28 May 2026 16:17:44 -0500 Subject: [PATCH 19/21] Honor user-set reader_threads when tail is false; default 1 when not tail and 4 when tail Signed-off-by: Srikanth Padakanti --- .../plugins/source/file/FileSource.java | 2 +- .../plugins/source/file/FileSourceConfig.java | 13 +++--- .../source/file/FileSourceConfigTest.java | 40 ++++++++++++++++++- 3 files changed, 47 insertions(+), 8 deletions(-) diff --git a/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileSource.java b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileSource.java index 65826bee85..1cb6d5a8da 100644 --- a/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileSource.java +++ b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileSource.java @@ -120,7 +120,7 @@ private void startModernPath(final Buffer> buffer) { LOG.info("Starting file source with paths: {}", fileSourceConfig.getAllPaths()); final int maxActiveFiles = fileSourceConfig.getMaxActiveFiles(); - final int readerThreads = fileSourceConfig.getEffectiveReaderThreads(); + final int readerThreads = fileSourceConfig.getReaderThreads(); if (readerThreads > 0 && maxActiveFiles / readerThreads > MAX_FILES_PER_THREAD_WARNING_THRESHOLD) { LOG.warn("max_active_files ({}) is {} times reader_threads ({}). Files with pending data may experience high latency.", maxActiveFiles, maxActiveFiles / readerThreads, readerThreads); diff --git a/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileSourceConfig.java b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileSourceConfig.java index f52d636f32..40c4954b23 100644 --- a/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileSourceConfig.java +++ b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileSourceConfig.java @@ -28,6 +28,8 @@ public class FileSourceConfig { static final int DEFAULT_TIMEOUT = 5_000; static final String DEFAULT_TYPE = "string"; static final String EVENT_TYPE = "event"; + static final int DEFAULT_READER_THREADS_TAIL = 4; + static final int DEFAULT_READER_THREADS_ONE_SHOT = 1; @JsonProperty(ATTRIBUTE_PATH) private String filePathToRead; @@ -66,7 +68,7 @@ public class FileSourceConfig { private int maxActiveFiles = 1000; @JsonProperty("reader_threads") - private int readerThreads = 4; + private Integer readerThreads; @JsonProperty("max_read_time_per_file") private Duration maxReadTimePerFile = Duration.ofSeconds(5); @@ -176,7 +178,10 @@ public int getMaxActiveFiles() { } public int getReaderThreads() { - return readerThreads; + if (readerThreads != null) { + return readerThreads; + } + return tail ? DEFAULT_READER_THREADS_TAIL : DEFAULT_READER_THREADS_ONE_SHOT; } public Duration getMaxReadTimePerFile() { @@ -261,10 +266,6 @@ public boolean isLegacyConfig() { return codec == null && !tail && paths.isEmpty() && excludePaths.isEmpty(); } - public int getEffectiveReaderThreads() { - return tail ? readerThreads : 1; - } - @AssertTrue(message = "The file source requires recordType to be event when using a codec.") boolean codeRequiresRecordTypeEvent() { return codec == null || recordType == RecordType.EVENT; diff --git a/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/FileSourceConfigTest.java b/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/FileSourceConfigTest.java index 649b191e93..fd9144aa14 100644 --- a/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/FileSourceConfigTest.java +++ b/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/FileSourceConfigTest.java @@ -170,7 +170,7 @@ void default_config_returns_expected_values() { assertThat(config.getEncoding(), equalTo("UTF-8")); assertThat(config.getReadBufferSize(), equalTo(65536)); assertThat(config.getMaxActiveFiles(), equalTo(1000)); - assertThat(config.getReaderThreads(), equalTo(4)); + assertThat(config.getReaderThreads(), equalTo(1)); assertThat(config.getMaxReadTimePerFile(), equalTo(Duration.ofSeconds(5))); assertThat(config.getRotateWait(), equalTo(Duration.ofSeconds(5))); assertThat(config.getRotationDrainTimeout(), equalTo(Duration.ofSeconds(30))); @@ -196,6 +196,44 @@ void exclude_paths_defaults_to_empty_list() { assertThat(config.getExcludePaths(), empty()); } + @Test + void reader_threads_defaults_to_four_when_tail_is_true() { + final Map configMap = Map.of("path", "/tmp/test.log", "tail", true); + final FileSourceConfig config = OBJECT_MAPPER.convertValue(configMap, FileSourceConfig.class); + + assertThat(config.getReaderThreads(), equalTo(4)); + } + + @Test + void reader_threads_defaults_to_one_when_tail_is_false() { + final Map configMap = Map.of("path", "/tmp/test.log", "tail", false); + final FileSourceConfig config = OBJECT_MAPPER.convertValue(configMap, FileSourceConfig.class); + + assertThat(config.getReaderThreads(), equalTo(1)); + } + + @Test + void reader_threads_honors_explicit_value_when_tail_is_false() { + final Map configMap = Map.of( + "path", "/tmp/test.log", + "tail", false, + "reader_threads", 8); + final FileSourceConfig config = OBJECT_MAPPER.convertValue(configMap, FileSourceConfig.class); + + assertThat(config.getReaderThreads(), equalTo(8)); + } + + @Test + void reader_threads_honors_explicit_value_when_tail_is_true() { + final Map configMap = Map.of( + "path", "/tmp/test.log", + "tail", true, + "reader_threads", 2); + final FileSourceConfig config = OBJECT_MAPPER.convertValue(configMap, FileSourceConfig.class); + + assertThat(config.getReaderThreads(), equalTo(2)); + } + @Test void exclude_paths_returns_configured_values() { final Map configMap = Map.of( From 96cd03d6f4cebdb5f19ce7cebc708217f2001316 Mon Sep 17 00:00:00 2001 From: Srikanth Padakanti Date: Fri, 5 Jun 2026 13:40:16 -0500 Subject: [PATCH 20/21] Read legacy path directly in ClassicFileStrategy ClassicFileStrategy runs only for the legacy single-path config (isLegacyConfig requires empty paths/exclude_paths, no codec, no tail), so GlobPathResolver added no value there and could glob a literal legacy path containing wildcard chars. Read the literal path directly, matching the original file source behavior. Glob resolution stays on the modern path. Addresses review feedback from @oeyh. Signed-off-by: Srikanth Padakanti --- .../plugins/source/file/FileSource.java | 14 +++----------- 1 file changed, 3 insertions(+), 11 deletions(-) diff --git a/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileSource.java b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileSource.java index 1cb6d5a8da..b7613438d3 100644 --- a/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileSource.java +++ b/data-prepper-plugins/file-source/src/main/java/org/opensearch/dataprepper/plugins/source/file/FileSource.java @@ -249,18 +249,10 @@ private interface FileStrategy { private class ClassicFileStrategy implements FileStrategy { @Override public void start(Buffer> buffer) { - final GlobPathResolver resolver = new GlobPathResolver( - fileSourceConfig.getAllPaths(), fileSourceConfig.getExcludePaths()); - final Set resolvedPaths = resolver.resolve(); - if (resolvedPaths.isEmpty() && fileSourceConfig.getFilePathToRead() != null) { - resolvedPaths.add(Paths.get(fileSourceConfig.getFilePathToRead()).toAbsolutePath().normalize()); - } - for (final Path filePath : resolvedPaths) { - if (isStopRequested) { - break; - } - readFile(filePath, buffer); + if (isStopRequested) { + return; } + readFile(Paths.get(fileSourceConfig.getFilePathToRead()), buffer); } private void readFile(final Path filePath, final Buffer> buffer) { From cd228ba3bd645fdcf61cdfb9b7bb268a9d1427bf Mon Sep 17 00:00:00 2001 From: Srikanth Padakanti Date: Fri, 5 Jun 2026 13:54:04 -0500 Subject: [PATCH 21/21] Test legacy path is read literally, not glob-expanded Regression test for ClassicFileStrategy: a wildcard in the legacy single 'path' must be treated as a literal filename (no glob expansion). Fails on the pre-fix globbing code, passes with the direct literal read. Addresses review feedback from @oeyh. Signed-off-by: Srikanth Padakanti --- .../dataprepper/plugins/source/file/FileSourceTests.java | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/FileSourceTests.java b/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/FileSourceTests.java index 8a2671bbe3..8d4791c965 100644 --- a/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/FileSourceTests.java +++ b/data-prepper-plugins/file-source/src/test/java/org/opensearch/dataprepper/plugins/source/file/FileSourceTests.java @@ -195,6 +195,15 @@ public void testFileSourceWithNonexistentFilePathDoesNotWriteToBuffer() throws T verify(buffer, after(500).never()).write(any(Record.class), anyInt()); } + @Test + public void testLegacyPathWithWildcardIsReadLiterallyAndNotGlobExpanded() throws TimeoutException { + buffer = mock(Buffer.class); + pluginSettings.put(FileSourceConfig.ATTRIBUTE_PATH, "src/test/resources/*.tst"); + fileSource = createObjectUnderTest(); + fileSource.start(buffer); + verify(buffer, after(500).never()).write(any(Record.class), anyInt()); + } + @Test public void testFileSourceWithNullFilePathThrowsNullPointerException() { pluginSettings.put(FileSourceConfig.ATTRIBUTE_PATH, null);