Skip to content

Commit 2e18baa

Browse files
committed
Fix javadocs + license
1 parent 128a2cd commit 2e18baa

File tree

5 files changed

+154
-48
lines changed

5 files changed

+154
-48
lines changed

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

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -1699,8 +1699,8 @@ private final class WriteOperationHelperCallbacksImpl
16991699
@Override
17001700
public CompletableFuture<Void> selectObjectContent(
17011701
SelectObjectContentRequest request,
1702-
SelectObjectContentResponseHandler th) {
1703-
return s3AsyncClient.selectObjectContent(request, th);
1702+
SelectObjectContentResponseHandler responseHandler) {
1703+
return s3AsyncClient.selectObjectContent(request, responseHandler);
17041704
}
17051705

17061706
@Override

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

Lines changed: 39 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -1,3 +1,21 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one
3+
* or more contributor license agreements. See the NOTICE file
4+
* distributed with this work for additional information
5+
* regarding copyright ownership. The ASF licenses this file
6+
* to you under the Apache License, Version 2.0 (the
7+
* "License"); you may not use this file except in compliance
8+
* with the License. You may obtain a copy of the License at
9+
*
10+
* http://www.apache.org/licenses/LICENSE-2.0
11+
*
12+
* Unless required by applicable law or agreed to in writing, software
13+
* distributed under the License is distributed on an "AS IS" BASIS,
14+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15+
* See the License for the specific language governing permissions and
16+
* limitations under the License.
17+
*/
18+
119
package org.apache.hadoop.fs.s3a.select;
220

321
import java.util.Enumeration;
@@ -7,9 +25,18 @@
725

826
import org.reactivestreams.Subscriber;
927
import org.reactivestreams.Subscription;
28+
1029
import software.amazon.awssdk.core.async.SdkPublisher;
1130
import software.amazon.awssdk.core.exception.SdkException;
1231

