Skip to content

Commit

Permalink
Heavily reduces contention on generating ids
Browse files Browse the repository at this point in the history
The main idea is to have batching id generators local to each transaction instance (which is pooled).
This batching sits in between store and typical creators of records at commit time and hands out ids,
grabbing batches of ids off of the stores. On clean shutdown any excess ids remaining unused will be
freed back into the store before closing.

In scenarios where there are many concurrent transactions committing data, where they all
co-operatively force the log then this id generator contention is the single biggest one.
With these changes this contention is basically removed entirely.

The default batch size is 20, which reduces contention dramatically and should be a good value
for most scenarios. The value is however controllable with a database setting.
  • Loading branch information
tinwelint committed Sep 1, 2017
1 parent 9d6c92b commit 7acf3b3
Show file tree
Hide file tree
Showing 50 changed files with 1,132 additions and 397 deletions.
Expand Up @@ -71,6 +71,7 @@
import static org.neo4j.kernel.configuration.Settings.min; import static org.neo4j.kernel.configuration.Settings.min;
import static org.neo4j.kernel.configuration.Settings.options; import static org.neo4j.kernel.configuration.Settings.options;
import static org.neo4j.kernel.configuration.Settings.pathSetting; import static org.neo4j.kernel.configuration.Settings.pathSetting;
import static org.neo4j.kernel.configuration.Settings.range;
import static org.neo4j.kernel.configuration.Settings.setting; import static org.neo4j.kernel.configuration.Settings.setting;


