Skip to content

Commit 2cf4d63

Browse files
HADOOP-19245. S3ABlockOutputStream no longer sends progress events in close() (#6974)
Contributed by Steve Loughran
1 parent c593c17 commit 2cf4d63

File tree

2 files changed

+10
-1
lines changed

2 files changed

+10
-1
lines changed

hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -1100,7 +1100,8 @@ private static class ProgressableListener implements ProgressListener {
11001100
this.progress = progress;
11011101
}
11021102

1103-
public void progressChanged(ProgressListenerEvent eventType, int bytesTransferred) {
1103+
@Override
1104+
public void progressChanged(ProgressListenerEvent eventType, long bytesTransferred) {
11041105
if (progress != null) {
11051106
progress.progress();
11061107
}

hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestCreateFileCost.java

Lines changed: 8 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -21,6 +21,7 @@
2121
import java.io.IOException;
2222
import java.util.Arrays;
2323
import java.util.Collection;
24+
import java.util.concurrent.atomic.AtomicLong;
2425

2526
import org.assertj.core.api.Assertions;
2627
import org.junit.Test;
@@ -213,8 +214,11 @@ public void testCreateFilePerformanceFlag() throws Throwable {
213214
S3AFileSystem fs = getFileSystem();
214215

215216
Path path = methodPath();
217+
// increment progress events
218+
AtomicLong progressEvents = new AtomicLong(0);
216219
FSDataOutputStreamBuilder builder = fs.createFile(path)
217220
.overwrite(false)
221+
.progress(progressEvents::incrementAndGet)
218222
.recursive();
219223

220224
// this has a broken return type; something to do with the return value of
@@ -225,6 +229,10 @@ public void testCreateFilePerformanceFlag() throws Throwable {
225229
always(NO_HEAD_OR_LIST),
226230
with(OBJECT_BULK_DELETE_REQUEST, 0),
227231
with(OBJECT_DELETE_REQUEST, 0));
232+
233+
Assertions.assertThat(progressEvents.get())
234+
.describedAs("progress events")
235+
.isGreaterThanOrEqualTo(1);
228236
}
229237

230238
@Test

0 commit comments

Comments
 (0)