From 130187646105346f735efb96d637e5498ec807fd Mon Sep 17 00:00:00 2001 From: Ragnar Mellbin Date: Tue, 29 May 2018 08:16:31 +0200 Subject: [PATCH] Generify Lucene index --- .../internal/kernel/api/IndexReference.java | 12 ++ .../neo4j/internal/kernel/api/SchemaRead.java | 8 + .../kernel/api/schema/SchemaDescriptor.java | 69 +++++++ .../kernel/impl/api/store/SchemaCache.java | 33 ++- .../kernel/impl/newapi/AllStoreHolder.java | 12 ++ .../recordstorage/RecordStorageReader.java | 6 + .../storageengine/api/StorageReader.java | 6 + .../storageengine/api/StubStorageCursors.java | 6 + .../api/impl/index/AbstractLuceneIndex.java | 79 ++++++- .../index/AbstractLuceneIndexAccessor.java | 192 ++++++++++++++++++ .../index/AbstractLuceneIndexProvider.java | 112 ++++++++++ .../kernel/api/impl/index/DatabaseIndex.java | 34 +++- .../index/ReadOnlyAbstractDatabaseIndex.java | 55 ++++- .../index/WritableAbstractDatabaseIndex.java | 62 +++++- .../api/impl/schema/LuceneIndexAccessor.java | 139 ++----------- .../schema/LuceneIndexProviderFactory.java | 1 - .../api/impl/schema/LuceneSchemaIndex.java | 102 ++-------- .../impl/schema/LuceneSchemaIndexBuilder.java | 1 - .../schema/ReadOnlyDatabaseSchemaIndex.java | 48 +---- .../kernel/api/impl/schema/SchemaIndex.java | 32 +-- .../schema/WritableDatabaseSchemaIndex.java | 69 +------ .../populator/LuceneIndexPopulator.java | 9 +- .../NonUniqueLuceneIndexPopulator.java | 2 +- .../populator/UniqueLuceneIndexPopulator.java | 2 +- .../LuceneAllEntriesIndexAccessorReader.java | 10 +- .../CompositeDuplicateCheckingCollector.java | 2 +- .../index/DatabaseIndexIntegrationTest.java | 19 +- 27 files changed, 736 insertions(+), 386 deletions(-) create mode 100644 community/lucene-index/src/main/java/org/neo4j/kernel/api/impl/index/AbstractLuceneIndexAccessor.java create mode 100644 community/lucene-index/src/main/java/org/neo4j/kernel/api/impl/index/AbstractLuceneIndexProvider.java diff --git a/community/kernel-api/src/main/java/org/neo4j/internal/kernel/api/IndexReference.java b/community/kernel-api/src/main/java/org/neo4j/internal/kernel/api/IndexReference.java index 1fe65307de4a..2605a1cfadd1 100644 --- a/community/kernel-api/src/main/java/org/neo4j/internal/kernel/api/IndexReference.java +++ b/community/kernel-api/src/main/java/org/neo4j/internal/kernel/api/IndexReference.java @@ -23,6 +23,7 @@ import java.util.List; import org.neo4j.helpers.collection.Iterators; +import org.neo4j.internal.kernel.api.schema.SchemaDescriptor; import org.neo4j.internal.kernel.api.schema.SchemaUtil; import org.neo4j.values.storable.ValueCategory; @@ -49,6 +50,11 @@ public interface IndexReference extends IndexCapability */ int[] properties(); + /** + * Returns the schema of this index. + */ + SchemaDescriptor schema(); + /** * Returns the key (or name) of the index provider that backs this index. */ @@ -117,6 +123,12 @@ public int[] properties() return new int[0]; } + @Override + public SchemaDescriptor schema() + { + return SchemaDescriptor.NO_SCHEMA; + } + @Override public String providerKey() { diff --git a/community/kernel-api/src/main/java/org/neo4j/internal/kernel/api/SchemaRead.java b/community/kernel-api/src/main/java/org/neo4j/internal/kernel/api/SchemaRead.java index 6ea1aa7a5b55..7e83197564aa 100644 --- a/community/kernel-api/src/main/java/org/neo4j/internal/kernel/api/SchemaRead.java +++ b/community/kernel-api/src/main/java/org/neo4j/internal/kernel/api/SchemaRead.java @@ -71,6 +71,14 @@ public interface SchemaRead */ Iterator indexesGetForLabel( int labelId ); + /** + * Returns the index with the given name + * + * @param name The name of the index you are looking for + * @return The index associated with the given name + */ + IndexReference indexGetForName( String name ); + /** * Returns all indexes used in the database * diff --git a/community/kernel-api/src/main/java/org/neo4j/internal/kernel/api/schema/SchemaDescriptor.java b/community/kernel-api/src/main/java/org/neo4j/internal/kernel/api/schema/SchemaDescriptor.java index e2750e1a47dd..d23e8285962e 100644 --- a/community/kernel-api/src/main/java/org/neo4j/internal/kernel/api/schema/SchemaDescriptor.java +++ b/community/kernel-api/src/main/java/org/neo4j/internal/kernel/api/schema/SchemaDescriptor.java @@ -38,6 +38,75 @@ */ public interface SchemaDescriptor extends SchemaDescriptorSupplier { + SchemaDescriptor NO_SCHEMA = new SchemaDescriptor() + { + @Override + public boolean isAffected( long[] entityIds ) + { + return false; + } + + @Override + public R computeWith( SchemaComputer computer ) + { + return null; + } + + @Override + public void processWith( SchemaProcessor processor ) + { + + } + + @Override + public String userDescription( TokenNameLookup tokenNameLookup ) + { + return "NO_SCHEMA"; + } + + @Override + public int[] getPropertyIds() + { + return new int[0]; + } + + @Override + public int[] getEntityTokenIds() + { + return new int[0]; + } + + @Override + public int keyId() + { + return 0; + } + + @Override + public ResourceType keyType() + { + return null; + } + + @Override + public EntityType entityType() + { + return null; + } + + @Override + public PropertySchemaType propertySchemaType() + { + return null; + } + + @Override + public SchemaDescriptor schema() + { + return null; + } + }; + int[] ANY_ENTITY_TOKEN = new int[0]; static boolean isAnyEntityTokenSchema( SchemaDescriptor schema ) diff --git a/community/kernel/src/main/java/org/neo4j/kernel/impl/api/store/SchemaCache.java b/community/kernel/src/main/java/org/neo4j/kernel/impl/api/store/SchemaCache.java index 4809c60bc4c9..434b9404a691 100644 --- a/community/kernel/src/main/java/org/neo4j/kernel/impl/api/store/SchemaCache.java +++ b/community/kernel/src/main/java/org/neo4j/kernel/impl/api/store/SchemaCache.java @@ -176,6 +176,11 @@ public Iterator indexesByProperty( int propertyId ) return schemaCacheState.indexesByProperty( propertyId ); } + public CapableIndexDescriptor indexDescriptorForName( String name ) + { + return schemaCacheState.indexDescriptorByName( name ); + } + private static class SchemaCacheState { private final ConstraintSemantics constraintSemantics; @@ -186,6 +191,7 @@ private static class SchemaCacheState private final Map indexDescriptors; private final MutableIntObjectMap> indexDescriptorsByLabel; + private final Map indexDescriptorsByName; private final Map,Object> dependantState; private final MutableIntObjectMap> indexByProperty; @@ -200,6 +206,7 @@ private static class SchemaCacheState this.indexDescriptors = new HashMap<>(); this.indexDescriptorsByLabel = new IntObjectHashMap<>(); + this.indexDescriptorsByName = new HashMap<>(); this.dependantState = new ConcurrentHashMap<>(); this.indexByProperty = new IntObjectHashMap<>(); load( rules ); @@ -214,6 +221,7 @@ private static class SchemaCacheState this.indexDescriptors = new HashMap<>( schemaCacheState.indexDescriptors ); this.indexDescriptorsByLabel = IntObjectHashMap.newMap( schemaCacheState.indexDescriptorsByLabel ); + this.indexDescriptorsByName = new HashMap<>( schemaCacheState.indexDescriptorsByName ); this.dependantState = new ConcurrentHashMap<>(); this.indexByProperty = IntObjectHashMap.newMap( schemaCacheState.indexByProperty ); this.indexProviderMap = schemaCacheState.indexProviderMap; @@ -262,6 +270,11 @@ CapableIndexDescriptor indexDescriptor( SchemaDescriptor descriptor ) return indexDescriptors.get( descriptor ); } + CapableIndexDescriptor indexDescriptorByName( String name ) + { + return indexDescriptorsByName.get( name ); + } + Iterator indexesByProperty( int propertyId ) { List indexes = indexByProperty.get( propertyId ); @@ -293,6 +306,14 @@ else if ( rule instanceof StoreIndexDescriptor ) indexDescriptorById.put( index.getId(), index ); SchemaDescriptor schemaDescriptor = index.schema(); indexDescriptors.put( schemaDescriptor, index ); + indexDescriptorsByName.put( rule.getName(), index ); + for ( int entityTokenId : schemaDescriptor.getEntityTokenIds() ) + { + Set forLabel = + indexDescriptorsByLabel.getIfAbsent( entityTokenId, HashSet::new ); + forLabel.add( index ); + } + Set forLabel = indexDescriptorsByLabel.getIfAbsentPut( schemaDescriptor.keyId(), HashSet::new ); @@ -319,12 +340,16 @@ else if ( indexDescriptorById.containsKey( id ) ) CapableIndexDescriptor index = indexDescriptorById.remove( id ); SchemaDescriptor schema = index.schema(); indexDescriptors.remove( schema ); + indexDescriptorsByName.remove( index.getName() ); - Set forLabel = indexDescriptorsByLabel.get( schema.keyId() ); - forLabel.remove( index ); - if ( forLabel.isEmpty() ) + for ( int entityTokenId : schema.getEntityTokenIds() ) { - indexDescriptorsByLabel.remove( schema.keyId() ); + Set forLabel = indexDescriptorsByLabel.get( entityTokenId ); + forLabel.remove( index ); + if ( forLabel.isEmpty() ) + { + indexDescriptorsByLabel.remove( entityTokenId ); + } } for ( int propertyId : index.schema().getPropertyIds() ) diff --git a/community/kernel/src/main/java/org/neo4j/kernel/impl/newapi/AllStoreHolder.java b/community/kernel/src/main/java/org/neo4j/kernel/impl/newapi/AllStoreHolder.java index bbfccb839e1e..323af69e11cd 100644 --- a/community/kernel/src/main/java/org/neo4j/kernel/impl/newapi/AllStoreHolder.java +++ b/community/kernel/src/main/java/org/neo4j/kernel/impl/newapi/AllStoreHolder.java @@ -410,6 +410,18 @@ public Iterator indexesGetForLabel( int labelId ) return (Iterator)iterator; } + @Override + public IndexReference indexGetForName( String name ) + { + ktx.assertOpen(); + + // This is only used for querying, and thus is is "fine" (at least for now) that it is not tx-state aware + + IndexDescriptor index = storageReader.indexGetForName( name ); + //TODO locking + return index; + } + @Override public Iterator indexesGetAll() { diff --git a/community/kernel/src/main/java/org/neo4j/kernel/impl/storageengine/impl/recordstorage/RecordStorageReader.java b/community/kernel/src/main/java/org/neo4j/kernel/impl/storageengine/impl/recordstorage/RecordStorageReader.java index e67257cccff6..263096f9fd97 100644 --- a/community/kernel/src/main/java/org/neo4j/kernel/impl/storageengine/impl/recordstorage/RecordStorageReader.java +++ b/community/kernel/src/main/java/org/neo4j/kernel/impl/storageengine/impl/recordstorage/RecordStorageReader.java @@ -140,6 +140,12 @@ public Iterator indexesGetForLabel( int labelId ) return schemaCache.indexDescriptorsForLabel( labelId ); } + @Override + public CapableIndexDescriptor indexGetForName( String name ) + { + return schemaCache.indexDescriptorForName( name ); + } + @Override public Iterator indexesGetAll() { diff --git a/community/kernel/src/main/java/org/neo4j/storageengine/api/StorageReader.java b/community/kernel/src/main/java/org/neo4j/storageengine/api/StorageReader.java index 40a313c93bee..e89ba3f068ae 100644 --- a/community/kernel/src/main/java/org/neo4j/storageengine/api/StorageReader.java +++ b/community/kernel/src/main/java/org/neo4j/storageengine/api/StorageReader.java @@ -126,6 +126,12 @@ public interface StorageReader extends AutoCloseable */ Iterator indexesGetForLabel( int labelId ); + /** + * @param name name of index to find + * @return {@link IndexDescriptor} associated with the given {@code name}. + */ + CapableIndexDescriptor indexGetForName( String name ); + /** * @return all {@link CapableIndexDescriptor} in storage. */ diff --git a/community/kernel/src/test/java/org/neo4j/storageengine/api/StubStorageCursors.java b/community/kernel/src/test/java/org/neo4j/storageengine/api/StubStorageCursors.java index dd2a8690ab81..7bc33aaf8b92 100644 --- a/community/kernel/src/test/java/org/neo4j/storageengine/api/StubStorageCursors.java +++ b/community/kernel/src/test/java/org/neo4j/storageengine/api/StubStorageCursors.java @@ -187,6 +187,12 @@ public Iterator indexesGetForLabel( int labelId ) throw new UnsupportedOperationException( "Not implemented yet" ); } + @Override + public CapableIndexDescriptor indexGetForName( String name ) + { + throw new UnsupportedOperationException( "Not implemented yet" ); + } + @Override public Iterator indexesGetAll() { diff --git a/community/lucene-index/src/main/java/org/neo4j/kernel/api/impl/index/AbstractLuceneIndex.java b/community/lucene-index/src/main/java/org/neo4j/kernel/api/impl/index/AbstractLuceneIndex.java index b5dc7c04322f..d55f42e912f1 100644 --- a/community/lucene-index/src/main/java/org/neo4j/kernel/api/impl/index/AbstractLuceneIndex.java +++ b/community/lucene-index/src/main/java/org/neo4j/kernel/api/impl/index/AbstractLuceneIndex.java @@ -43,7 +43,12 @@ import org.neo4j.kernel.api.impl.index.partition.IndexPartitionFactory; import org.neo4j.kernel.api.impl.index.partition.PartitionSearcher; import org.neo4j.kernel.api.impl.index.storage.PartitionedIndexStorage; +import org.neo4j.kernel.api.impl.schema.writer.LuceneIndexWriter; +import org.neo4j.kernel.api.impl.schema.writer.PartitionedIndexWriter; +import org.neo4j.kernel.api.schema.index.IndexDescriptor; +import org.neo4j.storageengine.api.schema.IndexReader; +import static java.util.Collections.singletonMap; import static java.util.stream.Collectors.toList; /** @@ -55,9 +60,13 @@ * @see WritableAbstractDatabaseIndex * @see ReadOnlyAbstractDatabaseIndex */ -public abstract class AbstractLuceneIndex +public abstract class AbstractLuceneIndex { + private static final String KEY_STATUS = "status"; + private static final String ONLINE = "online"; + private static final Map ONLINE_COMMIT_USER_DATA = singletonMap( KEY_STATUS, ONLINE ); protected final PartitionedIndexStorage indexStorage; + protected final IndexDescriptor descriptor; private final IndexPartitionFactory partitionFactory; // Note that we rely on the thread-safe internal snapshot feature of the CopyOnWriteArrayList @@ -66,10 +75,11 @@ public abstract class AbstractLuceneIndex private volatile boolean open; - public AbstractLuceneIndex( PartitionedIndexStorage indexStorage, IndexPartitionFactory partitionFactory ) + public AbstractLuceneIndex( PartitionedIndexStorage indexStorage, IndexPartitionFactory partitionFactory, IndexDescriptor descriptor ) { this.indexStorage = indexStorage; this.partitionFactory = partitionFactory; + this.descriptor = descriptor; } /** @@ -179,6 +189,24 @@ public boolean isValid() return true; } + public LuceneIndexWriter getIndexWriter( WritableAbstractDatabaseIndex writableAbstractDatabaseIndex ) + { + ensureOpen(); + return new PartitionedIndexWriter( writableAbstractDatabaseIndex ); + } + + public READER getIndexReader() throws IOException + { + ensureOpen(); + List partitions = getPartitions(); + return hasSinglePartition( partitions ) ? createSimpleReader( partitions ) : createPartitionedReader( partitions ); + } + + public IndexDescriptor getDescriptor() + { + return descriptor; + } + /** * Close index and deletes all it's partitions. */ @@ -399,4 +427,51 @@ private File createNewPartitionFolder() throws IOException indexStorage.prepareFolder( partitionFolder ); return partitionFolder; } + + /** + * Check if this index is marked as online. + * + * @return true if index is online, false otherwise + * @throws IOException + */ + public boolean isOnline() throws IOException + { + ensureOpen(); + AbstractIndexPartition partition = getFirstPartition( getPartitions() ); + Directory directory = partition.getDirectory(); + try ( DirectoryReader reader = DirectoryReader.open( directory ) ) + { + Map userData = reader.getIndexCommit().getUserData(); + return ONLINE.equals( userData.get( KEY_STATUS ) ); + } + } + + /** + * Marks index as online by including "status" -> "online" map into commit metadata of the first partition. + * + * @throws IOException + */ + public void markAsOnline() throws IOException + { + ensureOpen(); + AbstractIndexPartition partition = getFirstPartition( getPartitions() ); + IndexWriter indexWriter = partition.getIndexWriter(); + indexWriter.setCommitData( ONLINE_COMMIT_USER_DATA ); + flush( false ); + } + + /** + * Writes the given failure message to the failure storage. + * + * @param failure the failure message. + * @throws IOException + */ + public void markAsFailed( String failure ) throws IOException + { + indexStorage.storeIndexFailure( failure ); + } + + protected abstract READER createSimpleReader( List partitions ) throws IOException; + + protected abstract READER createPartitionedReader( List partitions ) throws IOException; } diff --git a/community/lucene-index/src/main/java/org/neo4j/kernel/api/impl/index/AbstractLuceneIndexAccessor.java b/community/lucene-index/src/main/java/org/neo4j/kernel/api/impl/index/AbstractLuceneIndexAccessor.java new file mode 100644 index 000000000000..7ca92da1f673 --- /dev/null +++ b/community/lucene-index/src/main/java/org/neo4j/kernel/api/impl/index/AbstractLuceneIndexAccessor.java @@ -0,0 +1,192 @@ +/* + * Copyright (c) 2002-2018 "Neo4j," + * Neo4j Sweden AB [http://neo4j.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 . + */ +package org.neo4j.kernel.api.impl.index; + +import org.apache.lucene.document.Document; + +import java.io.File; +import java.io.IOException; +import java.util.function.ToLongFunction; + +import org.neo4j.graphdb.ResourceIterator; +import org.neo4j.helpers.collection.BoundedIterable; +import org.neo4j.io.pagecache.IOLimiter; +import org.neo4j.kernel.api.exceptions.index.IndexEntryConflictException; +import org.neo4j.kernel.api.impl.schema.LuceneIndexReaderAcquisitionException; +import org.neo4j.kernel.api.impl.schema.reader.LuceneAllEntriesIndexAccessorReader; +import org.neo4j.kernel.api.impl.schema.writer.LuceneIndexWriter; +import org.neo4j.kernel.api.index.IndexAccessor; +import org.neo4j.kernel.api.index.IndexEntryUpdate; +import org.neo4j.kernel.api.index.IndexUpdater; +import org.neo4j.kernel.api.index.NodePropertyAccessor; +import org.neo4j.kernel.api.schema.index.IndexDescriptor; +import org.neo4j.kernel.impl.api.index.IndexUpdateMode; +import org.neo4j.storageengine.api.schema.IndexReader; +import org.neo4j.values.storable.Value; + +public abstract class AbstractLuceneIndexAccessor> + implements IndexAccessor +{ + protected final LuceneIndexWriter writer; + protected final INDEX luceneIndex; + protected final IndexDescriptor descriptor; + + protected AbstractLuceneIndexAccessor( INDEX luceneIndex, IndexDescriptor descriptor ) + { + this.writer = luceneIndex.isReadOnly() ? null : luceneIndex.getIndexWriter(); + this.luceneIndex = luceneIndex; + this.descriptor = descriptor; + } + + @Override + public IndexUpdater newUpdater( IndexUpdateMode mode ) + { + if ( luceneIndex.isReadOnly() ) + { + throw new UnsupportedOperationException( "Can't create updater for read only index." ); + } + return getIndexUpdater( mode ); + } + + protected abstract IndexUpdater getIndexUpdater( IndexUpdateMode mode ); + + @Override + public void drop() throws IOException + { + luceneIndex.drop(); + } + + @Override + public void force( IOLimiter ioLimiter ) throws IOException + { + // We never change status of read-only indexes. + if ( !luceneIndex.isReadOnly() ) + { + luceneIndex.markAsOnline(); + } + luceneIndex.maybeRefreshBlocking(); + } + + @Override + public void refresh() throws IOException + { + luceneIndex.maybeRefreshBlocking(); + } + + @Override + public void close() throws IOException + { + luceneIndex.close(); + } + + @Override + public READER newReader() + { + try + { + return luceneIndex.getIndexReader(); + } + catch ( IOException e ) + { + throw new LuceneIndexReaderAcquisitionException( "Can't acquire index reader", e ); + } + } + + public BoundedIterable newAllEntriesReader( ToLongFunction entityIdReader ) + { + return new LuceneAllEntriesIndexAccessorReader( luceneIndex.allDocumentsReader(), entityIdReader ); + } + + @Override + public ResourceIterator snapshotFiles() throws IOException + { + return luceneIndex.snapshot(); + } + + @Override + public abstract void verifyDeferredConstraints( NodePropertyAccessor propertyAccessor ) throws IndexEntryConflictException, IOException; + + @Override + public boolean isDirty() + { + return !luceneIndex.isValid(); + } + + protected abstract class AbstractLuceneIndexUpdater implements IndexUpdater + { + private final boolean idempotent; + private final boolean refresh; + + private boolean hasChanges; + + protected AbstractLuceneIndexUpdater( boolean idempotent, boolean refresh ) + { + this.idempotent = idempotent; + this.refresh = refresh; + } + + @Override + public void process( IndexEntryUpdate update ) throws IOException + { + // we do not support adding partial entries + assert update.indexKey().schema().equals( descriptor.schema() ); + + switch ( update.updateMode() ) + { + case ADDED: + if ( idempotent ) + { + addIdempotent( update.getEntityId(), update.values() ); + } + else + { + add( update.getEntityId(), update.values() ); + } + break; + case CHANGED: + change( update.getEntityId(), update.values() ); + break; + case REMOVED: + remove( update.getEntityId() ); + break; + default: + throw new UnsupportedOperationException(); + } + hasChanges = true; + } + + @Override + public void close() throws IOException + { + if ( hasChanges && refresh ) + { + luceneIndex.maybeRefreshBlocking(); + } + } + + protected abstract void addIdempotent( long nodeId, Value[] values ) throws IOException; + + protected abstract void add( long nodeId, Value[] values ) throws IOException; + + protected abstract void change( long nodeId, Value[] values ) throws IOException; + + protected abstract void remove( long nodeId ) throws IOException; + } +} diff --git a/community/lucene-index/src/main/java/org/neo4j/kernel/api/impl/index/AbstractLuceneIndexProvider.java b/community/lucene-index/src/main/java/org/neo4j/kernel/api/impl/index/AbstractLuceneIndexProvider.java new file mode 100644 index 000000000000..b26ba2c9ffbe --- /dev/null +++ b/community/lucene-index/src/main/java/org/neo4j/kernel/api/impl/index/AbstractLuceneIndexProvider.java @@ -0,0 +1,112 @@ +/* + * Copyright (c) 2002-2018 "Neo4j," + * Neo4j Sweden AB [http://neo4j.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 . + */ +package org.neo4j.kernel.api.impl.index; + +import java.io.IOException; + +import org.neo4j.internal.kernel.api.IndexCapability; +import org.neo4j.io.fs.FileSystemAbstraction; +import org.neo4j.io.pagecache.PageCache; +import org.neo4j.kernel.api.impl.index.storage.DirectoryFactory; +import org.neo4j.kernel.api.impl.index.storage.IndexStorageFactory; +import org.neo4j.kernel.api.impl.index.storage.PartitionedIndexStorage; +import org.neo4j.kernel.api.impl.schema.LuceneSchemaIndexBuilder; +import org.neo4j.kernel.api.impl.schema.SchemaIndex; +import org.neo4j.kernel.api.index.IndexDirectoryStructure; +import org.neo4j.kernel.api.index.IndexProvider; +import org.neo4j.kernel.api.schema.index.StoreIndexDescriptor; +import org.neo4j.kernel.configuration.Config; +import org.neo4j.kernel.impl.factory.OperationalMode; +import org.neo4j.kernel.impl.storemigration.StoreMigrationParticipant; +import org.neo4j.kernel.impl.storemigration.participant.SchemaIndexMigrator; +import org.neo4j.util.VisibleForTesting; + +public abstract class AbstractLuceneIndexProvider extends IndexProvider +{ + private final IndexStorageFactory indexStorageFactory; + protected final Config config; + protected final OperationalMode operationalMode; + protected final FileSystemAbstraction fileSystem; + + protected AbstractLuceneIndexProvider( Descriptor descriptor, int priority, IndexDirectoryStructure.Factory directoryStructureFactory, Config config, + OperationalMode operationalMode, FileSystemAbstraction fileSystem, DirectoryFactory directoryFactory ) + { + super( descriptor, priority, directoryStructureFactory ); + this.config = config; + this.operationalMode = operationalMode; + this.fileSystem = fileSystem; + this.indexStorageFactory = buildIndexStorageFactory( fileSystem, directoryFactory ); + } + + @Override + public void shutdown() + { // Nothing to shut down + } + + @Override + public StoreMigrationParticipant storeMigrationParticipant( final FileSystemAbstraction fs, PageCache pageCache ) + { + return new SchemaIndexMigrator( fs, this ); + } + + @Override + public String getPopulationFailure( StoreIndexDescriptor descriptor ) throws IllegalStateException + { + String failure = getIndexStorage( descriptor.getId() ).getStoredIndexFailure(); + if ( failure == null ) + { + throw new IllegalStateException( "Index " + descriptor.getId() + " isn't failed" ); + } + return failure; + } + + @Override + public IndexCapability getCapability() + { + return IndexCapability.NO_CAPABILITY; + } + + protected PartitionedIndexStorage getIndexStorage( long indexId ) + { + return indexStorageFactory.indexStorageOf( indexId ); + } + + protected boolean indexIsOnline( PartitionedIndexStorage indexStorage, StoreIndexDescriptor descriptor ) throws IOException + { + try ( SchemaIndex index = LuceneSchemaIndexBuilder.create( descriptor, config ).withIndexStorage( indexStorage ).build() ) + { + if ( index.exists() ) + { + index.open(); + return index.isOnline(); + } + return false; + } + } + + /** + * Visible only for testing. + */ + @VisibleForTesting + protected IndexStorageFactory buildIndexStorageFactory( FileSystemAbstraction fileSystem, DirectoryFactory directoryFactory ) + { + return new IndexStorageFactory( directoryFactory, fileSystem, directoryStructure() ); + } +} diff --git a/community/lucene-index/src/main/java/org/neo4j/kernel/api/impl/index/DatabaseIndex.java b/community/lucene-index/src/main/java/org/neo4j/kernel/api/impl/index/DatabaseIndex.java index bc20e201348e..f835fd10e996 100644 --- a/community/lucene-index/src/main/java/org/neo4j/kernel/api/impl/index/DatabaseIndex.java +++ b/community/lucene-index/src/main/java/org/neo4j/kernel/api/impl/index/DatabaseIndex.java @@ -27,12 +27,15 @@ import org.neo4j.graphdb.ResourceIterator; import org.neo4j.kernel.api.impl.index.backup.WritableIndexSnapshotFileIterator; import org.neo4j.kernel.api.impl.index.partition.AbstractIndexPartition; +import org.neo4j.kernel.api.impl.schema.writer.LuceneIndexWriter; +import org.neo4j.kernel.api.schema.index.IndexDescriptor; +import org.neo4j.storageengine.api.schema.IndexReader; /** * Lucene index that may consist of one or multiple separate lucene indexes that are represented as independent * {@link AbstractIndexPartition partitions}. */ -public interface DatabaseIndex extends Closeable +public interface DatabaseIndex extends Closeable { /** * Creates new index. @@ -121,4 +124,33 @@ public interface DatabaseIndex extends Closeable * @return list of index partition */ List getPartitions(); + + LuceneIndexWriter getIndexWriter(); + + READER getIndexReader() throws IOException; + + IndexDescriptor getDescriptor(); + + /** + * Check if this index is marked as online. + * + * @return true if index is online, false otherwise + * @throws IOException + */ + boolean isOnline() throws IOException; + + /** + * Marks index as online by including "status" -> "online" map into commit metadata of the first partition. + * + * @throws IOException + */ + void markAsOnline() throws IOException; + + /** + * Writes the given failure message to the failure storage. + * + * @param failure the failure message. + * @throws IOException + */ + void markAsFailed( String failure ) throws IOException; } diff --git a/community/lucene-index/src/main/java/org/neo4j/kernel/api/impl/index/ReadOnlyAbstractDatabaseIndex.java b/community/lucene-index/src/main/java/org/neo4j/kernel/api/impl/index/ReadOnlyAbstractDatabaseIndex.java index 5a2bdadf2d54..21678962bce3 100644 --- a/community/lucene-index/src/main/java/org/neo4j/kernel/api/impl/index/ReadOnlyAbstractDatabaseIndex.java +++ b/community/lucene-index/src/main/java/org/neo4j/kernel/api/impl/index/ReadOnlyAbstractDatabaseIndex.java @@ -25,17 +25,20 @@ import org.neo4j.graphdb.ResourceIterator; import org.neo4j.kernel.api.impl.index.partition.AbstractIndexPartition; +import org.neo4j.kernel.api.impl.schema.writer.LuceneIndexWriter; +import org.neo4j.kernel.api.schema.index.IndexDescriptor; +import org.neo4j.storageengine.api.schema.IndexReader; /** * Read only lucene index representation that wraps provided index implementation and * allow read only operations only on top of it. - * @param - particular index implementation + * @param - particular index implementation */ -public abstract class ReadOnlyAbstractDatabaseIndex implements DatabaseIndex +public abstract class ReadOnlyAbstractDatabaseIndex, READER extends IndexReader> implements DatabaseIndex { - protected T luceneIndex; + protected INDEX luceneIndex; - public ReadOnlyAbstractDatabaseIndex( T luceneIndex ) + public ReadOnlyAbstractDatabaseIndex( INDEX luceneIndex ) { this.luceneIndex = luceneIndex; } @@ -157,4 +160,48 @@ public List getPartitions() return luceneIndex.getPartitions(); } + @Override + public LuceneIndexWriter getIndexWriter() + { + throw new UnsupportedOperationException( "Can't get index writer for read only lucene index." ); + } + + @Override + public READER getIndexReader() throws IOException + { + return luceneIndex.getIndexReader(); + } + + @Override + public IndexDescriptor getDescriptor() + { + return luceneIndex.getDescriptor(); + } + + /** + * {@inheritDoc} + */ + @Override + public boolean isOnline() throws IOException + { + return luceneIndex.isOnline(); + } + + /** + * Unsupported operation in read only index. + */ + @Override + public void markAsOnline() + { + throw new UnsupportedOperationException( "Can't mark read only index." ); + } + + /** + * {@inheritDoc} + */ + @Override + public void markAsFailed( String failure ) throws IOException + { + luceneIndex.markAsFailed( failure ); + } } diff --git a/community/lucene-index/src/main/java/org/neo4j/kernel/api/impl/index/WritableAbstractDatabaseIndex.java b/community/lucene-index/src/main/java/org/neo4j/kernel/api/impl/index/WritableAbstractDatabaseIndex.java index e09677ba9c94..7e6bc63e5d1f 100644 --- a/community/lucene-index/src/main/java/org/neo4j/kernel/api/impl/index/WritableAbstractDatabaseIndex.java +++ b/community/lucene-index/src/main/java/org/neo4j/kernel/api/impl/index/WritableAbstractDatabaseIndex.java @@ -26,20 +26,23 @@ import org.neo4j.graphdb.ResourceIterator; import org.neo4j.kernel.api.impl.index.partition.AbstractIndexPartition; +import org.neo4j.kernel.api.impl.schema.writer.LuceneIndexWriter; +import org.neo4j.kernel.api.schema.index.IndexDescriptor; +import org.neo4j.storageengine.api.schema.IndexReader; /** * Writable lucene index representation that wraps provided index implementation and * allow read only operations only on top of it. * @param - particular index implementation */ -public class WritableAbstractDatabaseIndex implements DatabaseIndex +public class WritableAbstractDatabaseIndex, READER extends IndexReader> implements DatabaseIndex { // lock used to guard commits and close of lucene indexes from separate threads protected final ReentrantLock commitCloseLock = new ReentrantLock(); - protected T luceneIndex; + protected INDEX luceneIndex; - public WritableAbstractDatabaseIndex( T luceneIndex ) + public WritableAbstractDatabaseIndex( INDEX luceneIndex ) { this.luceneIndex = luceneIndex; } @@ -204,6 +207,59 @@ public List getPartitions() return luceneIndex.getPartitions(); } + /** + * {@inheritDoc} + */ + @Override + public void markAsOnline() throws IOException + { + commitCloseLock.lock(); + try + { + luceneIndex.markAsOnline(); + } + finally + { + commitCloseLock.unlock(); + } + } + + @Override + public LuceneIndexWriter getIndexWriter() + { + return luceneIndex.getIndexWriter( this ); + } + + @Override + public READER getIndexReader() throws IOException + { + return luceneIndex.getIndexReader(); + } + + @Override + public IndexDescriptor getDescriptor() + { + return luceneIndex.getDescriptor(); + } + + /** + * {@inheritDoc} + */ + @Override + public boolean isOnline() throws IOException + { + return luceneIndex.isOnline(); + } + + /** + * {@inheritDoc} + */ + @Override + public void markAsFailed( String failure ) throws IOException + { + luceneIndex.markAsFailed( failure ); + } + public boolean hasSinglePartition( List partitions ) { return luceneIndex.hasSinglePartition( partitions ); diff --git a/community/lucene-index/src/main/java/org/neo4j/kernel/api/impl/schema/LuceneIndexAccessor.java b/community/lucene-index/src/main/java/org/neo4j/kernel/api/impl/schema/LuceneIndexAccessor.java index c1e255d9755f..803fd13bdd57 100644 --- a/community/lucene-index/src/main/java/org/neo4j/kernel/api/impl/schema/LuceneIndexAccessor.java +++ b/community/lucene-index/src/main/java/org/neo4j/kernel/api/impl/schema/LuceneIndexAccessor.java @@ -19,17 +19,11 @@ */ package org.neo4j.kernel.api.impl.schema; -import java.io.File; import java.io.IOException; -import org.neo4j.graphdb.ResourceIterator; import org.neo4j.helpers.collection.BoundedIterable; -import org.neo4j.io.pagecache.IOLimiter; import org.neo4j.kernel.api.exceptions.index.IndexEntryConflictException; -import org.neo4j.kernel.api.impl.schema.reader.LuceneAllEntriesIndexAccessorReader; -import org.neo4j.kernel.api.impl.schema.writer.LuceneIndexWriter; -import org.neo4j.kernel.api.index.IndexAccessor; -import org.neo4j.kernel.api.index.IndexEntryUpdate; +import org.neo4j.kernel.api.impl.index.AbstractLuceneIndexAccessor; import org.neo4j.kernel.api.index.IndexUpdater; import org.neo4j.kernel.api.index.NodePropertyAccessor; import org.neo4j.kernel.api.schema.index.IndexDescriptor; @@ -38,81 +32,24 @@ import org.neo4j.storageengine.api.schema.IndexReader; import org.neo4j.values.storable.Value; -public class LuceneIndexAccessor implements IndexAccessor +public class LuceneIndexAccessor extends AbstractLuceneIndexAccessor { - private final LuceneIndexWriter writer; - private final SchemaIndex luceneIndex; - private final IndexDescriptor descriptor; public LuceneIndexAccessor( SchemaIndex luceneIndex, IndexDescriptor descriptor ) { - this.luceneIndex = luceneIndex; - this.descriptor = descriptor; - this.writer = luceneIndex.isReadOnly() ? null : luceneIndex.getIndexWriter(); + super( luceneIndex, descriptor ); } @Override - public IndexUpdater newUpdater( IndexUpdateMode mode ) + protected IndexUpdater getIndexUpdater( IndexUpdateMode mode ) { - if ( luceneIndex.isReadOnly() ) - { - throw new UnsupportedOperationException( "Can't create updater for read only index." ); - } - return new LuceneIndexUpdater( mode.requiresIdempotency(), mode.requiresRefresh() ); - } - - @Override - public void drop() throws IOException - { - luceneIndex.drop(); - } - - @Override - public void force( IOLimiter ioLimiter ) throws IOException - { - // We never change status of read-only indexes. - if ( !luceneIndex.isReadOnly() ) - { - luceneIndex.markAsOnline(); - } - luceneIndex.maybeRefreshBlocking(); - } - - @Override - public void refresh() throws IOException - { - luceneIndex.maybeRefreshBlocking(); - } - - @Override - public void close() throws IOException - { - luceneIndex.close(); - } - - @Override - public IndexReader newReader() - { - try - { - return luceneIndex.getIndexReader(); - } - catch ( IOException e ) - { - throw new LuceneIndexReaderAcquisitionException( "Can't acquire index reader", e ); - } + return new LuceneSchemaIndexUpdater( mode.requiresIdempotency(), mode.requiresRefresh() ); } @Override public BoundedIterable newAllEntriesReader() { - return new LuceneAllEntriesIndexAccessorReader( luceneIndex.allDocumentsReader() ); - } - - @Override - public ResourceIterator snapshotFiles() throws IOException - { - return luceneIndex.snapshot(); + return super.newAllEntriesReader( LuceneDocumentStructure::getNodeId ); } @Override @@ -122,12 +59,6 @@ public void verifyDeferredConstraints( NodePropertyAccessor nodePropertyAccessor luceneIndex.verifyUniqueness( nodePropertyAccessor, descriptor.schema().getPropertyIds() ); } - @Override - public boolean isDirty() - { - return !luceneIndex.isValid(); - } - @Override public void validateBeforeCommit( Value[] tuple ) { @@ -138,75 +69,35 @@ public void validateBeforeCommit( Value[] tuple ) } } - private class LuceneIndexUpdater implements IndexUpdater + private class LuceneSchemaIndexUpdater extends AbstractLuceneIndexUpdater { - private final boolean idempotent; - private final boolean refresh; - private boolean hasChanges; - - private LuceneIndexUpdater( boolean idempotent, boolean refresh ) - { - this.idempotent = idempotent; - this.refresh = refresh; - } - - @Override - public void process( IndexEntryUpdate update ) throws IOException + protected LuceneSchemaIndexUpdater( boolean idempotent, boolean refresh ) { - // we do not support adding partial entries - assert update.indexKey().schema().equals( descriptor.schema() ); - - switch ( update.updateMode() ) - { - case ADDED: - if ( idempotent ) - { - addIdempotent( update.getEntityId(), update.values() ); - } - else - { - add( update.getEntityId(), update.values() ); - } - break; - case CHANGED: - change( update.getEntityId(), update.values() ); - break; - case REMOVED: - remove( update.getEntityId() ); - break; - default: - throw new UnsupportedOperationException(); - } - hasChanges = true; + super( idempotent, refresh ); } @Override - public void close() throws IOException - { - if ( hasChanges && refresh ) - { - luceneIndex.maybeRefreshBlocking(); - } - } - - private void addIdempotent( long nodeId, Value[] values ) throws IOException + protected void addIdempotent( long nodeId, Value[] values ) throws IOException { writer.updateDocument( LuceneDocumentStructure.newTermForChangeOrRemove( nodeId ), LuceneDocumentStructure.documentRepresentingProperties( nodeId, values ) ); } - private void add( long nodeId, Value[] values ) throws IOException + @Override + protected void add( long nodeId, Value[] values ) throws IOException { writer.addDocument( LuceneDocumentStructure.documentRepresentingProperties( nodeId, values ) ); } - private void change( long nodeId, Value[] values ) throws IOException + @Override + protected void change( long nodeId, Value[] values ) throws IOException { writer.updateDocument( LuceneDocumentStructure.newTermForChangeOrRemove( nodeId ), LuceneDocumentStructure.documentRepresentingProperties( nodeId, values ) ); } + @Override protected void remove( long nodeId ) throws IOException { writer.deleteDocuments( LuceneDocumentStructure.newTermForChangeOrRemove( nodeId ) ); diff --git a/community/lucene-index/src/main/java/org/neo4j/kernel/api/impl/schema/LuceneIndexProviderFactory.java b/community/lucene-index/src/main/java/org/neo4j/kernel/api/impl/schema/LuceneIndexProviderFactory.java index 8abfa27a14e4..6cd44f7322af 100644 --- a/community/lucene-index/src/main/java/org/neo4j/kernel/api/impl/schema/LuceneIndexProviderFactory.java +++ b/community/lucene-index/src/main/java/org/neo4j/kernel/api/impl/schema/LuceneIndexProviderFactory.java @@ -120,5 +120,4 @@ private static IndexDirectoryStructure.Factory subProviderDirectoryStructure( Fi { return NativeLuceneFusionIndexProviderFactory.subProviderDirectoryStructure( storeDir, PROVIDER_DESCRIPTOR ); } - } diff --git a/community/lucene-index/src/main/java/org/neo4j/kernel/api/impl/schema/LuceneSchemaIndex.java b/community/lucene-index/src/main/java/org/neo4j/kernel/api/impl/schema/LuceneSchemaIndex.java index 3993b3086c94..eba0c27fe355 100644 --- a/community/lucene-index/src/main/java/org/neo4j/kernel/api/impl/schema/LuceneSchemaIndex.java +++ b/community/lucene-index/src/main/java/org/neo4j/kernel/api/impl/schema/LuceneSchemaIndex.java @@ -19,13 +19,8 @@ */ package org.neo4j.kernel.api.impl.schema; -import org.apache.lucene.index.DirectoryReader; -import org.apache.lucene.index.IndexWriter; -import org.apache.lucene.store.Directory; - import java.io.IOException; import java.util.List; -import java.util.Map; import java.util.concurrent.TimeUnit; import org.neo4j.helpers.TaskCoordinator; @@ -40,26 +35,18 @@ import org.neo4j.kernel.api.impl.schema.verification.PartitionedUniquenessVerifier; import org.neo4j.kernel.api.impl.schema.verification.SimpleUniquenessVerifier; import org.neo4j.kernel.api.impl.schema.verification.UniquenessVerifier; -import org.neo4j.kernel.api.impl.schema.writer.LuceneIndexWriter; -import org.neo4j.kernel.api.impl.schema.writer.PartitionedIndexWriter; import org.neo4j.kernel.api.index.NodePropertyAccessor; import org.neo4j.kernel.api.schema.index.IndexDescriptor; import org.neo4j.kernel.impl.api.index.sampling.IndexSamplingConfig; import org.neo4j.storageengine.api.schema.IndexReader; import org.neo4j.values.storable.Value; -import static java.util.Collections.singletonMap; - /** * Implementation of Lucene schema index that support multiple partitions. */ -class LuceneSchemaIndex extends AbstractLuceneIndex +class LuceneSchemaIndex extends AbstractLuceneIndex { - private static final String KEY_STATUS = "status"; - private static final String ONLINE = "online"; - private static final Map ONLINE_COMMIT_USER_DATA = singletonMap( KEY_STATUS, ONLINE ); - private final IndexDescriptor descriptor; private final IndexSamplingConfig samplingConfig; private final TaskCoordinator taskCoordinator = new TaskCoordinator( 10, TimeUnit.MILLISECONDS ); @@ -67,30 +54,10 @@ class LuceneSchemaIndex extends AbstractLuceneIndex LuceneSchemaIndex( PartitionedIndexStorage indexStorage, IndexDescriptor descriptor, IndexSamplingConfig samplingConfig, IndexPartitionFactory partitionFactory ) { - super( indexStorage, partitionFactory ); - this.descriptor = descriptor; + super( indexStorage, partitionFactory, descriptor ); this.samplingConfig = samplingConfig; } - public LuceneIndexWriter getIndexWriter( WritableDatabaseSchemaIndex writableLuceneSchemaIndex ) - { - ensureOpen(); - return new PartitionedIndexWriter( writableLuceneSchemaIndex ); - } - - public IndexReader getIndexReader() throws IOException - { - ensureOpen(); - List partitions = getPartitions(); - return hasSinglePartition( partitions ) ? createSimpleReader( partitions ) - : createPartitionedReader( partitions ); - } - - public IndexDescriptor getDescriptor() - { - return descriptor; - } - /** * Verifies uniqueness of property values present in this index. * @@ -144,49 +111,6 @@ public void drop() super.drop(); } - /** - * Check if this index is marked as online. - * - * @return true if index is online, false otherwise - * @throws IOException - */ - public boolean isOnline() throws IOException - { - ensureOpen(); - AbstractIndexPartition partition = getFirstPartition( getPartitions() ); - Directory directory = partition.getDirectory(); - try ( DirectoryReader reader = DirectoryReader.open( directory ) ) - { - Map userData = reader.getIndexCommit().getUserData(); - return ONLINE.equals( userData.get( KEY_STATUS ) ); - } - } - - /** - * Marks index as online by including "status" -> "online" map into commit metadata of the first partition. - * - * @throws IOException - */ - public void markAsOnline() throws IOException - { - ensureOpen(); - AbstractIndexPartition partition = getFirstPartition( getPartitions() ); - IndexWriter indexWriter = partition.getIndexWriter(); - indexWriter.setCommitData( ONLINE_COMMIT_USER_DATA ); - flush( false ); - } - - /** - * Writes the given failure message to the failure storage. - * - * @param failure the failure message. - * @throws IOException - */ - public void markAsFailed( String failure ) throws IOException - { - indexStorage.storeIndexFailure( failure ); - } - private UniquenessVerifier createUniquenessVerifier() throws IOException { ensureOpen(); @@ -196,12 +120,6 @@ private UniquenessVerifier createUniquenessVerifier() throws IOException : createPartitionedUniquenessVerifier( partitions ); } - private SimpleIndexReader createSimpleReader( List partitions ) throws IOException - { - AbstractIndexPartition singlePartition = getFirstPartition( partitions ); - return new SimpleIndexReader( singlePartition.acquireSearcher(), descriptor, samplingConfig, taskCoordinator ); - } - private UniquenessVerifier createSimpleUniquenessVerifier( List partitions ) throws IOException { AbstractIndexPartition singlePartition = getFirstPartition( partitions ); @@ -209,16 +127,24 @@ private UniquenessVerifier createSimpleUniquenessVerifier( List partitions ) throws IOException + private UniquenessVerifier createPartitionedUniquenessVerifier( List partitions ) throws IOException { List searchers = acquireSearchers( partitions ); - return new PartitionedIndexReader( searchers, descriptor, samplingConfig, taskCoordinator ); + return new PartitionedUniquenessVerifier( searchers ); } - private UniquenessVerifier createPartitionedUniquenessVerifier( List partitions ) throws IOException + @Override + protected SimpleIndexReader createSimpleReader( List partitions ) throws IOException + { + AbstractIndexPartition singlePartition = getFirstPartition( partitions ); + return new SimpleIndexReader( singlePartition.acquireSearcher(), descriptor, samplingConfig, taskCoordinator ); + } + + @Override + protected PartitionedIndexReader createPartitionedReader( List partitions ) throws IOException { List searchers = acquireSearchers( partitions ); - return new PartitionedUniquenessVerifier( searchers ); + return new PartitionedIndexReader( searchers, descriptor, samplingConfig, taskCoordinator ); } } diff --git a/community/lucene-index/src/main/java/org/neo4j/kernel/api/impl/schema/LuceneSchemaIndexBuilder.java b/community/lucene-index/src/main/java/org/neo4j/kernel/api/impl/schema/LuceneSchemaIndexBuilder.java index 9f214a986f65..c510ef703bd6 100644 --- a/community/lucene-index/src/main/java/org/neo4j/kernel/api/impl/schema/LuceneSchemaIndexBuilder.java +++ b/community/lucene-index/src/main/java/org/neo4j/kernel/api/impl/schema/LuceneSchemaIndexBuilder.java @@ -107,5 +107,4 @@ public SchemaIndex build() new WritableIndexPartitionFactory( writerConfigFactory ) ); } } - } diff --git a/community/lucene-index/src/main/java/org/neo4j/kernel/api/impl/schema/ReadOnlyDatabaseSchemaIndex.java b/community/lucene-index/src/main/java/org/neo4j/kernel/api/impl/schema/ReadOnlyDatabaseSchemaIndex.java index 4b1431da8e70..448a5f73ee93 100644 --- a/community/lucene-index/src/main/java/org/neo4j/kernel/api/impl/schema/ReadOnlyDatabaseSchemaIndex.java +++ b/community/lucene-index/src/main/java/org/neo4j/kernel/api/impl/schema/ReadOnlyDatabaseSchemaIndex.java @@ -26,7 +26,6 @@ import org.neo4j.kernel.api.impl.index.ReadOnlyAbstractDatabaseIndex; import org.neo4j.kernel.api.impl.index.partition.ReadOnlyIndexPartitionFactory; import org.neo4j.kernel.api.impl.index.storage.PartitionedIndexStorage; -import org.neo4j.kernel.api.impl.schema.writer.LuceneIndexWriter; import org.neo4j.kernel.api.index.NodePropertyAccessor; import org.neo4j.kernel.api.schema.index.IndexDescriptor; import org.neo4j.kernel.impl.api.index.sampling.IndexSamplingConfig; @@ -36,7 +35,7 @@ /** * Read only schema index */ -public class ReadOnlyDatabaseSchemaIndex extends ReadOnlyAbstractDatabaseIndex implements SchemaIndex +public class ReadOnlyDatabaseSchemaIndex extends ReadOnlyAbstractDatabaseIndex implements SchemaIndex { public ReadOnlyDatabaseSchemaIndex( PartitionedIndexStorage indexStorage, IndexDescriptor descriptor, IndexSamplingConfig samplingConfig, ReadOnlyIndexPartitionFactory readOnlyIndexPartitionFactory ) @@ -44,24 +43,6 @@ public ReadOnlyDatabaseSchemaIndex( PartitionedIndexStorage indexStorage, IndexD super( new LuceneSchemaIndex( indexStorage, descriptor, samplingConfig, readOnlyIndexPartitionFactory ) ); } - @Override - public LuceneIndexWriter getIndexWriter() - { - throw new UnsupportedOperationException( "Can't get index writer for read only lucene index." ); - } - - @Override - public IndexReader getIndexReader() throws IOException - { - return luceneIndex.getIndexReader(); - } - - @Override - public IndexDescriptor getDescriptor() - { - return luceneIndex.getDescriptor(); - } - /** * {@inheritDoc} */ @@ -81,31 +62,4 @@ public void verifyUniqueness( NodePropertyAccessor accessor, int[] propertyKeyId { luceneIndex.verifyUniqueness( accessor, propertyKeyIds, updatedValueTuples ); } - - /** - * {@inheritDoc} - */ - @Override - public boolean isOnline() throws IOException - { - return luceneIndex.isOnline(); - } - - /** - * Unsupported operation in read only index. - */ - @Override - public void markAsOnline() - { - throw new UnsupportedOperationException( "Can't mark read only index." ); - } - - /** - * {@inheritDoc} - */ - @Override - public void markAsFailed( String failure ) throws IOException - { - luceneIndex.markAsFailed( failure ); - } } diff --git a/community/lucene-index/src/main/java/org/neo4j/kernel/api/impl/schema/SchemaIndex.java b/community/lucene-index/src/main/java/org/neo4j/kernel/api/impl/schema/SchemaIndex.java index b95dc31f71cd..7d617781e243 100644 --- a/community/lucene-index/src/main/java/org/neo4j/kernel/api/impl/schema/SchemaIndex.java +++ b/community/lucene-index/src/main/java/org/neo4j/kernel/api/impl/schema/SchemaIndex.java @@ -25,22 +25,15 @@ import org.neo4j.kernel.api.exceptions.index.IndexEntryConflictException; import org.neo4j.kernel.api.impl.index.DatabaseIndex; import org.neo4j.kernel.api.impl.schema.verification.UniquenessVerifier; -import org.neo4j.kernel.api.impl.schema.writer.LuceneIndexWriter; import org.neo4j.kernel.api.index.NodePropertyAccessor; -import org.neo4j.kernel.api.schema.index.IndexDescriptor; import org.neo4j.storageengine.api.schema.IndexReader; import org.neo4j.values.storable.Value; /** * Partitioned lucene schema index. */ -public interface SchemaIndex extends DatabaseIndex +public interface SchemaIndex extends DatabaseIndex { - LuceneIndexWriter getIndexWriter(); - - IndexReader getIndexReader() throws IOException; - - IndexDescriptor getDescriptor(); /** * Verifies uniqueness of property values present in this index. @@ -66,27 +59,4 @@ void verifyUniqueness( NodePropertyAccessor accessor, int[] propertyKeyIds ) */ void verifyUniqueness( NodePropertyAccessor accessor, int[] propertyKeyIds, List updatedValueTuples ) throws IOException, IndexEntryConflictException; - - /** - * Check if this index is marked as online. - * - * @return true if index is online, false otherwise - * @throws IOException - */ - boolean isOnline() throws IOException; - - /** - * Marks index as online by including "status" -> "online" map into commit metadata of the first partition. - * - * @throws IOException - */ - void markAsOnline() throws IOException; - - /** - * Writes the given failure message to the failure storage. - * - * @param failure the failure message. - * @throws IOException - */ - void markAsFailed( String failure ) throws IOException; } diff --git a/community/lucene-index/src/main/java/org/neo4j/kernel/api/impl/schema/WritableDatabaseSchemaIndex.java b/community/lucene-index/src/main/java/org/neo4j/kernel/api/impl/schema/WritableDatabaseSchemaIndex.java index 1f71e36a1127..0f4e4588bffa 100644 --- a/community/lucene-index/src/main/java/org/neo4j/kernel/api/impl/schema/WritableDatabaseSchemaIndex.java +++ b/community/lucene-index/src/main/java/org/neo4j/kernel/api/impl/schema/WritableDatabaseSchemaIndex.java @@ -30,14 +30,16 @@ import org.neo4j.kernel.api.impl.schema.writer.LuceneIndexWriter; import org.neo4j.kernel.api.index.NodePropertyAccessor; import org.neo4j.kernel.api.schema.index.IndexDescriptor; +import org.neo4j.kernel.api.schema.index.StoreIndexDescriptor; import org.neo4j.kernel.impl.api.index.sampling.IndexSamplingConfig; +import org.neo4j.storageengine.api.schema.AbstractIndexReader; import org.neo4j.storageengine.api.schema.IndexReader; import org.neo4j.values.storable.Value; /** * Writable schema index */ -public class WritableDatabaseSchemaIndex extends WritableAbstractDatabaseIndex implements SchemaIndex +public class WritableDatabaseSchemaIndex extends WritableAbstractDatabaseIndex implements SchemaIndex { public WritableDatabaseSchemaIndex( PartitionedIndexStorage storage, IndexDescriptor descriptor, @@ -46,24 +48,6 @@ public WritableDatabaseSchemaIndex( PartitionedIndexStorage storage, IndexDescri super( new LuceneSchemaIndex( storage, descriptor, samplingConfig, writableIndexPartitionFactory ) ); } - @Override - public LuceneIndexWriter getIndexWriter() - { - return luceneIndex.getIndexWriter( this ); - } - - @Override - public IndexReader getIndexReader() throws IOException - { - return luceneIndex.getIndexReader(); - } - - @Override - public IndexDescriptor getDescriptor() - { - return luceneIndex.getDescriptor(); - } - /** * {@inheritDoc} */ @@ -83,51 +67,4 @@ public void verifyUniqueness( NodePropertyAccessor accessor, int[] propertyKeyId { luceneIndex.verifyUniqueness( accessor, propertyKeyIds, updatedValueTuples ); } - - /** - * {@inheritDoc} - */ - @Override - public boolean isOnline() throws IOException - { - return luceneIndex.isOnline(); - } - - /** - * {@inheritDoc} - */ - @Override - public void markAsOnline() throws IOException - { - commitCloseLock.lock(); - try - { - luceneIndex.markAsOnline(); - } - finally - { - commitCloseLock.unlock(); - } - } - - /** - * {@inheritDoc} - */ - @Override - public void markAsFailed( String failure ) throws IOException - { - luceneIndex.markAsFailed( failure ); - } - - @Override - public boolean hasSinglePartition( List partitions ) - { - return luceneIndex.hasSinglePartition( partitions ); - } - - @Override - public AbstractIndexPartition getFirstPartition( List partitions ) - { - return luceneIndex.getFirstPartition( partitions ); - } } diff --git a/community/lucene-index/src/main/java/org/neo4j/kernel/api/impl/schema/populator/LuceneIndexPopulator.java b/community/lucene-index/src/main/java/org/neo4j/kernel/api/impl/schema/populator/LuceneIndexPopulator.java index a1f085c4a247..d7aa506255ab 100644 --- a/community/lucene-index/src/main/java/org/neo4j/kernel/api/impl/schema/populator/LuceneIndexPopulator.java +++ b/community/lucene-index/src/main/java/org/neo4j/kernel/api/impl/schema/populator/LuceneIndexPopulator.java @@ -25,21 +25,22 @@ import java.util.Collection; import org.neo4j.io.IOUtils; +import org.neo4j.kernel.api.impl.index.DatabaseIndex; import org.neo4j.kernel.api.impl.schema.LuceneDocumentStructure; -import org.neo4j.kernel.api.impl.schema.SchemaIndex; import org.neo4j.kernel.api.impl.schema.writer.LuceneIndexWriter; import org.neo4j.kernel.api.index.IndexEntryUpdate; import org.neo4j.kernel.api.index.IndexPopulator; +import org.neo4j.storageengine.api.schema.IndexReader; /** * An {@link IndexPopulator} used to create, populate and mark as online a Lucene schema index. */ -public abstract class LuceneIndexPopulator implements IndexPopulator +public abstract class LuceneIndexPopulator> implements IndexPopulator { - protected SchemaIndex luceneIndex; + protected INDEX luceneIndex; protected LuceneIndexWriter writer; - LuceneIndexPopulator( SchemaIndex luceneIndex ) + protected LuceneIndexPopulator( INDEX luceneIndex ) { this.luceneIndex = luceneIndex; } diff --git a/community/lucene-index/src/main/java/org/neo4j/kernel/api/impl/schema/populator/NonUniqueLuceneIndexPopulator.java b/community/lucene-index/src/main/java/org/neo4j/kernel/api/impl/schema/populator/NonUniqueLuceneIndexPopulator.java index 1d9b93cbddcf..d39a724e6b3a 100644 --- a/community/lucene-index/src/main/java/org/neo4j/kernel/api/impl/schema/populator/NonUniqueLuceneIndexPopulator.java +++ b/community/lucene-index/src/main/java/org/neo4j/kernel/api/impl/schema/populator/NonUniqueLuceneIndexPopulator.java @@ -33,7 +33,7 @@ * A {@link LuceneIndexPopulator} used for non-unique Lucene schema indexes. * Performs sampling using {@link DefaultNonUniqueIndexSampler}. */ -public class NonUniqueLuceneIndexPopulator extends LuceneIndexPopulator +public class NonUniqueLuceneIndexPopulator extends LuceneIndexPopulator { private final IndexSamplingConfig samplingConfig; private NonUniqueIndexSampler sampler; diff --git a/community/lucene-index/src/main/java/org/neo4j/kernel/api/impl/schema/populator/UniqueLuceneIndexPopulator.java b/community/lucene-index/src/main/java/org/neo4j/kernel/api/impl/schema/populator/UniqueLuceneIndexPopulator.java index e69cfd08e884..a5f4e405aa19 100644 --- a/community/lucene-index/src/main/java/org/neo4j/kernel/api/impl/schema/populator/UniqueLuceneIndexPopulator.java +++ b/community/lucene-index/src/main/java/org/neo4j/kernel/api/impl/schema/populator/UniqueLuceneIndexPopulator.java @@ -36,7 +36,7 @@ * Verifies uniqueness of added and changed values using * {@link SchemaIndex#verifyUniqueness(NodePropertyAccessor, int[])} method. */ -public class UniqueLuceneIndexPopulator extends LuceneIndexPopulator +public class UniqueLuceneIndexPopulator extends LuceneIndexPopulator { private final int[] propertyKeyIds; private final UniqueIndexSampler sampler; diff --git a/community/lucene-index/src/main/java/org/neo4j/kernel/api/impl/schema/reader/LuceneAllEntriesIndexAccessorReader.java b/community/lucene-index/src/main/java/org/neo4j/kernel/api/impl/schema/reader/LuceneAllEntriesIndexAccessorReader.java index 6f2388c96dee..bb07377ac301 100644 --- a/community/lucene-index/src/main/java/org/neo4j/kernel/api/impl/schema/reader/LuceneAllEntriesIndexAccessorReader.java +++ b/community/lucene-index/src/main/java/org/neo4j/kernel/api/impl/schema/reader/LuceneAllEntriesIndexAccessorReader.java @@ -22,18 +22,20 @@ import org.apache.lucene.document.Document; import java.util.Iterator; +import java.util.function.ToLongFunction; import org.neo4j.helpers.collection.BoundedIterable; - -import static org.neo4j.kernel.api.impl.schema.LuceneDocumentStructure.getNodeId; +import org.neo4j.kernel.api.impl.schema.LuceneDocumentStructure; public class LuceneAllEntriesIndexAccessorReader implements BoundedIterable { private final BoundedIterable documents; + private final ToLongFunction entityIdReader; - public LuceneAllEntriesIndexAccessorReader( BoundedIterable documents ) + public LuceneAllEntriesIndexAccessorReader( BoundedIterable documents, ToLongFunction entityIdReader ) { this.documents = documents; + this.entityIdReader = entityIdReader; } @Override @@ -57,7 +59,7 @@ public boolean hasNext() @Override public Long next() { - return getNodeId( iterator.next() ); + return entityIdReader.applyAsLong( iterator.next() ); } }; } diff --git a/community/lucene-index/src/main/java/org/neo4j/kernel/api/impl/schema/verification/CompositeDuplicateCheckingCollector.java b/community/lucene-index/src/main/java/org/neo4j/kernel/api/impl/schema/verification/CompositeDuplicateCheckingCollector.java index 11f1623a5107..2c5f1805bf20 100644 --- a/community/lucene-index/src/main/java/org/neo4j/kernel/api/impl/schema/verification/CompositeDuplicateCheckingCollector.java +++ b/community/lucene-index/src/main/java/org/neo4j/kernel/api/impl/schema/verification/CompositeDuplicateCheckingCollector.java @@ -23,8 +23,8 @@ import java.io.IOException; -import org.neo4j.internal.kernel.api.exceptions.KernelException; import org.neo4j.kernel.api.StatementConstants; +import org.neo4j.internal.kernel.api.exceptions.KernelException; import org.neo4j.kernel.api.exceptions.index.IndexEntryConflictException; import org.neo4j.kernel.api.impl.schema.LuceneDocumentStructure; import org.neo4j.kernel.api.index.NodePropertyAccessor; diff --git a/community/lucene-index/src/test/java/org/neo4j/kernel/api/impl/index/DatabaseIndexIntegrationTest.java b/community/lucene-index/src/test/java/org/neo4j/kernel/api/impl/index/DatabaseIndexIntegrationTest.java index c8b7be1cbc1e..5285e606cfc4 100644 --- a/community/lucene-index/src/test/java/org/neo4j/kernel/api/impl/index/DatabaseIndexIntegrationTest.java +++ b/community/lucene-index/src/test/java/org/neo4j/kernel/api/impl/index/DatabaseIndexIntegrationTest.java @@ -59,6 +59,7 @@ import org.neo4j.test.extension.DefaultFileSystemExtension; import org.neo4j.test.extension.Inject; import org.neo4j.test.extension.TestDirectoryExtension; +import org.neo4j.storageengine.api.schema.AbstractIndexReader; import org.neo4j.test.rule.TestDirectory; import static java.time.Duration.ofSeconds; @@ -237,7 +238,7 @@ private IndexWriter firstPartitionWriter() return partition.getIndexWriter(); } - private static class WritableTestDatabaseIndex extends WritableAbstractDatabaseIndex + private static class WritableTestDatabaseIndex extends WritableAbstractDatabaseIndex { WritableTestDatabaseIndex( PartitionedIndexStorage indexStorage ) { @@ -246,12 +247,24 @@ private static class WritableTestDatabaseIndex extends WritableAbstractDatabaseI } } - private static class TestLuceneIndex extends AbstractLuceneIndex + private static class TestLuceneIndex extends AbstractLuceneIndex { TestLuceneIndex( PartitionedIndexStorage indexStorage, IndexPartitionFactory partitionFactory ) { - super( indexStorage, partitionFactory ); + super( indexStorage, partitionFactory, null ); + } + + @Override + protected AbstractIndexReader createSimpleReader( List partitions ) throws IOException + { + return null; + } + + @Override + protected AbstractIndexReader createPartitionedReader( List partitions ) throws IOException + { + return null; } }