Skip to content

Commit 19644c0

Browse files
HADOOP-17862. ABFS: Fix unchecked cast compiler warning for AbfsListStatusRemoteIterator (#3331)
closes #3331 Contributed by Sumangala Patki
1 parent 6f7b965 commit 19644c0

File tree

3 files changed

+152
-132
lines changed

3 files changed

+152
-132
lines changed
Lines changed: 57 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,57 @@
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+
* <p>
10+
* http://www.apache.org/licenses/LICENSE-2.0
11+
* <p>
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+
19+
package org.apache.hadoop.fs.azurebfs.services;
20+
21+
import java.io.IOException;
22+
import java.util.Collections;
23+
import java.util.Iterator;
24+
25+
import org.apache.hadoop.fs.FileStatus;
26+
27+
/**
28+
* Class to store listStatus results for AbfsListStatusRemoteIterator. The
29+
* results can either be of type Iterator or an exception thrown during the
30+
* operation
31+
*/
32+
public class AbfsListResult {
33+
private IOException listException = null;
34+
35+
private Iterator<FileStatus> fileStatusIterator
36+
= Collections.emptyIterator();
37+
38+
AbfsListResult(IOException ex) {
39+
this.listException = ex;
40+
}
41+
42+
AbfsListResult(Iterator<FileStatus> fileStatusIterator) {
43+
this.fileStatusIterator = fileStatusIterator;
44+
}
45+
46+
IOException getListingException() {
47+
return listException;
48+
}
49+
50+
Iterator<FileStatus> getFileStatusIterator() {
51+
return fileStatusIterator;
52+
}
53+
54+
boolean isFailedListing() {
55+
return (listException != null);
56+
}
57+
}

hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsListStatusRemoteIterator.java

Lines changed: 16 additions & 21 deletions
Original file line numberDiff line numberDiff line change
@@ -27,7 +27,6 @@
2727
import java.util.concurrent.ArrayBlockingQueue;
2828
import java.util.concurrent.CompletableFuture;
2929
import java.util.concurrent.TimeUnit;
30-
import javax.activation.UnsupportedDataTypeException;
3130

3231
import org.slf4j.Logger;
3332
import org.slf4j.LoggerFactory;
@@ -48,7 +47,7 @@ public class AbfsListStatusRemoteIterator
4847

4948
private final FileStatus fileStatus;
5049
private final ListingSupport listingSupport;
51-
private final ArrayBlockingQueue<Object> iteratorsQueue;
50+
private final ArrayBlockingQueue<AbfsListResult> listResultQueue;
5251
private final TracingContext tracingContext;
5352

5453
private volatile boolean isAsyncInProgress = false;
@@ -61,7 +60,7 @@ public AbfsListStatusRemoteIterator(final FileStatus fileStatus,
6160
this.fileStatus = fileStatus;
6261
this.listingSupport = listingSupport;
6362
this.tracingContext = tracingContext;
64-
iteratorsQueue = new ArrayBlockingQueue<>(MAX_QUEUE_SIZE);
63+
listResultQueue = new ArrayBlockingQueue<>(MAX_QUEUE_SIZE);
6564
currIterator = Collections.emptyIterator();
6665
fetchBatchesAsync();
6766
}
@@ -86,19 +85,17 @@ public FileStatus next() throws IOException {
8685
private Iterator<FileStatus> getNextIterator() throws IOException {
8786
fetchBatchesAsync();
8887
try {
89-
Object obj = null;
90-
while (obj == null
91-
&& (!isIterationComplete || !iteratorsQueue.isEmpty())) {
92-
obj = iteratorsQueue.poll(POLL_WAIT_TIME_IN_MS, TimeUnit.MILLISECONDS);
88+
AbfsListResult listResult = null;
89+
while (listResult == null
90+
&& (!isIterationComplete || !listResultQueue.isEmpty())) {
91+
listResult = listResultQueue.poll(POLL_WAIT_TIME_IN_MS, TimeUnit.MILLISECONDS);
9392
}
94-
if (obj == null) {
93+
if (listResult == null) {
9594
return Collections.emptyIterator();
96-
} else if (obj instanceof Iterator) {
97-
return (Iterator<FileStatus>) obj;
98-
} else if (obj instanceof IOException) {
99-
throw (IOException) obj;
95+
} else if (listResult.isFailedListing()) {
96+
throw listResult.getListingException();
10097
} else {
101-
throw new UnsupportedDataTypeException();
98+
return listResult.getFileStatusIterator();
10299
}
103100
} catch (InterruptedException e) {
104101
Thread.currentThread().interrupt();
@@ -122,13 +119,13 @@ private void fetchBatchesAsync() {
122119

123120
private void asyncOp() {
124121
try {
125-
while (!isIterationComplete && iteratorsQueue.size() <= MAX_QUEUE_SIZE) {
122+
while (!isIterationComplete && listResultQueue.size() <= MAX_QUEUE_SIZE) {
126123
addNextBatchIteratorToQueue();
127124
}
128125
} catch (IOException ioe) {
129126
LOG.error("Fetching filestatuses failed", ioe);
130127
try {
131-
iteratorsQueue.put(ioe);
128+
listResultQueue.put(new AbfsListResult(ioe));
132129
} catch (InterruptedException interruptedException) {
133130
Thread.currentThread().interrupt();
134131
LOG.error("Thread got interrupted: {}", interruptedException);
@@ -143,19 +140,17 @@ private void asyncOp() {
143140
}
144141
}
145142

146-
private void addNextBatchIteratorToQueue()
143+
private synchronized void addNextBatchIteratorToQueue()
147144
throws IOException, InterruptedException {
148145
List<FileStatus> fileStatuses = new ArrayList<>();
149146
continuation = listingSupport
150147
.listStatus(fileStatus.getPath(), null, fileStatuses, FETCH_ALL_FALSE,
151148
continuation, tracingContext);
152149
if (!fileStatuses.isEmpty()) {
153-
iteratorsQueue.put(fileStatuses.iterator());
150+
listResultQueue.put(new AbfsListResult(fileStatuses.iterator()));
154151
}
155-
synchronized (this) {
156-
if (continuation == null || continuation.isEmpty()) {
157-
isIterationComplete = true;
158-
}
152+
if (continuation == null || continuation.isEmpty()) {
153+
isIterationComplete = true;
159154
}
160155
}
161156

0 commit comments

Comments
 (0)