/** /**
Expand Down Expand Up @@ -459,8 +460,17 @@ public class GraphDatabaseSettings implements LoadableConfig
public static final Setting<Integer> label_block_size = setting("unsupported.dbms.block_size.labels", INTEGER, public static final Setting<Integer> label_block_size = setting("unsupported.dbms.block_size.labels", INTEGER,
"0", min( 0 ) ); "0", min( 0 ) );


@Description("An identifier that uniquely identifies this graph database instance within this JVM. " + @Description( "Specifies the size of id batches local to each transaction when committing. " +
"Defaults to an auto-generated number depending on how many instance are started in this JVM.") "Committing a transaction which contains changes most often results in new data records being created. " +
"For each record a new id needs to be generated from an id generator. " +
"It's more efficient to allocate a batch of ids from the contended id generator, which the transaction " +
"holds and generates ids from while creating these new records. " +
"This setting specifies how big those batches are. " +
"Remaining ids are freed back to id generator on clean shutdown." )
@Internal
public static final Setting<Integer> record_id_batch_size = setting( "unsupported.dbms.record_id_batch_size", INTEGER,
"20", range( 1, 1_000 ) );

@Internal @Internal
public static final Setting<String> forced_kernel_id = setting("unsupported.dbms.kernel_id", STRING, NO_DEFAULT, public static final Setting<String> forced_kernel_id = setting("unsupported.dbms.kernel_id", STRING, NO_DEFAULT,
illegalValueMessage("has to be a valid kernel identifier", matches("[a-zA-Z0-9]*"))); illegalValueMessage("has to be a valid kernel identifier", matches("[a-zA-Z0-9]*")));
Expand Down
Expand Up @@ -322,7 +322,7 @@ private void assertCurrentThreadIsNotBlockingNewTransactions()


private class KernelTransactionImplementationFactory implements Factory<KernelTransactionImplementation> private class KernelTransactionImplementationFactory implements Factory<KernelTransactionImplementation>
{ {
private Set<KernelTransactionImplementation> transactions; private final Set<KernelTransactionImplementation> transactions;


KernelTransactionImplementationFactory( Set<KernelTransactionImplementation> transactions ) KernelTransactionImplementationFactory( Set<KernelTransactionImplementation> transactions )
{ {
Expand All @@ -345,7 +345,7 @@ public KernelTransactionImplementation newInstance()


private class GlobalKernelTransactionPool extends LinkedQueuePool<KernelTransactionImplementation> private class GlobalKernelTransactionPool extends LinkedQueuePool<KernelTransactionImplementation>
{ {
private Set<KernelTransactionImplementation> transactions; private final Set<KernelTransactionImplementation> transactions;


GlobalKernelTransactionPool( Set<KernelTransactionImplementation> transactions, GlobalKernelTransactionPool( Set<KernelTransactionImplementation> transactions,
Factory<KernelTransactionImplementation> factory ) Factory<KernelTransactionImplementation> factory )
Expand Down
Expand Up @@ -417,7 +417,7 @@ private Cursor<PropertyItem> relationshipGetPropertyCursor( KernelStatement stat
@Override @Override
public long nodeCreate( KernelStatement state ) public long nodeCreate( KernelStatement state )
{ {
long nodeId = storeLayer.reserveNode(); long nodeId = state.getStoreStatement().reserveNode();
state.txState().nodeDoCreate(nodeId); state.txState().nodeDoCreate(nodeId);
return nodeId; return nodeId;
} }
Expand Down Expand Up @@ -452,7 +452,7 @@ public long relationshipCreate( KernelStatement state,
{ {
try ( Cursor<NodeItem> endNode = nodeCursorById( state, endNodeId ) ) try ( Cursor<NodeItem> endNode = nodeCursorById( state, endNodeId ) )
{ {
long id = storeLayer.reserveRelationship(); long id = state.getStoreStatement().reserveRelationship();
state.txState().relationshipDoCreate( id, relationshipTypeId, startNode.get().id(), endNode.get().id() ); state.txState().relationshipDoCreate( id, relationshipTypeId, startNode.get().id(), endNode.get().id() );
return id; return id;
} }
Expand Down
Expand Up @@ -21,11 +21,11 @@


import org.neo4j.kernel.impl.store.CommonAbstractStore; import org.neo4j.kernel.impl.store.CommonAbstractStore;


class AllIdIterator extends HighIdAwareIterator<CommonAbstractStore<?,?>> public class AllIdIterator extends HighIdAwareIterator<CommonAbstractStore<?,?>>
{ {
private long currentId; private long currentId;


AllIdIterator( CommonAbstractStore<?,?> store ) public AllIdIterator( CommonAbstractStore<?,?> store )
{ {
super( store ); super( store );
} }
Expand Down
Expand Up @@ -447,18 +447,6 @@ public Cursor<PropertyItem> relationshipGetProperty( StorageStatement statement,
return statement.acquireSinglePropertyCursor( relationship.nextPropertyId(), propertyKeyId, lock, assertOpen ); return statement.acquireSinglePropertyCursor( relationship.nextPropertyId(), propertyKeyId, lock, assertOpen );
} }


@Override
public long reserveNode()
{
return nodeStore.nextId();
}

@Override
public long reserveRelationship()
{
return relationshipStore.nextId();
}

@Override @Override
public void releaseNode( long id ) public void releaseNode( long id )
{ {
Expand Down
Expand Up @@ -52,7 +52,7 @@ public class StoreSingleNodeCursor implements Cursor<NodeItem>, NodeItem
private long nodeId = StatementConstants.NO_SUCH_NODE; private long nodeId = StatementConstants.NO_SUCH_NODE;
private long[] labels; private long[] labels;


StoreSingleNodeCursor( NodeRecord nodeRecord, Consumer<StoreSingleNodeCursor> instanceCache, public StoreSingleNodeCursor( NodeRecord nodeRecord, Consumer<StoreSingleNodeCursor> instanceCache,
RecordCursors recordCursors, LockService lockService ) RecordCursors recordCursors, LockService lockService )
{ {
this.nodeRecord = nodeRecord; this.nodeRecord = nodeRecord;
Expand Down
Expand Up @@ -35,7 +35,7 @@ public class StoreSingleRelationshipCursor extends StoreAbstractRelationshipCurs
private final InstanceCache<StoreSingleRelationshipCursor> instanceCache; private final InstanceCache<StoreSingleRelationshipCursor> instanceCache;
private long relationshipId = StatementConstants.NO_SUCH_RELATIONSHIP; private long relationshipId = StatementConstants.NO_SUCH_RELATIONSHIP;


StoreSingleRelationshipCursor( RelationshipRecord relationshipRecord, public StoreSingleRelationshipCursor( RelationshipRecord relationshipRecord,
InstanceCache<StoreSingleRelationshipCursor> instanceCache, RecordCursors cursors, LockService lockService ) InstanceCache<StoreSingleRelationshipCursor> instanceCache, RecordCursors cursors, LockService lockService )
{ {
super( relationshipRecord, cursors, lockService ); super( relationshipRecord, cursors, lockService );
Expand Down
@@ -0,0 +1,92 @@
/*
* Copyright (c) 2002-2017 "Neo Technology,"
* Network Engine for Objects in Lund AB [http://neotechnology.com]
*
* This file is part of Neo4j.
*
* Neo4j is free software: you can redistribute it and/or modify
* it under the terms of the GNU General Public License as published by
* the Free Software Foundation, either version 3 of the License, or
* (at your option) any later version.
*
* This program is distributed in the hope that it will be useful,
* but WITHOUT ANY WARRANTY; without even the implied warranty of
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
* GNU General Public License for more details.
*
* You should have received a copy of the GNU General Public License
* along with this program. If not, see <http://www.gnu.org/licenses/>.
*/
package org.neo4j.kernel.impl.storageengine.impl.recordstorage;

