Skip to content

Commit

Permalink
Rationalised on-disk directory layout and file naming for raft logs a…
Browse files Browse the repository at this point in the history
…nd cluster state.
  • Loading branch information
jimwebber committed Jan 15, 2016
1 parent 0c07b89 commit 37fcf20
Show file tree
Hide file tree
Showing 14 changed files with 125 additions and 55 deletions.
Expand Up @@ -68,6 +68,7 @@ public class NaiveDurableRaftLog extends LifecycleAdapter implements RaftLog
public static final int ENTRY_RECORD_LENGTH = 16; public static final int ENTRY_RECORD_LENGTH = 16;
public static final int CONTENT_LENGTH_BYTES = 4; public static final int CONTENT_LENGTH_BYTES = 4;
public static final int COMMIT_INDEX_BYTES = 8; public static final int COMMIT_INDEX_BYTES = 8;
public static final String DIRECTORY_NAME = "raft-log";


private final Set<Listener> listeners = new CopyOnWriteArraySet<>(); private final Set<Listener> listeners = new CopyOnWriteArraySet<>();


Expand All @@ -93,6 +94,8 @@ public NaiveDurableRaftLog( FileSystemAbstraction fileSystem, File directory, Se
this.commitIndexMonitor = monitors.newMonitor( RaftLogCommitIndexMonitor.class, getClass(), RaftLog this.commitIndexMonitor = monitors.newMonitor( RaftLogCommitIndexMonitor.class, getClass(), RaftLog
.COMMIT_INDEX_TAG ); .COMMIT_INDEX_TAG );


directory.mkdirs();

try try
{ {
entriesChannel = fileSystem.open( new File( directory, "entries.log" ), "rw" ); entriesChannel = fileSystem.open( new File( directory, "entries.log" ), "rw" );
Expand Down
Expand Up @@ -43,6 +43,7 @@
*/ */
public class OnDiskIdAllocationState extends LifecycleAdapter implements IdAllocationState public class OnDiskIdAllocationState extends LifecycleAdapter implements IdAllocationState
{ {
public static final String DIRECTORY_NAME = "id-allocation-state";
public static final String FILENAME = "id.allocation."; public static final String FILENAME = "id.allocation.";


private final InMemoryIdAllocationState inMemoryIdAllocationState; private final InMemoryIdAllocationState inMemoryIdAllocationState;
Expand All @@ -51,12 +52,12 @@ public class OnDiskIdAllocationState extends LifecycleAdapter implements IdAlloc
private final ByteBuffer workingBuffer; private final ByteBuffer workingBuffer;
private final InMemoryIdAllocationState.InMemoryIdAllocationStateMarshal marshal; private final InMemoryIdAllocationState.InMemoryIdAllocationStateMarshal marshal;


public OnDiskIdAllocationState( FileSystemAbstraction fileSystemAbstraction, File storeDir, public OnDiskIdAllocationState( FileSystemAbstraction fileSystemAbstraction, File stateDir,
int numberOfEntriesBeforeRotation, Supplier<DatabaseHealth> databaseHealthSupplier ) int numberOfEntriesBeforeRotation, Supplier<DatabaseHealth> databaseHealthSupplier )
throws IOException throws IOException
{ {
File fileA = new File( storeDir, FILENAME + "A" ); File fileA = new File( stateDir, FILENAME + "a" );
File fileB = new File( storeDir, FILENAME + "B" ); File fileB = new File( stateDir, FILENAME + "b" );


this.workingBuffer = ByteBuffer.allocate( NUMBER_OF_BYTES_PER_WRITE ); this.workingBuffer = ByteBuffer.allocate( NUMBER_OF_BYTES_PER_WRITE );


Expand Down
Expand Up @@ -35,6 +35,7 @@ public class OnDiskRaftMembershipState<MEMBER> extends LifecycleAdapter implemen
{ {
public static final int MAX_SIZE_OF_ADDRESS_STATE_ON_DISK = 2_000_000; public static final int MAX_SIZE_OF_ADDRESS_STATE_ON_DISK = 2_000_000;
private static final String FILENAME = "membership.state."; private static final String FILENAME = "membership.state.";
public static final String DIRECTORY_NAME = "membership-state";


private final StatePersister<InMemoryRaftMembershipState<MEMBER>> statePersister; private final StatePersister<InMemoryRaftMembershipState<MEMBER>> statePersister;


Expand All @@ -50,8 +51,8 @@ public OnDiskRaftMembershipState( FileSystemAbstraction fileSystemAbstraction,
InMemoryRaftMembershipState.InMemoryRaftMembershipStateMarshal<MEMBER> marshal = new InMemoryRaftMembershipState.InMemoryRaftMembershipStateMarshal<>( InMemoryRaftMembershipState.InMemoryRaftMembershipStateMarshal<MEMBER> marshal = new InMemoryRaftMembershipState.InMemoryRaftMembershipStateMarshal<>(
memberMarshal ); memberMarshal );


File fileA = new File( storeDir, FILENAME + "A" ); File fileA = new File( storeDir, FILENAME + "a" );
File fileB = new File( storeDir, FILENAME + "B" ); File fileB = new File( storeDir, FILENAME + "b" );


RaftMembershipStateRecoveryManager<MEMBER> recoveryManager = new RaftMembershipStateRecoveryManager<>( RaftMembershipStateRecoveryManager<MEMBER> recoveryManager = new RaftMembershipStateRecoveryManager<>(
fileSystemAbstraction, marshal ); fileSystemAbstraction, marshal );
Expand Down
Expand Up @@ -35,18 +35,19 @@
public class OnDiskTermState extends LifecycleAdapter implements TermState public class OnDiskTermState extends LifecycleAdapter implements TermState
{ {
public static final String FILENAME = "term."; public static final String FILENAME = "term.";
public static final String DIRECTORY_NAME = "term-state";


private final ByteBuffer workingBuffer; private final ByteBuffer workingBuffer;
private InMemoryTermState inMemoryTermState; private InMemoryTermState inMemoryTermState;


private final StatePersister<InMemoryTermState> statePersister; private final StatePersister<InMemoryTermState> statePersister;


public OnDiskTermState( FileSystemAbstraction fileSystemAbstraction, File storeDir, public OnDiskTermState( FileSystemAbstraction fileSystemAbstraction, File stateDir,
int numberOfEntriesBeforeRotation, Supplier<DatabaseHealth> databaseHealthSupplier ) int numberOfEntriesBeforeRotation, Supplier<DatabaseHealth> databaseHealthSupplier )
throws IOException throws IOException
{ {
File fileA = new File( storeDir, FILENAME + "A" ); File fileA = new File( stateDir, FILENAME + "a" );
File fileB = new File( storeDir, FILENAME + "B" ); File fileB = new File( stateDir, FILENAME + "b" );


workingBuffer = ByteBuffer.allocate( InMemoryVoteState.InMemoryVoteStateMarshal workingBuffer = ByteBuffer.allocate( InMemoryVoteState.InMemoryVoteStateMarshal
.NUMBER_OF_BYTES_PER_VOTE ); .NUMBER_OF_BYTES_PER_VOTE );
Expand Down
Expand Up @@ -35,17 +35,18 @@
public class OnDiskVoteState<MEMBER> extends LifecycleAdapter implements VoteState<MEMBER> public class OnDiskVoteState<MEMBER> extends LifecycleAdapter implements VoteState<MEMBER>
{ {
public static final String FILENAME = "vote."; public static final String FILENAME = "vote.";
public static final String DIRECTORY_NAME = "vote-state";


private final StatePersister<InMemoryVoteState<MEMBER>> statePersister; private final StatePersister<InMemoryVoteState<MEMBER>> statePersister;


private InMemoryVoteState<MEMBER> inMemoryVoteState; private InMemoryVoteState<MEMBER> inMemoryVoteState;


public OnDiskVoteState( FileSystemAbstraction fileSystemAbstraction, File storeDir, public OnDiskVoteState( FileSystemAbstraction fileSystemAbstraction, File stateDir,
int numberOfEntriesBeforeRotation, Supplier<DatabaseHealth> databaseHealthSupplier, int numberOfEntriesBeforeRotation, Supplier<DatabaseHealth> databaseHealthSupplier,
Marshal<MEMBER> memberMarshal ) throws IOException Marshal<MEMBER> memberMarshal ) throws IOException
{ {
File fileA = new File( storeDir, FILENAME + "A" ); File fileA = new File( stateDir, FILENAME + "a" );
File fileB = new File( storeDir, FILENAME + "B" ); File fileB = new File( stateDir, FILENAME + "b" );


ByteBuffer workingBuffer = ByteBuffer.allocate( InMemoryVoteState.InMemoryVoteStateMarshal ByteBuffer workingBuffer = ByteBuffer.allocate( InMemoryVoteState.InMemoryVoteStateMarshal
.NUMBER_OF_BYTES_PER_VOTE ); .NUMBER_OF_BYTES_PER_VOTE );
Expand Down
Expand Up @@ -62,9 +62,9 @@
import org.neo4j.coreedge.raft.replication.token.ReplicatedPropertyKeyTokenHolder; import org.neo4j.coreedge.raft.replication.token.ReplicatedPropertyKeyTokenHolder;
import org.neo4j.coreedge.raft.replication.token.ReplicatedRelationshipTypeTokenHolder; import org.neo4j.coreedge.raft.replication.token.ReplicatedRelationshipTypeTokenHolder;
import org.neo4j.coreedge.raft.replication.tx.CommittingTransactions; import org.neo4j.coreedge.raft.replication.tx.CommittingTransactions;
import org.neo4j.coreedge.raft.replication.tx.CommittingTransactionsRegistry;
import org.neo4j.coreedge.raft.replication.tx.ReplicatedTransactionCommitProcess; import org.neo4j.coreedge.raft.replication.tx.ReplicatedTransactionCommitProcess;
import org.neo4j.coreedge.raft.replication.tx.ReplicatedTransactionStateMachine; import org.neo4j.coreedge.raft.replication.tx.ReplicatedTransactionStateMachine;
import org.neo4j.coreedge.raft.replication.tx.CommittingTransactionsRegistry;
import org.neo4j.coreedge.raft.roles.Role; import org.neo4j.coreedge.raft.roles.Role;
import org.neo4j.coreedge.raft.state.id_allocation.OnDiskIdAllocationState; import org.neo4j.coreedge.raft.state.id_allocation.OnDiskIdAllocationState;
import org.neo4j.coreedge.raft.state.membership.OnDiskRaftMembershipState; import org.neo4j.coreedge.raft.state.membership.OnDiskRaftMembershipState;
Expand Down Expand Up @@ -130,6 +130,7 @@
public class EnterpriseCoreEditionModule public class EnterpriseCoreEditionModule
extends EditionModule extends EditionModule
{ {
public static final String CLUSTER_STATE_DIRECTORY_NAME = "cluster-state";
private final RaftInstance<CoreMember> raft; private final RaftInstance<CoreMember> raft;


public RaftInstance<CoreMember> raft() public RaftInstance<CoreMember> raft()
Expand All @@ -140,13 +141,14 @@ public RaftInstance<CoreMember> raft()
public EnterpriseCoreEditionModule( final PlatformModule platformModule, public EnterpriseCoreEditionModule( final PlatformModule platformModule,
DiscoveryServiceFactory discoveryServiceFactory ) DiscoveryServiceFactory discoveryServiceFactory )
{ {
org.neo4j.kernel.impl.util.Dependencies dependencies = platformModule.dependencies; final org.neo4j.kernel.impl.util.Dependencies dependencies = platformModule.dependencies;
Config config = platformModule.config; final Config config = platformModule.config;
LogService logging = platformModule.logging; final LogService logging = platformModule.logging;
FileSystemAbstraction fileSystem = platformModule.fileSystem; final FileSystemAbstraction fileSystem = platformModule.fileSystem;
File storeDir = platformModule.storeDir; final File storeDir = platformModule.storeDir;
LifeSupport life = platformModule.life; final File clusterStateDirectory = createClusterStateDirectory( storeDir, fileSystem );
GraphDatabaseFacade graphDatabaseFacade = platformModule.graphDatabaseFacade; final LifeSupport life = platformModule.life;
final GraphDatabaseFacade graphDatabaseFacade = platformModule.graphDatabaseFacade;


LogProvider logProvider = logging.getInternalLogProvider(); LogProvider logProvider = logging.getInternalLogProvider();


Expand Down Expand Up @@ -178,14 +180,15 @@ public EnterpriseCoreEditionModule( final PlatformModule platformModule,
final DelayedRenewableTimeoutService raftTimeoutService = final DelayedRenewableTimeoutService raftTimeoutService =
new DelayedRenewableTimeoutService( Clock.SYSTEM_CLOCK, logProvider ); new DelayedRenewableTimeoutService( Clock.SYSTEM_CLOCK, logProvider );


File raftLogsDirectory = createRaftLogsDirectory( platformModule.storeDir, fileSystem ); NaiveDurableRaftLog raftLog = life.add( new NaiveDurableRaftLog( fileSystem,
NaiveDurableRaftLog raftLog = life.add( new NaiveDurableRaftLog( fileSystem, raftLogsDirectory, new File( clusterStateDirectory, NaiveDurableRaftLog.DIRECTORY_NAME ),
new RaftContentSerializer(), platformModule.monitors ) ); new RaftContentSerializer(), platformModule.monitors ) );


TermState termState; TermState termState;
try try
{ {
termState = life.add( new OnDiskTermState( fileSystem, raftLogsDirectory, termState = life.add( new OnDiskTermState( fileSystem,
new File( clusterStateDirectory, OnDiskTermState.DIRECTORY_NAME ),
config.get( CoreEdgeClusterSettings.term_state_size ), databaseHealthSupplier ) ); config.get( CoreEdgeClusterSettings.term_state_size ), databaseHealthSupplier ) );
} }
catch ( IOException e ) catch ( IOException e )
Expand All @@ -196,7 +199,8 @@ public EnterpriseCoreEditionModule( final PlatformModule platformModule,
VoteState<CoreMember> voteState; VoteState<CoreMember> voteState;
try try
{ {
voteState = life.add( new OnDiskVoteState<>( fileSystem, raftLogsDirectory, voteState = life.add( new OnDiskVoteState<>( fileSystem,
new File( clusterStateDirectory, OnDiskVoteState.DIRECTORY_NAME ),
config.get( CoreEdgeClusterSettings.vote_state_size ), databaseHealthSupplier, config.get( CoreEdgeClusterSettings.vote_state_size ), databaseHealthSupplier,
new CoreMember.CoreMemberMarshal() ) ); new CoreMember.CoreMemberMarshal() ) );
} }
Expand All @@ -209,8 +213,9 @@ public EnterpriseCoreEditionModule( final PlatformModule platformModule,
RaftMembershipState<CoreMember> raftMembershipState; RaftMembershipState<CoreMember> raftMembershipState;
try try
{ {
raftMembershipState = life.add( new OnDiskRaftMembershipState<>( raftMembershipState = life.add( new OnDiskRaftMembershipState<>( fileSystem,
fileSystem, storeDir, config.get( CoreEdgeClusterSettings.raft_membership_state_size ), new File( clusterStateDirectory, OnDiskRaftMembershipState.DIRECTORY_NAME ),
config.get( CoreEdgeClusterSettings.raft_membership_state_size ),
databaseHealthSupplier, new CoreMember.CoreMemberMarshal() ) ); databaseHealthSupplier, new CoreMember.CoreMemberMarshal() ) );
} }
catch ( IOException e ) catch ( IOException e )
Expand All @@ -236,9 +241,9 @@ public EnterpriseCoreEditionModule( final PlatformModule platformModule,
final IdAllocationState idAllocationState; final IdAllocationState idAllocationState;
try try
{ {
idAllocationState = life.add( new OnDiskIdAllocationState( fileSystem, new File( idAllocationState = life.add( new OnDiskIdAllocationState( fileSystem,
storeDir, "id-alloc-store" ), config.get( new File( clusterStateDirectory, OnDiskIdAllocationState.DIRECTORY_NAME ),
CoreEdgeClusterSettings.id_alloc_state_size ), databaseHealthSupplier ) ); config.get( CoreEdgeClusterSettings.id_alloc_state_size ), databaseHealthSupplier ) );
} }
catch ( IOException e ) catch ( IOException e )
{ {
Expand Down Expand Up @@ -330,9 +335,9 @@ public boolean isLeader()
return raft.currentRole() == Role.LEADER; return raft.currentRole() == Role.LEADER;
} }


private File createRaftLogsDirectory( File dir, FileSystemAbstraction fileSystem ) private File createClusterStateDirectory( File dir, FileSystemAbstraction fileSystem )
{ {
File raftLogDir = new File( dir, "raft-logs" ); File raftLogDir = new File( dir, CLUSTER_STATE_DIRECTORY_NAME );


try try
{ {
Expand Down Expand Up @@ -360,7 +365,8 @@ public static CommitProcessFactory createCommitProcessFactory( final Replicator


CommittingTransactions committingTransactions = new CommittingTransactionsRegistry(); CommittingTransactions committingTransactions = new CommittingTransactionsRegistry();
ReplicatedTransactionStateMachine replicatedTxStateMachine = new ReplicatedTransactionStateMachine( ReplicatedTransactionStateMachine replicatedTxStateMachine = new ReplicatedTransactionStateMachine(
localCommit, localSessionPool.getGlobalSession(), currentReplicatedLockState, committingTransactions ); localCommit, localSessionPool.getGlobalSession(), currentReplicatedLockState,
committingTransactions );


dependencies.satisfyDependencies( replicatedTxStateMachine ); dependencies.satisfyDependencies( replicatedTxStateMachine );


Expand Down Expand Up @@ -464,8 +470,8 @@ protected ReplicatedIdGeneratorFactory createIdGeneratorFactory( FileSystemAbstr
} }


protected Locks createLockManager( final Config config, final LogService logging, final Replicator replicator, protected Locks createLockManager( final Config config, final LogService logging, final Replicator replicator,
CoreMember myself, ReplicatedLockStateMachine<CoreMember> CoreMember myself, ReplicatedLockStateMachine<CoreMember>
replicatedLockStateMachine, LeaderLocator<CoreMember> leaderLocator ) replicatedLockStateMachine, LeaderLocator<CoreMember> leaderLocator )
{ {
Locks local = CommunityEditionModule.createLockManager( config, logging ); Locks local = CommunityEditionModule.createLockManager( config, logging );


Expand Down
Expand Up @@ -599,7 +599,7 @@ public void shouldNotCommitAheadOfMatchingHistory() throws Exception
public void shouldIncludeLatestAppendedInResponse() throws Exception public void shouldIncludeLatestAppendedInResponse() throws Exception
{ {
// given: just a single appended entry at follower // given: just a single appended entry at follower
RaftLogEntry entryA = new RaftLogEntry( 1, ReplicatedString.valueOf( "A" ) ); RaftLogEntry entryA = new RaftLogEntry( 1, ReplicatedString.valueOf( "b" ) );


InMemoryRaftLog raftLog = new InMemoryRaftLog(); InMemoryRaftLog raftLog = new InMemoryRaftLog();
raftLog.append( entryA ); raftLog.append( entryA );
Expand All @@ -612,7 +612,7 @@ public void shouldIncludeLatestAppendedInResponse() throws Exception


Follower follower = new Follower(); Follower follower = new Follower();


RaftLogEntry entryB = new RaftLogEntry( 1, ReplicatedString.valueOf( "B" ) ); RaftLogEntry entryB = new RaftLogEntry( 1, ReplicatedString.valueOf( "b" ) );


// when: append request for item way forward (index=10, term=2) // when: append request for item way forward (index=10, term=2)
Outcome<RaftTestMember> outcome = follower.handle( Outcome<RaftTestMember> outcome = follower.handle(
Expand Down
Expand Up @@ -19,18 +19,14 @@
*/ */
package org.neo4j.coreedge.raft.state; package org.neo4j.coreedge.raft.state;


import static org.hamcrest.Matchers.greaterThanOrEqualTo;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertThat;
import static org.junit.Assert.fail;

import java.io.File; import java.io.File;
import java.io.IOException; import java.io.IOException;
import java.nio.BufferUnderflowException; import java.nio.BufferUnderflowException;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;


import org.junit.Rule; import org.junit.Rule;
import org.junit.Test; import org.junit.Test;

import org.neo4j.adversaries.CountingAdversary; import org.neo4j.adversaries.CountingAdversary;
import org.neo4j.adversaries.MethodGuardedAdversary; import org.neo4j.adversaries.MethodGuardedAdversary;
import org.neo4j.adversaries.fs.AdversarialFileSystemAbstraction; import org.neo4j.adversaries.fs.AdversarialFileSystemAbstraction;
Expand All @@ -46,6 +42,11 @@
import org.neo4j.logging.NullLog; import org.neo4j.logging.NullLog;
import org.neo4j.test.TargetDirectory; import org.neo4j.test.TargetDirectory;


import static org.hamcrest.Matchers.greaterThanOrEqualTo;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertThat;
import static org.junit.Assert.fail;

public class PersistedStateIT public class PersistedStateIT
{ {
@Rule @Rule
Expand Down Expand Up @@ -250,11 +251,11 @@ private static class LongState
private final StatePersister<Long> statePersister; private final StatePersister<Long> statePersister;
private long theState = -1; private long theState = -1;


public LongState( FileSystemAbstraction fileSystemAbstraction, File storeDir, public LongState( FileSystemAbstraction fileSystemAbstraction, File stateDir,
int numberOfEntriesBeforeRotation ) throws IOException int numberOfEntriesBeforeRotation ) throws IOException
{ {
File fileA = new File( storeDir, FILENAME + "A" ); File fileA = new File( stateDir, FILENAME + "a" );
File fileB = new File( storeDir, FILENAME + "B" ); File fileB = new File( stateDir, FILENAME + "b" );


ByteBuffer workingBuffer = ByteBuffer.allocate( InMemoryVoteState.InMemoryVoteStateMarshal ByteBuffer workingBuffer = ByteBuffer.allocate( InMemoryVoteState.InMemoryVoteStateMarshal
.NUMBER_OF_BYTES_PER_VOTE ); .NUMBER_OF_BYTES_PER_VOTE );
Expand Down
Expand Up @@ -133,11 +133,11 @@ public AtomicInteger unmarshal( ByteBuffer source )


private File stateFileA() private File stateFileA()
{ {
return new File( testDir.directory(), FILENAME + "A" ); return new File( testDir.directory(), FILENAME + "a" );
} }


private File stateFileB() private File stateFileB()
{ {
return new File( testDir.directory(), FILENAME + "B" ); return new File( testDir.directory(), FILENAME + "b" );
} }
} }
Expand Up @@ -65,7 +65,7 @@ public void shouldDiscardTermIfChannelFails() throws Exception


EphemeralFileSystemAbstraction fs = new EphemeralFileSystemAbstraction(); EphemeralFileSystemAbstraction fs = new EphemeralFileSystemAbstraction();
FileSystemAbstraction fileSystem = new SelectiveFileSystemAbstraction( new File( testDir.directory(), FileSystemAbstraction fileSystem = new SelectiveFileSystemAbstraction( new File( testDir.directory(),
"term.A" ), new AdversarialFileSystemAbstraction( adversary, fs ), fs ); "term.a" ), new AdversarialFileSystemAbstraction( adversary, fs ), fs );
TermState log = createTermStore( fileSystem ); TermState log = createTermStore( fileSystem );


log.update( 21 ); log.update( 21 );
Expand Down
Expand Up @@ -59,7 +59,7 @@ public void shouldDiscardVoteIfChannelFails() throws Exception


EphemeralFileSystemAbstraction fs = new EphemeralFileSystemAbstraction(); EphemeralFileSystemAbstraction fs = new EphemeralFileSystemAbstraction();
FileSystemAbstraction fileSystem = new SelectiveFileSystemAbstraction( FileSystemAbstraction fileSystem = new SelectiveFileSystemAbstraction(
new File( testDir.directory(), "vote.A" ), new AdversarialFileSystemAbstraction( adversary, fs ), fs ); new File( testDir.directory(), "vote.a" ), new AdversarialFileSystemAbstraction( adversary, fs ), fs );
VoteState<CoreMember> store = createVoteStore( fileSystem ); VoteState<CoreMember> store = createVoteStore( fileSystem );


final CoreMember member1 = new CoreMember( new AdvertisedSocketAddress( "host1:1001" ), final CoreMember member1 = new CoreMember( new AdvertisedSocketAddress( "host1:1001" ),
Expand Down
Expand Up @@ -144,11 +144,11 @@ public void shouldSwitchToWritingToPreviouslyInactiveFileOnRecovery() throws Exc


// then // then
assertEquals( 3 * NUMBER_OF_BYTES_PER_WRITE, assertEquals( 3 * NUMBER_OF_BYTES_PER_WRITE,
fsa.getFileSize( new File( testDir.directory(), "id.allocation.A" ) ) ); fsa.getFileSize( new File( testDir.directory(), "id.allocation.a" ) ) );




assertEquals( NUMBER_OF_BYTES_PER_WRITE, assertEquals( NUMBER_OF_BYTES_PER_WRITE,
fsa.getFileSize( new File( testDir.directory(), "id.allocation.B" ) ) ); fsa.getFileSize( new File( testDir.directory(), "id.allocation.b" ) ) );
} }


@Test @Test
Expand Down Expand Up @@ -191,6 +191,6 @@ public synchronized StoreChannel open( File fileName, String mode ) throws IOExc


private File stateFileA() private File stateFileA()
{ {
return new File( testDir.directory(), FILENAME + "A" ); return new File( testDir.directory(), FILENAME + "a" );
} }
} }
Expand Up @@ -19,17 +19,17 @@
*/ */
package org.neo4j.coreedge.scenarios; package org.neo4j.coreedge.scenarios;


import org.junit.After;
import org.junit.Ignore;
import org.junit.Rule;
import org.junit.Test;

import java.io.File; import java.io.File;
import java.util.Set; import java.util.Set;
import java.util.concurrent.ExecutorService; import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors; import java.util.concurrent.Executors;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;


import org.junit.After;
import org.junit.Ignore;
import org.junit.Rule;
import org.junit.Test;

import org.neo4j.cluster.InstanceId; import org.neo4j.cluster.InstanceId;
import org.neo4j.coreedge.discovery.Cluster; import org.neo4j.coreedge.discovery.Cluster;
import org.neo4j.coreedge.server.CoreEdgeClusterSettings; import org.neo4j.coreedge.server.CoreEdgeClusterSettings;
Expand All @@ -50,11 +50,13 @@
import static java.util.Collections.singletonList; import static java.util.Collections.singletonList;
import static java.util.concurrent.TimeUnit.MINUTES; import static java.util.concurrent.TimeUnit.MINUTES;
import static java.util.concurrent.TimeUnit.SECONDS; import static java.util.concurrent.TimeUnit.SECONDS;

import static junit.framework.TestCase.fail; import static junit.framework.TestCase.fail;
import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.greaterThan; import static org.hamcrest.Matchers.greaterThan;
import static org.hamcrest.core.Is.is; import static org.hamcrest.core.Is.is;
import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertEquals;

import static org.neo4j.cluster.ClusterSettings.server_id; import static org.neo4j.cluster.ClusterSettings.server_id;
import static org.neo4j.coreedge.server.CoreEdgeClusterSettings.raft_advertised_address; import static org.neo4j.coreedge.server.CoreEdgeClusterSettings.raft_advertised_address;
import static org.neo4j.graphdb.Label.label; import static org.neo4j.graphdb.Label.label;
Expand Down Expand Up @@ -116,6 +118,8 @@ public void shouldReplicateTransactionToCoreServers() throws Exception
tx.success(); tx.success();
} }
} }

fail();
} }


@Test @Test
Expand Down

0 comments on commit 37fcf20

Please sign in to comment.