diff --git a/driver/src/main/java/org/neo4j/driver/Driver.java b/driver/src/main/java/org/neo4j/driver/Driver.java index af732035ce..64238792cb 100644 --- a/driver/src/main/java/org/neo4j/driver/Driver.java +++ b/driver/src/main/java/org/neo4j/driver/Driver.java @@ -194,4 +194,17 @@ public interface Driver extends AutoCloseable * @return a {@link CompletionStage completion stage} that represents the asynchronous verification. */ CompletionStage verifyConnectivityAsync(); + + /** + * Returns true if the server or cluster the driver connects to supports multi-databases, otherwise false. + * @return true if the server or cluster the driver connects to supports multi-databases, otherwise false. + */ + boolean supportsMultiDb(); + + /** + * Asynchronous check if the server or cluster the driver connects to supports multi-databases. + * @return a {@link CompletionStage completion stage} that returns true if the server or cluster + * the driver connects to supports multi-databases, otherwise false. + */ + CompletionStage supportsMultiDbAsync(); } diff --git a/driver/src/main/java/org/neo4j/driver/internal/DirectConnectionProvider.java b/driver/src/main/java/org/neo4j/driver/internal/DirectConnectionProvider.java index bf87f68ed1..17cd5ff3d9 100644 --- a/driver/src/main/java/org/neo4j/driver/internal/DirectConnectionProvider.java +++ b/driver/src/main/java/org/neo4j/driver/internal/DirectConnectionProvider.java @@ -26,7 +26,7 @@ import org.neo4j.driver.internal.spi.ConnectionPool; import org.neo4j.driver.internal.spi.ConnectionProvider; -import static org.neo4j.driver.internal.async.ImmutableConnectionContext.simple; +import static org.neo4j.driver.internal.messaging.request.MultiDatabaseUtil.supportsMultiDatabase; /** * Simple {@link ConnectionProvider connection provider} that obtains connections form the given pool only for @@ -46,15 +46,13 @@ public class DirectConnectionProvider implements ConnectionProvider @Override public CompletionStage acquireConnection( ConnectionContext context ) { - return connectionPool.acquire( address ).thenApply( connection -> new DirectConnection( connection, context.databaseName(), context.mode() ) ); + return acquireConnection().thenApply( connection -> new DirectConnection( connection, context.databaseName(), context.mode() ) ); } @Override public CompletionStage verifyConnectivity() { - // We verify the connection by establishing a connection with the remote server specified by the address. - // Connection context will be ignored as no query is run in this connection and the connection is released immediately. - return acquireConnection( simple() ).thenCompose( Connection::release ); + return acquireConnection().thenCompose( Connection::release ); } @Override @@ -63,8 +61,26 @@ public CompletionStage close() return connectionPool.close(); } + @Override + public CompletionStage supportsMultiDb() + { + return acquireConnection().thenCompose( conn -> { + boolean supportsMultiDatabase = supportsMultiDatabase( conn ); + return conn.release().thenApply( ignored -> supportsMultiDatabase ); + } ); + } + public BoltServerAddress getAddress() { return address; } + + /** + * Used only for grabbing a connection with the server after hello message. + * This connection cannot be directly used for running any queries as it is missing necessary connection context + */ + private CompletionStage acquireConnection() + { + return connectionPool.acquire( address ); + } } diff --git a/driver/src/main/java/org/neo4j/driver/internal/InternalDriver.java b/driver/src/main/java/org/neo4j/driver/internal/InternalDriver.java index 05fa2b8c65..792c114664 100644 --- a/driver/src/main/java/org/neo4j/driver/internal/InternalDriver.java +++ b/driver/src/main/java/org/neo4j/driver/internal/InternalDriver.java @@ -141,6 +141,18 @@ public CompletionStage verifyConnectivityAsync() return sessionFactory.verifyConnectivity(); } + @Override + public boolean supportsMultiDb() + { + return Futures.blockingGet( supportsMultiDbAsync() ); + } + + @Override + public CompletionStage supportsMultiDbAsync() + { + return sessionFactory.supportsMultiDb(); + } + @Override public void verifyConnectivity() { diff --git a/driver/src/main/java/org/neo4j/driver/internal/SessionFactory.java b/driver/src/main/java/org/neo4j/driver/internal/SessionFactory.java index 60c6c72c43..61c0d33566 100644 --- a/driver/src/main/java/org/neo4j/driver/internal/SessionFactory.java +++ b/driver/src/main/java/org/neo4j/driver/internal/SessionFactory.java @@ -30,4 +30,6 @@ public interface SessionFactory CompletionStage verifyConnectivity(); CompletionStage close(); + + CompletionStage supportsMultiDb(); } diff --git a/driver/src/main/java/org/neo4j/driver/internal/SessionFactoryImpl.java b/driver/src/main/java/org/neo4j/driver/internal/SessionFactoryImpl.java index 46d8d1da04..201cfc36a5 100644 --- a/driver/src/main/java/org/neo4j/driver/internal/SessionFactoryImpl.java +++ b/driver/src/main/java/org/neo4j/driver/internal/SessionFactoryImpl.java @@ -79,6 +79,12 @@ public CompletionStage close() return connectionProvider.close(); } + @Override + public CompletionStage supportsMultiDb() + { + return connectionProvider.supportsMultiDb(); + } + /** * Get the underlying connection provider. *

diff --git a/driver/src/main/java/org/neo4j/driver/internal/async/ImmutableConnectionContext.java b/driver/src/main/java/org/neo4j/driver/internal/async/ImmutableConnectionContext.java index e031fdb3bd..633d9cf20a 100644 --- a/driver/src/main/java/org/neo4j/driver/internal/async/ImmutableConnectionContext.java +++ b/driver/src/main/java/org/neo4j/driver/internal/async/ImmutableConnectionContext.java @@ -24,6 +24,7 @@ import org.neo4j.driver.internal.spi.Connection; import static org.neo4j.driver.internal.DatabaseNameUtil.defaultDatabase; +import static org.neo4j.driver.internal.DatabaseNameUtil.systemDatabase; import static org.neo4j.driver.internal.InternalBookmark.empty; /** @@ -31,7 +32,8 @@ */ public class ImmutableConnectionContext implements ConnectionContext { - private static final ConnectionContext SIMPLE = new ImmutableConnectionContext( defaultDatabase(), empty(), AccessMode.READ ); + private static final ConnectionContext SINGLE_DB_CONTEXT = new ImmutableConnectionContext( defaultDatabase(), empty(), AccessMode.READ ); + private static final ConnectionContext MULTI_DB_CONTEXT = new ImmutableConnectionContext( systemDatabase(), empty(), AccessMode.READ ); private final DatabaseName databaseName; private final AccessMode mode; @@ -65,10 +67,10 @@ public Bookmark rediscoveryBookmark() /** * A simple context is used to test connectivity with a remote server/cluster. * As long as there is a read only service, the connection shall be established successfully. - * This context should be applicable for both bolt v4 and bolt v3 routing table rediscovery. + * Depending on whether multidb is supported or not, this method returns different context for routing table discovery. */ - public static ConnectionContext simple() + public static ConnectionContext simple( boolean supportsMultiDb ) { - return SIMPLE; + return supportsMultiDb ? MULTI_DB_CONTEXT : SINGLE_DB_CONTEXT; } } diff --git a/driver/src/main/java/org/neo4j/driver/internal/cluster/Rediscovery.java b/driver/src/main/java/org/neo4j/driver/internal/cluster/Rediscovery.java index c3c39ff5b0..0e0e47a3cf 100644 --- a/driver/src/main/java/org/neo4j/driver/internal/cluster/Rediscovery.java +++ b/driver/src/main/java/org/neo4j/driver/internal/cluster/Rediscovery.java @@ -18,12 +18,16 @@ */ package org.neo4j.driver.internal.cluster; +import java.util.List; import java.util.concurrent.CompletionStage; import org.neo4j.driver.Bookmark; +import org.neo4j.driver.internal.BoltServerAddress; import org.neo4j.driver.internal.spi.ConnectionPool; public interface Rediscovery { CompletionStage lookupClusterComposition( RoutingTable routingTable, ConnectionPool connectionPool, Bookmark bookmark ); + + List resolve(); } diff --git a/driver/src/main/java/org/neo4j/driver/internal/cluster/RediscoveryImpl.java b/driver/src/main/java/org/neo4j/driver/internal/cluster/RediscoveryImpl.java index 0c294de926..21b35d0434 100644 --- a/driver/src/main/java/org/neo4j/driver/internal/cluster/RediscoveryImpl.java +++ b/driver/src/main/java/org/neo4j/driver/internal/cluster/RediscoveryImpl.java @@ -198,7 +198,7 @@ private CompletionStage lookupOnInitialRouter( RoutingTable List addresses; try { - addresses = resolve( initialRouter ); + addresses = resolve(); } catch ( Throwable error ) { @@ -256,9 +256,10 @@ private ClusterComposition handleRoutingProcedureError( Throwable error, Routing return null; } - private List resolve( BoltServerAddress address ) + @Override + public List resolve() { - return resolver.resolve( address ) + return resolver.resolve( initialRouter ) .stream() .flatMap( resolved -> resolveAll( BoltServerAddress.from( resolved ) ) ) .collect( toList() ); // collect to list to preserve the order diff --git a/driver/src/main/java/org/neo4j/driver/internal/cluster/RoutingProcedureClusterCompositionProvider.java b/driver/src/main/java/org/neo4j/driver/internal/cluster/RoutingProcedureClusterCompositionProvider.java index 1dbbefaac1..8e024adc0a 100644 --- a/driver/src/main/java/org/neo4j/driver/internal/cluster/RoutingProcedureClusterCompositionProvider.java +++ b/driver/src/main/java/org/neo4j/driver/internal/cluster/RoutingProcedureClusterCompositionProvider.java @@ -23,16 +23,16 @@ import java.util.concurrent.CompletionStage; import org.neo4j.driver.Bookmark; -import org.neo4j.driver.Record; import org.neo4j.driver.Query; +import org.neo4j.driver.Record; import org.neo4j.driver.exceptions.ProtocolException; import org.neo4j.driver.exceptions.value.ValueException; import org.neo4j.driver.internal.DatabaseName; import org.neo4j.driver.internal.spi.Connection; import org.neo4j.driver.internal.util.Clock; -import org.neo4j.driver.internal.util.ServerVersion; import static java.lang.String.format; +import static org.neo4j.driver.internal.messaging.request.MultiDatabaseUtil.supportsMultiDatabase; public class RoutingProcedureClusterCompositionProvider implements ClusterCompositionProvider { @@ -59,7 +59,7 @@ public RoutingProcedureClusterCompositionProvider( Clock clock, RoutingContext r public CompletionStage getClusterComposition( Connection connection, DatabaseName databaseName, Bookmark bookmark ) { RoutingProcedureRunner runner; - if ( connection.serverVersion().greaterThanOrEqual( ServerVersion.v4_0_0 ) ) + if ( supportsMultiDatabase( connection ) ) { runner = multiDatabaseRoutingProcedureRunner; } diff --git a/driver/src/main/java/org/neo4j/driver/internal/cluster/loadbalancing/LoadBalancer.java b/driver/src/main/java/org/neo4j/driver/internal/cluster/loadbalancing/LoadBalancer.java index bdc8a15c01..2d7c00af70 100644 --- a/driver/src/main/java/org/neo4j/driver/internal/cluster/loadbalancing/LoadBalancer.java +++ b/driver/src/main/java/org/neo4j/driver/internal/cluster/loadbalancing/LoadBalancer.java @@ -20,12 +20,14 @@ import io.netty.util.concurrent.EventExecutorGroup; +import java.util.List; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletionStage; import org.neo4j.driver.AccessMode; import org.neo4j.driver.Logger; import org.neo4j.driver.Logging; +import org.neo4j.driver.exceptions.SecurityException; import org.neo4j.driver.exceptions.ServiceUnavailableException; import org.neo4j.driver.exceptions.SessionExpiredException; import org.neo4j.driver.internal.BoltServerAddress; @@ -49,6 +51,11 @@ import static java.lang.String.format; import static org.neo4j.driver.internal.async.ImmutableConnectionContext.simple; +import static org.neo4j.driver.internal.messaging.request.MultiDatabaseUtil.supportsMultiDatabase; +import static org.neo4j.driver.internal.util.Futures.completedWithNull; +import static org.neo4j.driver.internal.util.Futures.completionExceptionCause; +import static org.neo4j.driver.internal.util.Futures.failedFuture; +import static org.neo4j.driver.internal.util.Futures.onErrorContinue; public class LoadBalancer implements ConnectionProvider { @@ -58,20 +65,29 @@ public class LoadBalancer implements ConnectionProvider private final LoadBalancingStrategy loadBalancingStrategy; private final EventExecutorGroup eventExecutorGroup; private final Logger log; + private final Rediscovery rediscovery; public LoadBalancer( BoltServerAddress initialRouter, RoutingSettings settings, ConnectionPool connectionPool, EventExecutorGroup eventExecutorGroup, Clock clock, Logging logging, LoadBalancingStrategy loadBalancingStrategy, ServerAddressResolver resolver ) { - this( connectionPool, createRoutingTables( connectionPool, eventExecutorGroup, initialRouter, resolver, settings, clock, logging ), - loadBalancerLogger( logging ), loadBalancingStrategy, eventExecutorGroup ); + this( connectionPool, createRediscovery( eventExecutorGroup, initialRouter, resolver, settings, clock, logging ), settings, loadBalancingStrategy, + eventExecutorGroup, clock, loadBalancerLogger( logging ) ); } - LoadBalancer( ConnectionPool connectionPool, RoutingTableRegistry routingTables, Logger log, LoadBalancingStrategy loadBalancingStrategy, - EventExecutorGroup eventExecutorGroup ) + private LoadBalancer( ConnectionPool connectionPool, Rediscovery rediscovery, RoutingSettings settings, LoadBalancingStrategy loadBalancingStrategy, + EventExecutorGroup eventExecutorGroup, Clock clock, Logger log ) + { + this( connectionPool, createRoutingTables( connectionPool, rediscovery, settings, clock, log ), rediscovery, loadBalancingStrategy, eventExecutorGroup, + log ); + } + + LoadBalancer( ConnectionPool connectionPool, RoutingTableRegistry routingTables, Rediscovery rediscovery, LoadBalancingStrategy loadBalancingStrategy, + EventExecutorGroup eventExecutorGroup, Logger log ) { this.connectionPool = connectionPool; this.routingTables = routingTables; + this.rediscovery = rediscovery; this.loadBalancingStrategy = loadBalancingStrategy; this.eventExecutorGroup = eventExecutorGroup; this.log = log; @@ -88,14 +104,15 @@ public CompletionStage acquireConnection( ConnectionContext context @Override public CompletionStage verifyConnectivity() { - return routingTables.refreshRoutingTable( simple() ).handle( ( ignored, error ) -> { + return this.supportsMultiDb().thenCompose( supports -> routingTables.refreshRoutingTable( simple( supports ) ) ).handle( ( ignored, error ) -> { if ( error != null ) { - Throwable cause = Futures.completionExceptionCause( error ); + Throwable cause = completionExceptionCause( error ); if ( cause instanceof ServiceUnavailableException ) { throw Futures.asCompletionException( new ServiceUnavailableException( - "Unable to connect to database, ensure the database is running and that there is a working network connection to it.", cause ) ); + "Unable to connect to database management service, ensure the database is running and that there is a working network connection to it.", + cause ) ); } throw Futures.asCompletionException( cause ); } @@ -109,6 +126,54 @@ public CompletionStage close() return connectionPool.close(); } + @Override + public CompletionStage supportsMultiDb() + { + List addresses; + + try + { + addresses = rediscovery.resolve(); + } + catch ( Throwable error ) + { + return failedFuture( error ); + } + + CompletableFuture result = completedWithNull(); + Throwable baseError = new ServiceUnavailableException( "Failed to perform multi-databases feature detection with the following servers: " + addresses ); + + for ( BoltServerAddress address : addresses ) + { + result = onErrorContinue( result, baseError, completionError -> { + // We fail fast on security errors + Throwable error = completionExceptionCause( completionError ); + if ( error instanceof SecurityException ) + { + return failedFuture( error ); + } + return supportsMultiDb( address ); + } ); + } + return onErrorContinue( result, baseError, completionError -> { + // If we failed with security errors, then we rethrow the security error out, otherwise we throw the chained errors. + Throwable error = completionExceptionCause( completionError ); + if ( error instanceof SecurityException ) + { + return failedFuture( error ); + } + return failedFuture( baseError ); + } ); + } + + private CompletionStage supportsMultiDb( BoltServerAddress address ) + { + return connectionPool.acquire( address ).thenCompose( conn -> { + boolean supportsMultiDatabase = supportsMultiDatabase( conn ); + return conn.release().thenApply( ignored -> supportsMultiDatabase ); + } ); + } + private CompletionStage acquire( AccessMode mode, RoutingTable routingTable ) { AddressSet addresses = addressSet( mode, routingTable ); @@ -131,7 +196,7 @@ private void acquire( AccessMode mode, RoutingTable routingTable, AddressSet add connectionPool.acquire( address ).whenComplete( ( connection, completionError ) -> { - Throwable error = Futures.completionExceptionCause( completionError ); + Throwable error = completionExceptionCause( completionError ); if ( error != null ) { if ( error instanceof ServiceUnavailableException ) @@ -181,17 +246,16 @@ private BoltServerAddress selectAddress( AccessMode mode, AddressSet servers ) } } - private static RoutingTableRegistry createRoutingTables( ConnectionPool connectionPool, EventExecutorGroup eventExecutorGroup, BoltServerAddress initialRouter, - ServerAddressResolver resolver, RoutingSettings settings, Clock clock, Logging logging ) + private static RoutingTableRegistry createRoutingTables( ConnectionPool connectionPool, Rediscovery rediscovery, RoutingSettings settings, Clock clock, + Logger log ) { - Logger log = loadBalancerLogger( logging ); - Rediscovery rediscovery = createRediscovery( eventExecutorGroup, initialRouter, resolver, settings, clock, log ); return new RoutingTableRegistryImpl( connectionPool, rediscovery, clock, log, settings.routingTablePurgeDelayMs() ); } private static Rediscovery createRediscovery( EventExecutorGroup eventExecutorGroup, BoltServerAddress initialRouter, ServerAddressResolver resolver, - RoutingSettings settings, Clock clock, Logger log ) + RoutingSettings settings, Clock clock, Logging logging ) { + Logger log = loadBalancerLogger( logging ); ClusterCompositionProvider clusterCompositionProvider = new RoutingProcedureClusterCompositionProvider( clock, settings.routingContext() ); return new RediscoveryImpl( initialRouter, settings, clusterCompositionProvider, eventExecutorGroup, resolver, log ); } diff --git a/driver/src/main/java/org/neo4j/driver/internal/messaging/request/MultiDatabaseUtil.java b/driver/src/main/java/org/neo4j/driver/internal/messaging/request/MultiDatabaseUtil.java index 47b56bcce7..6710f0909e 100644 --- a/driver/src/main/java/org/neo4j/driver/internal/messaging/request/MultiDatabaseUtil.java +++ b/driver/src/main/java/org/neo4j/driver/internal/messaging/request/MultiDatabaseUtil.java @@ -20,6 +20,9 @@ import org.neo4j.driver.exceptions.ClientException; import org.neo4j.driver.internal.DatabaseName; +import org.neo4j.driver.internal.messaging.v4.BoltProtocolV4; +import org.neo4j.driver.internal.spi.Connection; +import org.neo4j.driver.internal.util.ServerVersion; public final class MultiDatabaseUtil { @@ -31,4 +34,9 @@ public static void assertEmptyDatabaseName( DatabaseName databaseName, int boltV "Database name: '%s'", boltVersion, databaseName.description() ) ); } } + + public static boolean supportsMultiDatabase( Connection connection ) + { + return connection.serverVersion().greaterThanOrEqual( ServerVersion.v4_0_0 ) && connection.protocol().version() >= BoltProtocolV4.VERSION; + } } diff --git a/driver/src/main/java/org/neo4j/driver/internal/spi/ConnectionProvider.java b/driver/src/main/java/org/neo4j/driver/internal/spi/ConnectionProvider.java index e9224da102..f94ef354a1 100644 --- a/driver/src/main/java/org/neo4j/driver/internal/spi/ConnectionProvider.java +++ b/driver/src/main/java/org/neo4j/driver/internal/spi/ConnectionProvider.java @@ -36,4 +36,6 @@ public interface ConnectionProvider CompletionStage verifyConnectivity(); CompletionStage close(); + + CompletionStage supportsMultiDb(); } diff --git a/driver/src/main/java/org/neo4j/driver/internal/util/Futures.java b/driver/src/main/java/org/neo4j/driver/internal/util/Futures.java index 5cbf96aa8c..be48778585 100644 --- a/driver/src/main/java/org/neo4j/driver/internal/util/Futures.java +++ b/driver/src/main/java/org/neo4j/driver/internal/util/Futures.java @@ -18,6 +18,7 @@ */ package org.neo4j.driver.internal.util; +import java.util.Objects; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletionException; import java.util.concurrent.CompletionStage; @@ -25,6 +26,7 @@ import java.util.concurrent.Future; import java.util.function.BiConsumer; import java.util.function.BiFunction; +import java.util.function.Function; import org.neo4j.driver.internal.async.connection.EventLoopGroupFactory; @@ -220,6 +222,52 @@ else if ( error2 != null ) } } + /** + * Given a future, if the future completes successfully then return a new completed future with the completed value. + * Otherwise if the future completes with an error, then this method first saves the error in the error recorder, and then continues with the onErrorAction. + * @param future the future. + * @param errorRecorder saves error if the given future completes with an error. + * @param onErrorAction continues the future with this action if the future completes with an error. + * @param type + * @return a new completed future with the same completed value if the given future completes successfully, otherwise continues with the onErrorAction. + */ + @SuppressWarnings( "ThrowableNotThrown" ) + public static CompletableFuture onErrorContinue( CompletableFuture future, Throwable errorRecorder, + Function> onErrorAction ) + { + Objects.requireNonNull( future ); + return future.handle( ( value, error ) -> { + if ( error != null ) + { + // record error + Futures.combineErrors( errorRecorder, error ); + return new CompletionResult( null, error ); + } + return new CompletionResult<>( value, null ); + } ).thenCompose( result -> { + if ( result.value != null ) + { + return completedFuture( result.value ); + } + else + { + return onErrorAction.apply( result.error ); + } + } ); + } + + private static class CompletionResult + { + T value; + Throwable error; + + CompletionResult( T value, Throwable error ) + { + this.value = value; + this.error = error; + } + } + private static void safeRun( Runnable runnable ) { try diff --git a/driver/src/test/java/org/neo4j/driver/integration/RoutingDriverBoltKitTest.java b/driver/src/test/java/org/neo4j/driver/integration/RoutingDriverBoltKitTest.java index 8003012d45..8348712041 100644 --- a/driver/src/test/java/org/neo4j/driver/integration/RoutingDriverBoltKitTest.java +++ b/driver/src/test/java/org/neo4j/driver/integration/RoutingDriverBoltKitTest.java @@ -1155,6 +1155,34 @@ void shouldRevertToInitialRouterIfKnownRouterThrowsProtocolErrors() throws Excep } } + @Test + void shouldServerWithBoltV4SupportMultiDb() throws Throwable + { + StubServer server = StubServer.start( "support_multidb_v4.script", 9001 ); + try ( Driver driver = GraphDatabase.driver( "neo4j://localhost:9001", INSECURE_CONFIG ) ) + { + assertTrue( driver.supportsMultiDb() ); + } + finally + { + assertEquals( 0, server.exitStatus() ); + } + } + + @Test + void shouldServerWithBoltV3NotSupportMultiDb() throws Throwable + { + StubServer server = StubServer.start( "support_multidb_v3.script", 9001 ); + try ( Driver driver = GraphDatabase.driver( "neo4j://localhost:9001", INSECURE_CONFIG ) ) + { + assertFalse( driver.supportsMultiDb() ); + } + finally + { + assertEquals( 0, server.exitStatus() ); + } + } + private static Driver newDriverWithSleeplessClock( String uriString, Config config ) { DriverFactory driverFactory = new DriverFactoryWithClock( new SleeplessClock() ); diff --git a/driver/src/test/java/org/neo4j/driver/integration/RoutingDriverMultidatabaseBoltKitTest.java b/driver/src/test/java/org/neo4j/driver/integration/RoutingDriverMultidatabaseBoltKitTest.java index ed78415c40..cb87e1e72a 100644 --- a/driver/src/test/java/org/neo4j/driver/integration/RoutingDriverMultidatabaseBoltKitTest.java +++ b/driver/src/test/java/org/neo4j/driver/integration/RoutingDriverMultidatabaseBoltKitTest.java @@ -155,7 +155,7 @@ void shouldBeAbleToServeReachableDatabase() throws IOException, InterruptedExcep @Test - void shouldVerifyConnectivityOnDriverCreation() throws Throwable + void shouldDriverVerifyConnectivity() throws Throwable { StubServer router = StubServer.start( "acquire_endpoints_v4_verify_connectivity.script", 9001 ); StubServer readServer = StubServer.start( "read_server_v4_read.script", 9005 ); diff --git a/driver/src/test/java/org/neo4j/driver/internal/DirectDriverBoltKitTest.java b/driver/src/test/java/org/neo4j/driver/internal/DirectDriverBoltKitTest.java index 68b66c7123..c8a68ebab4 100644 --- a/driver/src/test/java/org/neo4j/driver/internal/DirectDriverBoltKitTest.java +++ b/driver/src/test/java/org/neo4j/driver/internal/DirectDriverBoltKitTest.java @@ -56,6 +56,7 @@ import static org.hamcrest.core.IsEqual.equalTo; import static org.hamcrest.junit.MatcherAssert.assertThat; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -440,6 +441,34 @@ void shouldDiscardIfPullNotFinished() throws Throwable } } + @Test + void shouldServerWithBoltV4SupportMultiDb() throws Throwable + { + StubServer server = StubServer.start( "support_multidb_v4.script", 9001 ); + try ( Driver driver = GraphDatabase.driver( "bolt://localhost:9001", INSECURE_CONFIG ) ) + { + assertTrue( driver.supportsMultiDb() ); + } + finally + { + assertEquals( 0, server.exitStatus() ); + } + } + + @Test + void shouldServerWithBoltV3NotSupportMultiDb() throws Throwable + { + StubServer server = StubServer.start( "support_multidb_v3.script", 9001 ); + try ( Driver driver = GraphDatabase.driver( "bolt://localhost:9001", INSECURE_CONFIG ) ) + { + assertFalse( driver.supportsMultiDb() ); + } + finally + { + assertEquals( 0, server.exitStatus() ); + } + } + private static void testTxCloseErrorPropagation( String script, Consumer txAction, String expectedErrorMessage ) throws Exception { diff --git a/driver/src/test/java/org/neo4j/driver/internal/cluster/RoutingProcedureClusterCompositionProviderTest.java b/driver/src/test/java/org/neo4j/driver/internal/cluster/RoutingProcedureClusterCompositionProviderTest.java index 56f51ee800..924b93663a 100644 --- a/driver/src/test/java/org/neo4j/driver/internal/cluster/RoutingProcedureClusterCompositionProviderTest.java +++ b/driver/src/test/java/org/neo4j/driver/internal/cluster/RoutingProcedureClusterCompositionProviderTest.java @@ -34,6 +34,8 @@ import org.neo4j.driver.internal.DatabaseName; import org.neo4j.driver.internal.InternalBookmark; import org.neo4j.driver.internal.InternalRecord; +import org.neo4j.driver.internal.messaging.v3.BoltProtocolV3; +import org.neo4j.driver.internal.messaging.v4.BoltProtocolV4; import org.neo4j.driver.internal.spi.Connection; import org.neo4j.driver.internal.util.Clock; import org.neo4j.driver.internal.util.ServerVersion; @@ -304,18 +306,21 @@ private static RoutingProcedureResponse newRoutingResponse( Throwable error ) private static RoutingProcedureClusterCompositionProvider newClusterCompositionProvider( RoutingProcedureRunner runner, Connection connection ) { when( connection.serverVersion() ).thenReturn( ServerVersion.v3_5_0 ); + when( connection.protocol() ).thenReturn( BoltProtocolV3.INSTANCE ); return new RoutingProcedureClusterCompositionProvider( mock( Clock.class ), runner, newMultiDBProcedureRunnerMock() ); } private static RoutingProcedureClusterCompositionProvider newClusterCompositionProvider( MultiDatabasesRoutingProcedureRunner runner, Connection connection ) { when( connection.serverVersion() ).thenReturn( ServerVersion.v4_0_0 ); + when( connection.protocol() ).thenReturn( BoltProtocolV4.INSTANCE ); return new RoutingProcedureClusterCompositionProvider( mock( Clock.class ), newProcedureRunnerMock(), runner ); } private static RoutingProcedureClusterCompositionProvider newClusterCompositionProvider( MultiDatabasesRoutingProcedureRunner runner, Connection connection, Clock clock ) { when( connection.serverVersion() ).thenReturn( ServerVersion.v4_0_0 ); + when( connection.protocol() ).thenReturn( BoltProtocolV4.INSTANCE ); return new RoutingProcedureClusterCompositionProvider( clock, newProcedureRunnerMock(), runner ); } } diff --git a/driver/src/test/java/org/neo4j/driver/internal/cluster/RoutingTableHandlerTest.java b/driver/src/test/java/org/neo4j/driver/internal/cluster/RoutingTableHandlerTest.java index 86f2b61360..c4dcbb99c9 100644 --- a/driver/src/test/java/org/neo4j/driver/internal/cluster/RoutingTableHandlerTest.java +++ b/driver/src/test/java/org/neo4j/driver/internal/cluster/RoutingTableHandlerTest.java @@ -115,7 +115,7 @@ void acquireShouldUpdateRoutingTableWhenKnownRoutingTableIsStale() RoutingTableHandler handler = newRoutingTableHandler( routingTable, rediscovery, connectionPool ); - assertNotNull( await( handler.refreshRoutingTable( simple() ) ) ); + assertNotNull( await( handler.refreshRoutingTable( simple( false ) ) ) ); verify( rediscovery ).lookupClusterComposition( eq ( routingTable ) , eq ( connectionPool ), any() ); assertArrayEquals( new BoltServerAddress[]{reader1, reader2}, routingTable.readers().toArray() ); @@ -188,7 +188,7 @@ public void removeAged() RoutingTableHandler handler = newRoutingTableHandler( routingTable, rediscovery, connectionPool, registry ); - RoutingTable actual = await( handler.refreshRoutingTable( simple() ) ); + RoutingTable actual = await( handler.refreshRoutingTable( simple( false ) ) ); assertEquals( routingTable, actual ); verify( connectionPool ).retainAll( new HashSet<>( asList( A, B, C ) ) ); @@ -208,7 +208,7 @@ void shouldRemoveRoutingTableHandlerIfFailedToLookup() throws Throwable // When RoutingTableHandler handler = newRoutingTableHandler( routingTable, rediscovery, connectionPool, registry ); - assertThrows( RuntimeException.class, () -> await( handler.refreshRoutingTable( simple() ) ) ); + assertThrows( RuntimeException.class, () -> await( handler.refreshRoutingTable( simple( false ) ) ) ); // Then verify( registry ).remove( defaultDatabase() ); diff --git a/driver/src/test/java/org/neo4j/driver/internal/cluster/loadbalancing/LoadBalancerTest.java b/driver/src/test/java/org/neo4j/driver/internal/cluster/loadbalancing/LoadBalancerTest.java index 2bfeee482b..2396e7d797 100644 --- a/driver/src/test/java/org/neo4j/driver/internal/cluster/loadbalancing/LoadBalancerTest.java +++ b/driver/src/test/java/org/neo4j/driver/internal/cluster/loadbalancing/LoadBalancerTest.java @@ -29,8 +29,11 @@ import java.util.LinkedHashSet; import java.util.Set; import java.util.concurrent.CompletableFuture; +import java.util.function.Function; import org.neo4j.driver.AccessMode; +import org.neo4j.driver.exceptions.AuthenticationException; +import org.neo4j.driver.exceptions.SecurityException; import org.neo4j.driver.exceptions.ServiceUnavailableException; import org.neo4j.driver.exceptions.SessionExpiredException; import org.neo4j.driver.internal.BoltServerAddress; @@ -39,17 +42,21 @@ import org.neo4j.driver.internal.cluster.AddressSet; import org.neo4j.driver.internal.cluster.ClusterComposition; import org.neo4j.driver.internal.cluster.ClusterRoutingTable; +import org.neo4j.driver.internal.cluster.Rediscovery; import org.neo4j.driver.internal.cluster.RoutingTable; import org.neo4j.driver.internal.cluster.RoutingTableHandler; import org.neo4j.driver.internal.cluster.RoutingTableRegistry; +import org.neo4j.driver.internal.messaging.BoltProtocol; import org.neo4j.driver.internal.spi.Connection; import org.neo4j.driver.internal.spi.ConnectionPool; import org.neo4j.driver.internal.util.FakeClock; import org.neo4j.driver.internal.util.Futures; +import org.neo4j.driver.internal.util.ServerVersion; import static java.util.Arrays.asList; import static java.util.Collections.emptySet; import static java.util.concurrent.CompletableFuture.completedFuture; +import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.instanceOf; import static org.hamcrest.Matchers.startsWith; @@ -61,6 +68,7 @@ import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; import static org.neo4j.driver.AccessMode.READ; @@ -74,6 +82,8 @@ import static org.neo4j.driver.internal.util.ClusterCompositionUtil.A; import static org.neo4j.driver.internal.util.ClusterCompositionUtil.B; import static org.neo4j.driver.internal.util.ClusterCompositionUtil.C; +import static org.neo4j.driver.internal.util.ClusterCompositionUtil.D; +import static org.neo4j.driver.internal.util.Futures.completedWithNull; import static org.neo4j.driver.util.TestUtil.asOrderedSet; import static org.neo4j.driver.util.TestUtil.await; @@ -156,7 +166,7 @@ void shouldSelectLeastConnectedAddress() Set seenAddresses = new HashSet<>(); for ( int i = 0; i < 10; i++ ) { - Connection connection = await( loadBalancer.acquireConnection( simple() ) ); + Connection connection = await( loadBalancer.acquireConnection( newBoltV4ConnectionContext() ) ); seenAddresses.add( connection.serverAddress() ); } @@ -180,7 +190,7 @@ void shouldRoundRobinWhenNoActiveConnections() Set seenAddresses = new HashSet<>(); for ( int i = 0; i < 10; i++ ) { - Connection connection = await( loadBalancer.acquireConnection( simple() ) ); + Connection connection = await( loadBalancer.acquireConnection( newBoltV4ConnectionContext() ) ); seenAddresses.add( connection.serverAddress() ); } @@ -199,7 +209,7 @@ void shouldTryMultipleServersAfterRediscovery() LoadBalancer loadBalancer = newLoadBalancer( connectionPool, routingTable ); - Connection connection = await( loadBalancer.acquireConnection( simple() ) ); + Connection connection = await( loadBalancer.acquireConnection( newBoltV4ConnectionContext() ) ); assertNotNull( connection ); assertEquals( B, connection.serverAddress() ); @@ -207,13 +217,163 @@ void shouldTryMultipleServersAfterRediscovery() assertArrayEquals( new BoltServerAddress[]{B}, routingTable.readers().toArray() ); } + @Test + void shouldFailWithResolverError() throws Throwable + { + ConnectionPool pool = mock( ConnectionPool.class ); + Rediscovery rediscovery = mock( Rediscovery.class ); + when( rediscovery.resolve() ).thenThrow( new RuntimeException( "hi there" ) ); + + LoadBalancer loadBalancer = newLoadBalancer( pool, rediscovery ); + + RuntimeException exception = assertThrows( RuntimeException.class, () -> await( loadBalancer.supportsMultiDb() ) ); + assertThat( exception.getMessage(), equalTo( "hi there" ) ); + } + + @Test + void shouldFailAfterTryingAllServers() throws Throwable + { + Set unavailableAddresses = asOrderedSet( A, B ); + ConnectionPool connectionPool = newConnectionPoolMockWithFailures( unavailableAddresses ); + + Rediscovery rediscovery = mock( Rediscovery.class ); + when( rediscovery.resolve() ).thenReturn( Arrays.asList( A, B ) ); + + LoadBalancer loadBalancer = newLoadBalancer( connectionPool, rediscovery ); + + ServiceUnavailableException exception = assertThrows( ServiceUnavailableException.class, () -> await( loadBalancer.supportsMultiDb() ) ); + Throwable[] suppressed = exception.getSuppressed(); + assertThat( suppressed.length, equalTo( 2 ) ); // one for A, one for B + assertThat( suppressed[0].getMessage(), containsString( A.toString() ) ); + assertThat( suppressed[1].getMessage(), containsString( B.toString() ) ); + verify( connectionPool, times( 2 ) ).acquire( any() ); + } + + @Test + void shouldFailEarlyOnSecurityError() throws Throwable + { + Set unavailableAddresses = asOrderedSet( A, B ); + ConnectionPool connectionPool = newConnectionPoolMockWithFailures( unavailableAddresses, address -> new SecurityException( "code", "hi there" ) ); + + Rediscovery rediscovery = mock( Rediscovery.class ); + when( rediscovery.resolve() ).thenReturn( Arrays.asList( A, B ) ); + + LoadBalancer loadBalancer = newLoadBalancer( connectionPool, rediscovery ); + + SecurityException exception = assertThrows( SecurityException.class, () -> await( loadBalancer.supportsMultiDb() ) ); + assertThat( exception.getMessage(), startsWith( "hi there" ) ); + verify( connectionPool, times( 1 ) ).acquire( any() ); + } + + @Test + void shouldSuccessOnFirstSuccessfulServer() throws Throwable + { + Set unavailableAddresses = asOrderedSet( A, B ); + ConnectionPool connectionPool = newConnectionPoolMockWithFailures( unavailableAddresses ); + + Rediscovery rediscovery = mock( Rediscovery.class ); + when( rediscovery.resolve() ).thenReturn( Arrays.asList( A, B, C, D ) ); + + LoadBalancer loadBalancer = newLoadBalancer( connectionPool, rediscovery ); + + assertTrue( await( loadBalancer.supportsMultiDb() ) ); + verify( connectionPool, times( 3 ) ).acquire( any() ); + } + + @Test + void shouldThrowModifiedErrorWhenSupportMultiDbTestFails() throws Throwable + { + Set unavailableAddresses = asOrderedSet( A, B ); + ConnectionPool connectionPool = newConnectionPoolMockWithFailures( unavailableAddresses ); + + Rediscovery rediscovery = mock( Rediscovery.class ); + when( rediscovery.resolve() ).thenReturn( Arrays.asList( A, B ) ); + + LoadBalancer loadBalancer = newLoadBalancer( connectionPool, rediscovery ); + + ServiceUnavailableException exception = assertThrows( ServiceUnavailableException.class, () -> await( loadBalancer.verifyConnectivity() ) ); + assertThat( exception.getMessage(), startsWith( "Unable to connect to database management service," ) ); + } + + @Test + void shouldFailEarlyOnSecurityErrorWhenSupportMultiDbTestFails() throws Throwable + { + Set unavailableAddresses = asOrderedSet( A, B ); + ConnectionPool connectionPool = newConnectionPoolMockWithFailures( unavailableAddresses, address -> new AuthenticationException( "code", "error" ) ); + + Rediscovery rediscovery = mock( Rediscovery.class ); + when( rediscovery.resolve() ).thenReturn( Arrays.asList( A, B ) ); + + LoadBalancer loadBalancer = newLoadBalancer( connectionPool, rediscovery ); + + AuthenticationException exception = assertThrows( AuthenticationException.class, () -> await( loadBalancer.verifyConnectivity() ) ); + assertThat( exception.getMessage(), startsWith( "error" ) ); + } + + @Test + void shouldThrowModifiedErrorWhenRefreshRoutingTableFails() throws Throwable + { + ConnectionPool connectionPool = newConnectionPoolMock(); + + Rediscovery rediscovery = mock( Rediscovery.class ); + when( rediscovery.resolve() ).thenReturn( Arrays.asList( A, B ) ); + + RoutingTableRegistry routingTables = mock( RoutingTableRegistry.class ); + when( routingTables.refreshRoutingTable( any( ConnectionContext.class ) ) ).thenThrow( new ServiceUnavailableException( "boooo" ) ); + + LoadBalancer loadBalancer = newLoadBalancer( connectionPool, routingTables, rediscovery ); + + ServiceUnavailableException exception = assertThrows( ServiceUnavailableException.class, () -> await( loadBalancer.verifyConnectivity() ) ); + assertThat( exception.getMessage(), startsWith( "Unable to connect to database management service," ) ); + verify( routingTables ).refreshRoutingTable( any( ConnectionContext.class ) ); + } + + @Test + void shouldThrowOriginalErrorWhenRefreshRoutingTableFails() throws Throwable + { + ConnectionPool connectionPool = newConnectionPoolMock(); + + Rediscovery rediscovery = mock( Rediscovery.class ); + when( rediscovery.resolve() ).thenReturn( Arrays.asList( A, B ) ); + + RoutingTableRegistry routingTables = mock( RoutingTableRegistry.class ); + when( routingTables.refreshRoutingTable( any( ConnectionContext.class ) ) ).thenThrow( new RuntimeException( "boo" ) ); + + LoadBalancer loadBalancer = newLoadBalancer( connectionPool, routingTables, rediscovery ); + + RuntimeException exception = assertThrows( RuntimeException.class, () -> await( loadBalancer.verifyConnectivity() ) ); + assertThat( exception.getMessage(), startsWith( "boo" ) ); + verify( routingTables ).refreshRoutingTable( any( ConnectionContext.class ) ); + } + + @Test + void shouldReturnSuccessVerifyConnectivity() throws Throwable + { + ConnectionPool connectionPool = newConnectionPoolMock(); + + Rediscovery rediscovery = mock( Rediscovery.class ); + when( rediscovery.resolve() ).thenReturn( Arrays.asList( A, B ) ); + + RoutingTableRegistry routingTables = mock( RoutingTableRegistry.class ); + when( routingTables.refreshRoutingTable( any( ConnectionContext.class ) ) ).thenReturn( Futures.completedWithNull() ); + + LoadBalancer loadBalancer = newLoadBalancer( connectionPool, routingTables, rediscovery ); + + await( loadBalancer.verifyConnectivity() ); + verify( routingTables ).refreshRoutingTable( any( ConnectionContext.class ) ); + } + private static ConnectionPool newConnectionPoolMock() { return newConnectionPoolMockWithFailures( emptySet() ); } - private static ConnectionPool newConnectionPoolMockWithFailures( - Set unavailableAddresses ) + private static ConnectionPool newConnectionPoolMockWithFailures( Set unavailableAddresses ) + { + return newConnectionPoolMockWithFailures( unavailableAddresses, address -> new ServiceUnavailableException( address + " is unavailable!" ) ); + } + + private static ConnectionPool newConnectionPoolMockWithFailures( Set unavailableAddresses, Function errorAction ) { ConnectionPool pool = mock( ConnectionPool.class ); when( pool.acquire( any( BoltServerAddress.class ) ) ).then( invocation -> @@ -221,15 +381,29 @@ private static ConnectionPool newConnectionPoolMockWithFailures( BoltServerAddress requestedAddress = invocation.getArgument( 0 ); if ( unavailableAddresses.contains( requestedAddress ) ) { - return Futures.failedFuture( new ServiceUnavailableException( requestedAddress + " is unavailable!" ) ); + return Futures.failedFuture( errorAction.apply( requestedAddress ) ); } - Connection connection = mock( Connection.class ); - when( connection.serverAddress() ).thenReturn( requestedAddress ); - return completedFuture( connection ); + + return completedFuture( newBoltV4Connection( requestedAddress ) ); } ); return pool; } + private static Connection newBoltV4Connection( BoltServerAddress address ) + { + Connection connection = mock( Connection.class ); + when( connection.serverAddress() ).thenReturn( address ); + when( connection.protocol() ).thenReturn( BoltProtocol.forVersion( 4 ) ); + when( connection.serverVersion() ).thenReturn( ServerVersion.v4_0_0 ); + when( connection.release() ).thenReturn( completedWithNull() ); + return connection; + } + + private static ConnectionContext newBoltV4ConnectionContext() + { + return simple( true ); + } + private static LoadBalancer newLoadBalancer( ConnectionPool connectionPool, RoutingTable routingTable ) { // Used only in testing @@ -237,7 +411,22 @@ private static LoadBalancer newLoadBalancer( ConnectionPool connectionPool, Rout RoutingTableHandler handler = mock( RoutingTableHandler.class ); when( handler.routingTable() ).thenReturn( routingTable ); when( routingTables.refreshRoutingTable( any( ConnectionContext.class ) ) ).thenReturn( CompletableFuture.completedFuture( handler ) ); - return new LoadBalancer( connectionPool, routingTables, DEV_NULL_LOGGER, new LeastConnectedLoadBalancingStrategy( connectionPool, DEV_NULL_LOGGING ), - GlobalEventExecutor.INSTANCE ); + Rediscovery rediscovery = mock( Rediscovery.class ); + return new LoadBalancer( connectionPool, routingTables, rediscovery, new LeastConnectedLoadBalancingStrategy( connectionPool, DEV_NULL_LOGGING ), + GlobalEventExecutor.INSTANCE, DEV_NULL_LOGGER ); + } + + private static LoadBalancer newLoadBalancer( ConnectionPool connectionPool, Rediscovery rediscovery ) + { + // Used only in testing + RoutingTableRegistry routingTables = mock( RoutingTableRegistry.class ); + return newLoadBalancer( connectionPool, routingTables, rediscovery ); + } + + private static LoadBalancer newLoadBalancer( ConnectionPool connectionPool, RoutingTableRegistry routingTables, Rediscovery rediscovery ) + { + // Used only in testing + return new LoadBalancer( connectionPool, routingTables, rediscovery, new LeastConnectedLoadBalancingStrategy( connectionPool, DEV_NULL_LOGGING ), + GlobalEventExecutor.INSTANCE, DEV_NULL_LOGGER ); } } diff --git a/driver/src/test/java/org/neo4j/driver/internal/cluster/loadbalancing/RoutingTableAndConnectionPoolTest.java b/driver/src/test/java/org/neo4j/driver/internal/cluster/loadbalancing/RoutingTableAndConnectionPoolTest.java index 3cf9e434ac..a76f140b1f 100644 --- a/driver/src/test/java/org/neo4j/driver/internal/cluster/loadbalancing/RoutingTableAndConnectionPoolTest.java +++ b/driver/src/test/java/org/neo4j/driver/internal/cluster/loadbalancing/RoutingTableAndConnectionPoolTest.java @@ -324,8 +324,9 @@ private RoutingTableRegistryImpl newRoutingTables( ConnectionPool connectionPool private LoadBalancer newLoadBalancer( ConnectionPool connectionPool, RoutingTableRegistry routingTables ) { - return new LoadBalancer( connectionPool, routingTables, logging.getLog( "LB" ), new LeastConnectedLoadBalancingStrategy( connectionPool, logging ), - GlobalEventExecutor.INSTANCE ); + Rediscovery rediscovery = mock( Rediscovery.class ); + return new LoadBalancer( connectionPool, routingTables, rediscovery, new LeastConnectedLoadBalancingStrategy( connectionPool, logging ), + GlobalEventExecutor.INSTANCE, logging.getLog( "LB" ) ); } private CompletableFuture clusterComposition( BoltServerAddress... addresses ) @@ -368,5 +369,11 @@ public CompletionStage lookupClusterComposition( RoutingTabl ClusterComposition composition = new ClusterComposition( clock.millis() + 1, servers, servers, servers ); return CompletableFuture.completedFuture( composition ); } + + @Override + public List resolve() + { + throw new UnsupportedOperationException( "Not implemented" ); + } } } diff --git a/driver/src/test/resources/acquire_endpoints_v4_verify_connectivity.script b/driver/src/test/resources/acquire_endpoints_v4_verify_connectivity.script index 1511bfc923..3457b78378 100644 --- a/driver/src/test/resources/acquire_endpoints_v4_verify_connectivity.script +++ b/driver/src/test/resources/acquire_endpoints_v4_verify_connectivity.script @@ -3,7 +3,7 @@ !: AUTO HELLO !: AUTO GOODBYE -C: RUN "CALL dbms.routing.getRoutingTable($context, $database)" {"context": {}, "database": null} {"mode": "r", "db": "system"} +C: RUN "CALL dbms.routing.getRoutingTable($context, $database)" {"context": {}, "database": "system"} {"mode": "r", "db": "system"} PULL {"n": -1} S: SUCCESS {"fields": ["ttl", "servers"]} RECORD [9223372036854775807, [{"addresses": ["127.0.0.1:9007","127.0.0.1:9008"],"role": "WRITE"}, {"addresses": ["127.0.0.1:9005","127.0.0.1:9006"], "role": "READ"},{"addresses": ["127.0.0.1:9001","127.0.0.1:9002","127.0.0.1:9003"], "role": "ROUTE"}]] diff --git a/driver/src/test/resources/support_multidb_v3.script b/driver/src/test/resources/support_multidb_v3.script new file mode 100644 index 0000000000..0dbb929280 --- /dev/null +++ b/driver/src/test/resources/support_multidb_v3.script @@ -0,0 +1,4 @@ +!: BOLT 3 +!: AUTO RESET +!: AUTO HELLO +!: AUTO GOODBYE diff --git a/driver/src/test/resources/support_multidb_v4.script b/driver/src/test/resources/support_multidb_v4.script new file mode 100644 index 0000000000..2797dfaf9d --- /dev/null +++ b/driver/src/test/resources/support_multidb_v4.script @@ -0,0 +1,4 @@ +!: BOLT 4 +!: AUTO RESET +!: AUTO HELLO +!: AUTO GOODBYE