Skip to content

Commit d82b0cc

Browse files
authored
HADOOP-16080. hadoop-aws does not work with hadoop-client-api (#2510). Contributed by Chao Sun
1 parent f547cd4 commit d82b0cc

File tree

18 files changed

+59
-60
lines changed

18 files changed

+59
-60
lines changed

hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/BlockingThreadPoolExecutorService.java

Lines changed: 1 addition & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -28,8 +28,6 @@
2828
import org.slf4j.Logger;
2929
import org.slf4j.LoggerFactory;
3030

31-
import com.google.common.util.concurrent.MoreExecutors;
32-
3331
import org.apache.hadoop.classification.InterfaceAudience;
3432

3533
/**
@@ -105,8 +103,7 @@ public Thread newThread(Runnable r) {
105103

106104
private BlockingThreadPoolExecutorService(int permitCount,
107105
ThreadPoolExecutor eventProcessingExecutor) {
108-
super(MoreExecutors.listeningDecorator(eventProcessingExecutor),
109-
permitCount, false);
106+
super(eventProcessingExecutor, permitCount, false);
110107
this.eventProcessingExecutor = eventProcessingExecutor;
111108
}
112109

hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/SemaphoredDelegatingExecutor.java

Lines changed: 9 additions & 10 deletions
Original file line numberDiff line numberDiff line change
@@ -18,17 +18,16 @@
1818

1919
package org.apache.hadoop.util;
2020

21-
import com.google.common.util.concurrent.ForwardingListeningExecutorService;
21+
import com.google.common.util.concurrent.ForwardingExecutorService;
2222
import com.google.common.util.concurrent.Futures;
23-
import com.google.common.util.concurrent.ListenableFuture;
24-
import com.google.common.util.concurrent.ListeningExecutorService;
2523

2624
import org.apache.hadoop.classification.InterfaceAudience;
2725

2826
import java.util.Collection;
2927
import java.util.List;
3028
import java.util.concurrent.Callable;
3129
import java.util.concurrent.ExecutionException;
30+
import java.util.concurrent.ExecutorService;
3231
import java.util.concurrent.Future;
3332
import java.util.concurrent.Semaphore;
3433
import java.util.concurrent.TimeUnit;
@@ -49,10 +48,10 @@
4948
@SuppressWarnings("NullableProblems")
5049
@InterfaceAudience.Private
5150
public class SemaphoredDelegatingExecutor extends
52-
ForwardingListeningExecutorService {
51+
ForwardingExecutorService {
5352

5453
private final Semaphore queueingPermits;
55-
private final ListeningExecutorService executorDelegatee;
54+
private final ExecutorService executorDelegatee;
5655
private final int permitCount;
5756

5857
/**
@@ -62,7 +61,7 @@ public class SemaphoredDelegatingExecutor extends
6261
* @param fair should the semaphore be "fair"
6362
*/
6463
public SemaphoredDelegatingExecutor(
65-
ListeningExecutorService executorDelegatee,
64+
ExecutorService executorDelegatee,
6665
int permitCount,
6766
boolean fair) {
6867
this.permitCount = permitCount;
@@ -71,7 +70,7 @@ public SemaphoredDelegatingExecutor(
7170
}
7271

7372
@Override
74-
protected ListeningExecutorService delegate() {
73+
protected ExecutorService delegate() {
7574
return executorDelegatee;
7675
}
7776

@@ -102,7 +101,7 @@ public <T> T invokeAny(Collection<? extends Callable<T>> tasks, long timeout,
102101
}
103102

104103
@Override
105-
public <T> ListenableFuture<T> submit(Callable<T> task) {
104+
public <T> Future<T> submit(Callable<T> task) {
106105
try {
107106
queueingPermits.acquire();
108107
} catch (InterruptedException e) {
@@ -113,7 +112,7 @@ public <T> ListenableFuture<T> submit(Callable<T> task) {
113112
}
114113

115114
@Override
116-
public <T> ListenableFuture<T> submit(Runnable task, T result) {
115+
public <T> Future<T> submit(Runnable task, T result) {
117116
try {
118117
queueingPermits.acquire();
119118
} catch (InterruptedException e) {
@@ -124,7 +123,7 @@ public <T> ListenableFuture<T> submit(Runnable task, T result) {
124123
}
125124

126125
@Override
127-
public ListenableFuture<?> submit(Runnable task) {
126+
public Future<?> submit(Runnable task) {
128127
try {
129128
queueingPermits.acquire();
130129
} catch (InterruptedException e) {

hadoop-tools/hadoop-aliyun/src/main/java/org/apache/hadoop/fs/aliyun/oss/AliyunOSSFileSystem.java

Lines changed: 2 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -27,7 +27,6 @@
2727
import java.util.concurrent.ExecutorService;
2828
import java.util.concurrent.TimeUnit;
2929

30-
import com.google.common.util.concurrent.ListeningExecutorService;
3130
import com.google.common.util.concurrent.MoreExecutors;
3231
import org.apache.commons.collections.CollectionUtils;
3332
import org.apache.commons.lang3.StringUtils;
@@ -78,8 +77,8 @@ public class AliyunOSSFileSystem extends FileSystem {
7877
private int maxKeys;
7978
private int maxReadAheadPartNumber;
8079
private int maxConcurrentCopyTasksPerDir;
81-
private ListeningExecutorService boundedThreadPool;
82-
private ListeningExecutorService boundedCopyThreadPool;
80+
private ExecutorService boundedThreadPool;
81+
private ExecutorService boundedCopyThreadPool;
8382

8483
private static final PathFilter DEFAULT_FILTER = new PathFilter() {
8584
@Override

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

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -156,7 +156,7 @@ public String toString() {
156156
*/
157157
private static float validProbability(float p) {
158158
Preconditions.checkArgument(p >= 0.0f && p <= 1.0f,
159-
"Probability out of range 0 to 1 %s", p);
159+
String.format("Probability out of range 0 to 1 %s", p));
160160
return p;
161161
}
162162

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

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -294,8 +294,8 @@ private void addPrefixIfNotPresent(List<String> prefixes, String ancestor,
294294
String child) {
295295
Path prefixCandidate = new Path(child).getParent();
296296
Path ancestorPath = new Path(ancestor);
297-
Preconditions.checkArgument(child.startsWith(ancestor), "%s does not " +
298-
"start with %s", child, ancestor);
297+
Preconditions.checkArgument(child.startsWith(ancestor),
298+
String.format("%s does not start with %s", child, ancestor));
299299
while (!prefixCandidate.isRoot()) {
300300
Path nextParent = prefixCandidate.getParent();
301301
if (nextParent.equals(ancestorPath)) {

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

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -157,7 +157,7 @@ class S3ABlockOutputStream extends OutputStream implements
157157
this.writeOperationHelper = writeOperationHelper;
158158
this.putTracker = putTracker;
159159
Preconditions.checkArgument(blockSize >= Constants.MULTIPART_MIN_SIZE,
160-
"Block size is too small: %d", blockSize);
160+
String.format("Block size is too small: %d", blockSize));
161161
this.executorService = MoreExecutors.listeningDecorator(executorService);
162162
this.multiPartUpload = null;
163163
this.progressListener = (progress instanceof ProgressListener) ?

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

Lines changed: 4 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -75,7 +75,6 @@
7575
import com.amazonaws.event.ProgressListener;
7676
import com.google.common.annotations.VisibleForTesting;
7777
import com.google.common.base.Preconditions;
78-
import com.google.common.util.concurrent.ListeningExecutorService;
7978
import org.slf4j.Logger;
8079
import org.slf4j.LoggerFactory;
8180

@@ -181,7 +180,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities {
181180
private long partSize;
182181
private boolean enableMultiObjectsDelete;
183182
private TransferManager transfers;
184-
private ListeningExecutorService boundedThreadPool;
183+
private ExecutorService boundedThreadPool;
185184
private ExecutorService unboundedThreadPool;
186185
private long multiPartThreshold;
187186
public static final Logger LOG = LoggerFactory.getLogger(S3AFileSystem.class);
@@ -2254,9 +2253,9 @@ S3AFileStatus s3GetFileStatus(final Path path,
22542253
final boolean needEmptyDirectoryFlag) throws IOException {
22552254
LOG.debug("S3GetFileStatus {}", path);
22562255
Preconditions.checkArgument(!needEmptyDirectoryFlag
2257-
|| probes.contains(StatusProbeEnum.List),
2258-
"s3GetFileStatus(%s) wants to know if a directory is empty but"
2259-
+ " does not request a list probe", path);
2256+
|| probes.contains(StatusProbeEnum.List), String.format(
2257+
"s3GetFileStatus(%s) wants to know if a directory is empty but"
2258+
+ " does not request a list probe", path));
22602259

22612260
if (!key.isEmpty() && !key.endsWith("/")
22622261
&& probes.contains(StatusProbeEnum.Head)) {

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

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -71,7 +71,7 @@ public class S3AMultipartUploader extends MultipartUploader {
7171

7272
public S3AMultipartUploader(FileSystem fs, Configuration conf) {
7373
Preconditions.checkArgument(fs instanceof S3AFileSystem,
74-
"Wrong filesystem: expected S3A but got %s", fs);
74+
String.format("Wrong filesystem: expected S3A but got %s", fs));
7575
s3a = (S3AFileSystem) fs;
7676
}
7777

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

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -82,7 +82,7 @@ public S3AReadOpContext(
8282
dstFileStatus);
8383
this.path = checkNotNull(path);
8484
Preconditions.checkArgument(readahead >= 0,
85-
"invalid readahead %d", readahead);
85+
String.format("invalid readahead %d", readahead));
8686
this.inputPolicy = checkNotNull(inputPolicy);
8787
this.changeDetectionPolicy = checkNotNull(changeDetectionPolicy);
8888
this.readahead = readahead;

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

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -814,7 +814,7 @@ public static String lookupPassword(
814814
throws IOException {
815815
String initialVal;
816816
Preconditions.checkArgument(baseKey.startsWith(FS_S3A_PREFIX),
817-
"%s does not start with $%s", baseKey, FS_S3A_PREFIX);
817+
String.format("%s does not start with $%s", baseKey, FS_S3A_PREFIX));
818818
// if there's a bucket, work with it
819819
if (StringUtils.isNotEmpty(bucket)) {
820820
String subkey = baseKey.substring(FS_S3A_PREFIX.length());

0 commit comments

Comments
 (0)