diff --git a/packages/core/src/index.ts b/packages/core/src/index.ts index 404541824..15ef0193c 100644 --- a/packages/core/src/index.ts +++ b/packages/core/src/index.ts @@ -69,6 +69,7 @@ import Result, { QueryResult, ResultObserver } from './result' import ConnectionProvider from './connection-provider' import Connection from './connection' import Transaction from './transaction' +import TransactionPromise from './transaction-promise' import Session, { TransactionConfig } from './session' import Driver, * as driver from './driver' import auth from './auth' @@ -134,6 +135,7 @@ const forExport = { Stats, Result, Transaction, + TransactionPromise, Session, Driver, Connection, @@ -191,6 +193,7 @@ export { ConnectionProvider, Connection, Transaction, + TransactionPromise, Session, Driver, types, diff --git a/packages/core/src/session.ts b/packages/core/src/session.ts index 3f6f3b2d4..3c7598a40 100644 --- a/packages/core/src/session.ts +++ b/packages/core/src/session.ts @@ -31,6 +31,7 @@ import ConnectionProvider from './connection-provider' import { Query, SessionMode } from './types' import Connection from './connection' import { NumberOrInteger } from './graph-types' +import TransactionPromise from './transaction-promise' type ConnectionConsumer = (connection: Connection | void) => any | undefined type TransactionWork = (tx: Transaction) => Promise | T @@ -239,9 +240,9 @@ class Session { * While a transaction is open the session cannot be used to run queries outside the transaction. * * @param {TransactionConfig} [transactionConfig] - Configuration for the new auto-commit transaction. - * @returns {Transaction} New Transaction. + * @returns {TransactionPromise} New Transaction. */ - beginTransaction(transactionConfig?: TransactionConfig): Transaction { + beginTransaction(transactionConfig?: TransactionConfig): TransactionPromise { // this function needs to support bookmarks parameter for backwards compatibility // parameter was of type {string|string[]} and represented either a single or multiple bookmarks // that's why we need to check parameter type and decide how to interpret the value @@ -255,7 +256,7 @@ class Session { return this._beginTransaction(this._mode, txConfig) } - _beginTransaction(accessMode: SessionMode, txConfig: TxConfig): Transaction { + _beginTransaction(accessMode: SessionMode, txConfig: TxConfig): TransactionPromise { if (!this._open) { throw newError('Cannot begin a transaction on a closed session.') } @@ -271,7 +272,7 @@ class Session { connectionHolder.initializeConnection() this._hasTx = true - const tx = new Transaction({ + const tx = new TransactionPromise({ connectionHolder, impersonatedUser: this._impersonatedUser, onClose: this._transactionClosed.bind(this), diff --git a/packages/core/src/transaction-promise.ts b/packages/core/src/transaction-promise.ts new file mode 100644 index 000000000..d0a013151 --- /dev/null +++ b/packages/core/src/transaction-promise.ts @@ -0,0 +1,193 @@ +/** + * 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 Transaction from "./transaction" +import { + ConnectionHolder +} from './internal/connection-holder' + +import { Bookmarks } from './internal/bookmarks' +import { TxConfig } from "./internal/tx-config"; + +/** + * Represents a {@link Promise} object and a {@link Transaction} object. + * + * Resolving this object promise verifies the result of the transaction begin and returns the {@link Transaction} object in case of success. + * + * The object can still also used as {@link Transaction} for convenience. The result of begin will be checked + * during the next API calls in the object as it is in the transaction. + * + * @access public + */ +class TransactionPromise extends Transaction implements Promise{ + [Symbol.toStringTag]: string = "TransactionPromise" + private _beginError?: Error; + private _beginMetadata?: any; + private _beginPromise?: Promise; + private _reject?: (error: Error) => void; + private _resolve?: (value?: Transaction | PromiseLike | undefined) => void; + + /** + * @constructor + * @param {ConnectionHolder} connectionHolder - the connection holder to get connection from. + * @param {function()} onClose - Function to be called when transaction is committed or rolled back. + * @param {function(bookmarks: Bookmarks)} onBookmarks callback invoked when new bookmark is produced. + * @param {function()} onConnection - Function to be called when a connection is obtained to ensure the connection + * is not yet released. + * @param {boolean} reactive whether this transaction generates reactive streams + * @param {number} fetchSize - the record fetch size in each pulling batch. + * @param {string} impersonatedUser - The name of the user which should be impersonated for the duration of the session. + */ + constructor({ + connectionHolder, + onClose, + onBookmarks, + onConnection, + reactive, + fetchSize, + impersonatedUser, + highRecordWatermark, + lowRecordWatermark + }: { + connectionHolder: ConnectionHolder + onClose: () => void + onBookmarks: (bookmarks: Bookmarks) => void + onConnection: () => void + reactive: boolean + fetchSize: number + impersonatedUser?: string, + highRecordWatermark: number, + lowRecordWatermark: number + }) { + super({ + connectionHolder, + onClose, + onBookmarks, + onConnection, + reactive, + fetchSize, + impersonatedUser, + highRecordWatermark, + lowRecordWatermark + }) + } + + /** + * Waits for the begin to complete. + * + * @param {function(transaction: Transaction)} onFulfilled - function to be called when finished. + * @param {function(error: {message:string, code:string})} onRejected - function to be called upon errors. + * @return {Promise} promise. + */ + then( + onfulfilled?: + ((value: Transaction) => TResult1 | PromiseLike) + | null, + onrejected?: + ((reason: any) => TResult2 | PromiseLike) + | null + ): Promise { + return this._getOrCreateBeginPromise().then(onfulfilled, onrejected); + } + + /** + * Catch errors when using promises. + * + * @param {function(error: Neo4jError)} onRejected - Function to be called upon errors. + * @return {Promise} promise. + */ + catch(onrejected?: ((reason: any) => TResult | PromiseLike) | null): Promise { + return this._getOrCreateBeginPromise().catch(onrejected); + } + + /** + * Called when finally the begin is done + * + * @param {function()|null} onfinally - function when the promise finished + * @return {Promise} promise. + */ + finally(onfinally?: (() => void) | null): Promise { + return this._getOrCreateBeginPromise().finally(onfinally); + } + + private _getOrCreateBeginPromise(): Promise { + if (!this._beginPromise) { + this._beginPromise = new Promise((resolve, reject) => { + this._resolve = resolve; + this._reject = reject; + if (this._beginError) { + reject(this._beginError); + } + if (this._beginMetadata) { + resolve(this._toTransaction()); + } + }); + } + return this._beginPromise; + } + + /** + * @access private + */ + private _toTransaction(): Transaction { + //@ts-ignore + return { + ...this, + run: super.run.bind(this), + commit: super.commit.bind(this), + rollback: super.rollback.bind(this), + close: super.close.bind(this), + isOpen: super.isOpen.bind(this), + _begin: this._begin.bind(this), + } + } + + /** + * @access private + */ + _begin(bookmarks: string | Bookmarks | string[], txConfig: TxConfig): void { + return super._begin(bookmarks, txConfig, { + onError: this._onBeginError.bind(this), + onComplete: this._onBeginMetadata.bind(this) + }); + } + + /** + * @access private + */ + private _onBeginError(error: Error): void { + this._beginError = error; + if (this._reject) { + this._reject(error); + } + } + + /** + * @access private + */ + private _onBeginMetadata(metadata: any): void { + this._beginMetadata = metadata || {}; + if (this._resolve) { + this._resolve(this._toTransaction()); + } + } + +} + +export default TransactionPromise diff --git a/packages/core/src/transaction.ts b/packages/core/src/transaction.ts index f8685e550..8703e034a 100644 --- a/packages/core/src/transaction.ts +++ b/packages/core/src/transaction.ts @@ -61,11 +61,13 @@ class Transaction { * @param {ConnectionHolder} connectionHolder - the connection holder to get connection from. * @param {function()} onClose - Function to be called when transaction is committed or rolled back. * @param {function(bookmarks: Bookmarks)} onBookmarks callback invoked when new bookmark is produced. - * * @param {function()} onConnection - Function to be called when a connection is obtained to ensure the conneciton + * @param {function()} onConnection - Function to be called when a connection is obtained to ensure the conneciton * is not yet released. * @param {boolean} reactive whether this transaction generates reactive streams * @param {number} fetchSize - the record fetch size in each pulling batch. * @param {string} impersonatedUser - The name of the user which should be impersonated for the duration of the session. + * @param {number} highRecordWatermark - The high watermark for the record buffer. + * @param {number} lowRecordWatermark - The low watermark for the record buffer. */ constructor({ connectionHolder, @@ -109,7 +111,10 @@ class Transaction { * @param {TxConfig} txConfig * @returns {void} */ - _begin(bookmarks: Bookmarks | string | string[], txConfig: TxConfig): void { + _begin(bookmarks: Bookmarks | string | string[], txConfig: TxConfig, events?: { + onError: (error: Error) => void + onComplete: (metadata: any) => void + }): void { this._connectionHolder .getConnection() .then(connection => { @@ -121,14 +126,29 @@ class Transaction { mode: this._connectionHolder.mode(), database: this._connectionHolder.database(), impersonatedUser: this._impersonatedUser, - beforeError: this._onError, - afterComplete: this._onComplete + beforeError: (error: Error) => { + if (events) { + events.onError(error) + } + return this._onError(error).catch(() => {}) + }, + afterComplete: (metadata: any) => { + if (events) { + events.onComplete(metadata) + } + return this._onComplete(metadata) + } }) } else { throw newError('No connection available') } }) - .catch(error => this._onError(error)) + .catch(error => { + if (events) { + events.onError(error) + } + this._onError(error).catch(() => {}) + }) } /** diff --git a/packages/core/test/session.test.ts b/packages/core/test/session.test.ts index 829450e97..7f6601919 100644 --- a/packages/core/test/session.test.ts +++ b/packages/core/test/session.test.ts @@ -16,7 +16,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -import { ConnectionProvider, Session, Connection } from '../src' +import { ConnectionProvider, Session, Connection, TransactionPromise, Transaction } from '../src' import { bookmarks } from '../src/internal' import { ACCESS_MODE_READ, FETCH_ALL } from '../src/internal/constants' import FakeConnection from './utils/connection.fake' @@ -227,6 +227,35 @@ describe('session', () => { expect(session.lastBookmarks()).toEqual(bookmarks.values()) }) }) + + describe('.beginTransaction()', () => { + it('should return a TransactionPromise', () => { + const session = newSessionWithConnection(newFakeConnection(), false, 1000) + + const tx: Transaction = session.beginTransaction() + + expect(tx).toBeInstanceOf(TransactionPromise) + }) + + it('should resolves a Transaction', async () => { + const connection = newFakeConnection() + const protocol = connection.protocol() + // @ts-ignore + connection.protocol = () => { + return { + ...protocol, + beginTransaction: (params: { afterComplete: () => {} }) => { + params.afterComplete() + } + } + } + const session = newSessionWithConnection(connection, false, 1000) + + const tx: Transaction = await session.beginTransaction() + + expect(tx).toBeDefined() + }) + }) }) function newSessionWithConnection( diff --git a/packages/core/test/transaction.test.ts b/packages/core/test/transaction.test.ts index 302186e21..c47aa7a27 100644 --- a/packages/core/test/transaction.test.ts +++ b/packages/core/test/transaction.test.ts @@ -17,105 +17,455 @@ * limitations under the License. */ -import { ConnectionProvider, Transaction } from "../src"; +import { ConnectionProvider, newError, Transaction, TransactionPromise } from "../src"; import { Bookmarks } from "../src/internal/bookmarks"; import { ConnectionHolder } from "../src/internal/connection-holder"; +import { TxConfig } from "../src/internal/tx-config"; import FakeConnection from "./utils/connection.fake"; -describe('Transaction', () => { - describe('.run()', () => { - it('should call run with watermarks', async () => { - const connection = newFakeConnection() - const tx = newTransaction({ - connection, - fetchSize: 1000, - highRecordWatermark: 700, - lowRecordWatermark: 300 - }) +testTx('Transaction', newRegularTransaction) + +testTx('TransactionPromise', newTransactionPromise, () => { + describe('Promise', () => { + const syncContext = (fn: () => void) => fn() + const asyncContext = (fn: () => void) => setImmediate(fn) + + whenBeginSucceed('async', asyncContext) + whenBeginSucceed('sync', syncContext) + + function whenBeginSucceed(ctxName: string, ctx: (_: () => void) => void) { + describe(`when begin processed with success [${ctxName}]`, () => { + it('should result resolve with Transaction', async () => { + const [tx] = setupTx() + + const resolveTx: Transaction = await tx; + + // @ts-ignore + expect(resolveTx.then).toBeUndefined() + // @ts-ignore + expect(resolveTx.catch).toBeUndefined() + // @ts-ignore + expect(resolveTx.finally).toBeUndefined() + + expect(resolveTx.commit).toBeDefined() + expect(resolveTx.rollback).toBeDefined() + expect(resolveTx.close).toBeDefined() + expect(resolveTx.run).toBeDefined() + expect(resolveTx.isOpen).toBeDefined() + }) + + it('should resolve an open Transaction', async () => { + const [tx] = setupTx() + + const resolved = await tx; + + expect(resolved.isOpen()).toBe(true) + }) + + it('should be able to run queries in the resolved transaction', async () => { + const [tx] = setupTx() + + const resolvedTx = await tx + + await resolvedTx.run('RETURN 1'); + }) + + it('should be able to commit the resolved transaction', async () => { + const [tx] = setupTx() + + const resolvedTx = await tx + + await resolvedTx.commit(); + }) + + it('should be able to rollback the resolved transaction', async () => { + const [tx] = setupTx() + + const resolvedTx = await tx + + await resolvedTx.rollback(); + }) + + it('should be able to close the resolved transaction', async () => { + const [tx] = setupTx() + + const resolvedTx = await tx + + await resolvedTx.close(); + }) + + it('should the original tx be open', async () => { + const [tx] = setupTx() + + await tx; + + expect(tx.isOpen()).toBe(true) + }) + + it('should be able to run queries in the original transaction', async () => { + const [tx] = setupTx() + + await tx + + await tx.run('RETURN 1'); + }) + + it('should be able to commit the original transaction', async () => { + const [tx] = setupTx() + + await tx + + await tx.commit(); + }) + + it('should be able to rollback the original transaction', async () => { + const [tx] = setupTx() + + await tx + + await tx.rollback(); + }) - await tx.run('RETURN 1') + it('should be able to close the original transaction', async () => { + const [tx] = setupTx() + + await tx + + await tx.close(); + }) + + function setupTx(): [TransactionPromise] { + const connection = newFakeConnection() + const protocol = connection.protocol() + + // @ts-ignore + connection.protocol = () => { + return { + ...protocol, + beginTransaction: (params: { afterComplete: (meta: any) => void }) => { + ctx(() => params.afterComplete({})) + } + } + } + + const tx = newTransactionPromise({ + connection + }) + + tx._begin(Bookmarks.empty(), TxConfig.empty()) + + return [tx] + } - expect(connection.seenProtocolOptions[0]).toMatchObject({ - fetchSize: 1000, - lowRecordWatermark: 300, - highRecordWatermark: 700 }) - }) + } + + + whenBeginFails('async', asyncContext) + whenBeginFails('sync', syncContext) + + function whenBeginFails(ctxName: string, ctx: (fn: () => void) => void) { + describe(`when begin fails [${ctxName}]`, () => { + it('should fails to resolve the transaction', async () => { + const [tx, expectedError] = setupTx() + + try { + await tx; + fail('should have thrown') + } catch (e) { + expect(e).toEqual(expectedError) + } + }) + + it('should be closed', async () => { + const [tx] = setupTx() + + try { + await tx; + } catch (e) { + // thats fine + } + + expect(tx.isOpen()).toBe(false) + }) + + it('should not be able to run queries in the original transaction', async () => { + const [tx] = setupTx() - it('should configure result with watermarks', async () => { - const connection = newFakeConnection() - const tx = newTransaction({ - connection, - fetchSize: 1000, - highRecordWatermark: 700, - lowRecordWatermark: 300 + try { + await tx; + } catch (e) { + // thats fine + } + + try { + await tx.run('RETURN 1'); + fail('shoud not succeed') + } catch (e) { + expect(e).toEqual(newError( + 'Cannot run query in this transaction, because it has been rolled back either because of an error or explicit termination.' + )) + } + }) + + it('should not be able to commit the original transaction', async () => { + const [tx] = setupTx() + + try { + await tx; + } catch (e) { + // thats fine + } + + try { + await tx.commit(); + fail('shoud not succeed') + } catch (e) { + expect(e).toEqual(newError( + 'Cannot commit this transaction, because it has been rolled back either because of an error or explicit termination.' + )) + } + }) + + it('should be able to rollback the original transaction', async () => { + const [tx] = setupTx() + + try { + await tx; + } catch (e) { + // thats fine + } + + await tx.rollback(); + }) + + it('should be able to close the original transaction', async () => { + const [tx] = setupTx() + + try { + await tx; + } catch (e) { + // thats fine + } + + await tx.close(); + }) + + function setupTx(): [TransactionPromise, Error] { + const connection = newFakeConnection() + const protocol = connection.protocol() + const expectedError = newError('begin error') + + // @ts-ignore + connection.protocol = () => { + return { + ...protocol, + beginTransaction: (params: { beforeError: (error: Error) => void }) => { + ctx(() => params.beforeError(expectedError)) + } + } + } + + const tx = newTransactionPromise({ + connection + }) + + tx._begin(Bookmarks.empty(), TxConfig.empty()) + return [tx, expectedError] + } }) + } + + describe('when connection holder return a void connection', () => { + it('should fails to resolve the transaction', async () => { + const expectedError = newError('No connection available') - var result = tx.run('RETURN 1') + const tx = newTransactionPromise({ + connection: undefined + }) - // @ts-ignore - expect(result._watermarks).toEqual({ high: 700, low: 300 }) + tx._begin(Bookmarks.empty(), TxConfig.empty()) + + try { + await tx; + fail('should have thrown') + } catch (e) { + expect(e).toEqual(expectedError) + } + }) }) + describe('when connection holder fails returning a connection', () => { + it('should fails to resolve the transaction', async () => { + const expectedError = newError('Something wrong') + + const tx = newTransactionPromise({ + connection: undefined, + errorResolvingConnection: expectedError + }) + + tx._begin(Bookmarks.empty(), TxConfig.empty()) + + try { + await tx; + fail('should have thrown') + } catch (e) { + expect(e).toEqual(expectedError) + } + }) + }) }) +}) + +function testTx(transactionName: string, newTransaction: TransactionFactory, fn: jest.EmptyFunction = () => { }) { + describe(transactionName, () => { - describe('.close()', () => { - describe('when transaction is open', () => { - it('should roll back the transaction', async () => { + describe('.run()', () => { + it('should call run with watermarks', async () => { const connection = newFakeConnection() - const tx = newTransaction({ connection }) + const tx = newTransaction({ + connection, + fetchSize: 1000, + highRecordWatermark: 700, + lowRecordWatermark: 300 + }) await tx.run('RETURN 1') - await tx.close() - expect(connection.rollbackInvoked).toEqual(1) + expect(connection.seenProtocolOptions[0]).toMatchObject({ + fetchSize: 1000, + lowRecordWatermark: 300, + highRecordWatermark: 700 + }) }) - it('should surface errors during the rollback', async () => { - const expectedError = new Error('rollback error') - const connection = newFakeConnection().withRollbackError(expectedError) - const tx = newTransaction({ connection }) + it('should configure result with watermarks', async () => { + const connection = newFakeConnection() + const tx = newTransaction({ + connection, + fetchSize: 1000, + highRecordWatermark: 700, + lowRecordWatermark: 300 + }) - await tx.run('RETURN 1') + var result = tx.run('RETURN 1') - try { - await tx.close() - fail('should have thrown') - } catch (error) { - expect(error).toEqual(expectedError) - } + // @ts-ignore + expect(result._watermarks).toEqual({ high: 700, low: 300 }) }) + }) - describe('when transaction is closed', () => { - const commit = async (tx: Transaction) => tx.commit() - const rollback = async (tx: Transaction) => tx.rollback() - const error = async (tx: Transaction, conn: FakeConnection) => { - conn.withRollbackError(new Error('rollback error')) - return tx.rollback().catch(() => { }) - } - - it.each([ - ['commmited', commit], - ['rolled back', rollback], - ['with error', error] - ])('should not roll back the connection', async (_, operation) => { - const connection = newFakeConnection() - const tx = newTransaction({ connection }) + describe('.close()', () => { + describe('when transaction is open', () => { + it('should roll back the transaction', async () => { + const connection = newFakeConnection() + const tx = newTransaction({ connection }) + + await tx.run('RETURN 1') + await tx.close() + + expect(connection.rollbackInvoked).toEqual(1) + }) + + it('should surface errors during the rollback', async () => { + const expectedError = new Error('rollback error') + const connection = newFakeConnection().withRollbackError(expectedError) + const tx = newTransaction({ connection }) + + await tx.run('RETURN 1') + + try { + await tx.close() + fail('should have thrown') + } catch (error) { + expect(error).toEqual(expectedError) + } + }) + }) + + describe('when transaction is closed', () => { + const commit = async (tx: Transaction) => tx.commit() + const rollback = async (tx: Transaction) => tx.rollback() + const error = async (tx: Transaction, conn: FakeConnection) => { + conn.withRollbackError(new Error('rollback error')) + return tx.rollback().catch(() => { }) + } + + it.each([ + ['commmited', commit], + ['rolled back', rollback], + ['with error', error] + ])('should not roll back the connection', async (_, operation) => { + const connection = newFakeConnection() + const tx = newTransaction({ connection }) - await operation(tx, connection) - const rollbackInvokedAfterOperation = connection.rollbackInvoked + await operation(tx, connection) + const rollbackInvokedAfterOperation = connection.rollbackInvoked - await tx.close() + await tx.close() - expect(connection.rollbackInvoked).toEqual(rollbackInvokedAfterOperation) + expect(connection.rollbackInvoked).toEqual(rollbackInvokedAfterOperation) + }) }) }) + + fn() }) -}) +} + +interface TransactionFactory { + (_: { + connection: FakeConnection + fetchSize?: number + highRecordWatermark?: number, + lowRecordWatermark?: number + }): T +} + +function newTransactionPromise({ + connection, + fetchSize = 1000, + highRecordWatermark = 700, + lowRecordWatermark = 300, + errorResolvingConnection = undefined +}: { + connection: FakeConnection | void + fetchSize?: number + highRecordWatermark?: number, + lowRecordWatermark?: number + errorResolvingConnection?: Error +}): TransactionPromise { + const connectionProvider = new ConnectionProvider() + // @ts-ignore + connectionProvider.acquireConnection = () => { + if (errorResolvingConnection) { + return Promise.reject(errorResolvingConnection) + } + return Promise.resolve(connection) + } + connectionProvider.close = () => Promise.resolve() + + const connectionHolder = new ConnectionHolder({ connectionProvider }) + connectionHolder.initializeConnection() + + const transaction = new TransactionPromise({ + connectionHolder, + onClose: () => { }, + onBookmarks: (_: Bookmarks) => { }, + onConnection: () => { }, + reactive: false, + fetchSize, + impersonatedUser: "", + highRecordWatermark, + lowRecordWatermark + }) + + return transaction +} -function newTransaction({ +function newRegularTransaction({ connection, fetchSize = 1000, highRecordWatermark = 700, diff --git a/packages/neo4j-driver-lite/src/index.ts b/packages/neo4j-driver-lite/src/index.ts index 9a183d32a..112aac33a 100644 --- a/packages/neo4j-driver-lite/src/index.ts +++ b/packages/neo4j-driver-lite/src/index.ts @@ -62,6 +62,7 @@ import { NotificationPosition, Session, Transaction, + TransactionPromise, ServerInfo, Connection, driver as coreDriver, @@ -425,6 +426,7 @@ const forExport = { ServerInfo, Session, Transaction, + TransactionPromise, Point, Duration, LocalTime, @@ -473,6 +475,7 @@ export { ServerInfo, Session, Transaction, + TransactionPromise, Point, Duration, LocalTime, diff --git a/packages/testkit-backend/src/request-handlers.js b/packages/testkit-backend/src/request-handlers.js index 4c05da641..2e1e397a5 100644 --- a/packages/testkit-backend/src/request-handlers.js +++ b/packages/testkit-backend/src/request-handlers.js @@ -285,16 +285,21 @@ export function RetryableNegative (context, data, wire) { export function SessionBeginTransaction (context, data, wire) { const { sessionId, txMeta: metadata, timeout } = data const session = context.getSession(sessionId) - let tx + try { - tx = session.beginTransaction({ metadata, timeout }) + return session.beginTransaction({ metadata, timeout }) + .then(tx => { + const id = context.addTx(tx, sessionId) + wire.writeResponse('Transaction', { id }) + }).catch(e => { + console.log('got some err: ' + JSON.stringify(e)) + wire.writeError(e) + }) } catch (e) { console.log('got some err: ' + JSON.stringify(e)) wire.writeError(e) return } - const id = context.addTx(tx, sessionId) - wire.writeResponse('Transaction', { id }) } export function TransactionCommit (context, data, wire) { @@ -372,6 +377,7 @@ export function GetFeatures (_context, _params, wire) { 'Feature:Bolt:4.4', 'Feature:API:Result.List', 'Feature:API:Result.Peek', + 'Optimization:EagerTransactionBegin', 'Optimization:ImplicitDefaultArguments', 'Temporary:ConnectionAcquisitionTimeout', 'Temporary:CypherPathAndRelationship', diff --git a/packages/testkit-backend/src/skipped-tests/common.js b/packages/testkit-backend/src/skipped-tests/common.js index ca224c8f7..ad9db4d78 100644 --- a/packages/testkit-backend/src/skipped-tests/common.js +++ b/packages/testkit-backend/src/skipped-tests/common.js @@ -7,6 +7,11 @@ const skippedTests = [ ifEquals('stub.optimizations.test_optimizations.TestOptimizations.test_uses_implicit_default_arguments_multi_query'), ifEquals('stub.optimizations.test_optimizations.TestOptimizations.test_uses_implicit_default_arguments_multi_query_nested') ), + skip( + 'Eager verification not implemented for tx functions', + ifEquals('stub.tx_run.test_tx_run.TestTxRun.test_eager_begin_on_tx_func_run_with_error_on_begin'), + ifEquals('stub.tx_run.test_tx_run.TestTxRun.test_eager_begin_on_tx_func_run_with_disconnect_on_begin') + ), skip( 'Fail while enable Temporary::ResultKeys', ifEquals('neo4j.test_bookmarks.TestBookmarks.test_can_pass_bookmark_into_next_session'),