import org.neo4j.kernel.impl.store.NeoStores;
import org.neo4j.kernel.impl.store.StandardDynamicRecordAllocator;
import org.neo4j.kernel.impl.store.StoreType;
import org.neo4j.kernel.impl.store.id.RenewableBatchIdSequences;
import org.neo4j.kernel.impl.transaction.state.IntegrityValidator;
import org.neo4j.kernel.impl.transaction.state.Loaders;
import org.neo4j.kernel.impl.transaction.state.PropertyCreator;
import org.neo4j.kernel.impl.transaction.state.PropertyDeleter;
import org.neo4j.kernel.impl.transaction.state.PropertyTraverser;
import org.neo4j.kernel.impl.transaction.state.RecordChangeSet;
import org.neo4j.kernel.impl.transaction.state.RelationshipCreator;
import org.neo4j.kernel.impl.transaction.state.RelationshipDeleter;
import org.neo4j.kernel.impl.transaction.state.RelationshipGroupGetter;
import org.neo4j.kernel.impl.transaction.state.TransactionRecordState;
import org.neo4j.storageengine.api.CommandCreationContext;
import org.neo4j.storageengine.api.lock.ResourceLocker;

/**
* Holds commit data structures for creating records in a {@link NeoStores}.
*/
public class RecordStorageCommandCreationContext implements CommandCreationContext
{
private final NeoStores neoStores;
private final Loaders loaders;
private final RelationshipCreator relationshipCreator;
private final RelationshipDeleter relationshipDeleter;
private final PropertyCreator propertyCreator;
private final PropertyDeleter propertyDeleter;
private final RenewableBatchIdSequences idBatches;

RecordStorageCommandCreationContext( NeoStores neoStores, int denseNodeThreshold, int idBatchSize )
{
this.neoStores = neoStores;
this.idBatches = new RenewableBatchIdSequences( neoStores, idBatchSize );

this.loaders = new Loaders( neoStores );
RelationshipGroupGetter relationshipGroupGetter =
new RelationshipGroupGetter( idBatches.idGenerator( StoreType.RELATIONSHIP_GROUP ) );
this.relationshipCreator = new RelationshipCreator( relationshipGroupGetter, denseNodeThreshold );
PropertyTraverser propertyTraverser = new PropertyTraverser();
this.propertyDeleter = new PropertyDeleter( propertyTraverser );
this.relationshipDeleter = new RelationshipDeleter( relationshipGroupGetter, propertyDeleter );
this.propertyCreator = new PropertyCreator(
new StandardDynamicRecordAllocator( idBatches.idGenerator( StoreType.PROPERTY_STRING ),
neoStores.getPropertyStore().getStringStore().getRecordDataSize() ),
new StandardDynamicRecordAllocator( idBatches.idGenerator( StoreType.PROPERTY_ARRAY ),
neoStores.getPropertyStore().getArrayStore().getRecordDataSize() ),
idBatches.idGenerator( StoreType.PROPERTY ),
propertyTraverser );
}

public long nextId( StoreType storeType )
{
return idBatches.nextId( storeType );
}

@Override
public void close()
{
this.idBatches.close();
}

public TransactionRecordState createTransactionRecordState( IntegrityValidator integrityValidator, long lastTransactionIdWhenStarted,
ResourceLocker locks )
{
RecordChangeSet recordChangeSet = new RecordChangeSet( loaders );
return new TransactionRecordState( neoStores, integrityValidator,
recordChangeSet, lastTransactionIdWhenStarted, locks,
relationshipCreator, relationshipDeleter, propertyCreator, propertyDeleter );
}
}
Expand Up @@ -60,7 +60,6 @@
import org.neo4j.kernel.impl.api.scan.LabelScanStoreProvider; import org.neo4j.kernel.impl.api.scan.LabelScanStoreProvider;
import org.neo4j.kernel.impl.api.store.SchemaCache; import org.neo4j.kernel.impl.api.store.SchemaCache;
import org.neo4j.kernel.impl.api.store.StorageLayer; import org.neo4j.kernel.impl.api.store.StorageLayer;
import org.neo4j.kernel.impl.api.store.StoreStatement;
import org.neo4j.kernel.impl.cache.BridgingCacheAccess; import org.neo4j.kernel.impl.cache.BridgingCacheAccess;
import org.neo4j.kernel.impl.constraints.ConstraintSemantics; import org.neo4j.kernel.impl.constraints.ConstraintSemantics;
import org.neo4j.kernel.impl.core.CacheAccessBackDoor; import org.neo4j.kernel.impl.core.CacheAccessBackDoor;
Expand All @@ -87,14 +86,6 @@
import org.neo4j.kernel.impl.transaction.command.NeoStoreBatchTransactionApplier; import org.neo4j.kernel.impl.transaction.command.NeoStoreBatchTransactionApplier;
import org.neo4j.kernel.impl.transaction.state.DefaultSchemaIndexProviderMap; import org.neo4j.kernel.impl.transaction.state.DefaultSchemaIndexProviderMap;
import org.neo4j.kernel.impl.transaction.state.IntegrityValidator; import org.neo4j.kernel.impl.transaction.state.IntegrityValidator;
import org.neo4j.kernel.impl.transaction.state.Loaders;
import org.neo4j.kernel.impl.transaction.state.PropertyCreator;
import org.neo4j.kernel.impl.transaction.state.PropertyDeleter;
import org.neo4j.kernel.impl.transaction.state.PropertyTraverser;
import org.neo4j.kernel.impl.transaction.state.RecordChangeSet;
import org.neo4j.kernel.impl.transaction.state.RelationshipCreator;
import org.neo4j.kernel.impl.transaction.state.RelationshipDeleter;
import org.neo4j.kernel.impl.transaction.state.RelationshipGroupGetter;
import org.neo4j.kernel.impl.transaction.state.TransactionRecordState; import org.neo4j.kernel.impl.transaction.state.TransactionRecordState;
import org.neo4j.kernel.impl.transaction.state.storeview.DynamicIndexStoreView; import org.neo4j.kernel.impl.transaction.state.storeview.DynamicIndexStoreView;
import org.neo4j.kernel.impl.transaction.state.storeview.NeoStoreIndexStoreView; import org.neo4j.kernel.impl.transaction.state.storeview.NeoStoreIndexStoreView;
Expand Down Expand Up @@ -154,13 +145,8 @@ public class RecordStorageEngine implements StorageEngine, Lifecycle
private final PropertyPhysicalToLogicalConverter indexUpdatesConverter; private final PropertyPhysicalToLogicalConverter indexUpdatesConverter;
private final Supplier<StorageStatement> storeStatementSupplier; private final Supplier<StorageStatement> storeStatementSupplier;
private final IdController idController; private final IdController idController;

