diff --git a/CHANGELOG.md b/CHANGELOG.md index 9b9f08594..8f679fccc 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -78,6 +78,11 @@ It now raises a `ResultConsumedError`. - New method `Result.closed()` can be used to check for this condition if necessary. +- `driver.verify_connectivity()` + - All keyword arguments have been deprecated (they were experimental). + They are now ignored and will be removed in a future release. + - The undocumented return value has been removed. If you need information + about the remote server, use `driver.get_server_info()` instead. ## Version 4.4 diff --git a/bin/make-unasync b/bin/make-unasync index 34146f370..79d187d9c 100755 --- a/bin/make-unasync +++ b/bin/make-unasync @@ -213,6 +213,7 @@ def apply_unasync(files): additional_test_replacements = { "_async": "_sync", "mark_async_test": "mark_sync_test", + "assert_awaited_once": "assert_called_once", } additional_testkit_backend_replacements = {} rules = [ diff --git a/docs/source/api.rst b/docs/source/api.rst index ff7e21733..1dc77f203 100644 --- a/docs/source/api.rst +++ b/docs/source/api.rst @@ -145,7 +145,7 @@ This object holds the details required to establish connections with a Neo4j dat Closing a driver will immediately shut down all connections in the pool. .. autoclass:: neo4j.Driver() - :members: session, encrypted, close + :members: session, encrypted, close, verify_connectivity, get_server_info .. _driver-configuration-ref: diff --git a/docs/source/async_api.rst b/docs/source/async_api.rst index e571473c0..1d613b76c 100644 --- a/docs/source/async_api.rst +++ b/docs/source/async_api.rst @@ -10,6 +10,8 @@ Async API Documentation This means everything documented on this page might be removed or change its API at any time (including in patch releases). +.. versionadded:: 5.0 + ****************** AsyncGraphDatabase ****************** @@ -126,7 +128,7 @@ This object holds the details required to establish connections with a Neo4j dat Closing a driver will immediately shut down all connections in the pool. .. autoclass:: neo4j.AsyncDriver() - :members: session, encrypted, close + :members: session, encrypted, close, verify_connectivity, get_server_info .. _async-driver-configuration-ref: diff --git a/neo4j/_async/driver.py b/neo4j/_async/driver.py index d0dd8fad3..bde625a76 100644 --- a/neo4j/_async/driver.py +++ b/neo4j/_async/driver.py @@ -31,6 +31,10 @@ SessionConfig, WorkspaceConfig, ) +from ..exceptions import ( + ServiceUnavailable, + SessionExpired, +) from ..meta import ( deprecation_warn, experimental, @@ -236,9 +240,11 @@ class AsyncDriver: #: Flag if the driver has been closed _closed = False - def __init__(self, pool): + def __init__(self, pool, default_workspace_config): assert pool is not None + assert default_workspace_config is not None self._pool = pool + self._default_workspace_config = default_workspace_config async def __aenter__(self): return self @@ -285,17 +291,56 @@ async def close(self): await self._pool.close() self._closed = True - @experimental("The configuration may change in the future.") + # TODO: 6.0 - remove config argument async def verify_connectivity(self, **config): - """ This verifies if the driver can connect to a remote server or a cluster - by establishing a network connection with the remote and possibly exchanging - a few data before closing the connection. It throws exception if fails to connect. + """Verify that the driver can establish a connection to the server. + + This verifies if the driver can establish a reading connection to a + remote server or a cluster. Some data will be exchanged. - Use the exception to further understand the cause of the connectivity problem. + .. note:: + Even if this method raises an exception, the driver still needs to + be closed via :meth:`close` to free up all resources. - Note: Even if this method throws an exception, the driver still need to be closed via close() to free up all resources. + :raises DriverError: if the driver cannot connect to the remote. + Use the exception to further understand the cause of the + connectivity problem. + + .. versionchanged:: 5.0 the config parameters will be removed in + version 6 0. It has no effect starting in version 5.0. """ - raise NotImplementedError + if config: + deprecation_warn( + "verify_connectivity() will not accept any configuration " + "parameters starting with version 6.0." + ) + + await self.get_server_info() + + async def get_server_info(self): + """Get information about the connected Neo4j server. + + Try to establish a working read connection to the remote server or a + member of a cluster and exchange some data. Then return the contacted + server's information. + + In a cluster, there is no guarantee about which server will be + contacted. + + .. note:: + Even if this method raises an exception, the driver still needs to + be closed via :meth:`close` to free up all resources. + + :rtype: ServerInfo + + :raises DriverError: if the driver cannot connect to the remote. + Use the exception to further understand the cause of the + connectivity problem. + + .. versionadded:: 5.0 + """ + async with self.session() as session: + return await session._get_server_info() @experimental("Feature support query, based on Bolt Protocol Version and Neo4j Server Version will change in the future.") async def supports_multi_db(self): @@ -339,7 +384,7 @@ def open(cls, target, *, auth=None, **config): def __init__(self, pool, default_workspace_config): _Direct.__init__(self, pool.address) - AsyncDriver.__init__(self, pool) + AsyncDriver.__init__(self, pool, default_workspace_config) self._default_workspace_config = default_workspace_config def session(self, **config): @@ -354,17 +399,6 @@ def session(self, **config): SessionConfig.consume(config) # Consume the config return AsyncSession(self._pool, session_config) - @experimental("The configuration may change in the future.") - async def verify_connectivity(self, **config): - server_agent = None - config["fetch_size"] = -1 - async with self.session(**config) as session: - result = await session.run("RETURN 1 AS x") - value = await result.single().value() - summary = await result.consume() - server_agent = summary.server.agent - return server_agent - class AsyncNeo4jDriver(_Routing, AsyncDriver): """:class:`.AsyncNeo4jDriver` is instantiated for ``neo4j`` URIs. The @@ -387,45 +421,10 @@ def open(cls, *targets, auth=None, routing_context=None, **config): def __init__(self, pool, default_workspace_config): _Routing.__init__(self, pool.get_default_database_initial_router_addresses()) - AsyncDriver.__init__(self, pool) - self._default_workspace_config = default_workspace_config + AsyncDriver.__init__(self, pool, default_workspace_config) def session(self, **config): from .work import AsyncSession session_config = SessionConfig(self._default_workspace_config, config) SessionConfig.consume(config) # Consume the config return AsyncSession(self._pool, session_config) - - @experimental("The configuration may change in the future.") - async def verify_connectivity(self, **config): - """ - :raise ServiceUnavailable: raised if the server does not support routing or if routing support is broken. - """ - # TODO: Improve and update Stub Test Server to be able to test. - return await self._verify_routing_connectivity() - - async def _verify_routing_connectivity(self): - from ..exceptions import ( - Neo4jError, - ServiceUnavailable, - SessionExpired, - ) - - table = self._pool.get_routing_table_for_default_database() - routing_info = {} - for ix in list(table.routers): - try: - routing_info[ix] = await self._pool.fetch_routing_info( - address=table.routers[0], - database=self._default_workspace_config.database, - imp_user=self._default_workspace_config.impersonated_user, - bookmarks=None, - timeout=self._default_workspace_config - .connection_acquisition_timeout - ) - except (ServiceUnavailable, SessionExpired, Neo4jError): - routing_info[ix] = None - for key, val in routing_info.items(): - if val is not None: - return routing_info - raise ServiceUnavailable("Could not connect to any routing servers.") diff --git a/neo4j/_async/io/_bolt.py b/neo4j/_async/io/_bolt.py index 83c03c9eb..a62db09e2 100644 --- a/neo4j/_async/io/_bolt.py +++ b/neo4j/_async/io/_bolt.py @@ -72,6 +72,9 @@ class AsyncBolt: # The socket in_use = False + # When the connection was last put back into the pool + idle_since = float("-inf") + # The socket _closed = False @@ -104,6 +107,7 @@ def __init__(self, unresolved_address, sock, max_connection_lifetime, *, self._max_connection_lifetime = max_connection_lifetime self._creation_timestamp = perf_counter() self.routing_context = routing_context + self.idle_since = perf_counter() # Determine the user agent if user_agent: @@ -456,22 +460,28 @@ async def _send_all(self): except OSError as error: await self._set_defunct_write(error) self.outbox.clear() + self.idle_since = perf_counter() async def send_all(self): """ Send all queued messages to the server. """ if self.closed(): - raise ServiceUnavailable("Failed to write to closed connection {!r} ({!r})".format( - self.unresolved_address, self.server_info.address)) - + raise ServiceUnavailable( + "Failed to write to closed connection {!r} ({!r})".format( + self.unresolved_address, self.server_info.address + ) + ) if self.defunct(): - raise ServiceUnavailable("Failed to write to defunct connection {!r} ({!r})".format( - self.unresolved_address, self.server_info.address)) + raise ServiceUnavailable( + "Failed to write to defunct connection {!r} ({!r})".format( + self.unresolved_address, self.server_info.address + ) + ) await self._send_all() @abc.abstractmethod - async def fetch_message(self): + async def _fetch_message(self): """ Receive at most one message from the server, if available. :return: 2-tuple of number of detail messages and number of summary @@ -479,6 +489,26 @@ async def fetch_message(self): """ pass + async def fetch_message(self): + if self._closed: + raise ServiceUnavailable( + "Failed to read from closed connection {!r} ({!r})".format( + self.unresolved_address, self.server_info.address + ) + ) + if self._defunct: + raise ServiceUnavailable( + "Failed to read from defunct connection {!r} ({!r})".format( + self.unresolved_address, self.server_info.address + ) + ) + if not self.responses: + return 0, 0 + + res = await self._fetch_message() + self.idle_since = perf_counter() + return res + async def fetch_all(self): """ Fetch all outstanding messages. @@ -568,5 +598,15 @@ def closed(self): def defunct(self): pass + def is_idle_for(self, timeout): + """Check if connection has been idle for at least the given timeout. + + :param timeout: timeout in seconds + :type timeout: float + + :rtype: bool + """ + return perf_counter() - self.idle_since > timeout + AsyncBoltSocket.Bolt = AsyncBolt diff --git a/neo4j/_async/io/_bolt3.py b/neo4j/_async/io/_bolt3.py index febc442ab..5e5009b10 100644 --- a/neo4j/_async/io/_bolt3.py +++ b/neo4j/_async/io/_bolt3.py @@ -314,23 +314,12 @@ def fail(metadata): await self.send_all() await self.fetch_all() - async def fetch_message(self): + async def _fetch_message(self): """ Receive at most one message from the server, if available. :return: 2-tuple of number of detail messages and number of summary messages fetched """ - if self._closed: - raise ServiceUnavailable("Failed to read from closed connection {!r} ({!r})".format( - self.unresolved_address, self.server_info.address)) - - if self._defunct: - raise ServiceUnavailable("Failed to read from defunct connection {!r} ({!r})".format( - self.unresolved_address, self.server_info.address)) - - if not self.responses: - return 0, 0 - # Receive exactly one message details, summary_signature, summary_metadata = \ await AsyncUtil.next(self.inbox) diff --git a/neo4j/_async/io/_bolt4.py b/neo4j/_async/io/_bolt4.py index fae37a06c..8565d1955 100644 --- a/neo4j/_async/io/_bolt4.py +++ b/neo4j/_async/io/_bolt4.py @@ -265,23 +265,12 @@ def fail(metadata): await self.send_all() await self.fetch_all() - async def fetch_message(self): + async def _fetch_message(self): """ Receive at most one message from the server, if available. :return: 2-tuple of number of detail messages and number of summary messages fetched """ - if self._closed: - raise ServiceUnavailable("Failed to read from closed connection {!r} ({!r})".format( - self.unresolved_address, self.server_info.address)) - - if self._defunct: - raise ServiceUnavailable("Failed to read from defunct connection {!r} ({!r})".format( - self.unresolved_address, self.server_info.address)) - - if not self.responses: - return 0, 0 - # Receive exactly one message details, summary_signature, summary_metadata = \ await AsyncUtil.next(self.inbox) diff --git a/neo4j/_async/io/_pool.py b/neo4j/_async/io/_pool.py index 8d0905164..cbe84cb32 100644 --- a/neo4j/_async/io/_pool.py +++ b/neo4j/_async/io/_pool.py @@ -80,7 +80,60 @@ async def __aenter__(self): async def __aexit__(self, exc_type, exc_value, traceback): await self.close() - async def _acquire(self, address, timeout): + async def _acquire_from_pool(self, address, health_check): + for connection in list(self.connections.get(address, [])): + if connection.in_use: + continue + if not await health_check(connection): + # `close` is a noop on already closed connections. + # This is to make sure that the connection is + # gracefully closed, e.g. if it's just marked as + # `stale` but still alive. + if log.isEnabledFor(logging.DEBUG): + log.debug( + "[#%04X] C: removing old connection " + "(closed=%s, defunct=%s, stale=%s, in_use=%s)", + connection.local_port, + connection.closed(), connection.defunct(), + connection.stale(), connection.in_use + ) + await connection.close() + try: + self.connections.get(address, []).remove(connection) + except ValueError: + # If closure fails (e.g. because the server went + # down), all connections to the same address will + # be removed. Therefore, we silently ignore if the + # connection isn't in the pool anymore. + pass + continue + if not connection.in_use: + connection.in_use = True + return connection + return None + + async def _acquire_new(self, address, timeout, health_check): + connections = self.connections[address] + max_pool_size = self.pool_config.max_connection_pool_size + infinite_pool_size = (max_pool_size < 0 + or max_pool_size == float("inf")) + can_create_new_connection = (infinite_pool_size + or len(connections) < max_pool_size) + if can_create_new_connection: + timeout = min(self.pool_config.connection_timeout, timeout) + try: + connection = await self.opener(address, timeout) + except ServiceUnavailable: + await self.deactivate(address) + raise + else: + connection.pool = self + connection.in_use = True + connections.append(connection) + return connection + return None + + async def _acquire(self, address, timeout, lifeness_check_timeout): """ Acquire a connection to a given address from the pool. The address supplied should always be an IP address, not a host name. @@ -91,63 +144,37 @@ async def _acquire(self, address, timeout): if timeout is None: timeout = self.workspace_config.connection_acquisition_timeout - async with self.lock: - def time_remaining(): - t = timeout - (perf_counter() - t0) - return t if t > 0 else 0 + def time_remaining(): + t = timeout - (perf_counter() - t0) + return t if t > 0 else 0 + async def health_check(connection_): + if (connection_.closed() + or connection_.defunct() + or connection_.stale()): + return False + if lifeness_check_timeout is not None: + if connection_.is_idle_for(lifeness_check_timeout): + try: + await connection_.reset() + except (OSError, ServiceUnavailable, SessionExpired): + return False + return True + + async with self.lock: while True: # try to find a free connection in pool - for connection in list(self.connections.get(address, [])): - if (connection.closed() or connection.defunct() - or (connection.stale() and not connection.in_use)): - # `close` is a noop on already closed connections. - # This is to make sure that the connection is - # gracefully closed, e.g. if it's just marked as - # `stale` but still alive. - if log.isEnabledFor(logging.DEBUG): - log.debug( - "[#%04X] C: removing old connection " - "(closed=%s, defunct=%s, stale=%s, in_use=%s)", - connection.local_port, - connection.closed(), connection.defunct(), - connection.stale(), connection.in_use - ) - await connection.close() - try: - self.connections.get(address, []).remove(connection) - except ValueError: - # If closure fails (e.g. because the server went - # down), all connections to the same address will - # be removed. Therefore, we silently ignore if the - # connection isn't in the pool anymore. - pass - continue - if not connection.in_use: - connection.in_use = True - return connection + connection = await self._acquire_from_pool( + address, health_check + ) + if connection: + return connection # all connections in pool are in-use - connections = self.connections[address] - max_pool_size = self.pool_config.max_connection_pool_size - infinite_pool_size = (max_pool_size < 0 - or max_pool_size == float("inf")) - can_create_new_connection = ( - infinite_pool_size - or len(connections) < max_pool_size + connection = await self._acquire_new( + address, time_remaining(), health_check ) - if can_create_new_connection: - timeout = min(self.pool_config.connection_timeout, - time_remaining()) - try: - connection = await self.opener(address, timeout) - except ServiceUnavailable: - await self.remove(address) - raise - else: - connection.pool = self - connection.in_use = True - connections.append(connection) - return connection + if connection: + return connection # failed to obtain a connection from pool because the # pool is full and no free connection in the pool @@ -161,7 +188,8 @@ def time_remaining(): @abc.abstractmethod async def acquire( - self, access_mode=None, timeout=None, database=None, bookmarks=None + self, access_mode=None, timeout=None, database=None, bookmarks=None, + lifeness_check_timeout=None ): """ Acquire a connection to a server that can satisfy a set of parameters. @@ -169,6 +197,7 @@ async def acquire( :param timeout: :param database: :param bookmarks: + :param lifeness_check_timeout: """ async def release(self, *connections): @@ -284,11 +313,14 @@ def __repr__(self): self.address) async def acquire( - self, access_mode=None, timeout=None, database=None, bookmarks=None + self, access_mode=None, timeout=None, database=None, bookmarks=None, + lifeness_check_timeout=None ): # The access_mode and database is not needed for a direct connection, # it's just there for consistency. - return await self._acquire(self.address, timeout) + return await self._acquire( + self.address, timeout, lifeness_check_timeout + ) class AsyncNeo4jPool(AsyncIOPool): @@ -398,7 +430,7 @@ async def fetch_routing_info( :raise ServiceUnavailable: if the server does not support routing, or if routing support is broken or outdated """ - cx = await self._acquire(address, timeout) + cx = await self._acquire(address, timeout, None) try: routing_table = await cx.route( database or self.workspace_config.database, @@ -636,7 +668,8 @@ async def _select_address(self, *, access_mode, database): return choice(addresses_by_usage[min(addresses_by_usage)]) async def acquire( - self, access_mode=None, timeout=None, database=None, bookmarks=None + self, access_mode=None, timeout=None, database=None, bookmarks=None, + lifeness_check_timeout=None ): if access_mode not in (WRITE_ACCESS, READ_ACCESS): raise ClientError("Non valid 'access_mode'; {}".format(access_mode)) @@ -666,8 +699,10 @@ async def acquire( try: log.debug("[#0000] C: database=%r address=%r", database, address) # should always be a resolved address - connection = await self._acquire(address, timeout=timeout) - except ServiceUnavailable: + connection = await self._acquire( + address, timeout, lifeness_check_timeout + ) + except (ServiceUnavailable, SessionExpired): await self.deactivate(address=address) else: return connection diff --git a/neo4j/_async/work/session.py b/neo4j/_async/work/session.py index 7e3da9376..081acde63 100644 --- a/neo4j/_async/work/session.py +++ b/neo4j/_async/work/session.py @@ -112,10 +112,10 @@ def _prepare_bookmarks(self, bookmarks): raise TypeError("Bookmarks must be an instance of Bookmarks or an " "iterable of raw bookmarks (deprecated).") - async def _connect(self, access_mode): + async def _connect(self, access_mode, **access_kwargs): if access_mode is None: access_mode = self._config.default_access_mode - await super()._connect(access_mode) + await super()._connect(access_mode, **access_kwargs) def _collect_bookmark(self, bookmark): if bookmark: @@ -132,6 +132,13 @@ async def _result_error(self, _): self._auto_result = None await self._disconnect() + async def _get_server_info(self): + assert not self._connection + await self._connect(READ_ACCESS, lifeness_check_timeout=0) + server_info = self._connection.server_info + await self._disconnect() + return server_info + async def close(self): """Close the session. @@ -171,7 +178,7 @@ async def close(self): await self._disconnect() self._state_failed = False - self._closed = True + self._closed = True async def run(self, query, parameters=None, **kwargs): """Run a Cypher query within an auto-commit transaction. diff --git a/neo4j/_async/work/workspace.py b/neo4j/_async/work/workspace.py index b47430257..2098269d8 100644 --- a/neo4j/_async/work/workspace.py +++ b/neo4j/_async/work/workspace.py @@ -72,7 +72,7 @@ def _set_cached_database(self, database): self._cached_database = True self._config.database = database - async def _connect(self, access_mode): + async def _connect(self, access_mode, **acquire_kwargs): if self._connection: # TODO: Investigate this # log.warning("FIXME: should always disconnect before connect") @@ -96,12 +96,14 @@ async def _connect(self, access_mode): bookmarks=self._bookmarks, database_callback=self._set_cached_database ) - self._connection = await self._pool.acquire( - access_mode=access_mode, - timeout=self._config.connection_acquisition_timeout, - database=self._config.database, - bookmarks=self._bookmarks - ) + acquire_kwargs_ = { + "access_mode": access_mode, + "timeout": self._config.connection_acquisition_timeout, + "database": self._config.database, + "bookmarks": self._bookmarks, + } + acquire_kwargs_.update(acquire_kwargs) + self._connection = await self._pool.acquire(**acquire_kwargs_) self._connection_access_mode = access_mode async def _disconnect(self, sync=False): diff --git a/neo4j/_sync/driver.py b/neo4j/_sync/driver.py index 339653d3b..76084641f 100644 --- a/neo4j/_sync/driver.py +++ b/neo4j/_sync/driver.py @@ -31,6 +31,10 @@ SessionConfig, WorkspaceConfig, ) +from ..exceptions import ( + ServiceUnavailable, + SessionExpired, +) from ..meta import ( deprecation_warn, experimental, @@ -236,9 +240,11 @@ class Driver: #: Flag if the driver has been closed _closed = False - def __init__(self, pool): + def __init__(self, pool, default_workspace_config): assert pool is not None + assert default_workspace_config is not None self._pool = pool + self._default_workspace_config = default_workspace_config def __enter__(self): return self @@ -285,17 +291,56 @@ def close(self): self._pool.close() self._closed = True - @experimental("The configuration may change in the future.") + # TODO: 6.0 - remove config argument def verify_connectivity(self, **config): - """ This verifies if the driver can connect to a remote server or a cluster - by establishing a network connection with the remote and possibly exchanging - a few data before closing the connection. It throws exception if fails to connect. + """Verify that the driver can establish a connection to the server. + + This verifies if the driver can establish a reading connection to a + remote server or a cluster. Some data will be exchanged. - Use the exception to further understand the cause of the connectivity problem. + .. note:: + Even if this method raises an exception, the driver still needs to + be closed via :meth:`close` to free up all resources. - Note: Even if this method throws an exception, the driver still need to be closed via close() to free up all resources. + :raises DriverError: if the driver cannot connect to the remote. + Use the exception to further understand the cause of the + connectivity problem. + + .. versionchanged:: 5.0 the config parameters will be removed in + version 6 0. It has no effect starting in version 5.0. """ - raise NotImplementedError + if config: + deprecation_warn( + "verify_connectivity() will not accept any configuration " + "parameters starting with version 6.0." + ) + + self.get_server_info() + + def get_server_info(self): + """Get information about the connected Neo4j server. + + Try to establish a working read connection to the remote server or a + member of a cluster and exchange some data. Then return the contacted + server's information. + + In a cluster, there is no guarantee about which server will be + contacted. + + .. note:: + Even if this method raises an exception, the driver still needs to + be closed via :meth:`close` to free up all resources. + + :rtype: ServerInfo + + :raises DriverError: if the driver cannot connect to the remote. + Use the exception to further understand the cause of the + connectivity problem. + + .. versionadded:: 5.0 + """ + with self.session() as session: + return session._get_server_info() @experimental("Feature support query, based on Bolt Protocol Version and Neo4j Server Version will change in the future.") def supports_multi_db(self): @@ -339,7 +384,7 @@ def open(cls, target, *, auth=None, **config): def __init__(self, pool, default_workspace_config): _Direct.__init__(self, pool.address) - Driver.__init__(self, pool) + Driver.__init__(self, pool, default_workspace_config) self._default_workspace_config = default_workspace_config def session(self, **config): @@ -354,17 +399,6 @@ def session(self, **config): SessionConfig.consume(config) # Consume the config return Session(self._pool, session_config) - @experimental("The configuration may change in the future.") - def verify_connectivity(self, **config): - server_agent = None - config["fetch_size"] = -1 - with self.session(**config) as session: - result = session.run("RETURN 1 AS x") - value = result.single().value() - summary = result.consume() - server_agent = summary.server.agent - return server_agent - class Neo4jDriver(_Routing, Driver): """:class:`.Neo4jDriver` is instantiated for ``neo4j`` URIs. The @@ -387,45 +421,10 @@ def open(cls, *targets, auth=None, routing_context=None, **config): def __init__(self, pool, default_workspace_config): _Routing.__init__(self, pool.get_default_database_initial_router_addresses()) - Driver.__init__(self, pool) - self._default_workspace_config = default_workspace_config + Driver.__init__(self, pool, default_workspace_config) def session(self, **config): from .work import Session session_config = SessionConfig(self._default_workspace_config, config) SessionConfig.consume(config) # Consume the config return Session(self._pool, session_config) - - @experimental("The configuration may change in the future.") - def verify_connectivity(self, **config): - """ - :raise ServiceUnavailable: raised if the server does not support routing or if routing support is broken. - """ - # TODO: Improve and update Stub Test Server to be able to test. - return self._verify_routing_connectivity() - - def _verify_routing_connectivity(self): - from ..exceptions import ( - Neo4jError, - ServiceUnavailable, - SessionExpired, - ) - - table = self._pool.get_routing_table_for_default_database() - routing_info = {} - for ix in list(table.routers): - try: - routing_info[ix] = self._pool.fetch_routing_info( - address=table.routers[0], - database=self._default_workspace_config.database, - imp_user=self._default_workspace_config.impersonated_user, - bookmarks=None, - timeout=self._default_workspace_config - .connection_acquisition_timeout - ) - except (ServiceUnavailable, SessionExpired, Neo4jError): - routing_info[ix] = None - for key, val in routing_info.items(): - if val is not None: - return routing_info - raise ServiceUnavailable("Could not connect to any routing servers.") diff --git a/neo4j/_sync/io/_bolt.py b/neo4j/_sync/io/_bolt.py index 007bb6b48..7d11cfc5b 100644 --- a/neo4j/_sync/io/_bolt.py +++ b/neo4j/_sync/io/_bolt.py @@ -72,6 +72,9 @@ class Bolt: # The socket in_use = False + # When the connection was last put back into the pool + idle_since = float("-inf") + # The socket _closed = False @@ -104,6 +107,7 @@ def __init__(self, unresolved_address, sock, max_connection_lifetime, *, self._max_connection_lifetime = max_connection_lifetime self._creation_timestamp = perf_counter() self.routing_context = routing_context + self.idle_since = perf_counter() # Determine the user agent if user_agent: @@ -456,22 +460,28 @@ def _send_all(self): except OSError as error: self._set_defunct_write(error) self.outbox.clear() + self.idle_since = perf_counter() def send_all(self): """ Send all queued messages to the server. """ if self.closed(): - raise ServiceUnavailable("Failed to write to closed connection {!r} ({!r})".format( - self.unresolved_address, self.server_info.address)) - + raise ServiceUnavailable( + "Failed to write to closed connection {!r} ({!r})".format( + self.unresolved_address, self.server_info.address + ) + ) if self.defunct(): - raise ServiceUnavailable("Failed to write to defunct connection {!r} ({!r})".format( - self.unresolved_address, self.server_info.address)) + raise ServiceUnavailable( + "Failed to write to defunct connection {!r} ({!r})".format( + self.unresolved_address, self.server_info.address + ) + ) self._send_all() @abc.abstractmethod - def fetch_message(self): + def _fetch_message(self): """ Receive at most one message from the server, if available. :return: 2-tuple of number of detail messages and number of summary @@ -479,6 +489,26 @@ def fetch_message(self): """ pass + def fetch_message(self): + if self._closed: + raise ServiceUnavailable( + "Failed to read from closed connection {!r} ({!r})".format( + self.unresolved_address, self.server_info.address + ) + ) + if self._defunct: + raise ServiceUnavailable( + "Failed to read from defunct connection {!r} ({!r})".format( + self.unresolved_address, self.server_info.address + ) + ) + if not self.responses: + return 0, 0 + + res = self._fetch_message() + self.idle_since = perf_counter() + return res + def fetch_all(self): """ Fetch all outstanding messages. @@ -568,5 +598,15 @@ def closed(self): def defunct(self): pass + def is_idle_for(self, timeout): + """Check if connection has been idle for at least the given timeout. + + :param timeout: timeout in seconds + :type timeout: float + + :rtype: bool + """ + return perf_counter() - self.idle_since > timeout + BoltSocket.Bolt = Bolt diff --git a/neo4j/_sync/io/_bolt3.py b/neo4j/_sync/io/_bolt3.py index 8eac53f09..9166fe8f9 100644 --- a/neo4j/_sync/io/_bolt3.py +++ b/neo4j/_sync/io/_bolt3.py @@ -314,23 +314,12 @@ def fail(metadata): self.send_all() self.fetch_all() - def fetch_message(self): + def _fetch_message(self): """ Receive at most one message from the server, if available. :return: 2-tuple of number of detail messages and number of summary messages fetched """ - if self._closed: - raise ServiceUnavailable("Failed to read from closed connection {!r} ({!r})".format( - self.unresolved_address, self.server_info.address)) - - if self._defunct: - raise ServiceUnavailable("Failed to read from defunct connection {!r} ({!r})".format( - self.unresolved_address, self.server_info.address)) - - if not self.responses: - return 0, 0 - # Receive exactly one message details, summary_signature, summary_metadata = \ Util.next(self.inbox) diff --git a/neo4j/_sync/io/_bolt4.py b/neo4j/_sync/io/_bolt4.py index 44c807ce6..da7c22de3 100644 --- a/neo4j/_sync/io/_bolt4.py +++ b/neo4j/_sync/io/_bolt4.py @@ -265,23 +265,12 @@ def fail(metadata): self.send_all() self.fetch_all() - def fetch_message(self): + def _fetch_message(self): """ Receive at most one message from the server, if available. :return: 2-tuple of number of detail messages and number of summary messages fetched """ - if self._closed: - raise ServiceUnavailable("Failed to read from closed connection {!r} ({!r})".format( - self.unresolved_address, self.server_info.address)) - - if self._defunct: - raise ServiceUnavailable("Failed to read from defunct connection {!r} ({!r})".format( - self.unresolved_address, self.server_info.address)) - - if not self.responses: - return 0, 0 - # Receive exactly one message details, summary_signature, summary_metadata = \ Util.next(self.inbox) diff --git a/neo4j/_sync/io/_pool.py b/neo4j/_sync/io/_pool.py index 95d4fd4ee..b4ff5c1b0 100644 --- a/neo4j/_sync/io/_pool.py +++ b/neo4j/_sync/io/_pool.py @@ -80,7 +80,60 @@ def __enter__(self): def __exit__(self, exc_type, exc_value, traceback): self.close() - def _acquire(self, address, timeout): + def _acquire_from_pool(self, address, health_check): + for connection in list(self.connections.get(address, [])): + if connection.in_use: + continue + if not health_check(connection): + # `close` is a noop on already closed connections. + # This is to make sure that the connection is + # gracefully closed, e.g. if it's just marked as + # `stale` but still alive. + if log.isEnabledFor(logging.DEBUG): + log.debug( + "[#%04X] C: removing old connection " + "(closed=%s, defunct=%s, stale=%s, in_use=%s)", + connection.local_port, + connection.closed(), connection.defunct(), + connection.stale(), connection.in_use + ) + connection.close() + try: + self.connections.get(address, []).remove(connection) + except ValueError: + # If closure fails (e.g. because the server went + # down), all connections to the same address will + # be removed. Therefore, we silently ignore if the + # connection isn't in the pool anymore. + pass + continue + if not connection.in_use: + connection.in_use = True + return connection + return None + + def _acquire_new(self, address, timeout, health_check): + connections = self.connections[address] + max_pool_size = self.pool_config.max_connection_pool_size + infinite_pool_size = (max_pool_size < 0 + or max_pool_size == float("inf")) + can_create_new_connection = (infinite_pool_size + or len(connections) < max_pool_size) + if can_create_new_connection: + timeout = min(self.pool_config.connection_timeout, timeout) + try: + connection = self.opener(address, timeout) + except ServiceUnavailable: + self.deactivate(address) + raise + else: + connection.pool = self + connection.in_use = True + connections.append(connection) + return connection + return None + + def _acquire(self, address, timeout, lifeness_check_timeout): """ Acquire a connection to a given address from the pool. The address supplied should always be an IP address, not a host name. @@ -91,63 +144,37 @@ def _acquire(self, address, timeout): if timeout is None: timeout = self.workspace_config.connection_acquisition_timeout - with self.lock: - def time_remaining(): - t = timeout - (perf_counter() - t0) - return t if t > 0 else 0 + def time_remaining(): + t = timeout - (perf_counter() - t0) + return t if t > 0 else 0 + def health_check(connection_): + if (connection_.closed() + or connection_.defunct() + or connection_.stale()): + return False + if lifeness_check_timeout is not None: + if connection_.is_idle_for(lifeness_check_timeout): + try: + connection_.reset() + except (OSError, ServiceUnavailable, SessionExpired): + return False + return True + + with self.lock: while True: # try to find a free connection in pool - for connection in list(self.connections.get(address, [])): - if (connection.closed() or connection.defunct() - or (connection.stale() and not connection.in_use)): - # `close` is a noop on already closed connections. - # This is to make sure that the connection is - # gracefully closed, e.g. if it's just marked as - # `stale` but still alive. - if log.isEnabledFor(logging.DEBUG): - log.debug( - "[#%04X] C: removing old connection " - "(closed=%s, defunct=%s, stale=%s, in_use=%s)", - connection.local_port, - connection.closed(), connection.defunct(), - connection.stale(), connection.in_use - ) - connection.close() - try: - self.connections.get(address, []).remove(connection) - except ValueError: - # If closure fails (e.g. because the server went - # down), all connections to the same address will - # be removed. Therefore, we silently ignore if the - # connection isn't in the pool anymore. - pass - continue - if not connection.in_use: - connection.in_use = True - return connection + connection = self._acquire_from_pool( + address, health_check + ) + if connection: + return connection # all connections in pool are in-use - connections = self.connections[address] - max_pool_size = self.pool_config.max_connection_pool_size - infinite_pool_size = (max_pool_size < 0 - or max_pool_size == float("inf")) - can_create_new_connection = ( - infinite_pool_size - or len(connections) < max_pool_size + connection = self._acquire_new( + address, time_remaining(), health_check ) - if can_create_new_connection: - timeout = min(self.pool_config.connection_timeout, - time_remaining()) - try: - connection = self.opener(address, timeout) - except ServiceUnavailable: - self.remove(address) - raise - else: - connection.pool = self - connection.in_use = True - connections.append(connection) - return connection + if connection: + return connection # failed to obtain a connection from pool because the # pool is full and no free connection in the pool @@ -161,7 +188,8 @@ def time_remaining(): @abc.abstractmethod def acquire( - self, access_mode=None, timeout=None, database=None, bookmarks=None + self, access_mode=None, timeout=None, database=None, bookmarks=None, + lifeness_check_timeout=None ): """ Acquire a connection to a server that can satisfy a set of parameters. @@ -169,6 +197,7 @@ def acquire( :param timeout: :param database: :param bookmarks: + :param lifeness_check_timeout: """ def release(self, *connections): @@ -284,11 +313,14 @@ def __repr__(self): self.address) def acquire( - self, access_mode=None, timeout=None, database=None, bookmarks=None + self, access_mode=None, timeout=None, database=None, bookmarks=None, + lifeness_check_timeout=None ): # The access_mode and database is not needed for a direct connection, # it's just there for consistency. - return self._acquire(self.address, timeout) + return self._acquire( + self.address, timeout, lifeness_check_timeout + ) class Neo4jPool(IOPool): @@ -398,7 +430,7 @@ def fetch_routing_info( :raise ServiceUnavailable: if the server does not support routing, or if routing support is broken or outdated """ - cx = self._acquire(address, timeout) + cx = self._acquire(address, timeout, None) try: routing_table = cx.route( database or self.workspace_config.database, @@ -636,7 +668,8 @@ def _select_address(self, *, access_mode, database): return choice(addresses_by_usage[min(addresses_by_usage)]) def acquire( - self, access_mode=None, timeout=None, database=None, bookmarks=None + self, access_mode=None, timeout=None, database=None, bookmarks=None, + lifeness_check_timeout=None ): if access_mode not in (WRITE_ACCESS, READ_ACCESS): raise ClientError("Non valid 'access_mode'; {}".format(access_mode)) @@ -666,8 +699,10 @@ def acquire( try: log.debug("[#0000] C: database=%r address=%r", database, address) # should always be a resolved address - connection = self._acquire(address, timeout=timeout) - except ServiceUnavailable: + connection = self._acquire( + address, timeout, lifeness_check_timeout + ) + except (ServiceUnavailable, SessionExpired): self.deactivate(address=address) else: return connection diff --git a/neo4j/_sync/work/session.py b/neo4j/_sync/work/session.py index f188f952a..d372ba69b 100644 --- a/neo4j/_sync/work/session.py +++ b/neo4j/_sync/work/session.py @@ -112,10 +112,10 @@ def _prepare_bookmarks(self, bookmarks): raise TypeError("Bookmarks must be an instance of Bookmarks or an " "iterable of raw bookmarks (deprecated).") - def _connect(self, access_mode): + def _connect(self, access_mode, **access_kwargs): if access_mode is None: access_mode = self._config.default_access_mode - super()._connect(access_mode) + super()._connect(access_mode, **access_kwargs) def _collect_bookmark(self, bookmark): if bookmark: @@ -132,6 +132,13 @@ def _result_error(self, _): self._auto_result = None self._disconnect() + def _get_server_info(self): + assert not self._connection + self._connect(READ_ACCESS, lifeness_check_timeout=0) + server_info = self._connection.server_info + self._disconnect() + return server_info + def close(self): """Close the session. @@ -171,7 +178,7 @@ def close(self): self._disconnect() self._state_failed = False - self._closed = True + self._closed = True def run(self, query, parameters=None, **kwargs): """Run a Cypher query within an auto-commit transaction. diff --git a/neo4j/_sync/work/workspace.py b/neo4j/_sync/work/workspace.py index aa53d535a..38d312f41 100644 --- a/neo4j/_sync/work/workspace.py +++ b/neo4j/_sync/work/workspace.py @@ -72,7 +72,7 @@ def _set_cached_database(self, database): self._cached_database = True self._config.database = database - def _connect(self, access_mode): + def _connect(self, access_mode, **acquire_kwargs): if self._connection: # TODO: Investigate this # log.warning("FIXME: should always disconnect before connect") @@ -96,12 +96,14 @@ def _connect(self, access_mode): bookmarks=self._bookmarks, database_callback=self._set_cached_database ) - self._connection = self._pool.acquire( - access_mode=access_mode, - timeout=self._config.connection_acquisition_timeout, - database=self._config.database, - bookmarks=self._bookmarks - ) + acquire_kwargs_ = { + "access_mode": access_mode, + "timeout": self._config.connection_acquisition_timeout, + "database": self._config.database, + "bookmarks": self._bookmarks, + } + acquire_kwargs_.update(acquire_kwargs) + self._connection = self._pool.acquire(**acquire_kwargs_) self._connection_access_mode = access_mode def _disconnect(self, sync=False): diff --git a/neo4j/exceptions.py b/neo4j/exceptions.py index 7da2b2a7e..82a85c85a 100644 --- a/neo4j/exceptions.py +++ b/neo4j/exceptions.py @@ -269,7 +269,7 @@ class DriverError(Exception): class SessionExpired(DriverError): - """ Raised when no a session is no longer able to fulfil + """ Raised when a session is no longer able to fulfil the purpose described by its original parameters. """ diff --git a/neo4j/meta.py b/neo4j/meta.py index 2b4972493..d662bf0d0 100644 --- a/neo4j/meta.py +++ b/neo4j/meta.py @@ -88,14 +88,25 @@ def foo(x): pass """ - def f__(f): - @wraps(f) - def f_(*args, **kwargs): - from warnings import warn - warn(message, category=ExperimentalWarning, stacklevel=2) - return f(*args, **kwargs) - return f_ - return f__ + def decorator(f): + if asyncio.iscoroutinefunction(f): + @wraps(f) + async def inner(*args, **kwargs): + from warnings import warn + warn(message, category=ExperimentalWarning, stacklevel=2) + return await f(*args, **kwargs) + + return inner + else: + @wraps(f) + def inner(*args, **kwargs): + from warnings import warn + warn(message, category=ExperimentalWarning, stacklevel=2) + return f(*args, **kwargs) + + return inner + + return decorator def unclosed_resource_warn(obj): diff --git a/testkitbackend/_async/requests.py b/testkitbackend/_async/requests.py index 7da6f7872..e79d566f0 100644 --- a/testkitbackend/_async/requests.py +++ b/testkitbackend/_async/requests.py @@ -123,6 +123,17 @@ async def VerifyConnectivity(backend, data): await backend.send_response("Driver", {"id": driver_id}) +async def GetServerInfo(backend, data): + driver_id = data["driverId"] + driver = backend.drivers[driver_id] + server_info = await driver.get_server_info() + await backend.send_response("ServerInfo", { + "address": ":".join(map(str, server_info.address)), + "agent": server_info.agent, + "protocolVersion": ".".join(map(str, server_info.protocol_version)), + }) + + async def CheckMultiDBSupport(backend, data): driver_id = data["driverId"] driver = backend.drivers[driver_id] diff --git a/testkitbackend/_sync/requests.py b/testkitbackend/_sync/requests.py index a58abd51b..b67e413c7 100644 --- a/testkitbackend/_sync/requests.py +++ b/testkitbackend/_sync/requests.py @@ -123,6 +123,17 @@ def VerifyConnectivity(backend, data): backend.send_response("Driver", {"id": driver_id}) +def GetServerInfo(backend, data): + driver_id = data["driverId"] + driver = backend.drivers[driver_id] + server_info = driver.get_server_info() + backend.send_response("ServerInfo", { + "address": ":".join(map(str, server_info.address)), + "agent": server_info.agent, + "protocolVersion": ".".join(map(str, server_info.protocol_version)), + }) + + def CheckMultiDBSupport(backend, data): driver_id = data["driverId"] driver = backend.drivers[driver_id] diff --git a/testkitbackend/test_config.json b/testkitbackend/test_config.json index 706dc1336..6542510d3 100644 --- a/testkitbackend/test_config.json +++ b/testkitbackend/test_config.json @@ -1,13 +1,5 @@ { "skips": { - "stub.routing.test_routing_v4x1.RoutingV4x1.test_should_successfully_acquire_rt_when_router_ip_changes": - "Test makes assumptions about how verify_connectivity is implemented", - "stub.routing.test_routing_v3.RoutingV3.test_should_successfully_acquire_rt_when_router_ip_changes": - "Test makes assumptions about how verify_connectivity is implemented", - "stub.routing.test_routing_v4x3.RoutingV4x3.test_should_successfully_acquire_rt_when_router_ip_changes": - "Test makes assumptions about how verify_connectivity is implemented", - "stub.routing.test_routing_v4x4.RoutingV4x4.test_should_successfully_acquire_rt_when_router_ip_changes": - "Test makes assumptions about how verify_connectivity is implemented", "stub.retry.test_retry_clustering.TestRetryClustering.test_retry_ForbiddenOnReadOnlyDatabase_ChangingWriter": "Test makes assumptions about how verify_connectivity is implemented", "stub.authorization.test_authorization.TestAuthorizationV4x3.test_should_retry_on_auth_expired_on_begin_using_tx_function": @@ -28,7 +20,9 @@ "Driver emits deprecation warning. Behavior will be unified in 6.0." }, "features": { + "Feature:API:Driver:GetServerInfo": true, "Feature:API:Driver.IsEncrypted": true, + "Feature:API:Driver.VerifyConnectivity": true, "Feature:API:Liveness.Check": false, "Feature:API:Result.List": true, "Feature:API:Result.Peek": true, @@ -48,6 +42,7 @@ "Feature:TLS:1.1": "Driver blocks TLS 1.1 for security reasons.", "Feature:TLS:1.2": true, "Feature:TLS:1.3": "Depends on the machine (will be calculated dynamically).", + "AuthorizationExpiredTreatment": true, "Optimization:ConnectionReuse": true, "Optimization:EagerTransactionBegin": true, @@ -55,7 +50,12 @@ "Optimization:MinimalResets": true, "Optimization:PullPipelining": true, "Optimization:ResultListFetchAll": "The idiomatic way to cast to list is indistinguishable from iterating over the result.", + "ConfHint:connection.recv_timeout_seconds": true, + + "Backend:RTFetch": true, + "Backend:RTForceUpdate": true, + "Temporary:CypherPathAndRelationship": true, "Temporary:DriverFetchSize": true, "Temporary:DriverMaxTxRetryTime": true, diff --git a/tests/_async_compat/__init__.py b/tests/_async_compat/__init__.py index 285e3af5a..577335802 100644 --- a/tests/_async_compat/__init__.py +++ b/tests/_async_compat/__init__.py @@ -35,7 +35,6 @@ AsyncMagicMock = mock.AsyncMock MagicMock = mock.MagicMock Mock = mock.Mock -mock.NonCallableMagicMock class AsyncMock(AsyncMockMixin, Mock): diff --git a/tests/stub/__init__.py b/tests/stub/__init__.py deleted file mode 100644 index e69de29bb..000000000 diff --git a/tests/stub/conftest.py b/tests/stub/conftest.py deleted file mode 100644 index 7738a9495..000000000 --- a/tests/stub/conftest.py +++ /dev/null @@ -1,173 +0,0 @@ -# Copyright (c) "Neo4j" -# Neo4j Sweden AB [http://neo4j.com] -# -# This file is part of Neo4j. -# -# 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. - - -import logging -import os -from platform import system -import subprocess -from threading import Thread -from time import sleep - -from boltkit.server.stub import BoltStubService -from pytest import fixture - - -log = logging.getLogger("neo4j") - -# from neo4j.debug import watch -# watch("neo4j") - - -class StubServer: - - def __init__(self, port, script): - self.port = port - self.script = os.path.join(os.path.dirname(__file__), "scripts", script) - - def run(self): - shell = system() == "Windows" # I hate myself for doing this - self._process = subprocess.Popen(["python", "-m", "boltkit", "stub", "-v", "-l", ":{}".format(str(self.port)), "-t", "10", self.script], stdout=subprocess.PIPE, shell=shell) - # Need verbose for this to work - line = self._process.stdout.readline().decode("utf-8") - log.debug("started stub server {}".format(self.port)) - log.debug(line.strip("\n")) - - def wait(self): - while True: - return_code = self._process.poll() - if return_code is not None: - line = self._process.stdout.readline() - if not line: - break - try: - line = line.decode("utf-8") - line = line.strip("\n") - except UnicodeDecodeError: - pass - log.debug(line) - - return True - - def kill(self): - # Kill process if not already dead - if self._process.poll() is None: - self._process.kill() - - -class StubCluster: - - def __init__(self, servers): - self.servers = {port: StubServer(port, script) for port, script in dict(servers).items()} - - def __enter__(self): - self.start() - - def __exit__(self, exc_type, exc_value, traceback): - self.wait() - - def start(self): - for port, server in self.servers.items(): - server.run() - - def wait(self): - success = True - for port, server in self.servers.items(): - if not server.wait(): - success = False - server.kill() - - if not success: - raise Exception("Stub server failed") - - -class LegacyStubServer(Thread): - - def __init__(self, port, script): - super(LegacyStubServer, self).__init__() - self.port = port - self.script = os.path.join(os.path.dirname(__file__), "scripts", script) - - def run(self): - check_call(["python", "-m", "boltkit.legacy.stub", "-v", str(self.port), self.script]) - - -class LegacyStubCluster: - - def __init__(self, servers): - self.servers = {port: LegacyStubServer(port, script) for port, script in dict(servers).items()} - - def __enter__(self): - self.start() - - def __exit__(self, exc_type, exc_value, traceback): - self.wait() - - def start(self): - for port, server in self.servers.items(): - server.start() - sleep(0.5) - - def wait(self): - for port, server in self.servers.items(): - server.join() - - -class DefaultBoltStubService(BoltStubService): - - default_base_port = 9001 - - -class StubCluster(StubCluster): - - def __init__(self, *servers): - print("") - scripts = [os.path.join(os.path.dirname(__file__), "scripts", server) for server in servers] - - bss = DefaultBoltStubService.load(*scripts) - servers2 = {port: script.filename for port, script in bss.scripts.items()} - super().__init__(servers2) - - # def run(): - # check_call(["bolt", "stub", "-v", "-t", "10", "-l", ":9001"] + scripts) - - # self.thread = Thread(target=run) - - # def __enter__(self): - # self.thread.start() - # sleep(0.5) - - # def __exit__(self, exc_type, exc_value, traceback): - # self.thread.join(3) - - -@fixture -def script(): - return lambda *paths: path_join(dirname(__file__), "scripts", *paths) - - -@fixture -def driver_info(): - """ Base class for test cases that integrate with a server. - """ - return { - "uri_bolt": "bolt://localhost:9001", - "uri_neo4j": "neo4j://localhost:9001", - "user": "test", - "password": "test", - "auth_token": ("test", "test") - } diff --git a/tests/stub/io/__init__.py b/tests/stub/io/__init__.py deleted file mode 100644 index e69de29bb..000000000 diff --git a/tests/stub/scripts/v1/empty_explicit_hello_goodbye.script b/tests/stub/scripts/v1/empty_explicit_hello_goodbye.script deleted file mode 100644 index f55e38cc2..000000000 --- a/tests/stub/scripts/v1/empty_explicit_hello_goodbye.script +++ /dev/null @@ -1,6 +0,0 @@ -!: BOLT 1 - -C: INIT {"user_agent": "test", "scheme": "basic", "principal": "test", "credentials": "test"} -S: SUCCESS {"server": "Neo4j/3.3.0", "connection_id": "123e4567-e89b-12d3-a456-426655440000"} -C: RESET -S: diff --git a/tests/stub/scripts/v2/empty_explicit_hello_goodbye.script b/tests/stub/scripts/v2/empty_explicit_hello_goodbye.script deleted file mode 100644 index 8e661fdee..000000000 --- a/tests/stub/scripts/v2/empty_explicit_hello_goodbye.script +++ /dev/null @@ -1,6 +0,0 @@ -!: BOLT 2 - -C: INIT {"user_agent": "test", "scheme": "basic", "principal": "test", "credentials": "test"} -S: SUCCESS {"server": "Neo4j/3.4.0", "connection_id": "123e4567-e89b-12d3-a456-426655440000"} -C: RESET -S: diff --git a/tests/stub/scripts/v3/bookmark_chain.script b/tests/stub/scripts/v3/bookmark_chain.script deleted file mode 100644 index dac23edbf..000000000 --- a/tests/stub/scripts/v3/bookmark_chain.script +++ /dev/null @@ -1,15 +0,0 @@ -!: BOLT 3 -!: AUTO HELLO -!: AUTO GOODBYE -!: AUTO RESET -!: PORT 9004 - -C: BEGIN {"bookmarks": ["bookmark:0", "bookmark:1"], "mode": "r"} -S: SUCCESS {} -C: COMMIT -S: SUCCESS {"bookmark": "bookmark:2"} - -C: BEGIN {"bookmarks": ["bookmark:2"], "mode": "r"} -S: SUCCESS {} -C: COMMIT -S: SUCCESS {"bookmark": "bookmark:3"} diff --git a/tests/stub/scripts/v3/bookmark_chain_with_autocommit.script b/tests/stub/scripts/v3/bookmark_chain_with_autocommit.script deleted file mode 100644 index d1e6fae7c..000000000 --- a/tests/stub/scripts/v3/bookmark_chain_with_autocommit.script +++ /dev/null @@ -1,20 +0,0 @@ -!: BOLT 3 -!: AUTO HELLO -!: AUTO GOODBYE -!: AUTO RESET -!: PORT 9004 - -C: BEGIN {"bookmarks": ["bookmark:1"], "mode": "r"} -S: SUCCESS {} -C: COMMIT -S: SUCCESS {"bookmark": "bookmark:2"} - -C: RUN "RETURN 1" {} {"bookmarks": ["bookmark:2"], "mode": "r"} - PULL_ALL -S: SUCCESS {} - SUCCESS {"bookmark": "bookmark:3"} - -C: BEGIN {"bookmarks": ["bookmark:3"], "mode": "r"} -S: SUCCESS {} -C: COMMIT -S: SUCCESS {"bookmark": "bookmark:4"} diff --git a/tests/stub/scripts/v3/broken_router.script b/tests/stub/scripts/v3/broken_router.script deleted file mode 100644 index 7f3e72123..000000000 --- a/tests/stub/scripts/v3/broken_router.script +++ /dev/null @@ -1,12 +0,0 @@ -!: BOLT 3 -!: AUTO HELLO -!: AUTO GOODBYE -!: AUTO RESET -!: PORT 9001 - -C: RUN "CALL dbms.cluster.routing.getRoutingTable($context)" {"context": {"address": "localhost:9001"}} {} - PULL_ALL -S: FAILURE {"code": "Neo.DatabaseError.General.UnknownError", "message": "An unknown error occurred."} - IGNORED -C: RESET -S: SUCCESS {} diff --git a/tests/stub/scripts/v3/connection_error_on_commit.script b/tests/stub/scripts/v3/connection_error_on_commit.script deleted file mode 100644 index 7c1bf302d..000000000 --- a/tests/stub/scripts/v3/connection_error_on_commit.script +++ /dev/null @@ -1,13 +0,0 @@ -!: BOLT 3 -!: AUTO HELLO -!: AUTO GOODBYE -!: AUTO RESET - -C: BEGIN {} - RUN "CREATE (n {name:'Bob'})" {} {} - PULL_ALL -S: SUCCESS {} - SUCCESS {} - SUCCESS {} -C: COMMIT -S: diff --git a/tests/stub/scripts/v3/create_a.script b/tests/stub/scripts/v3/create_a.script deleted file mode 100644 index 8a8230aff..000000000 --- a/tests/stub/scripts/v3/create_a.script +++ /dev/null @@ -1,10 +0,0 @@ -!: BOLT 3 -!: AUTO HELLO -!: AUTO GOODBYE -!: AUTO RESET -!: PORT 9006 - -C: RUN "CREATE (a $x)" {"x": {"name": "Alice"}} {} - PULL_ALL -S: SUCCESS {"fields": []} - SUCCESS {} diff --git a/tests/stub/scripts/v3/database_unavailable.script b/tests/stub/scripts/v3/database_unavailable.script deleted file mode 100644 index 4bfe832fc..000000000 --- a/tests/stub/scripts/v3/database_unavailable.script +++ /dev/null @@ -1,13 +0,0 @@ -!: BOLT 3 -!: AUTO HELLO -!: AUTO GOODBYE -!: AUTO RESET -!: AUTO BEGIN {} -!: AUTO COMMIT -!: AUTO ROLLBACK -!: PORT 9004 - -C: RUN "RETURN 1" {} {"mode": "r"} -C: PULL_ALL -S: FAILURE {"code": "Neo.TransientError.General.DatabaseUnavailable", "message": "Database is busy doing store copy"} -S: IGNORED diff --git a/tests/stub/scripts/v3/dbms_cluster_routing_get_routing_table_system.script b/tests/stub/scripts/v3/dbms_cluster_routing_get_routing_table_system.script deleted file mode 100644 index e47a2e2ef..000000000 --- a/tests/stub/scripts/v3/dbms_cluster_routing_get_routing_table_system.script +++ /dev/null @@ -1,14 +0,0 @@ -!: BOLT 3 -!: PORT 9001 - -C: HELLO {"user_agent": "test", "scheme": "basic", "principal": "test", "credentials": "test"} -S: SUCCESS {"server": "Neo4j/3.5.0", "connection_id": "12345678-1234-1234-1234-123456789000"} - -C: RUN "CALL dbms.cluster.routing.getRoutingTable($context)" {"context": {"address": "localhost:9001"}} {"mode": "r"} - PULL_ALL -S: SUCCESS {"fields": ["ttl", "servers"]} - RECORD [1234, [{"role":"WRITE", "addresses":["127.0.0.1:9001"]}, {"role":"READ", "addresses":["127.0.0.1:9002", "127.0.0.1:9003"]}, {"role":"ROUTE", "addresses":["127.0.0.1:9001", "127.0.0.1:9002", "127.0.0.1:9003"]}]] - SUCCESS {} - -C: GOODBYE -S: diff --git a/tests/stub/scripts/v3/disconnect_after_init.script b/tests/stub/scripts/v3/disconnect_after_init.script deleted file mode 100644 index 9b0309ee5..000000000 --- a/tests/stub/scripts/v3/disconnect_after_init.script +++ /dev/null @@ -1,7 +0,0 @@ -!: BOLT 3 -!: AUTO GOODBYE -!: AUTO RESET - -C: HELLO {"user_agent": "test", "scheme": "basic", "principal": "test", "credentials": "test"} -S: SUCCESS {"server": "Neo4j/3.5.0", "connection_id": "bolt-0"} -S: diff --git a/tests/stub/scripts/v3/disconnect_on_pull_all.script b/tests/stub/scripts/v3/disconnect_on_pull_all.script deleted file mode 100644 index e509e4329..000000000 --- a/tests/stub/scripts/v3/disconnect_on_pull_all.script +++ /dev/null @@ -1,9 +0,0 @@ -!: BOLT 3 -!: AUTO HELLO -!: AUTO GOODBYE -!: AUTO RESET -!: PORT 9001 - -C: RUN "RETURN $x" {"x": 1} {"mode": "r"} - PULL_ALL -S: diff --git a/tests/stub/scripts/v3/disconnect_on_pull_all_9004.script b/tests/stub/scripts/v3/disconnect_on_pull_all_9004.script deleted file mode 100644 index d97d2d4f9..000000000 --- a/tests/stub/scripts/v3/disconnect_on_pull_all_9004.script +++ /dev/null @@ -1,9 +0,0 @@ -!: BOLT 3 -!: AUTO HELLO -!: AUTO GOODBYE -!: AUTO RESET -!: PORT 9004 - -C: RUN "RETURN $x" {"x": 1} {"mode": "r"} - PULL_ALL -S: diff --git a/tests/stub/scripts/v3/disconnect_on_run.script b/tests/stub/scripts/v3/disconnect_on_run.script deleted file mode 100644 index ca1640f66..000000000 --- a/tests/stub/scripts/v3/disconnect_on_run.script +++ /dev/null @@ -1,8 +0,0 @@ -!: BOLT 3 -!: AUTO HELLO -!: AUTO GOODBYE -!: AUTO RESET -!: PORT 9001 - -C: RUN "RETURN 1 AS x" {} {"mode": "r"} -S: diff --git a/tests/stub/scripts/v3/disconnect_on_run_9004.script b/tests/stub/scripts/v3/disconnect_on_run_9004.script deleted file mode 100644 index ca17f705a..000000000 --- a/tests/stub/scripts/v3/disconnect_on_run_9004.script +++ /dev/null @@ -1,8 +0,0 @@ -!: BOLT 3 -!: AUTO HELLO -!: AUTO GOODBYE -!: AUTO RESET -!: PORT 9004 - -C: RUN "RETURN $x" {"x": 1} {"mode": "r"} -S: diff --git a/tests/stub/scripts/v3/empty.script b/tests/stub/scripts/v3/empty.script deleted file mode 100644 index 3b38a3ca1..000000000 --- a/tests/stub/scripts/v3/empty.script +++ /dev/null @@ -1,5 +0,0 @@ -!: BOLT 3 -!: AUTO HELLO -!: AUTO GOODBYE -!: AUTO RESET -!: PORT 9001 diff --git a/tests/stub/scripts/v3/empty_explicit_hello_goodbye.script b/tests/stub/scripts/v3/empty_explicit_hello_goodbye.script deleted file mode 100644 index be98dd7aa..000000000 --- a/tests/stub/scripts/v3/empty_explicit_hello_goodbye.script +++ /dev/null @@ -1,7 +0,0 @@ -!: BOLT 3 -!: PORT 9001 - -C: HELLO {"user_agent": "test", "scheme": "basic", "principal": "test", "credentials": "test"} -S: SUCCESS {"server": "Neo4j/3.5.0", "connection_id": "123e4567-e89b-12d3-a456-426655440000"} -C: GOODBYE -S: diff --git a/tests/stub/scripts/v3/error_in_read_tx.script b/tests/stub/scripts/v3/error_in_read_tx.script deleted file mode 100644 index aeceb5c0c..000000000 --- a/tests/stub/scripts/v3/error_in_read_tx.script +++ /dev/null @@ -1,15 +0,0 @@ -!: BOLT 3 -!: AUTO HELLO -!: AUTO GOODBYE -!: PORT 9004 - -C: BEGIN {"mode": "r"} -S: SUCCESS {} - -C: RUN "X" {} {} - PULL_ALL -S: FAILURE {"code": "Neo.ClientError.Statement.SyntaxError", "message": "X"} - IGNORED {} - -C: RESET -S: SUCCESS {} diff --git a/tests/stub/scripts/v3/error_in_write_tx.script b/tests/stub/scripts/v3/error_in_write_tx.script deleted file mode 100644 index ab33f295d..000000000 --- a/tests/stub/scripts/v3/error_in_write_tx.script +++ /dev/null @@ -1,15 +0,0 @@ -!: BOLT 3 -!: AUTO HELLO -!: AUTO GOODBYE -!: PORT 9006 - -C: BEGIN {} -S: SUCCESS {} - -C: RUN "X" {} {} - PULL_ALL -S: FAILURE {"code": "Neo.ClientError.Statement.SyntaxError", "message": "X"} - IGNORED {} - -C: RESET -S: SUCCESS {} diff --git a/tests/stub/scripts/v3/fail_on_init.script b/tests/stub/scripts/v3/fail_on_init.script deleted file mode 100644 index b13869d6b..000000000 --- a/tests/stub/scripts/v3/fail_on_init.script +++ /dev/null @@ -1,6 +0,0 @@ -!: BOLT 3 -!: AUTO HELLO -!: AUTO GOODBYE -!: AUTO RESET - -S: diff --git a/tests/stub/scripts/v3/forbidden_on_read_only_database.script b/tests/stub/scripts/v3/forbidden_on_read_only_database.script deleted file mode 100644 index f49ac162b..000000000 --- a/tests/stub/scripts/v3/forbidden_on_read_only_database.script +++ /dev/null @@ -1,14 +0,0 @@ -!: BOLT 3 -!: AUTO HELLO -!: AUTO GOODBYE -!: AUTO RESET -!: AUTO DISCARD_ALL -!: AUTO BEGIN {} -!: AUTO COMMIT -!: AUTO ROLLBACK -!: PORT 9006 - -C: RUN "CREATE (n {name:'Bob'})" {} {} -C: PULL_ALL -S: FAILURE {"code": "Neo.ClientError.General.ForbiddenOnReadOnlyDatabase", "message": "Unable to write"} -S: IGNORED diff --git a/tests/stub/scripts/v3/get_routing_table.script b/tests/stub/scripts/v3/get_routing_table.script deleted file mode 100644 index 07e1ef208..000000000 --- a/tests/stub/scripts/v3/get_routing_table.script +++ /dev/null @@ -1,11 +0,0 @@ -!: BOLT 3 -!: AUTO HELLO -!: AUTO GOODBYE -!: AUTO RESET -!: PORT 9001 - -C: RUN "CALL dbms.cluster.routing.getRoutingTable($context)" {"context": {"address": "localhost:9001"}} {"mode": "r"} - PULL_ALL -S: SUCCESS {"fields": ["ttl", "servers"]} - RECORD [9223372036854775807, [{"addresses": ["127.0.0.1:9001"],"role": "WRITE"}, {"addresses": ["127.0.0.1:9002"], "role": "READ"},{"addresses": ["127.0.0.1:9001", "127.0.0.1:9002"], "role": "ROUTE"}]] - SUCCESS {} diff --git a/tests/stub/scripts/v3/get_routing_table_with_context.script b/tests/stub/scripts/v3/get_routing_table_with_context.script deleted file mode 100644 index 03a239672..000000000 --- a/tests/stub/scripts/v3/get_routing_table_with_context.script +++ /dev/null @@ -1,11 +0,0 @@ -!: BOLT 3 -!: AUTO HELLO -!: AUTO GOODBYE -!: AUTO RESET -!: PORT 9001 - -C: RUN "CALL dbms.cluster.routing.getRoutingTable($context)" {"context": {"name": "molly", "age": "1", "address": "localhost:9001"}} {"mode": "r"} - PULL_ALL -S: SUCCESS {"fields": ["ttl", "servers"]} - RECORD [9223372036854775807, [{"addresses": ["127.0.0.1:9001"],"role": "WRITE"}, {"addresses": ["127.0.0.1:9002"], "role": "READ"},{"addresses": ["127.0.0.1:9001", "127.0.0.1:9002"], "role": "ROUTE"}]] - SUCCESS {} diff --git a/tests/stub/scripts/v3/good_bookmarks.script b/tests/stub/scripts/v3/good_bookmarks.script deleted file mode 100644 index 40d3811e8..000000000 --- a/tests/stub/scripts/v3/good_bookmarks.script +++ /dev/null @@ -1,10 +0,0 @@ -!: BOLT 3 -!: AUTO HELLO -!: AUTO GOODBYE -!: AUTO RESET - -C: RUN "RETURN 1" {} {"bookmarks": ["bookmark1"]} - PULL_ALL -S: SUCCESS {"fields": ["1"]} - RECORD [1] - SUCCESS {} diff --git a/tests/stub/scripts/v3/good_metadata.script b/tests/stub/scripts/v3/good_metadata.script deleted file mode 100644 index 219f5bec2..000000000 --- a/tests/stub/scripts/v3/good_metadata.script +++ /dev/null @@ -1,10 +0,0 @@ -!: BOLT 3 -!: AUTO HELLO -!: AUTO GOODBYE -!: AUTO RESET - -C: RUN "RETURN 1" {} {"tx_metadata": {"foo": "bar"}} - PULL_ALL -S: SUCCESS {"fields": ["1"]} - RECORD [1] - SUCCESS {} diff --git a/tests/stub/scripts/v3/good_timeout.script b/tests/stub/scripts/v3/good_timeout.script deleted file mode 100644 index 3324e5d9e..000000000 --- a/tests/stub/scripts/v3/good_timeout.script +++ /dev/null @@ -1,10 +0,0 @@ -!: BOLT 3 -!: AUTO HELLO -!: AUTO GOODBYE -!: AUTO RESET - -C: RUN "RETURN 1" {} {"tx_timeout": 15000} - PULL_ALL -S: SUCCESS {"fields": ["1"]} - RECORD [1] - SUCCESS {} diff --git a/tests/stub/scripts/v3/incomplete_read_on_handshake.script b/tests/stub/scripts/v3/incomplete_read_on_handshake.script deleted file mode 100644 index a90e17a6c..000000000 --- a/tests/stub/scripts/v3/incomplete_read_on_handshake.script +++ /dev/null @@ -1,3 +0,0 @@ -!: HANDSHAKE 00 00 - -S: diff --git a/tests/stub/scripts/v3/incomplete_read_on_init.script b/tests/stub/scripts/v3/incomplete_read_on_init.script deleted file mode 100644 index 2888f0d29..000000000 --- a/tests/stub/scripts/v3/incomplete_read_on_init.script +++ /dev/null @@ -1,4 +0,0 @@ -!: BOLT 3 - -S: 00 40 B1 70 A2 -S: diff --git a/tests/stub/scripts/v3/non_router.script b/tests/stub/scripts/v3/non_router.script deleted file mode 100644 index f14da112f..000000000 --- a/tests/stub/scripts/v3/non_router.script +++ /dev/null @@ -1,11 +0,0 @@ -!: BOLT 3 -!: AUTO HELLO -!: AUTO GOODBYE -!: PORT 9001 - -C: RUN "CALL dbms.cluster.routing.getRoutingTable($context)" {"context": {"address": "localhost:9001"}} {"mode":"r"} - PULL_ALL -S: FAILURE {"code": "Neo.ClientError.Procedure.ProcedureNotFound", "message": "Not a router"} - IGNORED -C: RESET -S: SUCCESS {} diff --git a/tests/stub/scripts/v3/not_a_leader.script b/tests/stub/scripts/v3/not_a_leader.script deleted file mode 100644 index baa346523..000000000 --- a/tests/stub/scripts/v3/not_a_leader.script +++ /dev/null @@ -1,14 +0,0 @@ -!: BOLT 3 -!: AUTO HELLO -!: AUTO GOODBYE -!: AUTO RESET -!: AUTO DISCARD_ALL -!: AUTO BEGIN {} -!: AUTO COMMIT -!: AUTO ROLLBACK -!: PORT 9006 - -C: RUN "CREATE (n {name:'Bob'})" {} {} -C: PULL_ALL -S: FAILURE {"code": "Neo.ClientError.Cluster.NotALeader", "message": "Leader switched has happened"} -S: IGNORED diff --git a/tests/stub/scripts/v3/old_protocol.script b/tests/stub/scripts/v3/old_protocol.script deleted file mode 100644 index 889c9ce3f..000000000 --- a/tests/stub/scripts/v3/old_protocol.script +++ /dev/null @@ -1,4 +0,0 @@ -!: HANDSHAKE 00 00 01 00 - -S: 1 -S: diff --git a/tests/stub/scripts/v3/pull_all_port_9001.script b/tests/stub/scripts/v3/pull_all_port_9001.script deleted file mode 100644 index c7a89d0d1..000000000 --- a/tests/stub/scripts/v3/pull_all_port_9001.script +++ /dev/null @@ -1,15 +0,0 @@ -!: BOLT 3 -!: AUTO GOODBYE -!: AUTO RESET -!: PORT 9001 - -C: HELLO {"user_agent": "test", "scheme": "basic", "principal": "test", "credentials": "test"} -S: SUCCESS {"server": "Neo4j/3.0.0", "connection_id": "123e4567-e89b-12d3-a456-426655440000"} -C: RUN "UNWIND [1,2,3,4] AS x RETURN x" {} {"mode": "r"} -S: SUCCESS {"fields": ["x"], "t_first": 300} -C: PULL_ALL -S: RECORD [1] - RECORD [2] - RECORD [3] - RECORD [4] - SUCCESS {"bookmark": "neo4j:bookmark-test-1", "type": "r", "t_last": 500} diff --git a/tests/stub/scripts/v3/pull_all_port_9001_transaction_function.script b/tests/stub/scripts/v3/pull_all_port_9001_transaction_function.script deleted file mode 100644 index 1003b6efb..000000000 --- a/tests/stub/scripts/v3/pull_all_port_9001_transaction_function.script +++ /dev/null @@ -1,19 +0,0 @@ -!: BOLT 3 -!: AUTO GOODBYE -!: AUTO RESET -!: PORT 9001 - -C: HELLO {"user_agent": "test", "scheme": "basic", "principal": "test", "credentials": "test"} -S: SUCCESS {"server": "Neo4j/3.0.0", "connection_id": "123e4567-e89b-12d3-a456-426655440000"} -C: BEGIN {"mode": "r", "tx_metadata": {"foo": "bar"}, "tx_timeout": 3000} -S: SUCCESS {} -C: RUN "UNWIND [1,2,3,4] AS x RETURN x" {} {} -S: SUCCESS {"fields": ["x"], "t_first": 300} -C: PULL_ALL -S: RECORD [1] - RECORD [2] - RECORD [3] - RECORD [4] - SUCCESS {"type": "r", "t_last": 500} -C: COMMIT -S: SUCCESS {"bookmark": "neo4j:bookmark-test-1"} diff --git a/tests/stub/scripts/v3/readonly_false.script b/tests/stub/scripts/v3/readonly_false.script deleted file mode 100644 index 4e2dd91a3..000000000 --- a/tests/stub/scripts/v3/readonly_false.script +++ /dev/null @@ -1,10 +0,0 @@ -!: BOLT 3 -!: AUTO HELLO -!: AUTO GOODBYE -!: AUTO RESET - -C: RUN "RETURN 1" {} {} - PULL_ALL -S: SUCCESS {"fields": ["1"]} - RECORD [1] - SUCCESS {} diff --git a/tests/stub/scripts/v3/readonly_true.script b/tests/stub/scripts/v3/readonly_true.script deleted file mode 100644 index 0b80d8bc2..000000000 --- a/tests/stub/scripts/v3/readonly_true.script +++ /dev/null @@ -1,10 +0,0 @@ -!: BOLT 3 -!: AUTO HELLO -!: AUTO GOODBYE -!: AUTO RESET - -C: RUN "RETURN 1" {} {"mode": "r"} - PULL_ALL -S: SUCCESS {"fields": ["1"]} - RECORD [1] - SUCCESS {} diff --git a/tests/stub/scripts/v3/return_1.script b/tests/stub/scripts/v3/return_1.script deleted file mode 100644 index 70891f7e5..000000000 --- a/tests/stub/scripts/v3/return_1.script +++ /dev/null @@ -1,11 +0,0 @@ -!: BOLT 3 -!: AUTO HELLO -!: AUTO GOODBYE -!: AUTO RESET -!: PORT 9004 - -C: RUN "RETURN $x" {"x": 1} {"mode": "r"} - PULL_ALL -S: SUCCESS {"fields": ["x"]} - RECORD [1] - SUCCESS {} diff --git a/tests/stub/scripts/v3/return_1_four_times.script b/tests/stub/scripts/v3/return_1_four_times.script deleted file mode 100644 index 42ffdd8c8..000000000 --- a/tests/stub/scripts/v3/return_1_four_times.script +++ /dev/null @@ -1,29 +0,0 @@ -!: BOLT 3 -!: AUTO HELLO -!: AUTO GOODBYE -!: AUTO RESET -!: PORT 9004 - -C: RUN "RETURN $x" {"x": 1} {"mode": "r"} - PULL_ALL -S: SUCCESS {"fields": ["x"]} - RECORD [1] - SUCCESS {} - -C: RUN "RETURN $x" {"x": 1} {"mode": "r"} - PULL_ALL -S: SUCCESS {"fields": ["x"]} - RECORD [1] - SUCCESS {} - -C: RUN "RETURN $x" {"x": 1} {"mode": "r"} - PULL_ALL -S: SUCCESS {"fields": ["x"]} - RECORD [1] - SUCCESS {} - -C: RUN "RETURN $x" {"x": 1} {"mode": "r"} - PULL_ALL -S: SUCCESS {"fields": ["x"]} - RECORD [1] - SUCCESS {} diff --git a/tests/stub/scripts/v3/return_1_in_read_tx.script b/tests/stub/scripts/v3/return_1_in_read_tx.script deleted file mode 100644 index a759ac20b..000000000 --- a/tests/stub/scripts/v3/return_1_in_read_tx.script +++ /dev/null @@ -1,17 +0,0 @@ -!: BOLT 3 -!: AUTO HELLO -!: AUTO GOODBYE -!: AUTO RESET -!: PORT 9004 - -C: BEGIN {"mode": "r"} -S: SUCCESS {} - -C: RUN "RETURN 1" {} {} - PULL_ALL -S: SUCCESS {"fields": ["1"]} - RECORD [1] - SUCCESS {} - -C: COMMIT -S: SUCCESS {"bookmark": "bookmark:1"} diff --git a/tests/stub/scripts/v3/return_1_in_read_tx_twice.script b/tests/stub/scripts/v3/return_1_in_read_tx_twice.script deleted file mode 100644 index 19c2dc9f3..000000000 --- a/tests/stub/scripts/v3/return_1_in_read_tx_twice.script +++ /dev/null @@ -1,29 +0,0 @@ -!: BOLT 3 -!: AUTO HELLO -!: AUTO GOODBYE -!: AUTO RESET -!: PORT 9004 - -C: BEGIN {"mode": "r"} -S: SUCCESS {} - -C: RUN "RETURN 1" {} {} - PULL_ALL -S: SUCCESS {"fields": ["1"]} - RECORD [1] - SUCCESS {} - -C: COMMIT -S: SUCCESS {"bookmark": "bookmark:1"} - -C: BEGIN {"bookmarks": ["bookmark:1"], "mode": "r"} -S: SUCCESS {} - -C: RUN "RETURN 1" {} {} - PULL_ALL -S: SUCCESS {"fields": ["1"]} - RECORD [1] - SUCCESS {} - -C: COMMIT -S: SUCCESS {"bookmark": "bookmark:2"} diff --git a/tests/stub/scripts/v3/return_1_in_write_tx.script b/tests/stub/scripts/v3/return_1_in_write_tx.script deleted file mode 100644 index 7a84b752e..000000000 --- a/tests/stub/scripts/v3/return_1_in_write_tx.script +++ /dev/null @@ -1,17 +0,0 @@ -!: BOLT 3 -!: AUTO HELLO -!: AUTO GOODBYE -!: AUTO RESET -!: PORT 9006 - -C: BEGIN {} -S: SUCCESS {} - -C: RUN "RETURN 1" {} {} - PULL_ALL -S: SUCCESS {"fields": ["1"]} - RECORD [1] - SUCCESS {} - -C: COMMIT -S: SUCCESS {"bookmark": "bookmark:1"} diff --git a/tests/stub/scripts/v3/return_1_in_write_tx_twice.script b/tests/stub/scripts/v3/return_1_in_write_tx_twice.script deleted file mode 100644 index 2cabb275e..000000000 --- a/tests/stub/scripts/v3/return_1_in_write_tx_twice.script +++ /dev/null @@ -1,29 +0,0 @@ -!: BOLT 3 -!: AUTO HELLO -!: AUTO GOODBYE -!: AUTO RESET -!: PORT 9006 - -C: BEGIN {} -S: SUCCESS {} - -C: RUN "RETURN 1" {} {} - PULL_ALL -S: SUCCESS {"fields": ["1"]} - RECORD [1] - SUCCESS {} - -C: COMMIT -S: SUCCESS {"bookmark": "bookmark:1"} - -C: BEGIN {"bookmarks": ["bookmark:1"]} -S: SUCCESS {} - -C: RUN "RETURN 1" {} {} - PULL_ALL -S: SUCCESS {"fields": ["1"]} - RECORD [1] - SUCCESS {} - -C: COMMIT -S: SUCCESS {"bookmark": "bookmark:2"} diff --git a/tests/stub/scripts/v3/return_1_on_9002.script b/tests/stub/scripts/v3/return_1_on_9002.script deleted file mode 100644 index 6877f6ba6..000000000 --- a/tests/stub/scripts/v3/return_1_on_9002.script +++ /dev/null @@ -1,11 +0,0 @@ -!: BOLT 3 -!: AUTO HELLO -!: AUTO GOODBYE -!: AUTO RESET -!: PORT 9002 - -C: RUN "RETURN $x" {"x": 1} {"mode": "r"} - PULL_ALL -S: SUCCESS {"fields": ["x"]} - RECORD [1] - SUCCESS {} diff --git a/tests/stub/scripts/v3/return_1_on_9005.script b/tests/stub/scripts/v3/return_1_on_9005.script deleted file mode 100644 index 909421495..000000000 --- a/tests/stub/scripts/v3/return_1_on_9005.script +++ /dev/null @@ -1,11 +0,0 @@ -!: BOLT 3 -!: AUTO HELLO -!: AUTO GOODBYE -!: AUTO RESET -!: PORT 9005 - -C: RUN "RETURN $x" {"x": 1} {"mode": "r"} - PULL_ALL -S: SUCCESS {"fields": ["x"]} - RECORD [1] - SUCCESS {} diff --git a/tests/stub/scripts/v3/return_1_port_9001.script b/tests/stub/scripts/v3/return_1_port_9001.script deleted file mode 100644 index 5135a788d..000000000 --- a/tests/stub/scripts/v3/return_1_port_9001.script +++ /dev/null @@ -1,12 +0,0 @@ -!: BOLT 3 -!: AUTO GOODBYE -!: AUTO RESET -!: PORT 9001 - -C: HELLO {"user_agent": "test", "scheme": "basic", "principal": "test", "credentials": "test"} -S: SUCCESS {"server": "Neo4j/3.0.0", "connection_id": "123e4567-e89b-12d3-a456-426655440000"} -C: RUN "RETURN 1 AS x" {} {"mode": "r"} - PULL_ALL -S: SUCCESS {"fields": ["x"]} - RECORD [1] - SUCCESS {"bookmark": "neo4j:bookmark-test-1", "type": "r", "t_last": 5} diff --git a/tests/stub/scripts/v3/return_1_twice.script b/tests/stub/scripts/v3/return_1_twice.script deleted file mode 100644 index 7807a6a07..000000000 --- a/tests/stub/scripts/v3/return_1_twice.script +++ /dev/null @@ -1,17 +0,0 @@ -!: BOLT 3 -!: AUTO HELLO -!: AUTO GOODBYE -!: AUTO RESET -!: PORT 9004 - -C: RUN "RETURN $x" {"x": 1} {"mode": "r"} - PULL_ALL -S: SUCCESS {"fields": ["x"]} - RECORD [1] - SUCCESS {} - -C: RUN "RETURN $x" {"x": 1} {"mode": "r"} - PULL_ALL -S: SUCCESS {"fields": ["x"]} - RECORD [1] - SUCCESS {} diff --git a/tests/stub/scripts/v3/return_1_twice_in_read_tx.script b/tests/stub/scripts/v3/return_1_twice_in_read_tx.script deleted file mode 100644 index d6055cef7..000000000 --- a/tests/stub/scripts/v3/return_1_twice_in_read_tx.script +++ /dev/null @@ -1,23 +0,0 @@ -!: BOLT 3 -!: AUTO HELLO -!: AUTO GOODBYE -!: AUTO RESET -!: PORT 9004 - -C: BEGIN {"mode": "r"} -S: SUCCESS {} - -C: RUN "RETURN $x" {"x": 1} {} - PULL_ALL -S: SUCCESS {"fields": ["x"]} - RECORD [1] - SUCCESS {} - -C: RUN "RETURN $x" {"x": 1} {} - PULL_ALL -S: SUCCESS {"fields": ["x"]} - RECORD [1] - SUCCESS {} - -C: COMMIT -S: SUCCESS {"bookmark": "bookmark:1"} diff --git a/tests/stub/scripts/v3/return_2_in_read_tx.script b/tests/stub/scripts/v3/return_2_in_read_tx.script deleted file mode 100644 index bbda4f3d9..000000000 --- a/tests/stub/scripts/v3/return_2_in_read_tx.script +++ /dev/null @@ -1,17 +0,0 @@ -!: BOLT 3 -!: AUTO HELLO -!: AUTO GOODBYE -!: AUTO RESET -!: PORT 9004 - -C: BEGIN {"bookmarks": ["bookmark:1"], "mode": "r"} -S: SUCCESS {} - -C: RUN "RETURN 2" {} {} - PULL_ALL -S: SUCCESS {"fields": ["2"]} - RECORD [2] - SUCCESS {} - -C: COMMIT -S: SUCCESS {"bookmark": "bookmark:2"} diff --git a/tests/stub/scripts/v3/return_2_in_write_tx.script b/tests/stub/scripts/v3/return_2_in_write_tx.script deleted file mode 100644 index c03033a6a..000000000 --- a/tests/stub/scripts/v3/return_2_in_write_tx.script +++ /dev/null @@ -1,17 +0,0 @@ -!: BOLT 3 -!: AUTO HELLO -!: AUTO GOODBYE -!: AUTO RESET -!: PORT 9006 - -C: BEGIN {"bookmarks": ["bookmark:1"]} -S: SUCCESS {} - -C: RUN "RETURN 2" {} {} - PULL_ALL -S: SUCCESS {"fields": ["2"]} - RECORD [2] - SUCCESS {} - -C: COMMIT -S: SUCCESS {"bookmark": "bookmark:2"} diff --git a/tests/stub/scripts/v3/router.script b/tests/stub/scripts/v3/router.script deleted file mode 100644 index d5da9b0cb..000000000 --- a/tests/stub/scripts/v3/router.script +++ /dev/null @@ -1,11 +0,0 @@ -!: BOLT 3 -!: AUTO HELLO -!: AUTO GOODBYE -!: AUTO RESET -!: PORT 9001 - -C: RUN "CALL dbms.cluster.routing.getRoutingTable($context)" {"context": {"address": "localhost:9001"}} {"mode": "r"} - PULL_ALL -S: SUCCESS {"fields": ["ttl", "servers"]} - RECORD [300, [{"role":"ROUTE","addresses":["127.0.0.1:9001","127.0.0.1:9002","127.0.0.1:9003"]},{"role":"READ","addresses":["127.0.0.1:9004","127.0.0.1:9005"]},{"role":"WRITE","addresses":["127.0.0.1:9006"]}]] - SUCCESS {} diff --git a/tests/stub/scripts/v3/router_no_readers.script b/tests/stub/scripts/v3/router_no_readers.script deleted file mode 100644 index 8bcf624ff..000000000 --- a/tests/stub/scripts/v3/router_no_readers.script +++ /dev/null @@ -1,11 +0,0 @@ -!: BOLT 3 -!: AUTO HELLO -!: AUTO GOODBYE -!: AUTO RESET -!: PORT 9001 - -C: RUN "CALL dbms.cluster.routing.getRoutingTable($context)" {"context": {"address": "localhost:9001"}} {"mode": "r"} - PULL_ALL -S: SUCCESS {"fields": ["ttl", "servers"]} - RECORD [300, [{"role":"ROUTE","addresses":["127.0.0.1:9001","127.0.0.1:9002","127.0.0.1:9003"]},{"role":"READ","addresses":[]},{"role":"WRITE","addresses":["127.0.0.1:9006"]}]] - SUCCESS {} diff --git a/tests/stub/scripts/v3/router_no_routers.script b/tests/stub/scripts/v3/router_no_routers.script deleted file mode 100644 index c8b37d6f2..000000000 --- a/tests/stub/scripts/v3/router_no_routers.script +++ /dev/null @@ -1,11 +0,0 @@ -!: BOLT 3 -!: AUTO HELLO -!: AUTO GOODBYE -!: AUTO RESET -!: PORT 9001 - -C: RUN "CALL dbms.cluster.routing.getRoutingTable($context)" {"context": {"address": "localhost:9001"}} {"mode": "r"} - PULL_ALL -S: SUCCESS {"fields": ["ttl", "servers"]} - RECORD [300, [{"role":"ROUTE","addresses":[]},{"role":"READ","addresses":["127.0.0.1:9004","127.0.0.1:9005"]},{"role":"WRITE","addresses":["127.0.0.1:9006"]}]] - SUCCESS {} diff --git a/tests/stub/scripts/v3/router_no_writers.script b/tests/stub/scripts/v3/router_no_writers.script deleted file mode 100644 index 11b94d639..000000000 --- a/tests/stub/scripts/v3/router_no_writers.script +++ /dev/null @@ -1,11 +0,0 @@ -!: BOLT 3 -!: AUTO HELLO -!: AUTO GOODBYE -!: AUTO RESET -!: PORT 9001 - -C: RUN "CALL dbms.cluster.routing.getRoutingTable($context)" {"context": {"address": "localhost:9001"}} {"mode": "r"} - PULL_ALL -S: SUCCESS {"fields": ["ttl", "servers"]} - RECORD [300, [{"role":"ROUTE","addresses":["127.0.0.1:9001","127.0.0.1:9002","127.0.0.1:9003"]},{"role":"READ","addresses":["127.0.0.1:9004","127.0.0.1:9005"]},{"role":"WRITE","addresses":[]}]] - SUCCESS {} diff --git a/tests/stub/scripts/v3/router_with_multiple_servers.script b/tests/stub/scripts/v3/router_with_multiple_servers.script deleted file mode 100644 index fec9d25da..000000000 --- a/tests/stub/scripts/v3/router_with_multiple_servers.script +++ /dev/null @@ -1,11 +0,0 @@ -!: BOLT 3 -!: AUTO HELLO -!: AUTO GOODBYE -!: AUTO RESET -!: PORT 9001 - -C: RUN "CALL dbms.cluster.routing.getRoutingTable($context)" {"context": {"address": "localhost:9001"}} {"mode": "r"} - PULL_ALL -S: SUCCESS {"fields": ["ttl", "servers"]} - RECORD [300, [{"role":"ROUTE","addresses":["127.0.0.1:9001","127.0.0.1:9002"]},{"role":"READ","addresses":["127.0.0.1:9001","127.0.0.1:9003"]},{"role":"WRITE","addresses":["127.0.0.1:9004"]}]] - SUCCESS {} diff --git a/tests/stub/scripts/v3/router_with_multiple_writers.script b/tests/stub/scripts/v3/router_with_multiple_writers.script deleted file mode 100644 index c8e053727..000000000 --- a/tests/stub/scripts/v3/router_with_multiple_writers.script +++ /dev/null @@ -1,11 +0,0 @@ -!: BOLT 3 -!: AUTO HELLO -!: AUTO GOODBYE -!: AUTO RESET -!: PORT 9001 - -C: RUN "CALL dbms.cluster.routing.getRoutingTable($context)" {"context": {"address": "localhost:9001"}} {"mode": "r"} - PULL_ALL -S: SUCCESS {"fields": ["ttl", "servers"]} - RECORD [300, [{"role":"ROUTE","addresses":["127.0.0.1:9001","127.0.0.1:9002","127.0.0.1:9003"]},{"role":"READ","addresses":["127.0.0.1:9004","127.0.0.1:9005"]},{"role":"WRITE","addresses":["127.0.0.1:9006","127.0.0.1:9007"]}]] - SUCCESS {} diff --git a/tests/stub/scripts/v3/rude_reader.script b/tests/stub/scripts/v3/rude_reader.script deleted file mode 100644 index ea323e988..000000000 --- a/tests/stub/scripts/v3/rude_reader.script +++ /dev/null @@ -1,9 +0,0 @@ -!: BOLT 3 -!: AUTO HELLO -!: AUTO GOODBYE -!: AUTO RESET -!: PORT 9004 - -C: RUN "RETURN 1" {} {"mode": "r"} - PULL_ALL -S: diff --git a/tests/stub/scripts/v3/rude_router.script b/tests/stub/scripts/v3/rude_router.script deleted file mode 100644 index a29bbbd4c..000000000 --- a/tests/stub/scripts/v3/rude_router.script +++ /dev/null @@ -1,9 +0,0 @@ -!: BOLT 3 -!: AUTO HELLO -!: AUTO GOODBYE -!: AUTO RESET -!: PORT 9001 - -C: RUN "CALL dbms.cluster.routing.getRoutingTable($context)" {"context": {"address": "localhost:9001"}} {"mode": "r"} - PULL_ALL -S: diff --git a/tests/stub/scripts/v3/silent_router.script b/tests/stub/scripts/v3/silent_router.script deleted file mode 100644 index a844c7d7f..000000000 --- a/tests/stub/scripts/v3/silent_router.script +++ /dev/null @@ -1,10 +0,0 @@ -!: BOLT 3 -!: AUTO HELLO -!: AUTO GOODBYE -!: AUTO RESET -!: PORT 9001 - -C: RUN "CALL dbms.cluster.routing.getRoutingTable($context)" {"context": {"address": "localhost:9001"}} {"mode": "r"} - PULL_ALL -S: SUCCESS {"fields": ["ttl", "servers"]} - SUCCESS {} diff --git a/tests/stub/scripts/v3/unusable_value_on_handshake.script b/tests/stub/scripts/v3/unusable_value_on_handshake.script deleted file mode 100644 index 7433528e6..000000000 --- a/tests/stub/scripts/v3/unusable_value_on_handshake.script +++ /dev/null @@ -1,3 +0,0 @@ -!: HANDSHAKE DEAD BEEF - -S: diff --git a/tests/stub/scripts/v3/user_canceled_read.script b/tests/stub/scripts/v3/user_canceled_read.script deleted file mode 100644 index 2beeb89d3..000000000 --- a/tests/stub/scripts/v3/user_canceled_read.script +++ /dev/null @@ -1,15 +0,0 @@ -!: BOLT 3 -!: AUTO HELLO -!: AUTO GOODBYE -!: PORT 9004 - -C: BEGIN {"mode": "r"} -S: SUCCESS {} - -C: RUN "RETURN 1" {} {} - PULL_ALL -S: FAILURE {"code": "Neo.TransientError.Transaction.LockClientStopped", "message": "X"} - IGNORED {} - -C: RESET -S: SUCCESS {} diff --git a/tests/stub/scripts/v3/user_canceled_write.script b/tests/stub/scripts/v3/user_canceled_write.script deleted file mode 100644 index 88c3e1662..000000000 --- a/tests/stub/scripts/v3/user_canceled_write.script +++ /dev/null @@ -1,15 +0,0 @@ -!: BOLT 3 -!: AUTO HELLO -!: AUTO GOODBYE -!: PORT 9006 - -C: BEGIN {} -S: SUCCESS {} - -C: RUN "RETURN 1" {} {} - PULL_ALL -S: FAILURE {"code": "Neo.TransientError.Transaction.LockClientStopped", "message": "X"} - IGNORED {} - -C: RESET -S: SUCCESS {} diff --git a/tests/stub/scripts/v4x0/create_test_node_port_9006.script b/tests/stub/scripts/v4x0/create_test_node_port_9006.script deleted file mode 100644 index 9d7ad0b45..000000000 --- a/tests/stub/scripts/v4x0/create_test_node_port_9006.script +++ /dev/null @@ -1,10 +0,0 @@ -!: BOLT 4 -!: AUTO HELLO -!: AUTO GOODBYE -!: AUTO RESET -!: PORT 9006 - -C: RUN "CREATE (a $x)" {"x": {"name": "Alice"}} {} - PULL {"n": -1} -S: SUCCESS {"fields": []} - SUCCESS {} diff --git a/tests/stub/scripts/v4x0/dbms_routing_get_routing_table_system_default.script b/tests/stub/scripts/v4x0/dbms_routing_get_routing_table_system_default.script deleted file mode 100644 index ec865816d..000000000 --- a/tests/stub/scripts/v4x0/dbms_routing_get_routing_table_system_default.script +++ /dev/null @@ -1,14 +0,0 @@ -!: BOLT 4 -!: PORT 9001 - -C: HELLO {"user_agent": "test", "scheme": "basic", "principal": "test", "credentials": "test"} -S: SUCCESS {"server": "Neo4j/4.0.0", "connection_id": "12345678-1234-1234-1234-123456789000"} - -C: RUN "CALL dbms.routing.getRoutingTable($context)" {"context": {"address": "localhost:9001"}} {"mode": "r", "db": "system"} - PULL {"n": -1} -S: SUCCESS {"fields": ["ttl", "servers"]} - RECORD [1234, [{"role":"WRITE", "addresses":["127.0.0.1:9001"]}, {"role":"READ", "addresses":["127.0.0.1:9002", "127.0.0.1:9003"]}, {"role":"ROUTE", "addresses":["127.0.0.1:9001", "127.0.0.1:9002", "127.0.0.1:9003"]}]] - SUCCESS {"bookmark": "neo4j:bookmark-test-1", "type": "s", "t_last": 15, "db": "system"} - -C: GOODBYE -S: diff --git a/tests/stub/scripts/v4x0/dbms_routing_get_routing_table_system_neo4j.script b/tests/stub/scripts/v4x0/dbms_routing_get_routing_table_system_neo4j.script deleted file mode 100644 index ba93794d5..000000000 --- a/tests/stub/scripts/v4x0/dbms_routing_get_routing_table_system_neo4j.script +++ /dev/null @@ -1,14 +0,0 @@ -!: BOLT 4 -!: PORT 9001 - -C: HELLO {"user_agent": "test", "scheme": "basic", "principal": "test", "credentials": "test"} -S: SUCCESS {"server": "Neo4j/4.0.0", "connection_id": "12345678-1234-1234-1234-123456789000"} - -C: RUN "CALL dbms.routing.getRoutingTable($context, $database)" {"context": {"address": "localhost:9001"}, "database": "neo4j"} {"mode": "r", "db": "system"} - PULL {"n": -1} -S: SUCCESS {"fields": ["ttl", "servers"]} - RECORD [1234, [{"role":"WRITE", "addresses":["127.0.0.1:9001"]}, {"role":"READ", "addresses":["127.0.0.1:9002", "127.0.0.1:9003"]}, {"role":"ROUTE", "addresses":["127.0.0.1:9001", "127.0.0.1:9002", "127.0.0.1:9003"]}]] - SUCCESS {"bookmark": "neo4j:bookmark-test-1", "type": "r", "t_last": 15, "db": "neo4j"} - -C: GOODBYE -S: diff --git a/tests/stub/scripts/v4x0/disconnect_after_init.script b/tests/stub/scripts/v4x0/disconnect_after_init.script deleted file mode 100644 index bb2628df2..000000000 --- a/tests/stub/scripts/v4x0/disconnect_after_init.script +++ /dev/null @@ -1,7 +0,0 @@ -!: BOLT 4 -!: AUTO GOODBYE -!: AUTO RESET - -C: HELLO {"user_agent": "test", "scheme": "basic", "principal": "test", "credentials": "test"} -S: SUCCESS {"server": "Neo4j/4.0.0", "connection_id": "bolt-0"} -S: diff --git a/tests/stub/scripts/v4x0/disconnect_on_pull.script b/tests/stub/scripts/v4x0/disconnect_on_pull.script deleted file mode 100644 index 9200226ed..000000000 --- a/tests/stub/scripts/v4x0/disconnect_on_pull.script +++ /dev/null @@ -1,9 +0,0 @@ -!: BOLT 4 -!: AUTO HELLO -!: AUTO GOODBYE -!: AUTO RESET -!: PORT 9001 - -C: RUN "RETURN $x" {"x": 1} {"mode": "r"} - PULL {"n": -1} -S: diff --git a/tests/stub/scripts/v4x0/disconnect_on_pull_port_9004.script b/tests/stub/scripts/v4x0/disconnect_on_pull_port_9004.script deleted file mode 100644 index 59f5727fc..000000000 --- a/tests/stub/scripts/v4x0/disconnect_on_pull_port_9004.script +++ /dev/null @@ -1,9 +0,0 @@ -!: BOLT 4 -!: AUTO HELLO -!: AUTO GOODBYE -!: AUTO RESET -!: PORT 9004 - -C: RUN "RETURN $x" {"x": 1} {"mode": "r"} - PULL {"n": -1} -S: diff --git a/tests/stub/scripts/v4x0/disconnect_on_run.script b/tests/stub/scripts/v4x0/disconnect_on_run.script deleted file mode 100644 index 5508ee78b..000000000 --- a/tests/stub/scripts/v4x0/disconnect_on_run.script +++ /dev/null @@ -1,8 +0,0 @@ -!: BOLT 4 -!: AUTO HELLO -!: AUTO GOODBYE -!: AUTO RESET -!: PORT 9001 - -C: RUN "RETURN 1 AS x" {} {"mode": "r"} -S: diff --git a/tests/stub/scripts/v4x0/disconnect_on_run_port_9004.script b/tests/stub/scripts/v4x0/disconnect_on_run_port_9004.script deleted file mode 100644 index 42cfdde27..000000000 --- a/tests/stub/scripts/v4x0/disconnect_on_run_port_9004.script +++ /dev/null @@ -1,8 +0,0 @@ -!: BOLT 4 -!: AUTO HELLO -!: AUTO GOODBYE -!: AUTO RESET -!: PORT 9004 - -C: RUN "RETURN $x" {"x": 1} {"mode": "r"} -S: diff --git a/tests/stub/scripts/v4x0/empty.script b/tests/stub/scripts/v4x0/empty.script deleted file mode 100644 index 9b11fd90f..000000000 --- a/tests/stub/scripts/v4x0/empty.script +++ /dev/null @@ -1,5 +0,0 @@ -!: BOLT 4 -!: AUTO HELLO -!: AUTO GOODBYE -!: AUTO RESET -!: PORT 9001 diff --git a/tests/stub/scripts/v4x0/empty_explicit_hello_goodbye.script b/tests/stub/scripts/v4x0/empty_explicit_hello_goodbye.script deleted file mode 100644 index 5a2bdc883..000000000 --- a/tests/stub/scripts/v4x0/empty_explicit_hello_goodbye.script +++ /dev/null @@ -1,7 +0,0 @@ -!: BOLT 4 -!: PORT 9001 - -C: HELLO {"user_agent": "test", "scheme": "basic", "principal": "test", "credentials": "test"} -S: SUCCESS {"server": "Neo4j/4.0.0", "connection_id": "123e4567-e89b-12d3-a456-426655440000"} -C: GOODBYE -S: diff --git a/tests/stub/scripts/v4x0/pull_2_discard_all_port_9001.script b/tests/stub/scripts/v4x0/pull_2_discard_all_port_9001.script deleted file mode 100644 index 3a6ab5b86..000000000 --- a/tests/stub/scripts/v4x0/pull_2_discard_all_port_9001.script +++ /dev/null @@ -1,15 +0,0 @@ -!: BOLT 4 -!: AUTO GOODBYE -!: AUTO RESET -!: PORT 9001 - -C: HELLO {"user_agent": "test", "scheme": "basic", "principal": "test", "credentials": "test"} -S: SUCCESS {"server": "Neo4j/4.0.0", "connection_id": "123e4567-e89b-12d3-a456-426655440000"} -C: RUN "UNWIND [1,2,3,4] AS x RETURN x" {} {"mode": "r", "db": "test"} -S: SUCCESS {"fields": ["x"], "t_first": 300} -C: PULL {"n": 2} -S: RECORD [1] - RECORD [2] - SUCCESS {"has_more": true} -C: DISCARD {"n": -1} -S: SUCCESS {"bookmark": "neo4j:bookmark-test-1", "type": "r", "t_last": 500, "db": "test"} diff --git a/tests/stub/scripts/v4x0/pull_2_discard_all_pull_n_port_9001.script b/tests/stub/scripts/v4x0/pull_2_discard_all_pull_n_port_9001.script deleted file mode 100644 index 9a16e6631..000000000 --- a/tests/stub/scripts/v4x0/pull_2_discard_all_pull_n_port_9001.script +++ /dev/null @@ -1,24 +0,0 @@ -!: BOLT 4 -!: AUTO GOODBYE -!: AUTO RESET -!: PORT 9001 - -C: HELLO {"user_agent": "test", "scheme": "basic", "principal": "test", "credentials": "test"} -S: SUCCESS {"server": "Neo4j/4.0.0", "connection_id": "123e4567-e89b-12d3-a456-426655440000"} -C: RUN "UNWIND [1,2,3,4] AS x RETURN x" {} {"mode": "r", "db": "test"} -S: SUCCESS {"fields": ["x"], "t_first": 300} -C: PULL {"n": 2} -S: RECORD [1] - RECORD [2] - SUCCESS {"has_more": true} -C: DISCARD {"n": -1} -S: SUCCESS {"bookmark": "neo4j:bookmark-test-1", "type": "r", "t_last": 500, "db": "test"} -C: RUN "UNWIND [5,6,7,8] AS x RETURN x" {} {"mode": "r", "db": "test", "bookmarks": ["neo4j:bookmark-test-1"]} -C: PULL {"n": 2} -S: RECORD [5] - RECORD [6] - SUCCESS {"has_more": true} -C: PULL {"n": 2} -S: RECORD [7] - RECORD [8] -S: SUCCESS {"bookmark": "neo4j:bookmark-test-2", type": "r", "t_last": 500, "db": "test"} diff --git a/tests/stub/scripts/v4x0/pull_n_port_9001.script b/tests/stub/scripts/v4x0/pull_n_port_9001.script deleted file mode 100644 index 7b66ef1fa..000000000 --- a/tests/stub/scripts/v4x0/pull_n_port_9001.script +++ /dev/null @@ -1,17 +0,0 @@ -!: BOLT 4 -!: AUTO GOODBYE -!: AUTO RESET -!: PORT 9001 - -C: HELLO {"user_agent": "test", "scheme": "basic", "principal": "test", "credentials": "test"} -S: SUCCESS {"server": "Neo4j/4.0.0", "connection_id": "123e4567-e89b-12d3-a456-426655440000"} -C: RUN "UNWIND [1,2,3,4] AS x RETURN x" {} {"mode": "r", "db": "test"} -S: SUCCESS {"fields": ["x"], "t_first": 300} -C: PULL {"n": 2} -S: RECORD [1] - RECORD [2] - SUCCESS {"has_more": true} -C: PULL {"n": 2} -S: RECORD [3] - RECORD [4] - SUCCESS {"bookmark": "neo4j:bookmark-test-1", "type": "r", "t_last": 500, "db": "test"} diff --git a/tests/stub/scripts/v4x0/pull_n_port_9001_slow_network.script b/tests/stub/scripts/v4x0/pull_n_port_9001_slow_network.script deleted file mode 100644 index 3e02e96bb..000000000 --- a/tests/stub/scripts/v4x0/pull_n_port_9001_slow_network.script +++ /dev/null @@ -1,18 +0,0 @@ -!: BOLT 4 -!: AUTO GOODBYE -!: AUTO RESET -!: PORT 9001 - -C: HELLO {"user_agent": "test", "scheme": "basic", "principal": "test", "credentials": "test"} -S: SUCCESS {"server": "Neo4j/4.0.0", "connection_id": "123e4567-e89b-12d3-a456-426655440000"} -C: RUN "UNWIND [1,2,3,4] AS x RETURN x" {} {"mode": "r", "db": "test"} -S: SUCCESS {"fields": ["x"], "t_first": 300} -C: PULL {"n": 2} -S: RECORD [1] - RECORD [2] - SUCCESS {"has_more": true} -C: PULL {"n": 2} -S: 1 -S: RECORD [3] - RECORD [4] - SUCCESS {"bookmark": "neo4j:bookmark-test-1", "type": "r", "t_last": 500, "db": "test"} diff --git a/tests/stub/scripts/v4x0/return_1_four_times_port_9004.script b/tests/stub/scripts/v4x0/return_1_four_times_port_9004.script deleted file mode 100644 index a1f56d8b4..000000000 --- a/tests/stub/scripts/v4x0/return_1_four_times_port_9004.script +++ /dev/null @@ -1,29 +0,0 @@ -!: BOLT 4 -!: AUTO HELLO -!: AUTO GOODBYE -!: AUTO RESET -!: PORT 9004 - -C: RUN "RETURN 1" {} {"mode": "r"} - PULL {"n": -1} -S: SUCCESS {"fields": ["1"]} - RECORD [1] - SUCCESS {} - -C: RUN "RETURN 1" {} {"mode": "r"} - PULL {"n": -1} -S: SUCCESS {"fields": ["1"]} - RECORD [1] - SUCCESS {} - -C: RUN "RETURN 1" {} {"mode": "r"} - PULL {"n": -1} -S: SUCCESS {"fields": ["1"]} - RECORD [1] - SUCCESS {} - -C: RUN "RETURN 1" {} {"mode": "r"} - PULL {"n": -1} -S: SUCCESS {"fields": ["1"]} - RECORD [1] - SUCCESS {} diff --git a/tests/stub/scripts/v4x0/return_1_port_9001.script b/tests/stub/scripts/v4x0/return_1_port_9001.script deleted file mode 100644 index 188c18390..000000000 --- a/tests/stub/scripts/v4x0/return_1_port_9001.script +++ /dev/null @@ -1,12 +0,0 @@ -!: BOLT 4 -!: AUTO GOODBYE -!: AUTO RESET -!: PORT 9001 - -C: HELLO {"user_agent": "test", "scheme": "basic", "principal": "test", "credentials": "test"} -S: SUCCESS {"server": "Neo4j/4.0.0", "connection_id": "123e4567-e89b-12d3-a456-426655440000"} -C: RUN "RETURN 1 AS x" {} {"mode": "r"} - PULL {"n": -1} -S: SUCCESS {"fields": ["x"]} - RECORD [1] - SUCCESS {"bookmark": "neo4j:bookmark-test-1", "type": "r", "t_last": 5, "db": "system"} diff --git a/tests/stub/scripts/v4x0/return_1_port_9002.script b/tests/stub/scripts/v4x0/return_1_port_9002.script deleted file mode 100644 index c2778b13b..000000000 --- a/tests/stub/scripts/v4x0/return_1_port_9002.script +++ /dev/null @@ -1,11 +0,0 @@ -!: BOLT 4 -!: AUTO HELLO -!: AUTO GOODBYE -!: AUTO RESET -!: PORT 9002 - -C: RUN "RETURN 1 AS x" {} {"mode": "r"} - PULL {"n": -1} -S: SUCCESS {"fields": ["x"]} - RECORD [1] - SUCCESS {} diff --git a/tests/stub/scripts/v4x0/return_1_port_9004.script b/tests/stub/scripts/v4x0/return_1_port_9004.script deleted file mode 100644 index f523408c5..000000000 --- a/tests/stub/scripts/v4x0/return_1_port_9004.script +++ /dev/null @@ -1,11 +0,0 @@ -!: BOLT 4 -!: AUTO HELLO -!: AUTO GOODBYE -!: AUTO RESET -!: PORT 9004 - -C: RUN "RETURN $x" {"x": 1} {"mode": "r"} - PULL {"n": -1} -S: SUCCESS {"fields": ["x"]} - RECORD [1] - SUCCESS {} diff --git a/tests/stub/scripts/v4x0/return_1_port_9005.script b/tests/stub/scripts/v4x0/return_1_port_9005.script deleted file mode 100644 index d08f7246f..000000000 --- a/tests/stub/scripts/v4x0/return_1_port_9005.script +++ /dev/null @@ -1,11 +0,0 @@ -!: BOLT 4 -!: AUTO HELLO -!: AUTO GOODBYE -!: AUTO RESET -!: PORT 9005 - -C: RUN "RETURN 1 AS x" {} {"mode": "r"} - PULL {"n": -1} -S: SUCCESS {"fields": ["x"]} - RECORD [1] - SUCCESS {} diff --git a/tests/stub/scripts/v4x0/return_1_twice_port_9004.script b/tests/stub/scripts/v4x0/return_1_twice_port_9004.script deleted file mode 100644 index b97142b33..000000000 --- a/tests/stub/scripts/v4x0/return_1_twice_port_9004.script +++ /dev/null @@ -1,17 +0,0 @@ -!: BOLT 4 -!: AUTO HELLO -!: AUTO GOODBYE -!: AUTO RESET -!: PORT 9004 - -C: RUN "RETURN 1" {} {"mode": "r"} - PULL {"n": -1} -S: SUCCESS {"fields": ["1"]} - RECORD [1] - SUCCESS {} - -C: RUN "RETURN 1" {} {"mode": "r"} - PULL {"n": -1} -S: SUCCESS {"fields": ["1"]} - RECORD [1] - SUCCESS {} diff --git a/tests/stub/scripts/v4x0/router.script b/tests/stub/scripts/v4x0/router.script deleted file mode 100644 index 46dfcb64b..000000000 --- a/tests/stub/scripts/v4x0/router.script +++ /dev/null @@ -1,11 +0,0 @@ -!: BOLT 4 -!: AUTO HELLO -!: AUTO GOODBYE -!: AUTO RESET -!: PORT 9001 - -C: RUN "CALL dbms.routing.getRoutingTable($context)" {"context": {"address": "localhost:9001"}} {"mode": "r", "db": "system"} - PULL {"n": -1} -S: SUCCESS {"fields": ["ttl", "servers"]} - RECORD [300, [{"role":"ROUTE","addresses":["127.0.0.1:9001","127.0.0.1:9002","127.0.0.1:9003"]},{"role":"READ","addresses":["127.0.0.1:9004","127.0.0.1:9005"]},{"role":"WRITE","addresses":["127.0.0.1:9006"]}]] - SUCCESS {"bookmark": "neo4j:bookmark-test-1", "type": "s", "t_last": 15, "db": "system"} diff --git a/tests/stub/scripts/v4x0/router_get_routing_table_with_context.script b/tests/stub/scripts/v4x0/router_get_routing_table_with_context.script deleted file mode 100644 index 34c3d4cd1..000000000 --- a/tests/stub/scripts/v4x0/router_get_routing_table_with_context.script +++ /dev/null @@ -1,11 +0,0 @@ -!: BOLT 4 -!: AUTO HELLO -!: AUTO GOODBYE -!: AUTO RESET -!: PORT 9001 - -C: RUN "CALL dbms.routing.getRoutingTable($context)" {"context": {"name": "molly", "age": "1", "address": "localhost:9001"}} {"mode": "r", "db": "system"} - PULL {"n": -1} -S: SUCCESS {"fields": ["ttl", "servers"]} - RECORD [302, [{"role":"ROUTE", "addresses":["127.0.0.1:9001", "127.0.0.1:9002"]}, {"role":"READ", "addresses":["127.0.0.1:9002"]}, {"role":"WRITE", "addresses":["127.0.0.1:9001"]}]] - SUCCESS {"bookmark": "neo4j:bookmark-test-1", "type": "s", "t_last": 15, "db": "system"} diff --git a/tests/stub/scripts/v4x0/router_port_9001_one_read_port_9004_one_write_port_9006.script b/tests/stub/scripts/v4x0/router_port_9001_one_read_port_9004_one_write_port_9006.script deleted file mode 100644 index b373111f5..000000000 --- a/tests/stub/scripts/v4x0/router_port_9001_one_read_port_9004_one_write_port_9006.script +++ /dev/null @@ -1,13 +0,0 @@ -!: BOLT 4 -!: AUTO GOODBYE -!: AUTO RESET -!: PORT 9001 - -C: HELLO {"user_agent": "test", "scheme": "basic", "principal": "test", "credentials": "test"} -S: SUCCESS {"server": "Neo4j/4.0.0", "connection_id": "123e4567-e89b-12d3-a456-426655440000"} - -C: RUN "CALL dbms.routing.getRoutingTable($context)" {"context": {"address": "localhost:9001"}} {"mode": "r", "db": "system"} - PULL {"n": -1} -S: SUCCESS {"fields": ["ttl", "servers"]} - RECORD [300, [{"role":"ROUTE", "addresses":["127.0.0.1:9001", "127.0.0.1:9002"]}, {"role":"READ", "addresses":["127.0.0.1:9004"]}, {"role":"WRITE", "addresses":["127.0.0.1:9006"]}]] - SUCCESS {"bookmark": "neo4j:bookmark-test-1", "type": "s", "t_last": 5, "db": "system"} diff --git a/tests/stub/scripts/v4x0/router_role_route_share_port_with_role_read_and_role_write.script b/tests/stub/scripts/v4x0/router_role_route_share_port_with_role_read_and_role_write.script deleted file mode 100644 index 160afbee1..000000000 --- a/tests/stub/scripts/v4x0/router_role_route_share_port_with_role_read_and_role_write.script +++ /dev/null @@ -1,11 +0,0 @@ -!: BOLT 4 -!: AUTO HELLO -!: AUTO GOODBYE -!: AUTO RESET -!: PORT 9001 - -C: RUN "CALL dbms.routing.getRoutingTable($context)" {"context": {"address": "localhost:9001"}} {"mode": "r", "db": "system"} - PULL {"n": -1} -S: SUCCESS {"fields": ["ttl", "servers"]} - RECORD [302, [{"role":"ROUTE", "addresses":["127.0.0.1:9001", "127.0.0.1:9002"]}, {"role":"READ", "addresses":["127.0.0.1:9002"]}, {"role":"WRITE", "addresses":["127.0.0.1:9001"]}]] - SUCCESS {"bookmark": "neo4j:bookmark-test-1", "type": "s", "t_last": 5, "db": "system"} diff --git a/tests/stub/scripts/v4x0/router_with_no_role_read.script b/tests/stub/scripts/v4x0/router_with_no_role_read.script deleted file mode 100644 index 8039adddf..000000000 --- a/tests/stub/scripts/v4x0/router_with_no_role_read.script +++ /dev/null @@ -1,11 +0,0 @@ -!: BOLT 4 -!: AUTO HELLO -!: AUTO GOODBYE -!: AUTO RESET -!: PORT 9001 - -C: RUN "CALL dbms.routing.getRoutingTable($context)" {"context": {"address": "localhost:9001"}} {"mode": "r", "db": "system"} - PULL {"n": -1} -S: SUCCESS {"fields": ["ttl", "servers"]} - RECORD [304, [{"role":"ROUTE", "addresses":["127.0.0.1:9001", "127.0.0.1:9002", "127.0.0.1:9003"]}, {"role":"READ", "addresses":[]}, {"role":"WRITE", "addresses":["127.0.0.1:9006"]}]] - SUCCESS {"bookmark": "neo4j:bookmark-test-1", "type": "s", "t_last": 5, "db": "system"} diff --git a/tests/stub/scripts/v4x0/router_with_no_role_write.script b/tests/stub/scripts/v4x0/router_with_no_role_write.script deleted file mode 100644 index e1f8cbc9a..000000000 --- a/tests/stub/scripts/v4x0/router_with_no_role_write.script +++ /dev/null @@ -1,11 +0,0 @@ -!: BOLT 4 -!: AUTO HELLO -!: AUTO GOODBYE -!: AUTO RESET -!: PORT 9001 - -C: RUN "CALL dbms.routing.getRoutingTable($context)" {"context": {"address": "localhost:9001"}} {"mode": "r", "db": "system"} - PULL {"n": -1} -S: SUCCESS {"fields": ["ttl", "servers"]} - RECORD [303, [{"role":"ROUTE", "addresses":["127.0.0.1:9001", "127.0.0.1:9002", "127.0.0.1:9003"]}, {"role":"READ", "addresses":["127.0.0.1:9004", "127.0.0.1:9005"]}, {"role":"WRITE", "addresses":[]}]] - SUCCESS {"bookmark": "neo4j:bookmark-test-1", "type": "s", "t_last": 5, "db": "system"} diff --git a/tests/stub/scripts/v4x0/router_with_one_bookmark.script b/tests/stub/scripts/v4x0/router_with_one_bookmark.script deleted file mode 100644 index 6ec5dc8e3..000000000 --- a/tests/stub/scripts/v4x0/router_with_one_bookmark.script +++ /dev/null @@ -1,11 +0,0 @@ -!: BOLT 4 -!: AUTO HELLO -!: AUTO GOODBYE -!: AUTO RESET -!: PORT 9001 - -C: RUN "CALL dbms.routing.getRoutingTable($context)" {"context": {"address": "localhost:9001"}} {"mode": "r", "db": "system", "bookmarks": ["bookmark:1"]} - PULL {"n": -1} -S: SUCCESS {"fields": ["ttl", "servers"]} - RECORD [300, [{"role":"ROUTE","addresses":["127.0.0.1:9001","127.0.0.1:9002","127.0.0.1:9003"]},{"role":"READ","addresses":["127.0.0.1:9004","127.0.0.1:9005"]},{"role":"WRITE","addresses":["127.0.0.1:9006"]}]] - SUCCESS {"bookmark": "neo4j:bookmark-test-1", "type": "s", "t_last": 15, "db": "system"} diff --git a/tests/stub/scripts/v4x0/router_with_two_bookmarks.script b/tests/stub/scripts/v4x0/router_with_two_bookmarks.script deleted file mode 100644 index f87befa63..000000000 --- a/tests/stub/scripts/v4x0/router_with_two_bookmarks.script +++ /dev/null @@ -1,11 +0,0 @@ -!: BOLT 4 -!: AUTO HELLO -!: AUTO GOODBYE -!: AUTO RESET -!: PORT 9001 - -C: RUN "CALL dbms.routing.getRoutingTable($context)" {"context": {"address": "localhost:9001"}} {"mode": "r", "db": "system", "bookmarks": ["bookmark:0", "bookmark:1"]} - PULL {"n": -1} -S: SUCCESS {"fields": ["ttl", "servers"]} - RECORD [300, [{"role":"ROUTE","addresses":["127.0.0.1:9001","127.0.0.1:9002","127.0.0.1:9003"]},{"role":"READ","addresses":["127.0.0.1:9004","127.0.0.1:9005"]},{"role":"WRITE","addresses":["127.0.0.1:9006"]}]] - SUCCESS {"bookmark": "neo4j:bookmark-test-1", "type": "s", "t_last": 15, "db": "system"} diff --git a/tests/stub/scripts/v4x0/routing_table_failure_not_a_router.script b/tests/stub/scripts/v4x0/routing_table_failure_not_a_router.script deleted file mode 100644 index 520eca40c..000000000 --- a/tests/stub/scripts/v4x0/routing_table_failure_not_a_router.script +++ /dev/null @@ -1,10 +0,0 @@ -!: BOLT 4 -!: AUTO HELLO -!: AUTO GOODBYE - -C: RUN "CALL dbms.routing.getRoutingTable($context)" {"context": {"address": "localhost:9001"}} {"mode": "r", "db": "system"} - PULL {"n": -1} -S: FAILURE {"code": "Neo.ClientError.Procedure.ProcedureNotFound", "message": "Not a router"} - IGNORED -C: RESET -S: SUCCESS {} diff --git a/tests/stub/scripts/v4x0/routing_table_silent_router.script b/tests/stub/scripts/v4x0/routing_table_silent_router.script deleted file mode 100644 index ba0697646..000000000 --- a/tests/stub/scripts/v4x0/routing_table_silent_router.script +++ /dev/null @@ -1,9 +0,0 @@ -!: BOLT 4 -!: AUTO HELLO -!: AUTO GOODBYE -!: AUTO RESET - -C: RUN "CALL dbms.routing.getRoutingTable($context)" {"context": {"address": "localhost:9001"}} {"mode": "r", "db": "system"} - PULL {"n": -1} -S: SUCCESS {"fields": ["ttl", "servers"]} - SUCCESS {} diff --git a/tests/stub/scripts/v4x0/run_with_failure_cluster_not_a_leader.script b/tests/stub/scripts/v4x0/run_with_failure_cluster_not_a_leader.script deleted file mode 100644 index 48a9350cc..000000000 --- a/tests/stub/scripts/v4x0/run_with_failure_cluster_not_a_leader.script +++ /dev/null @@ -1,13 +0,0 @@ -!: BOLT 4 -!: AUTO HELLO -!: AUTO GOODBYE -!: AUTO RESET -!: AUTO BEGIN {} -!: AUTO COMMIT -!: AUTO ROLLBACK -!: PORT 9006 - -C: RUN "CREATE (n:TEST {name:'test'})" {} {} -C: PULL {"n": -1} -S: FAILURE {"code": "Neo.ClientError.Cluster.NotALeader", "message": "Leader switched has happened"} -S: IGNORED diff --git a/tests/stub/scripts/v4x0/run_with_failure_database_unavailable.script b/tests/stub/scripts/v4x0/run_with_failure_database_unavailable.script deleted file mode 100644 index 5796291a4..000000000 --- a/tests/stub/scripts/v4x0/run_with_failure_database_unavailable.script +++ /dev/null @@ -1,13 +0,0 @@ -!: BOLT 4 -!: AUTO HELLO -!: AUTO GOODBYE -!: AUTO RESET -!: AUTO BEGIN {} -!: AUTO COMMIT -!: AUTO ROLLBACK -!: PORT 9004 - -C: RUN "RETURN 1" {} {"mode": "r"} -C: PULL {"n": -1} -S: FAILURE {"code": "Neo.TransientError.General.DatabaseUnavailable", "message": "Database is busy doing store copy"} -S: IGNORED diff --git a/tests/stub/scripts/v4x0/run_with_failure_forbidden_on_read_only_database.script b/tests/stub/scripts/v4x0/run_with_failure_forbidden_on_read_only_database.script deleted file mode 100644 index 82d6043db..000000000 --- a/tests/stub/scripts/v4x0/run_with_failure_forbidden_on_read_only_database.script +++ /dev/null @@ -1,13 +0,0 @@ -!: BOLT 4 -!: AUTO HELLO -!: AUTO GOODBYE -!: AUTO RESET -!: AUTO BEGIN {} -!: AUTO COMMIT -!: AUTO ROLLBACK -!: PORT 9006 - -C: RUN "CREATE (n:TEST {name:'test'})" {} {} -C: PULL {"n": -1} -S: FAILURE {"code": "Neo.ClientError.General.ForbiddenOnReadOnlyDatabase", "message": "Unable to write"} -S: IGNORED diff --git a/tests/stub/scripts/v4x0/tx_bookmark_chain.script b/tests/stub/scripts/v4x0/tx_bookmark_chain.script deleted file mode 100644 index aa0e12704..000000000 --- a/tests/stub/scripts/v4x0/tx_bookmark_chain.script +++ /dev/null @@ -1,15 +0,0 @@ -!: BOLT 4 -!: AUTO HELLO -!: AUTO GOODBYE -!: AUTO RESET -!: PORT 9004 - -C: BEGIN {"bookmarks": ["bookmark:0", "bookmark:1"], "mode": "r"} -S: SUCCESS {} -C: COMMIT -S: SUCCESS {"bookmark": "bookmark:2"} - -C: BEGIN {"bookmarks": ["bookmark:2"], "mode": "r"} -S: SUCCESS {} -C: COMMIT -S: SUCCESS {"bookmark": "bookmark:3"} diff --git a/tests/stub/scripts/v4x0/tx_bookmark_chain_with_autocommit.script b/tests/stub/scripts/v4x0/tx_bookmark_chain_with_autocommit.script deleted file mode 100644 index 2230ecd23..000000000 --- a/tests/stub/scripts/v4x0/tx_bookmark_chain_with_autocommit.script +++ /dev/null @@ -1,20 +0,0 @@ -!: BOLT 4 -!: AUTO HELLO -!: AUTO GOODBYE -!: AUTO RESET -!: PORT 9004 - -C: BEGIN {"bookmarks": ["bookmark:1"], "mode": "r"} -S: SUCCESS {} -C: COMMIT -S: SUCCESS {"bookmark": "bookmark:2"} - -C: RUN "RETURN 1" {} {"bookmarks": ["bookmark:2"], "mode": "r"} - PULL {"n": -1} -S: SUCCESS {} - SUCCESS {"bookmark": "bookmark:3"} - -C: BEGIN {"bookmarks": ["bookmark:3"], "mode": "r"} -S: SUCCESS {} -C: COMMIT -S: SUCCESS {"bookmark": "bookmark:4"} diff --git a/tests/stub/scripts/v4x0/tx_connection_error_on_commit.script b/tests/stub/scripts/v4x0/tx_connection_error_on_commit.script deleted file mode 100644 index 79d07a519..000000000 --- a/tests/stub/scripts/v4x0/tx_connection_error_on_commit.script +++ /dev/null @@ -1,13 +0,0 @@ -!: BOLT 4 -!: AUTO HELLO -!: AUTO GOODBYE -!: AUTO RESET - -C: BEGIN {} - RUN "CREATE (n {name:'Bob'})" {} {} - PULL {"n": -1} -S: SUCCESS {} - SUCCESS {} - SUCCESS {} -C: COMMIT -S: diff --git a/tests/stub/scripts/v4x0/tx_pull_2_discard_all_port_9001.script b/tests/stub/scripts/v4x0/tx_pull_2_discard_all_port_9001.script deleted file mode 100644 index b2fc49827..000000000 --- a/tests/stub/scripts/v4x0/tx_pull_2_discard_all_port_9001.script +++ /dev/null @@ -1,19 +0,0 @@ -!: BOLT 4 -!: AUTO GOODBYE -!: AUTO RESET -!: PORT 9001 - -C: HELLO {"user_agent": "test", "scheme": "basic", "principal": "test", "credentials": "test"} -S: SUCCESS {"server": "Neo4j/4.0.0", "connection_id": "123e4567-e89b-12d3-a456-426655440000"} -C: BEGIN {"mode": "r", "db": "test", "tx_metadata": {"foo": "bar"}, "tx_timeout": 3000} -S: SUCCESS {} -C: RUN "UNWIND [1,2,3,4] AS x RETURN x" {} {} -S: SUCCESS {"fields": ["x"], "t_first": 300, "qid": 0} -C: PULL {"n": 2} -S: RECORD [1] - RECORD [2] - SUCCESS {"has_more": true} -C: DISCARD {"n": -1, "qid": 0} -S: SUCCESS {"type": "r", "t_last": 500, "db": "test"} -C: COMMIT -S: SUCCESS {"bookmark": "neo4j:bookmark-test-1"} diff --git a/tests/stub/scripts/v4x0/tx_pull_n_port_9001.script b/tests/stub/scripts/v4x0/tx_pull_n_port_9001.script deleted file mode 100644 index 9e8db0a69..000000000 --- a/tests/stub/scripts/v4x0/tx_pull_n_port_9001.script +++ /dev/null @@ -1,21 +0,0 @@ -!: BOLT 4 -!: AUTO GOODBYE -!: AUTO RESET -!: PORT 9001 - -C: HELLO {"user_agent": "test", "scheme": "basic", "principal": "test", "credentials": "test"} -S: SUCCESS {"server": "Neo4j/4.0.0", "connection_id": "123e4567-e89b-12d3-a456-426655440000"} -C: BEGIN {"mode": "r", "db": "test", "tx_metadata": {"foo": "bar"}, "tx_timeout": 3000} -S: SUCCESS {} -C: RUN "UNWIND [1,2,3,4] AS x RETURN x" {} {} -S: SUCCESS {"fields": ["x"], "t_first": 300, "qid": 0} -C: PULL {"n": 2} -S: RECORD [1] - RECORD [2] - SUCCESS {"has_more": true} -C: PULL {"n": 2, "qid": 0} -S: RECORD [3] - RECORD [4] - SUCCESS {"type": "r", "t_last": 500, "db": "test"} -C: COMMIT -S: SUCCESS {"bookmark": "neo4j:bookmark-test-1"} diff --git a/tests/stub/scripts/v4x0/tx_pull_n_port_9001_slow_network.script b/tests/stub/scripts/v4x0/tx_pull_n_port_9001_slow_network.script deleted file mode 100644 index fab6a370a..000000000 --- a/tests/stub/scripts/v4x0/tx_pull_n_port_9001_slow_network.script +++ /dev/null @@ -1,22 +0,0 @@ -!: BOLT 4 -!: AUTO GOODBYE -!: AUTO RESET -!: PORT 9001 - -C: HELLO {"user_agent": "test", "scheme": "basic", "principal": "test", "credentials": "test"} -S: SUCCESS {"server": "Neo4j/4.0.0", "connection_id": "123e4567-e89b-12d3-a456-426655440000"} -C: BEGIN {"mode": "r", "db": "test", "tx_metadata": {"foo": "bar"}, "tx_timeout": 3000} -S: SUCCESS {} -C: RUN "UNWIND [1,2,3,4] AS x RETURN x" {} {} -S: SUCCESS {"fields": ["x"], "t_first": 300, "qid": 0} -C: PULL {"n": 2} -S: RECORD [1] - RECORD [2] - SUCCESS {"has_more": true} -C: PULL {"n": 2, "qid": 0} -S: 1 -S: RECORD [3] - RECORD [4] - SUCCESS {"type": "r", "t_last": 500, "db": "test"} -C: COMMIT -S: SUCCESS {"bookmark": "neo4j:bookmark-test-1"} diff --git a/tests/stub/scripts/v4x0/tx_return_1_port_9004.script b/tests/stub/scripts/v4x0/tx_return_1_port_9004.script deleted file mode 100644 index 872b1ee27..000000000 --- a/tests/stub/scripts/v4x0/tx_return_1_port_9004.script +++ /dev/null @@ -1,17 +0,0 @@ -!: BOLT 4 -!: AUTO HELLO -!: AUTO GOODBYE -!: AUTO RESET -!: PORT 9004 - -C: BEGIN {"mode": "r"} -S: SUCCESS {} - -C: RUN "RETURN 1" {} {} - PULL {"n": -1} -S: SUCCESS {"fields": ["1"]} - RECORD [1] - SUCCESS {} - -C: COMMIT -S: SUCCESS {"bookmark": "bookmark:1"} diff --git a/tests/stub/scripts/v4x0/tx_return_1_port_9006.script b/tests/stub/scripts/v4x0/tx_return_1_port_9006.script deleted file mode 100644 index 5b78fb428..000000000 --- a/tests/stub/scripts/v4x0/tx_return_1_port_9006.script +++ /dev/null @@ -1,17 +0,0 @@ -!: BOLT 4 -!: AUTO HELLO -!: AUTO GOODBYE -!: AUTO RESET -!: PORT 9006 - -C: BEGIN {} -S: SUCCESS {} - -C: RUN "RETURN 1" {} {} - PULL {"n": -1} -S: SUCCESS {"fields": ["1"]} - RECORD [1] - SUCCESS {} - -C: COMMIT -S: SUCCESS {"bookmark": "bookmark:1"} diff --git a/tests/stub/scripts/v4x0/tx_return_1_reset_port_9004.script b/tests/stub/scripts/v4x0/tx_return_1_reset_port_9004.script deleted file mode 100644 index f655e90aa..000000000 --- a/tests/stub/scripts/v4x0/tx_return_1_reset_port_9004.script +++ /dev/null @@ -1,15 +0,0 @@ -!: BOLT 4 -!: AUTO HELLO -!: AUTO GOODBYE -!: PORT 9004 - -C: BEGIN {"mode": "r"} -S: SUCCESS {} - -C: RUN "RETURN 1" {} {} - PULL {"n": -1} -S: FAILURE {"code": "Neo.TransientError.Transaction.LockClientStopped", "message": "X"} - IGNORED {} - -C: RESET -S: SUCCESS {} diff --git a/tests/stub/scripts/v4x0/tx_return_1_reset_port_9006.script b/tests/stub/scripts/v4x0/tx_return_1_reset_port_9006.script deleted file mode 100644 index 4273d23d4..000000000 --- a/tests/stub/scripts/v4x0/tx_return_1_reset_port_9006.script +++ /dev/null @@ -1,15 +0,0 @@ -!: BOLT 4 -!: AUTO HELLO -!: AUTO GOODBYE -!: PORT 9006 - -C: BEGIN {} -S: SUCCESS {} - -C: RUN "RETURN 1" {} {} - PULL {"n": -1} -S: FAILURE {"code": "Neo.TransientError.Transaction.LockClientStopped", "message": "X"} - IGNORED {} - -C: RESET -S: SUCCESS {} diff --git a/tests/stub/scripts/v4x0/tx_return_1_twice_port_9004.script b/tests/stub/scripts/v4x0/tx_return_1_twice_port_9004.script deleted file mode 100644 index cf17f2644..000000000 --- a/tests/stub/scripts/v4x0/tx_return_1_twice_port_9004.script +++ /dev/null @@ -1,23 +0,0 @@ -!: BOLT 4 -!: AUTO HELLO -!: AUTO GOODBYE -!: AUTO RESET -!: PORT 9004 - -C: BEGIN {"mode": "r"} -S: SUCCESS {} - -C: RUN "RETURN 1" {} {} - PULL {"n": -1} -S: SUCCESS {"fields": ["1"]} - RECORD [1] - SUCCESS {} - -C: RUN "RETURN 1" {} {} - PULL {"n": -1} -S: SUCCESS {"fields": ["1"]} - RECORD [1] - SUCCESS {} - -C: COMMIT -S: SUCCESS {"bookmark": "bookmark:1"} diff --git a/tests/stub/scripts/v4x0/tx_return_2_with_bookmark_port_9004.script b/tests/stub/scripts/v4x0/tx_return_2_with_bookmark_port_9004.script deleted file mode 100644 index e6589258b..000000000 --- a/tests/stub/scripts/v4x0/tx_return_2_with_bookmark_port_9004.script +++ /dev/null @@ -1,17 +0,0 @@ -!: BOLT 4 -!: AUTO HELLO -!: AUTO GOODBYE -!: AUTO RESET -!: PORT 9004 - -C: BEGIN {"bookmarks": ["bookmark:1"], "mode": "r"} -S: SUCCESS {} - -C: RUN "RETURN 2" {} {} - PULL {"n": -1} -S: SUCCESS {"fields": ["2"]} - RECORD [2] - SUCCESS {} - -C: COMMIT -S: SUCCESS {"bookmark": "bookmark:2"} diff --git a/tests/stub/scripts/v4x0/tx_return_2_with_bookmark_port_9006.script b/tests/stub/scripts/v4x0/tx_return_2_with_bookmark_port_9006.script deleted file mode 100644 index 20281cc2b..000000000 --- a/tests/stub/scripts/v4x0/tx_return_2_with_bookmark_port_9006.script +++ /dev/null @@ -1,17 +0,0 @@ -!: BOLT 4 -!: AUTO HELLO -!: AUTO GOODBYE -!: AUTO RESET -!: PORT 9006 - -C: BEGIN {"bookmarks": ["bookmark:1"]} -S: SUCCESS {} - -C: RUN "RETURN 2" {} {} - PULL {"n": -1} -S: SUCCESS {"fields": ["2"]} - RECORD [2] - SUCCESS {} - -C: COMMIT -S: SUCCESS {"bookmark": "bookmark:2"} diff --git a/tests/stub/scripts/v4x0/tx_run_with_failure_syntax_error_port_9004.script b/tests/stub/scripts/v4x0/tx_run_with_failure_syntax_error_port_9004.script deleted file mode 100644 index 8bd711325..000000000 --- a/tests/stub/scripts/v4x0/tx_run_with_failure_syntax_error_port_9004.script +++ /dev/null @@ -1,15 +0,0 @@ -!: BOLT 4 -!: AUTO HELLO -!: AUTO GOODBYE -!: PORT 9004 - -C: BEGIN {"mode": "r"} -S: SUCCESS {} - -C: RUN "X" {} {} - PULL {"n": -1} -S: FAILURE {"code": "Neo.ClientError.Statement.SyntaxError", "message": "X"} - IGNORED {} - -C: RESET -S: SUCCESS {} diff --git a/tests/stub/scripts/v4x0/tx_run_with_failure_syntax_error_port_9006.script b/tests/stub/scripts/v4x0/tx_run_with_failure_syntax_error_port_9006.script deleted file mode 100644 index f877e4bfd..000000000 --- a/tests/stub/scripts/v4x0/tx_run_with_failure_syntax_error_port_9006.script +++ /dev/null @@ -1,15 +0,0 @@ -!: BOLT 4 -!: AUTO HELLO -!: AUTO GOODBYE -!: PORT 9006 - -C: BEGIN {} -S: SUCCESS {} - -C: RUN "X" {} {} - PULL {"n": -1} -S: FAILURE {"code": "Neo.ClientError.Statement.SyntaxError", "message": "X"} - IGNORED {} - -C: RESET -S: SUCCESS {} diff --git a/tests/stub/scripts/v4x0/tx_two_subsequent_return_1_port_9004.script b/tests/stub/scripts/v4x0/tx_two_subsequent_return_1_port_9004.script deleted file mode 100644 index 95afab577..000000000 --- a/tests/stub/scripts/v4x0/tx_two_subsequent_return_1_port_9004.script +++ /dev/null @@ -1,29 +0,0 @@ -!: BOLT 4 -!: AUTO HELLO -!: AUTO GOODBYE -!: AUTO RESET -!: PORT 9004 - -C: BEGIN {"mode": "r"} -S: SUCCESS {} - -C: RUN "RETURN 1" {} {} - PULL {"n": -1} -S: SUCCESS {"fields": ["1"]} - RECORD [1] - SUCCESS {} - -C: COMMIT -S: SUCCESS {"bookmark": "bookmark:1"} - -C: BEGIN {"bookmarks": ["bookmark:1"], "mode": "r"} -S: SUCCESS {} - -C: RUN "RETURN 1" {} {} - PULL {"n": -1} -S: SUCCESS {"fields": ["1"]} - RECORD [1] - SUCCESS {} - -C: COMMIT -S: SUCCESS {"bookmark": "bookmark:2"} diff --git a/tests/stub/scripts/v4x0/tx_two_subsequent_return_1_port_9006.script b/tests/stub/scripts/v4x0/tx_two_subsequent_return_1_port_9006.script deleted file mode 100644 index 727ad12bd..000000000 --- a/tests/stub/scripts/v4x0/tx_two_subsequent_return_1_port_9006.script +++ /dev/null @@ -1,29 +0,0 @@ -!: BOLT 4 -!: AUTO HELLO -!: AUTO GOODBYE -!: AUTO RESET -!: PORT 9006 - -C: BEGIN {} -S: SUCCESS {} - -C: RUN "RETURN 1" {} {} - PULL {"n": -1} -S: SUCCESS {"fields": ["1"]} - RECORD [1] - SUCCESS {} - -C: COMMIT -S: SUCCESS {"bookmark": "bookmark:1"} - -C: BEGIN {"bookmarks": ["bookmark:1"]} -S: SUCCESS {} - -C: RUN "RETURN 1" {} {} - PULL {"n": -1} -S: SUCCESS {"fields": ["1"]} - RECORD [1] - SUCCESS {} - -C: COMMIT -S: SUCCESS {"bookmark": "bookmark:2"} diff --git a/tests/stub/scripts/v4x1/empty_explicit_hello_goodbye.script b/tests/stub/scripts/v4x1/empty_explicit_hello_goodbye.script deleted file mode 100644 index 304979214..000000000 --- a/tests/stub/scripts/v4x1/empty_explicit_hello_goodbye.script +++ /dev/null @@ -1,7 +0,0 @@ -!: BOLT 4.1 -!: PORT 9001 - -C: HELLO {"user_agent": "test", "scheme": "basic", "principal": "test", "credentials": "test"} -S: SUCCESS {"server": "Neo4j/4.1.0", "connection_id": "123e4567-e89b-12d3-a456-426655440000"} -C: GOODBYE -S: diff --git a/tests/stub/scripts/v4x1/hello_with_routing_context_return_1_port_9002.script b/tests/stub/scripts/v4x1/hello_with_routing_context_return_1_port_9002.script deleted file mode 100644 index 8bd2038a1..000000000 --- a/tests/stub/scripts/v4x1/hello_with_routing_context_return_1_port_9002.script +++ /dev/null @@ -1,12 +0,0 @@ -!: BOLT 4.1 -!: AUTO GOODBYE -!: AUTO RESET -!: PORT 9002 - -C: HELLO {"scheme": "basic", "principal": "test", "credentials": "test", "user_agent": "test", "routing": {"address": "localhost:9001", "policy": "my_policy", "region": "china"}} -S: SUCCESS {"server": "Neo4j/4.1.0", "connection_id": "bolt-123456789"} -C: RUN "RETURN 1 AS x" {} {"mode": "r"} - PULL {"n": -1} -S: SUCCESS {"fields": ["x"]} - RECORD [1] - SUCCESS {"bookmark": "neo4j:bookmark-test-2", "type": "r", "t_last": 5, "db": "system"} diff --git a/tests/stub/scripts/v4x1/return_1_noop_port_9001.script b/tests/stub/scripts/v4x1/return_1_noop_port_9001.script deleted file mode 100644 index 41e5f1710..000000000 --- a/tests/stub/scripts/v4x1/return_1_noop_port_9001.script +++ /dev/null @@ -1,15 +0,0 @@ -!: BOLT 4.1 -!: AUTO GOODBYE -!: AUTO RESET -!: PORT 9001 - -C: HELLO {"user_agent": "test", "scheme": "basic", "principal": "test", "credentials": "test"} -S: SUCCESS {"server": "Neo4j/4.1.0", "connection_id": "123e4567-e89b-12d3-a456-426655440000"} -C: RUN "RETURN 1 AS x" {} {"mode": "r"} - PULL {"n": 2} -S: SUCCESS {"fields": ["x"]} - - - RECORD [1] - - SUCCESS {"bookmark": "neo4j:bookmark-test-1", "type": "r", "t_last": 5, "db": "neo4j"} diff --git a/tests/stub/scripts/v4x1/return_1_port_9001_bogus_server.script b/tests/stub/scripts/v4x1/return_1_port_9001_bogus_server.script deleted file mode 100644 index a41b8dab4..000000000 --- a/tests/stub/scripts/v4x1/return_1_port_9001_bogus_server.script +++ /dev/null @@ -1,10 +0,0 @@ -!: BOLT 4.1 -!: PORT 9001 - -C: HELLO {"user_agent": "test", "scheme": "basic", "principal": "test", "credentials": "test"} -S: SUCCESS {"server": "Bogus/4.1.0", "connection_id": "123e4567-e89b-12d3-a456-426655440000"} -C: RUN "RETURN 1 AS x" {} {"mode": "r"} - PULL {"n": -1} -S: SUCCESS {"fields": ["x"]} - RECORD [1] - SUCCESS {"bookmark": "neo4j:bookmark-test-1", "type": "r", "t_last": 5, "db": "neo4j"} diff --git a/tests/stub/scripts/v4x1/router_get_routing_table_with_context.script b/tests/stub/scripts/v4x1/router_get_routing_table_with_context.script deleted file mode 100644 index 830a1d7ab..000000000 --- a/tests/stub/scripts/v4x1/router_get_routing_table_with_context.script +++ /dev/null @@ -1,12 +0,0 @@ -!: BOLT 4.1 -!: AUTO GOODBYE -!: AUTO RESET -!: PORT 9001 - -C: HELLO {"scheme": "basic", "principal": "test", "credentials": "test", "user_agent": "test", "routing": {"address": "localhost:9001", "policy": "my_policy", "region": "china"}} -S: SUCCESS {"server": "Neo4j/4.1.0", "connection_id": "bolt-123456789"} -C: RUN "CALL dbms.routing.getRoutingTable($context)" {"context": {"address": "localhost:9001", "policy": "my_policy", "region": "china"}} {"mode": "r", "db": "system"} - PULL {"n": -1} -S: SUCCESS {"fields": ["ttl", "servers"]} - RECORD [4321, [{"addresses": ["127.0.0.1:9001"],"role": "WRITE"}, {"addresses": ["127.0.0.1:9002"], "role": "READ"}, {"addresses": ["127.0.0.1:9001", "127.0.0.1:9002"], "role": "ROUTE"}]] - SUCCESS {"bookmark": "neo4j:bookmark-test-1", "type": "r", "t_last": 5, "db": "system"} diff --git a/tests/stub/scripts/v4x2/empty_explicit_hello_goodbye.script b/tests/stub/scripts/v4x2/empty_explicit_hello_goodbye.script deleted file mode 100644 index efd6f6cdc..000000000 --- a/tests/stub/scripts/v4x2/empty_explicit_hello_goodbye.script +++ /dev/null @@ -1,7 +0,0 @@ -!: BOLT 4.2 -!: PORT 9001 - -C: HELLO {"user_agent": "test", "scheme": "basic", "principal": "test", "credentials": "test"} -S: SUCCESS {"server": "Neo4j/4.2.0", "connection_id": "123e4567-e89b-12d3-a456-426655440000"} -C: GOODBYE -S: diff --git a/tests/stub/scripts/v4x2/hello_with_routing_context_return_1_port_9002.script b/tests/stub/scripts/v4x2/hello_with_routing_context_return_1_port_9002.script deleted file mode 100644 index 9077e74b4..000000000 --- a/tests/stub/scripts/v4x2/hello_with_routing_context_return_1_port_9002.script +++ /dev/null @@ -1,12 +0,0 @@ -!: BOLT 4.2 -!: AUTO GOODBYE -!: AUTO RESET -!: PORT 9002 - -C: HELLO {"scheme": "basic", "principal": "test", "credentials": "test", "user_agent": "test", "routing": {"address": "localhost:9001", "policy": "my_policy", "region": "china"}} -S: SUCCESS {"server": "Neo4j/4.2.0", "connection_id": "bolt-123456789"} -C: RUN "RETURN 1 AS x" {} {"mode": "r"} - PULL {"n": -1} -S: SUCCESS {"fields": ["x"]} - RECORD [1] - SUCCESS {"bookmark": "neo4j:bookmark-test-2", "type": "r", "t_last": 5, "db": "system"} diff --git a/tests/stub/scripts/v4x2/return_1_noop_port_9001.script b/tests/stub/scripts/v4x2/return_1_noop_port_9001.script deleted file mode 100644 index c9335e538..000000000 --- a/tests/stub/scripts/v4x2/return_1_noop_port_9001.script +++ /dev/null @@ -1,15 +0,0 @@ -!: BOLT 4.2 -!: AUTO GOODBYE -!: AUTO RESET -!: PORT 9001 - -C: HELLO {"user_agent": "test", "scheme": "basic", "principal": "test", "credentials": "test"} -S: SUCCESS {"server": "Neo4j/4.2.0", "connection_id": "123e4567-e89b-12d3-a456-426655440000"} -C: RUN "RETURN 1 AS x" {} {"mode": "r"} - PULL {"n": 2} -S: SUCCESS {"fields": ["x"]} - - - RECORD [1] - - SUCCESS {"bookmark": "neo4j:bookmark-test-1", "type": "r", "t_last": 5, "db": "neo4j"} diff --git a/tests/stub/scripts/v4x2/return_1_port_9001_bogus_server.script b/tests/stub/scripts/v4x2/return_1_port_9001_bogus_server.script deleted file mode 100644 index 801a86fd7..000000000 --- a/tests/stub/scripts/v4x2/return_1_port_9001_bogus_server.script +++ /dev/null @@ -1,10 +0,0 @@ -!: BOLT 4.2 -!: PORT 9001 - -C: HELLO {"user_agent": "test", "scheme": "basic", "principal": "test", "credentials": "test"} -S: SUCCESS {"server": "Bogus/4.2.0", "connection_id": "123e4567-e89b-12d3-a456-426655440000"} -C: RUN "RETURN 1 AS x" {} {"mode": "r"} - PULL {"n": -1} -S: SUCCESS {"fields": ["x"]} - RECORD [1] - SUCCESS {"bookmark": "neo4j:bookmark-test-1", "type": "r", "t_last": 5, "db": "neo4j"} diff --git a/tests/stub/scripts/v4x2/router_get_routing_table_with_context.script b/tests/stub/scripts/v4x2/router_get_routing_table_with_context.script deleted file mode 100644 index 3c0878625..000000000 --- a/tests/stub/scripts/v4x2/router_get_routing_table_with_context.script +++ /dev/null @@ -1,12 +0,0 @@ -!: BOLT 4.2 -!: AUTO GOODBYE -!: AUTO RESET -!: PORT 9001 - -C: HELLO {"scheme": "basic", "principal": "test", "credentials": "test", "user_agent": "test", "routing": {"address": "localhost:9001", "policy": "my_policy", "region": "china"}} -S: SUCCESS {"server": "Neo4j/4.2.0", "connection_id": "bolt-123456789"} -C: RUN "CALL dbms.routing.getRoutingTable($context)" {"context": {"address": "localhost:9001", "policy": "my_policy", "region": "china"}} {"mode": "r", "db": "system"} - PULL {"n": -1} -S: SUCCESS {"fields": ["ttl", "servers"]} - RECORD [4321, [{"addresses": ["127.0.0.1:9001"],"role": "WRITE"}, {"addresses": ["127.0.0.1:9002"], "role": "READ"}, {"addresses": ["127.0.0.1:9001", "127.0.0.1:9002"], "role": "ROUTE"}]] - SUCCESS {"bookmark": "neo4j:bookmark-test-1", "type": "r", "t_last": 5, "db": "system"} diff --git a/tests/stub/test_directdriver.py b/tests/stub/test_directdriver.py deleted file mode 100644 index 492e323fc..000000000 --- a/tests/stub/test_directdriver.py +++ /dev/null @@ -1,93 +0,0 @@ -# Copyright (c) "Neo4j" -# Neo4j Sweden AB [http://neo4j.com] -# -# This file is part of Neo4j. -# -# 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. - - -import pytest - -from neo4j import ( - BoltDriver, - GraphDatabase, - READ_ACCESS, -) -from neo4j.exceptions import ServiceUnavailable -from tests.stub.conftest import StubCluster - - -# python -m pytest tests/stub/test_directdriver.py -s -v - - -driver_config = { - "encrypted": False, - "user_agent": "test", - "max_connection_lifetime": 1000, - "max_connection_pool_size": 10, - "keep_alive": True, - "resolver": None, -} - - -session_config = { - "default_access_mode": READ_ACCESS, - "connection_acquisition_timeout": 1.0, - "max_transaction_retry_time": 1.0, - "initial_retry_delay": 1.0, - "retry_delay_multiplier": 1.0, - "retry_delay_jitter_factor": 0.1, - "fetch_size": -1, -} - - -# TODO: those tests will stay until a uniform behavior across the drivers has -# been specified and tests are created in testkit -def test_direct_driver_with_wrong_port(driver_info): - # python -m pytest tests/stub/test_directdriver.py -s -v -k test_direct_driver_with_wrong_port - uri = "bolt://127.0.0.1:9002" - with pytest.raises(ServiceUnavailable): - driver = GraphDatabase.driver(uri, auth=driver_info["auth_token"], **driver_config) - driver.verify_connectivity() - - -@pytest.mark.parametrize( - "test_script, test_expected", - [ - ("v3/return_1_port_9001.script", "Neo4j/3.0.0"), - ("v4x0/return_1_port_9001.script", "Neo4j/4.0.0"), - ] -) -def test_direct_verify_connectivity(driver_info, test_script, test_expected): - # python -m pytest tests/stub/test_directdriver.py -s -v -k test_direct_verify_connectivity - with StubCluster(test_script): - uri = "bolt://localhost:9001" - with GraphDatabase.driver(uri, auth=driver_info["auth_token"], **driver_config) as driver: - assert isinstance(driver, BoltDriver) - assert driver.verify_connectivity(default_access_mode=READ_ACCESS) == test_expected - - -@pytest.mark.parametrize( - "test_script", - [ - "v3/disconnect_on_run.script", - "v4x0/disconnect_on_run.script", - ] -) -def test_direct_verify_connectivity_disconnect_on_run(driver_info, test_script): - # python -m pytest tests/stub/test_directdriver.py -s -v -k test_direct_verify_connectivity_disconnect_on_run - with StubCluster(test_script): - uri = "bolt://127.0.0.1:9001" - with GraphDatabase.driver(uri, auth=driver_info["auth_token"], **driver_config) as driver: - with pytest.raises(ServiceUnavailable): - driver.verify_connectivity(default_access_mode=READ_ACCESS) diff --git a/tests/stub/test_routingdriver.py b/tests/stub/test_routingdriver.py deleted file mode 100644 index c94b5d951..000000000 --- a/tests/stub/test_routingdriver.py +++ /dev/null @@ -1,61 +0,0 @@ -# Copyright (c) "Neo4j" -# Neo4j Sweden AB [http://neo4j.com] -# -# This file is part of Neo4j. -# -# 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. - - -import pytest - -from neo4j import ( - GraphDatabase, - Neo4jDriver, -) -from neo4j.exceptions import ServiceUnavailable -from tests.stub.conftest import StubCluster - - -# python -m pytest tests/stub/test_routingdriver.py -s -v -# TODO: those tests will stay until a uniform behavior across the drivers has -# been specified and tests are created in testkit -@pytest.mark.parametrize( - "test_script", - [ - "v3/router.script", - "v4x0/router_port_9001_one_read_port_9004_one_write_port_9006.script", - ] -) -def test_neo4j_driver_verify_connectivity(driver_info, test_script): - # python -m pytest tests/stub/test_routingdriver.py -s -v -k test_neo4j_driver_verify_connectivity - with StubCluster(test_script): - with GraphDatabase.driver(driver_info["uri_neo4j"], auth=driver_info["auth_token"], user_agent="test") as driver: - assert isinstance(driver, Neo4jDriver) - assert driver.verify_connectivity() is not None - - -# @pytest.mark.skip(reason="Flaky") -@pytest.mark.parametrize( - "test_script", - [ - "v3/router.script", - "v4x0/router_port_9001_one_read_port_9004_one_write_port_9006.script", - ] -) -def test_neo4j_driver_verify_connectivity_server_down(driver_info, test_script): - # python -m pytest tests/stub/test_routingdriver.py -s -v -k test_neo4j_driver_verify_connectivity_server_down - with GraphDatabase.driver(driver_info["uri_neo4j"], auth=driver_info["auth_token"], user_agent="test") as driver: - assert isinstance(driver, Neo4jDriver) - - with pytest.raises(ServiceUnavailable): - driver.verify_connectivity() diff --git a/tests/unit/async_/io/test_direct.py b/tests/unit/async_/io/test_direct.py index 10de2665d..666f93489 100644 --- a/tests/unit/async_/io/test_direct.py +++ b/tests/unit/async_/io/test_direct.py @@ -93,9 +93,11 @@ async def opener(addr, timeout): self.address = address async def acquire( - self, access_mode=None, timeout=None, database=None, bookmarks=None + self, access_mode=None, timeout=None, database=None, bookmarks=None, + lifeness_check_timeout=None ): - return await self._acquire(self.address, timeout) + return await self._acquire(self.address, timeout, + lifeness_check_timeout) @mark_async_test @@ -144,7 +146,7 @@ def assert_pool_size( address, expected_active, expected_inactive, pool): @mark_async_test async def test_pool_can_acquire(pool): address = ("127.0.0.1", 7687) - connection = await pool._acquire(address, timeout=3) + connection = await pool._acquire(address, 3, None) assert connection.address == address assert_pool_size(address, 1, 0, pool) @@ -152,8 +154,8 @@ async def test_pool_can_acquire(pool): @mark_async_test async def test_pool_can_acquire_twice(pool): address = ("127.0.0.1", 7687) - connection_1 = await pool._acquire(address, timeout=3) - connection_2 = await pool._acquire(address, timeout=3) + connection_1 = await pool._acquire(address, 3, None) + connection_2 = await pool._acquire(address, 3, None) assert connection_1.address == address assert connection_2.address == address assert connection_1 is not connection_2 @@ -164,8 +166,8 @@ async def test_pool_can_acquire_twice(pool): async def test_pool_can_acquire_two_addresses(pool): address_1 = ("127.0.0.1", 7687) address_2 = ("127.0.0.1", 7474) - connection_1 = await pool._acquire(address_1, timeout=3) - connection_2 = await pool._acquire(address_2, timeout=3) + connection_1 = await pool._acquire(address_1, 3, None) + connection_2 = await pool._acquire(address_2, 3, None) assert connection_1.address == address_1 assert connection_2.address == address_2 assert_pool_size(address_1, 1, 0, pool) @@ -175,7 +177,7 @@ async def test_pool_can_acquire_two_addresses(pool): @mark_async_test async def test_pool_can_acquire_and_release(pool): address = ("127.0.0.1", 7687) - connection = await pool._acquire(address, timeout=3) + connection = await pool._acquire(address, 3, None) assert_pool_size(address, 1, 0, pool) await pool.release(connection) assert_pool_size(address, 0, 1, pool) @@ -184,7 +186,7 @@ async def test_pool_can_acquire_and_release(pool): @mark_async_test async def test_pool_releasing_twice(pool): address = ("127.0.0.1", 7687) - connection = await pool._acquire(address, timeout=3) + connection = await pool._acquire(address, 3, None) await pool.release(connection) assert_pool_size(address, 0, 1, pool) await pool.release(connection) @@ -195,7 +197,7 @@ async def test_pool_releasing_twice(pool): async def test_pool_in_use_count(pool): address = ("127.0.0.1", 7687) assert pool.in_use_connection_count(address) == 0 - connection = await pool._acquire(address, timeout=3) + connection = await pool._acquire(address, 3, None) assert pool.in_use_connection_count(address) == 1 await pool.release(connection) assert pool.in_use_connection_count(address) == 0 @@ -205,10 +207,10 @@ async def test_pool_in_use_count(pool): async def test_pool_max_conn_pool_size(pool): async with AsyncFakeBoltPool((), max_connection_pool_size=1) as pool: address = ("127.0.0.1", 7687) - await pool._acquire(address, timeout=0) + await pool._acquire(address, 0, None) assert pool.in_use_connection_count(address) == 1 with pytest.raises(ClientError): - await pool._acquire(address, timeout=0) + await pool._acquire(address, 0, None) assert pool.in_use_connection_count(address) == 1 @@ -222,7 +224,7 @@ async def test_pool_reset_when_released(is_reset, pool): with mock.patch(f"{__name__}.{quick_connection_name}.reset", new_callable=AsyncMock) as reset_mock: is_reset_mock.return_value = is_reset - connection = await pool._acquire(address, timeout=3) + connection = await pool._acquire(address, 3, None) assert isinstance(connection, AsyncQuickConnection) assert is_reset_mock.call_count == 0 assert reset_mock.call_count == 0 diff --git a/tests/unit/async_/io/test_neo4j_pool.py b/tests/unit/async_/io/test_neo4j_pool.py index 8b70e10c9..cbf05f309 100644 --- a/tests/unit/async_/io/test_neo4j_pool.py +++ b/tests/unit/async_/io/test_neo4j_pool.py @@ -31,6 +31,10 @@ RoutingConfig, WorkspaceConfig, ) +from neo4j.exceptions import ( + ServiceUnavailable, + SessionExpired, +) from ...._async_compat import ( AsyncMock, @@ -257,3 +261,131 @@ async def test_release_does_not_resets_defunct_connections(opener): cx1.defunct.assert_called_once() cx1.is_reset_mock.asset_not_called() cx1.reset.asset_not_called() + + +@pytest.mark.parametrize("lifeness_timeout", (0, 1, 2)) +@mark_async_test +async def test_acquire_performs_no_lifeness_check_on_fresh_connection( + opener, lifeness_timeout +): + pool = AsyncNeo4jPool( + opener, PoolConfig(), WorkspaceConfig(), ROUTER_ADDRESS + ) + cx1 = await pool._acquire(READER_ADDRESS, 30, lifeness_timeout) + assert cx1.addr == READER_ADDRESS + cx1.reset.asset_not_called() + + +@pytest.mark.parametrize("lifeness_timeout", (0, 1, 2)) +@mark_async_test +async def test_acquire_performs_lifeness_check_on_existing_connection( + opener, lifeness_timeout +): + pool = AsyncNeo4jPool( + opener, PoolConfig(), WorkspaceConfig(), ROUTER_ADDRESS + ) + # populate the pool with a connection + cx1 = await pool._acquire(READER_ADDRESS, 30, lifeness_timeout) + + # make sure we assume the right state + assert cx1.addr == READER_ADDRESS + cx1.is_idle_for.assert_not_called() + cx1.reset.assert_not_called() + + cx1.is_idle_for.return_value = True + + # release the connection + await pool.release(cx1) + cx1.reset.assert_not_called() + + # then acquire it again and assert the lifeness check was performed + cx2 = await pool._acquire(READER_ADDRESS, 30, lifeness_timeout) + assert cx1 is cx2 + cx1.is_idle_for.assert_called_once_with(lifeness_timeout) + cx2.reset.assert_awaited_once() + + +@pytest.mark.parametrize("lifeness_error", + (OSError, ServiceUnavailable, SessionExpired)) +@mark_async_test +async def test_acquire_creates_connection_on_failed_lifeness_check( + opener, lifeness_error +): + def lifeness_side_effect(*args, **kwargs): + raise lifeness_error("lifeness check failed") + + lifeness_timeout = 1 + pool = AsyncNeo4jPool( + opener, PoolConfig(), WorkspaceConfig(), ROUTER_ADDRESS + ) + # populate the pool with a connection + cx1 = await pool._acquire(READER_ADDRESS, 30, lifeness_timeout) + + # make sure we assume the right state + assert cx1.addr == READER_ADDRESS + cx1.is_idle_for.assert_not_called() + cx1.reset.assert_not_called() + + cx1.is_idle_for.return_value = True + # simulate cx1 failing lifeness check + cx1.reset.side_effect = lifeness_side_effect + + # release the connection + await pool.release(cx1) + cx1.reset.assert_not_called() + + # then acquire it again and assert the lifeness check was performed + cx2 = await pool._acquire(READER_ADDRESS, 30, lifeness_timeout) + assert cx1 is not cx2 + assert cx1.addr == cx2.addr + cx1.is_idle_for.assert_called_once_with(lifeness_timeout) + cx2.reset.assert_not_called() + assert cx1 not in pool.connections[cx1.addr] + assert cx2 in pool.connections[cx1.addr] + + +@pytest.mark.parametrize("lifeness_error", + (OSError, ServiceUnavailable, SessionExpired)) +@mark_async_test +async def test_acquire_returns_other_connection_on_failed_lifeness_check( + opener, lifeness_error +): + def lifeness_side_effect(*args, **kwargs): + raise lifeness_error("lifeness check failed") + + lifeness_timeout = 1 + pool = AsyncNeo4jPool( + opener, PoolConfig(), WorkspaceConfig(), ROUTER_ADDRESS + ) + # populate the pool with a connection + cx1 = await pool._acquire(READER_ADDRESS, 30, lifeness_timeout) + cx2 = await pool._acquire(READER_ADDRESS, 30, lifeness_timeout) + + # make sure we assume the right state + assert cx1.addr == READER_ADDRESS + assert cx2.addr == READER_ADDRESS + assert cx1 is not cx2 + cx1.is_idle_for.assert_not_called() + cx2.is_idle_for.assert_not_called() + cx1.reset.assert_not_called() + + cx1.is_idle_for.return_value = True + cx2.is_idle_for.return_value = True + # simulate cx1 failing lifeness check + cx1.reset.side_effect = lifeness_side_effect + + # release the connection + await pool.release(cx1) + await pool.release(cx2) + cx1.reset.assert_not_called() + cx2.reset.assert_not_called() + + # then acquire it again and assert the lifeness check was performed + cx3 = await pool._acquire(READER_ADDRESS, 30, lifeness_timeout) + assert cx3 is cx2 + cx1.is_idle_for.assert_called_once_with(lifeness_timeout) + cx1.reset.assert_awaited_once() + cx3.is_idle_for.assert_called_once_with(lifeness_timeout) + cx3.reset.assert_awaited_once() + assert cx1 not in pool.connections[cx1.addr] + assert cx3 in pool.connections[cx1.addr] diff --git a/tests/unit/async_/test_driver.py b/tests/unit/async_/test_driver.py index 128504133..714a486f3 100644 --- a/tests/unit/async_/test_driver.py +++ b/tests/unit/async_/test_driver.py @@ -27,13 +27,17 @@ TRUST_ALL_CERTIFICATES, TRUST_SYSTEM_CA_SIGNED_CERTIFICATES, ) -from neo4j.api import WRITE_ACCESS +from neo4j.api import ( + READ_ACCESS, + WRITE_ACCESS, +) from neo4j.exceptions import ConfigurationError from ..._async_compat import ( mark_async_test, mock, ) +from .work import AsyncFakeConnection @pytest.mark.parametrize("protocol", ("bolt://", "bolt+s://", "bolt+ssc://")) @@ -195,3 +199,44 @@ async def test_driver_opens_write_session_by_default(uri, mocker): ) await driver.close() + + +@pytest.mark.parametrize("uri", ( + "bolt://127.0.0.1:9000", + "neo4j://127.0.0.1:9000", +)) +@mark_async_test +async def test_verify_connectivity(uri): + driver = AsyncGraphDatabase.driver(uri) + + try: + with mock.patch.object(driver, "_pool", autospec=True) as pool_mock: + ret = await driver.verify_connectivity() + finally: + await driver.close() + + assert ret is None + pool_mock.acquire.assert_awaited_once() + assert pool_mock.acquire.call_args.kwargs["lifeness_check_timeout"] == 0 + pool_mock.release.assert_awaited_once() + + +@pytest.mark.parametrize("uri", ( + "bolt://127.0.0.1:9000", + "neo4j://127.0.0.1:9000", +)) +@pytest.mark.parametrize("kwargs", ( + {"access_mode": WRITE_ACCESS}, + {"access_mode": READ_ACCESS}, + {"fetch_size": 69}, +)) +@mark_async_test +async def test_verify_connectivity_parameters_are_deprecated(uri, kwargs): + driver = AsyncGraphDatabase.driver(uri) + + try: + with mock.patch.object(driver, "_pool", autospec=True): + with pytest.warns(DeprecationWarning, match="configuration"): + await driver.verify_connectivity(**kwargs) + finally: + await driver.close() diff --git a/tests/unit/mixed/io/test_direct.py b/tests/unit/mixed/io/test_direct.py index 4cf6afa59..7f7577259 100644 --- a/tests/unit/mixed/io/test_direct.py +++ b/tests/unit/mixed/io/test_direct.py @@ -163,7 +163,7 @@ def assert_pool_size(self, address, expected_active, expected_inactive, def test_multithread(self): def acquire_release_conn(pool, address, acquired_counter, release_event): - conn = pool._acquire(address, timeout=3) + conn = pool._acquire(address, 3, None) acquired_counter.increment() release_event.wait() pool.release(conn) @@ -203,7 +203,7 @@ async def test_multi_coroutine(self): async def acquire_release_conn(pool_, address_, acquired_counter_, release_event_): try: - conn = await pool_._acquire(address_, timeout=3) + conn = await pool_._acquire(address_, 3, None) await acquired_counter_.increment() await release_event_.wait() await pool_.release(conn) diff --git a/tests/unit/sync/io/test_direct.py b/tests/unit/sync/io/test_direct.py index 9d1f2b568..0d8d6ad88 100644 --- a/tests/unit/sync/io/test_direct.py +++ b/tests/unit/sync/io/test_direct.py @@ -93,9 +93,11 @@ def opener(addr, timeout): self.address = address def acquire( - self, access_mode=None, timeout=None, database=None, bookmarks=None + self, access_mode=None, timeout=None, database=None, bookmarks=None, + lifeness_check_timeout=None ): - return self._acquire(self.address, timeout) + return self._acquire(self.address, timeout, + lifeness_check_timeout) @mark_sync_test @@ -144,7 +146,7 @@ def assert_pool_size( address, expected_active, expected_inactive, pool): @mark_sync_test def test_pool_can_acquire(pool): address = ("127.0.0.1", 7687) - connection = pool._acquire(address, timeout=3) + connection = pool._acquire(address, 3, None) assert connection.address == address assert_pool_size(address, 1, 0, pool) @@ -152,8 +154,8 @@ def test_pool_can_acquire(pool): @mark_sync_test def test_pool_can_acquire_twice(pool): address = ("127.0.0.1", 7687) - connection_1 = pool._acquire(address, timeout=3) - connection_2 = pool._acquire(address, timeout=3) + connection_1 = pool._acquire(address, 3, None) + connection_2 = pool._acquire(address, 3, None) assert connection_1.address == address assert connection_2.address == address assert connection_1 is not connection_2 @@ -164,8 +166,8 @@ def test_pool_can_acquire_twice(pool): def test_pool_can_acquire_two_addresses(pool): address_1 = ("127.0.0.1", 7687) address_2 = ("127.0.0.1", 7474) - connection_1 = pool._acquire(address_1, timeout=3) - connection_2 = pool._acquire(address_2, timeout=3) + connection_1 = pool._acquire(address_1, 3, None) + connection_2 = pool._acquire(address_2, 3, None) assert connection_1.address == address_1 assert connection_2.address == address_2 assert_pool_size(address_1, 1, 0, pool) @@ -175,7 +177,7 @@ def test_pool_can_acquire_two_addresses(pool): @mark_sync_test def test_pool_can_acquire_and_release(pool): address = ("127.0.0.1", 7687) - connection = pool._acquire(address, timeout=3) + connection = pool._acquire(address, 3, None) assert_pool_size(address, 1, 0, pool) pool.release(connection) assert_pool_size(address, 0, 1, pool) @@ -184,7 +186,7 @@ def test_pool_can_acquire_and_release(pool): @mark_sync_test def test_pool_releasing_twice(pool): address = ("127.0.0.1", 7687) - connection = pool._acquire(address, timeout=3) + connection = pool._acquire(address, 3, None) pool.release(connection) assert_pool_size(address, 0, 1, pool) pool.release(connection) @@ -195,7 +197,7 @@ def test_pool_releasing_twice(pool): def test_pool_in_use_count(pool): address = ("127.0.0.1", 7687) assert pool.in_use_connection_count(address) == 0 - connection = pool._acquire(address, timeout=3) + connection = pool._acquire(address, 3, None) assert pool.in_use_connection_count(address) == 1 pool.release(connection) assert pool.in_use_connection_count(address) == 0 @@ -205,10 +207,10 @@ def test_pool_in_use_count(pool): def test_pool_max_conn_pool_size(pool): with FakeBoltPool((), max_connection_pool_size=1) as pool: address = ("127.0.0.1", 7687) - pool._acquire(address, timeout=0) + pool._acquire(address, 0, None) assert pool.in_use_connection_count(address) == 1 with pytest.raises(ClientError): - pool._acquire(address, timeout=0) + pool._acquire(address, 0, None) assert pool.in_use_connection_count(address) == 1 @@ -222,7 +224,7 @@ def test_pool_reset_when_released(is_reset, pool): with mock.patch(f"{__name__}.{quick_connection_name}.reset", new_callable=Mock) as reset_mock: is_reset_mock.return_value = is_reset - connection = pool._acquire(address, timeout=3) + connection = pool._acquire(address, 3, None) assert isinstance(connection, QuickConnection) assert is_reset_mock.call_count == 0 assert reset_mock.call_count == 0 diff --git a/tests/unit/sync/io/test_neo4j_pool.py b/tests/unit/sync/io/test_neo4j_pool.py index fa46841e0..4d7b17380 100644 --- a/tests/unit/sync/io/test_neo4j_pool.py +++ b/tests/unit/sync/io/test_neo4j_pool.py @@ -31,6 +31,10 @@ RoutingConfig, WorkspaceConfig, ) +from neo4j.exceptions import ( + ServiceUnavailable, + SessionExpired, +) from ...._async_compat import ( mark_sync_test, @@ -257,3 +261,131 @@ def test_release_does_not_resets_defunct_connections(opener): cx1.defunct.assert_called_once() cx1.is_reset_mock.asset_not_called() cx1.reset.asset_not_called() + + +@pytest.mark.parametrize("lifeness_timeout", (0, 1, 2)) +@mark_sync_test +def test_acquire_performs_no_lifeness_check_on_fresh_connection( + opener, lifeness_timeout +): + pool = Neo4jPool( + opener, PoolConfig(), WorkspaceConfig(), ROUTER_ADDRESS + ) + cx1 = pool._acquire(READER_ADDRESS, 30, lifeness_timeout) + assert cx1.addr == READER_ADDRESS + cx1.reset.asset_not_called() + + +@pytest.mark.parametrize("lifeness_timeout", (0, 1, 2)) +@mark_sync_test +def test_acquire_performs_lifeness_check_on_existing_connection( + opener, lifeness_timeout +): + pool = Neo4jPool( + opener, PoolConfig(), WorkspaceConfig(), ROUTER_ADDRESS + ) + # populate the pool with a connection + cx1 = pool._acquire(READER_ADDRESS, 30, lifeness_timeout) + + # make sure we assume the right state + assert cx1.addr == READER_ADDRESS + cx1.is_idle_for.assert_not_called() + cx1.reset.assert_not_called() + + cx1.is_idle_for.return_value = True + + # release the connection + pool.release(cx1) + cx1.reset.assert_not_called() + + # then acquire it again and assert the lifeness check was performed + cx2 = pool._acquire(READER_ADDRESS, 30, lifeness_timeout) + assert cx1 is cx2 + cx1.is_idle_for.assert_called_once_with(lifeness_timeout) + cx2.reset.assert_called_once() + + +@pytest.mark.parametrize("lifeness_error", + (OSError, ServiceUnavailable, SessionExpired)) +@mark_sync_test +def test_acquire_creates_connection_on_failed_lifeness_check( + opener, lifeness_error +): + def lifeness_side_effect(*args, **kwargs): + raise lifeness_error("lifeness check failed") + + lifeness_timeout = 1 + pool = Neo4jPool( + opener, PoolConfig(), WorkspaceConfig(), ROUTER_ADDRESS + ) + # populate the pool with a connection + cx1 = pool._acquire(READER_ADDRESS, 30, lifeness_timeout) + + # make sure we assume the right state + assert cx1.addr == READER_ADDRESS + cx1.is_idle_for.assert_not_called() + cx1.reset.assert_not_called() + + cx1.is_idle_for.return_value = True + # simulate cx1 failing lifeness check + cx1.reset.side_effect = lifeness_side_effect + + # release the connection + pool.release(cx1) + cx1.reset.assert_not_called() + + # then acquire it again and assert the lifeness check was performed + cx2 = pool._acquire(READER_ADDRESS, 30, lifeness_timeout) + assert cx1 is not cx2 + assert cx1.addr == cx2.addr + cx1.is_idle_for.assert_called_once_with(lifeness_timeout) + cx2.reset.assert_not_called() + assert cx1 not in pool.connections[cx1.addr] + assert cx2 in pool.connections[cx1.addr] + + +@pytest.mark.parametrize("lifeness_error", + (OSError, ServiceUnavailable, SessionExpired)) +@mark_sync_test +def test_acquire_returns_other_connection_on_failed_lifeness_check( + opener, lifeness_error +): + def lifeness_side_effect(*args, **kwargs): + raise lifeness_error("lifeness check failed") + + lifeness_timeout = 1 + pool = Neo4jPool( + opener, PoolConfig(), WorkspaceConfig(), ROUTER_ADDRESS + ) + # populate the pool with a connection + cx1 = pool._acquire(READER_ADDRESS, 30, lifeness_timeout) + cx2 = pool._acquire(READER_ADDRESS, 30, lifeness_timeout) + + # make sure we assume the right state + assert cx1.addr == READER_ADDRESS + assert cx2.addr == READER_ADDRESS + assert cx1 is not cx2 + cx1.is_idle_for.assert_not_called() + cx2.is_idle_for.assert_not_called() + cx1.reset.assert_not_called() + + cx1.is_idle_for.return_value = True + cx2.is_idle_for.return_value = True + # simulate cx1 failing lifeness check + cx1.reset.side_effect = lifeness_side_effect + + # release the connection + pool.release(cx1) + pool.release(cx2) + cx1.reset.assert_not_called() + cx2.reset.assert_not_called() + + # then acquire it again and assert the lifeness check was performed + cx3 = pool._acquire(READER_ADDRESS, 30, lifeness_timeout) + assert cx3 is cx2 + cx1.is_idle_for.assert_called_once_with(lifeness_timeout) + cx1.reset.assert_called_once() + cx3.is_idle_for.assert_called_once_with(lifeness_timeout) + cx3.reset.assert_called_once() + assert cx1 not in pool.connections[cx1.addr] + assert cx3 in pool.connections[cx1.addr] diff --git a/tests/unit/sync/test_driver.py b/tests/unit/sync/test_driver.py index f084c709a..8b45a7f48 100644 --- a/tests/unit/sync/test_driver.py +++ b/tests/unit/sync/test_driver.py @@ -27,13 +27,17 @@ TRUST_ALL_CERTIFICATES, TRUST_SYSTEM_CA_SIGNED_CERTIFICATES, ) -from neo4j.api import WRITE_ACCESS +from neo4j.api import ( + READ_ACCESS, + WRITE_ACCESS, +) from neo4j.exceptions import ConfigurationError from ..._async_compat import ( mark_sync_test, mock, ) +from .work import FakeConnection @pytest.mark.parametrize("protocol", ("bolt://", "bolt+s://", "bolt+ssc://")) @@ -195,3 +199,44 @@ def test_driver_opens_write_session_by_default(uri, mocker): ) driver.close() + + +@pytest.mark.parametrize("uri", ( + "bolt://127.0.0.1:9000", + "neo4j://127.0.0.1:9000", +)) +@mark_sync_test +def test_verify_connectivity(uri): + driver = GraphDatabase.driver(uri) + + try: + with mock.patch.object(driver, "_pool", autospec=True) as pool_mock: + ret = driver.verify_connectivity() + finally: + driver.close() + + assert ret is None + pool_mock.acquire.assert_called_once() + assert pool_mock.acquire.call_args.kwargs["lifeness_check_timeout"] == 0 + pool_mock.release.assert_called_once() + + +@pytest.mark.parametrize("uri", ( + "bolt://127.0.0.1:9000", + "neo4j://127.0.0.1:9000", +)) +@pytest.mark.parametrize("kwargs", ( + {"access_mode": WRITE_ACCESS}, + {"access_mode": READ_ACCESS}, + {"fetch_size": 69}, +)) +@mark_sync_test +def test_verify_connectivity_parameters_are_deprecated(uri, kwargs): + driver = GraphDatabase.driver(uri) + + try: + with mock.patch.object(driver, "_pool", autospec=True): + with pytest.warns(DeprecationWarning, match="configuration"): + driver.verify_connectivity(**kwargs) + finally: + driver.close() diff --git a/tox.ini b/tox.ini index 1407cad26..aa4ea24fd 100644 --- a/tox.ini +++ b/tox.ini @@ -33,6 +33,5 @@ commands = coverage erase coverage run -m pytest -v {posargs} \ tests/unit \ - tests/stub \ tests/integration coverage report