Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

treewide: update "guava" package #369

Closed
wants to merge 1 commit into from
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
3 changes: 1 addition & 2 deletions build.xml
Original file line number Diff line number Diff line change
Expand Up @@ -363,11 +363,10 @@
<dependency groupId="org.xerial.snappy" artifactId="snappy-java" version="1.1.10.4"/>
<dependency groupId="net.jpountz.lz4" artifactId="lz4" version="1.3.0"/>
<dependency groupId="com.ning" artifactId="compress-lzf" version="0.8.4"/>
<dependency groupId="com.google.guava" artifactId="guava" version="18.0">
<dependency groupId="com.google.guava" artifactId="guava" version="32.1.3-jre">
<exclusion groupId="com.google.code.findbugs" artifactId="jsr305" />
<exclusion groupId="org.codehaus.mojo" artifactId="animal-sniffer-annotations" />
<exclusion groupId="com.google.guava" artifactId="listenablefuture" />
<exclusion groupId="com.google.guava" artifactId="failureaccess" />
<exclusion groupId="org.checkerframework" artifactId="checker-qual" />
<exclusion groupId="com.google.errorprone" artifactId="error_prone_annotations" />
</dependency>
Expand Down
2 changes: 1 addition & 1 deletion src/java/org/apache/cassandra/io/sstable/Descriptor.java
Original file line number Diff line number Diff line change
Expand Up @@ -270,7 +270,7 @@ public static Pair<Descriptor, String> fromFilename(File directory, String name,
nexttok = tokenStack.pop();
// generation OR format type
SSTableFormat.Type fmt = SSTableFormat.Type.LEGACY;
if (!CharMatcher.DIGIT.matchesAllOf(nexttok))
if (!CharMatcher.inRange('0', '9').matchesAllOf(nexttok))
{
fmt = SSTableFormat.Type.validate(nexttok);
nexttok = tokenStack.pop();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -60,7 +60,7 @@ private Type(String name, SSTableFormat info)
{
//Since format comes right after generation
//we disallow formats with numeric names
assert !CharMatcher.DIGIT.matchesAllOf(name);
assert !CharMatcher.inRange('0', '9').matchesAllOf(name);

this.name = name;
this.info = info;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -154,7 +154,7 @@ public static InetAddressAndPort getByNameOverrideDefaults(String name, Integer
{
port = hap.getPort();
}
return getByAddressOverrideDefaults(InetAddress.getByName(hap.getHostText()), port);
return getByAddressOverrideDefaults(InetAddress.getByName(hap.getHost()), port);
}

public static InetAddressAndPort getByAddress(byte[] address) throws UnknownHostException
Expand Down
8 changes: 4 additions & 4 deletions src/java/org/apache/cassandra/repair/RepairJob.java
Original file line number Diff line number Diff line change
Expand Up @@ -87,7 +87,7 @@ public void run()
}
// When all snapshot complete, send validation requests
ListenableFuture<List<InetAddress>> allSnapshotTasks = Futures.allAsList(snapshotTasks);
validations = Futures.transform(allSnapshotTasks, new AsyncFunction<List<InetAddress>, List<TreeResponse>>()
validations = Futures.transformAsync(allSnapshotTasks, new AsyncFunction<List<InetAddress>, List<TreeResponse>>()
{
public ListenableFuture<List<TreeResponse>> apply(List<InetAddress> endpoints)
{
Expand All @@ -105,7 +105,7 @@ public ListenableFuture<List<TreeResponse>> apply(List<InetAddress> endpoints)
}

// When all validations complete, submit sync tasks
ListenableFuture<List<SyncStat>> syncResults = Futures.transform(validations, new AsyncFunction<List<TreeResponse>, List<SyncStat>>()
ListenableFuture<List<SyncStat>> syncResults = Futures.transformAsync(validations, new AsyncFunction<List<TreeResponse>, List<SyncStat>>()
{
public ListenableFuture<List<SyncStat>> apply(List<TreeResponse> trees)
{
Expand Down Expand Up @@ -228,7 +228,7 @@ public void onSuccess(TreeResponse result)

// failure is handled at root of job chain
public void onFailure(Throwable t) {}
});
}, taskExecutor);
currentTask = nextTask;
}
// start running tasks
Expand Down Expand Up @@ -285,7 +285,7 @@ public void onSuccess(TreeResponse result)

// failure is handled at root of job chain
public void onFailure(Throwable t) {}
});
}, taskExecutor);
currentTask = nextTask;
}
// start running tasks
Expand Down
8 changes: 4 additions & 4 deletions src/java/org/apache/cassandra/repair/RepairRunnable.java
Original file line number Diff line number Diff line change
Expand Up @@ -309,7 +309,7 @@ public void onFailure(Throwable t)
totalProgress,
message));
}
});
}, executor);
futures.add(session);
}

