Skip to content

Commit

Permalink
Move the construction of IndexConfigStore into NeoStoreDataSource
Browse files Browse the repository at this point in the history
  • Loading branch information
chrisvest committed Nov 16, 2015
1 parent c99c04f commit 8c4897b
Show file tree
Hide file tree
Showing 10 changed files with 98 additions and 65 deletions.
Expand Up @@ -71,10 +71,8 @@
import org.neo4j.kernel.impl.api.index.IndexUpdatesValidator;
import org.neo4j.kernel.impl.api.index.IndexingService;
import org.neo4j.kernel.impl.api.index.RecoveryIndexingUpdatesValidator;
import org.neo4j.kernel.impl.api.index.SchemaIndexProviderMap;
import org.neo4j.kernel.impl.api.scan.LabelScanStoreProvider;
import org.neo4j.kernel.impl.api.state.ConstraintIndexCreator;
import org.neo4j.kernel.impl.api.store.ProcedureCache;
import org.neo4j.kernel.impl.api.store.StoreReadLayer;
import org.neo4j.kernel.impl.constraints.ConstraintSemantics;
import org.neo4j.kernel.impl.core.LabelTokenHolder;
Expand Down Expand Up @@ -115,6 +113,7 @@
import org.neo4j.kernel.impl.transaction.log.ReadableLogChannel;
import org.neo4j.kernel.impl.transaction.log.ReadableVersionableLogChannel;
import org.neo4j.kernel.impl.transaction.log.TransactionAppender;
import org.neo4j.kernel.impl.transaction.log.TransactionIdStore;
import org.neo4j.kernel.impl.transaction.log.TransactionMetadataCache;
import org.neo4j.kernel.impl.transaction.log.checkpoint.CheckPointScheduler;
import org.neo4j.kernel.impl.transaction.log.checkpoint.CheckPointThreshold;
Expand All @@ -135,7 +134,6 @@
import org.neo4j.kernel.impl.transaction.log.rotation.LogRotation;
import org.neo4j.kernel.impl.transaction.log.rotation.LogRotationImpl;
import org.neo4j.kernel.impl.transaction.log.rotation.StoreFlusher;
import org.neo4j.kernel.impl.transaction.state.IntegrityValidator;
import org.neo4j.kernel.impl.transaction.state.NeoStoreFileListing;
import org.neo4j.kernel.impl.transaction.state.NeoStoreTransactionContextFactory;
import org.neo4j.kernel.impl.transaction.state.NeoStoresSupplier;
Expand Down Expand Up @@ -315,7 +313,6 @@ boolean applicable( DiagnosticsPhase phase )
private final Guard guard;
private final Map<String,IndexImplementation> indexProviders = new HashMap<>();
private final LegacyIndexProviderLookup legacyIndexProviderLookup;
private final IndexConfigStore indexConfigStore;
private final ConstraintSemantics constraintSemantics;

