Skip to content

Commit

Permalink
Move indexProviderForOrDefault to IndexingService and hide it
Browse files Browse the repository at this point in the history
  • Loading branch information
fickludd authored and ragadeeshu committed May 21, 2018
1 parent 925b732 commit 50746a8
Show file tree
Hide file tree
Showing 11 changed files with 119 additions and 68 deletions.
Expand Up @@ -649,7 +649,7 @@ private void buildRecovery(

private NeoStoreKernelModule buildKernel( LogFiles logFiles, TransactionAppender appender,
IndexingService indexingService,
DatabaseSchemaState databaseSchemaState, LabelScanStore labelScanStore,
DatabaseSchemaState databaseSchemaState, LabelScanStore labelScanStore,
StorageEngine storageEngine, IndexConfigStore indexConfigStore, TransactionIdStore transactionIdStore,
AvailabilityGuard availabilityGuard, SystemNanoClock clock, PropertyAccessor propertyAccessor )
{
Expand Down
Expand Up @@ -67,12 +67,11 @@
import org.neo4j.kernel.api.exceptions.Status;
import org.neo4j.kernel.api.exceptions.schema.CreateConstraintFailureException;
import org.neo4j.kernel.api.explicitindex.AutoIndexing;
import org.neo4j.kernel.api.index.IndexProvider;
import org.neo4j.kernel.api.schema.index.IndexDescriptor;
import org.neo4j.kernel.api.txstate.ExplicitIndexTransactionState;
import org.neo4j.kernel.api.txstate.TransactionState;
import org.neo4j.kernel.api.txstate.TxStateHolder;
import org.neo4j.kernel.impl.api.index.IndexingService;
import org.neo4j.kernel.impl.api.index.IndexingProvidersService;
import org.neo4j.kernel.impl.api.state.ConstraintIndexCreator;
import org.neo4j.kernel.impl.api.state.TxState;
import org.neo4j.kernel.impl.constraints.ConstraintSemantics;
Expand Down Expand Up @@ -195,7 +194,7 @@ public KernelTransactionImplementation( StatementOperationParts statementOperati
StorageEngine storageEngine, AccessCapability accessCapability, DefaultCursors cursors, AutoIndexing autoIndexing,
ExplicitIndexStore explicitIndexStore, VersionContextSupplier versionContextSupplier,
CollectionsFactorySupplier collectionsFactorySupplier, ConstraintSemantics constraintSemantics,
SchemaState schemaState, IndexingService indexingService )
SchemaState schemaState, IndexingProvidersService indexProviders )
{
this.schemaWriteGuard = schemaWriteGuard;
this.hooks = hooks;
Expand Down Expand Up @@ -223,9 +222,14 @@ public KernelTransactionImplementation( StatementOperationParts statementOperati
this.operations =
new Operations(
allStoreHolder,
new IndexTxStateUpdater( storageReader, allStoreHolder, indexingService ), storageReader,
this, new KernelToken( storageReader, this ), cursors, autoIndexing, constraintIndexCreator,
constraintSemantics );
new IndexTxStateUpdater( storageReader, allStoreHolder, indexProviders ), storageReader,
this,
new KernelToken( storageReader, this ),
cursors,
autoIndexing,
constraintIndexCreator,
constraintSemantics,
indexProviders );
this.collectionsFactory = collectionsFactorySupplier.create();
}

Expand Down Expand Up @@ -1051,11 +1055,6 @@ public Statistics getStatistics()
return statistics;
}

public IndexProvider.Descriptor indexProviderForOrDefault( Optional<String> provider )
{
return storageEngine.indexProviderForOrDefault( provider );
}