Expand All @@ -318,7 +318,7 @@ public void onFailure(Throwable t)
final Collection<Range<Token>> successfulRanges = new ArrayList<>();
final AtomicBoolean hasFailure = new AtomicBoolean();
final ListenableFuture<List<RepairSessionResult>> allSessions = Futures.successfulAsList(futures);
ListenableFuture anticompactionResult = Futures.transform(allSessions, new AsyncFunction<List<RepairSessionResult>, Object>()
ListenableFuture anticompactionResult = Futures.transformAsync(allSessions, new AsyncFunction<List<RepairSessionResult>, Object>()
{
@SuppressWarnings("unchecked")
public ListenableFuture apply(List<RepairSessionResult> results)
Expand All @@ -337,7 +337,7 @@ public ListenableFuture apply(List<RepairSessionResult> results)
}
return ActiveRepairService.instance.finishParentSession(parentSession, allNeighbors, successfulRanges);
}
});
}, executor);
Futures.addCallback(anticompactionResult, new FutureCallback<Object>()
{
public void onSuccess(Object result)
Expand Down Expand Up @@ -385,7 +385,7 @@ private void repairComplete()
}
executor.shutdownNow();
}
});
}, executor);
}

private void addRangeToNeighbors(List<Pair<Set<InetAddress>, ? extends Collection<Range<Token>>>> neighborRangeList, Range<Token> range, Set<InetAddress> neighbors)
Expand Down
2 changes: 1 addition & 1 deletion src/java/org/apache/cassandra/repair/RepairSession.java
Original file line number Diff line number Diff line change
Expand Up @@ -296,7 +296,7 @@ public void onFailure(Throwable t)
Tracing.traceRepair("Session completed with the following error: {}", t);
forceShutdown(t);
}
});
}, taskExecutor);
}

public void terminate()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -183,7 +183,7 @@ public void run()
failureDetector.unregisterFailureDetectionEventListener(task);
gossiper.unregister(task);
}
}, MoreExecutors.sameThreadExecutor());
}, MoreExecutors.directExecutor());
}

public synchronized void terminateSessions()
Expand Down
5 changes: 3 additions & 2 deletions src/java/org/apache/cassandra/service/StorageService.java
Original file line number Diff line number Diff line change
Expand Up @@ -41,6 +41,7 @@
import com.google.common.base.Predicate;
import com.google.common.collect.*;
import com.google.common.util.concurrent.*;
import com.google.common.util.concurrent.MoreExecutors;

import org.apache.commons.lang3.StringUtils;

Expand Down Expand Up @@ -1726,7 +1727,7 @@ public void onFailure(Throwable e)
progressSupport.progress("bootstrap", new ProgressEvent(ProgressEventType.ERROR, 1, 1, message));
progressSupport.progress("bootstrap", new ProgressEvent(ProgressEventType.COMPLETE, 1, 1, "Resume bootstrap complete"));
}
});
}, MoreExecutors.directExecutor());
return true;
}
else
Expand Down Expand Up @@ -2847,7 +2848,7 @@ public void onFailure(Throwable t)
// We still want to send the notification
sendReplicationNotification(notifyEndpoint);
}
});
}, MoreExecutors.directExecutor());
}