private Dependencies dependencies;
Expand Down Expand Up @@ -347,18 +344,31 @@ boolean applicable( DiagnosticsPhase phase )
* core API are now slowly accumulating in the Kernel implementation. Over time, these components should be
* refactored into bigger components that wrap the very granular things we depend on here.
*/
public NeoStoreDataSource( File storeDir, Config config, IdGeneratorFactory idGeneratorFactory, LogProvider logProvider,
JobScheduler scheduler, TokenNameLookup tokenNameLookup, DependencyResolver dependencyResolver,
PropertyKeyTokenHolder propertyKeyTokens, LabelTokenHolder labelTokens,
RelationshipTypeTokenHolder relationshipTypeTokens, Locks lockManager,
SchemaWriteGuard schemaWriteGuard, TransactionEventHandlers transactionEventHandlers,
IndexingService.Monitor indexingServiceMonitor, FileSystemAbstraction fs,
StoreUpgrader storeMigrationProcess, TransactionMonitor transactionMonitor,
KernelHealth kernelHealth, PhysicalLogFile.Monitor physicalLogMonitor,
public NeoStoreDataSource(
File storeDir,
Config config,
IdGeneratorFactory idGeneratorFactory,
LogProvider logProvider,
JobScheduler scheduler,
TokenNameLookup tokenNameLookup,
DependencyResolver dependencyResolver,
PropertyKeyTokenHolder propertyKeyTokens,
LabelTokenHolder labelTokens,
RelationshipTypeTokenHolder relationshipTypeTokens,
Locks lockManager,
SchemaWriteGuard schemaWriteGuard,
TransactionEventHandlers transactionEventHandlers,
IndexingService.Monitor indexingServiceMonitor,
FileSystemAbstraction fs,
StoreUpgrader storeMigrationProcess,
TransactionMonitor transactionMonitor,
KernelHealth kernelHealth,
PhysicalLogFile.Monitor physicalLogMonitor,
TransactionHeaderInformationFactory transactionHeaderInformationFactory,
StartupStatisticsProvider startupStatistics,
NodeManager nodeManager, Guard guard,
IndexConfigStore indexConfigStore, CommitProcessFactory commitProcessFactory,
NodeManager nodeManager,
Guard guard,
CommitProcessFactory commitProcessFactory,
PageCache pageCache,
ConstraintSemantics constraintSemantics,
Monitors monitors,
Expand Down Expand Up @@ -386,7 +396,6 @@ public NeoStoreDataSource( File storeDir, Config config, IdGeneratorFactory idGe
this.startupStatistics = startupStatistics;
this.nodeManager = nodeManager;
this.guard = guard;
this.indexConfigStore = indexConfigStore;
this.constraintSemantics = constraintSemantics;
this.monitors = monitors;
this.tracers = tracers;
Expand Down Expand Up @@ -439,7 +448,10 @@ public void start() throws IOException
indexProvider = dependencyResolver.resolveDependency( SchemaIndexProvider.class,
SchemaIndexProvider.HIGHEST_PRIORITIZED_OR_NONE );

IndexConfigStore indexConfigStore = new IndexConfigStore( storeDir, fs );
dependencies.satisfyDependency( lockService );
dependencies.satisfyDependency( indexConfigStore );
life.add( indexConfigStore );

// Monitor listeners
LoggingLogFileMonitor loggingLogMonitor = new LoggingLogFileMonitor( msgLog );
Expand All @@ -460,7 +472,7 @@ public void start() throws IOException
// TODO Introduce a StorageEngine abstraction at the StoreLayerModule boundary
storageEngine = buildStorageEngine(
propertyKeyTokenHolder, labelTokens, relationshipTypeTokens, legacyIndexProviderLookup,
updateableSchemaState::clear );
indexConfigStore, updateableSchemaState::clear );

TransactionLogModule transactionLogModule =
buildTransactionLogs( storeDir, config, logProvider, scheduler,
Expand All @@ -473,13 +485,13 @@ public void start() throws IOException
transactionLogModule.logFiles(), transactionLogModule.storeFlusher(), startupStatistics,
storageEngine.legacyIndexApplierLookup(), storageEngine );

KernelModule kernelModule = buildKernel( storageEngine.integrityValidator(),
KernelModule kernelModule = buildKernel(
transactionLogModule.transactionAppender(), storageEngine.neoStores(),
transactionLogModule.storeApplier(), storageEngine.indexingService(),
storageEngine.indexUpdatesValidator(),
storageEngine.storeReadLayer(),
updateableSchemaState, storageEngine.labelScanStore(),
storageEngine.schemaIndexProviderMap(), storageEngine.procedureCache(), storageEngine );
storageEngine );


// Do these assignments last so that we can ensure no cyclical dependencies exist
Expand Down Expand Up @@ -568,15 +580,16 @@ private void upgradeStore( File storeDir, StoreUpgrader storeMigrationProcess, S
private StorageEngine buildStorageEngine(
PropertyKeyTokenHolder propertyKeyTokenHolder, LabelTokenHolder labelTokens,
RelationshipTypeTokenHolder relationshipTypeTokens,
LegacyIndexProviderLookup legacyIndexProviderLookup, Runnable schemaStateChangeCallback )
LegacyIndexProviderLookup legacyIndexProviderLookup, IndexConfigStore indexConfigStore,
Runnable schemaStateChangeCallback )
{
LabelScanStoreProvider labelScanStore = dependencyResolver.resolveDependency( LabelScanStoreProvider.class,
LabelScanStoreProvider.HIGHEST_PRIORITIZED );
return life.add(
new RecordStorageEngine( storeDir, config, idGeneratorFactory, pageCache, fs, logProvider, propertyKeyTokenHolder,
labelTokens, relationshipTypeTokens, schemaStateChangeCallback, constraintSemantics, scheduler,
tokenNameLookup, lockService, indexProvider, indexingServiceMonitor, kernelHealth,
labelScanStore, legacyIndexProviderLookup ) );
labelScanStore, legacyIndexProviderLookup, indexConfigStore ) );
}

