HADOOP-19245. S3ABlockOutputStream no longer sends progress events in close() (#6974)

Contributed by Steve Loughran
This commit is contained in:
Steve Loughran 2024-08-02 16:01:03 +01:00 committed by GitHub
parent c593c17255
commit 2cf4d638af
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
2 changed files with 10 additions and 1 deletions

View File

@ -1100,7 +1100,8 @@ private static class ProgressableListener implements ProgressListener {
this.progress = progress; this.progress = progress;
} }
public void progressChanged(ProgressListenerEvent eventType, int bytesTransferred) { @Override
public void progressChanged(ProgressListenerEvent eventType, long bytesTransferred) {
if (progress != null) { if (progress != null) {
progress.progress(); progress.progress();
} }

View File

@ -21,6 +21,7 @@
import java.io.IOException; import java.io.IOException;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collection; import java.util.Collection;
import java.util.concurrent.atomic.AtomicLong;
import org.assertj.core.api.Assertions; import org.assertj.core.api.Assertions;
import org.junit.Test; import org.junit.Test;
@ -213,8 +214,11 @@ public void testCreateFilePerformanceFlag() throws Throwable {
S3AFileSystem fs = getFileSystem(); S3AFileSystem fs = getFileSystem();
Path path = methodPath(); Path path = methodPath();
// increment progress events
AtomicLong progressEvents = new AtomicLong(0);
FSDataOutputStreamBuilder builder = fs.createFile(path) FSDataOutputStreamBuilder builder = fs.createFile(path)
.overwrite(false) .overwrite(false)
.progress(progressEvents::incrementAndGet)
.recursive(); .recursive();
// this has a broken return type; something to do with the return value of // this has a broken return type; something to do with the return value of
@ -225,6 +229,10 @@ public void testCreateFilePerformanceFlag() throws Throwable {
always(NO_HEAD_OR_LIST), always(NO_HEAD_OR_LIST),
with(OBJECT_BULK_DELETE_REQUEST, 0), with(OBJECT_BULK_DELETE_REQUEST, 0),
with(OBJECT_DELETE_REQUEST, 0)); with(OBJECT_DELETE_REQUEST, 0));
Assertions.assertThat(progressEvents.get())
.describedAs("progress events")
.isGreaterThanOrEqualTo(1);
} }
@Test @Test