HADOOP-16210. Update guava to 27.0-jre in hadoop-project trunk. Contributed by Gabor Bota.
This commit is contained in:
parent
d6c233fce6
commit
d7979079ea
@ -409,6 +409,13 @@
|
|||||||
<Bug pattern="NP_NULL_PARAM_DEREF"/>
|
<Bug pattern="NP_NULL_PARAM_DEREF"/>
|
||||||
</Match>
|
</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>
|
<Match>
|
||||||
<Class name="org.apache.hadoop.ipc.ExternalCall"/>
|
<Class name="org.apache.hadoop.ipc.ExternalCall"/>
|
||||||
<Filed name="done"/>
|
<Filed name="done"/>
|
||||||
|
@ -375,7 +375,7 @@ public void onFailure(Throwable t) {
|
|||||||
backgroundRefreshException.incrementAndGet();
|
backgroundRefreshException.incrementAndGet();
|
||||||
backgroundRefreshRunning.decrementAndGet();
|
backgroundRefreshRunning.decrementAndGet();
|
||||||
}
|
}
|
||||||
});
|
}, MoreExecutors.directExecutor());
|
||||||
return listenableFuture;
|
return listenableFuture;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -196,7 +196,7 @@ private Map<String, String> getNamenodesSubcluster(
|
|||||||
try {
|
try {
|
||||||
String nsId = nn.getNameserviceId();
|
String nsId = nn.getNameserviceId();
|
||||||
String rpcAddress = nn.getRpcAddress();
|
String rpcAddress = nn.getRpcAddress();
|
||||||
String hostname = HostAndPort.fromString(rpcAddress).getHostText();
|
String hostname = HostAndPort.fromString(rpcAddress).getHost();
|
||||||
ret.put(hostname, nsId);
|
ret.put(hostname, nsId);
|
||||||
if (hostname.equals(localHostname)) {
|
if (hostname.equals(localHostname)) {
|
||||||
ret.put(localIp, nsId);
|
ret.put(localIp, nsId);
|
||||||
|
@ -181,8 +181,8 @@
|
|||||||
<!-- More complex cleanup logic confuses findbugs -->
|
<!-- More complex cleanup logic confuses findbugs -->
|
||||||
<Match>
|
<Match>
|
||||||
<Class name="org.apache.hadoop.hdfs.qjournal.server.Journal" />
|
<Class name="org.apache.hadoop.hdfs.qjournal.server.Journal" />
|
||||||
<Method name="persistPaxosData" />
|
<Method name="getPersistedPaxosData" />
|
||||||
<Bug pattern="OS_OPEN_STREAM" />
|
<Bug pattern="NP_NULL_PARAM_DEREF" />
|
||||||
</Match>
|
</Match>
|
||||||
|
|
||||||
<!-- guava 27.0 update: @Nullable is not detected, however it's there -->
|
<!-- guava 27.0 update: @Nullable is not detected, however it's there -->
|
||||||
|
@ -447,7 +447,7 @@ public void onFailure(Throwable t) {
|
|||||||
public void onSuccess(Void t) {
|
public void onSuccess(Void t) {
|
||||||
unreserveQueueSpace(data.length);
|
unreserveQueueSpace(data.length);
|
||||||
}
|
}
|
||||||
});
|
}, MoreExecutors.directExecutor());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
return ret;
|
return ret;
|
||||||
|
@ -22,6 +22,7 @@
|
|||||||
import java.util.concurrent.TimeoutException;
|
import java.util.concurrent.TimeoutException;
|
||||||
import java.util.concurrent.TimeUnit;
|
import java.util.concurrent.TimeUnit;
|
||||||
|
|
||||||
|
import com.google.common.util.concurrent.MoreExecutors;
|
||||||
import org.apache.hadoop.ipc.RemoteException;
|
import org.apache.hadoop.ipc.RemoteException;
|
||||||
import org.apache.hadoop.util.StopWatch;
|
import org.apache.hadoop.util.StopWatch;
|
||||||
import org.apache.hadoop.util.Timer;
|
import org.apache.hadoop.util.Timer;
|
||||||
@ -80,7 +81,7 @@ public void onFailure(Throwable t) {
|
|||||||
public void onSuccess(RESULT res) {
|
public void onSuccess(RESULT res) {
|
||||||
qr.addResult(e.getKey(), res);
|
qr.addResult(e.getKey(), res);
|
||||||
}
|
}
|
||||||
});
|
}, MoreExecutors.directExecutor());
|
||||||
}
|
}
|
||||||
return qr;
|
return qr;
|
||||||
}
|
}
|
||||||
|
@ -1084,11 +1084,12 @@ private void persistPaxosData(long segmentTxId,
|
|||||||
fos.write('\n');
|
fos.write('\n');
|
||||||
// Write human-readable data after the protobuf. This is only
|
// Write human-readable data after the protobuf. This is only
|
||||||
// to assist in debugging -- it's not parsed at all.
|
// to assist in debugging -- it's not parsed at all.
|
||||||
OutputStreamWriter writer = new OutputStreamWriter(fos, Charsets.UTF_8);
|
try(OutputStreamWriter writer =
|
||||||
|
new OutputStreamWriter(fos, Charsets.UTF_8)) {
|
||||||
writer.write(String.valueOf(newData));
|
writer.write(String.valueOf(newData));
|
||||||
writer.write('\n');
|
writer.write('\n');
|
||||||
writer.flush();
|
writer.flush();
|
||||||
|
}
|
||||||
|
|
||||||
fos.flush();
|
fos.flush();
|
||||||
success = true;
|
success = true;
|
||||||
|
@ -24,6 +24,7 @@
|
|||||||
import com.google.common.util.concurrent.FutureCallback;
|
import com.google.common.util.concurrent.FutureCallback;
|
||||||
import com.google.common.util.concurrent.Futures;
|
import com.google.common.util.concurrent.Futures;
|
||||||
import com.google.common.util.concurrent.ListenableFuture;
|
import com.google.common.util.concurrent.ListenableFuture;
|
||||||
|
import com.google.common.util.concurrent.MoreExecutors;
|
||||||
import com.google.common.util.concurrent.ThreadFactoryBuilder;
|
import com.google.common.util.concurrent.ThreadFactoryBuilder;
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
||||||
@ -229,7 +230,7 @@ public void call(Set<FsVolumeSpi> ignored1,
|
|||||||
Set<FsVolumeSpi> ignored2) {
|
Set<FsVolumeSpi> ignored2) {
|
||||||
latch.countDown();
|
latch.countDown();
|
||||||
}
|
}
|
||||||
}));
|
}), MoreExecutors.directExecutor());
|
||||||
} else {
|
} else {
|
||||||
IOUtils.cleanup(null, reference);
|
IOUtils.cleanup(null, reference);
|
||||||
if (numVolumes.decrementAndGet() == 0) {
|
if (numVolumes.decrementAndGet() == 0) {
|
||||||
|
@ -182,7 +182,7 @@ public void onFailure(@Nonnull Throwable t) {
|
|||||||
t, timer.monotonicNow()));
|
t, timer.monotonicNow()));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
});
|
}, MoreExecutors.directExecutor());
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -20,6 +20,7 @@
|
|||||||
import com.google.common.util.concurrent.FutureCallback;
|
import com.google.common.util.concurrent.FutureCallback;
|
||||||
import com.google.common.util.concurrent.Futures;
|
import com.google.common.util.concurrent.Futures;
|
||||||
import com.google.common.util.concurrent.ListenableFuture;
|
import com.google.common.util.concurrent.ListenableFuture;
|
||||||
|
import com.google.common.util.concurrent.MoreExecutors;
|
||||||
import org.apache.hadoop.util.FakeTimer;
|
import org.apache.hadoop.util.FakeTimer;
|
||||||
import org.junit.Before;
|
import org.junit.Before;
|
||||||
import org.junit.Rule;
|
import org.junit.Rule;
|
||||||
@ -101,7 +102,7 @@ public void onFailure(Throwable t) {
|
|||||||
numCallbackInvocationsFailure.incrementAndGet();
|
numCallbackInvocationsFailure.incrementAndGet();
|
||||||
callbackResult.set(true);
|
callbackResult.set(true);
|
||||||
}
|
}
|
||||||
});
|
}, MoreExecutors.directExecutor());
|
||||||
|
|
||||||
while (!callbackResult.get()) {
|
while (!callbackResult.get()) {
|
||||||
// Wait for the callback
|
// Wait for the callback
|
||||||
@ -133,7 +134,8 @@ public void testDiskCheckTimeoutInvokesOneCallbackOnly() throws Exception {
|
|||||||
.schedule(target, true);
|
.schedule(target, true);
|
||||||
|
|
||||||
assertTrue(olf1.isPresent());
|
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
|
// Verify that timeout results in only 1 onFailure call and 0 onSuccess
|
||||||
// calls.
|
// calls.
|
||||||
@ -149,7 +151,8 @@ public void testDiskCheckTimeoutInvokesOneCallbackOnly() throws Exception {
|
|||||||
.schedule(target, true);
|
.schedule(target, true);
|
||||||
|
|
||||||
assertTrue(olf2.isPresent());
|
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.
|
// Verify that normal check (dummy) results in only 1 onSuccess call.
|
||||||
// Number of times onFailure is invoked should remain the same i.e. 1.
|
// Number of times onFailure is invoked should remain the same i.e. 1.
|
||||||
@ -187,7 +190,7 @@ public void onFailure(Throwable t) {
|
|||||||
throwable[0] = t;
|
throwable[0] = t;
|
||||||
callbackResult.set(true);
|
callbackResult.set(true);
|
||||||
}
|
}
|
||||||
});
|
}, MoreExecutors.directExecutor());
|
||||||
|
|
||||||
while (!callbackResult.get()) {
|
while (!callbackResult.get()) {
|
||||||
// Wait for the callback
|
// Wait for the callback
|
||||||
|
@ -120,7 +120,8 @@ public Iterable<FileStatus> getFileStatuses() throws InterruptedException,
|
|||||||
runningTasks.incrementAndGet();
|
runningTasks.incrementAndGet();
|
||||||
ListenableFuture<ProcessInitialInputPathCallable.Result> future = exec
|
ListenableFuture<ProcessInitialInputPathCallable.Result> future = exec
|
||||||
.submit(new ProcessInitialInputPathCallable(p, conf, inputFilter));
|
.submit(new ProcessInitialInputPathCallable(p, conf, inputFilter));
|
||||||
Futures.addCallback(future, processInitialInputPathCallback);
|
Futures.addCallback(future, processInitialInputPathCallback,
|
||||||
|
MoreExecutors.directExecutor());
|
||||||
}
|
}
|
||||||
|
|
||||||
runningTasks.decrementAndGet();
|
runningTasks.decrementAndGet();
|
||||||
@ -267,7 +268,8 @@ public void onSuccess(ProcessInputDirCallable.Result result) {
|
|||||||
ListenableFuture<ProcessInputDirCallable.Result> future = exec
|
ListenableFuture<ProcessInputDirCallable.Result> future = exec
|
||||||
.submit(new ProcessInputDirCallable(result.fs, fileStatus,
|
.submit(new ProcessInputDirCallable(result.fs, fileStatus,
|
||||||
recursive, inputFilter));
|
recursive, inputFilter));
|
||||||
Futures.addCallback(future, processInputDirCallback);
|
Futures.addCallback(future, processInputDirCallback,
|
||||||
|
MoreExecutors.directExecutor());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
decrementRunningAndCheckCompletion();
|
decrementRunningAndCheckCompletion();
|
||||||
@ -353,7 +355,8 @@ public void onSuccess(ProcessInitialInputPathCallable.Result result) {
|
|||||||
ListenableFuture<ProcessInputDirCallable.Result> future = exec
|
ListenableFuture<ProcessInputDirCallable.Result> future = exec
|
||||||
.submit(new ProcessInputDirCallable(result.fs, matched,
|
.submit(new ProcessInputDirCallable(result.fs, matched,
|
||||||
recursive, inputFilter));
|
recursive, inputFilter));
|
||||||
Futures.addCallback(future, processInputDirCallback);
|
Futures.addCallback(future, processInputDirCallback,
|
||||||
|
MoreExecutors.directExecutor());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
decrementRunningAndCheckCompletion();
|
decrementRunningAndCheckCompletion();
|
||||||
|
@ -92,7 +92,7 @@
|
|||||||
<spotbugs.version>3.1.0-RC1</spotbugs.version>
|
<spotbugs.version>3.1.0-RC1</spotbugs.version>
|
||||||
<dnsjava.version>2.1.7</dnsjava.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>
|
<guice.version>4.0</guice.version>
|
||||||
<joda-time.version>2.9.9</joda-time.version>
|
<joda-time.version>2.9.9</joda-time.version>
|
||||||
|
|
||||||
|
@ -87,7 +87,7 @@ public static List<HostAndPort> splitToHostsAndPorts(String hostPortQuorumList)
|
|||||||
public static String buildHostsOnlyList(List<HostAndPort> hostAndPorts) {
|
public static String buildHostsOnlyList(List<HostAndPort> hostAndPorts) {
|
||||||
StringBuilder sb = new StringBuilder();
|
StringBuilder sb = new StringBuilder();
|
||||||
for (HostAndPort hostAndPort : hostAndPorts) {
|
for (HostAndPort hostAndPort : hostAndPorts) {
|
||||||
sb.append(hostAndPort.getHostText()).append(",");
|
sb.append(hostAndPort.getHost()).append(",");
|
||||||
}
|
}
|
||||||
if (sb.length() > 0) {
|
if (sb.length() > 0) {
|
||||||
sb.delete(sb.length() - 1, sb.length());
|
sb.delete(sb.length() - 1, sb.length());
|
||||||
|
Loading…
Reference in New Issue
Block a user