From ea851acb02bf6944f489f639e034d8cdc49e2b4c Mon Sep 17 00:00:00 2001 From: Przemek Hugh Kaznowski Date: Thu, 1 Mar 2018 11:02:32 +0000 Subject: [PATCH] Resumable copy This PR introduces resumable store copy in the sense that failed store file download will be retried. Previously all store files would be streamed to the client in one response and if any issue would occur we would have to redo the entire process. The new strategy is as follows: CLIENT -> Prepare store request SERVER -> Grabs checkpoint lock and streams all "atomic-files" (count store, legacy index and additional providers). It then returns a response that includes last committed transaction id, a list of store file names, a list of lucene index descriptors. CLIENT -> for each store file: send get store file request SERVER -> stream requested file CLIENT -> for each descriptor: send get lucene snapshot request SERVER -> stream lucene snapshot files. If prepare store request fails then store copy fails. Each get file/snapshot request may be retried as long as the configured termination condition allows. Each file/snapshot request include the "last committed tx id" given by the prepare store copy respons so we can try to download from any instance in the cluster. In this PR all requests are to leader. As of yet, nothing has been done to "pull tx" which is called after the store files has been copied. This will be investigated in a different PR. KERNEL: NeoStoreFileListing have been modified to provide builders for listing files. IndexService has been modified to expose descriptors and to allow getting specific snapshot for descriptor. PrepareStoreCopyRequestHandler, GetStoreFileRequestHandler and GetLuceneSnapshotRequestHandler all use NeoStoreFileListing and would be good looking into. --- .../api/schema/LabelSchemaDescriptor.java | 3 + .../org/neo4j/kernel/NeoStoreDataSource.java | 15 +- .../api/schema/index/IndexDescriptor.java | 4 +- .../impl/api/index/IndexingService.java | 26 +- .../state/NeoStoreFileIndexListing.java | 108 ++++++ .../state/NeoStoreFileListing.java | 240 ++++++++---- .../neo4j/kernel/impl/util/MultiResource.java | 49 +++ .../impl/api/index/IndexingServiceTest.java | 4 +- .../state/NeoStoreFileListingTest.java | 7 +- .../org/neo4j/index/backup/IndexBackupIT.java | 12 +- .../neo4j/backup/impl/BackupDelegator.java | 3 +- .../backup/impl/BackupDelegatorTest.java | 15 +- .../CausalClusteringBackupStrategyTest.java | 6 - .../backup/impl/OnlineBackupCommandCcIT.java | 2 +- .../catchup/CatchUpClientChannelPipeline.java | 11 +- .../catchup/CatchUpResponseAdaptor.java | 26 +- .../catchup/CatchUpResponseCallback.java | 3 + .../catchup/CatchUpResponseHandler.java | 3 + .../catchup/CatchupAddressProvider.java | 114 ++++++ .../CatchupAddressResolutionException.java | 36 ++ .../catchup/CatchupClientProtocol.java | 2 + .../catchup/CatchupServer.java | 63 ++-- .../catchup/CatchupServerProtocol.java | 8 +- .../catchup/ClientMessageTypeHandler.java | 46 ++- .../catchup/RequestDecoderDispatcher.java | 4 +- .../catchup/RequestMessageType.java | 4 +- .../catchup/ResponseMessageType.java | 4 +- .../catchup/ServerMessageTypeHandler.java | 20 +- .../catchup/SimpleRequestDecoder.java | 4 +- .../catchup/StoreListingResponseHandler.java | 46 +++ .../catchup/TrackingResponseHandler.java | 11 + .../catchup/storecopy/CloseablesListener.java | 55 +++ .../catchup/storecopy/DataSourceChecks.java | 64 ++++ .../catchup/storecopy/FileChunk.java | 2 +- .../storecopy/GetIndexFilesRequest.java | 114 ++++++ .../GetIndexSnapshotRequestHandler.java | 102 ++++++ .../storecopy/GetStoreFileRequest.java | 120 ++++++ .../storecopy/GetStoreFileRequestHandler.java | 116 ++++++ .../storecopy/GetStoreRequestHandler.java | 60 --- .../storecopy/IndexDescriptorSerializer.java | 74 ++++ .../storecopy/MaximumTotalRetries.java | 64 ++++ .../storecopy/PrepareStoreCopyFiles.java | 112 ++++++ .../PrepareStoreCopyFilesProvider.java | 43 +++ ...uest.java => PrepareStoreCopyRequest.java} | 18 +- ...va => PrepareStoreCopyRequestDecoder.java} | 8 +- ...va => PrepareStoreCopyRequestEncoder.java} | 6 +- .../PrepareStoreCopyRequestHandler.java | 109 ++++++ .../storecopy/PrepareStoreCopyResponse.java | 228 ++++++++++++ .../PrepareStoreCopyResponseAdaptor.java | 69 ++++ .../catchup/storecopy/RemoteStore.java | 13 +- .../catchup/storecopy/StoreCopyClient.java | 168 +++++++-- .../storecopy/StoreCopyFinishedResponse.java | 16 +- .../StoreCopyFinishedResponseDecoder.java | 3 +- .../StoreCopyFinishedResponseEncoder.java | 3 +- .../catchup/storecopy/StoreCopyProcess.java | 8 +- ...l.java => StoreFileStreamingProtocol.java} | 24 +- .../storecopy/StoreStreamingProcess.java | 14 +- .../storecopy/TerminationCondition.java | 35 ++ .../catchup/tx/CatchupPollingProcess.java | 3 +- .../core/EnterpriseCoreEditionModule.java | 3 +- .../core/RaftServerModule.java | 12 +- .../core/server/CoreServerModule.java | 6 +- .../core/state/RaftMessageApplier.java | 11 +- .../state/snapshot/CoreStateDownloader.java | 40 +- .../snapshot/CoreStateDownloaderService.java | 6 +- .../PersistentSnapshotDownloader.java | 16 +- .../EnterpriseReadReplicaEditionModule.java | 3 +- .../ReadReplicaStartupProcess.java | 3 +- .../StrippedCatchupServer.java | 149 ++++++++ .../catchup/storecopy/CatchupServerIT.java | 343 ++++++++++++++++++ .../storecopy/CloseablesListenerTest.java | 84 +++++ .../catchup/storecopy/FakeCatchupServer.java | 210 +++++++++++ .../catchup/storecopy/FakeFile.java | 115 ++++++ .../GetIndexFilesRequestMarshalTest.java | 70 ++++ .../storecopy/GetStoreFileMarshalTest.java | 70 ++++ .../storecopy/GetStoreRequestHandlerTest.java | 77 ---- .../storecopy/InMemoryFileSystemStream.java | 57 +++ .../storecopy/MaximumTotalRetriesTest.java | 70 ++++ .../storecopy/PrepareStoreCopyFilesTest.java | 158 ++++++++ .../PrepareStoreCopyRequestHandlerTest.java | 158 ++++++++ .../PrepareStoreCopyRequestMarshalTest.java | 63 ++++ .../PrepareStoreCopyResponseMarshalTest.java | 114 ++++++ .../storecopy/RealStrippedCatchupServer.java | 93 +++++ .../catchup/storecopy/RemoteStoreTest.java | 15 +- .../storecopy/SimpleCatchupClient.java | 123 +++++++ .../catchup/storecopy/StoreCopyClientIT.java | 257 +++++++++++++ .../storecopy/StoreCopyClientTest.java | 232 ++++++++++++ ...eCopyFinishedResponseEncodeDecodeTest.java | 3 +- ...va => StoreFileStreamingProtocolTest.java} | 22 +- .../storecopy/StoreStreamingProcessTest.java | 9 +- .../catchup/tx/CatchupPollingProcessTest.java | 5 +- ...ava => EnterpriseCoreEditionModuleIT.java} | 2 +- .../CoreStateDownloaderServiceTest.java | 17 +- .../snapshot/CoreStateDownloaderTest.java | 18 +- .../PersistentSnapshotDownloaderTest.java | 57 +-- .../messaging/TestServer.java | 8 +- .../scenarios/ConnectionInfoIT.java | 3 +- .../neo4j/com/storecopy/StoreCopyServer.java | 3 +- .../impl/fulltext/FulltextProviderImpl.java | 2 +- 99 files changed, 4738 insertions(+), 519 deletions(-) create mode 100644 community/kernel/src/main/java/org/neo4j/kernel/impl/transaction/state/NeoStoreFileIndexListing.java create mode 100644 community/kernel/src/main/java/org/neo4j/kernel/impl/util/MultiResource.java create mode 100644 enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/CatchupAddressProvider.java create mode 100644 enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/CatchupAddressResolutionException.java create mode 100644 enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/StoreListingResponseHandler.java create mode 100644 enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/storecopy/CloseablesListener.java create mode 100644 enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/storecopy/DataSourceChecks.java create mode 100644 enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/storecopy/GetIndexFilesRequest.java create mode 100644 enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/storecopy/GetIndexSnapshotRequestHandler.java create mode 100644 enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/storecopy/GetStoreFileRequest.java create mode 100644 enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/storecopy/GetStoreFileRequestHandler.java delete mode 100644 enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/storecopy/GetStoreRequestHandler.java create mode 100644 enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/storecopy/IndexDescriptorSerializer.java create mode 100644 enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/storecopy/MaximumTotalRetries.java create mode 100644 enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/storecopy/PrepareStoreCopyFiles.java create mode 100644 enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/storecopy/PrepareStoreCopyFilesProvider.java rename enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/storecopy/{GetStoreRequest.java => PrepareStoreCopyRequest.java} (78%) rename enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/storecopy/{GetStoreRequestDecoder.java => PrepareStoreCopyRequestDecoder.java} (77%) rename enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/storecopy/{GetStoreRequestEncoder.java => PrepareStoreCopyRequestEncoder.java} (75%) create mode 100644 enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/storecopy/PrepareStoreCopyRequestHandler.java create mode 100644 enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/storecopy/PrepareStoreCopyResponse.java create mode 100644 enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/storecopy/PrepareStoreCopyResponseAdaptor.java rename enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/storecopy/{StoreStreamingProtocol.java => StoreFileStreamingProtocol.java} (66%) create mode 100644 enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/storecopy/TerminationCondition.java create mode 100644 enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/StrippedCatchupServer.java create mode 100644 enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/catchup/storecopy/CatchupServerIT.java create mode 100644 enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/catchup/storecopy/CloseablesListenerTest.java create mode 100644 enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/catchup/storecopy/FakeCatchupServer.java create mode 100644 enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/catchup/storecopy/FakeFile.java create mode 100644 enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/catchup/storecopy/GetIndexFilesRequestMarshalTest.java create mode 100644 enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/catchup/storecopy/GetStoreFileMarshalTest.java delete mode 100644 enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/catchup/storecopy/GetStoreRequestHandlerTest.java create mode 100644 enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/catchup/storecopy/InMemoryFileSystemStream.java create mode 100644 enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/catchup/storecopy/MaximumTotalRetriesTest.java create mode 100644 enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/catchup/storecopy/PrepareStoreCopyFilesTest.java create mode 100644 enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/catchup/storecopy/PrepareStoreCopyRequestHandlerTest.java create mode 100644 enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/catchup/storecopy/PrepareStoreCopyRequestMarshalTest.java create mode 100644 enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/catchup/storecopy/PrepareStoreCopyResponseMarshalTest.java create mode 100644 enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/catchup/storecopy/RealStrippedCatchupServer.java create mode 100644 enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/catchup/storecopy/SimpleCatchupClient.java create mode 100644 enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/catchup/storecopy/StoreCopyClientIT.java create mode 100644 enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/catchup/storecopy/StoreCopyClientTest.java rename enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/catchup/storecopy/{StoreStreamingProtocolTest.java => StoreFileStreamingProtocolTest.java} (88%) rename enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/core/{EnterpriseCoreEditionModuleIntegrationTest.java => EnterpriseCoreEditionModuleIT.java} (98%) diff --git a/community/kernel-api/src/main/java/org/neo4j/internal/kernel/api/schema/LabelSchemaDescriptor.java b/community/kernel-api/src/main/java/org/neo4j/internal/kernel/api/schema/LabelSchemaDescriptor.java index 89fb64743cd77..39d7d782565d4 100644 --- a/community/kernel-api/src/main/java/org/neo4j/internal/kernel/api/schema/LabelSchemaDescriptor.java +++ b/community/kernel-api/src/main/java/org/neo4j/internal/kernel/api/schema/LabelSchemaDescriptor.java @@ -19,6 +19,9 @@ */ package org.neo4j.internal.kernel.api.schema; +/** + * Please note that this interface has dependencies in other modules. See {@see org.neo4j.causalclustering.catchup.storecopy.IndexDescriptorSerializer} + */ public interface LabelSchemaDescriptor extends SchemaDescriptor, LabelSchemaSupplier { int getLabelId(); diff --git a/community/kernel/src/main/java/org/neo4j/kernel/NeoStoreDataSource.java b/community/kernel/src/main/java/org/neo4j/kernel/NeoStoreDataSource.java index 7740117725a18..fe761f9aa893a 100644 --- a/community/kernel/src/main/java/org/neo4j/kernel/NeoStoreDataSource.java +++ b/community/kernel/src/main/java/org/neo4j/kernel/NeoStoreDataSource.java @@ -127,6 +127,7 @@ import org.neo4j.kernel.impl.transaction.log.rotation.LogRotation; import org.neo4j.kernel.impl.transaction.log.rotation.LogRotationImpl; import org.neo4j.kernel.impl.transaction.state.DefaultSchemaIndexProviderMap; +import org.neo4j.kernel.impl.transaction.state.NeoStoreFileIndexListing; import org.neo4j.kernel.impl.transaction.state.NeoStoreFileListing; import org.neo4j.kernel.impl.util.Dependencies; import org.neo4j.kernel.impl.util.SynchronizedArrayIdOrderingQueue; @@ -828,7 +829,19 @@ public InwardKernel getKernel() public ResourceIterator listStoreFiles( boolean includeLogs ) throws IOException { - return kernelModule.fileListing().listStoreFiles( includeLogs ); + if ( includeLogs ) + { + return getNeoStoreFileListing().builder().build(); + } + else + { + return getNeoStoreFileListing().builder().excludeLogFiles().build(); + } + } + + public NeoStoreFileListing getNeoStoreFileListing() + { + return kernelModule.fileListing(); } public void registerDiagnosticsWith( DiagnosticsManager manager ) diff --git a/community/kernel/src/main/java/org/neo4j/kernel/api/schema/index/IndexDescriptor.java b/community/kernel/src/main/java/org/neo4j/kernel/api/schema/index/IndexDescriptor.java index 5bdeb80d1cb4b..3aa0b2e96116f 100644 --- a/community/kernel/src/main/java/org/neo4j/kernel/api/schema/index/IndexDescriptor.java +++ b/community/kernel/src/main/java/org/neo4j/kernel/api/schema/index/IndexDescriptor.java @@ -36,6 +36,8 @@ /** * Internal representation of a graph index, including the schema unit it targets (eg. label-property combination) * and the type of index. UNIQUE indexes are used to back uniqueness constraints. + * + * Please note that this class has dependencies in other modules. See {@see org.neo4j.causalclustering.catchup.storecopy.IndexDescriptorSerializer} */ public class IndexDescriptor implements LabelSchemaSupplier { @@ -81,7 +83,7 @@ public interface Supplier private final LabelSchemaDescriptor schema; private final IndexDescriptor.Type type; - IndexDescriptor( LabelSchemaDescriptor schema, Type type ) + public IndexDescriptor( LabelSchemaDescriptor schema, Type type ) { this.schema = schema; this.type = type; diff --git a/community/kernel/src/main/java/org/neo4j/kernel/impl/api/index/IndexingService.java b/community/kernel/src/main/java/org/neo4j/kernel/impl/api/index/IndexingService.java index 4b4286c36e54b..158b074460869 100644 --- a/community/kernel/src/main/java/org/neo4j/kernel/impl/api/index/IndexingService.java +++ b/community/kernel/src/main/java/org/neo4j/kernel/impl/api/index/IndexingService.java @@ -24,11 +24,14 @@ import java.util.ArrayList; import java.util.Collection; import java.util.EnumMap; +import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.concurrent.ExecutionException; import java.util.concurrent.Future; import java.util.function.BiConsumer; +import java.util.function.Predicate; import org.neo4j.collection.primitive.Primitive; import org.neo4j.collection.primitive.PrimitiveLongIterator; @@ -635,16 +638,35 @@ private void closeAllIndexes() } ); } - public ResourceIterator snapshotStoreFiles() throws IOException + public Set getIndexDescriptors() + { + Set indexDescriptors = new HashSet<>(); + for ( IndexProxy indexProxy : indexMapRef.getAllIndexProxies() ) + { + indexDescriptors.add( indexProxy.getDescriptor() ); + } + return indexDescriptors; + } + + public ResourceIterator snapshotIndexFiles( Predicate filter ) throws IOException { Collection> snapshots = new ArrayList<>(); for ( IndexProxy indexProxy : indexMapRef.getAllIndexProxies() ) { - snapshots.add( indexProxy.snapshotFiles() ); + IndexDescriptor providerDescriptor = indexProxy.getDescriptor(); + if ( filter.test( providerDescriptor ) ) + { + snapshots.add( indexProxy.snapshotFiles() ); + } } return Iterators.concatResourceIterators( snapshots.iterator() ); } + public ResourceIterator snapshotIndexFiles() throws IOException + { + return snapshotIndexFiles( d -> true ); + } + private IndexPopulationJob newIndexPopulationJob() { MultipleIndexPopulator multiPopulator = multiPopulatorFactory.create( storeView, logProvider ); diff --git a/community/kernel/src/main/java/org/neo4j/kernel/impl/transaction/state/NeoStoreFileIndexListing.java b/community/kernel/src/main/java/org/neo4j/kernel/impl/transaction/state/NeoStoreFileIndexListing.java new file mode 100644 index 0000000000000..5caddecd3b2f6 --- /dev/null +++ b/community/kernel/src/main/java/org/neo4j/kernel/impl/transaction/state/NeoStoreFileIndexListing.java @@ -0,0 +1,108 @@ +/* + * 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 . + */ +package org.neo4j.kernel.impl.transaction.state; + +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Set; +import java.util.function.Function; +import java.util.function.Predicate; + +import org.neo4j.graphdb.Resource; +import org.neo4j.graphdb.ResourceIterator; +import org.neo4j.kernel.api.index.SchemaIndexProvider; +import org.neo4j.kernel.api.labelscan.LabelScanStore; +import org.neo4j.kernel.api.schema.index.IndexDescriptor; +import org.neo4j.kernel.impl.api.ExplicitIndexProviderLookup; +import org.neo4j.kernel.impl.api.index.IndexingService; +import org.neo4j.kernel.impl.store.format.RecordFormat; +import org.neo4j.kernel.impl.util.MultiResource; +import org.neo4j.kernel.spi.explicitindex.IndexImplementation; +import org.neo4j.storageengine.api.StoreFileMetadata; + +import static org.neo4j.helpers.collection.Iterators.resourceIterator; + +public class NeoStoreFileIndexListing +{ + private final LabelScanStore labelScanStore; + private final IndexingService indexingService; + private final ExplicitIndexProviderLookup explicitIndexProviders; + + private static final Function toStoreFileMetatadata = file -> new StoreFileMetadata( file, RecordFormat.NO_RECORD_SIZE ); + + NeoStoreFileIndexListing( LabelScanStore labelScanStore, IndexingService indexingService, ExplicitIndexProviderLookup explicitIndexProviders ) + { + this.labelScanStore = labelScanStore; + this.indexingService = indexingService; + this.explicitIndexProviders = explicitIndexProviders; + } + + public Collection listIndexDescriptors() + { + return indexingService.getIndexDescriptors(); + } + + Resource gatherSchemaIndexFiles( Collection targetFiles ) throws IOException + { + ResourceIterator snapshot = indexingService.snapshotIndexFiles(); + getSnapshotFilesMetadata( snapshot, targetFiles ); + // Intentionally don't close the snapshot here, return it for closing by the consumer of + // the targetFiles list. + return snapshot; + } + + Resource gatherLabelScanStoreFiles( Collection targetFiles ) + { + ResourceIterator snapshot = labelScanStore.snapshotStoreFiles(); + getSnapshotFilesMetadata( snapshot, targetFiles ); + // Intentionally don't close the snapshot here, return it for closing by the consumer of + // the targetFiles list. + return snapshot; + } + + Resource gatherExplicitIndexFiles( Collection files ) throws IOException + { + final Collection> snapshots = new ArrayList<>(); + for ( IndexImplementation indexProvider : explicitIndexProviders.all() ) + { + ResourceIterator snapshot = indexProvider.listStoreFiles(); + snapshots.add( snapshot ); + getSnapshotFilesMetadata( snapshot, files ); + } + // Intentionally don't close the snapshot here, return it for closing by the consumer of + // the targetFiles list. + return new MultiResource( snapshots ); + } + + private void getSnapshotFilesMetadata( ResourceIterator snapshot, Collection targetFiles ) + { + snapshot.stream().map( toStoreFileMetatadata ).forEach( targetFiles::add ); + } + + public ResourceIterator getSnapshot( IndexDescriptor descriptor ) throws IOException + { + ResourceIterator snapshot = indexingService.snapshotIndexFiles( descriptor::equals ); + ArrayList files = new ArrayList<>(); + getSnapshotFilesMetadata( snapshot, files ); + return resourceIterator( files.iterator(), snapshot ); + } +} diff --git a/community/kernel/src/main/java/org/neo4j/kernel/impl/transaction/state/NeoStoreFileListing.java b/community/kernel/src/main/java/org/neo4j/kernel/impl/transaction/state/NeoStoreFileListing.java index 4166c587f63ef..80c2f9119e82c 100644 --- a/community/kernel/src/main/java/org/neo4j/kernel/impl/transaction/state/NeoStoreFileListing.java +++ b/community/kernel/src/main/java/org/neo4j/kernel/impl/transaction/state/NeoStoreFileListing.java @@ -31,6 +31,8 @@ import org.neo4j.graphdb.Resource; import org.neo4j.graphdb.ResourceIterator; +import org.neo4j.helpers.Exceptions; +import org.neo4j.io.IOUtils; import org.neo4j.kernel.api.labelscan.LabelScanStore; import org.neo4j.kernel.impl.api.ExplicitIndexProviderLookup; import org.neo4j.kernel.impl.api.index.IndexingService; @@ -38,7 +40,7 @@ import org.neo4j.kernel.impl.store.StoreType; import org.neo4j.kernel.impl.store.format.RecordFormat; import org.neo4j.kernel.impl.transaction.log.files.LogFiles; -import org.neo4j.kernel.spi.explicitindex.IndexImplementation; +import org.neo4j.kernel.impl.util.MultiResource; import org.neo4j.storageengine.api.StorageEngine; import org.neo4j.storageengine.api.StoreFileMetadata; @@ -48,14 +50,12 @@ public class NeoStoreFileListing { private final File storeDir; private final LogFiles logFiles; - private final LabelScanStore labelScanStore; - private final IndexingService indexingService; - private final ExplicitIndexProviderLookup explicitIndexProviders; private final StorageEngine storageEngine; private static final Function toNotAStoreTypeFile = file -> new StoreFileMetadata( file, RecordFormat.NO_RECORD_SIZE ); private static final Function logFileMapper = file -> new StoreFileMetadata( file, RecordFormat.NO_RECORD_SIZE, true ); + private final NeoStoreFileIndexListing neoStoreFileIndexListing; private final Collection additionalProviders; public NeoStoreFileListing( File storeDir, LogFiles logFiles, @@ -64,30 +64,19 @@ public NeoStoreFileListing( File storeDir, LogFiles logFiles, { this.storeDir = storeDir; this.logFiles = logFiles; - this.labelScanStore = labelScanStore; - this.indexingService = indexingService; - this.explicitIndexProviders = explicitIndexProviders; this.storageEngine = storageEngine; + this.neoStoreFileIndexListing = new NeoStoreFileIndexListing( labelScanStore, indexingService, explicitIndexProviders ); this.additionalProviders = new CopyOnWriteArrayList<>(); } - public ResourceIterator listStoreFiles( boolean includeLogs ) throws IOException + public StoreFileListingBuilder builder() { - List files = new ArrayList<>(); - gatherNonRecordStores( files, includeLogs ); - gatherNeoStoreFiles( files ); - List additionalResources = new ArrayList<>(); - additionalResources.add( gatherLabelScanStoreFiles( files ) ); - additionalResources.add( gatherSchemaIndexFiles( files ) ); - additionalResources.add( gatherExplicitIndexFiles( files ) ); - for ( StoreFileProvider additionalProvider : additionalProviders ) - { - additionalResources.add( additionalProvider.addFilesTo( files ) ); - } - - placeMetaDataStoreLast( files ); + return new StoreFileListingBuilder(); + } - return resourceIterator( files.iterator(), new MultiResource( additionalResources ) ); + public NeoStoreFileIndexListing getNeoStoreFileIndexListing() + { + return neoStoreFileIndexListing; } public void registerStoreFileProvider( StoreFileProvider provider ) @@ -144,76 +133,185 @@ private void gatherNonRecordStores( Collection files, boolean } } - private Resource gatherExplicitIndexFiles( Collection files ) throws IOException + public class StoreFileListingBuilder { - final Collection> snapshots = new ArrayList<>(); - for ( IndexImplementation indexProvider : explicitIndexProviders.all() ) + private boolean excludeLogFiles; + private boolean excludeNonRecordStoreFiles; + private boolean excludeNeoStoreFiles; + private boolean excludeLabelScanStoreFiles; + private boolean excludeSchemaIndexStoreFiles; + private boolean excludeExplicitIndexStoreFiles; + private boolean excludeAdditionalProviders; + + private StoreFileListingBuilder() { - ResourceIterator snapshot = indexProvider.listStoreFiles(); - snapshots.add( snapshot ); - files.addAll( getSnapshotFilesMetadata( snapshot ) ); } - // Intentionally don't close the snapshot here, return it for closing by the consumer of - // the targetFiles list. - return new MultiResource( snapshots ); - } - private Resource gatherSchemaIndexFiles( Collection targetFiles ) throws IOException - { - ResourceIterator snapshot = indexingService.snapshotStoreFiles(); - targetFiles.addAll( getSnapshotFilesMetadata( snapshot ) ); - // Intentionally don't close the snapshot here, return it for closing by the consumer of - // the targetFiles list. - return snapshot; - } + private void excludeAll( boolean initiateInclusive ) + { + this.excludeLogFiles = + this.excludeNonRecordStoreFiles = + this.excludeNeoStoreFiles = + this.excludeLabelScanStoreFiles = + this.excludeSchemaIndexStoreFiles = this.excludeAdditionalProviders = this.excludeExplicitIndexStoreFiles = initiateInclusive; + } - private Resource gatherLabelScanStoreFiles( Collection targetFiles ) - { - ResourceIterator snapshot = labelScanStore.snapshotStoreFiles(); - targetFiles.addAll( getSnapshotFilesMetadata( snapshot ) ); - // Intentionally don't close the snapshot here, return it for closing by the consumer of - // the targetFiles list. - return snapshot; - } + public StoreFileListingBuilder excludeAll() + { + excludeAll( true ); + return this; + } - public static List getSnapshotFilesMetadata( ResourceIterator snapshot ) - { - return snapshot.stream().map( toNotAStoreTypeFile ).collect( Collectors.toList() ); - } + public StoreFileListingBuilder includeAll() + { + excludeAll( false ); + return this; + } - private void gatherNeoStoreFiles( final Collection targetFiles ) - { - targetFiles.addAll( storageEngine.listStorageFiles() ); - } + public StoreFileListingBuilder excludeLogFiles() + { + excludeLogFiles = true; + return this; + } - public static final class MultiResource implements Resource - { - private final Collection snapshots; + public StoreFileListingBuilder excludeNonRecordStoreFiles() + { + excludeNonRecordStoreFiles = true; + return this; + } + + public StoreFileListingBuilder excludeNeoStoreFiles() + { + excludeNeoStoreFiles = true; + return this; + } + + public StoreFileListingBuilder excludeLabelScanStoreFiles() + { + excludeLabelScanStoreFiles = true; + return this; + } + + public StoreFileListingBuilder excludeSchemaIndexStoreFiles() + { + excludeSchemaIndexStoreFiles = true; + return this; + } + + public StoreFileListingBuilder excludeExplicitIndexStoreFiles() + { + excludeExplicitIndexStoreFiles = true; + return this; + } + + public StoreFileListingBuilder excludeAdditionalProviders() + { + excludeAdditionalProviders = true; + return this; + } - public MultiResource( Collection resources ) + public StoreFileListingBuilder includeLogFiles() { - this.snapshots = resources; + excludeLogFiles = false; + return this; } - @Override - public void close() + public StoreFileListingBuilder includeNonRecordStoreFiles() { - RuntimeException exception = null; - for ( Resource snapshot : snapshots ) + excludeNonRecordStoreFiles = false; + return this; + } + + public StoreFileListingBuilder includeNeoStoreFiles() + { + excludeNeoStoreFiles = false; + return this; + } + + public StoreFileListingBuilder includeLabelScanStoreFiles() + { + excludeLabelScanStoreFiles = false; + return this; + } + + public StoreFileListingBuilder includeSchemaIndexStoreFiles() + { + excludeSchemaIndexStoreFiles = false; + return this; + } + + public StoreFileListingBuilder includeExplicitIndexStoreStoreFiles() + { + excludeExplicitIndexStoreFiles = false; + return this; + } + + public StoreFileListingBuilder includeAdditionalProviders() + { + excludeAdditionalProviders = false; + return this; + } + + public ResourceIterator build() throws IOException + { + List files = new ArrayList<>(); + List resources = new ArrayList<>(); + try { - try + if ( !excludeNonRecordStoreFiles ) + { + gatherNonRecordStores( files, !excludeLogFiles ); + } + if ( !excludeNeoStoreFiles ) + { + gatherNeoStoreFiles( files ); + } + if ( !excludeLabelScanStoreFiles ) + { + resources.add( neoStoreFileIndexListing.gatherLabelScanStoreFiles( files ) ); + } + if ( !excludeSchemaIndexStoreFiles ) { - snapshot.close(); + resources.add( neoStoreFileIndexListing.gatherSchemaIndexFiles( files ) ); } - catch ( RuntimeException e ) + if ( !excludeExplicitIndexStoreFiles ) { - exception = e; + resources.add( neoStoreFileIndexListing.gatherExplicitIndexFiles( files ) ); } + if ( !excludeAdditionalProviders ) + { + for ( StoreFileProvider additionalProvider : additionalProviders ) + { + resources.add( additionalProvider.addFilesTo( files ) ); + } + } + + placeMetaDataStoreLast( files ); } - if ( exception != null ) + catch ( IOException e ) { - throw exception; + try + { + IOUtils.closeAll( resources ); + } + catch ( IOException e1 ) + { + e = Exceptions.chain( e, e1 ); + } + throw e; } + + return resourceIterator( files.iterator(), new MultiResource( resources ) ); } } + + public static List getSnapshotFilesMetadata( ResourceIterator snapshot ) + { + return snapshot.stream().map( toNotAStoreTypeFile ).collect( Collectors.toList() ); + } + + private void gatherNeoStoreFiles( final Collection targetFiles ) + { + targetFiles.addAll( storageEngine.listStorageFiles() ); + } } diff --git a/community/kernel/src/main/java/org/neo4j/kernel/impl/util/MultiResource.java b/community/kernel/src/main/java/org/neo4j/kernel/impl/util/MultiResource.java new file mode 100644 index 0000000000000..044e5bce93ba0 --- /dev/null +++ b/community/kernel/src/main/java/org/neo4j/kernel/impl/util/MultiResource.java @@ -0,0 +1,49 @@ +/* + * 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 . + */ +package org.neo4j.kernel.impl.util; + +import java.io.IOException; +import java.util.Collection; + +import org.neo4j.graphdb.Resource; +import org.neo4j.io.IOUtils; + +public final class MultiResource implements Resource +{ + private final Collection resources; + + public MultiResource( Collection resources ) + { + this.resources = resources; + } + + @Override + public void close() + { + try + { + IOUtils.closeAll( resources ); + } + catch ( IOException e ) + { + throw new RuntimeException( e ); + } + } +} diff --git a/community/kernel/src/test/java/org/neo4j/kernel/impl/api/index/IndexingServiceTest.java b/community/kernel/src/test/java/org/neo4j/kernel/impl/api/index/IndexingServiceTest.java index d5045f3f99036..94706883b46fd 100644 --- a/community/kernel/src/test/java/org/neo4j/kernel/impl/api/index/IndexingServiceTest.java +++ b/community/kernel/src/test/java/org/neo4j/kernel/impl/api/index/IndexingServiceTest.java @@ -484,7 +484,7 @@ public void shouldSnapshotOnlineIndexes() throws Exception life.start(); // WHEN - ResourceIterator files = indexing.snapshotStoreFiles(); + ResourceIterator files = indexing.snapshotIndexFiles(); // THEN // We get a snapshot per online index @@ -514,7 +514,7 @@ public void shouldNotSnapshotPopulatingIndexes() throws Exception life.start(); // WHEN - ResourceIterator files = indexing.snapshotStoreFiles(); + ResourceIterator files = indexing.snapshotIndexFiles(); populatorLatch.countDown(); // only now, after the snapshot, is the population job allowed to finish waitForIndexesToComeOnline( indexing, indexId, indexId2 ); diff --git a/community/kernel/src/test/java/org/neo4j/kernel/impl/transaction/state/NeoStoreFileListingTest.java b/community/kernel/src/test/java/org/neo4j/kernel/impl/transaction/state/NeoStoreFileListingTest.java index 3cc0bb5ec4f7a..d78002323c065 100644 --- a/community/kernel/src/test/java/org/neo4j/kernel/impl/transaction/state/NeoStoreFileListingTest.java +++ b/community/kernel/src/test/java/org/neo4j/kernel/impl/transaction/state/NeoStoreFileListingTest.java @@ -138,7 +138,7 @@ public void shouldCloseIndexAndLabelScanSnapshots() throws Exception new String[]{"blah/scan.store", "scan.more"} ); ResourceIterator indexSnapshot = indexFilesAre( indexingService, new String[]{"schema/index/my.index"} ); - ResourceIterator result = fileListing.listStoreFiles( false ); + ResourceIterator result = fileListing.builder().excludeLogFiles().build(); // When result.close(); @@ -202,8 +202,7 @@ public void shouldNotListTxLogFiles() throws Exception public void shouldListNeostoreFiles() throws Exception { StoreType[] values = StoreType.values(); - ResourceIterator storeFiles = - neoStoreDataSource.listStoreFiles( false ); + ResourceIterator storeFiles = neoStoreDataSource.listStoreFiles( false ); List listedStoreFiles = storeFiles.stream() .map( toStoreType ) .filter( Optional::isPresent ) @@ -249,7 +248,7 @@ private ResourceIterator indexFilesAre( IndexingService indexingService, S ArrayList files = new ArrayList<>(); mockFiles( fileNames, files, false ); ResourceIterator snapshot = spy( asResourceIterator( files.iterator() ) ); - when( indexingService.snapshotStoreFiles() ).thenReturn( snapshot ); + when( indexingService.snapshotIndexFiles() ).thenReturn( snapshot ); return snapshot; } diff --git a/community/neo4j/src/test/java/org/neo4j/index/backup/IndexBackupIT.java b/community/neo4j/src/test/java/org/neo4j/index/backup/IndexBackupIT.java index 6ab99e14dd377..70867cea920bc 100644 --- a/community/neo4j/src/test/java/org/neo4j/index/backup/IndexBackupIT.java +++ b/community/neo4j/src/test/java/org/neo4j/index/backup/IndexBackupIT.java @@ -79,20 +79,20 @@ public void concurrentIndexSnapshotUseDifferentSnapshots() throws Exception prepareDatabase( label ); forceCheckpoint( checkPointer ); - ResourceIterator firstCheckpointSnapshot = indexingService.snapshotStoreFiles(); + ResourceIterator firstCheckpointSnapshot = indexingService.snapshotIndexFiles(); generateData( label ); removeOldNodes( LongStream.range( 1, 20 ) ); updateOldNodes( LongStream.range( 30, 40 ) ); forceCheckpoint( checkPointer ); - ResourceIterator secondCheckpointSnapshot = indexingService.snapshotStoreFiles(); + ResourceIterator secondCheckpointSnapshot = indexingService.snapshotIndexFiles(); generateData( label ); removeOldNodes( LongStream.range( 50, 60 ) ); updateOldNodes( LongStream.range( 70, 80 ) ); forceCheckpoint( checkPointer ); - ResourceIterator thirdCheckpointSnapshot = indexingService.snapshotStoreFiles(); + ResourceIterator thirdCheckpointSnapshot = indexingService.snapshotIndexFiles(); Set firstSnapshotFileNames = getFileNames( firstCheckpointSnapshot ); Set secondSnapshotFileNames = getFileNames( secondCheckpointSnapshot ); @@ -114,11 +114,11 @@ public void snapshotFilesDeletedWhenSnapshotReleased() throws IOException Label label = Label.label( "testLabel" ); prepareDatabase( label ); - ResourceIterator firstCheckpointSnapshot = indexingService.snapshotStoreFiles(); + ResourceIterator firstCheckpointSnapshot = indexingService.snapshotIndexFiles(); generateData( label ); - ResourceIterator secondCheckpointSnapshot = indexingService.snapshotStoreFiles(); + ResourceIterator secondCheckpointSnapshot = indexingService.snapshotIndexFiles(); generateData( label ); - ResourceIterator thirdCheckpointSnapshot = indexingService.snapshotStoreFiles(); + ResourceIterator thirdCheckpointSnapshot = indexingService.snapshotIndexFiles(); Set firstSnapshotFileNames = getFileNames( firstCheckpointSnapshot ); Set secondSnapshotFileNames = getFileNames( secondCheckpointSnapshot ); diff --git a/enterprise/backup/src/main/java/org/neo4j/backup/impl/BackupDelegator.java b/enterprise/backup/src/main/java/org/neo4j/backup/impl/BackupDelegator.java index 4efebd0d0873c..d38a0e7e2423f 100644 --- a/enterprise/backup/src/main/java/org/neo4j/backup/impl/BackupDelegator.java +++ b/enterprise/backup/src/main/java/org/neo4j/backup/impl/BackupDelegator.java @@ -23,6 +23,7 @@ import java.nio.file.Path; import org.neo4j.causalclustering.catchup.CatchUpClient; +import org.neo4j.causalclustering.catchup.CatchupAddressProvider; import org.neo4j.causalclustering.catchup.CatchupResult; import org.neo4j.causalclustering.catchup.storecopy.RemoteStore; import org.neo4j.causalclustering.catchup.storecopy.StoreCopyClient; @@ -54,7 +55,7 @@ void copy( AdvertisedSocketAddress fromAddress, StoreId expectedStoreId, Path de { try { - remoteStore.copy( fromAddress, expectedStoreId, destDir.toFile() ); + remoteStore.copy( new CatchupAddressProvider.SingleAddressProvider( fromAddress ), expectedStoreId, destDir.toFile() ); } catch ( StreamingTransactionsFailedException e ) { diff --git a/enterprise/backup/src/test/java/org/neo4j/backup/impl/BackupDelegatorTest.java b/enterprise/backup/src/test/java/org/neo4j/backup/impl/BackupDelegatorTest.java index 0554e41b2d4b9..347694e44a9e4 100644 --- a/enterprise/backup/src/test/java/org/neo4j/backup/impl/BackupDelegatorTest.java +++ b/enterprise/backup/src/test/java/org/neo4j/backup/impl/BackupDelegatorTest.java @@ -21,12 +21,15 @@ import org.junit.Before; import org.junit.Test; +import org.mockito.ArgumentCaptor; import java.io.IOException; import java.nio.file.Path; import java.nio.file.Paths; import org.neo4j.causalclustering.catchup.CatchUpClient; +import org.neo4j.causalclustering.catchup.CatchupAddressProvider; +import org.neo4j.causalclustering.catchup.CatchupAddressResolutionException; import org.neo4j.causalclustering.catchup.storecopy.RemoteStore; import org.neo4j.causalclustering.catchup.storecopy.StoreCopyClient; import org.neo4j.causalclustering.catchup.storecopy.StoreCopyFailedException; @@ -36,6 +39,7 @@ import org.neo4j.helpers.AdvertisedSocketAddress; import static org.junit.Assert.assertEquals; +import static org.mockito.ArgumentMatchers.eq; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; @@ -60,7 +64,7 @@ public void setup() } @Test - public void tryCatchingUpDelegatesToRemoteStore() throws org.neo4j.causalclustering.catchup.storecopy.StoreCopyFailedException, IOException + public void tryCatchingUpDelegatesToRemoteStore() throws StoreCopyFailedException, IOException { // given AdvertisedSocketAddress fromAddress = new AdvertisedSocketAddress( "neo4j.com", 5432 ); @@ -112,7 +116,8 @@ public void fetchStoreIdDelegatesToStoreCopyClient() throws StoreIdDownloadFaile } @Test - public void retrieveStoreDelegatesToStoreCopyService() throws StoreCopyFailedException, StreamingTransactionsFailedException + public void retrieveStoreDelegatesToStoreCopyService() + throws StoreCopyFailedException, StreamingTransactionsFailedException, CatchupAddressResolutionException { // given StoreId storeId = new StoreId( 92, 5, 7, 32 ); @@ -122,6 +127,10 @@ public void retrieveStoreDelegatesToStoreCopyService() throws StoreCopyFailedExc subject.copy( anyAddress, storeId, anyFile ); // then - verify( remoteStore ).copy( anyAddress, storeId, anyFile.toFile() ); + ArgumentCaptor argumentCaptor = ArgumentCaptor.forClass( CatchupAddressProvider.class ); + verify( remoteStore ).copy( argumentCaptor.capture(), eq( storeId ), eq( anyFile.toFile() ) ); + + //and + assertEquals( anyAddress, argumentCaptor.getValue().primary() ); } } diff --git a/enterprise/backup/src/test/java/org/neo4j/backup/impl/CausalClusteringBackupStrategyTest.java b/enterprise/backup/src/test/java/org/neo4j/backup/impl/CausalClusteringBackupStrategyTest.java index a26dd87f0dd95..7e8ae198f189f 100644 --- a/enterprise/backup/src/test/java/org/neo4j/backup/impl/CausalClusteringBackupStrategyTest.java +++ b/enterprise/backup/src/test/java/org/neo4j/backup/impl/CausalClusteringBackupStrategyTest.java @@ -128,7 +128,6 @@ public void fullRunsRetrieveStoreWithTargetsStoreId() throws StoreIdDownloadFail public void failingToRetrieveStoreIdCausesFailWithStatus_incrementalBackup() throws StoreIdDownloadFailedException { // given - AdvertisedSocketAddress fromAddress = anyAddress(); StoreIdDownloadFailedException storeIdDownloadFailedException = new StoreIdDownloadFailedException( "Expected description" ); when( backupDelegator.fetchStoreId( any() ) ).thenThrow( storeIdDownloadFailedException ); @@ -224,9 +223,4 @@ private StoreId anyStoreId() { return new StoreId( 1, 2, 3, 4 ); } - - private AdvertisedSocketAddress anyAddress() - { - return new AdvertisedSocketAddress( "hostname", 1234 ); - } } diff --git a/enterprise/backup/src/test/java/org/neo4j/backup/impl/OnlineBackupCommandCcIT.java b/enterprise/backup/src/test/java/org/neo4j/backup/impl/OnlineBackupCommandCcIT.java index 7b1472948515e..be2ed2ce95e49 100644 --- a/enterprise/backup/src/test/java/org/neo4j/backup/impl/OnlineBackupCommandCcIT.java +++ b/enterprise/backup/src/test/java/org/neo4j/backup/impl/OnlineBackupCommandCcIT.java @@ -138,7 +138,7 @@ public void dataIsInAUsableStateAfterBackup() throws Exception AtomicBoolean populateDatabaseFlag = new AtomicBoolean( true ); new Thread( () -> repeatedlyPopulateDatabase( cluster, populateDatabaseFlag ) ) .start(); // populate db with number properties etc. - oneOffShutdownTasks.add( () -> populateDatabaseFlag.set( false ) ); // kill thread + oneOffShutdownTasks.add( () -> populateDatabaseFlag.set( false ) ); // kill thread after test is complete // then backup is successful String address = TestHelpers.backupAddressCc( clusterLeader( cluster ).database() ); diff --git a/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/CatchUpClientChannelPipeline.java b/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/CatchUpClientChannelPipeline.java index ab7d53091e510..b01a62c006e9d 100644 --- a/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/CatchUpClientChannelPipeline.java +++ b/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/CatchUpClientChannelPipeline.java @@ -31,12 +31,15 @@ import org.neo4j.causalclustering.catchup.storecopy.FileChunkHandler; import org.neo4j.causalclustering.catchup.storecopy.FileHeaderDecoder; import org.neo4j.causalclustering.catchup.storecopy.FileHeaderHandler; +import org.neo4j.causalclustering.catchup.storecopy.GetIndexFilesRequest; +import org.neo4j.causalclustering.catchup.storecopy.GetStoreFileRequest; import org.neo4j.causalclustering.catchup.storecopy.GetStoreIdRequestEncoder; import org.neo4j.causalclustering.catchup.storecopy.GetStoreIdResponseDecoder; import org.neo4j.causalclustering.catchup.storecopy.GetStoreIdResponseHandler; -import org.neo4j.causalclustering.catchup.storecopy.GetStoreRequestEncoder; +import org.neo4j.causalclustering.catchup.storecopy.PrepareStoreCopyRequestEncoder; import org.neo4j.causalclustering.catchup.storecopy.StoreCopyFinishedResponseDecoder; import org.neo4j.causalclustering.catchup.storecopy.StoreCopyFinishedResponseHandler; +import org.neo4j.causalclustering.catchup.storecopy.PrepareStoreCopyResponse; import org.neo4j.causalclustering.catchup.tx.TxPullRequestEncoder; import org.neo4j.causalclustering.catchup.tx.TxPullResponseDecoder; import org.neo4j.causalclustering.catchup.tx.TxPullResponseHandler; @@ -77,11 +80,13 @@ static void initChannel( SocketChannel ch, CatchUpResponseHandler handler, LogPr pipeline.addLast( new VersionPrepender() ); pipeline.addLast( new TxPullRequestEncoder() ); - pipeline.addLast( new GetStoreRequestEncoder() ); + pipeline.addLast( new GetIndexFilesRequest.Encoder() ); + pipeline.addLast( new GetStoreFileRequest.Encoder() ); pipeline.addLast( new CoreSnapshotRequestEncoder() ); pipeline.addLast( new GetStoreIdRequestEncoder() ); pipeline.addLast( new ResponseMessageTypeEncoder() ); pipeline.addLast( new RequestMessageTypeEncoder() ); + pipeline.addLast( new PrepareStoreCopyRequestEncoder() ); pipeline.addLast( new ClientMessageTypeHandler( protocol, logProvider ) ); @@ -95,6 +100,7 @@ static void initChannel( SocketChannel ch, CatchUpResponseHandler handler, LogPr decoderDispatcher.register( CatchupClientProtocol.State.TX_STREAM_FINISHED, new TxStreamFinishedResponseDecoder() ); decoderDispatcher.register( CatchupClientProtocol.State.FILE_HEADER, new FileHeaderDecoder() ); + decoderDispatcher.register( CatchupClientProtocol.State.PREPARE_STORE_COPY_RESPONSE, new PrepareStoreCopyResponse.Decoder() ); decoderDispatcher.register( CatchupClientProtocol.State.FILE_CONTENTS, new FileChunkDecoder() ); pipeline.addLast( decoderDispatcher ); @@ -111,5 +117,6 @@ static void initChannel( SocketChannel ch, CatchUpResponseHandler handler, LogPr pipeline.addLast( new ExceptionMonitoringHandler( monitors.newMonitor( ExceptionMonitoringHandler.Monitor.class, CatchUpClient.class ) ) ); pipeline.addLast( new ExceptionSwallowingHandler() ); + pipeline.addLast( new StoreListingResponseHandler( protocol, handler )); } } diff --git a/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/CatchUpResponseAdaptor.java b/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/CatchUpResponseAdaptor.java index 1560076fec592..1a48fe61a8f56 100644 --- a/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/CatchUpResponseAdaptor.java +++ b/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/CatchUpResponseAdaptor.java @@ -26,6 +26,7 @@ import org.neo4j.causalclustering.catchup.storecopy.FileHeader; import org.neo4j.causalclustering.catchup.storecopy.GetStoreIdResponse; import org.neo4j.causalclustering.catchup.storecopy.StoreCopyFinishedResponse; +import org.neo4j.causalclustering.catchup.storecopy.PrepareStoreCopyResponse; import org.neo4j.causalclustering.catchup.tx.TxPullResponse; import org.neo4j.causalclustering.catchup.tx.TxStreamFinishedResponse; import org.neo4j.causalclustering.core.state.snapshot.CoreSnapshot; @@ -35,43 +36,54 @@ public class CatchUpResponseAdaptor implements CatchUpResponseCallback @Override public void onFileHeader( CompletableFuture signal, FileHeader response ) { - signal.completeExceptionally( new CatchUpProtocolViolationException( "Unexpected response: %s", response ) ); + unimplementedMethod( signal, response ); } @Override public boolean onFileContent( CompletableFuture signal, FileChunk response ) throws IOException { - signal.completeExceptionally( new CatchUpProtocolViolationException( "Unexpected response: %s", response ) ); + unimplementedMethod( signal, response ); return false; } @Override public void onFileStreamingComplete( CompletableFuture signal, StoreCopyFinishedResponse response ) { - signal.completeExceptionally( new CatchUpProtocolViolationException( "Unexpected response: %s", response ) ); + unimplementedMethod( signal, response ); } @Override public void onTxPullResponse( CompletableFuture signal, TxPullResponse response ) { - signal.completeExceptionally( new CatchUpProtocolViolationException( "Unexpected response: %s", response ) ); + unimplementedMethod( signal, response ); } @Override public void onTxStreamFinishedResponse( CompletableFuture signal, TxStreamFinishedResponse response ) { - signal.completeExceptionally( new CatchUpProtocolViolationException( "Unexpected response: %s", response ) ); + unimplementedMethod( signal, response ); } @Override public void onGetStoreIdResponse( CompletableFuture signal, GetStoreIdResponse response ) { - signal.completeExceptionally( new CatchUpProtocolViolationException( "Unexpected response: %s", response ) ); + unimplementedMethod( signal, response ); } @Override public void onCoreSnapshot( CompletableFuture signal, CoreSnapshot response ) { - signal.completeExceptionally( new CatchUpProtocolViolationException( "Unexpected response: %s", response ) ); + unimplementedMethod( signal, response ); + } + + @Override + public void onStoreListingResponse( CompletableFuture signal, PrepareStoreCopyResponse response ) + { + unimplementedMethod( signal, response ); + } + + private void unimplementedMethod( CompletableFuture signal, U response ) + { + signal.completeExceptionally( new CatchUpProtocolViolationException( "This Adaptor has unimplemented methods for: %s", response ) ); } } diff --git a/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/CatchUpResponseCallback.java b/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/CatchUpResponseCallback.java index 53dc18794fd23..d326885282d9e 100644 --- a/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/CatchUpResponseCallback.java +++ b/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/CatchUpResponseCallback.java @@ -26,6 +26,7 @@ import org.neo4j.causalclustering.catchup.storecopy.FileHeader; import org.neo4j.causalclustering.catchup.storecopy.GetStoreIdResponse; import org.neo4j.causalclustering.catchup.storecopy.StoreCopyFinishedResponse; +import org.neo4j.causalclustering.catchup.storecopy.PrepareStoreCopyResponse; import org.neo4j.causalclustering.catchup.tx.TxPullResponse; import org.neo4j.causalclustering.catchup.tx.TxStreamFinishedResponse; import org.neo4j.causalclustering.core.state.snapshot.CoreSnapshot; @@ -45,4 +46,6 @@ public interface CatchUpResponseCallback void onGetStoreIdResponse( CompletableFuture signal, GetStoreIdResponse response ); void onCoreSnapshot( CompletableFuture signal, CoreSnapshot coreSnapshot ); + + void onStoreListingResponse( CompletableFuture signal, PrepareStoreCopyResponse prepareStoreCopyResponse ); } diff --git a/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/CatchUpResponseHandler.java b/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/CatchUpResponseHandler.java index c5d09dd0c998b..21908f4ce2080 100644 --- a/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/CatchUpResponseHandler.java +++ b/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/CatchUpResponseHandler.java @@ -25,6 +25,7 @@ import org.neo4j.causalclustering.catchup.storecopy.FileHeader; import org.neo4j.causalclustering.catchup.storecopy.GetStoreIdResponse; import org.neo4j.causalclustering.catchup.storecopy.StoreCopyFinishedResponse; +import org.neo4j.causalclustering.catchup.storecopy.PrepareStoreCopyResponse; import org.neo4j.causalclustering.catchup.tx.TxPullResponse; import org.neo4j.causalclustering.catchup.tx.TxStreamFinishedResponse; import org.neo4j.causalclustering.core.state.snapshot.CoreSnapshot; @@ -48,4 +49,6 @@ public interface CatchUpResponseHandler void onGetStoreIdResponse( GetStoreIdResponse response ); void onCoreSnapshot( CoreSnapshot coreSnapshot ); + + void onStoreListingResponse( PrepareStoreCopyResponse storeListingRequest ); } diff --git a/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/CatchupAddressProvider.java b/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/CatchupAddressProvider.java new file mode 100644 index 0000000000000..b6a7a2842a90b --- /dev/null +++ b/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/CatchupAddressProvider.java @@ -0,0 +1,114 @@ +/* + * 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 Affero 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 Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + */ +package org.neo4j.causalclustering.catchup; + +import java.util.ArrayList; +import java.util.List; + +import org.neo4j.causalclustering.core.consensus.NoLeaderFoundException; +import org.neo4j.causalclustering.core.consensus.RaftMachine; +import org.neo4j.causalclustering.discovery.TopologyService; +import org.neo4j.causalclustering.identity.MemberId; +import org.neo4j.helpers.AdvertisedSocketAddress; + +/** + * Address provider for catchup client. + */ +public interface CatchupAddressProvider +{ + /** + * @return The address to the primary location where up to date requests are required. For a cluster aware provider the obvious choice would be the + * leader address. + * @throws CatchupAddressResolutionException if the provider was unable to find an address to this location. + */ + AdvertisedSocketAddress primary() throws CatchupAddressResolutionException; + + /** + * @return The address to a secondary location that are not required to be up to date. If there are multiple secondary locations it is recommended to + * do some simple load balancing for returned addresses. This is to avoid re-sending failed requests to the same instance immediately. + * @throws CatchupAddressResolutionException if the provider was unable to find an address to this location. + */ + AdvertisedSocketAddress secondary() throws CatchupAddressResolutionException; + + class SingleAddressProvider implements CatchupAddressProvider + { + private final AdvertisedSocketAddress socketAddress; + + public SingleAddressProvider( AdvertisedSocketAddress socketAddress ) + { + this.socketAddress = socketAddress; + } + + @Override + public AdvertisedSocketAddress primary() + { + return socketAddress; + } + + @Override + public AdvertisedSocketAddress secondary() + { + return socketAddress; + } + } + + class TopologyBasedAddressProvider implements CatchupAddressProvider + { + private final RaftMachine raftMachine; + private final TopologyService topologyService; + + public TopologyBasedAddressProvider( RaftMachine raftMachine, TopologyService topologyService ) + { + this.raftMachine = raftMachine; + this.topologyService = topologyService; + } + + @Override + public AdvertisedSocketAddress primary() throws CatchupAddressResolutionException + { + try + { + MemberId leadMember = raftMachine.getLeader(); + return topologyService.findCatchupAddress( leadMember ).orElseThrow( () -> new CatchupAddressResolutionException( leadMember ) ); + } + catch ( NoLeaderFoundException e ) + { + throw new CatchupAddressResolutionException( e ); + } + } + + @Override + public AdvertisedSocketAddress secondary() throws CatchupAddressResolutionException + { + List potentialCoresAndReplicas = new ArrayList<>( raftMachine.votingMembers() ); + potentialCoresAndReplicas.addAll( raftMachine.replicationMembers() ); + int accountForRoundingDown = 1; + MemberId randomlySelectedCatchupServer = + potentialCoresAndReplicas.get( (int) (Math.random() * potentialCoresAndReplicas.size() + accountForRoundingDown) ); + return topologyService.findCatchupAddress( randomlySelectedCatchupServer ).orElseThrow( + () -> new CatchupAddressResolutionException( randomlySelectedCatchupServer ) ); + } + } + + static CatchupAddressProvider fromSingleAddress( AdvertisedSocketAddress advertisedSocketAddress ) + { + return new SingleAddressProvider( advertisedSocketAddress ); + } +} diff --git a/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/CatchupAddressResolutionException.java b/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/CatchupAddressResolutionException.java new file mode 100644 index 0000000000000..150fecda3f3dd --- /dev/null +++ b/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/CatchupAddressResolutionException.java @@ -0,0 +1,36 @@ +/* + * 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 Affero 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 Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + */ +package org.neo4j.causalclustering.catchup; + +import org.neo4j.causalclustering.core.state.snapshot.TopologyLookupException; +import org.neo4j.causalclustering.identity.MemberId; + +public class CatchupAddressResolutionException extends TopologyLookupException +{ + public CatchupAddressResolutionException( MemberId memberId ) + { + super( memberId ); + } + + public CatchupAddressResolutionException( Exception e ) + { + super( e ); + } +} diff --git a/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/CatchupClientProtocol.java b/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/CatchupClientProtocol.java index 495e0bb52884f..df2943c1ac6c5 100644 --- a/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/CatchupClientProtocol.java +++ b/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/CatchupClientProtocol.java @@ -35,6 +35,8 @@ public enum State STORE_COPY_FINISHED, TX_STREAM_FINISHED, FILE_HEADER, + PREPARE_STORE_COPY_RESPONSE, + INDEX_SNAPSHOT_RESPONSE, FILE_CONTENTS } } diff --git a/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/CatchupServer.java b/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/CatchupServer.java index cee4c7abb55e6..2dd303e8613bb 100644 --- a/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/CatchupServer.java +++ b/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/CatchupServer.java @@ -44,15 +44,19 @@ import org.neo4j.causalclustering.catchup.CatchupServerProtocol.State; import org.neo4j.causalclustering.catchup.storecopy.FileChunkEncoder; import org.neo4j.causalclustering.catchup.storecopy.FileHeaderEncoder; +import org.neo4j.causalclustering.catchup.storecopy.GetIndexFilesRequest; +import org.neo4j.causalclustering.catchup.storecopy.GetIndexSnapshotRequestHandler; +import org.neo4j.causalclustering.catchup.storecopy.GetStoreFileRequest; +import org.neo4j.causalclustering.catchup.storecopy.GetStoreFileRequestHandler; import org.neo4j.causalclustering.catchup.storecopy.GetStoreIdRequest; import org.neo4j.causalclustering.catchup.storecopy.GetStoreIdRequestHandler; import org.neo4j.causalclustering.catchup.storecopy.GetStoreIdResponseEncoder; -import org.neo4j.causalclustering.catchup.storecopy.GetStoreRequestDecoder; -import org.neo4j.causalclustering.catchup.storecopy.GetStoreRequestHandler; +import org.neo4j.causalclustering.catchup.storecopy.PrepareStoreCopyFilesProvider; +import org.neo4j.causalclustering.catchup.storecopy.PrepareStoreCopyRequestDecoder; +import org.neo4j.causalclustering.catchup.storecopy.PrepareStoreCopyRequestHandler; +import org.neo4j.causalclustering.catchup.storecopy.PrepareStoreCopyResponse; import org.neo4j.causalclustering.catchup.storecopy.StoreCopyFinishedResponseEncoder; -import org.neo4j.causalclustering.catchup.storecopy.StoreResourceStreamFactory; -import org.neo4j.causalclustering.catchup.storecopy.StoreStreamingProcess; -import org.neo4j.causalclustering.catchup.storecopy.StoreStreamingProtocol; +import org.neo4j.causalclustering.catchup.storecopy.StoreFileStreamingProtocol; import org.neo4j.causalclustering.catchup.tx.TxPullRequestDecoder; import org.neo4j.causalclustering.catchup.tx.TxPullRequestHandler; import org.neo4j.causalclustering.catchup.tx.TxPullResponseEncoder; @@ -143,9 +147,11 @@ public synchronized void start() workerGroup = new NioEventLoopGroup( 0, threadFactory ); - ServerBootstrap bootstrap = new ServerBootstrap().group( workerGroup ).channel( NioServerSocketChannel.class ) + ServerBootstrap bootstrap = new ServerBootstrap().group( workerGroup ) + .channel( NioServerSocketChannel.class ) .option( ChannelOption.SO_REUSEADDR, true ) - .localAddress( listenAddress.socketAddress() ).childHandler( new ChannelInitializer() + .localAddress( listenAddress.socketAddress() ) + .childHandler( new ChannelInitializer() { @Override protected void initChannel( SocketChannel ch ) throws Exception @@ -175,6 +181,7 @@ protected void initChannel( SocketChannel ch ) throws Exception pipeline.addLast( new TxStreamFinishedResponseEncoder() ); pipeline.addLast( new FileChunkEncoder() ); pipeline.addLast( new FileHeaderEncoder() ); + pipeline.addLast( new PrepareStoreCopyResponse.Encoder() ); pipeline.addLast( new ServerMessageTypeHandler( protocol, logProvider ) ); @@ -182,13 +189,18 @@ protected void initChannel( SocketChannel ch ) throws Exception pipeline.addLast( new ChunkedWriteHandler() ); - pipeline.addLast( new TxPullRequestHandler( protocol, storeIdSupplier, dataSourceAvailabilitySupplier, - transactionIdStoreSupplier, logicalTransactionStoreSupplier, monitors, logProvider ) ); - pipeline.addLast( new GetStoreRequestHandler( protocol, dataSourceSupplier, - new StoreStreamingProcess( new StoreStreamingProtocol(), checkPointerSupplier, storeCopyCheckPointMutex, - new StoreResourceStreamFactory( pageCache, fs, dataSourceSupplier ) ) ) ); + pipeline.addLast( new TxPullRequestHandler( protocol, storeIdSupplier, dataSourceAvailabilitySupplier, transactionIdStoreSupplier, + logicalTransactionStoreSupplier, monitors, logProvider ) ); pipeline.addLast( new GetStoreIdRequestHandler( protocol, storeIdSupplier ) ); + pipeline.addLast( + storeListingRequestHandler( protocol, checkPointerSupplier, storeCopyCheckPointMutex, dataSourceSupplier, pageCache, fs ) ); + pipeline.addLast( + new GetStoreFileRequestHandler( protocol, dataSourceSupplier, checkPointerSupplier, new StoreFileStreamingProtocol(), pageCache, + fs, logProvider ) ); + pipeline.addLast( + new GetIndexSnapshotRequestHandler( protocol, dataSourceSupplier, checkPointerSupplier, new StoreFileStreamingProtocol(), + pageCache, fs ) ); if ( snapshotService != null ) { @@ -196,9 +208,8 @@ protected void initChannel( SocketChannel ch ) throws Exception } pipeline.addLast( new ExceptionLoggingHandler( log ) ); - pipeline.addLast( new ExceptionMonitoringHandler( - monitors.newMonitor( ExceptionMonitoringHandler.Monitor.class, - CatchupServer.class ) ) ); + pipeline.addLast( + new ExceptionMonitoringHandler( monitors.newMonitor( ExceptionMonitoringHandler.Monitor.class, CatchupServer.class ) ) ); pipeline.addLast( new ExceptionSwallowingHandler() ); } } ); @@ -214,24 +225,32 @@ protected void initChannel( SocketChannel ch ) throws Exception //noinspection ConstantConditions if ( e instanceof BindException ) { - userLog.error( - "Address is already bound for setting: " + CausalClusteringSettings.transaction_listen_address + - " with value: " + listenAddress ); - log.error( - "Address is already bound for setting: " + CausalClusteringSettings.transaction_listen_address + - " with value: " + listenAddress, e ); + String message = String.format( "Address is already bound for setting: %s with value: %s", CausalClusteringSettings.transaction_listen_address, + listenAddress ); + userLog.error( message ); + log.error( message, e ); throw e; } } } + private PrepareStoreCopyRequestHandler storeListingRequestHandler( CatchupServerProtocol protocol, Supplier checkPointerSupplier, + StoreCopyCheckPointMutex storeCopyCheckPointMutex, Supplier dataSourceSupplier, PageCache pageCache, FileSystemAbstraction fs ) + { + PrepareStoreCopyFilesProvider prepareStoreCopyFilesProvider = new PrepareStoreCopyFilesProvider( pageCache, fs ); + return new PrepareStoreCopyRequestHandler( protocol, checkPointerSupplier, storeCopyCheckPointMutex, dataSourceSupplier, + prepareStoreCopyFilesProvider ); + } + private ChannelInboundHandler decoders( CatchupServerProtocol protocol ) { RequestDecoderDispatcher decoderDispatcher = new RequestDecoderDispatcher<>( protocol, logProvider ); decoderDispatcher.register( State.TX_PULL, new TxPullRequestDecoder() ); - decoderDispatcher.register( State.GET_STORE, new GetStoreRequestDecoder() ); decoderDispatcher.register( State.GET_STORE_ID, new SimpleRequestDecoder( GetStoreIdRequest::new ) ); decoderDispatcher.register( State.GET_CORE_SNAPSHOT, new SimpleRequestDecoder( CoreSnapshotRequest::new ) ); + decoderDispatcher.register( State.PREPARE_STORE_COPY, new PrepareStoreCopyRequestDecoder() ); + decoderDispatcher.register( State.GET_STORE_FILE, new GetStoreFileRequest.Decoder() ); + decoderDispatcher.register( State.GET_INDEX_SNAPSHOT, new GetIndexFilesRequest.Decoder() ); return decoderDispatcher; } diff --git a/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/CatchupServerProtocol.java b/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/CatchupServerProtocol.java index be271650079d2..8efdc5d32824d 100644 --- a/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/CatchupServerProtocol.java +++ b/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/CatchupServerProtocol.java @@ -28,6 +28,12 @@ public CatchupServerProtocol() public enum State { - MESSAGE_TYPE, GET_STORE, GET_STORE_ID, GET_CORE_SNAPSHOT, TX_PULL + MESSAGE_TYPE, + PREPARE_STORE_COPY, + GET_STORE_ID, + GET_CORE_SNAPSHOT, + GET_STORE_FILE, + GET_INDEX_SNAPSHOT, + TX_PULL } } diff --git a/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/ClientMessageTypeHandler.java b/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/ClientMessageTypeHandler.java index 117509f1f624f..69dae72d2aa22 100644 --- a/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/ClientMessageTypeHandler.java +++ b/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/ClientMessageTypeHandler.java @@ -50,26 +50,32 @@ public void channelRead( ChannelHandlerContext ctx, Object msg ) switch ( responseMessageType ) { - case STORE_ID: - protocol.expect( CatchupClientProtocol.State.STORE_ID ); - break; - case TX: - protocol.expect( CatchupClientProtocol.State.TX_PULL_RESPONSE ); - break; - case FILE: - protocol.expect( CatchupClientProtocol.State.FILE_HEADER ); - break; - case STORE_COPY_FINISHED: - protocol.expect( CatchupClientProtocol.State.STORE_COPY_FINISHED ); - break; - case CORE_SNAPSHOT: - protocol.expect( CatchupClientProtocol.State.CORE_SNAPSHOT ); - break; - case TX_STREAM_FINISHED: - protocol.expect( CatchupClientProtocol.State.TX_STREAM_FINISHED ); - break; - default: - log.warn( "No handler found for message type %s (%d)", responseMessageType.name(), byteValue ); + case STORE_ID: + protocol.expect( CatchupClientProtocol.State.STORE_ID ); + break; + case TX: + protocol.expect( CatchupClientProtocol.State.TX_PULL_RESPONSE ); + break; + case FILE: + protocol.expect( CatchupClientProtocol.State.FILE_HEADER ); + break; + case STORE_COPY_FINISHED: + protocol.expect( CatchupClientProtocol.State.STORE_COPY_FINISHED ); + break; + case CORE_SNAPSHOT: + protocol.expect( CatchupClientProtocol.State.CORE_SNAPSHOT ); + break; + case TX_STREAM_FINISHED: + protocol.expect( CatchupClientProtocol.State.TX_STREAM_FINISHED ); + break; + case PREPARE_STORE_COPY_RESPONSE: + protocol.expect( CatchupClientProtocol.State.PREPARE_STORE_COPY_RESPONSE ); + break; + case INDEX_SNAPSHOT_RESPONSE: + protocol.expect( CatchupClientProtocol.State.INDEX_SNAPSHOT_RESPONSE ); + break; + default: + log.warn( "No handler found for message type %s (%d)", responseMessageType.name(), byteValue ); } ReferenceCountUtil.release( msg ); diff --git a/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/RequestDecoderDispatcher.java b/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/RequestDecoderDispatcher.java index 1e78332cf8053..2efe069e910a7 100644 --- a/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/RequestDecoderDispatcher.java +++ b/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/RequestDecoderDispatcher.java @@ -30,13 +30,13 @@ import org.neo4j.logging.Log; import org.neo4j.logging.LogProvider; -class RequestDecoderDispatcher> extends ChannelInboundHandlerAdapter +public class RequestDecoderDispatcher> extends ChannelInboundHandlerAdapter { private final Map decoders = new HashMap<>(); private final Protocol protocol; private final Log log; - RequestDecoderDispatcher( Protocol protocol, LogProvider logProvider ) + public RequestDecoderDispatcher( Protocol protocol, LogProvider logProvider ) { this.protocol = protocol; this.log = logProvider.getLog( getClass() ); diff --git a/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/RequestMessageType.java b/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/RequestMessageType.java index 05c18d4f7d151..03273db9dcb56 100644 --- a/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/RequestMessageType.java +++ b/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/RequestMessageType.java @@ -26,9 +26,11 @@ public enum RequestMessageType implements Message { TX_PULL_REQUEST( (byte) 1 ), - STORE( (byte) 2 ), + PREPARE_STORE_COPY( (byte) 2 ), CORE_SNAPSHOT( (byte) 3 ), STORE_ID( (byte) 4 ), + STORE_FILE( (byte) 6 ), + INDEX_SNAPSHOT( (byte) 7 ), UNKNOWN( (byte) 404 ); private byte messageType; diff --git a/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/ResponseMessageType.java b/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/ResponseMessageType.java index 23504a728b8e1..37e3494053c75 100644 --- a/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/ResponseMessageType.java +++ b/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/ResponseMessageType.java @@ -29,7 +29,9 @@ public enum ResponseMessageType STORE_COPY_FINISHED( (byte) 4 ), CORE_SNAPSHOT( (byte) 5 ), TX_STREAM_FINISHED( (byte) 6 ), - UNKNOWN( (byte) 200 ),; + PREPARE_STORE_COPY_RESPONSE( (byte) 7 ), + INDEX_SNAPSHOT_RESPONSE( (byte) 8 ), + UNKNOWN( (byte) 200 ); private byte messageType; diff --git a/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/ServerMessageTypeHandler.java b/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/ServerMessageTypeHandler.java index 44ecca4f58c5c..298ed697ab6a4 100644 --- a/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/ServerMessageTypeHandler.java +++ b/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/ServerMessageTypeHandler.java @@ -28,12 +28,12 @@ import org.neo4j.logging.LogProvider; -class ServerMessageTypeHandler extends ChannelInboundHandlerAdapter +public class ServerMessageTypeHandler extends ChannelInboundHandlerAdapter { private final Log log; private final CatchupServerProtocol protocol; - ServerMessageTypeHandler( CatchupServerProtocol protocol, LogProvider logProvider ) + public ServerMessageTypeHandler( CatchupServerProtocol protocol, LogProvider logProvider ) { this.protocol = protocol; this.log = logProvider.getLog( getClass() ); @@ -50,10 +50,6 @@ public void channelRead( ChannelHandlerContext ctx, Object msg ) { protocol.expect( CatchupServerProtocol.State.TX_PULL ); } - else if ( requestMessageType.equals( RequestMessageType.STORE ) ) - { - protocol.expect( CatchupServerProtocol.State.GET_STORE ); - } else if ( requestMessageType.equals( RequestMessageType.STORE_ID ) ) { protocol.expect( CatchupServerProtocol.State.GET_STORE_ID ); @@ -62,6 +58,18 @@ else if ( requestMessageType.equals( RequestMessageType.CORE_SNAPSHOT ) ) { protocol.expect( CatchupServerProtocol.State.GET_CORE_SNAPSHOT ); } + else if ( requestMessageType.equals( RequestMessageType.PREPARE_STORE_COPY ) ) + { + protocol.expect( CatchupServerProtocol.State.PREPARE_STORE_COPY ); + } + else if ( requestMessageType.equals( RequestMessageType.STORE_FILE ) ) + { + protocol.expect( CatchupServerProtocol.State.GET_STORE_FILE ); + } + else if ( requestMessageType.equals( RequestMessageType.INDEX_SNAPSHOT ) ) + { + protocol.expect( CatchupServerProtocol.State.GET_INDEX_SNAPSHOT ); + } else { log.warn( "No handler found for message type %s", requestMessageType ); diff --git a/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/SimpleRequestDecoder.java b/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/SimpleRequestDecoder.java index 087d0a5f4618b..76f3e64766c5e 100644 --- a/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/SimpleRequestDecoder.java +++ b/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/SimpleRequestDecoder.java @@ -33,11 +33,11 @@ * {@link io.netty.handler.codec.ByteToMessageDecoder} instead the decode method would fail as no * bytes are consumed from the ByteBuf but an object is added in the out list. */ -class SimpleRequestDecoder extends MessageToMessageDecoder +public class SimpleRequestDecoder extends MessageToMessageDecoder { private Factory factory; - SimpleRequestDecoder( Factory factory ) + public SimpleRequestDecoder( Factory factory ) { this.factory = factory; } diff --git a/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/StoreListingResponseHandler.java b/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/StoreListingResponseHandler.java new file mode 100644 index 0000000000000..5bc782bc88499 --- /dev/null +++ b/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/StoreListingResponseHandler.java @@ -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 Affero 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 Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + */ +package org.neo4j.causalclustering.catchup; + +import io.netty.channel.ChannelHandlerContext; +import io.netty.channel.SimpleChannelInboundHandler; + +import org.neo4j.causalclustering.catchup.storecopy.PrepareStoreCopyResponse; + +public class StoreListingResponseHandler extends SimpleChannelInboundHandler +{ + private final CatchupClientProtocol protocol; + private final CatchUpResponseHandler handler; + + public StoreListingResponseHandler( CatchupClientProtocol protocol, + CatchUpResponseHandler handler ) + { + this.protocol = protocol; + this.handler = handler; + } + + @Override + protected void channelRead0( ChannelHandlerContext ctx, final PrepareStoreCopyResponse msg ) throws Exception + { + handler.onStoreListingResponse( msg ); + protocol.expect( CatchupClientProtocol.State.MESSAGE_TYPE ); + } +} + diff --git a/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/TrackingResponseHandler.java b/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/TrackingResponseHandler.java index 3d1ce7e48c2f5..5bb0db210c335 100644 --- a/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/TrackingResponseHandler.java +++ b/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/TrackingResponseHandler.java @@ -28,6 +28,7 @@ import org.neo4j.causalclustering.catchup.storecopy.FileHeader; import org.neo4j.causalclustering.catchup.storecopy.GetStoreIdResponse; import org.neo4j.causalclustering.catchup.storecopy.StoreCopyFinishedResponse; +import org.neo4j.causalclustering.catchup.storecopy.PrepareStoreCopyResponse; import org.neo4j.causalclustering.catchup.tx.TxPullResponse; import org.neo4j.causalclustering.catchup.tx.TxStreamFinishedResponse; import org.neo4j.causalclustering.core.state.snapshot.CoreSnapshot; @@ -125,6 +126,16 @@ public void onCoreSnapshot( CoreSnapshot coreSnapshot ) } } + @Override + public void onStoreListingResponse( PrepareStoreCopyResponse storeListingRequest ) + { + if ( !requestOutcomeSignal.isCancelled() ) + { + recordLastResponse(); + delegate.onStoreListingResponse( requestOutcomeSignal, storeListingRequest ); + } + } + Optional lastResponseTime() { return Optional.ofNullable( lastResponseTime ); diff --git a/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/storecopy/CloseablesListener.java b/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/storecopy/CloseablesListener.java new file mode 100644 index 0000000000000..b2a5668e14cd4 --- /dev/null +++ b/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/storecopy/CloseablesListener.java @@ -0,0 +1,55 @@ +/* + * 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 Affero 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 Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + */ +package org.neo4j.causalclustering.catchup.storecopy; + +import io.netty.util.concurrent.Future; +import io.netty.util.concurrent.GenericFutureListener; + +import java.util.ArrayList; +import java.util.List; + +import org.neo4j.io.IOUtils; + +class CloseablesListener implements AutoCloseable, GenericFutureListener> +{ + private final List closeables = new ArrayList<>(); + + T add( T closeable ) + { + if ( closeable == null ) + { + throw new IllegalArgumentException( "closeable cannot be null!" ); + } + closeables.add( closeable ); + return closeable; + } + + @Override + public void close() + { + IOUtils.closeAll( RuntimeException.class, closeables.toArray( new AutoCloseable[closeables.size()] ) ); + } + + @Override + public void operationComplete( Future future ) + { + close(); + } +} diff --git a/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/storecopy/DataSourceChecks.java b/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/storecopy/DataSourceChecks.java new file mode 100644 index 0000000000000..c23d345f9d29b --- /dev/null +++ b/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/storecopy/DataSourceChecks.java @@ -0,0 +1,64 @@ +/* + * 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 Affero 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 Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + */ +package org.neo4j.causalclustering.catchup.storecopy; + +import java.io.IOException; + +import org.neo4j.causalclustering.identity.StoreId; +import org.neo4j.kernel.NeoStoreDataSource; +import org.neo4j.kernel.impl.transaction.log.checkpoint.CheckPointer; +import org.neo4j.kernel.impl.transaction.log.checkpoint.SimpleTriggerInfo; + +class DataSourceChecks +{ + private DataSourceChecks() + { + } + + static boolean isTransactionWithinReach( long requiredTxId, CheckPointer checkpointer ) + { + if ( isWithinLastCheckPoint( requiredTxId, checkpointer ) ) + { + return true; + } + else + { + try + { + checkpointer.tryCheckPoint( new SimpleTriggerInfo( "Store file copy" ) ); + return isWithinLastCheckPoint( requiredTxId, checkpointer ); + } + catch ( IOException e ) + { + return false; + } + } + } + + private static boolean isWithinLastCheckPoint( long atLeast, CheckPointer checkPointer ) + { + return checkPointer.lastCheckPointedTransactionId() >= atLeast; + } + + static boolean hasSameStoreId( StoreId storeId, NeoStoreDataSource dataSource ) + { + return storeId.equalToKernelStoreId( dataSource.getStoreId() ); + } +} diff --git a/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/storecopy/FileChunk.java b/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/storecopy/FileChunk.java index 14438fe97420b..98eff488e6110 100644 --- a/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/storecopy/FileChunk.java +++ b/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/storecopy/FileChunk.java @@ -45,7 +45,7 @@ static FileChunk create( byte[] bytes, boolean last ) return new FileChunk( last ? bytes.length : USE_MAX_SIZE_AND_EXPECT_MORE_CHUNKS, bytes ); } - private FileChunk( int encodedLength, byte[] bytes ) + FileChunk( int encodedLength, byte[] bytes ) { this.encodedLength = encodedLength; this.bytes = bytes; diff --git a/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/storecopy/GetIndexFilesRequest.java b/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/storecopy/GetIndexFilesRequest.java new file mode 100644 index 0000000000000..5bb21c51dce48 --- /dev/null +++ b/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/storecopy/GetIndexFilesRequest.java @@ -0,0 +1,114 @@ +/* + * 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 Affero 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 Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + */ +package org.neo4j.causalclustering.catchup.storecopy; + +import io.netty.buffer.ByteBuf; +import io.netty.channel.ChannelHandlerContext; +import io.netty.handler.codec.ByteToMessageDecoder; +import io.netty.handler.codec.MessageToByteEncoder; + +import java.io.IOException; +import java.util.List; + +import org.neo4j.causalclustering.catchup.RequestMessageType; +import org.neo4j.causalclustering.core.state.storage.SafeChannelMarshal; +import org.neo4j.causalclustering.identity.StoreId; +import org.neo4j.causalclustering.messaging.CatchUpRequest; +import org.neo4j.causalclustering.messaging.EndOfStreamException; +import org.neo4j.causalclustering.messaging.NetworkFlushableByteBuf; +import org.neo4j.causalclustering.messaging.NetworkReadableClosableChannelNetty4; +import org.neo4j.causalclustering.messaging.marshalling.storeid.StoreIdMarshal; +import org.neo4j.kernel.api.schema.index.IndexDescriptor; +import org.neo4j.storageengine.api.ReadableChannel; +import org.neo4j.storageengine.api.WritableChannel; + +public class GetIndexFilesRequest implements CatchUpRequest +{ + private final StoreId expectedStoreId; + private final IndexDescriptor descriptor; + private final long lastTransactionId; + + public GetIndexFilesRequest( StoreId expectedStoreId, IndexDescriptor indexDescriptor, long lastTransactionId ) + { + this.expectedStoreId = expectedStoreId; + this.descriptor = indexDescriptor; + this.lastTransactionId = lastTransactionId; + } + + public StoreId expectedStoreId() + { + return expectedStoreId; + } + + public long requiredTransactionId() + { + return lastTransactionId; + } + + public IndexDescriptor descriptor() + { + return descriptor; + } + + @Override + public RequestMessageType messageType() + { + return RequestMessageType.INDEX_SNAPSHOT; + } + + static class IndexSnapshotRequestMarshall extends SafeChannelMarshal + { + @Override + protected GetIndexFilesRequest unmarshal0( ReadableChannel channel ) throws IOException, EndOfStreamException + { + StoreId storeId = StoreIdMarshal.INSTANCE.unmarshal( channel ); + long requiredTransactionId = channel.getLong(); + IndexDescriptor indexDescriptor = IndexDescriptorSerializer.deserialize( channel ); + return new GetIndexFilesRequest( storeId, indexDescriptor, requiredTransactionId ); + } + + @Override + public void marshal( GetIndexFilesRequest getIndexFilesRequest, WritableChannel channel ) throws IOException + { + StoreIdMarshal.INSTANCE.marshal( getIndexFilesRequest.expectedStoreId(), channel ); + channel.putLong( getIndexFilesRequest.requiredTransactionId() ); + IndexDescriptorSerializer.serialize( getIndexFilesRequest.descriptor(), channel ); + } + } + + public static class Encoder extends MessageToByteEncoder + { + @Override + protected void encode( ChannelHandlerContext ctx, GetIndexFilesRequest msg, ByteBuf out ) throws Exception + { + new IndexSnapshotRequestMarshall().marshal( msg, new NetworkFlushableByteBuf( out ) ); + } + } + + public static class Decoder extends ByteToMessageDecoder + { + @Override + protected void decode( ChannelHandlerContext ctx, ByteBuf in, List out ) throws Exception + { + GetIndexFilesRequest getIndexFilesRequest = new IndexSnapshotRequestMarshall().unmarshal0( new NetworkReadableClosableChannelNetty4( in ) ); + out.add( getIndexFilesRequest ); + } + } +} diff --git a/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/storecopy/GetIndexSnapshotRequestHandler.java b/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/storecopy/GetIndexSnapshotRequestHandler.java new file mode 100644 index 0000000000000..1e7127b12f6ae --- /dev/null +++ b/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/storecopy/GetIndexSnapshotRequestHandler.java @@ -0,0 +1,102 @@ +/* + * 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 Affero 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 Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + */ +package org.neo4j.causalclustering.catchup.storecopy; + +import io.netty.channel.ChannelHandlerContext; +import io.netty.channel.SimpleChannelInboundHandler; + +import java.io.File; +import java.io.IOException; +import java.util.function.Supplier; + +import org.neo4j.causalclustering.catchup.CatchupServerProtocol; +import org.neo4j.graphdb.ResourceIterator; +import org.neo4j.io.fs.FileSystemAbstraction; +import org.neo4j.io.pagecache.PageCache; +import org.neo4j.kernel.NeoStoreDataSource; +import org.neo4j.kernel.api.schema.index.IndexDescriptor; +import org.neo4j.kernel.impl.transaction.log.checkpoint.CheckPointer; +import org.neo4j.storageengine.api.StoreFileMetadata; + +import static org.neo4j.causalclustering.catchup.storecopy.DataSourceChecks.hasSameStoreId; +import static org.neo4j.causalclustering.catchup.storecopy.DataSourceChecks.isTransactionWithinReach; +import static org.neo4j.io.fs.FileUtils.relativePath; + +public class GetIndexSnapshotRequestHandler extends SimpleChannelInboundHandler +{ + private final CatchupServerProtocol protocol; + private final Supplier dataSource; + private final Supplier checkpointerSupplier; + private final StoreFileStreamingProtocol storeFileStreamingProtocol; + private final PageCache pageCache; + private final FileSystemAbstraction fs; + + public GetIndexSnapshotRequestHandler( CatchupServerProtocol protocol, Supplier dataSource, + Supplier checkpointerSupplier, StoreFileStreamingProtocol storeFileStreamingProtocol, PageCache pageCache, FileSystemAbstraction fs ) + { + this.protocol = protocol; + this.dataSource = dataSource; + this.checkpointerSupplier = checkpointerSupplier; + this.storeFileStreamingProtocol = storeFileStreamingProtocol; + this.pageCache = pageCache; + this.fs = fs; + } + + @Override + protected void channelRead0( ChannelHandlerContext ctx, GetIndexFilesRequest snapshotRequest ) throws IOException + { + CloseablesListener closeablesListener = new CloseablesListener(); + NeoStoreDataSource neoStoreDataSource = dataSource.get(); + if ( !hasSameStoreId( snapshotRequest.expectedStoreId(), neoStoreDataSource ) ) + { + storeFileStreamingProtocol.end( ctx, StoreCopyFinishedResponse.Status.E_STORE_ID_MISMATCH ); + protocol.expect( CatchupServerProtocol.State.MESSAGE_TYPE ); + } + else if ( !isTransactionWithinReach( snapshotRequest.requiredTransactionId(), checkpointerSupplier.get() ) ) + { + storeFileStreamingProtocol.end( ctx, StoreCopyFinishedResponse.Status.E_TOO_FAR_BEHIND ); + } + else + { + StoreCopyFinishedResponse.Status status = StoreCopyFinishedResponse.Status.E_UNKNOWN; + File storeDir = neoStoreDataSource.getStoreDir(); + ResourceIterator resourceIterator = + neoStoreDataSource.getNeoStoreFileListing().getNeoStoreFileIndexListing().getSnapshot( snapshotRequest.descriptor() ); + try + { + closeablesListener.add( resourceIterator ); + while ( resourceIterator.hasNext() ) + { + StoreFileMetadata storeFileMetadata = resourceIterator.next(); + File file = storeFileMetadata.file(); + String relativePath = relativePath( storeDir, file ); + int recordSize = storeFileMetadata.recordSize(); + storeFileStreamingProtocol.stream( ctx, new StoreResource( file, relativePath, recordSize, pageCache, fs ) ); + } + status = StoreCopyFinishedResponse.Status.SUCCESS; + } + finally + { + storeFileStreamingProtocol.end( ctx, status ).addListener( closeablesListener ); + protocol.expect( CatchupServerProtocol.State.MESSAGE_TYPE ); + } + } + } +} diff --git a/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/storecopy/GetStoreFileRequest.java b/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/storecopy/GetStoreFileRequest.java new file mode 100644 index 0000000000000..78dbf8731b088 --- /dev/null +++ b/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/storecopy/GetStoreFileRequest.java @@ -0,0 +1,120 @@ +/* + * 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 Affero 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 Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + */ +package org.neo4j.causalclustering.catchup.storecopy; + +import io.netty.buffer.ByteBuf; +import io.netty.channel.ChannelHandlerContext; +import io.netty.handler.codec.ByteToMessageDecoder; +import io.netty.handler.codec.MessageToByteEncoder; + +import java.io.File; +import java.io.IOException; +import java.util.List; + +import org.neo4j.causalclustering.catchup.RequestMessageType; +import org.neo4j.causalclustering.core.state.storage.SafeChannelMarshal; +import org.neo4j.causalclustering.identity.StoreId; +import org.neo4j.causalclustering.messaging.CatchUpRequest; +import org.neo4j.causalclustering.messaging.EndOfStreamException; +import org.neo4j.causalclustering.messaging.NetworkFlushableByteBuf; +import org.neo4j.causalclustering.messaging.NetworkReadableClosableChannelNetty4; +import org.neo4j.causalclustering.messaging.marshalling.storeid.StoreIdMarshal; +import org.neo4j.storageengine.api.ReadableChannel; +import org.neo4j.storageengine.api.WritableChannel; +import org.neo4j.string.UTF8; + +public class GetStoreFileRequest implements CatchUpRequest +{ + private final StoreId expectedStoreId; + private final File file; + private final long requiredTransactionId; + + public GetStoreFileRequest( StoreId expectedStoreId, File file, long requiredTransactionId ) + { + this.expectedStoreId = expectedStoreId; + this.file = file; + this.requiredTransactionId = requiredTransactionId; + } + + long requiredTransactionId() + { + return requiredTransactionId; + } + + StoreId expectedStoreId() + { + return expectedStoreId; + } + + File file() + { + return file; + } + + @Override + public RequestMessageType messageType() + { + return RequestMessageType.STORE_FILE; + } + + static class StoreFileRequestMarshall extends SafeChannelMarshal + { + @Override + protected GetStoreFileRequest unmarshal0( ReadableChannel channel ) throws IOException, EndOfStreamException + { + StoreId storeId = StoreIdMarshal.INSTANCE.unmarshal( channel ); + long requiredTransactionId = channel.getLong(); + int fileNameLength = channel.getInt(); + byte[] fileNameBytes = new byte[fileNameLength]; + channel.get( fileNameBytes, fileNameLength ); + return new GetStoreFileRequest( storeId, new File( UTF8.decode( fileNameBytes ) ), requiredTransactionId ); + } + + @Override + public void marshal( GetStoreFileRequest getStoreFileRequest, WritableChannel channel ) throws IOException + { + StoreIdMarshal.INSTANCE.marshal( getStoreFileRequest.expectedStoreId(), channel ); + channel.putLong( getStoreFileRequest.requiredTransactionId() ); + String name = getStoreFileRequest.file().getName(); + channel.putInt( name.length() ); + channel.put( UTF8.encode( name ), name.length() ); + } + } + + public static class Encoder extends MessageToByteEncoder + { + @Override + protected void encode( ChannelHandlerContext ctx, GetStoreFileRequest msg, ByteBuf out ) throws Exception + { + new GetStoreFileRequest.StoreFileRequestMarshall().marshal( msg, new NetworkFlushableByteBuf( out ) ); + } + } + + public static class Decoder extends ByteToMessageDecoder + { + @Override + protected void decode( ChannelHandlerContext ctx, ByteBuf in, List out ) throws Exception + { + GetStoreFileRequest getStoreFileRequest = + new GetStoreFileRequest.StoreFileRequestMarshall().unmarshal0( new NetworkReadableClosableChannelNetty4( in ) ); + out.add( getStoreFileRequest ); + } + } +} diff --git a/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/storecopy/GetStoreFileRequestHandler.java b/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/storecopy/GetStoreFileRequestHandler.java new file mode 100644 index 0000000000000..b9306efeb36e1 --- /dev/null +++ b/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/storecopy/GetStoreFileRequestHandler.java @@ -0,0 +1,116 @@ +/* + * 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 Affero 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 Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + */ +package org.neo4j.causalclustering.catchup.storecopy; + +import io.netty.channel.ChannelHandlerContext; +import io.netty.channel.SimpleChannelInboundHandler; + +import java.io.File; +import java.io.IOException; +import java.util.List; +import java.util.function.Predicate; +import java.util.function.Supplier; +import java.util.stream.Collectors; + +import org.neo4j.causalclustering.catchup.CatchupServerProtocol; +import org.neo4j.graphdb.ResourceIterator; +import org.neo4j.io.fs.FileSystemAbstraction; +import org.neo4j.io.pagecache.PageCache; +import org.neo4j.kernel.NeoStoreDataSource; +import org.neo4j.kernel.impl.transaction.log.checkpoint.CheckPointer; +import org.neo4j.logging.Log; +import org.neo4j.logging.LogProvider; +import org.neo4j.storageengine.api.StoreFileMetadata; + +import static java.lang.String.format; +import static org.neo4j.causalclustering.catchup.storecopy.DataSourceChecks.hasSameStoreId; +import static org.neo4j.causalclustering.catchup.storecopy.DataSourceChecks.isTransactionWithinReach; +import static org.neo4j.io.fs.FileUtils.relativePath; + +public class GetStoreFileRequestHandler extends SimpleChannelInboundHandler +{ + private final CatchupServerProtocol protocol; + private final Supplier dataSource; + private final Supplier checkpointerSupplier; + private final StoreFileStreamingProtocol storeFileStreamingProtocol; + private final PageCache pageCache; + private final FileSystemAbstraction fs; + private final Log log; + + public GetStoreFileRequestHandler( CatchupServerProtocol protocol, Supplier dataSource, Supplier checkpointerSupplier, + StoreFileStreamingProtocol storeFileStreamingProtocol, PageCache pageCache, FileSystemAbstraction fs, LogProvider logProvider ) + { + this.protocol = protocol; + this.dataSource = dataSource; + this.checkpointerSupplier = checkpointerSupplier; + this.storeFileStreamingProtocol = storeFileStreamingProtocol; + this.pageCache = pageCache; + this.fs = fs; + this.log = logProvider.getLog( GetStoreFileRequestHandler.class ); + } + + @Override + protected void channelRead0( ChannelHandlerContext ctx, GetStoreFileRequest fileRequest ) throws Exception + { + log.debug( "Requesting file %s", fileRequest.file() ); + NeoStoreDataSource neoStoreDataSource = dataSource.get(); + if ( !hasSameStoreId( fileRequest.expectedStoreId(), neoStoreDataSource ) ) + { + storeFileStreamingProtocol.end( ctx, StoreCopyFinishedResponse.Status.E_STORE_ID_MISMATCH ); + protocol.expect( CatchupServerProtocol.State.MESSAGE_TYPE ); + } + else if ( !isTransactionWithinReach( fileRequest.requiredTransactionId(), checkpointerSupplier.get() ) ) + { + storeFileStreamingProtocol.end( ctx, StoreCopyFinishedResponse.Status.E_TOO_FAR_BEHIND ); + } + else + { + File storeDir = neoStoreDataSource.getStoreDir(); + StoreFileMetadata storeFileMetadata = findFile( fileRequest.file().getName() ); + storeFileStreamingProtocol.stream( ctx, + new StoreResource( storeFileMetadata.file(), relativePath( storeDir, storeFileMetadata.file() ), storeFileMetadata.recordSize(), pageCache, + fs ) ); + storeFileStreamingProtocol.end( ctx, StoreCopyFinishedResponse.Status.SUCCESS ); + protocol.expect( CatchupServerProtocol.State.MESSAGE_TYPE ); + } + } + + private StoreFileMetadata findFile( String fileName ) throws IOException + { + try ( ResourceIterator resourceIterator = dataSource.get().listStoreFiles( false ) ) + { + return onlyOne( resourceIterator.stream().filter( matchesRequested( fileName ) ).collect( Collectors.toList() ), fileName ); + } + } + + private StoreFileMetadata onlyOne( List files, String description ) + { + if ( files.size() != 1 ) + { + throw new IllegalStateException( format( "Expected exactly one file '%s'. Got %d", description, files.size() ) ); + } + return files.get( 0 ); + } + + private static Predicate matchesRequested( String fileName ) + { + return f -> f.file().getName().equals( fileName ); + } +} diff --git a/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/storecopy/GetStoreRequestHandler.java b/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/storecopy/GetStoreRequestHandler.java deleted file mode 100644 index a33f30d11e4f0..0000000000000 --- a/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/storecopy/GetStoreRequestHandler.java +++ /dev/null @@ -1,60 +0,0 @@ -/* - * 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 Affero 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 Affero General Public License for more details. - * - * You should have received a copy of the GNU Affero General Public License - * along with this program. If not, see . - */ -package org.neo4j.causalclustering.catchup.storecopy; - -import io.netty.channel.ChannelHandlerContext; -import io.netty.channel.SimpleChannelInboundHandler; - -import java.util.function.Supplier; - -import org.neo4j.causalclustering.catchup.CatchupServerProtocol; -import org.neo4j.kernel.NeoStoreDataSource; - -import static org.neo4j.causalclustering.catchup.CatchupServerProtocol.State; -import static org.neo4j.causalclustering.catchup.storecopy.StoreCopyFinishedResponse.Status.E_STORE_ID_MISMATCH; - -public class GetStoreRequestHandler extends SimpleChannelInboundHandler -{ - private final CatchupServerProtocol protocol; - private final Supplier dataSource; - - private final StoreStreamingProcess storeStreamingProcess; - - public GetStoreRequestHandler( CatchupServerProtocol protocol, Supplier dataSource, StoreStreamingProcess storeStreamingProcess ) - { - this.protocol = protocol; - this.dataSource = dataSource; - this.storeStreamingProcess = storeStreamingProcess; - } - - @Override - protected void channelRead0( ChannelHandlerContext ctx, GetStoreRequest msg ) throws Exception - { - if ( !msg.expectedStoreId().equalToKernelStoreId( dataSource.get().getStoreId() ) ) - { - storeStreamingProcess.fail( ctx, E_STORE_ID_MISMATCH ); - } - else - { - storeStreamingProcess.perform( ctx ); - } - protocol.expect( State.MESSAGE_TYPE ); - } -} diff --git a/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/storecopy/IndexDescriptorSerializer.java b/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/storecopy/IndexDescriptorSerializer.java new file mode 100644 index 0000000000000..81711b99aa8a4 --- /dev/null +++ b/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/storecopy/IndexDescriptorSerializer.java @@ -0,0 +1,74 @@ +/* + * 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 Affero 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 Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + */ +package org.neo4j.causalclustering.catchup.storecopy; + +import java.io.IOException; + +import org.neo4j.kernel.api.schema.LabelSchemaDescriptor; +import org.neo4j.kernel.api.schema.index.IndexDescriptor; +import org.neo4j.storageengine.api.ReadableChannel; +import org.neo4j.storageengine.api.WritableChannel; + +class IndexDescriptorSerializer +{ + private static final byte LABEL_SCHEMA_DESCRIPTOR = 1; + + static void serialize( IndexDescriptor indexDescriptor, WritableChannel channel ) throws IOException + { + org.neo4j.internal.kernel.api.schema.LabelSchemaDescriptor schemaDescriptor = indexDescriptor.schema(); + if ( schemaDescriptor instanceof LabelSchemaDescriptor ) + { + channel.put( LABEL_SCHEMA_DESCRIPTOR ); + LabelSchemaDescriptor labelSchemaDescriptor = (LabelSchemaDescriptor) schemaDescriptor; + channel.putInt( labelSchemaDescriptor.getLabelId() ); + channel.putInt( labelSchemaDescriptor.getPropertyIds().length ); + for ( int propertyId : labelSchemaDescriptor.getPropertyIds() ) + { + channel.putInt( propertyId ); + } + channel.putInt( indexDescriptor.type().ordinal() ); + } + else + { + throw new IllegalArgumentException( "Not a recognized LabelSchemaDescriptor class: " + indexDescriptor.schema().getClass() ); + } + } + + static IndexDescriptor deserialize( ReadableChannel channel ) throws IOException + { + byte schemaDescriptor = channel.get(); + + switch ( schemaDescriptor ) + { + case LABEL_SCHEMA_DESCRIPTOR: + int labelId = channel.getInt(); + int length = channel.getInt(); + int[] propertyIds = new int[length]; + for ( int i = 0; i < length; i++ ) + { + propertyIds[i] = channel.getInt(); + } + IndexDescriptor.Type type = IndexDescriptor.Type.values()[channel.getInt()]; + return new IndexDescriptor( new LabelSchemaDescriptor( labelId, propertyIds ), type ); + default: + throw new IllegalStateException( "Unhandled schema descriptor: " + schemaDescriptor ); + } + } +} diff --git a/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/storecopy/MaximumTotalRetries.java b/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/storecopy/MaximumTotalRetries.java new file mode 100644 index 0000000000000..cc57d2ce4175f --- /dev/null +++ b/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/storecopy/MaximumTotalRetries.java @@ -0,0 +1,64 @@ +/* + * 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 Affero 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 Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + */ +package org.neo4j.causalclustering.catchup.storecopy; + +import java.time.Clock; + +public class MaximumTotalRetries implements TerminationCondition +{ + private final int maxRetries; + private final long allowedInBetweenTimeMillis; + private final Clock clock; + private int tries; + private long previousCheck; + + MaximumTotalRetries( int maxRetries, long allowedInBetweenTimeMillis ) + { + this( maxRetries, allowedInBetweenTimeMillis, Clock.systemUTC() ); + } + + MaximumTotalRetries( int maxRetries, long allowedInBetweenTimeMillis, Clock clock ) + { + this.clock = clock; + this.maxRetries = maxRetries; + this.allowedInBetweenTimeMillis = allowedInBetweenTimeMillis; + this.previousCheck = 0; + } + + @Override + public void assertContinue() throws StoreCopyFailedException + { + long currentTime = clock.millis(); + if ( timeHasExpired( previousCheck, currentTime ) ) + { + tries++; + previousCheck = currentTime; + } + if ( tries >= maxRetries ) + { + throw new StoreCopyFailedException( "Maximum allowed retries exceeded: " + maxRetries ); + } + } + + private boolean timeHasExpired( long previousCheck, long currentTime ) + { + return (currentTime - previousCheck) > allowedInBetweenTimeMillis; + } +} diff --git a/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/storecopy/PrepareStoreCopyFiles.java b/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/storecopy/PrepareStoreCopyFiles.java new file mode 100644 index 0000000000000..856c5ec799d87 --- /dev/null +++ b/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/storecopy/PrepareStoreCopyFiles.java @@ -0,0 +1,112 @@ +/* + * 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 Affero 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 Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + */ +package org.neo4j.causalclustering.catchup.storecopy; + +import java.io.File; +import java.io.IOException; +import java.util.Collection; +import java.util.function.Function; +import java.util.function.Predicate; +import java.util.stream.Stream; + +import org.neo4j.graphdb.ResourceIterator; +import org.neo4j.io.fs.FileSystemAbstraction; +import org.neo4j.io.pagecache.PageCache; +import org.neo4j.kernel.NeoStoreDataSource; +import org.neo4j.kernel.api.schema.index.IndexDescriptor; +import org.neo4j.kernel.impl.store.StoreType; +import org.neo4j.storageengine.api.StoreFileMetadata; + +import static org.neo4j.io.fs.FileUtils.relativePath; + +public class PrepareStoreCopyFiles implements AutoCloseable +{ + private final NeoStoreDataSource neoStoreDataSource; + private final PageCache pageCache; + private final FileSystemAbstraction fileSystemAbstraction; + private final CloseablesListener closeablesListener = new CloseablesListener(); + + PrepareStoreCopyFiles( NeoStoreDataSource neoStoreDataSource, PageCache pageCache, FileSystemAbstraction fileSystemAbstraction ) + { + this.neoStoreDataSource = neoStoreDataSource; + this.pageCache = pageCache; + this.fileSystemAbstraction = fileSystemAbstraction; + } + + IndexDescriptor[] getIndexDescriptors() + { + Collection descriptors = neoStoreDataSource.getNeoStoreFileListing().getNeoStoreFileIndexListing().listIndexDescriptors(); + return descriptors.toArray( new IndexDescriptor[descriptors.size()] ); + } + + StoreResource[] getAtomicFilesSnapshot() throws IOException + { + ResourceIterator neoStoreFilesIterator = + closeablesListener.add( neoStoreDataSource.getNeoStoreFileListing().builder().excludeAll().includeNeoStoreFiles().build() ); + ResourceIterator explicitIndexIterator = closeablesListener.add( + neoStoreDataSource.getNeoStoreFileListing().builder().excludeAll().includeExplicitIndexStoreStoreFiles().includeAdditionalProviders().build() ); + + return Stream.concat( neoStoreFilesIterator.stream().filter( isCountFile() ), explicitIndexIterator.stream() ).map( mapToStoreResource() ).toArray( + StoreResource[]::new ); + } + + private Function mapToStoreResource() + { + return storeFileMetadata -> + { + try + { + return toStoreResource( storeFileMetadata ); + } + catch ( IOException e ) + { + throw new IllegalStateException( "Unable to create store resource", e ); + } + }; + } + + File[] listReplayableFiles() throws IOException + { + try ( Stream stream = neoStoreDataSource.getNeoStoreFileListing().builder().excludeLogFiles() + .excludeExplicitIndexStoreFiles().excludeSchemaIndexStoreFiles().excludeAdditionalProviders().build().stream() ) + { + return stream.filter( isCountFile().negate() ).map( StoreFileMetadata::file ).toArray( File[]::new ); + } + } + + private static Predicate isCountFile() + { + return storeFileMetadata -> StoreType.typeOf( storeFileMetadata.file().getName() ).filter( f -> f == StoreType.COUNTS ).isPresent(); + } + + private StoreResource toStoreResource( StoreFileMetadata storeFileMetadata ) throws IOException + { + File storeDir = neoStoreDataSource.getStoreDir(); + File file = storeFileMetadata.file(); + String relativePath = relativePath( storeDir, file ); + return new StoreResource( file, relativePath, storeFileMetadata.recordSize(), pageCache, fileSystemAbstraction ); + } + + @Override + public void close() + { + closeablesListener.close(); + } +} diff --git a/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/storecopy/PrepareStoreCopyFilesProvider.java b/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/storecopy/PrepareStoreCopyFilesProvider.java new file mode 100644 index 0000000000000..69dee758b69b0 --- /dev/null +++ b/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/storecopy/PrepareStoreCopyFilesProvider.java @@ -0,0 +1,43 @@ +/* + * 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 Affero 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 Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + */ +package org.neo4j.causalclustering.catchup.storecopy; + +import java.util.function.Supplier; + +import org.neo4j.io.fs.FileSystemAbstraction; +import org.neo4j.io.pagecache.PageCache; +import org.neo4j.kernel.NeoStoreDataSource; + +public class PrepareStoreCopyFilesProvider +{ + private final PageCache pageCache; + private final FileSystemAbstraction fileSystemAbstraction; + + public PrepareStoreCopyFilesProvider( PageCache pageCache, FileSystemAbstraction fileSystemAbstraction ) + { + this.pageCache = pageCache; + this.fileSystemAbstraction = fileSystemAbstraction; + } + + PrepareStoreCopyFiles prepareStoreCopyFiles( NeoStoreDataSource neoStoreDataSource ) + { + return new PrepareStoreCopyFiles( neoStoreDataSource, pageCache, fileSystemAbstraction ); + } +} diff --git a/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/storecopy/GetStoreRequest.java b/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/storecopy/PrepareStoreCopyRequest.java similarity index 78% rename from enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/storecopy/GetStoreRequest.java rename to enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/storecopy/PrepareStoreCopyRequest.java index c9cdfcee7cd99..ff0f1d47bab3e 100644 --- a/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/storecopy/GetStoreRequest.java +++ b/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/storecopy/PrepareStoreCopyRequest.java @@ -23,23 +23,23 @@ import org.neo4j.causalclustering.identity.StoreId; import org.neo4j.causalclustering.messaging.CatchUpRequest; -public class GetStoreRequest implements CatchUpRequest +public class PrepareStoreCopyRequest implements CatchUpRequest { - private final StoreId expectedStoreId; + private final StoreId storeId; - GetStoreRequest( StoreId expectedStoreId ) + public PrepareStoreCopyRequest( StoreId expectedStoreId ) { - this.expectedStoreId = expectedStoreId; + this.storeId = expectedStoreId; } - @Override - public RequestMessageType messageType() + public StoreId getStoreId() { - return RequestMessageType.STORE; + return storeId; } - StoreId expectedStoreId() + @Override + public RequestMessageType messageType() { - return expectedStoreId; + return RequestMessageType.PREPARE_STORE_COPY; } } diff --git a/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/storecopy/GetStoreRequestDecoder.java b/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/storecopy/PrepareStoreCopyRequestDecoder.java similarity index 77% rename from enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/storecopy/GetStoreRequestDecoder.java rename to enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/storecopy/PrepareStoreCopyRequestDecoder.java index 724406487cad7..b8e9c69c832ca 100644 --- a/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/storecopy/GetStoreRequestDecoder.java +++ b/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/storecopy/PrepareStoreCopyRequestDecoder.java @@ -29,12 +29,12 @@ import org.neo4j.causalclustering.messaging.NetworkReadableClosableChannelNetty4; import org.neo4j.causalclustering.messaging.marshalling.storeid.StoreIdMarshal; -public class GetStoreRequestDecoder extends ByteToMessageDecoder +public class PrepareStoreCopyRequestDecoder extends ByteToMessageDecoder { @Override - protected void decode( ChannelHandlerContext ctx, ByteBuf msg, List out ) throws Exception + protected void decode( ChannelHandlerContext channelHandlerContext, ByteBuf byteBuf, List list ) throws Exception { - StoreId expectedStoreId = StoreIdMarshal.INSTANCE.unmarshal( new NetworkReadableClosableChannelNetty4( msg ) ); - out.add( new GetStoreRequest( expectedStoreId ) ); + StoreId storeId = StoreIdMarshal.INSTANCE.unmarshal( new NetworkReadableClosableChannelNetty4( byteBuf ) ); + list.add( new PrepareStoreCopyRequest( storeId ) ); } } diff --git a/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/storecopy/GetStoreRequestEncoder.java b/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/storecopy/PrepareStoreCopyRequestEncoder.java similarity index 75% rename from enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/storecopy/GetStoreRequestEncoder.java rename to enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/storecopy/PrepareStoreCopyRequestEncoder.java index c0ecdbda8a80a..9c3f24340a584 100644 --- a/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/storecopy/GetStoreRequestEncoder.java +++ b/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/storecopy/PrepareStoreCopyRequestEncoder.java @@ -26,11 +26,11 @@ import org.neo4j.causalclustering.messaging.NetworkFlushableChannelNetty4; import org.neo4j.causalclustering.messaging.marshalling.storeid.StoreIdMarshal; -public class GetStoreRequestEncoder extends MessageToByteEncoder +public class PrepareStoreCopyRequestEncoder extends MessageToByteEncoder { @Override - protected void encode( ChannelHandlerContext ctx, GetStoreRequest msg, ByteBuf out ) throws Exception + protected void encode( ChannelHandlerContext channelHandlerContext, PrepareStoreCopyRequest prepareStoreCopyRequest, ByteBuf byteBuf ) throws Exception { - StoreIdMarshal.INSTANCE.marshal( msg.expectedStoreId(), new NetworkFlushableChannelNetty4( out ) ); + StoreIdMarshal.INSTANCE.marshal( prepareStoreCopyRequest.getStoreId(), new NetworkFlushableChannelNetty4( byteBuf ) ); } } diff --git a/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/storecopy/PrepareStoreCopyRequestHandler.java b/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/storecopy/PrepareStoreCopyRequestHandler.java new file mode 100644 index 0000000000000..64b5510dc9ec9 --- /dev/null +++ b/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/storecopy/PrepareStoreCopyRequestHandler.java @@ -0,0 +1,109 @@ +/* + * 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 Affero 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 Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + */ +package org.neo4j.causalclustering.catchup.storecopy; + +import io.netty.channel.ChannelHandlerContext; +import io.netty.channel.SimpleChannelInboundHandler; + +import java.io.File; +import java.io.IOException; +import java.util.function.Supplier; + +import org.neo4j.causalclustering.catchup.CatchupServerProtocol; +import org.neo4j.causalclustering.catchup.ResponseMessageType; +import org.neo4j.graphdb.Resource; +import org.neo4j.kernel.NeoStoreDataSource; +import org.neo4j.kernel.api.index.SchemaIndexProvider; +import org.neo4j.kernel.api.schema.index.IndexDescriptor; +import org.neo4j.kernel.impl.transaction.log.checkpoint.CheckPointer; +import org.neo4j.kernel.impl.transaction.log.checkpoint.SimpleTriggerInfo; +import org.neo4j.kernel.impl.transaction.log.checkpoint.StoreCopyCheckPointMutex; + +import static org.neo4j.causalclustering.catchup.storecopy.DataSourceChecks.hasSameStoreId; + +public class PrepareStoreCopyRequestHandler extends SimpleChannelInboundHandler +{ + private final CatchupServerProtocol protocol; + private final Supplier checkPointerSupplier; + private final StoreCopyCheckPointMutex storeCopyCheckPointMutex; + private final PrepareStoreCopyFilesProvider prepareStoreCopyFilesProvider; + private final Supplier dataSourceSupplier; + private final StoreFileStreamingProtocol streamingProtocol = new StoreFileStreamingProtocol(); + + public PrepareStoreCopyRequestHandler( CatchupServerProtocol catchupServerProtocol, Supplier checkPointerSupplier, + StoreCopyCheckPointMutex storeCopyCheckPointMutex, Supplier dataSourceSupplier, + PrepareStoreCopyFilesProvider prepareStoreCopyFilesProvider ) + { + this.protocol = catchupServerProtocol; + this.checkPointerSupplier = checkPointerSupplier; + this.storeCopyCheckPointMutex = storeCopyCheckPointMutex; + this.prepareStoreCopyFilesProvider = prepareStoreCopyFilesProvider; + this.dataSourceSupplier = dataSourceSupplier; + } + + @Override + protected void channelRead0( ChannelHandlerContext channelHandlerContext, PrepareStoreCopyRequest prepareStoreCopyRequest ) throws IOException + { + CloseablesListener closeablesListener = new CloseablesListener(); + PrepareStoreCopyResponse response = PrepareStoreCopyResponse.error( PrepareStoreCopyResponse.Status.E_LISTING_STORE ); + try + { + NeoStoreDataSource neoStoreDataSource = dataSourceSupplier.get(); + if ( !hasSameStoreId( prepareStoreCopyRequest.getStoreId(), neoStoreDataSource ) ) + { + channelHandlerContext.write( ResponseMessageType.PREPARE_STORE_COPY_RESPONSE ); + response = PrepareStoreCopyResponse.error( PrepareStoreCopyResponse.Status.E_STORE_ID_MISMATCH ); + } + else + { + CheckPointer checkPointer = checkPointerSupplier.get(); + closeablesListener.add( tryCheckpointAndAcquireMutex( checkPointer ) ); + PrepareStoreCopyFiles prepareStoreCopyFiles = + closeablesListener.add( prepareStoreCopyFilesProvider.prepareStoreCopyFiles( neoStoreDataSource ) ); + + StoreResource[] nonReplayable = prepareStoreCopyFiles.getAtomicFilesSnapshot(); + for ( StoreResource storeResource : nonReplayable ) + { + streamingProtocol.stream( channelHandlerContext, storeResource ); + } + channelHandlerContext.write( ResponseMessageType.PREPARE_STORE_COPY_RESPONSE ); + response = createSuccessfulResponse( checkPointer, prepareStoreCopyFiles ); + } + } + finally + { + channelHandlerContext.writeAndFlush( response ).addListener( closeablesListener ); + protocol.expect( CatchupServerProtocol.State.MESSAGE_TYPE ); + } + } + + private PrepareStoreCopyResponse createSuccessfulResponse( CheckPointer checkPointer, PrepareStoreCopyFiles prepareStoreCopyFiles ) throws IOException + { + IndexDescriptor[] indexDescriptors = prepareStoreCopyFiles.getIndexDescriptors(); + File[] files = prepareStoreCopyFiles.listReplayableFiles(); + long lastCommittedTxId = checkPointer.lastCheckPointedTransactionId(); + return PrepareStoreCopyResponse.success( files, indexDescriptors, lastCommittedTxId ); + } + + private Resource tryCheckpointAndAcquireMutex( CheckPointer checkPointer ) throws IOException + { + return storeCopyCheckPointMutex.storeCopy( () -> checkPointer.tryCheckPoint( new SimpleTriggerInfo( "Store copy" ) ) ); + } +} diff --git a/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/storecopy/PrepareStoreCopyResponse.java b/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/storecopy/PrepareStoreCopyResponse.java new file mode 100644 index 0000000000000..1d0778da07418 --- /dev/null +++ b/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/storecopy/PrepareStoreCopyResponse.java @@ -0,0 +1,228 @@ +/* + * 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 Affero 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 Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + */ +package org.neo4j.causalclustering.catchup.storecopy; + +import io.netty.buffer.ByteBuf; +import io.netty.channel.ChannelHandlerContext; +import io.netty.handler.codec.ByteToMessageDecoder; +import io.netty.handler.codec.MessageToByteEncoder; + +import java.io.File; +import java.io.IOException; +import java.util.Arrays; +import java.util.List; +import java.util.Objects; + +import org.neo4j.causalclustering.core.state.storage.SafeChannelMarshal; +import org.neo4j.causalclustering.messaging.NetworkFlushableChannelNetty4; +import org.neo4j.causalclustering.messaging.NetworkReadableClosableChannelNetty4; +import org.neo4j.kernel.api.schema.index.IndexDescriptor; +import org.neo4j.storageengine.api.ReadableChannel; +import org.neo4j.storageengine.api.WritableChannel; +import org.neo4j.string.UTF8; + +public class PrepareStoreCopyResponse +{ + private final File[] files; + private final IndexDescriptor[] indexDescriptors; + private final Long transactionId; + private final Status status; + + public static PrepareStoreCopyResponse error( Status errorStatus ) + { + if ( errorStatus == Status.SUCCESS ) + { + throw new IllegalStateException( "Cannot create error result from state: " + errorStatus ); + } + return new PrepareStoreCopyResponse( new File[0], new IndexDescriptor[0], 0L, errorStatus ); + } + + public static PrepareStoreCopyResponse success( File[] storeFiles, IndexDescriptor[] indexDescriptors, long lastTransactionId ) + { + return new PrepareStoreCopyResponse( storeFiles, indexDescriptors, lastTransactionId, Status.SUCCESS ); + } + + IndexDescriptor[] getDescriptors() + { + return indexDescriptors; + } + + enum Status + { + SUCCESS, + E_STORE_ID_MISMATCH, + E_LISTING_STORE + } + + private PrepareStoreCopyResponse( File[] files, IndexDescriptor[] indexDescriptors, Long transactionId, Status status ) + { + this.files = files; + this.indexDescriptors = indexDescriptors; + this.transactionId = transactionId; + this.status = status; + } + + public File[] getFiles() + { + return files; + } + + long lastTransactionId() + { + return transactionId; + } + + public Status status() + { + return status; + } + + @Override + public boolean equals( Object o ) + { + if ( this == o ) + { + return true; + } + if ( o == null || getClass() != o.getClass() ) + { + return false; + } + PrepareStoreCopyResponse that = (PrepareStoreCopyResponse) o; + return Arrays.equals( files, that.files ) && Arrays.equals( indexDescriptors, that.indexDescriptors ) && + Objects.equals( transactionId, that.transactionId ) && status == that.status; + } + + @Override + public int hashCode() + { + return Objects.hash( files, indexDescriptors, transactionId, status ); + } + + public static class StoreListingMarshal extends SafeChannelMarshal + { + @Override + public void marshal( PrepareStoreCopyResponse prepareStoreCopyResponse, WritableChannel buffer ) throws IOException + { + buffer.putInt( prepareStoreCopyResponse.status.ordinal() ); + buffer.putLong( prepareStoreCopyResponse.transactionId ); + marshalFiles( buffer, prepareStoreCopyResponse.files ); + marshalIndexDescriptors( buffer, prepareStoreCopyResponse.indexDescriptors ); + } + + @Override + protected PrepareStoreCopyResponse unmarshal0( ReadableChannel channel ) throws IOException + { + int ordinal = channel.getInt(); + Status status = Status.values()[ordinal]; + Long transactionId = channel.getLong(); + File[] files = unmarshalFiles( channel ); + IndexDescriptor[] indexDescriptors = unmarshalIndexProviderDescriptors( channel ); + return new PrepareStoreCopyResponse( files, indexDescriptors, transactionId, status ); + } + + private static void marshalFiles( WritableChannel buffer, File[] files ) throws IOException + { + buffer.putInt( files.length ); + for ( File file : files ) + { + putBytes( buffer, file.getName() ); + } + } + + private void marshalIndexDescriptors( WritableChannel buffer, IndexDescriptor[] descriptors ) throws IOException + { + buffer.putInt( descriptors.length ); + for ( IndexDescriptor descriptor : descriptors ) + { + IndexDescriptorSerializer.serialize( descriptor, buffer ); + } + } + + private static File[] unmarshalFiles( ReadableChannel channel ) throws IOException + { + int numberOfFiles = channel.getInt(); + File[] files = new File[numberOfFiles]; + for ( int i = 0; i < numberOfFiles; i++ ) + { + files[i] = unmarshalFile( channel ); + } + return files; + } + + private static File unmarshalFile( ReadableChannel channel ) throws IOException + { + byte[] name = readBytes( channel ); + return new File( UTF8.decode( name ) ); + } + + private IndexDescriptor[] unmarshalIndexProviderDescriptors( ReadableChannel channel ) throws IOException + { + int numberOfDescriptors = channel.getInt(); + IndexDescriptor[] descriptors = new IndexDescriptor[numberOfDescriptors]; + for ( int i = 0; i < numberOfDescriptors; i++ ) + { + descriptors[i] = unmarshalIndexProviderDescriptor( channel ); + } + return descriptors; + } + + private IndexDescriptor unmarshalIndexProviderDescriptor( ReadableChannel channel ) throws IOException + { + return IndexDescriptorSerializer.deserialize( channel ); + } + + private static void putBytes( WritableChannel buffer, String value ) throws IOException + { + byte[] bytes = UTF8.encode( value ); + buffer.putInt( bytes.length ); + buffer.put( bytes, bytes.length ); + } + + private static byte[] readBytes( ReadableChannel channel ) throws IOException + { + int bytesLength = channel.getInt(); + byte[] bytes = new byte[bytesLength]; + channel.get( bytes, bytesLength ); + return bytes; + } + } + + public static class Encoder extends MessageToByteEncoder + { + + @Override + protected void encode( ChannelHandlerContext channelHandlerContext, PrepareStoreCopyResponse prepareStoreCopyResponse, ByteBuf byteBuf ) + throws Exception + { + new PrepareStoreCopyResponse.StoreListingMarshal().marshal( prepareStoreCopyResponse, new NetworkFlushableChannelNetty4( byteBuf ) ); + } + } + + public static class Decoder extends ByteToMessageDecoder + { + + @Override + protected void decode( ChannelHandlerContext channelHandlerContext, ByteBuf byteBuf, List list ) throws Exception + { + list.add( new PrepareStoreCopyResponse.StoreListingMarshal().unmarshal( new NetworkReadableClosableChannelNetty4( byteBuf ) ) ); + } + } +} diff --git a/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/storecopy/PrepareStoreCopyResponseAdaptor.java b/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/storecopy/PrepareStoreCopyResponseAdaptor.java new file mode 100644 index 0000000000000..02a20b2ca3e51 --- /dev/null +++ b/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/storecopy/PrepareStoreCopyResponseAdaptor.java @@ -0,0 +1,69 @@ +/* + * 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 Affero 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 Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + */ +package org.neo4j.causalclustering.catchup.storecopy; + +import java.io.IOException; +import java.util.HashSet; +import java.util.Set; +import java.util.concurrent.CompletableFuture; + +import org.neo4j.causalclustering.catchup.CatchUpResponseAdaptor; +import org.neo4j.logging.Log; +import org.neo4j.logging.LogProvider; + +/** + * Used client side for event handling of a store listing request + */ +public class PrepareStoreCopyResponseAdaptor extends CatchUpResponseAdaptor +{ + private final StoreFileStreams storeFileStreams; + private final Log log; + private String destination; + private int requiredAlignment; + + public PrepareStoreCopyResponseAdaptor( StoreFileStreams storeFileStreams, LogProvider logProvider ) + { + this.storeFileStreams = storeFileStreams; + log = logProvider.getLog( PrepareStoreCopyResponseAdaptor.class ); + } + + @Override + public void onStoreListingResponse( CompletableFuture signal, PrepareStoreCopyResponse response ) + { + log.debug( "Complete download of file %s", destination ); + signal.complete( response ); + } + + @Override + public void onFileHeader( CompletableFuture requestOutcomeSignal, FileHeader fileHeader ) + { + log.debug( "Received file header for file %s", fileHeader.fileName() ); + this.destination = fileHeader.fileName(); + this.requiredAlignment = fileHeader.requiredAlignment(); + } + + @Override + public boolean onFileContent( CompletableFuture signal, FileChunk fileChunk ) throws IOException + { + log.debug( "Received %b bytes for file %s", fileChunk.bytes(), destination ); + storeFileStreams.write( destination, requiredAlignment, fileChunk.bytes() ); + return fileChunk.isLast(); + } +} diff --git a/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/storecopy/RemoteStore.java b/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/storecopy/RemoteStore.java index 739b9822d4011..c8f9e75164682 100644 --- a/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/storecopy/RemoteStore.java +++ b/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/storecopy/RemoteStore.java @@ -21,9 +21,12 @@ import java.io.File; import java.io.IOException; +import java.util.function.Supplier; import org.neo4j.causalclustering.catchup.CatchUpClientException; +import org.neo4j.causalclustering.catchup.CatchupAddressResolutionException; import org.neo4j.causalclustering.catchup.CatchupResult; +import org.neo4j.causalclustering.catchup.CatchupAddressProvider; import org.neo4j.causalclustering.catchup.TxPullRequestResult; import org.neo4j.causalclustering.catchup.tx.TransactionLogCatchUpFactory; import org.neo4j.causalclustering.catchup.tx.TransactionLogCatchUpWriter; @@ -47,6 +50,7 @@ */ public class RemoteStore { + private static final Supplier DEFAULT_TERMINATION_CONDITIONS = () -> new MaximumTotalRetries( 10, 10 ); private final Log log; private final Config config; private final Monitors monitors; @@ -114,16 +118,15 @@ public CatchupResult tryCatchingUp( AdvertisedSocketAddress from, StoreId expect } } - public void copy( AdvertisedSocketAddress from, StoreId expectedStoreId, File destDir ) + public void copy( CatchupAddressProvider addressProvider, StoreId expectedStoreId, File destDir ) throws StoreCopyFailedException, StreamingTransactionsFailedException { try { - log.info( "Copying store from %s", from ); long lastFlushedTxId; try ( StreamToDisk storeFileStreams = new StreamToDisk( destDir, fs, pageCache, monitors ) ) { - lastFlushedTxId = storeCopyClient.copyStoreFiles( from, expectedStoreId, storeFileStreams ); + lastFlushedTxId = storeCopyClient.copyStoreFiles( addressProvider, expectedStoreId, storeFileStreams, DEFAULT_TERMINATION_CONDITIONS ); } log.info( "Store files need to be recovered starting from: %d", lastFlushedTxId ); @@ -132,13 +135,13 @@ public void copy( AdvertisedSocketAddress from, StoreId expectedStoreId, File de // because the destination directory is temporary. We will copy them to the correct place later. boolean keepTxLogsInStoreDir = true; CatchupResult catchupResult = - pullTransactions( from, expectedStoreId, destDir, lastFlushedTxId, true, keepTxLogsInStoreDir ); + pullTransactions( addressProvider.primary(), expectedStoreId, destDir, lastFlushedTxId, true, keepTxLogsInStoreDir ); if ( catchupResult != SUCCESS_END_OF_STREAM ) { throw new StreamingTransactionsFailedException( "Failed to pull transactions: " + catchupResult ); } } - catch ( IOException e ) + catch ( CatchupAddressResolutionException | IOException e ) { throw new StoreCopyFailedException( e ); } diff --git a/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/storecopy/StoreCopyClient.java b/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/storecopy/StoreCopyClient.java index e953097760f8e..ff692ec4912bb 100644 --- a/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/storecopy/StoreCopyClient.java +++ b/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/storecopy/StoreCopyClient.java @@ -19,14 +19,19 @@ */ package org.neo4j.causalclustering.catchup.storecopy; +import java.io.File; import java.io.IOException; import java.util.concurrent.CompletableFuture; +import java.util.function.Supplier; import org.neo4j.causalclustering.catchup.CatchUpClient; import org.neo4j.causalclustering.catchup.CatchUpClientException; import org.neo4j.causalclustering.catchup.CatchUpResponseAdaptor; +import org.neo4j.causalclustering.catchup.CatchupAddressProvider; +import org.neo4j.causalclustering.catchup.CatchupAddressResolutionException; import org.neo4j.causalclustering.identity.StoreId; import org.neo4j.helpers.AdvertisedSocketAddress; +import org.neo4j.kernel.api.schema.index.IndexDescriptor; import org.neo4j.logging.Log; import org.neo4j.logging.LogProvider; @@ -34,48 +39,112 @@ public class StoreCopyClient { private final CatchUpClient catchUpClient; private final Log log; + private final LogProvider logProvider; public StoreCopyClient( CatchUpClient catchUpClient, LogProvider logProvider ) { this.catchUpClient = catchUpClient; log = logProvider.getLog( getClass() ); + this.logProvider = logProvider; } - public long copyStoreFiles( AdvertisedSocketAddress from, StoreId expectedStoreId, StoreFileStreams storeFileStreams ) throws StoreCopyFailedException + long copyStoreFiles( CatchupAddressProvider catchupAddressProvider, StoreId expectedStoreId, StoreFileStreams storeFileStreams, + Supplier requestWiseTerminationCondition ) + throws StoreCopyFailedException { try { - return catchUpClient.makeBlockingRequest( from, new GetStoreRequest( expectedStoreId ), new CatchUpResponseAdaptor() - { - private String destination; - private int requiredAlignment; + PrepareStoreCopyResponse prepareStoreCopyResponse = listFiles( catchupAddressProvider.primary(), expectedStoreId, storeFileStreams ); + copyFilesIndividually( prepareStoreCopyResponse, expectedStoreId, catchupAddressProvider, storeFileStreams, requestWiseTerminationCondition ); + copyIndexSnapshotIndividually( prepareStoreCopyResponse, expectedStoreId, catchupAddressProvider, storeFileStreams, + requestWiseTerminationCondition ); + return prepareStoreCopyResponse.lastTransactionId(); + } + catch ( CatchupAddressResolutionException | CatchUpClientException e ) + { + throw new StoreCopyFailedException( e ); + } + } - @Override - public void onFileHeader( CompletableFuture requestOutcomeSignal, FileHeader fileHeader ) + private void copyFilesIndividually( PrepareStoreCopyResponse prepareStoreCopyResponse, StoreId expectedStoreId, CatchupAddressProvider addressProvider, + StoreFileStreams storeFileStreams, Supplier terminationConditions ) throws StoreCopyFailedException + { + CatchUpResponseAdaptor copyHandler = new StoreFileCopyResponseAdaptor( storeFileStreams, log ); + long lastTransactionId = prepareStoreCopyResponse.lastTransactionId(); + for ( File file : prepareStoreCopyResponse.getFiles() ) + { + TerminationCondition terminationCondition = terminationConditions.get(); + boolean successful; + do + { + try { - this.destination = fileHeader.fileName(); - this.requiredAlignment = fileHeader.requiredAlignment(); + AdvertisedSocketAddress from = addressProvider.primary(); + log.info( String.format( "Downloading file '%s' from '%s'", file, from ) ); + StoreCopyFinishedResponse response = + catchUpClient.makeBlockingRequest( from, new GetStoreFileRequest( expectedStoreId, file, lastTransactionId ), copyHandler ); + successful = successfulFileDownload( response ); } - - @Override - public boolean onFileContent( CompletableFuture signal, FileChunk fileChunk ) throws IOException + catch ( CatchUpClientException | CatchupAddressResolutionException e ) + { + successful = false; + } + if ( !successful ) { - storeFileStreams.write( destination, requiredAlignment, fileChunk.bytes() ); - return fileChunk.isLast(); + log.error( "Failed to download file '%s'", file ); + terminationCondition.assertContinue(); } + } + while ( !successful ); + } + } - @Override - public void onFileStreamingComplete( CompletableFuture signal, StoreCopyFinishedResponse response ) + private void copyIndexSnapshotIndividually( PrepareStoreCopyResponse prepareStoreCopyResponse, StoreId expectedStoreId, + CatchupAddressProvider addressProvider, + StoreFileStreams storeFileStreams, Supplier terminationConditions ) throws StoreCopyFailedException + { + CatchUpResponseAdaptor copyHandler = new StoreFileCopyResponseAdaptor( storeFileStreams, log ); + long lastTransactionId = prepareStoreCopyResponse.lastTransactionId(); + for ( IndexDescriptor descriptor : prepareStoreCopyResponse.getDescriptors() ) + { + TerminationCondition terminationCondition = terminationConditions.get(); + boolean successful; + do + { + try + { + AdvertisedSocketAddress from = addressProvider.primary(); + log.info( String.format( "Downloading snapshot '%s' from '%s'", descriptor, from ) ); + StoreCopyFinishedResponse response = + catchUpClient.makeBlockingRequest( from, new GetIndexFilesRequest( expectedStoreId, descriptor, lastTransactionId ), + copyHandler ); + successful = successfulFileDownload( response ); + } + catch ( CatchUpClientException | CatchupAddressResolutionException e ) + { + successful = false; + } + if ( !successful ) { - log.info( "Finished streaming" ); - signal.complete( response.lastCommittedTxBeforeStoreCopy() ); + log.error( "Failed to download file '%s'", descriptor ); + terminationCondition.assertContinue(); } - } ); + } + while ( !successful ); } - catch ( CatchUpClientException e ) + } + + private PrepareStoreCopyResponse listFiles( AdvertisedSocketAddress from, StoreId expectedStoreId, StoreFileStreams storeFileStreams ) + throws CatchUpClientException, StoreCopyFailedException + { + log.info( "Requesting store listing from: " + from ); + PrepareStoreCopyResponse prepareStoreCopyResponse = catchUpClient.makeBlockingRequest( from, new PrepareStoreCopyRequest( expectedStoreId ), + new PrepareStoreCopyResponseAdaptor( storeFileStreams, logProvider ) ); + if ( prepareStoreCopyResponse.status() != PrepareStoreCopyResponse.Status.SUCCESS ) { - throw new StoreCopyFailedException( e ); + throw new StoreCopyFailedException( "Preparing store failed due to: " + prepareStoreCopyResponse.status() ); } + return prepareStoreCopyResponse; } public StoreId fetchStoreId( AdvertisedSocketAddress fromAddress ) throws StoreIdDownloadFailedException @@ -97,4 +166,61 @@ public void onGetStoreIdResponse( CompletableFuture signal, GetStoreIdR throw new StoreIdDownloadFailedException( e ); } } + + private boolean successfulFileDownload( StoreCopyFinishedResponse response ) throws StoreCopyFailedException + { + StoreCopyFinishedResponse.Status responseStatus = response.status(); + log.debug( "Request for individual file resulted in response type: %s", response.status() ); + if ( responseStatus == StoreCopyFinishedResponse.Status.E_TOO_FAR_BEHIND ) + { + return false; + } + else if ( responseStatus == StoreCopyFinishedResponse.Status.SUCCESS ) + { + return true; + } + else if ( responseStatus == StoreCopyFinishedResponse.Status.E_STORE_ID_MISMATCH ) + { + throw new StoreCopyFailedException( "Store id mismatch" ); + } + else + { + throw new StoreCopyFailedException( "Unknown response type: " + responseStatus ); + } + } + + public static class StoreFileCopyResponseAdaptor extends CatchUpResponseAdaptor + { + private final StoreFileStreams storeFileStreams; + private final Log log; + private String destination; + private int requiredAlignment; + + StoreFileCopyResponseAdaptor( StoreFileStreams storeFileStreams, Log log ) + { + this.storeFileStreams = storeFileStreams; + this.log = log; + } + + @Override + public void onFileHeader( CompletableFuture requestOutcomeSignal, FileHeader fileHeader ) + { + this.destination = fileHeader.fileName(); + this.requiredAlignment = fileHeader.requiredAlignment(); + } + + @Override + public boolean onFileContent( CompletableFuture signal, FileChunk fileChunk ) throws IOException + { + storeFileStreams.write( destination, requiredAlignment, fileChunk.bytes() ); + return fileChunk.isLast(); + } + + @Override + public void onFileStreamingComplete( CompletableFuture signal, StoreCopyFinishedResponse response ) + { + log.info( "Finished streaming" ); + signal.complete( response ); + } + } } diff --git a/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/storecopy/StoreCopyFinishedResponse.java b/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/storecopy/StoreCopyFinishedResponse.java index dc91df9104521..06aa83e632a9c 100644 --- a/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/storecopy/StoreCopyFinishedResponse.java +++ b/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/storecopy/StoreCopyFinishedResponse.java @@ -26,22 +26,16 @@ public class StoreCopyFinishedResponse public enum Status { SUCCESS, + E_TOO_FAR_BEHIND, + E_UNKNOWN, E_STORE_ID_MISMATCH } private final Status status; - private final long lastCommittedTxBeforeStoreCopy; - public StoreCopyFinishedResponse( Status status, - long lastCommittedTxBeforeStoreCopy ) + public StoreCopyFinishedResponse( Status status ) { this.status = status; - this.lastCommittedTxBeforeStoreCopy = lastCommittedTxBeforeStoreCopy; - } - - long lastCommittedTxBeforeStoreCopy() - { - return lastCommittedTxBeforeStoreCopy; } Status status() @@ -61,12 +55,12 @@ public boolean equals( Object o ) return false; } StoreCopyFinishedResponse that = (StoreCopyFinishedResponse) o; - return lastCommittedTxBeforeStoreCopy == that.lastCommittedTxBeforeStoreCopy && status == that.status; + return status == that.status; } @Override public int hashCode() { - return Objects.hash( status, lastCommittedTxBeforeStoreCopy ); + return Objects.hash( status ); } } diff --git a/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/storecopy/StoreCopyFinishedResponseDecoder.java b/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/storecopy/StoreCopyFinishedResponseDecoder.java index 8aef6bcd058d4..8622bd77d270d 100644 --- a/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/storecopy/StoreCopyFinishedResponseDecoder.java +++ b/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/storecopy/StoreCopyFinishedResponseDecoder.java @@ -33,7 +33,6 @@ public class StoreCopyFinishedResponseDecoder extends ByteToMessageDecoder protected void decode( ChannelHandlerContext ctx, ByteBuf msg, List out ) { int statusOrdinal = msg.readInt(); - long lastCommittedTxBeforeStoreCopy = msg.readLong(); - out.add( new StoreCopyFinishedResponse( Status.values()[statusOrdinal], lastCommittedTxBeforeStoreCopy ) ); + out.add( new StoreCopyFinishedResponse( Status.values()[statusOrdinal] ) ); } } diff --git a/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/storecopy/StoreCopyFinishedResponseEncoder.java b/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/storecopy/StoreCopyFinishedResponseEncoder.java index 83b8f73f808da..1f166a27e8a34 100644 --- a/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/storecopy/StoreCopyFinishedResponseEncoder.java +++ b/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/storecopy/StoreCopyFinishedResponseEncoder.java @@ -29,6 +29,5 @@ public class StoreCopyFinishedResponseEncoder extends MessageToByteEncoder * Note that we do not block here. */ - void stream( ChannelHandlerContext ctx, RawCursor resources ) throws IOException + void stream( ChannelHandlerContext ctx, StoreResource resource ) { - while ( resources.next() ) - { - StoreResource resource = resources.get(); - - ctx.write( ResponseMessageType.FILE ); - ctx.write( new FileHeader( resource.path(), resource.recordSize() ) ); - ctx.write( new FileSender( resource ) ); - } + ctx.write( ResponseMessageType.FILE ); + ctx.write( new FileHeader( resource.path(), resource.recordSize() ) ); + ctx.write( new FileSender( resource ) ); } - Future end( ChannelHandlerContext ctx, StoreCopyFinishedResponse.Status status, long lastCommittedTxBeforeStoreCopy ) + Future end( ChannelHandlerContext ctx, StoreCopyFinishedResponse.Status status ) { ctx.write( ResponseMessageType.STORE_COPY_FINISHED ); - return ctx.writeAndFlush( new StoreCopyFinishedResponse( status, lastCommittedTxBeforeStoreCopy ) ); + return ctx.writeAndFlush( new StoreCopyFinishedResponse( status ) ); } } diff --git a/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/storecopy/StoreStreamingProcess.java b/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/storecopy/StoreStreamingProcess.java index cbe3f31f52c40..691836c84f0b5 100644 --- a/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/storecopy/StoreStreamingProcess.java +++ b/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/storecopy/StoreStreamingProcess.java @@ -35,12 +35,12 @@ public class StoreStreamingProcess { - private final StoreStreamingProtocol protocol; + private final StoreFileStreamingProtocol protocol; private final Supplier checkPointerSupplier; private final StoreCopyCheckPointMutex mutex; private final StoreResourceStreamFactory resourceStreamFactory; - public StoreStreamingProcess( StoreStreamingProtocol protocol, Supplier checkPointerSupplier, StoreCopyCheckPointMutex mutex, + public StoreStreamingProcess( StoreFileStreamingProtocol protocol, Supplier checkPointerSupplier, StoreCopyCheckPointMutex mutex, StoreResourceStreamFactory resourceStreamFactory ) { this.protocol = protocol; @@ -57,8 +57,12 @@ void perform( ChannelHandlerContext ctx ) throws IOException Future completion = null; try ( RawCursor resources = resourceStreamFactory.create() ) { - protocol.stream( ctx, resources ); - completion = protocol.end( ctx, SUCCESS, checkPointer.lastCheckPointedTransactionId() ); + while ( resources.next() ) + { + StoreResource resource = resources.get(); + protocol.stream( ctx, resource ); + } + completion = protocol.end( ctx, SUCCESS ); } finally { @@ -75,6 +79,6 @@ void perform( ChannelHandlerContext ctx ) throws IOException public void fail( ChannelHandlerContext ctx, StoreCopyFinishedResponse.Status failureCode ) { - protocol.end( ctx, failureCode, -1 ); + protocol.end( ctx, failureCode ); } } diff --git a/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/storecopy/TerminationCondition.java b/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/storecopy/TerminationCondition.java new file mode 100644 index 0000000000000..729ec9dda9a16 --- /dev/null +++ b/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/storecopy/TerminationCondition.java @@ -0,0 +1,35 @@ +/* + * 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 Affero 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 Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + */ +package org.neo4j.causalclustering.catchup.storecopy; + +@FunctionalInterface +interface TerminationCondition +{ + TerminationCondition CONTINUE_INDEFINITELY = () -> + { + }; + + /** + * If store copy client is allowed to continue sending store requests. + * + * @throws StoreCopyFailedException if the process should be stopped. + */ + void assertContinue() throws StoreCopyFailedException; +} diff --git a/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/tx/CatchupPollingProcess.java b/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/tx/CatchupPollingProcess.java index 9a0dcb4a1ad0a..e1e40523774f7 100644 --- a/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/tx/CatchupPollingProcess.java +++ b/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/catchup/tx/CatchupPollingProcess.java @@ -27,6 +27,7 @@ import org.neo4j.causalclustering.catchup.CatchUpClient; import org.neo4j.causalclustering.catchup.CatchUpClientException; import org.neo4j.causalclustering.catchup.CatchUpResponseAdaptor; +import org.neo4j.causalclustering.catchup.CatchupAddressProvider; import org.neo4j.causalclustering.catchup.storecopy.LocalDatabase; import org.neo4j.causalclustering.catchup.storecopy.StoreCopyFailedException; import org.neo4j.causalclustering.catchup.storecopy.StoreCopyProcess; @@ -332,7 +333,7 @@ private void downloadDatabase( MemberId upstream, StoreId localStoreId ) try { AdvertisedSocketAddress fromAddress = topologyService.findCatchupAddress( upstream ).orElseThrow( () -> new TopologyLookupException( upstream ) ); - storeCopyProcess.replaceWithStoreFrom( fromAddress, localStoreId ); + storeCopyProcess.replaceWithStoreFrom( CatchupAddressProvider.fromSingleAddress( fromAddress ), localStoreId ); } catch ( IOException | StoreCopyFailedException | StreamingTransactionsFailedException | TopologyLookupException e ) { diff --git a/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/core/EnterpriseCoreEditionModule.java b/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/core/EnterpriseCoreEditionModule.java index d35e7d5fa99f2..ef8673ca66da6 100644 --- a/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/core/EnterpriseCoreEditionModule.java +++ b/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/core/EnterpriseCoreEditionModule.java @@ -279,7 +279,8 @@ public EnterpriseCoreEditionModule( final PlatformModule platformModule, replicationModule, localDatabase, databaseHealthSupplier, clusterStateDirectory.get(), serverPipelineWrapper, clientPipelineWrapper ); serverInstalledProtocols = new RaftServerModule( - platformModule, consensusModule, identityModule, coreServerModule, localDatabase, serverPipelineBuilderFactory, messageLogger + platformModule, consensusModule, identityModule, coreServerModule, localDatabase, serverPipelineBuilderFactory, messageLogger, + topologyService ).raftServer()::installedProtocols; editionInvariants( platformModule, dependencies, config, logging, life ); diff --git a/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/core/RaftServerModule.java b/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/core/RaftServerModule.java index 5edf7e48df43f..f6d60d43514e6 100644 --- a/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/core/RaftServerModule.java +++ b/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/core/RaftServerModule.java @@ -32,6 +32,8 @@ import org.neo4j.causalclustering.core.consensus.RaftServer; import org.neo4j.causalclustering.core.server.CoreServerModule; import org.neo4j.causalclustering.core.state.RaftMessageApplier; +import org.neo4j.causalclustering.discovery.CoreTopologyService; +import org.neo4j.causalclustering.discovery.TopologyService; import org.neo4j.causalclustering.identity.MemberId; import org.neo4j.causalclustering.logging.MessageLogger; import org.neo4j.causalclustering.messaging.ComposableMessageHandler; @@ -58,10 +60,12 @@ class RaftServerModule private final MessageLogger messageLogger; private final LogProvider logProvider; private final NettyPipelineBuilderFactory pipelineBuilderFactory; + private final TopologyService topologyService; private final RaftServer raftServer; RaftServerModule( PlatformModule platformModule, ConsensusModule consensusModule, IdentityModule identityModule, CoreServerModule coreServerModule, - LocalDatabase localDatabase, NettyPipelineBuilderFactory pipelineBuilderFactory, MessageLogger messageLogger ) + LocalDatabase localDatabase, NettyPipelineBuilderFactory pipelineBuilderFactory, MessageLogger messageLogger, + CoreTopologyService topologyService ) { this.platformModule = platformModule; this.consensusModule = consensusModule; @@ -70,6 +74,7 @@ class RaftServerModule this.messageLogger = messageLogger; this.logProvider = platformModule.logging.getInternalLogProvider(); this.pipelineBuilderFactory = pipelineBuilderFactory; + this.topologyService = topologyService; LifecycleMessageHandler> messageHandlerChain = createMessageHandlerChain( coreServerModule ); @@ -105,8 +110,9 @@ private RaftServer createRaftServer( CoreServerModule coreServerModule, private LifecycleMessageHandler> createMessageHandlerChain( CoreServerModule coreServerModule ) { - RaftMessageApplier messageApplier = new RaftMessageApplier( localDatabase, logProvider, - consensusModule.raftMachine(), coreServerModule.downloadService(), coreServerModule.commandApplicationProcess() ); + RaftMessageApplier messageApplier = + new RaftMessageApplier( localDatabase, logProvider, consensusModule.raftMachine(), coreServerModule.downloadService(), + coreServerModule.commandApplicationProcess(), topologyService ); ComposableMessageHandler monitoringHandler = RaftMessageMonitoringHandler.composable( platformModule.clock, platformModule.monitors ); diff --git a/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/core/server/CoreServerModule.java b/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/core/server/CoreServerModule.java index 1b0ed08241a71..92b2ff20f2cb2 100644 --- a/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/core/server/CoreServerModule.java +++ b/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/core/server/CoreServerModule.java @@ -182,8 +182,8 @@ private CoreStateDownloader createCoreStateDownloader( LifeSupport servicesToSto CatchUpClient catchUpClient = platformModule.life.add( new CatchUpClient( logProvider, Clocks.systemClock(), inactivityTimeoutMillis, platformModule.monitors, clientPipelineWrapper ) ); - RemoteStore remoteStore = new RemoteStore( - logProvider, platformModule.fileSystem, platformModule.pageCache, new StoreCopyClient( catchUpClient, logProvider ), + RemoteStore remoteStore = new RemoteStore( logProvider, platformModule.fileSystem, platformModule.pageCache, + new StoreCopyClient( catchUpClient, logProvider ), new TxPullClient( catchUpClient, platformModule.monitors ), new TransactionLogCatchUpFactory(), config, platformModule.monitors ); CopiedStoreRecovery copiedStoreRecovery = platformModule.life.add( @@ -194,7 +194,7 @@ logProvider, platformModule.fileSystem, platformModule.pageCache, new StoreCopyC CommitStateHelper commitStateHelper = new CommitStateHelper( platformModule.pageCache, platformModule.fileSystem, config ); return new CoreStateDownloader( localDatabase, servicesToStopOnStoreCopy, remoteStore, catchUpClient, logProvider, - storeCopyProcess, coreStateMachinesModule.coreStateMachines, snapshotService, clusteringModule.topologyService(), commitStateHelper ); + storeCopyProcess, coreStateMachinesModule.coreStateMachines, snapshotService, commitStateHelper ); } private MembershipWaiterLifecycle createMembershipWaiterLifecycle() diff --git a/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/core/state/RaftMessageApplier.java b/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/core/state/RaftMessageApplier.java index c863c948c5708..2aec823ea4b5f 100644 --- a/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/core/state/RaftMessageApplier.java +++ b/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/core/state/RaftMessageApplier.java @@ -19,11 +19,13 @@ */ package org.neo4j.causalclustering.core.state; +import org.neo4j.causalclustering.catchup.CatchupAddressProvider; import org.neo4j.causalclustering.catchup.storecopy.LocalDatabase; import org.neo4j.causalclustering.core.consensus.RaftMachine; import org.neo4j.causalclustering.core.consensus.RaftMessages; import org.neo4j.causalclustering.core.consensus.outcome.ConsensusOutcome; import org.neo4j.causalclustering.core.state.snapshot.CoreStateDownloaderService; +import org.neo4j.causalclustering.discovery.TopologyService; import org.neo4j.causalclustering.identity.ClusterId; import org.neo4j.causalclustering.messaging.LifecycleMessageHandler; import org.neo4j.logging.Log; @@ -36,16 +38,17 @@ public class RaftMessageApplier implements LifecycleMessageHandler new TopologyLookupException( source )); - StoreId remoteStoreId = remoteStore.getStoreId( fromAddress ); + AdvertisedSocketAddress primary = addressProvider.primary(); + StoreId remoteStoreId = remoteStore.getStoreId( primary ); if ( !isEmptyStore && !remoteStoreId.equals( localDatabase.storeId() ) ) { throw new StoreCopyFailedException( "StoreId mismatch and not empty" ); @@ -105,7 +102,7 @@ void downloadSnapshot( MemberId source ) throws StoreCopyFailedException startStopOnStoreCopy.stop(); localDatabase.stopForStoreCopy(); - log.info( "Downloading snapshot from core server at %s", source ); + log.info( "Downloading snapshot from core server at %s", addressProvider ); /* The core snapshot must be copied before the store, because the store has a dependency on * the state of the state machines. The store will thus be at or ahead of the state machines, @@ -114,31 +111,30 @@ void downloadSnapshot( MemberId source ) throws StoreCopyFailedException * are ahead, and the correct decisions for their applicability have already been taken as encapsulated * in the copied store. */ - CoreSnapshot coreSnapshot = catchUpClient.makeBlockingRequest( fromAddress, new CoreSnapshotRequest(), - new CatchUpResponseAdaptor() - { - @Override - public void onCoreSnapshot( CompletableFuture signal, CoreSnapshot response ) - { - signal.complete( response ); - } - } ); + CoreSnapshot coreSnapshot = catchUpClient.makeBlockingRequest( primary, new CoreSnapshotRequest(), new CatchUpResponseAdaptor() + { + @Override + public void onCoreSnapshot( CompletableFuture signal, CoreSnapshot response ) + { + signal.complete( response ); + } + } ); if ( isEmptyStore ) { - storeCopyProcess.replaceWithStoreFrom( fromAddress, remoteStoreId ); + storeCopyProcess.replaceWithStoreFrom( addressProvider, remoteStoreId ); } else { StoreId localStoreId = localDatabase.storeId(); - CatchupResult catchupResult = remoteStore.tryCatchingUp( fromAddress, localStoreId, localDatabase.storeDir(), false ); + CatchupResult catchupResult = remoteStore.tryCatchingUp( primary, localStoreId, localDatabase.storeDir(), false ); if ( catchupResult == E_TRANSACTION_PRUNED ) { - log.info( format( "Failed to pull transactions from %s (%s). They may have been pruned away", source, fromAddress ) ); + log.info( format( "Failed to pull transactions from (%s). They may have been pruned away", primary ) ); localDatabase.delete(); - storeCopyProcess.replaceWithStoreFrom( fromAddress, localStoreId ); + storeCopyProcess.replaceWithStoreFrom( addressProvider, localStoreId ); } else if ( catchupResult != SUCCESS_END_OF_STREAM ) { diff --git a/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/core/state/snapshot/CoreStateDownloaderService.java b/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/core/state/snapshot/CoreStateDownloaderService.java index f078329ccdbb7..c64952aab7dfd 100644 --- a/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/core/state/snapshot/CoreStateDownloaderService.java +++ b/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/core/state/snapshot/CoreStateDownloaderService.java @@ -19,7 +19,7 @@ */ package org.neo4j.causalclustering.core.state.snapshot; -import org.neo4j.causalclustering.core.consensus.LeaderLocator; +import org.neo4j.causalclustering.catchup.CatchupAddressProvider; import org.neo4j.causalclustering.core.state.CommandApplicationProcess; import org.neo4j.causalclustering.helper.TimeoutStrategy; import org.neo4j.scheduler.JobScheduler; @@ -51,7 +51,7 @@ public CoreStateDownloaderService( JobScheduler jobScheduler, CoreStateDownloade this.downloaderPauseStrategy = downloaderPauseStrategy; } - public synchronized void scheduleDownload( LeaderLocator leaderLocator ) + public synchronized void scheduleDownload( CatchupAddressProvider addressProvider ) { if ( stopped ) { @@ -60,7 +60,7 @@ public synchronized void scheduleDownload( LeaderLocator leaderLocator ) if ( currentJob == null || currentJob.hasCompleted() ) { - currentJob = new PersistentSnapshotDownloader( leaderLocator, applicationProcess, downloader, log, + currentJob = new PersistentSnapshotDownloader( addressProvider, applicationProcess, downloader, log, downloaderPauseStrategy ); jobScheduler.schedule( downloadSnapshot, currentJob ); } diff --git a/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/core/state/snapshot/PersistentSnapshotDownloader.java b/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/core/state/snapshot/PersistentSnapshotDownloader.java index 1c694806e376b..023c78a569970 100644 --- a/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/core/state/snapshot/PersistentSnapshotDownloader.java +++ b/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/core/state/snapshot/PersistentSnapshotDownloader.java @@ -19,9 +19,8 @@ */ package org.neo4j.causalclustering.core.state.snapshot; +import org.neo4j.causalclustering.catchup.CatchupAddressProvider; import org.neo4j.causalclustering.catchup.storecopy.StoreCopyFailedException; -import org.neo4j.causalclustering.core.consensus.LeaderLocator; -import org.neo4j.causalclustering.core.consensus.NoLeaderFoundException; import org.neo4j.causalclustering.core.state.CommandApplicationProcess; import org.neo4j.causalclustering.helper.TimeoutStrategy; import org.neo4j.logging.Log; @@ -33,19 +32,18 @@ class PersistentSnapshotDownloader implements Runnable static final String OPERATION_NAME = "download of snapshot"; private final CommandApplicationProcess applicationProcess; - private final LeaderLocator leaderLocator; + private final CatchupAddressProvider addressProvider; private final CoreStateDownloader downloader; private final Log log; private final TimeoutStrategy.Timeout timeout; private volatile State state; private volatile boolean keepRunning; - PersistentSnapshotDownloader( LeaderLocator leaderLocator, - CommandApplicationProcess applicationProcess, CoreStateDownloader downloader, Log log, + PersistentSnapshotDownloader( CatchupAddressProvider addressProvider, CommandApplicationProcess applicationProcess, CoreStateDownloader downloader, Log log, TimeoutStrategy.Timeout pauseStrategy ) { this.applicationProcess = applicationProcess; - this.leaderLocator = leaderLocator; + this.addressProvider = addressProvider; this.downloader = downloader; this.log = log; this.timeout = pauseStrategy; @@ -75,17 +73,13 @@ public void run() { try { - downloader.downloadSnapshot( leaderLocator.getLeader() ); + downloader.downloadSnapshot( addressProvider ); break; } catch ( StoreCopyFailedException e ) { log.error( format( "Failed to download snapshot. Retrying in %s ms.", timeout.getMillis() ), e ); } - catch ( NoLeaderFoundException e ) - { - log.warn( "No leader found. Retrying in {} ms.", timeout.getMillis() ); - } Thread.sleep( timeout.getMillis() ); timeout.increment(); } diff --git a/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/readreplica/EnterpriseReadReplicaEditionModule.java b/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/readreplica/EnterpriseReadReplicaEditionModule.java index ba5abc1c82aa9..455192fa33182 100644 --- a/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/readreplica/EnterpriseReadReplicaEditionModule.java +++ b/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/readreplica/EnterpriseReadReplicaEditionModule.java @@ -232,7 +232,8 @@ public EnterpriseReadReplicaEditionModule( final PlatformModule platformModule, watcherService, platformModule.availabilityGuard, logProvider ); RemoteStore remoteStore = new RemoteStore( platformModule.logging.getInternalLogProvider(), fileSystem, platformModule.pageCache, - new StoreCopyClient( catchUpClient, logProvider ), new TxPullClient( catchUpClient, platformModule.monitors ), + new StoreCopyClient( catchUpClient, logProvider ), + new TxPullClient( catchUpClient, platformModule.monitors ), new TransactionLogCatchUpFactory(), config, platformModule.monitors ); CopiedStoreRecovery copiedStoreRecovery = new CopiedStoreRecovery( config, platformModule.kernelExtensions.listFactories(), platformModule.pageCache ); diff --git a/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/readreplica/ReadReplicaStartupProcess.java b/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/readreplica/ReadReplicaStartupProcess.java index aa1293aff551e..dfe5aaf5df228 100644 --- a/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/readreplica/ReadReplicaStartupProcess.java +++ b/enterprise/causal-clustering/src/main/java/org/neo4j/causalclustering/readreplica/ReadReplicaStartupProcess.java @@ -21,6 +21,7 @@ import java.io.IOException; +import org.neo4j.causalclustering.catchup.CatchupAddressProvider; import org.neo4j.causalclustering.catchup.storecopy.LocalDatabase; import org.neo4j.causalclustering.catchup.storecopy.RemoteStore; import org.neo4j.causalclustering.catchup.storecopy.StoreCopyFailedException; @@ -168,7 +169,7 @@ private void syncStoreWithUpstream( MemberId source ) debugLog.info( "Copying store from upstream server %s", source ); localDatabase.delete(); - storeCopyProcess.replaceWithStoreFrom( fromAddress, storeId ); + storeCopyProcess.replaceWithStoreFrom( CatchupAddressProvider.fromSingleAddress( fromAddress ), storeId ); debugLog.info( "Restarting local database after copy.", source ); } diff --git a/enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/StrippedCatchupServer.java b/enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/StrippedCatchupServer.java new file mode 100644 index 0000000000000..6c6bef43ad4e7 --- /dev/null +++ b/enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/StrippedCatchupServer.java @@ -0,0 +1,149 @@ +/* + * 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 Affero 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 Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + */ +package org.neo4j.causalclustering; + +import io.netty.channel.ChannelHandler; +import io.netty.channel.ChannelInboundHandler; +import io.netty.channel.ChannelInitializer; +import io.netty.channel.ChannelPipeline; +import io.netty.channel.socket.SocketChannel; +import io.netty.handler.codec.LengthFieldBasedFrameDecoder; +import io.netty.handler.codec.LengthFieldPrepender; +import io.netty.handler.stream.ChunkedWriteHandler; + +import org.neo4j.causalclustering.catchup.CatchupServerProtocol; +import org.neo4j.causalclustering.catchup.RequestDecoderDispatcher; +import org.neo4j.causalclustering.catchup.RequestMessageTypeEncoder; +import org.neo4j.causalclustering.catchup.ResponseMessageTypeEncoder; +import org.neo4j.causalclustering.catchup.ServerMessageTypeHandler; +import org.neo4j.causalclustering.catchup.storecopy.FileChunkEncoder; +import org.neo4j.causalclustering.catchup.storecopy.FileHeaderEncoder; +import org.neo4j.causalclustering.catchup.storecopy.GetIndexFilesRequest; +import org.neo4j.causalclustering.catchup.storecopy.GetStoreFileRequest; +import org.neo4j.causalclustering.catchup.storecopy.GetStoreIdResponseEncoder; +import org.neo4j.causalclustering.catchup.storecopy.PrepareStoreCopyRequestDecoder; +import org.neo4j.causalclustering.catchup.storecopy.PrepareStoreCopyResponse; +import org.neo4j.causalclustering.catchup.storecopy.StoreCopyFinishedResponseEncoder; +import org.neo4j.causalclustering.core.state.snapshot.CoreSnapshotEncoder; +import org.neo4j.causalclustering.handlers.ExceptionLoggingHandler; +import org.neo4j.causalclustering.handlers.ExceptionSwallowingHandler; +import org.neo4j.causalclustering.handlers.PipelineWrapper; +import org.neo4j.causalclustering.handlers.VoidPipelineWrapperFactory; +import org.neo4j.causalclustering.identity.StoreId; +import org.neo4j.causalclustering.messaging.TestServer; +import org.neo4j.logging.LogProvider; +import org.neo4j.logging.NullLogProvider; +import org.neo4j.ports.allocation.PortAuthority; + +public abstract class StrippedCatchupServer +{ + private TestServer catchupServer; + + private static final LogProvider LOG_PROVIDER = NullLogProvider.getInstance(); + private int port; + private StoreId storeId; + + public void before() + { + this.port = PortAuthority.allocatePort(); + this.storeId = new StoreId( 1, 2, 3, 4 ); + catchupServer = new TestServer( port, new ChannelInitializer() + { + + @Override + protected void initChannel( SocketChannel ch ) throws Exception + { + + CatchupServerProtocol protocol = new CatchupServerProtocol(); + ChannelPipeline pipeline = ch.pipeline(); + + PipelineWrapper pipelineWrapper = VoidPipelineWrapperFactory.VOID_WRAPPER; + for ( ChannelHandler handler : pipelineWrapper.handlersFor( ch ) ) + { + pipeline.addLast( handler ); + } + + pipeline.addLast( new LengthFieldBasedFrameDecoder( Integer.MAX_VALUE, 0, 4, 0, 4 ) ); + pipeline.addLast( new LengthFieldPrepender( 4 ) ); + + pipeline.addLast( new VersionDecoder( LOG_PROVIDER ) ); + pipeline.addLast( new VersionPrepender() ); + + pipeline.addLast( new ResponseMessageTypeEncoder() ); + pipeline.addLast( new RequestMessageTypeEncoder() ); + + pipeline.addLast( new CoreSnapshotEncoder() ); + pipeline.addLast( new GetStoreIdResponseEncoder() ); + pipeline.addLast( new StoreCopyFinishedResponseEncoder() ); + pipeline.addLast( new FileChunkEncoder() ); + pipeline.addLast( new FileHeaderEncoder() ); + pipeline.addLast( new PrepareStoreCopyResponse.Encoder() ); + + pipeline.addLast( new ServerMessageTypeHandler( protocol, LOG_PROVIDER ) ); + + pipeline.addLast( decoders( protocol ) ); + + pipeline.addLast( new ChunkedWriteHandler() ); + + pipeline.addLast( getStoreListingRequestHandler( protocol ) ); + pipeline.addLast( getStoreFileRequestHandler( protocol ) ); + pipeline.addLast( getIndexRequestHandler( protocol ) ); + + pipeline.addLast( new ExceptionLoggingHandler( LOG_PROVIDER.getLog( ExceptionLoggingHandler.class ) ) ); + pipeline.addLast( new ExceptionSwallowingHandler() ); + } + } ); + } + + public void start() + { + catchupServer.start(); + } + + public void stop() + { + catchupServer.stop(); + } + + private ChannelInboundHandler decoders( CatchupServerProtocol protocol ) + { + RequestDecoderDispatcher decoderDispatcher = new RequestDecoderDispatcher<>( protocol, LOG_PROVIDER ); + decoderDispatcher.register( CatchupServerProtocol.State.PREPARE_STORE_COPY, new PrepareStoreCopyRequestDecoder() ); + decoderDispatcher.register( CatchupServerProtocol.State.GET_STORE_FILE, new GetStoreFileRequest.Decoder() ); + decoderDispatcher.register( CatchupServerProtocol.State.GET_INDEX_SNAPSHOT, new GetIndexFilesRequest.Decoder() ); + return decoderDispatcher; + } + + protected abstract ChannelHandler getStoreFileRequestHandler( CatchupServerProtocol protocol ); + + protected abstract ChannelHandler getStoreListingRequestHandler( CatchupServerProtocol protocol ); + + public int getPort() + { + return port; + } + + public StoreId getStoreId() + { + return storeId; + } + + protected abstract ChannelHandler getIndexRequestHandler( CatchupServerProtocol protocol ); +} diff --git a/enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/catchup/storecopy/CatchupServerIT.java b/enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/catchup/storecopy/CatchupServerIT.java new file mode 100644 index 0000000000000..06aac7eec2faf --- /dev/null +++ b/enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/catchup/storecopy/CatchupServerIT.java @@ -0,0 +1,343 @@ +/* + * 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 Affero 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 Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + */ +package org.neo4j.causalclustering.catchup.storecopy; + +import org.junit.After; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; + +import java.io.File; +import java.io.IOException; +import java.time.Clock; +import java.util.Arrays; +import java.util.Collection; +import java.util.List; +import java.util.Optional; +import java.util.function.Predicate; +import java.util.stream.Stream; + +import org.neo4j.causalclustering.StrippedCatchupServer; +import org.neo4j.causalclustering.catchup.CatchUpClient; +import org.neo4j.causalclustering.handlers.VoidPipelineWrapperFactory; +import org.neo4j.causalclustering.identity.StoreId; +import org.neo4j.graphdb.Label; +import org.neo4j.graphdb.Node; +import org.neo4j.graphdb.RelationshipType; +import org.neo4j.graphdb.Transaction; +import org.neo4j.graphdb.index.Index; +import org.neo4j.io.pagecache.PageCache; +import org.neo4j.kernel.NeoStoreDataSource; +import org.neo4j.kernel.api.schema.index.IndexDescriptor; +import org.neo4j.kernel.impl.store.StoreType; +import org.neo4j.kernel.internal.GraphDatabaseAPI; +import org.neo4j.kernel.monitoring.Monitors; +import org.neo4j.logging.LogProvider; +import org.neo4j.logging.NullLogProvider; +import org.neo4j.storageengine.api.StoreFileMetadata; +import org.neo4j.test.TestGraphDatabaseFactory; +import org.neo4j.test.rule.TestDirectory; +import org.neo4j.test.rule.fs.DefaultFileSystemRule; + +import static java.util.stream.Collectors.toList; +import static org.hamcrest.Matchers.containsInAnyOrder; +import static org.hamcrest.Matchers.equalTo; +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotEquals; +import static org.junit.Assert.assertThat; +import static org.neo4j.causalclustering.catchup.storecopy.RealStrippedCatchupServer.getCheckPointer; +import static org.neo4j.causalclustering.catchup.storecopy.RealStrippedCatchupServer.getNeoStoreDataSource; +import static org.neo4j.graphdb.Label.label; +import static org.neo4j.io.fs.FileUtils.relativePath; + +public class CatchupServerIT +{ + private static final String EXISTING_FILE_NAME = "neostore.nodestore.db"; + private static final StoreId WRONG_STORE_ID = new StoreId( 123, 221, 3131, 45678 ); + private static final LogProvider LOG_PROVIDER = NullLogProvider.getInstance(); + + private static final String PROP_NAME = "name"; + private static final String PROP = "prop"; + public static final Label LABEL = label( "MyLabel" ); + private static final String INDEX = "index"; + private GraphDatabaseAPI graphDb; + private StrippedCatchupServer catchupServer; + private File temporaryDirectory; + + private PageCache pageCache; + + @Rule + public DefaultFileSystemRule fsa = new DefaultFileSystemRule(); + @Rule + public TestDirectory testDirectory = TestDirectory.testDirectory( fsa ); + private CatchUpClient catchUpClient; + + @Before + public void startDb() + { + temporaryDirectory = testDirectory.directory(); + graphDb = (GraphDatabaseAPI) new TestGraphDatabaseFactory().setFileSystem( fsa ).newEmbeddedDatabase( testDirectory.graphDbDir() ); + createLegacyIndex(); + createPropertyIndex(); + addData( graphDb ); + catchupServer = new RealStrippedCatchupServer( fsa, graphDb ); + catchupServer.before(); + catchupServer.start(); + catchUpClient = new CatchUpClient( LOG_PROVIDER, Clock.systemUTC(), 10000, new Monitors(), VoidPipelineWrapperFactory.VOID_WRAPPER ); + catchUpClient.start(); + pageCache = graphDb.getDependencyResolver().resolveDependency( PageCache.class ); + } + + @After + public void stopDb() throws IOException + { + pageCache.flushAndForce(); + if ( graphDb != null ) + { + graphDb.shutdown(); + } + Optional.ofNullable( catchupServer ).ifPresent( StrippedCatchupServer::stop ); + Optional.ofNullable( catchUpClient ).ifPresent( CatchUpClient::stop ); + } + + @Test + public void shouldListExpectedFilesCorrectly() throws Exception + { + // given (setup) required runtime subject dependencies + NeoStoreDataSource neoStoreDataSource = getNeoStoreDataSource( graphDb ); + SimpleCatchupClient simpleCatchupClient = new SimpleCatchupClient( graphDb, fsa, catchUpClient, catchupServer, temporaryDirectory, LOG_PROVIDER ); + + // when + PrepareStoreCopyResponse prepareStoreCopyResponse = simpleCatchupClient.requestListOfFilesFromServer(); + simpleCatchupClient.close(); + + // then + listOfDownloadedFilesMatchesServer( neoStoreDataSource, prepareStoreCopyResponse.getFiles() ); + + // and downloaded files are identical to source + List expectedCountStoreFiles = listServerExpectedNonReplayableFiles( neoStoreDataSource ); + for ( File snapshotedStorefile : expectedCountStoreFiles ) + { + fileContentEquals( databaseFileToClientFile( snapshotedStorefile ), snapshotedStorefile ); + } + + // and + assertTransactionIdMatches( prepareStoreCopyResponse.lastTransactionId() ); + + //and + assertDescriptorsMatch( prepareStoreCopyResponse.getDescriptors(), neoStoreDataSource ); + } + + @Test + public void shouldCommunicateErrorIfStoreIdDoesNotMatchRequest() throws Exception + { + // given (setup) required runtime subject dependencies + addData( graphDb ); + SimpleCatchupClient simpleCatchupClient = new SimpleCatchupClient( graphDb, fsa, catchUpClient, catchupServer, temporaryDirectory, LOG_PROVIDER ); + + // when the list of files are requested from the server with the wrong storeId + PrepareStoreCopyResponse prepareStoreCopyResponse = simpleCatchupClient.requestListOfFilesFromServer( WRONG_STORE_ID ); + simpleCatchupClient.close(); + + // then the response is not a list of files but an error + assertEquals( PrepareStoreCopyResponse.Status.E_STORE_ID_MISMATCH, prepareStoreCopyResponse.status() ); + + // and the list of files is empty because the request should have failed + File[] remoteFiles = prepareStoreCopyResponse.getFiles(); + assertArrayEquals( new File[]{}, remoteFiles ); + } + + @Test + public void individualFileCopyWorks() throws Exception + { + // given a file exists on the server + addData( graphDb ); + File existingFile = new File( temporaryDirectory, EXISTING_FILE_NAME ); + + // and + SimpleCatchupClient simpleCatchupClient = new SimpleCatchupClient( graphDb, fsa, catchUpClient, catchupServer, temporaryDirectory, LOG_PROVIDER ); + + // when we copy that file + pageCache.flushAndForce(); + StoreCopyFinishedResponse storeCopyFinishedResponse = simpleCatchupClient.requestIndividualFile( existingFile ); + simpleCatchupClient.close(); + + // then the response is successful + assertEquals( StoreCopyFinishedResponse.Status.SUCCESS, storeCopyFinishedResponse.status() ); + + // then the contents matches + fileContentEquals( clientFileToDatabaseFile( existingFile ), existingFile ); + } + + @Test + public void individualIndexSnapshotCopyWorks() throws Exception + { + + // given + NeoStoreDataSource neoStoreDataSource = getNeoStoreDataSource( graphDb ); + List expectingFiles = neoStoreDataSource.getNeoStoreFileListing().builder().excludeAll().includeSchemaIndexStoreFiles().build().stream().map( + StoreFileMetadata::file ).collect( toList() ); + SimpleCatchupClient simpleCatchupClient = new SimpleCatchupClient( graphDb, fsa, catchUpClient, catchupServer, temporaryDirectory, LOG_PROVIDER ); + + // and + Collection expectedDescriptors = getExpectedDescriptors( neoStoreDataSource ); + + // when + for ( IndexDescriptor expectedDescriptor : expectedDescriptors ) + { + StoreCopyFinishedResponse response = simpleCatchupClient.requestIndexSnapshot( expectedDescriptor ); + simpleCatchupClient.close(); + assertEquals( StoreCopyFinishedResponse.Status.SUCCESS, response.status() ); + } + + // then + fileContentEquals( expectingFiles ); + } + + @Test + public void individualFileCopyFailsIfStoreIdMismatch() throws Exception + { + // given a file exists on the server + addData( graphDb ); + File expectedExistingFile = new File( graphDb.getStoreDir(), EXISTING_FILE_NAME ); + + // and + SimpleCatchupClient simpleCatchupClient = new SimpleCatchupClient( graphDb, fsa, catchUpClient, catchupServer, temporaryDirectory, LOG_PROVIDER ); + + // when we copy that file using a different storeId + StoreCopyFinishedResponse storeCopyFinishedResponse = simpleCatchupClient.requestIndividualFile( expectedExistingFile, WRONG_STORE_ID ); + simpleCatchupClient.close(); + + // then the response from the server should be an error message that describes a store ID mismatch + assertEquals( StoreCopyFinishedResponse.Status.E_STORE_ID_MISMATCH, storeCopyFinishedResponse.status() ); + } + + private void assertTransactionIdMatches( long lastTxId ) + { + long expectedTransactionId = getCheckPointer( graphDb ).lastCheckPointedTransactionId(); + assertEquals( expectedTransactionId, lastTxId); + } + + private void fileContentEquals( Collection countStore ) throws IOException + { + for ( File file : countStore ) + { + fileContentEquals( databaseFileToClientFile( file ), file ); + } + } + + private File databaseFileToClientFile( File file ) throws IOException + { + String relativePathToDatabaseDir = relativePath( new File( temporaryDirectory, "graph-db" ), file ); + return new File( temporaryDirectory, relativePathToDatabaseDir ); + } + + private File clientFileToDatabaseFile( File file ) throws IOException + { + String relativePathToDatabaseDir = relativePath( temporaryDirectory, file ); + return new File( new File( temporaryDirectory, "graph-db" ), relativePathToDatabaseDir ); + } + + private void fileContentEquals( File fileA, File fileB ) + { + assertNotEquals( fileA.getPath(), fileB.getPath() ); + String message = String.format( "Expected file: %s\ndoes not match actual file: %s", fileA, fileB ); + assertEquals( message, StoreCopyClientIT.fileContent( fileA, fsa ), StoreCopyClientIT.fileContent( fileB, fsa ) ); + } + + private void listOfDownloadedFilesMatchesServer( NeoStoreDataSource neoStoreDataSource, File[] files ) + throws IOException + { + List expectedStoreFiles = getExpectedStoreFiles( neoStoreDataSource ); + List givenFile = Arrays.stream( files ).map( File::getName ).collect( toList() ); + assertThat( givenFile, containsInAnyOrder( expectedStoreFiles.toArray( new String[givenFile.size()] ) ) ); + } + + private void assertDescriptorsMatch( IndexDescriptor[] descriptors, NeoStoreDataSource neoStoreDataSource ) + { + Collection expectedDescriptors = getExpectedDescriptors( neoStoreDataSource ); + + assertThat( expectedDescriptors, containsInAnyOrder( descriptors ) ); + assertThat( expectedDescriptors.size(), equalTo( descriptors.length ) ); + } + + private Collection getExpectedDescriptors( NeoStoreDataSource neoStoreDataSource ) + { + return neoStoreDataSource.getNeoStoreFileListing().getNeoStoreFileIndexListing().listIndexDescriptors(); + } + + private List listServerExpectedNonReplayableFiles( NeoStoreDataSource neoStoreDataSource ) throws IOException + { + try ( Stream countStoreStream = neoStoreDataSource.getNeoStoreFileListing().builder().excludeAll() + .includeNeoStoreFiles().build().stream(); + Stream explicitIndexStream = neoStoreDataSource.getNeoStoreFileListing().builder().excludeAll() + .includeExplicitIndexStoreStoreFiles().build().stream() ) + { + return Stream.concat( countStoreStream.filter( isCountFile() ), explicitIndexStream ).map( StoreFileMetadata::file ).collect( toList() ); + } + } + + private List getExpectedStoreFiles( NeoStoreDataSource neoStoreDataSource ) throws IOException + { + try ( Stream stream = neoStoreDataSource.getNeoStoreFileListing().builder() + .excludeLogFiles().excludeExplicitIndexStoreFiles().excludeSchemaIndexStoreFiles().build().stream() ) + { + return stream.filter( isCountFile().negate() ).map( sfm -> sfm.file().getName() ).collect( toList() ); + } + } + + private static Predicate isCountFile() + { + return storeFileMetadata -> StoreType.typeOf( storeFileMetadata.file().getName() ).filter( f -> f == StoreType.COUNTS ).isPresent(); + } + + private void addData( GraphDatabaseAPI graphDb ) + { + try ( Transaction tx = graphDb.beginTx() ) + { + Node node = graphDb.createNode(); + node.addLabel( LABEL ); + node.setProperty( PROP_NAME, "Neo" ); + node.setProperty( PROP, Math.random() * 10000 ); + graphDb.createNode().createRelationshipTo( node, RelationshipType.withName( "KNOWS" ) ); + tx.success(); + } + } + + private void createPropertyIndex() + { + try ( Transaction tx = graphDb.beginTx() ) + { + graphDb.schema().indexFor( LABEL ).on( PROP_NAME ).create(); + tx.success(); + } + } + + private void createLegacyIndex() + { + try ( Transaction tx = graphDb.beginTx() ) + { + Index nodeIndex = graphDb.index().forNodes( INDEX ); + nodeIndex.add( graphDb.createNode(), "some-key", "som-value" ); + tx.success(); + } + } +} diff --git a/enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/catchup/storecopy/CloseablesListenerTest.java b/enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/catchup/storecopy/CloseablesListenerTest.java new file mode 100644 index 0000000000000..3f8c4e5132f85 --- /dev/null +++ b/enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/catchup/storecopy/CloseablesListenerTest.java @@ -0,0 +1,84 @@ +/* + * 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 Affero 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 Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + */ +package org.neo4j.causalclustering.catchup.storecopy; + +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; + +import static org.junit.Assert.assertTrue; + +public class CloseablesListenerTest +{ + @Rule + public final ExpectedException expectedException = ExpectedException.none(); + + @Test + public void shouldCloseAllReourcesBeforeException() throws Exception + { + // given + CloseablesListener closeablesListener = new CloseablesListener(); + RuntimeException exception = new RuntimeException( "fail" ); + CloseTrackingCloseable kindCloseable1 = new CloseTrackingCloseable(); + CloseTrackingCloseable unkindCloseable = new CloseTrackingCloseable( exception ); + CloseTrackingCloseable kindCloseable2 = new CloseTrackingCloseable(); + closeablesListener.add( kindCloseable1 ); + closeablesListener.add( unkindCloseable ); + closeablesListener.add( kindCloseable2 ); + + //then we expect an exception + expectedException.expect( exception.getClass() ); + + // when + closeablesListener.close(); + + //then we expect all have closed + assertTrue( kindCloseable1.wasClosed ); + assertTrue( unkindCloseable.wasClosed ); + assertTrue( kindCloseable2.wasClosed ); + } + + class CloseTrackingCloseable implements AutoCloseable + { + private final Exception throwOnClose; + + private CloseTrackingCloseable() + { + this( null ); + } + + CloseTrackingCloseable( Exception throwOnClose ) + { + this.throwOnClose = throwOnClose; + } + + boolean wasClosed; + + @Override + public void close() throws Exception + { + wasClosed = true; + if ( throwOnClose != null ) + { + throw throwOnClose; + } + } + } +} diff --git a/enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/catchup/storecopy/FakeCatchupServer.java b/enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/catchup/storecopy/FakeCatchupServer.java new file mode 100644 index 0000000000000..4974c929b5ba9 --- /dev/null +++ b/enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/catchup/storecopy/FakeCatchupServer.java @@ -0,0 +1,210 @@ +/* + * 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 Affero 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 Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + */ +package org.neo4j.causalclustering.catchup.storecopy; + +import io.netty.channel.ChannelHandler; +import io.netty.channel.ChannelHandlerContext; +import io.netty.channel.SimpleChannelInboundHandler; + +import java.io.File; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; + +import org.neo4j.causalclustering.StrippedCatchupServer; +import org.neo4j.causalclustering.catchup.CatchupServerProtocol; +import org.neo4j.causalclustering.catchup.ResponseMessageType; +import org.neo4j.io.pagecache.PageCache; +import org.neo4j.kernel.api.schema.LabelSchemaDescriptor; +import org.neo4j.kernel.api.schema.index.IndexDescriptor; +import org.neo4j.logging.Log; +import org.neo4j.logging.LogProvider; + +import static org.mockito.Mockito.mock; + +/** + * The purpose of having a fake catchup server, that has separate code from the production catchup server is so that we have finer grained control over how it + * behaves. Otherwise we would need to prepare difficult conditions on a production instance which could be difficult to compose or reproduce. + */ +class FakeCatchupServer extends StrippedCatchupServer +{ + private StoreCopyClientIT storeCopyClientIT; + private final Set filesystem = new HashSet<>(); + private final Set indexFiles = new HashSet<>(); + private final Map pathToRequestCountMapping = new HashMap<>(); + private final Log log; + + FakeCatchupServer( StoreCopyClientIT storeCopyClientIT, LogProvider logProvider ) + { + this.storeCopyClientIT = storeCopyClientIT; + log = logProvider.getLog( FakeCatchupServer.class ); + } + + void addFile( FakeFile fakeFile ) + { + filesystem.add( fakeFile ); + } + + void addIndexFile( FakeFile fakeFile ) + { + indexFiles.add( fakeFile ); + } + + int getRequestCount( String file ) + { + return pathToRequestCountMapping.getOrDefault( file, 0 ); + } + + @Override + protected ChannelHandler getStoreFileRequestHandler( CatchupServerProtocol protocol ) + { + return new SimpleChannelInboundHandler() + { + @Override + protected void channelRead0( ChannelHandlerContext channelHandlerContext, GetStoreFileRequest getStoreFileRequest ) + { + log.info( "Received request for file %s", getStoreFileRequest.file().getName() ); + incrementRequestCount( getStoreFileRequest.file() ); + try + { + if ( handleFileDoesNotExist( channelHandlerContext, getStoreFileRequest ) ) + { + protocol.expect( CatchupServerProtocol.State.MESSAGE_TYPE ); + return; + } + handleFileExists( channelHandlerContext, getStoreFileRequest.file() ); + } + finally + { + protocol.expect( CatchupServerProtocol.State.MESSAGE_TYPE ); + } + } + }; + } + + private boolean handleFileDoesNotExist( ChannelHandlerContext channelHandlerContext, GetStoreFileRequest getStoreFileRequest ) + { + FakeFile file = findFile( filesystem, getStoreFileRequest.file().getName() ); + if ( file.getRemainingFailed() > 0 ) + { + file.setRemainingFailed( file.getRemainingFailed() - 1 ); + log.info( "FakeServer failing for file %s", getStoreFileRequest.file() ); + failed( channelHandlerContext ); + return true; + } + else if ( file.getRemainingNoResponse() > 0 ) + { + log.info( "FakeServer not going to response for file %s", getStoreFileRequest.file() ); + file.setRemainingNoResponse( file.getRemainingNoResponse() - 1 ); + return true; // no response + } + return false; + } + + private void failed( ChannelHandlerContext channelHandlerContext ) + { + new StoreFileStreamingProtocol().end( channelHandlerContext, StoreCopyFinishedResponse.Status.E_TOO_FAR_BEHIND ); + } + + private FakeFile findFile( Set filesystem, String filename ) + { + return filesystem.stream() + .filter( fakeFile -> filename.equals( fakeFile.getFilename() ) ) + .findFirst() + .orElseThrow( () -> new RuntimeException( "FakeFile should handle all cases with regards to how server should respond" ) ); + } + + private boolean handleFileExists( ChannelHandlerContext channelHandlerContext, File file ) + { + log.info( "FakeServer File %s does exist", file ); + channelHandlerContext.writeAndFlush( ResponseMessageType.FILE ); + channelHandlerContext.writeAndFlush( new FileHeader( file.getName() ) ); + StoreResource storeResource = storeResourceFromEntry( file ); + channelHandlerContext.writeAndFlush( new FileSender( storeResource ) ); + new StoreFileStreamingProtocol().end( channelHandlerContext, StoreCopyFinishedResponse.Status.SUCCESS ); + return true; + } + + private void incrementRequestCount( File file ) + { + String path = file.getName(); + int count = pathToRequestCountMapping.getOrDefault( path, 0 ); + pathToRequestCountMapping.put( path, count + 1 ); + } + + private StoreResource storeResourceFromEntry( File file ) + { + file = storeCopyClientIT.testDirectory.file( file.getName() ); + return new StoreResource( file, file.getAbsolutePath(), 16, mock( PageCache.class ), storeCopyClientIT.fileSystemAbstraction ); + } + + @Override + protected ChannelHandler getStoreListingRequestHandler( CatchupServerProtocol protocol ) + { + return new SimpleChannelInboundHandler() + { + + @Override + protected void channelRead0( ChannelHandlerContext channelHandlerContext, PrepareStoreCopyRequest prepareStoreCopyRequest ) + { + channelHandlerContext.writeAndFlush( ResponseMessageType.PREPARE_STORE_COPY_RESPONSE ); + List list = filesystem.stream().map( FakeFile::getFile ).collect( Collectors.toList() ); + File[] files = new File[list.size()]; + files = list.toArray( files ); + long transactionId = 123L; + channelHandlerContext.writeAndFlush( PrepareStoreCopyResponse.success( files, + new IndexDescriptor[]{new IndexDescriptor( new LabelSchemaDescriptor( 1, 2, 3 ), IndexDescriptor.Type.GENERAL )}, transactionId ) ); + protocol.expect( CatchupServerProtocol.State.MESSAGE_TYPE ); + } + }; + } + + @Override + protected ChannelHandler getIndexRequestHandler( CatchupServerProtocol protocol ) + { + return new SimpleChannelInboundHandler() + { + @Override + protected void channelRead0( ChannelHandlerContext channelHandlerContext, GetIndexFilesRequest snapshotRequest ) + { + log.info( "Received request for index %s", snapshotRequest.descriptor() ); + try + { + for ( FakeFile indexFile : indexFiles ) + { + log.info( "FakeServer File %s does exist", indexFile.getFile() ); + channelHandlerContext.writeAndFlush( ResponseMessageType.FILE ); + channelHandlerContext.writeAndFlush( new FileHeader( indexFile.getFile().getName() ) ); + StoreResource storeResource = storeResourceFromEntry( indexFile.getFile() ); + channelHandlerContext.writeAndFlush( new FileSender( storeResource ) ); + } + new StoreFileStreamingProtocol().end( channelHandlerContext, StoreCopyFinishedResponse.Status.SUCCESS ); + } + finally + { + protocol.expect( CatchupServerProtocol.State.MESSAGE_TYPE ); + } + } + }; + } +} diff --git a/enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/catchup/storecopy/FakeFile.java b/enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/catchup/storecopy/FakeFile.java new file mode 100644 index 0000000000000..1c45beec4d340 --- /dev/null +++ b/enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/catchup/storecopy/FakeFile.java @@ -0,0 +1,115 @@ +/* + * 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 Affero 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 Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + */ +package org.neo4j.causalclustering.catchup.storecopy; + +import java.io.File; +import java.nio.file.Path; +import java.util.Optional; + +/** + * A fake file tracks a file, but also several counters and helpers that can be used in tests to invoke desired behaviour + */ +class FakeFile +{ + private File file; + private String filename; + private String content; + private int remainingNoResponse; + private int remainingFailed; + private Path relativePath; + + FakeFile( String name, String content ) + { + setFilename( name ); + this.content = content; + } + + public void setFilename( String filename ) + { + this.filename = filename; + this.file = getRelativePath().resolve( filename ).toFile(); + } + + public void setFile( File file ) + { + this.filename = file.getName(); + this.file = file; + } + + private Path getRelativePath() + { + return Optional.ofNullable( relativePath ).orElse( new File( "." ).toPath() ); + } + + public File getFile() + { + return file; + } + + public String getFilename() + { + return filename; + } + + public String getContent() + { + return content; + } + + public void setContent( String content ) + { + this.content = content; + } + + /** + * Number of times that requests for this file will result in communication breakdown (i.e. no response) + * + * @return + */ + int getRemainingNoResponse() + { + return remainingNoResponse; + } + + void setRemainingNoResponse( int remainingNoResponse ) + { + this.remainingNoResponse = remainingNoResponse; + } + + public void setRelativePath( Path relativePath ) + { + this.relativePath = relativePath; + } + + /** + * Clear response that the file has failed to copy (safe connection close, communication, ...) + * + * @return + */ + int getRemainingFailed() + { + return remainingFailed; + } + + void setRemainingFailed( int remainingFailed ) + { + this.remainingFailed = remainingFailed; + } +} diff --git a/enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/catchup/storecopy/GetIndexFilesRequestMarshalTest.java b/enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/catchup/storecopy/GetIndexFilesRequestMarshalTest.java new file mode 100644 index 0000000000000..2807b01e56d29 --- /dev/null +++ b/enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/catchup/storecopy/GetIndexFilesRequestMarshalTest.java @@ -0,0 +1,70 @@ +/* + * 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 Affero 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 Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + */ +package org.neo4j.causalclustering.catchup.storecopy; + +import io.netty.buffer.ByteBuf; +import io.netty.channel.embedded.EmbeddedChannel; +import org.junit.Before; +import org.junit.Test; + +import org.neo4j.causalclustering.identity.StoreId; +import org.neo4j.kernel.api.schema.LabelSchemaDescriptor; +import org.neo4j.kernel.api.schema.index.IndexDescriptor; + +import static org.junit.Assert.assertEquals; + +public class GetIndexFilesRequestMarshalTest +{ + private EmbeddedChannel embeddedChannel; + + @Before + public void setup() + { + embeddedChannel = new EmbeddedChannel( new GetIndexFilesRequest.Encoder(), new GetIndexFilesRequest.Decoder() ); + } + + private static final StoreId expectedStore = new StoreId( 1, 2, 3, 4 ); + private static final IndexDescriptor exepctedDescriptor = new IndexDescriptor( new LabelSchemaDescriptor( 1, 2, 3 ), IndexDescriptor.Type.GENERAL ); + private static final Long expectedLastTransaction = 1234L; + + @Test + public void getsTransmitted() + { + // given + GetIndexFilesRequest expectedIndexSnapshotRequest = new GetIndexFilesRequest( expectedStore, exepctedDescriptor, expectedLastTransaction ); + + // when + sendToChannel( expectedIndexSnapshotRequest, embeddedChannel ); + + // then + GetIndexFilesRequest actualIndexRequest = embeddedChannel.readInbound(); + assertEquals( expectedStore, actualIndexRequest.expectedStoreId() ); + assertEquals( exepctedDescriptor, actualIndexRequest.descriptor() ); + assertEquals( expectedLastTransaction.longValue(), actualIndexRequest.requiredTransactionId() ); + } + + private static void sendToChannel( GetIndexFilesRequest expectedIndexSnapshotRequest, EmbeddedChannel embeddedChannel ) + { + embeddedChannel.writeOutbound( expectedIndexSnapshotRequest ); + + ByteBuf object = embeddedChannel.readOutbound(); + embeddedChannel.writeInbound( object ); + } +} diff --git a/enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/catchup/storecopy/GetStoreFileMarshalTest.java b/enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/catchup/storecopy/GetStoreFileMarshalTest.java new file mode 100644 index 0000000000000..4ff1eb3fee4ed --- /dev/null +++ b/enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/catchup/storecopy/GetStoreFileMarshalTest.java @@ -0,0 +1,70 @@ +/* + * 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 Affero 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 Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + */ +package org.neo4j.causalclustering.catchup.storecopy; + +import io.netty.buffer.ByteBuf; +import io.netty.channel.embedded.EmbeddedChannel; +import org.junit.Before; +import org.junit.Test; + +import java.io.File; + +import org.neo4j.causalclustering.identity.StoreId; + +import static org.junit.Assert.assertEquals; + +public class GetStoreFileMarshalTest +{ + EmbeddedChannel embeddedChannel; + + @Before + public void setup() + { + embeddedChannel = new EmbeddedChannel( new GetStoreFileRequest.Encoder(), new GetStoreFileRequest.Decoder() ); + } + + private static final StoreId expectedStore = new StoreId( 1, 2, 3, 4 ); + private static final File expectedFile = new File( "abc.123" ); + private static final Long expectedLastTransaction = 1234L; + + @Test + public void getsTransmitted() + { + // given + GetStoreFileRequest expectedStoreRequest = new GetStoreFileRequest( expectedStore, expectedFile, expectedLastTransaction ); + + // when + sendToChannel( expectedStoreRequest, embeddedChannel ); + + // then + GetStoreFileRequest actualStoreRequest = embeddedChannel.readInbound(); + assertEquals( expectedStore, actualStoreRequest.expectedStoreId() ); + assertEquals( expectedFile, actualStoreRequest.file() ); + assertEquals( expectedLastTransaction.longValue(), actualStoreRequest.requiredTransactionId() ); + } + + private static void sendToChannel( GetStoreFileRequest getStoreFileRequest, EmbeddedChannel embeddedChannel ) + { + embeddedChannel.writeOutbound( getStoreFileRequest ); + + ByteBuf object = embeddedChannel.readOutbound(); + embeddedChannel.writeInbound( object ); + } +} diff --git a/enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/catchup/storecopy/GetStoreRequestHandlerTest.java b/enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/catchup/storecopy/GetStoreRequestHandlerTest.java deleted file mode 100644 index ae6c0e0704f10..0000000000000 --- a/enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/catchup/storecopy/GetStoreRequestHandlerTest.java +++ /dev/null @@ -1,77 +0,0 @@ -/* - * 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 Affero 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 Affero General Public License for more details. - * - * You should have received a copy of the GNU Affero General Public License - * along with this program. If not, see . - */ -package org.neo4j.causalclustering.catchup.storecopy; - -import io.netty.channel.ChannelHandlerContext; -import org.junit.Test; - -import org.neo4j.causalclustering.catchup.CatchupServerProtocol; -import org.neo4j.causalclustering.identity.StoreId; -import org.neo4j.kernel.NeoStoreDataSource; - -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.when; -import static org.neo4j.causalclustering.catchup.storecopy.StoreCopyFinishedResponse.Status.E_STORE_ID_MISMATCH; - -public class GetStoreRequestHandlerTest -{ - private final StoreStreamingProcess streamingProcess = mock( StoreStreamingProcess.class ); - private final NeoStoreDataSource dataSource = mock( NeoStoreDataSource.class ); - private final ChannelHandlerContext ctx = mock( ChannelHandlerContext.class ); - - private final CatchupServerProtocol protocol = new CatchupServerProtocol(); - - @Test - public void shouldInvokedStoreCopyProcess() throws Exception - { - // given - when( dataSource.getStoreId() ).thenReturn( new org.neo4j.kernel.impl.store.StoreId( 1, 2, 0, 3, 4 ) ); - GetStoreRequestHandler handler = new GetStoreRequestHandler( protocol, () -> dataSource, streamingProcess ); - - StoreId storeId = new StoreId( 1, 2, 3, 4 ); - GetStoreRequest msg = new GetStoreRequest( storeId ); - - // when - handler.channelRead0( ctx, msg ); - - // then - verify( streamingProcess ).perform( ctx ); - protocol.isExpecting( CatchupServerProtocol.State.MESSAGE_TYPE ); - } - - @Test - public void shouldFailStoreCopyProcessOnWrongStoreId() throws Exception - { - // given - GetStoreRequestHandler handler = new GetStoreRequestHandler( protocol, () -> dataSource, streamingProcess ); - when( dataSource.getStoreId() ).thenReturn( new org.neo4j.kernel.impl.store.StoreId( 5, 6, 7, 8, 9 ) ); - - StoreId storeId = new StoreId( 1, 2, 3, 4 ); - GetStoreRequest msg = new GetStoreRequest( storeId ); - - // when - handler.channelRead0( ctx, msg ); - - // then - verify( streamingProcess ).fail( ctx, E_STORE_ID_MISMATCH ); - protocol.isExpecting( CatchupServerProtocol.State.MESSAGE_TYPE ); - } -} diff --git a/enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/catchup/storecopy/InMemoryFileSystemStream.java b/enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/catchup/storecopy/InMemoryFileSystemStream.java new file mode 100644 index 0000000000000..5cfd977d43175 --- /dev/null +++ b/enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/catchup/storecopy/InMemoryFileSystemStream.java @@ -0,0 +1,57 @@ +/* + * 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 Affero 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 Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + */ +package org.neo4j.causalclustering.catchup.storecopy; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; + +class InMemoryFileSystemStream implements StoreFileStreams +{ + Map filesystem = new HashMap<>(); + + /** + * + * @param destination + * @param requiredAlignment + * @param data + * @throws IOException + */ + public void write( String destination, int requiredAlignment, byte[] data ) throws IOException + { + StringBuffer buffer = filesystem.getOrDefault( destination, new StringBuffer() ); + for ( byte b : data ) + { + buffer.append( (char) b ); + } + filesystem.put( destination, buffer ); + } + + @Override + public void close() throws Exception + { + throw new RuntimeException( "Unimplemented" ); + } + + public Map getFilesystem() + { + return filesystem; + } +} diff --git a/enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/catchup/storecopy/MaximumTotalRetriesTest.java b/enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/catchup/storecopy/MaximumTotalRetriesTest.java new file mode 100644 index 0000000000000..58ae1c8c01c6e --- /dev/null +++ b/enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/catchup/storecopy/MaximumTotalRetriesTest.java @@ -0,0 +1,70 @@ +/* + * 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 Affero 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 Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + */ +package org.neo4j.causalclustering.catchup.storecopy; + +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; + +import java.util.concurrent.TimeUnit; + +import org.neo4j.time.FakeClock; + +public class MaximumTotalRetriesTest +{ + @Rule + public ExpectedException expectedException = ExpectedException.none(); + + @Test + public void shouldRetryUntilMaximumRetries() throws Exception + { + FakeClock clock = new FakeClock(); + MaximumTotalRetries maximumTotalRetries = new MaximumTotalRetries( 4, -1, clock ); + + maximumTotalRetries.assertContinue(); + maximumTotalRetries.assertContinue(); + maximumTotalRetries.assertContinue(); + expectedException.expect( StoreCopyFailedException.class ); + maximumTotalRetries.assertContinue(); + } + + @Test + public void shouldContinueIfAllowedInBetweenTimeIsMet() throws Exception + { + // given + FakeClock clock = new FakeClock(); + MaximumTotalRetries maximumTotalRetries = new MaximumTotalRetries( 1, 0, clock ); + + // when we retry + maximumTotalRetries.assertContinue(); + + // then we can retry again because in between time == 0 + maximumTotalRetries.assertContinue(); + + //when we increment clock + clock.forward( 1, TimeUnit.MILLISECONDS ); + + //then we expected exception thrown + expectedException.expect( StoreCopyFailedException.class ); + + // when we retry + maximumTotalRetries.assertContinue(); + } +} diff --git a/enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/catchup/storecopy/PrepareStoreCopyFilesTest.java b/enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/catchup/storecopy/PrepareStoreCopyFilesTest.java new file mode 100644 index 0000000000000..b5ffcbc7fdf2f --- /dev/null +++ b/enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/catchup/storecopy/PrepareStoreCopyFilesTest.java @@ -0,0 +1,158 @@ +/* + * 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 Affero 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 Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + */ +package org.neo4j.causalclustering.catchup.storecopy; + +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; + +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; + +import org.neo4j.helpers.collection.Iterators; +import org.neo4j.io.fs.DefaultFileSystemAbstraction; +import org.neo4j.io.fs.FileSystemAbstraction; +import org.neo4j.io.pagecache.PageCache; +import org.neo4j.io.pagecache.impl.muninn.StandalonePageCacheFactory; +import org.neo4j.kernel.NeoStoreDataSource; +import org.neo4j.kernel.api.schema.LabelSchemaDescriptor; +import org.neo4j.kernel.api.schema.index.IndexDescriptor; +import org.neo4j.kernel.impl.transaction.state.NeoStoreFileIndexListing; +import org.neo4j.kernel.impl.transaction.state.NeoStoreFileListing; +import org.neo4j.storageengine.api.StoreFileMetadata; +import org.neo4j.test.rule.TestDirectory; + +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; +import static org.mockito.Mockito.CALLS_REAL_METHODS; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; +import static org.neo4j.io.fs.FileUtils.relativePath; + +public class PrepareStoreCopyFilesTest +{ + private final FileSystemAbstraction fileSystemAbstraction = new DefaultFileSystemAbstraction(); + + @Rule + public final TestDirectory testDirectory = TestDirectory.testDirectory( fileSystemAbstraction ); + private PrepareStoreCopyFiles prepareStoreCopyFiles; + private PageCache pageCache; + private NeoStoreFileIndexListing indexListingMock; + private File storeDir; + private NeoStoreFileListing.StoreFileListingBuilder fileListingBuilder; + + @Before + public void setUp() + { + pageCache = StandalonePageCacheFactory.createPageCache( fileSystemAbstraction ); + NeoStoreDataSource dataSource = mock( NeoStoreDataSource.class ); + fileListingBuilder = mock( NeoStoreFileListing.StoreFileListingBuilder.class, CALLS_REAL_METHODS ); + storeDir = testDirectory.graphDbDir(); + when( dataSource.getStoreDir() ).thenReturn( storeDir ); + indexListingMock = mock( NeoStoreFileIndexListing.class ); + when( indexListingMock.listIndexDescriptors() ).thenReturn( new ArrayList<>() ); + NeoStoreFileListing storeFileListing = mock( NeoStoreFileListing.class ); + when( storeFileListing.getNeoStoreFileIndexListing() ).thenReturn( indexListingMock ); + when( storeFileListing.builder() ).thenReturn( fileListingBuilder ); + when( dataSource.getNeoStoreFileListing() ).thenReturn( storeFileListing ); + prepareStoreCopyFiles = new PrepareStoreCopyFiles( dataSource, pageCache, fileSystemAbstraction ); + } + + @Test + public void shouldHanldeEmptyListOfFilesForeEachType() throws Exception + { + setExpectedFiles( new StoreFileMetadata[0] ); + File[] files = prepareStoreCopyFiles.listReplayableFiles(); + StoreResource[] atomicFilesSnapshot = prepareStoreCopyFiles.getAtomicFilesSnapshot(); + assertEquals( 0, files.length ); + assertEquals( 0, atomicFilesSnapshot.length ); + } + + private void setExpectedFiles( StoreFileMetadata[] expectedFiles ) throws IOException + { + doAnswer( invocation -> Iterators.asResourceIterator( Iterators.iterator( expectedFiles ) ) ).when( fileListingBuilder ).build(); + } + + @Test + public void shouldReturnExpectedListOfFileNamesForEachType() throws Exception + { + // given + StoreFileMetadata[] expectedFiles = + new StoreFileMetadata[]{new StoreFileMetadata( new File( storeDir, "a" ), 1 ), new StoreFileMetadata( new File( storeDir, "b" ), 2 )}; + setExpectedFiles( expectedFiles ); + + //when + File[] files = prepareStoreCopyFiles.listReplayableFiles(); + StoreResource[] atomicFilesSnapshot = prepareStoreCopyFiles.getAtomicFilesSnapshot(); + + //then + File[] expectedFilesConverted = Arrays.stream( expectedFiles ).map( StoreFileMetadata::file ).toArray( File[]::new ); + StoreResource[] exeptedAtomicFilesConverted = Arrays.stream( expectedFiles ).map( + f -> new StoreResource( f.file(), getRelativePath( f ), f.recordSize(), pageCache, fileSystemAbstraction ) ).toArray( StoreResource[]::new ); + assertArrayEquals( expectedFilesConverted, files ); + assertEquals( exeptedAtomicFilesConverted.length, atomicFilesSnapshot.length ); + for ( int i = 0; i < exeptedAtomicFilesConverted.length; i++ ) + { + StoreResource expected = exeptedAtomicFilesConverted[i]; + StoreResource storeResource = atomicFilesSnapshot[i]; + assertEquals( expected.path(), storeResource.path() ); + assertEquals( expected.recordSize(), storeResource.recordSize() ); + } + } + + @Test + public void shouldHandleEmptyDescriptors() + { + IndexDescriptor[] indexDescriptors = prepareStoreCopyFiles.getIndexDescriptors(); + + assertEquals( 0, indexDescriptors.length ); + } + + @Test + public void shouldReturnExpectedDescriptors() + { + IndexDescriptor[] expectedDescriptors = {indexDescriptor( 1, 2 )}; + when( indexListingMock.listIndexDescriptors() ).thenReturn( Arrays.asList( expectedDescriptors ) ); + + IndexDescriptor[] indexDescriptors = prepareStoreCopyFiles.getIndexDescriptors(); + + assertArrayEquals( expectedDescriptors, indexDescriptors ); + } + + private IndexDescriptor indexDescriptor( int labelId, int propertyId1 ) + { + return new IndexDescriptor( new LabelSchemaDescriptor( labelId, propertyId1 ), IndexDescriptor.Type.GENERAL ); + } + + private String getRelativePath( StoreFileMetadata f ) + { + try + { + return relativePath( storeDir, f.file() ); + } + catch ( IOException e ) + { + throw new RuntimeException( "Failed to create relative path" ); + } + } +} diff --git a/enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/catchup/storecopy/PrepareStoreCopyRequestHandlerTest.java b/enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/catchup/storecopy/PrepareStoreCopyRequestHandlerTest.java new file mode 100644 index 0000000000000..b5179360775e7 --- /dev/null +++ b/enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/catchup/storecopy/PrepareStoreCopyRequestHandlerTest.java @@ -0,0 +1,158 @@ +/* + * 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 Affero 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 Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + */ +package org.neo4j.causalclustering.catchup.storecopy; + +import io.netty.channel.ChannelHandlerContext; +import io.netty.channel.ChannelPromise; +import io.netty.channel.embedded.EmbeddedChannel; +import org.junit.Before; +import org.junit.Test; + +import java.io.File; +import java.io.IOException; +import java.util.concurrent.locks.ReentrantReadWriteLock; +import java.util.function.Supplier; + +import org.neo4j.causalclustering.catchup.CatchupServerProtocol; +import org.neo4j.causalclustering.catchup.ResponseMessageType; +import org.neo4j.causalclustering.identity.StoreId; +import org.neo4j.kernel.NeoStoreDataSource; +import org.neo4j.kernel.api.schema.LabelSchemaDescriptor; +import org.neo4j.kernel.api.schema.index.IndexDescriptor; +import org.neo4j.kernel.impl.transaction.log.checkpoint.CheckPointer; +import org.neo4j.kernel.impl.transaction.log.checkpoint.StoreCopyCheckPointMutex; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class PrepareStoreCopyRequestHandlerTest +{ + private static final StoreId STORE_ID_MATCHING = new StoreId( 1, 2, 3, 4 ); + private static final StoreId STORE_ID_MISMATCHING = new StoreId( 5000, 6000, 7000, 8000 ); + private final ChannelHandlerContext channelHandlerContext = mock( ChannelHandlerContext.class ); + private EmbeddedChannel embeddedChannel; + + private static final CheckPointer checkPointer = mock( CheckPointer.class ); + private static final NeoStoreDataSource neoStoreDataSource = mock( NeoStoreDataSource.class ); + private final CatchupServerProtocol catchupServerProtocol = new CatchupServerProtocol(); + private final PrepareStoreCopyFiles prepareStoreCopyFiles = mock( PrepareStoreCopyFiles.class ); + + @Before + public void setup() + { + StoreCopyCheckPointMutex storeCopyCheckPointMutex = new StoreCopyCheckPointMutex(); + PrepareStoreCopyRequestHandler subject = createHandler( storeCopyCheckPointMutex ); + embeddedChannel = new EmbeddedChannel( subject ); + } + + private PrepareStoreCopyRequestHandler createHandler( StoreCopyCheckPointMutex storeCopyCheckPointMutex ) + { + Supplier checkPointerSupplier = () -> checkPointer; + Supplier dataSourceSupplier = () -> neoStoreDataSource; + when( neoStoreDataSource.getStoreId() ).thenReturn( new org.neo4j.kernel.impl.store.StoreId( 1, 2, 5, 3, 4 ) ); + + PrepareStoreCopyFilesProvider prepareStoreCopyFilesProvider = mock( PrepareStoreCopyFilesProvider.class ); + when( prepareStoreCopyFilesProvider.prepareStoreCopyFiles( any() ) ).thenReturn( prepareStoreCopyFiles ); + + return new PrepareStoreCopyRequestHandler( catchupServerProtocol, checkPointerSupplier, storeCopyCheckPointMutex, dataSourceSupplier, + prepareStoreCopyFilesProvider ); + } + + @Test + public void shouldGiveErrorResponseIfStoreMismatch() + { + // given store id doesn't match + + // when PrepareStoreCopyRequest is written to channel + embeddedChannel.writeInbound( new PrepareStoreCopyRequest( STORE_ID_MISMATCHING ) ); + + // then there is a store id mismatch message + assertEquals( ResponseMessageType.PREPARE_STORE_COPY_RESPONSE, embeddedChannel.readOutbound() ); + PrepareStoreCopyResponse response = PrepareStoreCopyResponse.error( PrepareStoreCopyResponse.Status.E_STORE_ID_MISMATCH ); + assertEquals( response, embeddedChannel.readOutbound() ); + + // and the expected message type is reset back to message type + assertTrue( catchupServerProtocol.isExpecting( CatchupServerProtocol.State.MESSAGE_TYPE ) ); + } + + @Test + public void shouldGetSuccessfulResponseFromPrepareStoreCopyRequest() throws Exception + { + // given storeId matches + IndexDescriptor[] descriptors = new IndexDescriptor[]{new IndexDescriptor( new LabelSchemaDescriptor( 1, 2, 3 ), IndexDescriptor.Type.GENERAL )}; + File[] files = new File[]{new File( "file" )}; + long lastCheckpoint = 1; + + configureProvidedStoreCopyFiles( new StoreResource[0], files, descriptors, lastCheckpoint ); + + // when store listing is requested + embeddedChannel.writeInbound( channelHandlerContext, new PrepareStoreCopyRequest( STORE_ID_MATCHING ) ); + + // and the contents of the store listing response is sent + assertEquals( ResponseMessageType.PREPARE_STORE_COPY_RESPONSE, embeddedChannel.readOutbound() ); + PrepareStoreCopyResponse response = PrepareStoreCopyResponse.success( files, descriptors, lastCheckpoint ); + assertEquals( response, embeddedChannel.readOutbound() ); + + // and the protocol is reset to expect any message type after listing has been transmitted + assertTrue( catchupServerProtocol.isExpecting( CatchupServerProtocol.State.MESSAGE_TYPE ) ); + } + + @Test + public void shouldRetainLockWhileStreaming() throws Exception + { + // given + ChannelPromise channelPromise = embeddedChannel.newPromise(); + ChannelHandlerContext channelHandlerContext = mock( ChannelHandlerContext.class ); + when( channelHandlerContext.writeAndFlush( any( PrepareStoreCopyResponse.class ) ) ).thenReturn( channelPromise ); + + ReentrantReadWriteLock lock = new ReentrantReadWriteLock(); + PrepareStoreCopyRequestHandler subjectHandler = createHandler( new StoreCopyCheckPointMutex( lock ) ); + + // and + IndexDescriptor[] descriptors = new IndexDescriptor[]{new IndexDescriptor( new LabelSchemaDescriptor( 1, 2, 3 ), IndexDescriptor.Type.GENERAL )}; + File[] files = new File[]{new File( "file" )}; + long lastCheckpoint = 1; + configureProvidedStoreCopyFiles( new StoreResource[0], files, descriptors, lastCheckpoint ); + + // when + subjectHandler.channelRead0( channelHandlerContext, new PrepareStoreCopyRequest( STORE_ID_MATCHING ) ); + + // then + assertEquals( 1, lock.getReadLockCount() ); + + // when + channelPromise.setSuccess(); + + //then + assertEquals( 0, lock.getReadLockCount() ); + } + + private void configureProvidedStoreCopyFiles( StoreResource[] atomicFiles, File[] files, IndexDescriptor[] descriptors, long lastCommitedTx ) + throws IOException + { + when( prepareStoreCopyFiles.getAtomicFilesSnapshot() ).thenReturn( atomicFiles ); + when( prepareStoreCopyFiles.getIndexDescriptors() ).thenReturn( descriptors ); + when( prepareStoreCopyFiles.listReplayableFiles() ).thenReturn( files ); + when( checkPointer.lastCheckPointedTransactionId() ).thenReturn( lastCommitedTx ); + } +} diff --git a/enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/catchup/storecopy/PrepareStoreCopyRequestMarshalTest.java b/enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/catchup/storecopy/PrepareStoreCopyRequestMarshalTest.java new file mode 100644 index 0000000000000..97afcff48f5fc --- /dev/null +++ b/enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/catchup/storecopy/PrepareStoreCopyRequestMarshalTest.java @@ -0,0 +1,63 @@ +/* + * 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 Affero 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 Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + */ +package org.neo4j.causalclustering.catchup.storecopy; + +import io.netty.buffer.ByteBuf; +import io.netty.channel.embedded.EmbeddedChannel; +import org.junit.Before; +import org.junit.Test; + +import org.neo4j.causalclustering.identity.StoreId; + +import static org.junit.Assert.assertEquals; + +public class PrepareStoreCopyRequestMarshalTest +{ + EmbeddedChannel embeddedChannel; + + @Before + public void setup() + { + embeddedChannel = new EmbeddedChannel( new PrepareStoreCopyRequestEncoder(), new PrepareStoreCopyRequestDecoder() ); + } + + @Test + public void storeIdIsTransmitted() + { + // given store id requests transmit store id + StoreId storeId = new StoreId( 1, 2, 3, 4 ); + PrepareStoreCopyRequest prepareStoreCopyRequest = new PrepareStoreCopyRequest( storeId ); + + // when transmitted + sendToChannel( prepareStoreCopyRequest, embeddedChannel ); + + // then it can be received/deserialised + PrepareStoreCopyRequest prepareStoreCopyRequestRead = embeddedChannel.readInbound(); + assertEquals( prepareStoreCopyRequest.getStoreId(), prepareStoreCopyRequestRead.getStoreId() ); + } + + public static void sendToChannel( E e, EmbeddedChannel embeddedChannel ) + { + embeddedChannel.writeOutbound( e ); + + ByteBuf object = embeddedChannel.readOutbound(); + embeddedChannel.writeInbound( object ); + } +} diff --git a/enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/catchup/storecopy/PrepareStoreCopyResponseMarshalTest.java b/enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/catchup/storecopy/PrepareStoreCopyResponseMarshalTest.java new file mode 100644 index 0000000000000..3b7395b66d958 --- /dev/null +++ b/enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/catchup/storecopy/PrepareStoreCopyResponseMarshalTest.java @@ -0,0 +1,114 @@ +/* + * 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 Affero 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 Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + */ +package org.neo4j.causalclustering.catchup.storecopy; + +import io.netty.buffer.ByteBuf; +import io.netty.channel.embedded.EmbeddedChannel; +import org.junit.Before; +import org.junit.Test; + +import java.io.File; +import java.util.stream.Stream; + +import org.neo4j.kernel.api.index.SchemaIndexProvider; +import org.neo4j.kernel.api.schema.LabelSchemaDescriptor; +import org.neo4j.kernel.api.schema.index.IndexDescriptor; + +import static org.junit.Assert.assertEquals; + +public class PrepareStoreCopyResponseMarshalTest +{ + private EmbeddedChannel embeddedChannel; + + @Before + public void setup() + { + embeddedChannel = new EmbeddedChannel( new PrepareStoreCopyResponse.Encoder(), new PrepareStoreCopyResponse.Decoder() ); + } + + @Test + public void transactionIdGetsTransmitted() + { + // given + long transactionId = Long.MAX_VALUE; + + // when a transaction id is serialised + PrepareStoreCopyResponse prepareStoreCopyResponse = PrepareStoreCopyResponse.success( new File[0], new IndexDescriptor[0], transactionId ); + sendToChannel( prepareStoreCopyResponse, embeddedChannel ); + + // then it can be deserialised + PrepareStoreCopyResponse readPrepareStoreCopyResponse = embeddedChannel.readInbound(); + assertEquals( prepareStoreCopyResponse.lastTransactionId(), readPrepareStoreCopyResponse.lastTransactionId() ); + } + + @Test + public void fileListGetsTransmitted() + { + // given + File[] files = + new File[]{new File( "File a.txt" ), new File( "file-b" ), new File( "aoifnoasndfosidfoisndfoisnodainfsonidfaosiidfna" ), new File( "" )}; + + // when + PrepareStoreCopyResponse prepareStoreCopyResponse = PrepareStoreCopyResponse.success( files, new IndexDescriptor[0], 0L ); + sendToChannel( prepareStoreCopyResponse, embeddedChannel ); + + // then it can be deserialised + PrepareStoreCopyResponse readPrepareStoreCopyResponse = embeddedChannel.readInbound(); + assertEquals( prepareStoreCopyResponse.getFiles().length, readPrepareStoreCopyResponse.getFiles().length ); + for ( File file : files ) + { + assertEquals( 1, Stream.of( readPrepareStoreCopyResponse.getFiles() ).map( File::getName ).filter( f -> f.equals( file.getName() ) ).count() ); + } + } + + @Test + public void descriptorsGetTransmitted() + { + // given + File[] files = + new File[]{new File( "File a.txt" ), new File( "file-b" ), new File( "aoifnoasndfosidfoisndfoisnodainfsonidfaosiidfna" ), new File( "" )}; + IndexDescriptor[] descriptors = {new IndexDescriptor( new LabelSchemaDescriptor( 1, 2, 3 ), IndexDescriptor.Type.GENERAL )}; + + // when + PrepareStoreCopyResponse prepareStoreCopyResponse = PrepareStoreCopyResponse.success( files, descriptors, 1L ); + sendToChannel( prepareStoreCopyResponse, embeddedChannel ); + + // then it can be deserialised + PrepareStoreCopyResponse readPrepareStoreCopyResponse = embeddedChannel.readInbound(); + assertEquals( prepareStoreCopyResponse.getFiles().length, readPrepareStoreCopyResponse.getFiles().length ); + for ( File file : files ) + { + assertEquals( 1, Stream.of( readPrepareStoreCopyResponse.getFiles() ).map( File::getName ).filter( f -> f.equals( file.getName() ) ).count() ); + } + assertEquals( prepareStoreCopyResponse.getDescriptors().length, readPrepareStoreCopyResponse.getDescriptors().length ); + for ( IndexDescriptor descriptor : descriptors ) + { + assertEquals( 1, Stream.of( readPrepareStoreCopyResponse.getDescriptors() ).filter( f -> f.equals( descriptor ) ).count() ); + } + } + + private static void sendToChannel( PrepareStoreCopyResponse prepareStoreCopyResponse, EmbeddedChannel embeddedChannel ) + { + embeddedChannel.writeOutbound( prepareStoreCopyResponse ); + + ByteBuf object = embeddedChannel.readOutbound(); + embeddedChannel.writeInbound( object ); + } +} diff --git a/enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/catchup/storecopy/RealStrippedCatchupServer.java b/enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/catchup/storecopy/RealStrippedCatchupServer.java new file mode 100644 index 0000000000000..5b75ff8259da4 --- /dev/null +++ b/enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/catchup/storecopy/RealStrippedCatchupServer.java @@ -0,0 +1,93 @@ +/* + * 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 Affero 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 Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + */ +package org.neo4j.causalclustering.catchup.storecopy; + +import io.netty.channel.ChannelHandler; + +import java.util.function.Supplier; + +import org.neo4j.causalclustering.StrippedCatchupServer; +import org.neo4j.causalclustering.catchup.CatchupServerProtocol; +import org.neo4j.graphdb.DependencyResolver; +import org.neo4j.io.fs.FileSystemAbstraction; +import org.neo4j.io.pagecache.PageCache; +import org.neo4j.kernel.NeoStoreDataSource; +import org.neo4j.kernel.impl.transaction.log.checkpoint.CheckPointer; +import org.neo4j.kernel.impl.transaction.log.checkpoint.StoreCopyCheckPointMutex; +import org.neo4j.kernel.internal.GraphDatabaseAPI; +import org.neo4j.logging.LogProvider; +import org.neo4j.logging.NullLogProvider; + +/** + * A standalone catchup server that uses production dependencies + * (i.e. only relevant subset of a neo4j cluster instance) + */ +class RealStrippedCatchupServer extends StrippedCatchupServer +{ + private final StoreCopyCheckPointMutex storeCopyCheckPointMutex; + private final Supplier checkPointer; + private final Supplier neoStoreDataSourceSupplier; + private final PageCache pageCache; + private final FileSystemAbstraction fsa; + private final LogProvider logProvider; + + RealStrippedCatchupServer( FileSystemAbstraction fsa, GraphDatabaseAPI graphDb ) + { + this.fsa = fsa; + DependencyResolver dependencyResolver = graphDb.getDependencyResolver(); + storeCopyCheckPointMutex = dependencyResolver.resolveDependency( StoreCopyCheckPointMutex.class ); + checkPointer = () -> getCheckPointer( graphDb ); + neoStoreDataSourceSupplier = () -> getNeoStoreDataSource( graphDb ); + pageCache = graphDb.getDependencyResolver().resolveDependency( PageCache.class ); + logProvider = NullLogProvider.getInstance(); + } + + @Override + protected ChannelHandler getStoreFileRequestHandler( CatchupServerProtocol protocol ) + { + return new GetStoreFileRequestHandler( protocol, neoStoreDataSourceSupplier, checkPointer, new StoreFileStreamingProtocol(), + pageCache, fsa, logProvider ); + } + + @Override + protected ChannelHandler getStoreListingRequestHandler( CatchupServerProtocol protocol ) + { + PrepareStoreCopyFilesProvider prepareStoreCopyFilesProvider = new PrepareStoreCopyFilesProvider( pageCache, fsa ); + return new PrepareStoreCopyRequestHandler( protocol, checkPointer, storeCopyCheckPointMutex, neoStoreDataSourceSupplier, + prepareStoreCopyFilesProvider ); + } + + @Override + protected ChannelHandler getIndexRequestHandler( CatchupServerProtocol protocol ) + { + return new GetIndexSnapshotRequestHandler( protocol, neoStoreDataSourceSupplier, checkPointer, + new StoreFileStreamingProtocol(), pageCache, fsa ); + } + + static CheckPointer getCheckPointer( GraphDatabaseAPI graphDb ) + { + return graphDb.getDependencyResolver().resolveDependency( CheckPointer.class ); + } + + static NeoStoreDataSource getNeoStoreDataSource( GraphDatabaseAPI graphDb ) + { + return graphDb.getDependencyResolver().resolveDependency( NeoStoreDataSource.class ); + } +} diff --git a/enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/catchup/storecopy/RemoteStoreTest.java b/enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/catchup/storecopy/RemoteStoreTest.java index e1a26d8ca0a05..17a0ba432f647 100644 --- a/enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/catchup/storecopy/RemoteStoreTest.java +++ b/enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/catchup/storecopy/RemoteStoreTest.java @@ -23,8 +23,10 @@ import java.io.File; import java.io.IOException; +import java.util.function.Supplier; import org.neo4j.causalclustering.catchup.CatchUpClientException; +import org.neo4j.causalclustering.catchup.CatchupAddressProvider; import org.neo4j.causalclustering.catchup.TxPullRequestResult; import org.neo4j.causalclustering.catchup.tx.TransactionLogCatchUpFactory; import org.neo4j.causalclustering.catchup.tx.TransactionLogCatchUpWriter; @@ -66,10 +68,11 @@ public void shouldCopyStoreFilesAndPullTransactions() throws Exception // when AdvertisedSocketAddress localhost = new AdvertisedSocketAddress( "127.0.0.1", 1234 ); - remoteStore.copy( localhost, storeId, new File( "destination" ) ); + CatchupAddressProvider catchupAddressProvider = CatchupAddressProvider.fromSingleAddress( localhost ); + remoteStore.copy( catchupAddressProvider, storeId, new File( "destination" ) ); // then - verify( storeCopyClient ).copyStoreFiles( eq( localhost ), eq( storeId ), any( StoreFileStreams.class ) ); + verify( storeCopyClient ).copyStoreFiles( eq( catchupAddressProvider ), eq( storeId ), any( StoreFileStreams.class ), any() ); verify( txPullClient ).pullTransactions( eq( localhost ), eq( storeId ), anyLong(), isNull() ); } @@ -80,9 +83,10 @@ public void shouldSetLastPulledTransactionId() throws Exception long lastFlushedTxId = 12; StoreId wantedStoreId = new StoreId( 1, 2, 3, 4 ); AdvertisedSocketAddress localhost = new AdvertisedSocketAddress( "127.0.0.1", 1234 ); + CatchupAddressProvider catchupAddressProvider = CatchupAddressProvider.fromSingleAddress( localhost ); StoreCopyClient storeCopyClient = mock( StoreCopyClient.class ); - when( storeCopyClient.copyStoreFiles( eq( localhost ), eq( wantedStoreId ), any( StoreFileStreams.class ) ) ) + when( storeCopyClient.copyStoreFiles( eq( catchupAddressProvider ), eq( wantedStoreId ), any( StoreFileStreams.class ), any() ) ) .thenReturn( lastFlushedTxId ); TxPullClient txPullClient = mock( TxPullClient.class ); @@ -95,7 +99,7 @@ public void shouldSetLastPulledTransactionId() throws Exception null, storeCopyClient, txPullClient, factory( writer ), Config.defaults(), new Monitors() ); // when - remoteStore.copy( localhost, wantedStoreId, new File( "destination" ) ); + remoteStore.copy( catchupAddressProvider, wantedStoreId, new File( "destination" ) ); // then long previousTxId = lastFlushedTxId - 1; // the interface is defined as asking for the one preceding @@ -111,6 +115,7 @@ public void shouldCloseDownTxLogWriterIfTxStreamingFails() throws Exception StoreCopyClient storeCopyClient = mock( StoreCopyClient.class ); TxPullClient txPullClient = mock( TxPullClient.class ); TransactionLogCatchUpWriter writer = mock( TransactionLogCatchUpWriter.class ); + CatchupAddressProvider catchupAddressProvider = CatchupAddressProvider.fromSingleAddress( null ); RemoteStore remoteStore = new RemoteStore( NullLogProvider.getInstance(), mock( FileSystemAbstraction.class ), null, @@ -122,7 +127,7 @@ public void shouldCloseDownTxLogWriterIfTxStreamingFails() throws Exception // when try { - remoteStore.copy( null, storeId, null ); + remoteStore.copy( catchupAddressProvider, storeId, null ); } catch ( StoreCopyFailedException e ) { diff --git a/enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/catchup/storecopy/SimpleCatchupClient.java b/enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/catchup/storecopy/SimpleCatchupClient.java new file mode 100644 index 0000000000000..a84155b6d0489 --- /dev/null +++ b/enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/catchup/storecopy/SimpleCatchupClient.java @@ -0,0 +1,123 @@ +/* + * 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 Affero 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 Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + */ +package org.neo4j.causalclustering.catchup.storecopy; + +import java.io.File; +import java.io.IOException; + +import org.neo4j.causalclustering.StrippedCatchupServer; +import org.neo4j.causalclustering.catchup.CatchUpClient; +import org.neo4j.causalclustering.catchup.CatchUpClientException; +import org.neo4j.causalclustering.identity.StoreId; +import org.neo4j.helpers.AdvertisedSocketAddress; +import org.neo4j.io.IOUtils; +import org.neo4j.io.fs.FileSystemAbstraction; +import org.neo4j.io.pagecache.PageCache; +import org.neo4j.io.pagecache.impl.muninn.StandalonePageCacheFactory; +import org.neo4j.kernel.api.schema.index.IndexDescriptor; +import org.neo4j.kernel.internal.GraphDatabaseAPI; +import org.neo4j.kernel.monitoring.Monitors; +import org.neo4j.logging.Log; +import org.neo4j.logging.LogProvider; + +import static org.neo4j.causalclustering.catchup.storecopy.RealStrippedCatchupServer.getCheckPointer; + +class SimpleCatchupClient implements AutoCloseable +{ + private final GraphDatabaseAPI graphDb; + private final FileSystemAbstraction fsa; + private final CatchUpClient catchUpClient; + private final StrippedCatchupServer catchupServer; + + private final AdvertisedSocketAddress from; + private final StoreId correctStoreId; + private final StreamToDisk streamToDisk; + private final PageCache clientPageCache; + private final Log log; + private final LogProvider logProvider; + + SimpleCatchupClient( GraphDatabaseAPI graphDb, FileSystemAbstraction fileSystemAbstraction, CatchUpClient catchUpClient, + StrippedCatchupServer catchupServer, File temporaryDirectory, LogProvider logProvider ) throws IOException + { + this.graphDb = graphDb; + this.fsa = fileSystemAbstraction; + this.catchUpClient = catchUpClient; + this.catchupServer = catchupServer; + + from = getCatchupServerAddress(); + correctStoreId = getStoreIdFromKernelStoreId( graphDb ); + clientPageCache = createPageCache(); + streamToDisk = new StreamToDisk( temporaryDirectory, fsa, clientPageCache, new Monitors() ); + log = logProvider.getLog( SimpleCatchupClient.class ); + this.logProvider = logProvider; + } + + private PageCache createPageCache() + { + return StandalonePageCacheFactory.createPageCache( fsa ); + } + + PrepareStoreCopyResponse requestListOfFilesFromServer() throws CatchUpClientException + { + return requestListOfFilesFromServer( correctStoreId ); + } + + PrepareStoreCopyResponse requestListOfFilesFromServer( StoreId expectedStoreId ) throws CatchUpClientException + { + return catchUpClient.makeBlockingRequest( from, new PrepareStoreCopyRequest( expectedStoreId ), + new PrepareStoreCopyResponseAdaptor( streamToDisk, logProvider ) ); + } + + StoreCopyFinishedResponse requestIndividualFile( File file ) throws CatchUpClientException + { + return requestIndividualFile( file, correctStoreId ); + } + + StoreCopyFinishedResponse requestIndividualFile( File file, StoreId expectedStoreId ) throws CatchUpClientException + { + long lastTransactionId = getCheckPointer( graphDb ).lastCheckPointedTransactionId(); + GetStoreFileRequest storeFileRequest = new GetStoreFileRequest( expectedStoreId, file, lastTransactionId ); + return catchUpClient.makeBlockingRequest( from, storeFileRequest, new StoreCopyClient.StoreFileCopyResponseAdaptor( streamToDisk, log ) ); + } + + private StoreId getStoreIdFromKernelStoreId( GraphDatabaseAPI graphDb ) + { + org.neo4j.kernel.impl.store.StoreId storeId = graphDb.storeId(); + return new StoreId( storeId.getCreationTime(), storeId.getRandomId(), storeId.getUpgradeTime(), storeId.getUpgradeId() ); + } + + private AdvertisedSocketAddress getCatchupServerAddress() + { + return new AdvertisedSocketAddress( "localhost", catchupServer.getPort() ); + } + + StoreCopyFinishedResponse requestIndexSnapshot( IndexDescriptor expectedDescriptor ) throws CatchUpClientException + { + long lastCheckPointedTransactionId = getCheckPointer( graphDb ).lastCheckPointedTransactionId(); + return catchUpClient.makeBlockingRequest( from, new GetIndexFilesRequest( getStoreIdFromKernelStoreId( graphDb ), expectedDescriptor, + lastCheckPointedTransactionId ), new StoreCopyClient.StoreFileCopyResponseAdaptor( streamToDisk, log ) ); + } + + @Override + public void close() throws Exception + { + IOUtils.closeAll( streamToDisk, clientPageCache ); + } +} diff --git a/enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/catchup/storecopy/StoreCopyClientIT.java b/enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/catchup/storecopy/StoreCopyClientIT.java new file mode 100644 index 0000000000000..58bad19f74b4a --- /dev/null +++ b/enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/catchup/storecopy/StoreCopyClientIT.java @@ -0,0 +1,257 @@ +/* + * 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 Affero 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 Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + */ +package org.neo4j.causalclustering.catchup.storecopy; + +import org.junit.After; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; + +import java.io.File; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.time.Clock; +import java.util.Arrays; +import java.util.HashSet; +import java.util.Set; +import java.util.function.Predicate; +import java.util.function.Supplier; + +import org.neo4j.causalclustering.catchup.CatchUpClient; +import org.neo4j.causalclustering.catchup.CatchupAddressProvider; +import org.neo4j.causalclustering.handlers.VoidPipelineWrapperFactory; +import org.neo4j.function.ThrowingSupplier; +import org.neo4j.helpers.AdvertisedSocketAddress; +import org.neo4j.io.fs.DefaultFileSystemAbstraction; +import org.neo4j.io.fs.FileSystemAbstraction; +import org.neo4j.io.fs.OpenMode; +import org.neo4j.io.fs.StoreChannel; +import org.neo4j.kernel.monitoring.Monitors; +import org.neo4j.logging.FormattedLogProvider; +import org.neo4j.logging.Level; +import org.neo4j.logging.LogProvider; +import org.neo4j.test.rule.TestDirectory; + +import static org.junit.Assert.assertEquals; + +public class StoreCopyClientIT +{ + FileSystemAbstraction fileSystemAbstraction = new DefaultFileSystemAbstraction(); + private final LogProvider logProvider = FormattedLogProvider.withDefaultLogLevel( Level.DEBUG ).toOutputStream( System.out ); + private final TerminationCondition defaultTerminationCondition = TerminationCondition.CONTINUE_INDEFINITELY; + + @Rule + public TestDirectory testDirectory = TestDirectory.testDirectory( fileSystemAbstraction ); + + private FakeCatchupServer catchupServerRule; + + private StoreCopyClient subject; + private FakeFile fileA = new FakeFile( "fileA", "This is file a content" ); + private FakeFile fileB = new FakeFile( "another-file-b", "Totally different content 123" ); + + private FakeFile indexFileA = new FakeFile( "lucene", "Lucene 123" ); + + private static void writeContents( FileSystemAbstraction fileSystemAbstraction, File file, String contents ) + { + byte[] bytes = contents.getBytes(); + try ( StoreChannel storeChannel = fileSystemAbstraction.create( file ) ) + { + storeChannel.write( ByteBuffer.wrap( bytes ) ); + } + catch ( IOException e ) + { + throw new RuntimeException( e ); + } + } + + @Before + public void setup() + { + catchupServerRule = new FakeCatchupServer( this, logProvider ); + catchupServerRule.addFile( fileA ); + catchupServerRule.addFile( fileB ); + catchupServerRule.addIndexFile( indexFileA ); + writeContents( fileSystemAbstraction, relative( fileA.getFilename() ), fileA.getContent() ); + writeContents( fileSystemAbstraction, relative( fileB.getFilename() ), fileB.getContent() ); + writeContents( fileSystemAbstraction, relative( indexFileA.getFilename() ), indexFileA.getContent() ); + + CatchUpClient catchUpClient = + new CatchUpClient( logProvider, Clock.systemDefaultZone(), 2000, new Monitors(), VoidPipelineWrapperFactory.VOID_WRAPPER ); + catchUpClient.start(); + subject = new StoreCopyClient( catchUpClient, logProvider ); + } + + @After + public void shutdown() + { + catchupServerRule.stop(); + } + + @Test + public void canPerformCatchup() throws StoreCopyFailedException + { + // given remote node has a store + catchupServerRule.before(); // assume it is running + catchupServerRule.start(); + + // and local client has a store + InMemoryFileSystemStream storeFileStream = new InMemoryFileSystemStream(); + + // when catchup is performed for valid transactionId and StoreId + CatchupAddressProvider catchupAddressProvider = CatchupAddressProvider.fromSingleAddress( from( catchupServerRule.getPort() ) ); + subject.copyStoreFiles( catchupAddressProvider, catchupServerRule.getStoreId(), storeFileStream, () -> defaultTerminationCondition ); + + // then the catchup is successful + Set expectedFiles = new HashSet<>( Arrays.asList( fileA.getFilename(), fileB.getFilename(), indexFileA.getFilename() ) ); + assertEquals( expectedFiles, storeFileStream.filesystem.keySet() ); + assertEquals( fileContent( relative( fileA.getFilename() ) ), clientFileContents( storeFileStream, fileA.getFilename() ) ); + assertEquals( fileContent( relative( fileB.getFilename() ) ), clientFileContents( storeFileStream, fileB.getFilename() ) ); + } + + @Test + public void failedFileCopyShouldRetry() throws StoreCopyFailedException + { + // given a file will fail twice before succeeding + fileB.setRemainingFailed( 2 ); + + // and remote node has a store + catchupServerRule.before(); // assume it is running + catchupServerRule.start(); + + // and local client has a store + InMemoryFileSystemStream clientStoreFileStream = new InMemoryFileSystemStream(); + + // when catchup is performed for valid transactionId and StoreId + CatchupAddressProvider catchupAddressProvider = CatchupAddressProvider.fromSingleAddress( from( catchupServerRule.getPort() ) ); + subject.copyStoreFiles( catchupAddressProvider, catchupServerRule.getStoreId(), clientStoreFileStream, () -> defaultTerminationCondition ); + + // then the catchup is successful + Set expectedFiles = new HashSet<>( Arrays.asList( fileA.getFilename(), fileB.getFilename(), indexFileA.getFilename() ) ); + assertEquals( expectedFiles, clientStoreFileStream.filesystem.keySet() ); + + // and + assertEquals( fileContent( relative( fileA.getFilename() ) ), clientFileContents( clientStoreFileStream, fileA.getFilename() ) ); + assertEquals( fileContent( relative( fileB.getFilename() ) ), clientFileContents( clientStoreFileStream, fileB.getFilename() ) ); + + // and verify server had exactly 2 failed calls before having a 3rd succeeding request + assertEquals( 3, catchupServerRule.getRequestCount( fileB.getFilename() ) ); + + // and verify server had exactly 1 call for all other files + assertEquals( 1, catchupServerRule.getRequestCount( fileA.getFilename() ) ); + } + + @Test + public void reconnectingWorks() throws StoreCopyFailedException + { + // given a remote catchup will fail midway + catchupServerRule.before(); + catchupServerRule.start(); + + // and local client has a store + InMemoryFileSystemStream storeFileStream = new InMemoryFileSystemStream(); + + // and file B is broken once (after retry it works) + fileB.setRemainingNoResponse( 1 ); + + // when catchup is performed for valid transactionId and StoreId + CatchupAddressProvider catchupAddressProvider = CatchupAddressProvider.fromSingleAddress( from( catchupServerRule.getPort() ) ); + subject.copyStoreFiles( catchupAddressProvider, catchupServerRule.getStoreId(), storeFileStream, () -> defaultTerminationCondition ); + + // then the catchup is possible to complete + assertEquals( fileContent( relative( fileA.getFilename() ) ), clientFileContents( storeFileStream, fileA.getFilename() ) ); + assertEquals( fileContent( relative( fileB.getFilename() ) ), clientFileContents( storeFileStream, fileB.getFilename() ) ); + + // and verify server had exactly 2 calls for failing file + assertEquals( 2, catchupServerRule.getRequestCount( fileB.getFilename() ) ); + + // and verify server had exactly 1 call for all other files + assertEquals( 1, catchupServerRule.getRequestCount( fileA.getFilename() ) ); + } + + private static AdvertisedSocketAddress from( int port ) + { + return new AdvertisedSocketAddress( "localhost", port ); + } + + private File relative( String filename ) + { + return testDirectory.file( filename ); + } + + private String fileContent( File file ) + { + return fileContent( file, fileSystemAbstraction ); + } + + private static StringBuilder serverFileContentsStringBuilder( File file, FileSystemAbstraction fileSystemAbstraction ) + { + try ( StoreChannel storeChannel = fileSystemAbstraction.open( file, OpenMode.READ ) ) + { + int MAX_BUFFER_SIZE = 100; + ByteBuffer byteBuffer = ByteBuffer.wrap( new byte[MAX_BUFFER_SIZE] ); + StringBuilder stringBuilder = new StringBuilder(); + Predicate inRange = betweenZeroAndRange( MAX_BUFFER_SIZE ); + Supplier readNext = unchecked( () -> storeChannel.read( byteBuffer ) ); + for ( int readBytes = readNext.get(); inRange.test( readBytes ); readBytes = readNext.get() ) + { + for ( byte index = 0; index < readBytes; index++ ) + { + char actual = (char) byteBuffer.get( index ); + stringBuilder.append( actual ); + } + } + return stringBuilder; + } + catch ( IOException e ) + { + throw new RuntimeException( e ); + } + } + + static String fileContent( File file, FileSystemAbstraction fileSystemAbstraction ) + { + return serverFileContentsStringBuilder( file, fileSystemAbstraction ).toString(); + } + + private static Supplier unchecked( ThrowingSupplier throwableSupplier ) + { + return () -> + { + try + { + return throwableSupplier.get(); + } + catch ( Throwable throwable ) + { + throw new RuntimeException( throwable ); + } + }; + } + + private static Predicate betweenZeroAndRange( int RANGE ) + { + return bytes -> bytes > 0 && bytes <= RANGE; + } + + private String clientFileContents( InMemoryFileSystemStream storeFileStreams, String filename ) + { + return storeFileStreams.filesystem.get( filename ).toString(); + } +} diff --git a/enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/catchup/storecopy/StoreCopyClientTest.java b/enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/catchup/storecopy/StoreCopyClientTest.java new file mode 100644 index 0000000000000..9330d5b1f0c05 --- /dev/null +++ b/enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/catchup/storecopy/StoreCopyClientTest.java @@ -0,0 +1,232 @@ +/* + * 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 Affero 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 Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + */ +package org.neo4j.causalclustering.catchup.storecopy; + +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.mockito.ArgumentCaptor; + +import java.io.File; +import java.util.List; +import java.util.function.Supplier; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import org.neo4j.causalclustering.catchup.CatchUpClient; +import org.neo4j.causalclustering.catchup.CatchUpClientException; +import org.neo4j.causalclustering.catchup.CatchupAddressProvider; +import org.neo4j.causalclustering.identity.StoreId; +import org.neo4j.causalclustering.messaging.CatchUpRequest; +import org.neo4j.helpers.AdvertisedSocketAddress; +import org.neo4j.kernel.api.index.SchemaIndexProvider; +import org.neo4j.kernel.api.schema.LabelSchemaDescriptor; +import org.neo4j.kernel.api.schema.index.IndexDescriptor; +import org.neo4j.logging.FormattedLogProvider; +import org.neo4j.logging.Level; +import org.neo4j.logging.LogProvider; + +import static org.hamcrest.Matchers.containsInAnyOrder; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertThat; +import static org.junit.Assert.fail; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.atLeast; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +public class StoreCopyClientTest +{ + @Rule + public final ExpectedException expectedException = ExpectedException.none(); + + private final CatchUpClient catchUpClient = mock( CatchUpClient.class ); + + private StoreCopyClient subject; + private final LogProvider logProvider = FormattedLogProvider.withDefaultLogLevel( Level.DEBUG ).toOutputStream( System.out ); + + // params + private final AdvertisedSocketAddress expectedAdvertisedAddress = new AdvertisedSocketAddress( "host", 1234 ); + private final CatchupAddressProvider catchupAddressProvider = CatchupAddressProvider.fromSingleAddress( expectedAdvertisedAddress ); + private final StoreId expectedStoreId = new StoreId( 1, 2, 3, 4 ); + private final StoreFileStreams expectedStoreFileStreams = mock( StoreFileStreams.class ); + + // helpers + private File[] serverFiles = new File[]{new File( "fileA.txt" ), new File( "fileB.bmp" )}; + private IndexDescriptor[] descriptors = new IndexDescriptor[]{new IndexDescriptor( new LabelSchemaDescriptor( 1, 2, 3 ), IndexDescriptor.Type.GENERAL )}; + + @Before + public void setup() + { + subject = new StoreCopyClient( catchUpClient, logProvider ); + } + + @Test + public void clientRequestsAllFilesListedInListingResponse() throws StoreCopyFailedException, CatchUpClientException + { + // given a bunch of fake files on the server + PrepareStoreCopyResponse prepareStoreCopyResponse = PrepareStoreCopyResponse.success( serverFiles, descriptors, -123L ); + when( catchUpClient.makeBlockingRequest( any(), any( PrepareStoreCopyRequest.class ), any() ) ).thenReturn( prepareStoreCopyResponse ); + + // and any request for a file will be successful + StoreCopyFinishedResponse success = new StoreCopyFinishedResponse( StoreCopyFinishedResponse.Status.SUCCESS ); + when( catchUpClient.makeBlockingRequest( any(), any( GetStoreFileRequest.class ), any() ) ).thenReturn( success ); + + // and any request for a file will be successful + when( catchUpClient.makeBlockingRequest( any(), any( GetIndexFilesRequest.class ), any() ) ).thenReturn( success ); + + // when client requests catchup + subject.copyStoreFiles( catchupAddressProvider, expectedStoreId, expectedStoreFileStreams, continueIndefinitely() ); + + // then there are as many requests to the server for individual requests + List filteredRequests = filenamesFromIndividualFileRequests( getRequests() ); + List expectedFiles = Stream.of( serverFiles ).map( File::getName ).collect( Collectors.toList() ); + assertThat( expectedFiles, containsInAnyOrder( filteredRequests.toArray() ) ); + } + + private Supplier continueIndefinitely() + { + return () -> TerminationCondition.CONTINUE_INDEFINITELY; + } + + @Test + public void storeIdCanBeRetrieved() throws StoreIdDownloadFailedException, CatchUpClientException + { + // given remote has expected store ID + StoreId remoteStoreId = new StoreId( 6, 3, 2, 6 ); + + // and we know the remote address + AdvertisedSocketAddress remoteAddress = new AdvertisedSocketAddress( "host", 1234 ); + + // and server responds with correct data to correct params + when( catchUpClient.makeBlockingRequest( eq( remoteAddress ), any( GetStoreIdRequest.class ), any() ) ).thenReturn( remoteStoreId ); + + // when client requests the remote store id + StoreId actualStoreId = subject.fetchStoreId( remoteAddress ); + + // then store id matches + assertEquals( remoteStoreId, actualStoreId ); + } + + @Test + public void shouldFailIfTerminationConditionFails() throws CatchUpClientException + { + // given a file will fail an expected number of times + subject = new StoreCopyClient( catchUpClient, logProvider ); + + // and requesting the individual file will fail + when( catchUpClient.makeBlockingRequest( any(), any(), any() ) ).thenReturn( + new StoreCopyFinishedResponse( StoreCopyFinishedResponse.Status.E_TOO_FAR_BEHIND ) ); + + // and the initial list+count store files request is successful + PrepareStoreCopyResponse initialListingOfFilesResponse = PrepareStoreCopyResponse.success( serverFiles, descriptors, -123L ); + when( catchUpClient.makeBlockingRequest( any(), any( PrepareStoreCopyRequest.class ), any() ) ).thenReturn( initialListingOfFilesResponse ); + + // when we perform catchup + try + { + subject.copyStoreFiles( catchupAddressProvider, expectedStoreId, expectedStoreFileStreams, () -> () -> + { + throw new StoreCopyFailedException( "This can't go on" ); + } ); + fail( "Expected exception: " + StoreCopyFailedException.class ); + } + catch ( StoreCopyFailedException expectedException ) + { + assertEquals( "This can't go on", expectedException.getMessage() ); + return; + } + + fail( "Expected a StoreCopyFailedException" ); + } + + @Test + public void errorOnListingStore() throws CatchUpClientException, StoreCopyFailedException + { + // given store listing fails + PrepareStoreCopyResponse prepareStoreCopyResponse = PrepareStoreCopyResponse.error( PrepareStoreCopyResponse.Status.E_LISTING_STORE ); + when( catchUpClient.makeBlockingRequest( any(), any(), any() ) ).thenReturn( prepareStoreCopyResponse ) + .thenThrow( new RuntimeException( "Should not be accessible" ) ); + + // then + expectedException.expectMessage( "Preparing store failed due to: E_LISTING_STORE" ); + expectedException.expect( StoreCopyFailedException.class ); + + // when + subject.copyStoreFiles( catchupAddressProvider, expectedStoreId, expectedStoreFileStreams, continueIndefinitely() ); + } + + @Test + public void storeIdMismatchOnListing() throws CatchUpClientException, StoreCopyFailedException + { + // given store listing fails + PrepareStoreCopyResponse prepareStoreCopyResponse = PrepareStoreCopyResponse.error( PrepareStoreCopyResponse.Status.E_STORE_ID_MISMATCH ); + when( catchUpClient.makeBlockingRequest( any(), any(), any() ) ).thenReturn( prepareStoreCopyResponse ) + .thenThrow( new RuntimeException( "Should not be accessible" ) ); + + // then + expectedException.expectMessage( "Preparing store failed due to: E_STORE_ID_MISMATCH" ); + expectedException.expect( StoreCopyFailedException.class ); + + // when + subject.copyStoreFiles( catchupAddressProvider, expectedStoreId, expectedStoreFileStreams, continueIndefinitely() ); + } + + @Test + public void storeIdMismatchOnCopyIndividualFile() throws StoreCopyFailedException, CatchUpClientException + { + // given listing response will be successful + PrepareStoreCopyResponse prepareStoreCopyResponse = PrepareStoreCopyResponse.success( serverFiles, descriptors, -123L ); + when( catchUpClient.makeBlockingRequest( any(), any(), any() ) ).thenReturn( prepareStoreCopyResponse ); + + // and individual file requests get store id mismatch + StoreCopyFinishedResponse individualFileStoreCopyResposne = new StoreCopyFinishedResponse( StoreCopyFinishedResponse.Status.E_STORE_ID_MISMATCH ); + when( catchUpClient.makeBlockingRequest( any(), any(), any() ) ).thenReturn( prepareStoreCopyResponse, individualFileStoreCopyResposne ); + + // then exception denotes store id mismatch + expectedException.expect( StoreCopyFailedException.class ); + expectedException.expectMessage( "Store id mismatch" ); + + // when copy is performed + subject.copyStoreFiles( catchupAddressProvider, expectedStoreId, expectedStoreFileStreams, continueIndefinitely() ); + } + + private List getRequests() throws CatchUpClientException + { + ArgumentCaptor fileRequestArgumentCaptor = ArgumentCaptor.forClass( CatchUpRequest.class ); + verify( catchUpClient, atLeast( 0 ) ).makeBlockingRequest( any(), fileRequestArgumentCaptor.capture(), any() ); + return fileRequestArgumentCaptor.getAllValues(); + } + + private List filenamesFromIndividualFileRequests( List fileRequests ) + { + return fileRequests.stream() + .filter( GetStoreFileRequest.class::isInstance ) + .map( obj -> (GetStoreFileRequest) obj ) + .map( GetStoreFileRequest::file ) + .map( File::getName ) + .collect( Collectors.toList() ); + } +} + diff --git a/enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/catchup/storecopy/StoreCopyFinishedResponseEncodeDecodeTest.java b/enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/catchup/storecopy/StoreCopyFinishedResponseEncodeDecodeTest.java index 40175d97a977b..bea45ef72cbfa 100644 --- a/enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/catchup/storecopy/StoreCopyFinishedResponseEncodeDecodeTest.java +++ b/enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/catchup/storecopy/StoreCopyFinishedResponseEncodeDecodeTest.java @@ -35,8 +35,7 @@ public void shouldEncodeAndDecodePullRequestMessage() // given EmbeddedChannel channel = new EmbeddedChannel( new StoreCopyFinishedResponseEncoder(), new StoreCopyFinishedResponseDecoder() ); - final long arbitraryId = 23; - StoreCopyFinishedResponse sent = new StoreCopyFinishedResponse( Status.E_STORE_ID_MISMATCH, arbitraryId ); + StoreCopyFinishedResponse sent = new StoreCopyFinishedResponse( Status.E_STORE_ID_MISMATCH ); // when channel.writeOutbound( sent ); diff --git a/enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/catchup/storecopy/StoreStreamingProtocolTest.java b/enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/catchup/storecopy/StoreFileStreamingProtocolTest.java similarity index 88% rename from enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/catchup/storecopy/StoreStreamingProtocolTest.java rename to enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/catchup/storecopy/StoreFileStreamingProtocolTest.java index 87787ddf50326..fad11a26d0f26 100644 --- a/enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/catchup/storecopy/StoreStreamingProtocolTest.java +++ b/enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/catchup/storecopy/StoreFileStreamingProtocolTest.java @@ -44,7 +44,7 @@ import static org.neo4j.causalclustering.catchup.storecopy.StoreCopyFinishedResponse.Status.SUCCESS; import static org.neo4j.kernel.impl.util.Cursors.rawCursorOf; -public class StoreStreamingProtocolTest +public class StoreFileStreamingProtocolTest { @Rule public EphemeralFileSystemRule fs = new EphemeralFileSystemRule(); @@ -64,7 +64,7 @@ public void setup() public void shouldStreamResources() throws Exception { // given - StoreStreamingProtocol protocol = new StoreStreamingProtocol(); + StoreFileStreamingProtocol protocol = new StoreFileStreamingProtocol(); ChannelHandlerContext ctx = mock( ChannelHandlerContext.class ); fs.mkdir( new File( "dirA" ) ); @@ -80,7 +80,10 @@ public void shouldStreamResources() throws Exception RawCursor resources = rawCursorOf( resourceList ); // when - protocol.stream( ctx, resources ); + while ( resources.next() ) + { + protocol.stream( ctx, resources.get() ); + } // then InOrder inOrder = Mockito.inOrder( ctx ); @@ -98,16 +101,16 @@ public void shouldStreamResources() throws Exception public void shouldBeAbleToEndWithFailure() { // given - StoreStreamingProtocol protocol = new StoreStreamingProtocol(); + StoreFileStreamingProtocol protocol = new StoreFileStreamingProtocol(); ChannelHandlerContext ctx = mock( ChannelHandlerContext.class ); // when - protocol.end( ctx, E_STORE_ID_MISMATCH, -1 ); + protocol.end( ctx, E_STORE_ID_MISMATCH ); // then InOrder inOrder = Mockito.inOrder( ctx ); inOrder.verify( ctx ).write( ResponseMessageType.STORE_COPY_FINISHED ); - inOrder.verify( ctx ).writeAndFlush( new StoreCopyFinishedResponse( E_STORE_ID_MISMATCH, -1 ) ); + inOrder.verify( ctx ).writeAndFlush( new StoreCopyFinishedResponse( E_STORE_ID_MISMATCH ) ); inOrder.verifyNoMoreInteractions(); } @@ -115,17 +118,16 @@ public void shouldBeAbleToEndWithFailure() public void shouldBeAbleToEndWithSuccess() { // given - StoreStreamingProtocol protocol = new StoreStreamingProtocol(); + StoreFileStreamingProtocol protocol = new StoreFileStreamingProtocol(); ChannelHandlerContext ctx = mock( ChannelHandlerContext.class ); // when - int lastCommittedTxBeforeStoreCopy = 100000; - protocol.end( ctx, StoreCopyFinishedResponse.Status.SUCCESS, lastCommittedTxBeforeStoreCopy ); + protocol.end( ctx, StoreCopyFinishedResponse.Status.SUCCESS ); // then InOrder inOrder = Mockito.inOrder( ctx ); inOrder.verify( ctx ).write( ResponseMessageType.STORE_COPY_FINISHED ); - inOrder.verify( ctx ).writeAndFlush( new StoreCopyFinishedResponse( SUCCESS, lastCommittedTxBeforeStoreCopy ) ); + inOrder.verify( ctx ).writeAndFlush( new StoreCopyFinishedResponse( SUCCESS ) ); inOrder.verifyNoMoreInteractions(); } diff --git a/enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/catchup/storecopy/StoreStreamingProcessTest.java b/enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/catchup/storecopy/StoreStreamingProcessTest.java index fba31254bb403..0570bd355d244 100644 --- a/enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/catchup/storecopy/StoreStreamingProcessTest.java +++ b/enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/catchup/storecopy/StoreStreamingProcessTest.java @@ -46,7 +46,7 @@ public class StoreStreamingProcessTest { // mocks - private final StoreStreamingProtocol protocol = mock( StoreStreamingProtocol.class ); + private final StoreFileStreamingProtocol protocol = mock( StoreFileStreamingProtocol.class ); private final CheckPointer checkPointer = mock( CheckPointer.class ); private final StoreResourceStreamFactory resourceStream = mock( StoreResourceStreamFactory.class ); private final ChannelHandlerContext ctx = mock( ChannelHandlerContext.class ); @@ -69,7 +69,7 @@ public void shouldPerformSuccessfulStoreCopyProcess() throws Exception when( checkPointer.tryCheckPoint( any() ) ).thenReturn( lastCheckpointedTxId ); when( checkPointer.lastCheckPointedTransactionId() ).thenReturn( lastCheckpointedTxId ); - when( protocol.end( ctx, SUCCESS, lastCheckpointedTxId ) ).thenReturn( completionPromise ); + when( protocol.end( ctx, SUCCESS ) ).thenReturn( completionPromise ); when( resourceStream.create() ).thenReturn( resources ); // when @@ -78,8 +78,7 @@ public void shouldPerformSuccessfulStoreCopyProcess() throws Exception // then InOrder inOrder = Mockito.inOrder( protocol, checkPointer ); inOrder.verify( checkPointer ).tryCheckPoint( any() ); - inOrder.verify( protocol ).stream( ctx, resources ); - inOrder.verify( protocol ).end( ctx, SUCCESS, lastCheckpointedTxId ); + inOrder.verify( protocol ).end( ctx, SUCCESS ); inOrder.verifyNoMoreInteractions(); assertEquals( 1, lock.getReadLockCount() ); @@ -101,6 +100,6 @@ public void shouldSignalFailure() process.fail( ctx, E_STORE_ID_MISMATCH ); // then - verify( protocol ).end( ctx, E_STORE_ID_MISMATCH, -1 ); + verify( protocol ).end( ctx, E_STORE_ID_MISMATCH ); } } diff --git a/enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/catchup/tx/CatchupPollingProcessTest.java b/enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/catchup/tx/CatchupPollingProcessTest.java index d8f947b9cb95a..182f9d0f6cdf4 100644 --- a/enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/catchup/tx/CatchupPollingProcessTest.java +++ b/enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/catchup/tx/CatchupPollingProcessTest.java @@ -29,6 +29,7 @@ import org.neo4j.causalclustering.catchup.CatchUpClient; import org.neo4j.causalclustering.catchup.CatchUpResponseCallback; +import org.neo4j.causalclustering.catchup.CatchupAddressProvider; import org.neo4j.causalclustering.catchup.CatchupResult; import org.neo4j.causalclustering.catchup.storecopy.LocalDatabase; import org.neo4j.causalclustering.catchup.storecopy.StoreCopyProcess; @@ -53,6 +54,7 @@ import static org.mockito.ArgumentMatchers.eq; import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; @@ -80,6 +82,7 @@ public class CatchupPollingProcessTest private final LocalDatabase localDatabase = mock( LocalDatabase.class ); private final TopologyService topologyService = mock( TopologyService.class ); private final AdvertisedSocketAddress coreMemberAddress = new AdvertisedSocketAddress( "hostname", 1234 ); + private final CatchupAddressProvider catchupAddressProvider = CatchupAddressProvider.fromSingleAddress( coreMemberAddress ); { when( localDatabase.storeId() ).thenReturn( storeId ); @@ -179,7 +182,7 @@ public void nextStateShouldBeTxPullingAfterASuccessfulStoreCopy() throws Throwab // then verify( localDatabase ).stopForStoreCopy(); verify( startStopOnStoreCopy ).stop(); - verify( storeCopyProcess ).replaceWithStoreFrom( any( AdvertisedSocketAddress.class ), eq( storeId ) ); + verify( storeCopyProcess ).replaceWithStoreFrom( any( CatchupAddressProvider.class ), eq( storeId ) ); verify( localDatabase ).start(); verify( startStopOnStoreCopy ).start(); verify( txApplier ).refreshFromNewStore(); diff --git a/enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/core/EnterpriseCoreEditionModuleIntegrationTest.java b/enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/core/EnterpriseCoreEditionModuleIT.java similarity index 98% rename from enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/core/EnterpriseCoreEditionModuleIntegrationTest.java rename to enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/core/EnterpriseCoreEditionModuleIT.java index 45552840c7e29..48555a164fa86 100644 --- a/enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/core/EnterpriseCoreEditionModuleIntegrationTest.java +++ b/enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/core/EnterpriseCoreEditionModuleIT.java @@ -45,7 +45,7 @@ import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; -public class EnterpriseCoreEditionModuleIntegrationTest +public class EnterpriseCoreEditionModuleIT { @Rule public ClusterRule clusterRule = new ClusterRule(); diff --git a/enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/core/state/snapshot/CoreStateDownloaderServiceTest.java b/enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/core/state/snapshot/CoreStateDownloaderServiceTest.java index f83b9494a3fdf..b56aa01dc875b 100644 --- a/enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/core/state/snapshot/CoreStateDownloaderServiceTest.java +++ b/enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/core/state/snapshot/CoreStateDownloaderServiceTest.java @@ -29,11 +29,13 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; +import org.neo4j.causalclustering.catchup.CatchupAddressProvider; import org.neo4j.causalclustering.core.consensus.LeaderLocator; import org.neo4j.causalclustering.core.consensus.NoLeaderFoundException; import org.neo4j.causalclustering.core.state.CommandApplicationProcess; import org.neo4j.causalclustering.identity.MemberId; import org.neo4j.function.Predicates; +import org.neo4j.helpers.AdvertisedSocketAddress; import org.neo4j.kernel.impl.util.CountingJobScheduler; import org.neo4j.kernel.impl.util.Listener; import org.neo4j.kernel.impl.util.Neo4jJobScheduler; @@ -51,6 +53,8 @@ public class CoreStateDownloaderServiceTest { private final MemberId someMember = new MemberId( UUID.randomUUID() ); + private final AdvertisedSocketAddress someMemberAddress = new AdvertisedSocketAddress( "localhost", 1234 ); + private final CatchupAddressProvider catchupAddressProvider = CatchupAddressProvider.fromSingleAddress( someMemberAddress ); private Neo4jJobScheduler neo4jJobScheduler; @Before @@ -76,9 +80,7 @@ public void shouldRunPersistentDownloader() throws Exception CoreStateDownloaderService coreStateDownloaderService = new CoreStateDownloaderService( neo4jJobScheduler, coreStateDownloader, applicationProcess, logProvider( log ), new NoTimeout() ); - LeaderLocator leaderLocator = mock( LeaderLocator.class ); - when( leaderLocator.getLeader() ).thenReturn( someMember ); - coreStateDownloaderService.scheduleDownload( leaderLocator ); + coreStateDownloaderService.scheduleDownload( catchupAddressProvider ); waitForApplierToResume( applicationProcess ); verify( applicationProcess, times( 1 ) ).pauseApplier( OPERATION_NAME ); @@ -101,11 +103,10 @@ public void shouldOnlyScheduleOnePersistentDownloaderTaskAtTheTime() AtomicBoolean availableLeader = new AtomicBoolean( false ); - LeaderLocator leaderLocator = new ControllableLeaderLocator( availableLeader ); - coreStateDownloaderService.scheduleDownload( leaderLocator ); - coreStateDownloaderService.scheduleDownload( leaderLocator ); - coreStateDownloaderService.scheduleDownload( leaderLocator ); - coreStateDownloaderService.scheduleDownload( leaderLocator ); + coreStateDownloaderService.scheduleDownload( catchupAddressProvider ); + coreStateDownloaderService.scheduleDownload( catchupAddressProvider ); + coreStateDownloaderService.scheduleDownload( catchupAddressProvider ); + coreStateDownloaderService.scheduleDownload( catchupAddressProvider ); availableLeader.set( true ); diff --git a/enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/core/state/snapshot/CoreStateDownloaderTest.java b/enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/core/state/snapshot/CoreStateDownloaderTest.java index 521c222617676..ada47959b789f 100644 --- a/enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/core/state/snapshot/CoreStateDownloaderTest.java +++ b/enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/core/state/snapshot/CoreStateDownloaderTest.java @@ -27,6 +27,7 @@ import java.util.UUID; import org.neo4j.causalclustering.catchup.CatchUpClient; +import org.neo4j.causalclustering.catchup.CatchupAddressProvider; import org.neo4j.causalclustering.catchup.storecopy.CommitStateHelper; import org.neo4j.causalclustering.catchup.storecopy.LocalDatabase; import org.neo4j.causalclustering.catchup.storecopy.RemoteStore; @@ -67,12 +68,13 @@ public class CoreStateDownloaderTest private final MemberId remoteMember = new MemberId( UUID.randomUUID() ); private final AdvertisedSocketAddress remoteAddress = new AdvertisedSocketAddress( "remoteAddress", 1234 ); + private final CatchupAddressProvider catchupAddressProvider = CatchupAddressProvider.fromSingleAddress( remoteAddress ); private final StoreId storeId = new StoreId( 1, 2, 3, 4 ); private final File storeDir = new File( "graph.db" ); private final CoreStateDownloader downloader = new CoreStateDownloader( localDatabase, startStopLife, remoteStore, catchUpClient, logProvider, storeCopyProcess, coreStateMachines, - snapshotService, topologyService, commitStateHelper ); + snapshotService, commitStateHelper ); @Before public void commonMocking() @@ -91,11 +93,11 @@ public void shouldDownloadCompleteStoreWhenEmpty() throws Throwable when( localDatabase.isEmpty() ).thenReturn( true ); // when - downloader.downloadSnapshot( remoteMember ); + downloader.downloadSnapshot( catchupAddressProvider ); // then verify( remoteStore, never() ).tryCatchingUp( any(), any(), any(), anyBoolean() ); - verify( storeCopyProcess ).replaceWithStoreFrom( remoteAddress, remoteStoreId ); + verify( storeCopyProcess ).replaceWithStoreFrom( catchupAddressProvider, remoteStoreId ); } @Test @@ -105,7 +107,7 @@ public void shouldStopDatabaseDuringDownload() throws Throwable when( localDatabase.isEmpty() ).thenReturn( true ); // when - downloader.downloadSnapshot( remoteMember ); + downloader.downloadSnapshot( catchupAddressProvider ); // then verify( startStopLife ).stop(); @@ -125,7 +127,7 @@ public void shouldNotOverwriteNonEmptyMismatchingStore() throws Exception // when try { - downloader.downloadSnapshot( remoteMember ); + downloader.downloadSnapshot( catchupAddressProvider ); fail(); } catch ( StoreCopyFailedException e ) @@ -147,7 +149,7 @@ public void shouldCatchupIfPossible() throws Exception when( remoteStore.tryCatchingUp( remoteAddress, storeId, storeDir, false ) ).thenReturn( SUCCESS_END_OF_STREAM ); // when - downloader.downloadSnapshot( remoteMember ); + downloader.downloadSnapshot( catchupAddressProvider ); // then verify( remoteStore ).tryCatchingUp( remoteAddress, storeId, storeDir, false ); @@ -163,10 +165,10 @@ public void shouldDownloadWholeStoreIfCannotCatchUp() throws Exception when( remoteStore.tryCatchingUp( remoteAddress, storeId, storeDir, false ) ).thenReturn( E_TRANSACTION_PRUNED ); // when - downloader.downloadSnapshot( remoteMember ); + downloader.downloadSnapshot( catchupAddressProvider ); // then verify( remoteStore ).tryCatchingUp( remoteAddress, storeId, storeDir, false ); - verify( storeCopyProcess ).replaceWithStoreFrom( remoteAddress, storeId ); + verify( storeCopyProcess ).replaceWithStoreFrom( catchupAddressProvider, storeId ); } } diff --git a/enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/core/state/snapshot/PersistentSnapshotDownloaderTest.java b/enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/core/state/snapshot/PersistentSnapshotDownloaderTest.java index bf9a326260aa3..2e4ee2a51956f 100644 --- a/enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/core/state/snapshot/PersistentSnapshotDownloaderTest.java +++ b/enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/core/state/snapshot/PersistentSnapshotDownloaderTest.java @@ -21,16 +21,14 @@ import org.junit.Test; -import java.util.UUID; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; +import org.neo4j.causalclustering.catchup.CatchupAddressProvider; import org.neo4j.causalclustering.catchup.storecopy.StoreCopyFailedException; -import org.neo4j.causalclustering.core.consensus.LeaderLocator; -import org.neo4j.causalclustering.core.consensus.NoLeaderFoundException; import org.neo4j.causalclustering.core.state.CommandApplicationProcess; -import org.neo4j.causalclustering.identity.MemberId; import org.neo4j.function.Predicates; +import org.neo4j.helpers.AdvertisedSocketAddress; import org.neo4j.logging.Log; import org.neo4j.logging.NullLogProvider; @@ -40,13 +38,13 @@ import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.when; import static org.mockito.internal.verification.VerificationModeFactory.times; import static org.neo4j.causalclustering.core.state.snapshot.PersistentSnapshotDownloader.OPERATION_NAME; public class PersistentSnapshotDownloaderTest { - private final MemberId someMember = new MemberId( UUID.randomUUID() ); + private final AdvertisedSocketAddress fromAddress = new AdvertisedSocketAddress( "localhost", 1234 ); + private final CatchupAddressProvider catchupAddressProvider = CatchupAddressProvider.fromSingleAddress( fromAddress ); @Test public void shouldPauseAndResumeApplicationProcessIfDownloadIsSuccessful() throws Exception @@ -55,11 +53,8 @@ public void shouldPauseAndResumeApplicationProcessIfDownloadIsSuccessful() throw CoreStateDownloader coreStateDownloader = mock( CoreStateDownloader.class ); final CommandApplicationProcess applicationProcess = mock( CommandApplicationProcess.class ); final Log log = mock( Log.class ); - LeaderLocator leaderLocator = mock( LeaderLocator.class ); - when( leaderLocator.getLeader() ).thenReturn( someMember ); PersistentSnapshotDownloader persistentSnapshotDownloader = - new PersistentSnapshotDownloader( leaderLocator, applicationProcess, coreStateDownloader, log, - new NoTimeout() ); + new PersistentSnapshotDownloader( catchupAddressProvider, applicationProcess, coreStateDownloader, log, new NoTimeout() ); // when persistentSnapshotDownloader.run(); @@ -76,16 +71,13 @@ public void shouldResumeCommandApplicationProcessIfInterrupted() throws Exceptio { // given CoreStateDownloader coreStateDownloader = mock( CoreStateDownloader.class ); - doThrow( StoreCopyFailedException.class ).when( coreStateDownloader ).downloadSnapshot( someMember ); + doThrow( StoreCopyFailedException.class ).when( coreStateDownloader ).downloadSnapshot( catchupAddressProvider ); final CommandApplicationProcess applicationProcess = mock( CommandApplicationProcess.class ); - LeaderLocator leaderLocator = mock( LeaderLocator.class ); - when( leaderLocator.getLeader() ).thenReturn( someMember ); final Log log = mock( Log.class ); NoTimeout timeout = new NoTimeout(); PersistentSnapshotDownloader persistentSnapshotDownloader = - new PersistentSnapshotDownloader( leaderLocator, applicationProcess, coreStateDownloader, log, - timeout ); + new PersistentSnapshotDownloader( catchupAddressProvider, applicationProcess, coreStateDownloader, log, timeout ); // when Thread thread = new Thread( persistentSnapshotDownloader ); @@ -105,16 +97,14 @@ public void shouldResumeCommandApplicationProcessIfDownloaderIsStopped() throws { // given CoreStateDownloader coreStateDownloader = mock( CoreStateDownloader.class ); + doThrow( StoreCopyFailedException.class ).when( coreStateDownloader ).downloadSnapshot( any() ); + final CommandApplicationProcess applicationProcess = mock( CommandApplicationProcess.class ); - LeaderLocator leaderLocator = mock( LeaderLocator.class ); - doThrow( NoLeaderFoundException.class ).when( leaderLocator ).getLeader(); final Log log = mock( Log.class ); - NoTimeout timeout = new - NoTimeout(); + NoTimeout timeout = new NoTimeout(); PersistentSnapshotDownloader persistentSnapshotDownloader = - new PersistentSnapshotDownloader( leaderLocator, applicationProcess, coreStateDownloader, log, - timeout ); + new PersistentSnapshotDownloader( null, applicationProcess, coreStateDownloader, log, timeout ); // when Thread thread = new Thread( persistentSnapshotDownloader ); @@ -136,13 +126,10 @@ public void shouldEventuallySucceed() throws Exception CoreStateDownloader coreStateDownloader = new EventuallySuccessfulDownloader( 3 ); final CommandApplicationProcess applicationProcess = mock( CommandApplicationProcess.class ); - LeaderLocator leaderLocator = mock( LeaderLocator.class ); - when( leaderLocator.getLeader() ).thenReturn( someMember ); final Log log = mock( Log.class ); NoTimeout timeout = new NoTimeout(); PersistentSnapshotDownloader persistentSnapshotDownloader = - new PersistentSnapshotDownloader( leaderLocator, applicationProcess, coreStateDownloader, log, - timeout ); + new PersistentSnapshotDownloader( catchupAddressProvider, applicationProcess, coreStateDownloader, log, timeout ); // when persistentSnapshotDownloader.run(); @@ -160,20 +147,17 @@ public void shouldNotStartDownloadIfAlreadyCompleted() throws Exception // given CoreStateDownloader coreStateDownloader = mock( CoreStateDownloader.class ); final CommandApplicationProcess applicationProcess = mock( CommandApplicationProcess.class ); - LeaderLocator leaderLocator = mock( LeaderLocator.class ); - when( leaderLocator.getLeader() ).thenReturn( someMember ); final Log log = mock( Log.class ); PersistentSnapshotDownloader persistentSnapshotDownloader = - new PersistentSnapshotDownloader( leaderLocator, applicationProcess, coreStateDownloader, log, - new NoTimeout() ); + new PersistentSnapshotDownloader( catchupAddressProvider, applicationProcess, coreStateDownloader, log, new NoTimeout() ); // when persistentSnapshotDownloader.run(); persistentSnapshotDownloader.run(); // then - verify( coreStateDownloader, times( 1 ) ).downloadSnapshot( someMember ); + verify( coreStateDownloader, times( 1 ) ).downloadSnapshot( catchupAddressProvider ); verify( applicationProcess, times( 1 ) ).pauseApplier( OPERATION_NAME ); verify( applicationProcess, times( 1 ) ).resumeApplier( OPERATION_NAME ); } @@ -184,14 +168,12 @@ public void shouldNotStartIfCurrentlyRunning() throws Exception // given CoreStateDownloader coreStateDownloader = mock( CoreStateDownloader.class ); final CommandApplicationProcess applicationProcess = mock( CommandApplicationProcess.class ); - LeaderLocator leaderLocator = mock( LeaderLocator.class ); - doThrow( NoLeaderFoundException.class ).when( leaderLocator ).getLeader(); + doThrow( StoreCopyFailedException.class ).when( coreStateDownloader ).downloadSnapshot( any() ); final Log log = mock( Log.class ); NoTimeout timeout = new NoTimeout(); PersistentSnapshotDownloader persistentSnapshotDownloader = - new PersistentSnapshotDownloader( leaderLocator, applicationProcess, coreStateDownloader, log, - timeout ); + new PersistentSnapshotDownloader( catchupAddressProvider, applicationProcess, coreStateDownloader, log, timeout ); Thread thread = new Thread( persistentSnapshotDownloader ); @@ -218,14 +200,12 @@ private class EventuallySuccessfulDownloader extends CoreStateDownloader private EventuallySuccessfulDownloader( int after ) { - super( null, null, null, - null, NullLogProvider.getInstance(), null, null, - null, null, null ); + super( null, null, null, null, NullLogProvider.getInstance(), null, null, null, null ); this.after = after; } @Override - void downloadSnapshot( MemberId source ) throws StoreCopyFailedException + void downloadSnapshot( CatchupAddressProvider addressProvider ) throws StoreCopyFailedException { if ( after-- > 0 ) { @@ -233,5 +213,4 @@ void downloadSnapshot( MemberId source ) throws StoreCopyFailedException } } } - } diff --git a/enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/messaging/TestServer.java b/enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/messaging/TestServer.java index 52a8f8b614827..af29817aa8b65 100644 --- a/enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/messaging/TestServer.java +++ b/enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/messaging/TestServer.java @@ -33,7 +33,7 @@ import java.util.Set; import java.util.concurrent.ConcurrentSkipListSet; -class TestServer +public class TestServer { private final int port; private final ChildHandler childHandler; @@ -45,13 +45,13 @@ class TestServer this( port, null ); } - private TestServer( int port, ChannelInitializer channelInitializer ) + public TestServer( int port, ChannelInitializer channelInitializer ) { this.port = port; this.childHandler = new ChildHandler( channelInitializer ); } - void start() + public void start() { elg = new NioEventLoopGroup( 0 ); ServerBootstrap boot = new ServerBootstrap() @@ -63,7 +63,7 @@ void start() fServer.syncUninterruptibly(); } - void stop() + public void stop() { if ( elg != null ) { diff --git a/enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/scenarios/ConnectionInfoIT.java b/enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/scenarios/ConnectionInfoIT.java index a7aafe8e140cd..40482afe83141 100644 --- a/enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/scenarios/ConnectionInfoIT.java +++ b/enterprise/causal-clustering/src/test/java/org/neo4j/causalclustering/scenarios/ConnectionInfoIT.java @@ -30,6 +30,7 @@ import java.util.function.Supplier; import org.neo4j.causalclustering.catchup.CatchupServer; +import org.neo4j.causalclustering.catchup.CheckpointerSupplier; import org.neo4j.causalclustering.core.state.CoreSnapshotService; import org.neo4j.io.fs.FileSystemAbstraction; import org.neo4j.io.pagecache.PageCache; @@ -74,7 +75,7 @@ public void catchupServerMessage() throws Throwable CatchupServer catchupServer = new CatchupServer( logProvider, userLogProvider, mockSupplier(), mockSupplier(), mockSupplier(), mockSupplier(), mock( BooleanSupplier.class ), snapshotService, config, new Monitors(), - mockSupplier(), mock( FileSystemAbstraction.class ), mock( PageCache.class ), + mock( CheckpointerSupplier.class), mock( FileSystemAbstraction.class ), mock( PageCache.class ), new StoreCopyCheckPointMutex(), null ); //then diff --git a/enterprise/com/src/main/java/org/neo4j/com/storecopy/StoreCopyServer.java b/enterprise/com/src/main/java/org/neo4j/com/storecopy/StoreCopyServer.java index 13108d4042a51..8d987910d5ba0 100644 --- a/enterprise/com/src/main/java/org/neo4j/com/storecopy/StoreCopyServer.java +++ b/enterprise/com/src/main/java/org/neo4j/com/storecopy/StoreCopyServer.java @@ -164,8 +164,7 @@ public RequestContext flushStoresAndStreamStoreFiles( String triggerName, StoreW // Copy the store files long lastAppliedTransaction; - try ( Resource lock = mutex.storeCopy( checkPointAction ); - ResourceIterator files = dataSource.listStoreFiles( includeLogs ) ) + try ( Resource lock = mutex.storeCopy( checkPointAction ); ResourceIterator files = dataSource.listStoreFiles( includeLogs ) ) { lastAppliedTransaction = checkPointer.lastCheckPointedTransactionId(); monitor.startStreamingStoreFiles( storeCopyIdentifier ); diff --git a/enterprise/fulltext-addon/src/main/java/org/neo4j/kernel/api/impl/fulltext/FulltextProviderImpl.java b/enterprise/fulltext-addon/src/main/java/org/neo4j/kernel/api/impl/fulltext/FulltextProviderImpl.java index 1cc85ca7c6bac..d4ee3a1f9d32d 100644 --- a/enterprise/fulltext-addon/src/main/java/org/neo4j/kernel/api/impl/fulltext/FulltextProviderImpl.java +++ b/enterprise/fulltext-addon/src/main/java/org/neo4j/kernel/api/impl/fulltext/FulltextProviderImpl.java @@ -46,7 +46,7 @@ import org.neo4j.kernel.api.exceptions.InvalidArgumentsException; import org.neo4j.kernel.impl.transaction.log.TransactionIdStore; import org.neo4j.kernel.impl.transaction.state.NeoStoreFileListing; -import org.neo4j.kernel.impl.transaction.state.NeoStoreFileListing.MultiResource; +import org.neo4j.kernel.impl.util.MultiResource; import org.neo4j.logging.Log; import org.neo4j.scheduler.JobScheduler; import org.neo4j.storageengine.api.StoreFileMetadata;