From ef60c8f9888269b7366e60dff693213b31f83eb2 Mon Sep 17 00:00:00 2001 From: Dmitry Kropachev Date: Mon, 8 Dec 2025 22:05:44 -0400 Subject: [PATCH 1/2] Update guava to 33.3.1-jre It fixes ~8 vulnurabilities that comming from guava dependency. --- .../driver/core/GuavaCompatibility.java | 109 +----------------- .../core/AbstractReconnectionHandlerTest.java | 2 +- pom.xml | 4 +- 3 files changed, 8 insertions(+), 107 deletions(-) diff --git a/driver-core/src/main/java/com/datastax/driver/core/GuavaCompatibility.java b/driver-core/src/main/java/com/datastax/driver/core/GuavaCompatibility.java index 069f550b049..f5bda7d0919 100644 --- a/driver-core/src/main/java/com/datastax/driver/core/GuavaCompatibility.java +++ b/driver-core/src/main/java/com/datastax/driver/core/GuavaCompatibility.java @@ -17,8 +17,6 @@ import com.datastax.driver.core.exceptions.DriverInternalError; import com.google.common.base.Function; -import com.google.common.collect.BiMap; -import com.google.common.collect.Maps; import com.google.common.net.HostAndPort; import com.google.common.reflect.TypeToken; import com.google.common.util.concurrent.AsyncFunction; @@ -26,9 +24,6 @@ import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.MoreExecutors; -import java.lang.reflect.ParameterizedType; -import java.lang.reflect.Type; -import java.util.Map; import java.util.concurrent.Executor; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -36,13 +31,9 @@ /** * A compatibility layer to support a wide range of Guava versions. * - *

The driver is compatible with Guava 16.0.1 or higher, but Guava 20 introduced incompatible - * breaking changes in its API, that could in turn be breaking for legacy driver clients if we - * simply upgraded our dependency. We don't want to increment our major version "just" for Guava (we - * have other changes planned). + *

The driver is compatible with Guava 19.0 or higher. * - *

Therefore we depend on Guava 19, which has both the deprecated and the new APIs, and detect - * the actual version at runtime in order to call the relevant methods. + *

We detect the actual version at runtime in order to call the relevant methods. * *

This is a hack, and might not work with subsequent Guava releases; the real fix is to stop * exposing Guava in our public API. We'll address that in version 4 of the driver. @@ -188,84 +179,18 @@ public abstract ListenableFuture transformAsync( *

The method {@code HostAndPort.getHostText} has been replaced with {@code * HostAndPort.getHost} starting with Guava 20.0; it has been completely removed in Guava 22.0. */ - @SuppressWarnings("JavaReflectionMemberAccess") public String getHost(HostAndPort hostAndPort) { - try { - // Guava >= 20.0 - return (String) HostAndPort.class.getMethod("getHost").invoke(hostAndPort); - } catch (Exception e) { - // Guava < 22.0 - return hostAndPort.getHostText(); - } + // Guava >= 20.0 + return hostAndPort.getHost(); } private static GuavaCompatibility selectImplementation() { if (isGuava_19_0_OrHigher()) { logger.info("Detected Guava >= 19 in the classpath, using modern compatibility layer"); return new Version19OrHigher(); - } else if (isGuava_16_0_1_OrHigher()) { - logger.info("Detected Guava < 19 in the classpath, using legacy compatibility layer"); - return new Version18OrLower(); } else { throw new DriverInternalError( - "Detected incompatible version of Guava in the classpath. " - + "You need 16.0.1 or higher."); - } - } - - private static class Version18OrLower extends GuavaCompatibility { - - @Override - public ListenableFuture withFallback( - ListenableFuture input, final AsyncFunction fallback) { - return Futures.withFallback( - input, - new com.google.common.util.concurrent.FutureFallback() { - @Override - public ListenableFuture create(Throwable t) throws Exception { - return fallback.apply(t); - } - }); - } - - @Override - public ListenableFuture withFallback( - ListenableFuture input, - final AsyncFunction fallback, - Executor executor) { - return Futures.withFallback( - input, - new com.google.common.util.concurrent.FutureFallback() { - @Override - public ListenableFuture create(Throwable t) throws Exception { - return fallback.apply(t); - } - }, - executor); - } - - @Override - public ListenableFuture transformAsync( - ListenableFuture input, AsyncFunction function) { - return Futures.transform(input, function); - } - - @Override - public ListenableFuture transformAsync( - ListenableFuture input, - AsyncFunction function, - Executor executor) { - return Futures.transform(input, function, executor); - } - - @Override - public boolean isSupertypeOf(TypeToken target, TypeToken argument) { - return target.isAssignableFrom(argument); - } - - @Override - public Executor sameThreadExecutor() { - return MoreExecutors.sameThreadExecutor(); + "Detected incompatible version of Guava in the classpath. " + "You need 19.0 or higher."); } } @@ -319,30 +244,6 @@ private static boolean isGuava_19_0_OrHigher() { Executor.class); } - private static boolean isGuava_16_0_1_OrHigher() { - // Cheap check for < 16.0 - if (!methodExists(Maps.class, "asConverter", BiMap.class)) { - return false; - } - // More elaborate check to filter out 16.0, which has a bug in TypeToken. We need 16.0.1. - boolean resolved = false; - TypeToken> mapOfString = TypeTokens.mapOf(String.class, String.class); - Type type = mapOfString.getType(); - if (type instanceof ParameterizedType) { - ParameterizedType pType = (ParameterizedType) type; - Type[] types = pType.getActualTypeArguments(); - if (types.length == 2) { - TypeToken valueType = TypeToken.of(types[1]); - resolved = valueType.getRawType().equals(String.class); - } - } - if (!resolved) { - logger.debug( - "Detected Guava issue #1635 which indicates that version 16.0 is in the classpath"); - } - return resolved; - } - private static boolean methodExists( Class declaringClass, String methodName, Class... parameterTypes) { try { diff --git a/driver-core/src/test/java/com/datastax/driver/core/AbstractReconnectionHandlerTest.java b/driver-core/src/test/java/com/datastax/driver/core/AbstractReconnectionHandlerTest.java index 0f5b726f777..9d803333328 100644 --- a/driver-core/src/test/java/com/datastax/driver/core/AbstractReconnectionHandlerTest.java +++ b/driver-core/src/test/java/com/datastax/driver/core/AbstractReconnectionHandlerTest.java @@ -263,7 +263,7 @@ public void should_yield_to_another_running_handler() { */ @Test(groups = "unit") public void should_yield_to_another_handler_that_just_succeeded() { - future.set(Futures.immediateCheckedFuture(null)); + future.set(Futures.immediateFuture(null)); handler.start(); diff --git a/pom.xml b/pom.xml index bc618f4877a..6e8656a4478 100644 --- a/pom.xml +++ b/pom.xml @@ -55,7 +55,7 @@ 1.2.17 1.7.36 1.7.36 - 19.0 + 33.3.1-jre 4.1.127.Final netty-tcnative-boringssl-static 2.0.70.Final @@ -543,7 +543,7 @@ true - https://google.github.io/guava/releases/19.0/api/docs/ + https://google.github.io/guava/releases/33.3.1-jre/api/docs/ http://netty.io/4.0/api/ http://www.joda.org/joda-time/apidocs/ http://fasterxml.github.io/jackson-core/javadoc/2.8/ From 167f0d18599c8156b87372f2bdecdf9cd6f47501 Mon Sep 17 00:00:00 2001 From: Dmitry Kropachev Date: Fri, 12 Dec 2025 09:11:24 -0400 Subject: [PATCH 2/2] Drop GuavaCompatibility Driver do not support old guavas anymore, on modern guavas API look-alike, we don't need this layer anymore. --- clirr-ignores.xml | 6 + .../datastax/driver/core/AbstractSession.java | 7 +- .../driver/core/ChainedResultSetFuture.java | 7 +- .../com/datastax/driver/core/CloseFuture.java | 6 +- .../com/datastax/driver/core/Cluster.java | 23 +- .../com/datastax/driver/core/Connection.java | 27 +- .../driver/core/ControlConnection.java | 7 +- .../datastax/driver/core/EventDebouncer.java | 7 +- .../driver/core/GuavaCompatibility.java | 262 ------------------ .../driver/core/HostConnectionPool.java | 13 +- .../datastax/driver/core/PoolingOptions.java | 4 +- .../datastax/driver/core/QueryOptions.java | 7 +- .../datastax/driver/core/RequestHandler.java | 7 +- .../datastax/driver/core/SessionManager.java | 49 ++-- .../driver/core/utils/MoreFutures.java | 7 +- .../datastax/driver/core/AsyncQueryTest.java | 25 +- .../driver/core/AsyncResultSetTest.java | 7 +- .../driver/core/ConnectionReleaseTest.java | 7 +- .../driver/core/HostConnectionPoolTest.java | 12 +- .../driver/core/ReusedStreamIdTest.java | 7 +- .../driver/core/TimeoutStressTest.java | 7 +- .../com/datastax/driver/mapping/Mapper.java | 42 +-- .../datastax/driver/mapping/MethodMapper.java | 10 +- .../com/datastax/driver/mapping/Result.java | 8 +- .../driver/mapping/MapperAsyncResultTest.java | 8 +- .../driver/mapping/MapperAsyncTest.java | 28 +- .../datastax/driver/mapping/MapperTest.java | 5 +- driver-tests/osgi/README.md | 4 - faq/osgi/README.md | 16 +- 29 files changed, 223 insertions(+), 402 deletions(-) delete mode 100644 driver-core/src/main/java/com/datastax/driver/core/GuavaCompatibility.java diff --git a/clirr-ignores.xml b/clirr-ignores.xml index d7ff2a7c2ad..e405885f785 100644 --- a/clirr-ignores.xml +++ b/clirr-ignores.xml @@ -447,4 +447,10 @@ *one* False positive. Method is still present in parent interface (and was only introduced in ResultSet as a workaround for another clirr false positive) + + + 8001 + com/datastax/driver/core/GuavaCompatibility + Compatibility shim dropped now that we depend on modern Guava APIs directly. + diff --git a/driver-core/src/main/java/com/datastax/driver/core/AbstractSession.java b/driver-core/src/main/java/com/datastax/driver/core/AbstractSession.java index 61de3414fd2..e109c7a0fb5 100644 --- a/driver-core/src/main/java/com/datastax/driver/core/AbstractSession.java +++ b/driver-core/src/main/java/com/datastax/driver/core/AbstractSession.java @@ -16,7 +16,9 @@ package com.datastax.driver.core; import com.google.common.base.Function; +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.Uninterruptibles; import io.netty.util.concurrent.EventExecutor; import java.nio.ByteBuffer; @@ -114,7 +116,7 @@ public ListenableFuture prepareAsync(final RegularStatement s final CodecRegistry codecRegistry = getCluster().getConfiguration().getCodecRegistry(); ListenableFuture prepared = prepareAsync(statement.getQueryString(codecRegistry), statement.getOutgoingPayload()); - return GuavaCompatibility.INSTANCE.transform( + return Futures.transform( prepared, new Function() { @Override @@ -134,7 +136,8 @@ public PreparedStatement apply(PreparedStatement prepared) { return prepared; } - }); + }, + MoreExecutors.directExecutor()); } /** diff --git a/driver-core/src/main/java/com/datastax/driver/core/ChainedResultSetFuture.java b/driver-core/src/main/java/com/datastax/driver/core/ChainedResultSetFuture.java index 47d7767d332..1902411530c 100644 --- a/driver-core/src/main/java/com/datastax/driver/core/ChainedResultSetFuture.java +++ b/driver-core/src/main/java/com/datastax/driver/core/ChainedResultSetFuture.java @@ -17,6 +17,8 @@ import com.google.common.util.concurrent.AbstractFuture; import com.google.common.util.concurrent.FutureCallback; +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.MoreExecutors; import com.google.common.util.concurrent.Uninterruptibles; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; @@ -30,7 +32,7 @@ class ChainedResultSetFuture extends AbstractFuture implements Result void setSource(ResultSetFuture source) { if (this.isCancelled()) source.cancel(false); this.source = source; - GuavaCompatibility.INSTANCE.addCallback( + Futures.addCallback( source, new FutureCallback() { @Override @@ -42,7 +44,8 @@ public void onSuccess(ResultSet result) { public void onFailure(Throwable t) { ChainedResultSetFuture.this.setException(t); } - }); + }, + MoreExecutors.directExecutor()); } @Override diff --git a/driver-core/src/main/java/com/datastax/driver/core/CloseFuture.java b/driver-core/src/main/java/com/datastax/driver/core/CloseFuture.java index 9a6d4b5084e..d95fb95aaf4 100644 --- a/driver-core/src/main/java/com/datastax/driver/core/CloseFuture.java +++ b/driver-core/src/main/java/com/datastax/driver/core/CloseFuture.java @@ -18,6 +18,7 @@ import com.google.common.util.concurrent.AbstractFuture; import com.google.common.util.concurrent.FutureCallback; import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.MoreExecutors; import java.util.List; /** @@ -68,7 +69,7 @@ static class Forwarding extends CloseFuture { Forwarding(List futures) { this.futures = futures; - GuavaCompatibility.INSTANCE.addCallback( + Futures.addCallback( Futures.allAsList(futures), new FutureCallback>() { @Override @@ -80,7 +81,8 @@ public void onFailure(Throwable t) { public void onSuccess(List v) { Forwarding.this.onFuturesDone(); } - }); + }, + MoreExecutors.directExecutor()); } @Override diff --git a/driver-core/src/main/java/com/datastax/driver/core/Cluster.java b/driver-core/src/main/java/com/datastax/driver/core/Cluster.java index c323cdedf88..0cc335c9d70 100644 --- a/driver-core/src/main/java/com/datastax/driver/core/Cluster.java +++ b/driver-core/src/main/java/com/datastax/driver/core/Cluster.java @@ -119,8 +119,6 @@ public class Cluster implements Closeable { static { logDriverVersion(); - // Force initialization to fail fast if there is an issue detecting the version - GuavaCompatibility.init(); } @VisibleForTesting @@ -399,17 +397,19 @@ public ListenableFuture connectAsync(final String keyspace) { } else { final String useQuery = "USE " + keyspace; ListenableFuture keyspaceSet = - GuavaCompatibility.INSTANCE.transformAsync( + Futures.transformAsync( sessionInitialized, new AsyncFunction() { @Override public ListenableFuture apply(Session session) throws Exception { return session.executeAsync(useQuery); } - }); + }, + MoreExecutors.directExecutor()); ListenableFuture withErrorHandling = - GuavaCompatibility.INSTANCE.withFallback( + Futures.catchingAsync( keyspaceSet, + Throwable.class, new AsyncFunction() { @Override public ListenableFuture apply(Throwable t) throws Exception { @@ -427,8 +427,10 @@ public ListenableFuture apply(Throwable t) throws Exception { } throw Throwables.propagate(t); } - }); - return GuavaCompatibility.INSTANCE.transform(withErrorHandling, Functions.constant(session)); + }, + MoreExecutors.directExecutor()); + return Futures.transform( + withErrorHandling, Functions.constant(session), MoreExecutors.directExecutor()); } } @@ -2696,7 +2698,7 @@ public void run() { future.setResult(rs); } }, - GuavaCompatibility.INSTANCE.sameThreadExecutor()); + MoreExecutors.directExecutor()); } catch (Exception e) { logger.warn("Error while waiting for schema agreement", e); @@ -3136,7 +3138,7 @@ private ListenableFuture schedule(final ExceptionCatchingRunnable task) { @Override public void runMayThrow() throws Exception { ListenableFuture f = execute(task); - GuavaCompatibility.INSTANCE.addCallback( + Futures.addCallback( f, new FutureCallback() { @Override @@ -3148,7 +3150,8 @@ public void onSuccess(Object result) { public void onFailure(Throwable t) { future.setException(t); } - }); + }, + MoreExecutors.directExecutor()); } }, NEW_NODE_DELAY_SECONDS, diff --git a/driver-core/src/main/java/com/datastax/driver/core/Connection.java b/driver-core/src/main/java/com/datastax/driver/core/Connection.java index f0a98d2c4ce..f66bf012feb 100644 --- a/driver-core/src/main/java/com/datastax/driver/core/Connection.java +++ b/driver-core/src/main/java/com/datastax/driver/core/Connection.java @@ -47,6 +47,7 @@ import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.ListeningExecutorService; +import com.google.common.util.concurrent.MoreExecutors; import com.google.common.util.concurrent.SettableFuture; import com.google.common.util.concurrent.Uninterruptibles; import io.netty.bootstrap.Bootstrap; @@ -348,17 +349,18 @@ public void operationComplete(ChannelFuture future) throws Exception { factory.manager.configuration.getPoolingOptions().getInitializationExecutor(); ListenableFuture queryOptionsFuture = - GuavaCompatibility.INSTANCE.transformAsync( + Futures.transformAsync( channelReadyFuture, onChannelReady(protocolVersion, initExecutor), initExecutor); ListenableFuture initializeTransportFuture = - GuavaCompatibility.INSTANCE.transformAsync( + Futures.transformAsync( queryOptionsFuture, onOptionsReady(protocolVersion, initExecutor), initExecutor); // Fallback on initializeTransportFuture so we can properly propagate specific exceptions. ListenableFuture initFuture = - GuavaCompatibility.INSTANCE.withFallback( + Futures.catchingAsync( initializeTransportFuture, + Throwable.class, new AsyncFunction() { @Override public ListenableFuture apply(Throwable t) throws Exception { @@ -390,7 +392,7 @@ public ListenableFuture apply(Throwable t) throws Exception { initExecutor); // Ensure the connection gets closed if the caller cancels the returned future. - GuavaCompatibility.INSTANCE.addCallback( + Futures.addCallback( initFuture, new MoreFutures.FailureCallback() { @Override @@ -431,7 +433,8 @@ private static String extractMessage(Throwable t) { public ListenableFuture optionsQuery() { Future startupOptionsFuture = write(new Requests.Options()); - return GuavaCompatibility.INSTANCE.transformAsync(startupOptionsFuture, onSupportedResponse()); + return Futures.transformAsync( + startupOptionsFuture, onSupportedResponse(), MoreExecutors.directExecutor()); } private AsyncFunction onChannelReady( @@ -440,7 +443,7 @@ private AsyncFunction onChannelReady( @Override public ListenableFuture apply(Void input) throws Exception { Future startupOptionsFuture = write(new Requests.Options()); - return GuavaCompatibility.INSTANCE.transformAsync( + return Futures.transformAsync( startupOptionsFuture, onOptionsResponse(protocolVersion, initExecutor), initExecutor); } }; @@ -516,7 +519,7 @@ public ListenableFuture apply(Void input) throws Exception { write( new Requests.Startup( protocolOptions.getCompression(), protocolOptions.isNoCompact(), extraOptions)); - return GuavaCompatibility.INSTANCE.transformAsync( + return Futures.transformAsync( startupResponseFuture, onStartupResponse(protocolVersion, initExecutor), initExecutor); } }; @@ -624,7 +627,7 @@ private ListenableFuture checkClusterName( new Requests.Query("select cluster_name from system.local where key = 'local'")); try { write(clusterNameFuture); - return GuavaCompatibility.INSTANCE.transformAsync( + return Futures.transformAsync( clusterNameFuture, new AsyncFunction() { @Override @@ -663,7 +666,7 @@ private ListenableFuture authenticateV1( new Requests.Credentials(((ProtocolV1Authenticator) authenticator).getCredentials()); try { Future authResponseFuture = write(creds); - return GuavaCompatibility.INSTANCE.transformAsync( + return Futures.transformAsync( authResponseFuture, new AsyncFunction() { @Override @@ -699,7 +702,7 @@ private ListenableFuture authenticateV2( try { Future authResponseFuture = write(new Requests.AuthResponse(initialResponse)); - return GuavaCompatibility.INSTANCE.transformAsync( + return Futures.transformAsync( authResponseFuture, onV2AuthResponse(authenticator, protocolVersion, executor), executor); } catch (Exception e) { return Futures.immediateFailedFuture(e); @@ -730,7 +733,7 @@ public ListenableFuture apply(Message.Response authResponse) throws Except // Otherwise, send the challenge response back to the server logger.trace("{} Sending Auth response to challenge", this); Future nextResponseFuture = write(new Requests.AuthResponse(responseToServer)); - return GuavaCompatibility.INSTANCE.transformAsync( + return Futures.transformAsync( nextResponseFuture, onV2AuthResponse(authenticator, protocolVersion, executor), executor); @@ -898,7 +901,7 @@ ListenableFuture setKeyspaceAsync(final String keyspace) // Note: we quote the keyspace below, because the name is the one coming from Cassandra, so // it's in the right case already Future future = write(new Requests.Query("USE \"" + keyspace + '"')); - GuavaCompatibility.INSTANCE.addCallback( + Futures.addCallback( future, new FutureCallback() { diff --git a/driver-core/src/main/java/com/datastax/driver/core/ControlConnection.java b/driver-core/src/main/java/com/datastax/driver/core/ControlConnection.java index 5988708ff09..be8c7c88068 100644 --- a/driver-core/src/main/java/com/datastax/driver/core/ControlConnection.java +++ b/driver-core/src/main/java/com/datastax/driver/core/ControlConnection.java @@ -36,7 +36,9 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.Iterators; 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.SettableFuture; import java.net.InetAddress; import java.net.InetSocketAddress; @@ -721,7 +723,7 @@ private ListenableFuture selectPeersFuture(final Connection connectio connection.write(peersV2Future); final SettableFuture peersFuture = SettableFuture.create(); // if peers v2 query fails, query peers table instead. - GuavaCompatibility.INSTANCE.addCallback( + Futures.addCallback( peersV2Future, new FutureCallback() { @@ -745,7 +747,8 @@ public void onFailure(Throwable t) { peersFuture.setException(t); } } - }); + }, + MoreExecutors.directExecutor()); return peersFuture; } else { DefaultResultSetFuture peersFuture = diff --git a/driver-core/src/main/java/com/datastax/driver/core/EventDebouncer.java b/driver-core/src/main/java/com/datastax/driver/core/EventDebouncer.java index fc6cf90a71c..2d290024e5f 100644 --- a/driver-core/src/main/java/com/datastax/driver/core/EventDebouncer.java +++ b/driver-core/src/main/java/com/datastax/driver/core/EventDebouncer.java @@ -22,7 +22,9 @@ import com.datastax.driver.core.utils.MoreFutures; import com.google.common.collect.Lists; 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.SettableFuture; import java.util.List; import java.util.Queue; @@ -248,7 +250,7 @@ private void deliverEvents() { } else { logger.trace("{} debouncer: delivering {} events", name, toDeliver.size()); ListenableFuture delivered = callback.deliver(toDeliver); - GuavaCompatibility.INSTANCE.addCallback( + Futures.addCallback( delivered, new FutureCallback() { @Override @@ -260,7 +262,8 @@ public void onSuccess(Object result) { public void onFailure(Throwable t) { for (SettableFuture future : futures) future.setException(t); } - }); + }, + MoreExecutors.directExecutor()); } // If we didn't dequeue all events (or new ones arrived since we did), make sure we eventually diff --git a/driver-core/src/main/java/com/datastax/driver/core/GuavaCompatibility.java b/driver-core/src/main/java/com/datastax/driver/core/GuavaCompatibility.java deleted file mode 100644 index f5bda7d0919..00000000000 --- a/driver-core/src/main/java/com/datastax/driver/core/GuavaCompatibility.java +++ /dev/null @@ -1,262 +0,0 @@ -/* - * Copyright DataStax, Inc. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package com.datastax.driver.core; - -import com.datastax.driver.core.exceptions.DriverInternalError; -import com.google.common.base.Function; -import com.google.common.net.HostAndPort; -import com.google.common.reflect.TypeToken; -import com.google.common.util.concurrent.AsyncFunction; -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 java.util.concurrent.Executor; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * A compatibility layer to support a wide range of Guava versions. - * - *

The driver is compatible with Guava 19.0 or higher. - * - *

We detect the actual version at runtime in order to call the relevant methods. - * - *

This is a hack, and might not work with subsequent Guava releases; the real fix is to stop - * exposing Guava in our public API. We'll address that in version 4 of the driver. - */ -@SuppressWarnings("deprecation") -public abstract class GuavaCompatibility { - - private static final Logger logger = LoggerFactory.getLogger(GuavaCompatibility.class); - - /** - * The unique instance of this class, that is compatible with the Guava version found in the - * classpath. - */ - public static final GuavaCompatibility INSTANCE = selectImplementation(); - - /** - * Force the initialization of the class. This should be called early to ensure a fast failure if - * an incompatible version of Guava is in the classpath (the driver code calls it when loading the - * {@link Cluster} class). - */ - public static void init() { - // nothing to do, we just want the static initializers to run - } - - /** - * Returns a {@code Future} whose result is taken from the given primary {@code input} or, if the - * primary input fails, from the {@code Future} provided by the {@code fallback}. - * - * @see Futures#withFallback(ListenableFuture, com.google.common.util.concurrent.FutureFallback) - * @see Futures#catchingAsync(ListenableFuture, Class, AsyncFunction) - */ - public abstract ListenableFuture withFallback( - ListenableFuture input, AsyncFunction fallback); - - /** - * Returns a {@code Future} whose result is taken from the given primary {@code input} or, if the - * primary input fails, from the {@code Future} provided by the {@code fallback}. - * - * @see Futures#withFallback(ListenableFuture, com.google.common.util.concurrent.FutureFallback, - * Executor) - * @see Futures#catchingAsync(ListenableFuture, Class, AsyncFunction, Executor) - */ - public abstract ListenableFuture withFallback( - ListenableFuture input, AsyncFunction fallback, Executor executor); - - /** - * Registers separate success and failure callbacks to be run when the {@code Future}'s - * computation is {@linkplain java.util.concurrent.Future#isDone() complete} or, if the - * computation is already complete, immediately. - * - *

The callback is run in {@link #sameThreadExecutor()}. - * - * @see Futures#addCallback(ListenableFuture, FutureCallback, Executor) - */ - public void addCallback(ListenableFuture input, FutureCallback callback) { - addCallback(input, callback, sameThreadExecutor()); - } - - /** - * Registers separate success and failure callbacks to be run when the {@code Future}'s - * computation is {@linkplain java.util.concurrent.Future#isDone() complete} or, if the - * computation is already complete, immediately. - * - * @see Futures#addCallback(ListenableFuture, FutureCallback, Executor) - */ - public void addCallback( - ListenableFuture input, FutureCallback callback, Executor executor) { - Futures.addCallback(input, callback, executor); - } - - /** - * Returns a new {@code ListenableFuture} whose result is the product of applying the given {@code - * Function} to the result of the given {@code Future}. - * - *

The callback is run in {@link #sameThreadExecutor()}. - * - * @see Futures#transform(ListenableFuture, Function, Executor) - */ - public ListenableFuture transform( - ListenableFuture input, Function function) { - return transform(input, function, sameThreadExecutor()); - } - - /** - * Returns a new {@code ListenableFuture} whose result is the product of applying the given {@code - * Function} to the result of the given {@code Future}. - * - * @see Futures#transform(ListenableFuture, Function, Executor) - */ - public ListenableFuture transform( - ListenableFuture input, Function function, Executor executor) { - return Futures.transform(input, function, executor); - } - - /** - * Returns a new {@code ListenableFuture} whose result is asynchronously derived from the result - * of the given {@code Future}. More precisely, the returned {@code Future} takes its result from - * a {@code Future} produced by applying the given {@code AsyncFunction} to the result of the - * original {@code Future}. - * - * @see Futures#transform(ListenableFuture, AsyncFunction) - * @see Futures#transformAsync(ListenableFuture, AsyncFunction) - */ - public abstract ListenableFuture transformAsync( - ListenableFuture input, AsyncFunction function); - - /** - * Returns a new {@code ListenableFuture} whose result is asynchronously derived from the result - * of the given {@code Future}. More precisely, the returned {@code Future} takes its result from - * a {@code Future} produced by applying the given {@code AsyncFunction} to the result of the - * original {@code Future}. - * - * @see Futures#transform(ListenableFuture, AsyncFunction, Executor) - * @see Futures#transformAsync(ListenableFuture, AsyncFunction, Executor) - */ - public abstract ListenableFuture transformAsync( - ListenableFuture input, AsyncFunction function, Executor executor); - - /** - * Returns true if {@code target} is a supertype of {@code argument}. "Supertype" is defined - * according to the rules for type arguments introduced with Java generics. - * - * @see TypeToken#isAssignableFrom(Type) - * @see TypeToken#isSupertypeOf(Type) - */ - public abstract boolean isSupertypeOf(TypeToken target, TypeToken argument); - - /** - * Returns an {@link Executor} that runs each task in the thread that invokes {@link - * Executor#execute execute}, as in {@link - * java.util.concurrent.ThreadPoolExecutor.CallerRunsPolicy}. - * - * @see MoreExecutors#sameThreadExecutor() - * @see MoreExecutors#directExecutor() - */ - public abstract Executor sameThreadExecutor(); - - /** - * Returns the portion of the given {@link HostAndPort} instance that should represent the - * hostname or IPv4/IPv6 literal. - * - *

The method {@code HostAndPort.getHostText} has been replaced with {@code - * HostAndPort.getHost} starting with Guava 20.0; it has been completely removed in Guava 22.0. - */ - public String getHost(HostAndPort hostAndPort) { - // Guava >= 20.0 - return hostAndPort.getHost(); - } - - private static GuavaCompatibility selectImplementation() { - if (isGuava_19_0_OrHigher()) { - logger.info("Detected Guava >= 19 in the classpath, using modern compatibility layer"); - return new Version19OrHigher(); - } else { - throw new DriverInternalError( - "Detected incompatible version of Guava in the classpath. " + "You need 19.0 or higher."); - } - } - - private static class Version19OrHigher extends GuavaCompatibility { - - @Override - public ListenableFuture withFallback( - ListenableFuture input, AsyncFunction fallback) { - return withFallback(input, fallback, sameThreadExecutor()); - } - - @Override - public ListenableFuture withFallback( - ListenableFuture input, - AsyncFunction fallback, - Executor executor) { - return Futures.catchingAsync(input, Throwable.class, fallback, executor); - } - - @Override - public ListenableFuture transformAsync( - ListenableFuture input, AsyncFunction function) { - return transformAsync(input, function, sameThreadExecutor()); - } - - @Override - public ListenableFuture transformAsync( - ListenableFuture input, - AsyncFunction function, - Executor executor) { - return Futures.transformAsync(input, function, executor); - } - - @Override - public boolean isSupertypeOf(TypeToken target, TypeToken argument) { - return target.isSupertypeOf(argument); - } - - @Override - public Executor sameThreadExecutor() { - return MoreExecutors.directExecutor(); - } - } - - private static boolean isGuava_19_0_OrHigher() { - return methodExists( - Futures.class, - "transformAsync", - ListenableFuture.class, - AsyncFunction.class, - Executor.class); - } - - private static boolean methodExists( - Class declaringClass, String methodName, Class... parameterTypes) { - try { - declaringClass.getMethod(methodName, parameterTypes); - return true; - } catch (Exception e) { - logger.debug( - "Error while checking existence of method " - + declaringClass.getSimpleName() - + "." - + methodName, - e); - return false; - } - } -} diff --git a/driver-core/src/main/java/com/datastax/driver/core/HostConnectionPool.java b/driver-core/src/main/java/com/datastax/driver/core/HostConnectionPool.java index 14954b3b5dc..a27a70de265 100644 --- a/driver-core/src/main/java/com/datastax/driver/core/HostConnectionPool.java +++ b/driver-core/src/main/java/com/datastax/driver/core/HostConnectionPool.java @@ -38,6 +38,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.SettableFuture; import com.google.common.util.concurrent.Uninterruptibles; import io.netty.util.concurrent.EventExecutor; @@ -372,7 +373,7 @@ private void addCallback( manager.cluster.manager.configuration.getPoolingOptions().getInitializationExecutor(); final ListenableFuture> allConnectionsFuture = Futures.allAsList(connectionFutures); - GuavaCompatibility.INSTANCE.addCallback( + Futures.addCallback( allConnectionsFuture, new FutureCallback>() { @Override @@ -449,8 +450,9 @@ public void onFailure(Throwable t) { private ListenableFuture handleErrors( ListenableFuture connectionInitFuture, Executor executor) { - return GuavaCompatibility.INSTANCE.withFallback( + return Futures.catchingAsync( connectionInitFuture, + Throwable.class, new AsyncFunction() { @Override public ListenableFuture apply(Throwable t) throws Exception { @@ -708,7 +710,7 @@ private void dequeue(final Connection connection) { } else { // Otherwise the keyspace did need to be set, tie the pendingBorrow future to the set // keyspace completion. - GuavaCompatibility.INSTANCE.addCallback( + Futures.addCallback( setKeyspaceFuture, new FutureCallback() { @@ -726,7 +728,8 @@ public void onFailure(Throwable t) { pendingBorrow.setException(t); connection.inFlight.decrementAndGet(); } - }); + }, + MoreExecutors.directExecutor()); } } } @@ -1033,7 +1036,7 @@ public void run() { } } }, - GuavaCompatibility.INSTANCE.sameThreadExecutor()); + MoreExecutors.directExecutor()); futures.add(future); } } diff --git a/driver-core/src/main/java/com/datastax/driver/core/PoolingOptions.java b/driver-core/src/main/java/com/datastax/driver/core/PoolingOptions.java index 3741ef97ec0..03b7c5e87b1 100644 --- a/driver-core/src/main/java/com/datastax/driver/core/PoolingOptions.java +++ b/driver-core/src/main/java/com/datastax/driver/core/PoolingOptions.java @@ -20,6 +20,7 @@ import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableMap; +import com.google.common.util.concurrent.MoreExecutors; import java.util.Map; import java.util.concurrent.Executor; @@ -134,8 +135,7 @@ public class PoolingOptions { /** The default value for {@link #getHeartbeatIntervalSeconds()} ({@value}). */ public static final int DEFAULT_HEARTBEAT_INTERVAL_SECONDS = 30; - private static final Executor DEFAULT_INITIALIZATION_EXECUTOR = - GuavaCompatibility.INSTANCE.sameThreadExecutor(); + private static final Executor DEFAULT_INITIALIZATION_EXECUTOR = MoreExecutors.directExecutor(); private volatile Cluster.Manager manager; private volatile ProtocolVersion protocolVersion; diff --git a/driver-core/src/main/java/com/datastax/driver/core/QueryOptions.java b/driver-core/src/main/java/com/datastax/driver/core/QueryOptions.java index 3fa3e335b51..da3a4b0aaee 100644 --- a/driver-core/src/main/java/com/datastax/driver/core/QueryOptions.java +++ b/driver-core/src/main/java/com/datastax/driver/core/QueryOptions.java @@ -18,6 +18,8 @@ import com.datastax.driver.core.exceptions.UnsupportedFeatureException; import com.datastax.driver.core.utils.MoreFutures; import com.datastax.driver.core.utils.MoreObjects; +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.MoreExecutors; /** Options related to defaults for individual queries. */ public class QueryOptions { @@ -335,7 +337,7 @@ public QueryOptions setMetadataEnabled(boolean enabled) { // 1. call submitNodeListRefresh() first to // be able to compute the token map for the first time, // which will be incomplete due to the lack of keyspace metadata - GuavaCompatibility.INSTANCE.addCallback( + Futures.addCallback( manager.submitNodeListRefresh(), new MoreFutures.SuccessCallback() { @Override @@ -345,7 +347,8 @@ public void onSuccess(Void result) { // this time with information about keyspaces manager.submitSchemaRefresh(null, null, null, null); } - }); + }, + MoreExecutors.directExecutor()); } return this; } diff --git a/driver-core/src/main/java/com/datastax/driver/core/RequestHandler.java b/driver-core/src/main/java/com/datastax/driver/core/RequestHandler.java index ba23b4bbb35..27f6b67d184 100644 --- a/driver-core/src/main/java/com/datastax/driver/core/RequestHandler.java +++ b/driver-core/src/main/java/com/datastax/driver/core/RequestHandler.java @@ -44,7 +44,9 @@ import com.google.common.collect.Iterators; 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 io.netty.util.Timeout; import io.netty.util.TimerTask; import java.nio.ByteBuffer; @@ -470,7 +472,7 @@ private boolean query(final Host host) { routingKey, statementKeyspace, statementTable); - GuavaCompatibility.INSTANCE.addCallback( + Futures.addCallback( connectionFuture, new FutureCallback() { @Override @@ -522,7 +524,8 @@ public void onFailure(Throwable t) { } findNextHostAndQuery(); } - }); + }, + MoreExecutors.directExecutor()); return true; } diff --git a/driver-core/src/main/java/com/datastax/driver/core/SessionManager.java b/driver-core/src/main/java/com/datastax/driver/core/SessionManager.java index 54cceda6126..12cd2970394 100644 --- a/driver-core/src/main/java/com/datastax/driver/core/SessionManager.java +++ b/driver-core/src/main/java/com/datastax/driver/core/SessionManager.java @@ -38,6 +38,7 @@ import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.ListeningExecutorService; +import com.google.common.util.concurrent.MoreExecutors; import com.google.common.util.concurrent.SettableFuture; import com.google.common.util.concurrent.Uninterruptibles; import java.nio.ByteBuffer; @@ -99,7 +100,7 @@ public ListenableFuture initAsync() { Collection hosts = cluster.getMetadata().allHosts(); ListenableFuture allPoolsCreatedFuture = createPools(hosts); ListenableFuture allPoolsUpdatedFuture = - GuavaCompatibility.INSTANCE.transformAsync( + Futures.transformAsync( allPoolsCreatedFuture, new AsyncFunction() { @Override @@ -108,9 +109,10 @@ public ListenableFuture apply(Object input) throws Exception { isInit = true; return (ListenableFuture) updateCreatedPools(); } - }); + }, + MoreExecutors.directExecutor()); - GuavaCompatibility.INSTANCE.addCallback( + Futures.addCallback( allPoolsUpdatedFuture, new FutureCallback() { @Override @@ -123,7 +125,8 @@ public void onFailure(Throwable t) { SessionManager.this.closeAsync(); // don't leak the session myInitFuture.setException(t); } - }); + }, + MoreExecutors.directExecutor()); return myInitFuture; } @@ -219,7 +222,7 @@ public Session.State getState() { private ListenableFuture toPreparedStatement( final String query, final Connection.Future future) { - return GuavaCompatibility.INSTANCE.transformAsync( + return Futures.transformAsync( future, new AsyncFunction() { @Override @@ -309,7 +312,7 @@ ListenableFuture forceRenewPool(final Host host, Connection reusedConne final SettableFuture future = SettableFuture.create(); - GuavaCompatibility.INSTANCE.addCallback( + Futures.addCallback( poolInitFuture, new FutureCallback() { @Override @@ -337,7 +340,8 @@ public void onFailure(Throwable t) { logger.warn("Error creating pool to " + host, t); future.set(false); } - }); + }, + MoreExecutors.directExecutor()); return future; } @@ -369,7 +373,7 @@ private ListenableFuture replacePool( ListenableFuture poolInitFuture = newPool.initAsync(reusedConnection); - GuavaCompatibility.INSTANCE.addCallback( + Futures.addCallback( poolInitFuture, new FutureCallback() { @Override @@ -385,7 +389,8 @@ public void onSuccess(Void result) { public void onFailure(Throwable t) { pools.remove(host); } - }); + }, + MoreExecutors.directExecutor()); return poolInitFuture; } @@ -404,7 +409,7 @@ ListenableFuture maybeAddPool(final Host host, Connection reusedConnect final SettableFuture future = SettableFuture.create(); ListenableFuture newPoolInit = replacePool(host, distance, previous, reusedConnection); if (newPoolInit != null) { - GuavaCompatibility.INSTANCE.addCallback( + Futures.addCallback( newPoolInit, new FutureCallback() { @Override @@ -435,7 +440,8 @@ public void onFailure(Throwable t) { if (t instanceof Error) future.setException(t); else future.set(false); } - }); + }, + MoreExecutors.directExecutor()); return future; } } @@ -488,7 +494,7 @@ ListenableFuture updateCreatedPools() { // Wait pool creation before removing, so we don't lose connectivity ListenableFuture allPoolsCreatedFuture = Futures.allAsList(poolCreatedFutures); - return GuavaCompatibility.INSTANCE.transformAsync( + return Futures.transformAsync( allPoolsCreatedFuture, new AsyncFunction>() { @Override @@ -499,7 +505,8 @@ public ListenableFuture> apply(Object input) throws Exception { return Futures.successfulAsList(poolRemovedFuture); } - }); + }, + MoreExecutors.directExecutor()); } void updateCreatedPools(Host h) { @@ -756,13 +763,13 @@ private ListenableFuture prepare( statement.getQueryKeyspace(), statementTable); ListenableFuture prepareFuture = - GuavaCompatibility.INSTANCE.transformAsync( + Futures.transformAsync( connectionFuture, new AsyncFunction() { @Override public ListenableFuture apply(final Connection c) throws Exception { Connection.Future responseFuture = c.write(new Requests.Prepare(query)); - GuavaCompatibility.INSTANCE.addCallback( + Futures.addCallback( responseFuture, new FutureCallback() { @Override @@ -779,18 +786,22 @@ public void onFailure(Throwable t) { t); c.release(); } - }); + }, + MoreExecutors.directExecutor()); return responseFuture; } - }); + }, + MoreExecutors.directExecutor()); futures.add(prepareFuture); } catch (Exception e) { // Again, not being able to prepare the query right now is no big deal, so just ignore } } // Return the statement when all futures are done - return GuavaCompatibility.INSTANCE.transform( - Futures.successfulAsList(futures), Functions.constant(statement)); + return Futures.transform( + Futures.successfulAsList(futures), + Functions.constant(statement), + MoreExecutors.directExecutor()); } ResultSetFuture executeQuery(Message.Request msg, Statement statement) { diff --git a/driver-core/src/main/java/com/datastax/driver/core/utils/MoreFutures.java b/driver-core/src/main/java/com/datastax/driver/core/utils/MoreFutures.java index 5cbbe38c007..d3f7b549293 100644 --- a/driver-core/src/main/java/com/datastax/driver/core/utils/MoreFutures.java +++ b/driver-core/src/main/java/com/datastax/driver/core/utils/MoreFutures.java @@ -15,10 +15,10 @@ */ package com.datastax.driver.core.utils; -import com.datastax.driver.core.GuavaCompatibility; 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.SettableFuture; /** Helpers to work with Guava's {@link ListenableFuture}. */ @@ -51,7 +51,7 @@ public void onSuccess(V result) { */ public static void propagateFuture( final SettableFuture settable, ListenableFuture future) { - GuavaCompatibility.INSTANCE.addCallback( + Futures.addCallback( future, new FutureCallback() { @Override @@ -63,6 +63,7 @@ public void onSuccess(T result) { public void onFailure(Throwable t) { settable.setException(t); } - }); + }, + MoreExecutors.directExecutor()); } } diff --git a/driver-core/src/test/java/com/datastax/driver/core/AsyncQueryTest.java b/driver-core/src/test/java/com/datastax/driver/core/AsyncQueryTest.java index 30a1e981b76..d8e869ff45d 100644 --- a/driver-core/src/test/java/com/datastax/driver/core/AsyncQueryTest.java +++ b/driver-core/src/test/java/com/datastax/driver/core/AsyncQueryTest.java @@ -31,7 +31,9 @@ import com.google.common.base.Function; import com.google.common.base.Throwables; import com.google.common.util.concurrent.AsyncFunction; +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.Uninterruptibles; import java.nio.ByteBuffer; import java.util.Collection; @@ -118,7 +120,7 @@ public void should_init_cluster_and_session_if_needed() throws Exception { public void should_chain_query_on_async_session_init_with_same_executor(String keyspace) throws Exception { ListenableFuture resultFuture = - connectAndQuery(keyspace, GuavaCompatibility.INSTANCE.sameThreadExecutor()); + connectAndQuery(keyspace, MoreExecutors.directExecutor()); Integer result = Uninterruptibles.getUninterruptibly(resultFuture); assertThat(result).isEqualTo(1); @@ -140,7 +142,7 @@ public void should_chain_query_on_async_session_init_with_different_executor(Str @Test(groups = "short") public void should_propagate_error_to_chained_query_if_session_init_fails() throws Exception { ListenableFuture resultFuture = - connectAndQuery("wrong_keyspace", GuavaCompatibility.INSTANCE.sameThreadExecutor()); + connectAndQuery("wrong_keyspace", MoreExecutors.directExecutor()); try { Uninterruptibles.getUninterruptibly(resultFuture); @@ -157,7 +159,7 @@ public void should_fail_when_synchronous_call_on_io_thread() throws Exception { ResultSetFuture f = session().executeAsync("select release_version from system.local where key='local'"); ListenableFuture f2 = - GuavaCompatibility.INSTANCE.transform( + Futures.transform( f, new Function() { @Override @@ -165,7 +167,8 @@ public Thread apply(ResultSet input) { session().execute("select release_version from system.local where key='local'"); return Thread.currentThread(); } - }); + }, + MoreExecutors.directExecutor()); if (isFailed( f2, IllegalStateException.class, "Detected a synchronous call on an I/O thread")) { return; @@ -182,7 +185,7 @@ public void should_fail_when_synchronous_call_on_io_thread_with_session_wrapper( ResultSetFuture f = session.executeAsync("select release_version from system.local where key='local'"); ListenableFuture f2 = - GuavaCompatibility.INSTANCE.transform( + Futures.transform( f, new Function() { @Override @@ -190,7 +193,8 @@ public Thread apply(ResultSet input) { session.execute("select release_version from system.local where key='local'"); return Thread.currentThread(); } - }); + }, + MoreExecutors.directExecutor()); if (isFailed( f2, IllegalStateException.class, "Detected a synchronous call on an I/O thread")) { return; @@ -208,7 +212,7 @@ public void should_fail_when_auto_paging_on_io_thread() throws Exception { statement.setFetchSize(10); ResultSetFuture f = session().executeAsync(statement); ListenableFuture f2 = - GuavaCompatibility.INSTANCE.transform( + Futures.transform( f, new Function() { @Override @@ -216,7 +220,8 @@ public Thread apply(ResultSet rs) { rs.all(); // Consume the whole result set return Thread.currentThread(); } - }); + }, + MoreExecutors.directExecutor()); if (isFailed( f2, IllegalStateException.class, "Detected a synchronous call on an I/O thread")) { return; @@ -249,7 +254,7 @@ private boolean isFailed( private ListenableFuture connectAndQuery(String keyspace, Executor executor) { ListenableFuture sessionFuture = cluster().connectAsync(keyspace); ListenableFuture queryFuture = - GuavaCompatibility.INSTANCE.transformAsync( + Futures.transformAsync( sessionFuture, new AsyncFunction() { @Override @@ -258,7 +263,7 @@ public ListenableFuture apply(Session session) throws Exception { } }, executor); - return GuavaCompatibility.INSTANCE.transform( + return Futures.transform( queryFuture, new Function() { @Override diff --git a/driver-core/src/test/java/com/datastax/driver/core/AsyncResultSetTest.java b/driver-core/src/test/java/com/datastax/driver/core/AsyncResultSetTest.java index 4381702e5c1..110a2b1abde 100644 --- a/driver-core/src/test/java/com/datastax/driver/core/AsyncResultSetTest.java +++ b/driver-core/src/test/java/com/datastax/driver/core/AsyncResultSetTest.java @@ -21,6 +21,7 @@ import com.google.common.util.concurrent.AsyncFunction; import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; +import com.google.common.util.concurrent.MoreExecutors; import java.util.Set; import java.util.concurrent.ConcurrentSkipListSet; import org.testng.annotations.BeforeMethod; @@ -57,7 +58,8 @@ private void should_iterate_result_set_asynchronously(int totalCount, int fetchS ResultsAccumulator results = new ResultsAccumulator(); ListenableFuture future = - GuavaCompatibility.INSTANCE.transformAsync(session().executeAsync(statement), results); + Futures.transformAsync( + session().executeAsync(statement), results, MoreExecutors.directExecutor()); Futures.getUnchecked(future); @@ -77,7 +79,8 @@ public ListenableFuture apply(ResultSet rs) throws Exception { } boolean wasLastPage = rs.getExecutionInfo().getPagingState() == null; if (wasLastPage) return Futures.immediateFuture(rs); - else return GuavaCompatibility.INSTANCE.transformAsync(rs.fetchMoreResults(), this); + else + return Futures.transformAsync(rs.fetchMoreResults(), this, MoreExecutors.directExecutor()); } } } diff --git a/driver-core/src/test/java/com/datastax/driver/core/ConnectionReleaseTest.java b/driver-core/src/test/java/com/datastax/driver/core/ConnectionReleaseTest.java index 131fda2e175..5a4f81bab49 100644 --- a/driver-core/src/test/java/com/datastax/driver/core/ConnectionReleaseTest.java +++ b/driver-core/src/test/java/com/datastax/driver/core/ConnectionReleaseTest.java @@ -22,7 +22,9 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import com.google.common.util.concurrent.AsyncFunction; +import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; +import com.google.common.util.concurrent.MoreExecutors; import java.util.Collection; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; @@ -96,7 +98,7 @@ public void should_release_connection_before_completing_future() throws Exceptio mockFutures.add(session.executeAsync("mock query")); ListenableFuture future = - GuavaCompatibility.INSTANCE.transformAsync( + Futures.transformAsync( session.executeAsync("select c from test1 where k=1"), new AsyncFunction() { @Override @@ -108,7 +110,8 @@ public ListenableFuture apply(ResultSet result) { // was not release. return session.executeAsync("select n from test2 where c='" + c + "'"); } - }); + }, + MoreExecutors.directExecutor()); long waitTimeInMs = 2000; try { diff --git a/driver-core/src/test/java/com/datastax/driver/core/HostConnectionPoolTest.java b/driver-core/src/test/java/com/datastax/driver/core/HostConnectionPoolTest.java index 322590eda69..6147b7d0f09 100644 --- a/driver-core/src/test/java/com/datastax/driver/core/HostConnectionPoolTest.java +++ b/driver-core/src/test/java/com/datastax/driver/core/HostConnectionPoolTest.java @@ -57,8 +57,10 @@ import com.google.common.base.Throwables; import com.google.common.util.concurrent.ForwardingListeningExecutorService; 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.ListeningExecutorService; +import com.google.common.util.concurrent.MoreExecutors; import com.google.common.util.concurrent.Uninterruptibles; import java.net.InetSocketAddress; import java.nio.ByteBuffer; @@ -1598,7 +1600,7 @@ private MockRequest(HostConnectionPool pool, int timeoutMillis, int maxQueueSize pool.borrowConnection( timeoutMillis, MILLISECONDS, maxQueueSize, null, routingKey, null, null); requestInitialized = - GuavaCompatibility.INSTANCE.transform( + Futures.transform( this.connectionFuture, new Function() { @Override @@ -1609,7 +1611,8 @@ public Connection.ResponseHandler apply(Connection connection) { connection.dispatcher.add(thisRequest.responseHandler); return responseHandler; } - }); + }, + MoreExecutors.directExecutor()); } void simulateSuccessResponse() { @@ -1703,7 +1706,7 @@ public void blockUntilNextTaskCompleted() throws InterruptedException { @Override public ListenableFuture submit(Runnable task) { ListenableFuture future = super.submit(task); - GuavaCompatibility.INSTANCE.addCallback( + Futures.addCallback( future, new FutureCallback() { @Override @@ -1715,7 +1718,8 @@ public void onSuccess(Object result) { public void onFailure(Throwable t) { semaphore.release(1); } - }); + }, + MoreExecutors.directExecutor()); return future; } } diff --git a/driver-core/src/test/java/com/datastax/driver/core/ReusedStreamIdTest.java b/driver-core/src/test/java/com/datastax/driver/core/ReusedStreamIdTest.java index 79f06ab64fe..79f4076280b 100644 --- a/driver-core/src/test/java/com/datastax/driver/core/ReusedStreamIdTest.java +++ b/driver-core/src/test/java/com/datastax/driver/core/ReusedStreamIdTest.java @@ -21,6 +21,8 @@ import com.datastax.driver.core.exceptions.OperationTimedOutException; import com.google.common.util.concurrent.FutureCallback; +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.MoreExecutors; import com.google.common.util.concurrent.Uninterruptibles; import java.util.List; import java.util.Random; @@ -94,7 +96,7 @@ public void should_not_receive_wrong_response_when_callbacks_block_io_thread() { String query = String.format("select %s from system.local where key='local'", column); ResultSetFuture future = session().executeAsync(query); - GuavaCompatibility.INSTANCE.addCallback( + Futures.addCallback( future, new FutureCallback() { @Override @@ -131,7 +133,8 @@ public void onFailure(Throwable t) { errorTrigger.countDown(); } } - }); + }, + MoreExecutors.directExecutor()); } catch (InterruptedException e) { fail("Test interrupted", e); } diff --git a/driver-core/src/test/java/com/datastax/driver/core/TimeoutStressTest.java b/driver-core/src/test/java/com/datastax/driver/core/TimeoutStressTest.java index 41692dfed38..4733374c564 100644 --- a/driver-core/src/test/java/com/datastax/driver/core/TimeoutStressTest.java +++ b/driver-core/src/test/java/com/datastax/driver/core/TimeoutStressTest.java @@ -21,6 +21,8 @@ import com.datastax.driver.core.policies.ConstantReconnectionPolicy; import com.datastax.driver.core.utils.SocketChannelMonitor; import com.google.common.util.concurrent.FutureCallback; +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.MoreExecutors; import com.google.common.util.concurrent.ThreadFactoryBuilder; import com.google.common.util.concurrent.Uninterruptibles; import io.netty.channel.socket.SocketChannel; @@ -242,7 +244,7 @@ public void run() { try { concurrentQueries.acquire(); ResultSetFuture future = session.executeAsync(statement.bind("0")); - GuavaCompatibility.INSTANCE.addCallback( + Futures.addCallback( future, new FutureCallback() { @@ -265,7 +267,8 @@ public void onFailure(Throwable t) { // logger.error("Exception", t); } } - }); + }, + MoreExecutors.directExecutor()); } catch (Exception e) { logger.error("Failure while submitting query.", e); } diff --git a/driver-mapping/src/main/java/com/datastax/driver/mapping/Mapper.java b/driver-mapping/src/main/java/com/datastax/driver/mapping/Mapper.java index c6792634dfd..0cd56ee627a 100644 --- a/driver-mapping/src/main/java/com/datastax/driver/mapping/Mapper.java +++ b/driver-mapping/src/main/java/com/datastax/driver/mapping/Mapper.java @@ -21,7 +21,6 @@ import com.datastax.driver.core.AbstractSession; import com.datastax.driver.core.BoundStatement; import com.datastax.driver.core.ConsistencyLevel; -import com.datastax.driver.core.GuavaCompatibility; import com.datastax.driver.core.KeyspaceMetadata; import com.datastax.driver.core.PreparedStatement; import com.datastax.driver.core.ProtocolVersion; @@ -46,7 +45,9 @@ import com.google.common.collect.ImmutableSet; import com.google.common.util.concurrent.AsyncFunction; 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.SettableFuture; import com.google.common.util.concurrent.Uninterruptibles; import java.util.ArrayList; @@ -149,7 +150,7 @@ ListenableFuture getPreparedQueryAsync( SimpleStatement s = new SimpleStatement(queryString); // all queries generated by the mapper are idempotent s.setIdempotent(true); - GuavaCompatibility.INSTANCE.addCallback( + Futures.addCallback( session().prepareAsync(s), new FutureCallback() { @Override @@ -164,7 +165,8 @@ public void onFailure(Throwable t) { preparedQueries.remove(pqk, future); logger.error("Query preparation failed: " + queryString, t); } - }); + }, + MoreExecutors.directExecutor()); return future; } } else { @@ -268,7 +270,7 @@ private ListenableFuture saveQueryAsync( } } - return GuavaCompatibility.INSTANCE.transform( + return Futures.transform( getPreparedQueryAsync(QueryType.SAVE, columnToValue.keySet(), options), new Function() { @Override @@ -290,7 +292,8 @@ public BoundStatement apply(PreparedStatement input) { return bs; } - }); + }, + MoreExecutors.directExecutor()); } private static boolean shouldSaveNullFields(EnumMap options) { @@ -386,15 +389,16 @@ public ListenableFuture saveAsync(T entity, Option... options) { private ListenableFuture submitVoidQueryAsync(ListenableFuture bsFuture) { ListenableFuture rsFuture = - GuavaCompatibility.INSTANCE.transformAsync( + Futures.transformAsync( bsFuture, new AsyncFunction() { @Override public ListenableFuture apply(BoundStatement bs) throws Exception { return session().executeAsync(bs); } - }); - return GuavaCompatibility.INSTANCE.transform(rsFuture, TO_NULL); + }, + MoreExecutors.directExecutor()); + return Futures.transform(rsFuture, TO_NULL, MoreExecutors.directExecutor()); } /** @@ -457,7 +461,7 @@ private ListenableFuture getQueryAsync( "Invalid number of PRIMARY KEY columns provided, %d expected but got %d", mapper.primaryKeySize(), primaryKeys.size())); - return GuavaCompatibility.INSTANCE.transform( + return Futures.transform( getPreparedQueryAsync(QueryType.GET, options), new Function() { @Override @@ -483,7 +487,8 @@ public BoundStatement apply(PreparedStatement input) { } return bs; } - }); + }, + MoreExecutors.directExecutor()); } /** @@ -529,15 +534,16 @@ public T get(Object... objects) { public ListenableFuture getAsync(final Object... objects) { ListenableFuture bsFuture = getQueryAsync(objects); ListenableFuture rsFuture = - GuavaCompatibility.INSTANCE.transformAsync( + Futures.transformAsync( bsFuture, new AsyncFunction() { @Override public ListenableFuture apply(BoundStatement bs) throws Exception { return session().executeAsync(bs); } - }); - return GuavaCompatibility.INSTANCE.transform(rsFuture, mapOneFunction); + }, + MoreExecutors.directExecutor()); + return Futures.transform(rsFuture, mapOneFunction, MoreExecutors.directExecutor()); } /** @@ -676,7 +682,7 @@ private ListenableFuture deleteQueryAsync( "Invalid number of PRIMARY KEY columns provided, %d expected but got %d", mapper.primaryKeySize(), primaryKey.size())); - return GuavaCompatibility.INSTANCE.transform( + return Futures.transform( getPreparedQueryAsync(QueryType.DEL, options), new Function() { @Override @@ -705,7 +711,8 @@ public BoundStatement apply(PreparedStatement input) { } return bs; } - }); + }, + MoreExecutors.directExecutor()); } /** @@ -852,14 +859,15 @@ public Result map(ResultSet resultSet) { * returned mapped result set will consume results from that result set and vice-versa. */ public ListenableFuture> mapAsync(ResultSetFuture resultSetFuture) { - return GuavaCompatibility.INSTANCE.transform( + return Futures.transform( resultSetFuture, new Function>() { @Override public Result apply(ResultSet rs) { return map(rs); } - }); + }, + MoreExecutors.directExecutor()); } private boolean isFromMapperQuery(ResultSet resultSet) { diff --git a/driver-mapping/src/main/java/com/datastax/driver/mapping/MethodMapper.java b/driver-mapping/src/main/java/com/datastax/driver/mapping/MethodMapper.java index 1f29cec6031..08eb596a860 100644 --- a/driver-mapping/src/main/java/com/datastax/driver/mapping/MethodMapper.java +++ b/driver-mapping/src/main/java/com/datastax/driver/mapping/MethodMapper.java @@ -18,7 +18,6 @@ import com.datastax.driver.core.BoundStatement; import com.datastax.driver.core.ColumnDefinitions; import com.datastax.driver.core.ConsistencyLevel; -import com.datastax.driver.core.GuavaCompatibility; import com.datastax.driver.core.PreparedStatement; import com.datastax.driver.core.ResultSet; import com.datastax.driver.core.ResultSetFuture; @@ -28,7 +27,9 @@ import com.datastax.driver.mapping.annotations.Defaults; import com.google.common.collect.Sets; import com.google.common.reflect.TypeToken; +import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; +import com.google.common.util.concurrent.MoreExecutors; import java.lang.reflect.Method; import java.lang.reflect.ParameterizedType; import java.lang.reflect.Type; @@ -177,9 +178,10 @@ Object invoke(Object[] args) { if (returnMapper == null) return future; return mapOne - ? GuavaCompatibility.INSTANCE.transform(future, returnMapper.mapOneFunctionWithoutAliases) - : GuavaCompatibility.INSTANCE.transform( - future, returnMapper.mapAllFunctionWithoutAliases); + ? Futures.transform( + future, returnMapper.mapOneFunctionWithoutAliases, MoreExecutors.directExecutor()) + : Futures.transform( + future, returnMapper.mapAllFunctionWithoutAliases, MoreExecutors.directExecutor()); } else { ResultSet rs = session.execute(bs); if (returnMapper == null) return rs; diff --git a/driver-mapping/src/main/java/com/datastax/driver/mapping/Result.java b/driver-mapping/src/main/java/com/datastax/driver/mapping/Result.java index 98b0d7ede67..ada4ab491d8 100644 --- a/driver-mapping/src/main/java/com/datastax/driver/mapping/Result.java +++ b/driver-mapping/src/main/java/com/datastax/driver/mapping/Result.java @@ -16,13 +16,14 @@ package com.datastax.driver.mapping; import com.datastax.driver.core.ExecutionInfo; -import com.datastax.driver.core.GuavaCompatibility; import com.datastax.driver.core.PagingIterable; import com.datastax.driver.core.ResultSet; import com.datastax.driver.core.Row; import com.datastax.driver.core.TypeCodec; import com.google.common.base.Function; +import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; +import com.google.common.util.concurrent.MoreExecutors; import java.util.ArrayList; import java.util.Collection; import java.util.Iterator; @@ -126,14 +127,15 @@ public List getAllExecutionInfo() { @Override public ListenableFuture> fetchMoreResults() { - return GuavaCompatibility.INSTANCE.transform( + return Futures.transform( rs.fetchMoreResults(), new Function>() { @Override public Result apply(ResultSet rs) { return Result.this; } - }); + }, + MoreExecutors.directExecutor()); } @Override diff --git a/driver-mapping/src/test/java/com/datastax/driver/mapping/MapperAsyncResultTest.java b/driver-mapping/src/test/java/com/datastax/driver/mapping/MapperAsyncResultTest.java index f5f977a914f..25d4b8a86e2 100644 --- a/driver-mapping/src/test/java/com/datastax/driver/mapping/MapperAsyncResultTest.java +++ b/driver-mapping/src/test/java/com/datastax/driver/mapping/MapperAsyncResultTest.java @@ -18,7 +18,6 @@ import static org.assertj.core.api.Assertions.assertThat; import com.datastax.driver.core.CCMTestsSupport; -import com.datastax.driver.core.GuavaCompatibility; import com.datastax.driver.core.SimpleStatement; import com.datastax.driver.core.Statement; import com.datastax.driver.core.utils.CassandraVersion; @@ -27,6 +26,7 @@ import com.google.common.util.concurrent.AsyncFunction; import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; +import com.google.common.util.concurrent.MoreExecutors; import java.util.Set; import java.util.concurrent.ConcurrentSkipListSet; import org.testng.annotations.BeforeMethod; @@ -94,7 +94,7 @@ private void should_iterate_result_set_asynchronously(int totalCount, int fetchS ResultsAccumulator accumulator = new ResultsAccumulator(); ListenableFuture> results = mapper.mapAsync(session().executeAsync(statement)); ListenableFuture> future = - GuavaCompatibility.INSTANCE.transformAsync(results, accumulator); + Futures.transformAsync(results, accumulator, MoreExecutors.directExecutor()); Futures.getUnchecked(future); assertThat(accumulator.all.size()).isEqualTo(totalCount); } @@ -113,7 +113,9 @@ public ListenableFuture> apply(Result users) throws Exception } boolean wasLastPage = users.getExecutionInfo().getPagingState() == null; if (wasLastPage) return Futures.immediateFuture(users); - else return GuavaCompatibility.INSTANCE.transformAsync(users.fetchMoreResults(), this); + else + return Futures.transformAsync( + users.fetchMoreResults(), this, MoreExecutors.directExecutor()); } } } diff --git a/driver-mapping/src/test/java/com/datastax/driver/mapping/MapperAsyncTest.java b/driver-mapping/src/test/java/com/datastax/driver/mapping/MapperAsyncTest.java index 0982e0cfa76..f119136f00f 100644 --- a/driver-mapping/src/test/java/com/datastax/driver/mapping/MapperAsyncTest.java +++ b/driver-mapping/src/test/java/com/datastax/driver/mapping/MapperAsyncTest.java @@ -21,7 +21,6 @@ import com.datastax.driver.core.CCMTestsSupport; import com.datastax.driver.core.Cluster; -import com.datastax.driver.core.GuavaCompatibility; import com.datastax.driver.core.Session; import com.datastax.driver.core.utils.MoreFutures.SuccessCallback; import com.datastax.driver.core.utils.MoreObjects; @@ -30,7 +29,9 @@ import com.datastax.driver.mapping.annotations.PartitionKey; import com.datastax.driver.mapping.annotations.Table; import com.google.common.base.Function; +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.Uninterruptibles; import java.util.UUID; import java.util.concurrent.CountDownLatch; @@ -75,16 +76,17 @@ public void should_get_query_async_without_blocking() { .build()); ListenableFuture mappingManagerFuture = - GuavaCompatibility.INSTANCE.transform( + Futures.transform( cluster2.connectAsync(), new Function() { @Override public MappingManager apply(Session session) { return new MappingManager(session); } - }); + }, + MoreExecutors.directExecutor()); - GuavaCompatibility.INSTANCE.addCallback( + Futures.addCallback( mappingManagerFuture, new SuccessCallback() { @@ -93,34 +95,38 @@ public void onSuccess(MappingManager manager) { final Mapper mapper = manager.mapper(User.class); ListenableFuture saveFuture = mapper.saveAsync(paul); - GuavaCompatibility.INSTANCE.addCallback( + Futures.addCallback( saveFuture, new SuccessCallback() { @Override public void onSuccess(Void result) { ListenableFuture getFuture = mapper.getAsync(paul.getUserId()); - GuavaCompatibility.INSTANCE.addCallback( + Futures.addCallback( getFuture, new SuccessCallback() { @Override public void onSuccess(User paul) { - GuavaCompatibility.INSTANCE.addCallback( + Futures.addCallback( mapper.deleteAsync(paul), new SuccessCallback() { @Override public void onSuccess(Void result) { latch.countDown(); } - }); + }, + MoreExecutors.directExecutor()); } - }); + }, + MoreExecutors.directExecutor()); } - }); + }, + MoreExecutors.directExecutor()); } - }); + }, + MoreExecutors.directExecutor()); try { Uninterruptibles.awaitUninterruptibly(latch, 5, MINUTES); diff --git a/driver-mapping/src/test/java/com/datastax/driver/mapping/MapperTest.java b/driver-mapping/src/test/java/com/datastax/driver/mapping/MapperTest.java index 6948d2a4235..7afd200400d 100644 --- a/driver-mapping/src/test/java/com/datastax/driver/mapping/MapperTest.java +++ b/driver-mapping/src/test/java/com/datastax/driver/mapping/MapperTest.java @@ -24,7 +24,6 @@ import com.datastax.driver.core.BatchStatement; import com.datastax.driver.core.CCMTestsSupport; -import com.datastax.driver.core.GuavaCompatibility; import com.datastax.driver.core.ProtocolVersion; import com.datastax.driver.core.ResultSet; import com.datastax.driver.core.Session; @@ -42,7 +41,9 @@ import com.datastax.driver.mapping.annotations.Table; import com.google.common.base.Function; import com.google.common.base.Throwables; +import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; +import com.google.common.util.concurrent.MoreExecutors; import java.net.InetAddress; import java.util.Arrays; import java.util.HashSet; @@ -521,7 +522,7 @@ public Thread apply(Void v) { private void executeFunctionAndTestForException( User u, Mapper mapper, Function f2) { ListenableFuture f = mapper.saveAsync(u); - ListenableFuture toTest = GuavaCompatibility.INSTANCE.transform(f, f2); + ListenableFuture toTest = Futures.transform(f, f2, MoreExecutors.directExecutor()); try { Thread executedThread = toTest.get(); if (executedThread == Thread.currentThread()) { diff --git a/driver-tests/osgi/README.md b/driver-tests/osgi/README.md index b01ed55f05f..74d76afaa9e 100644 --- a/driver-tests/osgi/README.md +++ b/driver-tests/osgi/README.md @@ -45,7 +45,3 @@ configurations: 1. Default (default classifier with all dependencies) 2. Netty-Shaded (shaded classifier with all dependencies w/o Netty) -3. Guava 17 -4. Guava 18 -5. Guava 19 -6. Guava 20 diff --git a/faq/osgi/README.md b/faq/osgi/README.md index dde5bee194e..0ad6c0c1576 100644 --- a/faq/osgi/README.md +++ b/faq/osgi/README.md @@ -11,22 +11,20 @@ environment; please refer to our [OSGi examples repository]. ### How to override Guava's version? -The driver is compatible and tested with all versions of Guava in the range -`[16.0.1,26.0-jre)`. +The driver is built and tested with the Guava version declared in the root `pom.xml` +(`33.3.1-jre` at the time of writing). Overriding that dependency to an older Guava +release is not supported. -If using Maven, you can force a more specific version by re-declaring -the Guava dependency in your project, e.g.: +If you need to align with a newer release in your application, re-declare the same +major line and update your manifest accordingly, for example: com.google.guava guava - 19.0 + 33.3.1-jre -Make sure that your project's manifest is importing the right version -of Guava's packages, e.g. for 19.0: - - Import-Package: com.google.common.base;version="[19.0,20)" + Import-Package: com.google.common.base;version="[33.0,34)" ### How to enable compression?