From 7cb035979f62535fb2d5a88b086f2c490d0f6c2b Mon Sep 17 00:00:00 2001 From: Antonio Barcelos Date: Tue, 25 May 2021 12:38:35 +0200 Subject: [PATCH] Remove circular references in exceptions during the commit The error issue happened because the cursor failure is already the cause of the commit or rollback failure. This generates a circular reference when both of the exceptions are combined. This situation can create stack overflow when the exception is logged in some logging libraries. The solution don't add the cursor exception as the cause of the exception, this way it will not be wrongly combined creating a circular reference. --- .../internal/async/UnmanagedTransaction.java | 30 ++++--- .../neo4j/driver/integration/BookmarkIT.java | 9 +-- .../org/neo4j/driver/integration/QueryIT.java | 20 +++++ .../driver/integration/TransactionIT.java | 10 ++- .../integration/async/AsyncQueryIT.java | 81 +++++++++++++++++++ .../integration/async/AsyncSessionIT.java | 4 +- .../integration/async/AsyncTransactionIT.java | 6 ++ .../async/UnmanagedTransactionTest.java | 66 ++++++++++++++- .../java/org/neo4j/driver/util/TestUtil.java | 24 ++++++ 9 files changed, 228 insertions(+), 22 deletions(-) create mode 100644 driver/src/test/java/org/neo4j/driver/integration/async/AsyncQueryIT.java diff --git a/driver/src/main/java/org/neo4j/driver/internal/async/UnmanagedTransaction.java b/driver/src/main/java/org/neo4j/driver/internal/async/UnmanagedTransaction.java index 76683b1d54..acbd34e157 100644 --- a/driver/src/main/java/org/neo4j/driver/internal/async/UnmanagedTransaction.java +++ b/driver/src/main/java/org/neo4j/driver/internal/async/UnmanagedTransaction.java @@ -121,24 +121,29 @@ boolean isOpen() private volatile StateHolder state = StateHolder.of( State.ACTIVE ); - public UnmanagedTransaction(Connection connection, BookmarkHolder bookmarkHolder, long fetchSize ) + public UnmanagedTransaction( Connection connection, BookmarkHolder bookmarkHolder, long fetchSize ) + { + this( connection, bookmarkHolder, fetchSize, new ResultCursorsHolder() ); + } + + protected UnmanagedTransaction( Connection connection, BookmarkHolder bookmarkHolder, long fetchSize, ResultCursorsHolder resultCursors ) { this.connection = connection; this.protocol = connection.protocol(); this.bookmarkHolder = bookmarkHolder; - this.resultCursors = new ResultCursorsHolder(); + this.resultCursors = resultCursors; this.fetchSize = fetchSize; } - public CompletionStage beginAsync(Bookmark initialBookmark, TransactionConfig config ) + public CompletionStage beginAsync( Bookmark initialBookmark, TransactionConfig config ) { return protocol.beginTransaction( connection, initialBookmark, config ) - .handle( ( ignore, beginError ) -> - { - if ( beginError != null ) - { - if ( beginError instanceof AuthorizationExpiredException ) - { + .handle( ( ignore, beginError ) -> + { + if ( beginError != null ) + { + if ( beginError instanceof AuthorizationExpiredException ) + { connection.terminateAndRelease( AuthorizationExpiredException.DESCRIPTION ); } else @@ -176,7 +181,7 @@ else if ( state.value == State.ROLLED_BACK ) else { return resultCursors.retrieveNotConsumedError() - .thenCompose( error -> doCommitAsync().handle( handleCommitOrRollback( error ) ) ) + .thenCompose( error -> doCommitAsync( error ).handle( handleCommitOrRollback( error ) ) ) .whenComplete( ( ignore, error ) -> handleTransactionCompletion( State.COMMITTED, error ) ); } } @@ -249,12 +254,13 @@ else if ( state.value == State.TERMINATED ) } } - private CompletionStage doCommitAsync() + private CompletionStage doCommitAsync( Throwable cursorFailure ) { if ( state.value == State.TERMINATED ) { return failedFuture( new ClientException( "Transaction can't be committed. " + - "It has been rolled back either because of an error or explicit termination", state.causeOfTermination ) ); + "It has been rolled back either because of an error or explicit termination", + cursorFailure != state.causeOfTermination ? state.causeOfTermination : null ) ); } return protocol.commitTransaction( connection ).thenAccept( bookmarkHolder::setBookmark ); } diff --git a/driver/src/test/java/org/neo4j/driver/integration/BookmarkIT.java b/driver/src/test/java/org/neo4j/driver/integration/BookmarkIT.java index 93a917f5a8..d18494806c 100644 --- a/driver/src/test/java/org/neo4j/driver/integration/BookmarkIT.java +++ b/driver/src/test/java/org/neo4j/driver/integration/BookmarkIT.java @@ -18,20 +18,17 @@ */ package org.neo4j.driver.integration; -import org.hamcrest.BaseMatcher; -import org.hamcrest.Description; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.RegisterExtension; import java.util.UUID; +import org.neo4j.driver.Bookmark; import org.neo4j.driver.Driver; import org.neo4j.driver.Session; import org.neo4j.driver.Transaction; import org.neo4j.driver.exceptions.ClientException; -import org.neo4j.driver.Bookmark; -import org.neo4j.driver.internal.InternalBookmark; import org.neo4j.driver.internal.util.DisabledOnNeo4jWith; import org.neo4j.driver.internal.util.EnabledOnNeo4jWith; import org.neo4j.driver.internal.util.Neo4jFeature; @@ -47,6 +44,7 @@ import static org.neo4j.driver.internal.util.BookmarkUtil.assertBookmarkContainsSingleValue; import static org.neo4j.driver.internal.util.BookmarkUtil.assertBookmarkIsEmpty; import static org.neo4j.driver.internal.util.BookmarkUtil.assertBookmarksContainsSingleUniqueValues; +import static org.neo4j.driver.util.TestUtil.assertNoCircularReferences; @ParallelizableIT class BookmarkIT @@ -137,7 +135,8 @@ void bookmarkRemainsAfterTxFailure() Transaction tx = session.beginTransaction(); tx.run( "RETURN" ); - assertThrows( ClientException.class, tx::commit ); + ClientException e = assertThrows( ClientException.class, tx::commit ); + assertNoCircularReferences( e ); assertEquals( bookmark, session.lastBookmark() ); } diff --git a/driver/src/test/java/org/neo4j/driver/integration/QueryIT.java b/driver/src/test/java/org/neo4j/driver/integration/QueryIT.java index d6f7e0d903..c696756645 100644 --- a/driver/src/test/java/org/neo4j/driver/integration/QueryIT.java +++ b/driver/src/test/java/org/neo4j/driver/integration/QueryIT.java @@ -18,9 +18,13 @@ */ package org.neo4j.driver.integration; +import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.RegisterExtension; +import org.junit.jupiter.api.function.Executable; +import java.io.PrintWriter; +import java.io.StringWriter; import java.util.Collections; import java.util.Iterator; import java.util.List; @@ -36,8 +40,10 @@ import static java.util.Arrays.asList; import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.MatcherAssert.assertThat; +import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.neo4j.driver.Values.parameters; +import static org.neo4j.driver.util.TestUtil.assertNoCircularReferences; @ParallelizableIT class QueryIT @@ -182,4 +188,18 @@ void shouldFailForIllegalQueries() assertThrows( IllegalArgumentException.class, () -> session.run( (String) null ) ); assertThrows( IllegalArgumentException.class, () -> session.run( "" ) ); } + + @Test + void shouldBeAbleToLogSemanticWrongExceptions() { + try { + // When I run a query with the old syntax + session.writeTransaction(tx -> + tx.run( "MATCH (n:Element) WHERE n.name = {param} RETURN n", + parameters("param", "Luke" )).list()); + } catch ( Exception ex ) { + // And exception happens + // Then it should not have circular reference + assertNoCircularReferences(ex); + } + } } diff --git a/driver/src/test/java/org/neo4j/driver/integration/TransactionIT.java b/driver/src/test/java/org/neo4j/driver/integration/TransactionIT.java index 5be3f55849..0e77ff3edf 100644 --- a/driver/src/test/java/org/neo4j/driver/integration/TransactionIT.java +++ b/driver/src/test/java/org/neo4j/driver/integration/TransactionIT.java @@ -29,8 +29,8 @@ import org.neo4j.driver.Config; import org.neo4j.driver.Driver; import org.neo4j.driver.Record; -import org.neo4j.driver.Session; import org.neo4j.driver.Result; +import org.neo4j.driver.Session; import org.neo4j.driver.Transaction; import org.neo4j.driver.Value; import org.neo4j.driver.exceptions.ClientException; @@ -57,6 +57,7 @@ import static org.junit.jupiter.api.Assertions.assertTrue; import static org.neo4j.driver.internal.logging.DevNullLogging.DEV_NULL_LOGGING; import static org.neo4j.driver.internal.retry.RetrySettings.DEFAULT; +import static org.neo4j.driver.util.TestUtil.assertNoCircularReferences; @ParallelizableIT class TransactionIT @@ -291,7 +292,8 @@ void shouldRollBackTxIfErrorWithoutConsume() Transaction tx = session.beginTransaction(); tx.run( "invalid" ); // send run, pull_all - assertThrows( ClientException.class, tx::commit ); + ClientException e = assertThrows( ClientException.class, tx::commit ); + assertNoCircularReferences( e ); try ( Transaction anotherTx = session.beginTransaction() ) { @@ -385,7 +387,8 @@ void shouldBeResponsiveToThreadInterruptWhenWaitingForCommit() try { - assertThrows( ServiceUnavailableException.class, tx2::commit ); + ServiceUnavailableException e = assertThrows( ServiceUnavailableException.class, tx2::commit ); + assertNoCircularReferences( e ); } finally { @@ -481,6 +484,7 @@ void shouldRollbackWhenMarkedSuccessfulButOneQueryFails() } } ); + assertNoCircularReferences( error ); assertThat( error.code(), containsString( "SyntaxError" ) ); assertThat( error.getSuppressed().length, greaterThanOrEqualTo( 1 ) ); Throwable suppressed = error.getSuppressed()[0]; diff --git a/driver/src/test/java/org/neo4j/driver/integration/async/AsyncQueryIT.java b/driver/src/test/java/org/neo4j/driver/integration/async/AsyncQueryIT.java new file mode 100644 index 0000000000..5b28ab888d --- /dev/null +++ b/driver/src/test/java/org/neo4j/driver/integration/async/AsyncQueryIT.java @@ -0,0 +1,81 @@ +/* + * 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. + */ +package org.neo4j.driver.integration.async; + +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import reactor.core.publisher.Flux; +import reactor.core.publisher.Mono; + +import java.io.PrintWriter; +import java.io.StringWriter; +import java.util.ArrayList; +import java.util.concurrent.ExecutionException; + +import org.neo4j.driver.async.AsyncSession; +import org.neo4j.driver.util.DatabaseExtension; +import org.neo4j.driver.util.ParallelizableIT; + +import static org.neo4j.driver.Values.parameters; +import static org.neo4j.driver.util.TestUtil.assertNoCircularReferences; + +@ParallelizableIT +public class AsyncQueryIT +{ + private static final Logger LOGGER = LoggerFactory.getLogger( AsyncQueryIT.class ); + + @RegisterExtension + static final DatabaseExtension neo4j = new DatabaseExtension(); + + private AsyncSession session; + + @BeforeEach + void setUp() + { + session = neo4j.driver().asyncSession(); + } + + @AfterEach + void tearDown() + { + session.closeAsync(); + } + + @Test + void shouldBeAbleToLogSemanticWrongExceptions() throws ExecutionException, InterruptedException + { + session.writeTransactionAsync( tx -> Flux.from( + Mono.fromCompletionStage( + tx.runAsync( "MATCH (n:Element) WHERE n.name = {param} RETURN n", parameters("param", "Luke") ) + )).collectList().toFuture()) + + .toCompletableFuture() + .exceptionally( ex -> { + assertNoCircularReferences(ex); + return new ArrayList<>(); + } ) + .get(); + } + +} diff --git a/driver/src/test/java/org/neo4j/driver/integration/async/AsyncSessionIT.java b/driver/src/test/java/org/neo4j/driver/integration/async/AsyncSessionIT.java index 034b61fbd4..ed94aa2d70 100644 --- a/driver/src/test/java/org/neo4j/driver/integration/async/AsyncSessionIT.java +++ b/driver/src/test/java/org/neo4j/driver/integration/async/AsyncSessionIT.java @@ -83,6 +83,7 @@ import static org.neo4j.driver.internal.util.Matchers.syntaxError; import static org.neo4j.driver.internal.util.Neo4jFeature.BOLT_V3; import static org.neo4j.driver.internal.util.Neo4jFeature.BOLT_V4; +import static org.neo4j.driver.util.TestUtil.assertNoCircularReferences; import static org.neo4j.driver.util.TestUtil.await; import static org.neo4j.driver.util.TestUtil.awaitAll; @@ -381,7 +382,8 @@ void shouldRunAsyncTransactionThatCanNotBeRetried() InvocationTrackingWork work = new InvocationTrackingWork( "UNWIND [10, 5, 0] AS x CREATE (:Hi) RETURN 10/x" ); CompletionStage txStage = session.writeTransactionAsync( work ); - assertThrows( ClientException.class, () -> await( txStage ) ); + ClientException e = assertThrows( ClientException.class, () -> await( txStage ) ); + assertNoCircularReferences( e ); assertEquals( 1, work.invocationCount() ); assertEquals( 0, countNodesByLabel( "Hi" ) ); } diff --git a/driver/src/test/java/org/neo4j/driver/integration/async/AsyncTransactionIT.java b/driver/src/test/java/org/neo4j/driver/integration/async/AsyncTransactionIT.java index b710cdf87a..810ca50ab8 100644 --- a/driver/src/test/java/org/neo4j/driver/integration/async/AsyncTransactionIT.java +++ b/driver/src/test/java/org/neo4j/driver/integration/async/AsyncTransactionIT.java @@ -69,6 +69,7 @@ import static org.neo4j.driver.internal.util.Iterables.single; import static org.neo4j.driver.internal.util.Matchers.containsResultAvailableAfterAndResultConsumedAfter; import static org.neo4j.driver.internal.util.Matchers.syntaxError; +import static org.neo4j.driver.util.TestUtil.assertNoCircularReferences; import static org.neo4j.driver.util.TestUtil.await; @ParallelizableIT @@ -677,6 +678,7 @@ void shouldFailToCommitWhenQueriesFailAndErrorNotConsumed() throws InterruptedEx tx.runAsync( "CREATE (:TestNode)" ); ClientException e = assertThrows( ClientException.class, () -> await( tx.commitAsync() ) ); + assertNoCircularReferences( e ); assertEquals( "/ by zero", e.getMessage() ); } @@ -688,6 +690,7 @@ void shouldPropagateRunFailureFromCommit() tx.runAsync( "RETURN ILLEGAL" ); ClientException e = assertThrows( ClientException.class, () -> await( tx.commitAsync() ) ); + assertNoCircularReferences( e ); assertThat( e.getMessage(), containsString( "ILLEGAL" ) ); } @@ -699,6 +702,7 @@ void shouldPropagateBlockedRunFailureFromCommit() await( tx.runAsync( "RETURN 42 / 0" ) ); ClientException e = assertThrows( ClientException.class, () -> await( tx.commitAsync() ) ); + assertNoCircularReferences( e ); assertThat( e.getMessage(), containsString( "/ by zero" ) ); } @@ -732,6 +736,7 @@ void shouldPropagatePullAllFailureFromCommit() tx.runAsync( "UNWIND [1, 2, 3, 'Hi'] AS x RETURN 10 / x" ); ClientException e = assertThrows( ClientException.class, () -> await( tx.commitAsync() ) ); + assertNoCircularReferences( e ); assertThat( e.code(), containsString( "TypeError" ) ); } @@ -743,6 +748,7 @@ void shouldPropagateBlockedPullAllFailureFromCommit() await( tx.runAsync( "UNWIND [1, 2, 3, 'Hi'] AS x RETURN 10 / x" ) ); ClientException e = assertThrows( ClientException.class, () -> await( tx.commitAsync() ) ); + assertNoCircularReferences( e ); assertThat( e.code(), containsString( "TypeError" ) ); } diff --git a/driver/src/test/java/org/neo4j/driver/internal/async/UnmanagedTransactionTest.java b/driver/src/test/java/org/neo4j/driver/internal/async/UnmanagedTransactionTest.java index 831c579ffc..74881b4bc8 100644 --- a/driver/src/test/java/org/neo4j/driver/internal/async/UnmanagedTransactionTest.java +++ b/driver/src/test/java/org/neo4j/driver/internal/async/UnmanagedTransactionTest.java @@ -30,12 +30,14 @@ import org.neo4j.driver.TransactionConfig; import org.neo4j.driver.exceptions.ClientException; import org.neo4j.driver.internal.DefaultBookmarkHolder; +import org.neo4j.driver.internal.FailableCursor; import org.neo4j.driver.internal.InternalBookmark; import org.neo4j.driver.internal.messaging.v4.BoltProtocolV4; import org.neo4j.driver.internal.spi.Connection; import org.neo4j.driver.internal.spi.ResponseHandler; import static java.util.Collections.emptyMap; +import static java.util.concurrent.CompletableFuture.completedFuture; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertThrows; @@ -43,10 +45,13 @@ import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.argThat; import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.inOrder; +import static org.mockito.Mockito.mock; import static org.mockito.Mockito.never; import static org.mockito.Mockito.verify; import static org.neo4j.driver.internal.handlers.pulln.FetchSizeUtil.UNLIMITED_FETCH_SIZE; +import static org.neo4j.driver.util.TestUtil.assertNoCircularReferences; import static org.neo4j.driver.util.TestUtil.await; import static org.neo4j.driver.util.TestUtil.beginMessage; import static org.neo4j.driver.util.TestUtil.connectionMock; @@ -195,7 +200,7 @@ void shouldReleaseConnectionWhenTerminatedAndCommitted() Connection connection = connectionMock(); UnmanagedTransaction tx = new UnmanagedTransaction( connection, new DefaultBookmarkHolder(), UNLIMITED_FETCH_SIZE ); - tx.markTerminated( null ); + tx.markTerminated( null ); assertThrows( ClientException.class, () -> await( tx.commitAsync() ) ); @@ -203,6 +208,54 @@ void shouldReleaseConnectionWhenTerminatedAndCommitted() verify( connection ).release(); } + @Test + void shouldNotCreateCircularExceptionWhenTerminationCauseEqualsToCursorFailure() + { + Connection connection = connectionMock(); + ClientException terminationCause = new ClientException( "Custom exception" ); + ResultCursorsHolder resultCursorsHolder = mockResultCursorWith( terminationCause ); + UnmanagedTransaction tx = new UnmanagedTransaction( connection, new DefaultBookmarkHolder(), UNLIMITED_FETCH_SIZE, resultCursorsHolder ); + + tx.markTerminated( terminationCause ); + + ClientException e = assertThrows( ClientException.class, () -> await( tx.commitAsync() ) ); + assertNoCircularReferences( e ); + assertEquals( terminationCause, e ); + } + + @Test + void shouldNotCreateCircularExceptionWhenTerminationCauseDifferentFromCursorFailure() + { + Connection connection = connectionMock(); + ClientException terminationCause = new ClientException( "Custom exception" ); + ResultCursorsHolder resultCursorsHolder = mockResultCursorWith( new ClientException( "Cursor error" ) ); + UnmanagedTransaction tx = new UnmanagedTransaction( connection, new DefaultBookmarkHolder(), UNLIMITED_FETCH_SIZE, resultCursorsHolder ); + + tx.markTerminated( terminationCause ); + + ClientException e = assertThrows( ClientException.class, () -> await( tx.commitAsync() ) ); + assertNoCircularReferences( e ); + assertEquals( 1, e.getSuppressed().length ); + + Throwable suppressed = e.getSuppressed()[0]; + assertEquals( terminationCause, suppressed.getCause() ); + } + + @Test + void shouldNotCreateCircularExceptionWhenTerminatedWithoutFailure() + { + Connection connection = connectionMock(); + ClientException terminationCause = new ClientException( "Custom exception" ); + UnmanagedTransaction tx = new UnmanagedTransaction( connection, new DefaultBookmarkHolder(), UNLIMITED_FETCH_SIZE ); + + tx.markTerminated( terminationCause ); + + ClientException e = assertThrows( ClientException.class, () -> await( tx.commitAsync() ) ); + assertNoCircularReferences( e ); + + assertEquals( terminationCause, e.getCause() ); + } + @Test void shouldReleaseConnectionWhenTerminatedAndRolledBack() { @@ -250,4 +303,15 @@ private static Connection connectionWithBegin( Consumer beginBe return connection; } + + private ResultCursorsHolder mockResultCursorWith( ClientException clientException ) + { + ResultCursorsHolder resultCursorsHolder = new ResultCursorsHolder(); + FailableCursor cursor = mock( FailableCursor.class ); + doReturn( completedFuture( clientException ) ) + .when( cursor ) + .discardAllFailureAsync(); + resultCursorsHolder.add( completedFuture( cursor ) ); + return resultCursorsHolder; + } } diff --git a/driver/src/test/java/org/neo4j/driver/util/TestUtil.java b/driver/src/test/java/org/neo4j/driver/util/TestUtil.java index 835cb0c3c6..b0d4590316 100644 --- a/driver/src/test/java/org/neo4j/driver/util/TestUtil.java +++ b/driver/src/test/java/org/neo4j/driver/util/TestUtil.java @@ -29,6 +29,7 @@ import reactor.core.publisher.Mono; import java.time.Duration; +import java.util.ArrayList; import java.util.Arrays; import java.util.HashSet; import java.util.LinkedHashSet; @@ -644,6 +645,29 @@ public static ServerVersion anyServerVersion() return ServerVersion.v4_0_0; } + public static void assertNoCircularReferences(Throwable ex) + { + assertNoCircularReferences( ex, new ArrayList<>() ); + } + + private static void assertNoCircularReferences(Throwable ex, List list) + { + list.add( ex ); + if (ex.getCause() != null ) { + if (list.contains( ex.getCause() )) { + throw new AssertionError("Circular reference detected", ex.getCause()); + } + assertNoCircularReferences(ex.getCause(), list); + } + for ( Throwable suppressed: ex.getSuppressed() ) + { + if(list.contains( suppressed )) { + throw new AssertionError("Circular reference detected", suppressed); + } + assertNoCircularReferences( suppressed, list ); + } + } + private static void setupSuccessfulPullAll( Connection connection, String query ) { doAnswer( invocation ->