private TransactionLogModule buildTransactionLogs(
Expand All @@ -595,13 +608,13 @@ private TransactionLogModule buildTransactionLogs(
final IdOrderingQueue legacyIndexTransactionOrdering = new SynchronizedArrayIdOrderingQueue( 20 );
final TransactionRepresentationStoreApplier storeApplier = dependencies.satisfyDependency(
new TransactionRepresentationStoreApplier( alwaysCreateNewWriter( storageEngine.labelScanStore() ),
lockService, indexConfigStore,
lockService, storageEngine.indexConfigStore(),
legacyIndexTransactionOrdering, storageEngine ) );

MetaDataStore metaDataStore = storageEngine.metaDataStore();
TransactionIdStore transactionIdStore = storageEngine.transactionIdStore();
final PhysicalLogFile logFile = life.add( new PhysicalLogFile( fileSystemAbstraction, logFiles,
config.get( GraphDatabaseSettings.logical_log_rotation_threshold ), metaDataStore,
metaDataStore, physicalLogMonitor, transactionMetadataCache ) );
config.get( GraphDatabaseSettings.logical_log_rotation_threshold ), transactionIdStore,
storageEngine.logVersionRepository(), physicalLogMonitor, transactionMetadataCache ) );

final PhysicalLogFileInformation.LogVersionToTimestamp
logInformation = new PhysicalLogFileInformation.LogVersionToTimestamp()
Expand All @@ -626,7 +639,7 @@ public long getTimestampForVersion( long version ) throws IOException
}
};
final LogFileInformation logFileInformation =
new PhysicalLogFileInformation( logFiles, transactionMetadataCache, metaDataStore, logInformation );
new PhysicalLogFileInformation( logFiles, transactionMetadataCache, transactionIdStore, logInformation );

String pruningConf = config.get(
config.get( GraphDatabaseFacadeFactory.Configuration.ephemeral )
Expand All @@ -643,7 +656,7 @@ public long getTimestampForVersion( long version ) throws IOException
new LogRotationImpl( monitors.newMonitor( LogRotation.Monitor.class ), logFile, kernelHealth );

final TransactionAppender appender = life.add( new BatchingTransactionAppender(
logFile, logRotation, transactionMetadataCache, metaDataStore, legacyIndexTransactionOrdering,
logFile, logRotation, transactionMetadataCache, transactionIdStore, legacyIndexTransactionOrdering,
kernelHealth ) );
final LogicalTransactionStore logicalTransactionStore =
new PhysicalLogicalTransactionStore( logFile, transactionMetadataCache );
Expand All @@ -660,7 +673,7 @@ public long getTimestampForVersion( long version ) throws IOException
CheckPointThresholds.or( countCommittedTransactionThreshold, timeCheckPointThreshold );

final CheckPointerImpl checkPointer = new CheckPointerImpl(
metaDataStore, threshold, storeFlusher, logPruning, appender, kernelHealth, logProvider,
transactionIdStore, threshold, storeFlusher, logPruning, appender, kernelHealth, logProvider,
tracers.checkPointTracer );

long recurringPeriod = Math.min( timeMillisThreshold, TimeUnit.SECONDS.toMillis( 10 ) );
Expand Down Expand Up @@ -753,7 +766,7 @@ private void buildRecovery( final FileSystemAbstraction fileSystemAbstraction,
final RecoveryIndexingUpdatesValidator indexUpdatesValidator = new RecoveryIndexingUpdatesValidator( indexingService );
final TransactionRepresentationStoreApplier storeRecoverer =
new TransactionRepresentationStoreApplier( labelScanWriters,
lockService, indexConfigStore, IdOrderingQueue.BYPASS,
lockService, storageEngine.indexConfigStore(), IdOrderingQueue.BYPASS,
storageEngine );

RecoveryVisitor recoveryVisitor =
Expand Down Expand Up @@ -783,12 +796,11 @@ public void init() throws Throwable
} );
}