32+
/**
33+
* Implements the {@link Enumeration} interface by subscribing to a
34+
* {@link SdkPublisher} instance. The enumeration will buffer a fixed
35+
* number of events received from the publisher and block until the first is
36+
* consumed. Calls to {@link #hasMoreElements()} and {@link #nextElement()}
37+
* can also block while waiting for new events.
38+
* @param <T> the type of element.
39+
*/
1340
public final class BlockingEnumeration<T> implements Enumeration<T> {
1441
private static final class Token<T> {
1542
public T element;
@@ -31,6 +58,11 @@ public Token(Throwable error) {
3158
private final BlockingQueue<Token<T>> blockingQueue;
3259
private Token<T> current = null;
3360

61+
/**
62+
* Create an enumeration with a fixed buffer size.
63+
* @param publisher the publisher feeding the enumeration.
64+
* @param limit the buffer size.
65+
*/
3466
public BlockingEnumeration(SdkPublisher<T> publisher, final int limit) {
3567
this.blockingQueue = new LinkedBlockingQueue<>(limit);
3668
publisher.subscribe(new Subscriber<T>() {
@@ -68,6 +100,13 @@ private void enqueue(Token<T> token) {
68100
});
69101
}
70102

103+
/**
104+
* Create an enumeration with a fixed buffer size and an
105+
* injected first element.
106+
* @param publisher the publisher feeding the enumeration.
107+
* @param limit the buffer size.
108+
* @param firstElement first element the enumeration will return.
109+
*/
71110
public BlockingEnumeration(SdkPublisher<T> publisher,
72111
final int limit,
73112
T firstElement) {
Lines changed: 73 additions & 32 deletions
Original file line numberDiff line numberDiff line change
@@ -1,18 +1,30 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one
3+
* or more contributor license agreements. See the NOTICE file
4+
* distributed with this work for additional information
5+
* regarding copyright ownership. The ASF licenses this file
6+
* to you under the Apache License, Version 2.0 (the
7+
* "License"); you may not use this file except in compliance
8+
* with the License. You may obtain a copy of the License at
9+
*
10+
* http://www.apache.org/licenses/LICENSE-2.0
11+
*
12+
* Unless required by applicable law or agreed to in writing, software
13+
* distributed under the License is distributed on an "AS IS" BASIS,
14+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15+
* See the License for the specific language governing permissions and
16+
* limitations under the License.
17+
*/
18+
119
package org.apache.hadoop.fs.s3a.select;
220

321
import java.io.ByteArrayInputStream;
422
import java.io.InputStream;
523
import java.io.SequenceInputStream;
6-
import java.util.Enumeration;
7-
import java.util.Iterator;
8-
import java.util.NoSuchElementException;
9-
import java.util.concurrent.ArrayBlockingQueue;
10-
import java.util.concurrent.BlockingQueue;
1124
import java.util.concurrent.CompletableFuture;
1225
import java.util.function.Consumer;
1326

1427
import org.reactivestreams.Subscriber;
15-
import org.reactivestreams.Subscription;
1628

1729
import software.amazon.awssdk.core.async.SdkPublisher;
1830
import software.amazon.awssdk.http.AbortableInputStream;
@@ -23,26 +35,67 @@
2335
import software.amazon.awssdk.utils.ToString;
2436
import software.amazon.awssdk.utils.Validate;
2537

38+
/**
39+
* Async publisher of {@link SelectObjectContentEventStream}s returned
40+
* from a SelectObjectContent call.
41+
*/
2642
public final class SelectEventStreamPublisher implements
2743
SdkPublisher<SelectObjectContentEventStream> {
2844

29-
private final CompletableFuture<Void> future;
45+
private final CompletableFuture<Void> selectOperationFuture;
3046
private final SelectObjectContentResponse response;
3147
private final SdkPublisher<SelectObjectContentEventStream> publisher;
3248

49+
/**
50+
* Create the publisher.
51+
* @param selectOperationFuture SelectObjectContent future
52+
* @param response SelectObjectContent response
53+
* @param publisher SelectObjectContentEventStream publisher to wrap
54+
*/
3355
public SelectEventStreamPublisher(
34-
CompletableFuture<Void> future,
56+
CompletableFuture<Void> selectOperationFuture,
3557
SelectObjectContentResponse response,
3658
SdkPublisher<SelectObjectContentEventStream> publisher) {
37-
this.future = Validate.paramNotNull(future, "future");
38-
this.response = Validate.paramNotNull(response, "response");
39-
this.publisher = Validate.paramNotNull(publisher, "publisher");
59+
this.selectOperationFuture = selectOperationFuture;
60+
this.response = response;
61+
this.publisher = publisher;
4062
}
4163

42-
public void cancel() {
43-
future.cancel(true);
64+
/**
65+
* Retrieve an input stream to the subset of the S3 object that matched the select query.
66+
* This is equivalent to loading the content of all RecordsEvents into an InputStream.
67+
* This will lazily-load the content from S3, minimizing the amount of memory used.
68+
* @param onEndEvent callback on the end event
69+
* @return the input stream
70+
*/
71+
public AbortableInputStream toRecordsInputStream(Consumer<EndEvent> onEndEvent) {
72+
SdkPublisher<InputStream> recordInputStreams = this.publisher
73+
.filter(e -> {
74+
if (e instanceof RecordsEvent) {
75+
return true;
76+
} else if (e instanceof EndEvent) {
77+
onEndEvent.accept((EndEvent) e);
78+
}
79+
return false;
80+
})
81+
.map(e -> ((RecordsEvent) e).payload().asInputStream());
82+
83+
// Subscribe to the async publisher using an enumeration that will
84+
// buffer a single chunk (RecordsEvent's payload) at a time and
85+
// block until it is consumed.
86+
// Also inject an empty stream as the first element that
87+
// SequenceInputStream will request on construction.
88+
BlockingEnumeration enumeration =
89+
new BlockingEnumeration(recordInputStreams, 1, EMPTY_STREAM);
90+
return AbortableInputStream.create(
91+
new SequenceInputStream(enumeration),
92+
this::cancel);
4493
}
4594

95+
/**
96+
* The response from the SelectObjectContent call.
97+
* @return the response object
98+
*/
4699
public SelectObjectContentResponse response() {
47100
return response;
48101
}
@@ -52,6 +105,13 @@ public void subscribe(Subscriber<? super SelectObjectContentEventStream> subscri
52105
publisher.subscribe(subscriber);
53106
}
54107

108+
/**
109+
* Cancel the operation.
110+
*/
111+
public void cancel() {
112+
selectOperationFuture.cancel(true);
113+
}
114+
55115
@Override
56116
public String toString() {
57117
return ToString.builder("SelectObjectContentEventStream")
@@ -60,25 +120,6 @@ public String toString() {
60120
.build();
61121
}
62122

63-
public AbortableInputStream toRecordsInputStream(Consumer<EndEvent> onEndEvent) {
64-
SdkPublisher<InputStream> recordInputStreams = this.publisher
65-
.filter(e -> {
66-
if (e instanceof RecordsEvent) {
67-
return true;
68-
} else if (e instanceof EndEvent) {
69-
onEndEvent.accept((EndEvent) e);
70-
}
71-
return false;
72-
})
73-
.map(e -> ((RecordsEvent) e).payload().asInputStream());
74-
75-
return AbortableInputStream.create(
76-
new SequenceInputStream(
77-
new BlockingEnumeration(recordInputStreams, 1,
78-
EMPTY_STREAM)),
79-
this::cancel);
80-
}
81-
82123
private static final InputStream EMPTY_STREAM =
83124
new ByteArrayInputStream(new byte[0]);
84125
}

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

Lines changed: 1 addition & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -20,7 +20,6 @@
2020

2121
import java.io.EOFException;
2222
import java.io.IOException;
23-
import java.io.InputStream;
2423
import java.util.concurrent.atomic.AtomicBoolean;
2524
import java.util.concurrent.atomic.AtomicLong;
2625

@@ -43,9 +42,6 @@
4342

4443
import software.amazon.awssdk.core.exception.AbortedException;
4544
import software.amazon.awssdk.http.AbortableInputStream;
46-
import software.amazon.awssdk.services.s3.model.RecordsEvent;
47-
import software.amazon.awssdk.services.s3.model.SelectObjectContentEventStream.EventType;
48-
import software.amazon.awssdk.services.s3.model.SelectObjectContentResponseHandler;
4945

5046
import static org.apache.hadoop.util.Preconditions.checkNotNull;
5147
import static org.apache.commons.lang3.StringUtils.isNotEmpty;
@@ -115,7 +111,7 @@ public class SelectInputStream extends FSInputStream implements
115111
* The read attempt is initiated immediately.
116112
* @param readContext read context
117113
* @param objectAttributes object attributes from a HEAD request
118-
* @param selectResponse response from the already executed call
114+
* @param selectPublisher event stream publisher from the already executed call
119115
* @throws IOException failure
120116
*/
121117
@Retries.OnceTranslated

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

Lines changed: 39 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -1,3 +1,21 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one
3+
* or more contributor license agreements. See the NOTICE file
4+
* distributed with this work for additional information
5+
* regarding copyright ownership. The ASF licenses this file
6+
* to you under the Apache License, Version 2.0 (the
7+
* "License"); you may not use this file except in compliance
8+
* with the License. You may obtain a copy of the License at
9+
*
10+
* http://www.apache.org/licenses/LICENSE-2.0
11+
*
12+
* Unless required by applicable law or agreed to in writing, software
13+
* distributed under the License is distributed on an "AS IS" BASIS,
14+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15+
* See the License for the specific language governing permissions and
16+
* limitations under the License.
17+
*/
18+
119
package org.apache.hadoop.fs.s3a.select;
220

321
import java.io.IOException;
@@ -17,8 +35,20 @@
1735

1836
import static org.apache.hadoop.fs.s3a.WriteOperationHelper.WriteOperationHelperCallbacks;
1937

38+
/**
39+
* Helper for SelectObjectContent queries against an S3 Bucket.
40+
*/
2041
public final class SelectObjectContentHelper {
21-
42+
43+
/**
44+
* Execute an S3 Select operation.
45+
* @param writeOperationHelperCallbacks helper callbacks
46+
* @param source source for selection
47+
* @param request Select request to issue.
48+
* @param action the action for use in exception creation
49+
* @return the select response event stream publisher
50+
* @throws IOException on failure
51+
*/
2252
public static SelectEventStreamPublisher select(
2353
WriteOperationHelperCallbacks writeOperationHelperCallbacks,
2454
Path source,
@@ -27,9 +57,9 @@ public static SelectEventStreamPublisher select(
2757
throws IOException {
2858
try {
2959
Handler handler = new Handler();
30-
CompletableFuture<Void> operationFuture =
60+
CompletableFuture<Void> selectOperationFuture =
3161
writeOperationHelperCallbacks.selectObjectContent(request, handler);
32-
return handler.eventPublisher(operationFuture).join();
62+
return handler.eventPublisher(selectOperationFuture).join();
3363
} catch (Throwable e) {
3464
if (e instanceof CompletionException) {
3565
e = e.getCause();
@@ -47,15 +77,15 @@ public static SelectEventStreamPublisher select(
4777

4878
private static class Handler implements SelectObjectContentResponseHandler {
4979
private volatile CompletableFuture<Pair<SelectObjectContentResponse,
50-
SdkPublisher<SelectObjectContentEventStream>>> future =
80+
SdkPublisher<SelectObjectContentEventStream>>> responseAndPublisherFuture =
5181
new CompletableFuture<>();
5282

5383
private volatile SelectObjectContentResponse response;
5484

5585
public CompletableFuture<SelectEventStreamPublisher> eventPublisher(
56-
CompletableFuture<Void> operationFuture) {
57-
return future.thenApply(p ->
58-
new SelectEventStreamPublisher(operationFuture,
86+
CompletableFuture<Void> selectOperationFuture) {
87+
return responseAndPublisherFuture.thenApply(p ->
88+
new SelectEventStreamPublisher(selectOperationFuture,
5989
p.getLeft(), p.getRight()));
6090
}
6191

@@ -66,12 +96,12 @@ public void responseReceived(SelectObjectContentResponse response) {
6696

6797
@Override
6898
public void onEventStream(SdkPublisher<SelectObjectContentEventStream> publisher) {
69-
future.complete(Pair.of(response, publisher));
99+
responseAndPublisherFuture.complete(Pair.of(response, publisher));
70100
}
71101

72102
@Override
73103
public void exceptionOccurred(Throwable error) {
74-
future.completeExceptionally(error);
104+
responseAndPublisherFuture.completeExceptionally(error);
75105
}
76106

77107
@Override

0 commit comments

Comments
 (0)