HADOOP-16210. Update guava to 27.0-jre in hadoop-project trunk. Contributed by Gabor Bota.

This commit is contained in:
Gabor Bota 2018-11-29 16:27:55 +01:00 committed by Sean Mackrory
parent d6c233fce6
commit d7979079ea
13 changed files with 43 additions and 27 deletions

View File

@ -409,6 +409,13 @@
<Bug pattern="NP_NULL_PARAM_DEREF"/>
</Match>
<!-- Parameter is checked with Strings.isNullOrEmpty before accessing -->
<Match>
<Class name="org.apache.hadoop.conf.Configuration"/>
<Method name="asXmlDocument"/>
<Bug pattern="NP_PARAMETER_MUST_BE_NONNULL_BUT_MARKED_AS_NULLABLE"/>
</Match>
<Match>
<Class name="org.apache.hadoop.ipc.ExternalCall"/>
<Filed name="done"/>

View File

@ -375,7 +375,7 @@ public class Groups {
backgroundRefreshException.incrementAndGet();
backgroundRefreshRunning.decrementAndGet();
}
});
}, MoreExecutors.directExecutor());
return listenableFuture;
}

View File

@ -196,7 +196,7 @@ public class LocalResolver extends RouterResolver<String, String> {
try {
String nsId = nn.getNameserviceId();
String rpcAddress = nn.getRpcAddress();
String hostname = HostAndPort.fromString(rpcAddress).getHostText();
String hostname = HostAndPort.fromString(rpcAddress).getHost();
ret.put(hostname, nsId);
if (hostname.equals(localHostname)) {
ret.put(localIp, nsId);

View File

@ -181,8 +181,8 @@
<!-- More complex cleanup logic confuses findbugs -->
<Match>
<Class name="org.apache.hadoop.hdfs.qjournal.server.Journal" />
<Method name="persistPaxosData" />
<Bug pattern="OS_OPEN_STREAM" />
<Method name="getPersistedPaxosData" />
<Bug pattern="NP_NULL_PARAM_DEREF" />
</Match>
<!-- guava 27.0 update: @Nullable is not detected, however it's there -->

View File

@ -447,7 +447,7 @@ public class IPCLoggerChannel implements AsyncLogger {
public void onSuccess(Void t) {
unreserveQueueSpace(data.length);
}
});
}, MoreExecutors.directExecutor());
}
}
return ret;

View File

@ -22,6 +22,7 @@ import java.util.Map.Entry;
import java.util.concurrent.TimeoutException;
import java.util.concurrent.TimeUnit;
import com.google.common.util.concurrent.MoreExecutors;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.hadoop.util.StopWatch;
import org.apache.hadoop.util.Timer;
@ -80,7 +81,7 @@ class QuorumCall<KEY, RESULT> {
public void onSuccess(RESULT res) {
qr.addResult(e.getKey(), res);
}
});
}, MoreExecutors.directExecutor());
}
return qr;
}

View File