private final int denseNodeThreshold;
// Immutable state for creating/applying commands private final int recordIdBatchSize;
private final Loaders loaders;
private final RelationshipCreator relationshipCreator;
private final RelationshipDeleter relationshipDeleter;
private final PropertyCreator propertyCreator;
private final PropertyDeleter propertyDeleter;


public RecordStorageEngine( public RecordStorageEngine(
File storeDir, File storeDir,
Expand Down Expand Up @@ -233,16 +219,8 @@ public RecordStorageEngine(
commandReaderFactory = new RecordStorageCommandReaderFactory(); commandReaderFactory = new RecordStorageCommandReaderFactory();
indexUpdatesSync = new WorkSync<>( indexingService ); indexUpdatesSync = new WorkSync<>( indexingService );


// Immutable state for creating/applying commands denseNodeThreshold = config.get( GraphDatabaseSettings.dense_node_threshold );
loaders = new Loaders( neoStores ); recordIdBatchSize = config.get( GraphDatabaseSettings.record_id_batch_size );
RelationshipGroupGetter relationshipGroupGetter =
new RelationshipGroupGetter( neoStores.getRelationshipGroupStore() );
relationshipCreator = new RelationshipCreator( relationshipGroupGetter,
config.get( GraphDatabaseSettings.dense_node_threshold ) );
PropertyTraverser propertyTraverser = new PropertyTraverser();
propertyDeleter = new PropertyDeleter( propertyTraverser );
relationshipDeleter = new RelationshipDeleter( relationshipGroupGetter, propertyDeleter );
propertyCreator = new PropertyCreator( neoStores.getPropertyStore(), propertyTraverser );
} }
catch ( Throwable failure ) catch ( Throwable failure )
{ {
Expand All @@ -256,7 +234,8 @@ private Supplier<StorageStatement> storeStatementSupplier( NeoStores neoStores )
Supplier<IndexReaderFactory> indexReaderFactory = () -> new IndexReaderFactory.Caching( indexingService ); Supplier<IndexReaderFactory> indexReaderFactory = () -> new IndexReaderFactory.Caching( indexingService );
LockService lockService = takePropertyReadLocks ? this.lockService : NO_LOCK_SERVICE; LockService lockService = takePropertyReadLocks ? this.lockService : NO_LOCK_SERVICE;


return () -> new StoreStatement( neoStores, indexReaderFactory, labelScanStore::newReader, lockService ); return () -> new StoreStatement( neoStores, indexReaderFactory, labelScanStore::newReader, lockService,
allocateCommandCreationContext() );
} }


@Override @Override
Expand All @@ -265,6 +244,12 @@ public StoreReadLayer storeReadLayer()
return storeLayer; return storeLayer;
} }