private KernelModule buildKernel( IntegrityValidator integrityValidator, TransactionAppender appender,
private KernelModule buildKernel( TransactionAppender appender,
NeoStores neoStores, TransactionRepresentationStoreApplier storeApplier,
IndexingService indexingService,
IndexUpdatesValidator indexUpdatesValidator, StoreReadLayer storeLayer,
UpdateableSchemaState updateableSchemaState, LabelScanStore labelScanStore,
SchemaIndexProviderMap schemaIndexProviderMap, ProcedureCache procedureCache,
StorageEngine storageEngine )
{
TransactionCommitProcess transactionCommitProcess = commitProcessFactory.create( appender, storeApplier,
Expand All @@ -810,7 +822,7 @@ public KernelAPI get()
ConstraintIndexCreator constraintIndexCreator =
new ConstraintIndexCreator( kernelProvider, indexingService );

LegacyIndexStore legacyIndexStore = new LegacyIndexStore( config, indexConfigStore, kernelProvider,
LegacyIndexStore legacyIndexStore = new LegacyIndexStore( config, storageEngine.indexConfigStore(), kernelProvider,
legacyIndexProviderLookup );

LegacyPropertyTrackers legacyPropertyTrackers = new LegacyPropertyTrackers( propertyKeyTokenHolder,
Expand All @@ -828,7 +840,7 @@ public KernelAPI get()
locks, constraintIndexCreator,
statementOperations, updateableSchemaState, schemaWriteGuard,
transactionHeaderInformationFactory, transactionCommitProcess,
indexConfigStore, legacyIndexProviderLookup, hooks, constraintSemantics,
storageEngine.indexConfigStore(), legacyIndexProviderLookup, hooks, constraintSemantics,
transactionMonitor, life, tracers, storageEngine ) );

final Kernel kernel = new Kernel( kernelTransactions, hooks, kernelHealth, transactionMonitor );
Expand Down
Expand Up @@ -62,7 +62,6 @@
import org.neo4j.kernel.impl.coreapi.NodeAutoIndexerImpl;
import org.neo4j.kernel.impl.coreapi.RelationshipAutoIndexerImpl;
import org.neo4j.kernel.impl.coreapi.schema.SchemaImpl;
import org.neo4j.kernel.impl.index.IndexConfigStore;
import org.neo4j.kernel.impl.logging.LogService;
import org.neo4j.kernel.impl.query.QueryEngineProvider;
import org.neo4j.kernel.impl.query.QueryExecutionEngine;
Expand Down Expand Up @@ -135,9 +134,6 @@ public DataSourceModule( final GraphDatabaseFacadeFactory.Dependencies dependenc
transactionEventHandlers = new TransactionEventHandlers( nodeActions, relationshipActions,
threadToTransactionBridge );

IndexConfigStore indexStore =
life.add( deps.satisfyDependency( new IndexConfigStore( storeDir, fileSystem ) ) );

diagnosticsManager.prependProvider( config );

life.add( platformModule.kernelExtensions );
Expand Down Expand Up @@ -196,7 +192,7 @@ public GraphDatabaseService getGraphDatabaseService()
platformModule.monitors.newMonitor( IndexingService.Monitor.class ), fileSystem,
storeMigrationProcess, platformModule.transactionMonitor, kernelHealth,
platformModule.monitors.newMonitor( PhysicalLogFile.Monitor.class ),
editionModule.headerInformationFactory, startupStatistics, nodeManager, guard, indexStore,
editionModule.headerInformationFactory, startupStatistics, nodeManager, guard,
editionModule.commitProcessFactory, pageCache, editionModule.constraintSemantics,
platformModule.monitors, platformModule.tracers ) );
dataSourceManager.register( neoStoreDataSource );
Expand Down
Expand Up @@ -28,8 +28,11 @@
import org.neo4j.kernel.impl.api.store.ProcedureCache;
import org.neo4j.kernel.impl.api.store.StoreReadLayer;
import org.neo4j.kernel.impl.core.CacheAccessBackDoor;
import org.neo4j.kernel.impl.index.IndexConfigStore;
import org.neo4j.kernel.impl.store.MetaDataStore;
import org.neo4j.kernel.impl.store.NeoStores;
import org.neo4j.kernel.impl.transaction.log.LogVersionRepository;
import org.neo4j.kernel.impl.transaction.log.TransactionIdStore;
import org.neo4j.kernel.impl.transaction.state.IntegrityValidator;

/**
Expand All @@ -38,6 +41,8 @@
public interface StorageEngine
{
StoreReadLayer storeReadLayer();
TransactionIdStore transactionIdStore();
LogVersionRepository logVersionRepository();

@Deprecated
ProcedureCache procedureCache();
Expand Down Expand Up @@ -69,6 +74,9 @@ public interface StorageEngine
@Deprecated
LegacyIndexApplierLookup legacyIndexApplierLookup();

@Deprecated
IndexConfigStore indexConfigStore();

@Deprecated
KernelHealth kernelHealth();

Expand Down
Expand Up @@ -54,13 +54,16 @@
import org.neo4j.kernel.impl.core.LabelTokenHolder;
import org.neo4j.kernel.impl.core.PropertyKeyTokenHolder;
import org.neo4j.kernel.impl.core.RelationshipTypeTokenHolder;
import org.neo4j.kernel.impl.index.IndexConfigStore;
import org.neo4j.kernel.impl.locking.LockService;
import org.neo4j.kernel.impl.storageengine.StorageEngine;
import org.neo4j.kernel.impl.store.MetaDataStore;
import org.neo4j.kernel.impl.store.NeoStores;
import org.neo4j.kernel.impl.store.SchemaStorage;
import org.neo4j.kernel.impl.store.StoreFactory;
import org.neo4j.kernel.impl.store.record.SchemaRule;
import org.neo4j.kernel.impl.transaction.log.LogVersionRepository;
import org.neo4j.kernel.impl.transaction.log.TransactionIdStore;
import org.neo4j.kernel.impl.transaction.state.DefaultSchemaIndexProviderMap;
import org.neo4j.kernel.impl.transaction.state.IntegrityValidator;
import org.neo4j.kernel.impl.transaction.state.NeoStoreIndexStoreView;
Expand Down Expand Up @@ -91,6 +94,7 @@ public class RecordStorageEngine implements StorageEngine, Lifecycle
private final RelationshipTypeTokenHolder relationshipTypeTokenHolder;
private final LabelTokenHolder labelTokenHolder;
private final KernelHealth kernelHealth;
private final IndexConfigStore indexConfigStore;
private final SchemaCache schemaCache;
private final IntegrityValidator integrityValidator;
private final IndexUpdatesValidator indexUpdatesValidator;
Expand Down Expand Up @@ -119,12 +123,14 @@ public RecordStorageEngine(
IndexingService.Monitor indexingServiceMonitor,
KernelHealth kernelHealth,
LabelScanStoreProvider labelScanStoreProvider,
LegacyIndexProviderLookup legacyIndexProviderLookup )
LegacyIndexProviderLookup legacyIndexProviderLookup,
IndexConfigStore indexConfigStore )
{
this.propertyKeyTokenHolder = propertyKeyTokenHolder;
this.relationshipTypeTokenHolder = relationshipTypeTokens;
this.labelTokenHolder = labelTokens;
this.kernelHealth = kernelHealth;
this.indexConfigStore = indexConfigStore;
final StoreFactory storeFactory = new StoreFactory( storeDir, config, idGeneratorFactory, pageCache, fs, logProvider );
neoStores = storeFactory.openAllNeoStores( true );

Expand Down Expand Up @@ -176,6 +182,18 @@ public StoreReadLayer storeReadLayer()
return storeLayer;
}

@Override
public TransactionIdStore transactionIdStore()
{
return neoStores.getMetaDataStore();
}

@Override
public LogVersionRepository logVersionRepository()
{
return neoStores.getMetaDataStore();
}

@Override
public ProcedureCache procedureCache()
{
Expand Down Expand Up @@ -236,6 +254,12 @@ public LegacyIndexApplierLookup legacyIndexApplierLookup()
return legacyIndexApplierLookup;
}

@Override
public IndexConfigStore indexConfigStore()
{
return indexConfigStore;
}

@Override
public KernelHealth kernelHealth()
{
Expand Down

0 comments on commit 8c4897b

Please sign in to comment.