public static class Statistics
{
private volatile long cpuTimeNanosWhenQueryStarted;
Expand Down
Expand Up @@ -40,7 +40,7 @@
import org.neo4j.kernel.api.exceptions.Status;
import org.neo4j.kernel.api.explicitindex.AutoIndexing;
import org.neo4j.kernel.api.txstate.ExplicitIndexTransactionState;
import org.neo4j.kernel.impl.api.index.IndexingService;
import org.neo4j.kernel.impl.api.index.IndexingProvidersService;
import org.neo4j.kernel.impl.api.state.ConstraintIndexCreator;
import org.neo4j.kernel.impl.api.state.ExplicitIndexTransactionStateImpl;
import org.neo4j.kernel.impl.constraints.ConstraintSemantics;
Expand Down Expand Up @@ -102,7 +102,7 @@ public class KernelTransactions extends LifecycleAdapter implements Supplier<Ker
private final Supplier<DefaultCursors> cursorsSupplier;
private final AutoIndexing autoIndexing;
private final ExplicitIndexStore explicitIndexStore;
private final IndexingService indexingService;
private final IndexingProvidersService indexProviders;
private final PropertyKeyTokenHolder propertyKeyTokenHolder;
private final CollectionsFactorySupplier collectionsFactorySupplier;
private final SchemaState schemaState;
Expand Down Expand Up @@ -137,22 +137,26 @@ public class KernelTransactions extends LifecycleAdapter implements Supplier<Ker
private volatile boolean stopped = true;

public KernelTransactions( StatementLocksFactory statementLocksFactory,
ConstraintIndexCreator constraintIndexCreator, StatementOperationParts statementOperations,
ConstraintIndexCreator constraintIndexCreator,
StatementOperationParts statementOperations,
SchemaWriteGuard schemaWriteGuard, TransactionHeaderInformationFactory txHeaderFactory,
TransactionCommitProcess transactionCommitProcess, IndexConfigStore indexConfigStore,
ExplicitIndexProviderLookup explicitIndexProviderLookup, TransactionHooks hooks,
TransactionMonitor transactionMonitor, AvailabilityGuard availabilityGuard, Tracers tracers,
StorageEngine storageEngine, Procedures procedures, TransactionIdStore transactionIdStore,
TransactionMonitor transactionMonitor, AvailabilityGuard availabilityGuard,
Tracers tracers,
StorageEngine storageEngine, Procedures procedures,
TransactionIdStore transactionIdStore,
SystemNanoClock clock,
AtomicReference<CpuClock> cpuClockRef, AtomicReference<HeapAllocation> heapAllocationRef, AccessCapability accessCapability,
AtomicReference<CpuClock> cpuClockRef, AtomicReference<HeapAllocation> heapAllocationRef,
AccessCapability accessCapability,
Supplier<DefaultCursors> cursorsSupplier,
AutoIndexing autoIndexing,
ExplicitIndexStore explicitIndexStore,
VersionContextSupplier versionContextSupplier,
CollectionsFactorySupplier collectionsFactorySupplier,
ConstraintSemantics constraintSemantics,
SchemaState schemaState,
IndexingService indexingService,
IndexingProvidersService indexProviders,
PropertyKeyTokenHolder propertyKeyTokenHolder )
{
this.statementLocksFactory = statementLocksFactory;
Expand All @@ -173,7 +177,7 @@ public KernelTransactions( StatementLocksFactory statementLocksFactory,
this.accessCapability = accessCapability;
this.autoIndexing = autoIndexing;
this.explicitIndexStore = explicitIndexStore;
this.indexingService = indexingService;
this.indexProviders = indexProviders;
this.propertyKeyTokenHolder = propertyKeyTokenHolder;
this.explicitIndexTxStateSupplier = () ->
new CachingExplicitIndexTransactionState(
Expand Down Expand Up @@ -371,13 +375,13 @@ public KernelTransactionImplementation newInstance()
{
KernelTransactionImplementation tx =
new KernelTransactionImplementation( statementOperations, schemaWriteGuard, hooks,
constraintIndexCreator, procedures, transactionHeaderInformationFactory,
transactionCommitProcess, transactionMonitor, explicitIndexTxStateSupplier, localTxPool,
clock, cpuClockRef, heapAllocationRef, tracers.transactionTracer, tracers.lockTracer,
tracers.pageCursorTracerSupplier, storageEngine, accessCapability,
cursorsSupplier.get(), autoIndexing,
explicitIndexStore, versionContextSupplier, collectionsFactorySupplier, constraintSemantics,
schemaState, indexingService );
constraintIndexCreator, procedures, transactionHeaderInformationFactory,
transactionCommitProcess, transactionMonitor, explicitIndexTxStateSupplier, localTxPool,
clock, cpuClockRef, heapAllocationRef, tracers.transactionTracer, tracers.lockTracer,
tracers.pageCursorTracerSupplier, storageEngine, accessCapability,
cursorsSupplier.get(), autoIndexing,
explicitIndexStore, versionContextSupplier, collectionsFactorySupplier, constraintSemantics,
schemaState, indexProviders );
this.transactions.add( tx );
return tx;
}
Expand Down
@@ -0,0 +1,46 @@
/*
* Copyright (c) 2002-2018 "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.api.index;

import java.util.Optional;

import org.neo4j.internal.kernel.api.schema.SchemaDescriptor;
import org.neo4j.kernel.api.index.IndexProvider;
import org.neo4j.values.storable.Value;

public interface IndexingProvidersService
{
/**
* Get the index provider descriptor for the index provider with the given name, or the
* descriptor of the default index provider, if no name was given.
*
* @param providerName name of the wanted index provider
*/
IndexProvider.Descriptor indexProviderForNameOrDefault( Optional<String> providerName );

/**
* Validate that the given value tuple can be stored in the index associated with the given schema.
*
* @param schema index schema of the target index
* @param tuple value tuple to validate
*/
void validateBeforeCommit( SchemaDescriptor schema, Value[] tuple );
}

Expand Up @@ -35,6 +35,7 @@
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.concurrent.ExecutionException;

import org.neo4j.function.ThrowingConsumer;
Expand Down Expand Up @@ -90,7 +91,7 @@
* If, however, it is {@link InternalIndexState#ONLINE}, the index provider is required to
* also guarantee that the index had been flushed to disk.
*/
public class IndexingService extends LifecycleAdapter implements IndexingUpdateService
public class IndexingService extends LifecycleAdapter implements IndexingUpdateService, IndexingProvidersService
{
private final IndexSamplingController samplingController;
private final IndexProxyCreator indexProxyCreator;
Expand Down Expand Up @@ -427,11 +428,22 @@ public double indexUniqueValuesPercentage( SchemaDescriptor descriptor ) throws
}
}

@Override
public void validateBeforeCommit( SchemaDescriptor index, Value[] tuple )
{
indexMapRef.validateBeforeCommit( index, tuple );
}

@Override
public IndexProvider.Descriptor indexProviderForNameOrDefault( Optional<String> providerName )
{
if ( providerName.isPresent() )
{
return providerMap.lookup( providerName.get() ).getProviderDescriptor();
}
return providerMap.getDefaultProvider().getProviderDescriptor();
}

/**
* Applies updates from the given {@link IndexUpdates}, which may contain updates for one or more indexes.
* As long as index updates are derived from physical commands and store state there's special treatment
Expand Down
Expand Up @@ -29,7 +29,7 @@
import org.neo4j.internal.kernel.api.NodeCursor;
import org.neo4j.internal.kernel.api.PropertyCursor;
import org.neo4j.kernel.api.schema.index.IndexDescriptor;
import org.neo4j.kernel.impl.api.index.IndexingService;
import org.neo4j.kernel.impl.api.index.IndexingProvidersService;
import org.neo4j.storageengine.api.StorageReader;
import org.neo4j.values.storable.Value;
import org.neo4j.values.storable.ValueTuple;
Expand All @@ -44,15 +44,15 @@ public class IndexTxStateUpdater
{
private final StorageReader storageReader;
private final Read read;
private final IndexingService indexingService;
private final IndexingProvidersService indexProviders;

// We can use the StorageReader directly instead of the SchemaReadOps, because we know that in transactions
// where this class is needed we will never have index changes.
public IndexTxStateUpdater( StorageReader storageReader, Read read, IndexingService indexingService )
public IndexTxStateUpdater( StorageReader storageReader, Read read, IndexingProvidersService indexProviders )
{
this.storageReader = storageReader;
this.read = read;
this.indexingService = indexingService;
this.indexProviders = indexProviders;
}

// LABEL CHANGES
Expand Down Expand Up @@ -95,7 +95,7 @@ void onLabelChange( int labelId, NodeCursor node, PropertyCursor propertyCursor,
switch ( changeType )
{
case ADDED_LABEL:
indexingService.validateBeforeCommit( index.schema(), values );
indexProviders.validateBeforeCommit( index.schema(), values );
read.txState().indexDoUpdateEntry( index.schema(), node.nodeReference(), null, ValueTuple.of( values ) );
break;
case REMOVED_LABEL:
Expand Down Expand Up @@ -124,7 +124,7 @@ void onPropertyAdd( NodeCursor node, PropertyCursor propertyCursor, int property
( index, propertyKeyIds ) ->
{
Value[] values = getValueTuple( node, propertyCursor, propertyKeyId, value, index.schema().getPropertyIds() );
indexingService.validateBeforeCommit( index.schema(), values );
indexProviders.validateBeforeCommit( index.schema(), values );
read.txState().indexDoUpdateEntry( index.schema(), node.nodeReference(), null, ValueTuple.of( values ) );
} );
}
Expand Down Expand Up @@ -177,7 +177,7 @@ void onPropertyChange( NodeCursor node, PropertyCursor propertyCursor, int prope
valuesAfter[i] = value;
}
}
indexingService.validateBeforeCommit( index.schema(), valuesAfter );
indexProviders.validateBeforeCommit( index.schema(), valuesAfter );
read.txState().indexDoUpdateEntry( index.schema(), node.nodeReference(),
ValueTuple.of( valuesBefore ), ValueTuple.of( valuesAfter ) );
} );
Expand Down
Expand Up @@ -81,6 +81,7 @@
import org.neo4j.kernel.api.txstate.ExplicitIndexTransactionState;
import org.neo4j.kernel.api.txstate.TransactionState;
import org.neo4j.kernel.impl.api.KernelTransactionImplementation;
import org.neo4j.kernel.impl.api.index.IndexingProvidersService;
import org.neo4j.kernel.impl.api.state.ConstraintIndexCreator;
import org.neo4j.kernel.impl.constraints.ConstraintSemantics;
import org.neo4j.kernel.impl.index.IndexEntityType;
Expand Down Expand Up @@ -126,6 +127,7 @@ public class Operations implements Write, ExplicitIndexWrite, SchemaWrite
private final DefaultCursors cursors;
private final ConstraintIndexCreator constraintIndexCreator;
private final ConstraintSemantics constraintSemantics;
private final IndexingProvidersService indexProviders;

public Operations(
AllStoreHolder allStoreHolder,
Expand All @@ -136,7 +138,8 @@ public Operations(
DefaultCursors cursors,
AutoIndexing autoIndexing,
ConstraintIndexCreator constraintIndexCreator,
ConstraintSemantics constraintSemantics )
ConstraintSemantics constraintSemantics,
IndexingProvidersService indexProviders )
{
this.token = token;
this.autoIndexing = autoIndexing;
Expand All @@ -147,6 +150,7 @@ public Operations(
this.cursors = cursors;
this.constraintIndexCreator = constraintIndexCreator;
this.constraintSemantics = constraintSemantics;
this.indexProviders = indexProviders;
}

public void initialize()
Expand Down Expand Up @@ -887,7 +891,7 @@ public IndexReference indexCreate( SchemaDescriptor descriptor,
assertValidDescriptor( descriptor, SchemaKernelException.OperationContext.INDEX_CREATION );
assertIndexDoesNotExist( SchemaKernelException.OperationContext.INDEX_CREATION, descriptor );

IndexProvider.Descriptor providerDescriptor = ktx.indexProviderForOrDefault( provider );
IndexProvider.Descriptor providerDescriptor = indexProviders.indexProviderForNameOrDefault( provider );
IndexDescriptor index = IndexDescriptorFactory.forSchema( descriptor, name, providerDescriptor );
ktx.txState().indexDoAdd( index );
return index;
Expand All @@ -896,7 +900,7 @@ public IndexReference indexCreate( SchemaDescriptor descriptor,
// Note: this will be sneakily executed by an internal transaction, so no additional locking is required.
public IndexDescriptor indexUniqueCreate( SchemaDescriptor schema, Optional<String> provider )
{
IndexProvider.Descriptor providerDescriptor = ktx.indexProviderForOrDefault( provider );
IndexProvider.Descriptor providerDescriptor = indexProviders.indexProviderForNameOrDefault( provider );
IndexDescriptor index =
IndexDescriptorFactory.uniqueForSchema( schema,
Optional.empty(),
Expand Down
Expand Up @@ -248,16 +248,6 @@ public StorageReader newReader()
schemaCache, indexReaderFactory, labelScanStore::newReader, lockService, allocateCommandCreationContext() );
}

@Override
public IndexProvider.Descriptor indexProviderForOrDefault( Optional<String> providerName )
{
if ( providerName.isPresent() )
{
return indexProviderMap.lookup( providerName.get() ).getProviderDescriptor();
}
return indexProviderMap.getDefaultProvider().getProviderDescriptor();
}

@Override
public RecordStorageCommandCreationContext allocateCommandCreationContext()
{
Expand Down
Expand Up @@ -20,15 +20,12 @@
package org.neo4j.storageengine.api;

import java.util.Collection;
import java.util.Optional;
import java.util.stream.Stream;

import org.neo4j.internal.kernel.api.exceptions.TransactionFailureException;
import org.neo4j.internal.kernel.api.exceptions.schema.ConstraintValidationException;
import org.neo4j.io.pagecache.IOLimiter;
import org.neo4j.kernel.api.exceptions.schema.CreateConstraintFailureException;
import org.neo4j.kernel.api.index.IndexProvider;
import org.neo4j.kernel.impl.api.index.IndexProviderMap;
import org.neo4j.kernel.info.DiagnosticsManager;
import org.neo4j.storageengine.api.lock.ResourceLocker;
import org.neo4j.storageengine.api.txstate.ReadableTransactionState;
Expand All @@ -48,8 +45,6 @@ public interface StorageEngine
*/
StorageReader newReader();

IndexProvider.Descriptor indexProviderForOrDefault( Optional<String> providerName );

/**
* @return a new {@link CommandCreationContext} meant to be kept for multiple calls to
* {@link #createCommands(Collection, ReadableTransactionState, StorageReader, ResourceLocker,
Expand Down

0 comments on commit 50746a8

Please sign in to comment.