@Override
public RecordStorageCommandCreationContext allocateCommandCreationContext()
{
return new RecordStorageCommandCreationContext( neoStores, denseNodeThreshold, recordIdBatchSize );
}

@Override @Override
public CommandReaderFactory commandReaderFactory() public CommandReaderFactory commandReaderFactory()
{ {
Expand All @@ -283,10 +268,13 @@ public void createCommands(
{ {
if ( txState != null ) if ( txState != null )
{ {
RecordChangeSet recordChangeSet = new RecordChangeSet( loaders ); // We can make this cast here because we expected that the storageStatement passed in here comes from
TransactionRecordState recordState = new TransactionRecordState( neoStores, integrityValidator, // this storage engine itself, anything else is considered a bug. And we do know the inner workings
recordChangeSet, lastTransactionIdWhenStarted, locks, // of the storage statements that we create.
relationshipCreator, relationshipDeleter, propertyCreator, propertyDeleter ); RecordStorageCommandCreationContext creationContext =
(RecordStorageCommandCreationContext) storageStatement.getCommandCreationContext();
TransactionRecordState recordState =
creationContext.createTransactionRecordState( integrityValidator, lastTransactionIdWhenStarted, locks );


// Visit transaction state and populate these record state objects // Visit transaction state and populate these record state objects
TxStateVisitor txStateVisitor = new TransactionToRecordStateVisitor( recordState, TxStateVisitor txStateVisitor = new TransactionToRecordStateVisitor( recordState,
Expand Down
Expand Up @@ -17,7 +17,7 @@
* You should have received a copy of the GNU General Public License * You should have received a copy of the GNU General Public License
* along with this program. If not, see <http://www.gnu.org/licenses/>. * along with this program. If not, see <http://www.gnu.org/licenses/>.
*/ */
package org.neo4j.kernel.impl.api.store; package org.neo4j.kernel.impl.storageengine.impl.recordstorage;


import java.util.function.IntPredicate; import java.util.function.IntPredicate;
import java.util.function.Supplier; import java.util.function.Supplier;
Expand All @@ -27,15 +27,24 @@
import org.neo4j.kernel.api.exceptions.index.IndexNotFoundKernelException; import org.neo4j.kernel.api.exceptions.index.IndexNotFoundKernelException;
import org.neo4j.kernel.api.schema.index.IndexDescriptor; import org.neo4j.kernel.api.schema.index.IndexDescriptor;
import org.neo4j.kernel.impl.api.IndexReaderFactory; import org.neo4j.kernel.impl.api.IndexReaderFactory;
import org.neo4j.kernel.impl.api.store.AllIdIterator;
import org.neo4j.kernel.impl.api.store.StoreIteratorRelationshipCursor;
import org.neo4j.kernel.impl.api.store.StoreNodeRelationshipCursor;
import org.neo4j.kernel.impl.api.store.StorePropertyCursor;
import org.neo4j.kernel.impl.api.store.StoreSingleNodeCursor;
import org.neo4j.kernel.impl.api.store.StoreSinglePropertyCursor;
import org.neo4j.kernel.impl.api.store.StoreSingleRelationshipCursor;
import org.neo4j.kernel.impl.locking.Lock; import org.neo4j.kernel.impl.locking.Lock;
import org.neo4j.kernel.impl.locking.LockService; import org.neo4j.kernel.impl.locking.LockService;
import org.neo4j.kernel.impl.store.NeoStores; import org.neo4j.kernel.impl.store.NeoStores;
import org.neo4j.kernel.impl.store.NodeStore; import org.neo4j.kernel.impl.store.NodeStore;
import org.neo4j.kernel.impl.store.RecordCursors; import org.neo4j.kernel.impl.store.RecordCursors;
import org.neo4j.kernel.impl.store.RecordStore; import org.neo4j.kernel.impl.store.RecordStore;
import org.neo4j.kernel.impl.store.RelationshipStore; import org.neo4j.kernel.impl.store.RelationshipStore;
import org.neo4j.kernel.impl.store.StoreType;
import org.neo4j.kernel.impl.store.record.RelationshipGroupRecord; import org.neo4j.kernel.impl.store.record.RelationshipGroupRecord;
import org.neo4j.kernel.impl.util.InstanceCache; import org.neo4j.kernel.impl.util.InstanceCache;
import org.neo4j.storageengine.api.CommandCreationContext;
import org.neo4j.storageengine.api.Direction; import org.neo4j.storageengine.api.Direction;
import org.neo4j.storageengine.api.NodeItem; import org.neo4j.storageengine.api.NodeItem;
import org.neo4j.storageengine.api.PropertyItem; import org.neo4j.storageengine.api.PropertyItem;
Expand Down Expand Up @@ -66,6 +75,7 @@ public class StoreStatement implements StorageStatement
private final RecordCursors recordCursors; private final RecordCursors recordCursors;
private final Supplier<LabelScanReader> labelScanStore; private final Supplier<LabelScanReader> labelScanStore;
private final RecordStore<RelationshipGroupRecord> relationshipGroupStore; private final RecordStore<RelationshipGroupRecord> relationshipGroupStore;
private final RecordStorageCommandCreationContext commandCreationContext;


private IndexReaderFactory indexReaderFactory; private IndexReaderFactory indexReaderFactory;
private LabelScanReader labelScanReader; private LabelScanReader labelScanReader;
Expand All @@ -74,11 +84,13 @@ public class StoreStatement implements StorageStatement
private boolean closed; private boolean closed;


public StoreStatement( NeoStores neoStores, Supplier<IndexReaderFactory> indexReaderFactory, public StoreStatement( NeoStores neoStores, Supplier<IndexReaderFactory> indexReaderFactory,
Supplier<LabelScanReader> labelScanReaderSupplier, LockService lockService ) Supplier<LabelScanReader> labelScanReaderSupplier, LockService lockService,
RecordStorageCommandCreationContext commandCreationContext )
{ {
this.neoStores = neoStores; this.neoStores = neoStores;
this.indexReaderFactorySupplier = indexReaderFactory; this.indexReaderFactorySupplier = indexReaderFactory;
this.labelScanStore = labelScanReaderSupplier; this.labelScanStore = labelScanReaderSupplier;
this.commandCreationContext = commandCreationContext;
this.nodeStore = neoStores.getNodeStore(); this.nodeStore = neoStores.getNodeStore();
this.relationshipStore = neoStores.getRelationshipStore(); this.relationshipStore = neoStores.getRelationshipStore();
this.relationshipGroupStore = neoStores.getRelationshipGroupStore(); this.relationshipGroupStore = neoStores.getRelationshipGroupStore();
Expand Down Expand Up @@ -203,6 +215,7 @@ public void close()
assert !closed; assert !closed;
closeSchemaResources(); closeSchemaResources();
recordCursors.close(); recordCursors.close();
commandCreationContext.close();
closed = true; closed = true;
} }


Expand Down Expand Up @@ -250,4 +263,22 @@ public RecordCursors recordCursors()
{ {
return recordCursors; return recordCursors;
} }

@Override
public CommandCreationContext getCommandCreationContext()
{
return commandCreationContext;
}

@Override
public long reserveNode()
{
return commandCreationContext.nextId( StoreType.NODE );
}

@Override
public long reserveRelationship()
{
return commandCreationContext.nextId( StoreType.RELATIONSHIP );
}
} }

0 comments on commit 7acf3b3

Please sign in to comment.