Skip to content
Draft
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -91,7 +91,7 @@ public class FileCopyBasedReplicationConfig {
public FileCopyBasedReplicationConfig(VerifiableProperties verifiableProperties) {
fileCopyMetaDataFileName = verifiableProperties.getString(FILE_COPY_META_DATA_FILE_NAME, "segments_metadata_file");
fileCopyParallelPartitionHydrationCountPerDisk = verifiableProperties.getInt(FILE_COPY_PARALLEL_PARTITION_HYDRATION_COUNT_PER_DISK, 1);
fileCopyNumberOfFileCopyThreads = verifiableProperties.getInt(FILE_COPY_NUMBER_OF_FILE_COPY_THREADS, 4);
fileCopyNumberOfFileCopyThreads = verifiableProperties.getInt(FILE_COPY_NUMBER_OF_FILE_COPY_THREADS, 1);
fileCopyDataFlushIntervalInMbs = verifiableProperties.getLong(FILE_COPY_DATA_FLUSH_INTERVAL_IN_MBS, 1000);
fileCopyReplicaTimeoutSecs = verifiableProperties.getLong(FILE_COPY_REPLICA_TIMEOUT_SECS, 36000);
fileCopySchedulerWaitTimeSecs = verifiableProperties.getLong(FILE_COPY_SCHEDULER_WAIT_TIME_SECS, 30);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -53,6 +53,7 @@ StoreFileChunk readStoreFileChunkFromDisk(String fileName, long offset, long siz
*/
void moveAllRegularFiles(String srcDirPath, String destDirPath) throws IOException;

void cleanUpDirectory(String srcPath);
/**
* move a pre-allocated file in the target path from diskSpaceAllocator's pre-allocated files for the store
*
Expand Down
24 changes: 24 additions & 0 deletions ambry-api/src/main/java/com/github/ambry/store/SegmentTracker.java
Original file line number Diff line number Diff line change
@@ -0,0 +1,24 @@
package com.github.ambry.store;

import java.io.RandomAccessFile;
import java.nio.channels.FileChannel;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.CyclicBarrier;


public class SegmentTracker {
private final RandomAccessFile randomAccessFile;


public SegmentTracker(RandomAccessFile randomAccessFile) {
this.randomAccessFile = randomAccessFile;
}
public boolean isOpen(){
return this.randomAccessFile.getChannel().isOpen();
}

public FileChannel getRandomAccessFilechannel() {
return randomAccessFile.getChannel();
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -878,10 +878,33 @@ public void onPartitionBecomeBootstrapFromPreBootstrap(String partitionName) {

@Override
public void onPartitionBecomeBootstrapFromOffline(String partitionName) {
try {
Thread.sleep(10000);
} catch (InterruptedException e) {
throw new RuntimeException(e);
}
participantMetrics.incStateTransitionMetric(partitionName, ReplicaState.OFFLINE, ReplicaState.BOOTSTRAP);
try {
logger.info("FCH TEST: Transition For Partition PreBootstrap From Offline {}", partitionName);
// 1. take actions in storage manager (add new replica if necessary)
onPartitionBecomePreBootstrapFromOffline(partitionName);
try{
onPartitionBecomePreBootstrapFromOffline(partitionName);
}catch (Exception e){
logger.error("Error in onPartitionBecomePreBootstrapFromOffline", e);
localPartitionAndState.put(partitionName, ReplicaState.ERROR);
throw e;
}
logger.info("FCH TEST: Transition For Partition FileCopy from PreBootstrap {}", partitionName);
PartitionStateChangeListener fileCopyStateChangeListener =
partitionStateChangeListeners.get(StateModelListenerType.FileCopyManagerListener);

if (fileCopyStateChangeListener != null) {
logger.info("FCH TEST: Listener invoked for partition {}", partitionName);
fileCopyStateChangeListener.onPartitionBecomeBootstrapFromOffline(partitionName);
replicaSyncUpManager.waitForFileCopyCompleted(partitionName);
}
logger.info("FCH TEST: Transition For Partition Bootstrap from PreBootstrap {}", partitionName);

onPartitionBecomeBootstrapFromPreBootstrap(partitionName);
// 2. take actions in replication manager (add new replica if necessary)
PartitionStateChangeListener replicationManagerListener =
Expand All @@ -897,15 +920,19 @@ public void onPartitionBecomeBootstrapFromOffline(String partitionName) {
}
} catch (Exception e) {
localPartitionAndState.put(partitionName, ReplicaState.ERROR);
throw e;
try {
throw e;
} catch (InterruptedException ex) {
throw new RuntimeException(ex);
}
}
logger.info("Before setting partition {} to bootstrap", partitionName);
localPartitionAndState.put(partitionName, ReplicaState.BOOTSTRAP);
participantMetrics.decStateTransitionMetric(partitionName, ReplicaState.OFFLINE, ReplicaState.BOOTSTRAP);
}

@Override
public void onPartitionBecomeStandbyFromBootstrap(String partitionName) {
public void onPartitionBecomeStandbyFromBootstrap(String partitionName){
participantMetrics.incStateTransitionMetric(partitionName, ReplicaState.BOOTSTRAP, ReplicaState.STANDBY);
try {
// 1. take actions in replication manager (wait for replica to finish bootstrapping)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -110,6 +110,7 @@ public List<DiskId> getDiskIdsToHydrate() {
public void submitReplicaForHydration(ReplicaId replicaId, FileCopyStatusListener fileCopyStatusListener,
FileCopyHandler fileCopyHandler) {
try {
logger.info("FCH TEST: Submit");
threadQueueLock.lock();
DiskId diskId = replicaId.getDiskId();
FileCopyThread fileCopyThread = new FileCopyThread(fileCopyHandler, fileCopyStatusListener);
Expand All @@ -134,18 +135,18 @@ public void submitReplicaForHydration(ReplicaId replicaId, FileCopyStatusListene
@Override
public void stopAndRemoveReplicaFromThreadPool(ReplicaId replicaId) throws InterruptedException {
threadQueueLock.lock();
logger.info("Stopping and removing replica {} from thread pool", replicaId);
logger.info("FCH TEST: Stopping and removing replica {} from thread pool", replicaId);
FileCopyThread fileCopyThread = replicaToFileCopyThread.get(replicaId);

if (fileCopyThread == null || !fileCopyThread.isAlive()) {
logger.info("No thread found for replica {}. Nothing to stop.", replicaId);
logger.info("FCH TEST: No thread found for replica {}. Nothing to stop.", replicaId);
threadQueueLock.unlock();
return;
}
long threadShutDownInitiationTime = System.currentTimeMillis();
logger.info("Stopping thread for replica {}", replicaId);
logger.info("FCH TEST: Stopping thread for replica {}", replicaId);
fileCopyThread.shutDown();
logger.info("Thread for replica {} stopped in {} ms", replicaId,
logger.info("FCH TEST: Thread for replica {} stopped in {} ms", replicaId,
System.currentTimeMillis() - threadShutDownInitiationTime);
threadQueueLock.unlock();
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@
import com.codahale.metrics.MetricRegistry;
import com.github.ambry.clustermap.ClusterMap;
import com.github.ambry.clustermap.ClusterParticipant;
import com.github.ambry.clustermap.PartitionId;
import com.github.ambry.clustermap.PartitionStateChangeListener;
import com.github.ambry.clustermap.ReplicaId;
import com.github.ambry.clustermap.ReplicaSyncUpManager;
Expand All @@ -30,8 +31,16 @@
import com.github.ambry.replica.prioritization.PrioritizationManager;
import com.github.ambry.replica.prioritization.PrioritizationManagerFactory;
import com.github.ambry.server.StoreManager;
import java.io.BufferedReader;
import java.io.FileReader;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.Objects;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.stream.Collectors;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

Expand Down Expand Up @@ -81,7 +90,7 @@ public FileCopyBasedReplicationManager(FileCopyBasedReplicationConfig fileCopyBa
if (clusterParticipant != null) {
clusterParticipant.registerPartitionStateChangeListener(StateModelListenerType.FileCopyManagerListener,
new PartitionStateChangeListenerImpl());
logger.info("File Copy Manager's state change listener registered!");
logger.info("FCH TEST: File Copy Manager's state change listener registered!");
} else {
throw new InstantiationException("File Copy Manager cannot be instantiated without a ClusterParticipant");
}
Expand All @@ -90,6 +99,7 @@ public FileCopyBasedReplicationManager(FileCopyBasedReplicationConfig fileCopyBa
this.prioritizationManager = prioritizationManager;
this.fileCopyBasedReplicationScheduler = fileCopyBasedReplicationSchedulerFactory.getFileCopyBasedReplicationScheduler();
this.fileCopyBasedReplicationSchedulerThread = new Thread(fileCopyBasedReplicationScheduler);

if(!prioritizationManager.isRunning()) {
throw new InstantiationException("File Copy cannot run when Prioritization Manager is not running");
}
Expand All @@ -103,23 +113,79 @@ public void start() throws InterruptedException, IOException {
logger.info("Starting FileCopyBasedReplicationManager");
fileCopyBasedReplicationSchedulerThread.start();
isRunning = true;
logger.info("FileCopyBasedReplicationManager started");
logger.info("FCH TEST: FileCopyBasedReplicationManager started");
// PartitionStateChangeListenerImpl partitionStateChangeListener = new PartitionStateChangeListenerImpl();
// List<Long> partitionIds = Arrays.asList(20l, 127l);
//
// logger.info("FCH TEST: All Partitions to be hydrated up: {}", storeManager.getLocalPartitions().stream().map(
// PartitionId::getId).collect(Collectors.toList()));
//
// List<PartitionId> partitionIdList =
// storeManager.getLocalPartitions().stream().filter(p -> partitionIds.contains(p.getId())).collect(Collectors.toList());
//
// logger.info("FCH TEST: Partitions to be hydrated up: {}", partitionIdList);
// //Integrate clean up.
// ExecutorService executor = Executors.newFixedThreadPool(30); // use appropriate number of threads
//
// for (PartitionId partitionId : partitionIdList) {
// executor.execute(() -> {
// try {
// partitionStateChangeListener.onPartitionBecomeBootstrapFromOffline(String.valueOf(partitionId.getId()));
// } catch (Exception e) {
// logger.error("FCH TEST: Failed to build state for file copy for partition {}", partitionId, e);
// }
// });
// }
}

public void shutdown() throws InterruptedException {
logger.info("Shutting down FileCopyBasedReplicationManager");
logger.info("FCH TEST: Shutting down FileCopyBasedReplicationManager");
fileCopyBasedReplicationScheduler.shutdown();
fileCopyBasedReplicationSchedulerThread.join();
isRunning = false;
logger.info("FileCopyBasedReplicationManager shutdown");
logger.info("FCH TEST: FileCopyBasedReplicationManager shutdown");
}

public static List<String> readFromFile(String fileName) {
String line;
String[] values = new String[0];
try (BufferedReader br = new BufferedReader(new FileReader(fileName))) {
while ((line = br.readLine()) != null) {
// Split by comma
values = line.split(",");
for (String value : values) {
System.out.print(value.trim() + " ");
}
System.out.println(); // Newline after each row
}
} catch (IOException e) {
e.printStackTrace();
}
return Arrays.asList(values);
}

class PartitionStateChangeListenerImpl implements PartitionStateChangeListener {

@Override
public void onPartitionBecomeBootstrapFromOffline(String partitionName) {
//List<Long> partitionIds = Arrays.asList();//Arrays.asList(419l);
List<Long> fileCopyPartitionNames = readFromFile("/mnt/u001/partitions.txt").stream().map(Long::valueOf).collect(
Collectors.toList());
logger.info("FCH TEST: Filtered partitions {}",
fileCopyPartitionNames);
logger.info("FCH TEST: FileCopyBasedReplicationManager: onPartitionBecomeBootstrapFromOffline for partition {}",
partitionName);
logger.info("FCH TEST: All Partitions to be hydrated are: {}", storeManager.getLocalPartitions().stream().map(
PartitionId::getId).collect(Collectors.toList()));
if(!fileCopyPartitionNames.contains(Long.valueOf(partitionName))) {
logger.warn("FCH TEST: Partition {} is not part of the list of partitions to be hydrated up. Ignoring state change", partitionName);
return;
}

logger.info("Started hydration for partitions {}", partitionName);

if(!isRunning){
logger.info("FileCopyBasedReplicationManager is not running. Ignoring state change for partition: {}", partitionName);
logger.info("FCH TEST: FileCopyBasedReplicationManager is not running. Ignoring state change for partition: {}", partitionName);
throw new StateTransitionException("FileCopyBasedReplicationManager is not running. Ignoring state "
+ "change for partition: " + partitionName, StateTransitionException.
TransitionErrorCode.FileCopyBasedReplicationManagerNotRunning);
Expand All @@ -138,28 +204,30 @@ public void onPartitionBecomeBootstrapFromOffline(String partitionName) {
* If the file copy was already completed, then no need to do it again.
*/
if(storeManager.isFileExists(replicaId.getPartitionId(), storeConfig.storeFileCopyCompletedFileName)){
logger.info("File Copy Was Completed For Replica: " + replicaId.getPartitionId().toPathString());
logger.info("FCH TEST: File Copy Was Completed For Replica: " + replicaId.getPartitionId().toPathString());
return;
}

logger.info("Initiated File Copy Wait On ReplicaSyncUpManager for Replica: {}", replicaId.getPartitionId().toPathString());
logger.info("FCH TEST: Initiated File Copy Wait On ReplicaSyncUpManager for Replica: {}", replicaId.getPartitionId().toPathString());

replicaSyncUpManager.initiateFileCopy(replicaId);

logger.info("Adding Replica to Prioritization Manager For Replica: {}", replicaId.getPartitionId().toPathString());
logger.info("FCH TEST: Adding Replica to Prioritization Manager For Replica: {}", replicaId.getPartitionId().toPathString());
prioritizationManager.addReplica(replicaId);

try {
logger.info("Waiting for File Copy to be completed for Replica: {}", replicaId.getPartitionId().toPathString());
replicaSyncUpManager.waitForFileCopyCompleted(partitionName);
logger.info("File Copy Completed for Replica: {}", replicaId.getPartitionId().toPathString());
} catch (InterruptedException e) {
logger.info("FCH TEST: Waiting for File Copy to be completed for Replica: {}", replicaId.getPartitionId().toPathString());
//replicaSyncUpManager.waitForFileCopyCompleted(partitionName);
logger.info("FCH TEST: File Copy Completed for Replica: {}", replicaId.getPartitionId().toPathString());
} catch (Exception e) {
logger.error("File copy for partition {} was interrupted", partitionName);
throw new StateTransitionException("File copy for partition " + partitionName + " was interrupted",
StateTransitionException.TransitionErrorCode.FileCopyProtocolFailure);
} catch (StateTransitionException e){
logger.error("File copy for partition {} failed", partitionName);
throw e;
}
// } catch (StateTransitionException e){
// logger.error("File copy for partition {} failed", partitionName);
// throw e;
// }
}
@Override
public void onPartitionBecomeStandbyFromBootstrap(String partitionName) {
Expand Down
Loading