Introducing Data Replication To AsterixDB
This change includes the following:
- Add data replication properties to cluster properties and Managix validate command.
- Introduce Data Replication components.
- Add data replication required fields to LogRecord.
- Specialized LogManager for data replication.
- Fix for invalid cluster state on nodes failure.
- ASTERIXDB-139: Fix for cleaning workspace files on startup/shutdown.
- Fix for temp datasets storage reclamation.
- Allow MetadataNode rebinding with CC.
- Add flag to checkpoint to identify sharp checkpoints.
- ASTERIXDB-1170: Fix shutdown sequence
Change-Id: I729fdd1144dbc9ff039b4bc414494860d7553810
Reviewed-on: https://asterix-gerrit.ics.uci.edu/338
Tested-by: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
Reviewed-by: Young-Seok Kim <kisskys@gmail.com>
diff --git a/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/CommitRuntime.java b/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/CommitRuntime.java
index dfec32a..6f3078e 100644
--- a/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/CommitRuntime.java
+++ b/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/CommitRuntime.java
@@ -72,6 +72,7 @@
this.isWriteTransaction = isWriteTransaction;
this.longHashes = new long[2];
this.logRecord = new LogRecord();
+ logRecord.setNodeId(logMgr.getNodeId());
}
@Override
diff --git a/asterix-app/pom.xml b/asterix-app/pom.xml
index df9a0e5..66382e6 100644
--- a/asterix-app/pom.xml
+++ b/asterix-app/pom.xml
@@ -25,15 +25,14 @@
</parent>
<artifactId>asterix-app</artifactId>
- <licenses>
- <license>
- <name>Apache License, Version 2.0</name>
- <url>http://www.apache.org/licenses/LICENSE-2.0.txt</url>
- <distribution>repo</distribution>
- <comments>A business-friendly OSS license</comments>
- </license>
- </licenses>
-
+ <licenses>
+ <license>
+ <name>Apache License, Version 2.0</name>
+ <url>http://www.apache.org/licenses/LICENSE-2.0.txt</url>
+ <distribution>repo</distribution>
+ <comments>A business-friendly OSS license</comments>
+ </license>
+ </licenses>
<build>
<plugins>
<plugin>
@@ -216,5 +215,11 @@
<version>1.2.2</version>
<scope>test</scope>
</dependency>
+ <dependency>
+ <groupId>org.apache.asterix</groupId>
+ <artifactId>asterix-replication</artifactId>
+ <version>0.8.8-SNAPSHOT</version>
+ <scope>compile</scope>
+ </dependency>
</dependencies>
</project>
diff --git a/asterix-app/src/main/java/org/apache/asterix/api/common/AsterixAppRuntimeContext.java b/asterix-app/src/main/java/org/apache/asterix/api/common/AsterixAppRuntimeContext.java
index 2e7c23f..15252e9 100644
--- a/asterix-app/src/main/java/org/apache/asterix/api/common/AsterixAppRuntimeContext.java
+++ b/asterix-app/src/main/java/org/apache/asterix/api/common/AsterixAppRuntimeContext.java
@@ -25,12 +25,13 @@
import org.apache.asterix.common.api.AsterixThreadExecutor;
import org.apache.asterix.common.api.IAsterixAppRuntimeContext;
import org.apache.asterix.common.api.IDatasetLifecycleManager;
+import org.apache.asterix.common.config.AsterixBuildProperties;
import org.apache.asterix.common.config.AsterixCompilerProperties;
import org.apache.asterix.common.config.AsterixExternalProperties;
import org.apache.asterix.common.config.AsterixFeedProperties;
-import org.apache.asterix.common.config.AsterixBuildProperties;
import org.apache.asterix.common.config.AsterixMetadataProperties;
import org.apache.asterix.common.config.AsterixPropertiesAccessor;
+import org.apache.asterix.common.config.AsterixReplicationProperties;
import org.apache.asterix.common.config.AsterixStorageProperties;
import org.apache.asterix.common.config.AsterixTransactionProperties;
import org.apache.asterix.common.config.IAsterixPropertiesProvider;
@@ -39,10 +40,20 @@
import org.apache.asterix.common.exceptions.ACIDException;
import org.apache.asterix.common.exceptions.AsterixException;
import org.apache.asterix.common.feeds.api.IFeedManager;
+import org.apache.asterix.common.replication.IRemoteRecoveryManager;
+import org.apache.asterix.common.replication.IReplicaResourcesManager;
+import org.apache.asterix.common.replication.IReplicationChannel;
+import org.apache.asterix.common.replication.IReplicationManager;
import org.apache.asterix.common.transactions.IAsterixAppRuntimeContextProvider;
import org.apache.asterix.common.transactions.ITransactionSubsystem;
import org.apache.asterix.feeds.FeedManager;
import org.apache.asterix.metadata.bootstrap.MetadataPrimaryIndexes;
+import org.apache.asterix.om.util.AsterixClusterProperties;
+import org.apache.asterix.replication.management.ReplicationChannel;
+import org.apache.asterix.replication.management.ReplicationManager;
+import org.apache.asterix.replication.recovery.RemoteRecoveryManager;
+import org.apache.asterix.replication.storage.ReplicaResourcesManager;
+import org.apache.asterix.transaction.management.resource.PersistentLocalResourceRepository;
import org.apache.asterix.transaction.management.resource.PersistentLocalResourceRepositoryFactory;
import org.apache.asterix.transaction.management.service.transaction.TransactionSubsystem;
import org.apache.hyracks.api.application.INCApplicationContext;
@@ -95,6 +106,7 @@
private AsterixTransactionProperties txnProperties;
private AsterixFeedProperties feedProperties;
private AsterixBuildProperties buildProperties;
+ private AsterixReplicationProperties replicationProperties;
private AsterixThreadExecutor threadExecutor;
private IDatasetLifecycleManager datasetLifecycleManager;
@@ -110,6 +122,11 @@
private IFeedManager feedManager;
+ private IReplicationChannel replicationChannel;
+ private IReplicationManager replicationManager;
+ private IRemoteRecoveryManager remoteRecoveryManager;
+ private IReplicaResourcesManager replicaResourcesManager;
+
public AsterixAppRuntimeContext(INCApplicationContext ncApplicationContext) {
this.ncApplicationContext = ncApplicationContext;
compilerProperties = new AsterixCompilerProperties(ASTERIX_PROPERTIES_ACCESSOR);
@@ -119,6 +136,8 @@
txnProperties = new AsterixTransactionProperties(ASTERIX_PROPERTIES_ACCESSOR);
feedProperties = new AsterixFeedProperties(ASTERIX_PROPERTIES_ACCESSOR);
buildProperties = new AsterixBuildProperties(ASTERIX_PROPERTIES_ACCESSOR);
+ replicationProperties = new AsterixReplicationProperties(ASTERIX_PROPERTIES_ACCESSOR,
+ AsterixClusterProperties.INSTANCE.getCluster());
}
public void initialize() throws IOException, ACIDException, AsterixException {
@@ -131,8 +150,6 @@
ioManager = ncApplicationContext.getRootContext().getIOManager();
IPageReplacementStrategy prs = new ClockPageReplacementStrategy(allocator,
storageProperties.getBufferCachePageSize(), storageProperties.getBufferCacheNumPages());
- bufferCache = new BufferCache(ioManager, prs, pcp, fileMapManager,
- storageProperties.getBufferCacheMaxOpenFiles(), ncApplicationContext.getThreadFactory());
AsynchronousScheduler.INSTANCE.init(ncApplicationContext.getThreadFactory());
lsmIOScheduler = AsynchronousScheduler.INSTANCE;
@@ -142,7 +159,7 @@
ILocalResourceRepositoryFactory persistentLocalResourceRepositoryFactory = new PersistentLocalResourceRepositoryFactory(
ioManager, ncApplicationContext.getNodeId());
localResourceRepository = persistentLocalResourceRepositoryFactory.createRepository();
- resourceIdFactory = (new ResourceIdFactoryProvider(localResourceRepository)).createResourceIdFactory();
+ initializeResourceIdFactory();
IAsterixAppRuntimeContextProvider asterixAppRuntimeContextProvider = new AsterixAppRuntimeContextProdiverForRecovery(
this);
@@ -157,14 +174,61 @@
feedManager = new FeedManager(ncApplicationContext.getNodeId(), feedProperties,
compilerProperties.getFrameSize());
+ if (replicationProperties.isReplicationEnabled()) {
+ String nodeId = ncApplicationContext.getNodeId();
+
+ replicaResourcesManager = new ReplicaResourcesManager(ioManager.getIODevices(),
+ metadataProperties.getStores().get(nodeId)[0], nodeId, replicationProperties.getReplicationStore());
+
+ replicationManager = new ReplicationManager(nodeId, replicationProperties, replicaResourcesManager,
+ txnSubsystem.getLogManager(), asterixAppRuntimeContextProvider);
+
+ //pass replication manager to replication required object
+ //LogManager to replicate logs
+ txnSubsystem.getLogManager().setReplicationManager(replicationManager);
+
+ //PersistentLocalResourceRepository to replicate metadata files and delete backups on drop index
+ ((PersistentLocalResourceRepository) localResourceRepository).setReplicationManager(replicationManager);
+
+ //initialize replication channel
+ replicationChannel = new ReplicationChannel(nodeId, replicationProperties, txnSubsystem.getLogManager(),
+ replicaResourcesManager, replicationManager, ncApplicationContext,
+ asterixAppRuntimeContextProvider);
+
+ remoteRecoveryManager = new RemoteRecoveryManager(replicationManager, this, replicationProperties);
+
+ bufferCache = new BufferCache(ioManager, prs, pcp, fileMapManager,
+ storageProperties.getBufferCacheMaxOpenFiles(), ncApplicationContext.getThreadFactory(),
+ replicationManager);
+
+ } else {
+ bufferCache = new BufferCache(ioManager, prs, pcp, fileMapManager,
+ storageProperties.getBufferCacheMaxOpenFiles(), ncApplicationContext.getThreadFactory());
+ }
+
// The order of registration is important. The buffer cache must registered before recovery and transaction managers.
+ //Notes: registered components are stopped in reversed order
ILifeCycleComponentManager lccm = ncApplicationContext.getLifeCycleComponentManager();
lccm.register((ILifeCycleComponent) bufferCache);
- lccm.register((ILifeCycleComponent) txnSubsystem.getTransactionManager());
+ /**
+ * LogManager must be stopped after RecoveryManager, DatasetLifeCycleManager, and ReplicationManager
+ * to process any logs that might be generated during stopping these components
+ */
lccm.register((ILifeCycleComponent) txnSubsystem.getLogManager());
- lccm.register((ILifeCycleComponent) datasetLifecycleManager);
- lccm.register((ILifeCycleComponent) txnSubsystem.getLockManager());
lccm.register((ILifeCycleComponent) txnSubsystem.getRecoveryManager());
+ /**
+ * ReplicationManager must be stopped after indexLifecycleManager so that any logs/files generated
+ * during closing datasets are sent to remote replicas
+ */
+ if (replicationManager != null) {
+ lccm.register(replicationManager);
+ }
+ /**
+ * Stopping indexLifecycleManager will flush and close all datasets.
+ */
+ lccm.register((ILifeCycleComponent) datasetLifecycleManager);
+ lccm.register((ILifeCycleComponent) txnSubsystem.getTransactionManager());
+ lccm.register((ILifeCycleComponent) txnSubsystem.getLockManager());
}
public boolean isShuttingdown() {
@@ -276,4 +340,34 @@
public IFeedManager getFeedManager() {
return feedManager;
}
-}
+
+ @Override
+ public AsterixReplicationProperties getReplicationProperties() {
+ return replicationProperties;
+ }
+
+ @Override
+ public IReplicationChannel getReplicationChannel() {
+ return replicationChannel;
+ }
+
+ @Override
+ public IReplicaResourcesManager getReplicaResourcesManager() {
+ return replicaResourcesManager;
+ }
+
+ @Override
+ public IRemoteRecoveryManager getRemoteRecoveryManager() {
+ return remoteRecoveryManager;
+ }
+
+ @Override
+ public IReplicationManager getReplicationManager() {
+ return replicationManager;
+ }
+
+ @Override
+ public void initializeResourceIdFactory() throws HyracksDataException {
+ resourceIdFactory = (new ResourceIdFactoryProvider(localResourceRepository)).createResourceIdFactory();
+ }
+}
\ No newline at end of file
diff --git a/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplicationEntryPoint.java b/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplicationEntryPoint.java
index 5aae42a..90697ab 100644
--- a/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplicationEntryPoint.java
+++ b/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplicationEntryPoint.java
@@ -21,9 +21,6 @@
import java.util.logging.Level;
import java.util.logging.Logger;
-import org.eclipse.jetty.server.Server;
-import org.eclipse.jetty.servlet.ServletContextHandler;
-import org.eclipse.jetty.servlet.ServletHolder;
import org.apache.asterix.api.http.servlet.APIServlet;
import org.apache.asterix.api.http.servlet.AQLAPIServlet;
import org.apache.asterix.api.http.servlet.ConnectorAPIServlet;
@@ -35,14 +32,14 @@
import org.apache.asterix.api.http.servlet.ShutdownAPIServlet;
import org.apache.asterix.api.http.servlet.UpdateAPIServlet;
import org.apache.asterix.api.http.servlet.VersionAPIServlet;
-import org.apache.asterix.common.config.AsterixBuildProperties;
import org.apache.asterix.common.api.AsterixThreadFactory;
import org.apache.asterix.common.api.IClusterManagementWork.ClusterState;
+import org.apache.asterix.common.config.AsterixBuildProperties;
import org.apache.asterix.common.config.AsterixExternalProperties;
import org.apache.asterix.common.config.AsterixMetadataProperties;
+import org.apache.asterix.common.config.AsterixReplicationProperties;
import org.apache.asterix.common.feeds.api.ICentralFeedManager;
import org.apache.asterix.event.service.ILookupService;
-import org.apache.asterix.feeds.CentralFeedManager;
import org.apache.asterix.feeds.FeedLifecycleListener;
import org.apache.asterix.metadata.MetadataManager;
import org.apache.asterix.metadata.api.IAsterixStateProxy;
@@ -50,11 +47,15 @@
import org.apache.asterix.metadata.cluster.ClusterManager;
import org.apache.asterix.om.util.AsterixAppContextInfo;
import org.apache.asterix.om.util.AsterixClusterProperties;
+import org.apache.asterix.replication.management.ReplicationLifecycleListener;
import org.apache.hyracks.api.application.ICCApplicationContext;
import org.apache.hyracks.api.application.ICCApplicationEntryPoint;
import org.apache.hyracks.api.client.HyracksConnection;
import org.apache.hyracks.api.client.IHyracksClientConnection;
import org.apache.hyracks.api.lifecycle.LifeCycleComponentManager;
+import org.eclipse.jetty.server.Server;
+import org.eclipse.jetty.servlet.ServletContextHandler;
+import org.eclipse.jetty.servlet.ServletHolder;
public class CCApplicationEntryPoint implements ICCApplicationEntryPoint {
@@ -101,13 +102,18 @@
feedServer.start();
ExternalLibraryBootstrap.setUpExternaLibraries(false);
- centralFeedManager = CentralFeedManager.getInstance();
- centralFeedManager.start();
AsterixGlobalRecoveryManager.INSTANCE = new AsterixGlobalRecoveryManager(
(HyracksConnection) getNewHyracksClientConnection());
ClusterManager.INSTANCE.registerSubscriber(AsterixGlobalRecoveryManager.INSTANCE);
+ AsterixReplicationProperties asterixRepliactionProperties = AsterixAppContextInfo.getInstance()
+ .getReplicationProperties();
+ if (asterixRepliactionProperties.isReplicationEnabled()) {
+ ReplicationLifecycleListener.INSTANCE = new ReplicationLifecycleListener(asterixRepliactionProperties);
+ ClusterManager.INSTANCE.registerSubscriber(ReplicationLifecycleListener.INSTANCE);
+ }
+
ccAppCtx.addClusterLifecycleListener(ClusterLifecycleListener.INSTANCE);
}
diff --git a/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/ClusterLifecycleListener.java b/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/ClusterLifecycleListener.java
index e76d43b..9505692 100644
--- a/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/ClusterLifecycleListener.java
+++ b/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/ClusterLifecycleListener.java
@@ -33,6 +33,7 @@
import org.apache.asterix.common.api.IClusterManagementWorkResponse.Status;
import org.apache.asterix.common.exceptions.AsterixException;
import org.apache.asterix.event.schema.cluster.Node;
+import org.apache.asterix.metadata.MetadataManager;
import org.apache.asterix.metadata.cluster.AddNodeWork;
import org.apache.asterix.metadata.cluster.AddNodeWorkResponse;
import org.apache.asterix.metadata.cluster.ClusterManager;
@@ -95,6 +96,18 @@
LOGGER.info("NC: " + deadNode + " left");
}
AsterixClusterProperties.INSTANCE.removeNCConfiguration(deadNode);
+
+ //if metadata node failed, we need to rebind the proxy connection when it joins again.
+ //Note: the format for the NC should be (INSTANCE-NAME)_(NC-ID)
+ if (AsterixClusterProperties.INSTANCE.getCluster() != null) {
+ String instanceName = AsterixClusterProperties.INSTANCE.getCluster().getInstanceName();
+ String metadataNodeName = AsterixClusterProperties.INSTANCE.getCluster().getMetadataNode();
+ String completeMetadataNodeName = instanceName + "_" + metadataNodeName;
+ if (deadNode.equals(completeMetadataNodeName)) {
+ MetadataManager.INSTANCE.rebindMetadataNode = true;
+ }
+ }
+
}
updateProgress(ClusterEventType.NODE_FAILURE, deadNodeIds);
Set<IClusterEventsSubscriber> subscribers = ClusterManager.INSTANCE.getRegisteredClusterEventSubscribers();
diff --git a/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/NCApplicationEntryPoint.java b/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/NCApplicationEntryPoint.java
index 63f862c..a95d747 100644
--- a/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/NCApplicationEntryPoint.java
+++ b/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/NCApplicationEntryPoint.java
@@ -19,22 +19,22 @@
package org.apache.asterix.hyracks.bootstrap;
import java.io.File;
+import java.io.IOException;
import java.rmi.server.UnicastRemoteObject;
import java.util.HashMap;
import java.util.Map;
import java.util.logging.Level;
import java.util.logging.Logger;
-import org.kohsuke.args4j.CmdLineException;
-import org.kohsuke.args4j.CmdLineParser;
-import org.kohsuke.args4j.Option;
-
import org.apache.asterix.api.common.AsterixAppRuntimeContext;
import org.apache.asterix.common.api.AsterixThreadFactory;
import org.apache.asterix.common.api.IAsterixAppRuntimeContext;
import org.apache.asterix.common.config.AsterixMetadataProperties;
+import org.apache.asterix.common.config.AsterixReplicationProperties;
import org.apache.asterix.common.config.AsterixTransactionProperties;
import org.apache.asterix.common.config.IAsterixPropertiesProvider;
+import org.apache.asterix.common.context.DatasetLifecycleManager;
+import org.apache.asterix.common.replication.IRemoteRecoveryManager;
import org.apache.asterix.common.transactions.IRecoveryManager;
import org.apache.asterix.common.transactions.IRecoveryManager.SystemState;
import org.apache.asterix.event.schema.cluster.Cluster;
@@ -44,12 +44,18 @@
import org.apache.asterix.metadata.api.IAsterixStateProxy;
import org.apache.asterix.metadata.api.IMetadataNode;
import org.apache.asterix.metadata.bootstrap.MetadataBootstrap;
+import org.apache.asterix.metadata.declared.AqlMetadataProvider;
import org.apache.asterix.om.util.AsterixClusterProperties;
+import org.apache.asterix.replication.storage.AsterixFilesUtil;
import org.apache.asterix.transaction.management.resource.PersistentLocalResourceRepository;
+import org.apache.asterix.transaction.management.service.recovery.RecoveryManager;
import org.apache.hyracks.api.application.INCApplicationContext;
import org.apache.hyracks.api.application.INCApplicationEntryPoint;
import org.apache.hyracks.api.lifecycle.ILifeCycleComponentManager;
import org.apache.hyracks.api.lifecycle.LifeCycleComponentManager;
+import org.kohsuke.args4j.CmdLineException;
+import org.kohsuke.args4j.CmdLineParser;
+import org.kohsuke.args4j.Option;
public class NCApplicationEntryPoint implements INCApplicationEntryPoint {
private static final Logger LOGGER = Logger.getLogger(NCApplicationEntryPoint.class.getName());
@@ -66,7 +72,8 @@
private boolean isMetadataNode = false;
private boolean stopInitiated = false;
private SystemState systemState = SystemState.NEW_UNIVERSE;
- private final long NON_SHARP_CHECKPOINT_TARGET_LSN = -1;
+ private boolean performedRemoteRecovery = false;
+ private boolean replicationEnabled = false;
@Override
public void start(INCApplicationContext ncAppCtx, String[] args) throws Exception {
@@ -100,6 +107,12 @@
runtimeContext.initialize();
ncApplicationContext.setApplicationObject(runtimeContext);
+ //if replication is enabled, check if there is a replica for this node
+ AsterixReplicationProperties asterixReplicationProperties = ((IAsterixPropertiesProvider) runtimeContext)
+ .getReplicationProperties();
+
+ replicationEnabled = asterixReplicationProperties.isReplicationEnabled();
+
if (initialRun) {
LOGGER.info("System is being initialized. (first run)");
systemState = SystemState.NEW_UNIVERSE;
@@ -112,11 +125,42 @@
LOGGER.info("System is in a state: " + systemState);
}
+ if (replicationEnabled) {
+ if (systemState == SystemState.NEW_UNIVERSE || systemState == SystemState.CORRUPTED) {
+ //try to perform remote recovery
+ IRemoteRecoveryManager remoteRecoveryMgr = runtimeContext.getRemoteRecoveryManager();
+ remoteRecoveryMgr.performRemoteRecovery();
+ performedRemoteRecovery = true;
+ systemState = SystemState.HEALTHY;
+ }
+ }
+
if (systemState == SystemState.CORRUPTED) {
recoveryMgr.startRecovery(true);
}
}
+ if (replicationEnabled) {
+ startReplicationService();
+ }
+ }
+
+ private void startReplicationService() throws IOException {
+ //open replication channel
+ runtimeContext.getReplicationChannel().start();
+
+ //check the state of remote replicas
+ runtimeContext.getReplicationManager().initializeReplicasState();
+
+ if (performedRemoteRecovery) {
+ //notify remote replicas about the new IP Address if changed
+ //Note: this is a hack since each node right now maintains its own copy of the cluster configuration.
+ //Once the configuration is centralized on the CC, this step wont be needed.
+ runtimeContext.getReplicationManager().broadcastNewIPAddress();
+ }
+
+ //start replication after the state of remote replicas has been initialized.
+ runtimeContext.getReplicationManager().startReplicationThreads();
}
@Override
@@ -128,13 +172,14 @@
LOGGER.info("Stopping Asterix node controller: " + nodeId);
}
- IRecoveryManager recoveryMgr = runtimeContext.getTransactionSubsystem().getRecoveryManager();
- recoveryMgr.checkpoint(true, NON_SHARP_CHECKPOINT_TARGET_LSN);
-
if (isMetadataNode) {
MetadataBootstrap.stopUniverse();
}
+ //clean any temporary files
+ performLocalCleanUp();
+
+ //Note: stopping recovery manager will make a sharp checkpoint
ncApplicationContext.getLifeCycleComponentManager().stopAll(false);
runtimeContext.deinitialize();
} else {
@@ -211,7 +256,7 @@
lccm.startAll();
IRecoveryManager recoveryMgr = runtimeContext.getTransactionSubsystem().getRecoveryManager();
- recoveryMgr.checkpoint(true, NON_SHARP_CHECKPOINT_TARGET_LSN);
+ recoveryMgr.checkpoint(true, RecoveryManager.NON_SHARP_CHECKPOINT_TARGET_LSN);
if (isMetadataNode) {
IMetadataNode stub = null;
@@ -219,20 +264,31 @@
proxy.setMetadataNode(stub);
}
- // Reclaim storage for temporary datasets.
- String[] ioDevices = AsterixClusterProperties.INSTANCE.getIODevices(nodeId);
- String[] nodeStores = metadataProperties.getStores().get(nodeId);
- int numIoDevices = AsterixClusterProperties.INSTANCE.getNumberOfIODevices(nodeId);
- for (int j = 0; j < nodeStores.length; j++) {
- for (int k = 0; k < numIoDevices; k++) {
- File f = new File(ioDevices[k] + File.separator + nodeStores[j] + File.separator + "temp");
- f.delete();
- }
+ //clean any temporary files
+ performLocalCleanUp();
+ }
+
+ private void performLocalCleanUp() throws IOException {
+ //delete working area files from failed jobs
+ runtimeContext.getIOManager().deleteWorkspaceFiles();
+
+ //reclaim storage for temporary datasets.
+ PersistentLocalResourceRepository localResourceRepository = (PersistentLocalResourceRepository) runtimeContext
+ .getLocalResourceRepository();
+
+ String[] storageMountingPoints = localResourceRepository.getStorageMountingPoints();
+ String storageFolderName = ((IAsterixPropertiesProvider) runtimeContext).getMetadataProperties().getStores()
+ .get(nodeId)[0];
+
+ for (String mountPoint : storageMountingPoints) {
+ String tempDatasetFolder = mountPoint + storageFolderName + File.separator
+ + AqlMetadataProvider.TEMP_DATASETS_STORAGE_FOLDER;
+ AsterixFilesUtil.deleteFolder(tempDatasetFolder);
}
// TODO
- // reclaim storage for orphaned index artifacts in NCs.
-
+ //reclaim storage for orphaned index artifacts in NCs.
+ //Note: currently LSM indexes invalid components are deleted when an index is activated.
}
private void updateOnNodeJoin() {
diff --git a/asterix-common/pom.xml b/asterix-common/pom.xml
index 57f9f66..512502b 100644
--- a/asterix-common/pom.xml
+++ b/asterix-common/pom.xml
@@ -25,14 +25,14 @@
</parent>
<artifactId>asterix-common</artifactId>
- <licenses>
- <license>
- <name>Apache License, Version 2.0</name>
- <url>http://www.apache.org/licenses/LICENSE-2.0.txt</url>
- <distribution>repo</distribution>
- <comments>A business-friendly OSS license</comments>
- </license>
- </licenses>
+ <licenses>
+ <license>
+ <name>Apache License, Version 2.0</name>
+ <url>http://www.apache.org/licenses/LICENSE-2.0.txt</url>
+ <distribution>repo</distribution>
+ <comments>A business-friendly OSS license</comments>
+ </license>
+ </licenses>
<build>
<plugins>
diff --git a/asterix-common/src/main/java/org/apache/asterix/common/api/IAsterixAppRuntimeContext.java b/asterix-common/src/main/java/org/apache/asterix/common/api/IAsterixAppRuntimeContext.java
index f86ed8a..63851bf 100644
--- a/asterix-common/src/main/java/org/apache/asterix/common/api/IAsterixAppRuntimeContext.java
+++ b/asterix-common/src/main/java/org/apache/asterix/common/api/IAsterixAppRuntimeContext.java
@@ -25,6 +25,10 @@
import org.apache.asterix.common.exceptions.ACIDException;
import org.apache.asterix.common.exceptions.AsterixException;
import org.apache.asterix.common.feeds.api.IFeedManager;
+import org.apache.asterix.common.replication.IRemoteRecoveryManager;
+import org.apache.asterix.common.replication.IReplicaResourcesManager;
+import org.apache.asterix.common.replication.IReplicationChannel;
+import org.apache.asterix.common.replication.IReplicationManager;
import org.apache.asterix.common.transactions.ITransactionSubsystem;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.api.io.IIOManager;
@@ -76,4 +80,14 @@
public List<IVirtualBufferCache> getVirtualBufferCaches(int datasetID);
public IFeedManager getFeedManager();
+
+ public IRemoteRecoveryManager getRemoteRecoveryManager();
+
+ public IReplicaResourcesManager getReplicaResourcesManager();
+
+ public IReplicationManager getReplicationManager();
+
+ public IReplicationChannel getReplicationChannel();
+
+ public void initializeResourceIdFactory() throws HyracksDataException;
}
diff --git a/asterix-common/src/main/java/org/apache/asterix/common/api/IDatasetLifecycleManager.java b/asterix-common/src/main/java/org/apache/asterix/common/api/IDatasetLifecycleManager.java
index e1e6d96..803e708 100644
--- a/asterix-common/src/main/java/org/apache/asterix/common/api/IDatasetLifecycleManager.java
+++ b/asterix-common/src/main/java/org/apache/asterix/common/api/IDatasetLifecycleManager.java
@@ -21,6 +21,7 @@
import java.util.List;
import org.apache.asterix.common.context.DatasetLifecycleManager.DatasetInfo;
+import org.apache.asterix.common.context.DatasetLifecycleManager.IndexInfo;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.storage.am.common.api.IIndex;
import org.apache.hyracks.storage.am.common.api.IIndexLifecycleManager;
@@ -38,12 +39,14 @@
/**
* Flushes all open datasets synchronously.
+ *
* @throws HyracksDataException
*/
void flushAllDatasets() throws HyracksDataException;
/**
* Schedules asynchronous flush on datasets that have memory components with first LSN < nonSharpCheckpointTargetLSN.
+ *
* @param nonSharpCheckpointTargetLSN
* @throws HyracksDataException
*/
@@ -51,6 +54,7 @@
/**
* creates (if necessary) and returns the dataset info.
+ *
* @param datasetID
* @return
*/
@@ -67,6 +71,7 @@
/**
* creates (if necessary) and returns the primary index operation tracker of a dataset.
+ *
* @param datasetID
* @return
*/
@@ -74,8 +79,19 @@
/**
* creates (if necessary) and returns the dataset virtual buffer caches.
+ *
* @param datasetID
* @return
*/
List<IVirtualBufferCache> getVirtualBufferCaches(int datasetID);
+
+ /**
+ * Flushes then closes all open datasets
+ */
+ void closeAllDatasets() throws HyracksDataException;
+
+ /**
+ * @return a list of all indexes that are open at the time of the call.
+ */
+ List<IndexInfo> getOpenIndexesInfo();
}
diff --git a/asterix-common/src/main/java/org/apache/asterix/common/config/AsterixReplicationProperties.java b/asterix-common/src/main/java/org/apache/asterix/common/config/AsterixReplicationProperties.java
new file mode 100644
index 0000000..1ef7e3e
--- /dev/null
+++ b/asterix-common/src/main/java/org/apache/asterix/common/config/AsterixReplicationProperties.java
@@ -0,0 +1,206 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.common.config;
+
+import java.util.HashSet;
+import java.util.Set;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import org.apache.asterix.common.replication.Replica;
+import org.apache.asterix.event.schema.cluster.Cluster;
+import org.apache.asterix.event.schema.cluster.Node;
+
+public class AsterixReplicationProperties extends AbstractAsterixProperties {
+
+ private static final Logger LOGGER = Logger.getLogger(AsterixReplicationProperties.class.getName());
+
+ private static int REPLICATION_DATAPORT_DEFAULT = 2000;
+ private static int REPLICATION_FACTOR_DEFAULT = 1;
+ private static int REPLICATION_TIME_OUT_DEFAULT = 15;
+
+ private static final String NODE_IP_ADDRESS_DEFAULT = "127.0.0.1";
+ private static final String REPLICATION_STORE_DEFAULT = "asterix-replication";
+ private final String NODE_NAME_PREFIX;
+ private final Cluster cluster;
+
+ public AsterixReplicationProperties(AsterixPropertiesAccessor accessor, Cluster cluster) {
+ super(accessor);
+ this.cluster = cluster;
+
+ if (cluster != null) {
+ NODE_NAME_PREFIX = cluster.getInstanceName() + "_";
+ } else {
+ NODE_NAME_PREFIX = "";
+ }
+ }
+
+ public boolean isReplicationEnabled() {
+ if (cluster != null && cluster.getDataReplication() != null) {
+ if (getReplicationFactor() == 1) {
+ return false;
+ }
+
+ return cluster.getDataReplication().isEnabled();
+
+ } else {
+ return false;
+ }
+ }
+
+ public String getReplicaIPAddress(String nodeId) {
+ if (cluster != null) {
+
+ for (int i = 0; i < cluster.getNode().size(); i++) {
+ Node node = cluster.getNode().get(i);
+ if (getRealCluserNodeID(node.getId()).equals(nodeId)) {
+ return node.getClusterIp();
+ }
+ }
+ }
+ return NODE_IP_ADDRESS_DEFAULT;
+ }
+
+ public int getDataReplicationPort(String nodeId) {
+ if (cluster != null) {
+ return cluster.getDataReplication().getReplicationPort().intValue();
+ }
+
+ return REPLICATION_DATAPORT_DEFAULT;
+ }
+
+ public Set<Replica> getRemoteReplicas(String nodeId) {
+ Set<Replica> remoteReplicas = new HashSet<Replica>();;
+
+ int numberOfRemoteReplicas = getReplicationFactor() - 1;
+
+ //Using chained-declustering
+ if (cluster != null) {
+ int nodeIndex = -1;
+ for (int i = 0; i < cluster.getNode().size(); i++) {
+ Node node = cluster.getNode().get(i);
+ if (getRealCluserNodeID(node.getId()).equals(nodeId)) {
+ nodeIndex = i;
+ break;
+ }
+ }
+
+ if (nodeIndex == -1) {
+ LOGGER.log(Level.WARNING, "Could not find node " + getRealCluserNodeID(nodeId)
+ + " in cluster configurations");
+ return null;
+ }
+
+ for (int i = nodeIndex + 1; i < cluster.getNode().size(); i++) {
+ remoteReplicas.add(getReplicaByNodeIndex(i));
+
+ if (remoteReplicas.size() == numberOfRemoteReplicas) {
+ break;
+ }
+ }
+
+ if (remoteReplicas.size() != numberOfRemoteReplicas) {
+ for (int i = 0; i < cluster.getNode().size(); i++) {
+
+ remoteReplicas.add(getReplicaByNodeIndex(i));
+
+ if (remoteReplicas.size() == numberOfRemoteReplicas) {
+ break;
+ }
+ }
+ }
+ }
+ return remoteReplicas;
+ }
+
+ private Replica getReplicaByNodeIndex(int nodeIndex) {
+ Node node = cluster.getNode().get(nodeIndex);
+ Node replicaNode = new Node();
+ replicaNode.setId(getRealCluserNodeID(node.getId()));
+ replicaNode.setClusterIp(node.getClusterIp());
+ return new Replica(replicaNode);
+ }
+
+ public Replica getReplicaById(String nodeId) {
+ int nodeIndex = -1;
+ if (cluster != null) {
+ for (int i = 0; i < cluster.getNode().size(); i++) {
+ Node node = cluster.getNode().get(i);
+
+ if (getRealCluserNodeID(node.getId()).equals(nodeId)) {
+ nodeIndex = i;
+ break;
+ }
+ }
+ }
+
+ if (nodeIndex < 0) {
+ return null;
+ }
+
+ return getReplicaByNodeIndex(nodeIndex);
+ }
+
+ public Set<String> getRemoteReplicasIds(String nodeId) {
+ Set<String> remoteReplicasIds = new HashSet<String>();
+ Set<Replica> remoteReplicas = getRemoteReplicas(nodeId);
+
+ for (Replica replica : remoteReplicas) {
+ remoteReplicasIds.add(replica.getId());
+ }
+
+ return remoteReplicasIds;
+ }
+
+ public String getRealCluserNodeID(String nodeId) {
+ return NODE_NAME_PREFIX + nodeId;
+ }
+
+ public Set<String> getNodeReplicasIds(String nodeId) {
+ Set<String> replicaIds = new HashSet<String>();
+ replicaIds.add(nodeId);
+ replicaIds.addAll(getRemoteReplicasIds(nodeId));
+ return replicaIds;
+ }
+
+ public String getReplicationStore() {
+ if (cluster != null) {
+ return cluster.getDataReplication().getReplicationStore();
+ }
+ return REPLICATION_STORE_DEFAULT;
+ }
+
+ public int getReplicationFactor() {
+ if (cluster != null) {
+ if (cluster.getDataReplication() == null || cluster.getDataReplication().getReplicationFactor() == null) {
+ return REPLICATION_FACTOR_DEFAULT;
+ }
+ return cluster.getDataReplication().getReplicationFactor().intValue();
+ }
+ return REPLICATION_FACTOR_DEFAULT;
+ }
+
+ public int getReplicationTimeOut() {
+ if (cluster != null) {
+ return cluster.getDataReplication().getReplicationTimeOut().intValue();
+ }
+ return REPLICATION_TIME_OUT_DEFAULT;
+ }
+
+}
\ No newline at end of file
diff --git a/asterix-common/src/main/java/org/apache/asterix/common/config/IAsterixPropertiesProvider.java b/asterix-common/src/main/java/org/apache/asterix/common/config/IAsterixPropertiesProvider.java
index 93c58be..e6f383f 100644
--- a/asterix-common/src/main/java/org/apache/asterix/common/config/IAsterixPropertiesProvider.java
+++ b/asterix-common/src/main/java/org/apache/asterix/common/config/IAsterixPropertiesProvider.java
@@ -28,8 +28,10 @@
public AsterixMetadataProperties getMetadataProperties();
public AsterixExternalProperties getExternalProperties();
-
+
public AsterixFeedProperties getFeedProperties();
AsterixBuildProperties getBuildProperties();
+
+ public AsterixReplicationProperties getReplicationProperties();
}
diff --git a/asterix-common/src/main/java/org/apache/asterix/common/context/BaseOperationTracker.java b/asterix-common/src/main/java/org/apache/asterix/common/context/BaseOperationTracker.java
index 776549a..21500b7 100644
--- a/asterix-common/src/main/java/org/apache/asterix/common/context/BaseOperationTracker.java
+++ b/asterix-common/src/main/java/org/apache/asterix/common/context/BaseOperationTracker.java
@@ -39,7 +39,8 @@
@Override
public void beforeOperation(ILSMIndex index, LSMOperationType opType, ISearchOperationCallback searchCallback,
IModificationOperationCallback modificationCallback) throws HyracksDataException {
- if (opType == LSMOperationType.FLUSH || opType == LSMOperationType.MERGE) {
+ if (opType == LSMOperationType.FLUSH || opType == LSMOperationType.MERGE
+ || opType == LSMOperationType.REPLICATE) {
dsInfo.declareActiveIOOperation();
}
}
@@ -47,7 +48,8 @@
@Override
public void afterOperation(ILSMIndex index, LSMOperationType opType, ISearchOperationCallback searchCallback,
IModificationOperationCallback modificationCallback) throws HyracksDataException {
- if (opType == LSMOperationType.FLUSH || opType == LSMOperationType.MERGE) {
+ if (opType == LSMOperationType.FLUSH || opType == LSMOperationType.MERGE
+ || opType == LSMOperationType.REPLICATE) {
dsInfo.undeclareActiveIOOperation();
}
}
diff --git a/asterix-common/src/main/java/org/apache/asterix/common/context/DatasetLifecycleManager.java b/asterix-common/src/main/java/org/apache/asterix/common/context/DatasetLifecycleManager.java
index 20b07fa..adf1152 100644
--- a/asterix-common/src/main/java/org/apache/asterix/common/context/DatasetLifecycleManager.java
+++ b/asterix-common/src/main/java/org/apache/asterix/common/context/DatasetLifecycleManager.java
@@ -74,6 +74,7 @@
capacity = storageProperties.getMemoryComponentGlobalBudget();
used = 0;
logRecord = new LogRecord();
+ logRecord.setNodeId(logManager.getNodeId());
}
@Override
@@ -112,10 +113,10 @@
if (dsInfo.indexes.containsKey(resourceID)) {
throw new HyracksDataException("Index with resource ID " + resourceID + " already exists.");
}
- dsInfo.indexes.put(resourceID, new IndexInfo((ILSMIndex) index));
+ dsInfo.indexes.put(resourceID, new IndexInfo((ILSMIndex) index, dsInfo.datasetID, resourceID));
}
- private int getDIDfromResourceName(String resourceName) throws HyracksDataException {
+ public int getDIDfromResourceName(String resourceName) throws HyracksDataException {
LocalResource lr = resourceRepository.getResourceByName(resourceName);
if (lr == null) {
return -1;
@@ -123,7 +124,7 @@
return ((ILocalResourceMetadata) lr.getResourceObject()).getDatasetID();
}
- private long getResourceIDfromResourceName(String resourceName) throws HyracksDataException {
+ public long getResourceIDfromResourceName(String resourceName) throws HyracksDataException {
LocalResource lr = resourceRepository.getResourceByName(resourceName);
if (lr == null) {
return -1;
@@ -279,15 +280,25 @@
@Override
public synchronized List<IIndex> getOpenIndexes() {
+ List<IndexInfo> openIndexesInfo = getOpenIndexesInfo();
List<IIndex> openIndexes = new ArrayList<IIndex>();
+ for (IndexInfo iInfo : openIndexesInfo) {
+ openIndexes.add(iInfo.index);
+ }
+ return openIndexes;
+ }
+
+ @Override
+ public synchronized List<IndexInfo> getOpenIndexesInfo() {
+ List<IndexInfo> openIndexesInfo = new ArrayList<IndexInfo>();
for (DatasetInfo dsInfo : datasetInfos.values()) {
for (IndexInfo iInfo : dsInfo.indexes.values()) {
if (iInfo.isOpen) {
- openIndexes.add(iInfo.index);
+ openIndexesInfo.add(iInfo);
}
}
}
- return openIndexes;
+ return openIndexesInfo;
}
public List<IVirtualBufferCache> getVirtualBufferCaches(int datasetID) {
@@ -358,11 +369,27 @@
}
}
- private class IndexInfo extends Info {
+ public class IndexInfo extends Info {
private final ILSMIndex index;
+ private final long resourceId;
+ private final int datasetId;
- public IndexInfo(ILSMIndex index) {
+ public IndexInfo(ILSMIndex index, int datasetId, long resourceId) {
this.index = index;
+ this.datasetId = datasetId;
+ this.resourceId = resourceId;
+ }
+
+ public ILSMIndex getIndex() {
+ return index;
+ }
+
+ public long getResourceId() {
+ return resourceId;
+ }
+
+ public int getDatasetId() {
+ return datasetId;
}
}
@@ -456,14 +483,6 @@
+ ", lastAccess: " + lastAccess + ", isRegistered: " + isRegistered + ", memoryAllocated: "
+ memoryAllocated;
}
-
- public boolean isMemoryAllocated() {
- return memoryAllocated;
- }
-
- public int getDatasetID() {
- return datasetID;
- }
}
@Override
@@ -520,7 +539,7 @@
private void flushDatasetOpenIndexes(DatasetInfo dsInfo, boolean asyncFlush) throws HyracksDataException {
if (!dsInfo.isExternal) {
synchronized (logRecord) {
- logRecord.formFlushLogRecord(dsInfo.datasetID, null);
+ logRecord.formFlushLogRecord(dsInfo.datasetID, null, dsInfo.indexes.size());
try {
logManager.log(logRecord);
} catch (ACIDException e) {
@@ -588,15 +607,19 @@
removeDatasetFromCache(dsInfo.datasetID);
}
+ public void closeAllDatasets() throws HyracksDataException {
+ for (DatasetInfo dsInfo : datasetInfos.values()) {
+ closeDataset(dsInfo);
+ }
+ }
+
@Override
public synchronized void stop(boolean dumpState, OutputStream outputStream) throws IOException {
if (dumpState) {
dumpState(outputStream);
}
- for (DatasetInfo dsInfo : datasetInfos.values()) {
- closeDataset(dsInfo);
- }
+ closeAllDatasets();
datasetVirtualBufferCaches.clear();
datasetOpTrackers.clear();
@@ -686,4 +709,4 @@
int did = Integer.parseInt(resourceName);
allocateDatasetMemory(did);
}
-}
\ No newline at end of file
+}
diff --git a/asterix-common/src/main/java/org/apache/asterix/common/context/PrimaryIndexOperationTracker.java b/asterix-common/src/main/java/org/apache/asterix/common/context/PrimaryIndexOperationTracker.java
index b4a3ac9..437fac4 100644
--- a/asterix-common/src/main/java/org/apache/asterix/common/context/PrimaryIndexOperationTracker.java
+++ b/asterix-common/src/main/java/org/apache/asterix/common/context/PrimaryIndexOperationTracker.java
@@ -59,7 +59,8 @@
IModificationOperationCallback modificationCallback) throws HyracksDataException {
if (opType == LSMOperationType.MODIFICATION || opType == LSMOperationType.FORCE_MODIFICATION) {
incrementNumActiveOperations(modificationCallback);
- } else if (opType == LSMOperationType.FLUSH || opType == LSMOperationType.MERGE) {
+ } else if (opType == LSMOperationType.FLUSH || opType == LSMOperationType.MERGE
+ || opType == LSMOperationType.REPLICATE) {
dsInfo.declareActiveIOOperation();
}
}
@@ -68,7 +69,8 @@
public void afterOperation(ILSMIndex index, LSMOperationType opType, ISearchOperationCallback searchCallback,
IModificationOperationCallback modificationCallback) throws HyracksDataException {
// Searches are immediately considered complete, because they should not prevent the execution of flushes.
- if (opType == LSMOperationType.FLUSH || opType == LSMOperationType.MERGE) {
+ if (opType == LSMOperationType.FLUSH || opType == LSMOperationType.MERGE
+ || opType == LSMOperationType.REPLICATE) {
completeOperation(index, opType, searchCallback, modificationCallback);
}
}
@@ -84,7 +86,8 @@
} else if (numActiveOperations.get() < 0) {
throw new HyracksDataException("The number of active operations cannot be negative!");
}
- } else if (opType == LSMOperationType.FLUSH || opType == LSMOperationType.MERGE) {
+ } else if (opType == LSMOperationType.FLUSH || opType == LSMOperationType.MERGE
+ || opType == LSMOperationType.REPLICATE) {
dsInfo.undeclareActiveIOOperation();
}
}
@@ -119,7 +122,7 @@
}
LogRecord logRecord = new LogRecord();
- logRecord.formFlushLogRecord(datasetID, this);
+ logRecord.formFlushLogRecord(datasetID, this, logManager.getNodeId(), dsInfo.getDatasetIndexes().size());
try {
logManager.log(logRecord);
diff --git a/asterix-common/src/main/java/org/apache/asterix/common/dataflow/AsterixLSMIndexUtil.java b/asterix-common/src/main/java/org/apache/asterix/common/dataflow/AsterixLSMIndexUtil.java
index 2cd0554..4259a10 100644
--- a/asterix-common/src/main/java/org/apache/asterix/common/dataflow/AsterixLSMIndexUtil.java
+++ b/asterix-common/src/main/java/org/apache/asterix/common/dataflow/AsterixLSMIndexUtil.java
@@ -41,4 +41,10 @@
}
}
}
+
+ public static boolean lsmComponentFileHasLSN(AbstractLSMIndex lsmIndex, String componentFilePath) {
+ AbstractLSMIOOperationCallback ioOpCallback = (AbstractLSMIOOperationCallback) lsmIndex
+ .getIOOperationCallback();
+ return ioOpCallback.componentFileHasLSN(componentFilePath);
+ }
}
diff --git a/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/AbstractLSMIOOperationCallback.java b/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/AbstractLSMIOOperationCallback.java
index abf7ba9..76a11d1 100644
--- a/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/AbstractLSMIOOperationCallback.java
+++ b/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/AbstractLSMIOOperationCallback.java
@@ -151,4 +151,7 @@
}
return false;
}
+
+ public abstract boolean componentFileHasLSN(String componentFilePath);
+
}
diff --git a/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMBTreeIOOperationCallback.java b/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMBTreeIOOperationCallback.java
index 12680fd..8b4fa01 100644
--- a/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMBTreeIOOperationCallback.java
+++ b/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMBTreeIOOperationCallback.java
@@ -24,6 +24,7 @@
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.storage.am.btree.impls.BTree;
import org.apache.hyracks.storage.am.lsm.btree.impls.LSMBTreeDiskComponent;
+import org.apache.hyracks.storage.am.lsm.btree.impls.LSMBTreeFileManager;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent;
import org.apache.hyracks.storage.am.lsm.common.api.LSMOperationType;
@@ -60,4 +61,13 @@
}
return maxLSN;
}
+
+ @Override
+ public boolean componentFileHasLSN(String componentFilePath) {
+ if (componentFilePath.endsWith(LSMBTreeFileManager.BTREE_STRING)) {
+ return true;
+ }
+
+ return false;
+ }
}
diff --git a/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMBTreeWithBuddyIOOperationCallback.java b/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMBTreeWithBuddyIOOperationCallback.java
index b5ce879..229ccd6 100644
--- a/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMBTreeWithBuddyIOOperationCallback.java
+++ b/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMBTreeWithBuddyIOOperationCallback.java
@@ -22,6 +22,7 @@
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.storage.am.lsm.btree.impls.LSMBTreeWithBuddyDiskComponent;
+import org.apache.hyracks.storage.am.lsm.btree.impls.LSMBTreeWithBuddyFileManager;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent;
import org.apache.hyracks.storage.am.lsm.common.api.LSMOperationType;
@@ -55,4 +56,14 @@
return maxLSN;
}
+ @Override
+ public boolean componentFileHasLSN(String componentFilePath) {
+ if (componentFilePath.endsWith(LSMBTreeWithBuddyFileManager.BTREE_STRING)
+ || componentFilePath.endsWith(LSMBTreeWithBuddyFileManager.BUDDY_BTREE_STRING)) {
+ return true;
+ }
+
+ return false;
+ }
+
}
diff --git a/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMInvertedIndexIOOperationCallback.java b/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMInvertedIndexIOOperationCallback.java
index fd3cf12..3e4ff04 100644
--- a/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMInvertedIndexIOOperationCallback.java
+++ b/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMInvertedIndexIOOperationCallback.java
@@ -25,6 +25,7 @@
import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent;
import org.apache.hyracks.storage.am.lsm.common.api.LSMOperationType;
import org.apache.hyracks.storage.am.lsm.invertedindex.impls.LSMInvertedIndexDiskComponent;
+import org.apache.hyracks.storage.am.lsm.invertedindex.impls.LSMInvertedIndexFileManager;
public class LSMInvertedIndexIOOperationCallback extends AbstractLSMIOOperationCallback {
@@ -58,4 +59,13 @@
}
return maxLSN;
}
+
+ @Override
+ public boolean componentFileHasLSN(String componentFilePath) {
+ if (componentFilePath.endsWith(LSMInvertedIndexFileManager.DELETED_KEYS_BTREE_SUFFIX)) {
+ return true;
+ }
+
+ return false;
+ }
}
diff --git a/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMRTreeIOOperationCallback.java b/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMRTreeIOOperationCallback.java
index 5d243a3..7c483f3 100644
--- a/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMRTreeIOOperationCallback.java
+++ b/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMRTreeIOOperationCallback.java
@@ -25,6 +25,7 @@
import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent;
import org.apache.hyracks.storage.am.lsm.common.api.LSMOperationType;
import org.apache.hyracks.storage.am.lsm.rtree.impls.LSMRTreeDiskComponent;
+import org.apache.hyracks.storage.am.lsm.rtree.impls.LSMRTreeFileManager;
public class LSMRTreeIOOperationCallback extends AbstractLSMIOOperationCallback {
@@ -59,4 +60,14 @@
}
return maxLSN;
}
+
+ @Override
+ public boolean componentFileHasLSN(String componentFilePath) {
+ if (componentFilePath.endsWith(LSMRTreeFileManager.RTREE_STRING)
+ || componentFilePath.endsWith(LSMRTreeFileManager.BTREE_STRING)) {
+ return true;
+ }
+
+ return false;
+ }
}
diff --git a/asterix-common/src/main/java/org/apache/asterix/common/replication/AsterixReplicationJob.java b/asterix-common/src/main/java/org/apache/asterix/common/replication/AsterixReplicationJob.java
new file mode 100644
index 0000000..fc4f1ab
--- /dev/null
+++ b/asterix-common/src/main/java/org/apache/asterix/common/replication/AsterixReplicationJob.java
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.common.replication;
+
+import java.util.Set;
+
+import org.apache.hyracks.api.replication.impl.AbstractReplicationJob;
+
+/**
+ * LSMIndexReplicationJob is used for LSM Components only in Hyracks level.
+ * AsterixReplicationJob is used for everything else.
+ * Currently it is used to transfer indexes metadata files.
+ */
+public class AsterixReplicationJob extends AbstractReplicationJob {
+
+ public AsterixReplicationJob(ReplicationJobType jobType, ReplicationOperation operation,
+ ReplicationExecutionType executionType, Set<String> filesToReplicate) {
+ super(jobType, operation, executionType, filesToReplicate);
+ }
+
+}
\ No newline at end of file
diff --git a/asterix-common/src/main/java/org/apache/asterix/common/replication/IRemoteRecoveryManager.java b/asterix-common/src/main/java/org/apache/asterix/common/replication/IRemoteRecoveryManager.java
new file mode 100644
index 0000000..63d29a0
--- /dev/null
+++ b/asterix-common/src/main/java/org/apache/asterix/common/replication/IRemoteRecoveryManager.java
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.common.replication;
+
+public interface IRemoteRecoveryManager {
+
+ public void performRemoteRecovery();
+
+}
\ No newline at end of file
diff --git a/asterix-common/src/main/java/org/apache/asterix/common/replication/IReplicaResourcesManager.java b/asterix-common/src/main/java/org/apache/asterix/common/replication/IReplicaResourcesManager.java
new file mode 100644
index 0000000..f9481a0
--- /dev/null
+++ b/asterix-common/src/main/java/org/apache/asterix/common/replication/IReplicaResourcesManager.java
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.common.replication;
+
+import java.io.IOException;
+import java.util.Set;
+
+public interface IReplicaResourcesManager {
+
+ public String getIndexPath(String backupNodeId, int IODeviceNum, String dataverse, String dataset);
+
+ public String getLocalStorageFolder();
+
+ public long getMinRemoteLSN(Set<String> remoteNodes);
+
+ public void deleteAsterixStorageData() throws IOException;
+
+}
\ No newline at end of file
diff --git a/asterix-common/src/main/java/org/apache/asterix/common/replication/IReplicationChannel.java b/asterix-common/src/main/java/org/apache/asterix/common/replication/IReplicationChannel.java
new file mode 100644
index 0000000..56ae20f
--- /dev/null
+++ b/asterix-common/src/main/java/org/apache/asterix/common/replication/IReplicationChannel.java
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.common.replication;
+
+import java.io.IOException;
+
+public interface IReplicationChannel {
+
+ /**
+ * Opens the replication channel and starts accepting replication requests.
+ */
+ public void start();
+
+ /**
+ * Closes the replication channel.
+ *
+ * @throws IOException
+ */
+ public void close() throws IOException;
+
+}
\ No newline at end of file
diff --git a/asterix-common/src/main/java/org/apache/asterix/common/replication/IReplicationManager.java b/asterix-common/src/main/java/org/apache/asterix/common/replication/IReplicationManager.java
new file mode 100644
index 0000000..276d498
--- /dev/null
+++ b/asterix-common/src/main/java/org/apache/asterix/common/replication/IReplicationManager.java
@@ -0,0 +1,150 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.common.replication;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Set;
+
+import org.apache.asterix.common.exceptions.ACIDException;
+import org.apache.asterix.common.transactions.ILogRecord;
+import org.apache.hyracks.api.replication.IIOReplicationManager;
+
+public interface IReplicationManager extends IIOReplicationManager {
+
+ /**
+ * Asynchronously sends a serialized version of the record to remote replicas.
+ *
+ * @param logRecord
+ * The log record to be replicated,
+ */
+ public void replicateLog(ILogRecord logRecord);
+
+ /**
+ * Checks whether a log record has been replicated
+ *
+ * @param logRecord
+ * the log to check for.
+ * @return true, if all ACKs were received from remote replicas.
+ */
+ public boolean hasBeenReplicated(ILogRecord logRecord);
+
+ /**
+ * Requests txns logs from a remote replica.
+ *
+ * @param remoteReplicaId
+ * The replica id to send the request to.
+ * @param replicasDataToRecover
+ * Get logs that belong to those replicas.
+ * @param fromLSN
+ * Low water mark for logs to be requested.
+ * @return The logs received that belong to the local node.
+ * @throws IOException
+ * @throws ACIDException
+ */
+ public ArrayList<ILogRecord> requestReplicaLogs(String remoteReplicaId, Set<String> replicasDataToRecover,
+ long fromLSN) throws IOException, ACIDException;
+
+ /**
+ * Requests LSM components files from a remote replica.
+ *
+ * @param remoteReplicaId
+ * The replica id to send the request to.
+ * @param replicasDataToRecover
+ * Get files that belong to those replicas.
+ * @throws IOException
+ */
+ public void requestReplicaFiles(String remoteReplicaId, Set<String> replicasDataToRecover) throws IOException;
+
+ /**
+ * Requests current maximum LSN from remote replicas.
+ *
+ * @param remoteReplicaIds
+ * remote replicas to send the request to.
+ * @return The maximum of the received maximum LSNs.
+ * @throws IOException
+ */
+ public long getMaxRemoteLSN(Set<String> remoteReplicaIds) throws IOException;
+
+ /**
+ * Sends the IP address of the local replica to all remote replicas.
+ *
+ * @throws IOException
+ */
+ public void broadcastNewIPAddress() throws IOException;
+
+ /**
+ * @return The number of remote replicas that are in ACTIVE state.
+ */
+ public int getActiveReplicasCount();
+
+ /**
+ * @return The IDs of the remote replicas that are in DEAD state.
+ */
+ public Set<String> getDeadReplicasIds();
+
+ /**
+ * Starts processing of ASYNC replication jobs as well as Txn logs.
+ */
+ public void startReplicationThreads();
+
+ /**
+ * Checks and sets each remote replica state.
+ */
+ public void initializeReplicasState();
+
+ /**
+ * Updates remote replica (in-memory) information.
+ *
+ * @param replica
+ * the replica to update.
+ */
+ public void updateReplicaInfo(Replica replica);
+
+ /**
+ * @return The IDs of the remote replicas that are in ACTIVE state.
+ */
+ public Set<String> getActiveReplicasIds();
+
+ /**
+ * Submits a ReplicaEvent to ReplicationEventsMonitor thread.
+ *
+ * @param event
+ */
+ public void reportReplicaEvent(ReplicaEvent event);
+
+ /**
+ * Requests the current minimum LSN of a remote replica.
+ *
+ * @param replicaId
+ * The replica to send the request to.
+ * @return The returned minimum LSN from the remote replica.
+ * @throws IOException
+ */
+ public long requestReplicaMinLSN(String replicaId) throws IOException;
+
+ /**
+ * Sends a request to remote replicas to flush indexes that have LSN less than nonSharpCheckpointTargetLSN
+ *
+ * @param nonSharpCheckpointTargetLSN
+ * @throws IOException
+ */
+ public void requestFlushLaggingReplicaIndexes(long nonSharpCheckpointTargetLSN) throws IOException;
+
+}
diff --git a/asterix-common/src/main/java/org/apache/asterix/common/replication/IReplicationThread.java b/asterix-common/src/main/java/org/apache/asterix/common/replication/IReplicationThread.java
new file mode 100644
index 0000000..3e2569d
--- /dev/null
+++ b/asterix-common/src/main/java/org/apache/asterix/common/replication/IReplicationThread.java
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.common.replication;
+
+import org.apache.asterix.common.transactions.LogRecord;
+
+public interface IReplicationThread extends Runnable {
+
+ /**
+ * Sends a notification to this thread that logRecord has been flushed.
+ *
+ * @param logRecord
+ * The log that has been flushed.
+ */
+ public void notifyLogReplicationRequester(LogRecord logRecord);
+}
\ No newline at end of file
diff --git a/asterix-common/src/main/java/org/apache/asterix/common/replication/Replica.java b/asterix-common/src/main/java/org/apache/asterix/common/replication/Replica.java
new file mode 100644
index 0000000..4c3f728
--- /dev/null
+++ b/asterix-common/src/main/java/org/apache/asterix/common/replication/Replica.java
@@ -0,0 +1,92 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.common.replication;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.net.InetSocketAddress;
+
+import org.apache.asterix.common.config.AsterixReplicationProperties;
+import org.apache.asterix.event.schema.cluster.Node;
+
+public class Replica {
+
+ public enum ReplicaState {
+ ACTIVE,
+ DEAD,
+ UNKNOWN
+ }
+
+ final Node node;
+ private ReplicaState state = ReplicaState.UNKNOWN;
+
+ public Replica(Node node) {
+ this.node = node;
+ }
+
+ public ReplicaState getState() {
+ return state;
+ }
+
+ public void setState(ReplicaState state) {
+ this.state = state;
+ }
+
+ public Node getNode() {
+ return node;
+ }
+
+ public String getId() {
+ return node.getId();
+ }
+
+ public InetSocketAddress getAddress(AsterixReplicationProperties asterixReplicationProperties) {
+ String replicaIPAddress = node.getClusterIp();
+ int replicationPort = asterixReplicationProperties.getDataReplicationPort(node.getId());
+ InetSocketAddress replicaAddress = InetSocketAddress.createUnresolved(replicaIPAddress, replicationPort);
+ return replicaAddress;
+ }
+
+ public static Replica create(DataInput input) throws IOException {
+ Node node = new Node();
+ Replica replica = new Replica(node);
+ replica.readFields(input);
+ return replica;
+ }
+
+ public void writeFields(DataOutput output) throws IOException {
+ output.writeUTF(node.getId());
+ output.writeUTF(node.getClusterIp());
+ output.writeInt(state.ordinal());
+ }
+
+ public void readFields(DataInput input) throws IOException {
+ this.node.setId(input.readUTF());
+ this.node.setClusterIp(input.readUTF());
+ this.state = ReplicaState.values()[input.readInt()];
+ }
+
+ public void serialize(OutputStream out) throws IOException {
+ DataOutputStream dos = new DataOutputStream(out);
+ writeFields(dos);
+ }
+}
diff --git a/asterix-common/src/main/java/org/apache/asterix/common/replication/ReplicaEvent.java b/asterix-common/src/main/java/org/apache/asterix/common/replication/ReplicaEvent.java
new file mode 100644
index 0000000..0797a02
--- /dev/null
+++ b/asterix-common/src/main/java/org/apache/asterix/common/replication/ReplicaEvent.java
@@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.common.replication;
+
+import java.io.DataInput;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+
+public class ReplicaEvent {
+
+ /*
+ * FAIL: remote replica failed.
+ * JOIN: remote replica rejoined the cluster.
+ * SHUTDOWN: remote replica is shutting down normally
+ * */
+ public enum ReplicaEventType {
+ FAIL,
+ JOIN,
+ SHUTDOWN
+ }
+
+ Replica replica;
+ ReplicaEventType eventType;
+
+ public ReplicaEvent(Replica replica, ReplicaEventType eventType) {
+ this.replica = replica;
+ this.eventType = eventType;
+ }
+
+ public Replica getReplica() {
+ return replica;
+ }
+
+ public void setReplica(Replica replica) {
+ this.replica = replica;
+ }
+
+ public ReplicaEventType getEventType() {
+ return eventType;
+ }
+
+ public void setEventType(ReplicaEventType eventType) {
+ this.eventType = eventType;
+ }
+
+ public void serialize(OutputStream out) throws IOException {
+ DataOutputStream dos = new DataOutputStream(out);
+ replica.writeFields(dos);
+ dos.writeInt(eventType.ordinal());
+ }
+
+ public static ReplicaEvent create(DataInput input) throws IOException {
+ Replica replica = Replica.create(input);
+ ReplicaEventType eventType = ReplicaEventType.values()[input.readInt()];
+ ReplicaEvent event = new ReplicaEvent(replica, eventType);
+ return event;
+ }
+
+}
diff --git a/asterix-common/src/main/java/org/apache/asterix/common/transactions/ILogBuffer.java b/asterix-common/src/main/java/org/apache/asterix/common/transactions/ILogBuffer.java
index 9e28cda..02d69e5 100644
--- a/asterix-common/src/main/java/org/apache/asterix/common/transactions/ILogBuffer.java
+++ b/asterix-common/src/main/java/org/apache/asterix/common/transactions/ILogBuffer.java
@@ -24,4 +24,6 @@
public void flush();
+ public void appendWithReplication(ILogRecord logRecord, long appendLSN);
+
}
\ No newline at end of file
diff --git a/asterix-common/src/main/java/org/apache/asterix/common/transactions/ILogManager.java b/asterix-common/src/main/java/org/apache/asterix/common/transactions/ILogManager.java
index fc0b407..82a0cca 100644
--- a/asterix-common/src/main/java/org/apache/asterix/common/transactions/ILogManager.java
+++ b/asterix-common/src/main/java/org/apache/asterix/common/transactions/ILogManager.java
@@ -18,40 +18,71 @@
*/
package org.apache.asterix.common.transactions;
+import java.io.IOException;
+
import org.apache.asterix.common.exceptions.ACIDException;
+import org.apache.asterix.common.replication.IReplicationManager;
public interface ILogManager {
/**
* Submits a logRecord to log Manager which appends it to the log tail
+ *
* @param logRecord
* @throws ACIDException
*/
public void log(ILogRecord logRecord) throws ACIDException;
/**
- *
* @param isRecoveryMode
* @returnLogReader instance which enables reading the log files
*/
public ILogReader getLogReader(boolean isRecoveryMode);
-
+
/**
- *
* @return the last LSN the log manager used
*/
- public long getAppendLSN();
-
+ public long getAppendLSN();
+
/**
* Deletes all log partitions which have a maximum LSN less than checkpointLSN
+ *
* @param checkpointLSN
*/
public void deleteOldLogFiles(long checkpointLSN);
-
+
/**
- *
* @return the smallest readable LSN on the current log partitions
*/
public long getReadableSmallestLSN();
+ /**
+ * @return The local NC ID
+ */
+ public String getNodeId();
+
+ /**
+ * Delete all log files and start new log partition > LSNtoStartFrom
+ *
+ * @param LSNtoStartFrom
+ * @throws IOException
+ */
+ public void renewLogFilesAndStartFromLSN(long LSNtoStartFrom) throws IOException;
+
+ /**
+ * @return the log page size in bytes
+ */
+ public int getLogPageSize();
+
+ /**
+ * @param replicationManager
+ * the replication manager to be used to replicate logs
+ */
+ public void setReplicationManager(IReplicationManager replicationManager);
+
+ /**
+ * @return the number of log pages
+ */
+ public int getNumLogPages();
+
}
diff --git a/asterix-common/src/main/java/org/apache/asterix/common/transactions/ILogRecord.java b/asterix-common/src/main/java/org/apache/asterix/common/transactions/ILogRecord.java
index 16c51fe..9694949 100644
--- a/asterix-common/src/main/java/org/apache/asterix/common/transactions/ILogRecord.java
+++ b/asterix-common/src/main/java/org/apache/asterix/common/transactions/ILogRecord.java
@@ -20,22 +20,22 @@
import java.nio.ByteBuffer;
+import org.apache.asterix.common.replication.IReplicationThread;
import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
public interface ILogRecord {
- public static final int JOB_TERMINATE_LOG_SIZE = 13; //JOB_COMMIT or ABORT log type
- public static final int ENTITY_COMMIT_LOG_BASE_SIZE = 25;
- public static final int UPDATE_LOG_BASE_SIZE = 54;
- public static final int FLUSH_LOG_SIZE = 17;
-
-
public enum RECORD_STATUS{
TRUNCATED,
BAD_CHKSUM,
OK
}
+ public static final int JOB_TERMINATE_LOG_SIZE = 18; //JOB_COMMIT or ABORT log type
+ public static final int ENTITY_COMMIT_LOG_BASE_SIZE = 30;
+ public static final int UPDATE_LOG_BASE_SIZE = 59;
+ public static final int FLUSH_LOG_SIZE = 22;
+
public LogRecord.RECORD_STATUS readLogRecord(ByteBuffer buffer);
public void writeLogRecord(ByteBuffer buffer);
@@ -115,4 +115,26 @@
public void setPKValue(ITupleReference PKValue);
+ public String getNodeId();
+
+ public void setNodeId(String nodeId);
+
+ public int serialize(ByteBuffer buffer);
+
+ public void deserialize(ByteBuffer buffer, boolean remoteRecoveryLog, String localNodeId);
+
+ public void setReplicationThread(IReplicationThread replicationThread);
+
+ public void setLogSource(byte logSource);
+
+ public byte getLogSource();
+
+ public int getSerializedLogSize();
+
+ public void writeLogRecord(ByteBuffer buffer, long appendLSN);
+
+ public ByteBuffer getSerializedLog();
+
+ public void formJobTerminateLogRecord(int jobId, boolean isCommit, String nodeId);
+
}
diff --git a/asterix-common/src/main/java/org/apache/asterix/common/transactions/IRecoveryManager.java b/asterix-common/src/main/java/org/apache/asterix/common/transactions/IRecoveryManager.java
index e8d408b..9ea9957 100644
--- a/asterix-common/src/main/java/org/apache/asterix/common/transactions/IRecoveryManager.java
+++ b/asterix-common/src/main/java/org/apache/asterix/common/transactions/IRecoveryManager.java
@@ -19,6 +19,7 @@
package org.apache.asterix.common.transactions;
import java.io.IOException;
+import java.util.ArrayList;
import org.apache.asterix.common.exceptions.ACIDException;
import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -89,5 +90,25 @@
*/
public long checkpoint(boolean isSharpCheckpoint, long nonSharpCheckpointTargetLSN) throws ACIDException, HyracksDataException;
+ /**
+ * Performs recovery based on the passed logs
+ * @param remoteLogs the remote logs to be replayed
+ * @throws HyracksDataException
+ * @throws ACIDException
+ */
+ public void replayRemoteLogs(ArrayList<ILogRecord> remoteLogs) throws HyracksDataException, ACIDException;
+
+ /**
+ *
+ * @return min first LSN of the open indexes (including remote indexes if replication is enabled)
+ * @throws HyracksDataException
+ */
+ public long getMinFirstLSN() throws HyracksDataException;
+ /**
+ *
+ * @return min first LSN of the open indexes
+ * @throws HyracksDataException
+ */
+ public long getLocalMinFirstLSN() throws HyracksDataException;
}
diff --git a/asterix-common/src/main/java/org/apache/asterix/common/transactions/LogRecord.java b/asterix-common/src/main/java/org/apache/asterix/common/transactions/LogRecord.java
index a510e51..bbbe59e 100644
--- a/asterix-common/src/main/java/org/apache/asterix/common/transactions/LogRecord.java
+++ b/asterix-common/src/main/java/org/apache/asterix/common/transactions/LogRecord.java
@@ -20,10 +20,13 @@
import java.nio.BufferUnderflowException;
import java.nio.ByteBuffer;
+import java.util.HashMap;
+import java.util.Map;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.zip.CRC32;
import org.apache.asterix.common.context.PrimaryIndexOperationTracker;
+import org.apache.asterix.common.replication.IReplicationThread;
import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
import org.apache.hyracks.storage.am.common.tuples.SimpleTupleReference;
import org.apache.hyracks.storage.am.common.tuples.SimpleTupleWriter;
@@ -31,9 +34,12 @@
/*
* == LogRecordFormat ==
* ---------------------------
- * [Header1] (5 bytes) : for all log types
+ * [Header1] (10 bytes + NodeId Length) : for all log types
+ * LogSource(1)
* LogType(1)
* JobId(4)
+ * NodeIdLength(4)
+ * NodeId(?)
* ---------------------------
* [Header2] (12 bytes + PKValueSize) : for entity_commit and update log types
* DatasetId(4) //stored in dataset_dataset in Metadata Node
@@ -60,12 +66,15 @@
* 2) ENTITY_COMMIT: 25 + PKSize (5 + 12 + PKSize + 8)
* --> ENTITY_COMMIT_LOG_BASE_SIZE = 25
* 3) UPDATE: 54 + PKValueSize + NewValueSize (5 + 12 + PKValueSize + 20 + 9 + NewValueSize + 8)
- * 4) FLUSH: 5 + 8 + DatasetId(4)
- * --> UPDATE_LOG_BASE_SIZE = 54
+ * 4) FLUSH: 5 + 8 + DatasetId(4) (In case of serialize: + (8 bytes for LSN) + (4 bytes for number of flushed indexes)
*/
+
public class LogRecord implements ILogRecord {
//------------- fields in a log record (begin) ------------//
+ private byte logSource;
+ private String nodeId;
+ private int nodeIdLength;
private byte logType;
private int jobId;
private int datasetId;
@@ -92,6 +101,11 @@
private final CRC32 checksumGen;
private int[] PKFields;
private PrimaryIndexOperationTracker opTracker;
+ private IReplicationThread replicationThread;
+ private ByteBuffer serializedLog;
+ private final Map<String, byte[]> nodeIdsMap;
+ //this field is used for serialized flush logs only to indicate how many indexes were flushed using its LSN.
+ private int numOfFlushedIndexes;
public LogRecord() {
isFlushed = new AtomicBoolean(false);
@@ -99,29 +113,43 @@
readPKValue = new PrimaryKeyTupleReference();
readNewValue = (SimpleTupleReference) tupleWriter.createTupleReference();
checksumGen = new CRC32();
+ this.nodeIdsMap = new HashMap<String, byte[]>();
+ logSource = LogSource.LOCAL;
}
- private final static int TYPE_LEN = Byte.SIZE / Byte.SIZE;
- public final static int PKHASH_LEN = Integer.SIZE / Byte.SIZE;
- public final static int PKSZ_LEN = Integer.SIZE / Byte.SIZE;
- private final static int PRVLSN_LEN = Long.SIZE / Byte.SIZE;
- private final static int RSID_LEN = Long.SIZE / Byte.SIZE;
- private final static int LOGRCD_SZ_LEN = Integer.SIZE / Byte.SIZE;
- private final static int FLDCNT_LEN = Integer.SIZE / Byte.SIZE;
- private final static int NEWOP_LEN = Byte.SIZE / Byte.SIZE;
- private final static int NEWVALSZ_LEN = Integer.SIZE / Byte.SIZE;
- private final static int CHKSUM_LEN = Long.SIZE / Byte.SIZE;
+ private final static int LOG_SOURCE_LEN = Byte.BYTES;
+ private final static int NODE_ID_STRING_LENGTH = Integer.BYTES;
+ private final static int TYPE_LEN = Byte.BYTES;
+ public final static int PKHASH_LEN = Integer.BYTES;
+ public final static int PKSZ_LEN = Integer.BYTES;
+ private final static int PRVLSN_LEN = Long.BYTES;
+ private final static int RSID_LEN = Long.BYTES;
+ private final static int LOGRCD_SZ_LEN = Integer.BYTES;
+ private final static int FLDCNT_LEN = Integer.BYTES;
+ private final static int NEWOP_LEN = Byte.BYTES;
+ private final static int NEWVALSZ_LEN = Integer.BYTES;
+ private final static int CHKSUM_LEN = Long.BYTES;
- private final static int ALL_RECORD_HEADER_LEN = TYPE_LEN + JobId.BYTES;
+ private final static int ALL_RECORD_HEADER_LEN = LOG_SOURCE_LEN + TYPE_LEN + JobId.BYTES + NODE_ID_STRING_LENGTH;
private final static int ENTITYCOMMIT_UPDATE_HEADER_LEN = DatasetId.BYTES + PKHASH_LEN + PKSZ_LEN;
private final static int UPDATE_LSN_HEADER = PRVLSN_LEN + RSID_LEN + LOGRCD_SZ_LEN;
private final static int UPDATE_BODY_HEADER = FLDCNT_LEN + NEWOP_LEN + NEWVALSZ_LEN;
- @Override
- public void writeLogRecord(ByteBuffer buffer) {
- int beginOffset = buffer.position();
+ private void writeLogRecordCommonFields(ByteBuffer buffer) {
+ buffer.put(logSource);
buffer.put(logType);
buffer.putInt(jobId);
+ if (nodeIdsMap.containsKey(nodeId)) {
+ buffer.put(nodeIdsMap.get(nodeId));
+ } else {
+ //byte array for node id length and string
+ byte[] bytes = new byte[(Integer.SIZE / 8) + nodeId.length()];
+ buffer.putInt(nodeId.length());
+ buffer.put(nodeId.getBytes(java.nio.charset.StandardCharsets.UTF_8));
+ buffer.position(buffer.position() - bytes.length);
+ buffer.get(bytes, 0, bytes.length);
+ nodeIdsMap.put(nodeId, bytes);
+ }
if (logType == LogType.UPDATE || logType == LogType.ENTITY_COMMIT) {
buffer.putInt(datasetId);
buffer.putInt(PKHashValue);
@@ -144,15 +172,59 @@
if (logType == LogType.FLUSH) {
buffer.putInt(datasetId);
}
+ }
+
+ @Override
+ public void writeLogRecord(ByteBuffer buffer) {
+ int beginOffset = buffer.position();
+ writeLogRecordCommonFields(buffer);
+ checksum = generateChecksum(buffer, beginOffset, logSize - CHKSUM_LEN);
+ buffer.putLong(checksum);
+ }
+
+ //this method is used when replication is enabled to include the log record LSN in the serialized version
+ @Override
+ public void writeLogRecord(ByteBuffer buffer, long appendLSN) {
+ int beginOffset = buffer.position();
+ writeLogRecordCommonFields(buffer);
+
+ if (logSource == LogSource.LOCAL) {
+ //copy the serialized log to send it to replicas
+ int serializedLogSize = getSerializedLogSize(logType, logSize);
+
+ if (serializedLog == null || serializedLog.capacity() < serializedLogSize) {
+ serializedLog = ByteBuffer.allocate(serializedLogSize);
+ } else {
+ serializedLog.clear();
+ }
+
+ int currentPosition = buffer.position();
+ int currentLogSize = (currentPosition - beginOffset);
+
+ buffer.position(beginOffset);
+ buffer.get(serializedLog.array(), 0, currentLogSize);
+ serializedLog.position(currentLogSize);
+ if (logType == LogType.FLUSH) {
+ serializedLog.putLong(appendLSN);
+ serializedLog.putInt(numOfFlushedIndexes);
+ }
+ serializedLog.flip();
+ buffer.position(currentPosition);
+ }
checksum = generateChecksum(buffer, beginOffset, logSize - CHKSUM_LEN);
buffer.putLong(checksum);
}
private void writePKValue(ByteBuffer buffer) {
- int i;
- for (i = 0; i < PKFieldCnt; i++) {
- buffer.put(PKValue.getFieldData(0), PKValue.getFieldStart(PKFields[i]), PKValue.getFieldLength(PKFields[i]));
+ if (logSource == LogSource.LOCAL) {
+ for (int i = 0; i < PKFieldCnt; i++) {
+ buffer.put(PKValue.getFieldData(0), PKValue.getFieldStart(PKFields[i]),
+ PKValue.getFieldLength(PKFields[i]));
+ }
+ } else {
+ //since PKValue is already serialized in remote logs, just put it into buffer
+ buffer.put(PKValue.getFieldData(0), 0, PKValueSize);
}
}
@@ -171,13 +243,57 @@
@Override
public RECORD_STATUS readLogRecord(ByteBuffer buffer) {
int beginOffset = buffer.position();
- //first we need the logtype and Job ID, if the buffer isn't that big, then no dice.
- if (buffer.remaining() < ALL_RECORD_HEADER_LEN) {
+
+ //read header
+ RECORD_STATUS status = readLogHeader(buffer);
+ if (status != RECORD_STATUS.OK) {
+ buffer.position(beginOffset);
+ return status;
+ }
+
+ //read body
+ status = readLogBody(buffer, false);
+ if (status != RECORD_STATUS.OK) {
+ buffer.position(beginOffset);
+ return status;
+ }
+
+ //attempt to read checksum
+ if (buffer.remaining() < CHKSUM_LEN) {
buffer.position(beginOffset);
return RECORD_STATUS.TRUNCATED;
}
+ checksum = buffer.getLong();
+ if (checksum != generateChecksum(buffer, beginOffset, logSize - CHKSUM_LEN)) {
+ return RECORD_STATUS.BAD_CHKSUM;
+ }
+
+ return RECORD_STATUS.OK;
+ }
+
+ private RECORD_STATUS readLogHeader(ByteBuffer buffer) {
+ //first we need the logtype and Job ID, if the buffer isn't that big, then no dice.
+ if (buffer.remaining() < ALL_RECORD_HEADER_LEN) {
+ return RECORD_STATUS.TRUNCATED;
+ }
+ logSource = buffer.get();
logType = buffer.get();
jobId = buffer.getInt();
+ nodeIdLength = buffer.getInt();
+ //attempt to read node id
+ if (buffer.remaining() < nodeIdLength) {
+ return RECORD_STATUS.TRUNCATED;
+ }
+ //read node id string
+ nodeId = new String(buffer.array(), buffer.position() + buffer.arrayOffset(), nodeIdLength,
+ java.nio.charset.StandardCharsets.UTF_8);
+ //skip node id string bytes
+ buffer.position(buffer.position() + nodeIdLength);
+
+ return RECORD_STATUS.OK;
+ }
+
+ private RECORD_STATUS readLogBody(ByteBuffer buffer, boolean allocateTupleBuffer) {
if (logType != LogType.FLUSH) {
if (logType == LogType.JOB_COMMIT || logType == LogType.ABORT) {
datasetId = -1;
@@ -185,7 +301,6 @@
} else {
//attempt to read in the dsid, PK hash and PK length
if (buffer.remaining() < ENTITYCOMMIT_UPDATE_HEADER_LEN) {
- buffer.position(beginOffset);
return RECORD_STATUS.TRUNCATED;
}
datasetId = buffer.getInt();
@@ -193,7 +308,6 @@
PKValueSize = buffer.getInt();
//attempt to read in the PK
if (buffer.remaining() < PKValueSize) {
- buffer.position(beginOffset);
return RECORD_STATUS.TRUNCATED;
}
if (PKValueSize <= 0) {
@@ -201,10 +315,10 @@
}
PKValue = readPKValue(buffer);
}
+
if (logType == LogType.UPDATE) {
//attempt to read in the previous LSN, log size, new value size, and new record type
if (buffer.remaining() < UPDATE_LSN_HEADER + UPDATE_BODY_HEADER) {
- buffer.position(beginOffset);
return RECORD_STATUS.TRUNCATED;
}
prevLSN = buffer.getLong();
@@ -214,34 +328,50 @@
newOp = buffer.get();
newValueSize = buffer.getInt();
if (buffer.remaining() < newValueSize) {
- buffer.position(beginOffset);
return RECORD_STATUS.TRUNCATED;
}
- newValue = readTuple(buffer, readNewValue, fieldCnt, newValueSize);
+ if (!allocateTupleBuffer) {
+ newValue = readTuple(buffer, readNewValue, fieldCnt, newValueSize);
+ } else {
+ ByteBuffer tupleBuffer = ByteBuffer.allocate(newValueSize);
+ tupleBuffer.put(buffer.array(), buffer.position(), newValueSize);
+ tupleBuffer.flip();
+ newValue = readTuple(tupleBuffer, readNewValue, fieldCnt, newValueSize);
+ //skip tuple bytes
+ buffer.position(buffer.position() + newValueSize);
+ }
} else {
computeAndSetLogSize();
}
} else {
computeAndSetLogSize();
if (buffer.remaining() < DatasetId.BYTES) {
- buffer.position(beginOffset);
return RECORD_STATUS.TRUNCATED;
}
datasetId = buffer.getInt();
resourceId = 0l;
- }
- //atempt to read checksum
- if (buffer.remaining() < CHKSUM_LEN) {
- buffer.position(beginOffset);
- return RECORD_STATUS.TRUNCATED;
- }
- checksum = buffer.getLong();
- if (checksum != generateChecksum(buffer, beginOffset, logSize - CHKSUM_LEN)) {
- return RECORD_STATUS.BAD_CHKSUM;
+ computeAndSetLogSize();
}
return RECORD_STATUS.OK;
}
+ @Override
+ public void deserialize(ByteBuffer buffer, boolean remoteRecoveryLog, String localNodeId) {
+ readLogHeader(buffer);
+ if (!remoteRecoveryLog || !nodeId.equals(localNodeId)) {
+ readLogBody(buffer, false);
+ } else {
+ //need to allocate buffer for tuple since the logs will be kept in memory to use during remote recovery
+ //TODO when this is redesigned to spill remote recovery logs to disk, this will not be needed
+ readLogBody(buffer, true);
+ }
+
+ if (logType == LogType.FLUSH) {
+ LSN = buffer.getLong();
+ numOfFlushedIndexes = buffer.getInt();
+ }
+ }
+
private ITupleReference readPKValue(ByteBuffer buffer) {
if (buffer.position() + PKValueSize > buffer.limit()) {
throw new BufferUnderflowException();
@@ -251,7 +381,8 @@
return readPKValue;
}
- private ITupleReference readTuple(ByteBuffer srcBuffer, SimpleTupleReference destTuple, int fieldCnt, int size) {
+ private static ITupleReference readTuple(ByteBuffer srcBuffer, SimpleTupleReference destTuple, int fieldCnt,
+ int size) {
if (srcBuffer.position() + size > srcBuffer.limit()) {
throw new BufferUnderflowException();
}
@@ -264,18 +395,33 @@
@Override
public void formJobTerminateLogRecord(ITransactionContext txnCtx, boolean isCommit) {
this.txnCtx = txnCtx;
+ formJobTerminateLogRecord(txnCtx.getJobId().getId(), isCommit, nodeId);
+ }
+
+ @Override
+ public void formJobTerminateLogRecord(int jobId, boolean isCommit, String nodeId) {
this.logType = isCommit ? LogType.JOB_COMMIT : LogType.ABORT;
- this.jobId = txnCtx.getJobId().getId();
+ this.jobId = jobId;
this.datasetId = -1;
this.PKHashValue = -1;
+ setNodeId(nodeId);
computeAndSetLogSize();
}
- public void formFlushLogRecord(int datasetId, PrimaryIndexOperationTracker opTracker) {
+ public void formFlushLogRecord(int datasetId, PrimaryIndexOperationTracker opTracker, int numOfFlushedIndexes) {
+ formFlushLogRecord(datasetId, opTracker, null, numOfFlushedIndexes);
+ }
+
+ public void formFlushLogRecord(int datasetId, PrimaryIndexOperationTracker opTracker, String nodeId,
+ int numberOfIndexes) {
this.logType = LogType.FLUSH;
this.jobId = -1;
this.datasetId = datasetId;
this.opTracker = opTracker;
+ this.numOfFlushedIndexes = numberOfIndexes;
+ if (nodeId != null) {
+ setNodeId(nodeId);
+ }
computeAndSetLogSize();
}
@@ -326,11 +472,15 @@
default:
throw new IllegalStateException("Unsupported Log Type");
}
+
+ logSize += nodeIdLength;
}
@Override
public String getLogRecordForDisplay() {
StringBuilder builder = new StringBuilder();
+ builder.append(" Source : ").append(LogSource.toString(logSource));
+ builder.append(" NodeID : ").append(nodeId);
builder.append(" LSN : ").append(LSN);
builder.append(" LogType : ").append(LogType.toString(logType));
builder.append(" LogSize : ").append(logSize);
@@ -348,6 +498,20 @@
return builder.toString();
}
+ @Override
+ public int serialize(ByteBuffer buffer) {
+ int bufferBegin = buffer.position();
+ writeLogRecordCommonFields(buffer);
+
+ if (logType == LogType.FLUSH) {
+ buffer.putLong(LSN);
+ buffer.putInt(numOfFlushedIndexes);
+ }
+
+ buffer.putLong(LSN);
+ return buffer.position() - bufferBegin;
+ }
+
////////////////////////////////////////////
// getter and setter methods
////////////////////////////////////////////
@@ -438,6 +602,25 @@
}
@Override
+ public int getSerializedLogSize() {
+ return getSerializedLogSize(logType, logSize);
+ }
+
+ private static int getSerializedLogSize(Byte logType, int logSize) {
+ if (logType == LogType.FLUSH) {
+ //LSN
+ logSize += (Long.SIZE / 8);
+ //num of indexes
+ logSize += (Integer.SIZE / 8);
+ }
+
+ //checksum not included in serialized version
+ logSize -= CHKSUM_LEN;
+
+ return logSize;
+ }
+
+ @Override
public void setLogSize(int logSize) {
this.logSize = logSize;
}
@@ -517,4 +700,52 @@
public PrimaryIndexOperationTracker getOpTracker() {
return opTracker;
}
+
+ @Override
+ public ByteBuffer getSerializedLog() {
+ return serializedLog;
+ }
+
+ public void setSerializedLog(ByteBuffer serializedLog) {
+ this.serializedLog = serializedLog;
+ }
+
+ @Override
+ public String getNodeId() {
+ return nodeId;
+ }
+
+ @Override
+ public void setNodeId(String nodeId) {
+ this.nodeId = nodeId;
+ this.nodeIdLength = nodeId.length();
+ }
+
+ public IReplicationThread getReplicationThread() {
+ return replicationThread;
+ }
+
+ @Override
+ public void setReplicationThread(IReplicationThread replicationThread) {
+ this.replicationThread = replicationThread;
+ }
+
+ @Override
+ public void setLogSource(byte logSource) {
+ this.logSource = logSource;
+ }
+
+ @Override
+ public byte getLogSource() {
+ return logSource;
+ }
+
+ public int getNumOfFlushedIndexes() {
+ return numOfFlushedIndexes;
+ }
+
+ public void setNumOfFlushedIndexes(int numOfFlushedIndexes) {
+ this.numOfFlushedIndexes = numOfFlushedIndexes;
+ }
+
}
diff --git a/asterix-common/src/main/java/org/apache/asterix/common/transactions/LogSource.java b/asterix-common/src/main/java/org/apache/asterix/common/transactions/LogSource.java
new file mode 100644
index 0000000..34e81db
--- /dev/null
+++ b/asterix-common/src/main/java/org/apache/asterix/common/transactions/LogSource.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.common.transactions;
+
+public class LogSource {
+
+ public static final byte LOCAL = 0;
+ public static final byte REMOTE = 1;
+ public static final byte REMOTE_RECOVERY = 2;
+
+ private static final String STRING_LOCAL = "LOCAL";
+ private static final String STRING_REMOTE = "REMOTE";
+ private static final String STRING_REMOTE_RECOVERY = "REMOTE_RECOVERY";
+
+ private static final String STRING_INVALID_LOG_SOURCE = "INVALID_LOG_SOURCE";
+
+ public static String toString(byte LogSource) {
+ switch (LogSource) {
+ case LOCAL:
+ return STRING_LOCAL;
+ case REMOTE:
+ return STRING_REMOTE;
+ case REMOTE_RECOVERY:
+ return STRING_REMOTE_RECOVERY;
+ default:
+ return STRING_INVALID_LOG_SOURCE;
+ }
+ }
+
+}
\ No newline at end of file
diff --git a/asterix-common/src/main/resources/schema/cluster.xsd b/asterix-common/src/main/resources/schema/cluster.xsd
index d9b62c2..d3203d5 100644
--- a/asterix-common/src/main/resources/schema/cluster.xsd
+++ b/asterix-common/src/main/resources/schema/cluster.xsd
@@ -44,7 +44,11 @@
<xs:element name="http_port" type="xs:integer" />
<xs:element name="debug_port" type="xs:integer" />
<xs:element name="metadata_node" type="xs:string" />
-
+ <xs:element name="enabled" type="xs:boolean" />
+ <xs:element name="replication_port" type="xs:integer" />
+ <xs:element name="replication_factor" type="xs:integer" />
+ <xs:element name="replication_store" type="xs:string" />
+ <xs:element name="replication_time_out" type="xs:integer" />
<!-- definition of complex elements -->
<xs:element name="working_dir">
@@ -72,6 +76,18 @@
</xs:complexType>
</xs:element>
+ <xs:element name="data_replication">
+ <xs:complexType>
+ <xs:sequence>
+ <xs:element ref="cl:enabled" />
+ <xs:element ref="cl:replication_port" />
+ <xs:element ref="cl:replication_factor" />
+ <xs:element ref="cl:replication_store" />
+ <xs:element ref="cl:replication_time_out" />
+ </xs:sequence>
+ </xs:complexType>
+ </xs:element>
+
<xs:element name="property">
<xs:complexType>
<xs:sequence>
@@ -126,6 +142,7 @@
<xs:element ref="cl:iodevices" minOccurs="0" />
<xs:element ref="cl:working_dir" />
<xs:element ref="cl:metadata_node" />
+ <xs:element ref="cl:data_replication" minOccurs="0" />
<xs:element ref="cl:master_node" />
<xs:element ref="cl:node" maxOccurs="unbounded" />
<xs:element ref="cl:substitute_nodes" />
diff --git a/asterix-events/src/main/java/org/apache/asterix/event/util/PatternCreator.java b/asterix-events/src/main/java/org/apache/asterix/event/util/PatternCreator.java
index 2d7f56b..b6aaddb 100644
--- a/asterix-events/src/main/java/org/apache/asterix/event/util/PatternCreator.java
+++ b/asterix-events/src/main/java/org/apache/asterix/event/util/PatternCreator.java
@@ -319,6 +319,10 @@
patternList.addAll(createRemoveAsterixLogDirPattern(instance).getPattern());
patternList.addAll(createRemoveAsterixRootMetadata(instance).getPattern());
patternList.addAll(createRemoveAsterixTxnLogs(instance).getPattern());
+ if (instance.getCluster().getDataReplication() != null
+ && instance.getCluster().getDataReplication().isEnabled()) {
+ patternList.addAll(createRemoveAsterixReplicationPattern(instance).getPattern());
+ }
Patterns patterns = new Patterns(patternList);
return patterns;
}
@@ -645,5 +649,29 @@
Patterns patterns = new Patterns(patternList);
return patterns;
}
+
+ private Patterns createRemoveAsterixReplicationPattern(AsterixInstance instance) throws Exception {
+
+ List<Pattern> patternList = new ArrayList<Pattern>();
+ Cluster cluster = instance.getCluster();
+
+ Nodeid nodeid = null;
+ String pargs = null;
+ Event event = null;
+ for (Node node : cluster.getNode()) {
+ String[] nodeIODevices;
+ String iodevices = node.getIodevices() == null ? cluster.getIodevices() : node.getIodevices();
+ nodeIODevices = iodevices.trim().split(",");
+ for (String nodeIODevice : nodeIODevices) {
+ pargs = nodeIODevice + File.separator + cluster.getDataReplication().getReplicationStore();
+ nodeid = new Nodeid(new Value(null, node.getId()));
+ event = new Event("file_delete", nodeid, pargs);
+ patternList.add(new Pattern(null, 1, null, event));
+ }
+ }
+
+ Patterns patterns = new Patterns(patternList);
+ return patterns;
+ }
}
diff --git a/asterix-installer/src/main/java/org/apache/asterix/installer/command/ValidateCommand.java b/asterix-installer/src/main/java/org/apache/asterix/installer/command/ValidateCommand.java
index 159eeac..2254f6f 100644
--- a/asterix-installer/src/main/java/org/apache/asterix/installer/command/ValidateCommand.java
+++ b/asterix-installer/src/main/java/org/apache/asterix/installer/command/ValidateCommand.java
@@ -123,6 +123,8 @@
}
}
}
+
+ valid = valid & validateReplicationProperties(cluster);
}
if (valid) {
@@ -260,6 +262,79 @@
return valid;
}
+ private boolean validateReplicationProperties(Cluster cluster) {
+ boolean valid = true;
+
+ //if replication is disabled, no need to validate the settings
+ if (cluster.getDataReplication() != null && cluster.getDataReplication().isEnabled()) {
+
+ if (cluster.getDataReplication().getReplicationFactor() == null) {
+ if (cluster.getNode().size() >= 3) {
+ LOGGER.warn("Replication factor not defined. Using default value (3) " + WARNING);
+
+ } else {
+ valid = false;
+ LOGGER.fatal("Replication factor not defined for data repliaction. " + ERROR);
+ }
+
+ }
+
+ //replication factor = 1 means no replication
+ if (cluster.getDataReplication().getReplicationFactor().intValue() == 1) {
+ LOGGER.warn("Replication factor is set to 1. Disabling data replication" + WARNING);
+ return true;
+ }
+
+ if (cluster.getDataReplication().getReplicationFactor().intValue() > cluster.getNode().size()) {
+ LOGGER.fatal("Replication factor = " + cluster.getDataReplication().getReplicationFactor().intValue()
+ + " requires at least " + cluster.getDataReplication().getReplicationFactor().intValue()
+ + " nodes in the cluster" + ERROR);
+ valid = false;
+ }
+
+ if (cluster.getDataReplication().getReplicationStore() == null
+ || cluster.getDataReplication().getReplicationStore().length() == 0) {
+ valid = false;
+ LOGGER.fatal("Replication store not defined. " + ERROR);
+ }
+
+ if (cluster.getDataReplication().getReplicationPort() == null
+ || cluster.getDataReplication().getReplicationPort().toString().length() == 0) {
+ valid = false;
+ LOGGER.fatal("Replication data port not defined for data repliaction. " + ERROR);
+ }
+
+ if (cluster.getDataReplication().getReplicationTimeOut() == null
+ || (cluster.getDataReplication().getReplicationTimeOut().intValue() + "").length() == 0) {
+ LOGGER.warn("Replication maximum wait time not defined. Using default value (60 seconds) " + WARNING);
+ }
+
+ //validate all nodes have the same number of io devices
+ int numOfIODevices = 0;
+ Set<Integer> ioDevicesCount = new HashSet<Integer>();
+ for (int i = 0; i < cluster.getNode().size(); i++) {
+ Node node = cluster.getNode().get(i);
+
+ if (node.getIodevices() != null) {
+ numOfIODevices = node.getIodevices().length() - node.getIodevices().replace(",", "").length();
+ } else {
+ numOfIODevices = cluster.getIodevices().length() - cluster.getIodevices().replace(",", "").length();
+ }
+
+ ioDevicesCount.add(numOfIODevices);
+
+ if (ioDevicesCount.size() > 1) {
+ valid = false;
+ LOGGER.fatal("Replication requires all nodes to have the same number of IO devices." + ERROR);
+ break;
+ }
+ }
+
+ }
+
+ return valid;
+ }
+
}
class ValidateConfig extends CommandConfig {
diff --git a/asterix-lang-aql/pom.xml b/asterix-lang-aql/pom.xml
index d85ba7b..b8ea8c3 100644
--- a/asterix-lang-aql/pom.xml
+++ b/asterix-lang-aql/pom.xml
@@ -43,8 +43,8 @@
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-compiler-plugin</artifactId>
<configuration>
- <source>1.7</source>
- <target>1.7</target>
+ <source>1.8</source>
+ <target>1.8</target>
<fork>true</fork>
</configuration>
</plugin>
diff --git a/asterix-lang-sqlpp/pom.xml b/asterix-lang-sqlpp/pom.xml
index a76b403..492bcf1 100644
--- a/asterix-lang-sqlpp/pom.xml
+++ b/asterix-lang-sqlpp/pom.xml
@@ -43,8 +43,8 @@
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-compiler-plugin</artifactId>
<configuration>
- <source>1.7</source>
- <target>1.7</target>
+ <source>1.8</source>
+ <target>1.8</target>
<fork>true</fork>
</configuration>
</plugin>
diff --git a/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataManager.java b/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataManager.java
index 97260a1..9ae5650 100644
--- a/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataManager.java
+++ b/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataManager.java
@@ -96,6 +96,7 @@
private final ReadWriteLock metadataLatch;
private final AsterixMetadataProperties metadataProperties;
private IHyracksClientConnection hcc;
+ public boolean rebindMetadataNode = false;
public MetadataManager(IAsterixStateProxy proxy, AsterixMetadataProperties metadataProperties) {
if (proxy == null) {
@@ -121,7 +122,7 @@
public void init() throws RemoteException, MetadataException {
// Could be synchronized on any object. Arbitrarily chose proxy.
synchronized (proxy) {
- if (metadataNode != null) {
+ if (metadataNode != null && !rebindMetadataNode) {
return;
}
try {
@@ -130,6 +131,7 @@
while (retry++ < MAX_RETRY_COUNT) {
metadataNode = proxy.getMetadataNode();
if (metadataNode != null) {
+ rebindMetadataNode = false;
break;
}
Thread.sleep(sleep);
@@ -390,7 +392,8 @@
throw new MetadataException(e);
}
try {
- ctx.addDatatype(metadataNode.getDatatype(ctx.getJobId(),datatype.getDataverseName(),datatype.getDatatypeName()));
+ ctx.addDatatype(metadataNode.getDatatype(ctx.getJobId(), datatype.getDataverseName(),
+ datatype.getDatatypeName()));
} catch (RemoteException e) {
throw new MetadataException(e);
}
@@ -718,7 +721,6 @@
}
return adapter;
}
-
@Override
public void dropLibrary(MetadataTransactionContext ctx, String dataverseName, String libraryName)
@@ -801,7 +803,6 @@
}
return FeedPolicy;
}
-
@Override
public Feed getFeed(MetadataTransactionContext ctx, String dataverse, String feedName) throws MetadataException {
@@ -836,7 +837,6 @@
ctx.addFeed(feed);
}
-
public List<DatasourceAdapter> getDataverseAdapters(MetadataTransactionContext mdTxnCtx, String dataverse)
throws MetadataException {
List<DatasourceAdapter> dataverseAdapters;
@@ -847,7 +847,7 @@
}
return dataverseAdapters;
}
-
+
public void dropFeedPolicy(MetadataTransactionContext mdTxnCtx, String dataverseName, String policyName)
throws MetadataException {
FeedPolicy feedPolicy = null;
@@ -859,7 +859,7 @@
}
mdTxnCtx.dropFeedPolicy(feedPolicy);
}
-
+
public List<FeedPolicy> getDataversePolicies(MetadataTransactionContext mdTxnCtx, String dataverse)
throws MetadataException {
List<FeedPolicy> dataverseFeedPolicies;
@@ -871,7 +871,6 @@
return dataverseFeedPolicies;
}
-
@Override
public List<ExternalFile> getDatasetExternalFiles(MetadataTransactionContext mdTxnCtx, Dataset dataset)
throws MetadataException {
diff --git a/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/AqlMetadataProvider.java b/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/AqlMetadataProvider.java
index 2478dc7..d31dfcf 100644
--- a/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/AqlMetadataProvider.java
+++ b/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/AqlMetadataProvider.java
@@ -182,6 +182,7 @@
import org.apache.hyracks.storage.am.rtree.frames.RTreePolicyType;
public class AqlMetadataProvider implements IMetadataProvider<AqlSourceId, String> {
+ public static final String TEMP_DATASETS_STORAGE_FOLDER = "temp";
private static Logger LOGGER = Logger.getLogger(AqlMetadataProvider.class.getName());
private MetadataTransactionContext mdTxnCtx;
private boolean isWriteTransaction;
@@ -2195,7 +2196,8 @@
for (int j = 0; j < nodeStores.length; j++) {
for (int k = 0; k < numIODevices; k++) {
File f = new File(ioDevices[k] + File.separator + nodeStores[j]
- + (temp ? (File.separator + "temp") : "") + File.separator + relPathFile);
+ + (temp ? (File.separator + TEMP_DATASETS_STORAGE_FOLDER) : "") + File.separator
+ + relPathFile);
splitArray.add(new FileSplit(nd, new FileReference(f), k));
}
}
diff --git a/asterix-om/src/main/java/org/apache/asterix/om/util/AsterixAppContextInfo.java b/asterix-om/src/main/java/org/apache/asterix/om/util/AsterixAppContextInfo.java
index 8f9f577..866162b 100644
--- a/asterix-om/src/main/java/org/apache/asterix/om/util/AsterixAppContextInfo.java
+++ b/asterix-om/src/main/java/org/apache/asterix/om/util/AsterixAppContextInfo.java
@@ -26,6 +26,7 @@
import org.apache.asterix.common.config.AsterixBuildProperties;
import org.apache.asterix.common.config.AsterixMetadataProperties;
import org.apache.asterix.common.config.AsterixPropertiesAccessor;
+import org.apache.asterix.common.config.AsterixReplicationProperties;
import org.apache.asterix.common.config.AsterixStorageProperties;
import org.apache.asterix.common.config.AsterixTransactionProperties;
import org.apache.asterix.common.config.IAsterixPropertiesProvider;
@@ -54,6 +55,7 @@
private AsterixTransactionProperties txnProperties;
private AsterixFeedProperties feedProperties;
private AsterixBuildProperties buildProperties;
+ private AsterixReplicationProperties replicationProperties;
private IHyracksClientConnection hcc;
@@ -68,6 +70,8 @@
INSTANCE.storageProperties = new AsterixStorageProperties(propertiesAccessor);
INSTANCE.txnProperties = new AsterixTransactionProperties(propertiesAccessor);
INSTANCE.feedProperties = new AsterixFeedProperties(propertiesAccessor);
+ INSTANCE.replicationProperties = new AsterixReplicationProperties(propertiesAccessor,
+ AsterixClusterProperties.INSTANCE.getCluster());
INSTANCE.hcc = hcc;
INSTANCE.buildProperties = new AsterixBuildProperties(propertiesAccessor);
Logger.getLogger("org.apache").setLevel(INSTANCE.externalProperties.getLogLevel());
@@ -135,4 +139,9 @@
public IStorageManagerInterface getStorageManagerInterface() {
return AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER;
}
+
+ @Override
+ public AsterixReplicationProperties getReplicationProperties() {
+ return replicationProperties;
+ }
}
diff --git a/asterix-om/src/main/java/org/apache/asterix/om/util/AsterixClusterProperties.java b/asterix-om/src/main/java/org/apache/asterix/om/util/AsterixClusterProperties.java
index 9b8a40b..438805a 100644
--- a/asterix-om/src/main/java/org/apache/asterix/om/util/AsterixClusterProperties.java
+++ b/asterix-om/src/main/java/org/apache/asterix/om/util/AsterixClusterProperties.java
@@ -75,13 +75,15 @@
}
}
-
-
private ClusterState state = ClusterState.UNUSABLE;
public synchronized void removeNCConfiguration(String nodeId) {
- // state = State.UNUSABLE;
ncConfiguration.remove(nodeId);
+ if (ncConfiguration.keySet().size() != AsterixAppContextInfo.getInstance().getMetadataProperties()
+ .getNodeNames().size()) {
+ state = ClusterState.UNUSABLE;
+ LOGGER.info("Cluster now is in UNSABLE state");
+ }
resetClusterPartitionConstraint();
}
diff --git a/asterix-replication/pom.xml b/asterix-replication/pom.xml
new file mode 100644
index 0000000..b5595ef
--- /dev/null
+++ b/asterix-replication/pom.xml
@@ -0,0 +1,70 @@
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+ <modelVersion>4.0.0</modelVersion>
+ <parent>
+ <groupId>org.apache.asterix</groupId>
+ <artifactId>asterix</artifactId>
+ <version>0.8.8-SNAPSHOT</version>
+ </parent>
+ <artifactId>asterix-replication</artifactId>
+ <licenses>
+ <license>
+ <name>Apache License, Version 2.0</name>
+ <url>http://www.apache.org/licenses/LICENSE-2.0.txt</url>
+ <distribution>repo</distribution>
+ <comments>A business-friendly OSS license</comments>
+ </license>
+ </licenses>
+ <build>
+ <plugins>
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-compiler-plugin</artifactId>
+ <configuration>
+ <source>1.8</source>
+ <target>1.8</target>
+ <fork>true</fork>
+ </configuration>
+ </plugin>
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-jar-plugin</artifactId>
+ <configuration>
+ <includes>
+ <include>**/*.class</include>
+ <include>**/*.txt</include>
+ </includes>
+ </configuration>
+ <executions>
+ <execution>
+ <goals>
+ <goal>test-jar</goal>
+ </goals>
+ <phase>package</phase>
+ </execution>
+ </executions>
+ </plugin>
+ </plugins>
+ </build>
+ <dependencies>
+ <dependency>
+ <groupId>org.apache.asterix</groupId>
+ <artifactId>asterix-common</artifactId>
+ <version>0.8.8-SNAPSHOT</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.asterix</groupId>
+ <artifactId>asterix-metadata</artifactId>
+ <version>0.8.8-SNAPSHOT</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hyracks</groupId>
+ <artifactId>hyracks-api</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hyracks</groupId>
+ <artifactId>hyracks-storage-am-common</artifactId>
+ </dependency>
+ </dependencies>
+
+</project>
\ No newline at end of file
diff --git a/asterix-replication/src/main/java/org/apache/asterix/replication/functions/AsterixReplicationProtocol.java b/asterix-replication/src/main/java/org/apache/asterix/replication/functions/AsterixReplicationProtocol.java
new file mode 100644
index 0000000..f75ed6b
--- /dev/null
+++ b/asterix-replication/src/main/java/org/apache/asterix/replication/functions/AsterixReplicationProtocol.java
@@ -0,0 +1,346 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.replication.functions;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.SocketChannel;
+
+import org.apache.asterix.common.replication.Replica;
+import org.apache.asterix.common.replication.ReplicaEvent;
+import org.apache.asterix.common.transactions.ILogRecord;
+import org.apache.asterix.replication.management.NetworkingUtil;
+import org.apache.asterix.replication.storage.AsterixLSMIndexFileProperties;
+import org.apache.asterix.replication.storage.LSMComponentProperties;
+
+public class AsterixReplicationProtocol {
+
+ /**
+ * All replication messages start with ReplicationFunctions (4 bytes), then the length of the request in bytes
+ */
+ public static final String JOB_COMMIT_ACK = "$";
+
+ public final static int REPLICATION_REQUEST_TYPE_SIZE = Integer.BYTES;
+ public final static int REPLICATION_REQUEST_HEADER_SIZE = REPLICATION_REQUEST_TYPE_SIZE + Integer.BYTES;
+
+ /*
+ * ReplicationRequestType:
+ * REPLICATE_LOG: txn log replication
+ * REPLICATE_FILE: replicate a file(s)
+ * DELETE_FILE: delete a file(s)
+ * GET_REPLICA_FILES: used during remote recovery to request lost LSM Components
+ * GET_REPLICA_LOGS: used during remote recovery to request lost txn logs
+ * GET_REPLICA_MAX_LSN: used during remote recovery initialize a log manager LSN
+ * GET_REPLICA_MIN_LSN: used during remote recovery to specify the low water mark per replica
+ * UPDATE_REPLICA: used to update replica info such as IP Address change.
+ * GOODBYE: used to notify replicas that the replication request has been completed
+ * REPLICA_EVENT: used to notify replicas about a remote replica split/merge.
+ * LSM_COMPONENT_PROPERTIES: used to send the properties of an LSM Component before its physical files are sent
+ * ACK: used to notify the requesting replica that the request has been completed successfully
+ * FLUSH_INDEX: request remote replica to flush an LSM component
+ */
+ public enum ReplicationRequestType {
+ REPLICATE_LOG,
+ REPLICATE_FILE,
+ DELETE_FILE,
+ GET_REPLICA_FILES,
+ GET_REPLICA_LOGS,
+ GET_REPLICA_MAX_LSN,
+ GET_REPLICA_MIN_LSN,
+ UPDATE_REPLICA,
+ GOODBYE,
+ REPLICA_EVENT,
+ LSM_COMPONENT_PROPERTIES,
+ ACK,
+ FLUSH_INDEX
+ }
+
+ public static ByteBuffer readRequest(SocketChannel socketChannel, ByteBuffer dataBuffer) throws IOException {
+ //read request size
+ NetworkingUtil.readBytes(socketChannel, dataBuffer, Integer.BYTES);
+ int requestSize = dataBuffer.getInt();
+
+ if (dataBuffer.capacity() < requestSize) {
+ dataBuffer = ByteBuffer.allocate(requestSize);
+ }
+
+ //read request
+ NetworkingUtil.readBytes(socketChannel, dataBuffer, requestSize);
+
+ return dataBuffer;
+ }
+
+ public static ByteBuffer writeLSMComponentPropertiesRequest(LSMComponentProperties lsmCompProp, ByteBuffer buffer)
+ throws IOException {
+ ByteArrayOutputStream outputStream = new ByteArrayOutputStream();
+ DataOutputStream oos = new DataOutputStream(outputStream);
+ lsmCompProp.serialize(oos);
+ oos.close();
+
+ int requestSize = REPLICATION_REQUEST_HEADER_SIZE + oos.size();
+ if (buffer.capacity() < requestSize) {
+ buffer = ByteBuffer.allocate(requestSize);
+ } else {
+ buffer.clear();
+ }
+ buffer.putInt(ReplicationRequestType.LSM_COMPONENT_PROPERTIES.ordinal());
+ buffer.putInt(oos.size());
+ buffer.put(outputStream.toByteArray());
+ buffer.flip();
+ return buffer;
+ }
+
+ public static ReplicationRequestType getRequestType(SocketChannel socketChannel, ByteBuffer byteBuffer)
+ throws IOException {
+ //read replication request type
+ NetworkingUtil.readBytes(socketChannel, byteBuffer, REPLICATION_REQUEST_TYPE_SIZE);
+
+ ReplicationRequestType requestType = AsterixReplicationProtocol.ReplicationRequestType.values()[byteBuffer
+ .getInt()];
+ return requestType;
+ }
+
+ public static LSMComponentProperties readLSMPropertiesRequest(ByteBuffer buffer) throws IOException {
+ ByteArrayInputStream bais = new ByteArrayInputStream(buffer.array(), buffer.position(), buffer.limit());
+ DataInputStream dis = new DataInputStream(bais);
+ return LSMComponentProperties.create(dis);
+ }
+
+ public static ByteBuffer getGoodbyeBuffer() {
+ ByteBuffer bb = ByteBuffer.allocate(REPLICATION_REQUEST_TYPE_SIZE);
+ bb.putInt(ReplicationRequestType.GOODBYE.ordinal());
+ bb.flip();
+ return bb;
+ }
+
+ public static ByteBuffer getAckBuffer() {
+ ByteBuffer bb = ByteBuffer.allocate(REPLICATION_REQUEST_TYPE_SIZE);
+ bb.putInt(ReplicationRequestType.ACK.ordinal());
+ bb.flip();
+ return bb;
+ }
+
+ public static void writeReplicateLogRequest(ByteBuffer requestBuffer, ILogRecord logRecord) {
+ requestBuffer.clear();
+ //put request type (4 bytes)
+ requestBuffer.putInt(ReplicationRequestType.REPLICATE_LOG.ordinal());
+ //leave space for log size
+ requestBuffer.position(requestBuffer.position() + Integer.BYTES);
+ int logSize = logRecord.serialize(requestBuffer);
+ //put request type (4 bytes)
+ requestBuffer.putInt(4, logSize);
+ requestBuffer.flip();
+ }
+
+ public static void writeReplicateLogRequest(ByteBuffer requestBuffer, byte[] serializedLog) {
+ requestBuffer.clear();
+ //put request type (4 bytes)
+ requestBuffer.putInt(ReplicationRequestType.REPLICATE_LOG.ordinal());
+ //length of the log
+ requestBuffer.putInt(serializedLog.length);
+ //the log itself
+ requestBuffer.put(serializedLog);
+ requestBuffer.flip();
+ }
+
+ public static ByteBuffer writeFileReplicationRequest(ByteBuffer requestBuffer, AsterixLSMIndexFileProperties afp,
+ ReplicationRequestType requestType) throws IOException {
+ ByteArrayOutputStream outputStream = new ByteArrayOutputStream();
+ DataOutputStream oos = new DataOutputStream(outputStream);
+ afp.serialize(oos);
+ oos.close();
+
+ int requestSize = REPLICATION_REQUEST_HEADER_SIZE + oos.size();
+ if (requestBuffer.capacity() < requestSize) {
+ requestBuffer = ByteBuffer.allocate(requestSize);
+ } else {
+ requestBuffer.clear();
+ }
+ requestBuffer.putInt(requestType.ordinal());
+ requestBuffer.putInt(oos.size());
+ requestBuffer.put(outputStream.toByteArray());
+ requestBuffer.flip();
+ return requestBuffer;
+ }
+
+ public static AsterixLSMIndexFileProperties readFileReplicationRequest(ByteBuffer buffer) throws IOException {
+ ByteArrayInputStream bais = new ByteArrayInputStream(buffer.array(), buffer.position(), buffer.limit());
+ DataInputStream dis = new DataInputStream(bais);
+ return AsterixLSMIndexFileProperties.create(dis);
+ }
+
+ public static ReplicaLogsRequest readReplicaLogsRequest(ByteBuffer buffer) throws IOException {
+ ByteArrayInputStream bais = new ByteArrayInputStream(buffer.array(), buffer.position(), buffer.limit());
+ DataInputStream dis = new DataInputStream(bais);
+ return ReplicaLogsRequest.create(dis);
+ }
+
+ public static ByteBuffer writeGetReplicaLogsRequest(ByteBuffer requestBuffer, ReplicaLogsRequest request)
+ throws IOException {
+ ByteArrayOutputStream outputStream = new ByteArrayOutputStream();
+ DataOutputStream oos = new DataOutputStream(outputStream);
+ request.serialize(oos);
+ oos.close();
+
+ int requestSize = REPLICATION_REQUEST_HEADER_SIZE + oos.size();
+ if (requestBuffer.capacity() < requestSize) {
+ requestBuffer = ByteBuffer.allocate(requestSize);
+ } else {
+ requestBuffer.clear();
+ }
+ requestBuffer.putInt(ReplicationRequestType.GET_REPLICA_LOGS.ordinal());
+ requestBuffer.putInt(oos.size());
+ requestBuffer.put(outputStream.toByteArray());
+ requestBuffer.flip();
+ return requestBuffer;
+ }
+
+ public static ByteBuffer writeUpdateReplicaRequest(Replica replica) throws IOException {
+ ByteArrayOutputStream outputStream = new ByteArrayOutputStream();
+ DataOutputStream oos = new DataOutputStream(outputStream);
+
+ oos.writeInt(ReplicationRequestType.UPDATE_REPLICA.ordinal());
+ replica.writeFields(oos);
+ oos.close();
+
+ ByteBuffer buffer = ByteBuffer.allocate(REPLICATION_REQUEST_HEADER_SIZE + oos.size());
+ buffer.putInt(ReplicationRequestType.UPDATE_REPLICA.ordinal());
+ buffer.putInt(oos.size());
+ buffer.put(outputStream.toByteArray());
+ return buffer;
+ }
+
+ public static ByteBuffer writeReplicaEventRequest(ReplicaEvent event) throws IOException {
+ ByteArrayOutputStream outputStream = new ByteArrayOutputStream();
+ DataOutputStream oos = new DataOutputStream(outputStream);
+ event.serialize(oos);
+ oos.close();
+
+ ByteBuffer buffer = ByteBuffer.allocate(REPLICATION_REQUEST_HEADER_SIZE + oos.size());
+ buffer.putInt(ReplicationRequestType.REPLICA_EVENT.ordinal());
+ buffer.putInt(oos.size());
+ buffer.put(outputStream.toByteArray());
+ buffer.flip();
+ return buffer;
+ }
+
+ public static Replica readReplicaUpdateRequest(ByteBuffer buffer) throws IOException {
+ ByteArrayInputStream bais = new ByteArrayInputStream(buffer.array(), buffer.position(), buffer.limit());
+ DataInputStream dis = new DataInputStream(bais);
+ return Replica.create(dis);
+ }
+
+ public static ReplicaEvent readReplicaEventRequest(ByteBuffer buffer) throws IOException {
+ ByteArrayInputStream bais = new ByteArrayInputStream(buffer.array(), buffer.position(), buffer.limit());
+ DataInputStream dis = new DataInputStream(bais);
+
+ return ReplicaEvent.create(dis);
+ }
+
+ public static void writeGetReplicaFilesRequest(ByteBuffer buffer, ReplicaFilesRequest request) throws IOException {
+ ByteArrayOutputStream outputStream = new ByteArrayOutputStream();
+ DataOutputStream oos = new DataOutputStream(outputStream);
+ request.serialize(oos);
+ oos.close();
+
+ int requestSize = REPLICATION_REQUEST_HEADER_SIZE + oos.size();
+ if (buffer.capacity() < requestSize) {
+ buffer = ByteBuffer.allocate(requestSize);
+ } else {
+ buffer.clear();
+ }
+ buffer.putInt(ReplicationRequestType.GET_REPLICA_FILES.ordinal());
+ buffer.putInt(oos.size());
+ buffer.put(outputStream.toByteArray());
+ buffer.flip();
+ }
+
+ public static ByteBuffer writeGetReplicaIndexFlushRequest(ByteBuffer buffer, ReplicaIndexFlushRequest request)
+ throws IOException {
+ ByteArrayOutputStream outputStream = new ByteArrayOutputStream();
+ DataOutputStream oos = new DataOutputStream(outputStream);
+ request.serialize(oos);
+ oos.close();
+
+ int requestSize = REPLICATION_REQUEST_HEADER_SIZE + oos.size();
+ if (buffer.capacity() < requestSize) {
+ buffer = ByteBuffer.allocate(requestSize);
+ } else {
+ buffer.clear();
+ }
+ buffer.putInt(ReplicationRequestType.FLUSH_INDEX.ordinal());
+ buffer.putInt(oos.size());
+ buffer.put(outputStream.toByteArray());
+ buffer.flip();
+ return buffer;
+ }
+
+ public static ReplicaFilesRequest readReplicaFileRequest(ByteBuffer buffer) throws IOException {
+ ByteArrayInputStream bais = new ByteArrayInputStream(buffer.array(), buffer.position(), buffer.limit());
+ DataInputStream dis = new DataInputStream(bais);
+ return ReplicaFilesRequest.create(dis);
+ }
+
+ public static ReplicaIndexFlushRequest readReplicaIndexFlushRequest(ByteBuffer buffer) throws IOException {
+ ByteArrayInputStream bais = new ByteArrayInputStream(buffer.array(), buffer.position(), buffer.limit());
+ DataInputStream dis = new DataInputStream(bais);
+ return ReplicaIndexFlushRequest.create(dis);
+ }
+
+ public static void writeGetReplicaMaxLSNRequest(ByteBuffer requestBuffer) {
+ requestBuffer.clear();
+ requestBuffer.putInt(ReplicationRequestType.GET_REPLICA_MAX_LSN.ordinal());
+ requestBuffer.flip();
+ }
+
+ public static void writeMinLSNRequest(ByteBuffer requestBuffer) {
+ requestBuffer.clear();
+ requestBuffer.putInt(ReplicationRequestType.GET_REPLICA_MIN_LSN.ordinal());
+ requestBuffer.flip();
+ }
+
+ public static int getJobIdFromLogAckMessage(String msg) {
+ return Integer.parseInt(msg.substring((msg.indexOf(JOB_COMMIT_ACK) + 1)));
+ }
+
+ public static String getNodeIdFromLogAckMessage(String msg) {
+ return msg.substring(0, msg.indexOf(JOB_COMMIT_ACK));
+ }
+
+ /**
+ * Sends a goodbye request to a remote replica indicating the end of a replication request.
+ *
+ * @param socketChannel
+ * the remote replica socket.
+ * @throws IOException
+ */
+ public static void sendGoodbye(SocketChannel socketChannel) throws IOException {
+ ByteBuffer goodbyeBuffer = AsterixReplicationProtocol.getGoodbyeBuffer();
+ NetworkingUtil.transferBufferToChannel(socketChannel, goodbyeBuffer);
+ }
+
+ public static void sendAck(SocketChannel socketChannel) throws IOException {
+ ByteBuffer ackBuffer = AsterixReplicationProtocol.getAckBuffer();
+ NetworkingUtil.transferBufferToChannel(socketChannel, ackBuffer);
+ }
+}
\ No newline at end of file
diff --git a/asterix-replication/src/main/java/org/apache/asterix/replication/functions/ReplicaFilesRequest.java b/asterix-replication/src/main/java/org/apache/asterix/replication/functions/ReplicaFilesRequest.java
new file mode 100644
index 0000000..647a6a3
--- /dev/null
+++ b/asterix-replication/src/main/java/org/apache/asterix/replication/functions/ReplicaFilesRequest.java
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.replication.functions;
+
+import java.io.DataInput;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.HashSet;
+import java.util.Set;
+
+public class ReplicaFilesRequest {
+ Set<String> replicaIds;
+
+ public ReplicaFilesRequest(Set<String> replicaIds) {
+ this.replicaIds = replicaIds;
+ }
+
+ public void serialize(OutputStream out) throws IOException {
+ DataOutputStream dos = new DataOutputStream(out);
+ dos.writeInt(replicaIds.size());
+ for (String replicaId : replicaIds) {
+ dos.writeUTF(replicaId);
+ }
+ }
+
+ public static ReplicaFilesRequest create(DataInput input) throws IOException {
+ int size = input.readInt();
+ Set<String> replicaIds = new HashSet<String>(size);
+ for (int i = 0; i < size; i++) {
+ replicaIds.add(input.readUTF());
+ }
+
+ return new ReplicaFilesRequest(replicaIds);
+ }
+
+ public Set<String> getReplicaIds() {
+ return replicaIds;
+ }
+
+ public void setReplicaIds(Set<String> replicaIds) {
+ this.replicaIds = replicaIds;
+ }
+}
diff --git a/asterix-replication/src/main/java/org/apache/asterix/replication/functions/ReplicaIndexFlushRequest.java b/asterix-replication/src/main/java/org/apache/asterix/replication/functions/ReplicaIndexFlushRequest.java
new file mode 100644
index 0000000..a3c269d
--- /dev/null
+++ b/asterix-replication/src/main/java/org/apache/asterix/replication/functions/ReplicaIndexFlushRequest.java
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.replication.functions;
+
+import java.io.DataInput;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.HashSet;
+import java.util.Set;
+
+public class ReplicaIndexFlushRequest {
+ Set<Long> laggingRescouresIds;
+
+ public ReplicaIndexFlushRequest(Set<Long> laggingRescouresIds) {
+ this.laggingRescouresIds = laggingRescouresIds;
+ }
+
+ public void serialize(OutputStream out) throws IOException {
+ DataOutputStream dos = new DataOutputStream(out);
+ dos.writeInt(laggingRescouresIds.size());
+ for (Long resourceId : laggingRescouresIds) {
+ dos.writeLong(resourceId);
+ }
+ }
+
+ public static ReplicaIndexFlushRequest create(DataInput input) throws IOException {
+ int numOfResources = input.readInt();
+ Set<Long> laggingRescouresIds = new HashSet<Long>(numOfResources);
+ for (int i = 0; i < numOfResources; i++) {
+ laggingRescouresIds.add(input.readLong());
+ }
+ return new ReplicaIndexFlushRequest(laggingRescouresIds);
+ }
+
+ public Set<Long> getLaggingRescouresIds() {
+ return laggingRescouresIds;
+ }
+
+ public void setLaggingRescouresIds(Set<Long> laggingRescouresIds) {
+ this.laggingRescouresIds = laggingRescouresIds;
+ }
+
+}
diff --git a/asterix-replication/src/main/java/org/apache/asterix/replication/functions/ReplicaLogsRequest.java b/asterix-replication/src/main/java/org/apache/asterix/replication/functions/ReplicaLogsRequest.java
new file mode 100644
index 0000000..90df1e7
--- /dev/null
+++ b/asterix-replication/src/main/java/org/apache/asterix/replication/functions/ReplicaLogsRequest.java
@@ -0,0 +1,71 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.replication.functions;
+
+import java.io.DataInput;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.HashSet;
+import java.util.Set;
+
+public class ReplicaLogsRequest {
+ Set<String> replicaIds;
+ long fromLSN;
+
+ public ReplicaLogsRequest(Set<String> replicaIds, long fromLSN) {
+ this.replicaIds = replicaIds;
+ this.fromLSN = fromLSN;
+ }
+
+ public void serialize(OutputStream out) throws IOException {
+ DataOutputStream dos = new DataOutputStream(out);
+ dos.writeInt(replicaIds.size());
+ for (String replicaId : replicaIds) {
+ dos.writeUTF(replicaId);
+ }
+ dos.writeLong(fromLSN);
+ }
+
+ public static ReplicaLogsRequest create(DataInput input) throws IOException {
+ int size = input.readInt();
+ Set<String> replicaIds = new HashSet<String>(size);
+ for (int i = 0; i < size; i++) {
+ replicaIds.add(input.readUTF());
+ }
+ long fromLSN = input.readLong();
+ return new ReplicaLogsRequest(replicaIds, fromLSN);
+ }
+
+ public Set<String> getReplicaIds() {
+ return replicaIds;
+ }
+
+ public void setReplicaIds(Set<String> replicaIds) {
+ this.replicaIds = replicaIds;
+ }
+
+ public long getFromLSN() {
+ return fromLSN;
+ }
+
+ public void setFromLSN(long fromLSN) {
+ this.fromLSN = fromLSN;
+ }
+}
diff --git a/asterix-replication/src/main/java/org/apache/asterix/replication/logging/RemoteLogMapping.java b/asterix-replication/src/main/java/org/apache/asterix/replication/logging/RemoteLogMapping.java
new file mode 100644
index 0000000..20a5fd9
--- /dev/null
+++ b/asterix-replication/src/main/java/org/apache/asterix/replication/logging/RemoteLogMapping.java
@@ -0,0 +1,72 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.replication.logging;
+
+import java.util.concurrent.atomic.AtomicInteger;
+
+public class RemoteLogMapping {
+
+ private String remoteNodeID;
+ private long remoteLSN;
+ private boolean isFlushed = false;
+ private long localLSN;
+ public AtomicInteger numOfFlushedIndexes = new AtomicInteger();
+
+ public boolean isFlushed() {
+ return isFlushed;
+ }
+
+ public void setFlushed(boolean isFlushed) {
+ this.isFlushed = isFlushed;
+ }
+
+ public String getRemoteNodeID() {
+ return remoteNodeID;
+ }
+
+ public void setRemoteNodeID(String remoteNodeID) {
+ this.remoteNodeID = remoteNodeID;
+ }
+
+ public long getRemoteLSN() {
+ return remoteLSN;
+ }
+
+ public void setRemoteLSN(long remoteLSN) {
+ this.remoteLSN = remoteLSN;
+ }
+
+ public long getLocalLSN() {
+ return localLSN;
+ }
+
+ public void setLocalLSN(long localLSN) {
+ this.localLSN = localLSN;
+ }
+
+ @Override
+ public String toString() {
+ StringBuilder sb = new StringBuilder();
+ sb.append("Remote Node: " + remoteNodeID);
+ sb.append(" Remote LSN: " + remoteLSN);
+ sb.append(" Local LSN: " + localLSN);
+ sb.append(" isFlushed : " + isFlushed);
+ return sb.toString();
+ }
+}
\ No newline at end of file
diff --git a/asterix-replication/src/main/java/org/apache/asterix/replication/logging/ReplicationLogBuffer.java b/asterix-replication/src/main/java/org/apache/asterix/replication/logging/ReplicationLogBuffer.java
new file mode 100644
index 0000000..c0af2c8
--- /dev/null
+++ b/asterix-replication/src/main/java/org/apache/asterix/replication/logging/ReplicationLogBuffer.java
@@ -0,0 +1,158 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.replication.logging;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.SocketChannel;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.asterix.common.transactions.ILogRecord;
+import org.apache.asterix.replication.functions.AsterixReplicationProtocol;
+import org.apache.asterix.replication.management.NetworkingUtil;
+import org.apache.asterix.replication.management.ReplicationManager;
+
+public class ReplicationLogBuffer {
+ private final int logBufferSize;
+ private final AtomicBoolean full;
+ private int appendOffset;
+ private int flushOffset;
+ private final ByteBuffer appendBuffer;
+ private final ByteBuffer flushBuffer;
+ private boolean stop;
+ private Map<String, SocketChannel> replicaSockets;
+ private ReplicationManager replicationManager;
+
+ public ReplicationLogBuffer(ReplicationManager replicationManager, int logBufferSize) {
+ this.replicationManager = replicationManager;
+ this.logBufferSize = logBufferSize;
+ appendBuffer = ByteBuffer.allocate(logBufferSize);
+ flushBuffer = appendBuffer.duplicate();
+ full = new AtomicBoolean(false);
+ appendOffset = 0;
+ flushOffset = 0;
+ }
+
+ public void append(ILogRecord logRecord) {
+ appendBuffer.putInt(AsterixReplicationProtocol.ReplicationRequestType.REPLICATE_LOG.ordinal());
+ appendBuffer.putInt(logRecord.getSerializedLogSize());
+ appendBuffer.put(logRecord.getSerializedLog());
+
+ synchronized (this) {
+ appendOffset += getLogReplicationSize(logRecord);
+ this.notify();
+ }
+ }
+
+ public void setReplicationSockets(Map<String, SocketChannel> replicaSockets) {
+ this.replicaSockets = replicaSockets;
+ }
+
+ public synchronized void isFull(boolean full) {
+ this.full.set(full);
+ this.notify();
+ }
+
+ public boolean hasSpace(ILogRecord logRecord) {
+ return appendOffset + getLogReplicationSize(logRecord) <= logBufferSize;
+ }
+
+ private static int getLogReplicationSize(ILogRecord logRecord) {
+ //request type + request length + serialized log length
+ return Integer.BYTES + Integer.BYTES + logRecord.getSerializedLogSize();
+ }
+
+ public void reset() {
+ appendBuffer.position(0);
+ appendBuffer.limit(logBufferSize);
+ flushBuffer.position(0);
+ flushBuffer.limit(logBufferSize);
+ full.set(false);
+ appendOffset = 0;
+ flushOffset = 0;
+ }
+
+ public void flush() {
+ int endOffset;
+ while (!full.get()) {
+ synchronized (this) {
+ if (appendOffset - flushOffset == 0 && !full.get()) {
+ try {
+ if (stop) {
+ break;
+ }
+ this.wait();
+ } catch (InterruptedException e) {
+ continue;
+ }
+ }
+ endOffset = appendOffset;
+ }
+ internalFlush(flushOffset, endOffset);
+ }
+
+ internalFlush(flushOffset, appendOffset);
+ }
+
+ private void internalFlush(int beginOffset, int endOffset) {
+ if (endOffset > beginOffset) {
+ int begingPos = flushBuffer.position();
+ flushBuffer.limit(endOffset);
+ sendRequest(replicaSockets, flushBuffer);
+ flushBuffer.position(begingPos + (endOffset - beginOffset));
+ flushOffset = endOffset;
+ }
+ }
+
+ private void sendRequest(Map<String, SocketChannel> replicaSockets, ByteBuffer requestBuffer) {
+ Iterator<Map.Entry<String, SocketChannel>> iterator = replicaSockets.entrySet().iterator();
+ int begin = requestBuffer.position();
+ while (iterator.hasNext()) {
+ Entry<String, SocketChannel> replicaSocket = iterator.next();
+ SocketChannel clientSocket = replicaSocket.getValue();
+ try {
+ NetworkingUtil.transferBufferToChannel(clientSocket, requestBuffer);
+ } catch (IOException e) {
+ if (clientSocket.isOpen()) {
+ try {
+ clientSocket.close();
+ } catch (IOException e2) {
+ e2.printStackTrace();
+ }
+ }
+ replicationManager.reportFailedReplica(replicaSocket.getKey());
+ iterator.remove();
+ } finally {
+ requestBuffer.position(begin);
+ }
+ }
+
+ }
+
+ public boolean isStop() {
+ return stop;
+ }
+
+ public void isStop(boolean stop) {
+ this.stop = stop;
+ }
+}
diff --git a/asterix-replication/src/main/java/org/apache/asterix/replication/logging/ReplicationLogFlusher.java b/asterix-replication/src/main/java/org/apache/asterix/replication/logging/ReplicationLogFlusher.java
new file mode 100644
index 0000000..8541eaf
--- /dev/null
+++ b/asterix-replication/src/main/java/org/apache/asterix/replication/logging/ReplicationLogFlusher.java
@@ -0,0 +1,104 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.replication.logging;
+
+import java.util.concurrent.Callable;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.logging.Logger;
+
+import org.apache.asterix.common.transactions.LogRecord;
+
+/**
+ * This class is responsible for sending transactions logs to remote replicas.
+ */
+public class ReplicationLogFlusher implements Callable<Boolean> {
+ private final static Logger LOGGER = Logger.getLogger(ReplicationLogFlusher.class.getName());
+ private final static ReplicationLogBuffer POISON_PILL = new ReplicationLogBuffer(null,
+ LogRecord.JOB_TERMINATE_LOG_SIZE);
+ private final LinkedBlockingQueue<ReplicationLogBuffer> emptyQ;
+ private final LinkedBlockingQueue<ReplicationLogBuffer> flushQ;
+ private ReplicationLogBuffer flushPage;
+ private final AtomicBoolean isStarted;
+ private final AtomicBoolean terminateFlag;
+
+ public ReplicationLogFlusher(LinkedBlockingQueue<ReplicationLogBuffer> emptyQ,
+ LinkedBlockingQueue<ReplicationLogBuffer> flushQ) {
+ this.emptyQ = emptyQ;
+ this.flushQ = flushQ;
+ flushPage = null;
+ isStarted = new AtomicBoolean(false);
+ terminateFlag = new AtomicBoolean(false);
+ }
+
+ public void terminate() {
+ //make sure the LogFlusher thread started before terminating it.
+ synchronized (isStarted) {
+ while (!isStarted.get()) {
+ try {
+ isStarted.wait();
+ } catch (InterruptedException e) {
+ //ignore
+ }
+ }
+ }
+
+ terminateFlag.set(true);
+ if (flushPage != null) {
+ synchronized (flushPage) {
+ flushPage.isStop(true);
+ flushPage.notify();
+ }
+ }
+ //[Notice]
+ //The return value doesn't need to be checked
+ //since terminateFlag will trigger termination if the flushQ is full.
+ flushQ.offer(POISON_PILL);
+ }
+
+ @Override
+ public Boolean call() {
+ Thread.currentThread().setName("Replication Log Flusher");
+ synchronized (isStarted) {
+ isStarted.set(true);
+ isStarted.notify();
+ }
+ try {
+ while (true) {
+ flushPage = null;
+ try {
+ flushPage = flushQ.take();
+ if (flushPage == POISON_PILL || terminateFlag.get()) {
+ return true;
+ }
+ } catch (InterruptedException e) {
+ if (flushPage == null) {
+ continue;
+ }
+ }
+ flushPage.flush();
+ emptyQ.offer(flushPage);
+ }
+ } catch (Exception e) {
+ e.printStackTrace();
+ LOGGER.severe("ReplicationLogFlusher is terminating abnormally. Logs Replication Stopped.");
+ throw e;
+ }
+ }
+}
diff --git a/asterix-replication/src/main/java/org/apache/asterix/replication/management/NetworkingUtil.java b/asterix-replication/src/main/java/org/apache/asterix/replication/management/NetworkingUtil.java
new file mode 100644
index 0000000..90291d7
--- /dev/null
+++ b/asterix-replication/src/main/java/org/apache/asterix/replication/management/NetworkingUtil.java
@@ -0,0 +1,110 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.replication.management;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.NetworkInterface;
+import java.net.SocketException;
+import java.nio.ByteBuffer;
+import java.nio.MappedByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.channels.SocketChannel;
+import java.util.Enumeration;
+
+public class NetworkingUtil {
+
+ public static void readBytes(SocketChannel socketChannel, ByteBuffer byteBuffer, int length) throws IOException {
+ byteBuffer.clear();
+ byteBuffer.limit(length);
+
+ while (byteBuffer.remaining() > 0 && socketChannel.read(byteBuffer) > 0);
+
+ if (byteBuffer.remaining() > 0) {
+ throw new EOFException();
+ }
+
+ byteBuffer.flip();
+ }
+
+ public static void sendFile(FileChannel fileChannel, SocketChannel socketChannel) throws IOException {
+ long pos = 0;
+ long fileSize = fileChannel.size();
+ long remainingBytes = fileSize;
+ long transferredBytes = 0;
+
+ while ((transferredBytes += fileChannel.transferTo(pos, remainingBytes, socketChannel)) < fileSize) {
+ pos += transferredBytes;
+ remainingBytes -= transferredBytes;
+ }
+
+ socketChannel.socket().getOutputStream().flush();
+ }
+
+ public static void downloadFile(FileChannel fileChannel, SocketChannel socketChannel) throws IOException {
+ long pos = 0;
+ long fileSize = fileChannel.size();
+ long count = fileSize;
+ long numTransferred = 0;
+ while ((numTransferred += fileChannel.transferFrom(socketChannel, pos, count)) < fileSize) {
+ pos += numTransferred;
+ count -= numTransferred;
+ }
+ }
+
+ public static String getHostAddress(String hostIPAddressFirstOctet) throws SocketException {
+ String hostName = null;
+ Enumeration<NetworkInterface> nInterfaces = NetworkInterface.getNetworkInterfaces();
+ while (nInterfaces.hasMoreElements()) {
+ if (hostName != null) {
+ break;
+ }
+ Enumeration<InetAddress> inetAddresses = nInterfaces.nextElement().getInetAddresses();
+ while (inetAddresses.hasMoreElements()) {
+ String address = inetAddresses.nextElement().getHostAddress();
+ if (address.startsWith(hostIPAddressFirstOctet)) {
+ hostName = address;
+ break;
+ }
+ }
+ }
+ return hostName;
+ }
+
+ public static void transferBufferToChannel(SocketChannel socketChannel, ByteBuffer requestBuffer) throws IOException {
+ while (requestBuffer.hasRemaining()) {
+ socketChannel.write(requestBuffer);
+ }
+ }
+
+ //unused
+ public static void sendFileNIO(FileChannel fileChannel, SocketChannel socketChannel) throws IOException {
+ long fileSize = fileChannel.size();
+ MappedByteBuffer bb = fileChannel.map(FileChannel.MapMode.READ_ONLY, 0, fileSize);
+ socketChannel.write(bb);
+ }
+
+ //unused
+ public static void downloadFileNIO(FileChannel fileChannel, SocketChannel socketChannel) throws IOException {
+ long pos = 0;
+ long fileSize = fileChannel.size();
+ fileChannel.transferFrom(socketChannel, pos, fileSize);
+ }
+}
\ No newline at end of file
diff --git a/asterix-replication/src/main/java/org/apache/asterix/replication/management/ReplicaEventNotifier.java b/asterix-replication/src/main/java/org/apache/asterix/replication/management/ReplicaEventNotifier.java
new file mode 100644
index 0000000..633d87a
--- /dev/null
+++ b/asterix-replication/src/main/java/org/apache/asterix/replication/management/ReplicaEventNotifier.java
@@ -0,0 +1,109 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.replication.management;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.nio.ByteBuffer;
+import java.nio.channels.SocketChannel;
+import java.nio.channels.UnresolvedAddressException;
+import java.util.Set;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import org.apache.asterix.common.config.AsterixReplicationProperties;
+import org.apache.asterix.common.replication.Replica;
+import org.apache.asterix.common.replication.ReplicaEvent;
+import org.apache.asterix.replication.functions.AsterixReplicationProtocol;
+
+public class ReplicaEventNotifier implements Runnable {
+
+ private static final Logger LOGGER = Logger.getLogger(ReplicaEventNotifier.class.getName());
+
+ final int WAIT_TIME = 2000;
+ final Set<Replica> notifyReplicaNodes;
+
+ int notificationTimeOut;
+
+ final ReplicaEvent event;
+ final AsterixReplicationProperties asterixReplicationProperties;
+
+ public ReplicaEventNotifier(ReplicaEvent event, AsterixReplicationProperties asterixReplicationProperties) {
+ this.event = event;
+ this.asterixReplicationProperties = asterixReplicationProperties;
+ notificationTimeOut = asterixReplicationProperties.getReplicationTimeOut();
+ notifyReplicaNodes = asterixReplicationProperties.getRemoteReplicas(event.getReplica().getId());
+ }
+
+ @Override
+ public void run() {
+ Thread.currentThread().setName("ReplicaEventNotifier Thread");
+
+ if (notifyReplicaNodes == null) {
+ return;
+ }
+
+ ByteBuffer buffer = null;
+ try {
+ buffer = AsterixReplicationProtocol.writeReplicaEventRequest(event);
+ } catch (IOException e) {
+ e.printStackTrace();
+ }
+
+ for (Replica replica : notifyReplicaNodes) {
+ long startTime = System.currentTimeMillis();
+ InetSocketAddress replicaAddress = replica.getAddress(asterixReplicationProperties);
+ SocketChannel connection = null;
+
+ while (true) {
+ try {
+ connection = SocketChannel.open();
+ connection.configureBlocking(true);
+ connection.connect(new InetSocketAddress(replicaAddress.getHostString(), replicaAddress.getPort()));
+ //send replica event
+ connection.write(buffer);
+ //send goodbye
+ connection.write(AsterixReplicationProtocol.getGoodbyeBuffer());
+ break;
+ } catch (IOException | UnresolvedAddressException e) {
+ try {
+ Thread.sleep(WAIT_TIME);
+ } catch (InterruptedException e1) {
+ //ignore
+ }
+
+ //check if connection to replica timed out
+ if (((System.currentTimeMillis() - startTime) / 1000) >= notificationTimeOut) {
+ LOGGER.log(Level.WARNING, "Could not send ReplicaEvent to " + replica);
+ break;
+ }
+ } finally {
+ if (connection.isOpen()) {
+ try {
+ connection.close();
+ } catch (IOException e) {
+ e.printStackTrace();
+ }
+ }
+ buffer.position(0);
+ }
+ }
+ }
+ }
+}
diff --git a/asterix-replication/src/main/java/org/apache/asterix/replication/management/ReplicaStateChecker.java b/asterix-replication/src/main/java/org/apache/asterix/replication/management/ReplicaStateChecker.java
new file mode 100644
index 0000000..07ed144
--- /dev/null
+++ b/asterix-replication/src/main/java/org/apache/asterix/replication/management/ReplicaStateChecker.java
@@ -0,0 +1,86 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.replication.management;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.nio.ByteBuffer;
+import java.nio.channels.SocketChannel;
+import java.nio.channels.UnresolvedAddressException;
+import java.util.concurrent.Callable;
+
+import org.apache.asterix.common.config.AsterixReplicationProperties;
+import org.apache.asterix.common.replication.Replica;
+import org.apache.asterix.common.replication.Replica.ReplicaState;
+import org.apache.asterix.replication.functions.AsterixReplicationProtocol;
+
+public class ReplicaStateChecker implements Callable<Void> {
+
+ private final int WAIT_TIME = 2000;
+ private final Replica replica;
+ private final int replicationTimeOut;
+ private final ReplicationManager replicationManager;
+ private final AsterixReplicationProperties asterixReplicationProperties;
+ private final boolean suspendReplication;
+
+ public ReplicaStateChecker(Replica replica, int replicationTimeOut, ReplicationManager replicationManager,
+ AsterixReplicationProperties asterixReplicationProperties, boolean suspendReplication) {
+ this.replica = replica;
+ this.replicationTimeOut = replicationTimeOut;
+ this.replicationManager = replicationManager;
+ this.asterixReplicationProperties = asterixReplicationProperties;
+ this.suspendReplication = suspendReplication;
+ }
+
+ @Override
+ public Void call() throws Exception {
+ Thread.currentThread().setName("ReplicaConnector Thread");
+
+ long startTime = System.currentTimeMillis();
+ InetSocketAddress replicaAddress = replica.getAddress(asterixReplicationProperties);
+ SocketChannel connection = null;
+
+ while (true) {
+ try {
+ connection = SocketChannel.open();
+ connection.configureBlocking(true);
+ connection.connect(new InetSocketAddress(replicaAddress.getHostString(), replicaAddress.getPort()));
+ ByteBuffer buffer = AsterixReplicationProtocol.getGoodbyeBuffer();
+ connection.write(buffer);
+ replicationManager.updateReplicaState(replica.getId(), ReplicaState.ACTIVE, suspendReplication);
+ return null;
+ } catch (IOException | UnresolvedAddressException e) {
+ Thread.sleep(WAIT_TIME);
+
+ //check if connection to replica timed out
+ if (((System.currentTimeMillis() - startTime) / 1000) >= replicationTimeOut) {
+ replicationManager.updateReplicaState(replica.getId(), ReplicaState.DEAD, suspendReplication);
+ return null;
+ } else {
+ continue;
+ }
+ } finally {
+ if (connection.isOpen()) {
+ connection.close();
+ }
+ }
+ }
+ }
+
+}
diff --git a/asterix-replication/src/main/java/org/apache/asterix/replication/management/ReplicationChannel.java b/asterix-replication/src/main/java/org/apache/asterix/replication/management/ReplicationChannel.java
new file mode 100644
index 0000000..5a88626
--- /dev/null
+++ b/asterix-replication/src/main/java/org/apache/asterix/replication/management/ReplicationChannel.java
@@ -0,0 +1,640 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.replication.management;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.channels.ServerSocketChannel;
+import java.nio.channels.SocketChannel;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import org.apache.asterix.common.api.IDatasetLifecycleManager;
+import org.apache.asterix.common.config.AsterixReplicationProperties;
+import org.apache.asterix.common.context.DatasetLifecycleManager.IndexInfo;
+import org.apache.asterix.common.exceptions.ACIDException;
+import org.apache.asterix.common.ioopcallbacks.AbstractLSMIOOperationCallback;
+import org.apache.asterix.common.replication.IReplicaResourcesManager;
+import org.apache.asterix.common.replication.IReplicationChannel;
+import org.apache.asterix.common.replication.IReplicationManager;
+import org.apache.asterix.common.replication.IReplicationThread;
+import org.apache.asterix.common.replication.Replica;
+import org.apache.asterix.common.replication.ReplicaEvent;
+import org.apache.asterix.common.transactions.IAsterixAppRuntimeContextProvider;
+import org.apache.asterix.common.transactions.ILogManager;
+import org.apache.asterix.common.transactions.ILogReader;
+import org.apache.asterix.common.transactions.ILogRecord;
+import org.apache.asterix.common.transactions.LogRecord;
+import org.apache.asterix.common.transactions.LogSource;
+import org.apache.asterix.common.transactions.LogType;
+import org.apache.asterix.replication.functions.AsterixReplicationProtocol;
+import org.apache.asterix.replication.functions.AsterixReplicationProtocol.ReplicationRequestType;
+import org.apache.asterix.replication.functions.ReplicaFilesRequest;
+import org.apache.asterix.replication.functions.ReplicaIndexFlushRequest;
+import org.apache.asterix.replication.functions.ReplicaLogsRequest;
+import org.apache.asterix.replication.logging.RemoteLogMapping;
+import org.apache.asterix.replication.storage.AsterixLSMIndexFileProperties;
+import org.apache.asterix.replication.storage.LSMComponentLSNSyncTask;
+import org.apache.asterix.replication.storage.LSMComponentProperties;
+import org.apache.asterix.replication.storage.ReplicaResourcesManager;
+import org.apache.hyracks.api.application.INCApplicationContext;
+import org.apache.hyracks.storage.am.lsm.common.api.LSMOperationType;
+import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndex;
+
+/**
+ * This class is used to receive and process replication requests from remote replicas or replica events from CC
+ */
+public class ReplicationChannel extends Thread implements IReplicationChannel {
+
+ private static final Logger LOGGER = Logger.getLogger(ReplicationChannel.class.getName());
+ private final ExecutorService replicationThreads;
+ private final String localNodeID;
+ private final ILogManager logManager;
+ private final ReplicaResourcesManager replicaResourcesManager;
+ private SocketChannel socketChannel = null;
+ private ServerSocketChannel serverSocketChannel = null;
+ private final IReplicationManager replicationManager;
+ private final AsterixReplicationProperties replicationProperties;
+ private final IAsterixAppRuntimeContextProvider asterixAppRuntimeContextProvider;
+ private final static int INTIAL_BUFFER_SIZE = 4000; //4KB
+ private final LinkedBlockingQueue<LSMComponentLSNSyncTask> lsmComponentRemoteLSN2LocalLSNMappingTaskQ;
+ private final Map<String, LSMComponentProperties> lsmComponentId2PropertiesMap;
+ private final Map<Long, RemoteLogMapping> localLSN2RemoteLSNMap;
+ private final LSMComponentsSyncService lsmComponentLSNMappingService;
+
+ public ReplicationChannel(String nodeId, AsterixReplicationProperties replicationProperties, ILogManager logManager,
+ IReplicaResourcesManager replicaResoucesManager, IReplicationManager replicationManager,
+ INCApplicationContext appContext, IAsterixAppRuntimeContextProvider asterixAppRuntimeContextProvider) {
+ this.logManager = logManager;
+ this.localNodeID = nodeId;
+ this.replicaResourcesManager = (ReplicaResourcesManager) replicaResoucesManager;
+ this.replicationManager = replicationManager;
+ this.replicationProperties = replicationProperties;
+ this.asterixAppRuntimeContextProvider = asterixAppRuntimeContextProvider;
+ lsmComponentRemoteLSN2LocalLSNMappingTaskQ = new LinkedBlockingQueue<LSMComponentLSNSyncTask>();
+ lsmComponentId2PropertiesMap = new ConcurrentHashMap<String, LSMComponentProperties>();
+ localLSN2RemoteLSNMap = new ConcurrentHashMap<Long, RemoteLogMapping>();
+ lsmComponentLSNMappingService = new LSMComponentsSyncService();
+ replicationThreads = Executors.newCachedThreadPool(appContext.getThreadFactory());
+ }
+
+ @Override
+ public void run() {
+ Thread.currentThread().setName("Replication Channel Thread");
+
+ String nodeIP = replicationProperties.getReplicaIPAddress(localNodeID);
+ int dataPort = replicationProperties.getDataReplicationPort(localNodeID);
+ try {
+ serverSocketChannel = ServerSocketChannel.open();
+ serverSocketChannel.configureBlocking(true);
+ InetSocketAddress replicationChannelAddress = new InetSocketAddress(InetAddress.getByName(nodeIP),
+ dataPort);
+ serverSocketChannel.socket().bind(replicationChannelAddress);
+ lsmComponentLSNMappingService.start();
+
+ LOGGER.log(Level.INFO, "opened Replication Channel @ IP Address: " + nodeIP + ":" + dataPort);
+
+ //start accepting replication requests
+ while (true) {
+ socketChannel = serverSocketChannel.accept();
+ socketChannel.configureBlocking(true);
+ //start a new thread to handle the request
+ replicationThreads.execute(new ReplicationThread(socketChannel));
+ }
+ } catch (IOException e) {
+ throw new IllegalStateException(
+ "Could not opened replication channel @ IP Address: " + nodeIP + ":" + dataPort, e);
+ }
+ }
+
+ private void updateLSMComponentRemainingFiles(String lsmComponentId) throws IOException {
+ LSMComponentProperties lsmCompProp = lsmComponentId2PropertiesMap.get(lsmComponentId);
+ int remainingFile = lsmCompProp.markFileComplete();
+
+ //clean up when all the LSM component files have been received.
+ if (remainingFile == 0) {
+ if (lsmCompProp.getOpType() == LSMOperationType.FLUSH && lsmCompProp.getReplicaLSN() != null) {
+ //if this LSN wont be used for any other index, remove it
+ if (localLSN2RemoteLSNMap.containsKey(lsmCompProp.getReplicaLSN())) {
+ int remainingIndexes = localLSN2RemoteLSNMap.get(lsmCompProp.getReplicaLSN()).numOfFlushedIndexes
+ .decrementAndGet();
+ if (remainingIndexes == 0) {
+ //Note: there is a chance that this is never deleted because some index in the dataset was not flushed because it is empty.
+ //This could be solved by passing only the number of successfully flushed indexes
+ localLSN2RemoteLSNMap.remove(lsmCompProp.getReplicaLSN());
+ }
+ }
+ }
+
+ //delete mask to indicate that this component is now valid.
+ replicaResourcesManager.markLSMComponentReplicaAsValid(lsmCompProp);
+ lsmComponentId2PropertiesMap.remove(lsmComponentId);
+ LOGGER.log(Level.INFO, "Completed LSMComponent " + lsmComponentId + " Replication.");
+ }
+ }
+
+ /**
+ * @param replicaId
+ * the remote replica id this log belongs to.
+ * @param remoteLSN
+ * the remote LSN received from the remote replica.
+ * @return The local log mapping if found. Otherwise null.
+ */
+ private RemoteLogMapping getRemoteLogMapping(String replicaId, long remoteLSN) {
+ for (RemoteLogMapping mapping : localLSN2RemoteLSNMap.values()) {
+ if (mapping.getRemoteLSN() == remoteLSN && mapping.getRemoteNodeID().equals(replicaId)) {
+ return mapping;
+ }
+ }
+ return null;
+ }
+
+ @Override
+ public void close() throws IOException {
+ if (!serverSocketChannel.isOpen()) {
+ serverSocketChannel.close();
+ LOGGER.log(Level.INFO, "Replication channel closed.");
+ }
+ }
+
+ /**
+ * A replication thread is created per received replication request.
+ */
+ private class ReplicationThread implements IReplicationThread {
+ private final SocketChannel socketChannel;
+ private final LogRecord remoteLog;
+ private ByteBuffer inBuffer;
+ private ByteBuffer outBuffer;
+
+ public ReplicationThread(SocketChannel socketChannel) {
+ this.socketChannel = socketChannel;
+ inBuffer = ByteBuffer.allocate(INTIAL_BUFFER_SIZE);
+ outBuffer = ByteBuffer.allocate(INTIAL_BUFFER_SIZE);
+ remoteLog = new LogRecord();
+ }
+
+ @Override
+ public void run() {
+ Thread.currentThread().setName("Replication Thread");
+ try {
+ ReplicationRequestType replicationFunction = AsterixReplicationProtocol.getRequestType(socketChannel,
+ inBuffer);
+ while (replicationFunction != ReplicationRequestType.GOODBYE) {
+ switch (replicationFunction) {
+ case REPLICATE_LOG:
+ handleLogReplication();
+ break;
+ case LSM_COMPONENT_PROPERTIES:
+ handleLSMComponentProperties();
+ break;
+ case REPLICATE_FILE:
+ handleReplicateFile();
+ break;
+ case DELETE_FILE:
+ handleDeleteFile();
+ break;
+ case REPLICA_EVENT:
+ handleReplicaEvent();
+ break;
+ case UPDATE_REPLICA:
+ handleUpdateReplica();
+ break;
+ case GET_REPLICA_MAX_LSN:
+ handleGetReplicaMaxLSN();
+ break;
+ case GET_REPLICA_MIN_LSN:
+ handleGetReplicaMinLSN();
+ break;
+ case GET_REPLICA_FILES:
+ handleGetReplicaFiles();
+ break;
+ case GET_REPLICA_LOGS:
+ handleGetRemoteLogs();
+ break;
+ case FLUSH_INDEX:
+ handleFlushIndex();
+ break;
+ default: {
+ throw new IllegalStateException("Unknown replication request");
+ }
+ }
+ replicationFunction = AsterixReplicationProtocol.getRequestType(socketChannel, inBuffer);
+ }
+ } catch (Exception e) {
+ e.printStackTrace();
+ } finally {
+ if (socketChannel.isOpen()) {
+ try {
+ socketChannel.close();
+ } catch (IOException e) {
+ e.printStackTrace();
+ }
+ }
+ }
+ }
+
+ private void handleFlushIndex() throws IOException {
+ inBuffer = AsterixReplicationProtocol.readRequest(socketChannel, inBuffer);
+ //1. read which indexes are requested to be flushed from remote replica
+ ReplicaIndexFlushRequest request = AsterixReplicationProtocol.readReplicaIndexFlushRequest(inBuffer);
+ Set<Long> requestedIndexesToBeFlushed = request.getLaggingRescouresIds();
+
+ //2. check which indexes can be flushed (open indexes) and which cannot be flushed (closed or have empty memory component)
+ IDatasetLifecycleManager datasetLifeCycleManager = asterixAppRuntimeContextProvider.getDatasetLifecycleManager();
+ List<IndexInfo> openIndexesInfo = datasetLifeCycleManager.getOpenIndexesInfo();
+ Set<Integer> datasetsToForceFlush = new HashSet<Integer>();
+ for (IndexInfo iInfo : openIndexesInfo) {
+ if (requestedIndexesToBeFlushed.contains(iInfo.getResourceId())) {
+ AbstractLSMIOOperationCallback ioCallback = (AbstractLSMIOOperationCallback) iInfo.getIndex()
+ .getIOOperationCallback();
+ //if an index has a pending flush, then the request to flush it will succeed.
+ if (ioCallback.hasPendingFlush()) {
+ //remove index to indicate that it will be flushed
+ requestedIndexesToBeFlushed.remove(iInfo.getResourceId());
+ } else if (!((AbstractLSMIndex) iInfo.getIndex()).isCurrentMutableComponentEmpty()) {
+ //if an index has something to be flushed, then the request to flush it will succeed and we need to schedule it to be flushed.
+ datasetsToForceFlush.add(iInfo.getDatasetId());
+ //remove index to indicate that it will be flushed
+ requestedIndexesToBeFlushed.remove(iInfo.getResourceId());
+ }
+ }
+ }
+
+ //3. force flush datasets requested to be flushed
+ for (int datasetId : datasetsToForceFlush) {
+ datasetLifeCycleManager.flushDataset(datasetId, true);
+ }
+
+ //the remaining indexes in the requested set are those which cannot be flushed.
+ //4. respond back to the requester that those indexes cannot be flushed
+ ReplicaIndexFlushRequest laggingIndexesResponse = new ReplicaIndexFlushRequest(requestedIndexesToBeFlushed);
+ outBuffer = AsterixReplicationProtocol.writeGetReplicaIndexFlushRequest(outBuffer, laggingIndexesResponse);
+ NetworkingUtil.transferBufferToChannel(socketChannel, outBuffer);
+ }
+
+ private void handleLSMComponentProperties() throws IOException {
+ inBuffer = AsterixReplicationProtocol.readRequest(socketChannel, inBuffer);
+ LSMComponentProperties lsmCompProp = AsterixReplicationProtocol.readLSMPropertiesRequest(inBuffer);
+ //create mask to indicate that this component is not valid yet
+ replicaResourcesManager.createRemoteLSMComponentMask(lsmCompProp);
+ lsmComponentId2PropertiesMap.put(lsmCompProp.getComponentId(), lsmCompProp);
+ }
+
+ private void handleReplicateFile() throws IOException {
+ inBuffer = AsterixReplicationProtocol.readRequest(socketChannel, inBuffer);
+ AsterixLSMIndexFileProperties afp = AsterixReplicationProtocol.readFileReplicationRequest(inBuffer);
+
+ String replicaFolderPath = replicaResourcesManager.getIndexPath(afp.getNodeId(), afp.getIoDeviceNum(),
+ afp.getDataverse(), afp.getIdxName());
+
+ String replicaFilePath = replicaFolderPath + File.separator + afp.getFileName();
+
+ //create file
+ File destFile = new File(replicaFilePath);
+ destFile.createNewFile();
+
+ try (RandomAccessFile fileOutputStream = new RandomAccessFile(destFile, "rw");
+ FileChannel fileChannel = fileOutputStream.getChannel()) {
+ fileOutputStream.setLength(afp.getFileSize());
+ NetworkingUtil.downloadFile(fileChannel, socketChannel);
+ fileChannel.force(true);
+
+ if (afp.requiresAck()) {
+ AsterixReplicationProtocol.sendAck(socketChannel);
+ }
+ if (afp.isLSMComponentFile()) {
+ String compoentId = LSMComponentProperties.getLSMComponentID(afp.getFilePath(), afp.getNodeId());
+ if (afp.isRequireLSNSync()) {
+ LSMComponentLSNSyncTask syncTask = new LSMComponentLSNSyncTask(compoentId,
+ destFile.getAbsolutePath());
+ lsmComponentRemoteLSN2LocalLSNMappingTaskQ.offer(syncTask);
+ } else {
+ updateLSMComponentRemainingFiles(compoentId);
+ }
+ } else {
+ //index metadata file
+ replicaResourcesManager.initializeReplicaIndexLSNMap(replicaFolderPath, logManager.getAppendLSN());
+ }
+ }
+ }
+
+ private void handleGetReplicaMaxLSN() throws IOException {
+ long maxLNS = logManager.getAppendLSN();
+ outBuffer.clear();
+ outBuffer.putLong(maxLNS);
+ outBuffer.flip();
+ NetworkingUtil.transferBufferToChannel(socketChannel, outBuffer);
+ }
+
+ private void handleGetReplicaMinLSN() throws IOException {
+ long minLSN = asterixAppRuntimeContextProvider.getAppContext().getTransactionSubsystem()
+ .getRecoveryManager().getMinFirstLSN();
+ outBuffer.clear();
+ outBuffer.putLong(minLSN);
+ outBuffer.flip();
+ NetworkingUtil.transferBufferToChannel(socketChannel, outBuffer);
+ }
+
+ private void handleGetReplicaFiles() throws IOException {
+ inBuffer = AsterixReplicationProtocol.readRequest(socketChannel, inBuffer);
+ ReplicaFilesRequest request = AsterixReplicationProtocol.readReplicaFileRequest(inBuffer);
+
+ AsterixLSMIndexFileProperties fileProperties = new AsterixLSMIndexFileProperties();
+
+ List<String> filesList;
+ Set<String> replicaIds = request.getReplicaIds();
+
+ for (String replicaId : replicaIds) {
+ filesList = replicaResourcesManager.getResourcesForReplica(replicaId);
+
+ //start sending files
+ for (String filePath : filesList) {
+ try (RandomAccessFile fromFile = new RandomAccessFile(filePath, "r");
+ FileChannel fileChannel = fromFile.getChannel();) {
+ long fileSize = fileChannel.size();
+ fileProperties.initialize(filePath, fileSize, replicaId, false, false, false);
+
+ outBuffer = AsterixReplicationProtocol.writeFileReplicationRequest(outBuffer, fileProperties,
+ ReplicationRequestType.REPLICATE_FILE);
+
+ //send file info
+ NetworkingUtil.transferBufferToChannel(socketChannel, outBuffer);
+
+ //transfer file
+ NetworkingUtil.sendFile(fileChannel, socketChannel);
+ }
+ }
+ }
+
+ //send goodbye (end of files)
+ AsterixReplicationProtocol.sendGoodbye(socketChannel);
+ }
+
+ private void handleGetRemoteLogs() throws IOException, ACIDException {
+ inBuffer = AsterixReplicationProtocol.readRequest(socketChannel, inBuffer);
+ ReplicaLogsRequest request = AsterixReplicationProtocol.readReplicaLogsRequest(inBuffer);
+
+ Set<String> replicaIds = request.getReplicaIds();
+ long fromLSN = request.getFromLSN();
+ long minLocalFirstLSN = asterixAppRuntimeContextProvider.getAppContext().getTransactionSubsystem()
+ .getRecoveryManager().getLocalMinFirstLSN();
+
+ //get Log read
+ ILogReader logReader = logManager.getLogReader(true);
+ try {
+ if (fromLSN < logManager.getReadableSmallestLSN()) {
+ fromLSN = logManager.getReadableSmallestLSN();
+ }
+
+ logReader.initializeScan(fromLSN);
+ ILogRecord logRecord = logReader.next();
+ while (logRecord != null) {
+ //we should not send any local log which has already been converted to disk component
+ if (logRecord.getLogSource() == LogSource.LOCAL && logRecord.getLSN() < minLocalFirstLSN) {
+ logRecord = logReader.next();
+ continue;
+ }
+
+ //since flush logs are not required for recovery, skip them
+ if (replicaIds.contains(logRecord.getNodeId()) && logRecord.getLogType() != LogType.FLUSH) {
+ if (logRecord.getSerializedLogSize() > outBuffer.capacity()) {
+ int requestSize = logRecord.getSerializedLogSize()
+ + AsterixReplicationProtocol.REPLICATION_REQUEST_HEADER_SIZE;
+ outBuffer = ByteBuffer.allocate(requestSize);
+ }
+
+ //set log source to REMOTE_RECOVERY to avoid re-logging on the recipient side
+ logRecord.setLogSource(LogSource.REMOTE_RECOVERY);
+ AsterixReplicationProtocol.writeReplicateLogRequest(outBuffer, logRecord);
+ NetworkingUtil.transferBufferToChannel(socketChannel, outBuffer);
+ }
+ logRecord = logReader.next();
+ }
+ } finally {
+ logReader.close();
+ }
+
+ //send goodbye (end of logs)
+ AsterixReplicationProtocol.sendGoodbye(socketChannel);
+ }
+
+ private void handleUpdateReplica() throws IOException {
+ inBuffer = AsterixReplicationProtocol.readRequest(socketChannel, inBuffer);
+ Replica replica = AsterixReplicationProtocol.readReplicaUpdateRequest(inBuffer);
+ replicationManager.updateReplicaInfo(replica);
+ }
+
+ private void handleReplicaEvent() throws IOException {
+ inBuffer = AsterixReplicationProtocol.readRequest(socketChannel, inBuffer);
+ ReplicaEvent event = AsterixReplicationProtocol.readReplicaEventRequest(inBuffer);
+ replicationManager.reportReplicaEvent(event);
+ }
+
+ private void handleDeleteFile() throws IOException {
+ inBuffer = AsterixReplicationProtocol.readRequest(socketChannel, inBuffer);
+ AsterixLSMIndexFileProperties fileProp = AsterixReplicationProtocol.readFileReplicationRequest(inBuffer);
+ replicaResourcesManager.deleteRemoteFile(fileProp);
+ if (fileProp.requiresAck()) {
+ AsterixReplicationProtocol.sendAck(socketChannel);
+ }
+ }
+
+ private void handleLogReplication() throws IOException, ACIDException {
+ inBuffer = AsterixReplicationProtocol.readRequest(socketChannel, inBuffer);
+
+ //Deserialize log
+ remoteLog.deserialize(inBuffer, false, localNodeID);
+ remoteLog.setLogSource(LogSource.REMOTE);
+
+ if (remoteLog.getLogType() == LogType.JOB_COMMIT) {
+ LogRecord jobCommitLog = new LogRecord();
+ jobCommitLog.formJobTerminateLogRecord(remoteLog.getJobId(), true, remoteLog.getNodeId());
+ jobCommitLog.setReplicationThread(this);
+ jobCommitLog.setLogSource(LogSource.REMOTE);
+ logManager.log(jobCommitLog);
+ } else if (remoteLog.getLogType() == LogType.FLUSH) {
+ LogRecord flushLog = new LogRecord();
+ flushLog.formFlushLogRecord(remoteLog.getDatasetId(), null, remoteLog.getNodeId(),
+ remoteLog.getNumOfFlushedIndexes());
+ flushLog.setReplicationThread(this);
+ flushLog.setLogSource(LogSource.REMOTE);
+ synchronized (localLSN2RemoteLSNMap) {
+ logManager.log(flushLog);
+
+ //store mapping information for flush logs to use them in incoming LSM components.
+ RemoteLogMapping flushLogMap = new RemoteLogMapping();
+ flushLogMap.setRemoteLSN(remoteLog.getLSN());
+ flushLogMap.setRemoteNodeID(remoteLog.getNodeId());
+ flushLogMap.setLocalLSN(flushLog.getLSN());
+ flushLogMap.numOfFlushedIndexes.set(remoteLog.getNumOfFlushedIndexes());
+ localLSN2RemoteLSNMap.put(flushLog.getLSN(), flushLogMap);
+ localLSN2RemoteLSNMap.notifyAll();
+ }
+ } else {
+ //send log to LogManager as a remote log
+ logManager.log(remoteLog);
+ }
+ }
+
+ //this method is called sequentially by LogPage (notifyReplicationTerminator) for JOB_COMMIT and FLUSH log types
+ @Override
+ public void notifyLogReplicationRequester(LogRecord logRecord) {
+ //Note: this could be optimized by moving this to a different thread and freeing the LogPage thread faster
+ if (logRecord.getLogType() == LogType.JOB_COMMIT) {
+ //send ACK to requester
+ try {
+ socketChannel.socket().getOutputStream().write(
+ (localNodeID + AsterixReplicationProtocol.JOB_COMMIT_ACK + logRecord.getJobId() + "\n")
+ .getBytes());
+ socketChannel.socket().getOutputStream().flush();
+ } catch (IOException e) {
+ e.printStackTrace();
+ }
+ } else if (logRecord.getLogType() == LogType.FLUSH) {
+ synchronized (localLSN2RemoteLSNMap) {
+ RemoteLogMapping remoteLogMap = localLSN2RemoteLSNMap.get(logRecord.getLSN());
+ synchronized (remoteLogMap) {
+ remoteLogMap.setFlushed(true);
+ remoteLogMap.notifyAll();
+ }
+ }
+ }
+ }
+ }
+
+ /**
+ * This thread is responsible for synchronizing the LSN of the received LSM components to a local LSN.
+ */
+ private class LSMComponentsSyncService extends Thread {
+ @Override
+ public void run() {
+ Thread.currentThread().setName("LSMComponentsSyncService Thread");
+
+ while (true) {
+ try {
+ LSMComponentLSNSyncTask syncTask = lsmComponentRemoteLSN2LocalLSNMappingTaskQ.take();
+ LSMComponentProperties lsmCompProp = lsmComponentId2PropertiesMap.get(syncTask.getComponentId());
+ try {
+ syncLSMComponentFlushLSN(lsmCompProp, syncTask.getComponentFilePath());
+ updateLSMComponentRemainingFiles(lsmCompProp.getComponentId());
+ } catch (Exception e) {
+ e.printStackTrace();
+ }
+ } catch (InterruptedException e) {
+ //ignore
+ }
+ }
+ }
+
+ private void syncLSMComponentFlushLSN(LSMComponentProperties lsmCompProp, String filePath) throws Exception {
+ long remoteLSN = lsmCompProp.getOriginalLSN();
+ //LSN=0 (bulkload) does not need to be updated and there is no flush log corresponding to it
+ if (remoteLSN == 0) {
+ //since this is the first LSM component of this index,
+ //then set the mapping in the LSN_MAP to the current log LSN because
+ //no other log could've been received for this index since bulkload replication is synchronous.
+ lsmCompProp.setReplicaLSN(logManager.getAppendLSN());
+ return;
+ }
+
+ if (lsmCompProp.getReplicaLSN() == null) {
+ if (lsmCompProp.getOpType() == LSMOperationType.FLUSH) {
+ //need to look up LSN mapping from memory
+ RemoteLogMapping remoteLogMap = getRemoteLogMapping(lsmCompProp.getNodeId(), remoteLSN);
+
+ //wait until flush log arrives
+ while (remoteLogMap == null) {
+ synchronized (localLSN2RemoteLSNMap) {
+ localLSN2RemoteLSNMap.wait();
+ }
+ remoteLogMap = getRemoteLogMapping(lsmCompProp.getNodeId(), remoteLSN);
+ }
+
+ //wait until the log is flushed locally before updating the disk component LSN
+ synchronized (remoteLogMap) {
+ while (!remoteLogMap.isFlushed()) {
+ remoteLogMap.wait();
+ }
+ }
+
+ lsmCompProp.setReplicaLSN(remoteLogMap.getLocalLSN());
+ } else if (lsmCompProp.getOpType() == LSMOperationType.MERGE) {
+ //need to load the LSN mapping from disk
+ Map<Long, Long> lsmMap = replicaResourcesManager
+ .getReplicaIndexLSNMap(lsmCompProp.getReplicaComponentPath(replicaResourcesManager));
+ Long mappingLSN = lsmMap.get(lsmCompProp.getOriginalLSN());
+ if (mappingLSN == null) {
+ /*
+ * this shouldn't happen unless this node just recovered and the first component it received
+ * is a merged component due to an on-going merge operation while recovery on the remote replica.
+ * In this case, we use the current append LSN since no new records exist for this index,
+ * otherwise they would've been flushed.
+ * This could be prevented by waiting for any IO to finish on the remote replica during recovery.
+ *
+ */
+ mappingLSN = logManager.getAppendLSN();
+ } else {
+ lsmCompProp.setReplicaLSN(mappingLSN);
+ }
+ }
+ }
+
+ Path path = Paths.get(filePath);
+ if (Files.notExists(path)) {
+ /*
+ * This could happen when a merged component arrives and deletes the flushed
+ * component (which we are trying to update) before its flush log arrives since logs and components are received
+ * on different threads.
+ */
+ return;
+ }
+
+ File destFile = new File(filePath);
+ ByteBuffer metadataBuffer = ByteBuffer.allocate(Long.BYTES);
+ metadataBuffer.putLong(lsmCompProp.getReplicaLSN());
+ metadataBuffer.flip();
+
+ //replace the remote LSN value by the local one
+ try (RandomAccessFile fileOutputStream = new RandomAccessFile(destFile, "rw");
+ FileChannel fileChannel = fileOutputStream.getChannel()) {
+ while (metadataBuffer.hasRemaining()) {
+ fileChannel.write(metadataBuffer, lsmCompProp.getLSNOffset());
+ }
+ fileChannel.force(true);
+ }
+ }
+ }
+
+}
\ No newline at end of file
diff --git a/asterix-replication/src/main/java/org/apache/asterix/replication/management/ReplicationLifecycleListener.java b/asterix-replication/src/main/java/org/apache/asterix/replication/management/ReplicationLifecycleListener.java
new file mode 100644
index 0000000..39130a4
--- /dev/null
+++ b/asterix-replication/src/main/java/org/apache/asterix/replication/management/ReplicationLifecycleListener.java
@@ -0,0 +1,77 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.replication.management;
+
+import java.util.Set;
+
+import org.apache.asterix.common.api.IClusterEventsSubscriber;
+import org.apache.asterix.common.api.IClusterManagementWork;
+import org.apache.asterix.common.api.IClusterManagementWork.ClusterState;
+import org.apache.asterix.common.api.IClusterManagementWorkResponse;
+import org.apache.asterix.common.config.AsterixReplicationProperties;
+import org.apache.asterix.common.replication.Replica;
+import org.apache.asterix.common.replication.ReplicaEvent;
+import org.apache.asterix.common.replication.ReplicaEvent.ReplicaEventType;
+
+public class ReplicationLifecycleListener implements IClusterEventsSubscriber {
+
+ private final AsterixReplicationProperties asterixReplicationProperties;
+ public static ReplicationLifecycleListener INSTANCE;
+
+ public ReplicationLifecycleListener(AsterixReplicationProperties asterixReplicationProperties) {
+ this.asterixReplicationProperties = asterixReplicationProperties;
+ }
+
+ @Override
+ public Set<IClusterManagementWork> notifyNodeFailure(Set<String> deadNodeIds) {
+ //notify impacted replicas
+ for (String deadNodeId : deadNodeIds) {
+ Replica eventOnReplica = asterixReplicationProperties.getReplicaById(deadNodeId);
+ ReplicaEvent event = new ReplicaEvent(eventOnReplica, ReplicaEventType.FAIL);
+ ReplicaEventNotifier notifier = new ReplicaEventNotifier(event, asterixReplicationProperties);
+
+ //start notifier
+ new Thread(notifier).start();
+ }
+
+ return null;
+ }
+
+ @Override
+ public Set<IClusterManagementWork> notifyNodeJoin(String joinedNodeId) {
+ Replica eventOnReplica = asterixReplicationProperties.getReplicaById(joinedNodeId);
+ ReplicaEvent event = new ReplicaEvent(eventOnReplica, ReplicaEventType.JOIN);
+ ReplicaEventNotifier notifier = new ReplicaEventNotifier(event, asterixReplicationProperties);
+
+ //start notifier
+ new Thread(notifier).start();
+
+ return null;
+ }
+
+ @Override
+ public void notifyRequestCompletion(IClusterManagementWorkResponse response) {
+ //do nothing
+ }
+
+ @Override
+ public void notifyStateChange(ClusterState previousState, ClusterState newState) {
+ //do nothing
+ }
+}
diff --git a/asterix-replication/src/main/java/org/apache/asterix/replication/management/ReplicationManager.java b/asterix-replication/src/main/java/org/apache/asterix/replication/management/ReplicationManager.java
new file mode 100644
index 0000000..eaef9be
--- /dev/null
+++ b/asterix-replication/src/main/java/org/apache/asterix/replication/management/ReplicationManager.java
@@ -0,0 +1,1247 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.replication.management;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.OutputStream;
+import java.io.RandomAccessFile;
+import java.net.InetSocketAddress;
+import java.nio.ByteBuffer;
+import java.nio.channels.AsynchronousCloseException;
+import java.nio.channels.FileChannel;
+import java.nio.channels.SocketChannel;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import org.apache.asterix.common.config.AsterixReplicationProperties;
+import org.apache.asterix.common.dataflow.AsterixLSMIndexUtil;
+import org.apache.asterix.common.exceptions.ACIDException;
+import org.apache.asterix.common.replication.AsterixReplicationJob;
+import org.apache.asterix.common.replication.IReplicaResourcesManager;
+import org.apache.asterix.common.replication.IReplicationManager;
+import org.apache.asterix.common.replication.Replica;
+import org.apache.asterix.common.replication.Replica.ReplicaState;
+import org.apache.asterix.common.replication.ReplicaEvent;
+import org.apache.asterix.common.replication.ReplicaEvent.ReplicaEventType;
+import org.apache.asterix.common.transactions.IAsterixAppRuntimeContextProvider;
+import org.apache.asterix.common.transactions.ILogManager;
+import org.apache.asterix.common.transactions.ILogRecord;
+import org.apache.asterix.common.transactions.LogRecord;
+import org.apache.asterix.common.transactions.LogType;
+import org.apache.asterix.event.schema.cluster.Node;
+import org.apache.asterix.replication.functions.AsterixReplicationProtocol;
+import org.apache.asterix.replication.functions.AsterixReplicationProtocol.ReplicationRequestType;
+import org.apache.asterix.replication.functions.ReplicaFilesRequest;
+import org.apache.asterix.replication.functions.ReplicaIndexFlushRequest;
+import org.apache.asterix.replication.functions.ReplicaLogsRequest;
+import org.apache.asterix.replication.logging.ReplicationLogBuffer;
+import org.apache.asterix.replication.logging.ReplicationLogFlusher;
+import org.apache.asterix.replication.storage.AsterixLSMIndexFileProperties;
+import org.apache.asterix.replication.storage.LSMComponentProperties;
+import org.apache.asterix.replication.storage.ReplicaResourcesManager;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.replication.IReplicationJob;
+import org.apache.hyracks.api.replication.IReplicationJob.ReplicationExecutionType;
+import org.apache.hyracks.api.replication.IReplicationJob.ReplicationJobType;
+import org.apache.hyracks.api.replication.IReplicationJob.ReplicationOperation;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexReplicationJob;
+import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndex;
+
+/**
+ * This class is used to process replication jobs and maintain remote replicas states
+ */
+public class ReplicationManager implements IReplicationManager {
+
+ private static final Logger LOGGER = Logger.getLogger(ReplicationManager.class.getName());
+ private final int INITIAL_REPLICATION_FACTOR = 1;
+ private final String nodeId;
+ private ExecutorService replicationListenerThreads;
+ private final Map<Integer, Set<String>> jobCommitAcks;
+ private final Map<Integer, ILogRecord> replicationJobsPendingAcks;
+ private ByteBuffer dataBuffer;
+
+ private final LinkedBlockingQueue<IReplicationJob> replicationJobsQ;
+ private final LinkedBlockingQueue<ReplicaEvent> replicaEventsQ;
+
+ private int replicationFactor = 1;
+ private final ReplicaResourcesManager replicaResourcesManager;
+ private final ILogManager logManager;
+ private final IAsterixAppRuntimeContextProvider asterixAppRuntimeContextProvider;
+ private final AsterixReplicationProperties replicationProperties;
+ private final Map<String, Replica> replicas;
+
+ private final AtomicBoolean replicationSuspended;
+ private AtomicBoolean terminateJobsReplication;
+ private AtomicBoolean jobsReplicationSuspended;
+ private final static int INITIAL_BUFFER_SIZE = 4000; //4KB
+ private final Set<String> shuttingDownReplicaIds;
+ //replication threads
+ private ReplicationJobsProccessor replicationJobsProcessor;
+ private final ReplicasEventsMonitor replicationMonitor;
+ //dummy job used to stop ReplicationJobsProccessor thread.
+ private static final IReplicationJob replicationJobPoisonPill = new AsterixReplicationJob(
+ ReplicationJobType.METADATA, ReplicationOperation.STOP, ReplicationExecutionType.ASYNC, null);
+ //used to identify the correct IP address when the node has multiple network interfaces
+ private String hostIPAddressFirstOctet = null;
+
+ private LinkedBlockingQueue<ReplicationLogBuffer> emptyLogBuffersQ;
+ private LinkedBlockingQueue<ReplicationLogBuffer> pendingFlushLogBuffersQ;
+ protected ReplicationLogBuffer currentTxnLogBuffer;
+ private ReplicationLogFlusher txnlogsReplicator;
+ private Future<Object> txnLogReplicatorTask;
+ private Map<String, SocketChannel> logsReplicaSockets = null;
+
+ public ReplicationManager(String nodeId, AsterixReplicationProperties replicationProperties,
+ IReplicaResourcesManager remoteResoucesManager, ILogManager logManager,
+ IAsterixAppRuntimeContextProvider asterixAppRuntimeContextProvider) {
+ this.nodeId = nodeId;
+ this.replicationProperties = replicationProperties;
+ this.replicaResourcesManager = (ReplicaResourcesManager) remoteResoucesManager;
+ this.asterixAppRuntimeContextProvider = asterixAppRuntimeContextProvider;
+ this.hostIPAddressFirstOctet = replicationProperties.getReplicaIPAddress(nodeId).substring(0, 3);
+ this.logManager = logManager;
+ replicationJobsQ = new LinkedBlockingQueue<IReplicationJob>();
+ replicaEventsQ = new LinkedBlockingQueue<ReplicaEvent>();
+ terminateJobsReplication = new AtomicBoolean(false);
+ jobsReplicationSuspended = new AtomicBoolean(true);
+ replicationSuspended = new AtomicBoolean(true);
+
+ replicas = new HashMap<String, Replica>();
+ jobCommitAcks = new ConcurrentHashMap<Integer, Set<String>>();
+ replicationJobsPendingAcks = new ConcurrentHashMap<Integer, ILogRecord>();
+ shuttingDownReplicaIds = new HashSet<String>();
+ dataBuffer = ByteBuffer.allocate(INITIAL_BUFFER_SIZE);
+
+ //Used as async listeners from replicas
+ replicationListenerThreads = Executors.newCachedThreadPool();
+ replicationJobsProcessor = new ReplicationJobsProccessor();
+ replicationMonitor = new ReplicasEventsMonitor();
+
+ //add list of replicas from configurations (To be read from another source e.g. Zookeeper)
+ Set<Replica> replicaNodes = replicationProperties.getRemoteReplicas(nodeId);
+ if (replicaNodes != null) {
+ for (Replica replica : replicaNodes) {
+ replicas.put(replica.getNode().getId(), replica);
+ }
+ }
+ int numLogBuffers = logManager.getNumLogPages();
+ emptyLogBuffersQ = new LinkedBlockingQueue<ReplicationLogBuffer>(numLogBuffers);
+ pendingFlushLogBuffersQ = new LinkedBlockingQueue<ReplicationLogBuffer>(numLogBuffers);
+
+ int logBufferSize = logManager.getLogPageSize();
+ for (int i = 0; i < numLogBuffers; i++) {
+ emptyLogBuffersQ.offer(new ReplicationLogBuffer(this, logBufferSize));
+ }
+ }
+
+ /**
+ * Accepts a replication job. If the job execution type is ASYNC, it is queued.
+ * Otherwise, it is processed immediately.
+ */
+ @Override
+ public void submitJob(IReplicationJob job) throws IOException {
+ if (job.getExecutionType() == ReplicationExecutionType.ASYNC) {
+ replicationJobsQ.offer(job);
+ } else {
+ //wait until replication is resumed
+ while (replicationSuspended.get()) {
+ synchronized (replicationSuspended) {
+ try {
+ replicationSuspended.wait();
+ } catch (InterruptedException e) {
+ //ignore
+ }
+ }
+ }
+ processJob(job, null, null);
+ }
+ }
+
+ @Override
+ public void replicateLog(ILogRecord logRecord) {
+ if (logRecord.getLogType() == LogType.JOB_COMMIT) {
+ //if replication is suspended, wait until it is resumed.
+ while (replicationSuspended.get()) {
+ synchronized (replicationSuspended) {
+ try {
+ replicationSuspended.wait();
+ } catch (InterruptedException e) {
+ //ignore
+ }
+ }
+ }
+ Set<String> replicaIds = Collections.synchronizedSet(new HashSet<String>());
+ replicaIds.add(nodeId);
+ jobCommitAcks.put(logRecord.getJobId(), replicaIds);
+ }
+
+ appendToLogBuffer(logRecord);
+ }
+
+ protected void getAndInitNewPage() {
+ currentTxnLogBuffer = null;
+ while (currentTxnLogBuffer == null) {
+ try {
+ currentTxnLogBuffer = emptyLogBuffersQ.take();
+ } catch (InterruptedException e) {
+ //ignore
+ }
+ }
+ currentTxnLogBuffer.reset();
+ currentTxnLogBuffer.setReplicationSockets(logsReplicaSockets);
+ pendingFlushLogBuffersQ.offer(currentTxnLogBuffer);
+ }
+
+ private synchronized void appendToLogBuffer(ILogRecord logRecord) {
+ if (!currentTxnLogBuffer.hasSpace(logRecord)) {
+ currentTxnLogBuffer.isFull(true);
+ getAndInitNewPage();
+ }
+
+ currentTxnLogBuffer.append(logRecord);
+ }
+
+ /**
+ * Processes the replication job based on its specifications
+ *
+ * @param job
+ * The replication job
+ * @param replicasSockets
+ * The remote replicas sockets to send the request to.
+ * @param requestBuffer
+ * The buffer to use to send the request.
+ * @throws IOException
+ */
+ private void processJob(IReplicationJob job, Map<String, SocketChannel> replicasSockets, ByteBuffer requestBuffer)
+ throws IOException {
+ boolean isLSMComponentFile;
+ ByteBuffer responseBuffer = null;
+ AsterixLSMIndexFileProperties asterixFileProperties = new AsterixLSMIndexFileProperties();
+ if (requestBuffer == null) {
+ requestBuffer = ByteBuffer.allocate(INITIAL_BUFFER_SIZE);
+ }
+
+ isLSMComponentFile = job.getJobType() == ReplicationJobType.LSM_COMPONENT ? true : false;
+ try {
+ //if there isn't already a connection, establish a new one
+ if (replicasSockets == null) {
+ replicasSockets = getActiveRemoteReplicasSockets();
+ }
+
+ int remainingFiles = job.getJobFiles().size();
+
+ if (job.getOperation() == ReplicationOperation.REPLICATE) {
+ try {
+ //if the replication job is an LSM_COMPONENT, its properties are sent first, then its files.
+ ILSMIndexReplicationJob LSMComponentJob = null;
+ if (job.getJobType() == ReplicationJobType.LSM_COMPONENT) {
+ //send LSMComponent properties
+ LSMComponentJob = (ILSMIndexReplicationJob) job;
+ LSMComponentProperties lsmCompProp = new LSMComponentProperties(LSMComponentJob, nodeId);
+ requestBuffer = AsterixReplicationProtocol.writeLSMComponentPropertiesRequest(lsmCompProp,
+ requestBuffer);
+ sendRequest(replicasSockets, requestBuffer);
+ }
+
+ for (String filePath : job.getJobFiles()) {
+ remainingFiles--;
+ Path path = Paths.get(filePath);
+ if (Files.notExists(path)) {
+ LOGGER.log(Level.SEVERE, "File deleted before replication: " + filePath);
+ continue;
+ }
+
+ LOGGER.log(Level.INFO, "Replicating file: " + filePath);
+ //open file for reading
+ try (RandomAccessFile fromFile = new RandomAccessFile(filePath, "r");
+ FileChannel fileChannel = fromFile.getChannel();) {
+
+ long fileSize = fileChannel.size();
+
+ if (LSMComponentJob != null) {
+ boolean requireLSNSync = AsterixLSMIndexUtil.lsmComponentFileHasLSN(
+ (AbstractLSMIndex) LSMComponentJob.getLSMIndex(), filePath);
+ asterixFileProperties.initialize(filePath, fileSize, nodeId, isLSMComponentFile,
+ requireLSNSync, remainingFiles == 0);
+ } else {
+ asterixFileProperties.initialize(filePath, fileSize, nodeId, isLSMComponentFile, false,
+ remainingFiles == 0);
+ }
+
+ requestBuffer = AsterixReplicationProtocol.writeFileReplicationRequest(requestBuffer,
+ asterixFileProperties, ReplicationRequestType.REPLICATE_FILE);
+
+ Iterator<Map.Entry<String, SocketChannel>> iterator = replicasSockets.entrySet().iterator();
+ while (iterator.hasNext()) {
+ Map.Entry<String, SocketChannel> entry = iterator.next();
+ SocketChannel socketChannel = entry.getValue();
+ //transfer request header & file
+ try {
+ NetworkingUtil.transferBufferToChannel(socketChannel, requestBuffer);
+ NetworkingUtil.sendFile(fileChannel, socketChannel);
+ if (asterixFileProperties.requiresAck()) {
+ ReplicationRequestType responseType = waitForResponse(socketChannel,
+ responseBuffer);
+ if (responseType != ReplicationRequestType.ACK) {
+ throw new IOException(
+ "Could not receive ACK from replica " + entry.getKey());
+ }
+ }
+ } catch (IOException e) {
+ reportFailedReplica(entry.getKey());
+ iterator.remove();
+ } finally {
+ requestBuffer.position(0);
+ }
+ }
+ }
+ }
+ } finally {
+ if (job instanceof ILSMIndexReplicationJob) {
+ //exit the replicated LSM components
+ ILSMIndexReplicationJob aJob = (ILSMIndexReplicationJob) job;
+ aJob.endReplication();
+ }
+ }
+ } else if (job.getOperation() == ReplicationOperation.DELETE) {
+ for (String filePath : job.getJobFiles()) {
+ remainingFiles--;
+ asterixFileProperties.initialize(filePath, -1, nodeId, isLSMComponentFile, false,
+ remainingFiles == 0);
+ AsterixReplicationProtocol.writeFileReplicationRequest(requestBuffer, asterixFileProperties,
+ ReplicationRequestType.DELETE_FILE);
+
+ Iterator<Map.Entry<String, SocketChannel>> iterator = replicasSockets.entrySet().iterator();
+ while (iterator.hasNext()) {
+ Map.Entry<String, SocketChannel> entry = iterator.next();
+ SocketChannel socketChannel = entry.getValue();
+ try {
+ sendRequest(replicasSockets, requestBuffer);
+ if (asterixFileProperties.requiresAck()) {
+ waitForResponse(socketChannel, responseBuffer);
+ }
+ } catch (IOException e) {
+ reportFailedReplica(entry.getKey());
+ iterator.remove();
+ } finally {
+ requestBuffer.position(0);
+ }
+ }
+ }
+ }
+ } finally {
+ //if sync, close sockets with replicas since they wont be reused
+ if (job.getExecutionType() == ReplicationExecutionType.SYNC) {
+ closeReplicaSockets(replicasSockets);
+ }
+ }
+ }
+
+ /**
+ * Waits and reads a response from a remote replica
+ *
+ * @param socketChannel
+ * The socket to read the response from
+ * @param responseBuffer
+ * The response buffer to read the response to.
+ * @return The response type.
+ * @throws IOException
+ */
+ private static ReplicationRequestType waitForResponse(SocketChannel socketChannel, ByteBuffer responseBuffer)
+ throws IOException {
+ if (responseBuffer == null) {
+ responseBuffer = ByteBuffer.allocate(AsterixReplicationProtocol.REPLICATION_REQUEST_TYPE_SIZE);
+ } else {
+ responseBuffer.clear();
+ }
+
+ //read response from remote replicas
+ ReplicationRequestType responseFunction = AsterixReplicationProtocol.getRequestType(socketChannel,
+ responseBuffer);
+ return responseFunction;
+ }
+
+ @Override
+ public boolean isReplicationEnabled() {
+ return replicationProperties.isReplicationEnabled();
+ }
+
+ @Override
+ public synchronized void updateReplicaInfo(Replica replicaNode) {
+ Replica replica = replicas.get(replicaNode.getNode().getId());
+ //should not update the info of an active replica
+ if (replica.getState() == ReplicaState.ACTIVE) {
+ return;
+ }
+
+ replica.getNode().setClusterIp(replicaNode.getNode().getClusterIp());
+
+ /*
+ * This could be used to reconnect to replica without needing the Cluster notifications
+ if (replica.getState() == ReplicaState.DEAD) {
+ reportFailedReplica(replica.getNode().getId());
+ } else if (replica.getState() == ReplicaState.ACTIVE) {
+ checkReplicaState(replica.getNode().getId(), true);
+ }
+ */
+ }
+
+ /**
+ * Suspends proccessing replication jobs.
+ *
+ * @param force
+ * a flag indicates if replication should be suspended right away or when the pending jobs are completed.
+ */
+ private void suspendReplication(boolean force) {
+ //suspend replication jobs processing
+ if (replicationJobsProcessor != null && replicationJobsProcessor.isAlive()) {
+ if (force) {
+ terminateJobsReplication.set(true);
+ }
+ replicationJobsQ.offer(replicationJobPoisonPill);
+
+ //wait until the jobs are suspended
+ synchronized (jobsReplicationSuspended) {
+ while (!jobsReplicationSuspended.get()) {
+ try {
+ jobsReplicationSuspended.wait();
+ } catch (InterruptedException e) {
+ //ignore
+ }
+ }
+ }
+ }
+
+ //suspend logs replication
+ if (txnlogsReplicator != null) {
+ terminateTxnLogsReplicator();
+ }
+ }
+
+ /**
+ * Opens a new connection with Active remote replicas and starts a listen thread per connection.
+ */
+ private void establishTxnLogsReplicationConnection() {
+ logsReplicaSockets = getActiveRemoteReplicasSockets();
+ //start a listener thread per connection
+ for (Entry<String, SocketChannel> entry : logsReplicaSockets.entrySet()) {
+ replicationListenerThreads
+ .execute(new TxnLogsReplicationResponseListener(entry.getKey(), entry.getValue()));
+ }
+ }
+
+ /**
+ * Stops ReplicationFlusherThread and closes the sockets used to replicate logs.
+ */
+ private void terminateTxnLogsReplicator() {
+ LOGGER.log(Level.INFO, "Terminating ReplicationLogFlusher thread ...");
+ txnlogsReplicator.terminate();
+ try {
+ txnLogReplicatorTask.get();
+ } catch (ExecutionException | InterruptedException e) {
+ LOGGER.log(Level.WARNING, "RepicationLogFlusher thread terminated abnormally");
+ e.printStackTrace();
+ }
+ LOGGER.log(Level.INFO, "LogFlusher thread is terminated.");
+
+ if (logsReplicaSockets != null) {
+ //wait for any ACK to arrive before closing sockets.
+ synchronized (jobCommitAcks) {
+ while (jobCommitAcks.size() != 0) {
+ try {
+ jobCommitAcks.wait();
+ } catch (InterruptedException e) {
+ //ignore
+ }
+ }
+ }
+
+ //close log replication sockets
+ closeReplicaSockets(logsReplicaSockets);
+ logsReplicaSockets = null;
+ }
+ }
+
+ @Override
+ public void broadcastNewIPAddress() throws IOException {
+ String orignalIPAddress = replicationProperties.getReplicaIPAddress(nodeId);
+ String newAddress = NetworkingUtil.getHostAddress(hostIPAddressFirstOctet);
+
+ //IP Address didn't change after failure
+ if (orignalIPAddress.equals(newAddress)) {
+ return;
+ }
+
+ Node node = new Node();
+ node.setId(nodeId);
+ node.setClusterIp(newAddress);
+ Replica replica = new Replica(node);
+
+ ByteBuffer buffer = AsterixReplicationProtocol.writeUpdateReplicaRequest(replica);
+ Map<String, SocketChannel> replicaSockets = getActiveRemoteReplicasSockets();
+ sendRequest(replicaSockets, buffer);
+ closeReplicaSockets(replicaSockets);
+ }
+
+ /**
+ * Sends a shutdown event to remote replicas notifying them
+ * no more logs/files will be sent from this local replica.
+ *
+ * @throws IOException
+ */
+ private void sendShutdownNotifiction() throws IOException {
+ Node node = new Node();
+ node.setId(nodeId);
+ node.setClusterIp(NetworkingUtil.getHostAddress(hostIPAddressFirstOctet));
+ Replica replica = new Replica(node);
+ ReplicaEvent event = new ReplicaEvent(replica, ReplicaEventType.SHUTDOWN);
+ ByteBuffer buffer = AsterixReplicationProtocol.writeReplicaEventRequest(event);
+ Map<String, SocketChannel> replicaSockets = getActiveRemoteReplicasSockets();
+ sendRequest(replicaSockets, buffer);
+ closeReplicaSockets(replicaSockets);
+ }
+
+ /**
+ * Sends a request to remote replicas
+ *
+ * @param replicaSockets
+ * The sockets to send the request to.
+ * @param requestBuffer
+ * The buffer that contains the request.
+ */
+ private void sendRequest(Map<String, SocketChannel> replicaSockets, ByteBuffer requestBuffer) {
+ Iterator<Map.Entry<String, SocketChannel>> iterator = replicaSockets.entrySet().iterator();
+ while (iterator.hasNext()) {
+ Entry<String, SocketChannel> replicaSocket = iterator.next();
+ SocketChannel clientSocket = replicaSocket.getValue();
+ try {
+ NetworkingUtil.transferBufferToChannel(clientSocket, requestBuffer);
+ } catch (IOException e) {
+ if (clientSocket.isOpen()) {
+ try {
+ clientSocket.close();
+ } catch (IOException e2) {
+ e2.printStackTrace();
+ }
+ }
+ reportFailedReplica(replicaSocket.getKey());
+ iterator.remove();
+ } finally {
+ requestBuffer.position(0);
+ }
+ }
+ }
+
+ /**
+ * Closes the passed replication sockets by sending GOODBYE request to remote replicas.
+ *
+ * @param replicaSockets
+ */
+ private void closeReplicaSockets(Map<String, SocketChannel> replicaSockets) {
+ //send goodbye
+ ByteBuffer goodbyeBuffer = AsterixReplicationProtocol.getGoodbyeBuffer();
+ sendRequest(replicaSockets, goodbyeBuffer);
+
+ Iterator<Map.Entry<String, SocketChannel>> iterator = replicaSockets.entrySet().iterator();
+ while (iterator.hasNext()) {
+ Entry<String, SocketChannel> replicaSocket = iterator.next();
+ SocketChannel clientSocket = replicaSocket.getValue();
+ if (clientSocket.isOpen()) {
+ try {
+ clientSocket.close();
+ } catch (IOException e) {
+ e.printStackTrace();
+ }
+ }
+ }
+ }
+
+ @Override
+ public void initializeReplicasState() {
+ for (Replica replica : replicas.values()) {
+ checkReplicaState(replica.getNode().getId(), false, false);
+ }
+ }
+
+ /**
+ * Checks the state of a remote replica by trying to ping it.
+ *
+ * @param replicaId
+ * The replica to check the state for.
+ * @param async
+ * a flag indicating whether to wait for the result or not.
+ * @param suspendReplication
+ * a flag indicating whether to suspend replication on replica state change or not.
+ */
+ private void checkReplicaState(String replicaId, boolean async, boolean suspendReplication) {
+ Replica replica = replicas.get(replicaId);
+
+ ReplicaStateChecker connector = new ReplicaStateChecker(replica, replicationProperties.getReplicationTimeOut(),
+ this, replicationProperties, suspendReplication);
+ Future<Object> ft = asterixAppRuntimeContextProvider.getThreadExecutor().submit(connector);
+
+ if (!async) {
+ //wait until task is done
+ while (!ft.isDone()) {
+ try {
+ Thread.sleep(1000);
+ } catch (InterruptedException e) {
+ e.printStackTrace();
+ }
+ }
+ }
+ }
+
+ /**
+ * Updates the state of a remote replica.
+ *
+ * @param replicaId
+ * The replica id to update.
+ * @param newState
+ * The new state of the replica.
+ * @param suspendReplication
+ * a flag indicating whether to suspend replication on state change or not.
+ */
+ public synchronized void updateReplicaState(String replicaId, ReplicaState newState, boolean suspendReplication) {
+ Replica replica = replicas.get(replicaId);
+
+ if (replica.getState() == newState) {
+ return;
+ }
+
+ if (suspendReplication) {
+ //prevent new jobs/logs from coming in
+ replicationSuspended.set(true);
+
+ if (newState == ReplicaState.DEAD) {
+ //assume the dead replica ACK has been received for all pending jobs
+ synchronized (jobCommitAcks) {
+ for (Integer jobId : jobCommitAcks.keySet()) {
+ addAckToJob(jobId, replicaId);
+ }
+ }
+ }
+
+ //force replication threads to stop in order to change the replication factor
+ suspendReplication(true);
+ }
+
+ replica.setState(newState);
+
+ if (newState == ReplicaState.ACTIVE) {
+ replicationFactor++;
+ //TODO Extra check: make sure newly added replica is in sync.
+ //Since in the current design the whole cluster becomes UNUSABLE,
+ //no new jobs could start before the failed node rejoins
+ } else if (newState == ReplicaState.DEAD) {
+ if (replicationFactor > INITIAL_REPLICATION_FACTOR) {
+ replicationFactor--;
+ }
+ }
+
+ LOGGER.log(Level.WARNING, "Replica " + replicaId + " state changed to: " + newState.name()
+ + ". Replication factor changed to: " + replicationFactor);
+
+ if (suspendReplication) {
+ startReplicationThreads();
+ }
+ }
+
+ /**
+ * When an ACK for a JOB_COMMIT is received, it is added to the corresponding job.
+ *
+ * @param jobId
+ * @param replicaId
+ * The remote replica id the ACK received from.
+ */
+ private void addAckToJob(int jobId, String replicaId) {
+ //add ACK to the job
+ if (jobCommitAcks.containsKey(jobId)) {
+ Set<String> replicaIds = jobCommitAcks.get(jobId);
+ replicaIds.add(replicaId);
+ } else {
+ throw new IllegalStateException("Job ID not found in pending job commits " + jobId);
+ }
+
+ //if got ACKs from all remote replicas, notify pending jobs if any
+ if (jobCommitAcks.get(jobId).size() == replicationFactor) {
+ synchronized (replicationJobsPendingAcks) {
+ if (replicationJobsPendingAcks.containsKey(jobId)) {
+ ILogRecord pendingLog = replicationJobsPendingAcks.get(jobId);
+ synchronized (pendingLog) {
+ pendingLog.notify();
+ }
+ }
+ }
+ }
+ }
+
+ @Override
+ public boolean hasBeenReplicated(ILogRecord logRecord) {
+ if (jobCommitAcks.containsKey(logRecord.getJobId())) {
+ //check if all ACKs have been received
+ if (jobCommitAcks.get(logRecord.getJobId()).size() == replicationFactor) {
+ jobCommitAcks.remove(logRecord.getJobId());
+
+ if (replicationJobsPendingAcks.containsKey(logRecord.getJobId())) {
+ replicationJobsPendingAcks.remove(logRecord);
+ }
+
+ //notify any threads waiting for all jobs to finish
+ if (jobCommitAcks.size() == 0) {
+ synchronized (jobCommitAcks) {
+ jobCommitAcks.notifyAll();
+ }
+ }
+
+ return true;
+ } else {
+ if (!replicationJobsPendingAcks.containsKey(logRecord.getJobId())) {
+ synchronized (replicationJobsPendingAcks) {
+ replicationJobsPendingAcks.put(logRecord.getJobId(), logRecord);
+ }
+ }
+ return false;
+ }
+ }
+
+ //presume replicated
+ return true;
+ }
+
+ private Map<String, SocketChannel> getActiveRemoteReplicasSockets() {
+ Map<String, SocketChannel> replicaNodesSockets = new HashMap<String, SocketChannel>();
+ for (Replica replica : replicas.values()) {
+ if (replica.getState() == ReplicaState.ACTIVE) {
+ try {
+ SocketChannel sc = getReplicaSocket(replica.getId());
+ replicaNodesSockets.put(replica.getId(), sc);
+ } catch (IOException e) {
+ reportFailedReplica(replica.getId());
+ }
+ }
+ }
+ return replicaNodesSockets;
+ }
+
+ /**
+ * Establishes a connection with a remote replica.
+ *
+ * @param replicaId
+ * The replica to connect to.
+ * @return The socket of the remote replica
+ * @throws IOException
+ */
+ private SocketChannel getReplicaSocket(String replicaId) throws IOException {
+ Replica replica = replicas.get(replicaId);
+ SocketChannel sc = SocketChannel.open();
+ sc.configureBlocking(true);
+ InetSocketAddress address = replica.getAddress(replicationProperties);
+ sc.connect(new InetSocketAddress(address.getHostString(), address.getPort()));
+ return sc;
+ }
+
+ @Override
+ public Set<String> getDeadReplicasIds() {
+ Set<String> replicasIds = new HashSet<String>();
+ for (Replica replica : replicas.values()) {
+ if (replica.getState() == ReplicaState.DEAD) {
+ replicasIds.add(replica.getNode().getId());
+ }
+ }
+ return replicasIds;
+ }
+
+ @Override
+ public Set<String> getActiveReplicasIds() {
+ Set<String> replicasIds = new HashSet<String>();
+ for (Replica replica : replicas.values()) {
+ if (replica.getState() == ReplicaState.ACTIVE) {
+ replicasIds.add(replica.getNode().getId());
+ }
+ }
+ return replicasIds;
+ }
+
+ @Override
+ public int getActiveReplicasCount() {
+ return getActiveReplicasIds().size();
+ }
+
+ @Override
+ public void start() {
+ //do nothing
+ }
+
+ @Override
+ public void dumpState(OutputStream os) throws IOException {
+ //do nothing
+ }
+
+ /**
+ * Called during NC shutdown to notify remote replicas about the shutdown
+ * and wait for remote replicas shutdown notification then closes the local replication channel.
+ */
+ @Override
+ public void stop(boolean dumpState, OutputStream ouputStream) throws IOException {
+ try {
+ //stop replication thread afters all jobs/logs have been processed
+ suspendReplication(false);
+ //send shutdown event to remote replicas
+ sendShutdownNotifiction();
+ //wait until all shutdown events come from all remote replicas
+ synchronized (shuttingDownReplicaIds) {
+ while (!shuttingDownReplicaIds.containsAll(getActiveReplicasIds())) {
+ try {
+ shuttingDownReplicaIds.wait(1000);
+ } catch (InterruptedException e) {
+ e.printStackTrace();
+ }
+ }
+ }
+ LOGGER.log(Level.INFO, "Got shutdown notification from all remote replicas");
+ //close replication channel
+ asterixAppRuntimeContextProvider.getAppContext().getReplicationChannel().close();
+
+ LOGGER.log(Level.INFO, "Replication manager stopped.");
+ } catch (Exception e) {
+ e.printStackTrace();
+ }
+ }
+
+ @Override
+ public void reportReplicaEvent(ReplicaEvent event) {
+ synchronized (replicaEventsQ) {
+ replicaEventsQ.offer(event);
+ }
+ }
+
+ /**
+ * Suspends replications and sends a remote replica failure event to ReplicasEventsMonitor.
+ *
+ * @param replicaId
+ * the failed replica id.
+ */
+ public void reportFailedReplica(String replicaId) {
+ Replica replica = replicas.get(replicaId);
+ if (replica.getState() == ReplicaState.DEAD) {
+ return;
+ }
+
+ //need to stop processing any new logs or jobs
+ terminateJobsReplication.set(true);
+
+ ReplicaEvent event = new ReplicaEvent(replica, ReplicaEventType.FAIL);
+ reportReplicaEvent(event);
+ }
+
+ @Override
+ public void startReplicationThreads() {
+ replicationJobsProcessor = new ReplicationJobsProccessor();
+
+ //start/continue processing jobs/logs
+ if (logsReplicaSockets == null) {
+ establishTxnLogsReplicationConnection();
+ getAndInitNewPage();
+ txnlogsReplicator = new ReplicationLogFlusher(emptyLogBuffersQ, pendingFlushLogBuffersQ);
+ txnLogReplicatorTask = asterixAppRuntimeContextProvider.getThreadExecutor().submit(txnlogsReplicator);
+ }
+
+ replicationJobsProcessor.start();
+
+ if (!replicationMonitor.isAlive()) {
+ replicationMonitor.start();
+ }
+
+ //notify any waiting threads that replication has been resumed
+ synchronized (replicationSuspended) {
+ LOGGER.log(Level.INFO, "Replication started/resumed");
+ replicationSuspended.set(false);
+ replicationSuspended.notifyAll();
+ }
+ }
+
+ @Override
+ public void requestFlushLaggingReplicaIndexes(long nonSharpCheckpointTargetLSN) throws IOException {
+ long startLSN = logManager.getAppendLSN();
+ Set<String> replicaIds = getActiveReplicasIds();
+ ByteBuffer requestBuffer = ByteBuffer.allocate(INITIAL_BUFFER_SIZE);
+ for (String replicaId : replicaIds) {
+ //1. identify replica indexes with LSN less than nonSharpCheckpointTargetLSN.
+ HashMap<Long, String> laggingIndexes = replicaResourcesManager.getLaggingReplicaIndexesId2PathMap(replicaId,
+ nonSharpCheckpointTargetLSN);
+
+ if (laggingIndexes.size() > 0) {
+ //2. send request to remote replicas that have lagging indexes.
+ ReplicaIndexFlushRequest laggingIndexesResponse = null;
+ try (SocketChannel socketChannel = getReplicaSocket(replicaId)) {
+ ReplicaIndexFlushRequest laggingIndexesRequest = new ReplicaIndexFlushRequest(
+ laggingIndexes.keySet());
+ requestBuffer = AsterixReplicationProtocol.writeGetReplicaIndexFlushRequest(requestBuffer,
+ laggingIndexesRequest);
+ NetworkingUtil.transferBufferToChannel(socketChannel, requestBuffer);
+
+ //3. remote replicas will respond with indexes that were not flushed.
+ ReplicationRequestType responseFunction = waitForResponse(socketChannel, requestBuffer);
+
+ if (responseFunction == ReplicationRequestType.FLUSH_INDEX) {
+ requestBuffer = AsterixReplicationProtocol.readRequest(socketChannel, requestBuffer);
+ //returning the indexes that were not flushed
+ laggingIndexesResponse = AsterixReplicationProtocol.readReplicaIndexFlushRequest(requestBuffer);
+ }
+ //send goodbye
+ AsterixReplicationProtocol.sendGoodbye(socketChannel);
+ }
+
+ //4. update the LSN_MAP for indexes that were not flushed to the current append LSN to indicate no operations happend.
+ if (laggingIndexesResponse != null) {
+ for (Long resouceId : laggingIndexesResponse.getLaggingRescouresIds()) {
+ String indexPath = laggingIndexes.get(resouceId);
+ HashMap<Long, Long> indexLSNMap = replicaResourcesManager.getReplicaIndexLSNMap(indexPath);
+ indexLSNMap.put(ReplicaResourcesManager.REPLICA_INDEX_CREATION_LSN, startLSN);
+ replicaResourcesManager.updateReplicaIndexLSNMap(indexPath, indexLSNMap);
+ }
+ }
+ }
+ }
+ }
+
+ //Recovery Method
+ @Override
+ public long getMaxRemoteLSN(Set<String> remoteReplicas) throws IOException {
+ long maxRemoteLSN = 0;
+
+ AsterixReplicationProtocol.writeGetReplicaMaxLSNRequest(dataBuffer);
+ Map<String, SocketChannel> replicaSockets = new HashMap<String, SocketChannel>();
+ try {
+ for (String replicaId : remoteReplicas) {
+ replicaSockets.put(replicaId, getReplicaSocket(replicaId));
+ }
+
+ //send request
+ Iterator<Map.Entry<String, SocketChannel>> iterator = replicaSockets.entrySet().iterator();
+ while (iterator.hasNext()) {
+ Entry<String, SocketChannel> replicaSocket = iterator.next();
+ SocketChannel clientSocket = replicaSocket.getValue();
+ NetworkingUtil.transferBufferToChannel(clientSocket, dataBuffer);
+ dataBuffer.position(0);
+ }
+
+ iterator = replicaSockets.entrySet().iterator();
+ while (iterator.hasNext()) {
+ Entry<String, SocketChannel> replicaSocket = iterator.next();
+ SocketChannel clientSocket = replicaSocket.getValue();
+ //read response
+ NetworkingUtil.readBytes(clientSocket, dataBuffer, Long.BYTES);
+ maxRemoteLSN = Math.max(maxRemoteLSN, dataBuffer.getLong());
+ }
+ } finally {
+ closeReplicaSockets(replicaSockets);
+ }
+
+ return maxRemoteLSN;
+ }
+
+ //Recovery Method
+ @Override
+ public void requestReplicaFiles(String selectedReplicaId, Set<String> replicasDataToRecover) throws IOException {
+ ReplicaFilesRequest request = new ReplicaFilesRequest(replicasDataToRecover);
+ AsterixReplicationProtocol.writeGetReplicaFilesRequest(dataBuffer, request);
+
+ try (SocketChannel socketChannel = getReplicaSocket(selectedReplicaId)) {
+
+ //transfer request
+ NetworkingUtil.transferBufferToChannel(socketChannel, dataBuffer);
+
+ String destFolder;
+ String destFilePath;
+
+ ReplicationRequestType responseFunction = AsterixReplicationProtocol.getRequestType(socketChannel,
+ dataBuffer);
+ AsterixLSMIndexFileProperties fileProperties;
+ while (responseFunction != ReplicationRequestType.GOODBYE) {
+ dataBuffer = AsterixReplicationProtocol.readRequest(socketChannel, dataBuffer);
+
+ fileProperties = AsterixReplicationProtocol.readFileReplicationRequest(dataBuffer);
+ destFolder = replicaResourcesManager.getIndexPath(fileProperties.getNodeId(),
+ fileProperties.getIoDeviceNum(), fileProperties.getDataverse(), fileProperties.getIdxName());
+ destFilePath = destFolder + File.separator + fileProperties.getFileName();
+
+ //create file
+ File destFile = new File(destFilePath);
+ destFile.createNewFile();
+
+ try (RandomAccessFile fileOutputStream = new RandomAccessFile(destFile, "rw");
+ FileChannel fileChannel = fileOutputStream.getChannel();) {
+ fileOutputStream.setLength(fileProperties.getFileSize());
+
+ NetworkingUtil.downloadFile(fileChannel, socketChannel);
+ fileChannel.force(true);
+ }
+
+ //we need to create LSN map for .metadata files that belong to remote replicas
+ if (!fileProperties.isLSMComponentFile() && !fileProperties.getNodeId().equals(nodeId)) {
+ //replica index
+ replicaResourcesManager.initializeReplicaIndexLSNMap(destFolder, logManager.getAppendLSN());
+ }
+
+ responseFunction = AsterixReplicationProtocol.getRequestType(socketChannel, dataBuffer);
+ }
+
+ //send goodbye
+ AsterixReplicationProtocol.sendGoodbye(socketChannel);
+ }
+ }
+
+ //Recovery Method
+ @Override
+ public long requestReplicaMinLSN(String selectedReplicaId) throws IOException {
+ long minLSN = 0;
+ AsterixReplicationProtocol.writeMinLSNRequest(dataBuffer);
+ try (SocketChannel socketChannel = getReplicaSocket(selectedReplicaId);) {
+ //transfer request
+ NetworkingUtil.transferBufferToChannel(socketChannel, dataBuffer);
+
+ //read response
+ NetworkingUtil.readBytes(socketChannel, dataBuffer, Long.BYTES);
+ minLSN = dataBuffer.getLong();
+
+ //send goodbye
+ AsterixReplicationProtocol.sendGoodbye(socketChannel);
+ }
+
+ return minLSN;
+ }
+
+ //Recovery Method
+ @Override
+ public ArrayList<ILogRecord> requestReplicaLogs(String remoteNode, Set<String> nodeIdsToRecoverFor, long fromLSN)
+ throws IOException, ACIDException {
+ ReplicaLogsRequest request = new ReplicaLogsRequest(nodeIdsToRecoverFor, fromLSN);
+ dataBuffer = AsterixReplicationProtocol.writeGetReplicaLogsRequest(dataBuffer, request);
+
+ try (SocketChannel socketChannel = getReplicaSocket(remoteNode)) {
+ //transfer request
+ NetworkingUtil.transferBufferToChannel(socketChannel, dataBuffer);
+
+ //read response type
+ ReplicationRequestType responseType = AsterixReplicationProtocol.getRequestType(socketChannel, dataBuffer);
+
+ ArrayList<ILogRecord> recoveryLogs = new ArrayList<ILogRecord>();
+ ILogRecord logRecord = new LogRecord();
+ while (responseType != ReplicationRequestType.GOODBYE) {
+ dataBuffer = AsterixReplicationProtocol.readRequest(socketChannel, dataBuffer);
+ logRecord.deserialize(dataBuffer, true, nodeId);
+
+ if (logRecord.getNodeId().equals(nodeId)) {
+ //store log in memory to replay it for recovery
+ recoveryLogs.add(logRecord);
+ //this needs to be a new log object so that it is passed to recovery manager as a different object
+ logRecord = new LogRecord();
+ } else {
+ //send log to log manager as a remote recovery log
+ logManager.log(logRecord);
+ }
+
+ responseType = AsterixReplicationProtocol.getRequestType(socketChannel, dataBuffer);
+ }
+
+ //send goodbye
+ AsterixReplicationProtocol.sendGoodbye(socketChannel);
+ return recoveryLogs;
+ }
+ }
+
+ //supporting classes
+ /**
+ * This class is responsible for processing replica events.
+ */
+ private class ReplicasEventsMonitor extends Thread {
+ ReplicaEvent event;
+
+ @Override
+ public void run() {
+ while (true) {
+ try {
+ event = replicaEventsQ.take();
+
+ switch (event.getEventType()) {
+ case FAIL:
+ handleReplicaFailure(event.getReplica().getId());
+ break;
+ case JOIN:
+ checkReplicaState(event.getReplica().getId(), false, true);
+ break;
+ case SHUTDOWN:
+ handleShutdownEvent(event.getReplica().getId());
+ break;
+ default:
+ break;
+ }
+ } catch (InterruptedException e) {
+ //ignore
+ }
+ }
+ }
+
+ public void handleReplicaFailure(String replicaId) {
+ Replica replica = replicas.get(replicaId);
+
+ if (replica.getState() == ReplicaState.DEAD) {
+ return;
+ }
+
+ updateReplicaState(replicaId, ReplicaState.DEAD, true);
+
+ //delete any invalid LSMComponents for this replica
+ try {
+ replicaResourcesManager.cleanInvalidLSMComponents(replicaId);
+ } catch (HyracksDataException e) {
+ e.printStackTrace();
+ }
+ }
+
+ public void handleShutdownEvent(String replicaId) {
+ synchronized (shuttingDownReplicaIds) {
+ shuttingDownReplicaIds.add(replicaId);
+ shuttingDownReplicaIds.notifyAll();
+ }
+ }
+ }
+
+ /**
+ * This class process is responsible for processing ASYNC replication job.
+ */
+ private class ReplicationJobsProccessor extends Thread {
+ Map<String, SocketChannel> replicaSockets;
+ ByteBuffer reusableBuffer = ByteBuffer.allocate(INITIAL_BUFFER_SIZE);
+
+ @Override
+ public void run() {
+ Thread.currentThread().setName("ReplicationJobsProccessor Thread");
+ terminateJobsReplication.set(false);
+ jobsReplicationSuspended.set(false);
+
+ while (true) {
+ try {
+ if (terminateJobsReplication.get()) {
+ closeSockets();
+ break;
+ }
+
+ IReplicationJob job = replicationJobsQ.take();
+ if (job.getOperation() != ReplicationOperation.STOP) {
+ //if there isn't already a connection, establish a new one
+ if (replicaSockets == null) {
+ replicaSockets = getActiveRemoteReplicasSockets();
+ }
+
+ processJob(job, replicaSockets, reusableBuffer);
+ } else {
+ terminateJobsReplication.set(true);
+ continue;
+ }
+
+ //if no more jobs to process, close sockets
+ if (replicationJobsQ.size() == 0) {
+ LOGGER.log(Level.INFO, "No pending replication jobs. Closing connections to replicas");
+ closeSockets();
+ }
+ } catch (Exception e) {
+ e.printStackTrace();
+ }
+ }
+
+ synchronized (jobsReplicationSuspended) {
+ jobsReplicationSuspended.set(true);
+ jobsReplicationSuspended.notifyAll();
+ }
+ LOGGER.log(Level.INFO, "ReplicationJobsProccessor stopped. ");
+ }
+
+ private void closeSockets() {
+ if (replicaSockets != null) {
+ closeReplicaSockets(replicaSockets);
+ replicaSockets.clear();
+ replicaSockets = null;
+ }
+ }
+ }
+
+ /**
+ * This class is responsible for listening on sockets that belong to TxnLogsReplicator.
+ */
+ private class TxnLogsReplicationResponseListener implements Runnable {
+ final SocketChannel replicaSocket;
+ final String replicaId;
+
+ public TxnLogsReplicationResponseListener(String replicaId, SocketChannel replicaSocket) {
+ this.replicaId = replicaId;
+ this.replicaSocket = replicaSocket;
+ }
+
+ @Override
+ public void run() {
+ Thread.currentThread().setName("TxnLogs Replication Listener Thread");
+ LOGGER.log(Level.INFO, "Started listening on socket: " + replicaSocket.socket().getRemoteSocketAddress());
+
+ try {
+ BufferedReader incomingResponse = new BufferedReader(
+ new InputStreamReader(replicaSocket.socket().getInputStream()));
+ String responseLine = "";
+ while (true) {
+ responseLine = incomingResponse.readLine();
+ if (responseLine == null) {
+ break;
+ }
+ //read ACK for job commit log
+ String replicaId = AsterixReplicationProtocol.getNodeIdFromLogAckMessage(responseLine);
+ int jobId = AsterixReplicationProtocol.getJobIdFromLogAckMessage(responseLine);
+ addAckToJob(jobId, replicaId);
+ }
+ } catch (AsynchronousCloseException e1) {
+ LOGGER.log(Level.INFO, "Replication listener stopped for remote replica: " + replicaId);
+ } catch (IOException e2) {
+ reportFailedReplica(replicaId);
+ }
+ }
+ }
+
+}
diff --git a/asterix-replication/src/main/java/org/apache/asterix/replication/recovery/RemoteRecoveryManager.java b/asterix-replication/src/main/java/org/apache/asterix/replication/recovery/RemoteRecoveryManager.java
new file mode 100644
index 0000000..f424bc3
--- /dev/null
+++ b/asterix-replication/src/main/java/org/apache/asterix/replication/recovery/RemoteRecoveryManager.java
@@ -0,0 +1,208 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.replication.recovery;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import org.apache.asterix.common.api.IAsterixAppRuntimeContext;
+import org.apache.asterix.common.api.IDatasetLifecycleManager;
+import org.apache.asterix.common.config.AsterixReplicationProperties;
+import org.apache.asterix.common.context.DatasetLifecycleManager;
+import org.apache.asterix.common.replication.IRemoteRecoveryManager;
+import org.apache.asterix.common.replication.IReplicationManager;
+import org.apache.asterix.common.transactions.ILogManager;
+import org.apache.asterix.common.transactions.ILogRecord;
+import org.apache.asterix.transaction.management.resource.PersistentLocalResourceRepository;
+
+public class RemoteRecoveryManager implements IRemoteRecoveryManager {
+
+ private final IReplicationManager replicationManager;
+ private final ILogManager logManager;
+ public static final boolean IS_DEBUG_MODE = false;//true
+ private static final Logger LOGGER = Logger.getLogger(RemoteRecoveryManager.class.getName());
+ private final IAsterixAppRuntimeContext runtimeContext;
+ private final AsterixReplicationProperties replicationProperties;
+
+ public RemoteRecoveryManager(IReplicationManager replicationManager, IAsterixAppRuntimeContext runtimeContext,
+ AsterixReplicationProperties replicationProperties) {
+ this.replicationManager = replicationManager;
+ this.runtimeContext = runtimeContext;
+ this.logManager = runtimeContext.getTransactionSubsystem().getLogManager();
+ this.replicationProperties = replicationProperties;
+ }
+
+ @Override
+ public void performRemoteRecovery() {
+ //The whole remote recovery process should be atomic.
+ //Any error happens, we should start the recovery from the start until the recovery is complete or an illegal state is reached (cannot recovery).
+ int maxRecoveryAttempts = 10;
+
+ while (true) {
+ //start recovery recovery steps
+ try {
+ maxRecoveryAttempts--;
+
+ if (maxRecoveryAttempts == 0) {
+ //to avoid infinite loop in case of unexpected behavior.
+ throw new IllegalStateException("Failed to perform remote recovery.");
+ }
+
+ /*** Prepare for Recovery ***/
+ //1. check remote replicas states
+ replicationManager.initializeReplicasState();
+ int activeReplicasCount = replicationManager.getActiveReplicasCount();
+
+ if (activeReplicasCount == 0) {
+ throw new IllegalStateException("no ACTIVE remote replica(s) exists to performe remote recovery");
+ }
+
+ //2. clean any memory data that could've existed from previous failed recovery attempt
+ IDatasetLifecycleManager datasetLifeCycleManager = runtimeContext.getDatasetLifecycleManager();
+ datasetLifeCycleManager.closeAllDatasets();
+
+ //3. remove any existing storage data
+ runtimeContext.getReplicaResourcesManager().deleteAsterixStorageData();
+
+ //4. select remote replicas to recover from per lost replica data
+ Map<String, Set<String>> selectedRemoteReplicas = constructRemoteRecoveryPlan();
+
+ //5. get max LSN from selected remote replicas
+ long maxRemoteLSN = 0;
+ maxRemoteLSN = replicationManager.getMaxRemoteLSN(selectedRemoteReplicas.keySet());
+
+ //6. force LogManager to start from a partition > maxLSN in selected remote replicas
+ logManager.renewLogFilesAndStartFromLSN(maxRemoteLSN);
+
+ /*** Start Recovery Per Lost Replica ***/
+ for (Entry<String, Set<String>> remoteReplica : selectedRemoteReplicas.entrySet()) {
+ String replicaId = remoteReplica.getKey();
+ Set<String> replicasDataToRecover = remoteReplica.getValue();
+
+ //1. Request indexes metadata and LSM components
+ replicationManager.requestReplicaFiles(replicaId, replicasDataToRecover);
+
+ //2. Initialize local resources based on the newly received files (if we are recovering the primary replica on this node)
+ if (replicasDataToRecover.contains(logManager.getNodeId())) {
+ ((PersistentLocalResourceRepository) runtimeContext.getLocalResourceRepository()).initialize(
+ logManager.getNodeId(),
+ runtimeContext.getReplicaResourcesManager().getLocalStorageFolder());
+ //initialize resource id factor to correct max resource id
+ runtimeContext.initializeResourceIdFactory();
+ }
+
+ //3. Get min LSN to start requesting logs from
+ long minLSN = replicationManager.requestReplicaMinLSN(replicaId);
+
+ //4. Request remote logs from selected remote replicas
+ ArrayList<ILogRecord> remoteRecoveryLogs = replicationManager.requestReplicaLogs(replicaId,
+ replicasDataToRecover, minLSN);
+
+ //5. Replay remote logs using recovery manager
+ if (replicasDataToRecover.contains(logManager.getNodeId())) {
+ if (remoteRecoveryLogs.size() > 0) {
+ runtimeContext.getTransactionSubsystem().getRecoveryManager()
+ .replayRemoteLogs(remoteRecoveryLogs);
+ }
+ remoteRecoveryLogs.clear();
+ }
+ }
+
+ LOGGER.log(Level.INFO, "Completed remote recovery successfully!");
+ break;
+ } catch (Exception e) {
+ e.printStackTrace();
+ LOGGER.log(Level.WARNING, "Failed during remote recovery. Attempting again...");
+ }
+ }
+ }
+
+ private Map<String, Set<String>> constructRemoteRecoveryPlan() {
+
+ //1. identify which replicas reside in this node
+ String localNodeId = logManager.getNodeId();
+ Set<String> nodes = replicationProperties.getNodeReplicasIds(localNodeId);
+
+ Map<String, Set<String>> recoveryCandidates = new HashMap<String, Set<String>>();
+ Map<String, Integer> candidatesScore = new HashMap<String, Integer>();
+
+ //2. identify which nodes has backup per lost node data
+ for (String node : nodes) {
+ Set<String> locations = replicationProperties.getNodeReplicasIds(node);
+
+ //since the local node just started, remove it from candidates
+ locations.remove(localNodeId);
+
+ //remove any dead replicas
+ Set<String> deadReplicas = replicationManager.getDeadReplicasIds();
+ for (String deadReplica : deadReplicas) {
+ locations.remove(deadReplica);
+ }
+
+ //no active replicas to recover from
+ if (locations.size() == 0) {
+ throw new IllegalStateException("Could not find any ACTIVE replica to recover " + node + " data.");
+ }
+
+ for (String location : locations) {
+ if (candidatesScore.containsKey(location)) {
+ candidatesScore.put(location, candidatesScore.get(location) + 1);
+ } else {
+ candidatesScore.put(location, 1);
+ }
+ }
+ recoveryCandidates.put(node, locations);
+ }
+
+ Map<String, Set<String>> recoveryList = new HashMap<String, Set<String>>();
+
+ //3. find best candidate to recover from per lost replica data
+ for (Entry<String, Set<String>> entry : recoveryCandidates.entrySet()) {
+
+ int winnerScore = -1;
+ String winner = "";
+ for (String node : entry.getValue()) {
+
+ int nodeScore = candidatesScore.get(node);
+
+ if (nodeScore > winnerScore) {
+ winnerScore = nodeScore;
+ winner = node;
+ }
+ }
+
+ if (recoveryList.containsKey(winner)) {
+ recoveryList.get(winner).add(entry.getKey());
+ } else {
+ Set<String> nodesToRecover = new HashSet<String>();
+ nodesToRecover.add(entry.getKey());
+ recoveryList.put(winner, nodesToRecover);
+ }
+
+ }
+
+ return recoveryList;
+ }
+}
\ No newline at end of file
diff --git a/asterix-replication/src/main/java/org/apache/asterix/replication/storage/AsterixFilesUtil.java b/asterix-replication/src/main/java/org/apache/asterix/replication/storage/AsterixFilesUtil.java
new file mode 100644
index 0000000..67b39c4
--- /dev/null
+++ b/asterix-replication/src/main/java/org/apache/asterix/replication/storage/AsterixFilesUtil.java
@@ -0,0 +1,78 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.replication.storage;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.FileVisitResult;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.nio.file.SimpleFileVisitor;
+import java.nio.file.attribute.BasicFileAttributes;
+
+public class AsterixFilesUtil {
+
+ public static void deleteFolder(String folderPath) throws IOException {
+ File folder = new File(folderPath);
+ if (folder.exists()) {
+ //delete files inside the folder
+ while (deleteDirecotryFiles(folderPath) != true) {
+ //if there is a file being written (e.g. LSM Component), wait and try again to delete the file
+ try {
+ Thread.sleep(500);
+ } catch (InterruptedException e) {
+ //ignore
+ }
+ }
+
+ //delete the folder itself
+ folder.delete();
+ }
+ }
+
+ private static boolean deleteDirecotryFiles(String dirPath) throws IOException {
+ try {
+ Path directory = Paths.get(dirPath);
+ Files.walkFileTree(directory, new SimpleFileVisitor<Path>() {
+ @Override
+ public FileVisitResult visitFile(Path file, BasicFileAttributes attrs) throws IOException {
+ Files.delete(file);
+ return FileVisitResult.CONTINUE;
+ }
+
+ @Override
+ public FileVisitResult postVisitDirectory(Path dir, IOException exc) throws IOException {
+ Files.delete(dir);
+ return FileVisitResult.CONTINUE;
+ }
+
+ @Override
+ public FileVisitResult visitFileFailed(Path file, IOException exc) throws IOException {
+ return FileVisitResult.CONTINUE;
+ }
+
+ });
+ return true;
+ } catch (Exception e) {
+ e.printStackTrace();
+ return false;
+ }
+ }
+}
diff --git a/asterix-replication/src/main/java/org/apache/asterix/replication/storage/AsterixLSMIndexFileProperties.java b/asterix-replication/src/main/java/org/apache/asterix/replication/storage/AsterixLSMIndexFileProperties.java
new file mode 100644
index 0000000..30f2afc
--- /dev/null
+++ b/asterix-replication/src/main/java/org/apache/asterix/replication/storage/AsterixLSMIndexFileProperties.java
@@ -0,0 +1,189 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.replication.storage;
+
+import java.io.DataInput;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.IOException;
+import java.io.OutputStream;
+
+import org.apache.hyracks.storage.am.common.util.IndexFileNameUtil;
+
+public class AsterixLSMIndexFileProperties {
+
+ private String fileName;
+ private long fileSize;
+ private String nodeId;
+ private String dataverse;
+ private int ioDeviceNum;
+ private String idxName;
+ private boolean lsmComponentFile;
+ private boolean requireLSNSync;
+ private String filePath;
+ private boolean requiresAck = false;
+
+ public AsterixLSMIndexFileProperties() {
+ }
+
+ public AsterixLSMIndexFileProperties(String filePath, long fileSize, String nodeId, boolean lsmComponentFile,
+ boolean requireLSNSync, boolean requiresAck) {
+ initialize(filePath, fileSize, nodeId, lsmComponentFile, requireLSNSync, requiresAck);
+ }
+
+ public AsterixLSMIndexFileProperties(LSMComponentProperties lsmComponentProperties) {
+ initialize(lsmComponentProperties.getComponentId(), -1, lsmComponentProperties.getNodeId(), false, false, false);
+ }
+
+ public void initialize(String filePath, long fileSize, String nodeId, boolean lsmComponentFile,
+ boolean requireLSNSync, boolean requiresAck) {
+ this.filePath = filePath;
+ this.fileSize = fileSize;
+ this.nodeId = nodeId;
+ String[] tokens = filePath.split(File.separator);
+
+ int arraySize = tokens.length;
+ this.fileName = tokens[arraySize - 1];
+ this.ioDeviceNum = getDeviceIONumFromName(tokens[arraySize - 2]);
+ this.idxName = tokens[arraySize - 3];
+ this.dataverse = tokens[arraySize - 4];
+ this.lsmComponentFile = lsmComponentFile;
+ this.requireLSNSync = requireLSNSync;
+ this.requiresAck = requiresAck;
+ }
+
+ public static int getDeviceIONumFromName(String name) {
+ return Integer.parseInt(name.substring(IndexFileNameUtil.IO_DEVICE_NAME_PREFIX.length()));
+ }
+
+ public void serialize(OutputStream out) throws IOException {
+ DataOutputStream dos = new DataOutputStream(out);
+ dos.writeUTF(nodeId);
+ dos.writeUTF(filePath);
+ dos.writeLong(fileSize);
+ dos.writeBoolean(lsmComponentFile);
+ dos.writeBoolean(requireLSNSync);
+ dos.writeBoolean(requiresAck);
+ }
+
+ public static AsterixLSMIndexFileProperties create(DataInput input) throws IOException {
+ String nodeId = input.readUTF();
+ String filePath = input.readUTF();
+ long fileSize = input.readLong();
+ boolean lsmComponentFile = input.readBoolean();
+ boolean requireLSNSync = input.readBoolean();
+ boolean requiresAck = input.readBoolean();
+ AsterixLSMIndexFileProperties fileProp = new AsterixLSMIndexFileProperties();
+ fileProp.initialize(filePath, fileSize, nodeId, lsmComponentFile, requireLSNSync, requiresAck);
+ return fileProp;
+ }
+
+ public String getFilePath() {
+ return filePath;
+ }
+
+ public void setFilePath(String filePath) {
+ this.filePath = filePath;
+ }
+
+ public long getFileSize() {
+ return fileSize;
+ }
+
+ public String getFileName() {
+ return fileName;
+ }
+
+ public void setFileName(String fileName) {
+ this.fileName = fileName;
+ }
+
+ public String getNodeId() {
+ return nodeId;
+ }
+
+ public void setNodeId(String nodeId) {
+ this.nodeId = nodeId;
+ }
+
+ public String getDataverse() {
+ return dataverse;
+ }
+
+ public void setDataverse(String dataverse) {
+ this.dataverse = dataverse;
+ }
+
+ public void setFileSize(long fileSize) {
+ this.fileSize = fileSize;
+ }
+
+ public int getIoDeviceNum() {
+ return ioDeviceNum;
+ }
+
+ public void setIoDeviceNum(int ioDevoceNum) {
+ this.ioDeviceNum = ioDevoceNum;
+ }
+
+ public String getIdxName() {
+ return idxName;
+ }
+
+ public void setIdxName(String idxName) {
+ this.idxName = idxName;
+ }
+
+ public boolean isLSMComponentFile() {
+ return lsmComponentFile;
+ }
+
+ public void setLsmComponentFile(boolean lsmComponentFile) {
+ this.lsmComponentFile = lsmComponentFile;
+ }
+
+ public boolean isRequireLSNSync() {
+ return requireLSNSync;
+ }
+
+ public void setRequireLSNSync(boolean requireLSNSync) {
+ this.requireLSNSync = requireLSNSync;
+ }
+
+ public boolean requiresAck() {
+ return requiresAck;
+ }
+
+ public void setRequiresAck(boolean requiresAck) {
+ this.requiresAck = requiresAck;
+ }
+
+ @Override
+ public String toString() {
+ StringBuilder sb = new StringBuilder();
+ sb.append("File Name: " + fileName + " ");
+ sb.append("File Size: " + fileSize + " ");
+ sb.append("Node ID: " + nodeId + " ");
+ sb.append("I/O Device: " + ioDeviceNum + " ");
+ sb.append("IDX Name: " + idxName + " ");
+ sb.append("isLSMComponentFile : " + lsmComponentFile + " ");
+ sb.append("Dataverse: " + dataverse);
+ return sb.toString();
+ }
+}
\ No newline at end of file
diff --git a/asterix-replication/src/main/java/org/apache/asterix/replication/storage/LSMComponentLSNSyncTask.java b/asterix-replication/src/main/java/org/apache/asterix/replication/storage/LSMComponentLSNSyncTask.java
new file mode 100644
index 0000000..69f7d07
--- /dev/null
+++ b/asterix-replication/src/main/java/org/apache/asterix/replication/storage/LSMComponentLSNSyncTask.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.replication.storage;
+
+public class LSMComponentLSNSyncTask {
+ private String componentFilePath;
+ private String componentId;
+
+ public LSMComponentLSNSyncTask(String componentId, String componentFilePath) {
+ this.componentId = componentId;
+ this.componentFilePath = componentFilePath;
+ }
+
+ public String getComponentFilePath() {
+ return componentFilePath;
+ }
+
+ public void setComponentFilePath(String componentFilePath) {
+ this.componentFilePath = componentFilePath;
+ }
+
+ public String getComponentId() {
+ return componentId;
+ }
+
+ public void setComponentId(String componentId) {
+ this.componentId = componentId;
+ }
+}
diff --git a/asterix-replication/src/main/java/org/apache/asterix/replication/storage/LSMComponentProperties.java b/asterix-replication/src/main/java/org/apache/asterix/replication/storage/LSMComponentProperties.java
new file mode 100644
index 0000000..84d5dbe
--- /dev/null
+++ b/asterix-replication/src/main/java/org/apache/asterix/replication/storage/LSMComponentProperties.java
@@ -0,0 +1,203 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.replication.storage;
+
+import java.io.DataInput;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.asterix.common.ioopcallbacks.AbstractLSMIOOperationCallback;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.common.frames.LIFOMetaDataFrame;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexReplicationJob;
+import org.apache.hyracks.storage.am.lsm.common.api.LSMOperationType;
+import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndex;
+import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndexFileManager;
+
+public class LSMComponentProperties {
+
+ private AtomicInteger numberOfFiles;
+ private String componentId;
+ private long LSNOffset;
+ private long originalLSN;
+ private String nodeId;
+ private Long replicaLSN;
+ private String maskPath = null;
+ private String replicaPath = null;
+ private LSMOperationType opType;
+
+ public LSMComponentProperties(ILSMIndexReplicationJob job, String nodeId) {
+ this.nodeId = nodeId;
+ componentId = LSMComponentProperties.getLSMComponentID((String) job.getJobFiles().toArray()[0], nodeId);
+ numberOfFiles = new AtomicInteger(job.getJobFiles().size());
+ originalLSN = getLSMComponentLSN((AbstractLSMIndex) job.getLSMIndex(), job.getLSMIndexOperationContext());
+ //TODO this should be changed to a dynamic value when append only LSM indexes are implemented
+ LSNOffset = LIFOMetaDataFrame.lsnOff;
+ opType = job.getLSMOpType();
+ }
+
+ public LSMComponentProperties() {
+
+ }
+
+ public long getLSMComponentLSN(AbstractLSMIndex lsmIndex, ILSMIndexOperationContext ctx) {
+ long componentLSN = -1;
+ try {
+ componentLSN = ((AbstractLSMIOOperationCallback) lsmIndex.getIOOperationCallback()).getComponentLSN(ctx
+ .getComponentsToBeReplicated());
+ } catch (HyracksDataException e) {
+ e.printStackTrace();
+ }
+ if (componentLSN < 0) {
+ componentLSN = 0;
+ }
+ return componentLSN;
+ }
+
+ public void serialize(OutputStream out) throws IOException {
+ DataOutputStream dos = new DataOutputStream(out);
+ dos.writeUTF(componentId);
+ dos.writeUTF(nodeId);
+ dos.writeInt(numberOfFiles.get());
+ dos.writeLong(originalLSN);
+ dos.writeLong(LSNOffset);
+ dos.writeInt(opType.ordinal());
+ }
+
+ public static LSMComponentProperties create(DataInput input) throws IOException {
+ LSMComponentProperties lsmCompProp = new LSMComponentProperties();
+ lsmCompProp.componentId = input.readUTF();
+ lsmCompProp.nodeId = input.readUTF();
+ lsmCompProp.numberOfFiles = new AtomicInteger(input.readInt());
+ lsmCompProp.originalLSN = input.readLong();
+ lsmCompProp.LSNOffset = input.readLong();
+ lsmCompProp.opType = LSMOperationType.values()[input.readInt()];
+ return lsmCompProp;
+ }
+
+ public String getMaskPath(ReplicaResourcesManager resourceManager) {
+ if (maskPath == null) {
+ AsterixLSMIndexFileProperties afp = new AsterixLSMIndexFileProperties(this);
+ maskPath = getReplicaComponentPath(resourceManager) + File.separator + afp.getFileName()
+ + ReplicaResourcesManager.LSM_COMPONENT_MASK_SUFFIX;
+ }
+ return maskPath;
+ }
+
+ public String getReplicaComponentPath(ReplicaResourcesManager resourceManager) {
+ if (replicaPath == null) {
+ AsterixLSMIndexFileProperties afp = new AsterixLSMIndexFileProperties(this);
+ replicaPath = resourceManager.getIndexPath(afp.getNodeId(), afp.getIoDeviceNum(), afp.getDataverse(),
+ afp.getIdxName());
+ }
+ return replicaPath;
+ }
+
+ /***
+ * @param filePath
+ * any file of the LSM component
+ * @param nodeId
+ * @return a unique id based on the timestamp of the component
+ */
+ public static String getLSMComponentID(String filePath, String nodeId) {
+ String[] tokens = filePath.split(File.separator);
+
+ int arraySize = tokens.length;
+ String fileName = tokens[arraySize - 1];
+ String ioDevoceName = tokens[arraySize - 2];
+ String idxName = tokens[arraySize - 3];
+ String dataverse = tokens[arraySize - 4];
+
+ StringBuilder componentId = new StringBuilder();
+ componentId.append(nodeId);
+ componentId.append(File.separator);
+ componentId.append(dataverse);
+ componentId.append(File.separator);
+ componentId.append(idxName);
+ componentId.append(File.separator);
+ componentId.append(ioDevoceName);
+ componentId.append(File.separator);
+ componentId.append(fileName.substring(0, fileName.lastIndexOf(AbstractLSMIndexFileManager.SPLIT_STRING)));
+ return componentId.toString();
+ }
+
+ public String getComponentId() {
+ return componentId;
+ }
+
+ public void setComponentId(String componentId) {
+ this.componentId = componentId;
+ }
+
+ public long getLSNOffset() {
+ return LSNOffset;
+ }
+
+ public void setLSNOffset(long lSNOffset) {
+ LSNOffset = lSNOffset;
+ }
+
+ public long getOriginalLSN() {
+ return originalLSN;
+ }
+
+ public void setOriginalLSN(long lSN) {
+ originalLSN = lSN;
+ }
+
+ public String getNodeId() {
+ return nodeId;
+ }
+
+ public void setNodeId(String nodeId) {
+ this.nodeId = nodeId;
+ }
+
+ public int getNumberOfFiles() {
+ return numberOfFiles.get();
+ }
+
+ public int markFileComplete() {
+ return numberOfFiles.decrementAndGet();
+ }
+
+ public void setNumberOfFiles(AtomicInteger numberOfFiles) {
+ this.numberOfFiles = numberOfFiles;
+ }
+
+ public Long getReplicaLSN() {
+ return replicaLSN;
+ }
+
+ public void setReplicaLSN(Long replicaLSN) {
+ this.replicaLSN = replicaLSN;
+ }
+
+ public LSMOperationType getOpType() {
+ return opType;
+ }
+
+ public void setOpType(LSMOperationType opType) {
+ this.opType = opType;
+ }
+}
diff --git a/asterix-replication/src/main/java/org/apache/asterix/replication/storage/ReplicaResourcesManager.java b/asterix-replication/src/main/java/org/apache/asterix/replication/storage/ReplicaResourcesManager.java
new file mode 100644
index 0000000..5e321f1
--- /dev/null
+++ b/asterix-replication/src/main/java/org/apache/asterix/replication/storage/ReplicaResourcesManager.java
@@ -0,0 +1,415 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.replication.storage;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.FilenameFilter;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import org.apache.asterix.common.replication.IReplicaResourcesManager;
+import org.apache.asterix.transaction.management.resource.PersistentLocalResourceRepository;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.IODeviceHandle;
+import org.apache.hyracks.storage.am.common.util.IndexFileNameUtil;
+import org.apache.hyracks.storage.common.file.LocalResource;
+
+public class ReplicaResourcesManager implements IReplicaResourcesManager {
+ private static final Logger LOGGER = Logger.getLogger(ReplicaResourcesManager.class.getName());
+ private final String[] mountPoints;
+ private final int numIODevices;
+ private static final String REPLICA_FOLDER_SUFFIX = "_replica";
+ private final String replicationStorageFolder;
+ public final String localStorageFolder;
+ private final String localNodeID;
+ public final static String LSM_COMPONENT_MASK_SUFFIX = "_mask";
+ private final static String REPLICA_INDEX_LSN_MAP_NAME = ".LSN_MAP";
+ public static final long REPLICA_INDEX_CREATION_LSN = -1;
+ private final AtomicLong lastMinRemoteLSN;
+
+ public ReplicaResourcesManager(List<IODeviceHandle> devices, String localStorageFolder, String localNodeID,
+ String replicationStorageFolder) throws HyracksDataException {
+ numIODevices = devices.size();
+ this.mountPoints = new String[numIODevices];
+ for (int i = 0; i < numIODevices; i++) {
+ String mountPoint = devices.get(i).getPath().getPath();
+ File mountPointDir = new File(mountPoint);
+ if (!mountPointDir.exists()) {
+ throw new HyracksDataException(mountPointDir.getAbsolutePath() + " doesn't exist.");
+ }
+ if (!mountPoint.endsWith(System.getProperty("file.separator"))) {
+ mountPoints[i] = new String(mountPoint + System.getProperty("file.separator"));
+ } else {
+ mountPoints[i] = new String(mountPoint);
+ }
+ }
+ this.localStorageFolder = localStorageFolder;
+ this.localNodeID = localNodeID;
+ this.replicationStorageFolder = replicationStorageFolder;
+ lastMinRemoteLSN = new AtomicLong(-1);
+ }
+
+ @Override
+ public String getLocalStorageFolder() {
+ return localStorageFolder;
+ }
+
+ private String getReplicaStorageFolder(String replicaId, int IODeviceNum) {
+ if (replicaId.equals(localNodeID)) {
+ return mountPoints[IODeviceNum] + localStorageFolder;
+ } else {
+ return mountPoints[IODeviceNum] + replicationStorageFolder + File.separator + replicaId
+ + REPLICA_FOLDER_SUFFIX;
+ }
+ }
+
+ public void deleteRemoteFile(AsterixLSMIndexFileProperties afp) throws IOException {
+ String indexPath = getIndexPath(afp.getNodeId(), afp.getIoDeviceNum(), afp.getDataverse(), afp.getIdxName());
+ if (indexPath != null) {
+ if (afp.isLSMComponentFile()) {
+ String backupFilePath = indexPath + File.separator + afp.getFileName();
+
+ //delete file
+ File destFile = new File(backupFilePath);
+ if (destFile.exists()) {
+ destFile.delete();
+ }
+ } else {
+ //delete index files
+ indexPath = indexPath.substring(0, indexPath.lastIndexOf(File.separator));
+ AsterixFilesUtil.deleteFolder(indexPath);
+ }
+ }
+ }
+
+ public String getIndexPath(String replicaId, int IODeviceNum, String dataverse, String dataset) {
+ //mounting point/backupNodeId_replica/Dataverse/Dataset/device_id_#/
+ String remoteIndexFolderPath = getReplicaStorageFolder(replicaId, IODeviceNum) + File.separator + dataverse
+ + File.separator + dataset + File.separator + IndexFileNameUtil.IO_DEVICE_NAME_PREFIX + IODeviceNum;
+ Path path = Paths.get(remoteIndexFolderPath);
+ if (!Files.exists(path)) {
+ File indexFolder = new File(remoteIndexFolderPath);
+ indexFolder.mkdirs();
+ }
+ return remoteIndexFolderPath;
+ }
+
+ public void initializeReplicaIndexLSNMap(String indexPath, long currentLSN) throws IOException {
+ HashMap<Long, Long> lsnMap = new HashMap<Long, Long>();
+ lsnMap.put(REPLICA_INDEX_CREATION_LSN, currentLSN);
+ updateReplicaIndexLSNMap(indexPath, lsnMap);
+ }
+
+ public void createRemoteLSMComponentMask(LSMComponentProperties lsmComponentProperties) throws IOException {
+ String maskPath = lsmComponentProperties.getMaskPath(this);
+ Path path = Paths.get(maskPath);
+ if (!Files.exists(path)) {
+ File maskFile = new File(maskPath);
+ maskFile.createNewFile();
+ }
+ }
+
+ public void markLSMComponentReplicaAsValid(LSMComponentProperties lsmComponentProperties) throws IOException {
+ //remove mask to mark component as valid
+ String maskPath = lsmComponentProperties.getMaskPath(this);
+ Path path = Paths.get(maskPath);
+
+ if (Files.exists(path)) {
+ File maskFile = new File(maskPath);
+ maskFile.delete();
+ }
+
+ //add component LSN to the index LSNs map
+ HashMap<Long, Long> lsnMap = getReplicaIndexLSNMap(lsmComponentProperties.getReplicaComponentPath(this));
+ lsnMap.put(lsmComponentProperties.getOriginalLSN(), lsmComponentProperties.getReplicaLSN());
+
+ //update map on disk
+ updateReplicaIndexLSNMap(lsmComponentProperties.getReplicaComponentPath(this), lsnMap);
+
+ }
+
+ public List<String> getResourcesForReplica(String nodeId) throws HyracksDataException {
+ List<String> resourcesList = new ArrayList<String>();
+ String rootFolder;
+ for (int i = 0; i < numIODevices; i++) {
+ rootFolder = getReplicaStorageFolder(nodeId, i);
+ File rootDirFile = new File(rootFolder);
+ if (!rootDirFile.exists()) {
+ continue;
+ }
+
+ File[] dataverseFileList = rootDirFile.listFiles();
+ for (File dataverseFile : dataverseFileList) {
+ if (dataverseFile.isDirectory()) {
+ File[] indexFileList = dataverseFile.listFiles();
+ if (indexFileList != null) {
+ for (File indexFile : indexFileList) {
+ if (indexFile.isDirectory()) {
+ File[] ioDevicesList = indexFile.listFiles();
+ if (ioDevicesList != null) {
+ for (File ioDeviceFile : ioDevicesList) {
+ if (ioDeviceFile.isDirectory()) {
+ File[] metadataFiles = ioDeviceFile.listFiles(LSM_INDEX_FILES_FILTER);
+ if (metadataFiles != null) {
+ for (File metadataFile : metadataFiles) {
+ resourcesList.add(metadataFile.getAbsolutePath());
+ }
+ }
+ }
+ }
+ }
+ }
+ }
+ }
+ }
+ }
+ }
+ return resourcesList;
+ }
+
+ public Set<File> getReplicaIndexes(String replicaId) throws HyracksDataException {
+ Set<File> remoteIndexesPaths = new HashSet<File>();
+ for (int i = 0; i < numIODevices; i++) {
+ String rootReplicaFolder = getReplicaStorageFolder(replicaId, i);
+ File rootDirFile = new File(rootReplicaFolder);
+ if (!rootDirFile.exists()) {
+ continue;
+ }
+ File[] dataverseFileList = rootDirFile.listFiles();
+ for (File dataverseFile : dataverseFileList) {
+ if (dataverseFile.isDirectory()) {
+ File[] indexFileList = dataverseFile.listFiles();
+ if (indexFileList != null) {
+ for (File indexFile : indexFileList) {
+ if (indexFile.isDirectory()) {
+ File[] ioDevicesList = indexFile.listFiles();
+ if (ioDevicesList != null) {
+ for (File ioDeviceFile : ioDevicesList) {
+ if (ioDeviceFile.isDirectory()) {
+ remoteIndexesPaths.add(ioDeviceFile);
+ }
+ }
+ }
+ }
+ }
+ }
+ }
+ }
+ }
+ return remoteIndexesPaths;
+ }
+
+ @Override
+ public long getMinRemoteLSN(Set<String> replicaIds) {
+ if (lastMinRemoteLSN.get() != -1) {
+ return lastMinRemoteLSN.get();
+ }
+ long minRemoteLSN = Long.MAX_VALUE;
+ for (String replica : replicaIds) {
+ try {
+ //for every index in replica
+ Set<File> remoteIndexes = getReplicaIndexes(replica);
+ for (File indexFolder : remoteIndexes) {
+ //read LSN map
+ try {
+ //get max LSN per index
+ long remoteIndexMaxLSN = getReplicaIndexMaxLSN(indexFolder);
+
+ //get min of all maximums
+ minRemoteLSN = Math.min(minRemoteLSN, remoteIndexMaxLSN);
+ } catch (IOException e) {
+ LOGGER.log(Level.INFO, indexFolder.getAbsolutePath() + " Couldn't read LSN map for index "
+ + indexFolder);
+ continue;
+ }
+ }
+ } catch (HyracksDataException e) {
+ e.printStackTrace();
+ }
+ }
+ lastMinRemoteLSN.set(minRemoteLSN);
+ return minRemoteLSN;
+ }
+
+ public HashMap<Long, String> getLaggingReplicaIndexesId2PathMap(String replicaId, long targetLSN)
+ throws IOException {
+ HashMap<Long, String> laggingReplicaIndexes = new HashMap<Long, String>();
+ try {
+ //for every index in replica
+ Set<File> remoteIndexes = getReplicaIndexes(replicaId);
+ for (File indexFolder : remoteIndexes) {
+ if (getReplicaIndexMaxLSN(indexFolder) < targetLSN) {
+ File localResource = new File(indexFolder + File.separator
+ + PersistentLocalResourceRepository.METADATA_FILE_NAME);
+ LocalResource resource = PersistentLocalResourceRepository.readLocalResource(localResource);
+ laggingReplicaIndexes.put(resource.getResourceId(), indexFolder.getAbsolutePath());
+ }
+ }
+ } catch (HyracksDataException e) {
+ e.printStackTrace();
+ }
+
+ return laggingReplicaIndexes;
+ }
+
+ private long getReplicaIndexMaxLSN(File indexFolder) throws IOException {
+ long remoteIndexMaxLSN = 0;
+ //get max LSN per index
+ HashMap<Long, Long> lsnMap = getReplicaIndexLSNMap(indexFolder.getAbsolutePath());
+ if (lsnMap != null) {
+ for (Long lsn : lsnMap.values()) {
+ remoteIndexMaxLSN = Math.max(remoteIndexMaxLSN, lsn);
+ }
+ }
+
+ return remoteIndexMaxLSN;
+ }
+
+ public void cleanInvalidLSMComponents(String replicaId) throws HyracksDataException {
+ //for every index in replica
+ Set<File> remoteIndexes = getReplicaIndexes(replicaId);
+ for (File remoteIndexFile : remoteIndexes) {
+ //search for any mask
+ File[] masks = remoteIndexFile.listFiles(LSM_COMPONENTS_MASKS_FILTER);
+
+ for (File mask : masks) {
+ //delete all files belonging to this mask
+ deleteLSMComponentFilesForMask(mask);
+ //delete the mask itself
+ mask.delete();
+ }
+ }
+ }
+
+ private void deleteLSMComponentFilesForMask(File maskFile) {
+ String lsmComponentTimeStamp = maskFile.getName().substring(0,
+ maskFile.getName().length() - LSM_COMPONENT_MASK_SUFFIX.length());
+ File indexFolder = maskFile.getParentFile();
+ File[] lsmComponentsFiles = indexFolder.listFiles(LSM_COMPONENTS_NON_MASKS_FILTER);
+ for (File lsmComponentFile : lsmComponentsFiles) {
+ if (lsmComponentFile.getName().contains(lsmComponentTimeStamp)) {
+ //match based on time stamp
+ lsmComponentFile.delete();
+ }
+ }
+ }
+
+ @SuppressWarnings("unchecked")
+ public synchronized HashMap<Long, Long> getReplicaIndexLSNMap(String indexPath) throws IOException {
+ FileInputStream fis = null;
+ ObjectInputStream oisFromFis = null;
+ try {
+ fis = new FileInputStream(indexPath + File.separator + REPLICA_INDEX_LSN_MAP_NAME);
+ oisFromFis = new ObjectInputStream(fis);
+ Map<Long, Long> lsnMap = null;
+ try {
+ lsnMap = (Map<Long, Long>) oisFromFis.readObject();
+ } catch (ClassNotFoundException e) {
+ e.printStackTrace();
+ }
+ return (HashMap<Long, Long>) lsnMap;
+ } finally {
+ if (oisFromFis != null) {
+ oisFromFis.close();
+ }
+ if (oisFromFis == null && fis != null) {
+ fis.close();
+ }
+ }
+ }
+
+ public synchronized void updateReplicaIndexLSNMap(String indexPath, HashMap<Long, Long> lsnMap) throws IOException {
+ FileOutputStream fos = null;
+ ObjectOutputStream oosToFos = null;
+ try {
+ fos = new FileOutputStream(indexPath + File.separator + REPLICA_INDEX_LSN_MAP_NAME);
+ oosToFos = new ObjectOutputStream(fos);
+ oosToFos.writeObject(lsnMap);
+ oosToFos.flush();
+ lastMinRemoteLSN.set(-1);
+ } finally {
+ if (oosToFos != null) {
+ oosToFos.close();
+ }
+ if (oosToFos == null && fos != null) {
+ fos.close();
+ }
+ }
+ }
+
+ private static final FilenameFilter LSM_COMPONENTS_MASKS_FILTER = new FilenameFilter() {
+ public boolean accept(File dir, String name) {
+ if (name.endsWith(LSM_COMPONENT_MASK_SUFFIX)) {
+ return true;
+ } else {
+ return false;
+ }
+ }
+ };
+
+ private static final FilenameFilter LSM_COMPONENTS_NON_MASKS_FILTER = new FilenameFilter() {
+ public boolean accept(File dir, String name) {
+ if (!name.endsWith(LSM_COMPONENT_MASK_SUFFIX)) {
+ return true;
+ } else {
+ return false;
+ }
+ }
+ };
+
+ private static final FilenameFilter LSM_INDEX_FILES_FILTER = new FilenameFilter() {
+ public boolean accept(File dir, String name) {
+ if (name.equalsIgnoreCase(PersistentLocalResourceRepository.METADATA_FILE_NAME)) {
+ return true;
+ } else if (!name.startsWith(".")) {
+ return true;
+ } else {
+ return false;
+ }
+ }
+ };
+
+ @Override
+ public void deleteAsterixStorageData() throws IOException {
+ for (int i = 0; i < mountPoints.length; i++) {
+ File mountingPoint = new File(mountPoints[i]);
+
+ File[] storageFolders = mountingPoint.listFiles();
+
+ for (File storageFolder : storageFolders) {
+ AsterixFilesUtil.deleteFolder(storageFolder.getAbsolutePath());
+ }
+ }
+ }
+}
\ No newline at end of file
diff --git a/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/AbstractIndexModificationOperationCallback.java b/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/AbstractIndexModificationOperationCallback.java
index b76c94c..6646ffb 100644
--- a/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/AbstractIndexModificationOperationCallback.java
+++ b/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/AbstractIndexModificationOperationCallback.java
@@ -56,6 +56,7 @@
logRecord.setDatasetId(datasetId);
logRecord.setResourceId(resourceId);
logRecord.setNewOp((byte) (indexOp.ordinal()));
+ logRecord.setNodeId(txnSubsystem.getId());
}
protected void log(int PKHash, ITupleReference newValue)
diff --git a/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepository.java b/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepository.java
index 1eda9cc..6fa60ca 100644
--- a/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepository.java
+++ b/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepository.java
@@ -26,12 +26,19 @@
import java.io.ObjectInputStream;
import java.io.ObjectOutputStream;
import java.util.HashMap;
+import java.util.HashSet;
import java.util.List;
+import java.util.Set;
import java.util.logging.Level;
import java.util.logging.Logger;
+import org.apache.asterix.common.replication.AsterixReplicationJob;
+import org.apache.asterix.common.replication.IReplicationManager;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.api.io.IODeviceHandle;
+import org.apache.hyracks.api.replication.IReplicationJob.ReplicationExecutionType;
+import org.apache.hyracks.api.replication.IReplicationJob.ReplicationJobType;
+import org.apache.hyracks.api.replication.IReplicationJob.ReplicationOperation;
import org.apache.hyracks.storage.common.file.ILocalResourceRepository;
import org.apache.hyracks.storage.common.file.LocalResource;
@@ -45,11 +52,14 @@
private static final String ROOT_METADATA_DIRECTORY = "asterix_root_metadata";
private static final String ROOT_METADATA_FILE_NAME_PREFIX = ".asterix_root_metadata";
private static final long ROOT_LOCAL_RESOURCE_ID = -4321;
- private static final String METADATA_FILE_NAME = ".metadata";
+ public static final String METADATA_FILE_NAME = ".metadata";
private final Cache<String, LocalResource> resourceCache;
private final String nodeId;
private static final int MAX_CACHED_RESOURCES = 1000;
-
+ private IReplicationManager replicationManager;
+ private boolean isReplicationEnabled = false;
+ private Set<String> filesToBeReplicated;
+
public PersistentLocalResourceRepository(List<IODeviceHandle> devices, String nodeId) throws HyracksDataException {
mountPoints = new String[devices.size()];
this.nodeId = nodeId;
@@ -168,6 +178,12 @@
throw new HyracksDataException(e);
}
}
+
+ //if replication enabled, send resource metadata info to remote nodes
+ if (isReplicationEnabled && resource.getResourceId() != ROOT_LOCAL_RESOURCE_ID) {
+ String filePath = getFileName(resource.getResourceName(), resource.getResourceId());
+ createReplicationJob(ReplicationOperation.REPLICATE, filePath);
+ }
}
}
@@ -177,6 +193,11 @@
if (resourceFile.exists()) {
resourceFile.delete();
resourceCache.invalidate(name);
+
+ //if replication enabled, delete resource from remote replicas
+ if (isReplicationEnabled && !resourceFile.getName().startsWith(ROOT_METADATA_FILE_NAME_PREFIX)) {
+ createReplicationJob(ReplicationOperation.DELETE, resourceFile.getAbsolutePath());
+ }
} else {
throw new HyracksDataException("Resource doesn't exist");
}
@@ -310,7 +331,7 @@
}
}
- private LocalResource readLocalResource(File file) throws HyracksDataException {
+ public static LocalResource readLocalResource(File file) throws HyracksDataException {
FileInputStream fis = null;
ObjectInputStream oisFromFis = null;
@@ -348,4 +369,29 @@
}
}
};
+ public void setReplicationManager(IReplicationManager replicationManager) {
+ this.replicationManager = replicationManager;
+ isReplicationEnabled = replicationManager.isReplicationEnabled();
+
+ if (isReplicationEnabled) {
+ filesToBeReplicated = new HashSet<String>();
+ }
+ }
+
+ private void createReplicationJob(ReplicationOperation operation, String filePath) throws HyracksDataException {
+ filesToBeReplicated.clear();
+ filesToBeReplicated.add(filePath);
+ AsterixReplicationJob job = new AsterixReplicationJob(ReplicationJobType.METADATA, operation,
+ ReplicationExecutionType.SYNC, filesToBeReplicated);
+ try {
+ replicationManager.submitJob(job);
+ } catch (IOException e) {
+ throw new HyracksDataException(e);
+ }
+ }
+
+ public String[] getStorageMountingPoints() {
+ return mountPoints;
+ }
+
}
diff --git a/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/ConcurrentLockManager.java b/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/ConcurrentLockManager.java
index de8daf8..3114195 100644
--- a/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/ConcurrentLockManager.java
+++ b/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/ConcurrentLockManager.java
@@ -48,10 +48,9 @@
*/
public class ConcurrentLockManager implements ILockManager, ILifeCycleComponent {
- private static final Logger LOGGER
- = Logger.getLogger(ConcurrentLockManager.class.getName());
+ private static final Logger LOGGER = Logger.getLogger(ConcurrentLockManager.class.getName());
private static final Level LVL = Level.FINER;
-
+
public static final boolean DEBUG_MODE = false;//true
public static final boolean CHECK_CONSISTENCY = false;
@@ -62,8 +61,8 @@
private JobArenaManager jobArenaMgr;
private ConcurrentHashMap<Integer, Long> jobIdSlotMap;
private ThreadLocal<DatasetLockCache> dsLockCache;
- private LockManagerStats stats = new LockManagerStats(10000);
-
+ private LockManagerStats stats = new LockManagerStats(10000);
+
enum LockAction {
ERR(false, false),
GET(false, false),
@@ -118,8 +117,8 @@
throws ACIDException {
log("lock", datasetId.getId(), entityHashValue, lockMode, txnContext);
stats.lock();
-
- final int dsId = datasetId.getId();
+
+ final int dsId = datasetId.getId();
final int jobId = txnContext.getJobId().getId();
if (entityHashValue != -1) {
@@ -131,7 +130,7 @@
}
final long jobSlot = findOrAllocJobSlot(jobId);
-
+
final ResourceGroup group = table.get(dsId, entityHashValue);
group.getLatch();
try {
@@ -165,8 +164,9 @@
} finally {
group.releaseLatch();
}
-
- if (CHECK_CONSISTENCY) assertLocksCanBefoundInJobQueue();
+
+ if (CHECK_CONSISTENCY)
+ assertLocksCanBefoundInJobQueue();
}
private void enqueueWaiter(final ResourceGroup group, final long reqSlot, final long resSlot, final long jobSlot,
@@ -189,20 +189,30 @@
interface DeadlockTracker {
void pushResource(long resSlot);
+
void pushRequest(long reqSlot);
+
void pushJob(long jobSlot);
+
void pop();
}
-
- static class NOPTracker implements DeadlockTracker {
+
+ static class NOPTracker implements DeadlockTracker {
static final DeadlockTracker INSTANCE = new NOPTracker();
- public void pushResource(long resSlot) {}
- public void pushRequest(long reqSlot) {}
- public void pushJob(long jobSlot) {}
- public void pop() {}
+ public void pushResource(long resSlot) {
+ }
+
+ public void pushRequest(long reqSlot) {
+ }
+
+ public void pushJob(long jobSlot) {
+ }
+
+ public void pop() {
+ }
}
-
+
static class CollectingTracker implements DeadlockTracker {
ArrayList<Long> slots = new ArrayList<Long>();
ArrayList<String> types = new ArrayList<String>();
@@ -232,9 +242,9 @@
public void pop() {
System.err.println("pop " + types.get(types.size() - 1) + " " + slots.get(slots.size() - 1));
types.remove(types.size() - 1);
- slots.remove(slots.size() - 1);
+ slots.remove(slots.size() - 1);
}
-
+
@Override
public String toString() {
StringBuilder sb = new StringBuilder();
@@ -244,7 +254,7 @@
return sb.toString();
}
}
-
+
/**
* determine if adding a job to the waiters of a resource will introduce a
* cycle in the wait-graph where the job waits on itself
@@ -255,8 +265,7 @@
* the slot that contains the information about the job
* @return true if a cycle would be introduced, false otherwise
*/
- private boolean introducesDeadlock(final long resSlot, final long jobSlot,
- final DeadlockTracker tracker) {
+ private boolean introducesDeadlock(final long resSlot, final long jobSlot, final DeadlockTracker tracker) {
synchronized (jobArenaMgr) {
tracker.pushResource(resSlot);
long reqSlot = resArenaMgr.getLastHolder(resSlot);
@@ -288,14 +297,14 @@
return false;
}
}
-
+
@Override
public void instantLock(DatasetId datasetId, int entityHashValue, byte lockMode, ITransactionContext txnContext)
throws ACIDException {
log("instantLock", datasetId.getId(), entityHashValue, lockMode, txnContext);
stats.instantLock();
-
- final int dsId = datasetId.getId();
+
+ final int dsId = datasetId.getId();
final int jobId = txnContext.getJobId().getId();
if (entityHashValue != -1) {
@@ -348,7 +357,8 @@
} finally {
if (reqSlot != -1) {
// deallocate request, if we allocated one earlier
- if (DEBUG_MODE) LOGGER.finer("del req slot " + TypeUtil.Global.toString(reqSlot));
+ if (DEBUG_MODE)
+ LOGGER.finer("del req slot " + TypeUtil.Global.toString(reqSlot));
reqArenaMgr.deallocate(reqSlot);
}
group.releaseLatch();
@@ -360,12 +370,12 @@
throws ACIDException {
log("tryLock", datasetId.getId(), entityHashValue, lockMode, txnContext);
stats.tryLock();
-
+
final int dsId = datasetId.getId();
final int jobId = txnContext.getJobId().getId();
if (entityHashValue != -1) {
- if (! tryLock(datasetId, -1, LockMode.intentionMode(lockMode), txnContext)) {
+ if (!tryLock(datasetId, -1, LockMode.intentionMode(lockMode), txnContext)) {
return false;
}
} else {
@@ -375,7 +385,7 @@
}
final long jobSlot = findOrAllocJobSlot(jobId);
-
+
final ResourceGroup group = table.get(dsId, entityHashValue);
group.getLatch();
@@ -415,12 +425,12 @@
ITransactionContext txnContext) throws ACIDException {
log("instantTryLock", datasetId.getId(), entityHashValue, lockMode, txnContext);
stats.instantTryLock();
-
+
final int dsId = datasetId.getId();
final int jobId = txnContext.getJobId().getId();
if (entityHashValue != -1) {
- if (! tryLock(datasetId, -1, LockMode.intentionMode(lockMode), txnContext)) {
+ if (!tryLock(datasetId, -1, LockMode.intentionMode(lockMode), txnContext)) {
return false;
}
} else {
@@ -486,13 +496,15 @@
if (resource < 0) {
throw new IllegalStateException("resource (" + dsId + ", " + entityHashValue + ") not found");
}
-
- if (CHECK_CONSISTENCY) assertLocksCanBefoundInJobQueue();
-
+
+ if (CHECK_CONSISTENCY)
+ assertLocksCanBefoundInJobQueue();
+
long holder = removeLastHolder(resource, jobSlot, lockMode);
// deallocate request
- if (DEBUG_MODE) LOGGER.finer("del req slot " + TypeUtil.Global.toString(holder));
+ if (DEBUG_MODE)
+ LOGGER.finer("del req slot " + TypeUtil.Global.toString(holder));
reqArenaMgr.deallocate(holder);
// deallocate resource or fix max lock mode
if (resourceNotUsed(resource)) {
@@ -505,7 +517,8 @@
}
resArenaMgr.setNext(prev, resArenaMgr.getNext(resource));
}
- if (DEBUG_MODE) LOGGER.finer("del res slot " + TypeUtil.Global.toString(resource));
+ if (DEBUG_MODE)
+ LOGGER.finer("del res slot " + TypeUtil.Global.toString(resource));
resArenaMgr.deallocate(resource);
} else {
final int oldMaxMode = resArenaMgr.getMaxMode(resource);
@@ -554,7 +567,8 @@
holder = jobArenaMgr.getLastHolder(jobSlot);
}
}
- if (DEBUG_MODE) LOGGER.finer("del job slot " + TypeUtil.Global.toString(jobSlot));
+ if (DEBUG_MODE)
+ LOGGER.finer("del job slot " + TypeUtil.Global.toString(jobSlot));
jobArenaMgr.deallocate(jobSlot);
jobIdSlotMap.remove(jobId);
stats.logCounters(LOGGER, Level.INFO, true);
@@ -564,14 +578,16 @@
Long jobSlot = jobIdSlotMap.get(jobId);
if (jobSlot == null) {
jobSlot = new Long(jobArenaMgr.allocate());
- if (DEBUG_MODE) LOGGER.finer("new job slot " + TypeUtil.Global.toString(jobSlot) + " (" + jobId + ")");
+ if (DEBUG_MODE)
+ LOGGER.finer("new job slot " + TypeUtil.Global.toString(jobSlot) + " (" + jobId + ")");
jobArenaMgr.setJobId(jobSlot, jobId);
Long oldSlot = jobIdSlotMap.putIfAbsent(jobId, jobSlot);
if (oldSlot != null) {
// if another thread allocated a slot for this jobId between
// get(..) and putIfAbsent(..), we'll use that slot and
// deallocate the one we allocated
- if (DEBUG_MODE) LOGGER.finer("del job slot " + TypeUtil.Global.toString(jobSlot) + " due to conflict");
+ if (DEBUG_MODE)
+ LOGGER.finer("del job slot " + TypeUtil.Global.toString(jobSlot) + " due to conflict");
jobArenaMgr.deallocate(jobSlot);
jobSlot = oldSlot;
}
@@ -590,9 +606,13 @@
resArenaMgr.setPkHashVal(resSlot, entityHashValue);
resArenaMgr.setNext(resSlot, group.firstResourceIndex.get());
group.firstResourceIndex.set(resSlot);
- if (DEBUG_MODE) LOGGER.finer("new res slot " + TypeUtil.Global.toString(resSlot) + " (" + dsId + ", " + entityHashValue + ")");
+ if (DEBUG_MODE)
+ LOGGER.finer("new res slot " + TypeUtil.Global.toString(resSlot) + " (" + dsId + ", " + entityHashValue
+ + ")");
} else {
- if (DEBUG_MODE) LOGGER.finer("fnd res slot " + TypeUtil.Global.toString(resSlot) + " (" + dsId + ", " + entityHashValue + ")");
+ if (DEBUG_MODE)
+ LOGGER.finer("fnd res slot " + TypeUtil.Global.toString(resSlot) + " (" + dsId + ", " + entityHashValue
+ + ")");
}
return resSlot;
}
@@ -603,10 +623,8 @@
reqArenaMgr.setLockMode(reqSlot, lockMode); // lock mode is a byte!!
reqArenaMgr.setJobSlot(reqSlot, jobSlot);
if (DEBUG_MODE) {
- LOGGER.finer("new req slot " + TypeUtil.Global.toString(reqSlot)
- + " (" + TypeUtil.Global.toString(resSlot)
- + ", " + TypeUtil.Global.toString(jobSlot)
- + ", " + LockMode.toString(lockMode) + ")");
+ LOGGER.finer("new req slot " + TypeUtil.Global.toString(reqSlot) + " (" + TypeUtil.Global.toString(resSlot)
+ + ", " + TypeUtil.Global.toString(jobSlot) + ", " + LockMode.toString(lockMode) + ")");
}
return reqSlot;
}
@@ -904,9 +922,9 @@
/*
* Debugging support
*/
-
+
private void log(String string, int id, int entityHashValue, byte lockMode, ITransactionContext txnContext) {
- if (! LOGGER.isLoggable(LVL)) {
+ if (!LOGGER.isLoggable(LVL)) {
return;
}
StringBuilder sb = new StringBuilder();
@@ -955,23 +973,29 @@
}
}
}
-
+
private void assertLockCanBeFoundInJobQueue(int dsId, int entityHashValue, byte lockMode, int jobId) {
if (findLockInJobQueue(dsId, entityHashValue, jobId, lockMode) == -1) {
String msg = "request for " + LockMode.toString(lockMode) + " lock on dataset " + dsId + " entity "
- + entityHashValue + " not found for job " + jobId + " in thread " + Thread.currentThread().getName();
- LOGGER.severe(msg);
+ + entityHashValue + " not found for job " + jobId + " in thread "
+ + Thread.currentThread().getName();
+ LOGGER.severe(msg);
throw new IllegalStateException(msg);
}
}
/**
* tries to find a lock request searching though the job queue
- * @param dsId dataset id
- * @param entityHashValue primary key hash value
- * @param jobId job id
- * @param lockMode lock mode
- * @return the slot of the request, if the lock request is found, -1 otherwise
+ *
+ * @param dsId
+ * dataset id
+ * @param entityHashValue
+ * primary key hash value
+ * @param jobId
+ * job id
+ * @param lockMode
+ * lock mode
+ * @return the slot of the request, if the lock request is found, -1 otherwise
*/
private long findLockInJobQueue(final int dsId, final int entityHashValue, final int jobId, byte lockMode) {
Long jobSlot = jobIdSlotMap.get(jobId);
@@ -985,11 +1009,9 @@
}
while (holder != -1) {
long resource = reqArenaMgr.getResourceId(holder);
- if (dsId == resArenaMgr.getDatasetId(resource)
- && entityHashValue == resArenaMgr.getPkHashVal(resource)
+ if (dsId == resArenaMgr.getDatasetId(resource) && entityHashValue == resArenaMgr.getPkHashVal(resource)
&& jobSlot == reqArenaMgr.getJobSlot(holder)
- && (lockMode == reqArenaMgr.getLockMode(holder)
- || lockMode == LockMode.ANY)) {
+ && (lockMode == reqArenaMgr.getLockMode(holder) || lockMode == LockMode.ANY)) {
return holder;
}
synchronized (jobArenaMgr) {
@@ -1002,14 +1024,14 @@
private String resQueueToString(long resSlot) {
return appendResQueue(new StringBuilder(), resSlot).toString();
}
-
+
private StringBuilder appendResQueue(StringBuilder sb, long resSlot) {
resArenaMgr.appendRecord(sb, resSlot);
sb.append("\n");
appendReqQueue(sb, resArenaMgr.getLastHolder(resSlot));
return sb;
}
-
+
private StringBuilder appendReqQueue(StringBuilder sb, long head) {
while (head != -1) {
reqArenaMgr.appendRecord(sb, head);
@@ -1018,7 +1040,7 @@
}
return sb;
}
-
+
public StringBuilder append(StringBuilder sb) {
table.getAllLatches();
try {
@@ -1133,10 +1155,11 @@
ResourceGroup get(int dId, int entityHashValue) {
// TODO ensure good properties of hash function
int h = Math.abs(dId ^ entityHashValue);
- if (h < 0) h = 0;
+ if (h < 0)
+ h = 0;
return table[h % TABLE_SIZE];
}
-
+
ResourceGroup get(int i) {
return table[i];
}
@@ -1186,7 +1209,7 @@
log("latch");
latch.writeLock().lock();
}
-
+
boolean tryLatch(long timeout, TimeUnit unit) throws ACIDException {
log("tryLatch");
try {
@@ -1224,12 +1247,12 @@
void log(String s) {
if (LOGGER.isLoggable(LVL)) {
LOGGER.log(LVL, s + " " + toString());
- }
+ }
}
public String toString() {
- return "{ id : " + hashCode() + ", first : " + TypeUtil.Global.toString(firstResourceIndex.get()) + ", waiters : "
- + (hasWaiters() ? "true" : "false") + " }";
+ return "{ id : " + hashCode() + ", first : " + TypeUtil.Global.toString(firstResourceIndex.get())
+ + ", waiters : " + (hasWaiters() ? "true" : "false") + " }";
}
}
}
diff --git a/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogBuffer.java b/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogBuffer.java
index 4d50294..f4100ee 100644
--- a/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogBuffer.java
+++ b/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogBuffer.java
@@ -27,12 +27,14 @@
import org.apache.asterix.common.context.PrimaryIndexOperationTracker;
import org.apache.asterix.common.exceptions.ACIDException;
+import org.apache.asterix.common.replication.IReplicationThread;
import org.apache.asterix.common.transactions.DatasetId;
import org.apache.asterix.common.transactions.ILogBuffer;
import org.apache.asterix.common.transactions.ILogRecord;
import org.apache.asterix.common.transactions.ITransactionContext;
import org.apache.asterix.common.transactions.JobId;
import org.apache.asterix.common.transactions.LogRecord;
+import org.apache.asterix.common.transactions.LogSource;
import org.apache.asterix.common.transactions.LogType;
import org.apache.asterix.common.transactions.MutableLong;
import org.apache.asterix.transaction.management.service.transaction.TransactionManagementConstants.LockManagerConstants.LockMode;
@@ -56,7 +58,7 @@
private boolean isLastPage;
private final LinkedBlockingQueue<ILogRecord> syncCommitQ;
private final LinkedBlockingQueue<ILogRecord> flushQ;
-
+ private final LinkedBlockingQueue<ILogRecord> remoteJobsQ;
private FileChannel fileChannel;
private boolean stop;
private final DatasetId reusableDsId;
@@ -76,7 +78,7 @@
isLastPage = false;
syncCommitQ = new LinkedBlockingQueue<ILogRecord>(logPageSize / ILogRecord.JOB_TERMINATE_LOG_SIZE);
flushQ = new LinkedBlockingQueue<ILogRecord>();
-
+ remoteJobsQ = new LinkedBlockingQueue<ILogRecord>();
reusableDsId = new DatasetId(-1);
reusableJobId = new JobId(-1);
}
@@ -88,7 +90,6 @@
@Override
public void append(ILogRecord logRecord, long appendLSN) {
logRecord.writeLogRecord(appendBuffer);
- // mhubail Update impacted resource with the flushed lsn
if (logRecord.getLogType() != LogType.FLUSH) {
logRecord.getTxnCtx().setLastLSN(appendLSN);
}
@@ -109,6 +110,39 @@
}
}
+ @Override
+ public void appendWithReplication(ILogRecord logRecord, long appendLSN) {
+ logRecord.writeLogRecord(appendBuffer, appendLSN);
+
+ if (logRecord.getLogSource() == LogSource.LOCAL && logRecord.getLogType() != LogType.FLUSH) {
+ if (logRecord.getLogType() != LogType.FLUSH) {
+ logRecord.getTxnCtx().setLastLSN(appendLSN);
+ }
+ }
+ synchronized (this) {
+ appendOffset += logRecord.getLogSize();
+ if (IS_DEBUG_MODE) {
+ LOGGER.info("append()| appendOffset: " + appendOffset);
+ }
+ if (logRecord.getLogSource() == LogSource.LOCAL) {
+ if (logRecord.getLogType() == LogType.JOB_COMMIT || logRecord.getLogType() == LogType.ABORT) {
+ logRecord.isFlushed(false);
+ syncCommitQ.offer(logRecord);
+ }
+ if (logRecord.getLogType() == LogType.FLUSH) {
+ logRecord.isFlushed(false);
+ flushQ.offer(logRecord);
+ }
+ } else if (logRecord.getLogSource() == LogSource.REMOTE) {
+
+ if (logRecord.getLogType() == LogType.JOB_COMMIT || logRecord.getLogType() == LogType.FLUSH) {
+ remoteJobsQ.offer(logRecord);
+ }
+ }
+ this.notify();
+ }
+ }
+
public void setFileChannel(FileChannel fileChannel) {
this.fileChannel = fileChannel;
}
@@ -218,20 +252,26 @@
LogRecord logRecord = logBufferTailReader.next();
while (logRecord != null) {
- if (logRecord.getLogType() == LogType.ENTITY_COMMIT) {
- reusableJobId.setId(logRecord.getJobId());
- txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(reusableJobId, false);
- reusableDsId.setId(logRecord.getDatasetId());
- txnSubsystem.getLockManager()
- .unlock(reusableDsId, logRecord.getPKHashValue(), LockMode.ANY, txnCtx);
- txnCtx.notifyOptracker(false);
- } else if (logRecord.getLogType() == LogType.JOB_COMMIT || logRecord.getLogType() == LogType.ABORT) {
- reusableJobId.setId(logRecord.getJobId());
- txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(reusableJobId, false);
- txnCtx.notifyOptracker(true);
- notifyJobTerminator();
- } else if (logRecord.getLogType() == LogType.FLUSH) {
- notifyFlushTerminator();
+ if (logRecord.getLogSource() == LogSource.LOCAL) {
+ if (logRecord.getLogType() == LogType.ENTITY_COMMIT) {
+ reusableJobId.setId(logRecord.getJobId());
+ txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(reusableJobId, false);
+ reusableDsId.setId(logRecord.getDatasetId());
+ txnSubsystem.getLockManager().unlock(reusableDsId, logRecord.getPKHashValue(), LockMode.ANY,
+ txnCtx);
+ txnCtx.notifyOptracker(false);
+ } else if (logRecord.getLogType() == LogType.JOB_COMMIT || logRecord.getLogType() == LogType.ABORT) {
+ reusableJobId.setId(logRecord.getJobId());
+ txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(reusableJobId, false);
+ txnCtx.notifyOptracker(true);
+ notifyJobTerminator();
+ } else if (logRecord.getLogType() == LogType.FLUSH) {
+ notifyFlushTerminator();
+ }
+ } else if (logRecord.getLogSource() == LogSource.REMOTE) {
+ if (logRecord.getLogType() == LogType.JOB_COMMIT || logRecord.getLogType() == LogType.FLUSH) {
+ notifyReplicationTerminator();
+ }
}
logRecord = logBufferTailReader.next();
@@ -275,6 +315,25 @@
}
}
+ public void notifyReplicationTerminator() {
+ LogRecord logRecord = null;
+ try {
+ logRecord = (LogRecord) remoteJobsQ.take();
+ } catch (InterruptedException e) {
+ //ignore
+ }
+ logRecord.isFlushed(true);
+ IReplicationThread replicationThread = logRecord.getReplicationThread();
+
+ if (replicationThread != null) {
+ try {
+ replicationThread.notifyLogReplicationRequester(logRecord);
+ } catch (Exception e) {
+ e.printStackTrace();
+ }
+ }
+ }
+
public boolean isStop() {
return stop;
}
diff --git a/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogManager.java b/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogManager.java
index 87d0ad8..691a4d8 100644
--- a/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogManager.java
+++ b/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogManager.java
@@ -38,6 +38,7 @@
import java.util.logging.Logger;
import org.apache.asterix.common.exceptions.ACIDException;
+import org.apache.asterix.common.replication.IReplicationManager;
import org.apache.asterix.common.transactions.ILogManager;
import org.apache.asterix.common.transactions.ILogReader;
import org.apache.asterix.common.transactions.ILogRecord;
@@ -56,21 +57,22 @@
private final TransactionSubsystem txnSubsystem;
private final LogManagerProperties logManagerProperties;
- private final long logFileSize;
- private final int logPageSize;
+ protected final long logFileSize;
+ protected final int logPageSize;
private final int numLogPages;
private final String logDir;
private final String logFilePrefix;
private final MutableLong flushLSN;
private LinkedBlockingQueue<LogBuffer> emptyQ;
private LinkedBlockingQueue<LogBuffer> flushQ;
- private final AtomicLong appendLSN;
+ protected final AtomicLong appendLSN;
private FileChannel appendChannel;
- private LogBuffer appendPage;
+ protected LogBuffer appendPage;
private LogFlusher logFlusher;
private Future<Object> futureLogFlusher;
private static final long SMALLEST_LOG_FILE_ID = 0;
- private LinkedBlockingQueue<ILogRecord> flushLogsQ;
+ private final String nodeId;
+ protected LinkedBlockingQueue<ILogRecord> flushLogsQ;
private final FlushLogsLogger flushLogsLogger;
public LogManager(TransactionSubsystem txnSubsystem) {
@@ -84,6 +86,7 @@
logFilePrefix = logManagerProperties.getLogFilePrefix();
flushLSN = new MutableLong();
appendLSN = new AtomicLong();
+ nodeId = txnSubsystem.getId();
flushLogsQ = new LinkedBlockingQueue<>();
flushLogsLogger = new FlushLogsLogger();
initializeLogManager(SMALLEST_LOG_FILE_ID);
@@ -122,7 +125,7 @@
appendToLogTail(logRecord);
}
- private void appendToLogTail(ILogRecord logRecord) throws ACIDException {
+ protected void appendToLogTail(ILogRecord logRecord) throws ACIDException {
syncAppendToLogTail(logRecord);
if ((logRecord.getLogType() == LogType.JOB_COMMIT || logRecord.getLogType() == LogType.ABORT)
@@ -139,7 +142,7 @@
}
}
- private synchronized void syncAppendToLogTail(ILogRecord logRecord) throws ACIDException {
+ protected synchronized void syncAppendToLogTail(ILogRecord logRecord) throws ACIDException {
ITransactionContext txnCtx = null;
if (logRecord.getLogType() != LogType.FLUSH) {
@@ -168,7 +171,7 @@
appendLSN.addAndGet(logRecord.getLogSize());
}
- private void getAndInitNewPage() {
+ protected void getAndInitNewPage() {
appendPage = null;
while (appendPage == null) {
try {
@@ -182,7 +185,7 @@
flushQ.offer(appendPage);
}
- private void prepareNextLogFile() {
+ protected void prepareNextLogFile() {
appendLSN.addAndGet(logFileSize - getLogFileOffset(appendLSN.get()));
appendChannel = getFileChannel(appendLSN.get(), true);
appendPage.isLastPage(true);
@@ -454,6 +457,34 @@
}
}
+ @Override
+ public String getNodeId() {
+ return nodeId;
+ }
+
+ @Override
+ public int getLogPageSize() {
+ return logPageSize;
+ }
+
+ @Override
+ public void renewLogFilesAndStartFromLSN(long LSNtoStartFrom) throws IOException {
+ terminateLogFlusher();
+ deleteAllLogFiles();
+ long newLogFile = getLogFileId(LSNtoStartFrom);
+ initializeLogManager(newLogFile + 1);
+ }
+
+ @Override
+ public void setReplicationManager(IReplicationManager replicationManager) {
+ throw new IllegalStateException("This log manager does not support replication");
+ }
+
+ @Override
+ public int getNumLogPages() {
+ return numLogPages;
+ }
+
/**
* This class is used to log FLUSH logs.
* FLUSH logs are flushed on a different thread to avoid a possible deadlock in LogBuffer batchUnlock which calls PrimaryIndexOpeartionTracker.completeOperation
diff --git a/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogManagerWithReplication.java b/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogManagerWithReplication.java
new file mode 100644
index 0000000..2a40052
--- /dev/null
+++ b/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogManagerWithReplication.java
@@ -0,0 +1,131 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.transaction.management.service.logging;
+
+import org.apache.asterix.common.exceptions.ACIDException;
+import org.apache.asterix.common.replication.IReplicationManager;
+import org.apache.asterix.common.transactions.ILogRecord;
+import org.apache.asterix.common.transactions.ITransactionContext;
+import org.apache.asterix.common.transactions.ITransactionManager;
+import org.apache.asterix.common.transactions.LogSource;
+import org.apache.asterix.common.transactions.LogType;
+import org.apache.asterix.transaction.management.service.transaction.TransactionSubsystem;
+
+public class LogManagerWithReplication extends LogManager {
+
+ private IReplicationManager replicationManager;
+
+ public LogManagerWithReplication(TransactionSubsystem txnSubsystem) throws ACIDException {
+ super(txnSubsystem);
+ }
+
+ @Override
+ public void log(ILogRecord logRecord) throws ACIDException {
+ if (logRecord.getLogSize() > logPageSize) {
+ throw new IllegalStateException();
+ }
+
+ if (logRecord.getLogType() == LogType.FLUSH) {
+ flushLogsQ.offer(logRecord);
+ return;
+ }
+
+ appendToLogTail(logRecord);
+ }
+
+ @Override
+ protected void appendToLogTail(ILogRecord logRecord) throws ACIDException {
+ syncAppendToLogTail(logRecord);
+
+ if (logRecord.getLogSource() == LogSource.LOCAL) {
+ replicationManager.replicateLog(logRecord);
+ }
+
+ if (logRecord.getLogSource() == LogSource.LOCAL) {
+ if ((logRecord.getLogType() == LogType.JOB_COMMIT || logRecord.getLogType() == LogType.ABORT)
+ && !logRecord.isFlushed()) {
+ synchronized (logRecord) {
+ while (!logRecord.isFlushed()) {
+ try {
+ logRecord.wait();
+ } catch (InterruptedException e) {
+ //ignore
+ }
+ }
+ }
+ }
+
+ //wait for job ACK from replicas
+ //TODO should JOB_ABORT be added as well?
+ if ((logRecord.getLogType() == LogType.JOB_COMMIT) && !replicationManager.hasBeenReplicated(logRecord)) {
+ synchronized (logRecord) {
+ while (!replicationManager.hasBeenReplicated(logRecord)) {
+ try {
+ logRecord.wait();
+ } catch (InterruptedException e) {
+ //ignore
+ }
+ }
+ }
+ }
+ }
+ }
+
+ @Override
+ protected synchronized void syncAppendToLogTail(ILogRecord logRecord) throws ACIDException {
+ ITransactionContext txnCtx = null;
+
+ if (logRecord.getLogSource() == LogSource.LOCAL) {
+ if (logRecord.getLogType() != LogType.FLUSH) {
+ txnCtx = logRecord.getTxnCtx();
+ if (txnCtx.getTxnState() == ITransactionManager.ABORTED && logRecord.getLogType() != LogType.ABORT) {
+ throw new ACIDException(
+ "Aborted job(" + txnCtx.getJobId() + ") tried to write non-abort type log record.");
+ }
+ }
+ }
+
+ if (getLogFileOffset(appendLSN.get()) + logRecord.getLogSize() > logFileSize) {
+ prepareNextLogFile();
+ appendPage.isFull(true);
+ getAndInitNewPage();
+ } else if (!appendPage.hasSpace(logRecord.getLogSize())) {
+ appendPage.isFull(true);
+ getAndInitNewPage();
+ }
+ if (logRecord.getLogSource() == LogSource.LOCAL) {
+ if (logRecord.getLogType() == LogType.UPDATE) {
+ logRecord.setPrevLSN(txnCtx.getLastLSN());
+ }
+ }
+ appendPage.appendWithReplication(logRecord, appendLSN.get());
+
+ if (logRecord.getLogType() == LogType.FLUSH) {
+ logRecord.setLSN(appendLSN.get());
+ }
+
+ appendLSN.addAndGet(logRecord.getLogSize());
+ }
+
+ @Override
+ public void setReplicationManager(IReplicationManager replicationManager) {
+ this.replicationManager = replicationManager;
+ }
+
+}
\ No newline at end of file
diff --git a/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/recovery/CheckpointObject.java b/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/recovery/CheckpointObject.java
index 2e66a09..82f4609 100644
--- a/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/recovery/CheckpointObject.java
+++ b/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/recovery/CheckpointObject.java
@@ -28,12 +28,14 @@
private final long minMCTFirstLsn;
private final int maxJobId;
private final long timeStamp;
-
- public CheckpointObject(long checkpointLsn, long minMCTFirstLsn, int maxJobId, long timeStamp) {
+ private final boolean sharp;
+
+ public CheckpointObject(long checkpointLsn, long minMCTFirstLsn, int maxJobId, long timeStamp, boolean sharp) {
this.checkpointLsn = checkpointLsn;
this.minMCTFirstLsn = minMCTFirstLsn;
this.maxJobId = maxJobId;
this.timeStamp = timeStamp;
+ this.sharp = sharp;
}
public long getCheckpointLsn() {
@@ -52,6 +54,10 @@
return timeStamp;
}
+ public boolean isSharp() {
+ return sharp;
+ }
+
@Override
public int compareTo(CheckpointObject checkpointObject) {
long compareTimeStamp = checkpointObject.getTimeStamp();
diff --git a/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/recovery/RecoveryManager.java b/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/recovery/RecoveryManager.java
index 9bc4c53..f602156 100644
--- a/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/recovery/RecoveryManager.java
+++ b/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/recovery/RecoveryManager.java
@@ -49,8 +49,11 @@
import org.apache.asterix.common.api.IDatasetLifecycleManager;
import org.apache.asterix.common.api.ILocalResourceMetadata;
+import org.apache.asterix.common.config.IAsterixPropertiesProvider;
import org.apache.asterix.common.exceptions.ACIDException;
import org.apache.asterix.common.ioopcallbacks.AbstractLSMIOOperationCallback;
+import org.apache.asterix.common.replication.IReplicaResourcesManager;
+import org.apache.asterix.common.replication.IReplicationManager;
import org.apache.asterix.common.transactions.DatasetId;
import org.apache.asterix.common.transactions.IAsterixAppRuntimeContextProvider;
import org.apache.asterix.common.transactions.ILogReader;
@@ -92,6 +95,8 @@
private final LogManager logMgr;
private final int checkpointHistory;
private final long SHARP_CHECKPOINT_LSN = -1;
+ private final boolean replicationEnabled;
+ public static final long NON_SHARP_CHECKPOINT_TARGET_LSN = -1;
private final static String RECOVERY_FILES_DIR_NAME = "recovery_temp";
private static final long MEGABYTE = 1024L * 1024L;
private Map<Integer, JobEntityCommits> jobId2WinnerEntitiesMap = null;
@@ -108,6 +113,9 @@
this.txnSubsystem = txnSubsystem;
this.logMgr = (LogManager) txnSubsystem.getLogManager();
this.checkpointHistory = this.txnSubsystem.getTransactionProperties().getCheckpointHistory();
+ IAsterixPropertiesProvider propertiesProvider = (IAsterixPropertiesProvider) txnSubsystem
+ .getAsterixAppRuntimeContextProvider().getAppContext();
+ this.replicationEnabled = propertiesProvider.getReplicationProperties().isReplicationEnabled();
}
/**
@@ -133,28 +141,45 @@
return state;
}
- long readableSmallestLSN = logMgr.getReadableSmallestLSN();
- if (logMgr.getAppendLSN() == readableSmallestLSN) {
- if (checkpointObject.getMinMCTFirstLsn() != SHARP_CHECKPOINT_LSN) {
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("[Warning] ---------------------------------------------------");
- LOGGER.info("[Warning] Some(or all) of transaction log files are lost.");
- LOGGER.info("[Warning] ---------------------------------------------------");
- //No choice but continuing when the log files are lost.
- }
+ if (replicationEnabled) {
+ if (checkpointObject.getMinMCTFirstLsn() == SHARP_CHECKPOINT_LSN) {
+ //no logs exist
+ state = SystemState.HEALTHY;
+ return state;
+ } else if (checkpointObject.getCheckpointLsn() == logMgr.getAppendLSN() && checkpointObject.isSharp()) {
+ //only remote logs exist
+ state = SystemState.HEALTHY;
+ return state;
+ } else {
+ //need to perform remote recovery
+ state = SystemState.CORRUPTED;
+ return state;
}
- state = SystemState.HEALTHY;
- return state;
- } else if (checkpointObject.getCheckpointLsn() == logMgr.getAppendLSN()
- && checkpointObject.getMinMCTFirstLsn() == SHARP_CHECKPOINT_LSN) {
- state = SystemState.HEALTHY;
- return state;
} else {
- state = SystemState.CORRUPTED;
- return state;
+ long readableSmallestLSN = logMgr.getReadableSmallestLSN();
+ if (logMgr.getAppendLSN() == readableSmallestLSN) {
+ if (checkpointObject.getMinMCTFirstLsn() != SHARP_CHECKPOINT_LSN) {
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("[Warning] ---------------------------------------------------");
+ LOGGER.info("[Warning] Some(or all) of transaction log files are lost.");
+ LOGGER.info("[Warning] ---------------------------------------------------");
+ //No choice but continuing when the log files are lost.
+ }
+ }
+ state = SystemState.HEALTHY;
+ return state;
+ } else if (checkpointObject.getCheckpointLsn() == logMgr.getAppendLSN()
+ && checkpointObject.getMinMCTFirstLsn() == SHARP_CHECKPOINT_LSN) {
+ state = SystemState.HEALTHY;
+ return state;
+ } else {
+ state = SystemState.CORRUPTED;
+ return state;
+ }
}
}
+ //This method is used only when replication is disabled. Therefore, there is no need to check logs node ids
public void startRecovery(boolean synchronous) throws IOException, ACIDException {
//delete any recovery files from previous failed recovery attempts
deleteRecoveryTemporaryFiles();
@@ -383,6 +408,205 @@
}
@Override
+ public void replayRemoteLogs(ArrayList<ILogRecord> remoteLogs) throws HyracksDataException, ACIDException {
+ int updateLogCount = 0;
+ int entityCommitLogCount = 0;
+ int jobCommitLogCount = 0;
+ int redoCount = 0;
+ int abortLogCount = 0;
+ int jobId = -1;
+
+ state = SystemState.RECOVERING;
+
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("[RecoveryMgr] starting recovery ...");
+ }
+
+ Set<Integer> winnerJobSet = new HashSet<Integer>();
+ Map<Integer, Set<TxnId>> jobId2WinnerEntitiesMap = new HashMap<Integer, Set<TxnId>>();
+ //winnerEntity is used to add pairs, <committed TxnId, the most recent commit Lsn of the TxnId>
+ Set<TxnId> winnerEntitySet = null;
+ TxnId tempKeyTxnId = new TxnId(-1, -1, -1, null, -1, false);
+ TxnId winnerEntity = null;
+
+ //-------------------------------------------------------------------------
+ // [ analysis phase ]
+ // - collect all committed Lsn
+ //-------------------------------------------------------------------------
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("[RecoveryMgr] in analysis phase");
+ }
+
+ String nodeId = logMgr.getNodeId();
+ ILogRecord logRecord;
+ for (int i = 0; i < remoteLogs.size(); i++) {
+ logRecord = remoteLogs.get(i);
+ if (IS_DEBUG_MODE) {
+ LOGGER.info(logRecord.getLogRecordForDisplay());
+ }
+
+ if (logRecord.getNodeId().equals(nodeId)) {
+ //update max jobId
+ switch (logRecord.getLogType()) {
+ case LogType.UPDATE:
+ updateLogCount++;
+ break;
+ case LogType.JOB_COMMIT:
+ winnerJobSet.add(Integer.valueOf(logRecord.getJobId()));
+ jobId2WinnerEntitiesMap.remove(Integer.valueOf(logRecord.getJobId()));
+ jobCommitLogCount++;
+ break;
+ case LogType.ENTITY_COMMIT:
+ jobId = logRecord.getJobId();
+ winnerEntity = new TxnId(jobId, logRecord.getDatasetId(), logRecord.getPKHashValue(),
+ logRecord.getPKValue(), logRecord.getPKValueSize(), true);
+ if (!jobId2WinnerEntitiesMap.containsKey(Integer.valueOf(jobId))) {
+ winnerEntitySet = new HashSet<TxnId>();
+ jobId2WinnerEntitiesMap.put(Integer.valueOf(jobId), winnerEntitySet);
+ } else {
+ winnerEntitySet = jobId2WinnerEntitiesMap.get(Integer.valueOf(jobId));
+ }
+ winnerEntitySet.add(winnerEntity);
+ entityCommitLogCount++;
+ break;
+ case LogType.ABORT:
+ abortLogCount++;
+ break;
+ case LogType.FLUSH:
+ break;
+ default:
+ throw new ACIDException("Unsupported LogType: " + logRecord.getLogType());
+ }
+ }
+ }
+
+ //-------------------------------------------------------------------------
+ // [ redo phase ]
+ // - redo if
+ // 1) The TxnId is committed && --> guarantee durability
+ // 2) lsn > maxDiskLastLsn of the index --> guarantee idempotence
+ //-------------------------------------------------------------------------
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("[RecoveryMgr] in redo phase");
+ }
+
+ long resourceId;
+ long maxDiskLastLsn;
+ long LSN = -1;
+ ILSMIndex index = null;
+ LocalResource localResource = null;
+ ILocalResourceMetadata localResourceMetadata = null;
+ Map<Long, Long> resourceId2MaxLSNMap = new HashMap<Long, Long>();
+ boolean foundWinner = false;
+
+ //#. get indexLifeCycleManager
+ IAsterixAppRuntimeContextProvider appRuntimeContext = txnSubsystem.getAsterixAppRuntimeContextProvider();
+ IDatasetLifecycleManager datasetLifecycleManager = appRuntimeContext.getDatasetLifecycleManager();
+ ILocalResourceRepository localResourceRepository = appRuntimeContext.getLocalResourceRepository();
+ Map<Long, LocalResource> resourcesMap = ((PersistentLocalResourceRepository) localResourceRepository)
+ .loadAndGetAllResources();
+ //#. set log reader to the lowWaterMarkLsn again.
+ for (int i = 0; i < remoteLogs.size(); i++) {
+ logRecord = remoteLogs.get(i);
+ if (IS_DEBUG_MODE) {
+ LOGGER.info(logRecord.getLogRecordForDisplay());
+ }
+ if (logRecord.getNodeId().equals(nodeId)) {
+ LSN = logRecord.getLSN();
+ jobId = logRecord.getJobId();
+ foundWinner = false;
+ switch (logRecord.getLogType()) {
+ case LogType.UPDATE:
+ if (winnerJobSet.contains(Integer.valueOf(jobId))) {
+ foundWinner = true;
+ } else if (jobId2WinnerEntitiesMap.containsKey(Integer.valueOf(jobId))) {
+ winnerEntitySet = jobId2WinnerEntitiesMap.get(Integer.valueOf(jobId));
+ tempKeyTxnId.setTxnId(jobId, logRecord.getDatasetId(), logRecord.getPKHashValue(),
+ logRecord.getPKValue(), logRecord.getPKValueSize());
+ if (winnerEntitySet.contains(tempKeyTxnId)) {
+ foundWinner = true;
+ }
+ }
+ if (foundWinner) {
+ resourceId = logRecord.getResourceId();
+ localResource = resourcesMap.get(resourceId);
+
+ /*******************************************************************
+ * [Notice]
+ * -> Issue
+ * Delete index may cause a problem during redo.
+ * The index operation to be redone couldn't be redone because the corresponding index
+ * may not exist in NC due to the possible index drop DDL operation.
+ * -> Approach
+ * Avoid the problem during redo.
+ * More specifically, the problem will be detected when the localResource of
+ * the corresponding index is retrieved, which will end up with 'null'.
+ * If null is returned, then just go and process the next
+ * log record.
+ *******************************************************************/
+ if (localResource == null) {
+ continue;
+ }
+ /*******************************************************************/
+
+ //get index instance from IndexLifeCycleManager
+ //if index is not registered into IndexLifeCycleManager,
+ //create the index using LocalMetadata stored in LocalResourceRepository
+ index = (ILSMIndex) datasetLifecycleManager.getIndex(localResource.getResourceName());
+ if (index == null) {
+ //#. create index instance and register to indexLifeCycleManager
+ localResourceMetadata = (ILocalResourceMetadata) localResource.getResourceObject();
+ index = localResourceMetadata.createIndexInstance(appRuntimeContext,
+ localResource.getResourceName(), localResource.getPartition());
+ datasetLifecycleManager.register(localResource.getResourceName(), index);
+ datasetLifecycleManager.open(localResource.getResourceName());
+
+ //#. get maxDiskLastLSN
+ ILSMIndex lsmIndex = index;
+ maxDiskLastLsn = ((AbstractLSMIOOperationCallback) lsmIndex.getIOOperationCallback())
+ .getComponentLSN(lsmIndex.getImmutableComponents());
+
+ //#. set resourceId and maxDiskLastLSN to the map
+ resourceId2MaxLSNMap.put(Long.valueOf(resourceId), Long.valueOf(maxDiskLastLsn));
+ } else {
+ maxDiskLastLsn = resourceId2MaxLSNMap.get(Long.valueOf(resourceId));
+ }
+
+ if (LSN > maxDiskLastLsn) {
+ redo(logRecord, datasetLifecycleManager);
+ redoCount++;
+ }
+ }
+ break;
+ case LogType.JOB_COMMIT:
+ case LogType.ENTITY_COMMIT:
+ case LogType.ABORT:
+ case LogType.FLUSH:
+
+ //do nothing
+ break;
+
+ default:
+ throw new ACIDException("Unsupported LogType: " + logRecord.getLogType());
+ }
+ }
+ }
+
+ //close all indexes
+ Set<Long> resourceIdList = resourceId2MaxLSNMap.keySet();
+ for (long r : resourceIdList) {
+ datasetLifecycleManager.close(resourcesMap.get(r).getResourceName());
+ }
+
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("[RecoveryMgr] remote recovery is completed.");
+ LOGGER.info("[RecoveryMgr's recovery log count] update/entityCommit/jobCommit/abort/redo = "
+ + updateLogCount + "/" + entityCommitLogCount + "/" + jobCommitLogCount + "/" + abortLogCount + "/"
+ + redoCount);
+ }
+ }
+
+ @Override
public synchronized long checkpoint(boolean isSharpCheckpoint, long nonSharpCheckpointTargetLSN)
throws ACIDException, HyracksDataException {
long minMCTFirstLSN;
@@ -404,7 +628,26 @@
//flush all in-memory components if it is the sharp checkpoint
if (isSharpCheckpoint) {
datasetLifecycleManager.flushAllDatasets();
- minMCTFirstLSN = SHARP_CHECKPOINT_LSN;
+ if (!replicationEnabled) {
+ minMCTFirstLSN = SHARP_CHECKPOINT_LSN;
+ } else {
+ //if is shutting down, need to check if we need to keep any remote logs for dead replicas
+ if (txnSubsystem.getAsterixAppRuntimeContextProvider().getAppContext().isShuttingdown()) {
+ Set<String> deadReplicaIds = txnSubsystem.getAsterixAppRuntimeContextProvider().getAppContext()
+ .getReplicationManager().getDeadReplicasIds();
+ if (deadReplicaIds.isEmpty()) {
+ minMCTFirstLSN = SHARP_CHECKPOINT_LSN;
+ } else {
+ //get min LSN of dead replicas remote resources
+ IReplicaResourcesManager remoteResourcesManager = txnSubsystem
+ .getAsterixAppRuntimeContextProvider().getAppContext().getReplicaResourcesManager();
+ minMCTFirstLSN = remoteResourcesManager.getMinRemoteLSN(deadReplicaIds);
+ }
+ } else {
+ //start up complete checkpoint. Avoid deleting remote recovery logs.
+ minMCTFirstLSN = getMinFirstLSN();
+ }
+ }
} else {
minMCTFirstLSN = getMinFirstLSN();
if (minMCTFirstLSN >= nonSharpCheckpointTargetLSN) {
@@ -412,11 +655,21 @@
} else {
//flush datasets with indexes behind target checkpoint LSN
datasetLifecycleManager.scheduleAsyncFlushForLaggingDatasets(nonSharpCheckpointTargetLSN);
+ if (replicationEnabled) {
+ //request remote replicas to flush lagging indexes
+ IReplicationManager replicationManager = txnSubsystem.getAsterixAppRuntimeContextProvider()
+ .getAppContext().getReplicationManager();
+ try {
+ replicationManager.requestFlushLaggingReplicaIndexes(nonSharpCheckpointTargetLSN);
+ } catch (IOException e) {
+ e.printStackTrace();
+ }
+ }
}
}
CheckpointObject checkpointObject = new CheckpointObject(logMgr.getAppendLSN(), minMCTFirstLSN,
- txnMgr.getMaxJobId(), System.currentTimeMillis());
+ txnMgr.getMaxJobId(), System.currentTimeMillis(), isSharpCheckpoint);
FileOutputStream fos = null;
ObjectOutputStream oosToFos = null;
@@ -456,7 +709,11 @@
if (isSharpCheckpoint) {
try {
- logMgr.renewLogFiles();
+ if (minMCTFirstLSN == SHARP_CHECKPOINT_LSN) {
+ logMgr.renewLogFiles();
+ } else {
+ logMgr.deleteOldLogFiles(minMCTFirstLSN);
+ }
} catch (IOException e) {
throw new HyracksDataException(e);
}
@@ -475,6 +732,18 @@
}
public long getMinFirstLSN() throws HyracksDataException {
+ long minFirstLSN = getLocalMinFirstLSN();
+
+ //if replication is enabled, consider replica resources min LSN
+ if (replicationEnabled) {
+ long remoteMinFirstLSN = getRemoteMinFirstLSN();
+ minFirstLSN = Math.min(minFirstLSN, remoteMinFirstLSN);
+ }
+
+ return minFirstLSN;
+ }
+
+ public long getLocalMinFirstLSN() throws HyracksDataException {
IDatasetLifecycleManager datasetLifecycleManager = txnSubsystem.getAsterixAppRuntimeContextProvider()
.getDatasetLifecycleManager();
List<IIndex> openIndexList = datasetLifecycleManager.getOpenIndexes();
@@ -485,6 +754,7 @@
for (IIndex index : openIndexList) {
AbstractLSMIOOperationCallback ioCallback = (AbstractLSMIOOperationCallback) ((ILSMIndex) index)
.getIOOperationCallback();
+
if (!((AbstractLSMIndex) index).isCurrentMutableComponentEmpty() || ioCallback.hasPendingFlush()) {
firstLSN = ioCallback.getFirstLSN();
minFirstLSN = Math.min(minFirstLSN, firstLSN);
@@ -494,6 +764,18 @@
return minFirstLSN;
}
+ private long getRemoteMinFirstLSN() {
+ IAsterixPropertiesProvider propertiesProvider = (IAsterixPropertiesProvider) txnSubsystem
+ .getAsterixAppRuntimeContextProvider().getAppContext();
+
+ Set<String> replicaIds = propertiesProvider.getReplicationProperties()
+ .getRemoteReplicasIds(txnSubsystem.getId());
+ IReplicaResourcesManager remoteResourcesManager = txnSubsystem.getAsterixAppRuntimeContextProvider()
+ .getAppContext().getReplicaResourcesManager();
+
+ return remoteResourcesManager.getMinRemoteLSN(replicaIds);
+ }
+
private CheckpointObject readCheckpoint() throws ACIDException, FileNotFoundException {
CheckpointObject checkpointObject = null;
@@ -641,7 +923,7 @@
long currentLSN = -1;
TxnId loserEntity = null;
List<Long> undoLSNSet = null;
-
+ String nodeId = logMgr.getNodeId();
ILogReader logReader = logMgr.getLogReader(false);
try {
logReader.initializeScan(firstLSN);
@@ -652,48 +934,51 @@
break;
} else {
currentLSN = logRecord.getLSN();
+
if (IS_DEBUG_MODE) {
LOGGER.info(logRecord.getLogRecordForDisplay());
}
}
- logJobId = logRecord.getJobId();
- if (logJobId != abortedJobId) {
- continue;
- }
- tempKeyTxnId.setTxnId(logJobId, logRecord.getDatasetId(), logRecord.getPKHashValue(),
- logRecord.getPKValue(), logRecord.getPKValueSize());
- switch (logRecord.getLogType()) {
- case LogType.UPDATE:
- undoLSNSet = jobLoserEntity2LSNsMap.get(tempKeyTxnId);
- if (undoLSNSet == null) {
- loserEntity = new TxnId(logJobId, logRecord.getDatasetId(), logRecord.getPKHashValue(),
- logRecord.getPKValue(), logRecord.getPKValueSize(), true);
- undoLSNSet = new LinkedList<Long>();
- jobLoserEntity2LSNsMap.put(loserEntity, undoLSNSet);
- }
- undoLSNSet.add(currentLSN);
- updateLogCount++;
- if (IS_DEBUG_MODE) {
- LOGGER.info(Thread.currentThread().getId() + "======> update[" + currentLSN + "]:"
- + tempKeyTxnId);
- }
- break;
- case LogType.ENTITY_COMMIT:
- jobLoserEntity2LSNsMap.remove(tempKeyTxnId);
- entityCommitLogCount++;
- if (IS_DEBUG_MODE) {
- LOGGER.info(Thread.currentThread().getId() + "======> entity_commit[" + currentLSN + "]"
- + tempKeyTxnId);
- }
- break;
- case LogType.JOB_COMMIT:
- throw new ACIDException("Unexpected LogType(" + logRecord.getLogType() + ") during abort.");
- case LogType.ABORT:
- case LogType.FLUSH:
- //ignore
- break;
- default:
- throw new ACIDException("Unsupported LogType: " + logRecord.getLogType());
+ if (logRecord.getNodeId().equals(nodeId)) {
+ logJobId = logRecord.getJobId();
+ if (logJobId != abortedJobId) {
+ continue;
+ }
+ tempKeyTxnId.setTxnId(logJobId, logRecord.getDatasetId(), logRecord.getPKHashValue(),
+ logRecord.getPKValue(), logRecord.getPKValueSize());
+ switch (logRecord.getLogType()) {
+ case LogType.UPDATE:
+ undoLSNSet = jobLoserEntity2LSNsMap.get(tempKeyTxnId);
+ if (undoLSNSet == null) {
+ loserEntity = new TxnId(logJobId, logRecord.getDatasetId(), logRecord.getPKHashValue(),
+ logRecord.getPKValue(), logRecord.getPKValueSize(), true);
+ undoLSNSet = new LinkedList<Long>();
+ jobLoserEntity2LSNsMap.put(loserEntity, undoLSNSet);
+ }
+ undoLSNSet.add(currentLSN);
+ updateLogCount++;
+ if (IS_DEBUG_MODE) {
+ LOGGER.info(Thread.currentThread().getId() + "======> update[" + currentLSN + "]:"
+ + tempKeyTxnId);
+ }
+ break;
+ case LogType.ENTITY_COMMIT:
+ jobLoserEntity2LSNsMap.remove(tempKeyTxnId);
+ entityCommitLogCount++;
+ if (IS_DEBUG_MODE) {
+ LOGGER.info(Thread.currentThread().getId() + "======> entity_commit[" + currentLSN + "]"
+ + tempKeyTxnId);
+ }
+ break;
+ case LogType.JOB_COMMIT:
+ throw new ACIDException("Unexpected LogType(" + logRecord.getLogType() + ") during abort.");
+ case LogType.ABORT:
+ case LogType.FLUSH:
+ //ignore
+ break;
+ default:
+ throw new ACIDException("Unsupported LogType: " + logRecord.getLogType());
+ }
}
}
if (currentLSN != lastLSN) {
@@ -743,8 +1028,13 @@
}
@Override
- public void stop(boolean dumpState, OutputStream os) {
- //no op
+ public void stop(boolean dumpState, OutputStream os) throws IOException {
+ try {
+ checkpoint(true, NON_SHARP_CHECKPOINT_TARGET_LSN);
+ } catch (HyracksDataException | ACIDException e) {
+ e.printStackTrace();
+ throw new IOException(e);
+ }
}
@Override
@@ -784,6 +1074,7 @@
throw new IllegalStateException("Unsupported OperationType: " + logRecord.getNewOp());
}
} catch (Exception e) {
+ e.printStackTrace();
throw new IllegalStateException("Failed to redo", e);
}
}
@@ -1105,4 +1396,4 @@
}
return size;
}
-}
+}
\ No newline at end of file
diff --git a/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/transaction/TransactionContext.java b/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/transaction/TransactionContext.java
index ecb1bc9..bc644841 100644
--- a/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/transaction/TransactionContext.java
+++ b/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/transaction/TransactionContext.java
@@ -110,6 +110,7 @@
primaryIndex = null;
tempResourceIdForRegister = new MutableLong();
logRecord = new LogRecord();
+ logRecord.setNodeId(transactionSubsystem.getId());
}
@Override
diff --git a/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/transaction/TransactionSubsystem.java b/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/transaction/TransactionSubsystem.java
index eeb65e2..d371e94 100644
--- a/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/transaction/TransactionSubsystem.java
+++ b/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/transaction/TransactionSubsystem.java
@@ -18,7 +18,9 @@
*/
package org.apache.asterix.transaction.management.service.transaction;
+import org.apache.asterix.common.config.AsterixReplicationProperties;
import org.apache.asterix.common.config.AsterixTransactionProperties;
+import org.apache.asterix.common.config.IAsterixPropertiesProvider;
import org.apache.asterix.common.exceptions.ACIDException;
import org.apache.asterix.common.transactions.IAsterixAppRuntimeContextProvider;
import org.apache.asterix.common.transactions.ILockManager;
@@ -28,6 +30,7 @@
import org.apache.asterix.common.transactions.ITransactionSubsystem;
import org.apache.asterix.transaction.management.service.locking.ConcurrentLockManager;
import org.apache.asterix.transaction.management.service.logging.LogManager;
+import org.apache.asterix.transaction.management.service.logging.LogManagerWithReplication;
import org.apache.asterix.transaction.management.service.recovery.CheckpointThread;
import org.apache.asterix.transaction.management.service.recovery.RecoveryManager;
@@ -52,8 +55,21 @@
this.txnProperties = txnProperties;
this.transactionManager = new TransactionManager(this);
this.lockManager = new ConcurrentLockManager(this);
- this.logManager = new LogManager(this);
+
+ AsterixReplicationProperties asterixReplicationProperties = null;
+ if (asterixAppRuntimeContextProvider != null) {
+ asterixReplicationProperties = ((IAsterixPropertiesProvider) asterixAppRuntimeContextProvider
+ .getAppContext()).getReplicationProperties();
+ }
+
+ if (asterixReplicationProperties != null && asterixReplicationProperties.isReplicationEnabled()) {
+ this.logManager = new LogManagerWithReplication(this);
+ } else {
+ this.logManager = new LogManager(this);
+ }
+
this.recoveryManager = new RecoveryManager(this);
+
if (asterixAppRuntimeContextProvider != null) {
this.checkpointThread = new CheckpointThread(recoveryManager,
asterixAppRuntimeContextProvider.getDatasetLifecycleManager(),logManager,
diff --git a/pom.xml b/pom.xml
index a1a36f6..d8c7fc7 100644
--- a/pom.xml
+++ b/pom.xml
@@ -388,6 +388,7 @@
<module>asterix-doc</module>
<module>asterix-fuzzyjoin</module>
<module>asterix-yarn</module>
+ <module>asterix-replication</module>
</modules>
<repositories>