@ -1084,11 +1084,12 @@ public class Journal implements Closeable {
fos.write('\n');
// Write human-readable data after the protobuf. This is only
// to assist in debugging -- it's not parsed at all.
OutputStreamWriter writer = new OutputStreamWriter(fos, Charsets.UTF_8);
writer.write(String.valueOf(newData));
writer.write('\n');
writer.flush();
try(OutputStreamWriter writer =
new OutputStreamWriter(fos, Charsets.UTF_8)) {
writer.write(String.valueOf(newData));
writer.write('\n');
writer.flush();
}
fos.flush();
success = true;

View File

@ -24,6 +24,7 @@ import com.google.common.collect.Sets;
import com.google.common.util.concurrent.FutureCallback;
import com.google.common.util.concurrent.Futures;
import com.google.common.util.concurrent.ListenableFuture;
import com.google.common.util.concurrent.MoreExecutors;
import com.google.common.util.concurrent.ThreadFactoryBuilder;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.DFSConfigKeys;
@ -224,12 +225,12 @@ public class DatasetVolumeChecker {
Futures.addCallback(olf.get(),
new ResultHandler(reference, healthyVolumes, failedVolumes,
numVolumes, new Callback() {
@Override
public void call(Set<FsVolumeSpi> ignored1,
Set<FsVolumeSpi> ignored2) {
latch.countDown();
}
}));
@Override
public void call(Set<FsVolumeSpi> ignored1,
Set<FsVolumeSpi> ignored2) {
latch.countDown();
}
}), MoreExecutors.directExecutor());
} else {
IOUtils.cleanup(null, reference);
if (numVolumes.decrementAndGet() == 0) {

View File

@ -182,7 +182,7 @@ public class ThrottledAsyncChecker<K, V> implements AsyncChecker<K, V> {
t, timer.monotonicNow()));
}
}
});
}, MoreExecutors.directExecutor());
}
/**

View File

@ -20,6 +20,7 @@ package org.apache.hadoop.hdfs.server.datanode.checker;
import com.google.common.util.concurrent.FutureCallback;
import com.google.common.util.concurrent.Futures;
import com.google.common.util.concurrent.ListenableFuture;
import com.google.common.util.concurrent.MoreExecutors;
import org.apache.hadoop.util.FakeTimer;
import org.junit.Before;
import org.junit.Rule;
@ -101,7 +102,7 @@ public class TestThrottledAsyncCheckerTimeout {
numCallbackInvocationsFailure.incrementAndGet();
callbackResult.set(true);
}
});
}, MoreExecutors.directExecutor());
while (!callbackResult.get()) {
// Wait for the callback
@ -133,7 +134,8 @@ public class TestThrottledAsyncCheckerTimeout {
.schedule(target, true);
assertTrue(olf1.isPresent());
Futures.addCallback(olf1.get(), futureCallback);
Futures.addCallback(olf1.get(), futureCallback,
MoreExecutors.directExecutor());
// Verify that timeout results in only 1 onFailure call and 0 onSuccess
// calls.
@ -149,7 +151,8 @@ public class TestThrottledAsyncCheckerTimeout {
.schedule(target, true);
assertTrue(olf2.isPresent());
Futures.addCallback(olf2.get(), futureCallback);
Futures.addCallback(olf2.get(), futureCallback,
MoreExecutors.directExecutor());
// Verify that normal check (dummy) results in only 1 onSuccess call.
// Number of times onFailure is invoked should remain the same i.e. 1.
@ -187,7 +190,7 @@ public class TestThrottledAsyncCheckerTimeout {
throwable[0] = t;
callbackResult.set(true);
}
});
}, MoreExecutors.directExecutor());
while (!callbackResult.get()) {
// Wait for the callback

View File

@ -120,7 +120,8 @@ public class LocatedFileStatusFetcher {
runningTasks.incrementAndGet();
ListenableFuture<ProcessInitialInputPathCallable.Result> future = exec
.submit(new ProcessInitialInputPathCallable(p, conf, inputFilter));
Futures.addCallback(future, processInitialInputPathCallback);
Futures.addCallback(future, processInitialInputPathCallback,
MoreExecutors.directExecutor());
}
runningTasks.decrementAndGet();
@ -267,7 +268,8 @@ public class LocatedFileStatusFetcher {
ListenableFuture<ProcessInputDirCallable.Result> future = exec
.submit(new ProcessInputDirCallable(result.fs, fileStatus,
recursive, inputFilter));
Futures.addCallback(future, processInputDirCallback);
Futures.addCallback(future, processInputDirCallback,
MoreExecutors.directExecutor());
}
}
decrementRunningAndCheckCompletion();
@ -353,7 +355,8 @@ public class LocatedFileStatusFetcher {
ListenableFuture<ProcessInputDirCallable.Result> future = exec
.submit(new ProcessInputDirCallable(result.fs, matched,
recursive, inputFilter));
Futures.addCallback(future, processInputDirCallback);
Futures.addCallback(future, processInputDirCallback,
MoreExecutors.directExecutor());
}
}
decrementRunningAndCheckCompletion();

View File

@ -92,7 +92,7 @@
<spotbugs.version>3.1.0-RC1</spotbugs.version>
<dnsjava.version>2.1.7</dnsjava.version>
<guava.version>11.0.2</guava.version>
<guava.version>27.0-jre</guava.version>
<guice.version>4.0</guice.version>
<joda-time.version>2.9.9</joda-time.version>

View File

@ -87,7 +87,7 @@ public class ZookeeperUtils {
public static String buildHostsOnlyList(List<HostAndPort> hostAndPorts) {
StringBuilder sb = new StringBuilder();
for (HostAndPort hostAndPort : hostAndPorts) {
sb.append(hostAndPort.getHostText()).append(",");
sb.append(hostAndPort.getHost()).append(",");
}
if (sb.length() > 0) {
sb.delete(sb.length() - 1, sb.length());