From c18a1aa3a2afded0ee6baf240b076cd856f1a5cf Mon Sep 17 00:00:00 2001 From: Piotr Jastrzebski Date: Mon, 30 Jul 2018 16:10:15 +0200 Subject: [PATCH 01/52] Remove license maven plugin This is required to allow new code owned by ScyllaDB. Signed-off-by: Piotr Jastrzebski Signed-off-by: Piotr Jastrzebski Signed-off-by: Piotr Jastrzebski --- pom.xml | 51 --------------------------------------------------- 1 file changed, 51 deletions(-) diff --git a/pom.xml b/pom.xml index 350e56d4e0a..1654463eeb2 100644 --- a/pom.xml +++ b/pom.xml @@ -661,52 +661,6 @@ 2.5.1 - - com.mycila - license-maven-plugin - 3.0 - - - - - src/**/*.java - src/**/*.xml - src/**/*.properties - **/pom.xml - - - SLASHSTAR_STYLE - SCRIPT_STYLE - - true - - - - check-license - initialize - - check - - - - - org.codehaus.mojo animal-sniffer-maven-plugin @@ -901,11 +855,6 @@ limitations under the License. - - com.mycila - license-maven-plugin - - org.codehaus.mojo animal-sniffer-maven-plugin From 61b89b2774fa093f85380606900f783bf1fc0e8b Mon Sep 17 00:00:00 2001 From: Piotr Jastrzebski Date: Fri, 19 Apr 2019 10:24:22 +0200 Subject: [PATCH 02/52] Add Host field to Connection Part of Connection initialization will be obtaining info about shards in the host so Connection needs a reference to the host to be able to set up this info. Signed-off-by: Piotr Jastrzebski --- .../com/datastax/driver/core/Connection.java | 19 +++++++++---------- 1 file changed, 9 insertions(+), 10 deletions(-) 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 bf9c4ab5f84..2d6861bfe30 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 @@ -110,6 +110,7 @@ enum State { volatile long maxIdleTime; + private final Host host; final InetSocketAddress address; private final String name; @@ -144,8 +145,9 @@ enum State { * @param owner the component owning this connection (may be null). Note that an existing * connection can also be associated to an owner later with {@link #setOwner(Owner)}. */ - protected Connection(String name, InetSocketAddress address, Factory factory, Owner owner) { - this.address = address; + protected Connection(String name, Host host, Factory factory, Owner owner) { + this.host = host; + this.address = host.getSocketAddress(); this.factory = factory; this.dispatcher = new Dispatcher(); this.name = name; @@ -156,8 +158,8 @@ protected Connection(String name, InetSocketAddress address, Factory factory, Ow } /** Create a new connection to a Cassandra node. */ - Connection(String name, InetSocketAddress address, Factory factory) { - this(name, address, factory, null); + Connection(String name, Host host, Factory factory) { + this(name, host, factory, null); } ListenableFuture initAsync() { @@ -955,7 +957,7 @@ Connection open(Host host) if (isShutdown) throw new ConnectionException(address, "Connection factory is shut down"); host.convictionPolicy.signalConnectionsOpening(1); - Connection connection = new Connection(buildConnectionName(host), address, this); + Connection connection = new Connection(buildConnectionName(host), host, this); // This method opens the connection synchronously, so wait until it's initialized try { connection.initAsync().get(); @@ -970,8 +972,7 @@ Connection open(HostConnectionPool pool) throws ConnectionException, InterruptedException, UnsupportedProtocolVersionException, ClusterNameMismatchException { pool.host.convictionPolicy.signalConnectionsOpening(1); - Connection connection = - new Connection(buildConnectionName(pool.host), pool.host.getSocketAddress(), this, pool); + Connection connection = new Connection(buildConnectionName(pool.host), pool.host, this, pool); try { connection.initAsync().get(); return connection; @@ -988,9 +989,7 @@ List newConnections(HostConnectionPool pool, int count) { pool.host.convictionPolicy.signalConnectionsOpening(count); List connections = Lists.newArrayListWithCapacity(count); for (int i = 0; i < count; i++) - connections.add( - new Connection( - buildConnectionName(pool.host), pool.host.getSocketAddress(), this, pool)); + connections.add(new Connection(buildConnectionName(pool.host), pool.host, this, pool)); return connections; } From d5705ceb3e57db114525c3f21a8b8331c390105c Mon Sep 17 00:00:00 2001 From: Piotr Jastrzebski Date: Mon, 30 Jul 2018 16:07:55 +0200 Subject: [PATCH 03/52] Add ShardingInfo class to store details of shards on host Signed-off-by: Piotr Jastrzebski Signed-off-by: Piotr Jastrzebski --- .../datastax/driver/core/ShardingInfo.java | 106 ++++++++++++++++++ 1 file changed, 106 insertions(+) create mode 100644 driver-core/src/main/java/com/datastax/driver/core/ShardingInfo.java diff --git a/driver-core/src/main/java/com/datastax/driver/core/ShardingInfo.java b/driver-core/src/main/java/com/datastax/driver/core/ShardingInfo.java new file mode 100644 index 00000000000..192070b4aa7 --- /dev/null +++ b/driver-core/src/main/java/com/datastax/driver/core/ShardingInfo.java @@ -0,0 +1,106 @@ +/* + * Copyright (C) 2018 ScyllaDB + * + * 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 java.util.List; +import java.util.Map; + +/** Keeps the information the driver maintains on data layout of a given node. */ +public class ShardingInfo { + private static final String SCYLLA_SHARD_PARAM_KEY = "SCYLLA_SHARD"; + private static final String SCYLLA_NR_SHARDS_PARAM_KEY = "SCYLLA_NR_SHARDS"; + private static final String SCYLLA_PARTITIONER = "SCYLLA_PARTITIONER"; + private static final String SCYLLA_SHARDING_ALGORITHM = "SCYLLA_SHARDING_ALGORITHM"; + private static final String SCYLLA_SHARDING_IGNORE_MSB = "SCYLLA_SHARDING_IGNORE_MSB"; + + private final int shardsCount; + private final String partitioner; + private final String shardingAlgorithm; + private final int shardingIgnoreMSB; + + private ShardingInfo( + int shardsCount, String partitioner, String shardingAlgorithm, int shardingIgnoreMSB) { + this.shardsCount = shardsCount; + this.partitioner = partitioner; + this.shardingAlgorithm = shardingAlgorithm; + this.shardingIgnoreMSB = shardingIgnoreMSB; + } + + public int getShardsCount() { + return shardsCount; + } + + public int shardId(Token t) { + long token = Long.parseLong(t.toString()); + token += Long.MIN_VALUE; + token <<= shardingIgnoreMSB; + long tokLo = token & 0xffffffffL; + long tokHi = (token >>> 32) & 0xffffffffL; + long mul1 = tokLo * shardsCount; + long mul2 = tokHi * shardsCount; // logically shifted 32 bits + long sum = (mul1 >>> 32) + mul2; + return (int) (sum >>> 32); + } + + public static class ConnectionShardingInfo { + public final int shardId; + public final ShardingInfo shardingInfo; + + private ConnectionShardingInfo(int shardId, ShardingInfo shardingInfo) { + this.shardId = shardId; + this.shardingInfo = shardingInfo; + } + } + + public static ConnectionShardingInfo parseShardingInfo(Map> params) { + Integer shardId = parseInt(params, SCYLLA_SHARD_PARAM_KEY); + Integer shardsCount = parseInt(params, SCYLLA_NR_SHARDS_PARAM_KEY); + String partitioner = parseString(params, SCYLLA_PARTITIONER); + String shardingAlgorithm = parseString(params, SCYLLA_SHARDING_ALGORITHM); + Integer shardingIgnoreMSB = parseInt(params, SCYLLA_SHARDING_IGNORE_MSB); + if (shardId == null + || shardsCount == null + || partitioner == null + || shardingAlgorithm == null + || shardingIgnoreMSB == null + || !partitioner.equals("org.apache.cassandra.dht.Murmur3Partitioner") + || !shardingAlgorithm.equals("biased-token-round-robin")) { + return null; + } + return new ConnectionShardingInfo( + shardId, new ShardingInfo(shardsCount, partitioner, shardingAlgorithm, shardingIgnoreMSB)); + } + + private static String parseString(Map> params, String key) { + List val = params.get(key); + if (val == null || val.size() != 1) { + return null; + } + return val.get(0); + } + + private static Integer parseInt(Map> params, String key) { + String val = parseString(params, key); + if (val == null) { + return null; + } + try { + return Integer.valueOf(val); + } catch (Exception e) { + return null; + } + } +} From 6ccd1db3ee386b0221eaba5c17332987e11f6cee Mon Sep 17 00:00:00 2001 From: Piotr Jastrzebski Date: Fri, 19 Apr 2019 11:15:15 +0200 Subject: [PATCH 04/52] Add ProtocolVersion.isShardingSupported Signed-off-by: Piotr Jastrzebski --- .../main/java/com/datastax/driver/core/ProtocolVersion.java | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/driver-core/src/main/java/com/datastax/driver/core/ProtocolVersion.java b/driver-core/src/main/java/com/datastax/driver/core/ProtocolVersion.java index e165c6b17e7..3e5b888abca 100644 --- a/driver-core/src/main/java/com/datastax/driver/core/ProtocolVersion.java +++ b/driver-core/src/main/java/com/datastax/driver/core/ProtocolVersion.java @@ -63,6 +63,10 @@ public int toInt() { return asInt; } + public boolean isShardingSupported() { + return this != V1 && this != V2 && this != V3; + } + /** * Returns the highest supported version that is lower than this version. Returns {@code null} if * there isn't such a version. From cc66c5f24aea83d8a0cac70353f5cc30fd2eef9f Mon Sep 17 00:00:00 2001 From: Piotr Jastrzebski Date: Fri, 19 Apr 2019 11:18:37 +0200 Subject: [PATCH 05/52] Add shardingInfo field to Host This field will track the details of shards available at the host. It will be set during connection initialization. Multiple such initializations can occure at the same time but each should be setting the same number of shards. Signed-off-by: Piotr Jastrzebski --- .../src/main/java/com/datastax/driver/core/Host.java | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/driver-core/src/main/java/com/datastax/driver/core/Host.java b/driver-core/src/main/java/com/datastax/driver/core/Host.java index fea67bade97..bd75f76abc5 100644 --- a/driver-core/src/main/java/com/datastax/driver/core/Host.java +++ b/driver-core/src/main/java/com/datastax/driver/core/Host.java @@ -55,6 +55,9 @@ public class Host { private volatile UUID schemaVersion; + // Can be set concurrently but the value should always be the same. + private volatile ShardingInfo shardingInfo = null; + enum State { ADDED, DOWN, @@ -396,6 +399,14 @@ void setTokens(Set tokens) { this.tokens = tokens; } + public ShardingInfo getShardingInfo() { + return shardingInfo; + } + + public void setShardingInfo(ShardingInfo shardingInfo) { + this.shardingInfo = shardingInfo; + } + /** * Returns whether the host is considered up by the driver. * From f9f6dc10662dafb8feafea4287be5ea1b9155529 Mon Sep 17 00:00:00 2001 From: Piotr Jastrzebski Date: Fri, 19 Apr 2019 11:23:16 +0200 Subject: [PATCH 06/52] Add shardId field to Connection This field will be set during initialization and will describe the shard in the host that a connection is associated to. Signed-off-by: Piotr Jastrzebski --- .../src/main/java/com/datastax/driver/core/Connection.java | 5 +++++ 1 file changed, 5 insertions(+) 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 2d6861bfe30..84b3b90c1d3 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 @@ -113,6 +113,7 @@ enum State { private final Host host; final InetSocketAddress address; private final String name; + private volatile Integer shardId = null; @VisibleForTesting volatile Channel channel; private final Factory factory; @@ -803,6 +804,10 @@ boolean setOwner(Owner owner) { return ownerRef.compareAndSet(null, owner); } + public int shardId() { + return shardId == null ? 0 : shardId; + } + /** * If the connection is part of a pool, return it to the pool. The connection should generally not * be reused after that. From bc42e3156dec7f79b2cd1e2b70c2a5d48d623363 Mon Sep 17 00:00:00 2001 From: Piotr Jastrzebski Date: Fri, 19 Apr 2019 11:32:07 +0200 Subject: [PATCH 07/52] Fetch sharding info when initializing the connection Signed-off-by: Piotr Jastrzebski --- .../com/datastax/driver/core/Connection.java | 46 ++++++++++++++++++- 1 file changed, 45 insertions(+), 1 deletion(-) 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 84b3b90c1d3..5b148c561e4 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 @@ -19,6 +19,7 @@ import static io.netty.handler.timeout.IdleState.READER_IDLE; import com.datastax.driver.core.Responses.Result.SetKeyspace; +import com.datastax.driver.core.Responses.Supported; import com.datastax.driver.core.exceptions.AuthenticationException; import com.datastax.driver.core.exceptions.BusyConnectionException; import com.datastax.driver.core.exceptions.ConnectionException; @@ -246,10 +247,16 @@ public void operationComplete(ChannelFuture future) throws Exception { GuavaCompatibility.INSTANCE.transformAsync( channelReadyFuture, onChannelReady(protocolVersion, initExecutor), initExecutor); + ListenableFuture getShardingInfoFuture = + protocolVersion.isShardingSupported() + ? GuavaCompatibility.INSTANCE.transformAsync( + initializeTransportFuture, onTransportInitialized(initExecutor), initExecutor) + : initializeTransportFuture; + // Fallback on initializeTransportFuture so we can properly propagate specific exceptions. ListenableFuture initFuture = GuavaCompatibility.INSTANCE.withFallback( - initializeTransportFuture, + getShardingInfoFuture, new AsyncFunction() { @Override public ListenableFuture apply(Throwable t) throws Exception { @@ -369,6 +376,43 @@ public ListenableFuture apply(Message.Response response) throws Exception }; } + private AsyncFunction onTransportInitialized(final Executor initExecutor) { + return new AsyncFunction() { + @Override + public ListenableFuture apply(Void input) throws Exception { + Future shardingInfoResponseFuture = write(new Requests.Options()); + return GuavaCompatibility.INSTANCE.transformAsync( + shardingInfoResponseFuture, onShardingInfoResponse(initExecutor), initExecutor); + } + }; + } + + private AsyncFunction onShardingInfoResponse( + final Executor initExecutor) { + return new AsyncFunction() { + @Override + public ListenableFuture apply(Message.Response response) throws Exception { + switch (response.type) { + case SUPPORTED: + Responses.Supported msg = (Supported) response; + ShardingInfo.ConnectionShardingInfo sharding = + ShardingInfo.parseShardingInfo(msg.supported); + if (sharding != null) { + host.setShardingInfo(sharding.shardingInfo); + Connection.this.shardId = sharding.shardId; + } + return MoreFutures.VOID_SUCCESS; + default: + throw new TransportException( + address, + String.format( + "Unexpected %s response message from server to a OPTIONS message", + response.type)); + } + } + }; + } + // Due to C* gossip bugs, system.peers may report nodes that are gone from the cluster. // If these nodes have been recommissionned to another cluster and are up, nothing prevents the // driver from connecting From 2bd2e25a21d989a3d3c480cf6af9722a407436df Mon Sep 17 00:00:00 2001 From: Piotr Jastrzebski Date: Fri, 19 Apr 2019 12:43:29 +0200 Subject: [PATCH 08/52] Add shard awareness to HostConnectionPool Signed-off-by: Piotr Jastrzebski --- .../driver/core/HostConnectionPool.java | 365 ++++++++++++------ .../datastax/driver/core/RequestHandler.java | 8 +- .../datastax/driver/core/SessionManager.java | 4 +- .../datastax/driver/core/AsyncQueryTest.java | 7 +- .../datastax/driver/core/HeartbeatTest.java | 2 +- .../driver/core/HostConnectionPoolTest.java | 132 ++++--- .../driver/core/RequestHandlerTest.java | 2 +- 7 files changed, 338 insertions(+), 182 deletions(-) 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 aa17a0ea2b1..bf775d31ebb 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 @@ -35,10 +35,11 @@ import com.google.common.util.concurrent.SettableFuture; import com.google.common.util.concurrent.Uninterruptibles; import io.netty.util.concurrent.EventExecutor; +import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.List; -import java.util.ListIterator; import java.util.Queue; +import java.util.Random; import java.util.Set; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.CopyOnWriteArrayList; @@ -57,12 +58,14 @@ class HostConnectionPool implements Connection.Owner { private static final Logger logger = LoggerFactory.getLogger(HostConnectionPool.class); private static final int MAX_SIMULTANEOUS_CREATION = 1; + private static final Random RAND = new Random(); final Host host; volatile HostDistance hostDistance; protected final SessionManager manager; - final List connections; + private int connectionsPerShard; + List[] connections; private final AtomicInteger open; /** The total number of in-flight requests on all connections of this pool. */ final AtomicInteger totalInFlight = new AtomicInteger(); @@ -72,14 +75,12 @@ class HostConnectionPool implements Connection.Owner { */ private final AtomicInteger maxTotalInFlight = new AtomicInteger(); - @VisibleForTesting final Set trash = new CopyOnWriteArraySet(); + @VisibleForTesting Set[] trash; - private final Queue pendingBorrows = new ConcurrentLinkedQueue(); + private Queue[] pendingBorrows; final AtomicInteger pendingBorrowCount = new AtomicInteger(); - private final Runnable newConnectionTask; - - private final AtomicInteger scheduledForCreation = new AtomicInteger(); + private AtomicInteger[] scheduledForCreation; private final EventExecutor timeoutsExecutor; @@ -94,6 +95,20 @@ private enum Phase { protected final AtomicReference phase = new AtomicReference(Phase.INITIALIZING); + private class ConnectionTask implements Runnable { + private final int shardId; + + public ConnectionTask(int shardId) { + this.shardId = shardId; + } + + @Override + public void run() { + addConnectionIfUnderMaximum(shardId); + scheduledForCreation[shardId].decrementAndGet(); + } + } + // When a request times out, we may never release its stream ID. So over time, a given connection // may get less an less available streams. When the number of available ones go below the // following threshold, we just replace the connection by a new one. @@ -105,16 +120,6 @@ private enum Phase { this.hostDistance = hostDistance; this.manager = manager; - this.newConnectionTask = - new Runnable() { - @Override - public void run() { - addConnectionIfUnderMaximum(); - scheduledForCreation.decrementAndGet(); - } - }; - - this.connections = new CopyOnWriteArrayList(); this.open = new AtomicInteger(); this.minAllowedStreams = options().getMaxRequestsPerConnection(hostDistance) * 3 / 4; @@ -127,21 +132,48 @@ public void run() { * reuse as part of this pool. Might be null or already used by another pool. */ ListenableFuture initAsync(Connection reusedConnection) { + if (reusedConnection != null && reusedConnection.setOwner(this)) { + return initAsyncWithConnection(reusedConnection); + } + try { + return initAsyncWithConnection(manager.connectionFactory().open(this)); + } catch (Exception e) { + phase.compareAndSet(Phase.INITIALIZING, Phase.INIT_FAILED); + SettableFuture future = SettableFuture.create(); + future.setException(e); + return future; + } + } + + ListenableFuture initAsyncWithConnection(Connection reusedConnection) { Executor initExecutor = manager.cluster.manager.configuration.getPoolingOptions().getInitializationExecutor(); // Create initial core connections final int coreSize = options().getCoreConnectionsPerHost(hostDistance); - final List connections = Lists.newArrayListWithCapacity(coreSize); - final List> connectionFutures = Lists.newArrayListWithCapacity(coreSize); + final int shardsCount = + host.getShardingInfo() == null ? 1 : host.getShardingInfo().getShardsCount(); + + connectionsPerShard = coreSize / shardsCount + (coreSize % shardsCount > 0 ? 1 : 0); + int toCreate = shardsCount * connectionsPerShard; + + this.connections = new List[shardsCount]; + scheduledForCreation = new AtomicInteger[shardsCount]; + trash = new Set[shardsCount]; + pendingBorrows = new Queue[shardsCount]; + for (int i = 0; i < shardsCount; ++i) { + this.connections[i] = new CopyOnWriteArrayList(); + scheduledForCreation[i] = new AtomicInteger(); + trash[i] = new CopyOnWriteArraySet(); + pendingBorrows[i] = new ConcurrentLinkedQueue(); + } - int toCreate = coreSize; + final List connections = Lists.newArrayListWithCapacity(toCreate); + final List> connectionFutures = Lists.newArrayListWithCapacity(toCreate); - if (reusedConnection != null && toCreate > 0 && reusedConnection.setOwner(this)) { - toCreate -= 1; - connections.add(reusedConnection); - connectionFutures.add(MoreFutures.VOID_SUCCESS); - } + toCreate -= 1; + connections.add(reusedConnection); + connectionFutures.add(MoreFutures.VOID_SUCCESS); List newConnections = manager.connectionFactory().newConnections(this, toCreate); connections.addAll(newConnections); @@ -150,22 +182,42 @@ ListenableFuture initAsync(Connection reusedConnection) { connectionFutures.add(handleErrors(connectionFuture, initExecutor)); } - ListenableFuture> allConnectionsFuture = Futures.allAsList(connectionFutures); - final SettableFuture initFuture = SettableFuture.create(); + + addCallback(connections, connectionFutures, initFuture, new ArrayList()); + + return initFuture; + } + + private void addCallback( + final List connections, + final List> connectionFutures, + final SettableFuture initFuture, + final List toClose) { + + final Executor initExecutor = + manager.cluster.manager.configuration.getPoolingOptions().getInitializationExecutor(); + final ListenableFuture> allConnectionsFuture = Futures.allAsList(connectionFutures); + GuavaCompatibility.INSTANCE.addCallback( allConnectionsFuture, new FutureCallback>() { @Override public void onSuccess(List l) { - // Some of the connections might have failed, keep only the successful ones - ListIterator it = connections.listIterator(); - while (it.hasNext()) { - if (it.next().isClosed()) it.remove(); + int added = 0; + for (final Connection c : connections) { + if (!c.isClosed()) { + if (HostConnectionPool.this.connections[c.shardId()].size() + < HostConnectionPool.this.connectionsPerShard) { + ++added; + HostConnectionPool.this.connections[c.shardId()].add(c); + } else { + toClose.add(c); + } + } } - HostConnectionPool.this.connections.addAll(connections); - open.set(connections.size()); + open.addAndGet(added); if (isClosed()) { initFuture.setException( @@ -173,14 +225,35 @@ public void onSuccess(List l) { host.getSocketAddress(), "Pool was closed during initialization")); // we're not sure if closeAsync() saw the connections, so ensure they get closed forceClose(connections); + forceClose(toClose); + for (List shardConnections : HostConnectionPool.this.connections) { + forceClose(shardConnections); + } + open.set(0); } else { - logger.debug( - "Created connection pool to host {} ({} connections needed, {} successfully opened)", - host, - coreSize, - connections.size()); - phase.compareAndSet(Phase.INITIALIZING, Phase.READY); - initFuture.set(null); + int needed = 0; + for (final List shardsConnections : HostConnectionPool.this.connections) { + needed += + Math.max( + 0, HostConnectionPool.this.connectionsPerShard - shardsConnections.size()); + } + if (needed > 0) { + final List connections = + manager.connectionFactory().newConnections(HostConnectionPool.this, needed); + final List> connectionFutures = + Lists.newArrayListWithCapacity(needed); + for (Connection connection : connections) { + ListenableFuture connectionFuture = connection.initAsync(); + connectionFutures.add(handleErrors(connectionFuture, initExecutor)); + } + addCallback(connections, connectionFutures, initFuture, toClose); + } else { + for (final Connection c : toClose) { + c.closeAsync(); + } + phase.compareAndSet(Phase.INITIALIZING, Phase.READY); + initFuture.set(null); + } } } @@ -188,11 +261,15 @@ public void onSuccess(List l) { public void onFailure(Throwable t) { phase.compareAndSet(Phase.INITIALIZING, Phase.INIT_FAILED); forceClose(connections); + forceClose(toClose); + for (List shardConnections : HostConnectionPool.this.connections) { + forceClose(shardConnections); + } + open.set(0); initFuture.setException(t); } }, initExecutor); - return initFuture; } private ListenableFuture handleErrors( @@ -234,31 +311,42 @@ private PoolingOptions options() { return manager.configuration().getPoolingOptions(); } - ListenableFuture borrowConnection(long timeout, TimeUnit unit, int maxQueueSize) { + ListenableFuture borrowConnection( + long timeout, TimeUnit unit, int maxQueueSize, ByteBuffer routingKey) { Phase phase = this.phase.get(); if (phase != Phase.READY) return Futures.immediateFailedFuture( new ConnectionException(host.getSocketAddress(), "Pool is " + phase)); - if (connections.isEmpty()) { + int shardId = 0; + if (host.getShardingInfo() != null) { + if (routingKey != null) { + Metadata metadata = manager.cluster.getMetadata(); + Token t = metadata.newToken(routingKey); + shardId = host.getShardingInfo().shardId(t); + } else { + shardId = RAND.nextInt(host.getShardingInfo().getShardsCount()); + } + } + + if (connections[shardId].isEmpty()) { if (host.convictionPolicy.canReconnectNow()) { - int coreSize = options().getCoreConnectionsPerHost(hostDistance); - if (coreSize == 0) { - maybeSpawnNewConnection(); - } else if (scheduledForCreation.compareAndSet(0, coreSize)) { - for (int i = 0; i < coreSize; i++) { + if (connectionsPerShard == 0) { + maybeSpawnNewConnection(shardId); + } else if (scheduledForCreation[shardId].compareAndSet(0, connectionsPerShard)) { + for (int i = 0; i < connectionsPerShard; i++) { // We don't respect MAX_SIMULTANEOUS_CREATION here because it's only to // protect against creating connection in excess of core too quickly - manager.blockingExecutor().submit(newConnectionTask); + manager.blockingExecutor().submit(new ConnectionTask(shardId)); } } - return enqueue(timeout, unit, maxQueueSize); + return enqueue(timeout, unit, maxQueueSize, shardId); } } int minInFlight = Integer.MAX_VALUE; Connection leastBusy = null; - for (Connection connection : connections) { + for (Connection connection : connections[shardId]) { int inFlight = connection.inFlight.get(); if (inFlight < minInFlight) { minInFlight = inFlight; @@ -276,7 +364,7 @@ ListenableFuture borrowConnection(long timeout, TimeUnit unit, int m // the previous check to connections and now. But in that case, the line above will have // trigger the creation of // a new connection, so just wait that connection and move on - return enqueue(timeout, unit, maxQueueSize); + return enqueue(timeout, unit, maxQueueSize, shardId); } else { while (true) { int inFlight = leastBusy.inFlight.get(); @@ -285,7 +373,7 @@ ListenableFuture borrowConnection(long timeout, TimeUnit unit, int m >= Math.min( leastBusy.maxAvailableStreams(), options().getMaxRequestsPerConnection(hostDistance))) { - return enqueue(timeout, unit, maxQueueSize); + return enqueue(timeout, unit, maxQueueSize, shardId); } if (leastBusy.inFlight.compareAndSet(inFlight, inFlight + 1)) break; @@ -300,21 +388,22 @@ ListenableFuture borrowConnection(long timeout, TimeUnit unit, int m || maxTotalInFlight.compareAndSet(oldMax, totalInFlightCount)) break; } - int connectionCount = open.get() + scheduledForCreation.get(); - if (connectionCount < options().getCoreConnectionsPerHost(hostDistance)) { - maybeSpawnNewConnection(); + int connectionCount = connections[shardId].size() + scheduledForCreation[shardId].get(); + if (connectionCount < connectionsPerShard) { + maybeSpawnNewConnection(shardId); } else if (connectionCount < options().getMaxConnectionsPerHost(hostDistance)) { // Add a connection if we fill the first n-1 connections and almost fill the last one int currentCapacity = (connectionCount - 1) * options().getMaxRequestsPerConnection(hostDistance) + options().getNewConnectionThreshold(hostDistance); - if (totalInFlightCount > currentCapacity) maybeSpawnNewConnection(); + if (totalInFlightCount > currentCapacity) maybeSpawnNewConnection(shardId); } return leastBusy.setKeyspaceAsync(manager.poolsState.keyspace); } - private ListenableFuture enqueue(long timeout, TimeUnit unit, int maxQueueSize) { + private ListenableFuture enqueue( + long timeout, TimeUnit unit, int maxQueueSize, int shardId) { if (timeout == 0 || maxQueueSize == 0) { return Futures.immediateFailedFuture(new BusyPoolException(host.getSocketAddress(), 0)); } @@ -331,7 +420,7 @@ private ListenableFuture enqueue(long timeout, TimeUnit unit, int ma } PendingBorrow pendingBorrow = new PendingBorrow(timeout, unit, timeoutsExecutor); - pendingBorrows.add(pendingBorrow); + pendingBorrows[shardId].add(pendingBorrow); // If we raced with shutdown, make sure the future will be completed. This has no effect if it // was properly @@ -371,7 +460,7 @@ void returnConnection(Connection connection) { // When a connection gets returned to the pool, check if there are pending borrows that can be // completed with it. private void dequeue(final Connection connection) { - while (!pendingBorrows.isEmpty()) { + while (!pendingBorrows[connection.shardId()].isEmpty()) { // We can only reuse the connection if it's under its maximum number of inFlight requests. // Do this atomically, as we could be competing with other borrowConnection or dequeue calls. @@ -390,7 +479,7 @@ private void dequeue(final Connection connection) { } } - final PendingBorrow pendingBorrow = pendingBorrows.poll(); + final PendingBorrow pendingBorrow = pendingBorrows[connection.shardId()].poll(); if (pendingBorrow == null) { // Another thread has emptied the queue since our last check, restore the count connection.inFlight.decrementAndGet(); @@ -445,7 +534,7 @@ public void onFailure(Throwable t) { private void replaceConnection(Connection connection) { if (!connection.state.compareAndSet(OPEN, TRASHED)) return; open.decrementAndGet(); - maybeSpawnNewConnection(); + maybeSpawnNewConnection(connection.shardId()); connection.maxIdleTime = Long.MIN_VALUE; doTrashConnection(connection); } @@ -470,11 +559,11 @@ private boolean trashConnection(Connection connection) { } private void doTrashConnection(Connection connection) { - connections.remove(connection); - trash.add(connection); + connections[connection.shardId()].remove(connection); + trash[connection.shardId()].add(connection); } - private boolean addConnectionIfUnderMaximum() { + private boolean addConnectionIfUnderMaximum(int shardId) { // First, make sure we don't cross the allowed limit of open connections for (; ; ) { @@ -491,17 +580,30 @@ private boolean addConnectionIfUnderMaximum() { // Now really open the connection try { - Connection newConnection = tryResurrectFromTrash(); + Connection newConnection = tryResurrectFromTrash(shardId); if (newConnection == null) { if (!host.convictionPolicy.canReconnectNow()) { open.decrementAndGet(); return false; } logger.debug("Creating new connection on busy pool to {}", host); - newConnection = manager.connectionFactory().open(this); - newConnection.setKeyspace(manager.poolsState.keyspace); + List toClose = new ArrayList(); + try { + while (true) { + newConnection = manager.connectionFactory().open(this); + if (newConnection.shardId() == shardId) { + newConnection.setKeyspace(manager.poolsState.keyspace); + break; + } + toClose.add(newConnection); + } + } finally { + for (Connection c : toClose) { + c.closeAsync(); + } + } } - connections.add(newConnection); + connections[newConnection.shardId()].add(newConnection); newConnection.state.compareAndSet(RESURRECTING, OPEN); // no-op if it was already OPEN @@ -547,12 +649,12 @@ private boolean addConnectionIfUnderMaximum() { } } - private Connection tryResurrectFromTrash() { + private Connection tryResurrectFromTrash(int shardId) { long highestMaxIdleTime = System.currentTimeMillis(); Connection chosen = null; while (true) { - for (Connection connection : trash) + for (Connection connection : trash[shardId]) if (connection.maxIdleTime > highestMaxIdleTime && connection.maxAvailableStreams() > minAllowedStreams) { chosen = connection; @@ -563,26 +665,26 @@ private Connection tryResurrectFromTrash() { else if (chosen.state.compareAndSet(TRASHED, RESURRECTING)) break; } logger.trace("Resurrecting {}", chosen); - trash.remove(chosen); + trash[shardId].remove(chosen); return chosen; } - private void maybeSpawnNewConnection() { + private void maybeSpawnNewConnection(int shardId) { if (isClosed() || !host.convictionPolicy.canReconnectNow()) return; while (true) { - int inCreation = scheduledForCreation.get(); + int inCreation = scheduledForCreation[shardId].get(); if (inCreation >= MAX_SIMULTANEOUS_CREATION) return; - if (scheduledForCreation.compareAndSet(inCreation, inCreation + 1)) break; + if (scheduledForCreation[shardId].compareAndSet(inCreation, inCreation + 1)) break; } - manager.blockingExecutor().submit(newConnectionTask); + manager.blockingExecutor().submit(new ConnectionTask(shardId)); } @Override public void onConnectionDefunct(final Connection connection) { if (connection.state.compareAndSet(OPEN, GONE)) open.decrementAndGet(); - connections.remove(connection); + connections[connection.shardId()].remove(connection); // Don't try to replace the connection now. Connection.defunct already signaled the failure, // and either the host will be marked DOWN (which destroys all pools), or we want to prevent @@ -617,26 +719,36 @@ private void shrinkIfBelowCapacity() { if (toTrash <= 0) return; - for (Connection connection : connections) - if (trashConnection(connection)) { - toTrash -= 1; - if (toTrash == 0) return; + for (final List shardsConnections : connections) { + if (shardsConnections.size() > connectionsPerShard) { + for (Connection connection : shardsConnections) { + if (trashConnection(connection)) { + toTrash -= 1; + if (toTrash == 0) return; + if (shardsConnections.size() <= connectionsPerShard) { + break; + } + } + } } + } } /** Close connections that have been sitting in the trash for too long */ private void cleanupTrash(long now) { - for (Connection connection : trash) { - if (connection.maxIdleTime < now && connection.state.compareAndSet(TRASHED, GONE)) { - if (connection.inFlight.get() == 0) { - logger.trace("Cleaning up {}", connection); - trash.remove(connection); - close(connection); - } else { - // Given that idleTimeout >> request timeout, all outstanding requests should - // have finished by now, so we should not get here. - // Restore the status so that it's retried on the next cleanup. - connection.state.set(TRASHED); + for (Set shardConnections : trash) { + for (Connection connection : shardConnections) { + if (connection.maxIdleTime < now && connection.state.compareAndSet(TRASHED, GONE)) { + if (connection.inFlight.get() == 0) { + logger.trace("Cleaning up {}", connection); + shardConnections.remove(connection); + close(connection); + } else { + // Given that idleTimeout >> request timeout, all outstanding requests should + // have finished by now, so we should not get here. + // Restore the status so that it's retried on the next cleanup. + connection.state.set(TRASHED); + } } } } @@ -657,9 +769,11 @@ final CloseFuture closeAsync() { phase.set(Phase.CLOSING); - for (PendingBorrow pendingBorrow : pendingBorrows) { - pendingBorrow.setException( - new ConnectionException(host.getSocketAddress(), "Pool is shutdown")); + for (Queue queue : pendingBorrows) { + for (PendingBorrow pendingBorrow : queue) { + pendingBorrow.setException( + new ConnectionException(host.getSocketAddress(), "Pool is shutdown")); + } } future = new CloseFuture.Forwarding(discardAvailableConnections()); @@ -674,7 +788,11 @@ int opened() { } int trashed() { - return trash.size(); + int size = 0; + for (final Set shardConnections : trash) { + size += shardConnections.size(); + } + return size; } private List discardAvailableConnections() { @@ -682,23 +800,36 @@ private List discardAvailableConnections() { // will be empty, // so this will return an empty list - List futures = new ArrayList(connections.size() + trash.size()); - - for (final Connection connection : connections) { - CloseFuture future = connection.closeAsync(); - future.addListener( - new Runnable() { - @Override - public void run() { - if (connection.state.compareAndSet(OPEN, GONE)) open.decrementAndGet(); - } - }, - GuavaCompatibility.INSTANCE.sameThreadExecutor()); - futures.add(future); + int size = 0; + for (final Set shardConnections : trash) { + size += shardConnections.size(); + } + for (final List shardConnections : connections) { + size += shardConnections.size(); + } + List futures = new ArrayList(size); + + for (final List shardConnections : connections) { + for (final Connection connection : shardConnections) { + CloseFuture future = connection.closeAsync(); + future.addListener( + new Runnable() { + @Override + public void run() { + if (connection.state.compareAndSet(OPEN, GONE)) open.decrementAndGet(); + } + }, + GuavaCompatibility.INSTANCE.sameThreadExecutor()); + futures.add(future); + } } // Some connections in the trash might still be open if they hadn't reached their idle timeout - for (Connection connection : trash) futures.add(connection.closeAsync()); + for (final Set shardConnections : trash) { + for (final Connection connection : shardConnections) { + futures.add(connection.closeAsync()); + } + } return futures; } @@ -715,12 +846,14 @@ void ensureCoreConnections() { // more connection than maximum (and if we create more than core connection due to a race but // this isn't // justified by the load, the connection in excess will be quickly trashed anyway) - int opened = open.get(); - for (int i = opened; i < options().getCoreConnectionsPerHost(hostDistance); i++) { - // We don't respect MAX_SIMULTANEOUS_CREATION here because it's only to - // protect against creating connection in excess of core too quickly - scheduledForCreation.incrementAndGet(); - manager.blockingExecutor().submit(newConnectionTask); + for (int shardId = 0; shardId < connections.length; ++shardId) { + final List shardConnections = connections[shardId]; + for (int i = shardConnections.size(); i < connectionsPerShard; ++i) { + // We don't respect MAX_SIMULTANEOUS_CREATION here because it's only to + // protect against creating connection in excess of core too quickly + scheduledForCreation[shardId].incrementAndGet(); + manager.blockingExecutor().submit(new ConnectionTask(shardId)); + } } } 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 6b968f5d7be..735c176a3a3 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 @@ -42,6 +42,7 @@ import io.netty.util.Timeout; import io.netty.util.TimerTask; import java.net.InetSocketAddress; +import java.nio.ByteBuffer; import java.util.Collections; import java.util.Iterator; import java.util.List; @@ -392,12 +393,17 @@ private boolean query(final Host host) { if (allowSpeculativeExecutions && nextExecutionScheduled.compareAndSet(false, true)) scheduleExecution(speculativeExecutionPlan.nextExecution(host)); + ProtocolVersion protocolVersion = manager.cluster.manager.protocolVersion(); + CodecRegistry codecRegistry = manager.cluster.manager.configuration.getCodecRegistry(); + ByteBuffer routingKey = statement.getRoutingKey(protocolVersion, codecRegistry); + PoolingOptions poolingOptions = manager.configuration().getPoolingOptions(); ListenableFuture connectionFuture = pool.borrowConnection( poolingOptions.getPoolTimeoutMillis(), TimeUnit.MILLISECONDS, - poolingOptions.getMaxQueueSize()); + poolingOptions.getMaxQueueSize(), + routingKey); GuavaCompatibility.INSTANCE.addCallback( connectionFuture, new FutureCallback() { 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 a366dd82f33..3cffdb82216 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 @@ -719,7 +719,9 @@ private ListenableFuture prepare( // execute // the prepared query. So don't wait if no connection is available, simply abort. ListenableFuture connectionFuture = - entry.getValue().borrowConnection(0, TimeUnit.MILLISECONDS, 0); + entry + .getValue() + .borrowConnection(0, TimeUnit.MILLISECONDS, 0, statement.getRoutingKey()); ListenableFuture prepareFuture = GuavaCompatibility.INSTANCE.transformAsync( connectionFuture, 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 d59e9829bb1..2f01a01b7ab 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 @@ -30,6 +30,7 @@ import com.google.common.util.concurrent.Uninterruptibles; import java.nio.ByteBuffer; import java.util.Collection; +import java.util.List; import java.util.Map; import java.util.concurrent.ExecutionException; import java.util.concurrent.Executor; @@ -74,8 +75,10 @@ public void cancelled_query_should_release_the_connection() throws InterruptedEx TimeUnit.MILLISECONDS.sleep(100); HostConnectionPool pool = getPool(session()); - for (Connection connection : pool.connections) { - assertEquals(connection.inFlight.get(), 0); + for (List shardConnections : pool.connections) { + for (Connection connection : shardConnections) { + assertEquals(connection.inFlight.get(), 0); + } } } diff --git a/driver-core/src/test/java/com/datastax/driver/core/HeartbeatTest.java b/driver-core/src/test/java/com/datastax/driver/core/HeartbeatTest.java index c4de31642ef..4593b2017a4 100644 --- a/driver-core/src/test/java/com/datastax/driver/core/HeartbeatTest.java +++ b/driver-core/src/test/java/com/datastax/driver/core/HeartbeatTest.java @@ -179,7 +179,7 @@ public void should_send_heartbeat_when_requests_being_written_but_nothing_receiv // Find the connection in the connection pool. SessionManager session = (SessionManager) cluster.connect(); Host host = TestUtils.findHost(cluster, 1); - Connection connection = session.pools.get(host).connections.get(0); + Connection connection = session.pools.get(host).connections[0].get(0); // Extract connection name from toString implementation. String connectionName = 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 cad07e2ce27..3e73911ced1 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 @@ -54,6 +54,7 @@ import com.google.common.util.concurrent.ListeningExecutorService; import com.google.common.util.concurrent.Uninterruptibles; import java.net.InetSocketAddress; +import java.nio.ByteBuffer; import java.util.Collection; import java.util.Collections; import java.util.Iterator; @@ -84,6 +85,14 @@ public void reinitializeCluster() { cluster.close(); } + private static int size(HostConnectionPool pool) { + int size = 0; + for (List connections : pool.connections) { + size += connections.size(); + } + return size; + } + /** * Ensure the given pool has the given size within 5 seconds. * @@ -98,7 +107,7 @@ private void assertPoolSize(HostConnectionPool pool, final int expectedSize) { new Predicate() { @Override public boolean apply(HostConnectionPool input) { - return input.connections.size() == expectedSize; + return size(input) == expectedSize; } }) .becomesTrue(); @@ -133,8 +142,8 @@ public void fixed_size_pool_should_fill_its_core_connections_and_queue_and_then_ HostConnectionPool pool = createPool(cluster, 2, 2); int maxQueueSize = 256; - assertThat(pool.connections.size()).isEqualTo(2); - List coreConnections = newArrayList(pool.connections); + assertThat(size(pool)).isEqualTo(2); + List coreConnections = newArrayList(pool.connections[0]); // fill connections List requests = MockRequest.sendMany(2 * 128, pool); assertBorrowedConnections(requests, coreConnections); @@ -254,8 +263,8 @@ public void requests_with_enqueued_borrow_requests_should_be_failed_when_pool_cl HostConnectionPool pool = createPool(cluster, 2, 2); int maxQueueSize = 256; - assertThat(pool.connections.size()).isEqualTo(2); - List coreConnections = newArrayList(pool.connections); + assertThat(size(pool)).isEqualTo(2); + List coreConnections = newArrayList(pool.connections[0]); // fill connections requests = MockRequest.sendMany(2 * 128, pool); assertBorrowedConnections(requests, coreConnections); @@ -301,8 +310,8 @@ public void should_adjust_connection_keyspace_on_dequeue_if_pool_state_is_differ HostConnectionPool pool = createPool(cluster, 1, 1); int maxQueueSize = 256; - assertThat(pool.connections.size()).isEqualTo(1); - List coreConnections = newArrayList(pool.connections); + assertThat(size(pool)).isEqualTo(1); + List coreConnections = newArrayList(pool.connections[0]); // fill connections requests = MockRequest.sendMany(128, pool); assertBorrowedConnections(requests, coreConnections); @@ -378,7 +387,7 @@ public void should_adjust_connection_keyspace_on_dequeue_if_pool_state_is_differ .withQuery("USE \"slowks\"") .withThen(PrimingRequest.then().withFixedDelay(5000L))); - Connection connection = pool.connections.get(0); + Connection connection = pool.connections[0].get(0); connection.setKeyspaceAsync("slowks"); @@ -425,8 +434,8 @@ public void should_adjust_connection_keyspace_on_dequeue_if_pool_state_is_differ .setMaxRequestsPerConnection(HostDistance.LOCAL, 100); int maxQueueSize = 256; - assertThat(pool.connections.size()).isEqualTo(1); - List coreConnections = newArrayList(pool.connections); + assertThat(size(pool)).isEqualTo(1); + List coreConnections = newArrayList(pool.connections[0]); // fill connections requests = MockRequest.sendMany(100, pool); @@ -445,7 +454,7 @@ public void should_adjust_connection_keyspace_on_dequeue_if_pool_state_is_differ PrimingRequest.queryBuilder() .withQuery("USE \"slowks\"") .withThen(PrimingRequest.then().withFixedDelay(5000L))); - Connection connection = pool.connections.get(0); + Connection connection = pool.connections[0].get(0); connection.setKeyspaceAsync("slowks"); // Simulate change of keyspace on pool. @@ -486,8 +495,8 @@ public void variable_size_pool_should_fill_its_connections_and_then_reject() thr Connection.Factory factory = spy(cluster.manager.connectionFactory); cluster.manager.connectionFactory = factory; - assertThat(pool.connections.size()).isEqualTo(1); - Connection coreConnection = pool.connections.get(0); + assertThat(size(pool)).isEqualTo(1); + Connection coreConnection = pool.connections[0].get(0); // Fill enough connections to hit the threshold. List requests = MockRequest.sendMany(NEW_CONNECTION_THRESHOLD, pool); @@ -538,7 +547,7 @@ public void should_add_extra_connection_when_core_full() throws Exception { HostConnectionPool pool = createPool(cluster, 1, 2); Connection.Factory factory = spy(cluster.manager.connectionFactory); cluster.manager.connectionFactory = factory; - Connection core = pool.connections.get(0); + Connection core = pool.connections[0].get(0); // Fill core connection + 1 List requests = MockRequest.sendMany(NEW_CONNECTION_THRESHOLD, pool); @@ -574,7 +583,7 @@ public void should_resurrect_trashed_connection_within_idle_timeout() throws Exc HostConnectionPool pool = createPool(cluster, 1, 2); Connection.Factory factory = spy(cluster.manager.connectionFactory); cluster.manager.connectionFactory = factory; - Connection connection1 = pool.connections.get(0); + Connection connection1 = pool.connections[0].get(0); List requests = MockRequest.sendMany(NEW_CONNECTION_THRESHOLD, pool); assertBorrowedConnections(requests, Collections.singletonList(connection1)); @@ -583,7 +592,7 @@ public void should_resurrect_trashed_connection_within_idle_timeout() throws Exc verify(factory, after(2000).times(1)).open(any(HostConnectionPool.class)); assertPoolSize(pool, 2); - Connection connection2 = pool.connections.get(1); + Connection connection2 = pool.connections[0].get(1); assertThat(connection1.inFlight.get()).isEqualTo(101); assertThat(connection2.inFlight.get()).isEqualTo(0); @@ -596,14 +605,14 @@ public void should_resurrect_trashed_connection_within_idle_timeout() throws Exc // Given enough time, one connection gets trashed (and the implementation picks the first one) Uninterruptibles.sleepUninterruptibly(20, TimeUnit.SECONDS); - assertThat(pool.connections).containsExactly(connection2); - assertThat(pool.trash).containsExactly(connection1); + assertThat(pool.connections[0]).containsExactly(connection2); + assertThat(pool.trash[0]).containsExactly(connection1); // Now borrow enough to go just under the 1 connection threshold allRequests.addAll(MockRequest.sendMany(50, pool)); - assertThat(pool.connections).containsExactly(connection2); - assertThat(pool.trash).containsExactly(connection1); + assertThat(pool.connections[0]).containsExactly(connection2); + assertThat(pool.trash[0]).containsExactly(connection1); assertThat(connection1.inFlight.get()).isEqualTo(50); assertThat(connection2.inFlight.get()).isEqualTo(50); @@ -612,8 +621,8 @@ public void should_resurrect_trashed_connection_within_idle_timeout() throws Exc verify(factory, after(2000).times(1)).open(any(HostConnectionPool.class)); assertPoolSize(pool, 2); - assertThat(pool.connections).containsExactly(connection2, connection1); - assertThat(pool.trash).isEmpty(); + assertThat(pool.connections[0]).containsExactly(connection2, connection1); + assertThat(pool.trash[0]).isEmpty(); assertThat(connection1.inFlight.get()).isEqualTo(50); assertThat(connection2.inFlight.get()).isEqualTo(51); } finally { @@ -641,7 +650,7 @@ public void should_not_resurrect_trashed_connection_after_idle_timeout() throws HostConnectionPool pool = createPool(cluster, 1, 2); Connection.Factory factory = spy(cluster.manager.connectionFactory); cluster.manager.connectionFactory = factory; - Connection connection1 = pool.connections.get(0); + Connection connection1 = pool.connections[0].get(0); List requests = MockRequest.sendMany(NEW_CONNECTION_THRESHOLD, pool); assertBorrowedConnection(requests, connection1); @@ -651,7 +660,7 @@ public void should_not_resurrect_trashed_connection_after_idle_timeout() throws verify(factory, after(2000).times(1)).open(any(HostConnectionPool.class)); assertPoolSize(pool, 2); reset(factory); - Connection connection2 = pool.connections.get(1); + Connection connection2 = pool.connections[0].get(1); assertThat(connection1.inFlight.get()).isEqualTo(101); assertThat(connection2.inFlight.get()).isEqualTo(0); @@ -664,8 +673,8 @@ public void should_not_resurrect_trashed_connection_after_idle_timeout() throws // Given enough time, one connection gets trashed (and the implementation picks the first one) Uninterruptibles.sleepUninterruptibly(20, TimeUnit.SECONDS); - assertThat(pool.connections).containsExactly(connection2); - assertThat(pool.trash).containsExactly(connection1); + assertThat(pool.connections[0]).containsExactly(connection2); + assertThat(pool.trash[0]).containsExactly(connection1); // Return trashed connection down to 0 inFlight MockRequest.completeMany(50, allRequests); @@ -673,8 +682,8 @@ public void should_not_resurrect_trashed_connection_after_idle_timeout() throws // Give enough time for trashed connection to be cleaned up from the trash: Uninterruptibles.sleepUninterruptibly(30, TimeUnit.SECONDS); - assertThat(pool.connections).containsExactly(connection2); - assertThat(pool.trash).isEmpty(); + assertThat(pool.connections[0]).containsExactly(connection2); + assertThat(pool.trash[0]).isEmpty(); assertThat(connection1.isClosed()).isTrue(); // Fill the live connection to go over the threshold where a second one is needed @@ -718,7 +727,7 @@ public void should_not_close_trashed_connection_until_no_in_flight() throws Exce HostConnectionPool pool = createPool(cluster, 1, 2); Connection.Factory factory = spy(cluster.manager.connectionFactory); cluster.manager.connectionFactory = factory; - Connection connection1 = pool.connections.get(0); + Connection connection1 = pool.connections[0].get(0); // Fill core connection enough to trigger creation of another one List requests = MockRequest.sendMany(NEW_CONNECTION_THRESHOLD, pool); @@ -727,7 +736,7 @@ public void should_not_close_trashed_connection_until_no_in_flight() throws Exce allRequests.add(MockRequest.send(pool)); verify(factory, after(2000).times(1)).open(any(HostConnectionPool.class)); - assertThat(pool.connections).hasSize(2); + assertThat(pool.connections[0]).hasSize(2); // Return enough times to get back under the threshold where one connection is enough MockRequest.completeMany(50, allRequests); @@ -736,7 +745,7 @@ public void should_not_close_trashed_connection_until_no_in_flight() throws Exce // the first one. // It still has in-flight requests so should not get closed. Uninterruptibles.sleepUninterruptibly(30, TimeUnit.SECONDS); - assertThat(pool.trash).containsExactly(connection1); + assertThat(pool.trash[0]).containsExactly(connection1); assertThat(connection1.inFlight.get()).isEqualTo(51); assertThat(connection1.isClosed()).isFalse(); @@ -749,8 +758,8 @@ public void should_not_close_trashed_connection_until_no_in_flight() throws Exce // The connection should be now closed. // The trashed connection should be closed and not in the pool or trash. assertThat(connection1.isClosed()).isTrue(); - assertThat(pool.connections).doesNotContain(connection1); - assertThat(pool.trash).doesNotContain(connection1); + assertThat(pool.connections[0]).doesNotContain(connection1); + assertThat(pool.trash[0]).doesNotContain(connection1); } finally { MockRequest.completeAll(allRequests); cluster.close(); @@ -773,7 +782,7 @@ public void should_trash_on_returning_connection_with_insufficient_streams() thr HostConnectionPool pool = createPool(cluster, 1, 2); Connection.Factory factory = spy(cluster.manager.connectionFactory); cluster.manager.connectionFactory = factory; - Connection core = pool.connections.get(0); + Connection core = pool.connections[0].get(0); List requests = MockRequest.sendMany(NEW_CONNECTION_THRESHOLD, pool); assertBorrowedConnections(requests, Collections.singletonList(core)); @@ -781,15 +790,15 @@ public void should_trash_on_returning_connection_with_insufficient_streams() thr allRequests.add(MockRequest.send(pool)); verify(factory, after(2000).times(1)).open(any(HostConnectionPool.class)); - assertThat(pool.connections).hasSize(2); + assertThat(pool.connections[0]).hasSize(2); // Grab the new non-core connection and replace it with a spy. - Connection extra1 = spy(pool.connections.get(1)); - pool.connections.set(1, extra1); + Connection extra1 = spy(pool.connections[0].get(1)); + pool.connections[0].set(1, extra1); // Borrow 10 times to ensure pool is utilized. allRequests.addAll(MockRequest.sendMany(10, pool)); - assertThat(pool.connections).hasSize(2); + assertThat(pool.connections[0]).hasSize(2); // stub the maxAvailableStreams method to return 0, indicating there are no remaining streams. // this should cause the connection to be replaced and trashed on returnConnection. @@ -797,9 +806,9 @@ public void should_trash_on_returning_connection_with_insufficient_streams() thr // On returning of the connection, should detect that there are no available streams and trash // it. - assertThat(pool.trash).hasSize(0); + assertThat(pool.trash[0]).hasSize(0); pool.returnConnection(extra1); - assertThat(pool.trash).hasSize(1); + assertThat(pool.trash[0]).hasSize(1); } finally { MockRequest.completeAll(allRequests); cluster.close(); @@ -819,8 +828,8 @@ public void should_keep_host_up_when_one_connection_lost() throws Exception { Cluster cluster = createClusterBuilder().build(); try { HostConnectionPool pool = createPool(cluster, 2, 2); - Connection core0 = pool.connections.get(0); - Connection core1 = pool.connections.get(1); + Connection core0 = pool.connections[0].get(0); + Connection core1 = pool.connections[0].get(1); // Drop a connection and ensure the host stays up. currentClient.disableListener(); @@ -830,7 +839,7 @@ public void should_keep_host_up_when_one_connection_lost() throws Exception { // connection 0 should be down, while connection 1 and the Host should remain up. assertThat(core0.isClosed()).isTrue(); assertThat(core1.isClosed()).isFalse(); - assertThat(pool.connections).doesNotContain(core0); + assertThat(pool.connections[0]).doesNotContain(core0); assertThat(cluster).host(1).hasState(Host.State.UP); assertThat(cluster).hasOpenControlConnection(); } finally { @@ -866,7 +875,7 @@ public void should_mark_host_down_when_no_connections_remaining() throws Excepti HostConnectionPool pool = createPool(cluster, 8, 8); // copy list to track these connections. - List connections = newArrayList(pool.connections); + List connections = newArrayList(pool.connections[0]); reset(factory); @@ -950,9 +959,9 @@ public void should_create_new_connections_when_connection_lost_and_under_core_co cluster.manager.blockingExecutor = blockingExecutor; HostConnectionPool pool = createPool(cluster, 3, 3); - Connection core0 = pool.connections.get(0); - Connection core1 = pool.connections.get(1); - Connection core2 = pool.connections.get(2); + Connection core0 = pool.connections[0].get(0); + Connection core1 = pool.connections[0].get(1); + Connection core2 = pool.connections[0].get(2); // Drop two core connections. // Disable new connections initially and we'll eventually reenable it. @@ -963,7 +972,7 @@ public void should_create_new_connections_when_connection_lost_and_under_core_co // Since we have a connection left the host should remain up. assertThat(cluster).host(1).hasState(Host.State.UP); - assertThat(pool.connections).hasSize(1); + assertThat(pool.connections[0]).hasSize(1); // The borrowed connection should be the open one. MockRequest request = MockRequest.send(pool); @@ -1055,7 +1064,7 @@ public void should_not_schedule_reconnect_when_connection_lost_and_at_core_conne cluster.manager.connectionFactory = factory; HostConnectionPool pool = createPool(cluster, 1, 2); - Connection core0 = pool.connections.get(0); + Connection core0 = pool.connections[0].get(0); // Create enough inFlight requests to spawn another connection. List core0requests = newArrayList(); @@ -1067,13 +1076,13 @@ public void should_not_schedule_reconnect_when_connection_lost_and_at_core_conne // Pool should grow by 1. verify(factory, after(2000).times(1)).open(any(HostConnectionPool.class)); - assertThat(pool.connections).hasSize(2); + assertThat(pool.connections[0]).hasSize(2); // Reset factory mock as we'll be checking for new open() invokes later. reset(factory); // Grab the new non-core connection. - Connection extra1 = pool.connections.get(1); + Connection extra1 = pool.connections[0].get(1); // Drop a connection and disable listening. currentClient.closeConnection(CLOSE, ((InetSocketAddress) core0.channel.localAddress())); @@ -1091,7 +1100,7 @@ public void should_not_schedule_reconnect_when_connection_lost_and_at_core_conne List requests = MockRequest.sendMany(100, pool); assertBorrowedConnections(requests, Collections.singletonList(extra1)); allRequests.addAll(requests); - assertThat(pool.connections).hasSize(1); + assertThat(pool.connections[0]).hasSize(1); // A new connection should never have been spawned since we didn't max out core. verify(factory, after(readTimeout).never()).open(any(HostConnectionPool.class)); @@ -1104,7 +1113,7 @@ public void should_not_schedule_reconnect_when_connection_lost_and_at_core_conne // After some time the a connection should attempt to be opened (but will fail). verify(factory, timeout(readTimeout)).open(any(HostConnectionPool.class)); assertPoolSize(pool, 1); - assertThat(pool.connections).hasSize(1); + assertThat(pool.connections[0]).hasSize(1); // Wait some reasonable amount of time for connection to reestablish then check pool size. Uninterruptibles.sleepUninterruptibly(readTimeout * 2, TimeUnit.MILLISECONDS); @@ -1123,7 +1132,7 @@ public void should_not_schedule_reconnect_when_connection_lost_and_at_core_conne // Wait some reasonable amount of time for connection to reestablish then check pool size. Uninterruptibles.sleepUninterruptibly(readTimeout, TimeUnit.MILLISECONDS); // Reconnecting should have exceeded and pool will have grown. - assertThat(pool.connections).hasSize(2); + assertThat(pool.connections[0]).hasSize(2); // Borrowed connection should be the newly spawned connection since the other one has some // inflight requests. @@ -1171,7 +1180,7 @@ public void should_not_mark_host_down_if_some_connections_fail_on_init() throws reset(factory); // Pool size should show all successful connections. - assertThat(pool.connections).hasSize(4); + assertThat(pool.connections[0]).hasSize(4); // Control connection should remain up in addition to to host. assertThat(cluster).host(1).hasState(Host.State.UP); @@ -1235,7 +1244,7 @@ public void should_throw_exception_if_convicted_and_no_connections_available() { reset(factory); // Pool should be empty. - assertThat(pool.connections).hasSize(0); + assertThat(pool.connections[0]).hasSize(0); // Control connection should stay up with the host. assertThat(cluster).host(1).hasState(Host.State.UP); @@ -1288,7 +1297,7 @@ public void should_wait_on_connection_if_not_convicted_and_no_connections_availa HostConnectionPool pool = createPool(cluster, 8, 8); // Pool should be empty. - assertThat(pool.connections).hasSize(0); + assertThat(pool.connections[0]).hasSize(0); // Control connection should stay up with the host. assertThat(cluster).host(1).hasState(Host.State.UP); @@ -1344,7 +1353,7 @@ public void should_wait_on_connection_if_zero_core_connections() throws Exceptio HostConnectionPool pool = createPool(cluster, 0, 2); // Pool should be empty. - assertThat(pool.connections).hasSize(0); + assertThat(pool.connections[0]).hasSize(0); // Control connection should stay up with the host. assertThat(cluster).host(1).hasState(Host.State.UP); @@ -1477,7 +1486,7 @@ public void run() { verify(stateListener, times(1)).onUp(pool.host); // Pool should be empty. - assertThat(pool.connections).hasSize(0); + assertThat(pool.connections[0]).hasSize(0); // Control connection should stay up with the host. assertThat(cluster).host(1).hasState(Host.State.UP); @@ -1581,7 +1590,10 @@ private static void completeAll(List requests) { private MockRequest(HostConnectionPool pool, int timeoutMillis, int maxQueueSize) throws ConnectionException { - this.connectionFuture = pool.borrowConnection(timeoutMillis, MILLISECONDS, maxQueueSize); + ByteBuffer routingKey = ByteBuffer.allocate(4); + routingKey.putInt(0, 0); + this.connectionFuture = + pool.borrowConnection(timeoutMillis, MILLISECONDS, maxQueueSize, routingKey); requestInitialized = GuavaCompatibility.INSTANCE.transform( this.connectionFuture, diff --git a/driver-core/src/test/java/com/datastax/driver/core/RequestHandlerTest.java b/driver-core/src/test/java/com/datastax/driver/core/RequestHandlerTest.java index 22d7aad2dc8..a5986a4b2fc 100644 --- a/driver-core/src/test/java/com/datastax/driver/core/RequestHandlerTest.java +++ b/driver-core/src/test/java/com/datastax/driver/core/RequestHandlerTest.java @@ -91,6 +91,6 @@ public void should_handle_race_between_response_and_cancellation() { private Connection getSingleConnection(Session session) { HostConnectionPool pool = ((SessionManager) session).pools.values().iterator().next(); - return pool.connections.get(0); + return pool.connections[0].get(0); } } From 702d5f24f23f489eb36182922b12f7489f2ef883 Mon Sep 17 00:00:00 2001 From: Piotr Jastrzebski Date: Fri, 19 Apr 2019 12:45:53 +0200 Subject: [PATCH 09/52] Signal usage of optimized driver Signed-off-by: Piotr Jastrzebski --- driver-core/src/main/java/com/datastax/driver/core/Cluster.java | 1 + 1 file changed, 1 insertion(+) 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 b89e4c71c88..77cff96ad68 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 @@ -175,6 +175,7 @@ private Cluster( List contactPoints, Configuration configuration, Collection listeners) { + System.out.println("===== Using optimized driver!!! ====="); this.manager = new Manager(name, contactPoints, configuration, listeners); } From 1e6338d47a7d892dae404abd10dccb5880da34f1 Mon Sep 17 00:00:00 2001 From: Piotr Jastrzebski Date: Fri, 19 Apr 2019 13:42:14 +0200 Subject: [PATCH 10/52] Add lastHost field to Statement This field will be used for paging queries to fetch all pages from the same host if possible. Signed-off-by: Piotr Jastrzebski --- .../main/java/com/datastax/driver/core/Statement.java | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/driver-core/src/main/java/com/datastax/driver/core/Statement.java b/driver-core/src/main/java/com/datastax/driver/core/Statement.java index 56ffdcbc1bf..dc5b5eca270 100644 --- a/driver-core/src/main/java/com/datastax/driver/core/Statement.java +++ b/driver-core/src/main/java/com/datastax/driver/core/Statement.java @@ -74,6 +74,7 @@ public ConsistencyLevel getConsistencyLevel() { protected volatile Boolean idempotent; private volatile Map outgoingPayload; private volatile Host host; + private volatile Host lastHost; // We don't want to expose the constructor, because the code relies on this being only sub-classed // by RegularStatement, BoundStatement and BatchStatement @@ -648,4 +649,12 @@ public Statement setHost(Host host) { this.host = host; return this; } + + public Host getLastHost() { + return lastHost; + } + + public void setLastHost(Host host) { + lastHost = host; + } } From 4106ab1969c8ce89ee4b99247d370898dbbe9527 Mon Sep 17 00:00:00 2001 From: Piotr Jastrzebski Date: Wed, 24 Oct 2018 16:43:10 +0200 Subject: [PATCH 11/52] Add PagingOptimizingLoadBalancingPolicy This policy will try to fetch all pages from the same host if possible. Signed-off-by: Piotr Jastrzebski Signed-off-by: Piotr Jastrzebski --- .../PagingOptimizingLoadBalancingPolicy.java | 102 ++++++++++++++++++ 1 file changed, 102 insertions(+) create mode 100644 driver-core/src/main/java/com/datastax/driver/core/policies/PagingOptimizingLoadBalancingPolicy.java diff --git a/driver-core/src/main/java/com/datastax/driver/core/policies/PagingOptimizingLoadBalancingPolicy.java b/driver-core/src/main/java/com/datastax/driver/core/policies/PagingOptimizingLoadBalancingPolicy.java new file mode 100644 index 00000000000..edfe690900e --- /dev/null +++ b/driver-core/src/main/java/com/datastax/driver/core/policies/PagingOptimizingLoadBalancingPolicy.java @@ -0,0 +1,102 @@ +// Copyright (C) 2018 ScyllaDB +// Use of this source code is governed by a ALv2-style +// license that can be found in the LICENSE file. + +package com.datastax.driver.core.policies; + +import com.datastax.driver.core.Cluster; +import com.datastax.driver.core.Host; +import com.datastax.driver.core.HostDistance; +import com.datastax.driver.core.Statement; +import java.util.Collection; +import java.util.Iterator; +import java.util.concurrent.CopyOnWriteArrayList; + +public class PagingOptimizingLoadBalancingPolicy implements LoadBalancingPolicy { + private final LoadBalancingPolicy wrapped; + private volatile CopyOnWriteArrayList hosts; + + public PagingOptimizingLoadBalancingPolicy(LoadBalancingPolicy loadBalancingPolicy) { + wrapped = loadBalancingPolicy; + } + + @Override + public void init(Cluster cluster, Collection hosts) { + this.hosts = new CopyOnWriteArrayList(hosts); + wrapped.init(cluster, hosts); + } + + @Override + public HostDistance distance(Host host) { + return wrapped.distance(host); + } + + private class WithFirstIterator implements Iterator { + + private Host firstToReturn; + private final Iterator wrapped; + + public WithFirstIterator(Host host, Iterator iterator) { + firstToReturn = host; + wrapped = iterator; + } + + @Override + public boolean hasNext() { + return wrapped.hasNext() || hasValidFirstToReturn(); + } + + @Override + public Host next() { + if (hasValidFirstToReturn()) { + final Host result = firstToReturn; + firstToReturn = null; + return result; + } + return wrapped.next(); + } + + private boolean hasValidFirstToReturn() { + if (firstToReturn == null) { + return false; + } + return hosts.contains(firstToReturn); + } + } + + @Override + public Iterator newQueryPlan(String loggedKeyspace, Statement statement) { + final Iterator inner = wrapped.newQueryPlan(loggedKeyspace, statement); + final Host lastHost = statement.getLastHost(); + return lastHost == null ? inner : new WithFirstIterator(lastHost, inner); + } + + @Override + public void onAdd(Host host) { + hosts.addIfAbsent(host); + wrapped.onAdd(host); + } + + @Override + public void onUp(Host host) { + hosts.addIfAbsent(host); + wrapped.onUp(host); + } + + @Override + public void onDown(Host host) { + hosts.remove(host); + wrapped.onDown(host); + } + + @Override + public void onRemove(Host host) { + hosts.remove(host); + wrapped.onRemove(host); + } + + @Override + public void close() { + wrapped.close(); + } +} From 6511f1a532af54be65838abc1ef9283743832d03 Mon Sep 17 00:00:00 2001 From: Piotr Jastrzebski Date: Fri, 19 Apr 2019 13:52:45 +0200 Subject: [PATCH 12/52] Use PagingLoadBalancingPolicy in Cluster Signed-off-by: Piotr Jastrzebski --- .../com/datastax/driver/core/Cluster.java | 29 ++++++++++++++++++- 1 file changed, 28 insertions(+), 1 deletion(-) 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 77cff96ad68..cd5fd58f587 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 @@ -28,6 +28,7 @@ import com.datastax.driver.core.policies.IdentityTranslator; import com.datastax.driver.core.policies.LatencyAwarePolicy; import com.datastax.driver.core.policies.LoadBalancingPolicy; +import com.datastax.driver.core.policies.PagingOptimizingLoadBalancingPolicy; import com.datastax.driver.core.policies.Policies; import com.datastax.driver.core.policies.ReconnectionPolicy; import com.datastax.driver.core.policies.RetryPolicy; @@ -1445,7 +1446,33 @@ private Manager( Configuration configuration, Collection listeners) { this.clusterName = clusterName == null ? generateClusterName() : clusterName; - this.configuration = configuration; + if (configuration != null && configuration.getPolicies() != null) { + Policies policies = configuration.getPolicies(); + this.configuration = + Configuration.builder() + .withPolicies( + Policies.builder() + .withLoadBalancingPolicy( + new PagingOptimizingLoadBalancingPolicy( + policies.getLoadBalancingPolicy())) + .withReconnectionPolicy(policies.getReconnectionPolicy()) + .withRetryPolicy(policies.getRetryPolicy()) + .withAddressTranslator(policies.getAddressTranslator()) + .withTimestampGenerator(policies.getTimestampGenerator()) + .withSpeculativeExecutionPolicy(policies.getSpeculativeExecutionPolicy()) + .build()) + .withProtocolOptions(configuration.getProtocolOptions()) + .withPoolingOptions(configuration.getPoolingOptions()) + .withSocketOptions(configuration.getSocketOptions()) + .withMetricsOptions(configuration.getMetricsOptions()) + .withQueryOptions(configuration.getQueryOptions()) + .withThreadingOptions(configuration.getThreadingOptions()) + .withNettyOptions(configuration.getNettyOptions()) + .withCodecRegistry(configuration.getCodecRegistry()) + .build(); + } else { + this.configuration = configuration; + } this.contactPoints = contactPoints; this.listeners = new CopyOnWriteArraySet(listeners); } From 46d29d2e57cef9e0a834e4935ffe81317f0e9e1b Mon Sep 17 00:00:00 2001 From: Piotr Jastrzebski Date: Wed, 24 Oct 2018 17:07:39 +0200 Subject: [PATCH 13/52] Add PagingOptimizingLatencyTracker Signed-off-by: Piotr Jastrzebski Signed-off-by: Piotr Jastrzebski --- .../core/PagingOptimizingLatencyTracker.java | 26 +++++++++++++++++++ 1 file changed, 26 insertions(+) create mode 100644 driver-core/src/main/java/com/datastax/driver/core/PagingOptimizingLatencyTracker.java diff --git a/driver-core/src/main/java/com/datastax/driver/core/PagingOptimizingLatencyTracker.java b/driver-core/src/main/java/com/datastax/driver/core/PagingOptimizingLatencyTracker.java new file mode 100644 index 00000000000..aab0d9100c7 --- /dev/null +++ b/driver-core/src/main/java/com/datastax/driver/core/PagingOptimizingLatencyTracker.java @@ -0,0 +1,26 @@ +// Copyright (C) 2018 ScyllaDB +// Use of this source code is governed by a ALv2-style +// license that can be found in the LICENSE file. + +package com.datastax.driver.core; + +class PagingOptimizingLatencyTracker implements LatencyTracker { + + @Override + public void update(Host host, Statement statement, Exception exception, long newLatencyNanos) { + if (exception == null) { + statement.setLastHost(host); + } else { + final Host lastHost = statement.getLastHost(); + if (lastHost != null && lastHost.equals(host)) { + statement.setLastHost(null); + } + } + } + + @Override + public void onRegister(Cluster cluster) {} + + @Override + public void onUnregister(Cluster cluster) {} +} From 565d4efb0605fcdeca4a9c08b25f48369efe522a Mon Sep 17 00:00:00 2001 From: Piotr Jastrzebski Date: Fri, 19 Apr 2019 13:59:25 +0200 Subject: [PATCH 14/52] Use PagingOptimizingLatencyTracker Signed-off-by: Piotr Jastrzebski --- driver-core/src/main/java/com/datastax/driver/core/Cluster.java | 1 + 1 file changed, 1 insertion(+) 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 cd5fd58f587..97003e9cb85 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 @@ -207,6 +207,7 @@ private Cluster( * of an error while initializing the Cluster. */ public Cluster init() { + register(new PagingOptimizingLatencyTracker()); this.manager.init(); return this; } From 972e3f50af9088e2e8a7574e149a43ee2b9b51c7 Mon Sep 17 00:00:00 2001 From: Piotr Jastrzebski Date: Fri, 19 Apr 2019 14:02:44 +0200 Subject: [PATCH 15/52] Prevent schema.local full scans Signed-off-by: Piotr Jastrzebski --- .../src/main/java/com/datastax/driver/core/Connection.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) 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 5b148c561e4..93349997d67 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 @@ -429,7 +429,9 @@ private ListenableFuture checkClusterName( DefaultResultSetFuture clusterNameFuture = new DefaultResultSetFuture( - null, protocolVersion, new Requests.Query("select cluster_name from system.local")); + null, + protocolVersion, + new Requests.Query("select cluster_name from system.local where key='local'")); try { write(clusterNameFuture); return GuavaCompatibility.INSTANCE.transformAsync( From 4e10d5d062a8ac978b478fb2b7488b5d2f128493 Mon Sep 17 00:00:00 2001 From: Piotr Jastrzebski Date: Fri, 19 Apr 2019 14:05:08 +0200 Subject: [PATCH 16/52] Log channel creation exceptions Signed-off-by: Piotr Jastrzebski --- .../src/main/java/com/datastax/driver/core/Connection.java | 3 +++ 1 file changed, 3 insertions(+) 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 93349997d67..6b585815048 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 @@ -199,6 +199,9 @@ ListenableFuture initAsync() { @Override public void operationComplete(ChannelFuture future) throws Exception { writer.decrementAndGet(); + if (future.cause() != null) { + logger.warn("Error creating netty channel to " + address, future.cause()); + } channel = future.channel(); if (isClosed()) { channel From b6507ecde053c3e74bf7524adbd9417985443881 Mon Sep 17 00:00:00 2001 From: Piotr Jastrzebski Date: Fri, 19 Apr 2019 14:39:20 +0200 Subject: [PATCH 17/52] Stop cleaning uninitialized pool HostConnectionPool.cleanupIdleConnections is called from a recurring task. It can race with the initialization of the pool leading to NPE and other concurrency problems. Signed-off-by: Piotr Jastrzebski --- .../main/java/com/datastax/driver/core/HostConnectionPool.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 bf775d31ebb..6e5ae0ed5af 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 @@ -692,7 +692,7 @@ public void onConnectionDefunct(final Connection connection) { } void cleanupIdleConnections(long now) { - if (isClosed()) return; + if (isClosed() || phase.get() != Phase.READY) return; shrinkIfBelowCapacity(); cleanupTrash(now); From 5ce83a16a31b5a391762e052658df1ea94e59b9d Mon Sep 17 00:00:00 2001 From: Shlomi Livne Date: Tue, 6 Aug 2019 11:44:59 +0300 Subject: [PATCH 18/52] Optional coalscing of schema refresh queries By default the driver has the ability to coalesce schem refresh queries to try and reduce the number of queries sent. In case a view and a table refresh is needed a refresh for the keyspace would be generated. While in most cases thisoptimization is helpfull - in case there are 100s of tables in a single keyspace this will cause fetching all keyspace information which will cause extra load on the system (in scylla's case a single shard). Add the ability to disable this by reusing setting of maxPendingRefreshSchmaRequests value. If its 1 disable coalescing. Signed-off-by: Shlomi Livne --- .../com/datastax/driver/core/Cluster.java | 31 +++++++++++++------ 1 file changed, 21 insertions(+), 10 deletions(-) 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 97003e9cb85..906ce722b6c 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 @@ -2887,17 +2887,28 @@ public ListenableFuture deliver(final List events) { new ExceptionCatchingRunnable() { @Override public void runMayThrow() throws InterruptedException, ExecutionException { - SchemaRefreshRequest coalesced = null; - for (SchemaRefreshRequest request : events) { - coalesced = coalesced == null ? request : coalesced.coalesce(request); + if (schemaRefreshRequestDebouncer.maxPendingEvents() > 1) { + SchemaRefreshRequest coalesced = null; + for (SchemaRefreshRequest request : events) { + coalesced = coalesced == null ? request : coalesced.coalesce(request); + } + assert coalesced != null; + logger.trace("Coalesced schema refresh request: {}", coalesced); + controlConnection.refreshSchema( + coalesced.targetType, + coalesced.targetKeyspace, + coalesced.targetName, + coalesced.targetSignature); + } else { + for (SchemaRefreshRequest request : events) { + logger.trace("Schema refresh request: {}", request); + controlConnection.refreshSchema( + request.targetType, + request.targetKeyspace, + request.targetName, + request.targetSignature); + } } - assert coalesced != null; - logger.trace("Coalesced schema refresh request: {}", coalesced); - controlConnection.refreshSchema( - coalesced.targetType, - coalesced.targetKeyspace, - coalesced.targetName, - coalesced.targetSignature); } }); } From 934fd94cb54fd26b48d984d9bc8e88db863951ec Mon Sep 17 00:00:00 2001 From: Piotr Jastrzebski Date: Thu, 25 Apr 2019 10:36:33 +0200 Subject: [PATCH 19/52] Limit number of opened connections Signed-off-by: Piotr Jastrzebski --- .../driver/core/HostConnectionPool.java | 18 +++++++++++++----- 1 file changed, 13 insertions(+), 5 deletions(-) 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 6e5ae0ed5af..70915f5e494 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 @@ -36,7 +36,9 @@ import com.google.common.util.concurrent.Uninterruptibles; import io.netty.util.concurrent.EventExecutor; import java.nio.ByteBuffer; +import java.util.ArrayDeque; import java.util.ArrayList; +import java.util.Collection; import java.util.List; import java.util.Queue; import java.util.Random; @@ -184,7 +186,7 @@ ListenableFuture initAsyncWithConnection(Connection reusedConnection) { final SettableFuture initFuture = SettableFuture.create(); - addCallback(connections, connectionFutures, initFuture, new ArrayList()); + addCallback(connections, connectionFutures, initFuture, new ArrayDeque()); return initFuture; } @@ -193,7 +195,7 @@ private void addCallback( final List connections, final List> connectionFutures, final SettableFuture initFuture, - final List toClose) { + final Queue toClose) { final Executor initExecutor = manager.cluster.manager.configuration.getPoolingOptions().getInitializationExecutor(); @@ -238,10 +240,16 @@ public void onSuccess(List l) { 0, HostConnectionPool.this.connectionsPerShard - shardsConnections.size()); } if (needed > 0) { + int factor = (hostDistance == HostDistance.LOCAL) ? 2 : 1; + int limit = + HostConnectionPool.this.connections.length * connectionsPerShard * factor; + final List> connectionFutures = + Lists.newArrayListWithCapacity(needed + Math.max(0, toClose.size() - limit)); + while (toClose.size() > limit) { + connectionFutures.add(toClose.poll().closeAsync()); + } final List connections = manager.connectionFactory().newConnections(HostConnectionPool.this, needed); - final List> connectionFutures = - Lists.newArrayListWithCapacity(needed); for (Connection connection : connections) { ListenableFuture connectionFuture = connection.initAsync(); connectionFutures.add(handleErrors(connectionFuture, initExecutor)); @@ -301,7 +309,7 @@ public ListenableFuture apply(Throwable t) throws Exception { // Clean up if we got a fatal error at construction time but still created part of the core // connections - private void forceClose(List connections) { + private void forceClose(Collection connections) { for (Connection connection : connections) { connection.closeAsync().force(); } From e411a57909bd1313a44f14a097584c1c59948042 Mon Sep 17 00:00:00 2001 From: Piotr Jastrzebski Date: Wed, 7 Aug 2019 14:36:19 +0200 Subject: [PATCH 20/52] Prepare for publishing to Maven repo Make sure groupId is set everywhere to com.scylladb and artifactId has scylla- prefix instead of cassandra- prefix. Signed-off-by: Piotr Jastrzebski --- README.md | 18 +-- driver-core/pom.xml | 12 +- driver-dist/pom.xml | 32 +++-- driver-dist/src/assembly/binary-tarball.xml | 38 +++--- driver-examples/pom.xml | 20 +-- .../examples/json/JacksonJsonColumn.java | 4 +- .../examples/json/JacksonJsonFunction.java | 4 +- .../driver/examples/json/JacksonJsonRow.java | 4 +- .../examples/json/Jsr353JsonColumn.java | 4 +- .../examples/json/Jsr353JsonFunction.java | 4 +- .../driver/examples/json/Jsr353JsonRow.java | 4 +- driver-extras/pom.xml | 22 ++-- driver-mapping/pom.xml | 20 +-- driver-tests/osgi/pom.xml | 28 ++-- .../datastax/driver/osgi/BundleOptions.java | 6 +- driver-tests/pom.xml | 12 +- driver-tests/shading/pom.xml | 10 +- driver-tests/shading/shaded/pom.xml | 14 +- driver-tests/shading/unshaded/pom.xml | 14 +- driver-tests/stress/bin/stress | 2 +- driver-tests/stress/pom.xml | 16 +-- manual/compression/README.md | 2 +- manual/custom_codecs/extras/README.md | 6 +- manual/metrics/README.md | 6 +- manual/object_mapper/README.md | 6 +- manual/shaded_jar/README.md | 22 ++-- pom.xml | 124 ++++++++++++++---- 27 files changed, 269 insertions(+), 185 deletions(-) diff --git a/README.md b/README.md index 51df608aae8..0e29a674998 100644 --- a/README.md +++ b/README.md @@ -68,9 +68,9 @@ using DataStax Enterprise, install the [DataStax Enterprise Java driver][dse-dri ```xml - com.datastax.cassandra - cassandra-driver-core - 3.7.1 + com.scylladb + scylla-driver-core + 3.7.1-scylla-0-SNAPSHOT ``` @@ -78,9 +78,9 @@ Note that the object mapper is published as a separate artifact: ```xml - com.datastax.cassandra - cassandra-driver-mapping - 3.7.1 + com.scylladb + scylla-driver-mapping + 3.7.1-scylla-0-SNAPSHOT ``` @@ -88,9 +88,9 @@ The 'extras' module is also published as a separate artifact: ```xml - com.datastax.cassandra - cassandra-driver-extras - 3.7.1 + com.scylladb + scylla-driver-extras + 3.7.1-scylla-0-SNAPSHOT ``` diff --git a/driver-core/pom.xml b/driver-core/pom.xml index 5c4a954d218..7b1df2c4201 100644 --- a/driver-core/pom.xml +++ b/driver-core/pom.xml @@ -20,15 +20,15 @@ 4.0.0 - com.datastax.cassandra - cassandra-driver-parent - 3.7.1 + com.scylladb + scylla-driver-parent + 3.7.1-scylla-0-SNAPSHOT - cassandra-driver-core - DataStax Java Driver for Apache Cassandra - Core + scylla-driver-core + Java Driver for Scylla and Apache Cassandra - Core - A driver for Apache Cassandra 1.2+ that works exclusively with the Cassandra Query Language version 3 + A driver for Scylla and Apache Cassandra 1.2+ that works exclusively with the Cassandra Query Language version 3 (CQL3) and Cassandra's binary protocol. diff --git a/driver-dist/pom.xml b/driver-dist/pom.xml index 72ffa9fb504..b177150548e 100644 --- a/driver-dist/pom.xml +++ b/driver-dist/pom.xml @@ -20,39 +20,39 @@ 4.0.0 - com.datastax.cassandra - cassandra-driver-parent - 3.7.1 + com.scylladb + scylla-driver-parent + 3.7.1-scylla-0-SNAPSHOT - cassandra-driver-dist + scylla-driver-dist jar - DataStax Java Driver for Apache Cassandra - Binary distribution + Java Driver for Scylla and Apache Cassandra - Binary distribution - com.datastax.cassandra - cassandra-driver-core + com.scylladb + scylla-driver-core - com.datastax.cassandra - cassandra-driver-mapping + com.scylladb + scylla-driver-mapping - com.datastax.cassandra - cassandra-driver-extras + com.scylladb + scylla-driver-extras - cassandra-java-driver-${project.version} + scylla-java-driver-${project.version} @@ -87,6 +87,14 @@ true + + org.apache.maven.plugins + maven-gpg-plugin + 1.5 + + true + + diff --git a/driver-dist/src/assembly/binary-tarball.xml b/driver-dist/src/assembly/binary-tarball.xml index 1b9d2adf796..43f627b73e3 100644 --- a/driver-dist/src/assembly/binary-tarball.xml +++ b/driver-dist/src/assembly/binary-tarball.xml @@ -26,11 +26,11 @@ - + true - com.datastax.cassandra:cassandra-driver-core + com.scylladb:scylla-driver-core false @@ -39,9 +39,9 @@ lib - com.datastax.cassandra:cassandra-driver-core - com.datastax.cassandra:cassandra-driver-mapping - com.datastax.cassandra:cassandra-driver-extras + com.scylladb:scylla-driver-core + com.scylladb:scylla-driver-mapping + com.scylladb:scylla-driver-extras io.netty:netty-transport-native-epoll:* @@ -59,11 +59,11 @@ - + true - com.datastax.cassandra:cassandra-driver-mapping + com.scylladb:scylla-driver-mapping false @@ -72,9 +72,9 @@ lib/mapping - com.datastax.cassandra:cassandra-driver-core - com.datastax.cassandra:cassandra-driver-mapping - com.datastax.cassandra:cassandra-driver-extras + com.scylladb:scylla-driver-core + com.scylladb:scylla-driver-mapping + com.scylladb:scylla-driver-extras com.google.guava:guava org.slf4j:slf4j-api @@ -85,11 +85,11 @@ - + true - com.datastax.cassandra:cassandra-driver-extras + com.scylladb:scylla-driver-extras false @@ -98,9 +98,9 @@ lib/extras - com.datastax.cassandra:cassandra-driver-core - com.datastax.cassandra:cassandra-driver-mapping - com.datastax.cassandra:cassandra-driver-extras + com.scylladb:scylla-driver-core + com.scylladb:scylla-driver-mapping + com.scylladb:scylla-driver-extras com.google.guava:guava org.slf4j:slf4j-api @@ -115,10 +115,10 @@ true - com.datastax.cassandra:cassandra-driver-core - com.datastax.cassandra:cassandra-driver-mapping - com.datastax.cassandra:cassandra-driver-extras - com.datastax.cassandra:cassandra-driver-examples + com.scylladb:scylla-driver-core + com.scylladb:scylla-driver-mapping + com.scylladb:scylla-driver-extras + com.scylladb:scylla-driver-examples false diff --git a/driver-examples/pom.xml b/driver-examples/pom.xml index 0027db86f5f..64d78c7a723 100644 --- a/driver-examples/pom.xml +++ b/driver-examples/pom.xml @@ -20,27 +20,27 @@ 4.0.0 - com.datastax.cassandra - cassandra-driver-parent - 3.7.1 + com.scylladb + scylla-driver-parent + 3.7.1-scylla-0-SNAPSHOT - cassandra-driver-examples - DataStax Java Driver for Apache Cassandra - Examples - A collection of examples to demonstrate DataStax Java Driver for Apache Cassandra. + scylla-driver-examples + Java Driver for Scylla and Apache Cassandra - Examples + A collection of examples to demonstrate Java Driver for Scylla and Apache Cassandra. - com.datastax.cassandra - cassandra-driver-core + com.scylladb + scylla-driver-core - com.datastax.cassandra - cassandra-driver-extras + com.scylladb + scylla-driver-extras true diff --git a/driver-examples/src/main/java/com/datastax/driver/examples/json/JacksonJsonColumn.java b/driver-examples/src/main/java/com/datastax/driver/examples/json/JacksonJsonColumn.java index 8b35dbfbbc5..a06b9df62f4 100644 --- a/driver-examples/src/main/java/com/datastax/driver/examples/json/JacksonJsonColumn.java +++ b/driver-examples/src/main/java/com/datastax/driver/examples/json/JacksonJsonColumn.java @@ -42,8 +42,8 @@ * *
{@code
  * 
- *     com.datastax.cassandra
- *     cassandra-driver-extras
+ *     com.scylladb
+ *     scylla-driver-extras
  *     ${driver.version}
  * 
  *
diff --git a/driver-examples/src/main/java/com/datastax/driver/examples/json/JacksonJsonFunction.java b/driver-examples/src/main/java/com/datastax/driver/examples/json/JacksonJsonFunction.java
index 94a20ae99ba..533c79b1025 100644
--- a/driver-examples/src/main/java/com/datastax/driver/examples/json/JacksonJsonFunction.java
+++ b/driver-examples/src/main/java/com/datastax/driver/examples/json/JacksonJsonFunction.java
@@ -47,8 +47,8 @@
  *
  * 
{@code
  * 
- *     com.datastax.cassandra
- *     cassandra-driver-extras
+ *     com.scylladb
+ *     scylla-driver-extras
  *     ${driver.version}
  * 
  *
diff --git a/driver-examples/src/main/java/com/datastax/driver/examples/json/JacksonJsonRow.java b/driver-examples/src/main/java/com/datastax/driver/examples/json/JacksonJsonRow.java
index f2b6048912b..4e2ea8a6272 100644
--- a/driver-examples/src/main/java/com/datastax/driver/examples/json/JacksonJsonRow.java
+++ b/driver-examples/src/main/java/com/datastax/driver/examples/json/JacksonJsonRow.java
@@ -43,8 +43,8 @@
  *
  * 
{@code
  * 
- *     com.datastax.cassandra
- *     cassandra-driver-extras
+ *     com.scylladb
+ *     scylla-driver-extras
  *     ${driver.version}
  * 
  *
diff --git a/driver-examples/src/main/java/com/datastax/driver/examples/json/Jsr353JsonColumn.java b/driver-examples/src/main/java/com/datastax/driver/examples/json/Jsr353JsonColumn.java
index 101e6a51585..5f16e400960 100644
--- a/driver-examples/src/main/java/com/datastax/driver/examples/json/Jsr353JsonColumn.java
+++ b/driver-examples/src/main/java/com/datastax/driver/examples/json/Jsr353JsonColumn.java
@@ -44,8 +44,8 @@
  *
  * 
{@code
  * 
- *     com.datastax.cassandra
- *     cassandra-driver-extras
+ *     com.scylladb
+ *     scylla-driver-extras
  *     ${driver.version}
  * 
  *
diff --git a/driver-examples/src/main/java/com/datastax/driver/examples/json/Jsr353JsonFunction.java b/driver-examples/src/main/java/com/datastax/driver/examples/json/Jsr353JsonFunction.java
index b78928da63d..6d8c5cf0f88 100644
--- a/driver-examples/src/main/java/com/datastax/driver/examples/json/Jsr353JsonFunction.java
+++ b/driver-examples/src/main/java/com/datastax/driver/examples/json/Jsr353JsonFunction.java
@@ -45,8 +45,8 @@
  *
  * 
{@code
  * 
- *     com.datastax.cassandra
- *     cassandra-driver-extras
+ *     com.scylladb
+ *     scylla-driver-extras
  *     ${driver.version}
  * 
  *
diff --git a/driver-examples/src/main/java/com/datastax/driver/examples/json/Jsr353JsonRow.java b/driver-examples/src/main/java/com/datastax/driver/examples/json/Jsr353JsonRow.java
index 15a45c53c97..98d2a32d68f 100644
--- a/driver-examples/src/main/java/com/datastax/driver/examples/json/Jsr353JsonRow.java
+++ b/driver-examples/src/main/java/com/datastax/driver/examples/json/Jsr353JsonRow.java
@@ -44,8 +44,8 @@
  *
  * 
{@code
  * 
- *     com.datastax.cassandra
- *     cassandra-driver-extras
+ *     com.scylladb
+ *     scylla-driver-extras
  *     ${driver.version}
  * 
  *
diff --git a/driver-extras/pom.xml b/driver-extras/pom.xml
index a5d37984a9f..3ffe66a2b3e 100644
--- a/driver-extras/pom.xml
+++ b/driver-extras/pom.xml
@@ -20,21 +20,21 @@
     4.0.0
 
     
-        com.datastax.cassandra
-        cassandra-driver-parent
-        3.7.1
+        com.scylladb
+        scylla-driver-parent
+        3.7.1-scylla-0-SNAPSHOT
     
 
-    cassandra-driver-extras
+    scylla-driver-extras
     bundle
-    DataStax Java Driver for Apache Cassandra - Extras
+    Java Driver for Scylla and Apache Cassandra - Extras
     Extended functionality for the Java driver.
 
     
 
         
-            com.datastax.cassandra
-            cassandra-driver-core
+            com.scylladb
+            scylla-driver-core
         
 
         
@@ -67,15 +67,15 @@
         
 
         
-            com.datastax.cassandra
-            cassandra-driver-core
+            com.scylladb
+            scylla-driver-core
             test-jar
             test
         
 
         
-            com.datastax.cassandra
-            cassandra-driver-mapping
+            com.scylladb
+            scylla-driver-mapping
             test
         
 
diff --git a/driver-mapping/pom.xml b/driver-mapping/pom.xml
index bc60e34ee80..bc6ef0b2bb5 100644
--- a/driver-mapping/pom.xml
+++ b/driver-mapping/pom.xml
@@ -20,21 +20,21 @@
     4.0.0
 
     
-        com.datastax.cassandra
-        cassandra-driver-parent
-        3.7.1
+        com.scylladb
+        scylla-driver-parent
+        3.7.1-scylla-0-SNAPSHOT
     
 
-    cassandra-driver-mapping
+    scylla-driver-mapping
     bundle
-    DataStax Java Driver for Apache Cassandra - Object Mapping
-    Object mapper for the DataStax CQL Java Driver.
+    Java Driver for Scylla and Apache Cassandra - Object Mapping
+    Object mapper for the CQL Java Driver.
 
     
 
         
-            com.datastax.cassandra
-            cassandra-driver-core
+            com.scylladb
+            scylla-driver-core
         
 
         
@@ -54,8 +54,8 @@
         
 
         
-            com.datastax.cassandra
-            cassandra-driver-core
+            com.scylladb
+            scylla-driver-core
             test-jar
             test
         
diff --git a/driver-tests/osgi/pom.xml b/driver-tests/osgi/pom.xml
index 44acff7fe26..6b6884e58d7 100644
--- a/driver-tests/osgi/pom.xml
+++ b/driver-tests/osgi/pom.xml
@@ -20,20 +20,20 @@
     4.0.0
 
     
-        com.datastax.cassandra
-        cassandra-driver-tests-parent
-        3.7.1
+        com.scylladb
+        scylla-driver-tests-parent
+        3.7.1-scylla-0-SNAPSHOT
     
 
-    cassandra-driver-tests-osgi
-    DataStax Java Driver for Apache Cassandra Tests - OSGi
-    A test for the DataStax Java Driver in an OSGi container.
+    scylla-driver-tests-osgi
+    Java Driver for Scylla and Apache Cassandra Tests - OSGi
+    A test for the Java Driver in an OSGi container.
 
     
 
         
-            com.datastax.cassandra
-            cassandra-driver-core
+            com.scylladb
+            scylla-driver-core
             
                 
                     com.github.jnr
@@ -47,13 +47,13 @@
         
 
         
-            com.datastax.cassandra
-            cassandra-driver-mapping
+            com.scylladb
+            scylla-driver-mapping
         
 
         
-            com.datastax.cassandra
-            cassandra-driver-extras
+            com.scylladb
+            scylla-driver-extras
         
 
         
@@ -77,8 +77,8 @@
         
 
         
-            com.datastax.cassandra
-            cassandra-driver-core
+            com.scylladb
+            scylla-driver-core
             test-jar
             test
         
diff --git a/driver-tests/osgi/src/test/java/com/datastax/driver/osgi/BundleOptions.java b/driver-tests/osgi/src/test/java/com/datastax/driver/osgi/BundleOptions.java
index a60944b7e61..4866ad2887f 100644
--- a/driver-tests/osgi/src/test/java/com/datastax/driver/osgi/BundleOptions.java
+++ b/driver-tests/osgi/src/test/java/com/datastax/driver/osgi/BundleOptions.java
@@ -60,7 +60,7 @@ public static UrlProvisionOption driverBundle(boolean useShaded) {
     return bundle(
         "reference:file:"
             + PathUtils.getBaseDir()
-            + "/../../driver-core/target/cassandra-driver-core-"
+            + "/../../driver-core/target/scylla-driver-core-"
             + Cluster.getDriverVersion()
             + classifier
             + ".jar");
@@ -70,7 +70,7 @@ public static UrlProvisionOption mappingBundle() {
     return bundle(
         "reference:file:"
             + PathUtils.getBaseDir()
-            + "/../../driver-mapping/target/cassandra-driver-mapping-"
+            + "/../../driver-mapping/target/scylla-driver-mapping-"
             + Cluster.getDriverVersion()
             + ".jar");
   }
@@ -79,7 +79,7 @@ public static UrlProvisionOption extrasBundle() {
     return bundle(
         "reference:file:"
             + PathUtils.getBaseDir()
-            + "/../../driver-extras/target/cassandra-driver-extras-"
+            + "/../../driver-extras/target/scylla-driver-extras-"
             + Cluster.getDriverVersion()
             + ".jar");
   }
diff --git a/driver-tests/pom.xml b/driver-tests/pom.xml
index 8b097a20270..72760c7be00 100644
--- a/driver-tests/pom.xml
+++ b/driver-tests/pom.xml
@@ -20,15 +20,15 @@
     4.0.0
 
     
-        com.datastax.cassandra
-        cassandra-driver-parent
-        3.7.1
+        com.scylladb
+        scylla-driver-parent
+        3.7.1-scylla-0-SNAPSHOT
     
 
-    cassandra-driver-tests-parent
+    scylla-driver-tests-parent
     pom
-    DataStax Java Driver for Apache Cassandra Tests
-    Tests for the DataStax Java Driver for Apache Cassandra.
+    Java Driver for Scylla and Apache Cassandra Tests
+    Tests for the Java Driver for Scylla and Apache Cassandra.
 
     
         stress
diff --git a/driver-tests/shading/pom.xml b/driver-tests/shading/pom.xml
index d23c9e7daa1..ca1e0676116 100644
--- a/driver-tests/shading/pom.xml
+++ b/driver-tests/shading/pom.xml
@@ -20,14 +20,14 @@
     4.0.0
 
     
-        com.datastax.cassandra
-        cassandra-driver-tests-parent
-        3.7.1
+        com.scylladb
+        scylla-driver-tests-parent
+        3.7.1-scylla-0-SNAPSHOT
     
 
     pom
-    cassandra-driver-tests-shading
-    DataStax Java Driver for Apache Cassandra Tests - Shading
+    scylla-driver-tests-shading
+    Java Driver for Scylla and Apache Cassandra Tests - Shading
     A test project for tests which ensure that the shading of the driver didn't break anything.
 
     
diff --git a/driver-tests/shading/shaded/pom.xml b/driver-tests/shading/shaded/pom.xml
index a3531ca1b6b..d07613c9770 100644
--- a/driver-tests/shading/shaded/pom.xml
+++ b/driver-tests/shading/shaded/pom.xml
@@ -20,20 +20,20 @@
     4.0.0
 
     
-        com.datastax.cassandra
-        cassandra-driver-tests-shading
-        3.7.1
+        com.scylladb
+        scylla-driver-tests-shading
+        3.7.1-scylla-0-SNAPSHOT
     
 
-    cassandra-driver-tests-shading-shaded
-    DataStax Java Driver for Apache Cassandra Tests - Shading - Shaded
+    scylla-driver-tests-shading-shaded
+    Java Driver for Scylla and Apache Cassandra Tests - Shading - Shaded
     The shading detection tests for the shaded driver
 
     
 
         
-            com.datastax.cassandra
-            cassandra-driver-core
+            com.scylladb
+            scylla-driver-core
             shaded
             
                 
diff --git a/driver-tests/shading/unshaded/pom.xml b/driver-tests/shading/unshaded/pom.xml
index 4f1370b835d..3420a39bc89 100644
--- a/driver-tests/shading/unshaded/pom.xml
+++ b/driver-tests/shading/unshaded/pom.xml
@@ -20,20 +20,20 @@
     4.0.0
 
     
-        com.datastax.cassandra
-        cassandra-driver-tests-shading
-        3.7.1
+        com.scylladb
+        scylla-driver-tests-shading
+        3.7.1-scylla-0-SNAPSHOT
     
 
-    cassandra-driver-tests-shading-unshaded
-    DataStax Java Driver for Apache Cassandra Tests - Shading - Unshaded
+    scylla-driver-tests-shading-unshaded
+    Java Driver for Scylla and Apache Cassandra Tests - Shading - Unshaded
     The shading detection tests for the unshaded driver
 
     
 
         
-            com.datastax.cassandra
-            cassandra-driver-core
+            com.scylladb
+            scylla-driver-core
         
 
         
diff --git a/driver-tests/stress/bin/stress b/driver-tests/stress/bin/stress
index e47ec0d7955..c26f62cbe4a 100755
--- a/driver-tests/stress/bin/stress
+++ b/driver-tests/stress/bin/stress
@@ -12,7 +12,7 @@ fi
 
 if [ "x$STRESS_JAR" = "x" ]; then
 
-    STRESS_JAR="$SCRIPT_DIR/../target/cassandra-driver-tests-stress-*-jar-with-dependencies.jar"
+    STRESS_JAR="$SCRIPT_DIR/../target/scylla-driver-tests-stress-*-jar-with-dependencies.jar"
 
     if [ ! -f $STRESS_JAR ]; then
         # Trash the version file in case there was some crap in it
diff --git a/driver-tests/stress/pom.xml b/driver-tests/stress/pom.xml
index 8f04fbe5d73..828312d6922 100644
--- a/driver-tests/stress/pom.xml
+++ b/driver-tests/stress/pom.xml
@@ -20,20 +20,20 @@
     4.0.0
 
     
-        com.datastax.cassandra
-        cassandra-driver-tests-parent
-        3.7.1
+        com.scylladb
+        scylla-driver-tests-parent
+        3.7.1-scylla-0-SNAPSHOT
     
 
-    cassandra-driver-tests-stress
-    DataStax Java Driver for Apache Cassandra Tests - Stress
-    A stress test example for DataStax Java Driver for Apache Cassandra.
+    scylla-driver-tests-stress
+    Java Driver for Scylla and Apache Cassandra Tests - Stress
+    A stress test example for Java Driver for Apache Cassandra.
 
     
 
         
-            com.datastax.cassandra
-            cassandra-driver-core
+            com.scylladb
+            scylla-driver-core
         
 
         
diff --git a/manual/compression/README.md b/manual/compression/README.md
index d7d24fa8a31..b6dba5c73d7 100644
--- a/manual/compression/README.md
+++ b/manual/compression/README.md
@@ -88,4 +88,4 @@ cluster = Cluster.builder()
     .build();
 ```
 
-[pom]: https://repo1.maven.org/maven2/com/datastax/cassandra/cassandra-driver-parent/3.7.1/cassandra-driver-parent-3.7.1.pom
+[pom]: https://repo1.maven.org/maven2/com/datastax/cassandra/scylla-driver-parent/3.7.1/scylla-driver-parent-3.7.1.pom
diff --git a/manual/custom_codecs/extras/README.md b/manual/custom_codecs/extras/README.md
index 0bc4633ae24..af2a359e6bb 100644
--- a/manual/custom_codecs/extras/README.md
+++ b/manual/custom_codecs/extras/README.md
@@ -8,9 +8,9 @@ The module is published as a separate Maven artifact:
 
 ```xml
 
-  com.datastax.cassandra
-  cassandra-driver-extras
-  3.7.1
+  com.scylladb
+  scylla-driver-extras
+  3.7.1-scylla-0-SNAPSHOT
 
 ```
 
diff --git a/manual/metrics/README.md b/manual/metrics/README.md
index ee15107788c..1c1a3091e0e 100644
--- a/manual/metrics/README.md
+++ b/manual/metrics/README.md
@@ -36,9 +36,9 @@ To do this in a maven project:
 
 ```xml
 
-  com.datastax.cassandra
-  cassandra-driver-core
-  3.7.1
+  com.scylladb
+  scylla-driver-core
+  3.7.1-scylla-0-SNAPSHOT
   
     
       io.dropwizard.metrics
diff --git a/manual/object_mapper/README.md b/manual/object_mapper/README.md
index 3b9e26667f3..c32787f961a 100644
--- a/manual/object_mapper/README.md
+++ b/manual/object_mapper/README.md
@@ -9,9 +9,9 @@ The mapper is published as a separate Maven artifact:
 
 ```xml
 
-  com.datastax.cassandra
-  cassandra-driver-mapping
-  3.7.1
+  com.scylladb
+  scylla-driver-mapping
+  3.7.1-scylla-0-SNAPSHOT
 
 ```
 
diff --git a/manual/shaded_jar/README.md b/manual/shaded_jar/README.md
index 25d8bf4a124..fce55089d47 100644
--- a/manual/shaded_jar/README.md
+++ b/manual/shaded_jar/README.md
@@ -10,9 +10,9 @@ package name:
 
 ```xml
 
-  com.datastax.cassandra
-  cassandra-driver-core
-  3.7.1
+  com.scylladb
+  scylla-driver-core
+  3.7.1-scylla-0-SNAPSHOT
   shaded
   
@@ -30,9 +30,9 @@ non-shaded JAR:
 
 ```xml
 
-  com.datastax.cassandra
-  cassandra-driver-core
-  3.7.1
+  com.scylladb
+  scylla-driver-core
+  3.7.1-scylla-0-SNAPSHOT
   shaded
   
     
@@ -42,13 +42,13 @@ non-shaded JAR:
   
 
 
-  com.datastax.cassandra
-  cassandra-driver-mapping
-  3.7.1
+  com.scylladb
+  scylla-driver-mapping
+  3.7.1-scylla-0-SNAPSHOT
   
     
-      com.datastax.cassandra
-      cassandra-driver-core
+      com.scylladb
+      scylla-driver-core
     
   
 
diff --git a/pom.xml b/pom.xml
index 1654463eeb2..f2fd5a55db8 100644
--- a/pom.xml
+++ b/pom.xml
@@ -19,17 +19,17 @@
 
     4.0.0
 
-    com.datastax.cassandra
-    cassandra-driver-parent
-    3.7.1
+    com.scylladb
+    scylla-driver-parent
+    3.7.1-scylla-0-SNAPSHOT
     pom
-    DataStax Java Driver for Apache Cassandra
+    Java Driver for Scylla and Apache Cassandra
     
-        A driver for Apache Cassandra 1.2+ that works exclusively with the Cassandra Query Language version 3
+        A driver for Scylla and Apache Cassandra 1.2+ that works exclusively with the Cassandra Query Language version 3
         (CQL3) and Cassandra's binary protocol.
     
 
-    https://github.com/datastax/java-driver
+    https://github.com/scylladb/java-driver
 
     2012
 
@@ -94,34 +94,34 @@
         
 
             
-                com.datastax.cassandra
-                cassandra-driver-core
+                com.scylladb
+                scylla-driver-core
                 ${project.parent.version}
             
 
             
-                com.datastax.cassandra
-                cassandra-driver-core
+                com.scylladb
+                scylla-driver-core
                 ${project.parent.version}
                 shaded
             
 
             
-                com.datastax.cassandra
-                cassandra-driver-core
+                com.scylladb
+                scylla-driver-core
                 ${project.parent.version}
                 test-jar
             
 
             
-                com.datastax.cassandra
-                cassandra-driver-mapping
+                com.scylladb
+                scylla-driver-mapping
                 ${project.parent.version}
             
 
             
-                com.datastax.cassandra
-                cassandra-driver-extras
+                com.scylladb
+                scylla-driver-extras
                 ${project.parent.version}
             
 
@@ -599,10 +599,20 @@
                     
                         @{project.version}
                         true
-                        false
+                        
+                        true
+                        forked-path
+                        -Dgpg.passphrase=${gpg.passphrase}
                     
+                    
+                        
+                            org.apache.maven.scm
+                            maven-scm-provider-gitexe
+                            1.9.5
+                        
+                    
                 
 
                 
@@ -618,6 +628,15 @@
                 
                     maven-deploy-plugin
                     2.8.2
+                    
+                        
+                            default-deploy
+                            deploy
+                            
+                                deploy
+                            
+                        
+                    
                 
 
                 
@@ -770,8 +789,24 @@
                 
 
                 
+                    org.apache.maven.plugins
                     maven-gpg-plugin
                     1.5
+                    
+                        
+                            sign-artifacts
+                            verify
+                            
+                                sign
+                            
+                            
+                                
+                                    --pinentry-mode
+                                    loopback
+                                
+                            
+                        
+                    
                 
 
                 
@@ -779,6 +814,11 @@
                     nexus-staging-maven-plugin
                     1.6.8
                     true
+                    
+                        ossrh
+                        https://oss.sonatype.org/
+                        true
+                    
                 
 
                 
@@ -871,6 +911,33 @@
 
     
 
+        
+            release-sign-artifacts
+            
+                
+                    performRelease
+                    true
+                
+            
+            
+                
+                    
+                        org.apache.maven.plugins
+                        maven-gpg-plugin
+                        
+                            
+                                sign-artifacts
+                                verify
+                                
+                                    sign
+                                
+                            
+                        
+                    
+                
+            
+        
+
         
             short
             
@@ -976,6 +1043,7 @@
                         
                     
                     
+                        org.apache.maven.plugins
                         maven-gpg-plugin
                         
                             
@@ -994,8 +1062,8 @@
                         
                             ossrh
                             https://oss.sonatype.org/
-                            false
-                            true
+                            true
+                            
                         
                     
                 
@@ -1071,6 +1139,10 @@
     
 
     
+        
+            ossrh
+            https://oss.sonatype.org/content/repositories/snapshots
+        
         
             ossrh
             https://oss.sonatype.org/service/local/staging/deploy/maven2/
@@ -1087,10 +1159,10 @@
     
 
     
-        scm:git:git@github.com:datastax/java-driver.git
-        scm:git:git@github.com:datastax/java-driver.git
-        https://github.com/datastax/java-driver
-        3.7.1
+        scm:git:https://github.com/scylladb/java-driver
+        scm:git:https://github.com/scylladb/java-driver
+        https://github.com/scylladb/java-driver
+        HEAD
     
 
     
@@ -1098,6 +1170,10 @@
             Various
             DataStax
         
+        
+            Various
+            ScyllaDB
+        
     
 
 

From 2521e746dfe55d60287f14510c10aff9890563c3 Mon Sep 17 00:00:00 2001
From: Piotr Jastrzebski 
Date: Fri, 9 Aug 2019 17:44:58 +0200
Subject: [PATCH 21/52] [maven-release-plugin] prepare release 3.7.1-scylla-0

---
 driver-core/pom.xml                   | 2 +-
 driver-dist/pom.xml                   | 2 +-
 driver-examples/pom.xml               | 2 +-
 driver-extras/pom.xml                 | 2 +-
 driver-mapping/pom.xml                | 2 +-
 driver-tests/osgi/pom.xml             | 2 +-
 driver-tests/pom.xml                  | 2 +-
 driver-tests/shading/pom.xml          | 2 +-
 driver-tests/shading/shaded/pom.xml   | 2 +-
 driver-tests/shading/unshaded/pom.xml | 2 +-
 driver-tests/stress/pom.xml           | 2 +-
 pom.xml                               | 4 ++--
 12 files changed, 13 insertions(+), 13 deletions(-)

diff --git a/driver-core/pom.xml b/driver-core/pom.xml
index 7b1df2c4201..b5a3b9e7e33 100644
--- a/driver-core/pom.xml
+++ b/driver-core/pom.xml
@@ -22,7 +22,7 @@
     
         com.scylladb
         scylla-driver-parent
-        3.7.1-scylla-0-SNAPSHOT
+        3.7.1-scylla-0
     
 
     scylla-driver-core
diff --git a/driver-dist/pom.xml b/driver-dist/pom.xml
index b177150548e..5af69b491d0 100644
--- a/driver-dist/pom.xml
+++ b/driver-dist/pom.xml
@@ -22,7 +22,7 @@
     
         com.scylladb
         scylla-driver-parent
-        3.7.1-scylla-0-SNAPSHOT
+        3.7.1-scylla-0
     
 
     scylla-driver-dist
diff --git a/driver-examples/pom.xml b/driver-examples/pom.xml
index 64d78c7a723..946f6e83eb7 100644
--- a/driver-examples/pom.xml
+++ b/driver-examples/pom.xml
@@ -22,7 +22,7 @@
     
         com.scylladb
         scylla-driver-parent
-        3.7.1-scylla-0-SNAPSHOT
+        3.7.1-scylla-0
     
 
     scylla-driver-examples
diff --git a/driver-extras/pom.xml b/driver-extras/pom.xml
index 3ffe66a2b3e..294652d9273 100644
--- a/driver-extras/pom.xml
+++ b/driver-extras/pom.xml
@@ -22,7 +22,7 @@
     
         com.scylladb
         scylla-driver-parent
-        3.7.1-scylla-0-SNAPSHOT
+        3.7.1-scylla-0
     
 
     scylla-driver-extras
diff --git a/driver-mapping/pom.xml b/driver-mapping/pom.xml
index bc6ef0b2bb5..017322ed91f 100644
--- a/driver-mapping/pom.xml
+++ b/driver-mapping/pom.xml
@@ -22,7 +22,7 @@
     
         com.scylladb
         scylla-driver-parent
-        3.7.1-scylla-0-SNAPSHOT
+        3.7.1-scylla-0
     
 
     scylla-driver-mapping
diff --git a/driver-tests/osgi/pom.xml b/driver-tests/osgi/pom.xml
index 6b6884e58d7..82bf9396f09 100644
--- a/driver-tests/osgi/pom.xml
+++ b/driver-tests/osgi/pom.xml
@@ -22,7 +22,7 @@
     
         com.scylladb
         scylla-driver-tests-parent
-        3.7.1-scylla-0-SNAPSHOT
+        3.7.1-scylla-0
     
 
     scylla-driver-tests-osgi
diff --git a/driver-tests/pom.xml b/driver-tests/pom.xml
index 72760c7be00..7e144608e1a 100644
--- a/driver-tests/pom.xml
+++ b/driver-tests/pom.xml
@@ -22,7 +22,7 @@
     
         com.scylladb
         scylla-driver-parent
-        3.7.1-scylla-0-SNAPSHOT
+        3.7.1-scylla-0
     
 
     scylla-driver-tests-parent
diff --git a/driver-tests/shading/pom.xml b/driver-tests/shading/pom.xml
index ca1e0676116..5148618269b 100644
--- a/driver-tests/shading/pom.xml
+++ b/driver-tests/shading/pom.xml
@@ -22,7 +22,7 @@
     
         com.scylladb
         scylla-driver-tests-parent
-        3.7.1-scylla-0-SNAPSHOT
+        3.7.1-scylla-0
     
 
     pom
diff --git a/driver-tests/shading/shaded/pom.xml b/driver-tests/shading/shaded/pom.xml
index d07613c9770..9925587cc15 100644
--- a/driver-tests/shading/shaded/pom.xml
+++ b/driver-tests/shading/shaded/pom.xml
@@ -22,7 +22,7 @@
     
         com.scylladb
         scylla-driver-tests-shading
-        3.7.1-scylla-0-SNAPSHOT
+        3.7.1-scylla-0
     
 
     scylla-driver-tests-shading-shaded
diff --git a/driver-tests/shading/unshaded/pom.xml b/driver-tests/shading/unshaded/pom.xml
index 3420a39bc89..0910d4a2cce 100644
--- a/driver-tests/shading/unshaded/pom.xml
+++ b/driver-tests/shading/unshaded/pom.xml
@@ -22,7 +22,7 @@
     
         com.scylladb
         scylla-driver-tests-shading
-        3.7.1-scylla-0-SNAPSHOT
+        3.7.1-scylla-0
     
 
     scylla-driver-tests-shading-unshaded
diff --git a/driver-tests/stress/pom.xml b/driver-tests/stress/pom.xml
index 828312d6922..b7ba30e1cc6 100644
--- a/driver-tests/stress/pom.xml
+++ b/driver-tests/stress/pom.xml
@@ -22,7 +22,7 @@
     
         com.scylladb
         scylla-driver-tests-parent
-        3.7.1-scylla-0-SNAPSHOT
+        3.7.1-scylla-0
     
 
     scylla-driver-tests-stress
diff --git a/pom.xml b/pom.xml
index f2fd5a55db8..118ba63ae74 100644
--- a/pom.xml
+++ b/pom.xml
@@ -21,7 +21,7 @@
 
     com.scylladb
     scylla-driver-parent
-    3.7.1-scylla-0-SNAPSHOT
+    3.7.1-scylla-0
     pom
     Java Driver for Scylla and Apache Cassandra
     
@@ -1162,7 +1162,7 @@
         scm:git:https://github.com/scylladb/java-driver
         scm:git:https://github.com/scylladb/java-driver
         https://github.com/scylladb/java-driver
-        HEAD
+        3.7.1-scylla-0
     
 
     

From edad52b8c043e97893d29bb83c2b9d8b79d25d15 Mon Sep 17 00:00:00 2001
From: Piotr Jastrzebski 
Date: Fri, 9 Aug 2019 17:45:08 +0200
Subject: [PATCH 22/52] [maven-release-plugin] prepare for next development
 iteration

---
 driver-core/pom.xml                   | 2 +-
 driver-dist/pom.xml                   | 2 +-
 driver-examples/pom.xml               | 2 +-
 driver-extras/pom.xml                 | 2 +-
 driver-mapping/pom.xml                | 2 +-
 driver-tests/osgi/pom.xml             | 2 +-
 driver-tests/pom.xml                  | 2 +-
 driver-tests/shading/pom.xml          | 2 +-
 driver-tests/shading/shaded/pom.xml   | 2 +-
 driver-tests/shading/unshaded/pom.xml | 2 +-
 driver-tests/stress/pom.xml           | 2 +-
 pom.xml                               | 4 ++--
 12 files changed, 13 insertions(+), 13 deletions(-)

diff --git a/driver-core/pom.xml b/driver-core/pom.xml
index b5a3b9e7e33..352d1f0bc88 100644
--- a/driver-core/pom.xml
+++ b/driver-core/pom.xml
@@ -22,7 +22,7 @@
     
         com.scylladb
         scylla-driver-parent
-        3.7.1-scylla-0
+        3.7.1-scylla-1-SNAPSHOT
     
 
     scylla-driver-core
diff --git a/driver-dist/pom.xml b/driver-dist/pom.xml
index 5af69b491d0..fd4b4b9009f 100644
--- a/driver-dist/pom.xml
+++ b/driver-dist/pom.xml
@@ -22,7 +22,7 @@
     
         com.scylladb
         scylla-driver-parent
-        3.7.1-scylla-0
+        3.7.1-scylla-1-SNAPSHOT
     
 
     scylla-driver-dist
diff --git a/driver-examples/pom.xml b/driver-examples/pom.xml
index 946f6e83eb7..d7f73406243 100644
--- a/driver-examples/pom.xml
+++ b/driver-examples/pom.xml
@@ -22,7 +22,7 @@
     
         com.scylladb
         scylla-driver-parent
-        3.7.1-scylla-0
+        3.7.1-scylla-1-SNAPSHOT
     
 
     scylla-driver-examples
diff --git a/driver-extras/pom.xml b/driver-extras/pom.xml
index 294652d9273..cf7e0f30036 100644
--- a/driver-extras/pom.xml
+++ b/driver-extras/pom.xml
@@ -22,7 +22,7 @@
     
         com.scylladb
         scylla-driver-parent
-        3.7.1-scylla-0
+        3.7.1-scylla-1-SNAPSHOT
     
 
     scylla-driver-extras
diff --git a/driver-mapping/pom.xml b/driver-mapping/pom.xml
index 017322ed91f..b4ed74cb54c 100644
--- a/driver-mapping/pom.xml
+++ b/driver-mapping/pom.xml
@@ -22,7 +22,7 @@
     
         com.scylladb
         scylla-driver-parent
-        3.7.1-scylla-0
+        3.7.1-scylla-1-SNAPSHOT
     
 
     scylla-driver-mapping
diff --git a/driver-tests/osgi/pom.xml b/driver-tests/osgi/pom.xml
index 82bf9396f09..78090649b91 100644
--- a/driver-tests/osgi/pom.xml
+++ b/driver-tests/osgi/pom.xml
@@ -22,7 +22,7 @@
     
         com.scylladb
         scylla-driver-tests-parent
-        3.7.1-scylla-0
+        3.7.1-scylla-1-SNAPSHOT
     
 
     scylla-driver-tests-osgi
diff --git a/driver-tests/pom.xml b/driver-tests/pom.xml
index 7e144608e1a..afb955b9e4e 100644
--- a/driver-tests/pom.xml
+++ b/driver-tests/pom.xml
@@ -22,7 +22,7 @@
     
         com.scylladb
         scylla-driver-parent
-        3.7.1-scylla-0
+        3.7.1-scylla-1-SNAPSHOT
     
 
     scylla-driver-tests-parent
diff --git a/driver-tests/shading/pom.xml b/driver-tests/shading/pom.xml
index 5148618269b..263ed5c4a4f 100644
--- a/driver-tests/shading/pom.xml
+++ b/driver-tests/shading/pom.xml
@@ -22,7 +22,7 @@
     
         com.scylladb
         scylla-driver-tests-parent
-        3.7.1-scylla-0
+        3.7.1-scylla-1-SNAPSHOT
     
 
     pom
diff --git a/driver-tests/shading/shaded/pom.xml b/driver-tests/shading/shaded/pom.xml
index 9925587cc15..d2dbe1a2b95 100644
--- a/driver-tests/shading/shaded/pom.xml
+++ b/driver-tests/shading/shaded/pom.xml
@@ -22,7 +22,7 @@
     
         com.scylladb
         scylla-driver-tests-shading
-        3.7.1-scylla-0
+        3.7.1-scylla-1-SNAPSHOT
     
 
     scylla-driver-tests-shading-shaded
diff --git a/driver-tests/shading/unshaded/pom.xml b/driver-tests/shading/unshaded/pom.xml
index 0910d4a2cce..66f7056b127 100644
--- a/driver-tests/shading/unshaded/pom.xml
+++ b/driver-tests/shading/unshaded/pom.xml
@@ -22,7 +22,7 @@
     
         com.scylladb
         scylla-driver-tests-shading
-        3.7.1-scylla-0
+        3.7.1-scylla-1-SNAPSHOT
     
 
     scylla-driver-tests-shading-unshaded
diff --git a/driver-tests/stress/pom.xml b/driver-tests/stress/pom.xml
index b7ba30e1cc6..31d1a936c7e 100644
--- a/driver-tests/stress/pom.xml
+++ b/driver-tests/stress/pom.xml
@@ -22,7 +22,7 @@
     
         com.scylladb
         scylla-driver-tests-parent
-        3.7.1-scylla-0
+        3.7.1-scylla-1-SNAPSHOT
     
 
     scylla-driver-tests-stress
diff --git a/pom.xml b/pom.xml
index 118ba63ae74..97a094424bc 100644
--- a/pom.xml
+++ b/pom.xml
@@ -21,7 +21,7 @@
 
     com.scylladb
     scylla-driver-parent
-    3.7.1-scylla-0
+    3.7.1-scylla-1-SNAPSHOT
     pom
     Java Driver for Scylla and Apache Cassandra
     
@@ -1162,7 +1162,7 @@
         scm:git:https://github.com/scylladb/java-driver
         scm:git:https://github.com/scylladb/java-driver
         https://github.com/scylladb/java-driver
-        3.7.1-scylla-0
+        HEAD
     
 
     

From 09ecb301b9b318100fecd9447c75b9d0a8cb0ecd Mon Sep 17 00:00:00 2001
From: Piotr Jastrzebski 
Date: Tue, 20 Aug 2019 15:27:53 +0200
Subject: [PATCH 23/52] Add metrics for shard awareness

Those metrics expose the information the driver has
about number of shards each node has.

New metric is called shard-awareness-info and its
value is a Map.

For each host (which represents a node in a cluster) it presents
either the number of shards the host has or null if the node
does not provide sharding information.

Signed-off-by: Piotr Jastrzebski 
---
 .../com/datastax/driver/core/Metrics.java     | 25 ++++++++++++++++++-
 1 file changed, 24 insertions(+), 1 deletion(-)

diff --git a/driver-core/src/main/java/com/datastax/driver/core/Metrics.java b/driver-core/src/main/java/com/datastax/driver/core/Metrics.java
index 8f4498be490..f729bd8a77b 100644
--- a/driver-core/src/main/java/com/datastax/driver/core/Metrics.java
+++ b/driver-core/src/main/java/com/datastax/driver/core/Metrics.java
@@ -22,7 +22,10 @@
 import com.codahale.metrics.MetricRegistry;
 import com.codahale.metrics.Timer;
 import com.datastax.driver.core.policies.SpeculativeExecutionPolicy;
+import java.util.Collection;
+import java.util.HashMap;
 import java.util.HashSet;
+import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.ThreadPoolExecutor;
@@ -126,8 +129,9 @@ public Integer getValue() {
   private final Gauge blockingExecutorQueueDepth;
   private final Gauge reconnectionSchedulerQueueSize;
   private final Gauge taskSchedulerQueueSize;
+  private final Gauge> shardAwarenessInfo;
 
-  Metrics(Cluster.Manager manager) {
+  Metrics(final Cluster.Manager manager) {
     this.manager = manager;
     this.executorQueueDepth =
         registry.register("executor-queue-depth", buildQueueSizeGauge(manager.executorQueue));
@@ -148,6 +152,21 @@ public Integer getValue() {
     } else {
       this.jmxReporter = null;
     }
+    shardAwarenessInfo =
+        registry.register(
+            "shard-awareness-info",
+            new Gauge>() {
+              @Override
+              public Map getValue() {
+                Collection hosts = manager.metadata.allHosts();
+                Map result = new HashMap(hosts.size());
+                for (Host h : manager.metadata.allHosts()) {
+                  result.put(
+                      h, h.getShardingInfo() == null ? null : h.getShardingInfo().getShardsCount());
+                }
+                return result;
+              }
+            });
   }
 
   /**
@@ -345,6 +364,10 @@ public Gauge getTaskSchedulerQueueSize() {
     return taskSchedulerQueueSize;
   }
 
+  public Gauge> getShardAwarenessInfo() {
+    return shardAwarenessInfo;
+  }
+
   /**
    * Returns the number of bytes sent so far.
    *

From 9c28fb3798cce7b70f9671c18a24fb97955e901b Mon Sep 17 00:00:00 2001
From: Piotr Jastrzebski 
Date: Tue, 20 Aug 2019 16:08:20 +0200
Subject: [PATCH 24/52] Update sharding info when node restarts

Fixes #14

Signed-off-by: Piotr Jastrzebski 
---
 .../src/main/java/com/datastax/driver/core/Connection.java   | 5 ++++-
 1 file changed, 4 insertions(+), 1 deletion(-)

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 6b585815048..1ba9b17cb4a 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
@@ -403,6 +403,9 @@ public ListenableFuture apply(Message.Response response) throws Exception
             if (sharding != null) {
               host.setShardingInfo(sharding.shardingInfo);
               Connection.this.shardId = sharding.shardId;
+            } else {
+              host.setShardingInfo(null);
+              Connection.this.shardId = 0;
             }
             return MoreFutures.VOID_SUCCESS;
           default:
@@ -854,7 +857,7 @@ boolean setOwner(Owner owner) {
   }
 
   public int shardId() {
-    return shardId == null ? 0 : shardId;
+    return shardId == null || host.getShardingInfo() == null ? 0 : shardId;
   }
 
   /**

From 354d13d455f191acdff3aacab9913bd1d9e96194 Mon Sep 17 00:00:00 2001
From: Piotr Jastrzebski 
Date: Tue, 20 Aug 2019 16:15:19 +0200
Subject: [PATCH 25/52] [maven-release-plugin] prepare release 3.7.1-scylla-1

---
 driver-core/pom.xml                   | 2 +-
 driver-dist/pom.xml                   | 2 +-
 driver-examples/pom.xml               | 2 +-
 driver-extras/pom.xml                 | 2 +-
 driver-mapping/pom.xml                | 2 +-
 driver-tests/osgi/pom.xml             | 2 +-
 driver-tests/pom.xml                  | 2 +-
 driver-tests/shading/pom.xml          | 2 +-
 driver-tests/shading/shaded/pom.xml   | 2 +-
 driver-tests/shading/unshaded/pom.xml | 2 +-
 driver-tests/stress/pom.xml           | 2 +-
 pom.xml                               | 4 ++--
 12 files changed, 13 insertions(+), 13 deletions(-)

diff --git a/driver-core/pom.xml b/driver-core/pom.xml
index 352d1f0bc88..4d09966f1ce 100644
--- a/driver-core/pom.xml
+++ b/driver-core/pom.xml
@@ -22,7 +22,7 @@
     
         com.scylladb
         scylla-driver-parent
-        3.7.1-scylla-1-SNAPSHOT
+        3.7.1-scylla-1
     
 
     scylla-driver-core
diff --git a/driver-dist/pom.xml b/driver-dist/pom.xml
index fd4b4b9009f..617f83e4f6b 100644
--- a/driver-dist/pom.xml
+++ b/driver-dist/pom.xml
@@ -22,7 +22,7 @@
     
         com.scylladb
         scylla-driver-parent
-        3.7.1-scylla-1-SNAPSHOT
+        3.7.1-scylla-1
     
 
     scylla-driver-dist
diff --git a/driver-examples/pom.xml b/driver-examples/pom.xml
index d7f73406243..f12aa21ede1 100644
--- a/driver-examples/pom.xml
+++ b/driver-examples/pom.xml
@@ -22,7 +22,7 @@
     
         com.scylladb
         scylla-driver-parent
-        3.7.1-scylla-1-SNAPSHOT
+        3.7.1-scylla-1
     
 
     scylla-driver-examples
diff --git a/driver-extras/pom.xml b/driver-extras/pom.xml
index cf7e0f30036..7898c238b55 100644
--- a/driver-extras/pom.xml
+++ b/driver-extras/pom.xml
@@ -22,7 +22,7 @@
     
         com.scylladb
         scylla-driver-parent
-        3.7.1-scylla-1-SNAPSHOT
+        3.7.1-scylla-1
     
 
     scylla-driver-extras
diff --git a/driver-mapping/pom.xml b/driver-mapping/pom.xml
index b4ed74cb54c..edc5e098f36 100644
--- a/driver-mapping/pom.xml
+++ b/driver-mapping/pom.xml
@@ -22,7 +22,7 @@
     
         com.scylladb
         scylla-driver-parent
-        3.7.1-scylla-1-SNAPSHOT
+        3.7.1-scylla-1
     
 
     scylla-driver-mapping
diff --git a/driver-tests/osgi/pom.xml b/driver-tests/osgi/pom.xml
index 78090649b91..ac1e8b4899f 100644
--- a/driver-tests/osgi/pom.xml
+++ b/driver-tests/osgi/pom.xml
@@ -22,7 +22,7 @@
     
         com.scylladb
         scylla-driver-tests-parent
-        3.7.1-scylla-1-SNAPSHOT
+        3.7.1-scylla-1
     
 
     scylla-driver-tests-osgi
diff --git a/driver-tests/pom.xml b/driver-tests/pom.xml
index afb955b9e4e..95ac6bc0812 100644
--- a/driver-tests/pom.xml
+++ b/driver-tests/pom.xml
@@ -22,7 +22,7 @@
     
         com.scylladb
         scylla-driver-parent
-        3.7.1-scylla-1-SNAPSHOT
+        3.7.1-scylla-1
     
 
     scylla-driver-tests-parent
diff --git a/driver-tests/shading/pom.xml b/driver-tests/shading/pom.xml
index 263ed5c4a4f..0d5ebbf6e2f 100644
--- a/driver-tests/shading/pom.xml
+++ b/driver-tests/shading/pom.xml
@@ -22,7 +22,7 @@
     
         com.scylladb
         scylla-driver-tests-parent
-        3.7.1-scylla-1-SNAPSHOT
+        3.7.1-scylla-1
     
 
     pom
diff --git a/driver-tests/shading/shaded/pom.xml b/driver-tests/shading/shaded/pom.xml
index d2dbe1a2b95..170b3a789f5 100644
--- a/driver-tests/shading/shaded/pom.xml
+++ b/driver-tests/shading/shaded/pom.xml
@@ -22,7 +22,7 @@
     
         com.scylladb
         scylla-driver-tests-shading
-        3.7.1-scylla-1-SNAPSHOT
+        3.7.1-scylla-1
     
 
     scylla-driver-tests-shading-shaded
diff --git a/driver-tests/shading/unshaded/pom.xml b/driver-tests/shading/unshaded/pom.xml
index 66f7056b127..69ba60ad482 100644
--- a/driver-tests/shading/unshaded/pom.xml
+++ b/driver-tests/shading/unshaded/pom.xml
@@ -22,7 +22,7 @@
     
         com.scylladb
         scylla-driver-tests-shading
-        3.7.1-scylla-1-SNAPSHOT
+        3.7.1-scylla-1
     
 
     scylla-driver-tests-shading-unshaded
diff --git a/driver-tests/stress/pom.xml b/driver-tests/stress/pom.xml
index 31d1a936c7e..729839441ea 100644
--- a/driver-tests/stress/pom.xml
+++ b/driver-tests/stress/pom.xml
@@ -22,7 +22,7 @@
     
         com.scylladb
         scylla-driver-tests-parent
-        3.7.1-scylla-1-SNAPSHOT
+        3.7.1-scylla-1
     
 
     scylla-driver-tests-stress
diff --git a/pom.xml b/pom.xml
index 97a094424bc..9e672b6135f 100644
--- a/pom.xml
+++ b/pom.xml
@@ -21,7 +21,7 @@
 
     com.scylladb
     scylla-driver-parent
-    3.7.1-scylla-1-SNAPSHOT
+    3.7.1-scylla-1
     pom
     Java Driver for Scylla and Apache Cassandra
     
@@ -1162,7 +1162,7 @@
         scm:git:https://github.com/scylladb/java-driver
         scm:git:https://github.com/scylladb/java-driver
         https://github.com/scylladb/java-driver
-        HEAD
+        3.7.1-scylla-1
     
 
     

From ab9dd27d5784ff7082f07c6af1a74e92e93b7929 Mon Sep 17 00:00:00 2001
From: Piotr Jastrzebski 
Date: Tue, 20 Aug 2019 16:15:26 +0200
Subject: [PATCH 26/52] [maven-release-plugin] prepare for next development
 iteration

---
 driver-core/pom.xml                   | 2 +-
 driver-dist/pom.xml                   | 2 +-
 driver-examples/pom.xml               | 2 +-
 driver-extras/pom.xml                 | 2 +-
 driver-mapping/pom.xml                | 2 +-
 driver-tests/osgi/pom.xml             | 2 +-
 driver-tests/pom.xml                  | 2 +-
 driver-tests/shading/pom.xml          | 2 +-
 driver-tests/shading/shaded/pom.xml   | 2 +-
 driver-tests/shading/unshaded/pom.xml | 2 +-
 driver-tests/stress/pom.xml           | 2 +-
 pom.xml                               | 4 ++--
 12 files changed, 13 insertions(+), 13 deletions(-)

diff --git a/driver-core/pom.xml b/driver-core/pom.xml
index 4d09966f1ce..bf0fa12d868 100644
--- a/driver-core/pom.xml
+++ b/driver-core/pom.xml
@@ -22,7 +22,7 @@
     
         com.scylladb
         scylla-driver-parent
-        3.7.1-scylla-1
+        3.7.1-scylla-2-SNAPSHOT
     
 
     scylla-driver-core
diff --git a/driver-dist/pom.xml b/driver-dist/pom.xml
index 617f83e4f6b..b238a1ec497 100644
--- a/driver-dist/pom.xml
+++ b/driver-dist/pom.xml
@@ -22,7 +22,7 @@
     
         com.scylladb
         scylla-driver-parent
-        3.7.1-scylla-1
+        3.7.1-scylla-2-SNAPSHOT
     
 
     scylla-driver-dist
diff --git a/driver-examples/pom.xml b/driver-examples/pom.xml
index f12aa21ede1..69204c88a5a 100644
--- a/driver-examples/pom.xml
+++ b/driver-examples/pom.xml
@@ -22,7 +22,7 @@
     
         com.scylladb
         scylla-driver-parent
-        3.7.1-scylla-1
+        3.7.1-scylla-2-SNAPSHOT
     
 
     scylla-driver-examples
diff --git a/driver-extras/pom.xml b/driver-extras/pom.xml
index 7898c238b55..055f88c1444 100644
--- a/driver-extras/pom.xml
+++ b/driver-extras/pom.xml
@@ -22,7 +22,7 @@
     
         com.scylladb
         scylla-driver-parent
-        3.7.1-scylla-1
+        3.7.1-scylla-2-SNAPSHOT
     
 
     scylla-driver-extras
diff --git a/driver-mapping/pom.xml b/driver-mapping/pom.xml
index edc5e098f36..c0ee8484fac 100644
--- a/driver-mapping/pom.xml
+++ b/driver-mapping/pom.xml
@@ -22,7 +22,7 @@
     
         com.scylladb
         scylla-driver-parent
-        3.7.1-scylla-1
+        3.7.1-scylla-2-SNAPSHOT
     
 
     scylla-driver-mapping
diff --git a/driver-tests/osgi/pom.xml b/driver-tests/osgi/pom.xml
index ac1e8b4899f..cf51d20a71f 100644
--- a/driver-tests/osgi/pom.xml
+++ b/driver-tests/osgi/pom.xml
@@ -22,7 +22,7 @@
     
         com.scylladb
         scylla-driver-tests-parent
-        3.7.1-scylla-1
+        3.7.1-scylla-2-SNAPSHOT
     
 
     scylla-driver-tests-osgi
diff --git a/driver-tests/pom.xml b/driver-tests/pom.xml
index 95ac6bc0812..bd1019615d8 100644
--- a/driver-tests/pom.xml
+++ b/driver-tests/pom.xml
@@ -22,7 +22,7 @@
     
         com.scylladb
         scylla-driver-parent
-        3.7.1-scylla-1
+        3.7.1-scylla-2-SNAPSHOT
     
 
     scylla-driver-tests-parent
diff --git a/driver-tests/shading/pom.xml b/driver-tests/shading/pom.xml
index 0d5ebbf6e2f..ad81fa64c7b 100644
--- a/driver-tests/shading/pom.xml
+++ b/driver-tests/shading/pom.xml
@@ -22,7 +22,7 @@
     
         com.scylladb
         scylla-driver-tests-parent
-        3.7.1-scylla-1
+        3.7.1-scylla-2-SNAPSHOT
     
 
     pom
diff --git a/driver-tests/shading/shaded/pom.xml b/driver-tests/shading/shaded/pom.xml
index 170b3a789f5..d3ec0b815e7 100644
--- a/driver-tests/shading/shaded/pom.xml
+++ b/driver-tests/shading/shaded/pom.xml
@@ -22,7 +22,7 @@
     
         com.scylladb
         scylla-driver-tests-shading
-        3.7.1-scylla-1
+        3.7.1-scylla-2-SNAPSHOT
     
 
     scylla-driver-tests-shading-shaded
diff --git a/driver-tests/shading/unshaded/pom.xml b/driver-tests/shading/unshaded/pom.xml
index 69ba60ad482..e214519d4de 100644
--- a/driver-tests/shading/unshaded/pom.xml
+++ b/driver-tests/shading/unshaded/pom.xml
@@ -22,7 +22,7 @@
     
         com.scylladb
         scylla-driver-tests-shading
-        3.7.1-scylla-1
+        3.7.1-scylla-2-SNAPSHOT
     
 
     scylla-driver-tests-shading-unshaded
diff --git a/driver-tests/stress/pom.xml b/driver-tests/stress/pom.xml
index 729839441ea..0476fa9b0d2 100644
--- a/driver-tests/stress/pom.xml
+++ b/driver-tests/stress/pom.xml
@@ -22,7 +22,7 @@
     
         com.scylladb
         scylla-driver-tests-parent
-        3.7.1-scylla-1
+        3.7.1-scylla-2-SNAPSHOT
     
 
     scylla-driver-tests-stress
diff --git a/pom.xml b/pom.xml
index 9e672b6135f..14b39b9ac60 100644
--- a/pom.xml
+++ b/pom.xml
@@ -21,7 +21,7 @@
 
     com.scylladb
     scylla-driver-parent
-    3.7.1-scylla-1
+    3.7.1-scylla-2-SNAPSHOT
     pom
     Java Driver for Scylla and Apache Cassandra
     
@@ -1162,7 +1162,7 @@
         scm:git:https://github.com/scylladb/java-driver
         scm:git:https://github.com/scylladb/java-driver
         https://github.com/scylladb/java-driver
-        3.7.1-scylla-1
+        HEAD
     
 
     

From d14ef5d873f7296ee13b8a1297409bc1066c1232 Mon Sep 17 00:00:00 2001
From: Piotr Jastrzebski 
Date: Sun, 27 Oct 2019 16:22:03 +0100
Subject: [PATCH 27/52] HostConnectionPool: track number of opened connections
 per shard

Signed-off-by: Piotr Jastrzebski 
---
 .../driver/core/HostConnectionPool.java       | 85 +++++++++----------
 1 file changed, 42 insertions(+), 43 deletions(-)

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 70915f5e494..64b738f1337 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
@@ -67,8 +67,9 @@ class HostConnectionPool implements Connection.Owner {
   protected final SessionManager manager;
 
   private int connectionsPerShard;
+  private int maxConnectionsPerShard;
   List[] connections;
-  private final AtomicInteger open;
+  private AtomicInteger[] open;
   /** The total number of in-flight requests on all connections of this pool. */
   final AtomicInteger totalInFlight = new AtomicInteger();
   /**
@@ -122,8 +123,6 @@ public void run() {
     this.hostDistance = hostDistance;
     this.manager = manager;
 
-    this.open = new AtomicInteger();
-
     this.minAllowedStreams = options().getMaxRequestsPerConnection(hostDistance) * 3 / 4;
 
     this.timeoutsExecutor = manager.getCluster().manager.connectionFactory.eventLoopGroup.next();
@@ -153,19 +152,24 @@ ListenableFuture initAsyncWithConnection(Connection reusedConnection) {
 
     // Create initial core connections
     final int coreSize = options().getCoreConnectionsPerHost(hostDistance);
+    final int maxConnections = options().getMaxConnectionsPerHost(hostDistance);
     final int shardsCount =
         host.getShardingInfo() == null ? 1 : host.getShardingInfo().getShardsCount();
 
     connectionsPerShard = coreSize / shardsCount + (coreSize % shardsCount > 0 ? 1 : 0);
+    maxConnectionsPerShard =
+        maxConnections / shardsCount + (maxConnections % shardsCount > 0 ? 1 : 0);
     int toCreate = shardsCount * connectionsPerShard;
 
     this.connections = new List[shardsCount];
     scheduledForCreation = new AtomicInteger[shardsCount];
+    open = new AtomicInteger[shardsCount];
     trash = new Set[shardsCount];
     pendingBorrows = new Queue[shardsCount];
     for (int i = 0; i < shardsCount; ++i) {
       this.connections[i] = new CopyOnWriteArrayList();
       scheduledForCreation[i] = new AtomicInteger();
+      open[i] = new AtomicInteger();
       trash[i] = new CopyOnWriteArraySet();
       pendingBorrows[i] = new ConcurrentLinkedQueue();
     }
@@ -206,21 +210,18 @@ private void addCallback(
         new FutureCallback>() {
           @Override
           public void onSuccess(List l) {
-            int added = 0;
             for (final Connection c : connections) {
               if (!c.isClosed()) {
                 if (HostConnectionPool.this.connections[c.shardId()].size()
                     < HostConnectionPool.this.connectionsPerShard) {
-                  ++added;
                   HostConnectionPool.this.connections[c.shardId()].add(c);
+                  open[c.shardId()].addAndGet(1);
                 } else {
                   toClose.add(c);
                 }
               }
             }
 
-            open.addAndGet(added);
-
             if (isClosed()) {
               initFuture.setException(
                   new ConnectionException(
@@ -231,7 +232,9 @@ public void onSuccess(List l) {
               for (List shardConnections : HostConnectionPool.this.connections) {
                 forceClose(shardConnections);
               }
-              open.set(0);
+              for (AtomicInteger o : open) {
+                o.set(0);
+              }
             } else {
               int needed = 0;
               for (final List shardsConnections : HostConnectionPool.this.connections) {
@@ -273,7 +276,9 @@ public void onFailure(Throwable t) {
             for (List shardConnections : HostConnectionPool.this.connections) {
               forceClose(shardConnections);
             }
-            open.set(0);
+            for (AtomicInteger o : open) {
+              o.set(0);
+            }
             initFuture.setException(t);
           }
         },
@@ -399,7 +404,7 @@ ListenableFuture borrowConnection(
     int connectionCount = connections[shardId].size() + scheduledForCreation[shardId].get();
     if (connectionCount < connectionsPerShard) {
       maybeSpawnNewConnection(shardId);
-    } else if (connectionCount < options().getMaxConnectionsPerHost(hostDistance)) {
+    } else if (connectionCount < maxConnectionsPerShard) {
       // Add a connection if we fill the first n-1 connections and almost fill the last one
       int currentCapacity =
           (connectionCount - 1) * options().getMaxRequestsPerConnection(hostDistance)
@@ -541,7 +546,7 @@ public void onFailure(Throwable t) {
   // directly because we want to make sure the connection is always trashed.
   private void replaceConnection(Connection connection) {
     if (!connection.state.compareAndSet(OPEN, TRASHED)) return;
-    open.decrementAndGet();
+    open[connection.shardId()].decrementAndGet();
     maybeSpawnNewConnection(connection.shardId());
     connection.maxIdleTime = Long.MIN_VALUE;
     doTrashConnection(connection);
@@ -552,13 +557,13 @@ private boolean trashConnection(Connection connection) {
 
     // First, make sure we don't go below core connections
     for (; ; ) {
-      int opened = open.get();
+      int opened = open[connection.shardId()].get();
       if (opened <= options().getCoreConnectionsPerHost(hostDistance)) {
         connection.state.set(OPEN);
         return false;
       }
 
-      if (open.compareAndSet(opened, opened - 1)) break;
+      if (open[connection.shardId()].compareAndSet(opened, opened - 1)) break;
     }
     logger.trace("Trashing {}", connection);
     connection.maxIdleTime = System.currentTimeMillis() + options().getIdleTimeoutSeconds() * 1000;
@@ -575,14 +580,14 @@ private boolean addConnectionIfUnderMaximum(int shardId) {
 
     // First, make sure we don't cross the allowed limit of open connections
     for (; ; ) {
-      int opened = open.get();
-      if (opened >= options().getMaxConnectionsPerHost(hostDistance)) return false;
+      int opened = open[shardId].get();
+      if (opened >= maxConnectionsPerShard) return false;
 
-      if (open.compareAndSet(opened, opened + 1)) break;
+      if (open[shardId].compareAndSet(opened, opened + 1)) break;
     }
 
     if (phase.get() != Phase.READY) {
-      open.decrementAndGet();
+      open[shardId].decrementAndGet();
       return false;
     }
 
@@ -591,7 +596,7 @@ private boolean addConnectionIfUnderMaximum(int shardId) {
       Connection newConnection = tryResurrectFromTrash(shardId);
       if (newConnection == null) {
         if (!host.convictionPolicy.canReconnectNow()) {
-          open.decrementAndGet();
+          open[shardId].decrementAndGet();
           return false;
         }
         logger.debug("Creating new connection on busy pool to {}", host);
@@ -619,7 +624,7 @@ private boolean addConnectionIfUnderMaximum(int shardId) {
       // closed in case the pool did not do it.
       if (isClosed() && !newConnection.isClosed()) {
         close(newConnection);
-        open.decrementAndGet();
+        open[shardId].decrementAndGet();
         return false;
       }
 
@@ -628,28 +633,28 @@ private boolean addConnectionIfUnderMaximum(int shardId) {
     } catch (InterruptedException e) {
       Thread.currentThread().interrupt();
       // Skip the open but ignore otherwise
-      open.decrementAndGet();
+      open[shardId].decrementAndGet();
       return false;
     } catch (ConnectionException e) {
-      open.decrementAndGet();
+      open[shardId].decrementAndGet();
       logger.debug("Connection error to {} while creating additional connection", host);
       return false;
     } catch (AuthenticationException e) {
       // This shouldn't really happen in theory
-      open.decrementAndGet();
+      open[shardId].decrementAndGet();
       logger.error(
           "Authentication error while creating additional connection (error is: {})",
           e.getMessage());
       return false;
     } catch (UnsupportedProtocolVersionException e) {
       // This shouldn't happen since we shouldn't have been able to connect in the first place
-      open.decrementAndGet();
+      open[shardId].decrementAndGet();
       logger.error(
           "UnsupportedProtocolVersionException error while creating additional connection (error is: {})",
           e.getMessage());
       return false;
     } catch (ClusterNameMismatchException e) {
-      open.decrementAndGet();
+      open[shardId].decrementAndGet();
       logger.error(
           "ClusterNameMismatchException error while creating additional connection (error is: {})",
           e.getMessage());
@@ -691,7 +696,7 @@ private void maybeSpawnNewConnection(int shardId) {
 
   @Override
   public void onConnectionDefunct(final Connection connection) {
-    if (connection.state.compareAndSet(OPEN, GONE)) open.decrementAndGet();
+    if (connection.state.compareAndSet(OPEN, GONE)) open[connection.shardId()].decrementAndGet();
     connections[connection.shardId()].remove(connection);
 
     // Don't try to replace the connection now. Connection.defunct already signaled the failure,
@@ -715,27 +720,15 @@ private void shrinkIfBelowCapacity() {
     if (currentLoad % maxRequestsPerConnection > options().getNewConnectionThreshold(hostDistance))
       needed += 1;
     needed = Math.max(needed, options().getCoreConnectionsPerHost(hostDistance));
-    int actual = open.get();
-    int toTrash = Math.max(0, actual - needed);
-
-    logger.trace(
-        "Current inFlight = {}, {} connections needed, {} connections available, trashing {}",
-        currentLoad,
-        needed,
-        actual,
-        toTrash);
-
-    if (toTrash <= 0) return;
+    int neededPerShard = needed / connections.length + (needed % connections.length > 0 ? 1 : 0);
 
     for (final List shardsConnections : connections) {
-      if (shardsConnections.size() > connectionsPerShard) {
+      if (shardsConnections.size() > neededPerShard) {
+        int toTrash = shardsConnections.size() - neededPerShard;
         for (Connection connection : shardsConnections) {
           if (trashConnection(connection)) {
             toTrash -= 1;
-            if (toTrash == 0) return;
-            if (shardsConnections.size() <= connectionsPerShard) {
-              break;
-            }
+            if (toTrash == 0) break;
           }
         }
       }
@@ -792,7 +785,11 @@ final CloseFuture closeAsync() {
   }
 
   int opened() {
-    return open.get();
+    int result = 0;
+    for (AtomicInteger o : open) {
+      result += o.get();
+    }
+    return result;
   }
 
   int trashed() {
@@ -824,7 +821,9 @@ private List discardAvailableConnections() {
             new Runnable() {
               @Override
               public void run() {
-                if (connection.state.compareAndSet(OPEN, GONE)) open.decrementAndGet();
+                if (connection.state.compareAndSet(OPEN, GONE)) {
+                  open[connection.shardId()].decrementAndGet();
+                }
               }
             },
             GuavaCompatibility.INSTANCE.sameThreadExecutor());

From 7354c779f4fc580d077f3d8bd32db582b5b727ea Mon Sep 17 00:00:00 2001
From: Piotr Jastrzebski 
Date: Sun, 27 Oct 2019 18:13:01 +0100
Subject: [PATCH 28/52] HostConnectionPool: finish initializing after getting
 first connections

Previously the pool wasn't available for users before we have created
requested number of connections to each shard. This was realized by
not making a future returned from initAsync ready.

Some users were reporting the initialization taking
too long and timing out.

This commit changes the way initAsync works. From now on:
1. initAsync will try to open 2 * |number of shards| connections
2. Then it will close connections that are excessive for each shard
3. For shards that didn't get enough connections, connection tasks
   will be scheduled to create missing connections.

Also we are limiting the number of connections each ConnectionTask
holds to no more than the number of shards.

Signed-off-by: Piotr Jastrzebski 
---
 .../driver/core/HostConnectionPool.java       | 65 +++++++++----------
 1 file changed, 32 insertions(+), 33 deletions(-)

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 64b738f1337..629a8b27a2a 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
@@ -36,7 +36,6 @@
 import com.google.common.util.concurrent.Uninterruptibles;
 import io.netty.util.concurrent.EventExecutor;
 import java.nio.ByteBuffer;
-import java.util.ArrayDeque;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
@@ -174,14 +173,16 @@ ListenableFuture initAsyncWithConnection(Connection reusedConnection) {
       pendingBorrows[i] = new ConcurrentLinkedQueue();
     }
 
-    final List connections = Lists.newArrayListWithCapacity(toCreate);
-    final List> connectionFutures = Lists.newArrayListWithCapacity(toCreate);
+    final List connections = Lists.newArrayListWithCapacity(2 * toCreate);
+    final List> connectionFutures =
+        Lists.newArrayListWithCapacity(2 * toCreate);
 
     toCreate -= 1;
     connections.add(reusedConnection);
     connectionFutures.add(MoreFutures.VOID_SUCCESS);
 
-    List newConnections = manager.connectionFactory().newConnections(this, toCreate);
+    List newConnections =
+        manager.connectionFactory().newConnections(this, 2 * toCreate);
     connections.addAll(newConnections);
     for (Connection connection : newConnections) {
       ListenableFuture connectionFuture = connection.initAsync();
@@ -190,7 +191,7 @@ ListenableFuture initAsyncWithConnection(Connection reusedConnection) {
 
     final SettableFuture initFuture = SettableFuture.create();
 
-    addCallback(connections, connectionFutures, initFuture, new ArrayDeque());
+    addCallback(connections, connectionFutures, initFuture);
 
     return initFuture;
   }
@@ -198,8 +199,7 @@ ListenableFuture initAsyncWithConnection(Connection reusedConnection) {
   private void addCallback(
       final List connections,
       final List> connectionFutures,
-      final SettableFuture initFuture,
-      final Queue toClose) {
+      final SettableFuture initFuture) {
 
     final Executor initExecutor =
         manager.cluster.manager.configuration.getPoolingOptions().getInitializationExecutor();
@@ -217,7 +217,7 @@ public void onSuccess(List l) {
                   HostConnectionPool.this.connections[c.shardId()].add(c);
                   open[c.shardId()].addAndGet(1);
                 } else {
-                  toClose.add(c);
+                  c.closeAsync();
                 }
               }
             }
@@ -228,7 +228,6 @@ public void onSuccess(List l) {
                       host.getSocketAddress(), "Pool was closed during initialization"));
               // we're not sure if closeAsync() saw the connections, so ensure they get closed
               forceClose(connections);
-              forceClose(toClose);
               for (List shardConnections : HostConnectionPool.this.connections) {
                 forceClose(shardConnections);
               }
@@ -236,35 +235,32 @@ public void onSuccess(List l) {
                 o.set(0);
               }
             } else {
-              int needed = 0;
+              int shardId = 0;
+              int[] needed = new int[HostConnectionPool.this.connections.length];
               for (final List shardsConnections : HostConnectionPool.this.connections) {
-                needed +=
+                needed[shardId] =
                     Math.max(
                         0, HostConnectionPool.this.connectionsPerShard - shardsConnections.size());
+                ++shardId;
               }
-              if (needed > 0) {
-                int factor = (hostDistance == HostDistance.LOCAL) ? 2 : 1;
-                int limit =
-                    HostConnectionPool.this.connections.length * connectionsPerShard * factor;
-                final List> connectionFutures =
-                    Lists.newArrayListWithCapacity(needed + Math.max(0, toClose.size() - limit));
-                while (toClose.size() > limit) {
-                  connectionFutures.add(toClose.poll().closeAsync());
-                }
-                final List connections =
-                    manager.connectionFactory().newConnections(HostConnectionPool.this, needed);
-                for (Connection connection : connections) {
-                  ListenableFuture connectionFuture = connection.initAsync();
-                  connectionFutures.add(handleErrors(connectionFuture, initExecutor));
+              // First take permits for connection creation to make sure nothing else starts
+              // connecting
+              for (shardId = 0; shardId < HostConnectionPool.this.connections.length; ++shardId) {
+                if (needed[shardId] > 0) {
+                  if (!scheduledForCreation[shardId].compareAndSet(0, needed[shardId])) {
+                    needed[shardId] = 0;
+                  }
                 }
-                addCallback(connections, connectionFutures, initFuture, toClose);
-              } else {
-                for (final Connection c : toClose) {
-                  c.closeAsync();
+              }
+              // Then mark pool as ready
+              phase.compareAndSet(Phase.INITIALIZING, Phase.READY);
+              // Schedule connection tasks for missing connections
+              for (shardId = 0; shardId < HostConnectionPool.this.connections.length; ++shardId) {
+                while (needed[shardId]-- > 0) {
+                  manager.blockingExecutor().submit(new ConnectionTask(shardId));
                 }
-                phase.compareAndSet(Phase.INITIALIZING, Phase.READY);
-                initFuture.set(null);
               }
+              initFuture.set(null);
             }
           }
 
@@ -272,7 +268,6 @@ public void onSuccess(List l) {
           public void onFailure(Throwable t) {
             phase.compareAndSet(Phase.INITIALIZING, Phase.INIT_FAILED);
             forceClose(connections);
-            forceClose(toClose);
             for (List shardConnections : HostConnectionPool.this.connections) {
               forceClose(shardConnections);
             }
@@ -608,7 +603,11 @@ private boolean addConnectionIfUnderMaximum(int shardId) {
               newConnection.setKeyspace(manager.poolsState.keyspace);
               break;
             }
-            toClose.add(newConnection);
+            if (toClose.size() < connections.length) {
+              toClose.add(newConnection);
+            } else {
+              newConnection.closeAsync();
+            }
           }
         } finally {
           for (Connection c : toClose) {

From ba3443c77a5f8c3d1257d53a9cd59623a7595a67 Mon Sep 17 00:00:00 2001
From: Piotr Jastrzebski 
Date: Sun, 27 Oct 2019 19:39:14 +0100
Subject: [PATCH 29/52] HostConnectionPool: use connection for any shard

if connections for a target shard are still being
initialized.

Signed-off-by: Piotr Jastrzebski 
---
 .../driver/core/HostConnectionPool.java       | 37 +++++++++++++------
 1 file changed, 26 insertions(+), 11 deletions(-)

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 629a8b27a2a..99b1c891d4d 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
@@ -319,6 +319,19 @@ private PoolingOptions options() {
     return manager.configuration().getPoolingOptions();
   }
 
+  private Connection findLeastBusyForShard(int shardId) {
+    int minInFlight = Integer.MAX_VALUE;
+    Connection result = null;
+    for (Connection connection : connections[shardId]) {
+      int inFlight = connection.inFlight.get();
+      if (inFlight < minInFlight) {
+        minInFlight = inFlight;
+        result = connection;
+      }
+    }
+    return result;
+  }
+
   ListenableFuture borrowConnection(
       long timeout, TimeUnit unit, int maxQueueSize, ByteBuffer routingKey) {
     Phase phase = this.phase.get();
@@ -337,29 +350,31 @@ ListenableFuture borrowConnection(
       }
     }
 
+    Connection leastBusy = null;
+
     if (connections[shardId].isEmpty()) {
       if (host.convictionPolicy.canReconnectNow()) {
         if (connectionsPerShard == 0) {
           maybeSpawnNewConnection(shardId);
+          return enqueue(timeout, unit, maxQueueSize, shardId);
         } else if (scheduledForCreation[shardId].compareAndSet(0, connectionsPerShard)) {
           for (int i = 0; i < connectionsPerShard; i++) {
             // We don't respect MAX_SIMULTANEOUS_CREATION here because it's  only to
             // protect against creating connection in excess of core too quickly
             manager.blockingExecutor().submit(new ConnectionTask(shardId));
           }
+          return enqueue(timeout, unit, maxQueueSize, shardId);
         }
-        return enqueue(timeout, unit, maxQueueSize, shardId);
-      }
-    }
-
-    int minInFlight = Integer.MAX_VALUE;
-    Connection leastBusy = null;
-    for (Connection connection : connections[shardId]) {
-      int inFlight = connection.inFlight.get();
-      if (inFlight < minInFlight) {
-        minInFlight = inFlight;
-        leastBusy = connection;
       }
+      // connections for this shard are still being initialized so pick connection for any shard
+      int firstShardToCheck = RAND.nextInt(connections.length);
+      int shardToCheck = firstShardToCheck;
+      do {
+        leastBusy = findLeastBusyForShard(shardToCheck);
+        shardToCheck = (shardToCheck + 1) % connections.length;
+      } while (leastBusy == null && shardToCheck != firstShardToCheck);
+    } else {
+      leastBusy = findLeastBusyForShard(shardId);
     }
 
     if (leastBusy == null) {

From e432c4423fa865d43f8a96d1a75696663dd56dab Mon Sep 17 00:00:00 2001
From: Piotr Jastrzebski 
Date: Mon, 28 Oct 2019 07:56:22 +0100
Subject: [PATCH 30/52] HostConnectionPool: share excessive connections between
 ConnectionTasks

Previously each ConnectionTask was keeping its own excessive connections
(connections that are not to the shard it wants).

This patch changes the behaviour so that all ConnectionTasks
share excessive connections. This way we not only better limit
number of excessive connections but also will quicker find
the right connection for each task (some other task may open
a connection for it).

Signed-off-by: Piotr Jastrzebski 
---
 .../driver/core/HostConnectionPool.java       | 89 +++++++++++++++----
 1 file changed, 70 insertions(+), 19 deletions(-)

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 99b1c891d4d..45e8c673f35 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,7 +38,9 @@
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
 import java.util.Queue;
 import java.util.Random;
 import java.util.Set;
@@ -97,7 +99,60 @@ private enum Phase {
 
   protected final AtomicReference phase = new AtomicReference(Phase.INITIALIZING);
 
+  public static class ConnectionTasksSharedState {
+    private final Object lock = new Object();
+    private int tasksInFlight = 0;
+    private Map connectionsToClose = new HashMap();
+
+    public Connection registerTask(int shardId) {
+      Connection c = null;
+      synchronized (lock) {
+        c = connectionsToClose.remove(shardId);
+        if (c == null) {
+          ++tasksInFlight;
+        }
+      }
+      return c;
+    }
+
+    public void unregisterTask() {
+      Map toClose = null;
+      synchronized (lock) {
+        --tasksInFlight;
+        if (tasksInFlight == 0) {
+          toClose = connectionsToClose;
+          connectionsToClose = new HashMap();
+        }
+      }
+      if (toClose != null) {
+        for (Connection c : toClose.values()) {
+          c.closeAsync();
+        }
+      }
+    }
+
+    public Connection addConnectionToClose(int shardId, Connection c) {
+      Connection res = null;
+      boolean close = false;
+      synchronized (lock) {
+        res = connectionsToClose.remove(shardId);
+        close = connectionsToClose.get(c.shardId()) != null;
+        if (!close) {
+          connectionsToClose.put(c.shardId(), c);
+        }
+      }
+      if (close) {
+        c.closeAsync();
+      }
+      return res;
+    }
+  }
+
+  private final ConnectionTasksSharedState connectionTasksSharedState =
+      new ConnectionTasksSharedState();
+
   private class ConnectionTask implements Runnable {
+
     private final int shardId;
 
     public ConnectionTask(int shardId) {
@@ -106,7 +161,7 @@ public ConnectionTask(int shardId) {
 
     @Override
     public void run() {
-      addConnectionIfUnderMaximum(shardId);
+      addConnectionIfUnderMaximum(shardId, connectionTasksSharedState);
       scheduledForCreation[shardId].decrementAndGet();
     }
   }
@@ -586,7 +641,7 @@ private void doTrashConnection(Connection connection) {
     trash[connection.shardId()].add(connection);
   }
 
-  private boolean addConnectionIfUnderMaximum(int shardId) {
+  private boolean addConnectionIfUnderMaximum(int shardId, ConnectionTasksSharedState sharedState) {
 
     // First, make sure we don't cross the allowed limit of open connections
     for (; ; ) {
@@ -610,23 +665,19 @@ private boolean addConnectionIfUnderMaximum(int shardId) {
           return false;
         }
         logger.debug("Creating new connection on busy pool to {}", host);
-        List toClose = new ArrayList();
-        try {
-          while (true) {
-            newConnection = manager.connectionFactory().open(this);
-            if (newConnection.shardId() == shardId) {
-              newConnection.setKeyspace(manager.poolsState.keyspace);
-              break;
-            }
-            if (toClose.size() < connections.length) {
-              toClose.add(newConnection);
-            } else {
-              newConnection.closeAsync();
-            }
-          }
-        } finally {
-          for (Connection c : toClose) {
-            c.closeAsync();
+        newConnection = sharedState.registerTask(shardId);
+        if (newConnection == null) {
+          try {
+            do {
+              newConnection = manager.connectionFactory().open(this);
+              if (newConnection.shardId() == shardId) {
+                newConnection.setKeyspace(manager.poolsState.keyspace);
+              } else {
+                newConnection = sharedState.addConnectionToClose(shardId, newConnection);
+              }
+            } while (newConnection == null);
+          } finally {
+            sharedState.unregisterTask();
           }
         }
       }

From 9ab00c44621727531e639e03f43cf57783e5c1a4 Mon Sep 17 00:00:00 2001
From: Piotr Jastrzebski 
Date: Wed, 13 Nov 2019 10:14:48 +0100
Subject: [PATCH 31/52] [maven-release-plugin] prepare release 3.7.1-scylla-2

---
 driver-core/pom.xml                   | 2 +-
 driver-dist/pom.xml                   | 2 +-
 driver-examples/pom.xml               | 2 +-
 driver-extras/pom.xml                 | 2 +-
 driver-mapping/pom.xml                | 2 +-
 driver-tests/osgi/pom.xml             | 2 +-
 driver-tests/pom.xml                  | 2 +-
 driver-tests/shading/pom.xml          | 2 +-
 driver-tests/shading/shaded/pom.xml   | 2 +-
 driver-tests/shading/unshaded/pom.xml | 2 +-
 driver-tests/stress/pom.xml           | 2 +-
 pom.xml                               | 4 ++--
 12 files changed, 13 insertions(+), 13 deletions(-)

diff --git a/driver-core/pom.xml b/driver-core/pom.xml
index bf0fa12d868..7c64df570e1 100644
--- a/driver-core/pom.xml
+++ b/driver-core/pom.xml
@@ -22,7 +22,7 @@
     
         com.scylladb
         scylla-driver-parent
-        3.7.1-scylla-2-SNAPSHOT
+        3.7.1-scylla-2
     
 
     scylla-driver-core
diff --git a/driver-dist/pom.xml b/driver-dist/pom.xml
index b238a1ec497..1df0bc79aa7 100644
--- a/driver-dist/pom.xml
+++ b/driver-dist/pom.xml
@@ -22,7 +22,7 @@
     
         com.scylladb
         scylla-driver-parent
-        3.7.1-scylla-2-SNAPSHOT
+        3.7.1-scylla-2
     
 
     scylla-driver-dist
diff --git a/driver-examples/pom.xml b/driver-examples/pom.xml
index 69204c88a5a..776e8b83759 100644
--- a/driver-examples/pom.xml
+++ b/driver-examples/pom.xml
@@ -22,7 +22,7 @@
     
         com.scylladb
         scylla-driver-parent
-        3.7.1-scylla-2-SNAPSHOT
+        3.7.1-scylla-2
     
 
     scylla-driver-examples
diff --git a/driver-extras/pom.xml b/driver-extras/pom.xml
index 055f88c1444..4bd0e0737b9 100644
--- a/driver-extras/pom.xml
+++ b/driver-extras/pom.xml
@@ -22,7 +22,7 @@
     
         com.scylladb
         scylla-driver-parent
-        3.7.1-scylla-2-SNAPSHOT
+        3.7.1-scylla-2
     
 
     scylla-driver-extras
diff --git a/driver-mapping/pom.xml b/driver-mapping/pom.xml
index c0ee8484fac..df3d8fe18e4 100644
--- a/driver-mapping/pom.xml
+++ b/driver-mapping/pom.xml
@@ -22,7 +22,7 @@
     
         com.scylladb
         scylla-driver-parent
-        3.7.1-scylla-2-SNAPSHOT
+        3.7.1-scylla-2
     
 
     scylla-driver-mapping
diff --git a/driver-tests/osgi/pom.xml b/driver-tests/osgi/pom.xml
index cf51d20a71f..9b2304f50de 100644
--- a/driver-tests/osgi/pom.xml
+++ b/driver-tests/osgi/pom.xml
@@ -22,7 +22,7 @@
     
         com.scylladb
         scylla-driver-tests-parent
-        3.7.1-scylla-2-SNAPSHOT
+        3.7.1-scylla-2
     
 
     scylla-driver-tests-osgi
diff --git a/driver-tests/pom.xml b/driver-tests/pom.xml
index bd1019615d8..7ace92e4f62 100644
--- a/driver-tests/pom.xml
+++ b/driver-tests/pom.xml
@@ -22,7 +22,7 @@
     
         com.scylladb
         scylla-driver-parent
-        3.7.1-scylla-2-SNAPSHOT
+        3.7.1-scylla-2
     
 
     scylla-driver-tests-parent
diff --git a/driver-tests/shading/pom.xml b/driver-tests/shading/pom.xml
index ad81fa64c7b..4719211eef1 100644
--- a/driver-tests/shading/pom.xml
+++ b/driver-tests/shading/pom.xml
@@ -22,7 +22,7 @@
     
         com.scylladb
         scylla-driver-tests-parent
-        3.7.1-scylla-2-SNAPSHOT
+        3.7.1-scylla-2
     
 
     pom
diff --git a/driver-tests/shading/shaded/pom.xml b/driver-tests/shading/shaded/pom.xml
index d3ec0b815e7..95b07950b33 100644
--- a/driver-tests/shading/shaded/pom.xml
+++ b/driver-tests/shading/shaded/pom.xml
@@ -22,7 +22,7 @@
     
         com.scylladb
         scylla-driver-tests-shading
-        3.7.1-scylla-2-SNAPSHOT
+        3.7.1-scylla-2
     
 
     scylla-driver-tests-shading-shaded
diff --git a/driver-tests/shading/unshaded/pom.xml b/driver-tests/shading/unshaded/pom.xml
index e214519d4de..08287a09187 100644
--- a/driver-tests/shading/unshaded/pom.xml
+++ b/driver-tests/shading/unshaded/pom.xml
@@ -22,7 +22,7 @@
     
         com.scylladb
         scylla-driver-tests-shading
-        3.7.1-scylla-2-SNAPSHOT
+        3.7.1-scylla-2
     
 
     scylla-driver-tests-shading-unshaded
diff --git a/driver-tests/stress/pom.xml b/driver-tests/stress/pom.xml
index 0476fa9b0d2..e401034ec41 100644
--- a/driver-tests/stress/pom.xml
+++ b/driver-tests/stress/pom.xml
@@ -22,7 +22,7 @@
     
         com.scylladb
         scylla-driver-tests-parent
-        3.7.1-scylla-2-SNAPSHOT
+        3.7.1-scylla-2
     
 
     scylla-driver-tests-stress
diff --git a/pom.xml b/pom.xml
index 14b39b9ac60..c63e1f17cf6 100644
--- a/pom.xml
+++ b/pom.xml
@@ -21,7 +21,7 @@
 
     com.scylladb
     scylla-driver-parent
-    3.7.1-scylla-2-SNAPSHOT
+    3.7.1-scylla-2
     pom
     Java Driver for Scylla and Apache Cassandra
     
@@ -1162,7 +1162,7 @@
         scm:git:https://github.com/scylladb/java-driver
         scm:git:https://github.com/scylladb/java-driver
         https://github.com/scylladb/java-driver
-        HEAD
+        3.7.1-scylla-2
     
 
     

From f3ee452ae7547acc79c53cfbd45fd3bcf1705800 Mon Sep 17 00:00:00 2001
From: Piotr Jastrzebski 
Date: Wed, 13 Nov 2019 10:14:55 +0100
Subject: [PATCH 32/52] [maven-release-plugin] prepare for next development
 iteration

---
 driver-core/pom.xml                   | 2 +-
 driver-dist/pom.xml                   | 2 +-
 driver-examples/pom.xml               | 2 +-
 driver-extras/pom.xml                 | 2 +-
 driver-mapping/pom.xml                | 2 +-
 driver-tests/osgi/pom.xml             | 2 +-
 driver-tests/pom.xml                  | 2 +-
 driver-tests/shading/pom.xml          | 2 +-
 driver-tests/shading/shaded/pom.xml   | 2 +-
 driver-tests/shading/unshaded/pom.xml | 2 +-
 driver-tests/stress/pom.xml           | 2 +-
 pom.xml                               | 4 ++--
 12 files changed, 13 insertions(+), 13 deletions(-)

diff --git a/driver-core/pom.xml b/driver-core/pom.xml
index 7c64df570e1..fc619255c63 100644
--- a/driver-core/pom.xml
+++ b/driver-core/pom.xml
@@ -22,7 +22,7 @@
     
         com.scylladb
         scylla-driver-parent
-        3.7.1-scylla-2
+        3.7.1-scylla-3-SNAPSHOT
     
 
     scylla-driver-core
diff --git a/driver-dist/pom.xml b/driver-dist/pom.xml
index 1df0bc79aa7..8374a6462f7 100644
--- a/driver-dist/pom.xml
+++ b/driver-dist/pom.xml
@@ -22,7 +22,7 @@
     
         com.scylladb
         scylla-driver-parent
-        3.7.1-scylla-2
+        3.7.1-scylla-3-SNAPSHOT
     
 
     scylla-driver-dist
diff --git a/driver-examples/pom.xml b/driver-examples/pom.xml
index 776e8b83759..c1f89b7ce89 100644
--- a/driver-examples/pom.xml
+++ b/driver-examples/pom.xml
@@ -22,7 +22,7 @@
     
         com.scylladb
         scylla-driver-parent
-        3.7.1-scylla-2
+        3.7.1-scylla-3-SNAPSHOT
     
 
     scylla-driver-examples
diff --git a/driver-extras/pom.xml b/driver-extras/pom.xml
index 4bd0e0737b9..559c739015a 100644
--- a/driver-extras/pom.xml
+++ b/driver-extras/pom.xml
@@ -22,7 +22,7 @@
     
         com.scylladb
         scylla-driver-parent
-        3.7.1-scylla-2
+        3.7.1-scylla-3-SNAPSHOT
     
 
     scylla-driver-extras
diff --git a/driver-mapping/pom.xml b/driver-mapping/pom.xml
index df3d8fe18e4..401b8eb7f9c 100644
--- a/driver-mapping/pom.xml
+++ b/driver-mapping/pom.xml
@@ -22,7 +22,7 @@
     
         com.scylladb
         scylla-driver-parent
-        3.7.1-scylla-2
+        3.7.1-scylla-3-SNAPSHOT
     
 
     scylla-driver-mapping
diff --git a/driver-tests/osgi/pom.xml b/driver-tests/osgi/pom.xml
index 9b2304f50de..b02b5c12526 100644
--- a/driver-tests/osgi/pom.xml
+++ b/driver-tests/osgi/pom.xml
@@ -22,7 +22,7 @@
     
         com.scylladb
         scylla-driver-tests-parent
-        3.7.1-scylla-2
+        3.7.1-scylla-3-SNAPSHOT
     
 
     scylla-driver-tests-osgi
diff --git a/driver-tests/pom.xml b/driver-tests/pom.xml
index 7ace92e4f62..6eeb54ba4bd 100644
--- a/driver-tests/pom.xml
+++ b/driver-tests/pom.xml
@@ -22,7 +22,7 @@
     
         com.scylladb
         scylla-driver-parent
-        3.7.1-scylla-2
+        3.7.1-scylla-3-SNAPSHOT
     
 
     scylla-driver-tests-parent
diff --git a/driver-tests/shading/pom.xml b/driver-tests/shading/pom.xml
index 4719211eef1..6287a43fe60 100644
--- a/driver-tests/shading/pom.xml
+++ b/driver-tests/shading/pom.xml
@@ -22,7 +22,7 @@
     
         com.scylladb
         scylla-driver-tests-parent
-        3.7.1-scylla-2
+        3.7.1-scylla-3-SNAPSHOT
     
 
     pom
diff --git a/driver-tests/shading/shaded/pom.xml b/driver-tests/shading/shaded/pom.xml
index 95b07950b33..ddec5dc7371 100644
--- a/driver-tests/shading/shaded/pom.xml
+++ b/driver-tests/shading/shaded/pom.xml
@@ -22,7 +22,7 @@
     
         com.scylladb
         scylla-driver-tests-shading
-        3.7.1-scylla-2
+        3.7.1-scylla-3-SNAPSHOT
     
 
     scylla-driver-tests-shading-shaded
diff --git a/driver-tests/shading/unshaded/pom.xml b/driver-tests/shading/unshaded/pom.xml
index 08287a09187..9cef305fde1 100644
--- a/driver-tests/shading/unshaded/pom.xml
+++ b/driver-tests/shading/unshaded/pom.xml
@@ -22,7 +22,7 @@
     
         com.scylladb
         scylla-driver-tests-shading
-        3.7.1-scylla-2
+        3.7.1-scylla-3-SNAPSHOT
     
 
     scylla-driver-tests-shading-unshaded
diff --git a/driver-tests/stress/pom.xml b/driver-tests/stress/pom.xml
index e401034ec41..25765ae8912 100644
--- a/driver-tests/stress/pom.xml
+++ b/driver-tests/stress/pom.xml
@@ -22,7 +22,7 @@
     
         com.scylladb
         scylla-driver-tests-parent
-        3.7.1-scylla-2
+        3.7.1-scylla-3-SNAPSHOT
     
 
     scylla-driver-tests-stress
diff --git a/pom.xml b/pom.xml
index c63e1f17cf6..e39c75fe157 100644
--- a/pom.xml
+++ b/pom.xml
@@ -21,7 +21,7 @@
 
     com.scylladb
     scylla-driver-parent
-    3.7.1-scylla-2
+    3.7.1-scylla-3-SNAPSHOT
     pom
     Java Driver for Scylla and Apache Cassandra
     
@@ -1162,7 +1162,7 @@
         scm:git:https://github.com/scylladb/java-driver
         scm:git:https://github.com/scylladb/java-driver
         https://github.com/scylladb/java-driver
-        3.7.1-scylla-2
+        HEAD
     
 
     

From d8e0010548e0c3c86b2538abc6c7234b78e43ba0 Mon Sep 17 00:00:00 2001
From: David Garcia 
Date: Fri, 7 Aug 2020 13:47:35 +0200
Subject: [PATCH 33/52] Added scylladb docs style (#1)

---
 .github/workflows/pages.yml                   |   37 +
 .gitignore                                    |    5 +-
 README-dev.rst                                |   36 +
 changelog/README.md                           | 1624 -----------------
 docs.yaml                                     |   67 -
 docs/Makefile                                 |   72 +
 docs/_utils/api.html                          |    8 +
 docs/_utils/deploy.sh                         |   16 +
 docs/_utils/doxygen.sh                        |   12 +
 docs/_utils/multiversion.sh                   |    3 +
 docs/_utils/redirect.sh                       |   13 +
 docs/_utils/redirections.yaml                 |    1 +
 docs/_utils/setup.sh                          |   16 +
 docs/poetry.lock                              |  808 ++++++++
 docs/pyproject.toml                           |   23 +
 docs/source/api.rst                           |    3 +
 docs/source/changelog/index.md                | 1624 +++++++++++++++++
 docs/source/conf.py                           |  250 +++
 faq/README.md => docs/source/faq/index.md     |   24 +-
 .../source/faq/osgi/index.md                  |   12 +-
 docs/source/index.rst                         |   99 +
 docs/source/installation/index.md             |   76 +
 .../source/manual/address_resolution/index.md |    2 +-
 .../source/manual/async/index.md              |    2 +-
 .../source/manual/auth/index.md               |    0
 .../source/manual/compression/index.md        |    8 +-
 .../source/manual/control_connection/index.md |   12 +-
 .../manual/custom_codecs/extras/index.md      |   10 +-
 .../source/manual/custom_codecs/index.md      |    6 +-
 .../source/manual/custom_payloads/index.md    |    2 +-
 .../source/manual/idempotence/index.md        |    2 +-
 .../README.md => docs/source/manual/index.md  |   80 +-
 .../source/manual/load_balancing/index.md     |   18 +-
 .../source/manual/logging/index.md            |    0
 .../source/manual/metadata/index.md           |    2 +-
 .../source/manual/metrics/index.md            |    0
 .../source/manual/native_protocol/index.md    |    8 +-
 .../manual/object_mapper/creating/index.md    |    6 +-
 .../object_mapper/custom_codecs/index.md      |    6 +-
 .../source/manual/object_mapper/index.md      |   15 +-
 .../manual/object_mapper/using/index.md       |    4 +-
 .../source/manual/osgi/index.md               |    6 +-
 .../source/manual/paging/index.md             |    6 +-
 .../source/manual/pooling/index.md            |    8 +-
 .../source/manual/query_timestamps/index.md   |    4 +-
 .../source/manual/reconnection/index.md       |    4 +-
 .../source/manual/retries/index.md            |   10 +-
 .../source/manual/shaded_jar/index.md         |    2 +-
 .../source/manual/socket_options/index.md     |   12 +-
 .../manual/speculative_execution/index.md     |   14 +-
 .../source/manual/ssl/index.md                |    0
 .../source/manual/statements/batch/index.md   |    0
 .../source/manual/statements/built/index.md   |    6 +-
 .../source/manual/statements/index.md         |   18 +-
 .../manual/statements/prepared/index.md       |   12 +-
 .../source/manual/statements/simple/index.md  |    6 +-
 .../source/manual/tuples/index.md             |    0
 .../source/manual/udts/index.md               |    2 +-
 .../source/upgrade_guide/index.md             |   37 +-
 .../configuration/index.md                    |   22 +-
 .../migrating_from_astyanax/index.md          |   16 +-
 .../language_level_changes/index.md           |    8 +-
 .../queries_and_results/index.md              |   12 +-
 doxyfile                                      |    4 +-
 manual/object_mapper/.nav                     |    3 -
 manual/statements/.nav                        |    4 -
 upgrade_guide/migrating_from_astyanax/.nav    |    3 -
 67 files changed, 3361 insertions(+), 1870 deletions(-)
 create mode 100644 .github/workflows/pages.yml
 create mode 100644 README-dev.rst
 delete mode 100644 changelog/README.md
 delete mode 100644 docs.yaml
 create mode 100644 docs/Makefile
 create mode 100644 docs/_utils/api.html
 create mode 100755 docs/_utils/deploy.sh
 create mode 100755 docs/_utils/doxygen.sh
 create mode 100755 docs/_utils/multiversion.sh
 create mode 100755 docs/_utils/redirect.sh
 create mode 100644 docs/_utils/redirections.yaml
 create mode 100755 docs/_utils/setup.sh
 create mode 100644 docs/poetry.lock
 create mode 100644 docs/pyproject.toml
 create mode 100644 docs/source/api.rst
 create mode 100644 docs/source/changelog/index.md
 create mode 100644 docs/source/conf.py
 rename faq/README.md => docs/source/faq/index.md (94%)
 rename faq/osgi/README.md => docs/source/faq/osgi/index.md (95%)
 create mode 100644 docs/source/index.rst
 create mode 100644 docs/source/installation/index.md
 rename manual/address_resolution/README.md => docs/source/manual/address_resolution/index.md (98%)
 rename manual/async/README.md => docs/source/manual/async/index.md (98%)
 rename manual/auth/README.md => docs/source/manual/auth/index.md (100%)
 rename manual/compression/README.md => docs/source/manual/compression/index.md (92%)
 rename manual/control_connection/README.md => docs/source/manual/control_connection/index.md (67%)
 rename manual/custom_codecs/extras/README.md => docs/source/manual/custom_codecs/extras/index.md (96%)
 rename manual/custom_codecs/README.md => docs/source/manual/custom_codecs/index.md (99%)
 rename manual/custom_payloads/README.md => docs/source/manual/custom_payloads/index.md (98%)
 rename manual/idempotence/README.md => docs/source/manual/idempotence/index.md (97%)
 rename manual/README.md => docs/source/manual/index.md (86%)
 rename manual/load_balancing/README.md => docs/source/manual/load_balancing/index.md (94%)
 rename manual/logging/README.md => docs/source/manual/logging/index.md (100%)
 rename manual/metadata/README.md => docs/source/manual/metadata/index.md (99%)
 rename manual/metrics/README.md => docs/source/manual/metrics/index.md (100%)
 rename manual/native_protocol/README.md => docs/source/manual/native_protocol/index.md (96%)
 rename manual/object_mapper/creating/README.md => docs/source/manual/object_mapper/creating/index.md (98%)
 rename manual/object_mapper/custom_codecs/README.md => docs/source/manual/object_mapper/custom_codecs/index.md (95%)
 rename manual/object_mapper/README.md => docs/source/manual/object_mapper/index.md (69%)
 rename manual/object_mapper/using/README.md => docs/source/manual/object_mapper/using/index.md (98%)
 rename manual/osgi/README.md => docs/source/manual/osgi/index.md (84%)
 rename manual/paging/README.md => docs/source/manual/paging/index.md (98%)
 rename manual/pooling/README.md => docs/source/manual/pooling/index.md (98%)
 rename manual/query_timestamps/README.md => docs/source/manual/query_timestamps/index.md (97%)
 rename manual/reconnection/README.md => docs/source/manual/reconnection/index.md (89%)
 rename manual/retries/README.md => docs/source/manual/retries/index.md (97%)
 rename manual/shaded_jar/README.md => docs/source/manual/shaded_jar/index.md (97%)
 rename manual/socket_options/README.md => docs/source/manual/socket_options/index.md (93%)
 rename manual/speculative_execution/README.md => docs/source/manual/speculative_execution/index.md (97%)
 rename manual/ssl/README.md => docs/source/manual/ssl/index.md (100%)
 rename manual/statements/batch/README.md => docs/source/manual/statements/batch/index.md (100%)
 rename manual/statements/built/README.md => docs/source/manual/statements/built/index.md (98%)
 rename manual/statements/README.md => docs/source/manual/statements/index.md (83%)
 rename manual/statements/prepared/README.md => docs/source/manual/statements/prepared/index.md (98%)
 rename manual/statements/simple/README.md => docs/source/manual/statements/simple/index.md (97%)
 rename manual/tuples/README.md => docs/source/manual/tuples/index.md (100%)
 rename manual/udts/README.md => docs/source/manual/udts/index.md (97%)
 rename upgrade_guide/README.md => docs/source/upgrade_guide/index.md (97%)
 rename upgrade_guide/migrating_from_astyanax/configuration/README.md => docs/source/upgrade_guide/migrating_from_astyanax/configuration/index.md (96%)
 rename upgrade_guide/migrating_from_astyanax/README.md => docs/source/upgrade_guide/migrating_from_astyanax/index.md (56%)
 rename upgrade_guide/migrating_from_astyanax/language_level_changes/README.md => docs/source/upgrade_guide/migrating_from_astyanax/language_level_changes/index.md (98%)
 rename upgrade_guide/migrating_from_astyanax/queries_and_results/README.md => docs/source/upgrade_guide/migrating_from_astyanax/queries_and_results/index.md (89%)
 delete mode 100644 manual/object_mapper/.nav
 delete mode 100644 manual/statements/.nav
 delete mode 100644 upgrade_guide/migrating_from_astyanax/.nav

diff --git a/.github/workflows/pages.yml b/.github/workflows/pages.yml
new file mode 100644
index 00000000000..e9698efe8e9
--- /dev/null
+++ b/.github/workflows/pages.yml
@@ -0,0 +1,37 @@
+name: "CI Docs"
+
+on:
+  push:
+    branches:
+    - latest
+    tags:       
+    - '**'
+jobs:
+  release:
+    name: Build
+    runs-on: ubuntu-latest
+    steps:
+    - name: Checkout
+      uses: actions/checkout@v2
+      with:
+        persist-credentials: false
+        fetch-depth: 0
+    - name: Set up Python
+      uses: actions/setup-python@v1
+      with:
+        python-version: 3.7
+    - name: Set up Doxygen
+      run: sudo apt-get install doxygen
+    - name: Build Sphinx docs
+      run: |
+        export PATH=$PATH:~/.local/bin
+        cd docs
+        make multiversion
+    - name: Build Doxygen docs
+      run: |
+        ./docs/_utils/doxygen.sh
+    - name: Deploy
+      run : ./docs/_utils/deploy.sh
+      env:
+        GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }}
+        LATEST_VERSION: latest
diff --git a/.gitignore b/.gitignore
index 7cf0248307f..50bf5df9257 100644
--- a/.gitignore
+++ b/.gitignore
@@ -4,8 +4,9 @@ testing/
 .settings
 .classpath
 .project
-doc
-docs
+docs/_build
+html
+latex
 notes
 .DS_Store
 
diff --git a/README-dev.rst b/README-dev.rst
new file mode 100644
index 00000000000..378ecf78455
--- /dev/null
+++ b/README-dev.rst
@@ -0,0 +1,36 @@
+Building the Docs
+=================
+
+*Note*: The docs build instructions have been tested with Sphinx 2.4.4 and Fedora 32.
+
+To build and preview the theme locally, you will need to install the following software:
+
+- `Git `_
+- `Python 3.7 `_
+- `pip `_
+- `doxygen `_
+
+Run the following command to build the docs.
+
+.. code:: console
+
+    cd docs
+    make preview
+
+Once the command completes processing, open http://127.0.0.1:5500/ with your preferred browser.
+
+Building multiple documentation versions
+========================================
+
+Build Sphinx docs for all the versions defined in ``docs/conf.py``.
+
+The multiverson command does not build doxygen docs.
+
+```
+cd docs
+make multiversion
+```
+
+Then, open ``docs/_build/dirhtml//index.html`` with your preferred browser.
+
+**NOTE:** If you only can see docs generated for the master branch, try to run ``git fetch --tags`` to download the latest tags from remote.
diff --git a/changelog/README.md b/changelog/README.md
deleted file mode 100644
index 2b32d3ead94..00000000000
--- a/changelog/README.md
+++ /dev/null
@@ -1,1624 +0,0 @@
-## Changelog
-
-### 3.7.1
-
-- [bug] JAVA-2174: Metadata.needsQuote should accept empty strings.
-
-
-### 3.7.0
-
-- [improvement] JAVA-2025: Include exception message in Abstract\*Codec.accepts(null).
-- [improvement] JAVA-1980: Use covariant return types in RemoteEndpointAwareJdkSSLOptions.Builder methods.
-- [documentation] JAVA-2062: Document frozen collection preference with Mapper.
-- [bug] JAVA-2071: Fix NPE in ArrayBackedRow.toString().
-- [bug] JAVA-2070: Call onRemove instead of onDown when rack and/or DC information changes for a host.
-- [improvement] JAVA-1256: Log parameters of BuiltStatement in QueryLogger.
-- [documentation] JAVA-2074: Document preference for LZ4 over Snappy.
-- [bug] JAVA-1612: Include netty-common jar in binary tarball.
-- [improvement] JAVA-2003: Simplify CBUtil internal API to improve performance.
-- [improvement] JAVA-2002: Reimplement TypeCodec.accepts to improve performance.
-- [documentation] JAVA-2041: Deprecate cross-DC failover in DCAwareRoundRobinPolicy.
-- [documentation] JAVA-1159: Document workaround for using tuple with udt field in Mapper.
-- [documentation] JAVA-1964: Complete remaining "Coming Soon" sections in docs.
-- [improvement] JAVA-1950: Log server side warnings returned from a query.
-- [improvement] JAVA-2123: Allow to use QueryBuilder for building queries against Materialized Views.
-- [bug] JAVA-2082: Avoid race condition during cluster close and schema refresh.
-
-
-### 3.6.0
-
-- [improvement] JAVA-1394: Add request-queue-depth metric.
-- [improvement] JAVA-1857: Add Statement.setHost.
-- [bug] JAVA-1920: Use nanosecond precision in LocalTimeCodec#format().
-- [bug] JAVA-1794: Driver tries to create a connection array of size -1.
-- [new feature] JAVA-1899: Support virtual tables.
-- [bug] JAVA-1908: TableMetadata.asCQLQuery does not add table option 'memtable_flush_period_in_ms' in the generated query.
-- [bug] JAVA-1924: StatementWrapper setters should return the wrapping statement.
-- [new feature] JAVA-1532: Add Codec support for Java 8's LocalDateTime and ZoneId.
-- [improvement] JAVA-1786: Use Google code formatter.
-- [bug] JAVA-1871: Change LOCAL\_SERIAL.isDCLocal() to return true.
-- [documentation] JAVA-1902: Clarify unavailable & request error in DefaultRetryPolicy javadoc.
-- [new feature] JAVA-1903: Add WhiteListPolicy.ofHosts.
-- [bug] JAVA-1928: Fix GuavaCompatibility for Guava 26.
-- [bug] JAVA-1935: Add null check in QueryConsistencyException.getHost.
-- [improvement] JAVA-1771: Send driver name and version in STARTUP message.
-- [improvement] JAVA-1388: Add dynamic port discovery for system.peers\_v2.
-- [documentation] JAVA-1810: Note which setters are not propagated to PreparedStatement.
-- [bug] JAVA-1944: Surface Read and WriteFailureException to RetryPolicy.
-- [bug] JAVA-1211: Fix NPE in cluster close when cluster init fails.
-- [bug] JAVA-1220: Fail fast on cluster init if previous init failed.
-- [bug] JAVA-1929: Preempt session execute queries if session was closed.
-
-Merged from 3.5.x:
-
-- [bug] JAVA-1872: Retain table's views when processing table update.
-
-
-### 3.5.0
-
-- [improvement] JAVA-1448: TokenAwarePolicy should respect child policy ordering.
-- [bug] JAVA-1751: Include defaultTimestamp length in encodedSize for protocol version >= 3.
-- [bug] JAVA-1770: Fix message size when using Custom Payload.
-- [documentation] JAVA-1760: Add metrics documentation.
-- [improvement] JAVA-1765: Update dependencies to latest patch versions.
-- [improvement] JAVA-1752: Deprecate DowngradingConsistencyRetryPolicy.
-- [improvement] JAVA-1735: Log driver version on first use.
-- [documentation] JAVA-1380: Add FAQ entry for errors arising from incompatibilities.
-- [improvement] JAVA-1748: Support IS NOT NULL and != in query builder.
-- [documentation] JAVA-1740: Mention C*2.2/3.0 incompatibilities in paging state manual.
-- [improvement] JAVA-1725: Add a getNodeCount method to CCMAccess for easier automation.
-- [new feature] JAVA-708: Add means to measure request sizes.
-- [documentation] JAVA-1788: Add example for enabling host name verification to SSL docs.
-- [improvement] JAVA-1791: Revert "JAVA-1677: Warn if auth is configured on the client but not the server."
-- [bug] JAVA-1789: Account for flags in Prepare encodedSize.
-- [bug] JAVA-1797: Use jnr-ffi version required by jnr-posix.
-
-
-### 3.4.0
-
-- [improvement] JAVA-1671: Remove unnecessary test on prepared statement metadata.
-- [bug] JAVA-1694: Upgrade to jackson-databind 2.7.9.2 to address CVE-2015-15095.
-- [documentation] JAVA-1685: Clarify recommendation on preparing SELECT *.
-- [improvement] JAVA-1679: Improve error message on batch log write timeout.
-- [improvement] JAVA-1672: Remove schema agreement check when repreparing on up.
-- [improvement] JAVA-1677: Warn if auth is configured on the client but not the server.
-- [new feature] JAVA-1651: Add NO_COMPACT startup option.
-- [improvement] JAVA-1683: Add metrics to track writes to nodes.
-- [new feature] JAVA-1229: Allow specifying the keyspace for individual queries.
-- [improvement] JAVA-1682: Provide a way to record latencies for cancelled speculative executions.
-- [improvement] JAVA-1717: Add metrics to latency-aware policy.
-- [improvement] JAVA-1675: Remove dates from copyright headers.
-
-Merged from 3.3.x:
-
-- [bug] JAVA-1555: Include VIEW and CDC in WriteType.
-- [bug] JAVA-1599: exportAsString improvements (sort, format, clustering order)
-- [improvement] JAVA-1587: Deterministic ordering of columns used in Mapper#saveQuery
-- [improvement] JAVA-1500: Add a metric to report number of in-flight requests.
-- [bug] JAVA-1438: QueryBuilder check for empty orderings.
-- [improvement] JAVA-1490: Allow zero delay for speculative executions.
-- [documentation] JAVA-1607: Add FAQ entry for netty-transport-native-epoll.
-- [bug] JAVA-1630: Fix Metadata.addIfAbsent.
-- [improvement] JAVA-1619: Update QueryBuilder methods to support Iterable input.
-- [improvement] JAVA-1527: Expose host_id and schema_version on Host metadata.
-- [new feature] JAVA-1377: Add support for TWCS in SchemaBuilder.
-- [improvement] JAVA-1631: Publish a sources jar for driver-core-tests.
-- [improvement] JAVA-1632: Add a withIpPrefix(String) method to CCMBridge.Builder.
-- [bug] JAVA-1639: VersionNumber does not fullfill equals/hashcode contract.
-- [bug] JAVA-1613: Fix broken shaded Netty detection in NettyUtil.
-- [bug] JAVA-1666: Fix keyspace export when a UDT has case-sensitive field names.
-- [improvement] JAVA-1196: Include hash of result set metadata in prepared statement id.
-- [improvement] JAVA-1670: Support user-provided JMX ports for CCMBridge.
-- [improvement] JAVA-1661: Avoid String.toLowerCase if possible in Metadata.
-- [improvement] JAVA-1659: Expose low-level flusher tuning options.
-- [improvement] JAVA-1660: Support netty-transport-native-epoll in OSGi container.
-
-
-### 3.3.2
-
-- [bug] JAVA-1666: Fix keyspace export when a UDT has case-sensitive field names.
-- [improvement] JAVA-1196: Include hash of result set metadata in prepared statement id.
-- [improvement] JAVA-1670: Support user-provided JMX ports for CCMBridge.
-- [improvement] JAVA-1661: Avoid String.toLowerCase if possible in Metadata.
-- [improvement] JAVA-1659: Expose low-level flusher tuning options.
-- [improvement] JAVA-1660: Support netty-transport-native-epoll in OSGi container.
-
-
-### 3.3.1
-
-- [bug] JAVA-1555: Include VIEW and CDC in WriteType.
-- [bug] JAVA-1599: exportAsString improvements (sort, format, clustering order)
-- [improvement] JAVA-1587: Deterministic ordering of columns used in Mapper#saveQuery
-- [improvement] JAVA-1500: Add a metric to report number of in-flight requests.
-- [bug] JAVA-1438: QueryBuilder check for empty orderings.
-- [improvement] JAVA-1490: Allow zero delay for speculative executions.
-- [documentation] JAVA-1607: Add FAQ entry for netty-transport-native-epoll.
-- [bug] JAVA-1630: Fix Metadata.addIfAbsent.
-- [improvement] JAVA-1619: Update QueryBuilder methods to support Iterable input.
-- [improvement] JAVA-1527: Expose host_id and schema_version on Host metadata.
-- [new feature] JAVA-1377: Add support for TWCS in SchemaBuilder.
-- [improvement] JAVA-1631: Publish a sources jar for driver-core-tests.
-- [improvement] JAVA-1632: Add a withIpPrefix(String) method to CCMBridge.Builder.
-- [bug] JAVA-1639: VersionNumber does not fullfill equals/hashcode contract.
-- [bug] JAVA-1613: Fix broken shaded Netty detection in NettyUtil.
-
-
-### 3.3.0
-
-- [bug] JAVA-1469: Update LoggingRetryPolicy to deal with SLF4J-353.
-- [improvement] JAVA-1203: Upgrade Metrics to allow usage in OSGi.
-- [bug] JAVA-1407: KeyspaceMetadata exportAsString should export user types in topological sort order.
-- [bug] JAVA-1455: Mapper support using unset for null values.
-- [bug] JAVA-1464: Allow custom codecs with non public constructors in @Param.
-- [bug] JAVA-1470: Querying multiple pages overrides WrappedStatement.
-- [improvement] JAVA-1428: Upgrade logback and jackson dependencies.
-- [documentation] JAVA-1463: Revisit speculative execution docs.
-- [documentation] JAVA-1466: Revisit timestamp docs.
-- [documentation] JAVA-1445: Clarify how nodes are penalized in LatencyAwarePolicy docs.
-- [improvement] JAVA-1446: Support 'DEFAULT UNSET' in Query Builder JSON Insert.
-- [improvement] JAVA-1443: Add groupBy method to Select statement.
-- [improvement] JAVA-1458: Check thread in mapper sync methods.
-- [improvement] JAVA-1488: Upgrade Netty to 4.0.47.Final.
-- [improvement] JAVA-1460: Add speculative execution number to ExecutionInfo
-- [improvement] JAVA-1431: Improve error handling during pool initialization.
-
-
-### 3.2.0
-
-- [new feature] JAVA-1347: Add support for duration type.
-- [new feature] JAVA-1248: Implement "beta" flag for native protocol v5.
-- [new feature] JAVA-1362: Send query options flags as [int] for Protocol V5+.
-- [new feature] JAVA-1364: Enable creation of SSLHandler with remote address information.
-- [improvement] JAVA-1367: Make protocol negotiation more resilient.
-- [bug] JAVA-1397: Handle duration as native datatype in protocol v5+.
-- [improvement] JAVA-1308: CodecRegistry performance improvements.
-- [improvement] JAVA-1287: Add CDC to TableOptionsMetadata and Schema Builder.
-- [improvement] JAVA-1392: Reduce lock contention in RPTokenFactory.
-- [improvement] JAVA-1328: Provide compatibility with Guava 20.
-- [improvement] JAVA-1247: Disable idempotence warnings.
-- [improvement] JAVA-1286: Support setting and retrieving udt fields in QueryBuilder.
-- [bug] JAVA-1415: Correctly report if a UDT column is frozen.
-- [bug] JAVA-1418: Make Guava version detection more reliable.
-- [new feature] JAVA-1174: Add ifNotExists option to mapper.
-- [improvement] JAVA-1414: Optimize Metadata.escapeId and Metadata.handleId.
-- [improvement] JAVA-1310: Make mapper's ignored properties configurable.
-- [improvement] JAVA-1316: Add strategy for resolving properties into CQL names.
-- [bug] JAVA-1424: Handle new WRITE_FAILURE and READ_FAILURE format in v5 protocol.
-
-Merged from 3.1.x branch:
-
-- [bug] JAVA-1371: Reintroduce connection pool timeout.
-- [bug] JAVA-1313: Copy SerialConsistencyLevel to PreparedStatement.
-- [documentation] JAVA-1334: Clarify documentation of method `addContactPoints`.
-- [improvement] JAVA-1357: Document that getReplicas only returns replicas of the last token in range.
-- [bug] JAVA-1404: Fix min token handling in TokenRange.contains.
-- [bug] JAVA-1429: Prevent heartbeats until connection is fully initialized.
-
-
-### 3.1.4
-
-Merged from 3.0.x branch:
-
-- [bug] JAVA-1371: Reintroduce connection pool timeout.
-- [bug] JAVA-1313: Copy SerialConsistencyLevel to PreparedStatement.
-- [documentation] JAVA-1334: Clarify documentation of method `addContactPoints`.
-- [improvement] JAVA-1357: Document that getReplicas only returns replicas of the last token in range.
-
-
-### 3.1.3
-
-Merged from 3.0.x branch:
-
-- [bug] JAVA-1330: Add un/register for SchemaChangeListener in DelegatingCluster
-- [bug] JAVA-1351: Include Custom Payload in Request.copy.
-- [bug] JAVA-1346: Reset heartbeat only on client reads (not writes).
-- [improvement] JAVA-866: Support tuple notation in QueryBuilder.eq/in.
-
-
-### 3.1.2
-
-- [bug] JAVA-1321: Wrong OSGi dependency version for Guava.
-
-Merged from 3.0.x branch:
-
-- [bug] JAVA-1312: QueryBuilder modifies selected columns when manually selected.
-- [improvement] JAVA-1303: Add missing BoundStatement.setRoutingKey(ByteBuffer...)
-- [improvement] JAVA-262: Make internal executors customizable
-
-
-### 3.1.1
-
-- [bug] JAVA-1284: ClockFactory should check system property before attempting to load Native class.
-- [bug] JAVA-1255: Allow nested UDTs to be used in Mapper.
-- [bug] JAVA-1279: Mapper should exclude Groovy's "metaClass" property when looking for mapped properties
-
-Merged from 3.0.x branch:
-
-- [improvement] JAVA-1246: Driver swallows the real exception in a few cases
-- [improvement] JAVA-1261: Throw error when attempting to page in I/O thread.
-- [bug] JAVA-1258: Regression: Mapper cannot map a materialized view after JAVA-1126.
-- [bug] JAVA-1101: Batch and BatchStatement should consider inner statements to determine query idempotence
-- [improvement] JAVA-1262: Use ParseUtils for quoting & unquoting.
-- [improvement] JAVA-1275: Use Netty's default thread factory
-- [bug] JAVA-1285: QueryBuilder routing key auto-discovery should handle case-sensitive column names.
-- [bug] JAVA-1283: Don't cache failed query preparations in the mapper.
-- [improvement] JAVA-1277: Expose AbstractSession.checkNotInEventLoop.
-- [bug] JAVA-1272: BuiltStatement not able to print its query string if it contains mapped UDTs.
-- [bug] JAVA-1292: 'Adjusted frame length' error breaks driver's ability to read data.
-- [improvement] JAVA-1293: Make DecoderForStreamIdSize.MAX_FRAME_LENGTH configurable.
-- [improvement] JAVA-1053: Add a metric for authentication errors
-- [improvement] JAVA-1263: Eliminate unnecessary memory copies in FrameCompressor implementations.
-- [improvement] JAVA-893: Make connection pool non-blocking
-
-
-### 3.1.0
-
-- [new feature] JAVA-1153: Add PER PARTITION LIMIT to Select QueryBuilder.
-- [improvement] JAVA-743: Add JSON support to QueryBuilder.
-- [improvement] JAVA-1233: Update HdrHistogram to 2.1.9.
-- [improvement] JAVA-1233: Update Snappy to 1.1.2.6.
-- [bug] JAVA-1161: Preserve full time zone info in ZonedDateTimeCodec and DateTimeCodec.
-- [new feature] JAVA-1157: Allow asynchronous paging of Mapper Result.
-- [improvement] JAVA-1212: Don't retry non-idempotent statements by default.
-- [improvement] JAVA-1192: Make EventDebouncer settings updatable at runtime.
-- [new feature] JAVA-541: Add polymorphism support to object mapper.
-- [new feature] JAVA-636: Allow @Column annotations on getters/setters as well as fields.
-- [new feature] JAVA-984: Allow non-void setters in object mapping.
-- [new feature] JAVA-1055: Add ErrorAware load balancing policy.
-
-Merged from 3.0.x branch:
-
-- [bug] JAVA-1179: Request objects should be copied when executed.
-- [improvement] JAVA-1182: Throw error when synchronous call made on I/O thread.
-- [bug] JAVA-1184: Unwrap StatementWrappers when extracting column definitions.
-- [bug] JAVA-1132: Executing bound statement with no variables results in exception with protocol v1.
-- [improvement] JAVA-1040: SimpleStatement parameters support in QueryLogger.
-- [improvement] JAVA-1151: Fail fast if HdrHistogram is not in the classpath.
-- [improvement] JAVA-1154: Allow individual Statement to cancel the read timeout.
-- [bug] JAVA-1074: Fix documentation around default timestamp generator.
-- [improvement] JAVA-1109: Document SSLOptions changes in upgrade guide.
-- [improvement] JAVA-1065: Add method to create token from partition key values.
-- [improvement] JAVA-1136: Enable JDK signature check in module driver-extras.
-- [improvement] JAVA-866: Support tuple notation in QueryBuilder.eq/in.
-- [bug] JAVA-1140: Use same connection to check for schema agreement after a DDL query.
-- [improvement] JAVA-1113: Support Cassandra 3.4 LIKE operator in QueryBuilder.
-- [improvement] JAVA-1086: Support Cassandra 3.2 CAST function in QueryBuilder.
-- [bug] JAVA-1095: Check protocol version for custom payload before sending the query.
-- [improvement] JAVA-1133: Add OSGi headers to cassandra-driver-extras.
-- [bug] JAVA-1137: Incorrect string returned by DataType.asFunctionParameterString() for collections and tuples.
-- [bug] JAVA-1046: (Dynamic)CompositeTypes need to be parsed as string literal, not blob.
-- [improvement] JAVA-1164: Clarify documentation on Host.listenAddress and broadcastAddress.
-- [improvement] JAVA-1171: Add Host method to determine if DSE Graph is enabled.
-- [improvement] JAVA-1069: Bootstrap driver-examples module.
-- [documentation] JAVA-1150: Add example and FAQ entry about ByteBuffer/BLOB.
-- [improvement] JAVA-1011: Expose PoolingOptions default values.
-- [improvement] JAVA-630: Don't process DOWN events for nodes that have active connections.
-- [improvement] JAVA-851: Improve UUIDs javadoc with regard to user-provided timestamps.
-- [improvement] JAVA-979: Update javadoc for RegularStatement toString() and getQueryString() to indicate that consistency level and other parameters are not maintained in the query string.
-- [bug] JAVA-1068: Unwrap StatementWrappers when hashing the paging state.
-- [improvement] JAVA-1021: Improve error message when connect() is called with an invalid keyspace name.
-- [improvement] JAVA-879: Mapper.map() accepts mapper-generated and user queries.
-- [bug] JAVA-1100: Exception when connecting with shaded java driver in OSGI
-- [bug] JAVA-1064: getTable create statement doesn't properly handle quotes in primary key.
-- [bug] JAVA-1089: Set LWT made from BuiltStatements to non-idempotent.
-- [improvement] JAVA-923: Position idempotent flag on object mapper queries.
-- [bug] JAVA-1070: The Mapper should not prepare queries synchronously.
-- [new feature] JAVA-982: Introduce new method ConsistencyLevel.isSerial().
-- [bug] JAVA-764: Retry with the normal consistency level (not the serial one) when a write times out on the Paxos phase.
-- [improvement] JAVA-852: Ignore peers with null entries during discovery.
-- [bug] JAVA-1005: DowngradingConsistencyRetryPolicy does not work with EACH_QUORUM when 1 DC is down.
-- [bug] JAVA-1002: Avoid deadlock when re-preparing a statement on other hosts.
-- [bug] JAVA-1072: Ensure defunct connections are properly evicted from the pool.
-- [bug] JAVA-1152: Fix NPE at ControlConnection.refreshNodeListAndTokenMap().
-
-Merged from 2.1 branch:
-
-- [improvement] JAVA-1038: Fetch node info by rpc_address if its broadcast_address is not in system.peers.
-- [improvement] JAVA-888: Add cluster-wide percentile tracker.
-- [improvement] JAVA-963: Automatically register PercentileTracker from components that use it.
-- [new feature] JAVA-1019: SchemaBuilder support for CREATE/ALTER/DROP KEYSPACE.
-- [bug] JAVA-727: Allow monotonic timestamp generators to drift in the future + use microsecond precision when possible.
-- [improvement] JAVA-444: Add Java process information to UUIDs.makeNode() hash.
-
-
-### 3.0.7
-
-- [bug] JAVA-1371: Reintroduce connection pool timeout.
-- [bug] JAVA-1313: Copy SerialConsistencyLevel to PreparedStatement.
-- [documentation] JAVA-1334: Clarify documentation of method `addContactPoints`.
-- [improvement] JAVA-1357: Document that getReplicas only returns replicas of the last token in range.
-
-
-### 3.0.6
-
-- [bug] JAVA-1330: Add un/register for SchemaChangeListener in DelegatingCluster
-- [bug] JAVA-1351: Include Custom Payload in Request.copy.
-- [bug] JAVA-1346: Reset heartbeat only on client reads (not writes).
-- [improvement] JAVA-866: Support tuple notation in QueryBuilder.eq/in.
-
-
-### 3.0.5
-
-- [bug] JAVA-1312: QueryBuilder modifies selected columns when manually selected.
-- [improvement] JAVA-1303: Add missing BoundStatement.setRoutingKey(ByteBuffer...)
-- [improvement] JAVA-262: Make internal executors customizable
-- [bug] JAVA-1320: prevent unnecessary task creation on empty pool
-
-
-### 3.0.4
-
-- [improvement] JAVA-1246: Driver swallows the real exception in a few cases
-- [improvement] JAVA-1261: Throw error when attempting to page in I/O thread.
-- [bug] JAVA-1258: Regression: Mapper cannot map a materialized view after JAVA-1126.
-- [bug] JAVA-1101: Batch and BatchStatement should consider inner statements to determine query idempotence
-- [improvement] JAVA-1262: Use ParseUtils for quoting & unquoting.
-- [improvement] JAVA-1275: Use Netty's default thread factory
-- [bug] JAVA-1285: QueryBuilder routing key auto-discovery should handle case-sensitive column names.
-- [bug] JAVA-1283: Don't cache failed query preparations in the mapper.
-- [improvement] JAVA-1277: Expose AbstractSession.checkNotInEventLoop.
-- [bug] JAVA-1272: BuiltStatement not able to print its query string if it contains mapped UDTs.
-- [bug] JAVA-1292: 'Adjusted frame length' error breaks driver's ability to read data.
-- [improvement] JAVA-1293: Make DecoderForStreamIdSize.MAX_FRAME_LENGTH configurable.
-- [improvement] JAVA-1053: Add a metric for authentication errors
-- [improvement] JAVA-1263: Eliminate unnecessary memory copies in FrameCompressor implementations.
-- [improvement] JAVA-893: Make connection pool non-blocking
-
-
-### 3.0.3
-
-- [improvement] JAVA-1147: Upgrade Netty to 4.0.37.
-- [bug] JAVA-1213: Allow updates and inserts to BLOB column using read-only ByteBuffer.
-- [bug] JAVA-1209: ProtocolOptions.getProtocolVersion() should return null instead of throwing NPE if Cluster has not
-        been init'd.
-- [improvement] JAVA-1204: Update documentation to indicate tcnative version requirement.
-- [bug] JAVA-1186: Fix duplicated hosts in DCAwarePolicy warn message.
-- [bug] JAVA-1187: Fix warning message when local CL used with RoundRobinPolicy.
-- [improvement] JAVA-1175: Warn if DCAwarePolicy configuration is inconsistent.
-- [bug] JAVA-1139: ConnectionException.getMessage() throws NPE if address is null.
-- [bug] JAVA-1202: Handle null rpc_address when checking schema agreement.
-- [improvement] JAVA-1198: Document that BoundStatement is not thread-safe.
-- [improvement] JAVA-1200: Upgrade LZ4 to 1.3.0.
-- [bug] JAVA-1232: Fix NPE in IdempotenceAwareRetryPolicy.isIdempotent.
-- [improvement] JAVA-1227: Document "SELECT *" issue with prepared statement.
-- [bug] JAVA-1160: Fix NPE in VersionNumber.getPreReleaseLabels().
-- [improvement] JAVA-1126: Handle schema changes in Mapper.
-- [bug] JAVA-1193: Refresh token and replica metadata synchronously when schema is altered.
-- [bug] JAVA-1120: Skip schema refresh debouncer when checking for agreement as a result of schema change made by client.
-- [improvement] JAVA-1242: Fix driver-core dependency in driver-stress
-- [improvement] JAVA-1235: Move the query to the end of "re-preparing .." log message as a key value.
-
-
-### 3.0.2
-
-Merged from 2.1 branch:
-
-- [bug] JAVA-1179: Request objects should be copied when executed.
-- [improvement] JAVA-1182: Throw error when synchronous call made on I/O thread.
-- [bug] JAVA-1184: Unwrap StatementWrappers when extracting column definitions.
-
-
-### 3.0.1
-
-- [bug] JAVA-1132: Executing bound statement with no variables results in exception with protocol v1.
-- [improvement] JAVA-1040: SimpleStatement parameters support in QueryLogger.
-- [improvement] JAVA-1151: Fail fast if HdrHistogram is not in the classpath.
-- [improvement] JAVA-1154: Allow individual Statement to cancel the read timeout.
-- [bug] JAVA-1074: Fix documentation around default timestamp generator.
-- [improvement] JAVA-1109: Document SSLOptions changes in upgrade guide.
-- [improvement] JAVA-1065: Add method to create token from partition key values.
-- [improvement] JAVA-1136: Enable JDK signature check in module driver-extras.
-- [improvement] JAVA-866: Support tuple notation in QueryBuilder.eq/in.
-- [bug] JAVA-1140: Use same connection to check for schema agreement after a DDL query.
-- [improvement] JAVA-1113: Support Cassandra 3.4 LIKE operator in QueryBuilder.
-- [improvement] JAVA-1086: Support Cassandra 3.2 CAST function in QueryBuilder.
-- [bug] JAVA-1095: Check protocol version for custom payload before sending the query.
-- [improvement] JAVA-1133: Add OSGi headers to cassandra-driver-extras.
-- [bug] JAVA-1137: Incorrect string returned by DataType.asFunctionParameterString() for collections and tuples.
-- [bug] JAVA-1046: (Dynamic)CompositeTypes need to be parsed as string literal, not blob.
-- [improvement] JAVA-1164: Clarify documentation on Host.listenAddress and broadcastAddress.
-- [improvement] JAVA-1171: Add Host method to determine if DSE Graph is enabled.
-- [improvement] JAVA-1069: Bootstrap driver-examples module.
-- [documentation] JAVA-1150: Add example and FAQ entry about ByteBuffer/BLOB.
-
-Merged from 2.1 branch:
-
-- [improvement] JAVA-1011: Expose PoolingOptions default values.
-- [improvement] JAVA-630: Don't process DOWN events for nodes that have active connections.
-- [improvement] JAVA-851: Improve UUIDs javadoc with regard to user-provided timestamps.
-- [improvement] JAVA-979: Update javadoc for RegularStatement toString() and getQueryString() to indicate that consistency level and other parameters are not maintained in the query string.
-- [bug] JAVA-1068: Unwrap StatementWrappers when hashing the paging state.
-- [improvement] JAVA-1021: Improve error message when connect() is called with an invalid keyspace name.
-- [improvement] JAVA-879: Mapper.map() accepts mapper-generated and user queries.
-- [bug] JAVA-1100: Exception when connecting with shaded java driver in OSGI
-- [bug] JAVA-1064: getTable create statement doesn't properly handle quotes in primary key.
-- [bug] JAVA-1089: Set LWT made from BuiltStatements to non-idempotent.
-- [improvement] JAVA-923: Position idempotent flag on object mapper queries.
-- [bug] JAVA-1070: The Mapper should not prepare queries synchronously.
-- [new feature] JAVA-982: Introduce new method ConsistencyLevel.isSerial().
-- [bug] JAVA-764: Retry with the normal consistency level (not the serial one) when a write times out on the Paxos phase.
-- [improvement] JAVA-852: Ignore peers with null entries during discovery.
-- [bug] JAVA-1005: DowngradingConsistencyRetryPolicy does not work with EACH_QUORUM when 1 DC is down.
-- [bug] JAVA-1002: Avoid deadlock when re-preparing a statement on other hosts.
-- [bug] JAVA-1072: Ensure defunct connections are properly evicted from the pool.
-- [bug] JAVA-1152: Fix NPE at ControlConnection.refreshNodeListAndTokenMap().
-
-
-### 3.0.0
-
-- [bug] JAVA-1034: fix metadata parser for collections of custom types.
-- [improvement] JAVA-1035: Expose host broadcast_address and listen_address if available.
-- [new feature] JAVA-1037: Allow named parameters in simple statements.
-- [improvement] JAVA-1033: Allow per-statement read timeout.
-- [improvement] JAVA-1042: Include DSE version and workload in Host data.
-
-Merged from 2.1 branch:
-
-- [improvement] JAVA-1030: Log token to replica map computation times.
-- [bug] JAVA-1039: Minor bugs in Event Debouncer.
-
-
-### 3.0.0-rc1
-
-- [bug] JAVA-890: fix mapper for case-sensitive UDT.
-
-
-### 3.0.0-beta1
-
-- [bug] JAVA-993: Support for "custom" types after CASSANDRA-10365.
-- [bug] JAVA-999: Handle unset parameters in QueryLogger.
-- [bug] JAVA-998: SchemaChangeListener not invoked for Functions or Aggregates having UDT arguments.
-- [bug] JAVA-1009: use CL ONE to compute query plan when reconnecting
-  control connection.
-- [improvement] JAVA-1003: Change default consistency level to LOCAL_ONE (amends JAVA-926).
-- [improvement] JAVA-863: Idempotence propagation in prepared statements.
-- [improvement] JAVA-996: Make CodecRegistry available to ProtocolDecoder.
-- [bug] JAVA-819: Driver shouldn't retry on client timeout if statement is not idempotent.
-- [improvement] JAVA-1007: Make SimpleStatement and QueryBuilder "detached" again.
-
-Merged from 2.1 branch:
-
-- [improvement] JAVA-989: Include keyspace name when invalid replication found when generating token map.
-- [improvement] JAVA-664: Reduce heap consumption for TokenMap.
-- [bug] JAVA-994: Don't call on(Up|Down|Add|Remove) methods if Cluster is closed/closing.
-
-
-### 3.0.0-alpha5
-
-- [improvement] JAVA-958: Make TableOrView.Order visible.
-- [improvement] JAVA-968: Update metrics to the latest version.
-- [improvement] JAVA-965: Improve error handling for when a non-type 1 UUID is given to bind() on a timeuuid column.
-- [improvement] JAVA-885: Pass the authenticator name from the server to the auth provider.
-- [improvement] JAVA-961: Raise an exception when an older version of guava (<16.01) is found.
-- [bug] JAVA-972: TypeCodec.parse() implementations should be case insensitive when checking for keyword NULL.
-- [bug] JAVA-971: Make type codecs invariant.
-- [bug] JAVA-986: Update documentation links to reference 3.0.
-- [improvement] JAVA-841: Refactor SSLOptions API.
-- [improvement] JAVA-948: Don't limit cipher suites by default.
-- [improvement] JAVA-917: Document SSL configuration.
-- [improvement] JAVA-936: Adapt schema metadata parsing logic to new storage format of CQL types in C* 3.0.
-- [new feature] JAVA-846: Provide custom codecs library as an extra module.
-- [new feature] JAVA-742: Codec Support for JSON.
-- [new feature] JAVA-606: Codec support for Java 8.
-- [new feature] JAVA-565: Codec support for Java arrays.
-- [new feature] JAVA-605: Codec support for Java enums.
-- [bug] JAVA-884: Fix UDT mapper to process fields in the correct order.
-
-Merged from 2.1 branch:
-
-- [bug] JAVA-854: avoid early return in Cluster.init when a node doesn't support the protocol version.
-- [bug] JAVA-978: Fix quoting issue that caused Mapper.getTableMetadata() to return null.
-- [improvement] JAVA-920: Downgrade "error creating pool" message to WARN.
-- [bug] JAVA-954: Don't trigger reconnection before initialization complete.
-- [improvement] JAVA-914: Avoid rejected tasks at shutdown.
-- [improvement] JAVA-921: Add SimpleStatement.getValuesCount().
-- [bug] JAVA-901: Move call to connection.release() out of cancelHandler.
-- [bug] JAVA-960: Avoid race in control connection shutdown.
-- [bug] JAVA-656: Fix NPE in ControlConnection.updateLocationInfo.
-- [bug] JAVA-966: Count uninitialized connections in conviction policy.
-- [improvement] JAVA-917: Document SSL configuration.
-- [improvement] JAVA-652: Add DCAwareRoundRobinPolicy builder.
-- [improvement] JAVA-808: Add generic filtering policy that can be used to exclude specific DCs.
-- [bug] JAVA-988: Metadata.handleId should handle escaped double quotes.
-- [bug] JAVA-983: QueryBuilder cannot handle collections containing function calls.
-
-
-### 3.0.0-alpha4
-
-- [improvement] JAVA-926: Change default consistency level to LOCAL_QUORUM.
-- [bug] JAVA-942: Fix implementation of UserType.hashCode().
-- [improvement] JAVA-877: Don't delay UP/ADDED notifications if protocol version = V4.
-- [improvement] JAVA-938: Parse 'extensions' column in table metadata.
-- [bug] JAVA-900: Fix Configuration builder to allow disabled metrics.
-- [new feature] JAVA-902: Prepare API for async query trace.
-- [new feature] JAVA-930: Add BoundStatement#unset.
-- [bug] JAVA-946: Make table metadata options class visible.
-- [bug] JAVA-939: Add crcCheckChance to TableOptionsMetadata#equals/hashCode.
-- [bug] JAVA-922: Make TypeCodec return mutable collections.
-- [improvement] JAVA-932: Limit visibility of codec internals.
-- [improvement] JAVA-934: Warn if a custom codec collides with an existing one.
-- [improvement] JAVA-940: Allow typed getters/setters to target any CQL type.
-- [bug] JAVA-950: Fix Cluster.connect with a case-sensitive keyspace.
-- [bug] JAVA-953: Fix MaterializedViewMetadata when base table name is case sensitive.
-
-
-### 3.0.0-alpha3
-
-- [new feature] JAVA-571: Support new system tables in C* 3.0.
-- [improvement] JAVA-919: Move crc_check_chance out of compressions options.
-
-Merged from 2.0 branch:
-
-- [improvement] JAVA-718: Log streamid at the trace level on sending request and receiving response.
-- [bug] JAVA-796: Fix SpeculativeExecutionPolicy.init() and close() are never called.
-- [improvement] JAVA-710: Suppress unnecessary warning at shutdown.
-- [improvement] #340: Allow DNS name with multiple A-records as contact point.
-- [bug] JAVA-794: Allow tracing across multiple result pages.
-- [bug] JAVA-737: DowngradingConsistencyRetryPolicy ignores write timeouts.
-- [bug] JAVA-736: Forbid bind marker in QueryBuilder add/append/prepend.
-- [bug] JAVA-712: Prevent QueryBuilder.quote() from applying duplicate double quotes.
-- [bug] JAVA-688: Prevent QueryBuilder from trying to serialize raw string.
-- [bug] JAVA-679: Support bind marker in QueryBuilder DELETE's list index.
-- [improvement] JAVA-475: Improve QueryBuilder API for SELECT DISTINCT.
-- [improvement] JAVA-225: Create values() function for Insert builder using List.
-- [improvement] JAVA-702: Warn when ReplicationStrategy encounters invalid
-  replication factors.
-- [improvement] JAVA-662: Add PoolingOptions method to set both core and max
-  connections.
-- [improvement] JAVA-766: Do not include epoll JAR in binary distribution.
-- [improvement] JAVA-726: Optimize internal copies of Request objects.
-- [bug] JAVA-815: Preserve tracing across retries.
-- [improvement] JAVA-709: New RetryDecision.tryNextHost().
-- [bug] JAVA-733: Handle function calls and raw strings as non-idempotent in QueryBuilder.
-- [improvement] JAVA-765: Provide API to retrieve values of a Parameterized SimpleStatement.
-- [improvement] JAVA-827: implement UPDATE .. IF EXISTS in QueryBuilder.
-- [improvement] JAVA-618: Randomize contact points list to prevent hotspots.
-- [improvement] JAVA-720: Surface the coordinator used on query failure.
-- [bug] JAVA-792: Handle contact points removed during init.
-- [improvement] JAVA-719: Allow PlainTextAuthProvider to change its credentials at runtime.
-- [new feature] JAVA-151: Make it possible to register for SchemaChange Events.
-- [improvement] JAVA-861: Downgrade "Asked to rebuild table" log from ERROR to INFO level.
-- [improvement] JAVA-797: Provide an option to prepare statements only on one node.
-- [improvement] JAVA-658: Provide an option to not re-prepare all statements in onUp.
-- [improvement] JAVA-853: Customizable creation of netty timer.
-- [bug] JAVA-859: Avoid quadratic ring processing with invalid replication factors.
-- [improvement] JAVA-657: Debounce control connection queries.
-- [bug] JAVA-784: LoadBalancingPolicy.distance() called before init().
-- [new feature] JAVA-828: Make driver-side metadata optional.
-- [improvement] JAVA-544: Allow hosts to remain partially up.
-- [improvement] JAVA-821, JAVA-822: Remove internal blocking calls and expose async session
-  creation.
-- [improvement] JAVA-725: Use parallel calls when re-preparing statement on other
-  hosts.
-- [bug] JAVA-629: Don't use connection timeout for unrelated internal queries.
-- [bug] JAVA-892: Fix NPE in speculative executions when metrics disabled.
-
-
-### 3.0.0-alpha2
-
-- [new feature] JAVA-875, JAVA-882: Move secondary index metadata out of column definitions.
-
-Merged from 2.2 branch:
-
-- [bug] JAVA-847: Propagate CodecRegistry to nested UDTs.
-- [improvement] JAVA-848: Ability to store a default, shareable CodecRegistry
-  instance.
-- [bug] JAVA-880: Treat empty ByteBuffers as empty values in TupleCodec and
-  UDTCodec.
-
-
-### 3.0.0-alpha1
-
-- [new feature] JAVA-876: Support new system tables in C* 3.0.0-alpha1.
-
-Merged from 2.2 branch:
-
-- [improvement] JAVA-810: Rename DateWithoutTime to LocalDate.
-- [bug] JAVA-816: DateCodec does not format values correctly.
-- [bug] JAVA-817: TimeCodec does not format values correctly.
-- [bug] JAVA-818: TypeCodec.getDataTypeFor() does not handle LocalDate instances.
-- [improvement] JAVA-836: Make ResultSet#fetchMoreResult return a
-  ListenableFuture.
-- [improvement] JAVA-843: Disable frozen checks in mapper.
-- [improvement] JAVA-721: Allow user to register custom type codecs.
-- [improvement] JAVA-722: Support custom type codecs in mapper.
-
-
-### 2.2.0-rc3
-
-- [bug] JAVA-847: Propagate CodecRegistry to nested UDTs.
-- [improvement] JAVA-848: Ability to store a default, shareable CodecRegistry
-  instance.
-- [bug] JAVA-880: Treat empty ByteBuffers as empty values in TupleCodec and
-  UDTCodec.
-
-
-### 2.2.0-rc2
-
-- [improvement] JAVA-810: Rename DateWithoutTime to LocalDate.
-- [bug] JAVA-816: DateCodec does not format values correctly.
-- [bug] JAVA-817: TimeCodec does not format values correctly.
-- [bug] JAVA-818: TypeCodec.getDataTypeFor() does not handle LocalDate instances.
-- [improvement] JAVA-836: Make ResultSet#fetchMoreResult return a
-  ListenableFuture.
-- [improvement] JAVA-843: Disable frozen checks in mapper.
-- [improvement] JAVA-721: Allow user to register custom type codecs.
-- [improvement] JAVA-722: Support custom type codecs in mapper.
-
-Merged from 2.1 branch:
-
-- [bug] JAVA-834: Special case check for 'null' string in index_options column.
-- [improvement] JAVA-835: Allow accessor methods with less parameters in case
-  named bind markers are repeated.
-- [improvement] JAVA-475: Improve QueryBuilder API for SELECT DISTINCT.
-- [improvement] JAVA-715: Make NativeColumnType a top-level class.
-- [improvement] JAVA-700: Expose ProtocolVersion#toInt.
-- [bug] JAVA-542: Handle void return types in accessors.
-- [improvement] JAVA-225: Create values() function for Insert builder using List.
-- [improvement] JAVA-713: HashMap throws an OOM Exception when logging level is set to TRACE.
-- [bug] JAVA-679: Support bind marker in QueryBuilder DELETE's list index.
-- [improvement] JAVA-732: Expose KEYS and FULL indexing options in IndexMetadata.
-- [improvement] JAVA-589: Allow @Enumerated in Accessor method parameters.
-- [improvement] JAVA-554: Allow access to table metadata from Mapper.
-- [improvement] JAVA-661: Provide a way to map computed fields.
-- [improvement] JAVA-824: Ignore missing columns in mapper.
-- [bug] JAVA-724: Preserve default timestamp for retries and speculative executions.
-- [improvement] JAVA-738: Use same pool implementation for protocol v2 and v3.
-- [improvement] JAVA-677: Support CONTAINS / CONTAINS KEY in QueryBuilder.
-- [improvement] JAVA-477/JAVA-540: Add USING options in mapper for delete and save
-  operations.
-- [improvement] JAVA-473: Add mapper option to configure whether to save null fields.
-
-Merged from 2.0 branch:
-
-- [bug] JAVA-737: DowngradingConsistencyRetryPolicy ignores write timeouts.
-- [bug] JAVA-736: Forbid bind marker in QueryBuilder add/append/prepend.
-- [bug] JAVA-712: Prevent QueryBuilder.quote() from applying duplicate double quotes.
-- [bug] JAVA-688: Prevent QueryBuilder from trying to serialize raw string.
-- [bug] JAVA-679: Support bind marker in QueryBuilder DELETE's list index.
-- [improvement] JAVA-475: Improve QueryBuilder API for SELECT DISTINCT.
-- [improvement] JAVA-225: Create values() function for Insert builder using List.
-- [improvement] JAVA-702: Warn when ReplicationStrategy encounters invalid
-  replication factors.
-- [improvement] JAVA-662: Add PoolingOptions method to set both core and max
-  connections.
-- [improvement] JAVA-766: Do not include epoll JAR in binary distribution.
-- [improvement] JAVA-726: Optimize internal copies of Request objects.
-- [bug] JAVA-815: Preserve tracing across retries.
-- [improvement] JAVA-709: New RetryDecision.tryNextHost().
-- [bug] JAVA-733: Handle function calls and raw strings as non-idempotent in QueryBuilder.
-
-
-### 2.2.0-rc1
-
-- [new feature] JAVA-783: Protocol V4 enum support.
-- [new feature] JAVA-776: Use PK columns in protocol v4 PREPARED response.
-- [new feature] JAVA-777: Distinguish NULL and UNSET values.
-- [new feature] JAVA-779: Add k/v payload for 3rd party usage.
-- [new feature] JAVA-780: Expose server-side warnings on ExecutionInfo.
-- [new feature] JAVA-749: Expose new read/write failure exceptions.
-- [new feature] JAVA-747: Expose function and aggregate metadata.
-- [new feature] JAVA-778: Add new client exception for CQL function failure.
-- [improvement] JAVA-700: Expose ProtocolVersion#toInt.
-- [new feature] JAVA-404: Support new C* 2.2 CQL date and time types.
-
-Merged from 2.1 branch:
-
-- [improvement] JAVA-782: Unify "Target" enum for schema elements.
-
-
-### 2.1.10.2
-
-Merged from 2.0 branch:
-
-- [bug] JAVA-1179: Request objects should be copied when executed.
-- [improvement] JAVA-1182: Throw error when synchronous call made on I/O thread.
-- [bug] JAVA-1184: Unwrap StatementWrappers when extracting column definitions.
-
-
-### 2.1.10.1
-
-- [bug] JAVA-1152: Fix NPE at ControlConnection.refreshNodeListAndTokenMap().
-- [bug] JAVA-1156: Fix NPE at TableMetadata.equals().
-
-
-### 2.1.10
-
-- [bug] JAVA-988: Metadata.handleId should handle escaped double quotes.
-- [bug] JAVA-983: QueryBuilder cannot handle collections containing function calls.
-- [improvement] JAVA-863: Idempotence propagation in PreparedStatements.
-- [bug] JAVA-937: TypeCodec static initializers not always correctly executed.
-- [improvement] JAVA-989: Include keyspace name when invalid replication found when generating token map.
-- [improvement] JAVA-664: Reduce heap consumption for TokenMap.
-- [improvement] JAVA-1030: Log token to replica map computation times.
-- [bug] JAVA-1039: Minor bugs in Event Debouncer.
-- [improvement] JAVA-843: Disable frozen checks in mapper.
-- [improvement] JAVA-833: Improve message when a nested type can't be serialized.
-- [improvement] JAVA-1011: Expose PoolingOptions default values.
-- [improvement] JAVA-630: Don't process DOWN events for nodes that have active connections.
-- [improvement] JAVA-851: Improve UUIDs javadoc with regard to user-provided timestamps.
-- [improvement] JAVA-979: Update javadoc for RegularStatement toString() and getQueryString() to indicate that consistency level and other parameters are not maintained in the query string.
-- [improvement] JAVA-1038: Fetch node info by rpc_address if its broadcast_address is not in system.peers.
-- [improvement] JAVA-974: Validate accessor parameter types against bound statement.
-- [bug] JAVA-1068: Unwrap StatementWrappers when hashing the paging state.
-- [bug] JAVA-831: Mapper can't load an entity where the PK is a UDT.
-- [improvement] JAVA-1021: Improve error message when connect() is called with an invalid keyspace name.
-- [improvement] JAVA-879: Mapper.map() accepts mapper-generated and user queries.
-- [bug] JAVA-1100: Exception when connecting with shaded java driver in OSGI
-- [bug] JAVA-819: Expose more errors in RetryPolicy + provide idempotent-aware wrapper.
-- [improvement] JAVA-1040: SimpleStatement parameters support in QueryLogger.
-- [bug] JAVA-1064: getTable create statement doesn't properly handle quotes in primary key.
-- [improvement] JAVA-888: Add cluster-wide percentile tracker.
-- [improvement] JAVA-963: Automatically register PercentileTracker from components that use it.
-- [bug] JAVA-1089: Set LWT made from BuiltStatements to non-idempotent.
-- [improvement] JAVA-923: Position idempotent flag on object mapper queries.
-- [new feature] JAVA-1019: SchemaBuilder support for CREATE/ALTER/DROP KEYSPACE.
-- [bug] JAVA-1070: The Mapper should not prepare queries synchronously.
-- [new feature] JAVA-982: Introduce new method ConsistencyLevel.isSerial().
-- [bug] JAVA-764: Retry with the normal consistency level (not the serial one) when a write times out on the Paxos phase.
-- [bug] JAVA-727: Allow monotonic timestamp generators to drift in the future + use microsecond precision when possible.
-- [improvement] JAVA-444: Add Java process information to UUIDs.makeNode() hash.
-- [improvement] JAVA-977: Preserve original cause when BuiltStatement value can't be serialized.
-- [bug] JAVA-1094: Backport TypeCodec parse and format fixes from 3.0.
-- [improvement] JAVA-852: Ignore peers with null entries during discovery.
-- [bug] JAVA-1132: Executing bound statement with no variables results in exception with protocol v1.
-- [bug] JAVA-1005: DowngradingConsistencyRetryPolicy does not work with EACH_QUORUM when 1 DC is down.
-- [bug] JAVA-1002: Avoid deadlock when re-preparing a statement on other hosts.
-
-Merged from 2.0 branch:
-
-- [bug] JAVA-994: Don't call on(Up|Down|Add|Remove) methods if Cluster is closed/closing.
-- [improvement] JAVA-805: Document that metrics are null until Cluster is initialized.
-- [bug] JAVA-1072: Ensure defunct connections are properly evicted from the pool.
-
-
-### 2.1.9
-
-- [bug] JAVA-942: Fix implementation of UserType.hashCode().
-- [bug] JAVA-854: avoid early return in Cluster.init when a node doesn't support the protocol version.
-- [bug] JAVA-978: Fix quoting issue that caused Mapper.getTableMetadata() to return null.
-
-Merged from 2.0 branch:
-
-- [bug] JAVA-950: Fix Cluster.connect with a case-sensitive keyspace.
-- [improvement] JAVA-920: Downgrade "error creating pool" message to WARN.
-- [bug] JAVA-954: Don't trigger reconnection before initialization complete.
-- [improvement] JAVA-914: Avoid rejected tasks at shutdown.
-- [improvement] JAVA-921: Add SimpleStatement.getValuesCount().
-- [bug] JAVA-901: Move call to connection.release() out of cancelHandler.
-- [bug] JAVA-960: Avoid race in control connection shutdown.
-- [bug] JAVA-656: Fix NPE in ControlConnection.updateLocationInfo.
-- [bug] JAVA-966: Count uninitialized connections in conviction policy.
-- [improvement] JAVA-917: Document SSL configuration.
-- [improvement] JAVA-652: Add DCAwareRoundRobinPolicy builder.
-- [improvement] JAVA-808: Add generic filtering policy that can be used to exclude specific DCs.
-
-
-### 2.1.8
-
-Merged from 2.0 branch:
-
-- [improvement] JAVA-718: Log streamid at the trace level on sending request and receiving response.
-
-- [bug] JAVA-796: Fix SpeculativeExecutionPolicy.init() and close() are never called.
-- [improvement] JAVA-710: Suppress unnecessary warning at shutdown.
-- [improvement] #340: Allow DNS name with multiple A-records as contact point.
-- [bug] JAVA-794: Allow tracing across multiple result pages.
-- [bug] JAVA-737: DowngradingConsistencyRetryPolicy ignores write timeouts.
-- [bug] JAVA-736: Forbid bind marker in QueryBuilder add/append/prepend.
-- [bug] JAVA-712: Prevent QueryBuilder.quote() from applying duplicate double quotes.
-- [bug] JAVA-688: Prevent QueryBuilder from trying to serialize raw string.
-- [bug] JAVA-679: Support bind marker in QueryBuilder DELETE's list index.
-- [improvement] JAVA-475: Improve QueryBuilder API for SELECT DISTINCT.
-- [improvement] JAVA-225: Create values() function for Insert builder using List.
-- [improvement] JAVA-702: Warn when ReplicationStrategy encounters invalid
-  replication factors.
-- [improvement] JAVA-662: Add PoolingOptions method to set both core and max
-  connections.
-- [improvement] JAVA-766: Do not include epoll JAR in binary distribution.
-- [improvement] JAVA-726: Optimize internal copies of Request objects.
-- [bug] JAVA-815: Preserve tracing across retries.
-- [improvement] JAVA-709: New RetryDecision.tryNextHost().
-- [bug] JAVA-733: Handle function calls and raw strings as non-idempotent in QueryBuilder.
-- [improvement] JAVA-765: Provide API to retrieve values of a Parameterized SimpleStatement.
-- [improvement] JAVA-827: implement UPDATE .. IF EXISTS in QueryBuilder.
-- [improvement] JAVA-618: Randomize contact points list to prevent hotspots.
-- [improvement] JAVA-720: Surface the coordinator used on query failure.
-- [bug] JAVA-792: Handle contact points removed during init.
-- [improvement] JAVA-719: Allow PlainTextAuthProvider to change its credentials at runtime.
-- [new feature] JAVA-151: Make it possible to register for SchemaChange Events.
-- [improvement] JAVA-861: Downgrade "Asked to rebuild table" log from ERROR to INFO level.
-- [improvement] JAVA-797: Provide an option to prepare statements only on one node.
-- [improvement] JAVA-658: Provide an option to not re-prepare all statements in onUp.
-- [improvement] JAVA-853: Customizable creation of netty timer.
-- [bug] JAVA-859: Avoid quadratic ring processing with invalid replication factors.
-- [improvement] JAVA-657: Debounce control connection queries.
-- [bug] JAVA-784: LoadBalancingPolicy.distance() called before init().
-- [new feature] JAVA-828: Make driver-side metadata optional.
-- [improvement] JAVA-544: Allow hosts to remain partially up.
-- [improvement] JAVA-821, JAVA-822: Remove internal blocking calls and expose async session
-  creation.
-- [improvement] JAVA-725: Use parallel calls when re-preparing statement on other
-  hosts.
-- [bug] JAVA-629: Don't use connection timeout for unrelated internal queries.
-- [bug] JAVA-892: Fix NPE in speculative executions when metrics disabled.
-
-
-### 2.1.7.1
-
-- [bug] JAVA-834: Special case check for 'null' string in index_options column.
-- [improvement] JAVA-835: Allow accessor methods with less parameters in case
-  named bind markers are repeated.
-
-
-### 2.1.7
-
-- [improvement] JAVA-475: Improve QueryBuilder API for SELECT DISTINCT.
-- [improvement] JAVA-715: Make NativeColumnType a top-level class.
-- [improvement] JAVA-782: Unify "Target" enum for schema elements.
-- [improvement] JAVA-700: Expose ProtocolVersion#toInt.
-- [bug] JAVA-542: Handle void return types in accessors.
-- [improvement] JAVA-225: Create values() function for Insert builder using List.
-- [improvement] JAVA-713: HashMap throws an OOM Exception when logging level is set to TRACE.
-- [bug] JAVA-679: Support bind marker in QueryBuilder DELETE's list index.
-- [improvement] JAVA-732: Expose KEYS and FULL indexing options in IndexMetadata.
-- [improvement] JAVA-589: Allow @Enumerated in Accessor method parameters.
-- [improvement] JAVA-554: Allow access to table metadata from Mapper.
-- [improvement] JAVA-661: Provide a way to map computed fields.
-- [improvement] JAVA-824: Ignore missing columns in mapper.
-- [bug] JAVA-724: Preserve default timestamp for retries and speculative executions.
-- [improvement] JAVA-738: Use same pool implementation for protocol v2 and v3.
-- [improvement] JAVA-677: Support CONTAINS / CONTAINS KEY in QueryBuilder.
-- [improvement] JAVA-477/JAVA-540: Add USING options in mapper for delete and save
-  operations.
-- [improvement] JAVA-473: Add mapper option to configure whether to save null fields.
-
-Merged from 2.0 branch:
-
-- [bug] JAVA-737: DowngradingConsistencyRetryPolicy ignores write timeouts.
-- [bug] JAVA-736: Forbid bind marker in QueryBuilder add/append/prepend.
-- [bug] JAVA-712: Prevent QueryBuilder.quote() from applying duplicate double quotes.
-- [bug] JAVA-688: Prevent QueryBuilder from trying to serialize raw string.
-- [bug] JAVA-679: Support bind marker in QueryBuilder DELETE's list index.
-- [improvement] JAVA-475: Improve QueryBuilder API for SELECT DISTINCT.
-- [improvement] JAVA-225: Create values() function for Insert builder using List.
-- [improvement] JAVA-702: Warn when ReplicationStrategy encounters invalid
-  replication factors.
-- [improvement] JAVA-662: Add PoolingOptions method to set both core and max
-  connections.
-- [improvement] JAVA-766: Do not include epoll JAR in binary distribution.
-- [improvement] JAVA-726: Optimize internal copies of Request objects.
-- [bug] JAVA-815: Preserve tracing across retries.
-- [improvement] JAVA-709: New RetryDecision.tryNextHost().
-- [bug] JAVA-733: Handle function calls and raw strings as non-idempotent in QueryBuilder.
-
-
-### 2.1.6
-
-Merged from 2.0 branch:
-
-- [new feature] JAVA-584: Add getObject to BoundStatement and Row.
-- [improvement] JAVA-419: Improve connection pool resizing algorithm.
-- [bug] JAVA-599: Fix race condition between pool expansion and shutdown.
-- [improvement] JAVA-622: Upgrade Netty to 4.0.27.
-- [improvement] JAVA-562: Coalesce frames before flushing them to the connection.
-- [improvement] JAVA-583: Rename threads to indicate that they are for the driver.
-- [new feature] JAVA-550: Expose paging state.
-- [new feature] JAVA-646: Slow Query Logger.
-- [improvement] JAVA-698: Exclude some errors from measurements in LatencyAwarePolicy.
-- [bug] JAVA-641: Fix issue when executing a PreparedStatement from another cluster.
-- [improvement] JAVA-534: Log keyspace xxx does not exist at WARN level.
-- [improvement] JAVA-619: Allow Cluster subclasses to delegate to another instance.
-- [new feature] JAVA-669: Expose an API to check for schema agreement after a
-  schema-altering statement.
-- [improvement] JAVA-692: Make connection and pool creation fully async.
-- [improvement] JAVA-505: Optimize connection use after reconnection.
-- [improvement] JAVA-617: Remove "suspected" mechanism.
-- [improvement] reverts JAVA-425: Don't mark connection defunct on client timeout.
-- [new feature] JAVA-561: Speculative query executions.
-- [bug] JAVA-666: Release connection before completing the ResultSetFuture.
-- [new feature BETA] JAVA-723: Percentile-based variant of query logger and speculative
-  executions.
-- [bug] JAVA-734: Fix buffer leaks when compression is enabled.
-- [improvement] JAVA-756: Use Netty's pooled ByteBufAllocator by default.
-- [improvement] JAVA-759: Expose "unsafe" paging state API.
-- [bug] JAVA-768: Prevent race during pool initialization.
-
-
-### 2.1.5
-
-- [bug] JAVA-575: Authorize Null parameter in Accessor method.
-- [improvement] JAVA-570: Support C* 2.1.3's nested collections.
-- [bug] JAVA-612: Fix checks on mapped collection types.
-- [bug] JAVA-672: Fix QueryBuilder.putAll() when the collection contains UDTs.
-
-Merged from 2.0 branch:
-
-- [new feature] JAVA-518: Add AddressTranslater for EC2 multi-region deployment.
-- [improvement] JAVA-533: Add connection heartbeat.
-- [improvement] JAVA-568: Reduce level of logs on missing rpc_address.
-- [improvement] JAVA-312, JAVA-681: Expose node token and range information.
-- [bug] JAVA-595: Fix cluster name mismatch check at startup.
-- [bug] JAVA-620: Fix guava dependency when using OSGI.
-- [bug] JAVA-678: Fix handling of DROP events when ks name is case-sensitive.
-- [improvement] JAVA-631: Use List instead of List in QueryBuilder API.
-- [improvement] JAVA-654: Exclude Netty POM from META-INF in shaded JAR.
-- [bug] JAVA-655: Quote single quotes contained in table comments in asCQLQuery method.
-- [bug] JAVA-684: Empty TokenRange returned in a one token cluster.
-- [improvement] JAVA-687: Expose TokenRange#contains.
-- [bug] JAVA-614: Prevent race between cancellation and query completion.
-- [bug] JAVA-632: Prevent cancel and timeout from cancelling unrelated ResponseHandler if
-  streamId was already released and reused.
-- [bug] JAVA-642: Fix issue when newly opened pool fails before we could mark the node UP.
-- [bug] JAVA-613: Fix unwanted LBP notifications when a contact host is down.
-- [bug] JAVA-651: Fix edge cases where a connection was released twice.
-- [bug] JAVA-653: Fix edge cases in query cancellation.
-
-
-### 2.1.4
-
-Merged from 2.0 branch:
-
-- [improvement] JAVA-538: Shade Netty dependency.
-- [improvement] JAVA-543: Target schema refreshes more precisely.
-- [bug] JAVA-546: Don't check rpc_address for control host.
-- [improvement] JAVA-409: Improve message of NoHostAvailableException.
-- [bug] JAVA-556: Rework connection reaper to avoid deadlock.
-- [bug] JAVA-557: Avoid deadlock when multiple connections to the same host get write
-  errors.
-- [improvement] JAVA-504: Make shuffle=true the default for TokenAwarePolicy.
-- [bug] JAVA-577: Fix bug when SUSPECT reconnection succeeds, but one of the pooled
-  connections fails while bringing the node back up.
-- [bug] JAVA-419: JAVA-587: Prevent faulty control connection from ignoring reconnecting hosts.
-- temporarily revert "Add idle timeout to the connection pool".
-- [bug] JAVA-593: Ensure updateCreatedPools does not add pools for suspected hosts.
-- [bug] JAVA-594: Ensure state change notifications for a given host are handled serially.
-- [bug] JAVA-597: Ensure control connection reconnects when control host is removed.
-
-
-### 2.1.3
-
-- [bug] JAVA-510: Ignore static fields in mapper.
-- [bug] JAVA-509: Fix UDT parsing at init when using the default protocol version.
-- [bug] JAVA-495: Fix toString, equals and hashCode on accessor proxies.
-- [bug] JAVA-528: Allow empty name on Column and Field annotations.
-
-Merged from 2.0 branch:
-
-- [bug] JAVA-497: Ensure control connection does not trigger concurrent reconnects.
-- [improvement] JAVA-472: Keep trying to reconnect on authentication errors.
-- [improvement] JAVA-463: Expose close method on load balancing policy.
-- [improvement] JAVA-459: Allow load balancing policy to trigger refresh for a single host.
-- [bug] JAVA-493: Expose an API to cancel reconnection attempts.
-- [bug] JAVA-503: Fix NPE when a connection fails during pool construction.
-- [improvement] JAVA-423: Log datacenter name in DCAware policy's init when it is explicitly provided.
-- [improvement] JAVA-504: Shuffle the replicas in TokenAwarePolicy.newQueryPlan.
-- [improvement] JAVA-507: Make schema agreement wait tuneable.
-- [improvement] JAVA-494: Document how to inject the driver metrics into another registry.
-- [improvement] JAVA-419: Add idle timeout to the connection pool.
-- [bug] JAVA-516: LatencyAwarePolicy does not shutdown executor on invocation of close.
-- [improvement] JAVA-451: Throw an exception when DCAwareRoundRobinPolicy is built with
-  an explicit but null or empty local datacenter.
-- [bug] JAVA-511: Fix check for local contact points in DCAware policy's init.
-- [improvement] JAVA-457: Make timeout on saturated pool customizable.
-- [improvement] JAVA-521: Downgrade Guava to 14.0.1.
-- [bug] JAVA-526: Fix token awareness for case-sensitive keyspaces and tables.
-- [bug] JAVA-515: Check maximum number of values passed to SimpleStatement.
-- [improvement] JAVA-532: Expose the driver version through the API.
-- [improvement] JAVA-522: Optimize session initialization when some hosts are not
-  responsive.
-
-
-### 2.1.2
-
-- [improvement] JAVA-361, JAVA-364, JAVA-467: Support for native protocol v3.
-- [bug] JAVA-454: Fix UDT fields of type inet in QueryBuilder.
-- [bug] JAVA-455: Exclude transient fields from Frozen checks.
-- [bug] JAVA-453: Fix handling of null collections in mapper.
-- [improvement] JAVA-452: Make implicit column names case-insensitive in mapper.
-- [bug] JAVA-433: Fix named bind markers in QueryBuilder.
-- [bug] JAVA-458: Fix handling of BigInteger in object mapper.
-- [bug] JAVA-465: Ignore synthetic fields in mapper.
-- [improvement] JAVA-451: Throw an exception when DCAwareRoundRobinPolicy is built with
-  an explicit but null or empty local datacenter.
-- [improvement] JAVA-469: Add backwards-compatible DataType.serialize methods.
-- [bug] JAVA-487: Handle null enum fields in object mapper.
-- [bug] JAVA-499: Handle null UDT fields in object mapper.
-
-Merged from 2.0 branch:
-
-- [bug] JAVA-449: Handle null pool in PooledConnection.release.
-- [improvement] JAVA-425: Defunct connection on request timeout.
-- [improvement] JAVA-426: Try next host when we get a SERVER_ERROR.
-- [bug] JAVA-449, JAVA-460, JAVA-471: Handle race between query timeout and completion.
-- [bug] JAVA-496: Fix DCAwareRoundRobinPolicy datacenter auto-discovery.
-
-
-### 2.1.1
-
-- [new] JAVA-441: Support for new "frozen" keyword.
-
-Merged from 2.0 branch:
-
-- [bug] JAVA-397: Check cluster name when connecting to a new node.
-- [bug] JAVA-326: Add missing CAS delete support in QueryBuilder.
-- [bug] JAVA-363: Add collection and data length checks during serialization.
-- [improvement] JAVA-329: Surface number of retries in metrics.
-- [bug] JAVA-428: Do not use a host when no rpc_address found for it.
-- [improvement] JAVA-358: Add ResultSet.wasApplied() for conditional queries.
-- [bug] JAVA-349: Fix negative HostConnectionPool open count.
-- [improvement] JAVA-436: Log more connection details at trace and debug levels.
-- [bug] JAVA-445: Fix cluster shutdown.
-
-
-### 2.1.0
-
-- [bug] JAVA-408: ClusteringColumn annotation not working with specified ordering.
-- [improvement] JAVA-410: Fail BoundStatement if null values are not set explicitly.
-- [bug] JAVA-416: Handle UDT and tuples in BuiltStatement.toString.
-
-Merged from 2.0 branch:
-
-- [bug] JAVA-407: Release connections on ResultSetFuture#cancel.
-- [bug] JAVA-393: Fix handling of SimpleStatement with values in query builder
-  batches.
-- [bug] JAVA-417: Ensure pool is properly closed in onDown.
-- [bug] JAVA-415: Fix tokenMap initialization at startup.
-- [bug] JAVA-418: Avoid deadlock on close.
-
-
-### 2.1.0-rc1
-
-Merged from 2.0 branch:
-
-- [bug] JAVA-394: Ensure defunct connections are completely closed.
-- [bug] JAVA-342, JAVA-390: Fix memory and resource leak on closed Sessions.
-
-
-### 2.1.0-beta1
-
-- [new] Support for User Defined Types and tuples
-- [new] Simple object mapper
-
-Merged from 2.0 branch: everything up to 2.0.3 (included), and the following.
-
-- [improvement] JAVA-204: Better handling of dead connections.
-- [bug] JAVA-373: Fix potential NPE in ControlConnection.
-- [bug] JAVA-291: Throws NPE when passed null for a contact point.
-- [bug] JAVA-315: Avoid LoadBalancingPolicy onDown+onUp at startup.
-- [bug] JAVA-343: Avoid classloader leak in Tomcat.
-- [bug] JAVA-387: Avoid deadlock in onAdd/onUp.
-- [bug] JAVA-377, JAVA-391: Make metadata parsing more lenient.
-
-
-### 2.0.12.2
-
-- [bug] JAVA-1179: Request objects should be copied when executed.
-- [improvement] JAVA-1182: Throw error when synchronous call made on I/O thread.
-- [bug] JAVA-1184: Unwrap StatementWrappers when extracting column definitions.
-
-
-### 2.0.12.1
-
-- [bug] JAVA-994: Don't call on(Up|Down|Add|Remove) methods if Cluster is closed/closing.
-- [improvement] JAVA-805: Document that metrics are null until Cluster is initialized.
-- [bug] JAVA-1072: Ensure defunct connections are properly evicted from the pool.
-
-
-### 2.0.12
-
-- [bug] JAVA-950: Fix Cluster.connect with a case-sensitive keyspace.
-- [improvement] JAVA-920: Downgrade "error creating pool" message to WARN.
-- [bug] JAVA-954: Don't trigger reconnection before initialization complete.
-- [improvement] JAVA-914: Avoid rejected tasks at shutdown.
-- [improvement] JAVA-921: Add SimpleStatement.getValuesCount().
-- [bug] JAVA-901: Move call to connection.release() out of cancelHandler.
-- [bug] JAVA-960: Avoid race in control connection shutdown.
-- [bug] JAVA-656: Fix NPE in ControlConnection.updateLocationInfo.
-- [bug] JAVA-966: Count uninitialized connections in conviction policy.
-- [improvement] JAVA-917: Document SSL configuration.
-- [improvement] JAVA-652: Add DCAwareRoundRobinPolicy builder.
-- [improvement] JAVA-808: Add generic filtering policy that can be used to exclude specific DCs.
-
-
-### 2.0.11
-
-- [improvement] JAVA-718: Log streamid at the trace level on sending request and receiving response.
-- [bug] JAVA-796: Fix SpeculativeExecutionPolicy.init() and close() are never called.
-- [improvement] JAVA-710: Suppress unnecessary warning at shutdown.
-- [improvement] #340: Allow DNS name with multiple A-records as contact point.
-- [bug] JAVA-794: Allow tracing across multiple result pages.
-- [bug] JAVA-737: DowngradingConsistencyRetryPolicy ignores write timeouts.
-- [bug] JAVA-736: Forbid bind marker in QueryBuilder add/append/prepend.
-- [bug] JAVA-712: Prevent QueryBuilder.quote() from applying duplicate double quotes.
-- [bug] JAVA-688: Prevent QueryBuilder from trying to serialize raw string.
-- [bug] JAVA-679: Support bind marker in QueryBuilder DELETE's list index.
-- [improvement] JAVA-475: Improve QueryBuilder API for SELECT DISTINCT.
-- [improvement] JAVA-225: Create values() function for Insert builder using List.
-- [improvement] JAVA-702: Warn when ReplicationStrategy encounters invalid
-  replication factors.
-- [improvement] JAVA-662: Add PoolingOptions method to set both core and max
-  connections.
-- [improvement] JAVA-766: Do not include epoll JAR in binary distribution.
-- [improvement] JAVA-726: Optimize internal copies of Request objects.
-- [bug] JAVA-815: Preserve tracing across retries.
-- [improvement] JAVA-709: New RetryDecision.tryNextHost().
-- [bug] JAVA-733: Handle function calls and raw strings as non-idempotent in QueryBuilder.
-- [improvement] JAVA-765: Provide API to retrieve values of a Parameterized SimpleStatement.
-- [improvement] JAVA-827: implement UPDATE .. IF EXISTS in QueryBuilder.
-- [improvement] JAVA-618: Randomize contact points list to prevent hotspots.
-- [improvement] JAVA-720: Surface the coordinator used on query failure.
-- [bug] JAVA-792: Handle contact points removed during init.
-- [improvement] JAVA-719: Allow PlainTextAuthProvider to change its credentials at runtime.
-- [new feature] JAVA-151: Make it possible to register for SchemaChange Events.
-- [improvement] JAVA-861: Downgrade "Asked to rebuild table" log from ERROR to INFO level.
-- [improvement] JAVA-797: Provide an option to prepare statements only on one node.
-- [improvement] JAVA-658: Provide an option to not re-prepare all statements in onUp.
-- [improvement] JAVA-853: Customizable creation of netty timer.
-- [bug] JAVA-859: Avoid quadratic ring processing with invalid replication factors.
-- [improvement] JAVA-657: Debounce control connection queries.
-- [bug] JAVA-784: LoadBalancingPolicy.distance() called before init().
-- [new feature] JAVA-828: Make driver-side metadata optional.
-- [improvement] JAVA-544: Allow hosts to remain partially up.
-- [improvement] JAVA-821, JAVA-822: Remove internal blocking calls and expose async session
-  creation.
-- [improvement] JAVA-725: Use parallel calls when re-preparing statement on other
-  hosts.
-- [bug] JAVA-629: Don't use connection timeout for unrelated internal queries.
-- [bug] JAVA-892: Fix NPE in speculative executions when metrics disabled.
-
-Merged from 2.0.10_fixes branch:
-
-- [improvement] JAVA-756: Use Netty's pooled ByteBufAllocator by default.
-- [improvement] JAVA-759: Expose "unsafe" paging state API.
-- [bug] JAVA-767: Fix getObject by name.
-- [bug] JAVA-768: Prevent race during pool initialization.
-
-
-### 2.0.10.1
-
-- [improvement] JAVA-756: Use Netty's pooled ByteBufAllocator by default.
-- [improvement] JAVA-759: Expose "unsafe" paging state API.
-- [bug] JAVA-767: Fix getObject by name.
-- [bug] JAVA-768: Prevent race during pool initialization.
-
-
-### 2.0.10
-
-- [new feature] JAVA-518: Add AddressTranslater for EC2 multi-region deployment.
-- [improvement] JAVA-533: Add connection heartbeat.
-- [improvement] JAVA-568: Reduce level of logs on missing rpc_address.
-- [improvement] JAVA-312, JAVA-681: Expose node token and range information.
-- [bug] JAVA-595: Fix cluster name mismatch check at startup.
-- [bug] JAVA-620: Fix guava dependency when using OSGI.
-- [bug] JAVA-678: Fix handling of DROP events when ks name is case-sensitive.
-- [improvement] JAVA-631: Use List instead of List in QueryBuilder API.
-- [improvement] JAVA-654: Exclude Netty POM from META-INF in shaded JAR.
-- [bug] JAVA-655: Quote single quotes contained in table comments in asCQLQuery method.
-- [bug] JAVA-684: Empty TokenRange returned in a one token cluster.
-- [improvement] JAVA-687: Expose TokenRange#contains.
-- [new feature] JAVA-547: Expose values of BoundStatement.
-- [new feature] JAVA-584: Add getObject to BoundStatement and Row.
-- [improvement] JAVA-419: Improve connection pool resizing algorithm.
-- [bug] JAVA-599: Fix race condition between pool expansion and shutdown.
-- [improvement] JAVA-622: Upgrade Netty to 4.0.27.
-- [improvement] JAVA-562: Coalesce frames before flushing them to the connection.
-- [improvement] JAVA-583: Rename threads to indicate that they are for the driver.
-- [new feature] JAVA-550: Expose paging state.
-- [new feature] JAVA-646: Slow Query Logger.
-- [improvement] JAVA-698: Exclude some errors from measurements in LatencyAwarePolicy.
-- [bug] JAVA-641: Fix issue when executing a PreparedStatement from another cluster.
-- [improvement] JAVA-534: Log keyspace xxx does not exist at WARN level.
-- [improvement] JAVA-619: Allow Cluster subclasses to delegate to another instance.
-- [new feature] JAVA-669: Expose an API to check for schema agreement after a
-  schema-altering statement.
-- [improvement] JAVA-692: Make connection and pool creation fully async.
-- [improvement] JAVA-505: Optimize connection use after reconnection.
-- [improvement] JAVA-617: Remove "suspected" mechanism.
-- [improvement] reverts JAVA-425: Don't mark connection defunct on client timeout.
-- [new feature] JAVA-561: Speculative query executions.
-- [bug] JAVA-666: Release connection before completing the ResultSetFuture.
-- [new feature BETA] JAVA-723: Percentile-based variant of query logger and speculative
-  executions.
-- [bug] JAVA-734: Fix buffer leaks when compression is enabled.
-
-Merged from 2.0.9_fixes branch:
-
-- [bug] JAVA-614: Prevent race between cancellation and query completion.
-- [bug] JAVA-632: Prevent cancel and timeout from cancelling unrelated ResponseHandler if
-  streamId was already released and reused.
-- [bug] JAVA-642: Fix issue when newly opened pool fails before we could mark the node UP.
-- [bug] JAVA-613: Fix unwanted LBP notifications when a contact host is down.
-- [bug] JAVA-651: Fix edge cases where a connection was released twice.
-- [bug] JAVA-653: Fix edge cases in query cancellation.
-
-
-### 2.0.9.2
-
-- [bug] JAVA-651: Fix edge cases where a connection was released twice.
-- [bug] JAVA-653: Fix edge cases in query cancellation.
-
-
-### 2.0.9.1
-
-- [bug] JAVA-614: Prevent race between cancellation and query completion.
-- [bug] JAVA-632: Prevent cancel and timeout from cancelling unrelated ResponseHandler if
-  streamId was already released and reused.
-- [bug] JAVA-642: Fix issue when newly opened pool fails before we could mark the node UP.
-- [bug] JAVA-613: Fix unwanted LBP notifications when a contact host is down.
-
-
-### 2.0.9
-
-- [improvement] JAVA-538: Shade Netty dependency.
-- [improvement] JAVA-543: Target schema refreshes more precisely.
-- [bug] JAVA-546: Don't check rpc_address for control host.
-- [improvement] JAVA-409: Improve message of NoHostAvailableException.
-- [bug] JAVA-556: Rework connection reaper to avoid deadlock.
-- [bug] JAVA-557: Avoid deadlock when multiple connections to the same host get write
-  errors.
-- [improvement] JAVA-504: Make shuffle=true the default for TokenAwarePolicy.
-- [bug] JAVA-577: Fix bug when SUSPECT reconnection succeeds, but one of the pooled
-  connections fails while bringing the node back up.
-- [bug] JAVA-419: JAVA-587: Prevent faulty control connection from ignoring reconnecting hosts.
-- temporarily revert "Add idle timeout to the connection pool".
-- [bug] JAVA-593: Ensure updateCreatedPools does not add pools for suspected hosts.
-- [bug] JAVA-594: Ensure state change notifications for a given host are handled serially.
-- [bug] JAVA-597: Ensure control connection reconnects when control host is removed.
-
-
-### 2.0.8
-
-- [bug] JAVA-526: Fix token awareness for case-sensitive keyspaces and tables.
-- [bug] JAVA-515: Check maximum number of values passed to SimpleStatement.
-- [improvement] JAVA-532: Expose the driver version through the API.
-- [improvement] JAVA-522: Optimize session initialization when some hosts are not
-  responsive.
-
-
-### 2.0.7
-
-- [bug] JAVA-449: Handle null pool in PooledConnection.release.
-- [improvement] JAVA-425: Defunct connection on request timeout.
-- [improvement] JAVA-426: Try next host when we get a SERVER_ERROR.
-- [bug] JAVA-449, JAVA-460, JAVA-471: Handle race between query timeout and completion.
-- [bug] JAVA-496: Fix DCAwareRoundRobinPolicy datacenter auto-discovery.
-- [bug] JAVA-497: Ensure control connection does not trigger concurrent reconnects.
-- [improvement] JAVA-472: Keep trying to reconnect on authentication errors.
-- [improvement] JAVA-463: Expose close method on load balancing policy.
-- [improvement] JAVA-459: Allow load balancing policy to trigger refresh for a single host.
-- [bug] JAVA-493: Expose an API to cancel reconnection attempts.
-- [bug] JAVA-503: Fix NPE when a connection fails during pool construction.
-- [improvement] JAVA-423: Log datacenter name in DCAware policy's init when it is explicitly provided.
-- [improvement] JAVA-504: Shuffle the replicas in TokenAwarePolicy.newQueryPlan.
-- [improvement] JAVA-507: Make schema agreement wait tuneable.
-- [improvement] JAVA-494: Document how to inject the driver metrics into another registry.
-- [improvement] JAVA-419: Add idle timeout to the connection pool.
-- [bug] JAVA-516: LatencyAwarePolicy does not shutdown executor on invocation of close.
-- [improvement] JAVA-451: Throw an exception when DCAwareRoundRobinPolicy is built with
-  an explicit but null or empty local datacenter.
-- [bug] JAVA-511: Fix check for local contact points in DCAware policy's init.
-- [improvement] JAVA-457: Make timeout on saturated pool customizable.
-- [improvement] JAVA-521: Downgrade Guava to 14.0.1.
-
-
-### 2.0.6
-
-- [bug] JAVA-397: Check cluster name when connecting to a new node.
-- [bug] JAVA-326: Add missing CAS delete support in QueryBuilder.
-- [bug] JAVA-363: Add collection and data length checks during serialization.
-- [improvement] JAVA-329: Surface number of retries in metrics.
-- [bug] JAVA-428: Do not use a host when no rpc_address found for it.
-- [improvement] JAVA-358: Add ResultSet.wasApplied() for conditional queries.
-- [bug] JAVA-349: Fix negative HostConnectionPool open count.
-- [improvement] JAVA-436: Log more connection details at trace and debug levels.
-- [bug] JAVA-445: Fix cluster shutdown.
-- [improvement] JAVA-439: Expose child policy in chainable load balancing policies.
-
-
-### 2.0.5
-
-- [bug] JAVA-407: Release connections on ResultSetFuture#cancel.
-- [bug] JAVA-393: Fix handling of SimpleStatement with values in query builder
-  batches.
-- [bug] JAVA-417: Ensure pool is properly closed in onDown.
-- [bug] JAVA-415: Fix tokenMap initialization at startup.
-- [bug] JAVA-418: Avoid deadlock on close.
-
-
-### 2.0.4
-
-- [improvement] JAVA-204: Better handling of dead connections.
-- [bug] JAVA-373: Fix potential NPE in ControlConnection.
-- [bug] JAVA-291: Throws NPE when passed null for a contact point.
-- [bug] JAVA-315: Avoid LoadBalancingPolicy onDown+onUp at startup.
-- [bug] JAVA-343: Avoid classloader leak in Tomcat.
-- [bug] JAVA-387: Avoid deadlock in onAdd/onUp.
-- [bug] JAVA-377, JAVA-391: Make metadata parsing more lenient.
-- [bug] JAVA-394: Ensure defunct connections are completely closed.
-- [bug] JAVA-342, JAVA-390: Fix memory and resource leak on closed Sessions.
-
-
-### 2.0.3
-
-- [new] The new AbsractSession makes mocking of Session easier.
-- [new] JAVA-309: Allow to trigger a refresh of connected hosts.
-- [new] JAVA-265: New Session#getState method allows to grab information on
-  which nodes a session is connected to.
-- [new] JAVA-327: Add QueryBuilder syntax for tuples in where clauses (syntax
-  introduced in Cassandra 2.0.6).
-- [improvement] JAVA-359: Properly validate arguments of PoolingOptions methods.
-- [bug] JAVA-368: Fix bogus rejection of BigInteger in 'execute with values'.
-- [bug] JAVA-367: Signal connection failure sooner to avoid missing them.
-- [bug] JAVA-337: Throw UnsupportedOperationException for protocol batch
-  setSerialCL.
-
-Merged from 1.0 branch:
-
-- [bug] JAVA-325: Fix periodic reconnection to down hosts.
-
-
-### 2.0.2
-
-- [api] The type of the map key returned by NoHostAvailable#getErrors has changed from
-  InetAddress to InetSocketAddress. Same for Initializer#getContactPoints return and
-  for AuthProvider#newAuthenticator.
-- [api] JAVA-296: The default load balacing policy is now DCAwareRoundRobinPolicy, and the local
-  datacenter is automatically picked based on the first connected node. Furthermore,
-  the TokenAwarePolicy is also used by default.
-- [new] JAVA-145: New optional AddressTranslater.
-- [bug] JAVA-321: Don't remove quotes on keyspace in the query builder.
-- [bug] JAVA-320: Fix potential NPE while cluster undergo schema changes.
-- [bug] JAVA-319: Fix thread-safety of page fetching.
-- [bug] JAVA-318: Fix potential NPE using fetchMoreResults.
-
-Merged from 1.0 branch:
-
-- [new] JAVA-179: Expose the name of the partitioner in use in the cluster metadata.
-- [new] Add new WhiteListPolicy to limit the nodes connected to a particular list.
-- [improvement] JAVA-289: Do not hop DC for LOCAL_* CL in DCAwareRoundRobinPolicy.
-- [bug] JAVA-313: Revert back to longs for dates in the query builder.
-- [bug] JAVA-314: Don't reconnect to nodes ignored by the load balancing policy.
-
-
-### 2.0.1
-
-- [improvement] JAVA-278: Handle the static columns introduced in Cassandra 2.0.6.
-- [improvement] JAVA-208: Add Cluster#newSession method to create Session without connecting
-  right away.
-- [bug] JAVA-279: Add missing iso8601 patterns for parsing dates.
-- [bug] Properly parse BytesType as the blob type.
-- [bug] JAVA-280: Potential NPE when parsing schema of pre-CQL tables of C* 1.2 nodes.
-
-Merged from 1.0 branch:
-
-- [bug] JAVA-275: LatencyAwarePolicy.Builder#withScale doesn't set the scale.
-- [new] JAVA-114: Add methods to check if a Cluster/Session instance has been closed already.
-
-
-### 2.0.0
-
-- [api] JAVA-269: Case sensitive identifier by default in Metadata.
-- [bug] JAVA-274: Fix potential NPE in Cluster#connect.
-
-Merged from 1.0 branch:
-
-- [bug] JAVA-263: Always return the PreparedStatement object that is cache internally.
-- [bug] JAVA-261: Fix race when multiple connect are done in parallel.
-- [bug] JAVA-270: Don't connect at all to nodes that are ignored by the load balancing
-  policy.
-
-
-### 2.0.0-rc3
-
-- [improvement] The protocol version 1 is now supported (features only supported by the
-  version 2 of the protocol throw UnsupportedFeatureException).
-- [improvement] JAVA-195: Make most main objects interface to facilitate testing/mocking.
-- [improvement] Adds new getStatements and clear methods to BatchStatement.
-- [api] JAVA-247: Renamed shutdown to closeAsync and ShutdownFuture to CloseFuture. Clustering
-  and Session also now implement Closeable.
-- [bug] JAVA-232: Fix potential thread leaks when shutting down Metrics.
-- [bug] JAVA-231: Fix potential NPE in HostConnectionPool.
-- [bug] JAVA-244: Avoid NPE when node is in an unconfigured DC.
-- [bug] JAVA-258: Don't block for scheduled reconnections on Cluster#close.
-
-Merged from 1.0 branch:
-
-- [new] JAVA-224: Added Session#prepareAsync calls.
-- [new] JAVA-249: Added Cluster#getLoggedKeyspace.
-- [improvement] Avoid preparing a statement multiple time per host with multiple sessions.
-- [bug] JAVA-255: Make sure connections are returned to the right pools.
-- [bug] JAVA-264: Use date string in query build to work-around CASSANDRA-6718.
-
-
-### 2.0.0-rc2
-
-- [new] JAVA-207: Add LOCAL_ONE consistency level support (requires using C* 2.0.2+).
-- [bug] JAVA-219: Fix parsing of counter types.
-- [bug] JAVA-218: Fix missing whitespace for IN clause in the query builder.
-- [bug] JAVA-221: Fix replicas computation for token aware balancing.
-
-Merged from 1.0 branch:
-
-- [bug] JAVA-213: Fix regression from JAVA-201.
-- [improvement] New getter to obtain a snapshot of the scores maintained by
-  LatencyAwarePolicy.
-
-
-### 2.0.0-rc1
-
-- [new] JAVA-199: Mark compression dependencies optional in maven.
-- [api] Renamed TableMetadata#getClusteringKey to TableMetadata#getClusteringColumns.
-
-Merged from 1.0 branch:
-
-- [new] JAVA-142: OSGi bundle.
-- [improvement] JAVA-205: Make collections returned by Row immutable.
-- [improvement] JAVA-203: Limit internal thread pool size.
-- [bug] JAVA-201: Don't retain unused PreparedStatement in memory.
-- [bug] Add missing clustering order info in TableMetadata
-- [bug] JAVA-196: Allow bind markers for collections in the query builder.
-
-
-### 2.0.0-beta2
-
-- [api] BoundStatement#setX(String, X) methods now set all values (if there is
-  more than one) having the provided name, not just the first occurence.
-- [api] The Authenticator interface now has a onAuthenticationSuccess method that
-  allows to handle the potential last token sent by the server.
-- [new] The query builder don't serialize large values to strings anymore by
-  default by making use the new ability to send values alongside the query string.
-- [new] JAVA-140: The query builder has been updated for new CQL features.
-- [bug] Fix exception when a conditional write timeout C* side.
-- [bug] JAVA-182: Ensure connection is created when Cluster metadata are asked for.
-- [bug] JAVA-187: Fix potential NPE during authentication.
-
-
-### 2.0.0-beta1
-
-- [api] The 2.0 version is an API-breaking upgrade of the driver. While most
-  of the breaking changes are minor, there are too numerous to be listed here
-  and you are encouraged to look at the Upgrade_guide_to_2.0 file that describe
-  those changes in details.
-- [new] LZ4 compression is supported for the protocol.
-- [new] JAVA-39: The driver does not depend on cassandra-all anymore.
-- [new] New BatchStatement class allows to execute batch other statements.
-- [new] Large ResultSet are now paged (incrementally fetched) by default.
-- [new] SimpleStatement support values for bind-variables, to allow
-  prepare+execute behavior with one roundtrip.
-- [new] Query parameters defaults (Consistency level, page size, ...) can be
-  configured globally.
-- [new] New Cassandra 2.0 SERIAL and LOCAL_SERIAL consistency levels are
-  supported.
-- [new] JAVA-116: Cluster#shutdown now waits for ongoing queries to complete by default.
-- [new] Generic authentication through SASL is now exposed.
-- [bug] JAVA-88: TokenAwarePolicy now takes all replica into account, instead of only the
-  first one.
-
-
-### 1.0.5
-
-- [new] JAVA-142: OSGi bundle.
-- [new] JAVA-207: Add support for ConsistencyLevel.LOCAL_ONE; note that this
-  require Cassandra 1.2.12+.
-- [improvement] JAVA-205: Make collections returned by Row immutable.
-- [improvement] JAVA-203: Limit internal thread pool size.
-- [improvement] New getter to obtain a snapshot of the scores maintained by
-  LatencyAwarePolicy.
-- [improvement] JAVA-222: Avoid synchronization when getting codec for collection
-  types.
-- [bug] JAVA-201, JAVA-213: Don't retain unused PreparedStatement in memory.
-- [bug] Add missing clustering order info in TableMetadata
-- [bug] JAVA-196: Allow bind markers for collections in the query builder.
-
-
-### 1.0.4
-
-- [api] JAVA-163: The Cluster.Builder#poolingOptions and Cluster.Builder#socketOptions
-  are now deprecated. They are replaced by the new withPoolingOptions and
-  withSocketOptions methods.
-- [new] JAVA-129: A new LatencyAwarePolicy wrapping policy has been added, allowing to
-  add latency awareness to a wrapped load balancing policy.
-- [new] JAVA-161: Cluster.Builder#deferInitialization: Allow defering cluster initialization.
-- [new] JAVA-117: Add truncate statement in query builder.
-- [new] JAVA-106: Support empty IN in the query builder.
-- [bug] JAVA-166: Fix spurious "No current pool set; this should not happen" error
-  message.
-- [bug] JAVA-184: Fix potential overflow in RoundRobinPolicy and correctly errors if
-  a balancing policy throws.
-- [bug] Don't release Stream ID for timeouted queries (unless we do get back
-  the response)
-- [bug] Correctly escape identifiers and use fully qualified table names when
-  exporting schema as string.
-
-
-### 1.0.3
-
-- [api] The query builder now correctly throw an exception when given a value
-  of a type it doesn't know about.
-- [new] SocketOptions#setReadTimeout allows to set a timeout on how long we
-  wait for the answer of one node. See the javadoc for more details.
-- [new] New Session#prepare method that takes a Statement.
-- [bug] JAVA-143: Always take per-query CL, tracing, etc. into account for QueryBuilder
-  statements.
-- [bug] Temporary fixup for TimestampType when talking to C* 2.0 nodes.
-
-
-### 1.0.2
-
-- [api] Host#getMonitor and all Host.HealthMonitor methods have been
-  deprecated. The new Host#isUp method is now prefered to the method
-  in the monitor and you should now register Host.StateListener against
-  the Cluster object directly (registering against a host HealthMonitor
-  was much more limited anyway).
-- [new] JAVA-92: New serialize/deserialize methods in DataType to serialize/deserialize
-  values to/from bytes.
-- [new] JAVA-128: New getIndexOf() method in ColumnDefinitions to find the index of
-  a given column name.
-- [bug] JAVA-131: Fix a bug when thread could get blocked while setting the current
-  keyspace.
-- [bug] JAVA-136: Quote inet addresses in the query builder since CQL3 requires it.
-
-
-### 1.0.1
-
-- [api] JAVA-100: Function call handling in the query builder has been modified in a
-  backward incompatible way. Function calls are not parsed from string values
-  anymore as this wasn't safe. Instead the new 'fcall' method should be used.
-- [api] Some typos in method names in PoolingOptions have been fixed in a
-  backward incompatible way before the API get widespread.
-- [bug] JAVA-123: Don't destroy composite partition key with BoundStatement and
-  TokenAwarePolicy.
-- [new] null values support in the query builder.
-- [new] JAVA-5: SSL support (requires C* >= 1.2.1).
-- [new] JAVA-113: Allow generating unlogged batch in the query builder.
-- [improvement] Better error message when no host are available.
-- [improvement] Improves performance of the stress example application been.
-
-
-### 1.0.0
-
-- [api] The AuthInfoProvider has be (temporarily) removed. Instead, the
-  Cluster builder has a new withCredentials() method to provide a username
-  and password for use with Cassandra's PasswordAuthenticator. Custom
-  authenticator will be re-introduced in a future version but are not
-  supported at the moment.
-- [api] The isMetricsEnabled() method in Configuration has been replaced by
-  getMetricsOptions(). An option to disabled JMX reporting (on by default)
-  has been added.
-- [bug] JAVA-91: Don't make default load balancing policy a static singleton since it
-  is stateful.
-
-
-### 1.0.0-RC1
-
-- [new] JAVA-79: Null values are now supported in BoundStatement (but you will need at
-  least Cassandra 1.2.3 for it to work). The API of BoundStatement has been
-  slightly changed so that not binding a variable is not an error anymore,
-  the variable is simply considered null by default. The isReady() method has
-  been removed.
-- [improvement] JAVA-75: The Cluster/Session shutdown methods now properly block until
-  the shutdown is complete. A version with at timeout has been added.
-- [bug] JAVA-44: Fix use of CQL3 functions in the query builder.
-- [bug] JAVA-77: Fix case where multiple schema changes too quickly wouldn't work
-  (only triggered when 0.0.0.0 was used for the rpc_address on the Cassandra
-  nodes).
-- [bug] JAVA-72: Fix IllegalStateException thrown due to a reconnection made on an I/O
-  thread.
-- [bug] JAVA-82: Correctly reports errors during authentication phase.
-
-
-### 1.0.0-beta2
-
-- [new] JAVA-51, JAVA-60, JAVA-58: Support blob constants, BigInteger, BigDecimal and counter batches in
-  the query builder.
-- [new] JAVA-61: Basic support for custom CQL3 types.
-- [new] JAVA-65: Add "execution infos" for a result set (this also move the query
-  trace in the new ExecutionInfos object, so users of beta1 will have to
-  update).
-- [bug] JAVA-62: Fix failover bug in DCAwareRoundRobinPolicy.
-- [bug] JAVA-66: Fix use of bind markers for routing keys in the query builder.
-
-
-### 1.0.0-beta1
-
-- initial release
diff --git a/docs.yaml b/docs.yaml
deleted file mode 100644
index b38b86397d7..00000000000
--- a/docs.yaml
+++ /dev/null
@@ -1,67 +0,0 @@
-title:    Java Driver for Apache Cassandra
-summary:  High performance Java client for Apache Cassandra
-homepage: http://datastax.github.io/java-driver/
-theme: datastax
-sections:
-  - title:     Manual
-    prefix:    /manual
-    sources:
-      - type:  markdown
-        files: 'manual/**/*.md'
-  # The 'manual' section was called 'features' in older releases. Leave both
-  # definitions and Documentor will pick up whichever exists and ignore the
-  # other.
-  - title:     Features
-    prefix:    /features
-    sources:
-      - type:  markdown
-        files: 'features/**/*.md'
-  - title:     Changelog
-    prefix:    /changelog
-    sources:
-      - type:  markdown
-        files: 'changelog/**/*.md'
-  - title:     Upgrading
-    prefix:    /upgrade_guide
-    sources:
-      - type:  markdown
-        files: 'upgrade_guide/**/*.md'
-  - title:     FAQ
-    prefix:    /faq
-    sources:
-      - type:  markdown
-        files: 'faq/**/*.md'
-links:
-  - title: Code
-    href:  https://github.com/datastax/java-driver/
-  - title: Docs
-    href:  http://docs.datastax.com/en/developer/java-driver/
-  - title: Issues
-    href:  https://datastax-oss.atlassian.net/browse/JAVA/
-  - title: Mailing List
-    href:  https://groups.google.com/a/lists.datastax.com/forum/#!forum/java-driver-user
-  - title: Releases
-    href:  http://downloads.datastax.com/java-driver/
-api_docs:
-  3.3: http://docs.datastax.com/en/drivers/java/3.3
-  4.0-alpha: http://docs.datastax.com/en/drivers/java/4.0
-  3.2: http://docs.datastax.com/en/drivers/java/3.2
-  3.1: http://docs.datastax.com/en/drivers/java/3.1
-  3.0: http://docs.datastax.com/en/drivers/java/3.0
-  2.1: http://docs.datastax.com/en/drivers/java/2.1
-  2.0: http://docs.datastax.com/en/drivers/java/2.0
-versions:
-  - name: '3.3'
-    ref: '3.3.0'
-  - name: '4.0-alpha'
-    ref: '9f0edeb'
-  - name: '3.2'
-    ref: '3.2_docfixes'
-  - name: '3.1'
-    ref: '3.1_docfixes'
-  - name: '3.0'
-    ref: '3.0_docfixes'
-  - name: '2.1'
-    ref: '2.1.10.3'
-  - name: '2.0'
-    ref: '2.0.12.3'
diff --git a/docs/Makefile b/docs/Makefile
new file mode 100644
index 00000000000..7a18ae2c1af
--- /dev/null
+++ b/docs/Makefile
@@ -0,0 +1,72 @@
+# You can set these variables from the command line.
+SPHINXOPTS    =
+SPHINXBUILD   = poetry run sphinx-build
+PAPER         =
+BUILDDIR      = _build
+
+# Internal variables.
+PAPEROPT_a4     = -D latex_paper_size=a4
+PAPEROPT_letter = -D latex_paper_size=letter
+ALLSPHINXOPTS   = -d $(BUILDDIR)/doctrees $(PAPEROPT_$(PAPER)) $(SPHINXOPTS) source
+# the i18n builder cannot share the environment and doctrees with the others
+I18NSPHINXOPTS  = $(PAPEROPT_$(PAPER)) $(SPHINXOPTS) .
+
+.PHONY: all
+all: dirhtml
+
+.PHONY: pristine
+pristine: clean
+	git clean -dfX
+
+.PHONY: setup
+setup:
+	./_utils/setup.sh
+
+.PHONY: clean
+clean:
+	rm -rf $(BUILDDIR)/*
+
+.PHONY: preview
+preview: setup
+	cd .. && ./docs/_utils/doxygen.sh
+	poetry run sphinx-autobuild -b dirhtml $(ALLSPHINXOPTS) $(BUILDDIR)/dirhtml --port 5500
+
+.PHONY: dirhtml
+dirhtml: setup
+	$(SPHINXBUILD) -b dirhtml $(ALLSPHINXOPTS) $(BUILDDIR)/dirhtml
+	@echo
+	@echo "Build finished. The HTML pages are in $(BUILDDIR)/dirhtml."
+
+.PHONY: singlehtml
+singlehtml: setup
+	$(SPHINXBUILD) -b singlehtml $(ALLSPHINXOPTS) $(BUILDDIR)/singlehtml
+	@echo
+	@echo "Build finished. The HTML page is in $(BUILDDIR)/singlehtml."
+
+.PHONY: epub
+epub: 	setup
+	$(SPHINXBUILD) -b epub $(ALLSPHINXOPTS) $(BUILDDIR)/epub
+	@echo
+	@echo "Build finished. The epub file is in $(BUILDDIR)/epub."
+
+.PHONY: epub3
+epub3:	setup
+	$(SPHINXBUILD) -b epub3 $(ALLSPHINXOPTS) $(BUILDDIR)/epub3
+	@echo
+	@echo "Build finished. The epub3 file is in $(BUILDDIR)/epub3."
+
+.PHONY: dummy
+dummy: 	setup
+	$(SPHINXBUILD) -b dummy $(ALLSPHINXOPTS) $(BUILDDIR)/dummy
+	@echo
+	@echo "Build finished. Dummy builder generates no files."
+
+.PHONY: linkcheck
+linkcheck: setup
+	$(SPHINXBUILD) -b linkcheck . $(BUILDDIR)/linkcheck
+
+.PHONY: multiversion
+multiversion: setup
+	poetry run ./_utils/multiversion.sh
+	@echo
+	@echo "Build finished. The HTML pages are in $(BUILDDIR)/dirhtml."
diff --git a/docs/_utils/api.html b/docs/_utils/api.html
new file mode 100644
index 00000000000..684e559579c
--- /dev/null
+++ b/docs/_utils/api.html
@@ -0,0 +1,8 @@
+
+
+  
+    Redirecting to API Documentation
+    
+    
+  
+
\ No newline at end of file
diff --git a/docs/_utils/deploy.sh b/docs/_utils/deploy.sh
new file mode 100755
index 00000000000..0450baabb7a
--- /dev/null
+++ b/docs/_utils/deploy.sh
@@ -0,0 +1,16 @@
+#!/bin/bash
+
+# Clone repo
+git clone "https://x-access-token:${GITHUB_TOKEN}@github.com/${GITHUB_REPOSITORY}.git" --branch gh-pages --single-branch gh-pages
+cp -r docs/_build/dirhtml/* gh-pages
+# Redirections
+./docs/_utils/redirect.sh > gh-pages/index.html
+find  gh-pages/* -name "api.html" -exec cp ./docs/_utils/api.html {} \;
+# Deploy
+cd gh-pages
+touch .nojekyll
+git config --local user.email "action@scylladb.com"
+git config --local user.name "GitHub Action"
+git add .
+git commit -m "Publish docs" || true
+git push origin gh-pages --force
diff --git a/docs/_utils/doxygen.sh b/docs/_utils/doxygen.sh
new file mode 100755
index 00000000000..ebd995cd5ba
--- /dev/null
+++ b/docs/_utils/doxygen.sh
@@ -0,0 +1,12 @@
+#!/bin/bash
+
+GITHUB_VERSION=${GITHUB_REF##*/}
+
+OUTPUT_DIR="docs/_build/dirhtml/api"
+if [[ "$GITHUB_VERSION" != "" ]]; then
+    OUTPUT_DIR="docs/_build/dirhtml/$GITHUB_VERSION/api"
+    echo "HTML_OUTPUT = $OUTPUT_DIR" >> doxyfile
+fi
+echo $OUTPUT_DIR
+mkdir -p "$OUTPUT_DIR"
+doxygen doxyfile
\ No newline at end of file
diff --git a/docs/_utils/multiversion.sh b/docs/_utils/multiversion.sh
new file mode 100755
index 00000000000..91d59f72de7
--- /dev/null
+++ b/docs/_utils/multiversion.sh
@@ -0,0 +1,3 @@
+#! /bin/bash
+
+cd .. && sphinx-multiversion docs/source docs/_build/dirhtml
diff --git a/docs/_utils/redirect.sh b/docs/_utils/redirect.sh
new file mode 100755
index 00000000000..2721ca034fe
--- /dev/null
+++ b/docs/_utils/redirect.sh
@@ -0,0 +1,13 @@
+#!/bin/bash
+
+cat <<- _EOF_
+
+
+  
+    Redirecting to Driver
+    
+    
+    
+  
+
+_EOF_
diff --git a/docs/_utils/redirections.yaml b/docs/_utils/redirections.yaml
new file mode 100644
index 00000000000..36a73f25baa
--- /dev/null
+++ b/docs/_utils/redirections.yaml
@@ -0,0 +1 @@
+api: /api/classes.html
diff --git a/docs/_utils/setup.sh b/docs/_utils/setup.sh
new file mode 100755
index 00000000000..5c08b967d2e
--- /dev/null
+++ b/docs/_utils/setup.sh
@@ -0,0 +1,16 @@
+#! /bin/bash
+
+if pwd | egrep -q '\s'; then
+	echo "Working directory name contains one or more spaces."
+	exit 1
+fi
+
+which python3 || { echo "Failed to find python3. Try installing Python for your operative system: https://www.python.org/downloads/" && exit 1; }
+# install pipx
+which pipx || python3 -m pip install --user pipx
+python3 -m pipx ensurepath
+
+# install poetry
+which poetry || pipx install poetry
+poetry --version || { echo "Failed to find or install poetry. Try installing it manually: https://python-poetry.org/docs/#installation" && exit 1; }
+poetry install
diff --git a/docs/poetry.lock b/docs/poetry.lock
new file mode 100644
index 00000000000..e2f9009a078
--- /dev/null
+++ b/docs/poetry.lock
@@ -0,0 +1,808 @@
+[[package]]
+category = "main"
+description = "A configurable sidebar-enabled Sphinx theme"
+name = "alabaster"
+optional = false
+python-versions = "*"
+version = "0.7.12"
+
+[[package]]
+category = "main"
+description = "An unobtrusive argparse wrapper with natural syntax"
+name = "argh"
+optional = false
+python-versions = "*"
+version = "0.26.2"
+
+[[package]]
+category = "dev"
+description = "Atomic file writes."
+marker = "sys_platform == \"win32\""
+name = "atomicwrites"
+optional = false
+python-versions = ">=2.7, !=3.0.*, !=3.1.*, !=3.2.*, !=3.3.*"
+version = "1.4.0"
+
+[[package]]
+category = "dev"
+description = "Classes Without Boilerplate"
+name = "attrs"
+optional = false
+python-versions = ">=2.7, !=3.0.*, !=3.1.*, !=3.2.*, !=3.3.*"
+version = "19.3.0"
+
+[package.extras]
+azure-pipelines = ["coverage", "hypothesis", "pympler", "pytest (>=4.3.0)", "six", "zope.interface", "pytest-azurepipelines"]
+dev = ["coverage", "hypothesis", "pympler", "pytest (>=4.3.0)", "six", "zope.interface", "sphinx", "pre-commit"]
+docs = ["sphinx", "zope.interface"]
+tests = ["coverage", "hypothesis", "pympler", "pytest (>=4.3.0)", "six", "zope.interface"]
+
+[[package]]
+category = "main"
+description = "Internationalization utilities"
+name = "babel"
+optional = false
+python-versions = ">=2.7, !=3.0.*, !=3.1.*, !=3.2.*, !=3.3.*"
+version = "2.8.0"
+
+[package.dependencies]
+pytz = ">=2015.7"
+
+[[package]]
+category = "main"
+description = "Python package for providing Mozilla's CA Bundle."
+name = "certifi"
+optional = false
+python-versions = "*"
+version = "2020.6.20"
+
+[[package]]
+category = "main"
+description = "Universal encoding detector for Python 2 and 3"
+name = "chardet"
+optional = false
+python-versions = "*"
+version = "3.0.4"
+
+[[package]]
+category = "main"
+description = "Cross-platform colored terminal text."
+marker = "sys_platform == \"win32\""
+name = "colorama"
+optional = false
+python-versions = ">=2.7, !=3.0.*, !=3.1.*, !=3.2.*, !=3.3.*, !=3.4.*"
+version = "0.4.3"
+
+[[package]]
+category = "main"
+description = "Python parser for the CommonMark Markdown spec"
+name = "commonmark"
+optional = false
+python-versions = "*"
+version = "0.9.1"
+
+[package.extras]
+test = ["flake8 (3.7.8)", "hypothesis (3.55.3)"]
+
+[[package]]
+category = "main"
+description = "Docutils -- Python Documentation Utilities"
+name = "docutils"
+optional = false
+python-versions = ">=2.7, !=3.0.*, !=3.1.*, !=3.2.*, !=3.3.*, !=3.4.*"
+version = "0.16"
+
+[[package]]
+category = "main"
+description = "Internationalized Domain Names in Applications (IDNA)"
+name = "idna"
+optional = false
+python-versions = ">=2.7, !=3.0.*, !=3.1.*, !=3.2.*, !=3.3.*"
+version = "2.10"
+
+[[package]]
+category = "main"
+description = "Getting image size from png/jpeg/jpeg2000/gif file"
+name = "imagesize"
+optional = false
+python-versions = ">=2.7, !=3.0.*, !=3.1.*, !=3.2.*, !=3.3.*"
+version = "1.2.0"
+
+[[package]]
+category = "dev"
+description = "Read metadata from Python packages"
+marker = "python_version < \"3.8\""
+name = "importlib-metadata"
+optional = false
+python-versions = "!=3.0.*,!=3.1.*,!=3.2.*,!=3.3.*,!=3.4.*,>=2.7"
+version = "1.7.0"
+
+[package.dependencies]
+zipp = ">=0.5"
+
+[package.extras]
+docs = ["sphinx", "rst.linker"]
+testing = ["packaging", "pep517", "importlib-resources (>=1.3)"]
+
+[[package]]
+category = "main"
+description = "A very fast and expressive template engine."
+name = "jinja2"
+optional = false
+python-versions = ">=2.7, !=3.0.*, !=3.1.*, !=3.2.*, !=3.3.*, !=3.4.*"
+version = "2.11.2"
+
+[package.dependencies]
+MarkupSafe = ">=0.23"
+
+[package.extras]
+i18n = ["Babel (>=0.8)"]
+
+[[package]]
+category = "main"
+description = "Python LiveReload is an awesome tool for web developers"
+name = "livereload"
+optional = false
+python-versions = "*"
+version = "2.6.2"
+
+[package.dependencies]
+six = "*"
+
+[package.dependencies.tornado]
+python = ">=2.8"
+version = "*"
+
+[[package]]
+category = "main"
+description = "Safely add untrusted strings to HTML/XML markup."
+name = "markupsafe"
+optional = false
+python-versions = ">=2.7,!=3.0.*,!=3.1.*,!=3.2.*,!=3.3.*"
+version = "1.1.1"
+
+[[package]]
+category = "dev"
+description = "More routines for operating on iterables, beyond itertools"
+name = "more-itertools"
+optional = false
+python-versions = ">=3.5"
+version = "8.4.0"
+
+[[package]]
+category = "main"
+description = "Core utilities for Python packages"
+name = "packaging"
+optional = false
+python-versions = ">=2.7, !=3.0.*, !=3.1.*, !=3.2.*, !=3.3.*"
+version = "20.4"
+
+[package.dependencies]
+pyparsing = ">=2.0.2"
+six = "*"
+
+[[package]]
+category = "main"
+description = "File system general utilities"
+name = "pathtools"
+optional = false
+python-versions = "*"
+version = "0.1.2"
+
+[[package]]
+category = "dev"
+description = "plugin and hook calling mechanisms for python"
+name = "pluggy"
+optional = false
+python-versions = ">=2.7, !=3.0.*, !=3.1.*, !=3.2.*, !=3.3.*"
+version = "0.13.1"
+
+[package.dependencies]
+[package.dependencies.importlib-metadata]
+python = "<3.8"
+version = ">=0.12"
+
+[package.extras]
+dev = ["pre-commit", "tox"]
+
+[[package]]
+category = "main"
+description = "Utility that helps with local TCP ports managment. It can find an unused TCP localhost port and remember the association."
+name = "port-for"
+optional = false
+python-versions = "*"
+version = "0.3.1"
+
+[[package]]
+category = "dev"
+description = "library with cross-python path, ini-parsing, io, code, log facilities"
+name = "py"
+optional = false
+python-versions = ">=2.7, !=3.0.*, !=3.1.*, !=3.2.*, !=3.3.*"
+version = "1.9.0"
+
+[[package]]
+category = "main"
+description = "Pygments is a syntax highlighting package written in Python."
+name = "pygments"
+optional = false
+python-versions = "*"
+version = "2.2.0"
+
+[[package]]
+category = "main"
+description = "Python parsing module"
+name = "pyparsing"
+optional = false
+python-versions = ">=2.6, !=3.0.*, !=3.1.*, !=3.2.*"
+version = "2.4.7"
+
+[[package]]
+category = "dev"
+description = "pytest: simple powerful testing with Python"
+name = "pytest"
+optional = false
+python-versions = ">=3.5"
+version = "5.4.3"
+
+[package.dependencies]
+atomicwrites = ">=1.0"
+attrs = ">=17.4.0"
+colorama = "*"
+more-itertools = ">=4.0.0"
+packaging = "*"
+pluggy = ">=0.12,<1.0"
+py = ">=1.5.0"
+wcwidth = "*"
+
+[package.dependencies.importlib-metadata]
+python = "<3.8"
+version = ">=0.12"
+
+[package.extras]
+checkqa-mypy = ["mypy (v0.761)"]
+testing = ["argcomplete", "hypothesis (>=3.56)", "mock", "nose", "requests", "xmlschema"]
+
+[[package]]
+category = "main"
+description = "World timezone definitions, modern and historical"
+name = "pytz"
+optional = false
+python-versions = "*"
+version = "2020.1"
+
+[[package]]
+category = "main"
+description = "YAML parser and emitter for Python"
+name = "pyyaml"
+optional = false
+python-versions = ">=2.7, !=3.0.*, !=3.1.*, !=3.2.*, !=3.3.*, !=3.4.*"
+version = "5.3.1"
+
+[[package]]
+category = "main"
+description = "A docutils-compatibility bridge to CommonMark, enabling you to write CommonMark inside of Docutils & Sphinx projects."
+name = "recommonmark"
+optional = false
+python-versions = "*"
+version = "0.5.0"
+
+[package.dependencies]
+commonmark = ">=0.7.3"
+docutils = ">=0.11"
+sphinx = ">=1.3.1"
+
+[[package]]
+category = "main"
+description = "Python HTTP for Humans."
+name = "requests"
+optional = false
+python-versions = ">=2.7, !=3.0.*, !=3.1.*, !=3.2.*, !=3.3.*, !=3.4.*"
+version = "2.24.0"
+
+[package.dependencies]
+certifi = ">=2017.4.17"
+chardet = ">=3.0.2,<4"
+idna = ">=2.5,<3"
+urllib3 = ">=1.21.1,<1.25.0 || >1.25.0,<1.25.1 || >1.25.1,<1.26"
+
+[package.extras]
+security = ["pyOpenSSL (>=0.14)", "cryptography (>=1.3.4)"]
+socks = ["PySocks (>=1.5.6,<1.5.7 || >1.5.7)", "win-inet-pton"]
+
+[[package]]
+category = "main"
+description = "Python 2 and 3 compatibility utilities"
+name = "six"
+optional = false
+python-versions = ">=2.7, !=3.0.*, !=3.1.*, !=3.2.*"
+version = "1.15.0"
+
+[[package]]
+category = "main"
+description = "This package provides 26 stemmers for 25 languages generated from Snowball algorithms."
+name = "snowballstemmer"
+optional = false
+python-versions = "*"
+version = "2.0.0"
+
+[[package]]
+category = "main"
+description = "Python documentation generator"
+name = "sphinx"
+optional = false
+python-versions = ">=3.5"
+version = "2.4.4"
+
+[package.dependencies]
+Jinja2 = ">=2.3"
+Pygments = ">=2.0"
+alabaster = ">=0.7,<0.8"
+babel = ">=1.3,<2.0 || >2.0"
+colorama = ">=0.3.5"
+docutils = ">=0.12"
+imagesize = "*"
+packaging = "*"
+requests = ">=2.5.0"
+setuptools = "*"
+snowballstemmer = ">=1.1"
+sphinxcontrib-applehelp = "*"
+sphinxcontrib-devhelp = "*"
+sphinxcontrib-htmlhelp = "*"
+sphinxcontrib-jsmath = "*"
+sphinxcontrib-qthelp = "*"
+sphinxcontrib-serializinghtml = "*"
+
+[package.extras]
+docs = ["sphinxcontrib-websupport"]
+test = ["pytest (<5.3.3)", "pytest-cov", "html5lib", "flake8 (>=3.5.0)", "flake8-import-order", "mypy (>=0.761)", "docutils-stubs"]
+
+[[package]]
+category = "main"
+description = "Watch a Sphinx directory and rebuild the documentation when a change is detected. Also includes a livereload enabled web server."
+name = "sphinx-autobuild"
+optional = false
+python-versions = "*"
+version = "0.7.1"
+
+[package.dependencies]
+PyYAML = ">=3.10"
+argh = ">=0.24.1"
+livereload = ">=2.3.0"
+pathtools = ">=0.1.2"
+port-for = "0.3.1"
+tornado = ">=3.2"
+watchdog = ">=0.7.1"
+
+[[package]]
+category = "main"
+description = "Add a copy button to each of your code cells."
+name = "sphinx-copybutton"
+optional = false
+python-versions = "*"
+version = "0.2.12"
+
+[package.dependencies]
+sphinx = ">=1.8"
+
+[package.extras]
+code_style = ["flake8 (>=3.7.0,<3.8.0)", "black", "pre-commit (1.17.0)"]
+
+[[package]]
+category = "main"
+description = "Add support for multiple versions to sphinx"
+name = "sphinx-multiversion"
+optional = false
+python-versions = "*"
+version = "0.2.3"
+
+[package.dependencies]
+sphinx = ">=2.1"
+
+[[package]]
+category = "main"
+description = "A Sphinx Theme for ScyllaDB projects documentation"
+name = "sphinx-scylladb-theme"
+optional = false
+python-versions = ">=3.7,<4.0"
+version = "0.1.9"
+
+[package.dependencies]
+Sphinx = ">=2.4.4,<3.0.0"
+pyyaml = ">=5.3,<6.0"
+sphinx-copybutton = ">=0.2.8,<0.3.0"
+sphinx-multiversion = "0.2.3"
+sphinx-tabs = ">=1.1.13,<2.0.0"
+
+[[package]]
+category = "main"
+description = "Sitemap generator for Sphinx"
+name = "sphinx-sitemap"
+optional = false
+python-versions = "*"
+version = "2.1.0"
+
+[package.dependencies]
+six = "*"
+sphinx = ">=1.2"
+
+[[package]]
+category = "main"
+description = "Tab views for Sphinx"
+name = "sphinx-tabs"
+optional = false
+python-versions = "*"
+version = "1.1.13"
+
+[package.dependencies]
+sphinx = ">=1.4"
+
+[[package]]
+category = "main"
+description = "sphinxcontrib-applehelp is a sphinx extension which outputs Apple help books"
+name = "sphinxcontrib-applehelp"
+optional = false
+python-versions = ">=3.5"
+version = "1.0.2"
+
+[package.extras]
+lint = ["flake8", "mypy", "docutils-stubs"]
+test = ["pytest"]
+
+[[package]]
+category = "main"
+description = "sphinxcontrib-devhelp is a sphinx extension which outputs Devhelp document."
+name = "sphinxcontrib-devhelp"
+optional = false
+python-versions = ">=3.5"
+version = "1.0.2"
+
+[package.extras]
+lint = ["flake8", "mypy", "docutils-stubs"]
+test = ["pytest"]
+
+[[package]]
+category = "main"
+description = "sphinxcontrib-htmlhelp is a sphinx extension which renders HTML help files"
+name = "sphinxcontrib-htmlhelp"
+optional = false
+python-versions = ">=3.5"
+version = "1.0.3"
+
+[package.extras]
+lint = ["flake8", "mypy", "docutils-stubs"]
+test = ["pytest", "html5lib"]
+
+[[package]]
+category = "main"
+description = "A sphinx extension which renders display math in HTML via JavaScript"
+name = "sphinxcontrib-jsmath"
+optional = false
+python-versions = ">=3.5"
+version = "1.0.1"
+
+[package.extras]
+test = ["pytest", "flake8", "mypy"]
+
+[[package]]
+category = "main"
+description = "sphinxcontrib-qthelp is a sphinx extension which outputs QtHelp document."
+name = "sphinxcontrib-qthelp"
+optional = false
+python-versions = ">=3.5"
+version = "1.0.3"
+
+[package.extras]
+lint = ["flake8", "mypy", "docutils-stubs"]
+test = ["pytest"]
+
+[[package]]
+category = "main"
+description = "sphinxcontrib-serializinghtml is a sphinx extension which outputs \"serialized\" HTML files (json and pickle)."
+name = "sphinxcontrib-serializinghtml"
+optional = false
+python-versions = ">=3.5"
+version = "1.1.4"
+
+[package.extras]
+lint = ["flake8", "mypy", "docutils-stubs"]
+test = ["pytest"]
+
+[[package]]
+category = "main"
+description = "Tornado is a Python web framework and asynchronous networking library, originally developed at FriendFeed."
+name = "tornado"
+optional = false
+python-versions = ">= 3.5"
+version = "6.0.4"
+
+[[package]]
+category = "main"
+description = "HTTP library with thread-safe connection pooling, file post, and more."
+name = "urllib3"
+optional = false
+python-versions = ">=2.7, !=3.0.*, !=3.1.*, !=3.2.*, !=3.3.*, !=3.4.*, <4"
+version = "1.25.10"
+
+[package.extras]
+brotli = ["brotlipy (>=0.6.0)"]
+secure = ["certifi", "cryptography (>=1.3.4)", "idna (>=2.0.0)", "pyOpenSSL (>=0.14)", "ipaddress"]
+socks = ["PySocks (>=1.5.6,<1.5.7 || >1.5.7,<2.0)"]
+
+[[package]]
+category = "main"
+description = "Filesystem events monitoring"
+name = "watchdog"
+optional = false
+python-versions = "*"
+version = "0.10.3"
+
+[package.dependencies]
+pathtools = ">=0.1.1"
+
+[package.extras]
+watchmedo = ["PyYAML (>=3.10)", "argh (>=0.24.1)"]
+
+[[package]]
+category = "dev"
+description = "Measures the displayed width of unicode strings in a terminal"
+name = "wcwidth"
+optional = false
+python-versions = "*"
+version = "0.2.5"
+
+[[package]]
+category = "dev"
+description = "Backport of pathlib-compatible object wrapper for zip files"
+marker = "python_version < \"3.8\""
+name = "zipp"
+optional = false
+python-versions = ">=3.6"
+version = "3.1.0"
+
+[package.extras]
+docs = ["sphinx", "jaraco.packaging (>=3.2)", "rst.linker (>=1.9)"]
+testing = ["jaraco.itertools", "func-timeout"]
+
+[metadata]
+content-hash = "6318ee5d2bca7a8ac06a683541055ac8e6be7c8c8bb62c8c1fd443f06afadc47"
+python-versions = "^3.7"
+
+[metadata.files]
+alabaster = [
+    {file = "alabaster-0.7.12-py2.py3-none-any.whl", hash = "sha256:446438bdcca0e05bd45ea2de1668c1d9b032e1a9154c2c259092d77031ddd359"},
+    {file = "alabaster-0.7.12.tar.gz", hash = "sha256:a661d72d58e6ea8a57f7a86e37d86716863ee5e92788398526d58b26a4e4dc02"},
+]
+argh = [
+    {file = "argh-0.26.2-py2.py3-none-any.whl", hash = "sha256:a9b3aaa1904eeb78e32394cd46c6f37ac0fb4af6dc488daa58971bdc7d7fcaf3"},
+    {file = "argh-0.26.2.tar.gz", hash = "sha256:e9535b8c84dc9571a48999094fda7f33e63c3f1b74f3e5f3ac0105a58405bb65"},
+]
+atomicwrites = [
+    {file = "atomicwrites-1.4.0-py2.py3-none-any.whl", hash = "sha256:6d1784dea7c0c8d4a5172b6c620f40b6e4cbfdf96d783691f2e1302a7b88e197"},
+    {file = "atomicwrites-1.4.0.tar.gz", hash = "sha256:ae70396ad1a434f9c7046fd2dd196fc04b12f9e91ffb859164193be8b6168a7a"},
+]
+attrs = [
+    {file = "attrs-19.3.0-py2.py3-none-any.whl", hash = "sha256:08a96c641c3a74e44eb59afb61a24f2cb9f4d7188748e76ba4bb5edfa3cb7d1c"},
+    {file = "attrs-19.3.0.tar.gz", hash = "sha256:f7b7ce16570fe9965acd6d30101a28f62fb4a7f9e926b3bbc9b61f8b04247e72"},
+]
+babel = [
+    {file = "Babel-2.8.0-py2.py3-none-any.whl", hash = "sha256:d670ea0b10f8b723672d3a6abeb87b565b244da220d76b4dba1b66269ec152d4"},
+    {file = "Babel-2.8.0.tar.gz", hash = "sha256:1aac2ae2d0d8ea368fa90906567f5c08463d98ade155c0c4bfedd6a0f7160e38"},
+]
+certifi = [
+    {file = "certifi-2020.6.20-py2.py3-none-any.whl", hash = "sha256:8fc0819f1f30ba15bdb34cceffb9ef04d99f420f68eb75d901e9560b8749fc41"},
+    {file = "certifi-2020.6.20.tar.gz", hash = "sha256:5930595817496dd21bb8dc35dad090f1c2cd0adfaf21204bf6732ca5d8ee34d3"},
+]
+chardet = [
+    {file = "chardet-3.0.4-py2.py3-none-any.whl", hash = "sha256:fc323ffcaeaed0e0a02bf4d117757b98aed530d9ed4531e3e15460124c106691"},
+    {file = "chardet-3.0.4.tar.gz", hash = "sha256:84ab92ed1c4d4f16916e05906b6b75a6c0fb5db821cc65e70cbd64a3e2a5eaae"},
+]
+colorama = [
+    {file = "colorama-0.4.3-py2.py3-none-any.whl", hash = "sha256:7d73d2a99753107a36ac6b455ee49046802e59d9d076ef8e47b61499fa29afff"},
+    {file = "colorama-0.4.3.tar.gz", hash = "sha256:e96da0d330793e2cb9485e9ddfd918d456036c7149416295932478192f4436a1"},
+]
+commonmark = [
+    {file = "commonmark-0.9.1-py2.py3-none-any.whl", hash = "sha256:da2f38c92590f83de410ba1a3cbceafbc74fee9def35f9251ba9a971d6d66fd9"},
+    {file = "commonmark-0.9.1.tar.gz", hash = "sha256:452f9dc859be7f06631ddcb328b6919c67984aca654e5fefb3914d54691aed60"},
+]
+docutils = [
+    {file = "docutils-0.16-py2.py3-none-any.whl", hash = "sha256:0c5b78adfbf7762415433f5515cd5c9e762339e23369dbe8000d84a4bf4ab3af"},
+    {file = "docutils-0.16.tar.gz", hash = "sha256:c2de3a60e9e7d07be26b7f2b00ca0309c207e06c100f9cc2a94931fc75a478fc"},
+]
+idna = [
+    {file = "idna-2.10-py2.py3-none-any.whl", hash = "sha256:b97d804b1e9b523befed77c48dacec60e6dcb0b5391d57af6a65a312a90648c0"},
+    {file = "idna-2.10.tar.gz", hash = "sha256:b307872f855b18632ce0c21c5e45be78c0ea7ae4c15c828c20788b26921eb3f6"},
+]
+imagesize = [
+    {file = "imagesize-1.2.0-py2.py3-none-any.whl", hash = "sha256:6965f19a6a2039c7d48bca7dba2473069ff854c36ae6f19d2cde309d998228a1"},
+    {file = "imagesize-1.2.0.tar.gz", hash = "sha256:b1f6b5a4eab1f73479a50fb79fcf729514a900c341d8503d62a62dbc4127a2b1"},
+]
+importlib-metadata = [
+    {file = "importlib_metadata-1.7.0-py2.py3-none-any.whl", hash = "sha256:dc15b2969b4ce36305c51eebe62d418ac7791e9a157911d58bfb1f9ccd8e2070"},
+    {file = "importlib_metadata-1.7.0.tar.gz", hash = "sha256:90bb658cdbbf6d1735b6341ce708fc7024a3e14e99ffdc5783edea9f9b077f83"},
+]
+jinja2 = [
+    {file = "Jinja2-2.11.2-py2.py3-none-any.whl", hash = "sha256:f0a4641d3cf955324a89c04f3d94663aa4d638abe8f733ecd3582848e1c37035"},
+    {file = "Jinja2-2.11.2.tar.gz", hash = "sha256:89aab215427ef59c34ad58735269eb58b1a5808103067f7bb9d5836c651b3bb0"},
+]
+livereload = [
+    {file = "livereload-2.6.2.tar.gz", hash = "sha256:d1eddcb5c5eb8d2ca1fa1f750e580da624c0f7fcb734aa5780dc81b7dcbd89be"},
+]
+markupsafe = [
+    {file = "MarkupSafe-1.1.1-cp27-cp27m-macosx_10_6_intel.whl", hash = "sha256:09027a7803a62ca78792ad89403b1b7a73a01c8cb65909cd876f7fcebd79b161"},
+    {file = "MarkupSafe-1.1.1-cp27-cp27m-manylinux1_i686.whl", hash = "sha256:e249096428b3ae81b08327a63a485ad0878de3fb939049038579ac0ef61e17e7"},
+    {file = "MarkupSafe-1.1.1-cp27-cp27m-manylinux1_x86_64.whl", hash = "sha256:500d4957e52ddc3351cabf489e79c91c17f6e0899158447047588650b5e69183"},
+    {file = "MarkupSafe-1.1.1-cp27-cp27m-win32.whl", hash = "sha256:b2051432115498d3562c084a49bba65d97cf251f5a331c64a12ee7e04dacc51b"},
+    {file = "MarkupSafe-1.1.1-cp27-cp27m-win_amd64.whl", hash = "sha256:98c7086708b163d425c67c7a91bad6e466bb99d797aa64f965e9d25c12111a5e"},
+    {file = "MarkupSafe-1.1.1-cp27-cp27mu-manylinux1_i686.whl", hash = "sha256:cd5df75523866410809ca100dc9681e301e3c27567cf498077e8551b6d20e42f"},
+    {file = "MarkupSafe-1.1.1-cp27-cp27mu-manylinux1_x86_64.whl", hash = "sha256:43a55c2930bbc139570ac2452adf3d70cdbb3cfe5912c71cdce1c2c6bbd9c5d1"},
+    {file = "MarkupSafe-1.1.1-cp34-cp34m-macosx_10_6_intel.whl", hash = "sha256:1027c282dad077d0bae18be6794e6b6b8c91d58ed8a8d89a89d59693b9131db5"},
+    {file = "MarkupSafe-1.1.1-cp34-cp34m-manylinux1_i686.whl", hash = "sha256:62fe6c95e3ec8a7fad637b7f3d372c15ec1caa01ab47926cfdf7a75b40e0eac1"},
+    {file = "MarkupSafe-1.1.1-cp34-cp34m-manylinux1_x86_64.whl", hash = "sha256:88e5fcfb52ee7b911e8bb6d6aa2fd21fbecc674eadd44118a9cc3863f938e735"},
+    {file = "MarkupSafe-1.1.1-cp34-cp34m-win32.whl", hash = "sha256:ade5e387d2ad0d7ebf59146cc00c8044acbd863725f887353a10df825fc8ae21"},
+    {file = "MarkupSafe-1.1.1-cp34-cp34m-win_amd64.whl", hash = "sha256:09c4b7f37d6c648cb13f9230d847adf22f8171b1ccc4d5682398e77f40309235"},
+    {file = "MarkupSafe-1.1.1-cp35-cp35m-macosx_10_6_intel.whl", hash = "sha256:79855e1c5b8da654cf486b830bd42c06e8780cea587384cf6545b7d9ac013a0b"},
+    {file = "MarkupSafe-1.1.1-cp35-cp35m-manylinux1_i686.whl", hash = "sha256:c8716a48d94b06bb3b2524c2b77e055fb313aeb4ea620c8dd03a105574ba704f"},
+    {file = "MarkupSafe-1.1.1-cp35-cp35m-manylinux1_x86_64.whl", hash = "sha256:7c1699dfe0cf8ff607dbdcc1e9b9af1755371f92a68f706051cc8c37d447c905"},
+    {file = "MarkupSafe-1.1.1-cp35-cp35m-win32.whl", hash = "sha256:6dd73240d2af64df90aa7c4e7481e23825ea70af4b4922f8ede5b9e35f78a3b1"},
+    {file = "MarkupSafe-1.1.1-cp35-cp35m-win_amd64.whl", hash = "sha256:9add70b36c5666a2ed02b43b335fe19002ee5235efd4b8a89bfcf9005bebac0d"},
+    {file = "MarkupSafe-1.1.1-cp36-cp36m-macosx_10_6_intel.whl", hash = "sha256:24982cc2533820871eba85ba648cd53d8623687ff11cbb805be4ff7b4c971aff"},
+    {file = "MarkupSafe-1.1.1-cp36-cp36m-manylinux1_i686.whl", hash = "sha256:00bc623926325b26bb9605ae9eae8a215691f33cae5df11ca5424f06f2d1f473"},
+    {file = "MarkupSafe-1.1.1-cp36-cp36m-manylinux1_x86_64.whl", hash = "sha256:717ba8fe3ae9cc0006d7c451f0bb265ee07739daf76355d06366154ee68d221e"},
+    {file = "MarkupSafe-1.1.1-cp36-cp36m-win32.whl", hash = "sha256:535f6fc4d397c1563d08b88e485c3496cf5784e927af890fb3c3aac7f933ec66"},
+    {file = "MarkupSafe-1.1.1-cp36-cp36m-win_amd64.whl", hash = "sha256:b1282f8c00509d99fef04d8ba936b156d419be841854fe901d8ae224c59f0be5"},
+    {file = "MarkupSafe-1.1.1-cp37-cp37m-macosx_10_6_intel.whl", hash = "sha256:8defac2f2ccd6805ebf65f5eeb132adcf2ab57aa11fdf4c0dd5169a004710e7d"},
+    {file = "MarkupSafe-1.1.1-cp37-cp37m-manylinux1_i686.whl", hash = "sha256:46c99d2de99945ec5cb54f23c8cd5689f6d7177305ebff350a58ce5f8de1669e"},
+    {file = "MarkupSafe-1.1.1-cp37-cp37m-manylinux1_x86_64.whl", hash = "sha256:ba59edeaa2fc6114428f1637ffff42da1e311e29382d81b339c1817d37ec93c6"},
+    {file = "MarkupSafe-1.1.1-cp37-cp37m-win32.whl", hash = "sha256:b00c1de48212e4cc9603895652c5c410df699856a2853135b3967591e4beebc2"},
+    {file = "MarkupSafe-1.1.1-cp37-cp37m-win_amd64.whl", hash = "sha256:9bf40443012702a1d2070043cb6291650a0841ece432556f784f004937f0f32c"},
+    {file = "MarkupSafe-1.1.1-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:6788b695d50a51edb699cb55e35487e430fa21f1ed838122d722e0ff0ac5ba15"},
+    {file = "MarkupSafe-1.1.1-cp38-cp38-manylinux1_i686.whl", hash = "sha256:cdb132fc825c38e1aeec2c8aa9338310d29d337bebbd7baa06889d09a60a1fa2"},
+    {file = "MarkupSafe-1.1.1-cp38-cp38-manylinux1_x86_64.whl", hash = "sha256:13d3144e1e340870b25e7b10b98d779608c02016d5184cfb9927a9f10c689f42"},
+    {file = "MarkupSafe-1.1.1-cp38-cp38-win32.whl", hash = "sha256:596510de112c685489095da617b5bcbbac7dd6384aeebeda4df6025d0256a81b"},
+    {file = "MarkupSafe-1.1.1-cp38-cp38-win_amd64.whl", hash = "sha256:e8313f01ba26fbbe36c7be1966a7b7424942f670f38e666995b88d012765b9be"},
+    {file = "MarkupSafe-1.1.1.tar.gz", hash = "sha256:29872e92839765e546828bb7754a68c418d927cd064fd4708fab9fe9c8bb116b"},
+]
+more-itertools = [
+    {file = "more-itertools-8.4.0.tar.gz", hash = "sha256:68c70cc7167bdf5c7c9d8f6954a7837089c6a36bf565383919bb595efb8a17e5"},
+    {file = "more_itertools-8.4.0-py3-none-any.whl", hash = "sha256:b78134b2063dd214000685165d81c154522c3ee0a1c0d4d113c80361c234c5a2"},
+]
+packaging = [
+    {file = "packaging-20.4-py2.py3-none-any.whl", hash = "sha256:998416ba6962ae7fbd6596850b80e17859a5753ba17c32284f67bfff33784181"},
+    {file = "packaging-20.4.tar.gz", hash = "sha256:4357f74f47b9c12db93624a82154e9b120fa8293699949152b22065d556079f8"},
+]
+pathtools = [
+    {file = "pathtools-0.1.2.tar.gz", hash = "sha256:7c35c5421a39bb82e58018febd90e3b6e5db34c5443aaaf742b3f33d4655f1c0"},
+]
+pluggy = [
+    {file = "pluggy-0.13.1-py2.py3-none-any.whl", hash = "sha256:966c145cd83c96502c3c3868f50408687b38434af77734af1e9ca461a4081d2d"},
+    {file = "pluggy-0.13.1.tar.gz", hash = "sha256:15b2acde666561e1298d71b523007ed7364de07029219b604cf808bfa1c765b0"},
+]
+port-for = [
+    {file = "port-for-0.3.1.tar.gz", hash = "sha256:b16a84bb29c2954db44c29be38b17c659c9c27e33918dec16b90d375cc596f1c"},
+]
+py = [
+    {file = "py-1.9.0-py2.py3-none-any.whl", hash = "sha256:366389d1db726cd2fcfc79732e75410e5fe4d31db13692115529d34069a043c2"},
+    {file = "py-1.9.0.tar.gz", hash = "sha256:9ca6883ce56b4e8da7e79ac18787889fa5206c79dcc67fb065376cd2fe03f342"},
+]
+pygments = [
+    {file = "Pygments-2.2.0-py2.py3-none-any.whl", hash = "sha256:78f3f434bcc5d6ee09020f92ba487f95ba50f1e3ef83ae96b9d5ffa1bab25c5d"},
+    {file = "Pygments-2.2.0.tar.gz", hash = "sha256:dbae1046def0efb574852fab9e90209b23f556367b5a320c0bcb871c77c3e8cc"},
+]
+pyparsing = [
+    {file = "pyparsing-2.4.7-py2.py3-none-any.whl", hash = "sha256:ef9d7589ef3c200abe66653d3f1ab1033c3c419ae9b9bdb1240a85b024efc88b"},
+    {file = "pyparsing-2.4.7.tar.gz", hash = "sha256:c203ec8783bf771a155b207279b9bccb8dea02d8f0c9e5f8ead507bc3246ecc1"},
+]
+pytest = [
+    {file = "pytest-5.4.3-py3-none-any.whl", hash = "sha256:5c0db86b698e8f170ba4582a492248919255fcd4c79b1ee64ace34301fb589a1"},
+    {file = "pytest-5.4.3.tar.gz", hash = "sha256:7979331bfcba207414f5e1263b5a0f8f521d0f457318836a7355531ed1a4c7d8"},
+]
+pytz = [
+    {file = "pytz-2020.1-py2.py3-none-any.whl", hash = "sha256:a494d53b6d39c3c6e44c3bec237336e14305e4f29bbf800b599253057fbb79ed"},
+    {file = "pytz-2020.1.tar.gz", hash = "sha256:c35965d010ce31b23eeb663ed3cc8c906275d6be1a34393a1d73a41febf4a048"},
+]
+pyyaml = [
+    {file = "PyYAML-5.3.1-cp27-cp27m-win32.whl", hash = "sha256:74809a57b329d6cc0fdccee6318f44b9b8649961fa73144a98735b0aaf029f1f"},
+    {file = "PyYAML-5.3.1-cp27-cp27m-win_amd64.whl", hash = "sha256:240097ff019d7c70a4922b6869d8a86407758333f02203e0fc6ff79c5dcede76"},
+    {file = "PyYAML-5.3.1-cp35-cp35m-win32.whl", hash = "sha256:4f4b913ca1a7319b33cfb1369e91e50354d6f07a135f3b901aca02aa95940bd2"},
+    {file = "PyYAML-5.3.1-cp35-cp35m-win_amd64.whl", hash = "sha256:cc8955cfbfc7a115fa81d85284ee61147059a753344bc51098f3ccd69b0d7e0c"},
+    {file = "PyYAML-5.3.1-cp36-cp36m-win32.whl", hash = "sha256:7739fc0fa8205b3ee8808aea45e968bc90082c10aef6ea95e855e10abf4a37b2"},
+    {file = "PyYAML-5.3.1-cp36-cp36m-win_amd64.whl", hash = "sha256:69f00dca373f240f842b2931fb2c7e14ddbacd1397d57157a9b005a6a9942648"},
+    {file = "PyYAML-5.3.1-cp37-cp37m-win32.whl", hash = "sha256:d13155f591e6fcc1ec3b30685d50bf0711574e2c0dfffd7644babf8b5102ca1a"},
+    {file = "PyYAML-5.3.1-cp37-cp37m-win_amd64.whl", hash = "sha256:73f099454b799e05e5ab51423c7bcf361c58d3206fa7b0d555426b1f4d9a3eaf"},
+    {file = "PyYAML-5.3.1-cp38-cp38-win32.whl", hash = "sha256:06a0d7ba600ce0b2d2fe2e78453a470b5a6e000a985dd4a4e54e436cc36b0e97"},
+    {file = "PyYAML-5.3.1-cp38-cp38-win_amd64.whl", hash = "sha256:95f71d2af0ff4227885f7a6605c37fd53d3a106fcab511b8860ecca9fcf400ee"},
+    {file = "PyYAML-5.3.1.tar.gz", hash = "sha256:b8eac752c5e14d3eca0e6dd9199cd627518cb5ec06add0de9d32baeee6fe645d"},
+]
+recommonmark = [
+    {file = "recommonmark-0.5.0-py2.py3-none-any.whl", hash = "sha256:c85228b9b7aea7157662520e74b4e8791c5eacd375332ec68381b52bf10165be"},
+    {file = "recommonmark-0.5.0.tar.gz", hash = "sha256:a520b8d25071a51ae23a27cf6252f2fe387f51bdc913390d83b2b50617f5bb48"},
+]
+requests = [
+    {file = "requests-2.24.0-py2.py3-none-any.whl", hash = "sha256:fe75cc94a9443b9246fc7049224f75604b113c36acb93f87b80ed42c44cbb898"},
+    {file = "requests-2.24.0.tar.gz", hash = "sha256:b3559a131db72c33ee969480840fff4bb6dd111de7dd27c8ee1f820f4f00231b"},
+]
+six = [
+    {file = "six-1.15.0-py2.py3-none-any.whl", hash = "sha256:8b74bedcbbbaca38ff6d7491d76f2b06b3592611af620f8426e82dddb04a5ced"},
+    {file = "six-1.15.0.tar.gz", hash = "sha256:30639c035cdb23534cd4aa2dd52c3bf48f06e5f4a941509c8bafd8ce11080259"},
+]
+snowballstemmer = [
+    {file = "snowballstemmer-2.0.0-py2.py3-none-any.whl", hash = "sha256:209f257d7533fdb3cb73bdbd24f436239ca3b2fa67d56f6ff88e86be08cc5ef0"},
+    {file = "snowballstemmer-2.0.0.tar.gz", hash = "sha256:df3bac3df4c2c01363f3dd2cfa78cce2840a79b9f1c2d2de9ce8d31683992f52"},
+]
+sphinx = [
+    {file = "Sphinx-2.4.4-py3-none-any.whl", hash = "sha256:fc312670b56cb54920d6cc2ced455a22a547910de10b3142276495ced49231cb"},
+    {file = "Sphinx-2.4.4.tar.gz", hash = "sha256:b4c750d546ab6d7e05bdff6ac24db8ae3e8b8253a3569b754e445110a0a12b66"},
+]
+sphinx-autobuild = [
+    {file = "sphinx-autobuild-0.7.1.tar.gz", hash = "sha256:66388f81884666e3821edbe05dd53a0cfb68093873d17320d0610de8db28c74e"},
+    {file = "sphinx_autobuild-0.7.1-py2-none-any.whl", hash = "sha256:e60aea0789cab02fa32ee63c7acae5ef41c06f1434d9fd0a74250a61f5994692"},
+]
+sphinx-copybutton = [
+    {file = "sphinx-copybutton-0.2.12.tar.gz", hash = "sha256:9492883786984b6179c92c07ab0410237b26efa826adfa792acfd17b91a63e5c"},
+    {file = "sphinx_copybutton-0.2.12-py3-none-any.whl", hash = "sha256:517870030a931f313695705edbe14a8c30660829716100d3d24b379cf9257060"},
+]
+sphinx-multiversion = [
+    {file = "sphinx-multiversion-0.2.3.tar.gz", hash = "sha256:e46565ac2f703f3b55652f33c159c8059865f5d13dae7f0e8403e5afc2996f5f"},
+    {file = "sphinx_multiversion-0.2.3-py3-none-any.whl", hash = "sha256:dc0f18449122e3e2a61245771bfdb7fa83df4f6adbf8eafea31f5b0cfccb5dbe"},
+]
+sphinx-scylladb-theme = [
+    {file = "sphinx-scylladb-theme-0.1.9.tar.gz", hash = "sha256:2a2abaccedb3e00e57f412e35afda042c4a5d9baa66c288feb82362d9889294b"},
+    {file = "sphinx_scylladb_theme-0.1.9-py3-none-any.whl", hash = "sha256:50b3407bb1d2432f809ca48b7b20fdb124446c622584ddc04e8a390ce59f95b4"},
+]
+sphinx-sitemap = [
+    {file = "sphinx-sitemap-2.1.0.tar.gz", hash = "sha256:ba3576f38e8d1cabde483dabed2cc2b958af85ff97f2a4442f990b02e967d6a5"},
+]
+sphinx-tabs = [
+    {file = "sphinx-tabs-1.1.13.tar.gz", hash = "sha256:7ad881daa4d18799b254db4aa7feeb9d30256cbccf7d4f3de746d9fcc14e0196"},
+]
+sphinxcontrib-applehelp = [
+    {file = "sphinxcontrib-applehelp-1.0.2.tar.gz", hash = "sha256:a072735ec80e7675e3f432fcae8610ecf509c5f1869d17e2eecff44389cdbc58"},
+    {file = "sphinxcontrib_applehelp-1.0.2-py2.py3-none-any.whl", hash = "sha256:806111e5e962be97c29ec4c1e7fe277bfd19e9652fb1a4392105b43e01af885a"},
+]
+sphinxcontrib-devhelp = [
+    {file = "sphinxcontrib-devhelp-1.0.2.tar.gz", hash = "sha256:ff7f1afa7b9642e7060379360a67e9c41e8f3121f2ce9164266f61b9f4b338e4"},
+    {file = "sphinxcontrib_devhelp-1.0.2-py2.py3-none-any.whl", hash = "sha256:8165223f9a335cc1af7ffe1ed31d2871f325254c0423bc0c4c7cd1c1e4734a2e"},
+]
+sphinxcontrib-htmlhelp = [
+    {file = "sphinxcontrib-htmlhelp-1.0.3.tar.gz", hash = "sha256:e8f5bb7e31b2dbb25b9cc435c8ab7a79787ebf7f906155729338f3156d93659b"},
+    {file = "sphinxcontrib_htmlhelp-1.0.3-py2.py3-none-any.whl", hash = "sha256:3c0bc24a2c41e340ac37c85ced6dafc879ab485c095b1d65d2461ac2f7cca86f"},
+]
+sphinxcontrib-jsmath = [
+    {file = "sphinxcontrib-jsmath-1.0.1.tar.gz", hash = "sha256:a9925e4a4587247ed2191a22df5f6970656cb8ca2bd6284309578f2153e0c4b8"},
+    {file = "sphinxcontrib_jsmath-1.0.1-py2.py3-none-any.whl", hash = "sha256:2ec2eaebfb78f3f2078e73666b1415417a116cc848b72e5172e596c871103178"},
+]
+sphinxcontrib-qthelp = [
+    {file = "sphinxcontrib-qthelp-1.0.3.tar.gz", hash = "sha256:4c33767ee058b70dba89a6fc5c1892c0d57a54be67ddd3e7875a18d14cba5a72"},
+    {file = "sphinxcontrib_qthelp-1.0.3-py2.py3-none-any.whl", hash = "sha256:bd9fc24bcb748a8d51fd4ecaade681350aa63009a347a8c14e637895444dfab6"},
+]
+sphinxcontrib-serializinghtml = [
+    {file = "sphinxcontrib-serializinghtml-1.1.4.tar.gz", hash = "sha256:eaa0eccc86e982a9b939b2b82d12cc5d013385ba5eadcc7e4fed23f4405f77bc"},
+    {file = "sphinxcontrib_serializinghtml-1.1.4-py2.py3-none-any.whl", hash = "sha256:f242a81d423f59617a8e5cf16f5d4d74e28ee9a66f9e5b637a18082991db5a9a"},
+]
+tornado = [
+    {file = "tornado-6.0.4-cp35-cp35m-win32.whl", hash = "sha256:5217e601700f24e966ddab689f90b7ea4bd91ff3357c3600fa1045e26d68e55d"},
+    {file = "tornado-6.0.4-cp35-cp35m-win_amd64.whl", hash = "sha256:c98232a3ac391f5faea6821b53db8db461157baa788f5d6222a193e9456e1740"},
+    {file = "tornado-6.0.4-cp36-cp36m-win32.whl", hash = "sha256:5f6a07e62e799be5d2330e68d808c8ac41d4a259b9cea61da4101b83cb5dc673"},
+    {file = "tornado-6.0.4-cp36-cp36m-win_amd64.whl", hash = "sha256:c952975c8ba74f546ae6de2e226ab3cc3cc11ae47baf607459a6728585bb542a"},
+    {file = "tornado-6.0.4-cp37-cp37m-win32.whl", hash = "sha256:2c027eb2a393d964b22b5c154d1a23a5f8727db6fda837118a776b29e2b8ebc6"},
+    {file = "tornado-6.0.4-cp37-cp37m-win_amd64.whl", hash = "sha256:5618f72e947533832cbc3dec54e1dffc1747a5cb17d1fd91577ed14fa0dc081b"},
+    {file = "tornado-6.0.4-cp38-cp38-win32.whl", hash = "sha256:22aed82c2ea340c3771e3babc5ef220272f6fd06b5108a53b4976d0d722bcd52"},
+    {file = "tornado-6.0.4-cp38-cp38-win_amd64.whl", hash = "sha256:c58d56003daf1b616336781b26d184023ea4af13ae143d9dda65e31e534940b9"},
+    {file = "tornado-6.0.4.tar.gz", hash = "sha256:0fe2d45ba43b00a41cd73f8be321a44936dc1aba233dee979f17a042b83eb6dc"},
+]
+urllib3 = [
+    {file = "urllib3-1.25.10-py2.py3-none-any.whl", hash = "sha256:e7983572181f5e1522d9c98453462384ee92a0be7fac5f1413a1e35c56cc0461"},
+    {file = "urllib3-1.25.10.tar.gz", hash = "sha256:91056c15fa70756691db97756772bb1eb9678fa585d9184f24534b100dc60f4a"},
+]
+watchdog = [
+    {file = "watchdog-0.10.3.tar.gz", hash = "sha256:4214e1379d128b0588021880ccaf40317ee156d4603ac388b9adcf29165e0c04"},
+]
+wcwidth = [
+    {file = "wcwidth-0.2.5-py2.py3-none-any.whl", hash = "sha256:beb4802a9cebb9144e99086eff703a642a13d6a0052920003a230f3294bbe784"},
+    {file = "wcwidth-0.2.5.tar.gz", hash = "sha256:c4d647b99872929fdb7bdcaa4fbe7f01413ed3d98077df798530e5b04f116c83"},
+]
+zipp = [
+    {file = "zipp-3.1.0-py3-none-any.whl", hash = "sha256:aa36550ff0c0b7ef7fa639055d797116ee891440eac1a56f378e2d3179e0320b"},
+    {file = "zipp-3.1.0.tar.gz", hash = "sha256:c599e4d75c98f6798c509911d08a22e6c021d074469042177c8c86fb92eefd96"},
+]
diff --git a/docs/pyproject.toml b/docs/pyproject.toml
new file mode 100644
index 00000000000..d1cb108b855
--- /dev/null
+++ b/docs/pyproject.toml
@@ -0,0 +1,23 @@
+[tool.poetry]
+name = "java-driver"
+version = "3.7.1"
+description = "ScyllaDB Java Driver"
+authors = ["Java Driver Contributors"]
+
+[tool.poetry.dependencies]
+python = "^3.7"
+pyyaml = "^5.3"
+pygments = "2.2.0"
+recommonmark = "0.5.0"
+sphinx-scylladb-theme = "0.1.9"
+sphinx-sitemap = "2.1.0"
+sphinx-autobuild = "^0.7.1"
+Sphinx = "2.4.4"
+sphinx-multiversion = "0.2.3"
+
+[tool.poetry.dev-dependencies]
+pytest = "^5.2"
+
+[build-system]
+requires = ["poetry>=0.12"]
+build-backend = "poetry.masonry.api" 
diff --git a/docs/source/api.rst b/docs/source/api.rst
new file mode 100644
index 00000000000..850fb13929f
--- /dev/null
+++ b/docs/source/api.rst
@@ -0,0 +1,3 @@
+API Documentation
+=================
+
diff --git a/docs/source/changelog/index.md b/docs/source/changelog/index.md
new file mode 100644
index 00000000000..0429531d8ce
--- /dev/null
+++ b/docs/source/changelog/index.md
@@ -0,0 +1,1624 @@
+## Changelog
+
+### 3.7.1
+
+- ``[bug]`` JAVA-2174: Metadata.needsQuote should accept empty strings.
+
+
+### 3.7.0
+
+- ``[improvement]`` JAVA-2025: Include exception message in Abstract*Codec.accepts(null).
+- ``[improvement]`` JAVA-1980: Use covariant return types in RemoteEndpointAwareJdkSSLOptions.Builder methods.
+- ``[documentation]`` JAVA-2062: Document frozen collection preference with Mapper.
+- ``[bug]`` JAVA-2071: Fix NPE in ArrayBackedRow.toString().
+- ``[bug]`` JAVA-2070: Call onRemove instead of onDown when rack and/or DC information changes for a host.
+- ``[improvement]`` JAVA-1256: Log parameters of BuiltStatement in QueryLogger.
+- ``[documentation]`` JAVA-2074: Document preference for LZ4 over Snappy.
+- ``[bug]`` JAVA-1612: Include netty-common jar in binary tarball.
+- ``[improvement]`` JAVA-2003: Simplify CBUtil internal API to improve performance.
+- ``[improvement]`` JAVA-2002: Reimplement TypeCodec.accepts to improve performance.
+- ``[documentation]`` JAVA-2041: Deprecate cross-DC failover in DCAwareRoundRobinPolicy.
+- ``[documentation]`` JAVA-1159: Document workaround for using tuple with udt field in Mapper.
+- ``[documentation]`` JAVA-1964: Complete remaining "Coming Soon" sections in docs.
+- ``[improvement]`` JAVA-1950: Log server side warnings returned from a query.
+- ``[improvement]`` JAVA-2123: Allow to use QueryBuilder for building queries against Materialized Views.
+- ``[bug]`` JAVA-2082: Avoid race condition during cluster close and schema refresh.
+
+
+### 3.6.0
+
+- ``[improvement]`` JAVA-1394: Add request-queue-depth metric.
+- ``[improvement]`` JAVA-1857: Add Statement.setHost.
+- ``[bug]`` JAVA-1920: Use nanosecond precision in LocalTimeCodec#format().
+- ``[bug]`` JAVA-1794: Driver tries to create a connection array of size -1.
+- ``[new feature]`` JAVA-1899: Support virtual tables.
+- ``[bug]`` JAVA-1908: TableMetadata.asCQLQuery does not add table option 'memtable_flush_period_in_ms' in the generated query.
+- ``[bug]`` JAVA-1924: StatementWrapper setters should return the wrapping statement.
+- ``[new feature]`` JAVA-1532: Add Codec support for Java 8's LocalDateTime and ZoneId.
+- ``[improvement]`` JAVA-1786: Use Google code formatter.
+- ``[bug]`` JAVA-1871: Change LOCAL_SERIAL.isDCLocal() to return true.
+- ``[documentation]`` JAVA-1902: Clarify unavailable & request error in DefaultRetryPolicy javadoc.
+- ``[new feature]`` JAVA-1903: Add WhiteListPolicy.ofHosts.
+- ``[bug]`` JAVA-1928: Fix GuavaCompatibility for Guava 26.
+- ``[bug]`` JAVA-1935: Add null check in QueryConsistencyException.getHost.
+- ``[improvement]`` JAVA-1771: Send driver name and version in STARTUP message.
+- ``[improvement]`` JAVA-1388: Add dynamic port discovery for system.peers_v2.
+- ``[documentation]`` JAVA-1810: Note which setters are not propagated to PreparedStatement.
+- ``[bug]`` JAVA-1944: Surface Read and WriteFailureException to RetryPolicy.
+- ``[bug]`` JAVA-1211: Fix NPE in cluster close when cluster init fails.
+- ``[bug]`` JAVA-1220: Fail fast on cluster init if previous init failed.
+- ``[bug]`` JAVA-1929: Preempt session execute queries if session was closed.
+
+Merged from 3.5.x:
+
+- ``[bug]`` JAVA-1872: Retain table's views when processing table update.
+
+
+### 3.5.0
+
+- ``[improvement]`` JAVA-1448: TokenAwarePolicy should respect child policy ordering.
+- ``[bug]`` JAVA-1751: Include defaultTimestamp length in encodedSize for protocol version >= 3.
+- ``[bug]`` JAVA-1770: Fix message size when using Custom Payload.
+- ``[documentation]`` JAVA-1760: Add metrics documentation.
+- ``[improvement]`` JAVA-1765: Update dependencies to latest patch versions.
+- ``[improvement]`` JAVA-1752: Deprecate DowngradingConsistencyRetryPolicy.
+- ``[improvement]`` JAVA-1735: Log driver version on first use.
+- ``[documentation]`` JAVA-1380: Add FAQ entry for errors arising from incompatibilities.
+- ``[improvement]`` JAVA-1748: Support IS NOT NULL and != in query builder.
+- ``[documentation]`` JAVA-1740: Mention C*2.2/3.0 incompatibilities in paging state manual.
+- ``[improvement]`` JAVA-1725: Add a getNodeCount method to CCMAccess for easier automation.
+- ``[new feature]`` JAVA-708: Add means to measure request sizes.
+- ``[documentation]`` JAVA-1788: Add example for enabling host name verification to SSL docs.
+- ``[improvement]`` JAVA-1791: Revert "JAVA-1677: Warn if auth is configured on the client but not the server."
+- ``[bug]`` JAVA-1789: Account for flags in Prepare encodedSize.
+- ``[bug]`` JAVA-1797: Use jnr-ffi version required by jnr-posix.
+
+
+### 3.4.0
+
+- ``[improvement]`` JAVA-1671: Remove unnecessary test on prepared statement metadata.
+- ``[bug]`` JAVA-1694: Upgrade to jackson-databind 2.7.9.2 to address CVE-2015-15095.
+- ``[documentation]`` JAVA-1685: Clarify recommendation on preparing SELECT *.
+- ``[improvement]`` JAVA-1679: Improve error message on batch log write timeout.
+- ``[improvement]`` JAVA-1672: Remove schema agreement check when repreparing on up.
+- ``[improvement]`` JAVA-1677: Warn if auth is configured on the client but not the server.
+- ``[new feature]`` JAVA-1651: Add NO_COMPACT startup option.
+- ``[improvement]`` JAVA-1683: Add metrics to track writes to nodes.
+- ``[new feature]`` JAVA-1229: Allow specifying the keyspace for individual queries.
+- ``[improvement]`` JAVA-1682: Provide a way to record latencies for cancelled speculative executions.
+- ``[improvement]`` JAVA-1717: Add metrics to latency-aware policy.
+- ``[improvement]`` JAVA-1675: Remove dates from copyright headers.
+
+Merged from 3.3.x:
+
+- ``[bug]`` JAVA-1555: Include VIEW and CDC in WriteType.
+- ``[bug]`` JAVA-1599: exportAsString improvements (sort, format, clustering order)
+- ``[improvement]`` JAVA-1587: Deterministic ordering of columns used in Mapper#saveQuery
+- ``[improvement]`` JAVA-1500: Add a metric to report number of in-flight requests.
+- ``[bug]`` JAVA-1438: QueryBuilder check for empty orderings.
+- ``[improvement]`` JAVA-1490: Allow zero delay for speculative executions.
+- ``[documentation]`` JAVA-1607: Add FAQ entry for netty-transport-native-epoll.
+- ``[bug]`` JAVA-1630: Fix Metadata.addIfAbsent.
+- ``[improvement]`` JAVA-1619: Update QueryBuilder methods to support Iterable input.
+- ``[improvement]`` JAVA-1527: Expose host_id and schema_version on Host metadata.
+- ``[new feature]`` JAVA-1377: Add support for TWCS in SchemaBuilder.
+- ``[improvement]`` JAVA-1631: Publish a sources jar for driver-core-tests.
+- ``[improvement]`` JAVA-1632: Add a withIpPrefix(String) method to CCMBridge.Builder.
+- ``[bug]`` JAVA-1639: VersionNumber does not fullfill equals/hashcode contract.
+- ``[bug]`` JAVA-1613: Fix broken shaded Netty detection in NettyUtil.
+- ``[bug]`` JAVA-1666: Fix keyspace export when a UDT has case-sensitive field names.
+- ``[improvement]`` JAVA-1196: Include hash of result set metadata in prepared statement id.
+- ``[improvement]`` JAVA-1670: Support user-provided JMX ports for CCMBridge.
+- ``[improvement]`` JAVA-1661: Avoid String.toLowerCase if possible in Metadata.
+- ``[improvement]`` JAVA-1659: Expose low-level flusher tuning options.
+- ``[improvement]`` JAVA-1660: Support netty-transport-native-epoll in OSGi container.
+
+
+### 3.3.2
+
+- ``[bug]`` JAVA-1666: Fix keyspace export when a UDT has case-sensitive field names.
+- ``[improvement]`` JAVA-1196: Include hash of result set metadata in prepared statement id.
+- ``[improvement]`` JAVA-1670: Support user-provided JMX ports for CCMBridge.
+- ``[improvement]`` JAVA-1661: Avoid String.toLowerCase if possible in Metadata.
+- ``[improvement]`` JAVA-1659: Expose low-level flusher tuning options.
+- ``[improvement]`` JAVA-1660: Support netty-transport-native-epoll in OSGi container.
+
+
+### 3.3.1
+
+- ``[bug]`` JAVA-1555: Include VIEW and CDC in WriteType.
+- ``[bug]`` JAVA-1599: exportAsString improvements (sort, format, clustering order)
+- ``[improvement]`` JAVA-1587: Deterministic ordering of columns used in Mapper#saveQuery
+- ``[improvement]`` JAVA-1500: Add a metric to report number of in-flight requests.
+- ``[bug]`` JAVA-1438: QueryBuilder check for empty orderings.
+- ``[improvement]`` JAVA-1490: Allow zero delay for speculative executions.
+- ``[documentation]`` JAVA-1607: Add FAQ entry for netty-transport-native-epoll.
+- ``[bug]`` JAVA-1630: Fix Metadata.addIfAbsent.
+- ``[improvement]`` JAVA-1619: Update QueryBuilder methods to support Iterable input.
+- ``[improvement]`` JAVA-1527: Expose host_id and schema_version on Host metadata.
+- ``[new feature]`` JAVA-1377: Add support for TWCS in SchemaBuilder.
+- ``[improvement]`` JAVA-1631: Publish a sources jar for driver-core-tests.
+- ``[improvement]`` JAVA-1632: Add a withIpPrefix(String) method to CCMBridge.Builder.
+- ``[bug]`` JAVA-1639: VersionNumber does not fullfill equals/hashcode contract.
+- ``[bug]`` JAVA-1613: Fix broken shaded Netty detection in NettyUtil.
+
+
+### 3.3.0
+
+- ``[bug]`` JAVA-1469: Update LoggingRetryPolicy to deal with SLF4J-353.
+- ``[improvement]`` JAVA-1203: Upgrade Metrics to allow usage in OSGi.
+- ``[bug]`` JAVA-1407: KeyspaceMetadata exportAsString should export user types in topological sort order.
+- ``[bug]`` JAVA-1455: Mapper support using unset for null values.
+- ``[bug]`` JAVA-1464: Allow custom codecs with non public constructors in @Param.
+- ``[bug]`` JAVA-1470: Querying multiple pages overrides WrappedStatement.
+- ``[improvement]`` JAVA-1428: Upgrade logback and jackson dependencies.
+- ``[documentation]`` JAVA-1463: Revisit speculative execution docs.
+- ``[documentation]`` JAVA-1466: Revisit timestamp docs.
+- ``[documentation]`` JAVA-1445: Clarify how nodes are penalized in LatencyAwarePolicy docs.
+- ``[improvement]`` JAVA-1446: Support 'DEFAULT UNSET' in Query Builder JSON Insert.
+- ``[improvement]`` JAVA-1443: Add groupBy method to Select statement.
+- ``[improvement]`` JAVA-1458: Check thread in mapper sync methods.
+- ``[improvement]`` JAVA-1488: Upgrade Netty to 4.0.47.Final.
+- ``[improvement]`` JAVA-1460: Add speculative execution number to ExecutionInfo
+- ``[improvement]`` JAVA-1431: Improve error handling during pool initialization.
+
+
+### 3.2.0
+
+- ``[new feature]`` JAVA-1347: Add support for duration type.
+- ``[new feature]`` JAVA-1248: Implement "beta" flag for native protocol v5.
+- ``[new feature]`` JAVA-1362: Send query options flags as ``[int]`` for Protocol V5+.
+- ``[new feature]`` JAVA-1364: Enable creation of SSLHandler with remote address information.
+- ``[improvement]`` JAVA-1367: Make protocol negotiation more resilient.
+- ``[bug]`` JAVA-1397: Handle duration as native datatype in protocol v5+.
+- ``[improvement]`` JAVA-1308: CodecRegistry performance improvements.
+- ``[improvement]`` JAVA-1287: Add CDC to TableOptionsMetadata and Schema Builder.
+- ``[improvement]`` JAVA-1392: Reduce lock contention in RPTokenFactory.
+- ``[improvement]`` JAVA-1328: Provide compatibility with Guava 20.
+- ``[improvement]`` JAVA-1247: Disable idempotence warnings.
+- ``[improvement]`` JAVA-1286: Support setting and retrieving udt fields in QueryBuilder.
+- ``[bug]`` JAVA-1415: Correctly report if a UDT column is frozen.
+- ``[bug]`` JAVA-1418: Make Guava version detection more reliable.
+- ``[new feature]`` JAVA-1174: Add ifNotExists option to mapper.
+- ``[improvement]`` JAVA-1414: Optimize Metadata.escapeId and Metadata.handleId.
+- ``[improvement]`` JAVA-1310: Make mapper's ignored properties configurable.
+- ``[improvement]`` JAVA-1316: Add strategy for resolving properties into CQL names.
+- ``[bug]`` JAVA-1424: Handle new WRITE_FAILURE and READ_FAILURE format in v5 protocol.
+
+Merged from 3.1.x branch:
+
+- ``[bug]`` JAVA-1371: Reintroduce connection pool timeout.
+- ``[bug]`` JAVA-1313: Copy SerialConsistencyLevel to PreparedStatement.
+- ``[documentation]`` JAVA-1334: Clarify documentation of method `addContactPoints`.
+- ``[improvement]`` JAVA-1357: Document that getReplicas only returns replicas of the last token in range.
+- ``[bug]`` JAVA-1404: Fix min token handling in TokenRange.contains.
+- ``[bug]`` JAVA-1429: Prevent heartbeats until connection is fully initialized.
+
+
+### 3.1.4
+
+Merged from 3.0.x branch:
+
+- ``[bug]`` JAVA-1371: Reintroduce connection pool timeout.
+- ``[bug]`` JAVA-1313: Copy SerialConsistencyLevel to PreparedStatement.
+- ``[documentation]`` JAVA-1334: Clarify documentation of method `addContactPoints`.
+- ``[improvement]`` JAVA-1357: Document that getReplicas only returns replicas of the last token in range.
+
+
+### 3.1.3
+
+Merged from 3.0.x branch:
+
+- ``[bug]`` JAVA-1330: Add un/register for SchemaChangeListener in DelegatingCluster
+- ``[bug]`` JAVA-1351: Include Custom Payload in Request.copy.
+- ``[bug]`` JAVA-1346: Reset heartbeat only on client reads (not writes).
+- ``[improvement]`` JAVA-866: Support tuple notation in QueryBuilder.eq/in.
+
+
+### 3.1.2
+
+- ``[bug]`` JAVA-1321: Wrong OSGi dependency version for Guava.
+
+Merged from 3.0.x branch:
+
+- ``[bug]`` JAVA-1312: QueryBuilder modifies selected columns when manually selected.
+- ``[improvement]`` JAVA-1303: Add missing BoundStatement.setRoutingKey(ByteBuffer...)
+- ``[improvement]`` JAVA-262: Make internal executors customizable
+
+
+### 3.1.1
+
+- ``[bug]`` JAVA-1284: ClockFactory should check system property before attempting to load Native class.
+- ``[bug]`` JAVA-1255: Allow nested UDTs to be used in Mapper.
+- ``[bug]`` JAVA-1279: Mapper should exclude Groovy's "metaClass" property when looking for mapped properties
+
+Merged from 3.0.x branch:
+
+- ``[improvement]`` JAVA-1246: Driver swallows the real exception in a few cases
+- ``[improvement]`` JAVA-1261: Throw error when attempting to page in I/O thread.
+- ``[bug]`` JAVA-1258: Regression: Mapper cannot map a materialized view after JAVA-1126.
+- ``[bug]`` JAVA-1101: Batch and BatchStatement should consider inner statements to determine query idempotence
+- ``[improvement]`` JAVA-1262: Use ParseUtils for quoting & unquoting.
+- ``[improvement]`` JAVA-1275: Use Netty's default thread factory
+- ``[bug]`` JAVA-1285: QueryBuilder routing key auto-discovery should handle case-sensitive column names.
+- ``[bug]`` JAVA-1283: Don't cache failed query preparations in the mapper.
+- ``[improvement]`` JAVA-1277: Expose AbstractSession.checkNotInEventLoop.
+- ``[bug]`` JAVA-1272: BuiltStatement not able to print its query string if it contains mapped UDTs.
+- ``[bug]`` JAVA-1292: 'Adjusted frame length' error breaks driver's ability to read data.
+- ``[improvement]`` JAVA-1293: Make DecoderForStreamIdSize.MAX_FRAME_LENGTH configurable.
+- ``[improvement]`` JAVA-1053: Add a metric for authentication errors
+- ``[improvement]`` JAVA-1263: Eliminate unnecessary memory copies in FrameCompressor implementations.
+- ``[improvement]`` JAVA-893: Make connection pool non-blocking
+
+
+### 3.1.0
+
+- ``[new feature]`` JAVA-1153: Add PER PARTITION LIMIT to Select QueryBuilder.
+- ``[improvement]`` JAVA-743: Add JSON support to QueryBuilder.
+- ``[improvement]`` JAVA-1233: Update HdrHistogram to 2.1.9.
+- ``[improvement]`` JAVA-1233: Update Snappy to 1.1.2.6.
+- ``[bug]`` JAVA-1161: Preserve full time zone info in ZonedDateTimeCodec and DateTimeCodec.
+- ``[new feature]`` JAVA-1157: Allow asynchronous paging of Mapper Result.
+- ``[improvement]`` JAVA-1212: Don't retry non-idempotent statements by default.
+- ``[improvement]`` JAVA-1192: Make EventDebouncer settings updatable at runtime.
+- ``[new feature]`` JAVA-541: Add polymorphism support to object mapper.
+- ``[new feature]`` JAVA-636: Allow @Column annotations on getters/setters as well as fields.
+- ``[new feature]`` JAVA-984: Allow non-void setters in object mapping.
+- ``[new feature]`` JAVA-1055: Add ErrorAware load balancing policy.
+
+Merged from 3.0.x branch:
+
+- ``[bug]`` JAVA-1179: Request objects should be copied when executed.
+- ``[improvement]`` JAVA-1182: Throw error when synchronous call made on I/O thread.
+- ``[bug]`` JAVA-1184: Unwrap StatementWrappers when extracting column definitions.
+- ``[bug]`` JAVA-1132: Executing bound statement with no variables results in exception with protocol v1.
+- ``[improvement]`` JAVA-1040: SimpleStatement parameters support in QueryLogger.
+- ``[improvement]`` JAVA-1151: Fail fast if HdrHistogram is not in the classpath.
+- ``[improvement]`` JAVA-1154: Allow individual Statement to cancel the read timeout.
+- ``[bug]`` JAVA-1074: Fix documentation around default timestamp generator.
+- ``[improvement]`` JAVA-1109: Document SSLOptions changes in upgrade guide.
+- ``[improvement]`` JAVA-1065: Add method to create token from partition key values.
+- ``[improvement]`` JAVA-1136: Enable JDK signature check in module driver-extras.
+- ``[improvement]`` JAVA-866: Support tuple notation in QueryBuilder.eq/in.
+- ``[bug]`` JAVA-1140: Use same connection to check for schema agreement after a DDL query.
+- ``[improvement]`` JAVA-1113: Support Cassandra 3.4 LIKE operator in QueryBuilder.
+- ``[improvement]`` JAVA-1086: Support Cassandra 3.2 CAST function in QueryBuilder.
+- ``[bug]`` JAVA-1095: Check protocol version for custom payload before sending the query.
+- ``[improvement]`` JAVA-1133: Add OSGi headers to cassandra-driver-extras.
+- ``[bug]`` JAVA-1137: Incorrect string returned by DataType.asFunctionParameterString() for collections and tuples.
+- ``[bug]`` JAVA-1046: (Dynamic)CompositeTypes need to be parsed as string literal, not blob.
+- ``[improvement]`` JAVA-1164: Clarify documentation on Host.listenAddress and broadcastAddress.
+- ``[improvement]`` JAVA-1171: Add Host method to determine if DSE Graph is enabled.
+- ``[improvement]`` JAVA-1069: Bootstrap driver-examples module.
+- ``[documentation]`` JAVA-1150: Add example and FAQ entry about ByteBuffer/BLOB.
+- ``[improvement]`` JAVA-1011: Expose PoolingOptions default values.
+- ``[improvement]`` JAVA-630: Don't process DOWN events for nodes that have active connections.
+- ``[improvement]`` JAVA-851: Improve UUIDs javadoc with regard to user-provided timestamps.
+- ``[improvement]`` JAVA-979: Update javadoc for RegularStatement toString() and getQueryString() to indicate that consistency level and other parameters are not maintained in the query string.
+- ``[bug]`` JAVA-1068: Unwrap StatementWrappers when hashing the paging state.
+- ``[improvement]`` JAVA-1021: Improve error message when connect() is called with an invalid keyspace name.
+- ``[improvement]`` JAVA-879: Mapper.map() accepts mapper-generated and user queries.
+- ``[bug]`` JAVA-1100: Exception when connecting with shaded java driver in OSGI
+- ``[bug]`` JAVA-1064: getTable create statement doesn't properly handle quotes in primary key.
+- ``[bug]`` JAVA-1089: Set LWT made from BuiltStatements to non-idempotent.
+- ``[improvement]`` JAVA-923: Position idempotent flag on object mapper queries.
+- ``[bug]`` JAVA-1070: The Mapper should not prepare queries synchronously.
+- ``[new feature]`` JAVA-982: Introduce new method ConsistencyLevel.isSerial().
+- ``[bug]`` JAVA-764: Retry with the normal consistency level (not the serial one) when a write times out on the Paxos phase.
+- ``[improvement]`` JAVA-852: Ignore peers with null entries during discovery.
+- ``[bug]`` JAVA-1005: DowngradingConsistencyRetryPolicy does not work with EACH_QUORUM when 1 DC is down.
+- ``[bug]`` JAVA-1002: Avoid deadlock when re-preparing a statement on other hosts.
+- ``[bug]`` JAVA-1072: Ensure defunct connections are properly evicted from the pool.
+- ``[bug]`` JAVA-1152: Fix NPE at ControlConnection.refreshNodeListAndTokenMap().
+
+Merged from 2.1 branch:
+
+- ``[improvement]`` JAVA-1038: Fetch node info by rpc_address if its broadcast_address is not in system.peers.
+- ``[improvement]`` JAVA-888: Add cluster-wide percentile tracker.
+- ``[improvement]`` JAVA-963: Automatically register PercentileTracker from components that use it.
+- ``[new feature]`` JAVA-1019: SchemaBuilder support for CREATE/ALTER/DROP KEYSPACE.
+- ``[bug]`` JAVA-727: Allow monotonic timestamp generators to drift in the future + use microsecond precision when possible.
+- ``[improvement]`` JAVA-444: Add Java process information to UUIDs.makeNode() hash.
+
+
+### 3.0.7
+
+- ``[bug]`` JAVA-1371: Reintroduce connection pool timeout.
+- ``[bug]`` JAVA-1313: Copy SerialConsistencyLevel to PreparedStatement.
+- ``[documentation]`` JAVA-1334: Clarify documentation of method `addContactPoints`.
+- ``[improvement]`` JAVA-1357: Document that getReplicas only returns replicas of the last token in range.
+
+
+### 3.0.6
+
+- ``[bug]`` JAVA-1330: Add un/register for SchemaChangeListener in DelegatingCluster
+- ``[bug]`` JAVA-1351: Include Custom Payload in Request.copy.
+- ``[bug]`` JAVA-1346: Reset heartbeat only on client reads (not writes).
+- ``[improvement]`` JAVA-866: Support tuple notation in QueryBuilder.eq/in.
+
+
+### 3.0.5
+
+- ``[bug]`` JAVA-1312: QueryBuilder modifies selected columns when manually selected.
+- ``[improvement]`` JAVA-1303: Add missing BoundStatement.setRoutingKey(ByteBuffer...)
+- ``[improvement]`` JAVA-262: Make internal executors customizable
+- ``[bug]`` JAVA-1320: prevent unnecessary task creation on empty pool
+
+
+### 3.0.4
+
+- ``[improvement]`` JAVA-1246: Driver swallows the real exception in a few cases
+- ``[improvement]`` JAVA-1261: Throw error when attempting to page in I/O thread.
+- ``[bug]`` JAVA-1258: Regression: Mapper cannot map a materialized view after JAVA-1126.
+- ``[bug]`` JAVA-1101: Batch and BatchStatement should consider inner statements to determine query idempotence
+- ``[improvement]`` JAVA-1262: Use ParseUtils for quoting & unquoting.
+- ``[improvement]`` JAVA-1275: Use Netty's default thread factory
+- ``[bug]`` JAVA-1285: QueryBuilder routing key auto-discovery should handle case-sensitive column names.
+- ``[bug]`` JAVA-1283: Don't cache failed query preparations in the mapper.
+- ``[improvement]`` JAVA-1277: Expose AbstractSession.checkNotInEventLoop.
+- ``[bug]`` JAVA-1272: BuiltStatement not able to print its query string if it contains mapped UDTs.
+- ``[bug]`` JAVA-1292: 'Adjusted frame length' error breaks driver's ability to read data.
+- ``[improvement]`` JAVA-1293: Make DecoderForStreamIdSize.MAX_FRAME_LENGTH configurable.
+- ``[improvement]`` JAVA-1053: Add a metric for authentication errors
+- ``[improvement]`` JAVA-1263: Eliminate unnecessary memory copies in FrameCompressor implementations.
+- ``[improvement]`` JAVA-893: Make connection pool non-blocking
+
+
+### 3.0.3
+
+- ``[improvement]`` JAVA-1147: Upgrade Netty to 4.0.37.
+- ``[bug]`` JAVA-1213: Allow updates and inserts to BLOB column using read-only ByteBuffer.
+- ``[bug]`` JAVA-1209: ProtocolOptions.getProtocolVersion() should return null instead of throwing NPE if Cluster has not
+        been init'd.
+- ``[improvement]`` JAVA-1204: Update documentation to indicate tcnative version requirement.
+- ``[bug]`` JAVA-1186: Fix duplicated hosts in DCAwarePolicy warn message.
+- ``[bug]`` JAVA-1187: Fix warning message when local CL used with RoundRobinPolicy.
+- ``[improvement]`` JAVA-1175: Warn if DCAwarePolicy configuration is inconsistent.
+- ``[bug]`` JAVA-1139: ConnectionException.getMessage() throws NPE if address is null.
+- ``[bug]`` JAVA-1202: Handle null rpc_address when checking schema agreement.
+- ``[improvement]`` JAVA-1198: Document that BoundStatement is not thread-safe.
+- ``[improvement]`` JAVA-1200: Upgrade LZ4 to 1.3.0.
+- ``[bug]`` JAVA-1232: Fix NPE in IdempotenceAwareRetryPolicy.isIdempotent.
+- ``[improvement]`` JAVA-1227: Document "SELECT *" issue with prepared statement.
+- ``[bug]`` JAVA-1160: Fix NPE in VersionNumber.getPreReleaseLabels().
+- ``[improvement]`` JAVA-1126: Handle schema changes in Mapper.
+- ``[bug]`` JAVA-1193: Refresh token and replica metadata synchronously when schema is altered.
+- ``[bug]`` JAVA-1120: Skip schema refresh debouncer when checking for agreement as a result of schema change made by client.
+- ``[improvement]`` JAVA-1242: Fix driver-core dependency in driver-stress
+- ``[improvement]`` JAVA-1235: Move the query to the end of "re-preparing .." log message as a key value.
+
+
+### 3.0.2
+
+Merged from 2.1 branch:
+
+- ``[bug]`` JAVA-1179: Request objects should be copied when executed.
+- ``[improvement]`` JAVA-1182: Throw error when synchronous call made on I/O thread.
+- ``[bug]`` JAVA-1184: Unwrap StatementWrappers when extracting column definitions.
+
+
+### 3.0.1
+
+- ``[bug]`` JAVA-1132: Executing bound statement with no variables results in exception with protocol v1.
+- ``[improvement]`` JAVA-1040: SimpleStatement parameters support in QueryLogger.
+- ``[improvement]`` JAVA-1151: Fail fast if HdrHistogram is not in the classpath.
+- ``[improvement]`` JAVA-1154: Allow individual Statement to cancel the read timeout.
+- ``[bug]`` JAVA-1074: Fix documentation around default timestamp generator.
+- ``[improvement]`` JAVA-1109: Document SSLOptions changes in upgrade guide.
+- ``[improvement]`` JAVA-1065: Add method to create token from partition key values.
+- ``[improvement]`` JAVA-1136: Enable JDK signature check in module driver-extras.
+- ``[improvement]`` JAVA-866: Support tuple notation in QueryBuilder.eq/in.
+- ``[bug]`` JAVA-1140: Use same connection to check for schema agreement after a DDL query.
+- ``[improvement]`` JAVA-1113: Support Cassandra 3.4 LIKE operator in QueryBuilder.
+- ``[improvement]`` JAVA-1086: Support Cassandra 3.2 CAST function in QueryBuilder.
+- ``[bug]`` JAVA-1095: Check protocol version for custom payload before sending the query.
+- ``[improvement]`` JAVA-1133: Add OSGi headers to cassandra-driver-extras.
+- ``[bug]`` JAVA-1137: Incorrect string returned by DataType.asFunctionParameterString() for collections and tuples.
+- ``[bug]`` JAVA-1046: (Dynamic)CompositeTypes need to be parsed as string literal, not blob.
+- ``[improvement]`` JAVA-1164: Clarify documentation on Host.listenAddress and broadcastAddress.
+- ``[improvement]`` JAVA-1171: Add Host method to determine if DSE Graph is enabled.
+- ``[improvement]`` JAVA-1069: Bootstrap driver-examples module.
+- ``[documentation]`` JAVA-1150: Add example and FAQ entry about ByteBuffer/BLOB.
+
+Merged from 2.1 branch:
+
+- ``[improvement]`` JAVA-1011: Expose PoolingOptions default values.
+- ``[improvement]`` JAVA-630: Don't process DOWN events for nodes that have active connections.
+- ``[improvement]`` JAVA-851: Improve UUIDs javadoc with regard to user-provided timestamps.
+- ``[improvement]`` JAVA-979: Update javadoc for RegularStatement toString() and getQueryString() to indicate that consistency level and other parameters are not maintained in the query string.
+- ``[bug]`` JAVA-1068: Unwrap StatementWrappers when hashing the paging state.
+- ``[improvement]`` JAVA-1021: Improve error message when connect() is called with an invalid keyspace name.
+- ``[improvement]`` JAVA-879: Mapper.map() accepts mapper-generated and user queries.
+- ``[bug]`` JAVA-1100: Exception when connecting with shaded java driver in OSGI
+- ``[bug]`` JAVA-1064: getTable create statement doesn't properly handle quotes in primary key.
+- ``[bug]`` JAVA-1089: Set LWT made from BuiltStatements to non-idempotent.
+- ``[improvement]`` JAVA-923: Position idempotent flag on object mapper queries.
+- ``[bug]`` JAVA-1070: The Mapper should not prepare queries synchronously.
+- ``[new feature]`` JAVA-982: Introduce new method ConsistencyLevel.isSerial().
+- ``[bug]`` JAVA-764: Retry with the normal consistency level (not the serial one) when a write times out on the Paxos phase.
+- ``[improvement]`` JAVA-852: Ignore peers with null entries during discovery.
+- ``[bug]`` JAVA-1005: DowngradingConsistencyRetryPolicy does not work with EACH_QUORUM when 1 DC is down.
+- ``[bug]`` JAVA-1002: Avoid deadlock when re-preparing a statement on other hosts.
+- ``[bug]`` JAVA-1072: Ensure defunct connections are properly evicted from the pool.
+- ``[bug]`` JAVA-1152: Fix NPE at ControlConnection.refreshNodeListAndTokenMap().
+
+
+### 3.0.0
+
+- ``[bug]`` JAVA-1034: fix metadata parser for collections of custom types.
+- ``[improvement]`` JAVA-1035: Expose host broadcast_address and listen_address if available.
+- ``[new feature]`` JAVA-1037: Allow named parameters in simple statements.
+- ``[improvement]`` JAVA-1033: Allow per-statement read timeout.
+- ``[improvement]`` JAVA-1042: Include DSE version and workload in Host data.
+
+Merged from 2.1 branch:
+
+- ``[improvement]`` JAVA-1030: Log token to replica map computation times.
+- ``[bug]`` JAVA-1039: Minor bugs in Event Debouncer.
+
+
+### 3.0.0-rc1
+
+- ``[bug]`` JAVA-890: fix mapper for case-sensitive UDT.
+
+
+### 3.0.0-beta1
+
+- ``[bug]`` JAVA-993: Support for "custom" types after CASSANDRA-10365.
+- ``[bug]`` JAVA-999: Handle unset parameters in QueryLogger.
+- ``[bug]`` JAVA-998: SchemaChangeListener not invoked for Functions or Aggregates having UDT arguments.
+- ``[bug]`` JAVA-1009: use CL ONE to compute query plan when reconnecting
+  control connection.
+- ``[improvement]`` JAVA-1003: Change default consistency level to LOCAL_ONE (amends JAVA-926).
+- ``[improvement]`` JAVA-863: Idempotence propagation in prepared statements.
+- ``[improvement]`` JAVA-996: Make CodecRegistry available to ProtocolDecoder.
+- ``[bug]`` JAVA-819: Driver shouldn't retry on client timeout if statement is not idempotent.
+- ``[improvement]`` JAVA-1007: Make SimpleStatement and QueryBuilder "detached" again.
+
+Merged from 2.1 branch:
+
+- ``[improvement]`` JAVA-989: Include keyspace name when invalid replication found when generating token map.
+- ``[improvement]`` JAVA-664: Reduce heap consumption for TokenMap.
+- ``[bug]`` JAVA-994: Don't call on(Up|Down|Add|Remove) methods if Cluster is closed/closing.
+
+
+### 3.0.0-alpha5
+
+- ``[improvement]`` JAVA-958: Make TableOrView.Order visible.
+- ``[improvement]`` JAVA-968: Update metrics to the latest version.
+- ``[improvement]`` JAVA-965: Improve error handling for when a non-type 1 UUID is given to bind() on a timeuuid column.
+- ``[improvement]`` JAVA-885: Pass the authenticator name from the server to the auth provider.
+- ``[improvement]`` JAVA-961: Raise an exception when an older version of guava (<16.01) is found.
+- ``[bug]`` JAVA-972: TypeCodec.parse() implementations should be case insensitive when checking for keyword NULL.
+- ``[bug]`` JAVA-971: Make type codecs invariant.
+- ``[bug]`` JAVA-986: Update documentation links to reference 3.0.
+- ``[improvement]`` JAVA-841: Refactor SSLOptions API.
+- ``[improvement]`` JAVA-948: Don't limit cipher suites by default.
+- ``[improvement]`` JAVA-917: Document SSL configuration.
+- ``[improvement]`` JAVA-936: Adapt schema metadata parsing logic to new storage format of CQL types in C* 3.0.
+- ``[new feature]`` JAVA-846: Provide custom codecs library as an extra module.
+- ``[new feature]`` JAVA-742: Codec Support for JSON.
+- ``[new feature]`` JAVA-606: Codec support for Java 8.
+- ``[new feature]`` JAVA-565: Codec support for Java arrays.
+- ``[new feature]`` JAVA-605: Codec support for Java enums.
+- ``[bug]`` JAVA-884: Fix UDT mapper to process fields in the correct order.
+
+Merged from 2.1 branch:
+
+- ``[bug]`` JAVA-854: avoid early return in Cluster.init when a node doesn't support the protocol version.
+- ``[bug]`` JAVA-978: Fix quoting issue that caused Mapper.getTableMetadata() to return null.
+- ``[improvement]`` JAVA-920: Downgrade "error creating pool" message to WARN.
+- ``[bug]`` JAVA-954: Don't trigger reconnection before initialization complete.
+- ``[improvement]`` JAVA-914: Avoid rejected tasks at shutdown.
+- ``[improvement]`` JAVA-921: Add SimpleStatement.getValuesCount().
+- ``[bug]`` JAVA-901: Move call to connection.release() out of cancelHandler.
+- ``[bug]`` JAVA-960: Avoid race in control connection shutdown.
+- ``[bug]`` JAVA-656: Fix NPE in ControlConnection.updateLocationInfo.
+- ``[bug]`` JAVA-966: Count uninitialized connections in conviction policy.
+- ``[improvement]`` JAVA-917: Document SSL configuration.
+- ``[improvement]`` JAVA-652: Add DCAwareRoundRobinPolicy builder.
+- ``[improvement]`` JAVA-808: Add generic filtering policy that can be used to exclude specific DCs.
+- ``[bug]`` JAVA-988: Metadata.handleId should handle escaped double quotes.
+- ``[bug]`` JAVA-983: QueryBuilder cannot handle collections containing function calls.
+
+
+### 3.0.0-alpha4
+
+- ``[improvement]`` JAVA-926: Change default consistency level to LOCAL_QUORUM.
+- ``[bug]`` JAVA-942: Fix implementation of UserType.hashCode().
+- ``[improvement]`` JAVA-877: Don't delay UP/ADDED notifications if protocol version = V4.
+- ``[improvement]`` JAVA-938: Parse 'extensions' column in table metadata.
+- ``[bug]`` JAVA-900: Fix Configuration builder to allow disabled metrics.
+- ``[new feature]`` JAVA-902: Prepare API for async query trace.
+- ``[new feature]`` JAVA-930: Add BoundStatement#unset.
+- ``[bug]`` JAVA-946: Make table metadata options class visible.
+- ``[bug]`` JAVA-939: Add crcCheckChance to TableOptionsMetadata#equals/hashCode.
+- ``[bug]`` JAVA-922: Make TypeCodec return mutable collections.
+- ``[improvement]`` JAVA-932: Limit visibility of codec internals.
+- ``[improvement]`` JAVA-934: Warn if a custom codec collides with an existing one.
+- ``[improvement]`` JAVA-940: Allow typed getters/setters to target any CQL type.
+- ``[bug]`` JAVA-950: Fix Cluster.connect with a case-sensitive keyspace.
+- ``[bug]`` JAVA-953: Fix MaterializedViewMetadata when base table name is case sensitive.
+
+
+### 3.0.0-alpha3
+
+- ``[new feature]`` JAVA-571: Support new system tables in C* 3.0.
+- ``[improvement]`` JAVA-919: Move crc_check_chance out of compressions options.
+
+Merged from 2.0 branch:
+
+- ``[improvement]`` JAVA-718: Log streamid at the trace level on sending request and receiving response.
+- ``[bug]`` JAVA-796: Fix SpeculativeExecutionPolicy.init() and close() are never called.
+- ``[improvement]`` JAVA-710: Suppress unnecessary warning at shutdown.
+- ``[improvement]`` #340: Allow DNS name with multiple A-records as contact point.
+- ``[bug]`` JAVA-794: Allow tracing across multiple result pages.
+- ``[bug]`` JAVA-737: DowngradingConsistencyRetryPolicy ignores write timeouts.
+- ``[bug]`` JAVA-736: Forbid bind marker in QueryBuilder add/append/prepend.
+- ``[bug]`` JAVA-712: Prevent QueryBuilder.quote() from applying duplicate double quotes.
+- ``[bug]`` JAVA-688: Prevent QueryBuilder from trying to serialize raw string.
+- ``[bug]`` JAVA-679: Support bind marker in QueryBuilder DELETE's list index.
+- ``[improvement]`` JAVA-475: Improve QueryBuilder API for SELECT DISTINCT.
+- ``[improvement]`` JAVA-225: Create values() function for Insert builder using List.
+- ``[improvement]`` JAVA-702: Warn when ReplicationStrategy encounters invalid
+  replication factors.
+- ``[improvement]`` JAVA-662: Add PoolingOptions method to set both core and max
+  connections.
+- ``[improvement]`` JAVA-766: Do not include epoll JAR in binary distribution.
+- ``[improvement]`` JAVA-726: Optimize internal copies of Request objects.
+- ``[bug]`` JAVA-815: Preserve tracing across retries.
+- ``[improvement]`` JAVA-709: New RetryDecision.tryNextHost().
+- ``[bug]`` JAVA-733: Handle function calls and raw strings as non-idempotent in QueryBuilder.
+- ``[improvement]`` JAVA-765: Provide API to retrieve values of a Parameterized SimpleStatement.
+- ``[improvement]`` JAVA-827: implement UPDATE .. IF EXISTS in QueryBuilder.
+- ``[improvement]`` JAVA-618: Randomize contact points list to prevent hotspots.
+- ``[improvement]`` JAVA-720: Surface the coordinator used on query failure.
+- ``[bug]`` JAVA-792: Handle contact points removed during init.
+- ``[improvement]`` JAVA-719: Allow PlainTextAuthProvider to change its credentials at runtime.
+- ``[new feature]`` JAVA-151: Make it possible to register for SchemaChange Events.
+- ``[improvement]`` JAVA-861: Downgrade "Asked to rebuild table" log from ERROR to INFO level.
+- ``[improvement]`` JAVA-797: Provide an option to prepare statements only on one node.
+- ``[improvement]`` JAVA-658: Provide an option to not re-prepare all statements in onUp.
+- ``[improvement]`` JAVA-853: Customizable creation of netty timer.
+- ``[bug]`` JAVA-859: Avoid quadratic ring processing with invalid replication factors.
+- ``[improvement]`` JAVA-657: Debounce control connection queries.
+- ``[bug]`` JAVA-784: LoadBalancingPolicy.distance() called before init().
+- ``[new feature]`` JAVA-828: Make driver-side metadata optional.
+- ``[improvement]`` JAVA-544: Allow hosts to remain partially up.
+- ``[improvement]`` JAVA-821, JAVA-822: Remove internal blocking calls and expose async session
+  creation.
+- ``[improvement]`` JAVA-725: Use parallel calls when re-preparing statement on other
+  hosts.
+- ``[bug]`` JAVA-629: Don't use connection timeout for unrelated internal queries.
+- ``[bug]`` JAVA-892: Fix NPE in speculative executions when metrics disabled.
+
+
+### 3.0.0-alpha2
+
+- ``[new feature]`` JAVA-875, JAVA-882: Move secondary index metadata out of column definitions.
+
+Merged from 2.2 branch:
+
+- ``[bug]`` JAVA-847: Propagate CodecRegistry to nested UDTs.
+- ``[improvement]`` JAVA-848: Ability to store a default, shareable CodecRegistry
+  instance.
+- ``[bug]`` JAVA-880: Treat empty ByteBuffers as empty values in TupleCodec and
+  UDTCodec.
+
+
+### 3.0.0-alpha1
+
+- ``[new feature]`` JAVA-876: Support new system tables in C* 3.0.0-alpha1.
+
+Merged from 2.2 branch:
+
+- ``[improvement]`` JAVA-810: Rename DateWithoutTime to LocalDate.
+- ``[bug]`` JAVA-816: DateCodec does not format values correctly.
+- ``[bug]`` JAVA-817: TimeCodec does not format values correctly.
+- ``[bug]`` JAVA-818: TypeCodec.getDataTypeFor() does not handle LocalDate instances.
+- ``[improvement]`` JAVA-836: Make ResultSet#fetchMoreResult return a
+  ``ListenableFuture``.
+- ``[improvement]`` JAVA-843: Disable frozen checks in mapper.
+- ``[improvement]`` JAVA-721: Allow user to register custom type codecs.
+- ``[improvement]`` JAVA-722: Support custom type codecs in mapper.
+
+
+### 2.2.0-rc3
+
+- ``[bug]`` JAVA-847: Propagate CodecRegistry to nested UDTs.
+- ``[improvement]`` JAVA-848: Ability to store a default, shareable CodecRegistry
+  instance.
+- ``[bug]`` JAVA-880: Treat empty ByteBuffers as empty values in TupleCodec and
+  UDTCodec.
+
+
+### 2.2.0-rc2
+
+- ``[improvement]`` JAVA-810: Rename DateWithoutTime to LocalDate.
+- ``[bug]`` JAVA-816: DateCodec does not format values correctly.
+- ``[bug]`` JAVA-817: TimeCodec does not format values correctly.
+- ``[bug]`` JAVA-818: TypeCodec.getDataTypeFor() does not handle LocalDate instances.
+- ``[improvement]`` JAVA-836: Make ResultSet#fetchMoreResult return a
+  ``ListenableFuture``.
+- ``[improvement]`` JAVA-843: Disable frozen checks in mapper.
+- ``[improvement]`` JAVA-721: Allow user to register custom type codecs.
+- ``[improvement]`` JAVA-722: Support custom type codecs in mapper.
+
+Merged from 2.1 branch:
+
+- ``[bug]`` JAVA-834: Special case check for 'null' string in index_options column.
+- ``[improvement]`` JAVA-835: Allow accessor methods with less parameters in case
+  named bind markers are repeated.
+- ``[improvement]`` JAVA-475: Improve QueryBuilder API for SELECT DISTINCT.
+- ``[improvement]`` JAVA-715: Make NativeColumnType a top-level class.
+- ``[improvement]`` JAVA-700: Expose ProtocolVersion#toInt.
+- ``[bug]`` JAVA-542: Handle void return types in accessors.
+- ``[improvement]`` JAVA-225: Create values() function for Insert builder using List.
+- ``[improvement]`` JAVA-713: HashMap throws an OOM Exception when logging level is set to TRACE.
+- ``[bug]`` JAVA-679: Support bind marker in QueryBuilder DELETE's list index.
+- ``[improvement]`` JAVA-732: Expose KEYS and FULL indexing options in IndexMetadata.
+- ``[improvement]`` JAVA-589: Allow @Enumerated in Accessor method parameters.
+- ``[improvement]`` JAVA-554: Allow access to table metadata from Mapper.
+- ``[improvement]`` JAVA-661: Provide a way to map computed fields.
+- ``[improvement]`` JAVA-824: Ignore missing columns in mapper.
+- ``[bug]`` JAVA-724: Preserve default timestamp for retries and speculative executions.
+- ``[improvement]`` JAVA-738: Use same pool implementation for protocol v2 and v3.
+- ``[improvement]`` JAVA-677: Support CONTAINS / CONTAINS KEY in QueryBuilder.
+- ``[improvement]`` JAVA-477/JAVA-540: Add USING options in mapper for delete and save
+  operations.
+- ``[improvement]`` JAVA-473: Add mapper option to configure whether to save null fields.
+
+Merged from 2.0 branch:
+
+- ``[bug]`` JAVA-737: DowngradingConsistencyRetryPolicy ignores write timeouts.
+- ``[bug]`` JAVA-736: Forbid bind marker in QueryBuilder add/append/prepend.
+- ``[bug]`` JAVA-712: Prevent QueryBuilder.quote() from applying duplicate double quotes.
+- ``[bug]`` JAVA-688: Prevent QueryBuilder from trying to serialize raw string.
+- ``[bug]`` JAVA-679: Support bind marker in QueryBuilder DELETE's list index.
+- ``[improvement]`` JAVA-475: Improve QueryBuilder API for SELECT DISTINCT.
+- ``[improvement]`` JAVA-225: Create values() function for Insert builder using List.
+- ``[improvement]`` JAVA-702: Warn when ReplicationStrategy encounters invalid
+  replication factors.
+- ``[improvement]`` JAVA-662: Add PoolingOptions method to set both core and max
+  connections.
+- ``[improvement]`` JAVA-766: Do not include epoll JAR in binary distribution.
+- ``[improvement]`` JAVA-726: Optimize internal copies of Request objects.
+- ``[bug]`` JAVA-815: Preserve tracing across retries.
+- ``[improvement]`` JAVA-709: New RetryDecision.tryNextHost().
+- ``[bug]`` JAVA-733: Handle function calls and raw strings as non-idempotent in QueryBuilder.
+
+
+### 2.2.0-rc1
+
+- ``[new feature]`` JAVA-783: Protocol V4 enum support.
+- ``[new feature]`` JAVA-776: Use PK columns in protocol v4 PREPARED response.
+- ``[new feature]`` JAVA-777: Distinguish NULL and UNSET values.
+- ``[new feature]`` JAVA-779: Add k/v payload for 3rd party usage.
+- ``[new feature]`` JAVA-780: Expose server-side warnings on ExecutionInfo.
+- ``[new feature]`` JAVA-749: Expose new read/write failure exceptions.
+- ``[new feature]`` JAVA-747: Expose function and aggregate metadata.
+- ``[new feature]`` JAVA-778: Add new client exception for CQL function failure.
+- ``[improvement]`` JAVA-700: Expose ProtocolVersion#toInt.
+- ``[new feature]`` JAVA-404: Support new C* 2.2 CQL date and time types.
+
+Merged from 2.1 branch:
+
+- ``[improvement]`` JAVA-782: Unify "Target" enum for schema elements.
+
+
+### 2.1.10.2
+
+Merged from 2.0 branch:
+
+- ``[bug]`` JAVA-1179: Request objects should be copied when executed.
+- ``[improvement]`` JAVA-1182: Throw error when synchronous call made on I/O thread.
+- ``[bug]`` JAVA-1184: Unwrap StatementWrappers when extracting column definitions.
+
+
+### 2.1.10.1
+
+- ``[bug]`` JAVA-1152: Fix NPE at ControlConnection.refreshNodeListAndTokenMap().
+- ``[bug]`` JAVA-1156: Fix NPE at TableMetadata.equals().
+
+
+### 2.1.10
+
+- ``[bug]`` JAVA-988: Metadata.handleId should handle escaped double quotes.
+- ``[bug]`` JAVA-983: QueryBuilder cannot handle collections containing function calls.
+- ``[improvement]`` JAVA-863: Idempotence propagation in PreparedStatements.
+- ``[bug]`` JAVA-937: TypeCodec static initializers not always correctly executed.
+- ``[improvement]`` JAVA-989: Include keyspace name when invalid replication found when generating token map.
+- ``[improvement]`` JAVA-664: Reduce heap consumption for TokenMap.
+- ``[improvement]`` JAVA-1030: Log token to replica map computation times.
+- ``[bug]`` JAVA-1039: Minor bugs in Event Debouncer.
+- ``[improvement]`` JAVA-843: Disable frozen checks in mapper.
+- ``[improvement]`` JAVA-833: Improve message when a nested type can't be serialized.
+- ``[improvement]`` JAVA-1011: Expose PoolingOptions default values.
+- ``[improvement]`` JAVA-630: Don't process DOWN events for nodes that have active connections.
+- ``[improvement]`` JAVA-851: Improve UUIDs javadoc with regard to user-provided timestamps.
+- ``[improvement]`` JAVA-979: Update javadoc for RegularStatement toString() and getQueryString() to indicate that consistency level and other parameters are not maintained in the query string.
+- ``[improvement]`` JAVA-1038: Fetch node info by rpc_address if its broadcast_address is not in system.peers.
+- ``[improvement]`` JAVA-974: Validate accessor parameter types against bound statement.
+- ``[bug]`` JAVA-1068: Unwrap StatementWrappers when hashing the paging state.
+- ``[bug]`` JAVA-831: Mapper can't load an entity where the PK is a UDT.
+- ``[improvement]`` JAVA-1021: Improve error message when connect() is called with an invalid keyspace name.
+- ``[improvement]`` JAVA-879: Mapper.map() accepts mapper-generated and user queries.
+- ``[bug]`` JAVA-1100: Exception when connecting with shaded java driver in OSGI
+- ``[bug]`` JAVA-819: Expose more errors in RetryPolicy + provide idempotent-aware wrapper.
+- ``[improvement]`` JAVA-1040: SimpleStatement parameters support in QueryLogger.
+- ``[bug]`` JAVA-1064: getTable create statement doesn't properly handle quotes in primary key.
+- ``[improvement]`` JAVA-888: Add cluster-wide percentile tracker.
+- ``[improvement]`` JAVA-963: Automatically register PercentileTracker from components that use it.
+- ``[bug]`` JAVA-1089: Set LWT made from BuiltStatements to non-idempotent.
+- ``[improvement]`` JAVA-923: Position idempotent flag on object mapper queries.
+- ``[new feature]`` JAVA-1019: SchemaBuilder support for CREATE/ALTER/DROP KEYSPACE.
+- ``[bug]`` JAVA-1070: The Mapper should not prepare queries synchronously.
+- ``[new feature]`` JAVA-982: Introduce new method ConsistencyLevel.isSerial().
+- ``[bug]`` JAVA-764: Retry with the normal consistency level (not the serial one) when a write times out on the Paxos phase.
+- ``[bug]`` JAVA-727: Allow monotonic timestamp generators to drift in the future + use microsecond precision when possible.
+- ``[improvement]`` JAVA-444: Add Java process information to UUIDs.makeNode() hash.
+- ``[improvement]`` JAVA-977: Preserve original cause when BuiltStatement value can't be serialized.
+- ``[bug]`` JAVA-1094: Backport TypeCodec parse and format fixes from 3.0.
+- ``[improvement]`` JAVA-852: Ignore peers with null entries during discovery.
+- ``[bug]`` JAVA-1132: Executing bound statement with no variables results in exception with protocol v1.
+- ``[bug]`` JAVA-1005: DowngradingConsistencyRetryPolicy does not work with EACH_QUORUM when 1 DC is down.
+- ``[bug]`` JAVA-1002: Avoid deadlock when re-preparing a statement on other hosts.
+
+Merged from 2.0 branch:
+
+- ``[bug]`` JAVA-994: Don't call on(Up|Down|Add|Remove) methods if Cluster is closed/closing.
+- ``[improvement]`` JAVA-805: Document that metrics are null until Cluster is initialized.
+- ``[bug]`` JAVA-1072: Ensure defunct connections are properly evicted from the pool.
+
+
+### 2.1.9
+
+- ``[bug]`` JAVA-942: Fix implementation of UserType.hashCode().
+- ``[bug]`` JAVA-854: avoid early return in Cluster.init when a node doesn't support the protocol version.
+- ``[bug]`` JAVA-978: Fix quoting issue that caused Mapper.getTableMetadata() to return null.
+
+Merged from 2.0 branch:
+
+- ``[bug]`` JAVA-950: Fix Cluster.connect with a case-sensitive keyspace.
+- ``[improvement]`` JAVA-920: Downgrade "error creating pool" message to WARN.
+- ``[bug]`` JAVA-954: Don't trigger reconnection before initialization complete.
+- ``[improvement]`` JAVA-914: Avoid rejected tasks at shutdown.
+- ``[improvement]`` JAVA-921: Add SimpleStatement.getValuesCount().
+- ``[bug]`` JAVA-901: Move call to connection.release() out of cancelHandler.
+- ``[bug]`` JAVA-960: Avoid race in control connection shutdown.
+- ``[bug]`` JAVA-656: Fix NPE in ControlConnection.updateLocationInfo.
+- ``[bug]`` JAVA-966: Count uninitialized connections in conviction policy.
+- ``[improvement]`` JAVA-917: Document SSL configuration.
+- ``[improvement]`` JAVA-652: Add DCAwareRoundRobinPolicy builder.
+- ``[improvement]`` JAVA-808: Add generic filtering policy that can be used to exclude specific DCs.
+
+
+### 2.1.8
+
+Merged from 2.0 branch:
+
+- ``[improvement]`` JAVA-718: Log streamid at the trace level on sending request and receiving response.
+
+- ``[bug]`` JAVA-796: Fix SpeculativeExecutionPolicy.init() and close() are never called.
+- ``[improvement]`` JAVA-710: Suppress unnecessary warning at shutdown.
+- ``[improvement]`` #340: Allow DNS name with multiple A-records as contact point.
+- ``[bug]`` JAVA-794: Allow tracing across multiple result pages.
+- ``[bug]`` JAVA-737: DowngradingConsistencyRetryPolicy ignores write timeouts.
+- ``[bug]`` JAVA-736: Forbid bind marker in QueryBuilder add/append/prepend.
+- ``[bug]`` JAVA-712: Prevent QueryBuilder.quote() from applying duplicate double quotes.
+- ``[bug]`` JAVA-688: Prevent QueryBuilder from trying to serialize raw string.
+- ``[bug]`` JAVA-679: Support bind marker in QueryBuilder DELETE's list index.
+- ``[improvement]`` JAVA-475: Improve QueryBuilder API for SELECT DISTINCT.
+- ``[improvement]`` JAVA-225: Create values() function for Insert builder using List.
+- ``[improvement]`` JAVA-702: Warn when ReplicationStrategy encounters invalid
+  replication factors.
+- ``[improvement]`` JAVA-662: Add PoolingOptions method to set both core and max
+  connections.
+- ``[improvement]`` JAVA-766: Do not include epoll JAR in binary distribution.
+- ``[improvement]`` JAVA-726: Optimize internal copies of Request objects.
+- ``[bug]`` JAVA-815: Preserve tracing across retries.
+- ``[improvement]`` JAVA-709: New RetryDecision.tryNextHost().
+- ``[bug]`` JAVA-733: Handle function calls and raw strings as non-idempotent in QueryBuilder.
+- ``[improvement]`` JAVA-765: Provide API to retrieve values of a Parameterized SimpleStatement.
+- ``[improvement]`` JAVA-827: implement UPDATE .. IF EXISTS in QueryBuilder.
+- ``[improvement]`` JAVA-618: Randomize contact points list to prevent hotspots.
+- ``[improvement]`` JAVA-720: Surface the coordinator used on query failure.
+- ``[bug]`` JAVA-792: Handle contact points removed during init.
+- ``[improvement]`` JAVA-719: Allow PlainTextAuthProvider to change its credentials at runtime.
+- ``[new feature]`` JAVA-151: Make it possible to register for SchemaChange Events.
+- ``[improvement]`` JAVA-861: Downgrade "Asked to rebuild table" log from ERROR to INFO level.
+- ``[improvement]`` JAVA-797: Provide an option to prepare statements only on one node.
+- ``[improvement]`` JAVA-658: Provide an option to not re-prepare all statements in onUp.
+- ``[improvement]`` JAVA-853: Customizable creation of netty timer.
+- ``[bug]`` JAVA-859: Avoid quadratic ring processing with invalid replication factors.
+- ``[improvement]`` JAVA-657: Debounce control connection queries.
+- ``[bug]`` JAVA-784: LoadBalancingPolicy.distance() called before init().
+- ``[new feature]`` JAVA-828: Make driver-side metadata optional.
+- ``[improvement]`` JAVA-544: Allow hosts to remain partially up.
+- ``[improvement]`` JAVA-821, JAVA-822: Remove internal blocking calls and expose async session
+  creation.
+- ``[improvement]`` JAVA-725: Use parallel calls when re-preparing statement on other
+  hosts.
+- ``[bug]`` JAVA-629: Don't use connection timeout for unrelated internal queries.
+- ``[bug]`` JAVA-892: Fix NPE in speculative executions when metrics disabled.
+
+
+### 2.1.7.1
+
+- ``[bug]`` JAVA-834: Special case check for 'null' string in index_options column.
+- ``[improvement]`` JAVA-835: Allow accessor methods with less parameters in case
+  named bind markers are repeated.
+
+
+### 2.1.7
+
+- ``[improvement]`` JAVA-475: Improve QueryBuilder API for SELECT DISTINCT.
+- ``[improvement]`` JAVA-715: Make NativeColumnType a top-level class.
+- ``[improvement]`` JAVA-782: Unify "Target" enum for schema elements.
+- ``[improvement]`` JAVA-700: Expose ProtocolVersion#toInt.
+- ``[bug]`` JAVA-542: Handle void return types in accessors.
+- ``[improvement]`` JAVA-225: Create values() function for Insert builder using List.
+- ``[improvement]`` JAVA-713: HashMap throws an OOM Exception when logging level is set to TRACE.
+- ``[bug]`` JAVA-679: Support bind marker in QueryBuilder DELETE's list index.
+- ``[improvement]`` JAVA-732: Expose KEYS and FULL indexing options in IndexMetadata.
+- ``[improvement]`` JAVA-589: Allow @Enumerated in Accessor method parameters.
+- ``[improvement]`` JAVA-554: Allow access to table metadata from Mapper.
+- ``[improvement]`` JAVA-661: Provide a way to map computed fields.
+- ``[improvement]`` JAVA-824: Ignore missing columns in mapper.
+- ``[bug]`` JAVA-724: Preserve default timestamp for retries and speculative executions.
+- ``[improvement]`` JAVA-738: Use same pool implementation for protocol v2 and v3.
+- ``[improvement]`` JAVA-677: Support CONTAINS / CONTAINS KEY in QueryBuilder.
+- ``[improvement]`` JAVA-477/JAVA-540: Add USING options in mapper for delete and save
+  operations.
+- ``[improvement]`` JAVA-473: Add mapper option to configure whether to save null fields.
+
+Merged from 2.0 branch:
+
+- ``[bug]`` JAVA-737: DowngradingConsistencyRetryPolicy ignores write timeouts.
+- ``[bug]`` JAVA-736: Forbid bind marker in QueryBuilder add/append/prepend.
+- ``[bug]`` JAVA-712: Prevent QueryBuilder.quote() from applying duplicate double quotes.
+- ``[bug]`` JAVA-688: Prevent QueryBuilder from trying to serialize raw string.
+- ``[bug]`` JAVA-679: Support bind marker in QueryBuilder DELETE's list index.
+- ``[improvement]`` JAVA-475: Improve QueryBuilder API for SELECT DISTINCT.
+- ``[improvement]`` JAVA-225: Create values() function for Insert builder using List.
+- ``[improvement]`` JAVA-702: Warn when ReplicationStrategy encounters invalid
+  replication factors.
+- ``[improvement]`` JAVA-662: Add PoolingOptions method to set both core and max
+  connections.
+- ``[improvement]`` JAVA-766: Do not include epoll JAR in binary distribution.
+- ``[improvement]`` JAVA-726: Optimize internal copies of Request objects.
+- ``[bug]`` JAVA-815: Preserve tracing across retries.
+- ``[improvement]`` JAVA-709: New RetryDecision.tryNextHost().
+- ``[bug]`` JAVA-733: Handle function calls and raw strings as non-idempotent in QueryBuilder.
+
+
+### 2.1.6
+
+Merged from 2.0 branch:
+
+- ``[new feature]`` JAVA-584: Add getObject to BoundStatement and Row.
+- ``[improvement]`` JAVA-419: Improve connection pool resizing algorithm.
+- ``[bug]`` JAVA-599: Fix race condition between pool expansion and shutdown.
+- ``[improvement]`` JAVA-622: Upgrade Netty to 4.0.27.
+- ``[improvement]`` JAVA-562: Coalesce frames before flushing them to the connection.
+- ``[improvement]`` JAVA-583: Rename threads to indicate that they are for the driver.
+- ``[new feature]`` JAVA-550: Expose paging state.
+- ``[new feature]`` JAVA-646: Slow Query Logger.
+- ``[improvement]`` JAVA-698: Exclude some errors from measurements in LatencyAwarePolicy.
+- ``[bug]`` JAVA-641: Fix issue when executing a PreparedStatement from another cluster.
+- ``[improvement]`` JAVA-534: Log keyspace xxx does not exist at WARN level.
+- ``[improvement]`` JAVA-619: Allow Cluster subclasses to delegate to another instance.
+- ``[new feature]`` JAVA-669: Expose an API to check for schema agreement after a
+  schema-altering statement.
+- ``[improvement]`` JAVA-692: Make connection and pool creation fully async.
+- ``[improvement]`` JAVA-505: Optimize connection use after reconnection.
+- ``[improvement]`` JAVA-617: Remove "suspected" mechanism.
+- ``[improvement]`` reverts JAVA-425: Don't mark connection defunct on client timeout.
+- ``[new feature]`` JAVA-561: Speculative query executions.
+- ``[bug]`` JAVA-666: Release connection before completing the ResultSetFuture.
+- ``[new feature BETA]`` JAVA-723: Percentile-based variant of query logger and speculative
+  executions.
+- ``[bug]`` JAVA-734: Fix buffer leaks when compression is enabled.
+- ``[improvement]`` JAVA-756: Use Netty's pooled ByteBufAllocator by default.
+- ``[improvement]`` JAVA-759: Expose "unsafe" paging state API.
+- ``[bug]`` JAVA-768: Prevent race during pool initialization.
+
+
+### 2.1.5
+
+- ``[bug]`` JAVA-575: Authorize Null parameter in Accessor method.
+- ``[improvement]`` JAVA-570: Support C* 2.1.3's nested collections.
+- ``[bug]`` JAVA-612: Fix checks on mapped collection types.
+- ``[bug]`` JAVA-672: Fix QueryBuilder.putAll() when the collection contains UDTs.
+
+Merged from 2.0 branch:
+
+- ``[new feature]`` JAVA-518: Add AddressTranslater for EC2 multi-region deployment.
+- ``[improvement]`` JAVA-533: Add connection heartbeat.
+- ``[improvement]`` JAVA-568: Reduce level of logs on missing rpc_address.
+- ``[improvement]`` JAVA-312, JAVA-681: Expose node token and range information.
+- ``[bug]`` JAVA-595: Fix cluster name mismatch check at startup.
+- ``[bug]`` JAVA-620: Fix guava dependency when using OSGI.
+- ``[bug]`` JAVA-678: Fix handling of DROP events when ks name is case-sensitive.
+- ``[improvement]`` JAVA-631: Use ``List`` instead of ``List`` in QueryBuilder API.
+- ``[improvement]`` JAVA-654: Exclude Netty POM from META-INF in shaded JAR.
+- ``[bug]`` JAVA-655: Quote single quotes contained in table comments in asCQLQuery method.
+- ``[bug]`` JAVA-684: Empty TokenRange returned in a one token cluster.
+- ``[improvement]`` JAVA-687: Expose TokenRange#contains.
+- ``[bug]`` JAVA-614: Prevent race between cancellation and query completion.
+- ``[bug]`` JAVA-632: Prevent cancel and timeout from cancelling unrelated ResponseHandler if
+  streamId was already released and reused.
+- ``[bug]`` JAVA-642: Fix issue when newly opened pool fails before we could mark the node UP.
+- ``[bug]`` JAVA-613: Fix unwanted LBP notifications when a contact host is down.
+- ``[bug]`` JAVA-651: Fix edge cases where a connection was released twice.
+- ``[bug]`` JAVA-653: Fix edge cases in query cancellation.
+
+
+### 2.1.4
+
+Merged from 2.0 branch:
+
+- ``[improvement]`` JAVA-538: Shade Netty dependency.
+- ``[improvement]`` JAVA-543: Target schema refreshes more precisely.
+- ``[bug]`` JAVA-546: Don't check rpc_address for control host.
+- ``[improvement]`` JAVA-409: Improve message of NoHostAvailableException.
+- ``[bug]`` JAVA-556: Rework connection reaper to avoid deadlock.
+- ``[bug]`` JAVA-557: Avoid deadlock when multiple connections to the same host get write
+  errors.
+- ``[improvement]`` JAVA-504: Make shuffle=true the default for TokenAwarePolicy.
+- ``[bug]`` JAVA-577: Fix bug when SUSPECT reconnection succeeds, but one of the pooled
+  connections fails while bringing the node back up.
+- ``[bug]`` JAVA-419: JAVA-587: Prevent faulty control connection from ignoring reconnecting hosts.
+- temporarily revert "Add idle timeout to the connection pool".
+- ``[bug]`` JAVA-593: Ensure updateCreatedPools does not add pools for suspected hosts.
+- ``[bug]`` JAVA-594: Ensure state change notifications for a given host are handled serially.
+- ``[bug]`` JAVA-597: Ensure control connection reconnects when control host is removed.
+
+
+### 2.1.3
+
+- ``[bug]`` JAVA-510: Ignore static fields in mapper.
+- ``[bug]`` JAVA-509: Fix UDT parsing at init when using the default protocol version.
+- ``[bug]`` JAVA-495: Fix toString, equals and hashCode on accessor proxies.
+- ``[bug]`` JAVA-528: Allow empty name on Column and Field annotations.
+
+Merged from 2.0 branch:
+
+- ``[bug]`` JAVA-497: Ensure control connection does not trigger concurrent reconnects.
+- ``[improvement]`` JAVA-472: Keep trying to reconnect on authentication errors.
+- ``[improvement]`` JAVA-463: Expose close method on load balancing policy.
+- ``[improvement]`` JAVA-459: Allow load balancing policy to trigger refresh for a single host.
+- ``[bug]`` JAVA-493: Expose an API to cancel reconnection attempts.
+- ``[bug]`` JAVA-503: Fix NPE when a connection fails during pool construction.
+- ``[improvement]`` JAVA-423: Log datacenter name in DCAware policy's init when it is explicitly provided.
+- ``[improvement]`` JAVA-504: Shuffle the replicas in TokenAwarePolicy.newQueryPlan.
+- ``[improvement]`` JAVA-507: Make schema agreement wait tuneable.
+- ``[improvement]`` JAVA-494: Document how to inject the driver metrics into another registry.
+- ``[improvement]`` JAVA-419: Add idle timeout to the connection pool.
+- ``[bug]`` JAVA-516: LatencyAwarePolicy does not shutdown executor on invocation of close.
+- ``[improvement]`` JAVA-451: Throw an exception when DCAwareRoundRobinPolicy is built with
+  an explicit but null or empty local datacenter.
+- ``[bug]`` JAVA-511: Fix check for local contact points in DCAware policy's init.
+- ``[improvement]`` JAVA-457: Make timeout on saturated pool customizable.
+- ``[improvement]`` JAVA-521: Downgrade Guava to 14.0.1.
+- ``[bug]`` JAVA-526: Fix token awareness for case-sensitive keyspaces and tables.
+- ``[bug]`` JAVA-515: Check maximum number of values passed to SimpleStatement.
+- ``[improvement]`` JAVA-532: Expose the driver version through the API.
+- ``[improvement]`` JAVA-522: Optimize session initialization when some hosts are not
+  responsive.
+
+
+### 2.1.2
+
+- ``[improvement]`` JAVA-361, JAVA-364, JAVA-467: Support for native protocol v3.
+- ``[bug]`` JAVA-454: Fix UDT fields of type inet in QueryBuilder.
+- ``[bug]`` JAVA-455: Exclude transient fields from Frozen checks.
+- ``[bug]`` JAVA-453: Fix handling of null collections in mapper.
+- ``[improvement]`` JAVA-452: Make implicit column names case-insensitive in mapper.
+- ``[bug]`` JAVA-433: Fix named bind markers in QueryBuilder.
+- ``[bug]`` JAVA-458: Fix handling of BigInteger in object mapper.
+- ``[bug]`` JAVA-465: Ignore synthetic fields in mapper.
+- ``[improvement]`` JAVA-451: Throw an exception when DCAwareRoundRobinPolicy is built with
+  an explicit but null or empty local datacenter.
+- ``[improvement]`` JAVA-469: Add backwards-compatible DataType.serialize methods.
+- ``[bug]`` JAVA-487: Handle null enum fields in object mapper.
+- ``[bug]`` JAVA-499: Handle null UDT fields in object mapper.
+
+Merged from 2.0 branch:
+
+- ``[bug]`` JAVA-449: Handle null pool in PooledConnection.release.
+- ``[improvement]`` JAVA-425: Defunct connection on request timeout.
+- ``[improvement]`` JAVA-426: Try next host when we get a SERVER_ERROR.
+- ``[bug]`` JAVA-449, JAVA-460, JAVA-471: Handle race between query timeout and completion.
+- ``[bug]`` JAVA-496: Fix DCAwareRoundRobinPolicy datacenter auto-discovery.
+
+
+### 2.1.1
+
+- ``[new]`` JAVA-441: Support for new "frozen" keyword.
+
+Merged from 2.0 branch:
+
+- ``[bug]`` JAVA-397: Check cluster name when connecting to a new node.
+- ``[bug]`` JAVA-326: Add missing CAS delete support in QueryBuilder.
+- ``[bug]`` JAVA-363: Add collection and data length checks during serialization.
+- ``[improvement]`` JAVA-329: Surface number of retries in metrics.
+- ``[bug]`` JAVA-428: Do not use a host when no rpc_address found for it.
+- ``[improvement]`` JAVA-358: Add ResultSet.wasApplied() for conditional queries.
+- ``[bug]`` JAVA-349: Fix negative HostConnectionPool open count.
+- ``[improvement]`` JAVA-436: Log more connection details at trace and debug levels.
+- ``[bug]`` JAVA-445: Fix cluster shutdown.
+
+
+### 2.1.0
+
+- ``[bug]`` JAVA-408: ClusteringColumn annotation not working with specified ordering.
+- ``[improvement]`` JAVA-410: Fail BoundStatement if null values are not set explicitly.
+- ``[bug]`` JAVA-416: Handle UDT and tuples in BuiltStatement.toString.
+
+Merged from 2.0 branch:
+
+- ``[bug]`` JAVA-407: Release connections on ResultSetFuture#cancel.
+- ``[bug]`` JAVA-393: Fix handling of SimpleStatement with values in query builder
+  batches.
+- ``[bug]`` JAVA-417: Ensure pool is properly closed in onDown.
+- ``[bug]`` JAVA-415: Fix tokenMap initialization at startup.
+- ``[bug]`` JAVA-418: Avoid deadlock on close.
+
+
+### 2.1.0-rc1
+
+Merged from 2.0 branch:
+
+- ``[bug]`` JAVA-394: Ensure defunct connections are completely closed.
+- ``[bug]`` JAVA-342, JAVA-390: Fix memory and resource leak on closed Sessions.
+
+
+### 2.1.0-beta1
+
+- ``[new]`` Support for User Defined Types and tuples
+- ``[new]`` Simple object mapper
+
+Merged from 2.0 branch: everything up to 2.0.3 (included), and the following.
+
+- ``[improvement]`` JAVA-204: Better handling of dead connections.
+- ``[bug]`` JAVA-373: Fix potential NPE in ControlConnection.
+- ``[bug]`` JAVA-291: Throws NPE when passed null for a contact point.
+- ``[bug]`` JAVA-315: Avoid LoadBalancingPolicy onDown+onUp at startup.
+- ``[bug]`` JAVA-343: Avoid classloader leak in Tomcat.
+- ``[bug]`` JAVA-387: Avoid deadlock in onAdd/onUp.
+- ``[bug]`` JAVA-377, JAVA-391: Make metadata parsing more lenient.
+
+
+### 2.0.12.2
+
+- ``[bug]`` JAVA-1179: Request objects should be copied when executed.
+- ``[improvement]`` JAVA-1182: Throw error when synchronous call made on I/O thread.
+- ``[bug]`` JAVA-1184: Unwrap StatementWrappers when extracting column definitions.
+
+
+### 2.0.12.1
+
+- ``[bug]`` JAVA-994: Don't call on(Up|Down|Add|Remove) methods if Cluster is closed/closing.
+- ``[improvement]`` JAVA-805: Document that metrics are null until Cluster is initialized.
+- ``[bug]`` JAVA-1072: Ensure defunct connections are properly evicted from the pool.
+
+
+### 2.0.12
+
+- ``[bug]`` JAVA-950: Fix Cluster.connect with a case-sensitive keyspace.
+- ``[improvement]`` JAVA-920: Downgrade "error creating pool" message to WARN.
+- ``[bug]`` JAVA-954: Don't trigger reconnection before initialization complete.
+- ``[improvement]`` JAVA-914: Avoid rejected tasks at shutdown.
+- ``[improvement]`` JAVA-921: Add SimpleStatement.getValuesCount().
+- ``[bug]`` JAVA-901: Move call to connection.release() out of cancelHandler.
+- ``[bug]`` JAVA-960: Avoid race in control connection shutdown.
+- ``[bug]`` JAVA-656: Fix NPE in ControlConnection.updateLocationInfo.
+- ``[bug]`` JAVA-966: Count uninitialized connections in conviction policy.
+- ``[improvement]`` JAVA-917: Document SSL configuration.
+- ``[improvement]`` JAVA-652: Add DCAwareRoundRobinPolicy builder.
+- ``[improvement]`` JAVA-808: Add generic filtering policy that can be used to exclude specific DCs.
+
+
+### 2.0.11
+
+- ``[improvement]`` JAVA-718: Log streamid at the trace level on sending request and receiving response.
+- ``[bug]`` JAVA-796: Fix SpeculativeExecutionPolicy.init() and close() are never called.
+- ``[improvement]`` JAVA-710: Suppress unnecessary warning at shutdown.
+- ``[improvement]`` #340: Allow DNS name with multiple A-records as contact point.
+- ``[bug]`` JAVA-794: Allow tracing across multiple result pages.
+- ``[bug]`` JAVA-737: DowngradingConsistencyRetryPolicy ignores write timeouts.
+- ``[bug]`` JAVA-736: Forbid bind marker in QueryBuilder add/append/prepend.
+- ``[bug]`` JAVA-712: Prevent QueryBuilder.quote() from applying duplicate double quotes.
+- ``[bug]`` JAVA-688: Prevent QueryBuilder from trying to serialize raw string.
+- ``[bug]`` JAVA-679: Support bind marker in QueryBuilder DELETE's list index.
+- ``[improvement]`` JAVA-475: Improve QueryBuilder API for SELECT DISTINCT.
+- ``[improvement]`` JAVA-225: Create values() function for Insert builder using List.
+- ``[improvement]`` JAVA-702: Warn when ReplicationStrategy encounters invalid
+  replication factors.
+- ``[improvement]`` JAVA-662: Add PoolingOptions method to set both core and max
+  connections.
+- ``[improvement]`` JAVA-766: Do not include epoll JAR in binary distribution.
+- ``[improvement]`` JAVA-726: Optimize internal copies of Request objects.
+- ``[bug]`` JAVA-815: Preserve tracing across retries.
+- ``[improvement]`` JAVA-709: New RetryDecision.tryNextHost().
+- ``[bug]`` JAVA-733: Handle function calls and raw strings as non-idempotent in QueryBuilder.
+- ``[improvement]`` JAVA-765: Provide API to retrieve values of a Parameterized SimpleStatement.
+- ``[improvement]`` JAVA-827: implement UPDATE .. IF EXISTS in QueryBuilder.
+- ``[improvement]`` JAVA-618: Randomize contact points list to prevent hotspots.
+- ``[improvement]`` JAVA-720: Surface the coordinator used on query failure.
+- ``[bug]`` JAVA-792: Handle contact points removed during init.
+- ``[improvement]`` JAVA-719: Allow PlainTextAuthProvider to change its credentials at runtime.
+- ``[new feature]`` JAVA-151: Make it possible to register for SchemaChange Events.
+- ``[improvement]`` JAVA-861: Downgrade "Asked to rebuild table" log from ERROR to INFO level.
+- ``[improvement]`` JAVA-797: Provide an option to prepare statements only on one node.
+- ``[improvement]`` JAVA-658: Provide an option to not re-prepare all statements in onUp.
+- ``[improvement]`` JAVA-853: Customizable creation of netty timer.
+- ``[bug]`` JAVA-859: Avoid quadratic ring processing with invalid replication factors.
+- ``[improvement]`` JAVA-657: Debounce control connection queries.
+- ``[bug]`` JAVA-784: LoadBalancingPolicy.distance() called before init().
+- ``[new feature]`` JAVA-828: Make driver-side metadata optional.
+- ``[improvement]`` JAVA-544: Allow hosts to remain partially up.
+- ``[improvement]`` JAVA-821, JAVA-822: Remove internal blocking calls and expose async session
+  creation.
+- ``[improvement]`` JAVA-725: Use parallel calls when re-preparing statement on other
+  hosts.
+- ``[bug]`` JAVA-629: Don't use connection timeout for unrelated internal queries.
+- ``[bug]`` JAVA-892: Fix NPE in speculative executions when metrics disabled.
+
+Merged from 2.0.10_fixes branch:
+
+- ``[improvement]`` JAVA-756: Use Netty's pooled ByteBufAllocator by default.
+- ``[improvement]`` JAVA-759: Expose "unsafe" paging state API.
+- ``[bug]`` JAVA-767: Fix getObject by name.
+- ``[bug]`` JAVA-768: Prevent race during pool initialization.
+
+
+### 2.0.10.1
+
+- ``[improvement]`` JAVA-756: Use Netty's pooled ByteBufAllocator by default.
+- ``[improvement]`` JAVA-759: Expose "unsafe" paging state API.
+- ``[bug]`` JAVA-767: Fix getObject by name.
+- ``[bug]`` JAVA-768: Prevent race during pool initialization.
+
+
+### 2.0.10
+
+- ``[new feature]`` JAVA-518: Add AddressTranslater for EC2 multi-region deployment.
+- ``[improvement]`` JAVA-533: Add connection heartbeat.
+- ``[improvement]`` JAVA-568: Reduce level of logs on missing rpc_address.
+- ``[improvement]`` JAVA-312, JAVA-681: Expose node token and range information.
+- ``[bug]`` JAVA-595: Fix cluster name mismatch check at startup.
+- ``[bug]`` JAVA-620: Fix guava dependency when using OSGI.
+- ``[bug]`` JAVA-678: Fix handling of DROP events when ks name is case-sensitive.
+- ``[improvement]`` JAVA-631: Use ``List`` instead of ``List`` in QueryBuilder API.
+- ``[improvement]`` JAVA-654: Exclude Netty POM from META-INF in shaded JAR.
+- ``[bug]`` JAVA-655: Quote single quotes contained in table comments in asCQLQuery method.
+- ``[bug]`` JAVA-684: Empty TokenRange returned in a one token cluster.
+- ``[improvement]`` JAVA-687: Expose TokenRange#contains.
+- ``[new feature]`` JAVA-547: Expose values of BoundStatement.
+- ``[new feature]`` JAVA-584: Add getObject to BoundStatement and Row.
+- ``[improvement]`` JAVA-419: Improve connection pool resizing algorithm.
+- ``[bug]`` JAVA-599: Fix race condition between pool expansion and shutdown.
+- ``[improvement]`` JAVA-622: Upgrade Netty to 4.0.27.
+- ``[improvement]`` JAVA-562: Coalesce frames before flushing them to the connection.
+- ``[improvement]`` JAVA-583: Rename threads to indicate that they are for the driver.
+- ``[new feature]`` JAVA-550: Expose paging state.
+- ``[new feature]`` JAVA-646: Slow Query Logger.
+- ``[improvement]`` JAVA-698: Exclude some errors from measurements in LatencyAwarePolicy.
+- ``[bug]`` JAVA-641: Fix issue when executing a PreparedStatement from another cluster.
+- ``[improvement]`` JAVA-534: Log keyspace xxx does not exist at WARN level.
+- ``[improvement]`` JAVA-619: Allow Cluster subclasses to delegate to another instance.
+- ``[new feature]`` JAVA-669: Expose an API to check for schema agreement after a
+  schema-altering statement.
+- ``[improvement]`` JAVA-692: Make connection and pool creation fully async.
+- ``[improvement]`` JAVA-505: Optimize connection use after reconnection.
+- ``[improvement]`` JAVA-617: Remove "suspected" mechanism.
+- ``[improvement]`` reverts JAVA-425: Don't mark connection defunct on client timeout.
+- ``[new feature]`` JAVA-561: Speculative query executions.
+- ``[bug]`` JAVA-666: Release connection before completing the ResultSetFuture.
+- ``[new feature BETA]`` JAVA-723: Percentile-based variant of query logger and speculative
+  executions.
+- ``[bug]`` JAVA-734: Fix buffer leaks when compression is enabled.
+
+Merged from 2.0.9_fixes branch:
+
+- ``[bug]`` JAVA-614: Prevent race between cancellation and query completion.
+- ``[bug]`` JAVA-632: Prevent cancel and timeout from cancelling unrelated ResponseHandler if
+  streamId was already released and reused.
+- ``[bug]`` JAVA-642: Fix issue when newly opened pool fails before we could mark the node UP.
+- ``[bug]`` JAVA-613: Fix unwanted LBP notifications when a contact host is down.
+- ``[bug]`` JAVA-651: Fix edge cases where a connection was released twice.
+- ``[bug]`` JAVA-653: Fix edge cases in query cancellation.
+
+
+### 2.0.9.2
+
+- ``[bug]`` JAVA-651: Fix edge cases where a connection was released twice.
+- ``[bug]`` JAVA-653: Fix edge cases in query cancellation.
+
+
+### 2.0.9.1
+
+- ``[bug]`` JAVA-614: Prevent race between cancellation and query completion.
+- ``[bug]`` JAVA-632: Prevent cancel and timeout from cancelling unrelated ResponseHandler if
+  streamId was already released and reused.
+- ``[bug]`` JAVA-642: Fix issue when newly opened pool fails before we could mark the node UP.
+- ``[bug]`` JAVA-613: Fix unwanted LBP notifications when a contact host is down.
+
+
+### 2.0.9
+
+- ``[improvement]`` JAVA-538: Shade Netty dependency.
+- ``[improvement]`` JAVA-543: Target schema refreshes more precisely.
+- ``[bug]`` JAVA-546: Don't check rpc_address for control host.
+- ``[improvement]`` JAVA-409: Improve message of NoHostAvailableException.
+- ``[bug]`` JAVA-556: Rework connection reaper to avoid deadlock.
+- ``[bug]`` JAVA-557: Avoid deadlock when multiple connections to the same host get write
+  errors.
+- ``[improvement]`` JAVA-504: Make shuffle=true the default for TokenAwarePolicy.
+- ``[bug]`` JAVA-577: Fix bug when SUSPECT reconnection succeeds, but one of the pooled
+  connections fails while bringing the node back up.
+- ``[bug]`` JAVA-419: JAVA-587: Prevent faulty control connection from ignoring reconnecting hosts.
+- temporarily revert "Add idle timeout to the connection pool".
+- ``[bug]`` JAVA-593: Ensure updateCreatedPools does not add pools for suspected hosts.
+- ``[bug]`` JAVA-594: Ensure state change notifications for a given host are handled serially.
+- ``[bug]`` JAVA-597: Ensure control connection reconnects when control host is removed.
+
+
+### 2.0.8
+
+- ``[bug]`` JAVA-526: Fix token awareness for case-sensitive keyspaces and tables.
+- ``[bug]`` JAVA-515: Check maximum number of values passed to SimpleStatement.
+- ``[improvement]`` JAVA-532: Expose the driver version through the API.
+- ``[improvement]`` JAVA-522: Optimize session initialization when some hosts are not
+  responsive.
+
+
+### 2.0.7
+
+- ``[bug]`` JAVA-449: Handle null pool in PooledConnection.release.
+- ``[improvement]`` JAVA-425: Defunct connection on request timeout.
+- ``[improvement]`` JAVA-426: Try next host when we get a SERVER_ERROR.
+- ``[bug]`` JAVA-449, JAVA-460, JAVA-471: Handle race between query timeout and completion.
+- ``[bug]`` JAVA-496: Fix DCAwareRoundRobinPolicy datacenter auto-discovery.
+- ``[bug]`` JAVA-497: Ensure control connection does not trigger concurrent reconnects.
+- ``[improvement]`` JAVA-472: Keep trying to reconnect on authentication errors.
+- ``[improvement]`` JAVA-463: Expose close method on load balancing policy.
+- ``[improvement]`` JAVA-459: Allow load balancing policy to trigger refresh for a single host.
+- ``[bug]`` JAVA-493: Expose an API to cancel reconnection attempts.
+- ``[bug]`` JAVA-503: Fix NPE when a connection fails during pool construction.
+- ``[improvement]`` JAVA-423: Log datacenter name in DCAware policy's init when it is explicitly provided.
+- ``[improvement]`` JAVA-504: Shuffle the replicas in TokenAwarePolicy.newQueryPlan.
+- ``[improvement]`` JAVA-507: Make schema agreement wait tuneable.
+- ``[improvement]`` JAVA-494: Document how to inject the driver metrics into another registry.
+- ``[improvement]`` JAVA-419: Add idle timeout to the connection pool.
+- ``[bug]`` JAVA-516: LatencyAwarePolicy does not shutdown executor on invocation of close.
+- ``[improvement]`` JAVA-451: Throw an exception when DCAwareRoundRobinPolicy is built with
+  an explicit but null or empty local datacenter.
+- ``[bug]`` JAVA-511: Fix check for local contact points in DCAware policy's init.
+- ``[improvement]`` JAVA-457: Make timeout on saturated pool customizable.
+- ``[improvement]`` JAVA-521: Downgrade Guava to 14.0.1.
+
+
+### 2.0.6
+
+- ``[bug]`` JAVA-397: Check cluster name when connecting to a new node.
+- ``[bug]`` JAVA-326: Add missing CAS delete support in QueryBuilder.
+- ``[bug]`` JAVA-363: Add collection and data length checks during serialization.
+- ``[improvement]`` JAVA-329: Surface number of retries in metrics.
+- ``[bug]`` JAVA-428: Do not use a host when no rpc_address found for it.
+- ``[improvement]`` JAVA-358: Add ResultSet.wasApplied() for conditional queries.
+- ``[bug]`` JAVA-349: Fix negative HostConnectionPool open count.
+- ``[improvement]`` JAVA-436: Log more connection details at trace and debug levels.
+- ``[bug]`` JAVA-445: Fix cluster shutdown.
+- ``[improvement]`` JAVA-439: Expose child policy in chainable load balancing policies.
+
+
+### 2.0.5
+
+- ``[bug]`` JAVA-407: Release connections on ResultSetFuture#cancel.
+- ``[bug]`` JAVA-393: Fix handling of SimpleStatement with values in query builder
+  batches.
+- ``[bug]`` JAVA-417: Ensure pool is properly closed in onDown.
+- ``[bug]`` JAVA-415: Fix tokenMap initialization at startup.
+- ``[bug]`` JAVA-418: Avoid deadlock on close.
+
+
+### 2.0.4
+
+- ``[improvement]`` JAVA-204: Better handling of dead connections.
+- ``[bug]`` JAVA-373: Fix potential NPE in ControlConnection.
+- ``[bug]`` JAVA-291: Throws NPE when passed null for a contact point.
+- ``[bug]`` JAVA-315: Avoid LoadBalancingPolicy onDown+onUp at startup.
+- ``[bug]`` JAVA-343: Avoid classloader leak in Tomcat.
+- ``[bug]`` JAVA-387: Avoid deadlock in onAdd/onUp.
+- ``[bug]`` JAVA-377, JAVA-391: Make metadata parsing more lenient.
+- ``[bug]`` JAVA-394: Ensure defunct connections are completely closed.
+- ``[bug]`` JAVA-342, JAVA-390: Fix memory and resource leak on closed Sessions.
+
+
+### 2.0.3
+
+- ``[new]`` The new AbsractSession makes mocking of Session easier.
+- ``[new]`` JAVA-309: Allow to trigger a refresh of connected hosts.
+- ``[new]`` JAVA-265: New Session#getState method allows to grab information on
+  which nodes a session is connected to.
+- ``[new]`` JAVA-327: Add QueryBuilder syntax for tuples in where clauses (syntax
+  introduced in Cassandra 2.0.6).
+- ``[improvement]`` JAVA-359: Properly validate arguments of PoolingOptions methods.
+- ``[bug]`` JAVA-368: Fix bogus rejection of BigInteger in 'execute with values'.
+- ``[bug]`` JAVA-367: Signal connection failure sooner to avoid missing them.
+- ``[bug]`` JAVA-337: Throw UnsupportedOperationException for protocol batch
+  setSerialCL.
+
+Merged from 1.0 branch:
+
+- ``[bug]`` JAVA-325: Fix periodic reconnection to down hosts.
+
+
+### 2.0.2
+
+- ``[api]`` The type of the map key returned by NoHostAvailable#getErrors has changed from
+  InetAddress to InetSocketAddress. Same for Initializer#getContactPoints return and
+  for AuthProvider#newAuthenticator.
+- ``[api]`` JAVA-296: The default load balacing policy is now DCAwareRoundRobinPolicy, and the local
+  datacenter is automatically picked based on the first connected node. Furthermore,
+  the TokenAwarePolicy is also used by default.
+- ``[new]`` JAVA-145: New optional AddressTranslater.
+- ``[bug]`` JAVA-321: Don't remove quotes on keyspace in the query builder.
+- ``[bug]`` JAVA-320: Fix potential NPE while cluster undergo schema changes.
+- ``[bug]`` JAVA-319: Fix thread-safety of page fetching.
+- ``[bug]`` JAVA-318: Fix potential NPE using fetchMoreResults.
+
+Merged from 1.0 branch:
+
+- ``[new]`` JAVA-179: Expose the name of the partitioner in use in the cluster metadata.
+- ``[new]`` Add new WhiteListPolicy to limit the nodes connected to a particular list.
+- ``[improvement]`` JAVA-289: Do not hop DC for LOCAL_* CL in DCAwareRoundRobinPolicy.
+- ``[bug]`` JAVA-313: Revert back to longs for dates in the query builder.
+- ``[bug]`` JAVA-314: Don't reconnect to nodes ignored by the load balancing policy.
+
+
+### 2.0.1
+
+- ``[improvement]`` JAVA-278: Handle the static columns introduced in Cassandra 2.0.6.
+- ``[improvement]`` JAVA-208: Add Cluster#newSession method to create Session without connecting
+  right away.
+- ``[bug]`` JAVA-279: Add missing iso8601 patterns for parsing dates.
+- ``[bug]`` Properly parse BytesType as the blob type.
+- ``[bug]`` JAVA-280: Potential NPE when parsing schema of pre-CQL tables of C* 1.2 nodes.
+
+Merged from 1.0 branch:
+
+- ``[bug]`` JAVA-275: LatencyAwarePolicy.Builder#withScale doesn't set the scale.
+- ``[new]`` JAVA-114: Add methods to check if a Cluster/Session instance has been closed already.
+
+
+### 2.0.0
+
+- ``[api]`` JAVA-269: Case sensitive identifier by default in Metadata.
+- ``[bug]`` JAVA-274: Fix potential NPE in Cluster#connect.
+
+Merged from 1.0 branch:
+
+- ``[bug]`` JAVA-263: Always return the PreparedStatement object that is cache internally.
+- ``[bug]`` JAVA-261: Fix race when multiple connect are done in parallel.
+- ``[bug]`` JAVA-270: Don't connect at all to nodes that are ignored by the load balancing
+  policy.
+
+
+### 2.0.0-rc3
+
+- ``[improvement]`` The protocol version 1 is now supported (features only supported by the
+  version 2 of the protocol throw UnsupportedFeatureException).
+- ``[improvement]`` JAVA-195: Make most main objects interface to facilitate testing/mocking.
+- ``[improvement]`` Adds new getStatements and clear methods to BatchStatement.
+- ``[api]`` JAVA-247: Renamed shutdown to closeAsync and ShutdownFuture to CloseFuture. Clustering
+  and Session also now implement Closeable.
+- ``[bug]`` JAVA-232: Fix potential thread leaks when shutting down Metrics.
+- ``[bug]`` JAVA-231: Fix potential NPE in HostConnectionPool.
+- ``[bug]`` JAVA-244: Avoid NPE when node is in an unconfigured DC.
+- ``[bug]`` JAVA-258: Don't block for scheduled reconnections on Cluster#close.
+
+Merged from 1.0 branch:
+
+- ``[new]`` JAVA-224: Added Session#prepareAsync calls.
+- ``[new]`` JAVA-249: Added Cluster#getLoggedKeyspace.
+- ``[improvement]`` Avoid preparing a statement multiple time per host with multiple sessions.
+- ``[bug]`` JAVA-255: Make sure connections are returned to the right pools.
+- ``[bug]`` JAVA-264: Use date string in query build to work-around CASSANDRA-6718.
+
+
+### 2.0.0-rc2
+
+- ``[new]`` JAVA-207: Add LOCAL_ONE consistency level support (requires using C* 2.0.2+).
+- ``[bug]`` JAVA-219: Fix parsing of counter types.
+- ``[bug]`` JAVA-218: Fix missing whitespace for IN clause in the query builder.
+- ``[bug]`` JAVA-221: Fix replicas computation for token aware balancing.
+
+Merged from 1.0 branch:
+
+- ``[bug]`` JAVA-213: Fix regression from JAVA-201.
+- ``[improvement]`` New getter to obtain a snapshot of the scores maintained by
+  LatencyAwarePolicy.
+
+
+### 2.0.0-rc1
+
+- ``[new]`` JAVA-199: Mark compression dependencies optional in maven.
+- ``[api]`` Renamed TableMetadata#getClusteringKey to TableMetadata#getClusteringColumns.
+
+Merged from 1.0 branch:
+
+- ``[new]`` JAVA-142: OSGi bundle.
+- ``[improvement]`` JAVA-205: Make collections returned by Row immutable.
+- ``[improvement]`` JAVA-203: Limit internal thread pool size.
+- ``[bug]`` JAVA-201: Don't retain unused PreparedStatement in memory.
+- ``[bug]`` Add missing clustering order info in TableMetadata
+- ``[bug]`` JAVA-196: Allow bind markers for collections in the query builder.
+
+
+### 2.0.0-beta2
+
+- ``[api]`` BoundStatement#setX(String, X) methods now set all values (if there is
+  more than one) having the provided name, not just the first occurence.
+- ``[api]`` The Authenticator interface now has a onAuthenticationSuccess method that
+  allows to handle the potential last token sent by the server.
+- ``[new]`` The query builder don't serialize large values to strings anymore by
+  default by making use the new ability to send values alongside the query string.
+- ``[new]`` JAVA-140: The query builder has been updated for new CQL features.
+- ``[bug]`` Fix exception when a conditional write timeout C* side.
+- ``[bug]`` JAVA-182: Ensure connection is created when Cluster metadata are asked for.
+- ``[bug]`` JAVA-187: Fix potential NPE during authentication.
+
+
+### 2.0.0-beta1
+
+- ``[api]`` The 2.0 version is an API-breaking upgrade of the driver. While most
+  of the breaking changes are minor, there are too numerous to be listed here
+  and you are encouraged to look at the Upgrade_guide_to_2.0 file that describe
+  those changes in details.
+- ``[new]`` LZ4 compression is supported for the protocol.
+- ``[new]`` JAVA-39: The driver does not depend on cassandra-all anymore.
+- ``[new]`` New BatchStatement class allows to execute batch other statements.
+- ``[new]`` Large ResultSet are now paged (incrementally fetched) by default.
+- ``[new]`` SimpleStatement support values for bind-variables, to allow
+  prepare+execute behavior with one roundtrip.
+- ``[new]`` Query parameters defaults (Consistency level, page size, ...) can be
+  configured globally.
+- ``[new]`` New Cassandra 2.0 SERIAL and LOCAL_SERIAL consistency levels are
+  supported.
+- ``[new]`` JAVA-116: Cluster#shutdown now waits for ongoing queries to complete by default.
+- ``[new]`` Generic authentication through SASL is now exposed.
+- ``[bug]`` JAVA-88: TokenAwarePolicy now takes all replica into account, instead of only the
+  first one.
+
+
+### 1.0.5
+
+- ``[new]`` JAVA-142: OSGi bundle.
+- ``[new]`` JAVA-207: Add support for ConsistencyLevel.LOCAL_ONE; note that this
+  require Cassandra 1.2.12+.
+- ``[improvement]`` JAVA-205: Make collections returned by Row immutable.
+- ``[improvement]`` JAVA-203: Limit internal thread pool size.
+- ``[improvement]`` New getter to obtain a snapshot of the scores maintained by
+  LatencyAwarePolicy.
+- ``[improvement]`` JAVA-222: Avoid synchronization when getting codec for collection
+  types.
+- ``[bug]`` JAVA-201, JAVA-213: Don't retain unused PreparedStatement in memory.
+- ``[bug]`` Add missing clustering order info in TableMetadata
+- ``[bug]`` JAVA-196: Allow bind markers for collections in the query builder.
+
+
+### 1.0.4
+
+- ``[api]`` JAVA-163: The Cluster.Builder#poolingOptions and Cluster.Builder#socketOptions
+  are now deprecated. They are replaced by the new withPoolingOptions and
+  withSocketOptions methods.
+- ``[new]`` JAVA-129: A new LatencyAwarePolicy wrapping policy has been added, allowing to
+  add latency awareness to a wrapped load balancing policy.
+- ``[new]`` JAVA-161: Cluster.Builder#deferInitialization: Allow defering cluster initialization.
+- ``[new]`` JAVA-117: Add truncate statement in query builder.
+- ``[new]`` JAVA-106: Support empty IN in the query builder.
+- ``[bug]`` JAVA-166: Fix spurious "No current pool set; this should not happen" error
+  message.
+- ``[bug]`` JAVA-184: Fix potential overflow in RoundRobinPolicy and correctly errors if
+  a balancing policy throws.
+- ``[bug]`` Don't release Stream ID for timeouted queries (unless we do get back
+  the response)
+- ``[bug]`` Correctly escape identifiers and use fully qualified table names when
+  exporting schema as string.
+
+
+### 1.0.3
+
+- ``[api]`` The query builder now correctly throw an exception when given a value
+  of a type it doesn't know about.
+- ``[new]`` SocketOptions#setReadTimeout allows to set a timeout on how long we
+  wait for the answer of one node. See the javadoc for more details.
+- ``[new]`` New Session#prepare method that takes a Statement.
+- ``[bug]`` JAVA-143: Always take per-query CL, tracing, etc. into account for QueryBuilder
+  statements.
+- ``[bug]`` Temporary fixup for TimestampType when talking to C* 2.0 nodes.
+
+
+### 1.0.2
+
+- ``[api]`` Host#getMonitor and all Host.HealthMonitor methods have been
+  deprecated. The new Host#isUp method is now prefered to the method
+  in the monitor and you should now register Host.StateListener against
+  the Cluster object directly (registering against a host HealthMonitor
+  was much more limited anyway).
+- ``[new]`` JAVA-92: New serialize/deserialize methods in DataType to serialize/deserialize
+  values to/from bytes.
+- ``[new]`` JAVA-128: New getIndexOf() method in ColumnDefinitions to find the index of
+  a given column name.
+- ``[bug]`` JAVA-131: Fix a bug when thread could get blocked while setting the current
+  keyspace.
+- ``[bug]`` JAVA-136: Quote inet addresses in the query builder since CQL3 requires it.
+
+
+### 1.0.1
+
+- ``[api]`` JAVA-100: Function call handling in the query builder has been modified in a
+  backward incompatible way. Function calls are not parsed from string values
+  anymore as this wasn't safe. Instead the new 'fcall' method should be used.
+- ``[api]`` Some typos in method names in PoolingOptions have been fixed in a
+  backward incompatible way before the API get widespread.
+- ``[bug]`` JAVA-123: Don't destroy composite partition key with BoundStatement and
+  TokenAwarePolicy.
+- ``[new]`` null values support in the query builder.
+- ``[new]`` JAVA-5: SSL support (requires C* >= 1.2.1).
+- ``[new]`` JAVA-113: Allow generating unlogged batch in the query builder.
+- ``[improvement]`` Better error message when no host are available.
+- ``[improvement]`` Improves performance of the stress example application been.
+
+
+### 1.0.0
+
+- ``[api]`` The AuthInfoProvider has be (temporarily) removed. Instead, the
+  Cluster builder has a new withCredentials() method to provide a username
+  and password for use with Cassandra's PasswordAuthenticator. Custom
+  authenticator will be re-introduced in a future version but are not
+  supported at the moment.
+- ``[api]`` The isMetricsEnabled() method in Configuration has been replaced by
+  getMetricsOptions(). An option to disabled JMX reporting (on by default)
+  has been added.
+- ``[bug]`` JAVA-91: Don't make default load balancing policy a static singleton since it
+  is stateful.
+
+
+### 1.0.0-RC1
+
+- ``[new]`` JAVA-79: Null values are now supported in BoundStatement (but you will need at
+  least Cassandra 1.2.3 for it to work). The API of BoundStatement has been
+  slightly changed so that not binding a variable is not an error anymore,
+  the variable is simply considered null by default. The isReady() method has
+  been removed.
+- ``[improvement]`` JAVA-75: The Cluster/Session shutdown methods now properly block until
+  the shutdown is complete. A version with at timeout has been added.
+- ``[bug]`` JAVA-44: Fix use of CQL3 functions in the query builder.
+- ``[bug]`` JAVA-77: Fix case where multiple schema changes too quickly wouldn't work
+  (only triggered when 0.0.0.0 was used for the rpc_address on the Cassandra
+  nodes).
+- ``[bug]`` JAVA-72: Fix IllegalStateException thrown due to a reconnection made on an I/O
+  thread.
+- ``[bug]`` JAVA-82: Correctly reports errors during authentication phase.
+
+
+### 1.0.0-beta2
+
+- ``[new]`` JAVA-51, JAVA-60, JAVA-58: Support blob constants, BigInteger, BigDecimal and counter batches in
+  the query builder.
+- ``[new]`` JAVA-61: Basic support for custom CQL3 types.
+- ``[new]`` JAVA-65: Add "execution infos" for a result set (this also move the query
+  trace in the new ExecutionInfos object, so users of beta1 will have to
+  update).
+- ``[bug]`` JAVA-62: Fix failover bug in DCAwareRoundRobinPolicy.
+- ``[bug]`` JAVA-66: Fix use of bind markers for routing keys in the query builder.
+
+
+### 1.0.0-beta1
+
+- initial release
diff --git a/docs/source/conf.py b/docs/source/conf.py
new file mode 100644
index 00000000000..44dba55e341
--- /dev/null
+++ b/docs/source/conf.py
@@ -0,0 +1,250 @@
+# -*- coding: utf-8 -*-
+#
+# This is a Scylla Documentation build configuration file, created by
+# sphinx-quickstart on Tue Jul 26 13:41:02 2016.
+#
+# This file is execfile()d with the current directory set to its
+# containing dir.
+#
+# Note that not all possible configuration values are present in this
+# autogenerated file.
+#
+# All configuration values have a default; values that are commented out
+# serve to show the default.
+
+# If extensions (or modules to document with autodoc) are in another directory,
+# add these directories to sys.path here. If the directory is relative to the
+# documentation root, use os.path.abspath to make it absolute, like shown here.
+#
+import os
+import sys
+import yaml
+import re
+from docutils import nodes
+from sphinx.util import logging
+import recommonmark
+from recommonmark.transform import AutoStructify
+
+logger = logging.getLogger(__name__)
+
+# If extensions (or modules to document with autodoc) are in another directory,
+# add these directories to sys.path here. If the directory is relative to the
+# documentation root, use os.path.abspath to make it absolute, like shown here.
+# sys.path.insert(0, os.path.abspath('../..'))
+
+# Generate a redirection HTML file
+def write_html_redirect(redirect_to):
+    html = ""
+    return html
+
+# Read a YAML dictionary of redirections and generate an HTML file for each
+redirects_file = "_utils/redirections.yaml"
+
+def create_redirects(app, docname):
+    if not os.path.exists(redirects_file):
+        return
+    if not app.builder.name == 'dirhtml':
+        return
+    with open(redirects_file, 'r') as yaml_file:
+        for from_path, redirect_to in yaml.full_load(yaml_file).items():
+            target_path = app.outdir + '/' + from_path
+            if not os.path.exists(target_path):
+                os.makedirs(target_path)
+            with open(os.path.join(target_path + '/index.html'), 'w') as t_file:
+                t_file.write(write_html_redirect(redirect_to))
+
+# -- General configuration ------------------------------------------------
+
+# Add any Sphinx extension'¡' module names here, as strings. They can be
+# extensions coming with Sphinx (named 'sphinx.ext.*') or your custom
+# ones.
+extensions = [
+    'sphinx.ext.todo',
+    'sphinx.ext.mathjax',
+    'sphinx.ext.githubpages',
+    'sphinx.ext.extlinks',
+    'sphinx.ext.autosectionlabel',
+    'sphinx_scylladb_theme',
+    'sphinx_multiversion',
+    'recommonmark'
+]
+
+# Add Markdown support
+source_suffix = ['.rst', '.md']
+autosectionlabel_prefix_document = True
+
+def setup(app):
+    app.add_config_value('recommonmark_config', {
+        'enable_eval_rst': True,
+        'enable_auto_toc_tree': False,
+    }, True)
+    app.add_transform(AutoStructify)
+    app.connect('build-finished', create_redirects)
+
+# The master toctree document.
+master_doc = 'index'
+
+# General information about the project.
+project = 'Scylla Java Driver'
+copyright = u'2012, ScyllaDB. All rights reserved.'
+author = u'Scylla Project Contributors'
+
+# The version info for the project you're documenting, acts as replacement for
+# |version| and |release|, also used in various other places throughout the
+# built documents.
+#
+# The short X.Y version.
+version = u'3.7.1'
+# The full version, including alpha/beta/rc tags.
+release = u'3.7.1'
+
+# The language for content autogenerated by Sphinx. Refer to documentation
+# for a list of supported languages.
+language = None
+
+# List of patterns, relative to source directory, that match files and
+# directories to ignore when looking for source files.
+# This patterns also effect to html_static_path and html_extra_path
+exclude_patterns = ['_build', 'Thumbs.db', '.DS_Store', 'lib', 'lib64','**/_common/*', 'README.md', '.git', '.github', '_utils', '_templates', 'rst_include']
+
+# The name of the Pygments (syntax highlighting) style to use.
+pygments_style = 'sphinx'
+
+# If true, `todo` and `todoList` produce output, else they produce nothing.
+todo_include_todos = True
+
+
+# -- Options for HTML output ----------------------------------------------
+
+# The theme to use for HTML and HTML Help pages.  See the documentation for
+# a list of builtin themes.
+#
+html_theme = 'sphinx_scylladb_theme'
+# html_theme_path = ["../.."]
+
+html_style = ''
+
+# Theme options are theme-specific and customize the look and feel of a theme
+# further.  For a list of options available for each theme, see the
+# documentation.
+#
+html_theme_options = {
+    'header_links': [
+    ('Scylla Java Driver', 'https://scylladb.github.io/java-driver/'),
+    ('Scylla Cloud', 'https://docs.scylladb.com/scylla-cloud/'),
+    ('Scylla University', 'https://university.scylladb.com/'),
+    ('ScyllaDB Home', 'https://www.scylladb.com/')],
+    'github_issues_repository': 'scylladb/java-driver',
+    'show_sidebar_index': True,
+}
+
+# Add any paths that contain custom static files (such as style sheets) here,
+# relative to this directory. They are copied after the builtin static files,
+# so a file named "default.css" will overwrite the builtin "default.css".
+# html_static_path = ['_static']
+
+# Add any extra paths that contain custom files (such as robots.txt or
+# .htaccess) here, relative to this directory. These files are copied
+# directly to the root of the documentation.
+#
+#html_extra_path = ['404.html']
+
+# If not None, a 'Last updated on:' timestamp is inserted at every page
+# bottom, using the given strftime format.
+# The empty string is equivalent to '%b %d, %Y'.
+#
+html_last_updated_fmt = '%d %B %Y'
+
+# Custom sidebar templates, maps document names to template names.
+#
+html_sidebars = {'**': ['side-nav.html']}
+
+# Output file base name for HTML help builder.
+htmlhelp_basename = 'ScyllaDocumentationdoc'
+
+# -- Options for LaTeX output ---------------------------------------------
+
+latex_elements = {
+     # The paper size ('letterpaper' or 'a4paper').
+     #
+     # 'papersize': 'letterpaper',
+
+     # The font size ('10pt', '11pt' or '12pt').
+     #
+     # 'pointsize': '10pt',
+
+     # Additional stuff for the LaTeX preamble.
+     #
+     # 'preamble': '',
+
+     # Latex figure (float) alignment
+     #
+     # 'figure_align': 'htbp',
+}
+
+# Grouping the document tree into LaTeX files. List of tuples
+# (source start file, target name, title,
+#  author, documentclass [howto, manual, or own class]).
+latex_documents = [
+    (master_doc, 'ScyllaDocumentation.tex', u'Scylla Documentation Documentation',
+     u'Scylla Project Contributors', 'manual'),
+]
+
+# -- Options for manual page output ---------------------------------------
+
+# One entry per manual page. List of tuples
+# (source start file, name, description, authors, manual section).
+man_pages = [
+    (master_doc, 'scylladocumentation', u'Scylla Documentation Documentation',
+     [author], 1)
+]
+
+# -- Options for Texinfo output -------------------------------------------
+
+# Grouping the document tree into Texinfo files. List of tuples
+# (source start file, target name, title, author,
+#  dir menu entry, description, category)
+texinfo_documents = [
+    (master_doc, 'ScyllaDocumentation', u'Scylla Documentation Documentation',
+     author, 'ScyllaDocumentation', 'One line description of project.',
+     'Miscellaneous'),
+]
+
+# -- Options for Epub output ----------------------------------------------
+
+# Bibliographic Dublin Core info.
+epub_title = project
+epub_author = author
+epub_publisher = author
+epub_copyright = copyright
+
+# A list of files that should not be packed into the epub file.
+epub_exclude_files = ['search.html']
+
+extlinks = {
+    'manager': ('/operating-scylla/manager/%s/',''),
+    'manager_lst': ('/operating-scylla/manager/2.0/%s/',''),
+    'monitor': ('/operating-scylla/monitoring/%s/',''),
+    'monitor_lst': ('/operating-scylla/monitoring/3.1/%s/','')
+}
+
+#Adds version variables for monitoring and manager versions when used in inline text
+
+rst_epilog = """
+.. |mon_version| replace:: 3.1
+.. |man_version| replace:: 2.0
+.. |mon_root| replace::  :doc:`Scylla Monitoring Stack `
+"""
+
+# -- Options for multiversion --------------------------------------------
+# Whitelist pattern for tags (set to None to ignore all tags)
+smv_tag_whitelist = r'None'
+# Whitelist pattern for branches (set to None to ignore all branches)
+smv_branch_whitelist = r"^latest$"
+
+# Whitelist pattern for remotes (set to None to use local branches only)
+smv_remote_whitelist = r"^origin$"
+# Pattern for released versions
+smv_released_pattern = r'^tags/.*$'
+# Format for versioned output directories inside the build directory
+smv_outputdir_format = '{ref.name}'
diff --git a/faq/README.md b/docs/source/faq/index.md
similarity index 94%
rename from faq/README.md
rename to docs/source/faq/index.md
index dd76299e874..12c35e5b6d1 100644
--- a/faq/README.md
+++ b/docs/source/faq/index.md
@@ -2,10 +2,10 @@
 
 ### How do I implement paging?
 
-When using [native protocol](../manual/native_protocol/) version 2 or
+When using [native protocol](../manual/native_protocol/index) version 2 or
 higher, the driver automatically pages large result sets under the hood.
 You can also save the paging state to resume iteration later. See [this
-page](../manual/paging/) for more information.
+page](../manual/paging/index) for more information.
 
 Native protocol v1 does not support paging, but you can emulate it in
 CQL with `LIMIT` and the `token()` function. See
@@ -49,7 +49,7 @@ session.execute( "INSERT INTO contacts (email, firstname, lastname)
       VALUES (?, ?, ?)", "clint.barton@hawkeye.com", "Barney", "Barton");
 ```
 
-See [Simple statements](../manual/statements/simple/) for more information.
+See [Simple statements](../manual/statements/simple/index) for more information.
 
 
 ### Does a parameterized statement escape parameters?
@@ -69,7 +69,7 @@ the way that the parameters are sent. The difference is that a prepared statemen
 * sends only the statement id and its parameters (thus reducing the amount of data sent
   to the cluster)
 
-See [Prepared statements](../manual/statements/prepared/) for more information.
+See [Prepared statements](../manual/statements/prepared/index) for more information.
 
 
 ### Can I combine `PreparedStatements` and normal statements in a batch?
@@ -96,7 +96,7 @@ is currently no mechanism for Cassandra to invalidate the existing metadata.  Be
 the driver is not able to properly react to these changes and will improperly read rows after
 a schema change is made.
 
-See [Prepared statements](../manual/statements/prepared) for more information.
+See [Prepared statements](../manual/statements/prepared/index) for more information.
 
 
 ### Can I get the raw bytes of a text column?
@@ -135,7 +135,7 @@ Keep in mind that if your code iterates the `ResultSet` entirely, the driver may
 run additional background queries to fetch the rest of the data. The fetch size
 only affects what is retrieved at a time, not the overall number of rows.
 
-See [Paging](../manual/paging/) for more information.
+See [Paging](../manual/paging/index) for more information.
 
 
 ### What's the difference between using `setFetchSize()` and `LIMIT`?
@@ -147,7 +147,7 @@ For example, if you limit is 30 and your fetch size is 10, the `ResultSet` will
 30 rows, but under the hood the driver will perform 3 requests that will transfer 10
 rows each.
 
-See [Paging](../manual/paging/) for more information.
+See [Paging](../manual/paging/index) for more information.
 
 
 ### I'm reading a BLOB column and the driver returns incorrect data.
@@ -162,7 +162,7 @@ See [Blobs.java] in the `driver-examples` module for some examples and explanati
 
 ### How do I use the driver in an OSGi application?
 
-Read our [OSGi-specific FAQ section](osgi/) to find out.
+Read our [OSGi-specific FAQ section](osgi/index) to find out.
 
 
 ### Why am I seeing messages about `tombstone_warn_threshold` or `tombstone_fail_threshold` being exceeded in my Cassandra logs?
@@ -291,11 +291,11 @@ an update with compatibility to this driver version.
 
 [Blobs.java]: https://github.com/datastax/java-driver/tree/3.7.1/driver-examples/src/main/java/com/datastax/driver/examples/datatypes/Blobs.java
 [CASSANDRA-7304]: https://issues.apache.org/jira/browse/CASSANDRA-7304
-[Parameters and Binding]: ../manual/statements/prepared/#parameters-and-binding
-[Mapper options]: ../manual/object_mapper/using/#mapper-options
-[Acquisition queue]: ../manual/pooling/#acquisition-queue
+[Parameters and Binding]: ../manual/statements/prepared/index#parameters-and-binding
+[Mapper options]: ../manual/object_mapper/using/index#mapper-options
+[Acquisition queue]: ../manual/pooling/index#acquisition-queue
 [Semaphore]: https://docs.oracle.com/javase/8/docs/api/java/util/concurrent/Semaphore.html
 [Futures.allAsList]: https://google.github.io/guava/releases/19.0/api/docs/com/google/common/util/concurrent/Futures.html#allAsList(java.lang.Iterable)
 [DynaTrace]: https://www.dynatrace.com/
 [packaging data]: https://logback.qos.ch/reasonsToSwitch.html#packagingData
-[Using the shaded JAR]: ../manual/shaded_jar
+[Using the shaded JAR]: ../manual/shaded_jar/index
diff --git a/faq/osgi/README.md b/docs/source/faq/osgi/index.md
similarity index 95%
rename from faq/osgi/README.md
rename to docs/source/faq/osgi/index.md
index 18e1cfb831b..aaa7bd5b72f 100644
--- a/faq/osgi/README.md
+++ b/docs/source/faq/osgi/index.md
@@ -1,3 +1,7 @@
+```eval_rst
+:orphan:
+```
+
 ## Frequently Asked Questions - OSGi
 
 ### How to use the Java driver in an OSGi environment?
@@ -28,7 +32,7 @@ of Guava's packages, e.g. for 19.0:
 
 ### How to enable compression?
 
-First, read our [manual page on compression](../../manual/compression/)
+First, read our [manual page on compression](../../manual/compression/index)
 to understand how to enable compression for the Java driver.
             
 OSGi projects can use both Snappy or LZ4 compression algorithms. 
@@ -95,14 +99,14 @@ _not_ its Maven artifact id.
     
 ### How to use the driver shaded jar?
 
-The driver [shaded jar](../../manual/shaded_jar/) can be used 
+The driver [shaded jar](../../manual/shaded_jar/index) can be used 
 in any OSGi application, although the same limitations explained in
 the manual apply.
 
 
 ### How to get proper logs?
 
-The driver uses [SLF4j] for [logging](../../manual/logging/).
+The driver uses [SLF4j] for [logging](../../manual/logging/index).
 
 You OSGi runtime should therefore include the SLF4J API bundle, and
 one valid implementation bundle, such as [Logback].
@@ -131,7 +135,7 @@ Please consult their documentation for further details.
 
 The driver is able to perform native system calls through JNR in some cases,
 for example to achieve microsecond resolution when 
-[generating timestamps](../../manual/query_timestamps/).
+[generating timestamps](../../manual/query_timestamps/index).
 
 Unfortunately, some of the JNR artifacts available from Maven 
 are not valid OSGi bundles and cannot be used in OSGi applications.
diff --git a/docs/source/index.rst b/docs/source/index.rst
new file mode 100644
index 00000000000..8871edff250
--- /dev/null
+++ b/docs/source/index.rst
@@ -0,0 +1,99 @@
+Datastax Java Driver for Apache Cassandra®
+==========================================
+
+A modern, `feature-rich`_ and highly tunable Java client library for
+Apache Cassandra (2.1+) and using exclusively Cassandra's binary
+protocol and Cassandra Query Language v3. Use the `DataStax Enterprise
+Java driver `_ for better compatibility and support for
+DataStax Enterprise.
+
+**Features:**
+
+-  `Sync`_ and `Async`_ API
+-  `Simple`_, `Prepared`_, and `Batch`_ statements
+-  Asynchronous IO, parallel execution, request pipelining
+-  `Connection pooling`_
+-  Auto node discovery
+-  Automatic reconnection
+-  Configurable `load balancing`_ and `retry policies`_
+-  Works with any cluster size
+-  `Query builder`_
+-  `Object mapper`_
+
+The driver architecture is based on layers. At the bottom lies the
+driver core. This core handles everything related to the connections to
+a Cassandra cluster (for example, connection pool, discovering new
+nodes, etc.) and exposes a simple, relatively low-level API on top of
+which higher level layers can be built.
+
+The driver contains the following modules:
+
+-  driver-core: the core layer.
+-  driver-mapping: the object mapper.
+-  driver-extras: optional features for the Java driver.
+-  driver-examples: example applications using the other modules which
+   are only meant for demonstration purposes.
+-  driver-tests: tests for the java-driver.
+
+**Useful links:**
+
+-  JIRA (bug tracking): https://datastax-oss.atlassian.net/browse/JAVA
+-  MAILING LIST:
+   https://groups.google.com/a/lists.datastax.com/forum/#!forum/java-driver-user
+-  DATASTAX ACADEMY SLACK: #datastax-drivers on
+   https://academy.datastax.com/slack
+-  TWITTER: `@dsJavaDriver`_ tweets Java driver releases and important
+   announcements (low frequency). `@DataStaxEng`_ has more news
+   including other drivers, Cassandra, and DSE.
+-  DOCS: the `manual`_ has quick start material and technical details
+   about the driver and its features.
+-  API: http://www.datastax.com/drivers/java/3.7
+-  GITHUB REPOSITORY: https://github.com/datastax/java-driver
+-  `changelog`_
+-  `binary tarball`_
+
+.. _feature-rich: manual/index
+.. _Sync: manual/index
+.. _Async: manual/async/index
+.. _Simple: manual/statements/simple/index
+.. _Prepared: manual/statements/prepared/index
+.. _Batch: manual/statements/batch/index
+.. _Connection pooling: manual/pooling/index
+.. _load balancing: manual/load_balancing/index
+.. _retry policies: manual/retries/index
+.. _Query builder: manual/statements/built/index
+.. _Object mapper: manual/object_mapper/index
+.. _@dsJavaDriver: https://twitter.com/dsJavaDriver
+.. _@DataStaxEng: https://twitter.com/datastaxeng
+.. _manual: http://docs.datastax.com/en/developer/java-driver/3.7/manual/
+.. _changelog: changelog/index
+.. _binary tarball: http://downloads.datastax.com/java-driver/cassandra-java-driver-3.7.1.tar.gz
+
+
+License
+-------
+
+Copyright 2012-2018, DataStax
+
+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.
+
+.. toctree::
+   :hidden:
+   
+   api
+   installation/index
+   manual/index
+   upgrade_guide/index
+   faq/index
+   changelog/index
+
diff --git a/docs/source/installation/index.md b/docs/source/installation/index.md
new file mode 100644
index 00000000000..00b1897537a
--- /dev/null
+++ b/docs/source/installation/index.md
@@ -0,0 +1,76 @@
+## Installation
+
+### Getting the driver
+
+The last release of the driver is available on Maven Central. You can install
+it in your application using the following Maven dependency (_if
+using DataStax Enterprise, install the [DataStax Enterprise Java driver][dse-driver] instead_):
+
+```xml
+
+  com.scylladb
+  scylla-driver-core
+  3.7.1-scylla-0-SNAPSHOT
+
+```
+
+Note that the object mapper is published as a separate artifact:
+
+```xml
+
+  com.scylladb
+  scylla-driver-mapping
+  3.7.1-scylla-0-SNAPSHOT
+
+```
+
+The 'extras' module is also published as a separate artifact:
+
+```xml
+
+  com.scylladb
+  scylla-driver-extras
+  3.7.1-scylla-0-SNAPSHOT
+
+```
+
+
+We also provide a [shaded JAR](../manual/shaded_jar/index)
+to avoid the explicit dependency to Netty.
+
+If you can't use a dependency management tool, a
+[binary tarball](http://downloads.datastax.com/java-driver/cassandra-java-driver-3.7.1.tar.gz)
+is available for download.
+
+### Compatibility
+
+The Java client driver 3.7.1 ([branch 3.x](https://github.com/datastax/java-driver/tree/3.x)) is compatible with Apache
+Cassandra 2.1, 2.2 and 3.0+ (see [this page](http://docs.datastax.com/en/developer/java-driver/latest/manual/native_protocol/) for
+the most up-to-date compatibility information).
+
+UDT and tuple support is available only when using Apache Cassandra 2.1 or higher (see [CQL improvements in Cassandra 2.1](http://www.datastax.com/dev/blog/cql-in-2-1)).
+
+Other features are available only when using Apache Cassandra 2.0 or higher (e.g. result set paging,
+[BatchStatement](https://github.com/datastax/java-driver/blob/3.x/driver-core/src/main/java/com/datastax/driver/core/BatchStatement.java),
+[lightweight transactions](http://www.datastax.com/documentation/cql/3.1/cql/cql_using/use_ltweight_transaction_t.html) 
+-- see [What's new in Cassandra 2.0](http://www.datastax.com/documentation/cassandra/2.0/cassandra/features/features_key_c.html)). 
+Trying to use these with a cluster running Cassandra 1.2 will result in 
+an [UnsupportedFeatureException](https://github.com/datastax/java-driver/blob/3.x/driver-core/src/main/java/com/datastax/driver/core/exceptions/UnsupportedFeatureException.java) being thrown.
+
+The java driver supports Java JDK versions 6 and above.
+
+If using _DataStax Enterprise_, the [DataStax Enterprise Java driver][dse-driver] provides 
+more features and better compatibility.
+
+__Disclaimer__: Some _DataStax/DataStax Enterprise_ products might partially work on 
+big-endian systems, but _DataStax_ does not officially support these systems.
+
+### Upgrading from previous versions
+
+If you are upgrading from a previous version of the driver, be sure to have a look at
+the [upgrade guide](../upgrade_guide/index).
+
+If you are upgrading to _DataStax Enterprise_, use the [DataStax Enterprise Java driver][dse-driver] for more
+features and better compatibility.
+
+[dse-driver]: https://docs.datastax.com/en/developer/java-driver-dse/
diff --git a/manual/address_resolution/README.md b/docs/source/manual/address_resolution/index.md
similarity index 98%
rename from manual/address_resolution/README.md
rename to docs/source/manual/address_resolution/index.md
index 2f55461bd96..e7e1c28fc39 100644
--- a/manual/address_resolution/README.md
+++ b/docs/source/manual/address_resolution/index.md
@@ -5,7 +5,7 @@ connections.
 
 * for contact points, these are provided as part of configuring the `Cluster` object;
 * for other nodes, addresses will be discovered dynamically, either by inspecting `system.peers` on already connected
-  nodes, or via push notifications received from the [control host](../control_connection) when new nodes are discovered
+  nodes, or via push notifications received from the [control host](../control_connection/index) when new nodes are discovered
   by gossip.
 
 
diff --git a/manual/async/README.md b/docs/source/manual/async/index.md
similarity index 98%
rename from manual/async/README.md
rename to docs/source/manual/async/index.md
index a50d3180bbb..a58e120424b 100644
--- a/manual/async/README.md
+++ b/docs/source/manual/async/index.md
@@ -48,7 +48,7 @@ If you consume a `ResultSet` in a callback, be aware that iterating the
 rows will trigger synchronous queries as you page through the results.
 To avoid this, use [getAvailableWithoutFetching] to limit the iteration
 to the current page, and [fetchMoreResults] to get a future to the next
-page (see also the section on [paging](../paging/)).
+page (see also the section on [paging](../paging/index)).
 Here is a full example:
 
 [getAvailableWithoutFetching]: http://docs.datastax.com/en/drivers/java/3.7/com/datastax/driver/core/ResultSet.html#getAvailableWithoutFetching--
diff --git a/manual/auth/README.md b/docs/source/manual/auth/index.md
similarity index 100%
rename from manual/auth/README.md
rename to docs/source/manual/auth/index.md
diff --git a/manual/compression/README.md b/docs/source/manual/compression/index.md
similarity index 92%
rename from manual/compression/README.md
rename to docs/source/manual/compression/index.md
index b6dba5c73d7..d359fef7df4 100644
--- a/manual/compression/README.md
+++ b/docs/source/manual/compression/index.md
@@ -4,7 +4,7 @@ Cassandra's binary protocol supports optional compression of
 transport-level requests and responses, for example:
 
 * a query with its serialized parameters;
-* a [page](../paging/) from a result set, i.e. a list of serialized
+* a [page](../paging/index) from a result set, i.e. a list of serialized
   rows.
 
 It reduces network traffic at the cost of CPU overhead, therefore it
@@ -12,10 +12,10 @@ will likely be beneficial when you have larger payloads.
 
 Two algorithms are available:
 [LZ4](https://github.com/jpountz/lz4-java) and
-[Snappy](https://code.google.com/p/snappy/).  The LZ4 implementation is a good
+[Snappy](https://code.google.com/p/snappy/index).  The LZ4 implementation is a good
 first choice; it offers fallback implementations in case native libraries fail
 to load and
-[benchmarks](http://java-performance.info/performance-general-compression/)
+[benchmarks](http://java-performance.info/performance-general-compression/index)
 suggest that it offers better performance and compression ratios over Snappy.
 Both rely on third-party libraries, declared by the driver as *optional*
 dependencies. So if you use a build tool like Maven, you'll need to declare an
@@ -55,7 +55,7 @@ LZ4-java has three internal implementations (from fastest to slowest):
 
 It will pick the best implementation depending on what's possible on
 your platform. To find out which one was chosen, [enable INFO
-logs](../logging/) on the category
+logs](../logging/index) on the category
 `com.datastax.driver.core.FrameCompressor` and look for a log similar to
 this:
 
diff --git a/manual/control_connection/README.md b/docs/source/manual/control_connection/index.md
similarity index 67%
rename from manual/control_connection/README.md
rename to docs/source/manual/control_connection/index.md
index 6b6385fd30a..1f23bfeb52e 100644
--- a/manual/control_connection/README.md
+++ b/docs/source/manual/control_connection/index.md
@@ -3,19 +3,19 @@
 The control connection is a dedicated connection used for administrative tasks:
 
 * querying system tables to learn about the cluster's topology and
-  [schema](../metadata/#schema-metadata);
-* checking [schema agreement](../metadata/#schema-agreement);
+  [schema](../metadata/index#schema-metadata);
+* checking [schema agreement](../metadata/index#schema-agreement);
 * reacting to server events, which are used to notify the driver of external topology or schema
   changes.
 
 When the driver starts, the control connection is established to the first contacted node. If that
-node goes down, a [reconnection](../reconnection/) is started to find another node; it is governed
+node goes down, a [reconnection](../reconnection/index) is started to find another node; it is governed
 by the same policy as regular connections and tries the nodes according to a query plan from the
-[load balancing policy](../load_balancing/).
+[load balancing policy](../load_balancing/index).
 
-The control connection is managed independently from [regular pooled connections](../pooling/), and
+The control connection is managed independently from [regular pooled connections](../pooling/index), and
 used exclusively for administrative requests. It is included in [Session.State.getOpenConnections],
-as well as the `open-connections` [metric](../metrics); for example, if you've configured a pool
+as well as the `open-connections` [metric](../metrics/index); for example, if you've configured a pool
 size of 2, the control node will have 3 connections.
 
 [Session.State.getOpenConnections]: https://docs.datastax.com/en/drivers/java/3.6/com/datastax/driver/core/Session.State.html#getOpenConnections-com.datastax.driver.core.Host-
\ No newline at end of file
diff --git a/manual/custom_codecs/extras/README.md b/docs/source/manual/custom_codecs/extras/index.md
similarity index 96%
rename from manual/custom_codecs/extras/README.md
rename to docs/source/manual/custom_codecs/extras/index.md
index af2a359e6bb..79a47fe6d49 100644
--- a/manual/custom_codecs/extras/README.md
+++ b/docs/source/manual/custom_codecs/extras/index.md
@@ -1,6 +1,10 @@
+```eval_rst
+:orphan:
+```
+
 ## Optional codecs
 
-The driver's "extras" module provides additional [codec](../) implementations: these codecs are not required by core
+The driver's "extras" module provides additional [codec](../index) implementations: these codecs are not required by core
 driver features, but will probably prove useful in a lot of client applications. You can also study their source code as
 a reference to write your own.
 
@@ -182,10 +186,10 @@ session.execute("insert into ordinal_example (id, state) values (1, ?)", State.I
 // state saved as 0
 ```
 
-Note that if you registered an `EnumNameCodec` and an `EnumOrdinalCodec` _for the same enum_ at the same time, there could be a problem when executing [simple statements](../../statements/simple/), because in a simple statement, the target CQL type of a given query parameter is not known in advance, so the driver, on a best-effort attempt, will pick one or the other, whichever was registered first. If the chosen codec proves to be the wrong one, the request would fail on the server side.
+Note that if you registered an `EnumNameCodec` and an `EnumOrdinalCodec` _for the same enum_ at the same time, there could be a problem when executing [simple statements](../../statements/simple/index), because in a simple statement, the target CQL type of a given query parameter is not known in advance, so the driver, on a best-effort attempt, will pick one or the other, whichever was registered first. If the chosen codec proves to be the wrong one, the request would fail on the server side.
 
 In practice, this is unlikely to happen, because you'll probably stick to a single CQL type for a given enum type;
-however, if you ever run into that issue, the workaround is to use [prepared statements](../../statements/prepared/), for which the driver knows the CQL type and can pick the exact codec.
+however, if you ever run into that issue, the workaround is to use [prepared statements](../../statements/prepared/index), for which the driver knows the CQL type and can pick the exact codec.
 
 [EnumNameCodec]: http://docs.datastax.com/en/drivers/java/3.7/com/datastax/driver/extras/codecs/enums/EnumNameCodec.html
 [EnumOrdinalCodec]: http://docs.datastax.com/en/drivers/java/3.7/com/datastax/driver/extras/codecs/enums/EnumOrdinalCodec.html
diff --git a/manual/custom_codecs/README.md b/docs/source/manual/custom_codecs/index.md
similarity index 99%
rename from manual/custom_codecs/README.md
rename to docs/source/manual/custom_codecs/index.md
index 15e90d48a3f..19ab0dbbaf1 100644
--- a/manual/custom_codecs/README.md
+++ b/docs/source/manual/custom_codecs/index.md
@@ -12,7 +12,7 @@ Practical use cases that justify such a feature are numerous:
 * etc.
 
 This page explains the implementation and how to write your own custom codecs. Note that the driver also provides a set
-of [optional codecs](extras/) that might fit your needs.
+of [optional codecs](extras/index) that might fit your needs.
 
 ### Overview of the serialization mechanism
 
@@ -203,7 +203,7 @@ make sure that you use preferably the `get()` and `set()` methods: they
 avoid any ambiguity by requiring the user to explicitly specify the desired Java type,
 thus forcing the driver to pick the right codec for the right task.
 
-Custom codecs also work with the driver's [object mapper](../object_mapper/custom_codecs/).
+Custom codecs also work with the driver's [object mapper](../object_mapper/custom_codecs/index).
 
 ### On-the-fly codec generation
 
@@ -364,7 +364,7 @@ serializing and deserializing. It's possible to bypass this step with a
 lower-level implementation that manipulates the binary stream directly.
 That's also how the object mapper handles UDTs, and you can rely on the
 mapper to generate UDT codecs for you; see
-[this page](../object_mapper/custom_codecs/#implicit-udt-codecs) for more
+[this page](../object_mapper/custom_codecs/index#implicit-udt-codecs) for more
 information.
 
 ### Support for generic (parameterized) types
diff --git a/manual/custom_payloads/README.md b/docs/source/manual/custom_payloads/index.md
similarity index 98%
rename from manual/custom_payloads/README.md
rename to docs/source/manual/custom_payloads/index.md
index 924c97ee188..568b50bdda3 100644
--- a/manual/custom_payloads/README.md
+++ b/docs/source/manual/custom_payloads/index.md
@@ -1,6 +1,6 @@
 ## Custom Payloads
 
-The [native protocol](../native_protocol/) version 4 introduces a new feature called [Custom Payloads][CASSANDRA-8553].
+The [native protocol](../native_protocol/index) version 4 introduces a new feature called [Custom Payloads][CASSANDRA-8553].
 
 According to the [protocol V4 specification][v4spec], custom payloads are generic key-value maps
 where keys are strings and each value is an arbitrary sequence of bytes. Currently payloads 
diff --git a/manual/idempotence/README.md b/docs/source/manual/idempotence/index.md
similarity index 97%
rename from manual/idempotence/README.md
rename to docs/source/manual/idempotence/index.md
index a130a6952cd..a93790136f7 100644
--- a/manual/idempotence/README.md
+++ b/docs/source/manual/idempotence/index.md
@@ -8,7 +8,7 @@ example:
 * `update my_table set list_col = [1] + list_col where pk = 1` is not idempotent: if `list_col` was initially empty,
   it will contain `[1]` after the first execution, `[1, 1]` after the second, etc.
 
-Idempotence matters for [retries](../retries/) and [speculative query executions](../speculative_execution/). The driver
+Idempotence matters for [retries](../retries/index) and [speculative query executions](../speculative_execution/index). The driver
 will bypass those features if the [Statement#isIdempotent()][isIdempotent] flag is set to `false`, to ensure that the
 statement does not get executed more than once.
 
diff --git a/manual/README.md b/docs/source/manual/index.md
similarity index 86%
rename from manual/README.md
rename to docs/source/manual/index.md
index 283d835933a..0b50525d6ce 100644
--- a/manual/README.md
+++ b/docs/source/manual/index.md
@@ -21,7 +21,7 @@ try {
 ```
 
 1. the [Cluster] object is the main entry point of the driver. It holds the known state of the actual Cassandra cluster
-   (notably the [Metadata](metadata/)). This class is thread-safe, you should create a single instance (per target
+   (notably the [Metadata](metadata/index)). This class is thread-safe, you should create a single instance (per target
    Cassandra cluster), and share it throughout your application;
 2. the [Session] is what you use to execute queries. Likewise, it is thread-safe and should be reused;
 3. we use `execute` to send a query to Cassandra. This returns a [ResultSet], which is essentially a collection of [Row]
@@ -31,7 +31,7 @@ try {
    cluster. This step is important because it frees underlying resources (TCP connections, thread pools...). In a real
    application, you would typically do this at shutdown (for example, when undeploying your webapp).
 
-Note: this example uses the synchronous API. Most methods have [asynchronous](async/) equivalents.
+Note: this example uses the synchronous API. Most methods have [asynchronous](async/index) equivalents.
 
 
 ### Setting up the driver
@@ -65,19 +65,19 @@ that the driver can fallback if the first one is down.
 
 The other aspects that you can configure on the `Cluster` are:
 
-* [address translation](address_resolution/);
-* [authentication](auth/);
-* [compression](compression/);
-* [load balancing](load_balancing/);
-* [metrics](metrics/);
+* [address translation](address_resolution/index);
+* [authentication](auth/index);
+* [compression](compression/index);
+* [load balancing](load_balancing/index);
+* [metrics](metrics/index);
 * low-level [Netty configuration][NettyOptions];
 * [query options][QueryOptions];
-* [reconnections](reconnection/);
-* [retries](retries/);
-* [socket options](socket_options/);
-* [SSL](ssl/);
-* [speculative executions](speculative_execution/);
-* [query timestamps](query_timestamps/).
+* [reconnections](reconnection/index);
+* [retries](retries/index);
+* [socket options](socket_options/index);
+* [SSL](ssl/index);
+* [speculative executions](speculative_execution/index);
+* [query timestamps](query_timestamps/index).
 
 In addition, you can register various types of listeners to be notified of cluster events; see [Host.StateListener],
 [LatencyTracker], and [SchemaChangeListener].
@@ -98,8 +98,8 @@ The initialization sequence is the following:
 * try to connect to each of the contact points in sequence. The order is not deterministic (in fact, the driver shuffles
   the list to avoid hotspots if a large number of clients share the same contact points). If no contact point replies,
   a [NoHostAvailableException] is thrown and the process stops here;
-* otherwise, the successful contact point is elected as the [control host](control_connection/). The driver negotiates
-  the [native protocol version](native_protocol/) with it, and queries its system tables to discover the addresses of
+* otherwise, the successful contact point is elected as the [control host](control_connection/index). The driver negotiates
+  the [native protocol version](native_protocol/index) with it, and queries its system tables to discover the addresses of
   the other hosts.
 
 Note that, at this stage, only the control connection has been established. Connections to other hosts will only be
@@ -124,7 +124,7 @@ session.execute("select * from otherKeyspace.otherTable where id = 1");
 ```
 
 You might be tempted to open a separate session for each keyspace used in your application; however, note that
-[connection pools](pooling/) are created at the session level, so each new session will consume additional system
+[connection pools](pooling/index) are created at the session level, so each new session will consume additional system
 resources:
 
 ```java
@@ -133,7 +133,7 @@ Session session1 = cluster.connect("ks1");
 Session session2 = cluster.connect("ks2");
 ```
 
-Also, there is currently a [known limitation](async/#known-limitations) with named sessions, that causes the driver to
+Also, there is currently a [known limitation](async/index#known-limitations) with named sessions, that causes the driver to
 unexpectedly block the calling thread in certain circumstances; if you use a fully asynchronous model, you should use a
 session with no keyspace.
 
@@ -164,7 +164,7 @@ ResultSet rs = session.execute("select release_version from system.local");
 ```
 
 As shown here, the simplest form is to pass a query string directly. You can also pass an instance of
-[Statement](statements/).
+[Statement](statements/index).
 
 #### Processing rows
 
@@ -179,7 +179,7 @@ for (Row row : rs) {
 
 Note that this will return **all results** without limit (even though the driver might use multiple queries in the
 background). To handle large result sets, you might want to use a `LIMIT` clause in your CQL query, or use one of the
-techniques described in the [paging](paging/) documentation.
+techniques described in the [paging](paging/index) documentation.
 
 When you know that there is only one row (or are only interested in the first one), the driver provides a convenience
 method:
@@ -231,7 +231,7 @@ String firstName = row.getString("first_name");
      varint getVarint java.math.BigInteger 
 
 
-In addition to these default mappings, you can register your own types with [custom codecs](custom_codecs/).
+In addition to these default mappings, you can register your own types with [custom codecs](custom_codecs/index).
 
 ##### Primitive types
 
@@ -280,13 +280,13 @@ for (ColumnDefinitions.Definition definition : row.getColumnDefinitions()) {
 ### Object mapping
 
 Besides explicit work with queries and rows, you can also use
-[Object Mapper](object_mapper/) to simplify retrieval & store of your data.
+[Object Mapper](object_mapper/index) to simplify retrieval & store of your data.
 
 
 ### More information
 
 If you're reading this from the [generated HTML documentation on
-github.io](http://datastax.github.io/java-driver/), use the "Contents"
+github.io](http://datastax.github.io/java-driver), use the "Contents"
 menu on the left hand side to navigate sub-sections. If you're [browsing the source files on
 github.com](https://github.com/datastax/java-driver/tree/3.x/manual),
 simply navigate to each sub-directory.
@@ -304,4 +304,38 @@ simply navigate to each sub-directory.
 [LatencyTracker]: http://docs.datastax.com/en/drivers/java/3.7/com/datastax/driver/core/LatencyTracker.html
 [SchemaChangeListener]: http://docs.datastax.com/en/drivers/java/3.7/com/datastax/driver/core/SchemaChangeListener.html
 [NoHostAvailableException]: http://docs.datastax.com/en/drivers/java/3.7/com/datastax/driver/core/exceptions/NoHostAvailableException.html
-[LocalDate]: http://docs.datastax.com/en/drivers/java/3.7/com/datastax/driver/core/LocalDate.html
\ No newline at end of file
+[LocalDate]: http://docs.datastax.com/en/drivers/java/3.7/com/datastax/driver/core/LocalDate.html
+
+```eval_rst
+.. toctree::
+   :hidden:
+   
+   address_resolution/index
+   async/index
+   auth/index
+   address_resolution/index
+   compression/index
+   control_connection/index
+   custom_codecs/index
+   custom_payloads/index
+   idempotence/index
+   load_balancing/index
+   logging/index
+   metadata/index
+   metrics/index
+   native_protocol/index
+   object_mapper/index
+   osgi/index
+   paging/index
+   pooling/index
+   query_timestamps/index
+   reconnection/index
+   retries/index
+   shaded_jar/index
+   socket_options/index
+   speculative_execution/index
+   ssl/index
+   statements/index
+   tuples/index
+   udts/index
+```
\ No newline at end of file
diff --git a/manual/load_balancing/README.md b/docs/source/manual/load_balancing/index.md
similarity index 94%
rename from manual/load_balancing/README.md
rename to docs/source/manual/load_balancing/index.md
index 40ac9bd3f34..8db03ea4925 100644
--- a/manual/load_balancing/README.md
+++ b/docs/source/manual/load_balancing/index.md
@@ -38,8 +38,8 @@ For each host, the policy computes a **[distance][HostDistance]** that determine
 to it:
 
 * `LOCAL` and `REMOTE` are "active" distances, meaning that the driver will keep open connections to the host. They
-  differ in the number of connections opened, depending on your [pooling options](../pooling/). Also, the
-  [control connection](../control_connection/) will  favor local nodes if possible.
+  differ in the number of connections opened, depending on your [pooling options](../pooling/index). Also, the
+  [control connection](../control_connection/index) will  favor local nodes if possible.
 * `IGNORED`, as the name suggests, means that the driver will not attempt to connect.
 
 Typically, the distance will reflect network topology (e.g. local vs. remote datacenter), although that is entirely up
@@ -57,8 +57,8 @@ cluster.getConfiguration().getPoolingOptions().refreshConnectedHost(host);
 #### Query plan
 
 Each time the driver executes a query, it asks the policy to compute a **query plan**, which is a list of hosts. The
-driver will then try each host in sequence, according to the [retry policy](../retries/) and
-[speculative execution policy](../speculative_execution).
+driver will then try each host in sequence, according to the [retry policy](../retries/index) and
+[speculative execution policy](../speculative_execution/index).
 
 The contents and order of query plans are entirely up to your policy, but implementations typically return plans that:
 
@@ -108,7 +108,7 @@ Cluster cluster = Cluster.builder()
 This policy queries nodes of the local data-center in a round-robin fashion.
 
 Call `withLocalDc` to specify the name of your local datacenter. You can also leave it out, and the driver will use the
-datacenter of the first contact point that was reached [at initialization](../#cluster-initialization). However,
+datacenter of the first contact point that was reached [at initialization](../index#cluster-initialization). However,
 remember that the driver shuffles the initial list of contact points, so this assumes that all contact points are in the
 local datacenter. In general, providing the datacenter name explicitly is a safer option.
 
@@ -144,7 +144,7 @@ CREATE TABLE testKs.sensor_data(id int, year int, ts timestamp, data double,
                                 PRIMARY KEY ((id, year), ts));
 ```
 
-For [simple statements](../statements/simple/), routing information can never be computed automatically:
+For [simple statements](../statements/simple/index), routing information can never be computed automatically:
 
 ```java
 SimpleStatement statement = new SimpleStatement(
@@ -166,7 +166,7 @@ statement.setRoutingKey(
 session.execute(statement);
 ```
 
-For [built statements](../statements/built/), the keyspace is available if it was provided while building the query; the
+For [built statements](../statements/built/index), the keyspace is available if it was provided while building the query; the
 routing key is available only if the statement was built using the table metadata, and all components of the partition
 key appear in the query:
 
@@ -192,7 +192,7 @@ assert statement2.getKeyspace() != null;
 assert statement2.getRoutingKey() == null;
 ```
 
-For [bound statements](../statements/prepared/), the keyspace is always available; the routing key is only available if
+For [bound statements](../statements/prepared/index), the keyspace is always available; the routing key is only available if
 all components of the partition key are bound as variables:
 
 ```java
@@ -211,7 +211,7 @@ assert statement2.getKeyspace() != null;
 assert statement2.getRoutingKey() == null;
 ```
 
-For [batch statements](../statements/batch/), the routing information of each child statement is inspected; the first
+For [batch statements](../statements/batch/index), the routing information of each child statement is inspected; the first
 non-null keyspace is used as the keyspace of the batch, and the first non-null routing key as its routing key (the idea
 is that all childs should have the same routing information, since batches are supposed to operate on a single
 partition). All children might have null information, in which case you need to provide the information manually as
diff --git a/manual/logging/README.md b/docs/source/manual/logging/index.md
similarity index 100%
rename from manual/logging/README.md
rename to docs/source/manual/logging/index.md
diff --git a/manual/metadata/README.md b/docs/source/manual/metadata/index.md
similarity index 99%
rename from manual/metadata/README.md
rename to docs/source/manual/metadata/index.md
index 0155bb82882..d355701eeb9 100644
--- a/manual/metadata/README.md
+++ b/docs/source/manual/metadata/index.md
@@ -61,7 +61,7 @@ executing a schema-altering query. This is to ensure that subsequent
 requests (which might get routed to different nodes) see an up-to-date
 version of the schema.
 
-```ditaa
+```console
  Application             Driver           Cassandra
 ------+--------------------+------------------+-----
       |                    |                  |
diff --git a/manual/metrics/README.md b/docs/source/manual/metrics/index.md
similarity index 100%
rename from manual/metrics/README.md
rename to docs/source/manual/metrics/index.md
diff --git a/manual/native_protocol/README.md b/docs/source/manual/native_protocol/index.md
similarity index 96%
rename from manual/native_protocol/README.md
rename to docs/source/manual/native_protocol/index.md
index 7ee0f4414fb..12eb17d8cfa 100644
--- a/manual/native_protocol/README.md
+++ b/docs/source/manual/native_protocol/index.md
@@ -97,17 +97,17 @@ To avoid this issue, you can use one the following workarounds:
 * bound variables in simple statements
   ([Session#execute(String, Object...)](http://docs.datastax.com/en/drivers/java/3.6/com/datastax/driver/core/Session.html#execute-java.lang.String-java.lang.Object...-))
 * [batch statements](http://docs.datastax.com/en/drivers/java/3.6/com/datastax/driver/core/BatchStatement.html)
-* [query paging](../paging/)
+* [query paging](../paging/index)
 
 #### v2 to v3
 
 * the number of stream ids per connection goes from 128 to 32768 (see
-  [Connection pooling](../pooling/))
+  [Connection pooling](../pooling/index))
 * [serial consistency on batch statements](http://docs.datastax.com/en/drivers/java/3.6/com/datastax/driver/core/BatchStatement.html#setSerialConsistencyLevel-com.datastax.driver.core.ConsistencyLevel-)
-* [client-side timestamps](../query_timestamps/)
+* [client-side timestamps](../query_timestamps/index)
 
 #### v3 to v4
 
 * [query warnings](http://docs.datastax.com/en/drivers/java/3.6/com/datastax/driver/core/ExecutionInfo.html#getWarnings--)
 * allowed unset values in bound statements
-* [Custom payloads](../custom_payloads/)
+* [Custom payloads](../custom_payloads/index)
diff --git a/manual/object_mapper/creating/README.md b/docs/source/manual/object_mapper/creating/index.md
similarity index 98%
rename from manual/object_mapper/creating/README.md
rename to docs/source/manual/object_mapper/creating/index.md
index ab337e071e3..ece05702664 100644
--- a/manual/object_mapper/creating/README.md
+++ b/docs/source/manual/object_mapper/creating/index.md
@@ -244,8 +244,8 @@ int f;
 ```
 
 Finally, computed properties are only supported with [basic read
-operations](../using/#basic-crud-operations) at this time.
-Support in [accessors](../using/#accessors) is planned for a future
+operations](../using/index#basic-crud-operations) at this time.
+Support in [accessors](../using/index#accessors) is planned for a future
 version (see
 [JAVA-832](https://datastax-oss.atlassian.net/browse/JAVA-832)).
 
@@ -321,7 +321,7 @@ public class Company {
 This also works with UDTs inside collections or other UDTs, with any arbitrary
 nesting level.
 
-[User Defined Types]: ../../udts/
+[User Defined Types]: ../../udts/index
 [udt]:http://docs.datastax.com/en/drivers/java/3.6/com/datastax/driver/mapping/annotations/UDT.html
 [field]:http://docs.datastax.com/en/drivers/java/3.6/com/datastax/driver/mapping/annotations/Field.html
 
diff --git a/manual/object_mapper/custom_codecs/README.md b/docs/source/manual/object_mapper/custom_codecs/index.md
similarity index 95%
rename from manual/object_mapper/custom_codecs/README.md
rename to docs/source/manual/object_mapper/custom_codecs/index.md
index e423b3bae79..de870ebe26c 100644
--- a/manual/object_mapper/custom_codecs/README.md
+++ b/docs/source/manual/object_mapper/custom_codecs/index.md
@@ -1,6 +1,6 @@
 # Using custom codecs
 
-The mapper can take advantage of [custom codecs](../../custom_codecs/)
+The mapper can take advantage of [custom codecs](../../custom_codecs/index)
 to apply custom conversions to mapped columns and fields.
 
 ## Declaring codecs
@@ -106,9 +106,9 @@ This also works with [@Field][field] and [@Param][param] annotations.
 ## Implicit UDT codecs
 
 The mapper uses custom codecs internally to handle UDT conversions: when
-you register an [entity](../using/#entity-mappers), the mapper inspects
+you register an [entity](../using/index#entity-mappers), the mapper inspects
 the type of all fields to find classes annotated with
-[@UDT](../creating/#mapping-user-types) (this works recursively with
+[@UDT](../creating/index#mapping-user-types) (this works recursively with
 nested UDTs and collections). For each class, the mapper creates a codec
 and registers it with the underlying `Cluster`.
 
diff --git a/manual/object_mapper/README.md b/docs/source/manual/object_mapper/index.md
similarity index 69%
rename from manual/object_mapper/README.md
rename to docs/source/manual/object_mapper/index.md
index c32787f961a..411889f641a 100644
--- a/manual/object_mapper/README.md
+++ b/docs/source/manual/object_mapper/index.md
@@ -17,6 +17,15 @@ The mapper is published as a separate Maven artifact:
 
 See the child pages for more information:
 
-* [definition of mapped classes](creating/)
-* [using the mapper](using/)
-* [using custom codecs](custom_codecs/)
+* [definition of mapped classes](creating/index)
+* [using the mapper](using/index)
+* [using custom codecs](custom_codecs/index)
+
+```eval_rst
+.. toctree::
+   :hidden:
+      
+   creating/index
+   custom_codecs/index
+   using/index
+```
diff --git a/manual/object_mapper/using/README.md b/docs/source/manual/object_mapper/using/index.md
similarity index 98%
rename from manual/object_mapper/using/README.md
rename to docs/source/manual/object_mapper/using/index.md
index f55c19e843e..d9a18dd473b 100644
--- a/manual/object_mapper/using/README.md
+++ b/docs/source/manual/object_mapper/using/index.md
@@ -117,7 +117,7 @@ Some options don't apply to all operations:
 
 
 Note that `Option.consistencyLevel` is redundant with the consistency
-level defined by [@Table](../creating/#creating-a-table-entity).
+level defined by [@Table](../creating/index#creating-a-table-entity).
 If both are defined, the option will take precedence over the
 annotation.
 
@@ -233,7 +233,7 @@ ResultSet insert(@Param("u") UUID userId, @Param("n") String name);
 
 If a method argument is a Java enumeration, it must be annotated with
 `@Enumerated` to indicate how to convert it to a CQL type (the rules are
-the same as in [mapping definition](../creating/#enumerations)):
+the same as in [mapping definition](../creating/index#enumerations)):
 
 ```java
 @Query("insert into user (key, gender) values (?,?)")
diff --git a/manual/osgi/README.md b/docs/source/manual/osgi/index.md
similarity index 84%
rename from manual/osgi/README.md
rename to docs/source/manual/osgi/index.md
index 36a91b7155d..6e5c6722c16 100644
--- a/manual/osgi/README.md
+++ b/docs/source/manual/osgi/index.md
@@ -1,3 +1,7 @@
+```eval_rst
+:orphan:
+```
+
 # OSGi
 
 The driver is available as an [OSGi] bundle. More specifically, 
@@ -16,7 +20,7 @@ environment; please refer to our [OSGi examples repository].
 
 ## Troubleshooting OSGi applications
 
-Please read our [OSGi-specific FAQ section](../../faq/osgi/) to find tips
+Please read our [OSGi-specific FAQ section](../../faq/osgi/index) to find tips
 for troubleshooting OSGi applications.
 
 [OSGi]:https://www.osgi.org
diff --git a/manual/paging/README.md b/docs/source/manual/paging/index.md
similarity index 98%
rename from manual/paging/README.md
rename to docs/source/manual/paging/index.md
index ea4e9febf15..e77aef75971 100644
--- a/manual/paging/README.md
+++ b/docs/source/manual/paging/index.md
@@ -42,7 +42,7 @@ The fetch size limits the number of results that are returned in one
 page; if you iterate past that, the driver will run background queries
 to fetch subsequent pages. Here's an example with a fetch size of 20:
 
-```ditaa
+```console
     client         Session                          Cassandra
     --+--------------+---------------------------------+-----
       |execute(query)|                                 |
@@ -95,7 +95,7 @@ for (Row row : rs) {
 
 If you use paging with the async API, you'll also want to use those
 methods to avoid triggering synchronous fetches unintentionally; see
-[async paging](../async/#async-paging).
+[async paging](../async/index#async-paging).
 
 
 ### Saving and reusing the paging state
@@ -181,7 +181,7 @@ if (nextPage != null) {
 
 
 Due to internal implementation details, `PagingState` instances are not
-portable across [native protocol](../native_protocol/) versions. This
+portable across [native protocol](../native_protocol/index) versions. This
 could become a problem in the following scenario:
 
 * you're using the driver 2.0.x and Cassandra 2.0.x, and therefore
diff --git a/manual/pooling/README.md b/docs/source/manual/pooling/index.md
similarity index 98%
rename from manual/pooling/README.md
rename to docs/source/manual/pooling/index.md
index 1838e2f926b..294d87807a2 100644
--- a/manual/pooling/README.md
+++ b/docs/source/manual/pooling/index.md
@@ -22,16 +22,16 @@ connection to handle multiple simultaneous requests:
 You don't need to manage connections yourself. You simply interact with a `Session` object, which takes care of it.
 
 **For each `Session`, there is one connection pool per connected host** (a host is connected when it is up and
-not ignored by the [load balancing policy](../load_balancing)).
+not ignored by the [load balancing policy](../load_balancing/index)).
 
 The number of connections per pool is configurable (this will be
 described in the next section).  The number of stream ids depends on the
-[native protocol version](../native_protocol/):
+[native protocol version](../native_protocol/index):
 
 * protocol v2 or below: 128 stream ids per connection.
 * protocol v3 or above: up to 32768 stream ids per connection.
 
-```ditaa
+```console
 +-------+1   n+-------+1   n+----+1   n+----------+1   128/32K+-------+
 |Cluster+-----+Session+-----+Pool+-----+Connection+-----------+Request+
 +-------+     +-------+     +----+     +----------+           +-------+
@@ -198,7 +198,7 @@ Two options control that queue: a maximum size ([PoolingOptions.setMaxQueueSize]
 * otherwise, the attempt is enqueued; if a connection becomes available before `poolTimeoutMillis` has elapsed,
   then the attempt succeeds, otherwise it is rejected.
 
-If the attempt is rejected, the driver will move to the next host in the [query plan](../load_balancing/#query-plan),
+If the attempt is rejected, the driver will move to the next host in the [query plan](../load_balancing/index#query-plan),
 and try to acquire a connection again.
 
 If all hosts are busy with a full queue, the request will fail with a
diff --git a/manual/query_timestamps/README.md b/docs/source/manual/query_timestamps/index.md
similarity index 97%
rename from manual/query_timestamps/README.md
rename to docs/source/manual/query_timestamps/index.md
index fcdc66c7189..56a65fdc9a4 100644
--- a/manual/query_timestamps/README.md
+++ b/docs/source/manual/query_timestamps/index.md
@@ -17,7 +17,7 @@ session.execute("INSERT INTO my_table(c1, c2) values (1, 1) " +
 ### Client-side generation
 
 This is enabled by default if you're using the driver 3.0+ and a version
-of Cassandra that supports [native protocol](../native_protocol) v3 or
+of Cassandra that supports [native protocol](../native_protocol/index) v3 or
 above.
 
 The driver will use a [TimestampGenerator] to generate a timestamp for each query, and send it
@@ -60,7 +60,7 @@ property `com.datastax.driver.USE_NATIVE_CLOCK` is explicitly set to `false`.
 To check what's available on your system:
 
 * make sure your `Cluster` uses a `TimestampGenerator`;
-* [configure your logging framework](../logging/) to use level `INFO` for the category
+* [configure your logging framework](../logging/index) to use level `INFO` for the category
   `com.datastax.driver.core.ClockFactory`;
 * look for one of the following messages at startup:
 
diff --git a/manual/reconnection/README.md b/docs/source/manual/reconnection/index.md
similarity index 89%
rename from manual/reconnection/README.md
rename to docs/source/manual/reconnection/index.md
index ee078e61751..4db89977c08 100644
--- a/manual/reconnection/README.md
+++ b/docs/source/manual/reconnection/index.md
@@ -3,9 +3,9 @@
 If the driver loses a connection to a node, it tries to re-establish it according to a configurable
 policy. This is used in two places:
 
-* [connection pools](../pooling/): for each node, a session has a fixed-size pool of connections to
+* [connection pools](../pooling/index): for each node, a session has a fixed-size pool of connections to
   execute user requests. If a node is detected as down, a reconnection is started.
-* [control connection](../control_connection/): a session uses a single connection to an arbitrary
+* [control connection](../control_connection/index): a session uses a single connection to an arbitrary
   node for administrative requests. If that connection goes down, a reconnection gets started; each
   attempt iterates through all active nodes until one of them accepts a connection. This goes on
   until we have a control node again.
diff --git a/manual/retries/README.md b/docs/source/manual/retries/index.md
similarity index 97%
rename from manual/retries/README.md
rename to docs/source/manual/retries/index.md
index 5c450ce1ebd..004d097cf3c 100644
--- a/manual/retries/README.md
+++ b/docs/source/manual/retries/index.md
@@ -174,9 +174,9 @@ implementations to handle idempotence (the new behavior is equivalent to what yo
 [AlreadyExistsException]:               http://docs.datastax.com/en/drivers/java/3.6/com/datastax/driver/core/exceptions/AlreadyExistsException.html
 [TruncateException]:                    http://docs.datastax.com/en/drivers/java/3.6/com/datastax/driver/core/exceptions/TruncateException.html
 
-[query plan]: ../load_balancing/#query-plan
-[connection pool]: ../pooling/
-[prepared]: ../statements/prepared/#preparing-on-multiple-nodes
-[driver read timeout]: ../socket_options/#driver-read-timeout
+[query plan]: ../load_balancing/index#query-plan
+[connection pool]: ../pooling/index
+[prepared]: ../statements/prepared/index#preparing-on-multiple-nodes/index
+[driver read timeout]: ../socket_options/index#driver-read-timeout/index
 [hinted handoffs]: https://docs.datastax.com/en/cassandra/2.1/cassandra/dml/dml_about_hh_c.html?scroll=concept_ds_ifg_jqx_zj__performance
-[idempotence]: ../idempotence/
\ No newline at end of file
+[idempotence]: ../idempotence/index
\ No newline at end of file
diff --git a/manual/shaded_jar/README.md b/docs/source/manual/shaded_jar/index.md
similarity index 97%
rename from manual/shaded_jar/README.md
rename to docs/source/manual/shaded_jar/index.md
index fce55089d47..88f4f9c89bc 100644
--- a/manual/shaded_jar/README.md
+++ b/docs/source/manual/shaded_jar/index.md
@@ -1,6 +1,6 @@
 ## Using the shaded JAR
 
-The default driver JAR depends on [Netty](http://netty.io/), which is
+The default driver JAR depends on [Netty](http://netty.io/index), which is
 used internally for networking.
 
 This explicit dependency can be a problem if your application already
diff --git a/manual/socket_options/README.md b/docs/source/manual/socket_options/index.md
similarity index 93%
rename from manual/socket_options/README.md
rename to docs/source/manual/socket_options/index.md
index 26e5fdad733..0399bc397b3 100644
--- a/manual/socket_options/README.md
+++ b/docs/source/manual/socket_options/index.md
@@ -71,8 +71,8 @@ session.execute(
 Do not set the read timeout too low, or the driver might give up on requests that had a chance of succeeding.
 
 If the timeout is reached, the driver will receive an [OperationTimedOutException], and invoke [onRequestError] on the
-[retry policy](../retries/) to decide what to do (the default is to retry on the next node in the
-[query plan](../load_balancing/#query-plan)).
+[retry policy](../retries/index) to decide what to do (the default is to retry on the next node in the
+[query plan](../load_balancing/index#query-plan)).
 
 #### Limiting overall query time
 
@@ -97,22 +97,22 @@ ResultSet execute(Statement statement, long timeout, TimeUnit unit)
 }
 ```
 
-A complementary approach is to enable [speculative executions](../speculative_execution/), to have the driver query
+A complementary approach is to enable [speculative executions](../speculative_execution/index), to have the driver query
 multiple nodes in parallel. This way you won't have to wait for the full timeout if the first node is unresponsive.
 
 #### Driver read timeout vs. server read timeout
 
 Unfortunately, the term "read timeout" clashes with another concept that is not directly related: a Cassandra node may
-reply with a [READ_TIMEOUT](../retries/#on-read-timeout) error when it didn't hear back from enough replicas during a
+reply with a [READ_TIMEOUT](../retries/index#on-read-timeout) error when it didn't hear back from enough replicas during a
 read query.
 
 To clarify:
 
 * **driver read timeout:** the driver did not receive any response from the current coordinator within
-  `SocketOptions.setReadTimeoutMillis`. It invokes [onRequestError] on the [retry policy](../retries/) with an
+  `SocketOptions.setReadTimeoutMillis`. It invokes [onRequestError] on the [retry policy](../retries/index) with an
   [OperationTimedOutException] to decide what to do.
 * **server read timeout:** the driver *did* receive a response, but that response indicates that the coordinator timed
-  out while waiting for other replicas. It invokes [onReadTimeout] on the [retry policy](../retries/) to decide what to
+  out while waiting for other replicas. It invokes [onReadTimeout] on the [retry policy](../retries/index) to decide what to
   do.
 
 We might rename `SocketOptions.setReadTimeoutMillis` in a future version to clear up any confusion.
diff --git a/manual/speculative_execution/README.md b/docs/source/manual/speculative_execution/index.md
similarity index 97%
rename from manual/speculative_execution/README.md
rename to docs/source/manual/speculative_execution/index.md
index 157cfae3b6c..e617698f97a 100644
--- a/manual/speculative_execution/README.md
+++ b/docs/source/manual/speculative_execution/index.md
@@ -12,7 +12,7 @@ note that "cancelling" in this context simply means discarding the response
 when it arrives later, Cassandra does not support cancellation of in flight
 requests at this stage):
 
-```ditaa
+```console
 client           driver          exec1  exec2
 --+----------------+--------------+------+---
   | execute(query) |
@@ -37,7 +37,7 @@ Or the first node could reply just after the second execution was
 started. In this case, we cancel the second execution. In other words,
 whichever node replies faster "wins" and completes the client query:
 
-```ditaa
+```console
 client           driver          exec1  exec2
 --+----------------+--------------+------+---
   | execute(query) |
@@ -63,7 +63,7 @@ sections cover the practical details and how to enable them.
 
 ### Query idempotence
 
-If a query is [not idempotent](../idempotence/), the driver will never schedule speculative executions for it, because
+If a query is [not idempotent](../idempotence/index), the driver will never schedule speculative executions for it, because
 there is no way to guarantee that only one node will apply the mutation.
 
 ### Enabling speculative executions
@@ -157,7 +157,7 @@ Cluster cluster = Cluster.builder()
 ```
 
 Note that `PercentileTracker` may also be used with a slow query
-logger (see the [Logging](../logging/) section). In that case, you would
+logger (see the [Logging](../logging/index) section). In that case, you would
 create a single tracker object and share it with both components.
 
 [PercentileSpeculativeExecutionPolicy]: http://docs.datastax.com/en/drivers/java/3.6/com/datastax/driver/core/policies/PercentileSpeculativeExecutionPolicy.html
@@ -173,10 +173,10 @@ As with all policies, you are free to provide your own by implementing
 
 ### How speculative executions affect retries
 
-Turning speculative executions on doesn't change the driver's [retry](../retries/) behavior. Each
+Turning speculative executions on doesn't change the driver's [retry](../retries/index) behavior. Each
 parallel execution will trigger retries independently:
 
-```ditaa
+```console
 client           driver          exec1  exec2
 --+----------------+--------------+------+---
   | execute(query) |
@@ -286,5 +286,5 @@ The workaround is to use a timestamp with your queries:
     insert into my_table (k, v) values (1, 1) USING TIMESTAMP 1432764000;
 
 If you're using native protocol v3, you can also enable [client-side
-timestamps](../query_timestamps/#client-side-generation) to have this done
+timestamps](../query_timestamps/index#client-side-generation) to have this done
 automatically.
diff --git a/manual/ssl/README.md b/docs/source/manual/ssl/index.md
similarity index 100%
rename from manual/ssl/README.md
rename to docs/source/manual/ssl/index.md
diff --git a/manual/statements/batch/README.md b/docs/source/manual/statements/batch/index.md
similarity index 100%
rename from manual/statements/batch/README.md
rename to docs/source/manual/statements/batch/index.md
diff --git a/manual/statements/built/README.md b/docs/source/manual/statements/built/index.md
similarity index 98%
rename from manual/statements/built/README.md
rename to docs/source/manual/statements/built/index.md
index bafc20f3eeb..6f9266a1bfd 100644
--- a/manual/statements/built/README.md
+++ b/docs/source/manual/statements/built/index.md
@@ -10,7 +10,7 @@ invalid queries.
 Queries built with `QueryBuilder` are executed the same way as other queries--via
 `execute` or `executeAsync`.  When a query is built with inlined values, then it doesn't
 differ much from a statement specified as a string.  But it's also possible to build
-the query with bind markers inside it, and then convert it into a [prepared statement](../prepared/).
+the query with bind markers inside it, and then convert it into a [prepared statement](../prepared/index).
 
 ### Basics
 
@@ -24,7 +24,7 @@ a [TableMetadata] object.
 
 Note: The `QueryBuilder` doesn't provide support for the full set of CQL.  For
 most of DDL operations (`CREATE TABLE`, etc.) you can use the [SchemaBuilder].  To perform other
-operations, for example, for role management, you still need to use [simple statements](../simple/).
+operations, for example, for role management, you still need to use [simple statements](../simple/index).
 
 ### Selecting data
 
@@ -238,7 +238,7 @@ conditional deletes with `ifExists` & `onlyIf`.
 ### Prepared statements
 
 If you're repeating the same operation very often, the more effective way will be to
-create a [prepared statement](../prepared/) from the `BuiltStatement`. To do this, instead
+create a [prepared statement](../prepared/index) from the `BuiltStatement`. To do this, instead
 of the real values, use bind markers created either by calling `bindMarker` (which generates
 a positional placeholder), or by calling `bindMarker("name")` (which creates a named
 placeholder).  After the statement is generated, just prepare it as usual, then bind, and
diff --git a/manual/statements/README.md b/docs/source/manual/statements/index.md
similarity index 83%
rename from manual/statements/README.md
rename to docs/source/manual/statements/index.md
index d06f5f33387..8da0ee3a11f 100644
--- a/manual/statements/README.md
+++ b/docs/source/manual/statements/index.md
@@ -3,15 +3,15 @@
 To execute a query, you  create a [Statement] instance and pass it to [Session#execute()][execute] or
 [Session#executeAsync][executeAsync]. The driver provides various implementations:
 
-* [SimpleStatement](simple/): a simple implementation built directly from a
+* [SimpleStatement](simple/index): a simple implementation built directly from a
   character string. Typically used for queries that are executed only
   once or a few times.
-* [BoundStatement](prepared/): obtained by binding values to a prepared
+* [BoundStatement](prepared/index): obtained by binding values to a prepared
   statement. Typically used for queries that are executed
   often, with different values.
-* [BuiltStatement](built/): a statement built with the [QueryBuilder] DSL. It
+* [BuiltStatement](built/index): a statement built with the [QueryBuilder] DSL. It
   can be executed directly like a simple statement, or prepared.
-* [BatchStatement](batch/): a statement that groups multiple statements to be
+* [BatchStatement](batch/index): a statement that groups multiple statements to be
   executed as a batch.
 
 
@@ -40,3 +40,13 @@ wrap your statements in a custom [StatementWrapper] implementation.
 [SpeculativeExecutionPolicy]: http://docs.datastax.com/en/drivers/java/3.6/com/datastax/driver/core/policies/SpeculativeExecutionPolicy.html
 [execute]:                    http://docs.datastax.com/en/drivers/java/3.6/com/datastax/driver/core/Session.html#execute-com.datastax.driver.core.Statement-
 [executeAsync]:               http://docs.datastax.com/en/drivers/java/3.6/com/datastax/driver/core/Session.html#executeAsync-com.datastax.driver.core.Statement-
+
+```eval_rst
+.. toctree::
+   :hidden:
+   
+   simple/index
+   prepared/index
+   built/index
+   batch/index
+```
diff --git a/manual/statements/prepared/README.md b/docs/source/manual/statements/prepared/index.md
similarity index 98%
rename from manual/statements/prepared/README.md
rename to docs/source/manual/statements/prepared/index.md
index a1556b8c2c1..6b3b5326baa 100644
--- a/manual/statements/prepared/README.md
+++ b/docs/source/manual/statements/prepared/index.md
@@ -15,7 +15,7 @@ session.execute(prepared.bind("987274", "Keyboard"));
 When you prepare the statement, Cassandra will parse the query string, cache the result and return a unique identifier
 (the `PreparedStatement` object keeps an internal reference to that identifier):
 
-```ditaa
+```console
 client                   driver           Cassandra
 --+------------------------+----------------+------
   |                        |                |
@@ -38,7 +38,7 @@ client                   driver           Cassandra
 When you bind and execute a prepared statement, the driver will only send the identifier, which allows Cassandra to
 skip the parsing phase:
 
-```ditaa
+```console
 client                            driver                Cassandra
 --+---------------------------------+---------------------+------
   |                                 |                     |
@@ -62,12 +62,12 @@ You should prepare only once, and cache the `PreparedStatement` in your applicat
 `prepare` multiple times with the same query string, the driver will log a warning.
 
 If you execute a query only once, a prepared statement is inefficient because it requires two roundtrips. Consider a
-[simple statement](../simple/) instead.
+[simple statement](../simple/index) instead.
 
 ### Parameters and binding
 
 Parameters can be either anonymous or named (named parameters are only
-available with [native protocol](../../native_protocol) v2 or above):
+available with [native protocol](../../native_protocol/index) v2 or above):
 
 ```java
 ps1 = session.prepare("insert into product (sku, description) values (?, ?)");
@@ -167,7 +167,7 @@ date. It uses a number of strategies to achieve this:
     the query string is wrong). Once that node replies successfully, the
     driver re-prepares on all remaining nodes:
 
-    ```ditaa
+    ```console
     client                   driver           node1          node2  node3
     --+------------------------+----------------+--------------+------+---
       |                        |                |              |      |
@@ -201,7 +201,7 @@ date. It uses a number of strategies to achieve this:
     statement on the fly (this is transparent for the client, but will cost
     two extra roundtrips):
 
-    ```ditaa
+    ```console
     client                          driver                         node1
     --+-------------------------------+------------------------------+--
       |                               |                              |
diff --git a/manual/statements/simple/README.md b/docs/source/manual/statements/simple/index.md
similarity index 97%
rename from manual/statements/simple/README.md
rename to docs/source/manual/statements/simple/index.md
index 386937c1ec2..2b98446f9dd 100644
--- a/manual/statements/simple/README.md
+++ b/docs/source/manual/statements/simple/index.md
@@ -17,7 +17,7 @@ session.execute("SELECT value FROM application_params WHERE name = 'greeting_mes
 Each time you execute a simple statement, Cassandra will parse the query string again; nothing is cached (neither on the
 client nor on the server):
 
-```ditaa
+```console
 client                             driver                Cassandra
 --+----------------------------------+---------------------+------
   |                                  |                     |
@@ -37,7 +37,7 @@ client                             driver                Cassandra
 ```
 
 If you execute the same query often (or a similar query with different column values), consider a
-[prepared statement](../prepared/) instead.
+[prepared statement](../prepared/index) instead.
 
 
 ### Using values
@@ -85,7 +85,7 @@ session.execute(
 ### Value type inference
 
 Another consequence of not parsing query strings is that the driver has to make a guess on how to serialize values,
-based on their Java type (see the [default type mappings](../../#cql-to-java-type-mapping)). This can be tricky, in
+based on their Java type (see the [default type mappings](../../index#cql-to-java-type-mapping)). This can be tricky, in
 particular for numeric types:
 
 ```java
diff --git a/manual/tuples/README.md b/docs/source/manual/tuples/index.md
similarity index 100%
rename from manual/tuples/README.md
rename to docs/source/manual/tuples/index.md
diff --git a/manual/udts/README.md b/docs/source/manual/udts/index.md
similarity index 97%
rename from manual/udts/README.md
rename to docs/source/manual/udts/index.md
index 7221ba5cc4b..a1336a46046 100644
--- a/manual/udts/README.md
+++ b/docs/source/manual/udts/index.md
@@ -58,7 +58,7 @@ various ways to get it:
     UserType udt = (UserType) ps.getVariables().getType("v");
     ```
 
-* from the driver's [schema metadata](../metadata/#schema-metadata):
+* from the driver's [schema metadata](../metadata/index#schema-metadata):
 
     ```java
     UserType udt = session.getMetadata().getKeyspace("ks").getUserType("type1");
diff --git a/upgrade_guide/README.md b/docs/source/upgrade_guide/index.md
similarity index 97%
rename from upgrade_guide/README.md
rename to docs/source/upgrade_guide/index.md
index e85262ed847..87f8137ce36 100644
--- a/upgrade_guide/README.md
+++ b/docs/source/upgrade_guide/index.md
@@ -125,7 +125,7 @@ queries that were retried before).
 
 In 3.1.0, the default is now to **not retry** after a write timeout or request error if the statement is not idempotent.
 This is handled internally, the retry policy methods are not even invoked in those cases (and therefore
-`IdempotenceAwareRetryPolicy` has been deprecated). See the manual section about [retries](../manual/retries/) for more
+`IdempotenceAwareRetryPolicy` has been deprecated). See the manual section about [retries](../manual/retries/index) for more
 information.
 
 In practice, here's what upgrading to 3.1.0 means for you:
@@ -135,7 +135,7 @@ In practice, here's what upgrading to 3.1.0 means for you:
 * otherwise, you might want to review how your code positions the `setIdempotent` flag on statements. In most cases the
   driver can't compute in automatically (because it doesn't parse query strings), so it takes a conservative approach
   and sets it to `false` by default. If you know the query is idempotent, you should set it to `true` manually. See the
-  [query idempotence](../manual/idempotence/) section of the manual.
+  [query idempotence](../manual/idempotence/index) section of the manual.
 
 The driver logs a warning the first time it ignores a non-idempotent request; this warning will be removed in version
 3.2.0.
@@ -166,7 +166,7 @@ The main changes were introduced by the custom codecs feature (see below).
 We've also seized the opportunity to remove code that was deprecated in 2.1.
 
 1.  The default consistency level in `QueryOptions` is now `LOCAL_ONE`.
-2.  [Custom codecs](../manual/custom_codecs/)
+2.  [Custom codecs](../manual/custom_codecs/index)
     ([JAVA-721](https://datastax-oss.atlassian.net/browse/JAVA-721))
     introduce several breaking changes and also modify a few runtime behaviors.
 
@@ -230,7 +230,7 @@ We've also seized the opportunity to remove code that was deprecated in 2.1.
     corresponding method `MappingManager#udtMapper`.
 
     The mapper now uses custom codecs to convert UDTs. See more
-    explanations [here](../manual/object_mapper/custom_codecs/#implicit-udt-codecs).
+    explanations [here](../manual/object_mapper/custom_codecs/index#implicit-udt-codecs).
 
 5.  All methods that took the protocol version as an `int` or assumed a
     default version have been removed (they were already deprecated in
@@ -289,9 +289,9 @@ We've also seized the opportunity to remove code that was deprecated in 2.1.
     callback methods (`onUp`, `onDown`, etc.) have been duplicated. This
     is unlikely to affect clients.
 
-12. [Client-side timestamp generation](../manual/query_timestamps/) is
+12. [Client-side timestamp generation](../manual/query_timestamps/index) is
     now the default (provided that [native
-    protocol](../manual/native_protocol) v3 or higher is in use). The
+    protocol](../manual/native_protocol/index) v3 or higher is in use). The
     generator used is `AtomicMonotonicTimestampGenerator`.
 
 13. If a DNS name resolves to multiple A-records,
@@ -300,7 +300,7 @@ We've also seized the opportunity to remove code that was deprecated in 2.1.
     maintaining contact points in DNS configuration, and having a single,
     static contact point in your Java code.
 
-14. The following methods were added for [Custom payloads](../manual/custom_payloads):
+14. The following methods were added for [Custom payloads](../manual/custom_payloads/index):
     * in `PreparedStatement`: `getIncomingPayload()`,
       `getOutgoingPayload()` and
       `setOutgoingPayload(Map)`
@@ -425,7 +425,7 @@ We've also seized the opportunity to remove code that was deprecated in 2.1.
 
 29. `SSLOptions` has been refactored to allow the option to choose between JDK and Netty-based
     SSL implementations.  See [JAVA-841](https://datastax-oss.atlassian.net/browse/JAVA-841) and
-    the [SSL documentation](../manual/ssl) for more details.
+    the [SSL documentation](../manual/ssl/index) for more details.
 
 
 ### 2.1.8
@@ -463,7 +463,7 @@ binary compatibility.
     ones.
 
     Also, note that the connection pool for protocol v3 can now be configured to
-    use multiple connections. See [this page](../manual/pooling) for more
+    use multiple connections. See [this page](../manual/pooling/index) for more
     information.
 
 3. `MappingManager(Session)` will now force the initialization of the `Session`
@@ -496,7 +496,7 @@ the impact on the public API has been kept as low as possible.
    version that takes an `int` (the exception being `RegularStatement`,
    described below). For new code, prefer the enum version.
 
-#### Internal API Changes
+#### 2.1.6 Internal API Changes
 
 1. `RegularStatement.getValues` now takes the protocol version as a
    `ProtocolVersion` instead of an `int`. This is transparent for callers
@@ -539,7 +539,7 @@ These features are only active when the native protocol v3 is in use.
 
 ### 2.1.1
 
-#### Internal API Changes
+#### 2.1.1 Internal API Changes
 
 1. The `ResultSet` interface has a new `wasApplied()` method. This will
    only affect clients that provide their own implementation of this interface.
@@ -547,7 +547,7 @@ These features are only active when the native protocol v3 is in use.
 
 ### 2.1.0
 
-#### User API Changes
+#### 2.1.0 User API Changes
 
 1. The `getCaching` method of `TableMetadata#Options` now returns a
    `Map` to account for changes to Cassandra 2.1. Also, the
@@ -564,7 +564,7 @@ These features are only active when the native protocol v3 is in use.
    set to `null`" behavior.
 
 
-#### Internal API Changes
+#### 2.1.0 Internal API Changes
 
 The changes listed in this section should normally not impact end users of the
 driver, but rather third-party frameworks and tools.
@@ -632,7 +632,7 @@ The following might also be of interest:
    default like in previous versions.
 
 3. Netty is not shaded anymore in the default Maven artifact. However we publish a
-   [shaded artifact](../manual/shaded_jar/) under a different classifier.
+   [shaded artifact](../manual/shaded_jar/index) under a different classifier.
 
 4. The internal initialization sequence of the Cluster object has been slightly changed:
    some fields that were previously initialized in the constructor are now set when
@@ -687,7 +687,7 @@ you have trouble compiling your application after an upgrade.
 
 5. The `Metrics` class now uses the Codahale metrics library version 3 (version 2 was
    used previously). This new major version of the library has many API changes
-   compared to its version 2 (see the [release notes](https://dropwizard.github.io/metrics/3.1.0/about/release-notes/) for details),
+   compared to its version 2 (see the [release notes](https://dropwizard.github.io/metrics/3.1.0/about/release-notes/index) for details),
    which can thus impact consumers of the Metrics class.
    Furthermore, the default `JmxReporter` now includes a name specific to the
    cluster instance (to avoid conflicts when multiple Cluster instances are created
@@ -826,3 +826,10 @@ exhaustive list of new features in 2.0.
     * the `ResultSet` object provides 2 methods to check the state of paging
       (`ResultSet#getAvailableWithoutFetching` and `ResultSet#isFullyFetched`)
       as well as a mean to force the pre-fetching of the next page (`ResultSet#fetchMoreResults`).
+
+```eval_rst
+.. toctree::
+   :hidden:
+   
+   migrating_from_astyanax/index
+```
\ No newline at end of file
diff --git a/upgrade_guide/migrating_from_astyanax/configuration/README.md b/docs/source/upgrade_guide/migrating_from_astyanax/configuration/index.md
similarity index 96%
rename from upgrade_guide/migrating_from_astyanax/configuration/README.md
rename to docs/source/upgrade_guide/migrating_from_astyanax/configuration/index.md
index c6629113d7f..dc1edbc4b22 100644
--- a/upgrade_guide/migrating_from_astyanax/configuration/README.md
+++ b/docs/source/upgrade_guide/migrating_from_astyanax/configuration/index.md
@@ -1,3 +1,7 @@
+```eval_rst
+:orphan:
+```
+
 # Configuration
 
 ## How Configuring the Java driver works
@@ -29,7 +33,7 @@ that the *Java driver* has categorized in multiple *Option* and *Policy* kinds.
 
 ### Connections pools internals
 Everything concerning the internal pools of connections to the *Cassandra nodes*
-will be gathered in the Java driver in the [`PoolingOptions`](../../../manual/pooling):
+will be gathered in the Java driver in the [`PoolingOptions`](../../../manual/pooling/index):
 
 *Astyanax*:
 
@@ -51,11 +55,11 @@ The first number is the initial number of connections, the second is the maximum
 of connections the driver is allowed to create for each host.
 
 Note that the *Java driver* allows multiple simultaneous requests on one single
-connection, as it is based upon the [*Native protocol*](../../../manual/native_protocol),
+connection, as it is based upon the [*Native protocol*](../../../manual/native_protocol/index),
 an asynchronous binary protocol that can handle up to 32768 simultaneous requests on a 
 single connection. The Java driver is able to manage and distribute simultaneous requests
 by itself even under high contention, and changing the default `PoolingOptions` is not
-necessary most of the time except for very [specific use cases](../../../manual/pooling/#tuning-protocol-v3-for-very-high-throughputs).
+necessary most of the time except for very [specific use cases](../../../manual/pooling/index#tuning-protocol-v3-for-very-high-throughputs).
 
 ### Timeouts
 
@@ -132,12 +136,12 @@ query needs to have the *Row key* specified explicitly on a `Statement` in order
 to benefit from the *TokenAware* routing (the *Row key* in the *Java driver* is 
 referenced as *Routing Key*), unlike the *Astyanax* driver. 
 Some differences occur related to the different kinds of `Statements` the *Java
-driver* provides. Please see [this link](../../../manual/load_balancing/#token-aware-policy) 
+driver* provides. Please see [this link](../../../manual/load_balancing/index#token-aware-policy) 
 for specific information.
 
 Custom load balancing policies can easily be implemented by users, and supplied to 
 the *Java driver* for specific use cases. All information necessary is available
-in the [Load balaning policies docs](../../../manual/load_balancing).
+in the [Load balaning policies docs](../../../manual/load_balancing/index).
 
 ## Consistency levels
 Consistency levels can be set per-statement, or globally through the `QueryOptions`.
@@ -165,7 +169,7 @@ Level can be set per-statement, you can either set it on every statement, or use
 this case, setting the CL on the `PreparedStatement`, causes the `BoundStatements` to 
 inherit the CL from the prepared statements they were prepared from. More
 informations about how `Statement`s work in the *Java driver* are detailed
-in the [“Queries and Results” section](../queries_and_results/).
+in the [“Queries and Results” section](../queries_and_results/index).
 
 
 ## Authentication
@@ -189,7 +193,7 @@ AuthProvider authProvider = new PlainTextAuthProvider("username", "password");
 ```
 
 The class `AuthProvider` can be easily implemented to suit the user’s needs,
-documentation about the classes needed is [available there](../../../manual/auth/).
+documentation about the classes needed is [available there](../../../manual/auth/index).
 
 ## Hosts and ports
 
@@ -243,8 +247,8 @@ A few best practices are summed up in [this blog post](http://www.datastax.com/d
 Concerning connection pools, the Java driver’s default settings should allow 
 most of the users to get the best out of the driver in terms of throughput, 
 they have been thoroughly tested and tweaked to accommodate the users’ needs. 
-If one still wishes to change those, first [Monitoring the pools](../../../manual/pooling/#monitoring-and-tuning-the-pool) is
-advised, then a [deep dive in the Pools management mechanism](../../../manual/pooling/) should
+If one still wishes to change those, first [Monitoring the pools](../../../manual/pooling/index#monitoring-and-tuning-the-pool) is
+advised, then a [deep dive in the Pools management mechanism](../../../manual/pooling/index) should
 provide enough insight.
 
 A lot more options are available in the different `XxxxOption`s classes, policies are
diff --git a/upgrade_guide/migrating_from_astyanax/README.md b/docs/source/upgrade_guide/migrating_from_astyanax/index.md
similarity index 56%
rename from upgrade_guide/migrating_from_astyanax/README.md
rename to docs/source/upgrade_guide/migrating_from_astyanax/index.md
index 0b518322a28..9894394d313 100644
--- a/upgrade_guide/migrating_from_astyanax/README.md
+++ b/docs/source/upgrade_guide/migrating_from_astyanax/index.md
@@ -5,6 +5,16 @@ migrating to the *DataStax Java driver*.
 
 See the child pages for more information:
 
-* [Changes at the language level](language_level_changes/)
-* [Migrating Astyanax configurations to DataStax Java driver configurations](configuration/)
-* [Querying and retrieving results comparisons.](queries_and_results/)
+* [Changes at the language level](language_level_changes/index)
+* [Migrating Astyanax configurations to DataStax Java driver configurations](configuration/index)
+* [Querying and retrieving results comparisons.](queries_and_results/index)
+
+```eval_rst
+.. toctree::
+   :hidden:
+   
+   configuration/index
+   language_level_changes/index
+   queries_and_results/index
+
+```
\ No newline at end of file
diff --git a/upgrade_guide/migrating_from_astyanax/language_level_changes/README.md b/docs/source/upgrade_guide/migrating_from_astyanax/language_level_changes/index.md
similarity index 98%
rename from upgrade_guide/migrating_from_astyanax/language_level_changes/README.md
rename to docs/source/upgrade_guide/migrating_from_astyanax/language_level_changes/index.md
index 8116e82b566..d37f4abdcc3 100644
--- a/upgrade_guide/migrating_from_astyanax/language_level_changes/README.md
+++ b/docs/source/upgrade_guide/migrating_from_astyanax/language_level_changes/index.md
@@ -1,3 +1,7 @@
+```eval_rst
+:orphan:
+```
+
 # Language change : from Thrift to CQL
 The data model changes when using *CQL* (Cassandra Query Language).
 *CQL* is providing an abstraction of the low-level data stored in *Cassandra*, in
@@ -11,7 +15,7 @@ as a *Table*, the **Table Rows** get a **fixed and finite number of named column
 *Thrift*’s columns inside the *Rows* get distributed in a tabular way through the
 _Table Rows_. See the following figure:
 
-```ditaa
+```console
                                                      Thrift
              /-                                                                                          -\
              |                                                                                            |
@@ -70,6 +74,6 @@ composing the *Clustering Column*, but there can be others if the Thrift's Colum
 is a CompositeType).
 
 Here is the basic architectural concept of *CQL*, a detailed explanation and *CQL*
-examples can be found in this article : [http://www.planetcassandra.org/making-the-change-from-thrift-to-cql/](http://www.planetcassandra.org/making-the-change-from-thrift-to-cql/).
+examples can be found in this article : [http://www.planetcassandra.org/making-the-change-from-thrift-to-cql/](http://www.planetcassandra.org/making-the-change-from-thrift-to-cql/index).
 Understanding the *CQL* abstraction plays a key role in developing performing
 and scaling applications.
diff --git a/upgrade_guide/migrating_from_astyanax/queries_and_results/README.md b/docs/source/upgrade_guide/migrating_from_astyanax/queries_and_results/index.md
similarity index 89%
rename from upgrade_guide/migrating_from_astyanax/queries_and_results/README.md
rename to docs/source/upgrade_guide/migrating_from_astyanax/queries_and_results/index.md
index 3f14620bac7..ee6fb834685 100644
--- a/upgrade_guide/migrating_from_astyanax/queries_and_results/README.md
+++ b/docs/source/upgrade_guide/migrating_from_astyanax/queries_and_results/index.md
@@ -1,3 +1,7 @@
+```eval_rst
+:orphan:
+```
+
 # Queries and Results
 There are many resources such as [this post][planetCCqlLink] or [this post][dsBlogCqlLink] to learn
 how to transform previous Thrift operations to CQL queries.
@@ -6,10 +10,10 @@ The *Java driver* executes CQL queries through the `Session`.
 The queries can either be simple *CQL* Strings or represented in the form of 
 `Statement`s. The driver offers 4 kinds of statements, `SimpleStatement`, 
 `Prepared/BoundStatement`, `BuiltStatement`, and `BatchStatement`. All necessary
-information can be [found here](../../../manual/statements/) about the nature of the different
+information can be [found here](../../../manual/statements/index) about the nature of the different
 `Statement`s.
 
-As explained in [the running section](../../../manual/#running-queries),
+As explained in [the running section](../../../manual/index#running-queries),
 results of a *CQL* query will be in the form of *Rows* from *Tables*, composed 
 of fixed set of columns, each with a type and a name. The driver exposes the 
 set of *Rows* returned from a query as a ResultSet, thus containing *Rows* on 
@@ -97,8 +101,8 @@ for (Row row : rs) {
 
 ## Async
 The *Java driver* provides native support for asynchronous programming since it 
-is built on top of an [asynchronous protocol](../../../manual/native_protocol/),
-please see [this page](../../../manual/async/) for best practices regarding asynchronous programming
+is built on top of an [asynchronous protocol](../../../manual/native_protocol/index),
+please see [this page](../../../manual/async/index) for best practices regarding asynchronous programming
 with the *Java driver*.
 
 
diff --git a/doxyfile b/doxyfile
index 414bdbd7ec4..ea043740253 100644
--- a/doxyfile
+++ b/doxyfile
@@ -160,7 +160,7 @@ IGNORE_PREFIX          =
 # Configuration options related to the HTML output
 #---------------------------------------------------------------------------
 GENERATE_HTML          = YES
-HTML_OUTPUT            = html
+HTML_OUTPUT            = docs/_build/dirhtml/api
 HTML_FILE_EXTENSION    = .html
 HTML_HEADER            = 
 HTML_FOOTER            = 
@@ -333,4 +333,4 @@ MAX_DOT_GRAPH_DEPTH    = 0
 DOT_TRANSPARENT        = NO
 DOT_MULTI_TARGETS      = NO
 GENERATE_LEGEND        = YES
-DOT_CLEANUP            = YES
\ No newline at end of file
+DOT_CLEANUP            = YES
diff --git a/manual/object_mapper/.nav b/manual/object_mapper/.nav
deleted file mode 100644
index 1aaac91ac9c..00000000000
--- a/manual/object_mapper/.nav
+++ /dev/null
@@ -1,3 +0,0 @@
-creating
-using
-custom_codecs
\ No newline at end of file
diff --git a/manual/statements/.nav b/manual/statements/.nav
deleted file mode 100644
index 89273161ff0..00000000000
--- a/manual/statements/.nav
+++ /dev/null
@@ -1,4 +0,0 @@
-simple
-prepared
-built
-batch
\ No newline at end of file
diff --git a/upgrade_guide/migrating_from_astyanax/.nav b/upgrade_guide/migrating_from_astyanax/.nav
deleted file mode 100644
index 2267b910143..00000000000
--- a/upgrade_guide/migrating_from_astyanax/.nav
+++ /dev/null
@@ -1,3 +0,0 @@
-language_level_changes
-configuration
-queries_and_results
\ No newline at end of file

From 2aef9fcf542bab147660fec44abf6f7a5813d744 Mon Sep 17 00:00:00 2001
From: dgarcia360 
Date: Fri, 7 Aug 2020 13:56:00 +0200
Subject: [PATCH 34/52] Updated dev docs

---
 README-dev.rst | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/README-dev.rst b/README-dev.rst
index 378ecf78455..666b760dd47 100644
--- a/README-dev.rst
+++ b/README-dev.rst
@@ -3,7 +3,7 @@ Building the Docs
 
 *Note*: The docs build instructions have been tested with Sphinx 2.4.4 and Fedora 32.
 
-To build and preview the theme locally, you will need to install the following software:
+To build and preview the docs locally, you will need to install the following software:
 
 - `Git `_
 - `Python 3.7 `_

From 958ba52af5c72830bc8bbaace4dd32b5ea68b696 Mon Sep 17 00:00:00 2001
From: dgarcia360 
Date: Fri, 7 Aug 2020 13:56:58 +0200
Subject: [PATCH 35/52] Clean conf.py

---
 docs/source/conf.py | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/docs/source/conf.py b/docs/source/conf.py
index 44dba55e341..6ad69aaf651 100644
--- a/docs/source/conf.py
+++ b/docs/source/conf.py
@@ -238,7 +238,7 @@ def setup(app):
 
 # -- Options for multiversion --------------------------------------------
 # Whitelist pattern for tags (set to None to ignore all tags)
-smv_tag_whitelist = r'None'
+smv_tag_whitelist = 'None'
 # Whitelist pattern for branches (set to None to ignore all branches)
 smv_branch_whitelist = r"^latest$"
 

From 478bb85638cba16808f9fb3fc1b310709bcc977a Mon Sep 17 00:00:00 2001
From: dgarcia360 
Date: Fri, 14 Aug 2020 12:33:48 +0200
Subject: [PATCH 36/52] Fixed link

---
 docs/source/manual/compression/index.md | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/docs/source/manual/compression/index.md b/docs/source/manual/compression/index.md
index d8ac4094390..ca74446a4c3 100644
--- a/docs/source/manual/compression/index.md
+++ b/docs/source/manual/compression/index.md
@@ -55,7 +55,7 @@ LZ4-java has three internal implementations (from fastest to slowest):
 
 It will pick the best implementation depending on what's possible on
 your platform. To find out which one was chosen, [enable INFO
-logs](../logging/) on the category
+logs](../logging/index) on the category
 `com.datastax.driver.core.LZ4Compressor` and look for a log similar to
 this:
 

From 9cd786990985b311f8b5e43511965892830790b1 Mon Sep 17 00:00:00 2001
From: dgarcia360 
Date: Fri, 14 Aug 2020 12:43:00 +0200
Subject: [PATCH 37/52] Added ditaa lexer

---
 docs/source/conf.py                                       | 8 ++++++++
 docs/source/manual/metadata/index.md                      | 2 +-
 docs/source/manual/paging/index.md                        | 2 +-
 docs/source/manual/pooling/index.md                       | 2 +-
 docs/source/manual/speculative_execution/index.md         | 6 +++---
 docs/source/manual/statements/prepared/index.md           | 8 ++++----
 docs/source/manual/statements/simple/index.md             | 2 +-
 .../language_level_changes/index.md                       | 2 +-
 8 files changed, 20 insertions(+), 12 deletions(-)

diff --git a/docs/source/conf.py b/docs/source/conf.py
index 6ad69aaf651..4876bf9034f 100644
--- a/docs/source/conf.py
+++ b/docs/source/conf.py
@@ -236,6 +236,14 @@ def setup(app):
 .. |mon_root| replace::  :doc:`Scylla Monitoring Stack `
 """
 
+# Custom lexer
+from pygments.lexers.shell import BashLexer
+from sphinx.highlighting import lexers
+
+class DitaaLexer(BashLexer):
+    pass
+lexers['ditaa'] = DitaaLexer(startinline=True)
+
 # -- Options for multiversion --------------------------------------------
 # Whitelist pattern for tags (set to None to ignore all tags)
 smv_tag_whitelist = 'None'
diff --git a/docs/source/manual/metadata/index.md b/docs/source/manual/metadata/index.md
index 42457db931b..fd65ed90e4e 100644
--- a/docs/source/manual/metadata/index.md
+++ b/docs/source/manual/metadata/index.md
@@ -61,7 +61,7 @@ executing a schema-altering query. This is to ensure that subsequent
 requests (which might get routed to different nodes) see an up-to-date
 version of the schema.
 
-```console
+```ditaa
  Application             Driver           Cassandra
 ------+--------------------+------------------+-----
       |                    |                  |
diff --git a/docs/source/manual/paging/index.md b/docs/source/manual/paging/index.md
index 3b333568466..e9e3f1f8a79 100644
--- a/docs/source/manual/paging/index.md
+++ b/docs/source/manual/paging/index.md
@@ -42,7 +42,7 @@ The fetch size limits the number of results that are returned in one
 page; if you iterate past that, the driver will run background queries
 to fetch subsequent pages. Here's an example with a fetch size of 20:
 
-```console
+```ditaa
     client         Session                          Cassandra
     --+--------------+---------------------------------+-----
       |execute(query)|                                 |
diff --git a/docs/source/manual/pooling/index.md b/docs/source/manual/pooling/index.md
index 17137aa227b..ae654439095 100644
--- a/docs/source/manual/pooling/index.md
+++ b/docs/source/manual/pooling/index.md
@@ -31,7 +31,7 @@ described in the next section).  The number of stream ids depends on the
 * protocol v2 or below: 128 stream ids per connection.
 * protocol v3 or above: up to 32768 stream ids per connection.
 
-```console
+```ditaa
 +-------+1   n+-------+1   n+----+1   n+----------+1   128/32K+-------+
 |Cluster+-----+Session+-----+Pool+-----+Connection+-----------+Request+
 +-------+     +-------+     +----+     +----------+           +-------+
diff --git a/docs/source/manual/speculative_execution/index.md b/docs/source/manual/speculative_execution/index.md
index a8c22e0cfe7..d7f321c0c2c 100644
--- a/docs/source/manual/speculative_execution/index.md
+++ b/docs/source/manual/speculative_execution/index.md
@@ -12,7 +12,7 @@ note that "cancelling" in this context simply means discarding the response
 when it arrives later, Cassandra does not support cancellation of in flight
 requests at this stage):
 
-```console
+```ditaa
 client           driver          exec1  exec2
 --+----------------+--------------+------+---
   | execute(query) |
@@ -37,7 +37,7 @@ Or the first node could reply just after the second execution was
 started. In this case, we cancel the second execution. In other words,
 whichever node replies faster "wins" and completes the client query:
 
-```console
+```ditaa
 client           driver          exec1  exec2
 --+----------------+--------------+------+---
   | execute(query) |
@@ -176,7 +176,7 @@ As with all policies, you are free to provide your own by implementing
 Turning speculative executions on doesn't change the driver's [retry](../retries/index) behavior. Each
 parallel execution will trigger retries independently:
 
-```console
+```ditaa
 client           driver          exec1  exec2
 --+----------------+--------------+------+---
   | execute(query) |
diff --git a/docs/source/manual/statements/prepared/index.md b/docs/source/manual/statements/prepared/index.md
index 1a6c768994b..6bba541e3c9 100644
--- a/docs/source/manual/statements/prepared/index.md
+++ b/docs/source/manual/statements/prepared/index.md
@@ -15,7 +15,7 @@ session.execute(prepared.bind("987274", "Keyboard"));
 When you prepare the statement, Cassandra will parse the query string, cache the result and return a unique identifier
 (the `PreparedStatement` object keeps an internal reference to that identifier):
 
-```console
+```ditaa
 client                   driver           Cassandra
 --+------------------------+----------------+------
   |                        |                |
@@ -38,7 +38,7 @@ client                   driver           Cassandra
 When you bind and execute a prepared statement, the driver will only send the identifier, which allows Cassandra to
 skip the parsing phase:
 
-```console
+```ditaa
 client                            driver                Cassandra
 --+---------------------------------+---------------------+------
   |                                 |                     |
@@ -167,7 +167,7 @@ date. It uses a number of strategies to achieve this:
     the query string is wrong). Once that node replies successfully, the
     driver re-prepares on all remaining nodes:
 
-    ```console
+    ```ditaa
     client                   driver           node1          node2  node3
     --+------------------------+----------------+--------------+------+---
       |                        |                |              |      |
@@ -201,7 +201,7 @@ date. It uses a number of strategies to achieve this:
     statement on the fly (this is transparent for the client, but will cost
     two extra roundtrips):
 
-    ```console
+    ```ditaa
     client                          driver                         node1
     --+-------------------------------+------------------------------+--
       |                               |                              |
diff --git a/docs/source/manual/statements/simple/index.md b/docs/source/manual/statements/simple/index.md
index 2562a187fd6..b35f8e1f811 100644
--- a/docs/source/manual/statements/simple/index.md
+++ b/docs/source/manual/statements/simple/index.md
@@ -17,7 +17,7 @@ session.execute("SELECT value FROM application_params WHERE name = 'greeting_mes
 Each time you execute a simple statement, Cassandra will parse the query string again; nothing is cached (neither on the
 client nor on the server):
 
-```console
+```ditaa
 client                             driver                Cassandra
 --+----------------------------------+---------------------+------
   |                                  |                     |
diff --git a/docs/source/upgrade_guide/migrating_from_astyanax/language_level_changes/index.md b/docs/source/upgrade_guide/migrating_from_astyanax/language_level_changes/index.md
index d37f4abdcc3..9dc332c5a06 100644
--- a/docs/source/upgrade_guide/migrating_from_astyanax/language_level_changes/index.md
+++ b/docs/source/upgrade_guide/migrating_from_astyanax/language_level_changes/index.md
@@ -15,7 +15,7 @@ as a *Table*, the **Table Rows** get a **fixed and finite number of named column
 *Thrift*’s columns inside the *Rows* get distributed in a tabular way through the
 _Table Rows_. See the following figure:
 
-```console
+```ditaa
                                                      Thrift
              /-                                                                                          -\
              |                                                                                            |

From 1ea9b0315d44eedfc66a5d0426ae0b11711b6627 Mon Sep 17 00:00:00 2001
From: dgarcia360 
Date: Fri, 14 Aug 2020 12:47:10 +0200
Subject: [PATCH 38/52] Fixed merge

---
 driver-tests/osgi/pom.xml | 3 +++
 1 file changed, 3 insertions(+)

diff --git a/driver-tests/osgi/pom.xml b/driver-tests/osgi/pom.xml
index 5c8733eefe1..e192a53fe9e 100644
--- a/driver-tests/osgi/pom.xml
+++ b/driver-tests/osgi/pom.xml
@@ -22,7 +22,10 @@
     
         com.scylladb
         scylla-driver-tests-parent
+        3.10.1-scylla-1-SNAPSHOT	
+    
     scylla-driver-tests-osgi
+    Java Driver for Scylla and Apache Cassandra Tests - OSGi	
     A test for the Java Driver in an OSGi container.
 
     

From 26ebfeea96497b3421012a1dcfca6f481a7c93c8 Mon Sep 17 00:00:00 2001
From: dgarcia360 
Date: Fri, 14 Aug 2020 12:59:47 +0200
Subject: [PATCH 39/52] Clean PR

---
 docs/_utils/api.html                        | 2 +-
 docs/_utils/doxygen.sh                      | 2 +-
 docs/source/changelog/index.md              | 8 ++++----
 docs/source/manual/native_protocol/index.md | 2 +-
 driver-tests/osgi/pom.xml                   | 4 ++--
 5 files changed, 9 insertions(+), 9 deletions(-)

diff --git a/docs/_utils/api.html b/docs/_utils/api.html
index 684e559579c..1e876cb1a4c 100644
--- a/docs/_utils/api.html
+++ b/docs/_utils/api.html
@@ -5,4 +5,4 @@
     
     
   
-
\ No newline at end of file
+
diff --git a/docs/_utils/doxygen.sh b/docs/_utils/doxygen.sh
index ebd995cd5ba..ab1200765a0 100755
--- a/docs/_utils/doxygen.sh
+++ b/docs/_utils/doxygen.sh
@@ -9,4 +9,4 @@ if [[ "$GITHUB_VERSION" != "" ]]; then
 fi
 echo $OUTPUT_DIR
 mkdir -p "$OUTPUT_DIR"
-doxygen doxyfile
\ No newline at end of file
+doxygen doxyfile
diff --git a/docs/source/changelog/index.md b/docs/source/changelog/index.md
index 0be7fd1bc27..2fc74c44480 100644
--- a/docs/source/changelog/index.md
+++ b/docs/source/changelog/index.md
@@ -5,17 +5,17 @@
   3.x versions get published.
 -->
 
-## 3.10.1
+### 3.10.1
 
 - [bug] JAVA-2857: Fix NPE when built statements without parameters are logged at TRACE level.
 - [bug] JAVA-2843: Successfully parse DSE table schema in OSS driver.
 
-## 3.10.0
+### 3.10.0
 
 - [improvement] JAVA-2676: Don't reschedule flusher after empty runs
 - [new feature] JAVA-2772: Support new protocol v5 message format
 
-## 3.9.0
+### 3.9.0
 
 - [bug] JAVA-2627: Avoid logging error message including stack trace in request handler.
 - [new feature] JAVA-2706: Add now_in_seconds to protocol v5 query messages.
@@ -23,7 +23,7 @@
 - [improvement] JAVA-2702: Transient Replication Support for Cassandra® 4.0
 
 
-## 3.8.0
+### 3.8.0
 
 - [new feature] JAVA-2356: Support for DataStax Cloud API.
 - [improvement] JAVA-2483: Allow to provide secure bundle via URL.
diff --git a/docs/source/manual/native_protocol/index.md b/docs/source/manual/native_protocol/index.md
index a60a5b4b559..ced7ced52c0 100644
--- a/docs/source/manual/native_protocol/index.md
+++ b/docs/source/manual/native_protocol/index.md
@@ -5,7 +5,7 @@ between the driver and Cassandra over TCP. As a driver user, you don't
 need to know the fine details (although the protocol spec is [in the
 Cassandra codebase][native_spec] if you're curious); the most visible
 aspect is that some features are only available with specific protocol
-versions.f
+versions.
 
 [native_spec]: https://github.com/apache/cassandra/tree/trunk/doc
 
diff --git a/driver-tests/osgi/pom.xml b/driver-tests/osgi/pom.xml
index e192a53fe9e..b9dbec3b4f6 100644
--- a/driver-tests/osgi/pom.xml
+++ b/driver-tests/osgi/pom.xml
@@ -22,10 +22,10 @@
     
         com.scylladb
         scylla-driver-tests-parent
-        3.10.1-scylla-1-SNAPSHOT	
+        3.10.1-scylla-1-SNAPSHOT
     
     scylla-driver-tests-osgi
-    Java Driver for Scylla and Apache Cassandra Tests - OSGi	
+    Java Driver for Scylla and Apache Cassandra Tests - OSGi
     A test for the Java Driver in an OSGi container.
 
     

From 254b12407969c9f36eeee06ab0b1eae25ba7e45d Mon Sep 17 00:00:00 2001
From: dgarcia360 
Date: Fri, 28 Aug 2020 16:16:03 +0200
Subject: [PATCH 40/52] Replace index before building

---
 docs/source/conf.py                           | 10 +++++
 docs/source/faq/index.md                      | 22 +++++-----
 docs/source/faq/osgi/index.md                 |  8 ++--
 docs/source/installation/index.md             |  4 +-
 .../source/manual/address_resolution/index.md |  2 +-
 docs/source/manual/async/index.md             |  2 +-
 docs/source/manual/cloud/index.md             |  2 +-
 docs/source/manual/compression/index.md       |  8 ++--
 .../source/manual/control_connection/index.md | 12 +++---
 .../manual/custom_codecs/extras/index.md      |  6 +--
 docs/source/manual/custom_codecs/index.md     |  6 +--
 docs/source/manual/custom_payloads/index.md   |  2 +-
 docs/source/manual/idempotence/index.md       |  2 +-
 docs/source/manual/index.md                   | 42 +++++++++----------
 docs/source/manual/load_balancing/index.md    | 18 ++++----
 docs/source/manual/native_protocol/index.md   |  8 ++--
 .../manual/object_mapper/creating/index.md    |  4 +-
 .../object_mapper/custom_codecs/index.md      |  6 +--
 docs/source/manual/object_mapper/index.md     |  6 +--
 .../manual/object_mapper/using/index.md       |  4 +-
 docs/source/manual/osgi/index.md              |  2 +-
 docs/source/manual/paging/index.md            |  4 +-
 docs/source/manual/pooling/index.md           |  6 +--
 docs/source/manual/query_timestamps/index.md  |  4 +-
 docs/source/manual/reconnection/index.md      |  4 +-
 docs/source/manual/retries/index.md           |  6 +--
 docs/source/manual/shaded_jar/index.md        |  2 +-
 docs/source/manual/socket_options/index.md    | 12 +++---
 .../manual/speculative_execution/index.md     |  8 ++--
 docs/source/manual/statements/built/index.md  |  6 +--
 docs/source/manual/statements/index.md        |  8 ++--
 .../manual/statements/prepared/index.md       |  4 +-
 docs/source/manual/statements/simple/index.md |  4 +-
 docs/source/manual/udts/index.md              |  2 +-
 docs/source/upgrade_guide/index.md            | 22 +++++-----
 .../configuration/index.md                    | 18 ++++----
 .../migrating_from_astyanax/index.md          |  6 +--
 .../language_level_changes/index.md           |  2 +-
 .../queries_and_results/index.md              |  8 ++--
 39 files changed, 156 insertions(+), 146 deletions(-)

diff --git a/docs/source/conf.py b/docs/source/conf.py
index 4876bf9034f..1854fce4aec 100644
--- a/docs/source/conf.py
+++ b/docs/source/conf.py
@@ -73,7 +73,17 @@ def create_redirects(app, docname):
 source_suffix = ['.rst', '.md']
 autosectionlabel_prefix_document = True
 
+def replace_relative_links(app, docname, source):
+    result = source[0]
+    for key in app.config.replacements:
+        result = result.replace(key, app.config.replacements[key])
+    source[0] = result
+
 def setup(app):
+    replacements = {"/)" : "/index)", "/#": "/index#"}
+    app.add_config_value('replacements', replacements, True)
+    app.connect('source-read', replace_relative_links)
+
     app.add_config_value('recommonmark_config', {
         'enable_eval_rst': True,
         'enable_auto_toc_tree': False,
diff --git a/docs/source/faq/index.md b/docs/source/faq/index.md
index 565f4c7f114..843ba66aa12 100644
--- a/docs/source/faq/index.md
+++ b/docs/source/faq/index.md
@@ -2,10 +2,10 @@
 
 ### How do I implement paging?
 
-When using [native protocol](../manual/native_protocol/index) version 2 or
+When using [native protocol](../manual/native_protocol/) version 2 or
 higher, the driver automatically pages large result sets under the hood.
 You can also save the paging state to resume iteration later. See [this
-page](../manual/paging/index) for more information.
+page](../manual/paging/) for more information.
 
 Native protocol v1 does not support paging, but you can emulate it in
 CQL with `LIMIT` and the `token()` function. See
@@ -49,7 +49,7 @@ session.execute( "INSERT INTO contacts (email, firstname, lastname)
       VALUES (?, ?, ?)", "clint.barton@hawkeye.com", "Barney", "Barton");
 ```
 
-See [Simple statements](../manual/statements/simple/index) for more information.
+See [Simple statements](../manual/statements/simple/) for more information.
 
 
 ### Does a parameterized statement escape parameters?
@@ -69,7 +69,7 @@ the way that the parameters are sent. The difference is that a prepared statemen
 * sends only the statement id and its parameters (thus reducing the amount of data sent
   to the cluster)
 
-See [Prepared statements](../manual/statements/prepared/index) for more information.
+See [Prepared statements](../manual/statements/prepared/) for more information.
 
 
 ### Can I combine `PreparedStatements` and normal statements in a batch?
@@ -96,7 +96,7 @@ is currently no mechanism for Cassandra to invalidate the existing metadata.  Be
 the driver is not able to properly react to these changes and will improperly read rows after
 a schema change is made.
 
-See [Prepared statements](../manual/statements/prepared/index) for more information.
+See [Prepared statements](../manual/statements/prepared/) for more information.
 
 
 ### Can I get the raw bytes of a text column?
@@ -135,7 +135,7 @@ Keep in mind that if your code iterates the `ResultSet` entirely, the driver may
 run additional background queries to fetch the rest of the data. The fetch size
 only affects what is retrieved at a time, not the overall number of rows.
 
-See [Paging](../manual/paging/index) for more information.
+See [Paging](../manual/paging/) for more information.
 
 
 ### What's the difference between using `setFetchSize()` and `LIMIT`?
@@ -147,7 +147,7 @@ For example, if you limit is 30 and your fetch size is 10, the `ResultSet` will
 30 rows, but under the hood the driver will perform 3 requests that will transfer 10
 rows each.
 
-See [Paging](../manual/paging/index) for more information.
+See [Paging](../manual/paging/) for more information.
 
 
 ### I'm reading a BLOB column and the driver returns incorrect data.
@@ -162,7 +162,7 @@ See [Blobs.java] in the `driver-examples` module for some examples and explanati
 
 ### How do I use the driver in an OSGi application?
 
-Read our [OSGi-specific FAQ section](osgi/index) to find out.
+Read our [OSGi-specific FAQ section](osgi/) to find out.
 
 
 ### Why am I seeing messages about `tombstone_warn_threshold` or `tombstone_fail_threshold` being exceeded in my Cassandra logs?
@@ -291,9 +291,9 @@ an update with compatibility to this driver version.
 
 [Blobs.java]: https://github.com/datastax/java-driver/tree/3.10.0/driver-examples/src/main/java/com/datastax/driver/examples/datatypes/Blobs.java
 [CASSANDRA-7304]: https://issues.apache.org/jira/browse/CASSANDRA-7304
-[Parameters and Binding]: ../manual/statements/prepared/index#parameters-and-binding
-[Mapper options]: ../manual/object_mapper/using/index#mapper-options
-[Acquisition queue]: ../manual/pooling/index#acquisition-queue
+[Parameters and Binding]: ../manual/statements/prepared/#parameters-and-binding
+[Mapper options]: ../manual/object_mapper/using/#mapper-options
+[Acquisition queue]: ../manual/pooling/#acquisition-queue
 [Semaphore]: https://docs.oracle.com/javase/8/docs/api/java/util/concurrent/Semaphore.html
 [Futures.allAsList]: https://google.github.io/guava/releases/19.0/api/docs/com/google/common/util/concurrent/Futures.html#allAsList(java.lang.Iterable)
 [DynaTrace]: https://www.dynatrace.com/
diff --git a/docs/source/faq/osgi/index.md b/docs/source/faq/osgi/index.md
index 9cbe520f99e..5b2d22af0ae 100644
--- a/docs/source/faq/osgi/index.md
+++ b/docs/source/faq/osgi/index.md
@@ -32,7 +32,7 @@ of Guava's packages, e.g. for 19.0:
 
 ### How to enable compression?
 
-First, read our [manual page on compression](../../manual/compression/index)
+First, read our [manual page on compression](../../manual/compression/)
 to understand how to enable compression for the Java driver.
             
 OSGi projects can use both Snappy or LZ4 compression algorithms. 
@@ -99,14 +99,14 @@ _not_ its Maven artifact id.
     
 ### How to use the driver shaded jar?
 
-The driver [shaded jar](../../manual/shaded_jar/index) can be used 
+The driver [shaded jar](../../manual/shaded_jar/) can be used 
 in any OSGi application, although the same limitations explained in
 the manual apply.
 
 
 ### How to get proper logs?
 
-The driver uses [SLF4j] for [logging](../../manual/logging/index).
+The driver uses [SLF4j] for [logging](../../manual/logging/).
 
 You OSGi runtime should therefore include the SLF4J API bundle, and
 one valid implementation bundle, such as [Logback].
@@ -135,7 +135,7 @@ Please consult their documentation for further details.
 
 The driver is able to perform native system calls through JNR in some cases,
 for example to achieve microsecond resolution when 
-[generating timestamps](../../manual/query_timestamps/index).
+[generating timestamps](../../manual/query_timestamps/).
 
 Unfortunately, some of the JNR artifacts available from Maven 
 are not valid OSGi bundles and cannot be used in OSGi applications.
diff --git a/docs/source/installation/index.md b/docs/source/installation/index.md
index 00b1897537a..be6b9cf5496 100644
--- a/docs/source/installation/index.md
+++ b/docs/source/installation/index.md
@@ -35,7 +35,7 @@ The 'extras' module is also published as a separate artifact:
 ```
 
 
-We also provide a [shaded JAR](../manual/shaded_jar/index)
+We also provide a [shaded JAR](../manual/shaded_jar/)
 to avoid the explicit dependency to Netty.
 
 If you can't use a dependency management tool, a
@@ -68,7 +68,7 @@ big-endian systems, but _DataStax_ does not officially support these systems.
 ### Upgrading from previous versions
 
 If you are upgrading from a previous version of the driver, be sure to have a look at
-the [upgrade guide](../upgrade_guide/index).
+the [upgrade guide](../upgrade_guide/).
 
 If you are upgrading to _DataStax Enterprise_, use the [DataStax Enterprise Java driver][dse-driver] for more
 features and better compatibility.
diff --git a/docs/source/manual/address_resolution/index.md b/docs/source/manual/address_resolution/index.md
index 6bea7597161..635d079d0c3 100644
--- a/docs/source/manual/address_resolution/index.md
+++ b/docs/source/manual/address_resolution/index.md
@@ -5,7 +5,7 @@ connections.
 
 * for contact points, these are provided as part of configuring the `Cluster` object;
 * for other nodes, addresses will be discovered dynamically, either by inspecting `system.peers` on already connected
-  nodes, or via push notifications received from the [control host](../control_connection/index) when new nodes are discovered
+  nodes, or via push notifications received from the [control host](../control_connection/) when new nodes are discovered
   by gossip.
 
 
diff --git a/docs/source/manual/async/index.md b/docs/source/manual/async/index.md
index 4d8c0a8d9dc..21a70f7b767 100644
--- a/docs/source/manual/async/index.md
+++ b/docs/source/manual/async/index.md
@@ -48,7 +48,7 @@ If you consume a `ResultSet` in a callback, be aware that iterating the
 rows will trigger synchronous queries as you page through the results.
 To avoid this, use [getAvailableWithoutFetching] to limit the iteration
 to the current page, and [fetchMoreResults] to get a future to the next
-page (see also the section on [paging](../paging/index)).
+page (see also the section on [paging](../paging/)).
 Here is a full example:
 
 [getAvailableWithoutFetching]: https://docs.datastax.com/en/drivers/java/3.10/com/datastax/driver/core/ResultSet.html#getAvailableWithoutFetching--
diff --git a/docs/source/manual/cloud/index.md b/docs/source/manual/cloud/index.md
index 3faba9223fd..5b9268643a2 100644
--- a/docs/source/manual/cloud/index.md
+++ b/docs/source/manual/cloud/index.md
@@ -103,4 +103,4 @@ interacting with any other Cassandra cluster. The exceptions being:
 [Access an Apollo database - AWS]: https://helpdocs.datastax.com/aws/dscloud/apollo/dscloudShareClusterDetails.html
 [Download the secure connect bundle - GCP]: https://helpdocs.datastax.com/gcp/dscloud/apollo/dscloudObtainingCredentials.html
 [Download the secure connect bundle - AWS]: https://helpdocs.datastax.com/aws/dscloud/apollo/dscloudObtainingCredentials.html
-[pom.xml dependency]: ../../index#getting-the-driver
+[pom.xml dependency]: ../../#getting-the-driver
diff --git a/docs/source/manual/compression/index.md b/docs/source/manual/compression/index.md
index ca74446a4c3..ffc56d79495 100644
--- a/docs/source/manual/compression/index.md
+++ b/docs/source/manual/compression/index.md
@@ -4,7 +4,7 @@ Cassandra's binary protocol supports optional compression of
 transport-level requests and responses, for example:
 
 * a query with its serialized parameters;
-* a [page](../paging/index) from a result set, i.e. a list of serialized
+* a [page](../paging/) from a result set, i.e. a list of serialized
   rows.
 
 It reduces network traffic at the cost of CPU overhead, therefore it
@@ -12,10 +12,10 @@ will likely be beneficial when you have larger payloads.
 
 Two algorithms are available:
 [LZ4](https://github.com/jpountz/lz4-java) and
-[Snappy](https://code.google.com/p/snappy/index).  The LZ4 implementation is a good
+[Snappy](https://code.google.com/p/snappy/).  The LZ4 implementation is a good
 first choice; it offers fallback implementations in case native libraries fail
 to load and
-[benchmarks](http://java-performance.info/performance-general-compression/index)
+[benchmarks](http://java-performance.info/performance-general-compression/)
 suggest that it offers better performance and compression ratios over Snappy.
 Both rely on third-party libraries, declared by the driver as *optional*
 dependencies. So if you use a build tool like Maven, you'll need to declare an
@@ -55,7 +55,7 @@ LZ4-java has three internal implementations (from fastest to slowest):
 
 It will pick the best implementation depending on what's possible on
 your platform. To find out which one was chosen, [enable INFO
-logs](../logging/index) on the category
+logs](../logging/) on the category
 `com.datastax.driver.core.LZ4Compressor` and look for a log similar to
 this:
 
diff --git a/docs/source/manual/control_connection/index.md b/docs/source/manual/control_connection/index.md
index 3e3b8ee98fd..60aad531528 100644
--- a/docs/source/manual/control_connection/index.md
+++ b/docs/source/manual/control_connection/index.md
@@ -3,19 +3,19 @@
 The control connection is a dedicated connection used for administrative tasks:
 
 * querying system tables to learn about the cluster's topology and
-  [schema](../metadata/index#schema-metadata);
-* checking [schema agreement](../metadata/index#schema-agreement);
+  [schema](../metadata/#schema-metadata);
+* checking [schema agreement](../metadata/#schema-agreement);
 * reacting to server events, which are used to notify the driver of external topology or schema
   changes.
 
 When the driver starts, the control connection is established to the first contacted node. If that
-node goes down, a [reconnection](../reconnection/index) is started to find another node; it is governed
+node goes down, a [reconnection](../reconnection/) is started to find another node; it is governed
 by the same policy as regular connections and tries the nodes according to a query plan from the
-[load balancing policy](../load_balancing/index).
+[load balancing policy](../load_balancing/).
 
-The control connection is managed independently from [regular pooled connections](../pooling/index), and
+The control connection is managed independently from [regular pooled connections](../pooling/), and
 used exclusively for administrative requests. It is included in [Session.State.getOpenConnections],
-as well as the `open-connections` [metric](../metrics/index); for example, if you've configured a pool
+as well as the `open-connections` [metric](../metrics/); for example, if you've configured a pool
 size of 2, the control node will have 3 connections.
 
 [Session.State.getOpenConnections]: https://docs.datastax.com/en/drivers/java/3.10/com/datastax/driver/core/Session.State.html#getOpenConnections-com.datastax.driver.core.Host-
\ No newline at end of file
diff --git a/docs/source/manual/custom_codecs/extras/index.md b/docs/source/manual/custom_codecs/extras/index.md
index 523a3eaceb3..f89102e259c 100644
--- a/docs/source/manual/custom_codecs/extras/index.md
+++ b/docs/source/manual/custom_codecs/extras/index.md
@@ -4,7 +4,7 @@
 
 ## Optional codecs
 
-The driver's "extras" module provides additional [codec](../index) implementations: these codecs are not required by core
+The driver's "extras" module provides additional [codec](../) implementations: these codecs are not required by core
 driver features, but will probably prove useful in a lot of client applications. You can also study their source code as
 a reference to write your own.
 
@@ -189,10 +189,10 @@ session.execute("insert into ordinal_example (id, state) values (1, ?)", State.I
 // state saved as 0
 ```
 
-Note that if you registered an `EnumNameCodec` and an `EnumOrdinalCodec` _for the same enum_ at the same time, there could be a problem when executing [simple statements](../../statements/simple/index), because in a simple statement, the target CQL type of a given query parameter is not known in advance, so the driver, on a best-effort attempt, will pick one or the other, whichever was registered first. If the chosen codec proves to be the wrong one, the request would fail on the server side.
+Note that if you registered an `EnumNameCodec` and an `EnumOrdinalCodec` _for the same enum_ at the same time, there could be a problem when executing [simple statements](../../statements/simple/), because in a simple statement, the target CQL type of a given query parameter is not known in advance, so the driver, on a best-effort attempt, will pick one or the other, whichever was registered first. If the chosen codec proves to be the wrong one, the request would fail on the server side.
 
 In practice, this is unlikely to happen, because you'll probably stick to a single CQL type for a given enum type;
-however, if you ever run into that issue, the workaround is to use [prepared statements](../../statements/prepared/index), for which the driver knows the CQL type and can pick the exact codec.
+however, if you ever run into that issue, the workaround is to use [prepared statements](../../statements/prepared/), for which the driver knows the CQL type and can pick the exact codec.
 
 [EnumNameCodec]: https://docs.datastax.com/en/drivers/java/3.10/com/datastax/driver/extras/codecs/enums/EnumNameCodec.html
 [EnumOrdinalCodec]: https://docs.datastax.com/en/drivers/java/3.10/com/datastax/driver/extras/codecs/enums/EnumOrdinalCodec.html
diff --git a/docs/source/manual/custom_codecs/index.md b/docs/source/manual/custom_codecs/index.md
index dcf399f4d6f..108d7dab1d4 100644
--- a/docs/source/manual/custom_codecs/index.md
+++ b/docs/source/manual/custom_codecs/index.md
@@ -12,7 +12,7 @@ Practical use cases that justify such a feature are numerous:
 * etc.
 
 This page explains the implementation and how to write your own custom codecs. Note that the driver also provides a set
-of [optional codecs](extras/index) that might fit your needs.
+of [optional codecs](extras/) that might fit your needs.
 
 ### Overview of the serialization mechanism
 
@@ -203,7 +203,7 @@ make sure that you use preferably the `get()` and `set()` methods: they
 avoid any ambiguity by requiring the user to explicitly specify the desired Java type,
 thus forcing the driver to pick the right codec for the right task.
 
-Custom codecs also work with the driver's [object mapper](../object_mapper/custom_codecs/index).
+Custom codecs also work with the driver's [object mapper](../object_mapper/custom_codecs/).
 
 ### On-the-fly codec generation
 
@@ -364,7 +364,7 @@ serializing and deserializing. It's possible to bypass this step with a
 lower-level implementation that manipulates the binary stream directly.
 That's also how the object mapper handles UDTs, and you can rely on the
 mapper to generate UDT codecs for you; see
-[this page](../object_mapper/custom_codecs/index#implicit-udt-codecs) for more
+[this page](../object_mapper/custom_codecs/#implicit-udt-codecs) for more
 information.
 
 ### Support for generic (parameterized) types
diff --git a/docs/source/manual/custom_payloads/index.md b/docs/source/manual/custom_payloads/index.md
index 5c0a8015617..a31cee5161b 100644
--- a/docs/source/manual/custom_payloads/index.md
+++ b/docs/source/manual/custom_payloads/index.md
@@ -1,6 +1,6 @@
 ## Custom Payloads
 
-The [native protocol](../native_protocol/index) version 4 introduces a new feature called [Custom Payloads][CASSANDRA-8553].
+The [native protocol](../native_protocol/) version 4 introduces a new feature called [Custom Payloads][CASSANDRA-8553].
 
 According to the [protocol V4 specification][v4spec], custom payloads are generic key-value maps
 where keys are strings and each value is an arbitrary sequence of bytes. Currently payloads 
diff --git a/docs/source/manual/idempotence/index.md b/docs/source/manual/idempotence/index.md
index 6e670578d73..71246748735 100644
--- a/docs/source/manual/idempotence/index.md
+++ b/docs/source/manual/idempotence/index.md
@@ -8,7 +8,7 @@ example:
 * `update my_table set list_col = [1] + list_col where pk = 1` is not idempotent: if `list_col` was initially empty,
   it will contain `[1]` after the first execution, `[1, 1]` after the second, etc.
 
-Idempotence matters for [retries](../retries/index) and [speculative query executions](../speculative_execution/index). The driver
+Idempotence matters for [retries](../retries/) and [speculative query executions](../speculative_execution/). The driver
 will bypass those features if the [Statement#isIdempotent()][isIdempotent] flag is set to `false`, to ensure that the
 statement does not get executed more than once.
 
diff --git a/docs/source/manual/index.md b/docs/source/manual/index.md
index 10eccc5d4ca..17d4544b5b3 100644
--- a/docs/source/manual/index.md
+++ b/docs/source/manual/index.md
@@ -21,7 +21,7 @@ try {
 ```
 
 1. the [Cluster] object is the main entry point of the driver. It holds the known state of the actual Cassandra cluster
-   (notably the [Metadata](metadata/index)). This class is thread-safe, you should create a single instance (per target
+   (notably the [Metadata](metadata/)). This class is thread-safe, you should create a single instance (per target
    Cassandra cluster), and share it throughout your application;
 2. the [Session] is what you use to execute queries. Likewise, it is thread-safe and should be reused;
 3. we use `execute` to send a query to Cassandra. This returns a [ResultSet], which is essentially a collection of [Row]
@@ -31,7 +31,7 @@ try {
    cluster. This step is important because it frees underlying resources (TCP connections, thread pools...). In a real
    application, you would typically do this at shutdown (for example, when undeploying your webapp).
 
-Note: this example uses the synchronous API. Most methods have [asynchronous](async/index) equivalents.
+Note: this example uses the synchronous API. Most methods have [asynchronous](async/) equivalents.
 
 
 ### Setting up the driver
@@ -65,19 +65,19 @@ that the driver can fallback if the first one is down.
 
 The other aspects that you can configure on the `Cluster` are:
 
-* [address translation](address_resolution/index);
-* [authentication](auth/index);
-* [compression](compression/index);
-* [load balancing](load_balancing/index);
-* [metrics](metrics/index);
+* [address translation](address_resolution/);
+* [authentication](auth/);
+* [compression](compression/);
+* [load balancing](load_balancing/);
+* [metrics](metrics/);
 * low-level [Netty configuration][NettyOptions];
 * [query options][QueryOptions];
-* [reconnections](reconnection/index);
-* [retries](retries/index);
-* [socket options](socket_options/index);
-* [SSL](ssl/index);
-* [speculative executions](speculative_execution/index);
-* [query timestamps](query_timestamps/index).
+* [reconnections](reconnection/);
+* [retries](retries/);
+* [socket options](socket_options/);
+* [SSL](ssl/);
+* [speculative executions](speculative_execution/);
+* [query timestamps](query_timestamps/).
 
 In addition, you can register various types of listeners to be notified of cluster events; see [Host.StateListener],
 [LatencyTracker], and [SchemaChangeListener].
@@ -98,8 +98,8 @@ The initialization sequence is the following:
 * try to connect to each of the contact points in sequence. The order is not deterministic (in fact, the driver shuffles
   the list to avoid hotspots if a large number of clients share the same contact points). If no contact point replies,
   a [NoHostAvailableException] is thrown and the process stops here;
-* otherwise, the successful contact point is elected as the [control host](control_connection/index). The driver negotiates
-  the [native protocol version](native_protocol/index) with it, and queries its system tables to discover the addresses of
+* otherwise, the successful contact point is elected as the [control host](control_connection/). The driver negotiates
+  the [native protocol version](native_protocol/) with it, and queries its system tables to discover the addresses of
   the other hosts.
 
 Note that, at this stage, only the control connection has been established. Connections to other hosts will only be
@@ -124,7 +124,7 @@ session.execute("select * from otherKeyspace.otherTable where id = 1");
 ```
 
 You might be tempted to open a separate session for each keyspace used in your application; however, note that
-[connection pools](pooling/index) are created at the session level, so each new session will consume additional system
+[connection pools](pooling/) are created at the session level, so each new session will consume additional system
 resources:
 
 ```java
@@ -133,7 +133,7 @@ Session session1 = cluster.connect("ks1");
 Session session2 = cluster.connect("ks2");
 ```
 
-Also, there is currently a [known limitation](async/index#known-limitations) with named sessions, that causes the driver to
+Also, there is currently a [known limitation](async/#known-limitations) with named sessions, that causes the driver to
 unexpectedly block the calling thread in certain circumstances; if you use a fully asynchronous model, you should use a
 session with no keyspace.
 
@@ -164,7 +164,7 @@ ResultSet rs = session.execute("select release_version from system.local");
 ```
 
 As shown here, the simplest form is to pass a query string directly. You can also pass an instance of
-[Statement](statements/index).
+[Statement](statements/).
 
 #### Processing rows
 
@@ -179,7 +179,7 @@ for (Row row : rs) {
 
 Note that this will return **all results** without limit (even though the driver might use multiple queries in the
 background). To handle large result sets, you might want to use a `LIMIT` clause in your CQL query, or use one of the
-techniques described in the [paging](paging/index) documentation.
+techniques described in the [paging](paging/) documentation.
 
 When you know that there is only one row (or are only interested in the first one), the driver provides a convenience
 method:
@@ -231,7 +231,7 @@ String firstName = row.getString("first_name");
      varint getVarint java.math.BigInteger 
 
 
-In addition to these default mappings, you can register your own types with [custom codecs](custom_codecs/index).
+In addition to these default mappings, you can register your own types with [custom codecs](custom_codecs/).
 
 ##### Primitive types
 
@@ -280,7 +280,7 @@ for (ColumnDefinitions.Definition definition : row.getColumnDefinitions()) {
 ### Object mapping
 
 Besides explicit work with queries and rows, you can also use
-[Object Mapper](object_mapper/index) to simplify retrieval & store of your data.
+[Object Mapper](object_mapper/) to simplify retrieval & store of your data.
 
 
 ### More information
diff --git a/docs/source/manual/load_balancing/index.md b/docs/source/manual/load_balancing/index.md
index c1b505793e2..686277a9bcf 100644
--- a/docs/source/manual/load_balancing/index.md
+++ b/docs/source/manual/load_balancing/index.md
@@ -38,8 +38,8 @@ For each host, the policy computes a **[distance][HostDistance]** that determine
 to it:
 
 * `LOCAL` and `REMOTE` are "active" distances, meaning that the driver will keep open connections to the host. They
-  differ in the number of connections opened, depending on your [pooling options](../pooling/index). Also, the
-  [control connection](../control_connection/index) will  favor local nodes if possible.
+  differ in the number of connections opened, depending on your [pooling options](../pooling/). Also, the
+  [control connection](../control_connection/) will  favor local nodes if possible.
 * `IGNORED`, as the name suggests, means that the driver will not attempt to connect.
 
 Typically, the distance will reflect network topology (e.g. local vs. remote datacenter), although that is entirely up
@@ -57,8 +57,8 @@ cluster.getConfiguration().getPoolingOptions().refreshConnectedHost(host);
 #### Query plan
 
 Each time the driver executes a query, it asks the policy to compute a **query plan**, which is a list of hosts. The
-driver will then try each host in sequence, according to the [retry policy](../retries/index) and
-[speculative execution policy](../speculative_execution/index).
+driver will then try each host in sequence, according to the [retry policy](../retries/) and
+[speculative execution policy](../speculative_execution/).
 
 The contents and order of query plans are entirely up to your policy, but implementations typically return plans that:
 
@@ -108,7 +108,7 @@ Cluster cluster = Cluster.builder()
 This policy queries nodes of the local data-center in a round-robin fashion.
 
 Call `withLocalDc` to specify the name of your local datacenter. You can also leave it out, and the driver will use the
-datacenter of the first contact point that was reached [at initialization](../index#cluster-initialization). However,
+datacenter of the first contact point that was reached [at initialization](../#cluster-initialization). However,
 remember that the driver shuffles the initial list of contact points, so this assumes that all contact points are in the
 local datacenter. In general, providing the datacenter name explicitly is a safer option.
 
@@ -144,7 +144,7 @@ CREATE TABLE testKs.sensor_data(id int, year int, ts timestamp, data double,
                                 PRIMARY KEY ((id, year), ts));
 ```
 
-For [simple statements](../statements/simple/index), routing information can never be computed automatically:
+For [simple statements](../statements/simple/), routing information can never be computed automatically:
 
 ```java
 SimpleStatement statement = new SimpleStatement(
@@ -166,7 +166,7 @@ statement.setRoutingKey(
 session.execute(statement);
 ```
 
-For [built statements](../statements/built/index), the keyspace is available if it was provided while building the query; the
+For [built statements](../statements/built/), the keyspace is available if it was provided while building the query; the
 routing key is available only if the statement was built using the table metadata, and all components of the partition
 key appear in the query:
 
@@ -192,7 +192,7 @@ assert statement2.getKeyspace() != null;
 assert statement2.getRoutingKey() == null;
 ```
 
-For [bound statements](../statements/prepared/index), the keyspace is always available; the routing key is only available if
+For [bound statements](../statements/prepared/), the keyspace is always available; the routing key is only available if
 all components of the partition key are bound as variables:
 
 ```java
@@ -211,7 +211,7 @@ assert statement2.getKeyspace() != null;
 assert statement2.getRoutingKey() == null;
 ```
 
-For [batch statements](../statements/batch/index), the routing information of each child statement is inspected; the first
+For [batch statements](../statements/batch/), the routing information of each child statement is inspected; the first
 non-null keyspace is used as the keyspace of the batch, and the first non-null routing key as its routing key (the idea
 is that all childs should have the same routing information, since batches are supposed to operate on a single
 partition). All children might have null information, in which case you need to provide the information manually as
diff --git a/docs/source/manual/native_protocol/index.md b/docs/source/manual/native_protocol/index.md
index ced7ced52c0..e07ff9fe8bd 100644
--- a/docs/source/manual/native_protocol/index.md
+++ b/docs/source/manual/native_protocol/index.md
@@ -97,17 +97,17 @@ To avoid this issue, you can use one the following workarounds:
 * bound variables in simple statements
   ([Session#execute(String, Object...)](https://docs.datastax.com/en/drivers/java/3.10/com/datastax/driver/core/Session.html#execute-java.lang.String-java.lang.Object...-))
 * [batch statements](https://docs.datastax.com/en/drivers/java/3.10/com/datastax/driver/core/BatchStatement.html)
-* [query paging](../paging/index)
+* [query paging](../paging/)
 
 #### v2 to v3
 
 * the number of stream ids per connection goes from 128 to 32768 (see
-  [Connection pooling](../pooling/index))
+  [Connection pooling](../pooling/))
 * [serial consistency on batch statements](https://docs.datastax.com/en/drivers/java/3.10/com/datastax/driver/core/BatchStatement.html#setSerialConsistencyLevel-com.datastax.driver.core.ConsistencyLevel-)
-* [client-side timestamps](../query_timestamps/index)
+* [client-side timestamps](../query_timestamps/)
 
 #### v3 to v4
 
 * [query warnings](https://docs.datastax.com/en/drivers/java/3.10/com/datastax/driver/core/ExecutionInfo.html#getWarnings--)
 * allowed unset values in bound statements
-* [Custom payloads](../custom_payloads/index)
+* [Custom payloads](../custom_payloads/)
diff --git a/docs/source/manual/object_mapper/creating/index.md b/docs/source/manual/object_mapper/creating/index.md
index 7946b5ccf0d..9ff0aebd19f 100644
--- a/docs/source/manual/object_mapper/creating/index.md
+++ b/docs/source/manual/object_mapper/creating/index.md
@@ -244,8 +244,8 @@ int f;
 ```
 
 Finally, computed properties are only supported with [basic read
-operations](../using/index#basic-crud-operations) at this time.
-Support in [accessors](../using/index#accessors) is planned for a future
+operations](../using/#basic-crud-operations) at this time.
+Support in [accessors](../using/#accessors) is planned for a future
 version (see
 [JAVA-832](https://datastax-oss.atlassian.net/browse/JAVA-832)).
 
diff --git a/docs/source/manual/object_mapper/custom_codecs/index.md b/docs/source/manual/object_mapper/custom_codecs/index.md
index bed66d60538..c68f59ba67f 100644
--- a/docs/source/manual/object_mapper/custom_codecs/index.md
+++ b/docs/source/manual/object_mapper/custom_codecs/index.md
@@ -1,6 +1,6 @@
 # Using custom codecs
 
-The mapper can take advantage of [custom codecs](../../custom_codecs/index)
+The mapper can take advantage of [custom codecs](../../custom_codecs/)
 to apply custom conversions to mapped columns and fields.
 
 ## Declaring codecs
@@ -106,9 +106,9 @@ This also works with [@Field][field] and [@Param][param] annotations.
 ## Implicit UDT codecs
 
 The mapper uses custom codecs internally to handle UDT conversions: when
-you register an [entity](../using/index#entity-mappers), the mapper inspects
+you register an [entity](../using/#entity-mappers), the mapper inspects
 the type of all fields to find classes annotated with
-[@UDT](../creating/index#mapping-user-types) (this works recursively with
+[@UDT](../creating/#mapping-user-types) (this works recursively with
 nested UDTs and collections). For each class, the mapper creates a codec
 and registers it with the underlying `Cluster`.
 
diff --git a/docs/source/manual/object_mapper/index.md b/docs/source/manual/object_mapper/index.md
index 49cb7984ad6..6c55e55714c 100644
--- a/docs/source/manual/object_mapper/index.md
+++ b/docs/source/manual/object_mapper/index.md
@@ -17,9 +17,9 @@ The mapper is published as a separate Maven artifact:
 
 See the child pages for more information:
 
-* [definition of mapped classes](creating/index)
-* [using the mapper](using/index)
-* [using custom codecs](custom_codecs/index)
+* [definition of mapped classes](creating/)
+* [using the mapper](using/)
+* [using custom codecs](custom_codecs/)
 
 ```eval_rst
 .. toctree::
diff --git a/docs/source/manual/object_mapper/using/index.md b/docs/source/manual/object_mapper/using/index.md
index a9ac0d49611..f6367ecf203 100644
--- a/docs/source/manual/object_mapper/using/index.md
+++ b/docs/source/manual/object_mapper/using/index.md
@@ -117,7 +117,7 @@ Some options don't apply to all operations:
 
 
 Note that `Option.consistencyLevel` is redundant with the consistency
-level defined by [@Table](../creating/index#creating-a-table-entity).
+level defined by [@Table](../creating/#creating-a-table-entity).
 If both are defined, the option will take precedence over the
 annotation.
 
@@ -233,7 +233,7 @@ ResultSet insert(@Param("u") UUID userId, @Param("n") String name);
 
 If a method argument is a Java enumeration, it must be annotated with
 `@Enumerated` to indicate how to convert it to a CQL type (the rules are
-the same as in [mapping definition](../creating/index#enumerations)):
+the same as in [mapping definition](../creating/#enumerations)):
 
 ```java
 @Query("insert into user (key, gender) values (?,?)")
diff --git a/docs/source/manual/osgi/index.md b/docs/source/manual/osgi/index.md
index 6e5c6722c16..42f9ba1693c 100644
--- a/docs/source/manual/osgi/index.md
+++ b/docs/source/manual/osgi/index.md
@@ -20,7 +20,7 @@ environment; please refer to our [OSGi examples repository].
 
 ## Troubleshooting OSGi applications
 
-Please read our [OSGi-specific FAQ section](../../faq/osgi/index) to find tips
+Please read our [OSGi-specific FAQ section](../../faq/osgi/) to find tips
 for troubleshooting OSGi applications.
 
 [OSGi]:https://www.osgi.org
diff --git a/docs/source/manual/paging/index.md b/docs/source/manual/paging/index.md
index e9e3f1f8a79..d3628cbfaa8 100644
--- a/docs/source/manual/paging/index.md
+++ b/docs/source/manual/paging/index.md
@@ -95,7 +95,7 @@ for (Row row : rs) {
 
 If you use paging with the async API, you'll also want to use those
 methods to avoid triggering synchronous fetches unintentionally; see
-[async paging](../async/index#async-paging).
+[async paging](../async/#async-paging).
 
 
 ### Saving and reusing the paging state
@@ -181,7 +181,7 @@ if (nextPage != null) {
 
 
 Due to internal implementation details, `PagingState` instances are not
-portable across [native protocol](../native_protocol/index) versions. This
+portable across [native protocol](../native_protocol/) versions. This
 could become a problem in the following scenario:
 
 * you're using the driver 2.0.x and Cassandra 2.0.x, and therefore
diff --git a/docs/source/manual/pooling/index.md b/docs/source/manual/pooling/index.md
index ae654439095..4e51d416f00 100644
--- a/docs/source/manual/pooling/index.md
+++ b/docs/source/manual/pooling/index.md
@@ -22,11 +22,11 @@ connection to handle multiple simultaneous requests:
 You don't need to manage connections yourself. You simply interact with a `Session` object, which takes care of it.
 
 **For each `Session`, there is one connection pool per connected host** (a host is connected when it is up and
-not ignored by the [load balancing policy](../load_balancing/index)).
+not ignored by the [load balancing policy](../load_balancing/)).
 
 The number of connections per pool is configurable (this will be
 described in the next section).  The number of stream ids depends on the
-[native protocol version](../native_protocol/index):
+[native protocol version](../native_protocol/):
 
 * protocol v2 or below: 128 stream ids per connection.
 * protocol v3 or above: up to 32768 stream ids per connection.
@@ -198,7 +198,7 @@ Two options control that queue: a maximum size ([PoolingOptions.setMaxQueueSize]
 * otherwise, the attempt is enqueued; if a connection becomes available before `poolTimeoutMillis` has elapsed,
   then the attempt succeeds, otherwise it is rejected.
 
-If the attempt is rejected, the driver will move to the next host in the [query plan](../load_balancing/index#query-plan),
+If the attempt is rejected, the driver will move to the next host in the [query plan](../load_balancing/#query-plan),
 and try to acquire a connection again.
 
 If all hosts are busy with a full queue, the request will fail with a
diff --git a/docs/source/manual/query_timestamps/index.md b/docs/source/manual/query_timestamps/index.md
index 8c553acb5df..dab15c56add 100644
--- a/docs/source/manual/query_timestamps/index.md
+++ b/docs/source/manual/query_timestamps/index.md
@@ -17,7 +17,7 @@ session.execute("INSERT INTO my_table(c1, c2) values (1, 1) " +
 ### Client-side generation
 
 This is enabled by default if you're using the driver 3.0+ and a version
-of Cassandra that supports [native protocol](../native_protocol/index) v3 or
+of Cassandra that supports [native protocol](../native_protocol/) v3 or
 above.
 
 The driver will use a [TimestampGenerator] to generate a timestamp for each query, and send it
@@ -60,7 +60,7 @@ property `com.datastax.driver.USE_NATIVE_CLOCK` is explicitly set to `false`.
 To check what's available on your system:
 
 * make sure your `Cluster` uses a `TimestampGenerator`;
-* [configure your logging framework](../logging/index) to use level `INFO` for the category
+* [configure your logging framework](../logging/) to use level `INFO` for the category
   `com.datastax.driver.core.ClockFactory`;
 * look for one of the following messages at startup:
 
diff --git a/docs/source/manual/reconnection/index.md b/docs/source/manual/reconnection/index.md
index 4b149b69119..f99be3ef99f 100644
--- a/docs/source/manual/reconnection/index.md
+++ b/docs/source/manual/reconnection/index.md
@@ -3,9 +3,9 @@
 If the driver loses a connection to a node, it tries to re-establish it according to a configurable
 policy. This is used in two places:
 
-* [connection pools](../pooling/index): for each node, a session has a fixed-size pool of connections to
+* [connection pools](../pooling/): for each node, a session has a fixed-size pool of connections to
   execute user requests. If a node is detected as down, a reconnection is started.
-* [control connection](../control_connection/index): a session uses a single connection to an arbitrary
+* [control connection](../control_connection/): a session uses a single connection to an arbitrary
   node for administrative requests. If that connection goes down, a reconnection gets started; each
   attempt iterates through all active nodes until one of them accepts a connection. This goes on
   until we have a control node again.
diff --git a/docs/source/manual/retries/index.md b/docs/source/manual/retries/index.md
index 48c1f3676fd..2a9917ab665 100644
--- a/docs/source/manual/retries/index.md
+++ b/docs/source/manual/retries/index.md
@@ -174,9 +174,9 @@ implementations to handle idempotence (the new behavior is equivalent to what yo
 [AlreadyExistsException]:               https://docs.datastax.com/en/drivers/java/3.10/com/datastax/driver/core/exceptions/AlreadyExistsException.html
 [TruncateException]:                    https://docs.datastax.com/en/drivers/java/3.10/com/datastax/driver/core/exceptions/TruncateException.html
 
-[query plan]: ../load_balancing/index#query-plan
+[query plan]: ../load_balancing/#query-plan
 [connection pool]: ../pooling/index
-[prepared]: ../statements/prepared/index#preparing-on-multiple-nodes/index
-[driver read timeout]: ../socket_options/index#driver-read-timeout/index
+[prepared]: ../statements/prepared/#preparing-on-multiple-nodes/index
+[driver read timeout]: ../socket_options/#driver-read-timeout/index
 [hinted handoffs]: https://docs.datastax.com/en/cassandra/2.1/cassandra/dml/dml_about_hh_c.html?scroll=concept_ds_ifg_jqx_zj__performance
 [idempotence]: ../idempotence/index
\ No newline at end of file
diff --git a/docs/source/manual/shaded_jar/index.md b/docs/source/manual/shaded_jar/index.md
index 63a50d2ac2b..d3bd52ff360 100644
--- a/docs/source/manual/shaded_jar/index.md
+++ b/docs/source/manual/shaded_jar/index.md
@@ -1,6 +1,6 @@
 ## Using the shaded JAR
 
-The default driver JAR depends on [Netty](http://netty.io/index), which is
+The default driver JAR depends on [Netty](http://netty.io/), which is
 used internally for networking.
 
 This explicit dependency can be a problem if your application already
diff --git a/docs/source/manual/socket_options/index.md b/docs/source/manual/socket_options/index.md
index f71e24f1ec4..3edd9a74718 100644
--- a/docs/source/manual/socket_options/index.md
+++ b/docs/source/manual/socket_options/index.md
@@ -71,8 +71,8 @@ session.execute(
 Do not set the read timeout too low, or the driver might give up on requests that had a chance of succeeding.
 
 If the timeout is reached, the driver will receive an [OperationTimedOutException], and invoke [onRequestError] on the
-[retry policy](../retries/index) to decide what to do (the default is to retry on the next node in the
-[query plan](../load_balancing/index#query-plan)).
+[retry policy](../retries/) to decide what to do (the default is to retry on the next node in the
+[query plan](../load_balancing/#query-plan)).
 
 #### Limiting overall query time
 
@@ -97,22 +97,22 @@ ResultSet execute(Statement statement, long timeout, TimeUnit unit)
 }
 ```
 
-A complementary approach is to enable [speculative executions](../speculative_execution/index), to have the driver query
+A complementary approach is to enable [speculative executions](../speculative_execution/), to have the driver query
 multiple nodes in parallel. This way you won't have to wait for the full timeout if the first node is unresponsive.
 
 #### Driver read timeout vs. server read timeout
 
 Unfortunately, the term "read timeout" clashes with another concept that is not directly related: a Cassandra node may
-reply with a [READ_TIMEOUT](../retries/index#on-read-timeout) error when it didn't hear back from enough replicas during a
+reply with a [READ_TIMEOUT](../retries/#on-read-timeout) error when it didn't hear back from enough replicas during a
 read query.
 
 To clarify:
 
 * **driver read timeout:** the driver did not receive any response from the current coordinator within
-  `SocketOptions.setReadTimeoutMillis`. It invokes [onRequestError] on the [retry policy](../retries/index) with an
+  `SocketOptions.setReadTimeoutMillis`. It invokes [onRequestError] on the [retry policy](../retries/) with an
   [OperationTimedOutException] to decide what to do.
 * **server read timeout:** the driver *did* receive a response, but that response indicates that the coordinator timed
-  out while waiting for other replicas. It invokes [onReadTimeout] on the [retry policy](../retries/index) to decide what to
+  out while waiting for other replicas. It invokes [onReadTimeout] on the [retry policy](../retries/) to decide what to
   do.
 
 We might rename `SocketOptions.setReadTimeoutMillis` in a future version to clear up any confusion.
diff --git a/docs/source/manual/speculative_execution/index.md b/docs/source/manual/speculative_execution/index.md
index d7f321c0c2c..3ef65f60242 100644
--- a/docs/source/manual/speculative_execution/index.md
+++ b/docs/source/manual/speculative_execution/index.md
@@ -63,7 +63,7 @@ sections cover the practical details and how to enable them.
 
 ### Query idempotence
 
-If a query is [not idempotent](../idempotence/index), the driver will never schedule speculative executions for it, because
+If a query is [not idempotent](../idempotence/), the driver will never schedule speculative executions for it, because
 there is no way to guarantee that only one node will apply the mutation.
 
 ### Enabling speculative executions
@@ -157,7 +157,7 @@ Cluster cluster = Cluster.builder()
 ```
 
 Note that `PercentileTracker` may also be used with a slow query
-logger (see the [Logging](../logging/index) section). In that case, you would
+logger (see the [Logging](../logging/) section). In that case, you would
 create a single tracker object and share it with both components.
 
 [PercentileSpeculativeExecutionPolicy]: https://docs.datastax.com/en/drivers/java/3.10/com/datastax/driver/core/policies/PercentileSpeculativeExecutionPolicy.html
@@ -173,7 +173,7 @@ As with all policies, you are free to provide your own by implementing
 
 ### How speculative executions affect retries
 
-Turning speculative executions on doesn't change the driver's [retry](../retries/index) behavior. Each
+Turning speculative executions on doesn't change the driver's [retry](../retries/) behavior. Each
 parallel execution will trigger retries independently:
 
 ```ditaa
@@ -286,5 +286,5 @@ The workaround is to use a timestamp with your queries:
     insert into my_table (k, v) values (1, 1) USING TIMESTAMP 1432764000;
 
 If you're using native protocol v3, you can also enable [client-side
-timestamps](../query_timestamps/index#client-side-generation) to have this done
+timestamps](../query_timestamps/#client-side-generation) to have this done
 automatically.
diff --git a/docs/source/manual/statements/built/index.md b/docs/source/manual/statements/built/index.md
index f376636ce92..d54f6e30faa 100644
--- a/docs/source/manual/statements/built/index.md
+++ b/docs/source/manual/statements/built/index.md
@@ -10,7 +10,7 @@ invalid queries.
 Queries built with `QueryBuilder` are executed the same way as other queries--via
 `execute` or `executeAsync`.  When a query is built with inlined values, then it doesn't
 differ much from a statement specified as a string.  But it's also possible to build
-the query with bind markers inside it, and then convert it into a [prepared statement](../prepared/index).
+the query with bind markers inside it, and then convert it into a [prepared statement](../prepared/).
 
 ### Basics
 
@@ -24,7 +24,7 @@ a [TableMetadata] object.
 
 Note: The `QueryBuilder` doesn't provide support for the full set of CQL.  For
 most of DDL operations (`CREATE TABLE`, etc.) you can use the [SchemaBuilder].  To perform other
-operations, for example, for role management, you still need to use [simple statements](../simple/index).
+operations, for example, for role management, you still need to use [simple statements](../simple/).
 
 ### Selecting data
 
@@ -238,7 +238,7 @@ conditional deletes with `ifExists` & `onlyIf`.
 ### Prepared statements
 
 If you're repeating the same operation very often, the more effective way will be to
-create a [prepared statement](../prepared/index) from the `BuiltStatement`. To do this, instead
+create a [prepared statement](../prepared/) from the `BuiltStatement`. To do this, instead
 of the real values, use bind markers created either by calling `bindMarker` (which generates
 a positional placeholder), or by calling `bindMarker("name")` (which creates a named
 placeholder).  After the statement is generated, just prepare it as usual, then bind, and
diff --git a/docs/source/manual/statements/index.md b/docs/source/manual/statements/index.md
index 7d453d49060..03705d49f27 100644
--- a/docs/source/manual/statements/index.md
+++ b/docs/source/manual/statements/index.md
@@ -3,15 +3,15 @@
 To execute a query, you  create a [Statement] instance and pass it to [Session#execute()][execute] or
 [Session#executeAsync][executeAsync]. The driver provides various implementations:
 
-* [SimpleStatement](simple/index): a simple implementation built directly from a
+* [SimpleStatement](simple/): a simple implementation built directly from a
   character string. Typically used for queries that are executed only
   once or a few times.
-* [BoundStatement](prepared/index): obtained by binding values to a prepared
+* [BoundStatement](prepared/): obtained by binding values to a prepared
   statement. Typically used for queries that are executed
   often, with different values.
-* [BuiltStatement](built/index): a statement built with the [QueryBuilder] DSL. It
+* [BuiltStatement](built/): a statement built with the [QueryBuilder] DSL. It
   can be executed directly like a simple statement, or prepared.
-* [BatchStatement](batch/index): a statement that groups multiple statements to be
+* [BatchStatement](batch/): a statement that groups multiple statements to be
   executed as a batch.
 
 
diff --git a/docs/source/manual/statements/prepared/index.md b/docs/source/manual/statements/prepared/index.md
index 6bba541e3c9..fd53848bc59 100644
--- a/docs/source/manual/statements/prepared/index.md
+++ b/docs/source/manual/statements/prepared/index.md
@@ -62,12 +62,12 @@ You should prepare only once, and cache the `PreparedStatement` in your applicat
 `prepare` multiple times with the same query string, the driver will log a warning.
 
 If you execute a query only once, a prepared statement is inefficient because it requires two roundtrips. Consider a
-[simple statement](../simple/index) instead.
+[simple statement](../simple/) instead.
 
 ### Parameters and binding
 
 Parameters can be either anonymous or named (named parameters are only
-available with [native protocol](../../native_protocol/index) v2 or above):
+available with [native protocol](../../native_protocol/) v2 or above):
 
 ```java
 ps1 = session.prepare("insert into product (sku, description) values (?, ?)");
diff --git a/docs/source/manual/statements/simple/index.md b/docs/source/manual/statements/simple/index.md
index b35f8e1f811..1144ba64c83 100644
--- a/docs/source/manual/statements/simple/index.md
+++ b/docs/source/manual/statements/simple/index.md
@@ -37,7 +37,7 @@ client                             driver                Cassandra
 ```
 
 If you execute the same query often (or a similar query with different column values), consider a
-[prepared statement](../prepared/index) instead.
+[prepared statement](../prepared/) instead.
 
 
 ### Using values
@@ -85,7 +85,7 @@ session.execute(
 ### Value type inference
 
 Another consequence of not parsing query strings is that the driver has to make a guess on how to serialize values,
-based on their Java type (see the [default type mappings](../../index#cql-to-java-type-mapping)). This can be tricky, in
+based on their Java type (see the [default type mappings](../../#cql-to-java-type-mapping)). This can be tricky, in
 particular for numeric types:
 
 ```java
diff --git a/docs/source/manual/udts/index.md b/docs/source/manual/udts/index.md
index e7a629acf4a..f2f4d514496 100644
--- a/docs/source/manual/udts/index.md
+++ b/docs/source/manual/udts/index.md
@@ -58,7 +58,7 @@ various ways to get it:
     UserType udt = (UserType) ps.getVariables().getType("v");
     ```
 
-* from the driver's [schema metadata](../metadata/index#schema-metadata):
+* from the driver's [schema metadata](../metadata/#schema-metadata):
 
     ```java
     UserType udt = session.getCluster().getMetadata().getKeyspace("ks").getUserType("type1");
diff --git a/docs/source/upgrade_guide/index.md b/docs/source/upgrade_guide/index.md
index 87f8137ce36..63023bab816 100644
--- a/docs/source/upgrade_guide/index.md
+++ b/docs/source/upgrade_guide/index.md
@@ -125,7 +125,7 @@ queries that were retried before).
 
 In 3.1.0, the default is now to **not retry** after a write timeout or request error if the statement is not idempotent.
 This is handled internally, the retry policy methods are not even invoked in those cases (and therefore
-`IdempotenceAwareRetryPolicy` has been deprecated). See the manual section about [retries](../manual/retries/index) for more
+`IdempotenceAwareRetryPolicy` has been deprecated). See the manual section about [retries](../manual/retries/) for more
 information.
 
 In practice, here's what upgrading to 3.1.0 means for you:
@@ -135,7 +135,7 @@ In practice, here's what upgrading to 3.1.0 means for you:
 * otherwise, you might want to review how your code positions the `setIdempotent` flag on statements. In most cases the
   driver can't compute in automatically (because it doesn't parse query strings), so it takes a conservative approach
   and sets it to `false` by default. If you know the query is idempotent, you should set it to `true` manually. See the
-  [query idempotence](../manual/idempotence/index) section of the manual.
+  [query idempotence](../manual/idempotence/) section of the manual.
 
 The driver logs a warning the first time it ignores a non-idempotent request; this warning will be removed in version
 3.2.0.
@@ -166,7 +166,7 @@ The main changes were introduced by the custom codecs feature (see below).
 We've also seized the opportunity to remove code that was deprecated in 2.1.
 
 1.  The default consistency level in `QueryOptions` is now `LOCAL_ONE`.
-2.  [Custom codecs](../manual/custom_codecs/index)
+2.  [Custom codecs](../manual/custom_codecs/)
     ([JAVA-721](https://datastax-oss.atlassian.net/browse/JAVA-721))
     introduce several breaking changes and also modify a few runtime behaviors.
 
@@ -230,7 +230,7 @@ We've also seized the opportunity to remove code that was deprecated in 2.1.
     corresponding method `MappingManager#udtMapper`.
 
     The mapper now uses custom codecs to convert UDTs. See more
-    explanations [here](../manual/object_mapper/custom_codecs/index#implicit-udt-codecs).
+    explanations [here](../manual/object_mapper/custom_codecs/#implicit-udt-codecs).
 
 5.  All methods that took the protocol version as an `int` or assumed a
     default version have been removed (they were already deprecated in
@@ -289,9 +289,9 @@ We've also seized the opportunity to remove code that was deprecated in 2.1.
     callback methods (`onUp`, `onDown`, etc.) have been duplicated. This
     is unlikely to affect clients.
 
-12. [Client-side timestamp generation](../manual/query_timestamps/index) is
+12. [Client-side timestamp generation](../manual/query_timestamps/) is
     now the default (provided that [native
-    protocol](../manual/native_protocol/index) v3 or higher is in use). The
+    protocol](../manual/native_protocol/) v3 or higher is in use). The
     generator used is `AtomicMonotonicTimestampGenerator`.
 
 13. If a DNS name resolves to multiple A-records,
@@ -300,7 +300,7 @@ We've also seized the opportunity to remove code that was deprecated in 2.1.
     maintaining contact points in DNS configuration, and having a single,
     static contact point in your Java code.
 
-14. The following methods were added for [Custom payloads](../manual/custom_payloads/index):
+14. The following methods were added for [Custom payloads](../manual/custom_payloads/):
     * in `PreparedStatement`: `getIncomingPayload()`,
       `getOutgoingPayload()` and
       `setOutgoingPayload(Map)`
@@ -425,7 +425,7 @@ We've also seized the opportunity to remove code that was deprecated in 2.1.
 
 29. `SSLOptions` has been refactored to allow the option to choose between JDK and Netty-based
     SSL implementations.  See [JAVA-841](https://datastax-oss.atlassian.net/browse/JAVA-841) and
-    the [SSL documentation](../manual/ssl/index) for more details.
+    the [SSL documentation](../manual/ssl/) for more details.
 
 
 ### 2.1.8
@@ -463,7 +463,7 @@ binary compatibility.
     ones.
 
     Also, note that the connection pool for protocol v3 can now be configured to
-    use multiple connections. See [this page](../manual/pooling/index) for more
+    use multiple connections. See [this page](../manual/pooling/) for more
     information.
 
 3. `MappingManager(Session)` will now force the initialization of the `Session`
@@ -632,7 +632,7 @@ The following might also be of interest:
    default like in previous versions.
 
 3. Netty is not shaded anymore in the default Maven artifact. However we publish a
-   [shaded artifact](../manual/shaded_jar/index) under a different classifier.
+   [shaded artifact](../manual/shaded_jar/) under a different classifier.
 
 4. The internal initialization sequence of the Cluster object has been slightly changed:
    some fields that were previously initialized in the constructor are now set when
@@ -687,7 +687,7 @@ you have trouble compiling your application after an upgrade.
 
 5. The `Metrics` class now uses the Codahale metrics library version 3 (version 2 was
    used previously). This new major version of the library has many API changes
-   compared to its version 2 (see the [release notes](https://dropwizard.github.io/metrics/3.1.0/about/release-notes/index) for details),
+   compared to its version 2 (see the [release notes](https://dropwizard.github.io/metrics/3.1.0/about/release-notes/) for details),
    which can thus impact consumers of the Metrics class.
    Furthermore, the default `JmxReporter` now includes a name specific to the
    cluster instance (to avoid conflicts when multiple Cluster instances are created
diff --git a/docs/source/upgrade_guide/migrating_from_astyanax/configuration/index.md b/docs/source/upgrade_guide/migrating_from_astyanax/configuration/index.md
index dc1edbc4b22..0c9eba1c2a5 100644
--- a/docs/source/upgrade_guide/migrating_from_astyanax/configuration/index.md
+++ b/docs/source/upgrade_guide/migrating_from_astyanax/configuration/index.md
@@ -33,7 +33,7 @@ that the *Java driver* has categorized in multiple *Option* and *Policy* kinds.
 
 ### Connections pools internals
 Everything concerning the internal pools of connections to the *Cassandra nodes*
-will be gathered in the Java driver in the [`PoolingOptions`](../../../manual/pooling/index):
+will be gathered in the Java driver in the [`PoolingOptions`](../../../manual/pooling/):
 
 *Astyanax*:
 
@@ -55,11 +55,11 @@ The first number is the initial number of connections, the second is the maximum
 of connections the driver is allowed to create for each host.
 
 Note that the *Java driver* allows multiple simultaneous requests on one single
-connection, as it is based upon the [*Native protocol*](../../../manual/native_protocol/index),
+connection, as it is based upon the [*Native protocol*](../../../manual/native_protocol/),
 an asynchronous binary protocol that can handle up to 32768 simultaneous requests on a 
 single connection. The Java driver is able to manage and distribute simultaneous requests
 by itself even under high contention, and changing the default `PoolingOptions` is not
-necessary most of the time except for very [specific use cases](../../../manual/pooling/index#tuning-protocol-v3-for-very-high-throughputs).
+necessary most of the time except for very [specific use cases](../../../manual/pooling/#tuning-protocol-v3-for-very-high-throughputs).
 
 ### Timeouts
 
@@ -136,12 +136,12 @@ query needs to have the *Row key* specified explicitly on a `Statement` in order
 to benefit from the *TokenAware* routing (the *Row key* in the *Java driver* is 
 referenced as *Routing Key*), unlike the *Astyanax* driver. 
 Some differences occur related to the different kinds of `Statements` the *Java
-driver* provides. Please see [this link](../../../manual/load_balancing/index#token-aware-policy) 
+driver* provides. Please see [this link](../../../manual/load_balancing/#token-aware-policy) 
 for specific information.
 
 Custom load balancing policies can easily be implemented by users, and supplied to 
 the *Java driver* for specific use cases. All information necessary is available
-in the [Load balaning policies docs](../../../manual/load_balancing/index).
+in the [Load balaning policies docs](../../../manual/load_balancing/).
 
 ## Consistency levels
 Consistency levels can be set per-statement, or globally through the `QueryOptions`.
@@ -169,7 +169,7 @@ Level can be set per-statement, you can either set it on every statement, or use
 this case, setting the CL on the `PreparedStatement`, causes the `BoundStatements` to 
 inherit the CL from the prepared statements they were prepared from. More
 informations about how `Statement`s work in the *Java driver* are detailed
-in the [“Queries and Results” section](../queries_and_results/index).
+in the [“Queries and Results” section](../queries_and_results/).
 
 
 ## Authentication
@@ -193,7 +193,7 @@ AuthProvider authProvider = new PlainTextAuthProvider("username", "password");
 ```
 
 The class `AuthProvider` can be easily implemented to suit the user’s needs,
-documentation about the classes needed is [available there](../../../manual/auth/index).
+documentation about the classes needed is [available there](../../../manual/auth/).
 
 ## Hosts and ports
 
@@ -247,8 +247,8 @@ A few best practices are summed up in [this blog post](http://www.datastax.com/d
 Concerning connection pools, the Java driver’s default settings should allow 
 most of the users to get the best out of the driver in terms of throughput, 
 they have been thoroughly tested and tweaked to accommodate the users’ needs. 
-If one still wishes to change those, first [Monitoring the pools](../../../manual/pooling/index#monitoring-and-tuning-the-pool) is
-advised, then a [deep dive in the Pools management mechanism](../../../manual/pooling/index) should
+If one still wishes to change those, first [Monitoring the pools](../../../manual/pooling/#monitoring-and-tuning-the-pool) is
+advised, then a [deep dive in the Pools management mechanism](../../../manual/pooling/) should
 provide enough insight.
 
 A lot more options are available in the different `XxxxOption`s classes, policies are
diff --git a/docs/source/upgrade_guide/migrating_from_astyanax/index.md b/docs/source/upgrade_guide/migrating_from_astyanax/index.md
index 9894394d313..b3f6fc14846 100644
--- a/docs/source/upgrade_guide/migrating_from_astyanax/index.md
+++ b/docs/source/upgrade_guide/migrating_from_astyanax/index.md
@@ -5,9 +5,9 @@ migrating to the *DataStax Java driver*.
 
 See the child pages for more information:
 
-* [Changes at the language level](language_level_changes/index)
-* [Migrating Astyanax configurations to DataStax Java driver configurations](configuration/index)
-* [Querying and retrieving results comparisons.](queries_and_results/index)
+* [Changes at the language level](language_level_changes/)
+* [Migrating Astyanax configurations to DataStax Java driver configurations](configuration/)
+* [Querying and retrieving results comparisons.](queries_and_results/)
 
 ```eval_rst
 .. toctree::
diff --git a/docs/source/upgrade_guide/migrating_from_astyanax/language_level_changes/index.md b/docs/source/upgrade_guide/migrating_from_astyanax/language_level_changes/index.md
index 9dc332c5a06..0c6a7a91b2d 100644
--- a/docs/source/upgrade_guide/migrating_from_astyanax/language_level_changes/index.md
+++ b/docs/source/upgrade_guide/migrating_from_astyanax/language_level_changes/index.md
@@ -74,6 +74,6 @@ composing the *Clustering Column*, but there can be others if the Thrift's Colum
 is a CompositeType).
 
 Here is the basic architectural concept of *CQL*, a detailed explanation and *CQL*
-examples can be found in this article : [http://www.planetcassandra.org/making-the-change-from-thrift-to-cql/](http://www.planetcassandra.org/making-the-change-from-thrift-to-cql/index).
+examples can be found in this article : [http://www.planetcassandra.org/making-the-change-from-thrift-to-cql/](http://www.planetcassandra.org/making-the-change-from-thrift-to-cql/).
 Understanding the *CQL* abstraction plays a key role in developing performing
 and scaling applications.
diff --git a/docs/source/upgrade_guide/migrating_from_astyanax/queries_and_results/index.md b/docs/source/upgrade_guide/migrating_from_astyanax/queries_and_results/index.md
index ee6fb834685..ff9d7998476 100644
--- a/docs/source/upgrade_guide/migrating_from_astyanax/queries_and_results/index.md
+++ b/docs/source/upgrade_guide/migrating_from_astyanax/queries_and_results/index.md
@@ -10,10 +10,10 @@ The *Java driver* executes CQL queries through the `Session`.
 The queries can either be simple *CQL* Strings or represented in the form of 
 `Statement`s. The driver offers 4 kinds of statements, `SimpleStatement`, 
 `Prepared/BoundStatement`, `BuiltStatement`, and `BatchStatement`. All necessary
-information can be [found here](../../../manual/statements/index) about the nature of the different
+information can be [found here](../../../manual/statements/) about the nature of the different
 `Statement`s.
 
-As explained in [the running section](../../../manual/index#running-queries),
+As explained in [the running section](../../../manual/#running-queries),
 results of a *CQL* query will be in the form of *Rows* from *Tables*, composed 
 of fixed set of columns, each with a type and a name. The driver exposes the 
 set of *Rows* returned from a query as a ResultSet, thus containing *Rows* on 
@@ -101,8 +101,8 @@ for (Row row : rs) {
 
 ## Async
 The *Java driver* provides native support for asynchronous programming since it 
-is built on top of an [asynchronous protocol](../../../manual/native_protocol/index),
-please see [this page](../../../manual/async/index) for best practices regarding asynchronous programming
+is built on top of an [asynchronous protocol](../../../manual/native_protocol/),
+please see [this page](../../../manual/async/) for best practices regarding asynchronous programming
 with the *Java driver*.
 
 

From 7d2b859223feef2c774b73fd4b1e4ca7935b667a Mon Sep 17 00:00:00 2001
From: dgarcia360 
Date: Thu, 10 Sep 2020 17:29:55 +0200
Subject: [PATCH 41/52] Overwrite md relative link parser

---
 docs/source/conf.py                           | 44 +++++++++++++------
 docs/source/faq/index.md                      |  4 +-
 .../source/manual/address_resolution/index.md |  2 +-
 .../source/manual/control_connection/index.md |  2 +-
 docs/source/manual/index.md                   |  2 +-
 docs/source/manual/load_balancing/index.md    |  2 +-
 .../manual/object_mapper/creating/index.md    |  2 +-
 docs/source/manual/pooling/index.md           |  2 +-
 docs/source/manual/query_timestamps/index.md  |  2 +-
 docs/source/manual/retries/index.md           |  8 ++--
 .../manual/statements/prepared/index.md       |  2 +-
 docs/source/upgrade_guide/index.md            |  8 ++--
 .../configuration/index.md                    |  6 +--
 .../migrating_from_astyanax/index.md          |  1 -
 driver-tests/osgi/pom.xml                     |  1 +
 15 files changed, 53 insertions(+), 35 deletions(-)

diff --git a/docs/source/conf.py b/docs/source/conf.py
index 1854fce4aec..8a6982b3137 100644
--- a/docs/source/conf.py
+++ b/docs/source/conf.py
@@ -22,8 +22,8 @@
 import re
 from docutils import nodes
 from sphinx.util import logging
-import recommonmark
 from recommonmark.transform import AutoStructify
+from recommonmark.parser import CommonMarkParser, splitext, urlparse
 
 logger = logging.getLogger(__name__)
 
@@ -66,24 +66,42 @@ def create_redirects(app, docname):
     'sphinx.ext.autosectionlabel',
     'sphinx_scylladb_theme',
     'sphinx_multiversion',
-    'recommonmark'
 ]
 
 # Add Markdown support
-source_suffix = ['.rst', '.md']
+source_suffix = {
+    '.rst': 'restructuredtext',
+    '.md': 'markdown',
+}
 autosectionlabel_prefix_document = True
 
-def replace_relative_links(app, docname, source):
-    result = source[0]
-    for key in app.config.replacements:
-        result = result.replace(key, app.config.replacements[key])
-    source[0] = result
+class CustomCommonMarkParser(CommonMarkParser):
+    
+    def visit_document(self, node):
+        pass
+    
+    def visit_link(self, mdnode):
+        # Override to avoid checking if relative links exists
+        ref_node = nodes.reference()
+        destination = mdnode.destination
+        _, ext = splitext(destination)
+
+        url_check = urlparse(destination)
+        scheme_known = bool(url_check.scheme)
+
+        if not scheme_known and ext.replace('.', '') in self.supported:
+            destination = destination.replace(ext, '')
+        ref_node['refuri'] = destination
+        ref_node.line = self._get_line(mdnode)
+        if mdnode.title:
+            ref_node['title'] = mdnode.title
+        next_node = ref_node
+
+        self.current_node.append(next_node)
+        self.current_node = ref_node
 
 def setup(app):
-    replacements = {"/)" : "/index)", "/#": "/index#"}
-    app.add_config_value('replacements', replacements, True)
-    app.connect('source-read', replace_relative_links)
-
+    app.add_source_parser(CustomCommonMarkParser)
     app.add_config_value('recommonmark_config', {
         'enable_eval_rst': True,
         'enable_auto_toc_tree': False,
@@ -238,7 +256,7 @@ def setup(app):
     'monitor_lst': ('/operating-scylla/monitoring/3.1/%s/','')
 }
 
-#Adds version variables for monitoring and manager versions when used in inline text
+# Adds version variables for monitoring and manager versions when used in inline text
 
 rst_epilog = """
 .. |mon_version| replace:: 3.1
diff --git a/docs/source/faq/index.md b/docs/source/faq/index.md
index 843ba66aa12..6a442c6d797 100644
--- a/docs/source/faq/index.md
+++ b/docs/source/faq/index.md
@@ -69,7 +69,7 @@ the way that the parameters are sent. The difference is that a prepared statemen
 * sends only the statement id and its parameters (thus reducing the amount of data sent
   to the cluster)
 
-See [Prepared statements](../manual/statements/prepared/) for more information.
+See [Prepared statements](../manual/statements/prepared) for more information.
 
 
 ### Can I combine `PreparedStatements` and normal statements in a batch?
@@ -298,4 +298,4 @@ an update with compatibility to this driver version.
 [Futures.allAsList]: https://google.github.io/guava/releases/19.0/api/docs/com/google/common/util/concurrent/Futures.html#allAsList(java.lang.Iterable)
 [DynaTrace]: https://www.dynatrace.com/
 [packaging data]: https://logback.qos.ch/reasonsToSwitch.html#packagingData
-[Using the shaded JAR]: ../manual/shaded_jar/index
+[Using the shaded JAR]: ../manual/shaded_jar
diff --git a/docs/source/manual/address_resolution/index.md b/docs/source/manual/address_resolution/index.md
index 635d079d0c3..8f0ffd33740 100644
--- a/docs/source/manual/address_resolution/index.md
+++ b/docs/source/manual/address_resolution/index.md
@@ -5,7 +5,7 @@ connections.
 
 * for contact points, these are provided as part of configuring the `Cluster` object;
 * for other nodes, addresses will be discovered dynamically, either by inspecting `system.peers` on already connected
-  nodes, or via push notifications received from the [control host](../control_connection/) when new nodes are discovered
+  nodes, or via push notifications received from the [control host](../control_connection) when new nodes are discovered
   by gossip.
 
 
diff --git a/docs/source/manual/control_connection/index.md b/docs/source/manual/control_connection/index.md
index 60aad531528..eb610627941 100644
--- a/docs/source/manual/control_connection/index.md
+++ b/docs/source/manual/control_connection/index.md
@@ -15,7 +15,7 @@ by the same policy as regular connections and tries the nodes according to a que
 
 The control connection is managed independently from [regular pooled connections](../pooling/), and
 used exclusively for administrative requests. It is included in [Session.State.getOpenConnections],
-as well as the `open-connections` [metric](../metrics/); for example, if you've configured a pool
+as well as the `open-connections` [metric](../metrics); for example, if you've configured a pool
 size of 2, the control node will have 3 connections.
 
 [Session.State.getOpenConnections]: https://docs.datastax.com/en/drivers/java/3.10/com/datastax/driver/core/Session.State.html#getOpenConnections-com.datastax.driver.core.Host-
\ No newline at end of file
diff --git a/docs/source/manual/index.md b/docs/source/manual/index.md
index 17d4544b5b3..c2f72356a08 100644
--- a/docs/source/manual/index.md
+++ b/docs/source/manual/index.md
@@ -286,7 +286,7 @@ Besides explicit work with queries and rows, you can also use
 ### More information
 
 If you're reading this from the [generated HTML documentation on
-github.io](http://datastax.github.io/java-driver), use the "Contents"
+github.io](http://datastax.github.io/java-driver/), use the "Contents"
 menu on the left hand side to navigate sub-sections. If you're [browsing the source files on
 github.com](https://github.com/datastax/java-driver/tree/3.x/manual),
 simply navigate to each sub-directory.
diff --git a/docs/source/manual/load_balancing/index.md b/docs/source/manual/load_balancing/index.md
index 686277a9bcf..dad615adeb8 100644
--- a/docs/source/manual/load_balancing/index.md
+++ b/docs/source/manual/load_balancing/index.md
@@ -58,7 +58,7 @@ cluster.getConfiguration().getPoolingOptions().refreshConnectedHost(host);
 
 Each time the driver executes a query, it asks the policy to compute a **query plan**, which is a list of hosts. The
 driver will then try each host in sequence, according to the [retry policy](../retries/) and
-[speculative execution policy](../speculative_execution/).
+[speculative execution policy](../speculative_execution).
 
 The contents and order of query plans are entirely up to your policy, but implementations typically return plans that:
 
diff --git a/docs/source/manual/object_mapper/creating/index.md b/docs/source/manual/object_mapper/creating/index.md
index 9ff0aebd19f..e3fa32b4cba 100644
--- a/docs/source/manual/object_mapper/creating/index.md
+++ b/docs/source/manual/object_mapper/creating/index.md
@@ -321,7 +321,7 @@ public class Company {
 This also works with UDTs inside collections or other UDTs, with any arbitrary
 nesting level.
 
-[User Defined Types]: ../../udts/index
+[User Defined Types]: ../../udts
 [udt]:https://docs.datastax.com/en/drivers/java/3.10/com/datastax/driver/mapping/annotations/UDT.html
 [field]:https://docs.datastax.com/en/drivers/java/3.10/com/datastax/driver/mapping/annotations/Field.html
 
diff --git a/docs/source/manual/pooling/index.md b/docs/source/manual/pooling/index.md
index 4e51d416f00..ab13653b481 100644
--- a/docs/source/manual/pooling/index.md
+++ b/docs/source/manual/pooling/index.md
@@ -22,7 +22,7 @@ connection to handle multiple simultaneous requests:
 You don't need to manage connections yourself. You simply interact with a `Session` object, which takes care of it.
 
 **For each `Session`, there is one connection pool per connected host** (a host is connected when it is up and
-not ignored by the [load balancing policy](../load_balancing/)).
+not ignored by the [load balancing policy](../load_balancing)).
 
 The number of connections per pool is configurable (this will be
 described in the next section).  The number of stream ids depends on the
diff --git a/docs/source/manual/query_timestamps/index.md b/docs/source/manual/query_timestamps/index.md
index dab15c56add..20f4e9c6a47 100644
--- a/docs/source/manual/query_timestamps/index.md
+++ b/docs/source/manual/query_timestamps/index.md
@@ -17,7 +17,7 @@ session.execute("INSERT INTO my_table(c1, c2) values (1, 1) " +
 ### Client-side generation
 
 This is enabled by default if you're using the driver 3.0+ and a version
-of Cassandra that supports [native protocol](../native_protocol/) v3 or
+of Cassandra that supports [native protocol](../native_protocol) v3 or
 above.
 
 The driver will use a [TimestampGenerator] to generate a timestamp for each query, and send it
diff --git a/docs/source/manual/retries/index.md b/docs/source/manual/retries/index.md
index 2a9917ab665..80fbb0f6db5 100644
--- a/docs/source/manual/retries/index.md
+++ b/docs/source/manual/retries/index.md
@@ -175,8 +175,8 @@ implementations to handle idempotence (the new behavior is equivalent to what yo
 [TruncateException]:                    https://docs.datastax.com/en/drivers/java/3.10/com/datastax/driver/core/exceptions/TruncateException.html
 
 [query plan]: ../load_balancing/#query-plan
-[connection pool]: ../pooling/index
-[prepared]: ../statements/prepared/#preparing-on-multiple-nodes/index
-[driver read timeout]: ../socket_options/#driver-read-timeout/index
+[connection pool]: ../pooling
+[prepared]: ../statements/prepared/#preparing-on-multiple-nodes
+[driver read timeout]: ../socket_options/#driver-read-timeout
 [hinted handoffs]: https://docs.datastax.com/en/cassandra/2.1/cassandra/dml/dml_about_hh_c.html?scroll=concept_ds_ifg_jqx_zj__performance
-[idempotence]: ../idempotence/index
\ No newline at end of file
+[idempotence]: ../idempotence
\ No newline at end of file
diff --git a/docs/source/manual/statements/prepared/index.md b/docs/source/manual/statements/prepared/index.md
index fd53848bc59..d9dd7fee278 100644
--- a/docs/source/manual/statements/prepared/index.md
+++ b/docs/source/manual/statements/prepared/index.md
@@ -67,7 +67,7 @@ If you execute a query only once, a prepared statement is inefficient because it
 ### Parameters and binding
 
 Parameters can be either anonymous or named (named parameters are only
-available with [native protocol](../../native_protocol/) v2 or above):
+available with [native protocol](../../native_protocol) v2 or above):
 
 ```java
 ps1 = session.prepare("insert into product (sku, description) values (?, ?)");
diff --git a/docs/source/upgrade_guide/index.md b/docs/source/upgrade_guide/index.md
index 63023bab816..702786fd071 100644
--- a/docs/source/upgrade_guide/index.md
+++ b/docs/source/upgrade_guide/index.md
@@ -291,7 +291,7 @@ We've also seized the opportunity to remove code that was deprecated in 2.1.
 
 12. [Client-side timestamp generation](../manual/query_timestamps/) is
     now the default (provided that [native
-    protocol](../manual/native_protocol/) v3 or higher is in use). The
+    protocol](../manual/native_protocol) v3 or higher is in use). The
     generator used is `AtomicMonotonicTimestampGenerator`.
 
 13. If a DNS name resolves to multiple A-records,
@@ -300,7 +300,7 @@ We've also seized the opportunity to remove code that was deprecated in 2.1.
     maintaining contact points in DNS configuration, and having a single,
     static contact point in your Java code.
 
-14. The following methods were added for [Custom payloads](../manual/custom_payloads/):
+14. The following methods were added for [Custom payloads](../manual/custom_payloads):
     * in `PreparedStatement`: `getIncomingPayload()`,
       `getOutgoingPayload()` and
       `setOutgoingPayload(Map)`
@@ -425,7 +425,7 @@ We've also seized the opportunity to remove code that was deprecated in 2.1.
 
 29. `SSLOptions` has been refactored to allow the option to choose between JDK and Netty-based
     SSL implementations.  See [JAVA-841](https://datastax-oss.atlassian.net/browse/JAVA-841) and
-    the [SSL documentation](../manual/ssl/) for more details.
+    the [SSL documentation](../manual/ssl) for more details.
 
 
 ### 2.1.8
@@ -463,7 +463,7 @@ binary compatibility.
     ones.
 
     Also, note that the connection pool for protocol v3 can now be configured to
-    use multiple connections. See [this page](../manual/pooling/) for more
+    use multiple connections. See [this page](../manual/pooling) for more
     information.
 
 3. `MappingManager(Session)` will now force the initialization of the `Session`
diff --git a/docs/source/upgrade_guide/migrating_from_astyanax/configuration/index.md b/docs/source/upgrade_guide/migrating_from_astyanax/configuration/index.md
index 0c9eba1c2a5..b8abcc95bbd 100644
--- a/docs/source/upgrade_guide/migrating_from_astyanax/configuration/index.md
+++ b/docs/source/upgrade_guide/migrating_from_astyanax/configuration/index.md
@@ -33,7 +33,7 @@ that the *Java driver* has categorized in multiple *Option* and *Policy* kinds.
 
 ### Connections pools internals
 Everything concerning the internal pools of connections to the *Cassandra nodes*
-will be gathered in the Java driver in the [`PoolingOptions`](../../../manual/pooling/):
+will be gathered in the Java driver in the [`PoolingOptions`](../../../manual/pooling):
 
 *Astyanax*:
 
@@ -55,7 +55,7 @@ The first number is the initial number of connections, the second is the maximum
 of connections the driver is allowed to create for each host.
 
 Note that the *Java driver* allows multiple simultaneous requests on one single
-connection, as it is based upon the [*Native protocol*](../../../manual/native_protocol/),
+connection, as it is based upon the [*Native protocol*](../../../manual/native_protocol),
 an asynchronous binary protocol that can handle up to 32768 simultaneous requests on a 
 single connection. The Java driver is able to manage and distribute simultaneous requests
 by itself even under high contention, and changing the default `PoolingOptions` is not
@@ -141,7 +141,7 @@ for specific information.
 
 Custom load balancing policies can easily be implemented by users, and supplied to 
 the *Java driver* for specific use cases. All information necessary is available
-in the [Load balaning policies docs](../../../manual/load_balancing/).
+in the [Load balaning policies docs](../../../manual/load_balancing).
 
 ## Consistency levels
 Consistency levels can be set per-statement, or globally through the `QueryOptions`.
diff --git a/docs/source/upgrade_guide/migrating_from_astyanax/index.md b/docs/source/upgrade_guide/migrating_from_astyanax/index.md
index b3f6fc14846..b683b84dd02 100644
--- a/docs/source/upgrade_guide/migrating_from_astyanax/index.md
+++ b/docs/source/upgrade_guide/migrating_from_astyanax/index.md
@@ -16,5 +16,4 @@ See the child pages for more information:
    configuration/index
    language_level_changes/index
    queries_and_results/index
-
 ```
\ No newline at end of file
diff --git a/driver-tests/osgi/pom.xml b/driver-tests/osgi/pom.xml
index b9dbec3b4f6..bd56f08898b 100644
--- a/driver-tests/osgi/pom.xml
+++ b/driver-tests/osgi/pom.xml
@@ -24,6 +24,7 @@
         scylla-driver-tests-parent
         3.10.1-scylla-1-SNAPSHOT
     
+
     scylla-driver-tests-osgi
     Java Driver for Scylla and Apache Cassandra Tests - OSGi
     A test for the Java Driver in an OSGi container.

From 5341cbaccead5994de30b5c28d0026cae7761df9 Mon Sep 17 00:00:00 2001
From: dgarcia360 
Date: Thu, 10 Sep 2020 17:33:10 +0200
Subject: [PATCH 42/52] Add original markers

---
 docs/source/faq/index.md                           | 4 ++--
 docs/source/manual/object_mapper/creating/index.md | 2 +-
 docs/source/manual/retries/index.md                | 4 ++--
 3 files changed, 5 insertions(+), 5 deletions(-)

diff --git a/docs/source/faq/index.md b/docs/source/faq/index.md
index 6a442c6d797..8ef0dd017b6 100644
--- a/docs/source/faq/index.md
+++ b/docs/source/faq/index.md
@@ -69,7 +69,7 @@ the way that the parameters are sent. The difference is that a prepared statemen
 * sends only the statement id and its parameters (thus reducing the amount of data sent
   to the cluster)
 
-See [Prepared statements](../manual/statements/prepared) for more information.
+See [Prepared statements](../manual/statements/prepared/) for more information.
 
 
 ### Can I combine `PreparedStatements` and normal statements in a batch?
@@ -96,7 +96,7 @@ is currently no mechanism for Cassandra to invalidate the existing metadata.  Be
 the driver is not able to properly react to these changes and will improperly read rows after
 a schema change is made.
 
-See [Prepared statements](../manual/statements/prepared/) for more information.
+See [Prepared statements](../manual/statements/prepared) for more information.
 
 
 ### Can I get the raw bytes of a text column?
diff --git a/docs/source/manual/object_mapper/creating/index.md b/docs/source/manual/object_mapper/creating/index.md
index e3fa32b4cba..dbd5fc5b826 100644
--- a/docs/source/manual/object_mapper/creating/index.md
+++ b/docs/source/manual/object_mapper/creating/index.md
@@ -321,7 +321,7 @@ public class Company {
 This also works with UDTs inside collections or other UDTs, with any arbitrary
 nesting level.
 
-[User Defined Types]: ../../udts
+[User Defined Types]: ../../udts/
 [udt]:https://docs.datastax.com/en/drivers/java/3.10/com/datastax/driver/mapping/annotations/UDT.html
 [field]:https://docs.datastax.com/en/drivers/java/3.10/com/datastax/driver/mapping/annotations/Field.html
 
diff --git a/docs/source/manual/retries/index.md b/docs/source/manual/retries/index.md
index 80fbb0f6db5..231707022c9 100644
--- a/docs/source/manual/retries/index.md
+++ b/docs/source/manual/retries/index.md
@@ -175,8 +175,8 @@ implementations to handle idempotence (the new behavior is equivalent to what yo
 [TruncateException]:                    https://docs.datastax.com/en/drivers/java/3.10/com/datastax/driver/core/exceptions/TruncateException.html
 
 [query plan]: ../load_balancing/#query-plan
-[connection pool]: ../pooling
+[connection pool]: ../pooling/
 [prepared]: ../statements/prepared/#preparing-on-multiple-nodes
 [driver read timeout]: ../socket_options/#driver-read-timeout
 [hinted handoffs]: https://docs.datastax.com/en/cassandra/2.1/cassandra/dml/dml_about_hh_c.html?scroll=concept_ds_ifg_jqx_zj__performance
-[idempotence]: ../idempotence
\ No newline at end of file
+[idempotence]: ../idempotence/
\ No newline at end of file

From f1923257a29e2129dc683e646dce7b6ddf9f6d95 Mon Sep 17 00:00:00 2001
From: dgarcia360 
Date: Mon, 14 Sep 2020 14:47:22 +0200
Subject: [PATCH 43/52] Keep original folder structure

---
 {docs/source/changelog => changelog}/index.md |   0
 docs/source/changelog                         |   1 +
 docs/source/faq                               |   1 +
 docs/source/index.rst                         |  32 ++--
 .../index.md => installation.md}              |   4 +-
 docs/source/manual                            |   1 +
 docs/source/upgrade_guide                     |   1 +
 driver-examples/index.md                      |  10 ++
 driver-tests/osgi/index.md                    |  51 ++++++
 driver-tests/stress/index.md                  |  25 +++
 {docs/source/faq => faq}/index.md             |   0
 {docs/source/faq => faq}/osgi/index.md        |   0
 index.md                                      | 148 ++++++++++++++++++
 .../address_resolution/index.md               |   0
 {docs/source/manual => manual}/async/index.md |   0
 {docs/source/manual => manual}/auth/index.md  |   0
 {docs/source/manual => manual}/cloud/index.md |   0
 .../manual => manual}/compression/index.md    |   0
 .../control_connection/index.md               |   0
 .../custom_codecs/extras/index.md             |   0
 .../manual => manual}/custom_codecs/index.md  |   0
 .../custom_payloads/index.md                  |   0
 .../manual => manual}/idempotence/index.md    |   0
 {docs/source/manual => manual}/index.md       |   0
 .../manual => manual}/load_balancing/index.md |   0
 .../source/manual => manual}/logging/index.md |   0
 .../manual => manual}/metadata/index.md       |   0
 .../source/manual => manual}/metrics/index.md |   0
 .../native_protocol/index.md                  |   0
 .../object_mapper/creating/index.md           |   0
 .../object_mapper/custom_codecs/index.md      |   0
 .../manual => manual}/object_mapper/index.md  |   0
 .../object_mapper/using/index.md              |   0
 {docs/source/manual => manual}/osgi/index.md  |   0
 .../source/manual => manual}/paging/index.md  |   0
 .../source/manual => manual}/pooling/index.md |   0
 .../query_timestamps/index.md                 |   0
 .../manual => manual}/reconnection/index.md   |   0
 .../source/manual => manual}/retries/index.md |   0
 .../manual => manual}/shaded_jar/index.md     |   0
 .../manual => manual}/socket_options/index.md |   0
 .../speculative_execution/index.md            |   0
 {docs/source/manual => manual}/ssl/index.md   |   0
 .../statements/batch/index.md                 |   0
 .../statements/built/index.md                 |   0
 .../manual => manual}/statements/index.md     |   0
 .../statements/prepared/index.md              |   0
 .../statements/simple/index.md                |   0
 .../source/manual => manual}/tuples/index.md  |   0
 {docs/source/manual => manual}/udts/index.md  |   0
 .../upgrade_guide => upgrade_guide}/index.md  |   0
 .../configuration/index.md                    |   0
 .../migrating_from_astyanax/index.md          |   0
 .../language_level_changes/index.md           |   0
 .../queries_and_results/index.md              |   0
 55 files changed, 256 insertions(+), 18 deletions(-)
 rename {docs/source/changelog => changelog}/index.md (100%)
 create mode 120000 docs/source/changelog
 create mode 120000 docs/source/faq
 rename docs/source/{installation/index.md => installation.md} (96%)
 create mode 120000 docs/source/manual
 create mode 120000 docs/source/upgrade_guide
 create mode 100644 driver-examples/index.md
 create mode 100644 driver-tests/osgi/index.md
 create mode 100644 driver-tests/stress/index.md
 rename {docs/source/faq => faq}/index.md (100%)
 rename {docs/source/faq => faq}/osgi/index.md (100%)
 create mode 100644 index.md
 rename {docs/source/manual => manual}/address_resolution/index.md (100%)
 rename {docs/source/manual => manual}/async/index.md (100%)
 rename {docs/source/manual => manual}/auth/index.md (100%)
 rename {docs/source/manual => manual}/cloud/index.md (100%)
 rename {docs/source/manual => manual}/compression/index.md (100%)
 rename {docs/source/manual => manual}/control_connection/index.md (100%)
 rename {docs/source/manual => manual}/custom_codecs/extras/index.md (100%)
 rename {docs/source/manual => manual}/custom_codecs/index.md (100%)
 rename {docs/source/manual => manual}/custom_payloads/index.md (100%)
 rename {docs/source/manual => manual}/idempotence/index.md (100%)
 rename {docs/source/manual => manual}/index.md (100%)
 rename {docs/source/manual => manual}/load_balancing/index.md (100%)
 rename {docs/source/manual => manual}/logging/index.md (100%)
 rename {docs/source/manual => manual}/metadata/index.md (100%)
 rename {docs/source/manual => manual}/metrics/index.md (100%)
 rename {docs/source/manual => manual}/native_protocol/index.md (100%)
 rename {docs/source/manual => manual}/object_mapper/creating/index.md (100%)
 rename {docs/source/manual => manual}/object_mapper/custom_codecs/index.md (100%)
 rename {docs/source/manual => manual}/object_mapper/index.md (100%)
 rename {docs/source/manual => manual}/object_mapper/using/index.md (100%)
 rename {docs/source/manual => manual}/osgi/index.md (100%)
 rename {docs/source/manual => manual}/paging/index.md (100%)
 rename {docs/source/manual => manual}/pooling/index.md (100%)
 rename {docs/source/manual => manual}/query_timestamps/index.md (100%)
 rename {docs/source/manual => manual}/reconnection/index.md (100%)
 rename {docs/source/manual => manual}/retries/index.md (100%)
 rename {docs/source/manual => manual}/shaded_jar/index.md (100%)
 rename {docs/source/manual => manual}/socket_options/index.md (100%)
 rename {docs/source/manual => manual}/speculative_execution/index.md (100%)
 rename {docs/source/manual => manual}/ssl/index.md (100%)
 rename {docs/source/manual => manual}/statements/batch/index.md (100%)
 rename {docs/source/manual => manual}/statements/built/index.md (100%)
 rename {docs/source/manual => manual}/statements/index.md (100%)
 rename {docs/source/manual => manual}/statements/prepared/index.md (100%)
 rename {docs/source/manual => manual}/statements/simple/index.md (100%)
 rename {docs/source/manual => manual}/tuples/index.md (100%)
 rename {docs/source/manual => manual}/udts/index.md (100%)
 rename {docs/source/upgrade_guide => upgrade_guide}/index.md (100%)
 rename {docs/source/upgrade_guide => upgrade_guide}/migrating_from_astyanax/configuration/index.md (100%)
 rename {docs/source/upgrade_guide => upgrade_guide}/migrating_from_astyanax/index.md (100%)
 rename {docs/source/upgrade_guide => upgrade_guide}/migrating_from_astyanax/language_level_changes/index.md (100%)
 rename {docs/source/upgrade_guide => upgrade_guide}/migrating_from_astyanax/queries_and_results/index.md (100%)

diff --git a/docs/source/changelog/index.md b/changelog/index.md
similarity index 100%
rename from docs/source/changelog/index.md
rename to changelog/index.md
diff --git a/docs/source/changelog b/docs/source/changelog
new file mode 120000
index 00000000000..2281eaa6b7b
--- /dev/null
+++ b/docs/source/changelog
@@ -0,0 +1 @@
+../../changelog/
\ No newline at end of file
diff --git a/docs/source/faq b/docs/source/faq
new file mode 120000
index 00000000000..bec65510d7f
--- /dev/null
+++ b/docs/source/faq
@@ -0,0 +1 @@
+../../faq/
\ No newline at end of file
diff --git a/docs/source/index.rst b/docs/source/index.rst
index 8871edff250..000d4919ab2 100644
--- a/docs/source/index.rst
+++ b/docs/source/index.rst
@@ -52,17 +52,17 @@ The driver contains the following modules:
 -  `changelog`_
 -  `binary tarball`_
 
-.. _feature-rich: manual/index
-.. _Sync: manual/index
-.. _Async: manual/async/index
-.. _Simple: manual/statements/simple/index
-.. _Prepared: manual/statements/prepared/index
-.. _Batch: manual/statements/batch/index
-.. _Connection pooling: manual/pooling/index
-.. _load balancing: manual/load_balancing/index
-.. _retry policies: manual/retries/index
-.. _Query builder: manual/statements/built/index
-.. _Object mapper: manual/object_mapper/index
+.. _feature-rich: ../../manual/index
+.. _Sync: ../../manual/index
+.. _Async: ../../manual/async/index
+.. _Simple: ../../manual/statements/simple/index
+.. _Prepared: ../../manual/statements/prepared/index
+.. _Batch: ../../manual/statements/batch/index
+.. _Connection pooling: ../../manual/pooling/index
+.. _load balancing: ../../manual/load_balancing/index
+.. _retry policies: ../../manual/retries/index
+.. _Query builder: ../../manual/statements/built/index
+.. _Object mapper: ../../manual/object_mapper/index
 .. _@dsJavaDriver: https://twitter.com/dsJavaDriver
 .. _@DataStaxEng: https://twitter.com/datastaxeng
 .. _manual: http://docs.datastax.com/en/developer/java-driver/3.7/manual/
@@ -91,9 +91,9 @@ limitations under the License.
    :hidden:
    
    api
-   installation/index
-   manual/index
-   upgrade_guide/index
-   faq/index
-   changelog/index
+   installation
+   manual/README
+   ../../upgrade_guide/README
+   ../../faq/README
+   ../../changelog/README
 
diff --git a/docs/source/installation/index.md b/docs/source/installation.md
similarity index 96%
rename from docs/source/installation/index.md
rename to docs/source/installation.md
index be6b9cf5496..95e874c45c6 100644
--- a/docs/source/installation/index.md
+++ b/docs/source/installation.md
@@ -35,7 +35,7 @@ The 'extras' module is also published as a separate artifact:
 ```
 
 
-We also provide a [shaded JAR](../manual/shaded_jar/)
+We also provide a [shaded JAR](../../manual/shaded_jar/)
 to avoid the explicit dependency to Netty.
 
 If you can't use a dependency management tool, a
@@ -68,7 +68,7 @@ big-endian systems, but _DataStax_ does not officially support these systems.
 ### Upgrading from previous versions
 
 If you are upgrading from a previous version of the driver, be sure to have a look at
-the [upgrade guide](../upgrade_guide/).
+the [upgrade guide](../../upgrade_guide/).
 
 If you are upgrading to _DataStax Enterprise_, use the [DataStax Enterprise Java driver][dse-driver] for more
 features and better compatibility.
diff --git a/docs/source/manual b/docs/source/manual
new file mode 120000
index 00000000000..fb3a8517e30
--- /dev/null
+++ b/docs/source/manual
@@ -0,0 +1 @@
+../../manual/
\ No newline at end of file
diff --git a/docs/source/upgrade_guide b/docs/source/upgrade_guide
new file mode 120000
index 00000000000..a1b678e2fa5
--- /dev/null
+++ b/docs/source/upgrade_guide
@@ -0,0 +1 @@
+../../upgrade_guide/
\ No newline at end of file
diff --git a/driver-examples/index.md b/driver-examples/index.md
new file mode 100644
index 00000000000..553cdbc5117
--- /dev/null
+++ b/driver-examples/index.md
@@ -0,0 +1,10 @@
+# DataStax Java Driver for Apache Cassandra - Examples
+
+This module contains examples of how to use the DataStax Java driver for
+Apache Cassandra.
+
+## Usage
+
+Unless otherwise stated, all examples assume that you have a single-node Cassandra 3.0 cluster 
+listening on localhost:9042.
+
diff --git a/driver-tests/osgi/index.md b/driver-tests/osgi/index.md
new file mode 100644
index 00000000000..b01ed55f05f
--- /dev/null
+++ b/driver-tests/osgi/index.md
@@ -0,0 +1,51 @@
+# OSGi Tests
+
+A collection of simple tests for the Java Driver in an OSGi environment.
+
+This project is mainly intended as a tool for validating
+that each new release of the driver is fully OSGi-compatible. 
+It is _not_ meant as an example application.
+
+If you are looking for examples demonstrating usage of the driver in an OSGi
+environment, please refer to our [OSGi examples repository].
+
+[OSGi examples repository]:https://github.com/datastax/java-driver-examples-osgi
+
+## Usage
+
+`MailboxService` is an OSGi service that uses Cassandra to
+store messages that can be retrieved by email address.
+
+Before running the tests, there are some prerequisites that must be met:
+
+The tests in this module will try to locate and load the following artifacts:
+
+1. The jars of 3 dependent modules:
+   `driver-core`, `driver-mapping` and `driver-extras`. 
+   These are expected to be found in their respective `target` directory;
+2. The test-jar for `driver-core`. Tests will attempt to load it from your local Maven repository.
+
+Therefore, before you can launch the tests, it is required to run `mvn install` 
+first _on the entire project_ to make sure all the above artifacts will be present.
+
+If the above prerequisites are met, then it is possible to build 
+this project and run its tests by executing the following Maven goal:
+
+    mvn verify -P short
+
+The "short" profile needs to be activated since the tests run under
+this group.
+
+Once `mvn verify` completes, the bundle jar will be present in the `target/` directory.
+
+The project includes integration tests that verify that the service can
+be activated and used in an OSGi container.  It also verifies that
+the Java driver can be used in an OSGi container in the following
+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/driver-tests/stress/index.md b/driver-tests/stress/index.md
new file mode 100644
index 00000000000..ea0ea61bc88
--- /dev/null
+++ b/driver-tests/stress/index.md
@@ -0,0 +1,25 @@
+# Stress application
+
+A simple example application that uses the Java driver to stress test
+Cassandra. This also somewhat stress tests the Java driver as a result.
+
+Please note that this simple example is far from being a complete stress
+application. In particular it currently supports a very limited number of
+stress scenarios.
+
+## Usage
+
+You will need to build the stress application fist:
+
+    ./bin/build
+
+After which you can run it using for instance:
+
+    ./bin/stress insert_prepared
+
+Of course, you will need to have at least one Cassandra node running (on
+127.0.0.1 by default) for this to work. Please refer to:
+
+    ./bin/stress -h
+
+for more details on the options available.
diff --git a/docs/source/faq/index.md b/faq/index.md
similarity index 100%
rename from docs/source/faq/index.md
rename to faq/index.md
diff --git a/docs/source/faq/osgi/index.md b/faq/osgi/index.md
similarity index 100%
rename from docs/source/faq/osgi/index.md
rename to faq/osgi/index.md
diff --git a/index.md b/index.md
new file mode 100644
index 00000000000..7f184b637c1
--- /dev/null
+++ b/index.md
@@ -0,0 +1,148 @@
+# Datastax Java Driver for Apache Cassandra®
+
+*If you're reading this on github.com, please note that this is the readme
+for the development version and that some features described here might
+not yet have been released. You can find the documentation for the latest
+version through the [Java driver
+docs](http://docs.datastax.com/en/developer/java-driver/latest/index.html) or via the release tags,
+[e.g.
+3.10.0](https://github.com/datastax/java-driver/tree/3.10.0).*
+
+A modern, [feature-rich](manual/) and highly tunable Java client
+library for Apache Cassandra (2.1+) and using exclusively Cassandra's binary protocol 
+and Cassandra Query Language v3. _Use the [DataStax Enterprise Java driver][dse-driver]
+for better compatibility and support for DataStax Enterprise._
+
+**Features:**
+
+* [Sync](manual/) and [Async](manual/async/) API
+* [Simple](manual/statements/simple/), [Prepared](manual/statements/prepared/), and [Batch](manual/statements/batch/)
+  statements
+* Asynchronous IO, parallel execution, request pipelining
+* [Connection pooling](manual/pooling/)
+* Auto node discovery
+* Automatic reconnection
+* Configurable [load balancing](manual/load_balancing/) and [retry policies](manual/retries/)
+* Works with any cluster size
+* [Query builder](manual/statements/built/)
+* [Object mapper](manual/object_mapper/)
+
+The driver architecture is based on layers. At the bottom lies the driver core.
+This core handles everything related to the connections to a Cassandra
+cluster (for example, connection pool, discovering new nodes, etc.) and exposes a simple,
+relatively low-level API on top of which higher level layers can be built.
+
+The driver contains the following modules:
+
+- driver-core: the core layer.
+- driver-mapping: the object mapper.
+- driver-extras: optional features for the Java driver.
+- driver-examples: example applications using the other modules which are
+  only meant for demonstration purposes.
+- driver-tests: tests for the java-driver.
+
+**Useful links:**
+
+- JIRA (bug tracking): https://datastax-oss.atlassian.net/browse/JAVA
+- MAILING LIST: https://groups.google.com/a/lists.datastax.com/forum/#!forum/java-driver-user
+- DATASTAX ACADEMY SLACK: #datastax-drivers on https://academy.datastax.com/slack 
+- TWITTER: [@dsJavaDriver](https://twitter.com/dsJavaDriver) tweets Java
+  driver releases and important announcements (low frequency).
+  [@DataStaxEng](https://twitter.com/datastaxeng) has more news including
+  other drivers, Cassandra, and DSE.
+- DOCS: the [manual](http://docs.datastax.com/en/developer/java-driver/3.10/manual/) has quick
+  start material and technical details about the driver and its features.
+- API: https://docs.datastax.com/en/drivers/java/3.10
+- GITHUB REPOSITORY: https://github.com/datastax/java-driver
+- [changelog](changelog/)
+- [binary tarball](http://downloads.datastax.com/java-driver/cassandra-java-driver-3.10.0.tar.gz)
+
+## Getting the driver
+
+The last release of the driver is available on Maven Central. You can install
+it in your application using the following Maven dependency (_if
+using DataStax Enterprise, install the [DataStax Enterprise Java driver][dse-driver] instead_):
+
+```xml
+
+  com.scylladb
+  scylla-driver-core
+  3.10.1-scylla-0-SNAPSHOT
+
+```
+
+Note that the object mapper is published as a separate artifact:
+
+```xml
+
+  com.scylladb
+  scylla-driver-mapping
+  3.10.1-scylla-0-SNAPSHOT
+
+```
+
+The 'extras' module is also published as a separate artifact:
+
+```xml
+
+  com.scylladb
+  scylla-driver-extras
+  3.10.1-scylla-0-SNAPSHOT
+
+```
+
+
+We also provide a [shaded JAR](manual/shaded_jar/)
+to avoid the explicit dependency to Netty.
+
+If you can't use a dependency management tool, a
+[binary tarball](http://downloads.datastax.com/java-driver/cassandra-java-driver-3.10.0.tar.gz)
+is available for download.
+
+## Compatibility
+
+The Java client driver 3.10.0 ([branch 3.x](https://github.com/datastax/java-driver/tree/3.x)) is compatible with Apache
+Cassandra 2.1, 2.2 and 3.0+ (see [this page](http://docs.datastax.com/en/developer/java-driver/latest/manual/native_protocol/) for
+the most up-to-date compatibility information).
+
+UDT and tuple support is available only when using Apache Cassandra 2.1 or higher (see [CQL improvements in Cassandra 2.1](http://www.datastax.com/dev/blog/cql-in-2-1)).
+
+Other features are available only when using Apache Cassandra 2.0 or higher (e.g. result set paging,
+[BatchStatement](https://github.com/datastax/java-driver/blob/3.x/driver-core/src/main/java/com/datastax/driver/core/BatchStatement.java),
+[lightweight transactions](http://www.datastax.com/documentation/cql/3.1/cql/cql_using/use_ltweight_transaction_t.html) 
+-- see [What's new in Cassandra 2.0](http://www.datastax.com/documentation/cassandra/2.0/cassandra/features/features_key_c.html)). 
+Trying to use these with a cluster running Cassandra 1.2 will result in 
+an [UnsupportedFeatureException](https://github.com/datastax/java-driver/blob/3.x/driver-core/src/main/java/com/datastax/driver/core/exceptions/UnsupportedFeatureException.java) being thrown.
+
+The java driver supports Java JDK versions 6 and above.
+
+If using _DataStax Enterprise_, the [DataStax Enterprise Java driver][dse-driver] provides 
+more features and better compatibility.
+
+__Disclaimer__: Some _DataStax/DataStax Enterprise_ products might partially work on 
+big-endian systems, but _DataStax_ does not officially support these systems.
+
+## Upgrading from previous versions
+
+If you are upgrading from a previous version of the driver, be sure to have a look at
+the [upgrade guide](/upgrade_guide/).
+
+If you are upgrading to _DataStax Enterprise_, use the [DataStax Enterprise Java driver][dse-driver] for more
+features and better compatibility.
+
+## License
+© 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.
+
+[dse-driver]: http://docs.datastax.com/en/developer/java-driver-dse/latest/
diff --git a/docs/source/manual/address_resolution/index.md b/manual/address_resolution/index.md
similarity index 100%
rename from docs/source/manual/address_resolution/index.md
rename to manual/address_resolution/index.md
diff --git a/docs/source/manual/async/index.md b/manual/async/index.md
similarity index 100%
rename from docs/source/manual/async/index.md
rename to manual/async/index.md
diff --git a/docs/source/manual/auth/index.md b/manual/auth/index.md
similarity index 100%
rename from docs/source/manual/auth/index.md
rename to manual/auth/index.md
diff --git a/docs/source/manual/cloud/index.md b/manual/cloud/index.md
similarity index 100%
rename from docs/source/manual/cloud/index.md
rename to manual/cloud/index.md
diff --git a/docs/source/manual/compression/index.md b/manual/compression/index.md
similarity index 100%
rename from docs/source/manual/compression/index.md
rename to manual/compression/index.md
diff --git a/docs/source/manual/control_connection/index.md b/manual/control_connection/index.md
similarity index 100%
rename from docs/source/manual/control_connection/index.md
rename to manual/control_connection/index.md
diff --git a/docs/source/manual/custom_codecs/extras/index.md b/manual/custom_codecs/extras/index.md
similarity index 100%
rename from docs/source/manual/custom_codecs/extras/index.md
rename to manual/custom_codecs/extras/index.md
diff --git a/docs/source/manual/custom_codecs/index.md b/manual/custom_codecs/index.md
similarity index 100%
rename from docs/source/manual/custom_codecs/index.md
rename to manual/custom_codecs/index.md
diff --git a/docs/source/manual/custom_payloads/index.md b/manual/custom_payloads/index.md
similarity index 100%
rename from docs/source/manual/custom_payloads/index.md
rename to manual/custom_payloads/index.md
diff --git a/docs/source/manual/idempotence/index.md b/manual/idempotence/index.md
similarity index 100%
rename from docs/source/manual/idempotence/index.md
rename to manual/idempotence/index.md
diff --git a/docs/source/manual/index.md b/manual/index.md
similarity index 100%
rename from docs/source/manual/index.md
rename to manual/index.md
diff --git a/docs/source/manual/load_balancing/index.md b/manual/load_balancing/index.md
similarity index 100%
rename from docs/source/manual/load_balancing/index.md
rename to manual/load_balancing/index.md
diff --git a/docs/source/manual/logging/index.md b/manual/logging/index.md
similarity index 100%
rename from docs/source/manual/logging/index.md
rename to manual/logging/index.md
diff --git a/docs/source/manual/metadata/index.md b/manual/metadata/index.md
similarity index 100%
rename from docs/source/manual/metadata/index.md
rename to manual/metadata/index.md
diff --git a/docs/source/manual/metrics/index.md b/manual/metrics/index.md
similarity index 100%
rename from docs/source/manual/metrics/index.md
rename to manual/metrics/index.md
diff --git a/docs/source/manual/native_protocol/index.md b/manual/native_protocol/index.md
similarity index 100%
rename from docs/source/manual/native_protocol/index.md
rename to manual/native_protocol/index.md
diff --git a/docs/source/manual/object_mapper/creating/index.md b/manual/object_mapper/creating/index.md
similarity index 100%
rename from docs/source/manual/object_mapper/creating/index.md
rename to manual/object_mapper/creating/index.md
diff --git a/docs/source/manual/object_mapper/custom_codecs/index.md b/manual/object_mapper/custom_codecs/index.md
similarity index 100%
rename from docs/source/manual/object_mapper/custom_codecs/index.md
rename to manual/object_mapper/custom_codecs/index.md
diff --git a/docs/source/manual/object_mapper/index.md b/manual/object_mapper/index.md
similarity index 100%
rename from docs/source/manual/object_mapper/index.md
rename to manual/object_mapper/index.md
diff --git a/docs/source/manual/object_mapper/using/index.md b/manual/object_mapper/using/index.md
similarity index 100%
rename from docs/source/manual/object_mapper/using/index.md
rename to manual/object_mapper/using/index.md
diff --git a/docs/source/manual/osgi/index.md b/manual/osgi/index.md
similarity index 100%
rename from docs/source/manual/osgi/index.md
rename to manual/osgi/index.md
diff --git a/docs/source/manual/paging/index.md b/manual/paging/index.md
similarity index 100%
rename from docs/source/manual/paging/index.md
rename to manual/paging/index.md
diff --git a/docs/source/manual/pooling/index.md b/manual/pooling/index.md
similarity index 100%
rename from docs/source/manual/pooling/index.md
rename to manual/pooling/index.md
diff --git a/docs/source/manual/query_timestamps/index.md b/manual/query_timestamps/index.md
similarity index 100%
rename from docs/source/manual/query_timestamps/index.md
rename to manual/query_timestamps/index.md
diff --git a/docs/source/manual/reconnection/index.md b/manual/reconnection/index.md
similarity index 100%
rename from docs/source/manual/reconnection/index.md
rename to manual/reconnection/index.md
diff --git a/docs/source/manual/retries/index.md b/manual/retries/index.md
similarity index 100%
rename from docs/source/manual/retries/index.md
rename to manual/retries/index.md
diff --git a/docs/source/manual/shaded_jar/index.md b/manual/shaded_jar/index.md
similarity index 100%
rename from docs/source/manual/shaded_jar/index.md
rename to manual/shaded_jar/index.md
diff --git a/docs/source/manual/socket_options/index.md b/manual/socket_options/index.md
similarity index 100%
rename from docs/source/manual/socket_options/index.md
rename to manual/socket_options/index.md
diff --git a/docs/source/manual/speculative_execution/index.md b/manual/speculative_execution/index.md
similarity index 100%
rename from docs/source/manual/speculative_execution/index.md
rename to manual/speculative_execution/index.md
diff --git a/docs/source/manual/ssl/index.md b/manual/ssl/index.md
similarity index 100%
rename from docs/source/manual/ssl/index.md
rename to manual/ssl/index.md
diff --git a/docs/source/manual/statements/batch/index.md b/manual/statements/batch/index.md
similarity index 100%
rename from docs/source/manual/statements/batch/index.md
rename to manual/statements/batch/index.md
diff --git a/docs/source/manual/statements/built/index.md b/manual/statements/built/index.md
similarity index 100%
rename from docs/source/manual/statements/built/index.md
rename to manual/statements/built/index.md
diff --git a/docs/source/manual/statements/index.md b/manual/statements/index.md
similarity index 100%
rename from docs/source/manual/statements/index.md
rename to manual/statements/index.md
diff --git a/docs/source/manual/statements/prepared/index.md b/manual/statements/prepared/index.md
similarity index 100%
rename from docs/source/manual/statements/prepared/index.md
rename to manual/statements/prepared/index.md
diff --git a/docs/source/manual/statements/simple/index.md b/manual/statements/simple/index.md
similarity index 100%
rename from docs/source/manual/statements/simple/index.md
rename to manual/statements/simple/index.md
diff --git a/docs/source/manual/tuples/index.md b/manual/tuples/index.md
similarity index 100%
rename from docs/source/manual/tuples/index.md
rename to manual/tuples/index.md
diff --git a/docs/source/manual/udts/index.md b/manual/udts/index.md
similarity index 100%
rename from docs/source/manual/udts/index.md
rename to manual/udts/index.md
diff --git a/docs/source/upgrade_guide/index.md b/upgrade_guide/index.md
similarity index 100%
rename from docs/source/upgrade_guide/index.md
rename to upgrade_guide/index.md
diff --git a/docs/source/upgrade_guide/migrating_from_astyanax/configuration/index.md b/upgrade_guide/migrating_from_astyanax/configuration/index.md
similarity index 100%
rename from docs/source/upgrade_guide/migrating_from_astyanax/configuration/index.md
rename to upgrade_guide/migrating_from_astyanax/configuration/index.md
diff --git a/docs/source/upgrade_guide/migrating_from_astyanax/index.md b/upgrade_guide/migrating_from_astyanax/index.md
similarity index 100%
rename from docs/source/upgrade_guide/migrating_from_astyanax/index.md
rename to upgrade_guide/migrating_from_astyanax/index.md
diff --git a/docs/source/upgrade_guide/migrating_from_astyanax/language_level_changes/index.md b/upgrade_guide/migrating_from_astyanax/language_level_changes/index.md
similarity index 100%
rename from docs/source/upgrade_guide/migrating_from_astyanax/language_level_changes/index.md
rename to upgrade_guide/migrating_from_astyanax/language_level_changes/index.md
diff --git a/docs/source/upgrade_guide/migrating_from_astyanax/queries_and_results/index.md b/upgrade_guide/migrating_from_astyanax/queries_and_results/index.md
similarity index 100%
rename from docs/source/upgrade_guide/migrating_from_astyanax/queries_and_results/index.md
rename to upgrade_guide/migrating_from_astyanax/queries_and_results/index.md

From b15ff8a4b967bc13a6c41c34d7528ec22d561768 Mon Sep 17 00:00:00 2001
From: dgarcia360 
Date: Mon, 14 Sep 2020 14:55:18 +0200
Subject: [PATCH 44/52] Fixed links

---
 docs/source/index.rst        |  32 ++++----
 docs/source/installation.md  |   4 +-
 driver-examples/index.md     |  10 ---
 driver-tests/osgi/index.md   |  51 ------------
 driver-tests/stress/index.md |  25 ------
 index.md                     | 148 -----------------------------------
 6 files changed, 18 insertions(+), 252 deletions(-)
 delete mode 100644 driver-examples/index.md
 delete mode 100644 driver-tests/osgi/index.md
 delete mode 100644 driver-tests/stress/index.md
 delete mode 100644 index.md

diff --git a/docs/source/index.rst b/docs/source/index.rst
index 000d4919ab2..edd5d90c973 100644
--- a/docs/source/index.rst
+++ b/docs/source/index.rst
@@ -52,21 +52,21 @@ The driver contains the following modules:
 -  `changelog`_
 -  `binary tarball`_
 
-.. _feature-rich: ../../manual/index
-.. _Sync: ../../manual/index
-.. _Async: ../../manual/async/index
-.. _Simple: ../../manual/statements/simple/index
-.. _Prepared: ../../manual/statements/prepared/index
-.. _Batch: ../../manual/statements/batch/index
-.. _Connection pooling: ../../manual/pooling/index
-.. _load balancing: ../../manual/load_balancing/index
-.. _retry policies: ../../manual/retries/index
-.. _Query builder: ../../manual/statements/built/index
-.. _Object mapper: ../../manual/object_mapper/index
+.. _feature-rich: manual
+.. _Sync: manual
+.. _Async: manual/async
+.. _Simple: manual/statements/simple
+.. _Prepared: manual/statements/prepared
+.. _Batch: manual/statements/batch
+.. _Connection pooling: manual/pooling
+.. _load balancing: manual/load_balancing
+.. _retry policies: manual/retries
+.. _Query builder: manual/statements/built
+.. _Object mapper: manual/object_mapper
 .. _@dsJavaDriver: https://twitter.com/dsJavaDriver
 .. _@DataStaxEng: https://twitter.com/datastaxeng
 .. _manual: http://docs.datastax.com/en/developer/java-driver/3.7/manual/
-.. _changelog: changelog/index
+.. _changelog: changelog
 .. _binary tarball: http://downloads.datastax.com/java-driver/cassandra-java-driver-3.7.1.tar.gz
 
 
@@ -92,8 +92,8 @@ limitations under the License.
    
    api
    installation
-   manual/README
-   ../../upgrade_guide/README
-   ../../faq/README
-   ../../changelog/README
+   manual/index
+   upgrade_guide/index
+   faq/index
+   changelog/index
 
diff --git a/docs/source/installation.md b/docs/source/installation.md
index 95e874c45c6..51f400b44c5 100644
--- a/docs/source/installation.md
+++ b/docs/source/installation.md
@@ -35,7 +35,7 @@ The 'extras' module is also published as a separate artifact:
 ```
 
 
-We also provide a [shaded JAR](../../manual/shaded_jar/)
+We also provide a [shaded JAR](manual/shaded_jar/)
 to avoid the explicit dependency to Netty.
 
 If you can't use a dependency management tool, a
@@ -68,7 +68,7 @@ big-endian systems, but _DataStax_ does not officially support these systems.
 ### Upgrading from previous versions
 
 If you are upgrading from a previous version of the driver, be sure to have a look at
-the [upgrade guide](../../upgrade_guide/).
+the [upgrade guide](upgrade_guide/).
 
 If you are upgrading to _DataStax Enterprise_, use the [DataStax Enterprise Java driver][dse-driver] for more
 features and better compatibility.
diff --git a/driver-examples/index.md b/driver-examples/index.md
deleted file mode 100644
index 553cdbc5117..00000000000
--- a/driver-examples/index.md
+++ /dev/null
@@ -1,10 +0,0 @@
-# DataStax Java Driver for Apache Cassandra - Examples
-
-This module contains examples of how to use the DataStax Java driver for
-Apache Cassandra.
-
-## Usage
-
-Unless otherwise stated, all examples assume that you have a single-node Cassandra 3.0 cluster 
-listening on localhost:9042.
-
diff --git a/driver-tests/osgi/index.md b/driver-tests/osgi/index.md
deleted file mode 100644
index b01ed55f05f..00000000000
--- a/driver-tests/osgi/index.md
+++ /dev/null
@@ -1,51 +0,0 @@
-# OSGi Tests
-
-A collection of simple tests for the Java Driver in an OSGi environment.
-
-This project is mainly intended as a tool for validating
-that each new release of the driver is fully OSGi-compatible. 
-It is _not_ meant as an example application.
-
-If you are looking for examples demonstrating usage of the driver in an OSGi
-environment, please refer to our [OSGi examples repository].
-
-[OSGi examples repository]:https://github.com/datastax/java-driver-examples-osgi
-
-## Usage
-
-`MailboxService` is an OSGi service that uses Cassandra to
-store messages that can be retrieved by email address.
-
-Before running the tests, there are some prerequisites that must be met:
-
-The tests in this module will try to locate and load the following artifacts:
-
-1. The jars of 3 dependent modules:
-   `driver-core`, `driver-mapping` and `driver-extras`. 
-   These are expected to be found in their respective `target` directory;
-2. The test-jar for `driver-core`. Tests will attempt to load it from your local Maven repository.
-
-Therefore, before you can launch the tests, it is required to run `mvn install` 
-first _on the entire project_ to make sure all the above artifacts will be present.
-
-If the above prerequisites are met, then it is possible to build 
-this project and run its tests by executing the following Maven goal:
-
-    mvn verify -P short
-
-The "short" profile needs to be activated since the tests run under
-this group.
-
-Once `mvn verify` completes, the bundle jar will be present in the `target/` directory.
-
-The project includes integration tests that verify that the service can
-be activated and used in an OSGi container.  It also verifies that
-the Java driver can be used in an OSGi container in the following
-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/driver-tests/stress/index.md b/driver-tests/stress/index.md
deleted file mode 100644
index ea0ea61bc88..00000000000
--- a/driver-tests/stress/index.md
+++ /dev/null
@@ -1,25 +0,0 @@
-# Stress application
-
-A simple example application that uses the Java driver to stress test
-Cassandra. This also somewhat stress tests the Java driver as a result.
-
-Please note that this simple example is far from being a complete stress
-application. In particular it currently supports a very limited number of
-stress scenarios.
-
-## Usage
-
-You will need to build the stress application fist:
-
-    ./bin/build
-
-After which you can run it using for instance:
-
-    ./bin/stress insert_prepared
-
-Of course, you will need to have at least one Cassandra node running (on
-127.0.0.1 by default) for this to work. Please refer to:
-
-    ./bin/stress -h
-
-for more details on the options available.
diff --git a/index.md b/index.md
deleted file mode 100644
index 7f184b637c1..00000000000
--- a/index.md
+++ /dev/null
@@ -1,148 +0,0 @@
-# Datastax Java Driver for Apache Cassandra®
-
-*If you're reading this on github.com, please note that this is the readme
-for the development version and that some features described here might
-not yet have been released. You can find the documentation for the latest
-version through the [Java driver
-docs](http://docs.datastax.com/en/developer/java-driver/latest/index.html) or via the release tags,
-[e.g.
-3.10.0](https://github.com/datastax/java-driver/tree/3.10.0).*
-
-A modern, [feature-rich](manual/) and highly tunable Java client
-library for Apache Cassandra (2.1+) and using exclusively Cassandra's binary protocol 
-and Cassandra Query Language v3. _Use the [DataStax Enterprise Java driver][dse-driver]
-for better compatibility and support for DataStax Enterprise._
-
-**Features:**
-
-* [Sync](manual/) and [Async](manual/async/) API
-* [Simple](manual/statements/simple/), [Prepared](manual/statements/prepared/), and [Batch](manual/statements/batch/)
-  statements
-* Asynchronous IO, parallel execution, request pipelining
-* [Connection pooling](manual/pooling/)
-* Auto node discovery
-* Automatic reconnection
-* Configurable [load balancing](manual/load_balancing/) and [retry policies](manual/retries/)
-* Works with any cluster size
-* [Query builder](manual/statements/built/)
-* [Object mapper](manual/object_mapper/)
-
-The driver architecture is based on layers. At the bottom lies the driver core.
-This core handles everything related to the connections to a Cassandra
-cluster (for example, connection pool, discovering new nodes, etc.) and exposes a simple,
-relatively low-level API on top of which higher level layers can be built.
-
-The driver contains the following modules:
-
-- driver-core: the core layer.
-- driver-mapping: the object mapper.
-- driver-extras: optional features for the Java driver.
-- driver-examples: example applications using the other modules which are
-  only meant for demonstration purposes.
-- driver-tests: tests for the java-driver.
-
-**Useful links:**
-
-- JIRA (bug tracking): https://datastax-oss.atlassian.net/browse/JAVA
-- MAILING LIST: https://groups.google.com/a/lists.datastax.com/forum/#!forum/java-driver-user
-- DATASTAX ACADEMY SLACK: #datastax-drivers on https://academy.datastax.com/slack 
-- TWITTER: [@dsJavaDriver](https://twitter.com/dsJavaDriver) tweets Java
-  driver releases and important announcements (low frequency).
-  [@DataStaxEng](https://twitter.com/datastaxeng) has more news including
-  other drivers, Cassandra, and DSE.
-- DOCS: the [manual](http://docs.datastax.com/en/developer/java-driver/3.10/manual/) has quick
-  start material and technical details about the driver and its features.
-- API: https://docs.datastax.com/en/drivers/java/3.10
-- GITHUB REPOSITORY: https://github.com/datastax/java-driver
-- [changelog](changelog/)
-- [binary tarball](http://downloads.datastax.com/java-driver/cassandra-java-driver-3.10.0.tar.gz)
-
-## Getting the driver
-
-The last release of the driver is available on Maven Central. You can install
-it in your application using the following Maven dependency (_if
-using DataStax Enterprise, install the [DataStax Enterprise Java driver][dse-driver] instead_):
-
-```xml
-
-  com.scylladb
-  scylla-driver-core
-  3.10.1-scylla-0-SNAPSHOT
-
-```
-
-Note that the object mapper is published as a separate artifact:
-
-```xml
-
-  com.scylladb
-  scylla-driver-mapping
-  3.10.1-scylla-0-SNAPSHOT
-
-```
-
-The 'extras' module is also published as a separate artifact:
-
-```xml
-
-  com.scylladb
-  scylla-driver-extras
-  3.10.1-scylla-0-SNAPSHOT
-
-```
-
-
-We also provide a [shaded JAR](manual/shaded_jar/)
-to avoid the explicit dependency to Netty.
-
-If you can't use a dependency management tool, a
-[binary tarball](http://downloads.datastax.com/java-driver/cassandra-java-driver-3.10.0.tar.gz)
-is available for download.
-
-## Compatibility
-
-The Java client driver 3.10.0 ([branch 3.x](https://github.com/datastax/java-driver/tree/3.x)) is compatible with Apache
-Cassandra 2.1, 2.2 and 3.0+ (see [this page](http://docs.datastax.com/en/developer/java-driver/latest/manual/native_protocol/) for
-the most up-to-date compatibility information).
-
-UDT and tuple support is available only when using Apache Cassandra 2.1 or higher (see [CQL improvements in Cassandra 2.1](http://www.datastax.com/dev/blog/cql-in-2-1)).
-
-Other features are available only when using Apache Cassandra 2.0 or higher (e.g. result set paging,
-[BatchStatement](https://github.com/datastax/java-driver/blob/3.x/driver-core/src/main/java/com/datastax/driver/core/BatchStatement.java),
-[lightweight transactions](http://www.datastax.com/documentation/cql/3.1/cql/cql_using/use_ltweight_transaction_t.html) 
--- see [What's new in Cassandra 2.0](http://www.datastax.com/documentation/cassandra/2.0/cassandra/features/features_key_c.html)). 
-Trying to use these with a cluster running Cassandra 1.2 will result in 
-an [UnsupportedFeatureException](https://github.com/datastax/java-driver/blob/3.x/driver-core/src/main/java/com/datastax/driver/core/exceptions/UnsupportedFeatureException.java) being thrown.
-
-The java driver supports Java JDK versions 6 and above.
-
-If using _DataStax Enterprise_, the [DataStax Enterprise Java driver][dse-driver] provides 
-more features and better compatibility.
-
-__Disclaimer__: Some _DataStax/DataStax Enterprise_ products might partially work on 
-big-endian systems, but _DataStax_ does not officially support these systems.
-
-## Upgrading from previous versions
-
-If you are upgrading from a previous version of the driver, be sure to have a look at
-the [upgrade guide](/upgrade_guide/).
-
-If you are upgrading to _DataStax Enterprise_, use the [DataStax Enterprise Java driver][dse-driver] for more
-features and better compatibility.
-
-## License
-© 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.
-
-[dse-driver]: http://docs.datastax.com/en/developer/java-driver-dse/latest/

From 7fc6f211b2da9fcf12385a21bec4f6628192733a Mon Sep 17 00:00:00 2001
From: dgarcia360 
Date: Mon, 14 Sep 2020 15:54:39 +0200
Subject: [PATCH 45/52] Remove index references from toctree

---
 docs/source/index.rst                         | 11 ++--
 manual/index.md                               | 58 +++++++++----------
 manual/object_mapper/index.md                 |  7 ++-
 manual/statements/index.md                    |  9 +--
 upgrade_guide/index.md                        |  3 +-
 .../migrating_from_astyanax/index.md          |  7 ++-
 6 files changed, 50 insertions(+), 45 deletions(-)

diff --git a/docs/source/index.rst b/docs/source/index.rst
index edd5d90c973..23c8e618702 100644
--- a/docs/source/index.rst
+++ b/docs/source/index.rst
@@ -89,11 +89,12 @@ limitations under the License.
 
 .. toctree::
    :hidden:
-   
+   :glob:
+
    api
    installation
-   manual/index
-   upgrade_guide/index
-   faq/index
-   changelog/index
+   manual/*
+   upgrade_guide/*
+   faq/*
+   changelog/*
 
diff --git a/manual/index.md b/manual/index.md
index c2f72356a08..49cda94a082 100644
--- a/manual/index.md
+++ b/manual/index.md
@@ -309,34 +309,34 @@ simply navigate to each sub-directory.
 ```eval_rst
 .. toctree::
    :hidden:
+   :glob:
    
-   address_resolution/index
-   async/index
-   auth/index
-   address_resolution/index
-   cloud/index
-   compression/index
-   control_connection/index
-   custom_codecs/index
-   custom_payloads/index
-   idempotence/index
-   load_balancing/index
-   logging/index
-   metadata/index
-   metrics/index
-   native_protocol/index
-   object_mapper/index
-   osgi/index
-   paging/index
-   pooling/index
-   query_timestamps/index
-   reconnection/index
-   retries/index
-   shaded_jar/index
-   socket_options/index
-   speculative_execution/index
-   ssl/index
-   statements/index
-   tuples/index
-   udts/index
+   address_resolution/*
+   async/*
+   auth/*
+   cloud/*
+   compression/*
+   control_connection/*
+   custom_codecs/*
+   custom_payloads/*
+   idempotence/*
+   load_balancing/*
+   logging/*
+   metadata/*
+   metrics/*
+   native_protocol/*
+   object_mapper/*
+   osgi/*
+   paging/*
+   pooling/*
+   query_timestamps/*
+   reconnection/*
+   retries/*
+   shaded_jar/*
+   socket_options/*
+   speculative_execution/*
+   ssl/*
+   statements/*
+   tuples/*
+   udts/*
 ```
\ No newline at end of file
diff --git a/manual/object_mapper/index.md b/manual/object_mapper/index.md
index 6c55e55714c..ffc8fb4b764 100644
--- a/manual/object_mapper/index.md
+++ b/manual/object_mapper/index.md
@@ -24,8 +24,9 @@ See the child pages for more information:
 ```eval_rst
 .. toctree::
    :hidden:
+   :glob:
       
-   creating/index
-   custom_codecs/index
-   using/index
+   creating/*
+   custom_codecs/*
+   using/*
 ```
diff --git a/manual/statements/index.md b/manual/statements/index.md
index 03705d49f27..9a941c30128 100644
--- a/manual/statements/index.md
+++ b/manual/statements/index.md
@@ -44,9 +44,10 @@ wrap your statements in a custom [StatementWrapper] implementation.
 ```eval_rst
 .. toctree::
    :hidden:
+   :glob:
    
-   simple/index
-   prepared/index
-   built/index
-   batch/index
+   simple/*
+   prepared/*
+   built/*
+   batch/*
 ```
diff --git a/upgrade_guide/index.md b/upgrade_guide/index.md
index 702786fd071..081cb571c26 100644
--- a/upgrade_guide/index.md
+++ b/upgrade_guide/index.md
@@ -830,6 +830,7 @@ exhaustive list of new features in 2.0.
 ```eval_rst
 .. toctree::
    :hidden:
+   :glob:
    
-   migrating_from_astyanax/index
+   migrating_from_astyanax/*
 ```
\ No newline at end of file
diff --git a/upgrade_guide/migrating_from_astyanax/index.md b/upgrade_guide/migrating_from_astyanax/index.md
index b683b84dd02..c899252d63d 100644
--- a/upgrade_guide/migrating_from_astyanax/index.md
+++ b/upgrade_guide/migrating_from_astyanax/index.md
@@ -12,8 +12,9 @@ See the child pages for more information:
 ```eval_rst
 .. toctree::
    :hidden:
+   :glob:
    
-   configuration/index
-   language_level_changes/index
-   queries_and_results/index
+   configuration/*
+   language_level_changes/*
+   queries_and_results/*
 ```
\ No newline at end of file

From 9b9b5fb4d225fa0b8dac8b023eba089fe3d16a3e Mon Sep 17 00:00:00 2001
From: dgarcia360 
Date: Mon, 21 Sep 2020 11:01:45 +0200
Subject: [PATCH 46/52] Rollback index.md to README.md

---
 .github/workflows/pages.yml                      |  3 ---
 .gitignore                                       |  1 +
 changelog/{index.md => README.md}                |  0
 docs/Makefile                                    | 15 +++++++++++----
 docs/_utils/api.html                             |  8 --------
 docs/_utils/deploy.sh                            |  1 -
 docs/_utils/doxygen.sh                           | 12 +++++-------
 docs/_utils/multiversion.sh                      |  3 ---
 docs/poetry.lock                                 | 16 +++++++++++++++-
 docs/pyproject.toml                              |  2 +-
 faq/{index.md => README.md}                      |  0
 faq/osgi/{index.md => README.md}                 |  0
 manual/{index.md => README.md}                   |  0
 .../address_resolution/{index.md => README.md}   |  0
 manual/async/{index.md => README.md}             |  0
 manual/auth/{index.md => README.md}              |  0
 manual/cloud/{index.md => README.md}             |  0
 manual/compression/{index.md => README.md}       |  0
 .../control_connection/{index.md => README.md}   |  0
 manual/custom_codecs/{index.md => README.md}     |  0
 .../custom_codecs/extras/{index.md => README.md} |  0
 manual/custom_payloads/{index.md => README.md}   |  0
 manual/idempotence/{index.md => README.md}       |  0
 manual/load_balancing/{index.md => README.md}    |  0
 manual/logging/{index.md => README.md}           |  0
 manual/metadata/{index.md => README.md}          |  0
 manual/metrics/{index.md => README.md}           |  0
 manual/native_protocol/{index.md => README.md}   |  0
 manual/object_mapper/{index.md => README.md}     |  0
 .../creating/{index.md => README.md}             |  0
 .../custom_codecs/{index.md => README.md}        |  0
 .../object_mapper/using/{index.md => README.md}  |  0
 manual/osgi/{index.md => README.md}              |  0
 manual/paging/{index.md => README.md}            |  0
 manual/pooling/{index.md => README.md}           |  0
 manual/query_timestamps/{index.md => README.md}  |  0
 manual/reconnection/{index.md => README.md}      |  0
 manual/retries/{index.md => README.md}           |  0
 manual/shaded_jar/{index.md => README.md}        |  0
 manual/socket_options/{index.md => README.md}    |  0
 .../{index.md => README.md}                      |  0
 manual/ssl/{index.md => README.md}               |  0
 manual/statements/{index.md => README.md}        |  0
 manual/statements/batch/{index.md => README.md}  |  0
 manual/statements/built/{index.md => README.md}  |  0
 .../statements/prepared/{index.md => README.md}  |  0
 manual/statements/simple/{index.md => README.md} |  0
 manual/tuples/{index.md => README.md}            |  0
 manual/udts/{index.md => README.md}              |  0
 upgrade_guide/{index.md => README.md}            |  0
 .../{index.md => README.md}                      |  0
 .../configuration/{index.md => README.md}        |  0
 .../{index.md => README.md}                      |  0
 .../queries_and_results/{index.md => README.md}  |  0
 54 files changed, 33 insertions(+), 28 deletions(-)
 rename changelog/{index.md => README.md} (100%)
 delete mode 100644 docs/_utils/api.html
 delete mode 100755 docs/_utils/multiversion.sh
 rename faq/{index.md => README.md} (100%)
 rename faq/osgi/{index.md => README.md} (100%)
 rename manual/{index.md => README.md} (100%)
 rename manual/address_resolution/{index.md => README.md} (100%)
 rename manual/async/{index.md => README.md} (100%)
 rename manual/auth/{index.md => README.md} (100%)
 rename manual/cloud/{index.md => README.md} (100%)
 rename manual/compression/{index.md => README.md} (100%)
 rename manual/control_connection/{index.md => README.md} (100%)
 rename manual/custom_codecs/{index.md => README.md} (100%)
 rename manual/custom_codecs/extras/{index.md => README.md} (100%)
 rename manual/custom_payloads/{index.md => README.md} (100%)
 rename manual/idempotence/{index.md => README.md} (100%)
 rename manual/load_balancing/{index.md => README.md} (100%)
 rename manual/logging/{index.md => README.md} (100%)
 rename manual/metadata/{index.md => README.md} (100%)
 rename manual/metrics/{index.md => README.md} (100%)
 rename manual/native_protocol/{index.md => README.md} (100%)
 rename manual/object_mapper/{index.md => README.md} (100%)
 rename manual/object_mapper/creating/{index.md => README.md} (100%)
 rename manual/object_mapper/custom_codecs/{index.md => README.md} (100%)
 rename manual/object_mapper/using/{index.md => README.md} (100%)
 rename manual/osgi/{index.md => README.md} (100%)
 rename manual/paging/{index.md => README.md} (100%)
 rename manual/pooling/{index.md => README.md} (100%)
 rename manual/query_timestamps/{index.md => README.md} (100%)
 rename manual/reconnection/{index.md => README.md} (100%)
 rename manual/retries/{index.md => README.md} (100%)
 rename manual/shaded_jar/{index.md => README.md} (100%)
 rename manual/socket_options/{index.md => README.md} (100%)
 rename manual/speculative_execution/{index.md => README.md} (100%)
 rename manual/ssl/{index.md => README.md} (100%)
 rename manual/statements/{index.md => README.md} (100%)
 rename manual/statements/batch/{index.md => README.md} (100%)
 rename manual/statements/built/{index.md => README.md} (100%)
 rename manual/statements/prepared/{index.md => README.md} (100%)
 rename manual/statements/simple/{index.md => README.md} (100%)
 rename manual/tuples/{index.md => README.md} (100%)
 rename manual/udts/{index.md => README.md} (100%)
 rename upgrade_guide/{index.md => README.md} (100%)
 rename upgrade_guide/migrating_from_astyanax/{index.md => README.md} (100%)
 rename upgrade_guide/migrating_from_astyanax/configuration/{index.md => README.md} (100%)
 rename upgrade_guide/migrating_from_astyanax/language_level_changes/{index.md => README.md} (100%)
 rename upgrade_guide/migrating_from_astyanax/queries_and_results/{index.md => README.md} (100%)

diff --git a/.github/workflows/pages.yml b/.github/workflows/pages.yml
index e9698efe8e9..a0859c3da6c 100644
--- a/.github/workflows/pages.yml
+++ b/.github/workflows/pages.yml
@@ -27,9 +27,6 @@ jobs:
         export PATH=$PATH:~/.local/bin
         cd docs
         make multiversion
-    - name: Build Doxygen docs
-      run: |
-        ./docs/_utils/doxygen.sh
     - name: Deploy
       run : ./docs/_utils/deploy.sh
       env:
diff --git a/.gitignore b/.gitignore
index 50bf5df9257..b631491efeb 100644
--- a/.gitignore
+++ b/.gitignore
@@ -5,6 +5,7 @@ testing/
 .classpath
 .project
 docs/_build
+docs/_source
 html
 latex
 notes
diff --git a/changelog/index.md b/changelog/README.md
similarity index 100%
rename from changelog/index.md
rename to changelog/README.md
diff --git a/docs/Makefile b/docs/Makefile
index 7a18ae2c1af..73f9a79732a 100644
--- a/docs/Makefile
+++ b/docs/Makefile
@@ -3,11 +3,12 @@ SPHINXOPTS    =
 SPHINXBUILD   = poetry run sphinx-build
 PAPER         =
 BUILDDIR      = _build
+SOURCE_DIR    = _source
 
 # Internal variables.
 PAPEROPT_a4     = -D latex_paper_size=a4
 PAPEROPT_letter = -D latex_paper_size=letter
-ALLSPHINXOPTS   = -d $(BUILDDIR)/doctrees $(PAPEROPT_$(PAPER)) $(SPHINXOPTS) source
+ALLSPHINXOPTS   = -d $(BUILDDIR)/doctrees $(PAPEROPT_$(PAPER)) $(SPHINXOPTS) $(SOURCE_DIR)
 # the i18n builder cannot share the environment and doctrees with the others
 I18NSPHINXOPTS  = $(PAPEROPT_$(PAPER)) $(SPHINXOPTS) .
 
@@ -21,14 +22,16 @@ pristine: clean
 .PHONY: setup
 setup:
 	./_utils/setup.sh
-
+	cp -Tr source $(SOURCE_DIR)
+	cd $(SOURCE_DIR) && find -name README.md -execdir mv '{}' index.md ';'
 .PHONY: clean
 clean:
 	rm -rf $(BUILDDIR)/*
+	rm -rf $(SOURCE_DIR)/*
 
 .PHONY: preview
 preview: setup
-	cd .. && ./docs/_utils/doxygen.sh
+	./_utils/doxygen.sh
 	poetry run sphinx-autobuild -b dirhtml $(ALLSPHINXOPTS) $(BUILDDIR)/dirhtml --port 5500
 
 .PHONY: dirhtml
@@ -67,6 +70,10 @@ linkcheck: setup
 
 .PHONY: multiversion
 multiversion: setup
-	poetry run ./_utils/multiversion.sh
+	cd .. && sphinx-multiversion docs/_source docs/$(BUILDDIR)/dirhtml  \
+		--pre-build="./_utils/doxygen.sh"  \
+		--pre-build="cp -Tr source $(SOURCE_DIR)"  \
+		--pre-build="cd $(SOURCE_DIR) && find -name README.md -execdir mv '{}' index.md ';'"
 	@echo
 	@echo "Build finished. The HTML pages are in $(BUILDDIR)/dirhtml."
+
diff --git a/docs/_utils/api.html b/docs/_utils/api.html
deleted file mode 100644
index 1e876cb1a4c..00000000000
--- a/docs/_utils/api.html
+++ /dev/null
@@ -1,8 +0,0 @@
-
-
-  
-    Redirecting to API Documentation
-    
-    
-  
-
diff --git a/docs/_utils/deploy.sh b/docs/_utils/deploy.sh
index 0450baabb7a..77321c192f2 100755
--- a/docs/_utils/deploy.sh
+++ b/docs/_utils/deploy.sh
@@ -5,7 +5,6 @@ git clone "https://x-access-token:${GITHUB_TOKEN}@github.com/${GITHUB_REPOSITORY
 cp -r docs/_build/dirhtml/* gh-pages
 # Redirections
 ./docs/_utils/redirect.sh > gh-pages/index.html
-find  gh-pages/* -name "api.html" -exec cp ./docs/_utils/api.html {} \;
 # Deploy
 cd gh-pages
 touch .nojekyll
diff --git a/docs/_utils/doxygen.sh b/docs/_utils/doxygen.sh
index ab1200765a0..906337747d3 100755
--- a/docs/_utils/doxygen.sh
+++ b/docs/_utils/doxygen.sh
@@ -1,12 +1,10 @@
 #!/bin/bash
 
-GITHUB_VERSION=${GITHUB_REF##*/}
-
-OUTPUT_DIR="docs/_build/dirhtml/api"
-if [[ "$GITHUB_VERSION" != "" ]]; then
-    OUTPUT_DIR="docs/_build/dirhtml/$GITHUB_VERSION/api"
-    echo "HTML_OUTPUT = $OUTPUT_DIR" >> doxyfile
+OUTPUT_DIR="_build/dirhtml/api"
+if [[ "$SPHINX_MULTIVERSION_OUTPUTDIR" != "" ]]; then
+    echo $SPHINX_MULTIVERSION_OUTPUTDIR
+    OUTPUT_DIR="$SPHINX_MULTIVERSION_OUTPUTDIR/api"
 fi
 echo $OUTPUT_DIR
 mkdir -p "$OUTPUT_DIR"
-doxygen doxyfile
+cd .. && ( cat Doxyfile | echo "HTML_OUTPUT = $OUTPUT_DIR" )  | doxygen doxyfile
diff --git a/docs/_utils/multiversion.sh b/docs/_utils/multiversion.sh
deleted file mode 100755
index 91d59f72de7..00000000000
--- a/docs/_utils/multiversion.sh
+++ /dev/null
@@ -1,3 +0,0 @@
-#! /bin/bash
-
-cd .. && sphinx-multiversion docs/source docs/_build/dirhtml
diff --git a/docs/poetry.lock b/docs/poetry.lock
index e2f9009a078..6c2826388dd 100644
--- a/docs/poetry.lock
+++ b/docs/poetry.lock
@@ -399,6 +399,17 @@ version = "0.2.3"
 [package.dependencies]
 sphinx = ">=2.1"
 
+[[package]]
+category = "main"
+description = "Add support for multiple versions to sphinx"
+name = "sphinx-multiversion-scylla"
+optional = false
+python-versions = "*"
+version = "0.2.4"
+
+[package.dependencies]
+sphinx = ">=2.1"
+
 [[package]]
 category = "main"
 description = "A Sphinx Theme for ScyllaDB projects documentation"
@@ -565,7 +576,7 @@ docs = ["sphinx", "jaraco.packaging (>=3.2)", "rst.linker (>=1.9)"]
 testing = ["jaraco.itertools", "func-timeout"]
 
 [metadata]
-content-hash = "6318ee5d2bca7a8ac06a683541055ac8e6be7c8c8bb62c8c1fd443f06afadc47"
+content-hash = "cf1a2cd671a207948f537aa0b005efb04395998ac6ea7e1beb8e8430ea5b6d75"
 python-versions = "^3.7"
 
 [metadata.files]
@@ -746,6 +757,9 @@ sphinx-multiversion = [
     {file = "sphinx-multiversion-0.2.3.tar.gz", hash = "sha256:e46565ac2f703f3b55652f33c159c8059865f5d13dae7f0e8403e5afc2996f5f"},
     {file = "sphinx_multiversion-0.2.3-py3-none-any.whl", hash = "sha256:dc0f18449122e3e2a61245771bfdb7fa83df4f6adbf8eafea31f5b0cfccb5dbe"},
 ]
+sphinx-multiversion-scylla = [
+    {file = "sphinx-multiversion-scylla-0.2.4.tar.gz", hash = "sha256:a44fced382c9efac454749cc3b113e971a1ad63a8901c0aebd1299d131b102b2"},
+]
 sphinx-scylladb-theme = [
     {file = "sphinx-scylladb-theme-0.1.9.tar.gz", hash = "sha256:2a2abaccedb3e00e57f412e35afda042c4a5d9baa66c288feb82362d9889294b"},
     {file = "sphinx_scylladb_theme-0.1.9-py3-none-any.whl", hash = "sha256:50b3407bb1d2432f809ca48b7b20fdb124446c622584ddc04e8a390ce59f95b4"},
diff --git a/docs/pyproject.toml b/docs/pyproject.toml
index d1cb108b855..469e0f4ad6f 100644
--- a/docs/pyproject.toml
+++ b/docs/pyproject.toml
@@ -13,7 +13,7 @@ sphinx-scylladb-theme = "0.1.9"
 sphinx-sitemap = "2.1.0"
 sphinx-autobuild = "^0.7.1"
 Sphinx = "2.4.4"
-sphinx-multiversion = "0.2.3"
+sphinx-multiversion-scylla = "^0.2.4"
 
 [tool.poetry.dev-dependencies]
 pytest = "^5.2"
diff --git a/faq/index.md b/faq/README.md
similarity index 100%
rename from faq/index.md
rename to faq/README.md
diff --git a/faq/osgi/index.md b/faq/osgi/README.md
similarity index 100%
rename from faq/osgi/index.md
rename to faq/osgi/README.md
diff --git a/manual/index.md b/manual/README.md
similarity index 100%
rename from manual/index.md
rename to manual/README.md
diff --git a/manual/address_resolution/index.md b/manual/address_resolution/README.md
similarity index 100%
rename from manual/address_resolution/index.md
rename to manual/address_resolution/README.md
diff --git a/manual/async/index.md b/manual/async/README.md
similarity index 100%
rename from manual/async/index.md
rename to manual/async/README.md
diff --git a/manual/auth/index.md b/manual/auth/README.md
similarity index 100%
rename from manual/auth/index.md
rename to manual/auth/README.md
diff --git a/manual/cloud/index.md b/manual/cloud/README.md
similarity index 100%
rename from manual/cloud/index.md
rename to manual/cloud/README.md
diff --git a/manual/compression/index.md b/manual/compression/README.md
similarity index 100%
rename from manual/compression/index.md
rename to manual/compression/README.md
diff --git a/manual/control_connection/index.md b/manual/control_connection/README.md
similarity index 100%
rename from manual/control_connection/index.md
rename to manual/control_connection/README.md
diff --git a/manual/custom_codecs/index.md b/manual/custom_codecs/README.md
similarity index 100%
rename from manual/custom_codecs/index.md
rename to manual/custom_codecs/README.md
diff --git a/manual/custom_codecs/extras/index.md b/manual/custom_codecs/extras/README.md
similarity index 100%
rename from manual/custom_codecs/extras/index.md
rename to manual/custom_codecs/extras/README.md
diff --git a/manual/custom_payloads/index.md b/manual/custom_payloads/README.md
similarity index 100%
rename from manual/custom_payloads/index.md
rename to manual/custom_payloads/README.md
diff --git a/manual/idempotence/index.md b/manual/idempotence/README.md
similarity index 100%
rename from manual/idempotence/index.md
rename to manual/idempotence/README.md
diff --git a/manual/load_balancing/index.md b/manual/load_balancing/README.md
similarity index 100%
rename from manual/load_balancing/index.md
rename to manual/load_balancing/README.md
diff --git a/manual/logging/index.md b/manual/logging/README.md
similarity index 100%
rename from manual/logging/index.md
rename to manual/logging/README.md
diff --git a/manual/metadata/index.md b/manual/metadata/README.md
similarity index 100%
rename from manual/metadata/index.md
rename to manual/metadata/README.md
diff --git a/manual/metrics/index.md b/manual/metrics/README.md
similarity index 100%
rename from manual/metrics/index.md
rename to manual/metrics/README.md
diff --git a/manual/native_protocol/index.md b/manual/native_protocol/README.md
similarity index 100%
rename from manual/native_protocol/index.md
rename to manual/native_protocol/README.md
diff --git a/manual/object_mapper/index.md b/manual/object_mapper/README.md
similarity index 100%
rename from manual/object_mapper/index.md
rename to manual/object_mapper/README.md
diff --git a/manual/object_mapper/creating/index.md b/manual/object_mapper/creating/README.md
similarity index 100%
rename from manual/object_mapper/creating/index.md
rename to manual/object_mapper/creating/README.md
diff --git a/manual/object_mapper/custom_codecs/index.md b/manual/object_mapper/custom_codecs/README.md
similarity index 100%
rename from manual/object_mapper/custom_codecs/index.md
rename to manual/object_mapper/custom_codecs/README.md
diff --git a/manual/object_mapper/using/index.md b/manual/object_mapper/using/README.md
similarity index 100%
rename from manual/object_mapper/using/index.md
rename to manual/object_mapper/using/README.md
diff --git a/manual/osgi/index.md b/manual/osgi/README.md
similarity index 100%
rename from manual/osgi/index.md
rename to manual/osgi/README.md
diff --git a/manual/paging/index.md b/manual/paging/README.md
similarity index 100%
rename from manual/paging/index.md
rename to manual/paging/README.md
diff --git a/manual/pooling/index.md b/manual/pooling/README.md
similarity index 100%
rename from manual/pooling/index.md
rename to manual/pooling/README.md
diff --git a/manual/query_timestamps/index.md b/manual/query_timestamps/README.md
similarity index 100%
rename from manual/query_timestamps/index.md
rename to manual/query_timestamps/README.md
diff --git a/manual/reconnection/index.md b/manual/reconnection/README.md
similarity index 100%
rename from manual/reconnection/index.md
rename to manual/reconnection/README.md
diff --git a/manual/retries/index.md b/manual/retries/README.md
similarity index 100%
rename from manual/retries/index.md
rename to manual/retries/README.md
diff --git a/manual/shaded_jar/index.md b/manual/shaded_jar/README.md
similarity index 100%
rename from manual/shaded_jar/index.md
rename to manual/shaded_jar/README.md
diff --git a/manual/socket_options/index.md b/manual/socket_options/README.md
similarity index 100%
rename from manual/socket_options/index.md
rename to manual/socket_options/README.md
diff --git a/manual/speculative_execution/index.md b/manual/speculative_execution/README.md
similarity index 100%
rename from manual/speculative_execution/index.md
rename to manual/speculative_execution/README.md
diff --git a/manual/ssl/index.md b/manual/ssl/README.md
similarity index 100%
rename from manual/ssl/index.md
rename to manual/ssl/README.md
diff --git a/manual/statements/index.md b/manual/statements/README.md
similarity index 100%
rename from manual/statements/index.md
rename to manual/statements/README.md
diff --git a/manual/statements/batch/index.md b/manual/statements/batch/README.md
similarity index 100%
rename from manual/statements/batch/index.md
rename to manual/statements/batch/README.md
diff --git a/manual/statements/built/index.md b/manual/statements/built/README.md
similarity index 100%
rename from manual/statements/built/index.md
rename to manual/statements/built/README.md
diff --git a/manual/statements/prepared/index.md b/manual/statements/prepared/README.md
similarity index 100%
rename from manual/statements/prepared/index.md
rename to manual/statements/prepared/README.md
diff --git a/manual/statements/simple/index.md b/manual/statements/simple/README.md
similarity index 100%
rename from manual/statements/simple/index.md
rename to manual/statements/simple/README.md
diff --git a/manual/tuples/index.md b/manual/tuples/README.md
similarity index 100%
rename from manual/tuples/index.md
rename to manual/tuples/README.md
diff --git a/manual/udts/index.md b/manual/udts/README.md
similarity index 100%
rename from manual/udts/index.md
rename to manual/udts/README.md
diff --git a/upgrade_guide/index.md b/upgrade_guide/README.md
similarity index 100%
rename from upgrade_guide/index.md
rename to upgrade_guide/README.md
diff --git a/upgrade_guide/migrating_from_astyanax/index.md b/upgrade_guide/migrating_from_astyanax/README.md
similarity index 100%
rename from upgrade_guide/migrating_from_astyanax/index.md
rename to upgrade_guide/migrating_from_astyanax/README.md
diff --git a/upgrade_guide/migrating_from_astyanax/configuration/index.md b/upgrade_guide/migrating_from_astyanax/configuration/README.md
similarity index 100%
rename from upgrade_guide/migrating_from_astyanax/configuration/index.md
rename to upgrade_guide/migrating_from_astyanax/configuration/README.md
diff --git a/upgrade_guide/migrating_from_astyanax/language_level_changes/index.md b/upgrade_guide/migrating_from_astyanax/language_level_changes/README.md
similarity index 100%
rename from upgrade_guide/migrating_from_astyanax/language_level_changes/index.md
rename to upgrade_guide/migrating_from_astyanax/language_level_changes/README.md
diff --git a/upgrade_guide/migrating_from_astyanax/queries_and_results/index.md b/upgrade_guide/migrating_from_astyanax/queries_and_results/README.md
similarity index 100%
rename from upgrade_guide/migrating_from_astyanax/queries_and_results/index.md
rename to upgrade_guide/migrating_from_astyanax/queries_and_results/README.md

From c56731cbc9a32aa2d4c50358464362538be8eabd Mon Sep 17 00:00:00 2001
From: dgarcia360 
Date: Mon, 21 Sep 2020 11:42:05 +0200
Subject: [PATCH 47/52] Fixed multiversion

---
 docs/Makefile               | 5 +----
 docs/_utils/multiversion.sh | 6 ++++++
 2 files changed, 7 insertions(+), 4 deletions(-)
 create mode 100755 docs/_utils/multiversion.sh

diff --git a/docs/Makefile b/docs/Makefile
index 73f9a79732a..dacd12da26f 100644
--- a/docs/Makefile
+++ b/docs/Makefile
@@ -70,10 +70,7 @@ linkcheck: setup
 
 .PHONY: multiversion
 multiversion: setup
-	cd .. && sphinx-multiversion docs/_source docs/$(BUILDDIR)/dirhtml  \
-		--pre-build="./_utils/doxygen.sh"  \
-		--pre-build="cp -Tr source $(SOURCE_DIR)"  \
-		--pre-build="cd $(SOURCE_DIR) && find -name README.md -execdir mv '{}' index.md ';'"
+	poetry run ./_utils/multiversion.sh
 	@echo
 	@echo "Build finished. The HTML pages are in $(BUILDDIR)/dirhtml."
 
diff --git a/docs/_utils/multiversion.sh b/docs/_utils/multiversion.sh
new file mode 100755
index 00000000000..0699e34c6cb
--- /dev/null
+++ b/docs/_utils/multiversion.sh
@@ -0,0 +1,6 @@
+#! /bin/bash	
+
+cd .. && sphinx-multiversion docs/_source docs/_build/dirhtml \
+    --pre-build="./_utils/doxygen.sh" \
+    --pre-build="cp -Tr source _source" \
+    --pre-build="cd _source && find -name README.md -execdir mv '{}' index.md ';'"
\ No newline at end of file

From 2883628cc7c5394a46f999ca313901a3b80a78ab Mon Sep 17 00:00:00 2001
From: dgarcia360 
Date: Mon, 21 Sep 2020 12:45:15 +0200
Subject: [PATCH 48/52] Fixed doxygen multiversion

---
 docs/Makefile               | 2 +-
 docs/_utils/doxygen.sh      | 8 +++-----
 docs/_utils/multiversion.sh | 7 +++----
 docs/source/conf.py         | 9 +++------
 4 files changed, 10 insertions(+), 16 deletions(-)

diff --git a/docs/Makefile b/docs/Makefile
index dacd12da26f..997841395b3 100644
--- a/docs/Makefile
+++ b/docs/Makefile
@@ -31,7 +31,7 @@ clean:
 
 .PHONY: preview
 preview: setup
-	./_utils/doxygen.sh
+	cd .. && ./docs/_utils/doxygen.sh
 	poetry run sphinx-autobuild -b dirhtml $(ALLSPHINXOPTS) $(BUILDDIR)/dirhtml --port 5500
 
 .PHONY: dirhtml
diff --git a/docs/_utils/doxygen.sh b/docs/_utils/doxygen.sh
index 906337747d3..6d47250a586 100755
--- a/docs/_utils/doxygen.sh
+++ b/docs/_utils/doxygen.sh
@@ -1,10 +1,8 @@
 #!/bin/bash
 
-OUTPUT_DIR="_build/dirhtml/api"
+OUTPUT_DIR="docs/_build/dirhtml/api"
 if [[ "$SPHINX_MULTIVERSION_OUTPUTDIR" != "" ]]; then
-    echo $SPHINX_MULTIVERSION_OUTPUTDIR
-    OUTPUT_DIR="$SPHINX_MULTIVERSION_OUTPUTDIR/api"
+    OUTPUT_DIR="docs/$SPHINX_MULTIVERSION_OUTPUTDIR/api"
 fi
-echo $OUTPUT_DIR
 mkdir -p "$OUTPUT_DIR"
-cd .. && ( cat Doxyfile | echo "HTML_OUTPUT = $OUTPUT_DIR" )  | doxygen doxyfile
+( cat doxyfile | echo "HTML_OUTPUT = $OUTPUT_DIR" )  | doxygen doxyfile
diff --git a/docs/_utils/multiversion.sh b/docs/_utils/multiversion.sh
index 0699e34c6cb..ff70afd4739 100755
--- a/docs/_utils/multiversion.sh
+++ b/docs/_utils/multiversion.sh
@@ -1,6 +1,5 @@
 #! /bin/bash	
 
-cd .. && sphinx-multiversion docs/_source docs/_build/dirhtml \
-    --pre-build="./_utils/doxygen.sh" \
-    --pre-build="cp -Tr source _source" \
-    --pre-build="cd _source && find -name README.md -execdir mv '{}' index.md ';'"
\ No newline at end of file
+cd .. && sphinx-multiversion docs/source docs/_build/dirhtml \
+    --pre-build './docs/_utils/doxygen.sh' \
+    --pre-build "find docs/source -name README.md -execdir mv '{}' index.md ';'"
\ No newline at end of file
diff --git a/docs/source/conf.py b/docs/source/conf.py
index 8a6982b3137..f42383899fa 100644
--- a/docs/source/conf.py
+++ b/docs/source/conf.py
@@ -43,8 +43,6 @@ def write_html_redirect(redirect_to):
 def create_redirects(app, docname):
     if not os.path.exists(redirects_file):
         return
-    if not app.builder.name == 'dirhtml':
-        return
     with open(redirects_file, 'r') as yaml_file:
         for from_path, redirect_to in yaml.full_load(yaml_file).items():
             target_path = app.outdir + '/' + from_path
@@ -274,12 +272,11 @@ class DitaaLexer(BashLexer):
 
 # -- Options for multiversion --------------------------------------------
 # Whitelist pattern for tags (set to None to ignore all tags)
-smv_tag_whitelist = 'None'
 # Whitelist pattern for branches (set to None to ignore all branches)
-smv_branch_whitelist = r"^latest$"
-
+smv_tag_whitelist = r'None'
+smv_branch_whitelist = r'^docs-automation$'
 # Whitelist pattern for remotes (set to None to use local branches only)
-smv_remote_whitelist = r"^origin$"
+smv_remote_whitelist = None
 # Pattern for released versions
 smv_released_pattern = r'^tags/.*$'
 # Format for versioned output directories inside the build directory

From ec0148dd19e4f0c18f6cb11e97d22e96099dcf93 Mon Sep 17 00:00:00 2001
From: GitHub Action 
Date: Mon, 21 Sep 2020 13:45:35 +0200
Subject: [PATCH 49/52] Fixed redirection multiversion

---
 docs/_utils/doxygen.sh | 5 +++--
 docs/source/conf.py    | 7 +++++--
 2 files changed, 8 insertions(+), 4 deletions(-)

diff --git a/docs/_utils/doxygen.sh b/docs/_utils/doxygen.sh
index 6d47250a586..095ea787e71 100755
--- a/docs/_utils/doxygen.sh
+++ b/docs/_utils/doxygen.sh
@@ -2,7 +2,8 @@
 
 OUTPUT_DIR="docs/_build/dirhtml/api"
 if [[ "$SPHINX_MULTIVERSION_OUTPUTDIR" != "" ]]; then
-    OUTPUT_DIR="docs/$SPHINX_MULTIVERSION_OUTPUTDIR/api"
+    OUTPUT_DIR="$SPHINX_MULTIVERSION_OUTPUTDIR/api"
+    echo "HTML_OUTPUT = $OUTPUT_DIR" >> doxyfile
 fi
 mkdir -p "$OUTPUT_DIR"
-( cat doxyfile | echo "HTML_OUTPUT = $OUTPUT_DIR" )  | doxygen doxyfile
+doxygen doxyfile
\ No newline at end of file
diff --git a/docs/source/conf.py b/docs/source/conf.py
index f42383899fa..a504e5e32f4 100644
--- a/docs/source/conf.py
+++ b/docs/source/conf.py
@@ -41,9 +41,12 @@ def write_html_redirect(redirect_to):
 redirects_file = "_utils/redirections.yaml"
 
 def create_redirects(app, docname):
-    if not os.path.exists(redirects_file):
+    redirects = redirects_file
+    if os.path.exists('docs'):
+        redirects = 'docs/' + redirects_file
+    if not os.path.exists(redirects):
         return
-    with open(redirects_file, 'r') as yaml_file:
+    with open(redirects, 'r') as yaml_file:
         for from_path, redirect_to in yaml.full_load(yaml_file).items():
             target_path = app.outdir + '/' + from_path
             if not os.path.exists(target_path):

From 848fe5e973d5b4dfe2793e15337e5b8960307ae7 Mon Sep 17 00:00:00 2001
From: GitHub Action 
Date: Mon, 21 Sep 2020 13:49:03 +0200
Subject: [PATCH 50/52] Fixed conf.py

---
 docs/source/conf.py | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)

diff --git a/docs/source/conf.py b/docs/source/conf.py
index a504e5e32f4..bbe616c9756 100644
--- a/docs/source/conf.py
+++ b/docs/source/conf.py
@@ -277,9 +277,9 @@ class DitaaLexer(BashLexer):
 # Whitelist pattern for tags (set to None to ignore all tags)
 # Whitelist pattern for branches (set to None to ignore all branches)
 smv_tag_whitelist = r'None'
-smv_branch_whitelist = r'^docs-automation$'
+smv_branch_whitelist = r"^latest$"
 # Whitelist pattern for remotes (set to None to use local branches only)
-smv_remote_whitelist = None
+smv_remote_whitelist = r"^origin$"
 # Pattern for released versions
 smv_released_pattern = r'^tags/.*$'
 # Format for versioned output directories inside the build directory

From 8d3be899d15f682dde007194ad414da57b260bbe Mon Sep 17 00:00:00 2001
From: dgarcia360 
Date: Mon, 21 Sep 2020 14:14:23 +0200
Subject: [PATCH 51/52] Fixed multiversion redirects

---
 docs/source/conf.py | 5 +++++
 1 file changed, 5 insertions(+)

diff --git a/docs/source/conf.py b/docs/source/conf.py
index bbe616c9756..53e731f943e 100644
--- a/docs/source/conf.py
+++ b/docs/source/conf.py
@@ -49,10 +49,15 @@ def create_redirects(app, docname):
     with open(redirects, 'r') as yaml_file:
         for from_path, redirect_to in yaml.full_load(yaml_file).items():
             target_path = app.outdir + '/' + from_path
+            if os.getenv("SPHINX_MULTIVERSION_NAME") is not None:
+                redirect_to = '/' + os.environ['SPHINX_MULTIVERSION_NAME'] + redirect_to
             if not os.path.exists(target_path):
                 os.makedirs(target_path)
             with open(os.path.join(target_path + '/index.html'), 'w') as t_file:
                 t_file.write(write_html_redirect(redirect_to))
+            with open(os.path.join(target_path + '.html'), 'w') as t_file:
+                t_file.write(write_html_redirect(redirect_to))
+
 
 # -- General configuration ------------------------------------------------
 

From 032009941bc830db23625c7a0f3a5a799efc6a83 Mon Sep 17 00:00:00 2001
From: dgarcia360 
Date: Mon, 21 Sep 2020 14:16:40 +0200
Subject: [PATCH 52/52] Fixed spaces

---
 docs/_utils/doxygen.sh      | 2 +-
 docs/_utils/multiversion.sh | 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)

diff --git a/docs/_utils/doxygen.sh b/docs/_utils/doxygen.sh
index 095ea787e71..d739cf81997 100755
--- a/docs/_utils/doxygen.sh
+++ b/docs/_utils/doxygen.sh
@@ -6,4 +6,4 @@ if [[ "$SPHINX_MULTIVERSION_OUTPUTDIR" != "" ]]; then
     echo "HTML_OUTPUT = $OUTPUT_DIR" >> doxyfile
 fi
 mkdir -p "$OUTPUT_DIR"
-doxygen doxyfile
\ No newline at end of file
+doxygen doxyfile
diff --git a/docs/_utils/multiversion.sh b/docs/_utils/multiversion.sh
index ff70afd4739..ae56ca3574e 100755
--- a/docs/_utils/multiversion.sh
+++ b/docs/_utils/multiversion.sh
@@ -2,4 +2,4 @@
 
 cd .. && sphinx-multiversion docs/source docs/_build/dirhtml \
     --pre-build './docs/_utils/doxygen.sh' \
-    --pre-build "find docs/source -name README.md -execdir mv '{}' index.md ';'"
\ No newline at end of file
+    --pre-build "find docs/source -name README.md -execdir mv '{}' index.md ';'"