Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -375,7 +375,7 @@ public void onFailure(Throwable t) {
backgroundRefreshException.incrementAndGet();
backgroundRefreshRunning.decrementAndGet();
}
});
}, MoreExecutors.directExecutor());
return listenableFuture;
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -196,7 +196,7 @@ private Map<String, String> getNamenodesSubcluster(
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);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -447,7 +447,7 @@ public void onFailure(Throwable t) {
public void onSuccess(Void t) {
unreserveQueueSpace(data.length);
}
});
}, MoreExecutors.directExecutor());
}
}
return ret;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@
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;
Expand Down Expand Up @@ -80,7 +81,7 @@ public void onFailure(Throwable t) {
public void onSuccess(RESULT res) {
qr.addResult(e.getKey(), res);
}
});
}, MoreExecutors.directExecutor());
}
return qr;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@
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;
Expand Down Expand Up @@ -224,12 +225,12 @@ public Set<FsVolumeSpi> checkAllVolumes(
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) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -182,7 +182,7 @@ public void onFailure(@Nonnull Throwable t) {
t, timer.monotonicNow()));
}
}
});
}, MoreExecutors.directExecutor());
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@
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;
Expand Down Expand Up @@ -101,7 +102,7 @@ public void onFailure(Throwable t) {
numCallbackInvocationsFailure.incrementAndGet();
callbackResult.set(true);
}
});
}, MoreExecutors.directExecutor());

while (!callbackResult.get()) {
// Wait for the callback
Expand Down Expand Up @@ -133,7 +134,8 @@ public void testDiskCheckTimeoutInvokesOneCallbackOnly() throws Exception {
.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.
Expand All @@ -149,7 +151,8 @@ public void testDiskCheckTimeoutInvokesOneCallbackOnly() throws Exception {
.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.
Expand Down Expand Up @@ -187,7 +190,7 @@ public void onFailure(Throwable t) {
throwable[0] = t;
callbackResult.set(true);
}
});
}, MoreExecutors.directExecutor());

while (!callbackResult.get()) {
// Wait for the callback
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -120,7 +120,8 @@ public Iterable<FileStatus> getFileStatuses() throws InterruptedException,
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();
Expand Down Expand Up @@ -267,7 +268,8 @@ public void onSuccess(ProcessInputDirCallable.Result result) {
ListenableFuture<ProcessInputDirCallable.Result> future = exec
.submit(new ProcessInputDirCallable(result.fs, fileStatus,
recursive, inputFilter));
Futures.addCallback(future, processInputDirCallback);
Futures.addCallback(future, processInputDirCallback,
MoreExecutors.directExecutor());
}
}
decrementRunningAndCheckCompletion();
Expand Down Expand Up @@ -353,7 +355,8 @@ public void onSuccess(ProcessInitialInputPathCallable.Result result) {
ListenableFuture<ProcessInputDirCallable.Result> future = exec
.submit(new ProcessInputDirCallable(result.fs, matched,
recursive, inputFilter));
Futures.addCallback(future, processInputDirCallback);
Futures.addCallback(future, processInputDirCallback,
MoreExecutors.directExecutor());
}
}
decrementRunningAndCheckCompletion();
Expand Down
2 changes: 1 addition & 1 deletion hadoop-project/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -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>

Expand Down
2 changes: 2 additions & 0 deletions hadoop-tools/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -68,3 +68,5 @@
</plugins>
</build>
</project>

<!-- THIS COMMENT IS FOR TRIGGER YETUS -->
Original file line number Diff line number Diff line change
Expand Up @@ -87,7 +87,7 @@ public static List<HostAndPort> splitToHostsAndPorts(String hostPortQuorumList)
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());
Expand Down