HADOOP-19245. S3ABlockOutputStream no longer sends progress events in close() (#6974)
Contributed by Steve Loughran
This commit is contained in:
parent
c593c17255
commit
2cf4d638af
@ -1100,7 +1100,8 @@ private static class ProgressableListener implements ProgressListener {
|
||||
this.progress = progress;
|
||||
}
|
||||
|
||||
public void progressChanged(ProgressListenerEvent eventType, int bytesTransferred) {
|
||||
@Override
|
||||
public void progressChanged(ProgressListenerEvent eventType, long bytesTransferred) {
|
||||
if (progress != null) {
|
||||
progress.progress();
|
||||
}
|
||||
|
@ -21,6 +21,7 @@
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
||||
import org.assertj.core.api.Assertions;
|
||||
import org.junit.Test;
|
||||
@ -213,8 +214,11 @@ public void testCreateFilePerformanceFlag() throws Throwable {
|
||||
S3AFileSystem fs = getFileSystem();
|
||||
|
||||
Path path = methodPath();
|
||||
// increment progress events
|
||||
AtomicLong progressEvents = new AtomicLong(0);
|
||||
FSDataOutputStreamBuilder builder = fs.createFile(path)
|
||||
.overwrite(false)
|
||||
.progress(progressEvents::incrementAndGet)
|
||||
.recursive();
|
||||
|
||||
// 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),
|
||||
with(OBJECT_BULK_DELETE_REQUEST, 0),
|
||||
with(OBJECT_DELETE_REQUEST, 0));
|
||||
|
||||
Assertions.assertThat(progressEvents.get())
|
||||
.describedAs("progress events")
|
||||
.isGreaterThanOrEqualTo(1);
|
||||
}
|
||||
|
||||
@Test
|
||||
|
Loading…
Reference in New Issue
Block a user