// needs to be modified to accept either a keyspace or ARS.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@

import com.google.common.util.concurrent.AbstractFuture;
import com.google.common.util.concurrent.Futures;
import com.google.common.util.concurrent.MoreExecutors;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

Expand Down Expand Up @@ -140,7 +141,7 @@ private void attachConnection(InetAddress from, int sessionIndex, IncomingStream

public void addEventListener(StreamEventHandler listener)
{
Futures.addCallback(this, listener);
Futures.addCallback(this, listener, MoreExecutors.directExecutor());
eventListeners.add(listener);
}

Expand Down
3 changes: 2 additions & 1 deletion test/unit/org/apache/cassandra/hints/HintsServiceTest.java
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@

import com.google.common.util.concurrent.Futures;
import com.google.common.util.concurrent.ListenableFuture;
import com.google.common.util.concurrent.MoreExecutors;
import org.junit.After;
import org.junit.Before;
import org.junit.BeforeClass;
Expand Down Expand Up @@ -125,7 +126,7 @@ public void onSuccess(@Nullable Boolean aBoolean)
{
HintsService.instance.resumeDispatch();
}
});
}, MoreExecutors.directExecutor());

Futures.allAsList(
noMessagesWhilePaused,
Expand Down
4 changes: 2 additions & 2 deletions test/unit/org/apache/cassandra/net/MockMessagingSpy.java
Original file line number Diff line number Diff line change
Expand Up @@ -54,7 +54,7 @@ public class MockMessagingSpy
*/
public ListenableFuture<MessageIn<?>> captureMockedMessageIn()
{
return Futures.transform(captureMockedMessageInN(1), (List<MessageIn<?>> result) -> result.isEmpty() ? null : result.get(0));
return Futures.transform(captureMockedMessageInN(1), (List<MessageIn<?>> result) -> result.isEmpty() ? null : result.get(0), executor);
}

/**
Expand Down Expand Up @@ -90,7 +90,7 @@ public ListenableFuture<Boolean> expectMockedMessageIn(int noOfMessages)
*/
public ListenableFuture<MessageOut<?>> captureMessageOut()
{
return Futures.transform(captureMessageOut(1), (List<MessageOut<?>> result) -> result.isEmpty() ? null : result.get(0));
return Futures.transform(captureMessageOut(1), (List<MessageOut<?>> result) -> result.isEmpty() ? null : result.get(0), executor);
}

/**
Expand Down
2 changes: 1 addition & 1 deletion test/unit/org/apache/cassandra/repair/RepairJobTest.java
Original file line number Diff line number Diff line change
Expand Up @@ -237,7 +237,7 @@ public void testNoTreesRetainedAfterDifference() throws Throwable
// SyncTasks themselves should not contain significant memory
assertTrue(ObjectSizes.measureDeep(syncTasks) < 0.8 * singleTreeSize);

ListenableFuture<List<SyncStat>> syncResults = Futures.transform(Futures.immediateFuture(mockTreeResponses), new AsyncFunction<List<TreeResponse>, List<SyncStat>>()
ListenableFuture<List<SyncStat>> syncResults = Futures.transformAsync(Futures.immediateFuture(mockTreeResponses), new AsyncFunction<List<TreeResponse>, List<SyncStat>>()
{
public ListenableFuture<List<SyncStat>> apply(List<TreeResponse> treeResponses)
{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@
import com.google.common.collect.Iterables;
import com.google.common.util.concurrent.FutureCallback;
import com.google.common.util.concurrent.Futures;
import com.google.common.util.concurrent.MoreExecutors;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.runner.RunWith;
Expand Down Expand Up @@ -125,7 +126,7 @@ public void onFailure(Throwable t)
{
fail();
}
});
}, MoreExecutors.directExecutor());
// should be complete immediately
futureResult.get(100, TimeUnit.MILLISECONDS);
}
Expand Down