Asterix NCs Failback Support
- Allow Failed NCs to failback and takeover their partitions.
- New cluster API servlet for cluster state description.
- Remove nodeId from txn logs except remote FLUSH_LOG.
- Add partition id in UPDATE and Entity_COMMIT logs.
- Adapt remote recovery to new logs format.
- Refactor RecoveryManager and split Analysis and Redo phase.
- Spill remote recover logs to temporary file.
- Replicate files to remote replicas for partitions of interest only.
- Introduce NC active/inactive partitions concept.
- Test case for failback.
Change-Id: Id17819542d6b9c4e32647e64737c4a467b630f24
Reviewed-on: https://asterix-gerrit.ics.uci.edu/613
Tested-by: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
Reviewed-by: abdullah alamoudi <bamousaa@gmail.com>
diff --git a/.gitignore b/.gitignore
index 90efb11..d6ab1f1 100644
--- a/.gitignore
+++ b/.gitignore
@@ -16,6 +16,7 @@
asterix-app/opt_parserts/
asterix-app/runtime_parserts/
asterix-installer/ittest/
+asterix-installer/repliationtest/
build
asterix_logs
bin/
diff --git a/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/CommitPOperator.java b/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/CommitPOperator.java
index 5112fcf..2d1cf1e 100644
--- a/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/CommitPOperator.java
+++ b/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/CommitPOperator.java
@@ -37,20 +37,25 @@
import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext;
import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenHelper;
import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.dataflow.std.file.FileSplit;
public class CommitPOperator extends AbstractPhysicalOperator {
private final List<LogicalVariable> primaryKeyLogicalVars;
private final JobId jobId;
private final int datasetId;
+ private final String dataverse;
+ private final String dataset;
private final LogicalVariable upsertVar;
- public CommitPOperator(JobId jobId, int datasetId, List<LogicalVariable> primaryKeyLogicalVars,
- LogicalVariable upsertVar) {
+ public CommitPOperator(JobId jobId, String dataverse, String dataset, int datasetId,
+ List<LogicalVariable> primaryKeyLogicalVars, LogicalVariable upsertVar) {
this.jobId = jobId;
this.datasetId = datasetId;
this.primaryKeyLogicalVars = primaryKeyLogicalVars;
this.upsertVar = upsertVar;
+ this.dataverse = dataverse;
+ this.dataset = dataset;
}
@Override
@@ -84,13 +89,23 @@
RecordDescriptor recDesc = JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op), propagatedSchema,
context);
int[] primaryKeyFields = JobGenHelper.variablesToFieldIndexes(primaryKeyLogicalVars, inputSchemas[0]);
+
+ //get dataset splits
+ FileSplit[] splitsForDataset = metadataProvider.splitsForDataset(metadataProvider.getMetadataTxnContext(),
+ dataverse, dataset, dataset, metadataProvider.isTemporaryDatasetWriteJob());
+ int[] datasetPartitions = new int[splitsForDataset.length];
+ for (int i = 0; i < splitsForDataset.length; i++) {
+ datasetPartitions[i] = splitsForDataset[i].getPartition();
+ }
+
int upsertVarIdx = -1;
CommitRuntimeFactory runtime = null;
if (upsertVar != null) {
upsertVarIdx = inputSchemas[0].findVariable(upsertVar);
}
runtime = new CommitRuntimeFactory(jobId, datasetId, primaryKeyFields,
- metadataProvider.isTemporaryDatasetWriteJob(), metadataProvider.isWriteTransaction(), upsertVarIdx);
+ metadataProvider.isTemporaryDatasetWriteJob(), metadataProvider.isWriteTransaction(), upsertVarIdx,
+ datasetPartitions);
builder.contributeMicroOperator(op, runtime, recDesc);
ILogicalOperator src = op.getInputs().get(0).getValue();
builder.contributeGraphEdge(src, 0, op, 0);
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 b72018a..94519cf 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
@@ -28,8 +28,8 @@
import org.apache.asterix.common.transactions.ITransactionManager;
import org.apache.asterix.common.transactions.JobId;
import org.apache.asterix.common.transactions.LogRecord;
+import org.apache.asterix.common.transactions.LogType;
import org.apache.asterix.common.utils.TransactionUtil;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.runtime.base.IPushRuntime;
import org.apache.hyracks.api.comm.IFrameWriter;
import org.apache.hyracks.api.context.IHyracksTaskContext;
@@ -58,9 +58,10 @@
protected ITransactionContext transactionContext;
protected FrameTupleAccessor frameTupleAccessor;
+ protected final int resourcePartition;
public CommitRuntime(IHyracksTaskContext ctx, JobId jobId, int datasetId, int[] primaryKeyFields,
- boolean isTemporaryDatasetWriteJob, boolean isWriteTransaction) {
+ boolean isTemporaryDatasetWriteJob, boolean isWriteTransaction, int resourcePartition) {
IAsterixAppRuntimeContext runtimeCtx = (IAsterixAppRuntimeContext) ctx.getJobletContext()
.getApplicationContext().getApplicationObject();
this.ctx = ctx;
@@ -72,9 +73,9 @@
this.frameTupleReference = new FrameTupleReference();
this.isTemporaryDatasetWriteJob = isTemporaryDatasetWriteJob;
this.isWriteTransaction = isWriteTransaction;
- this.longHashes = new long[2];
- this.logRecord = new LogRecord();
- logRecord.setNodeId(logMgr.getNodeId());
+ this.resourcePartition = resourcePartition;
+ longHashes = new long[2];
+ logRecord = new LogRecord();
}
@Override
@@ -109,17 +110,17 @@
try {
formLogRecord(buffer, t);
logMgr.log(logRecord);
- } catch (ACIDException | AlgebricksException e) {
+ } catch (ACIDException e) {
throw new HyracksDataException(e);
}
}
}
}
- protected void formLogRecord(ByteBuffer buffer, int t) throws AlgebricksException {
+ protected void formLogRecord(ByteBuffer buffer, int t) {
int pkHash = computePrimaryKeyHashValue(frameTupleReference, primaryKeyFields);
TransactionUtil.formEntityCommitLogRecord(logRecord, transactionContext, datasetId, pkHash, frameTupleReference,
- primaryKeyFields);
+ primaryKeyFields, resourcePartition, LogType.ENTITY_COMMIT);
}
protected int computePrimaryKeyHashValue(ITupleReference tuple, int[] primaryKeyFields) {
diff --git a/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/CommitRuntimeFactory.java b/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/CommitRuntimeFactory.java
index 7d03796..4f28b9d 100644
--- a/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/CommitRuntimeFactory.java
+++ b/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/CommitRuntimeFactory.java
@@ -35,15 +35,17 @@
private final boolean isTemporaryDatasetWriteJob;
private final boolean isWriteTransaction;
private final int upsertVarIdx;
+ private int[] datasetPartitions;
public CommitRuntimeFactory(JobId jobId, int datasetId, int[] primaryKeyFields, boolean isTemporaryDatasetWriteJob,
- boolean isWriteTransaction, int upsertVarIdx) {
+ boolean isWriteTransaction, int upsertVarIdx, int[] datasetPartitions) {
this.jobId = jobId;
this.datasetId = datasetId;
this.primaryKeyFields = primaryKeyFields;
this.isTemporaryDatasetWriteJob = isTemporaryDatasetWriteJob;
this.isWriteTransaction = isWriteTransaction;
this.upsertVarIdx = upsertVarIdx;
+ this.datasetPartitions = datasetPartitions;
}
@Override
@@ -55,10 +57,11 @@
public IPushRuntime createPushRuntime(IHyracksTaskContext ctx) throws AlgebricksException {
if (upsertVarIdx >= 0) {
return new UpsertCommitRuntime(ctx, jobId, datasetId, primaryKeyFields, isTemporaryDatasetWriteJob,
- isWriteTransaction, upsertVarIdx);
+ isWriteTransaction, datasetPartitions[ctx.getTaskAttemptId().getTaskId().getPartition()],
+ upsertVarIdx);
} else {
return new CommitRuntime(ctx, jobId, datasetId, primaryKeyFields, isTemporaryDatasetWriteJob,
- isWriteTransaction);
+ isWriteTransaction, datasetPartitions[ctx.getTaskAttemptId().getTaskId().getPartition()]);
}
}
}
diff --git a/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/UpsertCommitRuntime.java b/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/UpsertCommitRuntime.java
index 81996d1..7358700 100644
--- a/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/UpsertCommitRuntime.java
+++ b/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/UpsertCommitRuntime.java
@@ -21,22 +21,23 @@
import java.nio.ByteBuffer;
import org.apache.asterix.common.transactions.JobId;
+import org.apache.asterix.common.transactions.LogType;
import org.apache.asterix.common.utils.TransactionUtil;
import org.apache.asterix.dataflow.data.nontagged.serde.ABooleanSerializerDeserializer;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.api.context.IHyracksTaskContext;
public class UpsertCommitRuntime extends CommitRuntime {
private final int upsertIdx;
public UpsertCommitRuntime(IHyracksTaskContext ctx, JobId jobId, int datasetId, int[] primaryKeyFields,
- boolean isTemporaryDatasetWriteJob, boolean isWriteTransaction, int upsertIdx) throws AlgebricksException {
- super(ctx, jobId, datasetId, primaryKeyFields, isTemporaryDatasetWriteJob, isWriteTransaction);
+ boolean isTemporaryDatasetWriteJob, boolean isWriteTransaction, int resourcePartition, int upsertIdx) {
+ super(ctx, jobId, datasetId, primaryKeyFields, isTemporaryDatasetWriteJob, isWriteTransaction,
+ resourcePartition);
this.upsertIdx = upsertIdx;
}
@Override
- protected void formLogRecord(ByteBuffer buffer, int t) throws AlgebricksException {
+ protected void formLogRecord(ByteBuffer buffer, int t) {
boolean isNull = ABooleanSerializerDeserializer.getBoolean(buffer.array(),
frameTupleAccessor.getFieldSlotsLength() + frameTupleAccessor.getTupleStartOffset(t)
+ frameTupleAccessor.getFieldStartOffset(t, upsertIdx) + 1);
@@ -46,8 +47,8 @@
} else {
// Previous record found (delete + insert)
int pkHash = computePrimaryKeyHashValue(frameTupleReference, primaryKeyFields);
- TransactionUtil.formEntityUpsertCommitLogRecord(logRecord, transactionContext, datasetId, pkHash,
- frameTupleReference, primaryKeyFields);
+ TransactionUtil.formEntityCommitLogRecord(logRecord, transactionContext, datasetId, pkHash,
+ frameTupleReference, primaryKeyFields, resourcePartition, LogType.UPSERT_ENTITY_COMMIT);
}
}
}
diff --git a/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ReplaceSinkOpWithCommitOpRule.java b/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ReplaceSinkOpWithCommitOpRule.java
index f8df183..ef8b4a3 100644
--- a/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ReplaceSinkOpWithCommitOpRule.java
+++ b/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ReplaceSinkOpWithCommitOpRule.java
@@ -68,6 +68,8 @@
List<Mutable<ILogicalExpression>> primaryKeyExprs = null;
int datasetId = 0;
+ String dataverse = null;
+ String datasetName = null;
AbstractLogicalOperator descendantOp = (AbstractLogicalOperator) sinkOperator.getInputs().get(0).getValue();
LogicalVariable upsertVar = null;
AssignOperator upsertFlagAssign = null;
@@ -79,6 +81,10 @@
primaryKeyExprs = indexInsertDeleteUpsertOperator.getPrimaryKeyExpressions();
datasetId = ((DatasetDataSource) indexInsertDeleteUpsertOperator.getDataSourceIndex()
.getDataSource()).getDataset().getDatasetId();
+ dataverse = ((DatasetDataSource) indexInsertDeleteUpsertOperator.getDataSourceIndex()
+ .getDataSource()).getDataset().getDataverseName();
+ datasetName = ((DatasetDataSource) indexInsertDeleteUpsertOperator.getDataSourceIndex()
+ .getDataSource()).getDataset().getDatasetName();
break;
}
} else if (descendantOp.getOperatorTag() == LogicalOperatorTag.INSERT_DELETE_UPSERT) {
@@ -87,6 +93,10 @@
primaryKeyExprs = insertDeleteUpsertOperator.getPrimaryKeyExpressions();
datasetId = ((DatasetDataSource) insertDeleteUpsertOperator.getDataSource()).getDataset()
.getDatasetId();
+ dataverse = ((DatasetDataSource) insertDeleteUpsertOperator.getDataSource()).getDataset()
+ .getDataverseName();
+ datasetName = ((DatasetDataSource) insertDeleteUpsertOperator.getDataSource()).getDataset()
+ .getDatasetName();
if (insertDeleteUpsertOperator.getOperation() == Kind.UPSERT) {
//we need to add a function that checks if previous record was found
upsertVar = context.newVar();
@@ -132,7 +142,8 @@
//create the logical and physical operator
CommitOperator commitOperator = new CommitOperator(primaryKeyLogicalVars, upsertVar);
- CommitPOperator commitPOperator = new CommitPOperator(jobId, datasetId, primaryKeyLogicalVars, upsertVar);
+ CommitPOperator commitPOperator = new CommitPOperator(jobId, dataverse, datasetName, datasetId,
+ primaryKeyLogicalVars, upsertVar);
commitOperator.setPhysicalOperator(commitPOperator);
//create ExtensionOperator and put the commitOperator in it.
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 8a40876..5532e79 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
@@ -22,12 +22,14 @@
import java.rmi.RemoteException;
import java.rmi.server.UnicastRemoteObject;
import java.util.List;
+import java.util.Set;
import java.util.logging.Level;
import java.util.logging.Logger;
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.cluster.ClusterPartition;
import org.apache.asterix.common.config.AsterixBuildProperties;
import org.apache.asterix.common.config.AsterixCompilerProperties;
import org.apache.asterix.common.config.AsterixExternalProperties;
@@ -105,8 +107,6 @@
}
}
- private static final int METADATA_IO_DEVICE_ID = 0;
-
private ILSMMergePolicyFactory metadataMergePolicyFactory;
private final INCApplicationContext ncApplicationContext;
@@ -126,7 +126,7 @@
private ITransactionSubsystem txnSubsystem;
private ILSMIOOperationScheduler lsmIOScheduler;
- private ILocalResourceRepository localResourceRepository;
+ private PersistentLocalResourceRepository localResourceRepository;
private IResourceIdFactory resourceIdFactory;
private IIOManager ioManager;
private boolean isShuttingdown;
@@ -153,6 +153,7 @@
this.metadataRmiPort = metadataRmiPort;
}
+ @Override
public void initialize(boolean initialRun) throws IOException, ACIDException, AsterixException {
Logger.getLogger("org.apache").setLevel(externalProperties.getLogLevel());
@@ -172,7 +173,7 @@
ILocalResourceRepositoryFactory persistentLocalResourceRepositoryFactory = new PersistentLocalResourceRepositoryFactory(
ioManager, ncApplicationContext.getNodeId(), metadataProperties);
- localResourceRepository = persistentLocalResourceRepositoryFactory.createRepository();
+ localResourceRepository = (PersistentLocalResourceRepository) persistentLocalResourceRepositoryFactory.createRepository();
IAsterixAppRuntimeContextProvider asterixAppRuntimeContextProvider = new AsterixAppRuntimeContextProdiverForRecovery(
this);
@@ -183,7 +184,7 @@
SystemState systemState = recoveryMgr.getSystemState();
if (initialRun || systemState == SystemState.NEW_UNIVERSE) {
//delete any storage data before the resource factory is initialized
- ((PersistentLocalResourceRepository) localResourceRepository).deleteStorageData(true);
+ localResourceRepository.deleteStorageData(true);
}
initializeResourceIdFactory();
@@ -208,7 +209,22 @@
txnSubsystem.getLogManager().setReplicationManager(replicationManager);
//PersistentLocalResourceRepository to replicate metadata files and delete backups on drop index
- ((PersistentLocalResourceRepository) localResourceRepository).setReplicationManager(replicationManager);
+ localResourceRepository.setReplicationManager(replicationManager);
+
+ /**
+ * add the partitions that will be replicated in this node as inactive partitions
+ */
+ //get nodes which replicate to this node
+ Set<String> replicationClients = replicationProperties.getNodeReplicationClients(nodeId);
+ //remove the node itself
+ replicationClients.remove(nodeId);
+ for (String clientId : replicationClients) {
+ //get the partitions of each client
+ ClusterPartition[] clientPartitions = metadataProperties.getNodePartitions().get(clientId);
+ for (ClusterPartition partition : clientPartitions) {
+ localResourceRepository.addInactivePartition(partition.getPartitionId());
+ }
+ }
//initialize replication channel
replicationChannel = new ReplicationChannel(nodeId, replicationProperties, txnSubsystem.getLogManager(),
@@ -220,7 +236,6 @@
bufferCache = new BufferCache(ioManager, prs, pcp, fileMapManager,
storageProperties.getBufferCacheMaxOpenFiles(), ncApplicationContext.getThreadFactory(),
replicationManager);
-
} else {
bufferCache = new BufferCache(ioManager, prs, pcp, fileMapManager,
storageProperties.getBufferCacheMaxOpenFiles(), ncApplicationContext.getThreadFactory());
@@ -251,57 +266,65 @@
lccm.register((ILifeCycleComponent) txnSubsystem.getLockManager());
}
+ @Override
public boolean isShuttingdown() {
return isShuttingdown;
}
+ @Override
public void setShuttingdown(boolean isShuttingdown) {
this.isShuttingdown = isShuttingdown;
}
+ @Override
public void deinitialize() throws HyracksDataException {
}
+ @Override
public IBufferCache getBufferCache() {
return bufferCache;
}
+ @Override
public IFileMapProvider getFileMapManager() {
return fileMapManager;
}
+ @Override
public ITransactionSubsystem getTransactionSubsystem() {
return txnSubsystem;
}
+ @Override
public IDatasetLifecycleManager getDatasetLifecycleManager() {
return datasetLifecycleManager;
}
+ @Override
public double getBloomFilterFalsePositiveRate() {
return storageProperties.getBloomFilterFalsePositiveRate();
}
+ @Override
public ILSMIOOperationScheduler getLSMIOScheduler() {
return lsmIOScheduler;
}
+ @Override
public ILocalResourceRepository getLocalResourceRepository() {
return localResourceRepository;
}
+ @Override
public IResourceIdFactory getResourceIdFactory() {
return resourceIdFactory;
}
+ @Override
public IIOManager getIOManager() {
return ioManager;
}
- public int getMetaDataIODeviceId() {
- return METADATA_IO_DEVICE_ID;
- }
-
@Override
public AsterixStorageProperties getStorageProperties() {
return storageProperties;
@@ -352,6 +375,7 @@
return threadExecutor;
}
+ @Override
public ILSMMergePolicyFactory getMetadataMergePolicyFactory() {
return metadataMergePolicyFactory;
}
@@ -421,4 +445,9 @@
IMetadataNode stub = (IMetadataNode) UnicastRemoteObject.exportObject(MetadataNode.INSTANCE, metadataRmiPort);
((IAsterixStateProxy) ncApplicationContext.getDistributedState()).setMetadataNode(stub);
}
+
+ @Override
+ public void unexportMetadataNodeStub() throws RemoteException {
+ UnicastRemoteObject.unexportObject(MetadataNode.INSTANCE, false);
+ }
}
diff --git a/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/ClusterAPIServlet.java b/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/ClusterAPIServlet.java
new file mode 100644
index 0000000..dce10ab
--- /dev/null
+++ b/asterix-app/src/main/java/org/apache/asterix/api/http/servlet/ClusterAPIServlet.java
@@ -0,0 +1,51 @@
+/*
+ * 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.api.http.servlet;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+
+import javax.servlet.http.HttpServlet;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+
+import org.apache.asterix.om.util.AsterixClusterProperties;
+import org.apache.asterix.result.ResultUtils;
+import org.json.JSONException;
+import org.json.JSONObject;
+
+public class ClusterAPIServlet extends HttpServlet {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public void doGet(HttpServletRequest request, HttpServletResponse response) throws IOException {
+ response.setContentType("application/json");
+ response.setCharacterEncoding("utf-8");
+ PrintWriter responseWriter = response.getWriter();
+ try {
+ JSONObject responseObject = AsterixClusterProperties.INSTANCE.getClusterStateDescription();
+ responseWriter.write(responseObject.toString());
+ response.setStatus(HttpServletResponse.SC_OK);
+ } catch (JSONException e) {
+ ResultUtils.apiErrorHandler(responseWriter, e);
+ response.setStatus(HttpServletResponse.SC_INTERNAL_SERVER_ERROR);
+ }
+ responseWriter.flush();
+ }
+}
\ 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 bee284d..adf0a4d 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
@@ -23,6 +23,7 @@
import org.apache.asterix.api.http.servlet.APIServlet;
import org.apache.asterix.api.http.servlet.AQLAPIServlet;
+import org.apache.asterix.api.http.servlet.ClusterAPIServlet;
import org.apache.asterix.api.http.servlet.ConnectorAPIServlet;
import org.apache.asterix.api.http.servlet.DDLAPIServlet;
import org.apache.asterix.api.http.servlet.FeedServlet;
@@ -37,7 +38,7 @@
import org.apache.asterix.common.api.IClusterManagementWork.ClusterState;
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.utils.ServletUtil.Servlets;
import org.apache.asterix.compiler.provider.AqlCompilationProvider;
import org.apache.asterix.compiler.provider.SqlppCompilationProvider;
import org.apache.asterix.event.service.ILookupService;
@@ -51,7 +52,6 @@
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;
@@ -88,11 +88,9 @@
}
appCtx.setThreadFactory(new AsterixThreadFactory(new LifeCycleComponentManager()));
- GlobalRecoveryManager.INSTANCE = new GlobalRecoveryManager(
- (HyracksConnection) getNewHyracksClientConnection());
+ GlobalRecoveryManager.INSTANCE = new GlobalRecoveryManager((HyracksConnection) getNewHyracksClientConnection());
- AsterixAppContextInfo.initialize(appCtx, getNewHyracksClientConnection(),
- GlobalRecoveryManager.INSTANCE);
+ AsterixAppContextInfo.initialize(appCtx, getNewHyracksClientConnection(), GlobalRecoveryManager.INSTANCE);
proxy = AsterixStateProxy.registerRemoteObject();
appCtx.setDistributedState(proxy);
@@ -118,13 +116,6 @@
ClusterManager.INSTANCE.registerSubscriber(GlobalRecoveryManager.INSTANCE);
- AsterixReplicationProperties asterixRepliactionProperties = AsterixAppContextInfo.getInstance()
- .getReplicationProperties();
- if (asterixRepliactionProperties.isReplicationEnabled()) {
- ReplicationLifecycleListener.INSTANCE = new ReplicationLifecycleListener(asterixRepliactionProperties);
- ClusterManager.INSTANCE.registerSubscriber(ReplicationLifecycleListener.INSTANCE);
- }
-
ccAppCtx.addClusterLifecycleListener(ClusterLifecycleListener.INSTANCE);
ccAppCtx.setMessageBroker(messageBroker);
}
@@ -178,25 +169,32 @@
jsonAPIServer.setHandler(context);
// AQL rest APIs.
- context.addServlet(new ServletHolder(new QueryAPIServlet(new AqlCompilationProvider())), "/query");
- context.addServlet(new ServletHolder(new UpdateAPIServlet(new AqlCompilationProvider())), "/update");
- context.addServlet(new ServletHolder(new DDLAPIServlet(new AqlCompilationProvider())), "/ddl");
- context.addServlet(new ServletHolder(new AQLAPIServlet(new AqlCompilationProvider())), "/aql");
+ context.addServlet(new ServletHolder(new QueryAPIServlet(new AqlCompilationProvider())),
+ Servlets.AQL_QUERY.getPath());
+ context.addServlet(new ServletHolder(new UpdateAPIServlet(new AqlCompilationProvider())),
+ Servlets.AQL_UPDATE.getPath());
+ context.addServlet(new ServletHolder(new DDLAPIServlet(new AqlCompilationProvider())),
+ Servlets.AQL_DDL.getPath());
+ context.addServlet(new ServletHolder(new AQLAPIServlet(new AqlCompilationProvider())), Servlets.AQL.getPath());
// SQL++ rest APIs.
- context.addServlet(new ServletHolder(new QueryAPIServlet(new SqlppCompilationProvider())), "/query/sqlpp");
- context.addServlet(new ServletHolder(new UpdateAPIServlet(new SqlppCompilationProvider())), "/update/sqlpp");
- context.addServlet(new ServletHolder(new DDLAPIServlet(new SqlppCompilationProvider())), "/ddl/sqlpp");
- context.addServlet(new ServletHolder(new AQLAPIServlet(new SqlppCompilationProvider())), "/sqlpp");
+ context.addServlet(new ServletHolder(new QueryAPIServlet(new SqlppCompilationProvider())),
+ Servlets.SQLPP_QUERY.getPath());
+ context.addServlet(new ServletHolder(new UpdateAPIServlet(new SqlppCompilationProvider())),
+ Servlets.SQLPP_UPDATE.getPath());
+ context.addServlet(new ServletHolder(new DDLAPIServlet(new SqlppCompilationProvider())),
+ Servlets.SQLPP_DDL.getPath());
+ context.addServlet(new ServletHolder(new AQLAPIServlet(new SqlppCompilationProvider())),
+ Servlets.SQLPP.getPath());
// Other APIs.
- context.addServlet(new ServletHolder(new QueryStatusAPIServlet()), "/query/status");
- context.addServlet(new ServletHolder(new QueryResultAPIServlet()), "/query/result");
- context.addServlet(new ServletHolder(new ConnectorAPIServlet()), "/connector");
- context.addServlet(new ServletHolder(new ShutdownAPIServlet()), "/admin/shutdown");
- context.addServlet(new ServletHolder(new VersionAPIServlet()), "/admin/version");
-
- context.addServlet(new ServletHolder(new QueryServiceServlet()), "/query/service");
+ context.addServlet(new ServletHolder(new QueryStatusAPIServlet()), Servlets.QUERY_STATUS.getPath());
+ context.addServlet(new ServletHolder(new QueryResultAPIServlet()), Servlets.QUERY_RESULT.getPath());
+ context.addServlet(new ServletHolder(new QueryServiceServlet()), Servlets.QUERY_SERVICE.getPath());
+ context.addServlet(new ServletHolder(new ConnectorAPIServlet()), Servlets.CONNECTOR.getPath());
+ context.addServlet(new ServletHolder(new ShutdownAPIServlet()), Servlets.SHUTDOWN.getPath());
+ context.addServlet(new ServletHolder(new VersionAPIServlet()), Servlets.VERSION.getPath());
+ context.addServlet(new ServletHolder(new ClusterAPIServlet()), Servlets.CLUSTER_STATE.getPath());
}
private void setupFeedServer(AsterixExternalProperties externalProperties) throws Exception {
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 00b7391..4514fee 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
@@ -62,17 +62,15 @@
t.start();
}
- public enum ClusterEventType {
- NODE_JOIN,
- NODE_FAILURE
- }
-
@Override
public void notifyNodeJoin(String nodeId, Map<String, String> ncConfiguration) {
if (LOGGER.isLoggable(Level.INFO)) {
LOGGER.info("NC: " + nodeId + " joined");
}
AsterixClusterProperties.INSTANCE.addNCConfiguration(nodeId, ncConfiguration);
+ //if metadata node rejoining, we need to rebind the proxy connection when it is active again.
+ MetadataManager.INSTANCE.rebindMetadataNode = !AsterixClusterProperties.INSTANCE.isMetadataNodeActive();
+
Set<String> nodeAddition = new HashSet<String>();
nodeAddition.add(nodeId);
updateProgress(ClusterEventType.NODE_JOIN, nodeAddition);
@@ -90,17 +88,16 @@
}
+ @Override
public void notifyNodeFailure(Set<String> deadNodeIds) {
for (String deadNode : deadNodeIds) {
if (LOGGER.isLoggable(Level.INFO)) {
LOGGER.info("NC: " + deadNode + " left");
}
- //if metadata node failed, we need to rebind the proxy connection when it joins again.
- String metadataNode = AsterixClusterProperties.INSTANCE.getCurrentMetadataNode();
- if (deadNode.equals(metadataNode)) {
- MetadataManager.INSTANCE.rebindMetadataNode = true;
- }
AsterixClusterProperties.INSTANCE.removeNCConfiguration(deadNode);
+
+ //if metadata node failed, we need to rebind the proxy connection when it is active again
+ MetadataManager.INSTANCE.rebindMetadataNode = !AsterixClusterProperties.INSTANCE.isMetadataNodeActive();
}
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 dac9af5..fcb196d 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,7 +19,6 @@
package org.apache.asterix.hyracks.bootstrap;
import java.io.File;
-import java.io.IOException;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
@@ -30,7 +29,6 @@
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.messaging.api.INCMessageBroker;
@@ -71,8 +69,7 @@
private boolean isMetadataNode = false;
private boolean stopInitiated = false;
private SystemState systemState = SystemState.NEW_UNIVERSE;
- private boolean performedRemoteRecovery = false;
- private boolean replicationEnabled = false;
+ private boolean pendingFailbackCompletion = false;
private IMessageBroker messageBroker;
@Override
@@ -90,8 +87,7 @@
ncAppCtx.setThreadFactory(new AsterixThreadFactory(ncAppCtx.getLifeCycleComponentManager()));
ncApplicationContext = ncAppCtx;
- messageBroker = new NCMessageBroker((NodeControllerService) ncAppCtx.getControllerService());
- ncApplicationContext.setMessageBroker(messageBroker);
+
nodeId = ncApplicationContext.getNodeId();
if (LOGGER.isLoggable(Level.INFO)) {
LOGGER.info("Starting Asterix node controller: " + nodeId);
@@ -108,17 +104,14 @@
}
runtimeContext.initialize(initialRun);
ncApplicationContext.setApplicationObject(runtimeContext);
+ messageBroker = new NCMessageBroker((NodeControllerService) ncAppCtx.getControllerService());
+ ncApplicationContext.setMessageBroker(messageBroker);
- //If replication is enabled, check if there is a replica for this node
- AsterixReplicationProperties asterixReplicationProperties = ((IAsterixPropertiesProvider) runtimeContext)
- .getReplicationProperties();
-
- replicationEnabled = asterixReplicationProperties.isReplicationEnabled();
-
+ boolean replicationEnabled = AsterixClusterProperties.INSTANCE.isReplicationEnabled();
+ boolean autoFailover = AsterixClusterProperties.INSTANCE.isAutoFailoverEnabled();
if (initialRun) {
LOGGER.info("System is being initialized. (first run)");
} else {
- //#. recover if the system is corrupted by checking system state.
IRecoveryManager recoveryMgr = runtimeContext.getTransactionSubsystem().getRecoveryManager();
systemState = recoveryMgr.getSystemState();
@@ -130,36 +123,41 @@
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 (autoFailover) {
+ remoteRecoveryMgr.startFailbackProcess();
+ systemState = SystemState.RECOVERING;
+ pendingFailbackCompletion = true;
+ } else {
+ remoteRecoveryMgr.performRemoteRecovery();
+ systemState = SystemState.HEALTHY;
+ }
}
- }
-
- if (systemState == SystemState.CORRUPTED) {
- recoveryMgr.startRecovery(true);
+ } else {
+ //recover if the system is corrupted by checking system state.
+ if (systemState == SystemState.CORRUPTED) {
+ recoveryMgr.startRecovery(true);
+ }
}
}
- if (replicationEnabled) {
+ /**
+ * if the node pending failback completion, the replication channel
+ * should not be opened to avoid other nodes connecting to it before
+ * the node completes its failback. CC will notify other replicas once
+ * this node is ready to receive replication requests.
+ */
+ if (replicationEnabled && !pendingFailbackCompletion) {
startReplicationService();
}
}
- private void startReplicationService() throws IOException {
+ private void startReplicationService() {
//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();
}
@@ -211,10 +209,10 @@
}
isMetadataNode = nodeId.equals(metadataProperties.getMetadataNodeName());
- if (isMetadataNode) {
+ if (isMetadataNode && !pendingFailbackCompletion) {
runtimeContext.initializeMetadata(systemState == SystemState.NEW_UNIVERSE);
}
- ExternalLibraryBootstrap.setUpExternaLibraries(isMetadataNode);
+ ExternalLibraryBootstrap.setUpExternaLibraries(isMetadataNode && !pendingFailbackCompletion);
if (LOGGER.isLoggable(Level.INFO)) {
LOGGER.info("Starting lifecycle components");
@@ -237,11 +235,13 @@
lccm.startAll();
- IRecoveryManager recoveryMgr = runtimeContext.getTransactionSubsystem().getRecoveryManager();
- recoveryMgr.checkpoint(true, RecoveryManager.NON_SHARP_CHECKPOINT_TARGET_LSN);
+ if (!pendingFailbackCompletion) {
+ IRecoveryManager recoveryMgr = runtimeContext.getTransactionSubsystem().getRecoveryManager();
+ recoveryMgr.checkpoint(true, RecoveryManager.NON_SHARP_CHECKPOINT_TARGET_LSN);
- if (isMetadataNode) {
- runtimeContext.exportMetadataNodeStub();
+ if (isMetadataNode) {
+ runtimeContext.exportMetadataNodeStub();
+ }
}
//Clean any temporary files
diff --git a/asterix-app/src/main/java/org/apache/asterix/messaging/CCMessageBroker.java b/asterix-app/src/main/java/org/apache/asterix/messaging/CCMessageBroker.java
index aeaef59..258bc35 100644
--- a/asterix-app/src/main/java/org/apache/asterix/messaging/CCMessageBroker.java
+++ b/asterix-app/src/main/java/org/apache/asterix/messaging/CCMessageBroker.java
@@ -22,9 +22,12 @@
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.messaging.AbstractApplicationMessage;
+import org.apache.asterix.common.messaging.PreparePartitionsFailbackResponseMessage;
+import org.apache.asterix.common.messaging.CompleteFailbackResponseMessage;
import org.apache.asterix.common.messaging.ReportMaxResourceIdMessage;
import org.apache.asterix.common.messaging.ReportMaxResourceIdRequestMessage;
import org.apache.asterix.common.messaging.ResourceIdRequestMessage;
@@ -54,6 +57,9 @@
@Override
public void receivedMessage(IMessage message, String nodeId) throws Exception {
AbstractApplicationMessage absMessage = (AbstractApplicationMessage) message;
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Received message: " + absMessage.getMessageType().name());
+ }
switch (absMessage.getMessageType()) {
case RESOURCE_ID_REQUEST:
handleResourceIdRequest(message, nodeId);
@@ -67,6 +73,12 @@
case TAKEOVER_METADATA_NODE_RESPONSE:
handleTakeoverMetadataNodeResponse(message);
break;
+ case PREPARE_PARTITIONS_FAILBACK_RESPONSE:
+ handleClosePartitionsResponse(message);
+ break;
+ case COMPLETE_FAILBACK_RESPONSE:
+ handleCompleteFailbcakResponse(message);
+ break;
default:
LOGGER.warning("Unknown message: " + absMessage.getMessageType());
break;
@@ -78,7 +90,7 @@
ResourceIdRequestResponseMessage reponse = new ResourceIdRequestResponseMessage();
reponse.setId(msg.getId());
//cluster is not active
- if (!AsterixClusterProperties.isClusterActive()) {
+ if (!AsterixClusterProperties.INSTANCE.isClusterActive()) {
reponse.setResourceId(-1);
reponse.setException(new Exception("Cannot generate global resource id when cluster is not active."));
} else if (nodesReportedMaxResourceId.size() < AsterixClusterProperties.getNumberOfNodes()) {
@@ -126,4 +138,14 @@
TakeoverMetadataNodeResponseMessage msg = (TakeoverMetadataNodeResponseMessage) message;
AsterixClusterProperties.INSTANCE.processMetadataNodeTakeoverResponse(msg);
}
+
+ private void handleCompleteFailbcakResponse(IMessage message) {
+ CompleteFailbackResponseMessage msg = (CompleteFailbackResponseMessage) message;
+ AsterixClusterProperties.INSTANCE.processCompleteFailbackResponse(msg);
+ }
+
+ private void handleClosePartitionsResponse(IMessage message) {
+ PreparePartitionsFailbackResponseMessage msg = (PreparePartitionsFailbackResponseMessage) message;
+ AsterixClusterProperties.INSTANCE.processPreparePartitionsFailbackResponse(msg);
+ }
}
diff --git a/asterix-app/src/main/java/org/apache/asterix/messaging/NCMessageBroker.java b/asterix-app/src/main/java/org/apache/asterix/messaging/NCMessageBroker.java
index 8f8723e..0a0a917 100644
--- a/asterix-app/src/main/java/org/apache/asterix/messaging/NCMessageBroker.java
+++ b/asterix-app/src/main/java/org/apache/asterix/messaging/NCMessageBroker.java
@@ -21,9 +21,16 @@
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.atomic.AtomicLong;
+import java.util.logging.Level;
+import java.util.logging.Logger;
import org.apache.asterix.common.api.IAsterixAppRuntimeContext;
import org.apache.asterix.common.messaging.AbstractApplicationMessage;
+import org.apache.asterix.common.messaging.CompleteFailbackRequestMessage;
+import org.apache.asterix.common.messaging.CompleteFailbackResponseMessage;
+import org.apache.asterix.common.messaging.PreparePartitionsFailbackRequestMessage;
+import org.apache.asterix.common.messaging.PreparePartitionsFailbackResponseMessage;
+import org.apache.asterix.common.messaging.ReplicaEventMessage;
import org.apache.asterix.common.messaging.ReportMaxResourceIdMessage;
import org.apache.asterix.common.messaging.TakeoverMetadataNodeResponseMessage;
import org.apache.asterix.common.messaging.TakeoverPartitionsRequestMessage;
@@ -32,18 +39,26 @@
import org.apache.asterix.common.messaging.api.IApplicationMessageCallback;
import org.apache.asterix.common.messaging.api.INCMessageBroker;
import org.apache.asterix.common.replication.IRemoteRecoveryManager;
+import org.apache.asterix.common.replication.Replica;
+import org.apache.asterix.common.replication.ReplicaEvent;
+import org.apache.asterix.event.schema.cluster.Node;
import org.apache.asterix.metadata.bootstrap.MetadataIndexImmutableProperties;
+import org.apache.asterix.transaction.management.resource.PersistentLocalResourceRepository;
import org.apache.hyracks.api.messages.IMessage;
import org.apache.hyracks.api.util.JavaSerializationUtils;
import org.apache.hyracks.control.nc.NodeControllerService;
public class NCMessageBroker implements INCMessageBroker {
+ private final static Logger LOGGER = Logger.getLogger(NCMessageBroker.class.getName());
+
private final NodeControllerService ncs;
private final AtomicLong messageId = new AtomicLong(0);
private final Map<Long, IApplicationMessageCallback> callbacks;
+ private final IAsterixAppRuntimeContext appContext;
public NCMessageBroker(NodeControllerService ncs) {
this.ncs = ncs;
+ appContext = (IAsterixAppRuntimeContext) ncs.getApplicationContext().getApplicationObject();
callbacks = new ConcurrentHashMap<Long, IApplicationMessageCallback>();
}
@@ -67,56 +82,72 @@
@Override
public void receivedMessage(IMessage message, String nodeId) throws Exception {
- AbstractApplicationMessage absMessage = (AbstractApplicationMessage) message;
- //if the received message is a response to a sent message, deliver it to the sender
- IApplicationMessageCallback callback = callbacks.remove(absMessage.getId());
- if (callback != null) {
- callback.deliverMessageResponse(absMessage);
- }
+ try {
+ AbstractApplicationMessage absMessage = (AbstractApplicationMessage) message;
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Received message: " + absMessage.getMessageType().name());
+ }
+ //if the received message is a response to a sent message, deliver it to the sender
+ IApplicationMessageCallback callback = callbacks.remove(absMessage.getId());
+ if (callback != null) {
+ callback.deliverMessageResponse(absMessage);
+ }
- //handle requests from CC
- switch (absMessage.getMessageType()) {
- case REPORT_MAX_RESOURCE_ID_REQUEST:
- reportMaxResourceId();
- break;
- case TAKEOVER_PARTITIONS_REQUEST:
- handleTakeoverPartitons(message);
- break;
- case TAKEOVER_METADATA_NODE_REQUEST:
- handleTakeoverMetadataNode(message);
- break;
- default:
- break;
+ //handle requests from CC
+ switch (absMessage.getMessageType()) {
+ case REPORT_MAX_RESOURCE_ID_REQUEST:
+ reportMaxResourceId();
+ break;
+ case TAKEOVER_PARTITIONS_REQUEST:
+ handleTakeoverPartitons(message);
+ break;
+ case TAKEOVER_METADATA_NODE_REQUEST:
+ handleTakeoverMetadataNode(message);
+ break;
+ case PREPARE_PARTITIONS_FAILBACK_REQUEST:
+ handlePreparePartitionsFailback(message);
+ break;
+ case COMPLETE_FAILBACK_REQUEST:
+ handleCompleteFailbackRequest(message);
+ break;
+ case REPLICA_EVENT:
+ handleReplicaEvent(message);
+ break;
+ default:
+ break;
+ }
+ } catch (Exception e) {
+ e.printStackTrace();
+ throw e;
}
}
private void handleTakeoverPartitons(IMessage message) throws Exception {
TakeoverPartitionsRequestMessage msg = (TakeoverPartitionsRequestMessage) message;
- IAsterixAppRuntimeContext appContext = (IAsterixAppRuntimeContext) ncs.getApplicationContext()
- .getApplicationObject();
- IRemoteRecoveryManager remoteRecoeryManager = appContext.getRemoteRecoveryManager();
- remoteRecoeryManager.takeoverPartitons(msg.getFailedNode(), msg.getPartitions());
- //send response after takeover is completed
- TakeoverPartitionsResponseMessage reponse = new TakeoverPartitionsResponseMessage(msg.getRequestId(),
- appContext.getTransactionSubsystem().getId(), msg.getPartitions());
- sendMessage(reponse, null);
+ try {
+ IRemoteRecoveryManager remoteRecoeryManager = appContext.getRemoteRecoveryManager();
+ remoteRecoeryManager.takeoverPartitons(msg.getPartitions());
+ } finally {
+ //send response after takeover is completed
+ TakeoverPartitionsResponseMessage reponse = new TakeoverPartitionsResponseMessage(msg.getRequestId(),
+ appContext.getTransactionSubsystem().getId(), msg.getPartitions());
+ sendMessage(reponse, null);
+ }
}
private void handleTakeoverMetadataNode(IMessage message) throws Exception {
- IAsterixAppRuntimeContext appContext = (IAsterixAppRuntimeContext) ncs.getApplicationContext()
- .getApplicationObject();
- appContext.initializeMetadata(false);
- appContext.exportMetadataNodeStub();
- //send response after takeover is completed
- TakeoverMetadataNodeResponseMessage reponse = new TakeoverMetadataNodeResponseMessage(
- appContext.getTransactionSubsystem().getId());
- sendMessage(reponse, null);
+ try {
+ appContext.initializeMetadata(false);
+ appContext.exportMetadataNodeStub();
+ } finally {
+ TakeoverMetadataNodeResponseMessage reponse = new TakeoverMetadataNodeResponseMessage(
+ appContext.getTransactionSubsystem().getId());
+ sendMessage(reponse, null);
+ }
}
@Override
public void reportMaxResourceId() throws Exception {
- IAsterixAppRuntimeContext appContext = (IAsterixAppRuntimeContext) ncs.getApplicationContext()
- .getApplicationObject();
ReportMaxResourceIdMessage maxResourceIdMsg = new ReportMaxResourceIdMessage();
//resource ids < FIRST_AVAILABLE_USER_DATASET_ID are reserved for metadata indexes.
long maxResourceId = Math.max(appContext.getLocalResourceRepository().getMaxResourceID(),
@@ -124,4 +155,58 @@
maxResourceIdMsg.setMaxResourceId(maxResourceId);
sendMessage(maxResourceIdMsg, null);
}
+
+ private void handleReplicaEvent(IMessage message) {
+ ReplicaEventMessage msg = (ReplicaEventMessage) message;
+ Node node = new Node();
+ node.setId(msg.getNodeId());
+ node.setClusterIp(msg.getNodeIPAddress());
+ Replica replica = new Replica(node);
+ ReplicaEvent event = new ReplicaEvent(replica, msg.getEvent());
+ appContext.getReplicationManager().reportReplicaEvent(event);
+ }
+
+ private void handlePreparePartitionsFailback(IMessage message) throws Exception {
+ PreparePartitionsFailbackRequestMessage msg = (PreparePartitionsFailbackRequestMessage) message;
+ /**
+ * if the metadata partition will be failed back
+ * we need to flush and close all datasets including metadata datasets
+ * otherwise we need to close all non-metadata datasets and flush metadata datasets
+ * so that their memory components will be copied to the failing back node
+ */
+ if (msg.isReleaseMetadataNode()) {
+ appContext.getDatasetLifecycleManager().closeAllDatasets();
+ //remove the metadata node stub from RMI registry
+ appContext.unexportMetadataNodeStub();
+ } else {
+ //close all non-metadata datasets
+ appContext.getDatasetLifecycleManager().closeUserDatasets();
+ //flush the remaining metadata datasets that were not closed
+ appContext.getDatasetLifecycleManager().flushAllDatasets();
+ }
+
+ //mark the partitions to be closed as inactive
+ PersistentLocalResourceRepository localResourceRepo = (PersistentLocalResourceRepository) appContext
+ .getLocalResourceRepository();
+ for (Integer partitionId : msg.getPartitions()) {
+ localResourceRepo.addInactivePartition(partitionId);
+ }
+
+ //send response after partitions prepared for failback
+ PreparePartitionsFailbackResponseMessage reponse = new PreparePartitionsFailbackResponseMessage(msg.getPlanId(),
+ msg.getRequestId(), msg.getPartitions());
+ sendMessage(reponse, null);
+ }
+
+ private void handleCompleteFailbackRequest(IMessage message) throws Exception {
+ CompleteFailbackRequestMessage msg = (CompleteFailbackRequestMessage) message;
+ try {
+ IRemoteRecoveryManager remoteRecoeryManager = appContext.getRemoteRecoveryManager();
+ remoteRecoeryManager.completeFailbackProcess();
+ } finally {
+ CompleteFailbackResponseMessage reponse = new CompleteFailbackResponseMessage(msg.getPlanId(),
+ msg.getRequestId(), msg.getPartitions());
+ sendMessage(reponse, null);
+ }
+ }
}
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 975180b..496a10e 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
@@ -55,8 +55,6 @@
public ILSMMergePolicyFactory getMetadataMergePolicyFactory();
- public int getMetaDataIODeviceId();
-
public IBufferCache getBufferCache();
public IFileMapProvider getFileMapManager();
@@ -93,14 +91,23 @@
/**
* Exports the metadata node to the metadata RMI port.
+ *
* @throws RemoteException
*/
public void exportMetadataNodeStub() throws RemoteException;
/**
* Initializes the metadata node and bootstraps the metadata.
+ *
* @param newUniverse
* @throws Exception
*/
public void initializeMetadata(boolean newUniverse) throws Exception;
+
+ /**
+ * Unexports the metadata node from the RMI registry
+ *
+ * @throws RemoteException
+ */
+ public void unexportMetadataNodeStub() throws RemoteException;
}
diff --git a/asterix-common/src/main/java/org/apache/asterix/common/api/IClusterManagementWork.java b/asterix-common/src/main/java/org/apache/asterix/common/api/IClusterManagementWork.java
index 75f1f82..adf8e38 100644
--- a/asterix-common/src/main/java/org/apache/asterix/common/api/IClusterManagementWork.java
+++ b/asterix-common/src/main/java/org/apache/asterix/common/api/IClusterManagementWork.java
@@ -28,7 +28,8 @@
public enum ClusterState {
STARTING,
ACTIVE,
- UNUSABLE
+ UNUSABLE,
+ REBALANCING
}
public WorkType getClusterManagementWorkType();
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 7e02faf..3b4617a 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
@@ -94,4 +94,11 @@
* @return a list of all indexes that are open at the time of the call.
*/
List<IndexInfo> getOpenIndexesInfo();
+
+ /**
+ * Flushes and closes all user datasets (non-metadata datasets)
+ *
+ * @throws HyracksDataException
+ */
+ void closeUserDatasets() throws HyracksDataException;
}
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
index fa5b503..019c168 100644
--- 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
@@ -34,7 +34,7 @@
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 int MAX_REMOTE_RECOVERY_ATTEMPTS = 5;
private static final String NODE_IP_ADDRESS_DEFAULT = "127.0.0.1";
private final String NODE_NAME_PREFIX;
private final Cluster cluster;
@@ -88,10 +88,10 @@
Set<Replica> remoteReplicas = new HashSet<Replica>();;
int numberOfRemoteReplicas = getReplicationFactor() - 1;
-
//Using chained-declustering
if (cluster != null) {
int nodeIndex = -1;
+ //find the node index in the cluster config
for (int i = 0; i < cluster.getNode().size(); i++) {
Node node = cluster.getNode().get(i);
if (getRealCluserNodeID(node.getId()).equals(nodeId)) {
@@ -106,19 +106,18 @@
return null;
}
+ //find nodes to the right of this node
for (int i = nodeIndex + 1; i < cluster.getNode().size(); i++) {
remoteReplicas.add(getReplicaByNodeIndex(i));
-
if (remoteReplicas.size() == numberOfRemoteReplicas) {
break;
}
}
+ //if not all remote replicas have been found, start from the beginning
if (remoteReplicas.size() != numberOfRemoteReplicas) {
for (int i = 0; i < cluster.getNode().size(); i++) {
-
remoteReplicas.add(getReplicaByNodeIndex(i));
-
if (remoteReplicas.size() == numberOfRemoteReplicas) {
break;
}
@@ -194,4 +193,51 @@
}
return REPLICATION_TIME_OUT_DEFAULT;
}
+
+ /**
+ * @param nodeId
+ * @return The set of nodes which replicate to this node, including the node itself
+ */
+ public Set<String> getNodeReplicationClients(String nodeId) {
+ Set<String> clientReplicas = new HashSet<>();
+ clientReplicas.add(nodeId);
+
+ int clientsCount = getReplicationFactor();
+
+ //Using chained-declustering backwards
+ if (cluster != null) {
+ int nodeIndex = -1;
+ //find the node index in the cluster config
+ for (int i = 0; i < cluster.getNode().size(); i++) {
+ Node node = cluster.getNode().get(i);
+ if (getRealCluserNodeID(node.getId()).equals(nodeId)) {
+ nodeIndex = i;
+ break;
+ }
+ }
+
+ //find nodes to the left of this node
+ for (int i = nodeIndex - 1; i >= 0; i--) {
+ clientReplicas.add(getReplicaByNodeIndex(i).getId());
+ if (clientReplicas.size() == clientsCount) {
+ break;
+ }
+ }
+
+ //if not all client replicas have been found, start from the end
+ if (clientReplicas.size() != clientsCount) {
+ for (int i = cluster.getNode().size() - 1; i >= 0; i--) {
+ clientReplicas.add(getReplicaByNodeIndex(i).getId());
+ if (clientReplicas.size() == clientsCount) {
+ break;
+ }
+ }
+ }
+ }
+ return clientReplicas;
+ }
+
+ public int getMaxRemoteRecoveryAttempts() {
+ return MAX_REMOTE_RECOVERY_ATTEMPTS;
+ }
}
\ No newline at end of file
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 53902e1..0cd88d6 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
@@ -75,7 +75,6 @@
capacity = storageProperties.getMemoryComponentGlobalBudget();
used = 0;
logRecord = new LogRecord();
- logRecord.setNodeId(logManager.getNodeId());
}
@Override
@@ -543,7 +542,8 @@
private void flushDatasetOpenIndexes(DatasetInfo dsInfo, boolean asyncFlush) throws HyracksDataException {
if (!dsInfo.isExternal) {
synchronized (logRecord) {
- TransactionUtil.formFlushLogRecord(logRecord, dsInfo.datasetID, null, dsInfo.indexes.size());
+ TransactionUtil.formFlushLogRecord(logRecord, dsInfo.datasetID, null, logManager.getNodeId(),
+ dsInfo.indexes.size());
try {
logManager.log(logRecord);
} catch (ACIDException e) {
@@ -612,13 +612,24 @@
}
@Override
- public void closeAllDatasets() throws HyracksDataException {
- for (DatasetInfo dsInfo : datasetInfos.values()) {
+ public synchronized void closeAllDatasets() throws HyracksDataException {
+ List<DatasetInfo> openDatasets = new ArrayList<>(datasetInfos.values());
+ for (DatasetInfo dsInfo : openDatasets) {
closeDataset(dsInfo);
}
}
@Override
+ public synchronized void closeUserDatasets() throws HyracksDataException {
+ List<DatasetInfo> openDatasets = new ArrayList<>(datasetInfos.values());
+ for (DatasetInfo dsInfo : openDatasets) {
+ if (dsInfo.datasetID >= firstAvilableUserDatasetID) {
+ closeDataset(dsInfo);
+ }
+ }
+ }
+
+ @Override
public synchronized void stop(boolean dumpState, OutputStream outputStream) throws IOException {
if (dumpState) {
dumpState(outputStream);
diff --git a/asterix-common/src/main/java/org/apache/asterix/common/dataflow/AsterixLSMInsertDeleteOperatorNodePushable.java b/asterix-common/src/main/java/org/apache/asterix/common/dataflow/AsterixLSMInsertDeleteOperatorNodePushable.java
index d25e51f..afdbb31 100644
--- a/asterix-common/src/main/java/org/apache/asterix/common/dataflow/AsterixLSMInsertDeleteOperatorNodePushable.java
+++ b/asterix-common/src/main/java/org/apache/asterix/common/dataflow/AsterixLSMInsertDeleteOperatorNodePushable.java
@@ -65,7 +65,8 @@
try {
writer.open();
modCallback = opDesc.getModificationOpCallbackFactory().createModificationOperationCallback(
- indexHelper.getResourcePath(), indexHelper.getResourceID(), lsmIndex, ctx);
+ indexHelper.getResourcePath(), indexHelper.getResourceID(), indexHelper.getResourcePartition(),
+ lsmIndex, ctx);
indexAccessor = lsmIndex.createAccessor(modCallback, NoOpOperationCallback.INSTANCE);
ITupleFilterFactory tupleFilterFactory = opDesc.getTupleFilterFactory();
if (tupleFilterFactory != null) {
diff --git a/asterix-common/src/main/java/org/apache/asterix/common/messaging/AbstractFailbackPlanMessage.java b/asterix-common/src/main/java/org/apache/asterix/common/messaging/AbstractFailbackPlanMessage.java
new file mode 100644
index 0000000..2c72051
--- /dev/null
+++ b/asterix-common/src/main/java/org/apache/asterix/common/messaging/AbstractFailbackPlanMessage.java
@@ -0,0 +1,39 @@
+/*
+ * 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.messaging;
+
+public abstract class AbstractFailbackPlanMessage extends AbstractApplicationMessage {
+
+ private static final long serialVersionUID = 1L;
+ protected final long planId;
+ protected final int requestId;
+
+ public AbstractFailbackPlanMessage(long planId, int requestId) {
+ this.planId = planId;
+ this.requestId = requestId;
+ }
+
+ public long getPlanId() {
+ return planId;
+ }
+
+ public int getRequestId() {
+ return requestId;
+ }
+}
\ No newline at end of file
diff --git a/asterix-common/src/main/java/org/apache/asterix/common/messaging/CompleteFailbackRequestMessage.java b/asterix-common/src/main/java/org/apache/asterix/common/messaging/CompleteFailbackRequestMessage.java
new file mode 100644
index 0000000..510817a
--- /dev/null
+++ b/asterix-common/src/main/java/org/apache/asterix/common/messaging/CompleteFailbackRequestMessage.java
@@ -0,0 +1,56 @@
+/*
+ * 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.messaging;
+
+import java.util.Set;
+
+public class CompleteFailbackRequestMessage extends AbstractFailbackPlanMessage {
+
+ private static final long serialVersionUID = 1L;
+ private final Set<Integer> partitions;
+ private final String nodeId;
+
+ public CompleteFailbackRequestMessage(long planId, int requestId, String nodeId, Set<Integer> partitions) {
+ super(planId, requestId);
+ this.nodeId = nodeId;
+ this.partitions = partitions;
+ }
+
+ @Override
+ public ApplicationMessageType getMessageType() {
+ return ApplicationMessageType.COMPLETE_FAILBACK_REQUEST;
+ }
+
+ public Set<Integer> getPartitions() {
+ return partitions;
+ }
+
+ public String getNodeId() {
+ return nodeId;
+ }
+
+ @Override
+ public String toString() {
+ StringBuilder sb = new StringBuilder();
+ sb.append("Plan ID: " + planId);
+ sb.append(" Node ID: " + nodeId);
+ sb.append(" Partitions: " + partitions);
+ return sb.toString();
+ }
+}
\ No newline at end of file
diff --git a/asterix-common/src/main/java/org/apache/asterix/common/messaging/CompleteFailbackResponseMessage.java b/asterix-common/src/main/java/org/apache/asterix/common/messaging/CompleteFailbackResponseMessage.java
new file mode 100644
index 0000000..6d77920
--- /dev/null
+++ b/asterix-common/src/main/java/org/apache/asterix/common/messaging/CompleteFailbackResponseMessage.java
@@ -0,0 +1,49 @@
+/*
+ * 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.messaging;
+
+import java.util.Set;
+
+public class CompleteFailbackResponseMessage extends AbstractFailbackPlanMessage {
+
+ private static final long serialVersionUID = 1L;
+ private final Set<Integer> partitions;
+
+ public CompleteFailbackResponseMessage(long planId, int requestId, Set<Integer> partitions) {
+ super(planId, requestId);
+ this.partitions = partitions;
+ }
+
+ @Override
+ public ApplicationMessageType getMessageType() {
+ return ApplicationMessageType.COMPLETE_FAILBACK_RESPONSE;
+ }
+
+ public Set<Integer> getPartitions() {
+ return partitions;
+ }
+
+ @Override
+ public String toString() {
+ StringBuilder sb = new StringBuilder();
+ sb.append("Plan ID: " + planId);
+ sb.append(" Partitions: " + partitions);
+ return sb.toString();
+ }
+}
\ No newline at end of file
diff --git a/asterix-common/src/main/java/org/apache/asterix/common/messaging/PreparePartitionsFailbackRequestMessage.java b/asterix-common/src/main/java/org/apache/asterix/common/messaging/PreparePartitionsFailbackRequestMessage.java
new file mode 100644
index 0000000..7909a35
--- /dev/null
+++ b/asterix-common/src/main/java/org/apache/asterix/common/messaging/PreparePartitionsFailbackRequestMessage.java
@@ -0,0 +1,65 @@
+/*
+ * 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.messaging;
+
+import java.util.Set;
+
+public class PreparePartitionsFailbackRequestMessage extends AbstractFailbackPlanMessage {
+
+ private static final long serialVersionUID = 1L;
+ private final Set<Integer> partitions;
+ private boolean releaseMetadataNode = false;
+ private final String nodeID;
+
+ public PreparePartitionsFailbackRequestMessage(long planId, int requestId, String nodeId, Set<Integer> partitions) {
+ super(planId, requestId);
+ this.nodeID = nodeId;
+ this.partitions = partitions;
+ }
+
+ @Override
+ public ApplicationMessageType getMessageType() {
+ return ApplicationMessageType.PREPARE_PARTITIONS_FAILBACK_REQUEST;
+ }
+
+ public Set<Integer> getPartitions() {
+ return partitions;
+ }
+
+ public boolean isReleaseMetadataNode() {
+ return releaseMetadataNode;
+ }
+
+ public void setReleaseMetadataNode(boolean releaseMetadataNode) {
+ this.releaseMetadataNode = releaseMetadataNode;
+ }
+
+ public String getNodeID() {
+ return nodeID;
+ }
+
+ @Override
+ public String toString() {
+ StringBuilder sb = new StringBuilder();
+ sb.append("Plan ID: " + planId);
+ sb.append(" Partitions: " + partitions);
+ sb.append(" releaseMetadataNode: " + releaseMetadataNode);
+ return sb.toString();
+ }
+}
\ No newline at end of file
diff --git a/asterix-common/src/main/java/org/apache/asterix/common/messaging/PreparePartitionsFailbackResponseMessage.java b/asterix-common/src/main/java/org/apache/asterix/common/messaging/PreparePartitionsFailbackResponseMessage.java
new file mode 100644
index 0000000..6b058c7
--- /dev/null
+++ b/asterix-common/src/main/java/org/apache/asterix/common/messaging/PreparePartitionsFailbackResponseMessage.java
@@ -0,0 +1,41 @@
+/*
+ * 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.messaging;
+
+import java.util.Set;
+
+public class PreparePartitionsFailbackResponseMessage extends AbstractFailbackPlanMessage {
+
+ private static final long serialVersionUID = 1L;
+ private final Set<Integer> partitions;
+
+ public PreparePartitionsFailbackResponseMessage(long planId, int requestId, Set<Integer> partitions) {
+ super(planId, requestId);
+ this.partitions = partitions;
+ }
+
+ @Override
+ public ApplicationMessageType getMessageType() {
+ return ApplicationMessageType.PREPARE_PARTITIONS_FAILBACK_RESPONSE;
+ }
+
+ public Set<Integer> getPartitions() {
+ return partitions;
+ }
+}
\ No newline at end of file
diff --git a/asterix-common/src/main/java/org/apache/asterix/common/messaging/ReplicaEventMessage.java b/asterix-common/src/main/java/org/apache/asterix/common/messaging/ReplicaEventMessage.java
new file mode 100644
index 0000000..28fd36f
--- /dev/null
+++ b/asterix-common/src/main/java/org/apache/asterix/common/messaging/ReplicaEventMessage.java
@@ -0,0 +1,52 @@
+/*
+ * 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.messaging;
+
+import org.apache.hyracks.api.application.IClusterLifecycleListener.ClusterEventType;
+
+public class ReplicaEventMessage extends AbstractApplicationMessage {
+
+ private static final long serialVersionUID = 1L;
+ private final String nodeId;
+ private final ClusterEventType event;
+ private final String nodeIPAddress;
+
+ public ReplicaEventMessage(String nodeId, String nodeIPAddress, ClusterEventType event) {
+ this.nodeId = nodeId;
+ this.nodeIPAddress = nodeIPAddress;
+ this.event = event;
+ }
+
+ @Override
+ public ApplicationMessageType getMessageType() {
+ return ApplicationMessageType.REPLICA_EVENT;
+ }
+
+ public String getNodeId() {
+ return nodeId;
+ }
+
+ public ClusterEventType getEvent() {
+ return event;
+ }
+
+ public String getNodeIPAddress() {
+ return nodeIPAddress;
+ }
+}
\ No newline at end of file
diff --git a/asterix-common/src/main/java/org/apache/asterix/common/messaging/TakeoverPartitionsRequestMessage.java b/asterix-common/src/main/java/org/apache/asterix/common/messaging/TakeoverPartitionsRequestMessage.java
index abfa7d2..ad5234b 100644
--- a/asterix-common/src/main/java/org/apache/asterix/common/messaging/TakeoverPartitionsRequestMessage.java
+++ b/asterix-common/src/main/java/org/apache/asterix/common/messaging/TakeoverPartitionsRequestMessage.java
@@ -22,15 +22,12 @@
private static final long serialVersionUID = 1L;
private final Integer[] partitions;
- private final String failedNode;
private final long requestId;
private final String nodeId;
- public TakeoverPartitionsRequestMessage(long requestId, String nodeId, String failedNode,
- Integer[] partitionsToTakeover) {
+ public TakeoverPartitionsRequestMessage(long requestId, String nodeId, Integer[] partitionsToTakeover) {
this.requestId = requestId;
this.nodeId = nodeId;
- this.failedNode = failedNode;
this.partitions = partitionsToTakeover;
}
@@ -47,10 +44,6 @@
return requestId;
}
- public String getFailedNode() {
- return failedNode;
- }
-
public String getNodeId() {
return nodeId;
}
@@ -60,7 +53,6 @@
StringBuilder sb = new StringBuilder();
sb.append("Request ID: " + requestId);
sb.append(" Node ID: " + nodeId);
- sb.append(" Failed Node: " + failedNode);
sb.append(" Partitions: ");
for (Integer partitionId : partitions) {
sb.append(partitionId + ",");
diff --git a/asterix-common/src/main/java/org/apache/asterix/common/messaging/api/IApplicationMessage.java b/asterix-common/src/main/java/org/apache/asterix/common/messaging/api/IApplicationMessage.java
index 57a0dae..5d2e263 100644
--- a/asterix-common/src/main/java/org/apache/asterix/common/messaging/api/IApplicationMessage.java
+++ b/asterix-common/src/main/java/org/apache/asterix/common/messaging/api/IApplicationMessage.java
@@ -30,7 +30,12 @@
TAKEOVER_PARTITIONS_REQUEST,
TAKEOVER_PARTITIONS_RESPONSE,
TAKEOVER_METADATA_NODE_REQUEST,
- TAKEOVER_METADATA_NODE_RESPONSE
+ TAKEOVER_METADATA_NODE_RESPONSE,
+ PREPARE_PARTITIONS_FAILBACK_REQUEST,
+ PREPARE_PARTITIONS_FAILBACK_RESPONSE,
+ COMPLETE_FAILBACK_REQUEST,
+ COMPLETE_FAILBACK_RESPONSE,
+ REPLICA_EVENT
}
public abstract ApplicationMessageType getMessageType();
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
index ecc9494..6d5918f 100644
--- 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
@@ -36,6 +36,17 @@
* @throws IOException
* @throws ACIDException
*/
- public void takeoverPartitons(String failedNode, Integer[] partitions) throws IOException, ACIDException;
+ public void takeoverPartitons(Integer[] partitions) throws IOException, ACIDException;
+ /**
+ * Construct failback plan and requests LSM disk components from active remote replicas.
+ */
+ public void startFailbackProcess();
+
+ /**
+ * Requests the remaining LSM disk components files from active remote replicas.
+ *
+ * @throws IOException
+ */
+ public void completeFailbackProcess() throws IOException;
}
\ 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
index f13d300..e22fafc 100644
--- 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
@@ -23,14 +23,8 @@
public interface IReplicaResourcesManager {
/**
- * @param remoteNodes
- * @return The minimum LSN of all indexes that belong to {@code remoteNodes}.
- */
- public long getMinRemoteLSN(Set<String> remoteNodes);
-
- /**
* @param partitions
* @return the minimum LSN of all indexes that belong to {@code partitions}.
*/
- public long getPartitionsMinLSN(Integer[] partitions);
+ public long getPartitionsMinLSN(Set<Integer> partitions);
}
\ 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
index 76f8767..3fc2af0 100644
--- 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
@@ -18,8 +18,8 @@
*/
package org.apache.asterix.common.replication;
+import java.io.File;
import java.io.IOException;
-import java.util.ArrayList;
import java.util.Set;
import org.apache.asterix.common.exceptions.ACIDException;
@@ -54,12 +54,13 @@
* 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.
+ * @param recoveryLogsFile
+ * a temporary file to store the logs required for recovery
* @throws IOException
* @throws ACIDException
*/
- public ArrayList<ILogRecord> requestReplicaLogs(String remoteReplicaId, Set<String> replicasDataToRecover,
- long fromLSN) throws IOException, ACIDException;
+ public void requestReplicaLogs(String remoteReplicaId, Set<String> replicasDataToRecover, long fromLSN,
+ File recoveryLogsFile) throws IOException, ACIDException;
/**
* Requests LSM components files from a remote replica.
@@ -68,9 +69,12 @@
* The replica id to send the request to.
* @param replicasDataToRecover
* Get files that belong to those replicas.
+ * @param existingFiles
+ * a list of already existing files on the requester
* @throws IOException
*/
- public void requestReplicaFiles(String remoteReplicaId, Set<String> replicasDataToRecover) throws IOException;
+ public void requestReplicaFiles(String remoteReplicaId, Set<String> replicasDataToRecover,
+ Set<String> existingFiles) throws IOException;
/**
* Requests current maximum LSN from remote replicas.
@@ -83,13 +87,6 @@
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();
@@ -146,5 +143,4 @@
* @throws IOException
*/
public void requestFlushLaggingReplicaIndexes(long nonSharpCheckpointTargetLSN) throws IOException;
-
}
diff --git a/asterix-common/src/main/java/org/apache/asterix/common/replication/NodeFailbackPlan.java b/asterix-common/src/main/java/org/apache/asterix/common/replication/NodeFailbackPlan.java
new file mode 100644
index 0000000..0591644
--- /dev/null
+++ b/asterix-common/src/main/java/org/apache/asterix/common/replication/NodeFailbackPlan.java
@@ -0,0 +1,209 @@
+/*
+ * 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.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.asterix.common.messaging.CompleteFailbackRequestMessage;
+import org.apache.asterix.common.messaging.PreparePartitionsFailbackRequestMessage;
+
+public class NodeFailbackPlan {
+
+ public enum FailbackPlanState {
+ /**
+ * Initial state while selecting the nodes that will participate
+ * in the node failback plan.
+ */
+ PREPARING,
+ /**
+ * Once a pending {@link PreparePartitionsFailbackRequestMessage} request is added,
+ * the state is changed from PREPARING to PENDING_PARTICIPANT_REPONSE to indicate
+ * a response is expected and need to wait for it.
+ */
+ PENDING_PARTICIPANT_REPONSE,
+ /**
+ * Upon receiving the last {@link PreparePartitionsFailbackResponseMessage} response,
+ * the state changes from PENDING_PARTICIPANT_REPONSE to PENDING_COMPLETION to indicate
+ * the need to send {@link CompleteFailbackRequestMessage} to the failing back node.
+ */
+ PENDING_COMPLETION,
+ /**
+ * if any of the participants fail or the failing back node itself fails during
+ * and of these states (PREPARING, PENDING_PARTICIPANT_REPONSE, PENDING_COMPLETION),
+ * the state is changed to FAILED.
+ */
+ FAILED,
+ /**
+ * if the state is FAILED, and all pending responses (if any) have been received,
+ * the state changes from FAILED to PENDING_ROLLBACK to indicate the need to revert
+ * the effects of this plan (if any).
+ */
+ PENDING_ROLLBACK
+ }
+
+ private static long planIdGenerator = 0;
+ private long planId;
+ private final String nodeId;
+ private final Set<String> participants;
+ private final Map<Integer, String> partition2nodeMap;
+ private String nodeToReleaseMetadataManager;
+ private int requestId;
+ private Map<Integer, PreparePartitionsFailbackRequestMessage> pendingRequests;
+ private FailbackPlanState state;
+
+ public static NodeFailbackPlan createPlan(String nodeId) {
+ return new NodeFailbackPlan(planIdGenerator++, nodeId);
+ }
+
+ private NodeFailbackPlan(long planId, String nodeId) {
+ this.planId = planId;
+ this.nodeId = nodeId;
+ participants = new HashSet<>();
+ partition2nodeMap = new HashMap<>();
+ pendingRequests = new HashMap<>();
+ state = FailbackPlanState.PREPARING;
+ }
+
+ public synchronized void addPartitionToFailback(int partitionId, String currentActiveNode) {
+ partition2nodeMap.put(partitionId, currentActiveNode);
+ }
+
+ public synchronized void addParticipant(String nodeId) {
+ participants.add(nodeId);
+ }
+
+ public synchronized void notifyNodeFailure(String failedNode) {
+ if (participants.contains(failedNode)) {
+ if (state == FailbackPlanState.PREPARING) {
+ state = FailbackPlanState.FAILED;
+ } else if (state == FailbackPlanState.PENDING_PARTICIPANT_REPONSE) {
+ /**
+ * if there is any pending request from this failed node,
+ * it should be marked as completed and the plan should be marked as failed
+ */
+ Set<Integer> failedRequests = new HashSet<>();
+ for (PreparePartitionsFailbackRequestMessage request : pendingRequests.values()) {
+ if (request.getNodeID().equals(failedNode)) {
+ failedRequests.add(request.getRequestId());
+ }
+ }
+
+ if (failedRequests.size() > 0) {
+ state = FailbackPlanState.FAILED;
+ for (Integer failedRequestId : failedRequests) {
+ markRequestCompleted(failedRequestId);
+ }
+ }
+ }
+ } else if (nodeId.equals(failedNode)) {
+ //if the failing back node is the failed node itself
+ state = FailbackPlanState.FAILED;
+ updateState();
+ }
+ }
+
+ public synchronized Set<Integer> getPartitionsToFailback() {
+ return new HashSet<>(partition2nodeMap.keySet());
+ }
+
+ public synchronized void addPendingRequest(PreparePartitionsFailbackRequestMessage msg) {
+ //if this is the first request
+ if (pendingRequests.size() == 0) {
+ state = FailbackPlanState.PENDING_PARTICIPANT_REPONSE;
+ }
+ pendingRequests.put(msg.getRequestId(), msg);
+ }
+
+ public synchronized void markRequestCompleted(int requestId) {
+ pendingRequests.remove(requestId);
+ updateState();
+ }
+
+ private void updateState() {
+ if (pendingRequests.size() == 0) {
+ switch (state) {
+ case PREPARING:
+ case FAILED:
+ state = FailbackPlanState.PENDING_ROLLBACK;
+ break;
+ case PENDING_PARTICIPANT_REPONSE:
+ state = FailbackPlanState.PENDING_COMPLETION;
+ break;
+ default:
+ break;
+ }
+ }
+ }
+
+ public synchronized Set<PreparePartitionsFailbackRequestMessage> getPlanFailbackRequests() {
+ Set<PreparePartitionsFailbackRequestMessage> node2Partitions = new HashSet<>();
+ /**
+ * for each participant, construct a request with the partitions
+ * that will be failed back or flushed.
+ */
+ for (String participant : participants) {
+ Set<Integer> partitionToPrepareForFailback = new HashSet<>();
+ for (Map.Entry<Integer, String> entry : partition2nodeMap.entrySet()) {
+ if (entry.getValue().equals(participant)) {
+ partitionToPrepareForFailback.add(entry.getKey());
+ }
+ }
+ PreparePartitionsFailbackRequestMessage msg = new PreparePartitionsFailbackRequestMessage(planId,
+ requestId++, participant, partitionToPrepareForFailback);
+ if (participant.equals(nodeToReleaseMetadataManager)) {
+ msg.setReleaseMetadataNode(true);
+ }
+ node2Partitions.add(msg);
+ }
+ return node2Partitions;
+ }
+
+ public synchronized CompleteFailbackRequestMessage getCompleteFailbackRequestMessage() {
+ return new CompleteFailbackRequestMessage(planId, requestId++, nodeId, getPartitionsToFailback());
+ }
+
+ public String getNodeId() {
+ return nodeId;
+ }
+
+ public long getPlanId() {
+ return planId;
+ }
+
+ public void setNodeToReleaseMetadataManager(String nodeToReleaseMetadataManager) {
+ this.nodeToReleaseMetadataManager = nodeToReleaseMetadataManager;
+ }
+
+ public synchronized FailbackPlanState getState() {
+ return state;
+ }
+
+ @Override
+ public String toString() {
+ StringBuilder sb = new StringBuilder();
+ sb.append("Plan ID: " + planId);
+ sb.append(" Failing back node: " + nodeId);
+ sb.append(" Participants: " + participants);
+ sb.append(" Partitions to Failback: " + partition2nodeMap.keySet());
+ return sb.toString();
+ }
+}
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
index 0797a02..ae02ca9 100644
--- 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
@@ -23,23 +23,14 @@
import java.io.IOException;
import java.io.OutputStream;
+import org.apache.hyracks.api.application.IClusterLifecycleListener.ClusterEventType;
+
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;
+ ClusterEventType eventType;
- public ReplicaEvent(Replica replica, ReplicaEventType eventType) {
+ public ReplicaEvent(Replica replica, ClusterEventType eventType) {
this.replica = replica;
this.eventType = eventType;
}
@@ -52,11 +43,11 @@
this.replica = replica;
}
- public ReplicaEventType getEventType() {
+ public ClusterEventType getEventType() {
return eventType;
}
- public void setEventType(ReplicaEventType eventType) {
+ public void setEventType(ClusterEventType eventType) {
this.eventType = eventType;
}
@@ -68,7 +59,7 @@
public static ReplicaEvent create(DataInput input) throws IOException {
Replica replica = Replica.create(input);
- ReplicaEventType eventType = ReplicaEventType.values()[input.readInt()];
+ ClusterEventType eventType = ClusterEventType.values()[input.readInt()];
ReplicaEvent event = new ReplicaEvent(replica, eventType);
return event;
}
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 a88c985..7e27c54 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
@@ -31,10 +31,10 @@
OK
}
- public static final int JOB_TERMINATE_LOG_SIZE = 18; //JOB_COMMIT or ABORT log type
+ public static final int JOB_TERMINATE_LOG_SIZE = 14; //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 static final int FLUSH_LOG_SIZE = 18;
public LogRecord.RECORD_STATUS readLogRecord(ByteBuffer buffer);
@@ -112,11 +112,9 @@
public String getNodeId();
- public void setNodeId(String nodeId);
-
public int writeRemoteRecoveryLog(ByteBuffer buffer);
- public void readRemoteLog(ByteBuffer buffer, boolean remoteRecoveryLog, String localNodeId);
+ public RECORD_STATUS readRemoteLog(ByteBuffer buffer, boolean remoteRecoveryLog);
public void setReplicationThread(IReplicationThread replicationThread);
@@ -130,4 +128,16 @@
public ByteBuffer getSerializedLog();
+ public void setNodeId(String nodeId);
+
+ public int getResourcePartition();
+
+ public void setResourcePartition(int resourcePartition);
+
+ public void setReplicated(boolean replicated);
+
+ /**
+ * @return a flag indicating whether the log record should be sent to remote replicas
+ */
+ public boolean isReplicated();
}
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 cb3f8c3..a3115e7 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
@@ -18,8 +18,9 @@
*/
package org.apache.asterix.common.transactions;
+import java.io.File;
import java.io.IOException;
-import java.util.ArrayList;
+import java.util.Set;
import org.apache.asterix.common.exceptions.ACIDException;
import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -82,31 +83,25 @@
/**
* Makes a system checkpoint.
- * @param isSharpCheckpoint a flag indicating whether to perform a sharp or non-sharp checkpoint.
- * @param nonSharpCheckpointTargetLSN if a non-sharp checkpoint to be performed, what is the minimum LSN it should target.
+ *
+ * @param isSharpCheckpoint
+ * a flag indicating whether to perform a sharp or non-sharp checkpoint.
+ * @param nonSharpCheckpointTargetLSN
+ * if a non-sharp checkpoint to be performed, what is the minimum LSN it should target.
* @return the LSN at which the checkpoint was performed.
* @throws ACIDException
* @throws HyracksDataException
*/
- public long checkpoint(boolean isSharpCheckpoint, long nonSharpCheckpointTargetLSN) throws ACIDException, HyracksDataException;
+ 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
*/
@@ -114,11 +109,29 @@
/**
* Replay the logs that belong to the passed {@code partitions} starting from the {@code lowWaterMarkLSN}
+ *
* @param partitions
* @param lowWaterMarkLSN
* @param failedNode
* @throws IOException
* @throws ACIDException
*/
- public void replayPartitionsLogs(Integer[] partitions, long lowWaterMarkLSN, String failedNode) throws IOException, ACIDException;
+ public void replayPartitionsLogs(Set<Integer> partitions, ILogReader logReader, long lowWaterMarkLSN)
+ throws IOException, ACIDException;
+
+ /**
+ * Creates a temporary file to be used during recovery
+ *
+ * @param jobId
+ * @param fileName
+ * @return A file to the created temporary file
+ * @throws IOException
+ * if the file for the specified {@code jobId} with the {@code fileName} already exists
+ */
+ public File createJobRecoveryFile(int jobId, String fileName) throws IOException;
+
+ /**
+ * Deletes all temporary recovery files
+ */
+ public void deleteRecoveryTemporaryFiles();
}
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 d4a96a5..d09f6ca 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,8 +20,6 @@
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;
@@ -34,15 +32,14 @@
/*
* == LogRecordFormat ==
* ---------------------------
- * [Header1] (10 bytes + NodeId Length) : for all log types
+ * [Header1] (6 bytes) : for all log types
* LogSource(1)
* LogType(1)
* JobId(4)
- * NodeIdLength(4)
- * NodeId(?)
* ---------------------------
- * [Header2] (12 bytes + PKValueSize) : for entity_commit, upsert_entity_commit, and update log types
+ * [Header2] (16 bytes + PKValueSize) : for entity_commit, upsert_entity_commit, and update log types
* DatasetId(4) //stored in dataset_dataset in Metadata Node
+ * ResourcePartition(4)
* PKHashValue(4)
* PKValueSize(4)
* PKValue(PKValueSize)
@@ -52,7 +49,7 @@
* ResourceId(8) //stored in .metadata of the corresponding index in NC node
* LogRecordSize(4)
* ---------------------------
- * [Body] (Variable size) : only for update log type
+ * [Body] (9 bytes + NewValueSize) : only for update log type
* FieldCnt(4)
* NewOp(1)
* NewValueSize(4)
@@ -62,19 +59,18 @@
* Checksum(8)
* ---------------------------
* = LogSize =
- * 1) JOB_COMMIT_LOG_SIZE: 13 bytes (5 + 8)
- * 2) ENTITY_COMMIT || UPSERT_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) (In case of serialize: + (8 bytes for LSN) + (4 bytes for number of flushed indexes)
+ * 1) JOB_COMMIT_LOG_SIZE: 14 bytes (Header1(6) + Tail(8))
+ * 2) ENTITY_COMMIT || UPSERT_ENTITY_COMMIT: (Header1(6) + Header2(16) + Tail(8)) + PKValueSize
+ * --> ENTITY_COMMIT_LOG_BASE_SIZE = 30
+ * 3) UPDATE: (Header1(6) + Header2(16) + + Header3(20) + Body(9) + Tail(8)) + PKValueSize + NewValueSize
+ * --> UPDATE_LOG_BASE_SIZE = 59
+ * 4) FLUSH: 18 bytes (Header1(6) + DatasetId(4) + Tail(8))
*/
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;
@@ -83,6 +79,7 @@
private ITupleReference PKValue;
private long prevLSN;
private long resourceId;
+ private int resourcePartition;
private int logSize;
private int fieldCnt;
private byte newOp;
@@ -103,9 +100,13 @@
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.
+ /**
+ * The fields (numOfFlushedIndexes and nodeId) are used for serialized flush logs only
+ * to indicate the source of the log and how many indexes were flushed using its LSN.
+ */
private int numOfFlushedIndexes;
+ private String nodeId;
+ private boolean replicated = false;
public LogRecord() {
isFlushed = new AtomicBoolean(false);
@@ -113,16 +114,15 @@
readPKValue = new PrimaryKeyTupleReference();
readNewValue = (SimpleTupleReference) tupleWriter.createTupleReference();
checksumGen = new CRC32();
- this.nodeIdsMap = new HashMap<String, byte[]>();
logSource = LogSource.LOCAL;
}
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 RS_PARTITION_LEN = Integer.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;
@@ -130,27 +130,19 @@
private final static int NEWVALSZ_LEN = Integer.BYTES;
private final static int CHKSUM_LEN = Long.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 ALL_RECORD_HEADER_LEN = LOG_SOURCE_LEN + TYPE_LEN + JobId.BYTES;
+ private final static int ENTITYCOMMIT_UPDATE_HEADER_LEN = RS_PARTITION_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;
+ private final static int REMOTE_FLUSH_LOG_EXTRA_FIELDS_LEN = Long.BYTES + Integer.BYTES + Integer.BYTES;
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 || logType == LogType.UPSERT_ENTITY_COMMIT) {
+ buffer.putInt(resourcePartition);
buffer.putInt(datasetId);
buffer.putInt(PKHashValue);
if (PKValueSize <= 0) {
@@ -168,7 +160,6 @@
buffer.putInt(newValueSize);
writeTuple(buffer, newValue, newValueSize);
}
-
if (logType == LogType.FLUSH) {
buffer.putInt(datasetId);
}
@@ -188,10 +179,9 @@
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 (replicated) {
+ //copy the serialized log to send it to replicas
+ int serializedLogSize = getSerializedLogSize();
if (serializedLog == null || serializedLog.capacity() < serializedLogSize) {
serializedLog = ByteBuffer.allocate(serializedLogSize);
} else {
@@ -207,6 +197,8 @@
if (logType == LogType.FLUSH) {
serializedLog.putLong(appendLSN);
serializedLog.putInt(numOfFlushedIndexes);
+ serializedLog.putInt(nodeId.length());
+ serializedLog.put(nodeId.getBytes());
}
serializedLog.flip();
buffer.position(currentPosition);
@@ -244,15 +236,8 @@
public RECORD_STATUS readLogRecord(ByteBuffer buffer) {
int beginOffset = buffer.position();
- // read header
- RECORD_STATUS status = readLogHeader(buffer);
- if (status != RECORD_STATUS.OK) {
- buffer.position(beginOffset);
- return status;
- }
-
- // read body
- status = readLogBody(buffer, false);
+ //read common fields
+ RECORD_STATUS status = readLogCommonFields(buffer);
if (status != RECORD_STATUS.OK) {
buffer.position(beginOffset);
return status;
@@ -271,38 +256,25 @@
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.
+ private RECORD_STATUS readLogCommonFields(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;
PKHashValue = -1;
} else {
- // attempt to read in the dsid, PK hash and PK length
+ //attempt to read in the resourcePartition, dsid, PK hash and PK length
if (buffer.remaining() < ENTITYCOMMIT_UPDATE_HEADER_LEN) {
return RECORD_STATUS.TRUNCATED;
}
+ resourcePartition = buffer.getInt();
datasetId = buffer.getInt();
PKHashValue = buffer.getInt();
PKValueSize = buffer.getInt();
@@ -330,16 +302,7 @@
if (buffer.remaining() < newValueSize) {
return RECORD_STATUS.TRUNCATED;
}
- 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);
- }
+ newValue = readTuple(buffer, readNewValue, fieldCnt, newValueSize);
} else {
computeAndSetLogSize();
}
@@ -356,25 +319,47 @@
}
@Override
- public void readRemoteLog(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);
+ public RECORD_STATUS readRemoteLog(ByteBuffer buffer, boolean remoteRecoveryLog) {
+ int beginOffset = buffer.position();
+
+ //read common fields
+ RECORD_STATUS status = readLogCommonFields(buffer);
+ if (status != RECORD_STATUS.OK) {
+ buffer.position(beginOffset);
+ return status;
}
if (logType == LogType.FLUSH) {
- LSN = buffer.getLong();
- numOfFlushedIndexes = buffer.getInt();
+ if (buffer.remaining() >= REMOTE_FLUSH_LOG_EXTRA_FIELDS_LEN) {
+ LSN = buffer.getLong();
+ numOfFlushedIndexes = buffer.getInt();
+ //read serialized node id
+ int nodeIdLength = buffer.getInt();
+ if (buffer.remaining() >= nodeIdLength) {
+ byte[] nodeIdBytes = new byte[nodeIdLength];
+ buffer.get(nodeIdBytes);
+ nodeId = new String(nodeIdBytes);
+ } else {
+ buffer.position(beginOffset);
+ return RECORD_STATUS.TRUNCATED;
+ }
+ } else {
+ buffer.position(beginOffset);
+ return RECORD_STATUS.TRUNCATED;
+ }
}
- // remote recovery logs need to have the LSN to check which should be replayed
- if (remoteRecoveryLog && nodeId.equals(localNodeId)) {
- LSN = buffer.getLong();
+ //remote recovery logs need to have the LSN to check which should be replayed
+ if (remoteRecoveryLog) {
+ if (buffer.remaining() >= Long.BYTES) {
+ LSN = buffer.getLong();
+ } else {
+ buffer.position(beginOffset);
+ return RECORD_STATUS.TRUNCATED;
+ }
}
+
+ return RECORD_STATUS.OK;
}
private ITupleReference readPKValue(ByteBuffer buffer) {
@@ -430,21 +415,19 @@
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);
builder.append(" JobId : ").append(jobId);
if (logType == LogType.ENTITY_COMMIT || logType == LogType.UPDATE) {
builder.append(" DatasetId : ").append(datasetId);
+ builder.append(" ResourcePartition : ").append(resourcePartition);
builder.append(" PKHashValue : ").append(PKHashValue);
builder.append(" PKFieldCnt : ").append(PKFieldCnt);
builder.append(" PKSize: ").append(PKValueSize);
@@ -460,11 +443,8 @@
public int writeRemoteRecoveryLog(ByteBuffer buffer) {
int bufferBegin = buffer.position();
writeLogRecordCommonFields(buffer);
- if (logType == LogType.FLUSH) {
- buffer.putLong(LSN);
- buffer.putInt(numOfFlushedIndexes);
- }
- // LSN must be included in all remote recovery logs (not only FLUSH)
+ //FLUSH logs should not included in remote recovery
+ //LSN must be included in all remote recovery logs
buffer.putLong(LSN);
return buffer.position() - bufferBegin;
}
@@ -560,21 +540,21 @@
@Override
public int getSerializedLogSize() {
- return getSerializedLogSize(logType, logSize);
- }
-
- private static int getSerializedLogSize(Byte logType, int logSize) {
+ int serilizedSize = logSize;
if (logType == LogType.FLUSH) {
- // LSN
- logSize += (Long.SIZE / 8);
- // num of indexes
- logSize += (Integer.SIZE / 8);
+ //LSN
+ serilizedSize += Long.BYTES;
+ //num of indexes
+ serilizedSize += Integer.BYTES;
+ //serialized node id String
+ serilizedSize += Integer.BYTES + nodeId.length();
}
-
- // checksum not included in serialized version
- logSize -= CHKSUM_LEN;
-
- return logSize;
+ if (logSource == LogSource.REMOTE_RECOVERY) {
+ //for LSN;
+ serilizedSize += Long.BYTES;
+ }
+ serilizedSize -= CHKSUM_LEN;
+ return serilizedSize;
}
@Override
@@ -675,7 +655,6 @@
@Override
public void setNodeId(String nodeId) {
this.nodeId = nodeId;
- this.nodeIdLength = nodeId.length();
}
public IReplicationThread getReplicationThread() {
@@ -713,4 +692,23 @@
this.opTracker = opTracker;
}
-}
+ @Override
+ public int getResourcePartition() {
+ return resourcePartition;
+ }
+
+ @Override
+ public void setResourcePartition(int resourcePartition) {
+ this.resourcePartition = resourcePartition;
+ }
+
+ @Override
+ public void setReplicated(boolean replicate) {
+ this.replicated = replicate;
+ }
+
+ @Override
+ public boolean isReplicated() {
+ return replicated;
+ }
+}
\ No newline at end of file
diff --git a/asterix-common/src/main/java/org/apache/asterix/common/utils/ServletUtil.java b/asterix-common/src/main/java/org/apache/asterix/common/utils/ServletUtil.java
new file mode 100644
index 0000000..b75d16c
--- /dev/null
+++ b/asterix-common/src/main/java/org/apache/asterix/common/utils/ServletUtil.java
@@ -0,0 +1,54 @@
+/*
+ * 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.utils;
+
+public class ServletUtil {
+
+ public enum Servlets {
+ AQL("/aql"),
+ AQL_QUERY("/query"),
+ AQL_UPDATE("/update"),
+ AQL_DDL("/ddl"),
+ SQLPP("/sqlpp"),
+ SQLPP_QUERY("/query/sqlpp"),
+ SQLPP_UPDATE("/update/sqlpp"),
+ SQLPP_DDL("/ddl/sqlpp"),
+ QUERY_STATUS("/query/status"),
+ QUERY_RESULT("/query/result"),
+ QUERY_SERVICE("/query/service"),
+ CONNECTOR("/connector"),
+ SHUTDOWN("/admin/shutdown"),
+ VERSION("/admin/version"),
+ CLUSTER_STATE("/admin/cluster");
+
+ private final String path;
+
+ private Servlets(String path) {
+ this.path = path;
+ }
+
+ public String getPath() {
+ return path;
+ }
+ }
+
+ private ServletUtil() {
+ throw new AssertionError("No objects of this class should be created.");
+ }
+}
diff --git a/asterix-common/src/main/java/org/apache/asterix/common/utils/StoragePathUtil.java b/asterix-common/src/main/java/org/apache/asterix/common/utils/StoragePathUtil.java
index 48e42bd..5b4035c 100644
--- a/asterix-common/src/main/java/org/apache/asterix/common/utils/StoragePathUtil.java
+++ b/asterix-common/src/main/java/org/apache/asterix/common/utils/StoragePathUtil.java
@@ -66,4 +66,8 @@
private static String prepareFullIndexName(String datasetName, String idxName) {
return (datasetName + DATASET_INDEX_NAME_SEPARATOR + idxName);
}
+
+ public static int getPartitonNumFromName(String name) {
+ return Integer.parseInt(name.substring(PARTITION_DIR_PREFIX.length()));
+ }
}
diff --git a/asterix-common/src/main/java/org/apache/asterix/common/utils/TransactionUtil.java b/asterix-common/src/main/java/org/apache/asterix/common/utils/TransactionUtil.java
index 97674e6..1d5b15e 100644
--- a/asterix-common/src/main/java/org/apache/asterix/common/utils/TransactionUtil.java
+++ b/asterix-common/src/main/java/org/apache/asterix/common/utils/TransactionUtil.java
@@ -28,61 +28,39 @@
public static void formJobTerminateLogRecord(ITransactionContext txnCtx, LogRecord logRecord, boolean isCommit) {
logRecord.setTxnCtx(txnCtx);
- TransactionUtil.formJobTerminateLogRecord(logRecord, txnCtx.getJobId().getId(), isCommit,
- logRecord.getNodeId());
+ TransactionUtil.formJobTerminateLogRecord(logRecord, txnCtx.getJobId().getId(), isCommit);
}
- public static void formJobTerminateLogRecord(LogRecord logRecord, int jobId, boolean isCommit, String nodeId) {
+ public static void formJobTerminateLogRecord(LogRecord logRecord, int jobId, boolean isCommit) {
logRecord.setLogType(isCommit ? LogType.JOB_COMMIT : LogType.ABORT);
logRecord.setDatasetId(-1);
logRecord.setPKHashValue(-1);
logRecord.setJobId(jobId);
- logRecord.setNodeId(nodeId);
logRecord.computeAndSetLogSize();
}
public static void formFlushLogRecord(LogRecord logRecord, int datasetId, PrimaryIndexOperationTracker opTracker,
- int numOfFlushedIndexes) {
- formFlushLogRecord(logRecord, datasetId, opTracker, null, numOfFlushedIndexes);
- }
-
- public static void formFlushLogRecord(LogRecord logRecord, int datasetId, PrimaryIndexOperationTracker opTracker,
String nodeId, int numberOfIndexes) {
logRecord.setLogType(LogType.FLUSH);
logRecord.setJobId(-1);
logRecord.setDatasetId(datasetId);
logRecord.setOpTracker(opTracker);
logRecord.setNumOfFlushedIndexes(numberOfIndexes);
- if (nodeId != null) {
- logRecord.setNodeId(nodeId);
- }
+ logRecord.setNodeId(nodeId);
logRecord.computeAndSetLogSize();
}
public static void formEntityCommitLogRecord(LogRecord logRecord, ITransactionContext txnCtx, int datasetId,
- int PKHashValue, ITupleReference PKValue, int[] PKFields) {
+ int PKHashValue, ITupleReference PKValue, int[] PKFields, int resourcePartition, byte entityCommitType) {
logRecord.setTxnCtx(txnCtx);
- logRecord.setLogType(LogType.ENTITY_COMMIT);
+ logRecord.setLogType(entityCommitType);
logRecord.setJobId(txnCtx.getJobId().getId());
logRecord.setDatasetId(datasetId);
logRecord.setPKHashValue(PKHashValue);
logRecord.setPKFieldCnt(PKFields.length);
logRecord.setPKValue(PKValue);
logRecord.setPKFields(PKFields);
- logRecord.computeAndSetPKValueSize();
- logRecord.computeAndSetLogSize();
- }
-
- public static void formEntityUpsertCommitLogRecord(LogRecord logRecord, ITransactionContext txnCtx, int datasetId,
- int PKHashValue, ITupleReference PKValue, int[] PKFields) {
- logRecord.setTxnCtx(txnCtx);
- logRecord.setLogType(LogType.UPSERT_ENTITY_COMMIT);
- logRecord.setJobId(txnCtx.getJobId().getId());
- logRecord.setDatasetId(datasetId);
- logRecord.setPKHashValue(PKHashValue);
- logRecord.setPKFieldCnt(PKFields.length);
- logRecord.setPKValue(PKValue);
- logRecord.setPKFields(PKFields);
+ logRecord.setResourcePartition(resourcePartition);
logRecord.computeAndSetPKValueSize();
logRecord.computeAndSetLogSize();
}
diff --git a/asterix-common/src/test/java/org/apache/asterix/test/aql/TestExecutor.java b/asterix-common/src/test/java/org/apache/asterix/test/aql/TestExecutor.java
index f54db63..29e08b5 100644
--- a/asterix-common/src/test/java/org/apache/asterix/test/aql/TestExecutor.java
+++ b/asterix-common/src/test/java/org/apache/asterix/test/aql/TestExecutor.java
@@ -24,7 +24,6 @@
import java.io.FileInputStream;
import java.io.FileOutputStream;
import java.io.FileReader;
-import java.io.IOException;
import java.io.InputStream;
import java.io.InputStreamReader;
import java.io.PrintWriter;
@@ -39,6 +38,7 @@
import java.util.logging.Logger;
import org.apache.asterix.common.config.GlobalConfig;
+import org.apache.asterix.common.utils.ServletUtil.Servlets;
import org.apache.asterix.testframework.context.TestCaseContext;
import org.apache.asterix.testframework.context.TestCaseContext.OutputFormat;
import org.apache.asterix.testframework.context.TestFileContext;
@@ -56,7 +56,6 @@
import org.apache.commons.httpclient.methods.StringRequestEntity;
import org.apache.commons.httpclient.params.HttpMethodParams;
import org.apache.commons.io.IOUtils;
-import org.apache.commons.lang3.StringUtils;
import org.json.JSONObject;
public class TestExecutor {
@@ -262,6 +261,17 @@
return method.getResponseBodyAsStream();
}
+ public InputStream executeClusterStateQuery(OutputFormat fmt, String url) throws Exception {
+ HttpMethodBase method = new GetMethod(url);
+
+ //Set accepted output response type
+ method.setRequestHeader("Accept", fmt.mimeType());
+ // Provide custom retry handler is necessary
+ method.getParams().setParameter(HttpMethodParams.RETRY_HANDLER, new DefaultHttpMethodRetryHandler(3, false));
+ executeHttpMethod(method);
+ return method.getResponseBodyAsStream();
+ }
+
// To execute Update statements
// Insert and Delete statements are executed here
public void executeUpdate(String str, String url) throws Exception {
@@ -301,7 +311,7 @@
}
private InputStream getHandleResult(String handle, OutputFormat fmt) throws Exception {
- final String url = "http://" + host + ":" + port + "/query/result";
+ final String url = "http://" + host + ":" + port + Servlets.QUERY_RESULT.getPath();
// Create a method instance.
GetMethod method = new GetMethod(url);
@@ -372,6 +382,14 @@
return IOUtils.toString(input, StandardCharsets.UTF_8.name());
}
+ private static String executeVagrantManagix(ProcessBuilder pb, String command) throws Exception {
+ pb.command("vagrant", "ssh", "cc", "--", pb.environment().get("MANAGIX_HOME") + command);
+ Process p = pb.start();
+ p.waitFor();
+ InputStream input = p.getInputStream();
+ return IOUtils.toString(input, StandardCharsets.UTF_8.name());
+ }
+
private static String getScriptPath(String queryPath, String scriptBasePath, String scriptFileName) {
String targetWord = "queries" + File.separator;
int targetWordSize = targetWord.lastIndexOf(File.separator);
@@ -439,9 +457,9 @@
switch (ctx.getType()) {
case "ddl":
if (ctx.getFile().getName().endsWith("aql")) {
- executeDDL(statement, "http://" + host + ":" + port + "/ddl");
+ executeDDL(statement, "http://" + host + ":" + port + Servlets.AQL_DDL.getPath());
} else {
- executeDDL(statement, "http://" + host + ":" + port + "/ddl/sqlpp");
+ executeDDL(statement, "http://" + host + ":" + port + Servlets.SQLPP_DDL.getPath());
}
break;
case "update":
@@ -451,9 +469,10 @@
"127.0.0.1://../../../../../../asterix-app/");
}
if (ctx.getFile().getName().endsWith("aql")) {
- executeUpdate(statement, "http://" + host + ":" + port + "/update");
+ executeUpdate(statement, "http://" + host + ":" + port + Servlets.AQL_UPDATE.getPath());
} else {
- executeUpdate(statement, "http://" + host + ":" + port + "/update/sqlpp");
+ executeUpdate(statement,
+ "http://" + host + ":" + port + Servlets.SQLPP_UPDATE.getPath());
}
break;
case "query":
@@ -471,24 +490,26 @@
if (ctx.getFile().getName().endsWith("aql")) {
if (ctx.getType().equalsIgnoreCase("query")) {
resultStream = executeQuery(statement, fmt,
- "http://" + host + ":" + port + "/query", cUnit.getParameter());
+ "http://" + host + ":" + port + Servlets.AQL_QUERY.getPath(),
+ cUnit.getParameter());
} else if (ctx.getType().equalsIgnoreCase("async")) {
resultStream = executeAnyAQLAsync(statement, false, fmt,
- "http://" + host + ":" + port + "/aql");
+ "http://" + host + ":" + port + Servlets.AQL.getPath());
} else if (ctx.getType().equalsIgnoreCase("asyncdefer")) {
resultStream = executeAnyAQLAsync(statement, true, fmt,
- "http://" + host + ":" + port + "/aql");
+ "http://" + host + ":" + port + Servlets.AQL.getPath());
}
} else {
if (ctx.getType().equalsIgnoreCase("query")) {
resultStream = executeQuery(statement, fmt,
- "http://" + host + ":" + port + "/query/sqlpp", cUnit.getParameter());
+ "http://" + host + ":" + port + Servlets.SQLPP_QUERY.getPath(),
+ cUnit.getParameter());
} else if (ctx.getType().equalsIgnoreCase("async")) {
resultStream = executeAnyAQLAsync(statement, false, fmt,
- "http://" + host + ":" + port + "/sqlpp");
+ "http://" + host + ":" + port + Servlets.SQLPP.getPath());
} else if (ctx.getType().equalsIgnoreCase("asyncdefer")) {
resultStream = executeAnyAQLAsync(statement, true, fmt,
- "http://" + host + ":" + port + "/sqlpp");
+ "http://" + host + ":" + port + Servlets.SQLPP.getPath());
}
}
@@ -505,9 +526,6 @@
runScriptAndCompareWithResult(testFile, new PrintWriter(System.err), expectedResultFile,
actualResultFile);
- LOGGER.info("[TEST]: " + testCaseCtx.getTestCase().getFilePath() + "/" + cUnit.getName()
- + " PASSED ");
-
queryCount++;
break;
case "mgx":
@@ -515,7 +533,7 @@
break;
case "txnqbc": //qbc represents query before crash
resultStream = executeQuery(statement, OutputFormat.forCompilationUnit(cUnit),
- "http://" + host + ":" + port + "/query", cUnit.getParameter());
+ "http://" + host + ":" + port + Servlets.AQL_QUERY.getPath(), cUnit.getParameter());
qbcFile = new File(actualPath + File.separator
+ testCaseCtx.getTestCase().getFilePath().replace(File.separator, "_") + "_"
+ cUnit.getName() + "_qbc.adm");
@@ -524,20 +542,17 @@
break;
case "txnqar": //qar represents query after recovery
resultStream = executeQuery(statement, OutputFormat.forCompilationUnit(cUnit),
- "http://" + host + ":" + port + "/query", cUnit.getParameter());
+ "http://" + host + ":" + port + Servlets.AQL_QUERY.getPath(), cUnit.getParameter());
qarFile = new File(actualPath + File.separator
+ testCaseCtx.getTestCase().getFilePath().replace(File.separator, "_") + "_"
+ cUnit.getName() + "_qar.adm");
qarFile.getParentFile().mkdirs();
writeOutputToFile(qarFile, resultStream);
runScriptAndCompareWithResult(testFile, new PrintWriter(System.err), qbcFile, qarFile);
-
- LOGGER.info("[TEST]: " + testCaseCtx.getTestCase().getFilePath() + "/" + cUnit.getName()
- + " PASSED ");
break;
case "txneu": //eu represents erroneous update
try {
- executeUpdate(statement, "http://" + host + ":" + port + "/update");
+ executeUpdate(statement, "http://" + host + ":" + port + Servlets.AQL_UPDATE.getPath());
} catch (Exception e) {
//An exception is expected.
failed = true;
@@ -565,7 +580,7 @@
break;
case "errddl": // a ddlquery that expects error
try {
- executeDDL(statement, "http://" + host + ":" + port + "/ddl");
+ executeDDL(statement, "http://" + host + ":" + port + Servlets.AQL_DDL.getPath());
} catch (Exception e) {
// expected error happens
failed = true;
@@ -576,7 +591,7 @@
}
System.err.println("...but that was expected.");
break;
- case "vagrant_script":
+ case "vscript": //a script that will be executed on a vagrant virtual node
try {
String[] command = statement.trim().split(" ");
if (command.length != 2) {
@@ -592,6 +607,32 @@
throw new Exception("Test \"" + testFile + "\" FAILED!\n", e);
}
break;
+ case "vmgx": //a managix command that will be executed on vagrant cc node
+ try {
+ String output = executeVagrantManagix(pb, statement);
+ if (output.contains("ERROR")) {
+ throw new Exception(output);
+ }
+ } catch (Exception e) {
+ throw new Exception("Test \"" + testFile + "\" FAILED!\n", e);
+ }
+ break;
+ case "cstate": //cluster state query
+ try {
+ fmt = OutputFormat.forCompilationUnit(cUnit);
+ resultStream = executeClusterStateQuery(fmt,
+ "http://" + host + ":" + port + Servlets.CLUSTER_STATE.getPath());
+ expectedResultFile = expectedResultFileCtxs.get(queryCount).getFile();
+ actualResultFile = testCaseCtx.getActualResultFile(cUnit, new File(actualPath));
+ actualResultFile.getParentFile().mkdirs();
+ writeOutputToFile(actualResultFile, resultStream);
+ runScriptAndCompareWithResult(testFile, new PrintWriter(System.err), expectedResultFile,
+ actualResultFile);
+ queryCount++;
+ } catch (Exception e) {
+ throw new Exception("Test \"" + testFile + "\" FAILED!\n", e);
+ }
+ break;
default:
throw new IllegalArgumentException("No statements of type " + ctx.getType());
}
@@ -626,6 +667,9 @@
"Test \"" + cUnit.getName() + "\" FAILED!\nExpected error was not thrown...");
e.printStackTrace();
throw e;
+ } else if (numOfFiles == testFileCtxs.size()) {
+ LOGGER.info("[TEST]: " + testCaseCtx.getTestCase().getFilePath() + "/" + cUnit.getName()
+ + " PASSED ");
}
}
}
diff --git a/asterix-installer/src/test/java/org/apache/asterix/installer/test/ReplicationIT.java b/asterix-installer/src/test/java/org/apache/asterix/installer/test/ReplicationIT.java
index 86c15ae..62ca9bf 100644
--- a/asterix-installer/src/test/java/org/apache/asterix/installer/test/ReplicationIT.java
+++ b/asterix-installer/src/test/java/org/apache/asterix/installer/test/ReplicationIT.java
@@ -58,7 +58,9 @@
private static ProcessBuilder pb;
private static Map<String, String> env;
private final static TestExecutor testExecutor = new TestExecutor(CLUSTER_CC_ADDRESS, CLUSTER_CC_API_PORT);
- private static String SCRIPT_HOME;
+ private static String SCRIPT_HOME = "/vagrant/scripts/";
+ private static String MANAGIX_HOME = "/tmp/asterix/bin/managix ";
+ private static final String INSTANCE_NAME = "asterix";
protected TestCaseContext tcCtx;
public ReplicationIT(TestCaseContext tcCtx) {
@@ -107,10 +109,10 @@
remoteInvoke("cp -r /vagrant/" + managixFolderName + " /tmp/asterix");
- SCRIPT_HOME = "/vagrant/scripts/";
pb = new ProcessBuilder();
env = pb.environment();
env.put("SCRIPT_HOME", SCRIPT_HOME);
+ env.put("MANAGIX_HOME", MANAGIX_HOME);
File cwd = new File(asterixProjectDir.toString() + "/" + CLUSTER_BASE);
pb.directory(cwd);
pb.redirectErrorStream(true);
@@ -141,13 +143,13 @@
@Before
public void beforeTest() throws Exception {
//create instance
- managixInvoke("create -n vagrant-ssh -c /vagrant/cluster_with_replication.xml").getInputStream();
+ managixInvoke("create -n " + INSTANCE_NAME + " -c /vagrant/cluster_with_replication.xml").getInputStream();
}
@After
public void afterTest() throws Exception {
//stop instance
- managixInvoke("stop -n vagrant-ssh");
+ managixInvoke("stop -n " + INSTANCE_NAME);
//verify that all processes have been stopped
String killProcesses = "kill_cc_and_nc.sh";
@@ -162,7 +164,7 @@
executeVagrantScript("nc2", deleteStorage);
//delete instance
- managixInvoke("delete -n vagrant-ssh");
+ managixInvoke("delete -n " + INSTANCE_NAME);
}
@Test
@@ -244,7 +246,7 @@
}
private static Process managixInvoke(String cmd) throws Exception {
- return remoteInvoke("/tmp/asterix/bin/managix " + cmd);
+ return remoteInvoke(MANAGIX_HOME + cmd);
}
private static String executeVagrantScript(String node, String scriptName) throws Exception {
diff --git a/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.1.ddl.aql b/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.1.ddl.aql
new file mode 100644
index 0000000..2c49a01
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.1.ddl.aql
@@ -0,0 +1,59 @@
+/*
+ * 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.
+ */
+/*
+ * Test case Name : node_failback.aql
+ * Description : Make sure node failback completes as expected.
+ The test goes as follows:
+ start 2 nodes, bulkload a dataset, copy it to in-memory dataset,
+ kill one node and wait until the failover complete, query cluster state,
+ query data, insert new data, start the killed node and wait for failback,
+ query cluster state, query data.
+ * Expected Result : Success
+ * Date : February 3 2016
+ */
+
+drop dataverse TinySocial if exists;
+create dataverse TinySocial;
+use dataverse TinySocial;
+
+create type EmploymentType as open {
+ organization-name: string,
+ start-date: date,
+ end-date: date?
+}
+
+create type FacebookUserType as closed {
+ id: int,
+ alias: string,
+ name: string,
+ user-since: datetime,
+ friend-ids: {{ int32 }},
+ employment: [EmploymentType]
+}
+
+/********* 2. Create Datasets ***********/
+use dataverse TinySocial;
+
+drop dataset FacebookUsers if exists;
+
+create dataset FacebookUsers(FacebookUserType)
+primary key id;
+
+create dataset FacebookUsersInMemory(FacebookUserType)
+primary key id;
\ No newline at end of file
diff --git a/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.10.cstate.aql b/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.10.cstate.aql
new file mode 100644
index 0000000..bd01d99
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.10.cstate.aql
@@ -0,0 +1,29 @@
+/*
+ * 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.
+ */
+/*
+ * Test case Name : node_failback.aql
+ * Description : Make sure node failback completes as expected.
+ The test goes as follows:
+ start 2 nodes, bulkload a dataset, copy it to in-memory dataset,
+ kill one node and wait until the failover complete, query cluster state,
+ query data, insert new data, start the killed node and wait for failback,
+ query cluster state, query data.
+ * Expected Result : Success
+ * Date : February 3 2016
+ */
\ No newline at end of file
diff --git a/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.11.query.aql b/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.11.query.aql
new file mode 100644
index 0000000..b09c3d3
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.11.query.aql
@@ -0,0 +1,33 @@
+/*
+ * 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.
+ */
+/*
+ * Test case Name : node_failback.aql
+ * Description : Make sure node failback completes as expected.
+ The test goes as follows:
+ start 2 nodes, bulkload a dataset, copy it to in-memory dataset,
+ kill one node and wait until the failover complete, query cluster state,
+ query data, insert new data, start the killed node and wait for failback,
+ query cluster state, query data.
+ * Expected Result : Success
+ * Date : February 3 2016
+ */
+
+use dataverse TinySocial;
+
+count (for $x in dataset FacebookUsersInMemory return $x);
\ No newline at end of file
diff --git a/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.2.update.aql b/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.2.update.aql
new file mode 100644
index 0000000..47f5975
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.2.update.aql
@@ -0,0 +1,35 @@
+/*
+ * 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.
+ */
+/*
+ * Test case Name : node_failback.aql
+ * Description : Make sure node failback completes as expected.
+ The test goes as follows:
+ start 2 nodes, bulkload a dataset, copy it to in-memory dataset,
+ kill one node and wait until the failover complete, query cluster state,
+ query data, insert new data, start the killed node and wait for failback,
+ query cluster state, query data.
+ * Expected Result : Success
+ * Date : February 3 2016
+ */
+use dataverse TinySocial;
+
+load dataset FacebookUsers using localfs
+(("path"="asterix_nc1:///vagrant/data/fbu.adm"),("format"="adm"));
+
+insert into dataset TinySocial.FacebookUsersInMemory(for $x in dataset TinySocial.FacebookUsers return $x);
\ No newline at end of file
diff --git a/asterix-installer/src/test/resources/integrationts/replication/queries/failover/metadata_node/metadata_node.3.vagrant_script.aql b/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.3.vscript.aql
similarity index 100%
copy from asterix-installer/src/test/resources/integrationts/replication/queries/failover/metadata_node/metadata_node.3.vagrant_script.aql
copy to asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.3.vscript.aql
diff --git a/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.4.sleep.aql b/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.4.sleep.aql
new file mode 100644
index 0000000..51b0e76
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.4.sleep.aql
@@ -0,0 +1 @@
+60000
\ No newline at end of file
diff --git a/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.5.cstate.aql b/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.5.cstate.aql
new file mode 100644
index 0000000..bd01d99
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.5.cstate.aql
@@ -0,0 +1,29 @@
+/*
+ * 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.
+ */
+/*
+ * Test case Name : node_failback.aql
+ * Description : Make sure node failback completes as expected.
+ The test goes as follows:
+ start 2 nodes, bulkload a dataset, copy it to in-memory dataset,
+ kill one node and wait until the failover complete, query cluster state,
+ query data, insert new data, start the killed node and wait for failback,
+ query cluster state, query data.
+ * Expected Result : Success
+ * Date : February 3 2016
+ */
\ No newline at end of file
diff --git a/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.6.query.aql b/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.6.query.aql
new file mode 100644
index 0000000..b09c3d3
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.6.query.aql
@@ -0,0 +1,33 @@
+/*
+ * 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.
+ */
+/*
+ * Test case Name : node_failback.aql
+ * Description : Make sure node failback completes as expected.
+ The test goes as follows:
+ start 2 nodes, bulkload a dataset, copy it to in-memory dataset,
+ kill one node and wait until the failover complete, query cluster state,
+ query data, insert new data, start the killed node and wait for failback,
+ query cluster state, query data.
+ * Expected Result : Success
+ * Date : February 3 2016
+ */
+
+use dataverse TinySocial;
+
+count (for $x in dataset FacebookUsersInMemory return $x);
\ No newline at end of file
diff --git a/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.7.update.aql b/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.7.update.aql
new file mode 100644
index 0000000..56a88a2
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.7.update.aql
@@ -0,0 +1,51 @@
+/*
+ * 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.
+ */
+/*
+ * Test case Name : node_failback.aql
+ * Description : Make sure node failback completes as expected.
+ The test goes as follows:
+ start 2 nodes, bulkload a dataset, copy it to in-memory dataset,
+ kill one node and wait until the failover complete, query cluster state,
+ query data, insert new data, start the killed node and wait for failback,
+ query cluster state, query data.
+ * Expected Result : Success
+ * Date : February 3 2016
+ */
+use dataverse TinySocial;
+
+/* insert ids 11-20 */
+insert into dataset TinySocial.FacebookUsersInMemory {"id":11,"alias":"Margarita","name":"MargaritaStoddard","user-since":datetime("2012-08-20T10:10:00"),"friend-ids":{{2,3,6,10}},"employment":[{"organization-name":"Codetechno","start-date":date("2006-08-06")}]};
+
+insert into dataset TinySocial.FacebookUsersInMemory {"id":12,"alias":"Margarita","name":"MargaritaStoddard","user-since":datetime("2012-08-20T10:10:00"),"friend-ids":{{2,3,6,10}},"employment":[{"organization-name":"Codetechno","start-date":date("2006-08-06")}]};
+
+insert into dataset TinySocial.FacebookUsersInMemory {"id":13,"alias":"Margarita","name":"MargaritaStoddard","user-since":datetime("2012-08-20T10:10:00"),"friend-ids":{{2,3,6,10}},"employment":[{"organization-name":"Codetechno","start-date":date("2006-08-06")}]};
+
+insert into dataset TinySocial.FacebookUsersInMemory {"id":14,"alias":"Margarita","name":"MargaritaStoddard","user-since":datetime("2012-08-20T10:10:00"),"friend-ids":{{2,3,6,10}},"employment":[{"organization-name":"Codetechno","start-date":date("2006-08-06")}]};
+
+insert into dataset TinySocial.FacebookUsersInMemory {"id":15,"alias":"Margarita","name":"MargaritaStoddard","user-since":datetime("2012-08-20T10:10:00"),"friend-ids":{{2,3,6,10}},"employment":[{"organization-name":"Codetechno","start-date":date("2006-08-06")}]};
+
+insert into dataset TinySocial.FacebookUsersInMemory {"id":16,"alias":"Margarita","name":"MargaritaStoddard","user-since":datetime("2012-08-20T10:10:00"),"friend-ids":{{2,3,6,10}},"employment":[{"organization-name":"Codetechno","start-date":date("2006-08-06")}]};
+
+insert into dataset TinySocial.FacebookUsersInMemory {"id":17,"alias":"Margarita","name":"MargaritaStoddard","user-since":datetime("2012-08-20T10:10:00"),"friend-ids":{{2,3,6,10}},"employment":[{"organization-name":"Codetechno","start-date":date("2006-08-06")}]};
+
+insert into dataset TinySocial.FacebookUsersInMemory {"id":18,"alias":"Margarita","name":"MargaritaStoddard","user-since":datetime("2012-08-20T10:10:00"),"friend-ids":{{2,3,6,10}},"employment":[{"organization-name":"Codetechno","start-date":date("2006-08-06")}]};
+
+insert into dataset TinySocial.FacebookUsersInMemory {"id":19,"alias":"Margarita","name":"MargaritaStoddard","user-since":datetime("2012-08-20T10:10:00"),"friend-ids":{{2,3,6,10}},"employment":[{"organization-name":"Codetechno","start-date":date("2006-08-06")}]};
+
+insert into dataset TinySocial.FacebookUsersInMemory {"id":20,"alias":"Margarita","name":"MargaritaStoddard","user-since":datetime("2012-08-20T10:10:00"),"friend-ids":{{2,3,6,10}},"employment":[{"organization-name":"Codetechno","start-date":date("2006-08-06")}]};
\ No newline at end of file
diff --git a/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.8.vmgx.aql b/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.8.vmgx.aql
new file mode 100644
index 0000000..67b492c
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.8.vmgx.aql
@@ -0,0 +1 @@
+startnode -n asterix -nodes nc1
\ No newline at end of file
diff --git a/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.9.sleep.aql b/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.9.sleep.aql
new file mode 100644
index 0000000..1746da6
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.9.sleep.aql
@@ -0,0 +1 @@
+10000
\ No newline at end of file
diff --git a/asterix-installer/src/test/resources/integrationts/replication/queries/failover/bulkload/bulkload.2.update.aql b/asterix-installer/src/test/resources/integrationts/replication/queries/failover/bulkload/bulkload.2.update.aql
index ae14ad0..94ecc27 100644
--- a/asterix-installer/src/test/resources/integrationts/replication/queries/failover/bulkload/bulkload.2.update.aql
+++ b/asterix-installer/src/test/resources/integrationts/replication/queries/failover/bulkload/bulkload.2.update.aql
@@ -29,4 +29,4 @@
use dataverse TinySocial;
load dataset FacebookUsers using localfs
-(("path"="vagrant-ssh_nc1:///vagrant/data/fbu.adm"),("format"="adm"));
\ No newline at end of file
+(("path"="asterix_nc1:///vagrant/data/fbu.adm"),("format"="adm"));
\ No newline at end of file
diff --git a/asterix-installer/src/test/resources/integrationts/replication/queries/failover/bulkload/bulkload.4.vagrant_script.aql b/asterix-installer/src/test/resources/integrationts/replication/queries/failover/bulkload/bulkload.4.vscript.aql
similarity index 100%
rename from asterix-installer/src/test/resources/integrationts/replication/queries/failover/bulkload/bulkload.4.vagrant_script.aql
rename to asterix-installer/src/test/resources/integrationts/replication/queries/failover/bulkload/bulkload.4.vscript.aql
diff --git a/asterix-installer/src/test/resources/integrationts/replication/queries/failover/mem_component_recovery/mem_component_recovery.2.update.aql b/asterix-installer/src/test/resources/integrationts/replication/queries/failover/mem_component_recovery/mem_component_recovery.2.update.aql
index 8087689..d97f786 100644
--- a/asterix-installer/src/test/resources/integrationts/replication/queries/failover/mem_component_recovery/mem_component_recovery.2.update.aql
+++ b/asterix-installer/src/test/resources/integrationts/replication/queries/failover/mem_component_recovery/mem_component_recovery.2.update.aql
@@ -29,6 +29,6 @@
use dataverse TinySocial;
load dataset FacebookUsers using localfs
-(("path"="vagrant-ssh_nc1:///vagrant/data/fbu.adm"),("format"="adm"));
+(("path"="asterix_nc1:///vagrant/data/fbu.adm"),("format"="adm"));
insert into dataset TinySocial.FacebookUsersInMemory(for $x in dataset TinySocial.FacebookUsers return $x);
\ No newline at end of file
diff --git a/asterix-installer/src/test/resources/integrationts/replication/queries/failover/mem_component_recovery/mem_component_recovery.4.vagrant_script.aql b/asterix-installer/src/test/resources/integrationts/replication/queries/failover/mem_component_recovery/mem_component_recovery.4.vscript.aql
similarity index 100%
rename from asterix-installer/src/test/resources/integrationts/replication/queries/failover/mem_component_recovery/mem_component_recovery.4.vagrant_script.aql
rename to asterix-installer/src/test/resources/integrationts/replication/queries/failover/mem_component_recovery/mem_component_recovery.4.vscript.aql
diff --git a/asterix-installer/src/test/resources/integrationts/replication/queries/failover/metadata_node/metadata_node.3.vagrant_script.aql b/asterix-installer/src/test/resources/integrationts/replication/queries/failover/metadata_node/metadata_node.3.vscript.aql
similarity index 100%
rename from asterix-installer/src/test/resources/integrationts/replication/queries/failover/metadata_node/metadata_node.3.vagrant_script.aql
rename to asterix-installer/src/test/resources/integrationts/replication/queries/failover/metadata_node/metadata_node.3.vscript.aql
diff --git a/asterix-installer/src/test/resources/integrationts/replication/results/failback/node_failback/node_failback.cluster_state.10.adm b/asterix-installer/src/test/resources/integrationts/replication/results/failback/node_failback/node_failback.cluster_state.10.adm
new file mode 100644
index 0000000..61322c9
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/replication/results/failback/node_failback/node_failback.cluster_state.10.adm
@@ -0,0 +1 @@
+{"State":"ACTIVE","Metadata_Node":"asterix_nc1","partition_0":"asterix_nc1","partition_1":"asterix_nc1","partition_2":"asterix_nc2","partition_3":"asterix_nc2"}
\ No newline at end of file
diff --git a/asterix-installer/src/test/resources/integrationts/replication/results/failback/node_failback/node_failback.cluster_state.5.adm b/asterix-installer/src/test/resources/integrationts/replication/results/failback/node_failback/node_failback.cluster_state.5.adm
new file mode 100644
index 0000000..587a97a
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/replication/results/failback/node_failback/node_failback.cluster_state.5.adm
@@ -0,0 +1 @@
+{"State":"ACTIVE","Metadata_Node":"asterix_nc2","partition_0":"asterix_nc2","partition_1":"asterix_nc2","partition_2":"asterix_nc2","partition_3":"asterix_nc2"}
\ No newline at end of file
diff --git a/asterix-installer/src/test/resources/integrationts/replication/results/failback/node_failback/node_failback.query.11.adm b/asterix-installer/src/test/resources/integrationts/replication/results/failback/node_failback/node_failback.query.11.adm
new file mode 100644
index 0000000..2edeafb
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/replication/results/failback/node_failback/node_failback.query.11.adm
@@ -0,0 +1 @@
+20
\ No newline at end of file
diff --git a/asterix-installer/src/test/resources/integrationts/replication/results/failback/node_failback/node_failback.query.6.adm b/asterix-installer/src/test/resources/integrationts/replication/results/failback/node_failback/node_failback.query.6.adm
new file mode 100644
index 0000000..9a03714
--- /dev/null
+++ b/asterix-installer/src/test/resources/integrationts/replication/results/failback/node_failback/node_failback.query.6.adm
@@ -0,0 +1 @@
+10
\ No newline at end of file
diff --git a/asterix-installer/src/test/resources/integrationts/replication/testsuite.xml b/asterix-installer/src/test/resources/integrationts/replication/testsuite.xml
index f033086..36c3992 100644
--- a/asterix-installer/src/test/resources/integrationts/replication/testsuite.xml
+++ b/asterix-installer/src/test/resources/integrationts/replication/testsuite.xml
@@ -17,21 +17,28 @@
! under the License.
!-->
<test-suite xmlns="urn:xml.testframework.asterix.apache.org" ResultOffsetPath="results" QueryOffsetPath="queries" QueryFileExtension=".aql">
- <test-group name="failover">
- <test-case FilePath="failover">
- <compilation-unit name="bulkload">
- <output-dir compare="Text">bulkload</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="failover">
- <compilation-unit name="mem_component_recovery">
- <output-dir compare="Text">mem_component_recovery</output-dir>
- </compilation-unit>
- </test-case>
- <test-case FilePath="failover">
- <compilation-unit name="metadata_node">
- <output-dir compare="Text">metadata_node</output-dir>
- </compilation-unit>
- </test-case>
- </test-group>
+ <test-group name="failover">
+ <test-case FilePath="failover">
+ <compilation-unit name="bulkload">
+ <output-dir compare="Text">bulkload</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="failover">
+ <compilation-unit name="mem_component_recovery">
+ <output-dir compare="Text">mem_component_recovery</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="failover">
+ <compilation-unit name="metadata_node">
+ <output-dir compare="Text">metadata_node</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="failback">
+ <test-case FilePath="failback">
+ <compilation-unit name="node_failback">
+ <output-dir compare="Text">node_failback</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
</test-suite>
\ No newline at end of file
diff --git a/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataNode.java b/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataNode.java
index 4e6a3df..088a85b 100644
--- a/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataNode.java
+++ b/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataNode.java
@@ -28,6 +28,7 @@
import org.apache.asterix.common.api.IDatasetLifecycleManager;
import org.apache.asterix.common.config.DatasetConfig.DatasetType;
import org.apache.asterix.common.config.DatasetConfig.IndexType;
+import org.apache.asterix.common.config.IAsterixPropertiesProvider;
import org.apache.asterix.common.dataflow.AsterixLSMIndexUtil;
import org.apache.asterix.common.exceptions.ACIDException;
import org.apache.asterix.common.functions.FunctionSignature;
@@ -113,6 +114,7 @@
private IDatasetLifecycleManager datasetLifecycleManager;
private ITransactionSubsystem transactionSubsystem;
+ private int metadataStoragePartition;
public static final MetadataNode INSTANCE = new MetadataNode();
@@ -123,6 +125,8 @@
public void initialize(IAsterixAppRuntimeContext runtimeContext) {
this.transactionSubsystem = runtimeContext.getTransactionSubsystem();
this.datasetLifecycleManager = runtimeContext.getDatasetLifecycleManager();
+ this.metadataStoragePartition = ((IAsterixPropertiesProvider) runtimeContext).getMetadataProperties()
+ .getMetadataPartition().getPartitionId();
}
@Override
@@ -305,11 +309,11 @@
if (metadataIndex.isPrimaryIndex()) {
return new PrimaryIndexModificationOperationCallback(metadataIndex.getDatasetId().getId(),
metadataIndex.getPrimaryKeyIndexes(), txnCtx, transactionSubsystem.getLockManager(),
- transactionSubsystem, resourceId, ResourceType.LSM_BTREE, indexOp);
+ transactionSubsystem, resourceId, metadataStoragePartition, ResourceType.LSM_BTREE, indexOp);
} else {
return new SecondaryIndexModificationOperationCallback(metadataIndex.getDatasetId().getId(),
metadataIndex.getPrimaryKeyIndexes(), txnCtx, transactionSubsystem.getLockManager(),
- transactionSubsystem, resourceId, ResourceType.LSM_BTREE, indexOp);
+ transactionSubsystem, resourceId, metadataStoragePartition, ResourceType.LSM_BTREE, indexOp);
}
}
@@ -641,8 +645,7 @@
}
}
- private List<Datatype> getDataverseDatatypes(JobId jobId, String dataverseName)
- throws MetadataException, RemoteException {
+ private List<Datatype> getDataverseDatatypes(JobId jobId, String dataverseName) throws MetadataException {
try {
ITupleReference searchKey = createTuple(dataverseName);
DatatypeTupleTranslator tupleReaderWriter = new DatatypeTupleTranslator(jobId, this, false);
@@ -673,7 +676,7 @@
}
}
- public List<Dataset> getAllDatasets(JobId jobId) throws MetadataException, RemoteException {
+ public List<Dataset> getAllDatasets(JobId jobId) throws MetadataException {
try {
ITupleReference searchKey = null;
DatasetTupleTranslator tupleReaderWriter = new DatasetTupleTranslator(false);
@@ -686,7 +689,7 @@
}
}
- public List<Datatype> getAllDatatypes(JobId jobId) throws MetadataException, RemoteException {
+ public List<Datatype> getAllDatatypes(JobId jobId) throws MetadataException {
try {
ITupleReference searchKey = null;
DatatypeTupleTranslator tupleReaderWriter = new DatatypeTupleTranslator(jobId, this, false);
@@ -699,8 +702,7 @@
}
}
- private void confirmDataverseCanBeDeleted(JobId jobId, String dataverseName)
- throws MetadataException, RemoteException {
+ private void confirmDataverseCanBeDeleted(JobId jobId, String dataverseName) throws MetadataException {
//If a dataset from a DIFFERENT dataverse
//uses a type from this dataverse
//throw an error
@@ -717,13 +719,13 @@
}
private void confirmDatatypeIsUnused(JobId jobId, String dataverseName, String datatypeName)
- throws MetadataException, RemoteException {
+ throws MetadataException {
confirmDatatypeIsUnusedByDatatypes(jobId, dataverseName, datatypeName);
confirmDatatypeIsUnusedByDatasets(jobId, dataverseName, datatypeName);
}
private void confirmDatatypeIsUnusedByDatasets(JobId jobId, String dataverseName, String datatypeName)
- throws MetadataException, RemoteException {
+ throws MetadataException {
//If any dataset uses this type, throw an error
List<Dataset> datasets = getAllDatasets(jobId);
for (Dataset set : datasets) {
@@ -735,7 +737,7 @@
}
private void confirmDatatypeIsUnusedByDatatypes(JobId jobId, String dataverseName, String datatypeName)
- throws MetadataException, RemoteException {
+ throws MetadataException {
//If any datatype uses this type, throw an error
//TODO: Currently this loads all types into memory. This will need to be fixed for large numbers of types
List<Datatype> datatypes = getAllDatatypes(jobId);
@@ -768,7 +770,7 @@
}
public List<String> getDatasetNamesPartitionedOnThisNodeGroup(JobId jobId, String nodegroup)
- throws MetadataException, RemoteException {
+ throws MetadataException {
//this needs to scan the datasets and return the datasets that use this nodegroup
List<String> nodeGroupDatasets = new ArrayList<String>();
List<Dataset> datasets = getAllDatasets(jobId);
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 2744630..f79385c 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
@@ -22,6 +22,8 @@
import java.util.ArrayList;
import java.util.HashMap;
import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
import java.util.List;
import java.util.Map;
import java.util.Set;
@@ -36,14 +38,24 @@
import org.apache.asterix.common.api.IClusterManagementWork.ClusterState;
import org.apache.asterix.common.cluster.ClusterPartition;
import org.apache.asterix.common.config.AsterixReplicationProperties;
+import org.apache.asterix.common.messaging.CompleteFailbackRequestMessage;
+import org.apache.asterix.common.messaging.CompleteFailbackResponseMessage;
+import org.apache.asterix.common.messaging.PreparePartitionsFailbackRequestMessage;
+import org.apache.asterix.common.messaging.PreparePartitionsFailbackResponseMessage;
+import org.apache.asterix.common.messaging.ReplicaEventMessage;
import org.apache.asterix.common.messaging.TakeoverMetadataNodeRequestMessage;
import org.apache.asterix.common.messaging.TakeoverMetadataNodeResponseMessage;
import org.apache.asterix.common.messaging.TakeoverPartitionsRequestMessage;
import org.apache.asterix.common.messaging.TakeoverPartitionsResponseMessage;
import org.apache.asterix.common.messaging.api.ICCMessageBroker;
+import org.apache.asterix.common.replication.NodeFailbackPlan;
+import org.apache.asterix.common.replication.NodeFailbackPlan.FailbackPlanState;
import org.apache.asterix.event.schema.cluster.Cluster;
import org.apache.asterix.event.schema.cluster.Node;
import org.apache.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
+import org.apache.hyracks.api.application.IClusterLifecycleListener.ClusterEventType;
+import org.json.JSONException;
+import org.json.JSONObject;
/**
* A holder class for properties related to the Asterix cluster.
@@ -57,15 +69,16 @@
*/
private static final Logger LOGGER = Logger.getLogger(AsterixClusterProperties.class.getName());
-
public static final AsterixClusterProperties INSTANCE = new AsterixClusterProperties();
public static final String CLUSTER_CONFIGURATION_FILE = "cluster.xml";
+ private static final String CLUSTER_NET_IP_ADDRESS_KEY = "cluster-net-ip-address";
private static final String IO_DEVICES = "iodevices";
private static final String DEFAULT_STORAGE_DIR_NAME = "storage";
- private Map<String, Map<String, String>> ncConfiguration = new HashMap<String, Map<String, String>>();
+ private Map<String, Map<String, String>> activeNcConfiguration = new HashMap<String, Map<String, String>>();
private final Cluster cluster;
+ private ClusterState state = ClusterState.UNUSABLE;
private AlgebricksAbsolutePartitionConstraint clusterPartitionConstraint;
@@ -75,10 +88,14 @@
private SortedMap<Integer, ClusterPartition> clusterPartitions = null;
private Map<Long, TakeoverPartitionsRequestMessage> pendingTakeoverRequests = null;
- private long takeoverRequestId = 0;
+ private long clusterRequestId = 0;
private String currentMetadataNode = null;
- private boolean isMetadataNodeActive = false;
+ private boolean metadataNodeActive = false;
private boolean autoFailover = false;
+ private boolean replicationEnabled = false;
+ private Set<String> failedNodes = new HashSet<>();
+ private LinkedList<NodeFailbackPlan> pendingProcessingFailbackPlans;
+ private Map<Long, NodeFailbackPlan> planId2FailbackPlanMap;
private AsterixClusterProperties() {
InputStream is = this.getClass().getClassLoader().getResourceAsStream(CLUSTER_CONFIGURATION_FILE);
@@ -99,43 +116,73 @@
node2PartitionsMap = AsterixAppContextInfo.getInstance().getMetadataProperties().getNodePartitions();
clusterPartitions = AsterixAppContextInfo.getInstance().getMetadataProperties().getClusterPartitions();
currentMetadataNode = AsterixAppContextInfo.getInstance().getMetadataProperties().getMetadataNodeName();
- if (isAutoFailoverEnabled()) {
- autoFailover = cluster.getDataReplication().isAutoFailover();
- }
+ replicationEnabled = isReplicationEnabled();
+ autoFailover = isAutoFailoverEnabled();
if (autoFailover) {
pendingTakeoverRequests = new HashMap<>();
+ pendingProcessingFailbackPlans = new LinkedList<>();
+ planId2FailbackPlanMap = new HashMap<>();
}
}
}
}
- private ClusterState state = ClusterState.UNUSABLE;
-
public synchronized void removeNCConfiguration(String nodeId) {
- updateNodePartitions(nodeId, false);
- ncConfiguration.remove(nodeId);
- if (nodeId.equals(currentMetadataNode)) {
- isMetadataNodeActive = false;
- LOGGER.info("Metadata node is now inactive");
- }
if (LOGGER.isLoggable(Level.INFO)) {
LOGGER.info("Removing configuration parameters for node id " + nodeId);
}
- if (autoFailover) {
- requestPartitionsTakeover(nodeId);
+ activeNcConfiguration.remove(nodeId);
+
+ //if this node was waiting for failback and failed before it completed
+ if (failedNodes.contains(nodeId)) {
+ if (autoFailover) {
+ notifyFailbackPlansNodeFailure(nodeId);
+ revertFailedFailbackPlanEffects();
+ }
+ } else {
+ //an active node failed
+ failedNodes.add(nodeId);
+ if (nodeId.equals(currentMetadataNode)) {
+ metadataNodeActive = false;
+ LOGGER.info("Metadata node is now inactive");
+ }
+ updateNodePartitions(nodeId, false);
+ if (replicationEnabled) {
+ notifyImpactedReplicas(nodeId, ClusterEventType.NODE_FAILURE);
+ if (autoFailover) {
+ notifyFailbackPlansNodeFailure(nodeId);
+ requestPartitionsTakeover(nodeId);
+ }
+ }
}
}
public synchronized void addNCConfiguration(String nodeId, Map<String, String> configuration) {
- ncConfiguration.put(nodeId, configuration);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Registering configuration parameters for node id " + nodeId);
+ }
+ activeNcConfiguration.put(nodeId, configuration);
+
+ //a node trying to come back after failure
+ if (failedNodes.contains(nodeId)) {
+ if (autoFailover) {
+ prepareFailbackPlan(nodeId);
+ return;
+ } else {
+ //a node completed local or remote recovery and rejoined
+ failedNodes.remove(nodeId);
+ if (replicationEnabled) {
+ //notify other replica to reconnect to this node
+ notifyImpactedReplicas(nodeId, ClusterEventType.NODE_JOIN);
+ }
+ }
+ }
+
if (nodeId.equals(currentMetadataNode)) {
- isMetadataNodeActive = true;
+ metadataNodeActive = true;
LOGGER.info("Metadata node is now active");
}
updateNodePartitions(nodeId, true);
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info(" Registering configuration parameters for node id " + nodeId);
- }
}
private synchronized void updateNodePartitions(String nodeId, boolean added) {
@@ -163,11 +210,17 @@
}
}
//if all storage partitions are active as well as the metadata node, then the cluster is active
- if (isMetadataNodeActive) {
+ if (metadataNodeActive) {
state = ClusterState.ACTIVE;
LOGGER.info("Cluster is now ACTIVE");
//start global recovery
AsterixAppContextInfo.getInstance().getGlobalRecoveryManager().startGlobalRecovery();
+ if (autoFailover) {
+ //if there are any pending failback requests, process them
+ if (pendingProcessingFailbackPlans.size() > 0) {
+ processPendingFailbackPlans();
+ }
+ }
} else {
requestMetadataNodeTakeover();
}
@@ -196,7 +249,7 @@
* if it does not correspond to the set of registered Node Controllers.
*/
public synchronized String[] getIODevices(String nodeId) {
- Map<String, String> ncConfig = ncConfiguration.get(nodeId);
+ Map<String, String> ncConfig = activeNcConfiguration.get(nodeId);
if (ncConfig == null) {
if (LOGGER.isLoggable(Level.WARNING)) {
LOGGER.warning("Configuration parameters for nodeId " + nodeId
@@ -222,7 +275,7 @@
public synchronized Set<String> getParticipantNodes() {
Set<String> participantNodes = new HashSet<String>();
- for (String pNode : ncConfiguration.keySet()) {
+ for (String pNode : activeNcConfiguration.keySet()) {
participantNodes.add(pNode);
}
return participantNodes;
@@ -254,12 +307,12 @@
this.globalRecoveryCompleted = globalRecoveryCompleted;
}
- public static boolean isClusterActive() {
- if (AsterixClusterProperties.INSTANCE.getCluster() == null) {
+ public boolean isClusterActive() {
+ if (cluster == null) {
// this is a virtual cluster
return true;
}
- return AsterixClusterProperties.INSTANCE.getState() == ClusterState.ACTIVE;
+ return state == ClusterState.ACTIVE;
}
public static int getNumberOfNodes() {
@@ -279,8 +332,8 @@
public synchronized ClusterPartition[] getClusterPartitons() {
ArrayList<ClusterPartition> partitons = new ArrayList<>();
- for (ClusterPartition cluster : clusterPartitions.values()) {
- partitons.add(cluster);
+ for (ClusterPartition partition : clusterPartitions.values()) {
+ partitons.add(partition);
}
return partitons.toArray(new ClusterPartition[] {});
}
@@ -301,44 +354,53 @@
//collect the partitions of the failed NC
List<ClusterPartition> lostPartitions = getNodeAssignedPartitions(failedNodeId);
- for (ClusterPartition partition : lostPartitions) {
- //find replicas for this partitions
- Set<String> partitionReplicas = replicationProperties.getNodeReplicasIds(partition.getNodeId());
- //find a replica that is still active
- for (String replica : partitionReplicas) {
- //TODO (mhubail) currently this assigns the partition to the first found active replica.
- //It needs to be modified to consider load balancing.
- if (ncConfiguration.containsKey(replica)) {
- if (!partitionRecoveryPlan.containsKey(replica)) {
- List<Integer> replicaPartitions = new ArrayList<>();
- replicaPartitions.add(partition.getPartitionId());
- partitionRecoveryPlan.put(replica, replicaPartitions);
- } else {
- partitionRecoveryPlan.get(replica).add(partition.getPartitionId());
+ if (lostPartitions.size() > 0) {
+ for (ClusterPartition partition : lostPartitions) {
+ //find replicas for this partitions
+ Set<String> partitionReplicas = replicationProperties.getNodeReplicasIds(partition.getNodeId());
+ //find a replica that is still active
+ for (String replica : partitionReplicas) {
+ //TODO (mhubail) currently this assigns the partition to the first found active replica.
+ //It needs to be modified to consider load balancing.
+ if (activeNcConfiguration.containsKey(replica) && !failedNodes.contains(replica)) {
+ if (!partitionRecoveryPlan.containsKey(replica)) {
+ List<Integer> replicaPartitions = new ArrayList<>();
+ replicaPartitions.add(partition.getPartitionId());
+ partitionRecoveryPlan.put(replica, replicaPartitions);
+ } else {
+ partitionRecoveryPlan.get(replica).add(partition.getPartitionId());
+ }
}
}
}
- }
- ICCMessageBroker messageBroker = (ICCMessageBroker) AsterixAppContextInfo.getInstance()
- .getCCApplicationContext().getMessageBroker();
- //For each replica, send a request to takeover the assigned partitions
- for (String replica : partitionRecoveryPlan.keySet()) {
- Integer[] partitionsToTakeover = partitionRecoveryPlan.get(replica).toArray(new Integer[] {});
- long requestId = takeoverRequestId++;
- TakeoverPartitionsRequestMessage takeoverRequest = new TakeoverPartitionsRequestMessage(requestId, replica,
- failedNodeId, partitionsToTakeover);
- pendingTakeoverRequests.put(requestId, takeoverRequest);
- try {
- messageBroker.sendApplicationMessageToNC(takeoverRequest, replica);
- } catch (Exception e) {
- /**
- * if we fail to send the request, it means the NC we tried to send the request to
- * has failed. When the failure notification arrives, we will send any pending request
- * that belongs to the failed NC to a different active replica.
- */
- LOGGER.warning("Failed to send takeover request: " + takeoverRequest);
- e.printStackTrace();
+ if (partitionRecoveryPlan.size() == 0) {
+ //no active replicas were found for the failed node
+ LOGGER.severe("Could not find active replicas for the partitions " + lostPartitions);
+ return;
+ } else {
+ LOGGER.info("Partitions to recover: " + lostPartitions);
+ }
+ ICCMessageBroker messageBroker = (ICCMessageBroker) AsterixAppContextInfo.getInstance()
+ .getCCApplicationContext().getMessageBroker();
+ //For each replica, send a request to takeover the assigned partitions
+ for (String replica : partitionRecoveryPlan.keySet()) {
+ Integer[] partitionsToTakeover = partitionRecoveryPlan.get(replica).toArray(new Integer[] {});
+ long requestId = clusterRequestId++;
+ TakeoverPartitionsRequestMessage takeoverRequest = new TakeoverPartitionsRequestMessage(requestId,
+ replica, partitionsToTakeover);
+ pendingTakeoverRequests.put(requestId, takeoverRequest);
+ try {
+ messageBroker.sendApplicationMessageToNC(takeoverRequest, replica);
+ } catch (Exception e) {
+ /**
+ * if we fail to send the request, it means the NC we tried to send the request to
+ * has failed. When the failure notification arrives, we will send any pending request
+ * that belongs to the failed NC to a different active replica.
+ */
+ LOGGER.warning("Failed to send takeover request: " + takeoverRequest);
+ e.printStackTrace();
+ }
}
}
}
@@ -368,7 +430,6 @@
for (Long requestId : failedTakeoverRequests) {
pendingTakeoverRequests.remove(requestId);
}
-
return nodePartitions;
}
@@ -406,19 +467,223 @@
public synchronized void processMetadataNodeTakeoverResponse(TakeoverMetadataNodeResponseMessage reponse) {
currentMetadataNode = reponse.getNodeId();
- isMetadataNodeActive = true;
+ metadataNodeActive = true;
LOGGER.info("Current metadata node: " + currentMetadataNode);
updateClusterState();
}
- public synchronized String getCurrentMetadataNode() {
- return currentMetadataNode;
+ private synchronized void prepareFailbackPlan(String failingBackNodeId) {
+ NodeFailbackPlan plan = NodeFailbackPlan.createPlan(failingBackNodeId);
+ pendingProcessingFailbackPlans.add(plan);
+ planId2FailbackPlanMap.put(plan.getPlanId(), plan);
+
+ //get all partitions this node requires to resync
+ AsterixReplicationProperties replicationProperties = AsterixAppContextInfo.getInstance()
+ .getReplicationProperties();
+ Set<String> nodeReplicas = replicationProperties.getNodeReplicationClients(failingBackNodeId);
+ for (String replicaId : nodeReplicas) {
+ ClusterPartition[] nodePartitions = node2PartitionsMap.get(replicaId);
+ for (ClusterPartition partition : nodePartitions) {
+ plan.addParticipant(partition.getActiveNodeId());
+ /**
+ * if the partition original node is the returning node,
+ * add it to the list of the partitions which will be failed back
+ */
+ if (partition.getNodeId().equals(failingBackNodeId)) {
+ plan.addPartitionToFailback(partition.getPartitionId(), partition.getActiveNodeId());
+ }
+ }
+ }
+
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Prepared Failback plan: " + plan.toString());
+ }
+
+ processPendingFailbackPlans();
+ }
+
+ private synchronized void processPendingFailbackPlans() {
+ /**
+ * if the cluster state is not ACTIVE, then failbacks should not be processed
+ * since some partitions are not active
+ */
+ if (state == ClusterState.ACTIVE) {
+ while (!pendingProcessingFailbackPlans.isEmpty()) {
+ //take the first pending failback plan
+ NodeFailbackPlan plan = pendingProcessingFailbackPlans.pop();
+ /**
+ * A plan at this stage will be in one of two states:
+ * 1. PREPARING -> the participants were selected but we haven't sent any request.
+ * 2. PENDING_ROLLBACK -> a participant failed before we send any requests
+ */
+ if (plan.getState() == FailbackPlanState.PREPARING) {
+ //set the partitions that will be failed back as inactive
+ String failbackNode = plan.getNodeId();
+ for (Integer partitionId : plan.getPartitionsToFailback()) {
+ ClusterPartition clusterPartition = clusterPartitions.get(partitionId);
+ clusterPartition.setActive(false);
+ //partition expected to be returned to the failing back node
+ clusterPartition.setActiveNodeId(failbackNode);
+ }
+
+ /**
+ * if the returning node is the original metadata node,
+ * then metadata node will change after the failback completes
+ */
+ String originalMetadataNode = AsterixAppContextInfo.getInstance().getMetadataProperties()
+ .getMetadataNodeName();
+ if (originalMetadataNode.equals(failbackNode)) {
+ plan.setNodeToReleaseMetadataManager(currentMetadataNode);
+ currentMetadataNode = "";
+ metadataNodeActive = false;
+ }
+
+ //force new jobs to wait
+ state = ClusterState.REBALANCING;
+
+ ICCMessageBroker messageBroker = (ICCMessageBroker) AsterixAppContextInfo.getInstance()
+ .getCCApplicationContext().getMessageBroker();
+ //send requests to other nodes to complete on-going jobs and prepare partitions for failback
+ Set<PreparePartitionsFailbackRequestMessage> planFailbackRequests = plan.getPlanFailbackRequests();
+ for (PreparePartitionsFailbackRequestMessage request : planFailbackRequests) {
+ try {
+ messageBroker.sendApplicationMessageToNC(request, request.getNodeID());
+ plan.addPendingRequest(request);
+ } catch (Exception e) {
+ LOGGER.warning("Failed to send failback request to: " + request.getNodeID());
+ e.printStackTrace();
+ plan.notifyNodeFailure(request.getNodeID());
+ revertFailedFailbackPlanEffects();
+ break;
+ }
+ }
+ /**
+ * wait until the current plan is completed before processing the next plan.
+ * when the current one completes or is reverted, the cluster state will be
+ * ACTIVE again, and the next failback plan (if any) will be processed.
+ */
+ break;
+ } else if (plan.getState() == FailbackPlanState.PENDING_ROLLBACK) {
+ //this plan failed before sending any requests -> nothing to rollback
+ planId2FailbackPlanMap.remove(plan.getPlanId());
+ }
+ }
+ }
+ }
+
+ public synchronized void processPreparePartitionsFailbackResponse(PreparePartitionsFailbackResponseMessage msg) {
+ NodeFailbackPlan plan = planId2FailbackPlanMap.get(msg.getPlanId());
+ plan.markRequestCompleted(msg.getRequestId());
+ /**
+ * A plan at this stage will be in one of three states:
+ * 1. PENDING_PARTICIPANT_REPONSE -> one or more responses are still expected (wait).
+ * 2. PENDING_COMPLETION -> all responses received (time to send completion request).
+ * 3. PENDING_ROLLBACK -> the plan failed and we just received the final pending response (revert).
+ */
+ if (plan.getState() == FailbackPlanState.PENDING_COMPLETION) {
+ CompleteFailbackRequestMessage request = plan.getCompleteFailbackRequestMessage();
+
+ //send complete resync and takeover partitions to the failing back node
+ ICCMessageBroker messageBroker = (ICCMessageBroker) AsterixAppContextInfo.getInstance()
+ .getCCApplicationContext().getMessageBroker();
+ try {
+ messageBroker.sendApplicationMessageToNC(request, request.getNodeId());
+ } catch (Exception e) {
+ LOGGER.warning("Failed to send complete failback request to: " + request.getNodeId());
+ e.printStackTrace();
+ notifyFailbackPlansNodeFailure(request.getNodeId());
+ revertFailedFailbackPlanEffects();
+ }
+ } else if (plan.getState() == FailbackPlanState.PENDING_ROLLBACK) {
+ revertFailedFailbackPlanEffects();
+ }
+ }
+
+ public synchronized void processCompleteFailbackResponse(CompleteFailbackResponseMessage reponse) {
+ /**
+ * the failback plan completed successfully:
+ * Remove all references to it.
+ * Remove the the failing back node from the failed nodes list.
+ * Notify its replicas to reconnect to it.
+ * Set the failing back node partitions as active.
+ */
+ NodeFailbackPlan plan = planId2FailbackPlanMap.remove(reponse.getPlanId());
+ String nodeId = plan.getNodeId();
+ failedNodes.remove(nodeId);
+ //notify impacted replicas they can reconnect to this node
+ notifyImpactedReplicas(nodeId, ClusterEventType.NODE_JOIN);
+ updateNodePartitions(nodeId, true);
+ }
+
+ private synchronized void notifyImpactedReplicas(String nodeId, ClusterEventType event) {
+ AsterixReplicationProperties replicationProperties = AsterixAppContextInfo.getInstance()
+ .getReplicationProperties();
+ Set<String> remoteReplicas = replicationProperties.getRemoteReplicasIds(nodeId);
+ String nodeIdAddress = "";
+ //in case the node joined with a new IP address, we need to send it to the other replicas
+ if (event == ClusterEventType.NODE_JOIN) {
+ nodeIdAddress = activeNcConfiguration.get(nodeId).get(CLUSTER_NET_IP_ADDRESS_KEY);
+ }
+
+ ReplicaEventMessage msg = new ReplicaEventMessage(nodeId, nodeIdAddress, event);
+ ICCMessageBroker messageBroker = (ICCMessageBroker) AsterixAppContextInfo.getInstance()
+ .getCCApplicationContext().getMessageBroker();
+ for (String replica : remoteReplicas) {
+ //if the remote replica is alive, send the event
+ if (activeNcConfiguration.containsKey(replica)) {
+ try {
+ messageBroker.sendApplicationMessageToNC(msg, replica);
+ } catch (Exception e) {
+ e.printStackTrace();
+ }
+ }
+ }
+ }
+
+ private synchronized void revertFailedFailbackPlanEffects() {
+ Iterator<NodeFailbackPlan> iterator = planId2FailbackPlanMap.values().iterator();
+ while (iterator.hasNext()) {
+ NodeFailbackPlan plan = iterator.next();
+ if (plan.getState() == FailbackPlanState.PENDING_ROLLBACK) {
+ //TODO if the failing back node is still active, notify it to construct a new plan for it
+ iterator.remove();
+
+ //reassign the partitions that were supposed to be failed back to an active replica
+ requestPartitionsTakeover(plan.getNodeId());
+ }
+ }
+ }
+
+ private synchronized void notifyFailbackPlansNodeFailure(String nodeId) {
+ Iterator<NodeFailbackPlan> iterator = planId2FailbackPlanMap.values().iterator();
+ while (iterator.hasNext()) {
+ NodeFailbackPlan plan = iterator.next();
+ plan.notifyNodeFailure(nodeId);
+ }
+ }
+
+ public synchronized boolean isMetadataNodeActive() {
+ return metadataNodeActive;
+ }
+
+ public boolean isReplicationEnabled() {
+ if (cluster != null && cluster.getDataReplication() != null) {
+ return cluster.getDataReplication().isEnabled();
+ }
+ return false;
}
public boolean isAutoFailoverEnabled() {
- if (cluster != null && cluster.getDataReplication() != null && cluster.getDataReplication().isEnabled()) {
- return cluster.getDataReplication().isAutoFailover();
+ return isReplicationEnabled() && cluster.getDataReplication().isAutoFailover();
+ }
+
+ public synchronized JSONObject getClusterStateDescription() throws JSONException {
+ JSONObject stateDescription = new JSONObject();
+ stateDescription.put("State", state.name());
+ stateDescription.put("Metadata_Node", currentMetadataNode);
+ for (ClusterPartition partition : clusterPartitions.values()) {
+ stateDescription.put("partition_" + partition.getPartitionId(), partition.getActiveNodeId());
}
- return false;
+ return stateDescription;
}
}
\ 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
index 647a6a3..7502737 100644
--- 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
@@ -26,10 +26,12 @@
import java.util.Set;
public class ReplicaFilesRequest {
- Set<String> replicaIds;
+ private final Set<String> replicaIds;
+ private final Set<String> existingFiles;
- public ReplicaFilesRequest(Set<String> replicaIds) {
+ public ReplicaFilesRequest(Set<String> replicaIds, Set<String> existingFiles) {
this.replicaIds = replicaIds;
+ this.existingFiles = existingFiles;
}
public void serialize(OutputStream out) throws IOException {
@@ -38,6 +40,10 @@
for (String replicaId : replicaIds) {
dos.writeUTF(replicaId);
}
+ dos.writeInt(existingFiles.size());
+ for (String fileName : existingFiles) {
+ dos.writeUTF(fileName);
+ }
}
public static ReplicaFilesRequest create(DataInput input) throws IOException {
@@ -46,15 +52,19 @@
for (int i = 0; i < size; i++) {
replicaIds.add(input.readUTF());
}
-
- return new ReplicaFilesRequest(replicaIds);
+ int filesCount = input.readInt();
+ Set<String> existingFiles = new HashSet<String>(filesCount);
+ for (int i = 0; i < filesCount; i++) {
+ existingFiles.add(input.readUTF());
+ }
+ return new ReplicaFilesRequest(replicaIds, existingFiles);
}
public Set<String> getReplicaIds() {
return replicaIds;
}
- public void setReplicaIds(Set<String> replicaIds) {
- this.replicaIds = replicaIds;
+ public Set<String> getExistingFiles() {
+ return existingFiles;
}
}
diff --git a/asterix-replication/src/main/java/org/apache/asterix/replication/functions/ReplicationProtocol.java b/asterix-replication/src/main/java/org/apache/asterix/replication/functions/ReplicationProtocol.java
index 790df66..d2380c1 100644
--- a/asterix-replication/src/main/java/org/apache/asterix/replication/functions/ReplicationProtocol.java
+++ b/asterix-replication/src/main/java/org/apache/asterix/replication/functions/ReplicationProtocol.java
@@ -26,7 +26,6 @@
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;
@@ -52,7 +51,6 @@
* 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
@@ -67,7 +65,6 @@
GET_REPLICA_LOGS,
GET_REPLICA_MAX_LSN,
GET_REPLICA_MIN_LSN,
- UPDATE_REPLICA,
GOODBYE,
REPLICA_EVENT,
LSM_COMPONENT_PROPERTIES,
@@ -115,8 +112,7 @@
//read replication request type
NetworkingUtil.readBytes(socketChannel, byteBuffer, REPLICATION_REQUEST_TYPE_SIZE);
- ReplicationRequestType requestType = ReplicationProtocol.ReplicationRequestType.values()[byteBuffer
- .getInt()];
+ ReplicationRequestType requestType = ReplicationProtocol.ReplicationRequestType.values()[byteBuffer.getInt()];
return requestType;
}
@@ -215,21 +211,6 @@
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);
@@ -244,12 +225,6 @@
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);
@@ -257,22 +232,24 @@
return ReplicaEvent.create(dis);
}
- public static void writeGetReplicaFilesRequest(ByteBuffer buffer, ReplicaFilesRequest request) throws IOException {
+ public static ByteBuffer writeGetReplicaFilesRequest(ByteBuffer buffer, ReplicaFilesRequest request)
+ throws IOException {
ByteArrayOutputStream outputStream = new ByteArrayOutputStream();
- DataOutputStream oos = new DataOutputStream(outputStream);
- request.serialize(oos);
- oos.close();
+ try (DataOutputStream oos = new DataOutputStream(outputStream)) {
+ request.serialize(oos);
- int requestSize = REPLICATION_REQUEST_HEADER_SIZE + oos.size();
- if (buffer.capacity() < requestSize) {
- buffer = ByteBuffer.allocate(requestSize);
- } else {
- buffer.clear();
+ 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();
+ return buffer;
}
- 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)
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
deleted file mode 100644
index 9915c83..0000000
--- a/asterix-replication/src/main/java/org/apache/asterix/replication/management/ReplicaEventNotifier.java
+++ /dev/null
@@ -1,109 +0,0 @@
-/*
- * 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.ReplicationProtocol;
-
-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 = ReplicationProtocol.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(ReplicationProtocol.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/ReplicationChannel.java b/asterix-replication/src/main/java/org/apache/asterix/replication/management/ReplicationChannel.java
index e4d94b4..b9447af 100644
--- 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
@@ -30,6 +30,7 @@
import java.nio.file.Files;
import java.nio.file.Path;
import java.nio.file.Paths;
+import java.util.ArrayList;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
@@ -52,7 +53,6 @@
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;
@@ -72,6 +72,7 @@
import org.apache.asterix.replication.storage.LSMComponentProperties;
import org.apache.asterix.replication.storage.LSMIndexFileProperties;
import org.apache.asterix.replication.storage.ReplicaResourcesManager;
+import org.apache.asterix.transaction.management.resource.PersistentLocalResourceRepository;
import org.apache.hyracks.api.application.INCApplicationContext;
import org.apache.hyracks.storage.am.common.api.IMetaDataPageManager;
import org.apache.hyracks.storage.am.lsm.common.api.LSMOperationType;
@@ -97,6 +98,7 @@
private final Map<String, LSMComponentProperties> lsmComponentId2PropertiesMap;
private final Map<Long, RemoteLogMapping> localLSN2RemoteLSNMap;
private final LSMComponentsSyncService lsmComponentLSNMappingService;
+ private final Set<Integer> nodeHostedPartitions;
public ReplicationChannel(String nodeId, AsterixReplicationProperties replicationProperties, ILogManager logManager,
IReplicaResourcesManager replicaResoucesManager, IReplicationManager replicationManager,
@@ -112,6 +114,17 @@
localLSN2RemoteLSNMap = new ConcurrentHashMap<Long, RemoteLogMapping>();
lsmComponentLSNMappingService = new LSMComponentsSyncService();
replicationThreads = Executors.newCachedThreadPool(appContext.getThreadFactory());
+ Map<String, ClusterPartition[]> nodePartitions = ((IAsterixPropertiesProvider) asterixAppRuntimeContextProvider
+ .getAppContext()).getMetadataProperties().getNodePartitions();
+ Set<String> nodeReplicationClients = replicationProperties.getNodeReplicationClients(nodeId);
+ List<Integer> clientsPartitions = new ArrayList<>();
+ for (String clientId : nodeReplicationClients) {
+ for (ClusterPartition clusterPartition : nodePartitions.get(clientId)) {
+ clientsPartitions.add(clusterPartition.getPartitionId());
+ }
+ }
+ nodeHostedPartitions = new HashSet<>(clientsPartitions.size());
+ nodeHostedPartitions.addAll(clientsPartitions);
}
@Override
@@ -193,6 +206,18 @@
}
}
+ private static void sendRemoteRecoveryLog(ILogRecord logRecord, SocketChannel socketChannel, ByteBuffer outBuffer)
+ throws IOException {
+ logRecord.setLogSource(LogSource.REMOTE_RECOVERY);
+ if (logRecord.getSerializedLogSize() > outBuffer.capacity()) {
+ int requestSize = logRecord.getSerializedLogSize() + ReplicationProtocol.REPLICATION_REQUEST_HEADER_SIZE;
+ outBuffer = ByteBuffer.allocate(requestSize);
+ }
+ //set log source to REMOTE_RECOVERY to avoid re-logging on the recipient side
+ ReplicationProtocol.writeRemoteRecoveryLogRequest(outBuffer, logRecord);
+ NetworkingUtil.transferBufferToChannel(socketChannel, outBuffer);
+ }
+
/**
* A replication thread is created per received replication request.
*/
@@ -232,9 +257,6 @@
case REPLICA_EVENT:
handleReplicaEvent();
break;
- case UPDATE_REPLICA:
- handleUpdateReplica();
- break;
case GET_REPLICA_MAX_LSN:
handleGetReplicaMaxLSN();
break;
@@ -379,29 +401,33 @@
List<String> filesList;
Set<String> replicaIds = request.getReplicaIds();
+ Set<String> requesterExistingFiles = request.getExistingFiles();
Map<String, ClusterPartition[]> nodePartitions = ((IAsterixPropertiesProvider) asterixAppRuntimeContextProvider
.getAppContext()).getMetadataProperties().getNodePartitions();
for (String replicaId : replicaIds) {
//get replica partitions
ClusterPartition[] replicaPatitions = nodePartitions.get(replicaId);
for (ClusterPartition partition : replicaPatitions) {
- filesList = replicaResourcesManager.getPartitionIndexesFiles(partition.getPartitionId());
-
+ filesList = replicaResourcesManager.getPartitionIndexesFiles(partition.getPartitionId(), false);
//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,
- IMetaDataPageManager.INVALID_LSN_OFFSET, false);
- outBuffer = ReplicationProtocol.writeFileReplicationRequest(outBuffer, fileProperties,
- ReplicationRequestType.REPLICATE_FILE);
+ String relativeFilePath = PersistentLocalResourceRepository.getResourceRelativePath(filePath);
+ //if the file already exists on the requester, skip it
+ if (!requesterExistingFiles.contains(relativeFilePath)) {
+ try (RandomAccessFile fromFile = new RandomAccessFile(filePath, "r");
+ FileChannel fileChannel = fromFile.getChannel();) {
+ long fileSize = fileChannel.size();
+ fileProperties.initialize(filePath, fileSize, replicaId, false,
+ IMetaDataPageManager.INVALID_LSN_OFFSET, false);
+ outBuffer = ReplicationProtocol.writeFileReplicationRequest(outBuffer, fileProperties,
+ ReplicationRequestType.REPLICATE_FILE);
- //send file info
- NetworkingUtil.transferBufferToChannel(socketChannel, outBuffer);
+ //send file info
+ NetworkingUtil.transferBufferToChannel(socketChannel, outBuffer);
- //transfer file
- NetworkingUtil.sendFile(fileChannel, socketChannel);
+ //transfer file
+ NetworkingUtil.sendFile(fileChannel, socketChannel);
+ }
}
}
}
@@ -416,11 +442,23 @@
ReplicaLogsRequest request = ReplicationProtocol.readReplicaLogsRequest(inBuffer);
Set<String> replicaIds = request.getReplicaIds();
+ //get list of partitions that belong to the replicas in the request
+ Set<Integer> requestedPartitions = new HashSet<>();
+ Map<String, ClusterPartition[]> nodePartitions = ((IAsterixPropertiesProvider) asterixAppRuntimeContextProvider
+ .getAppContext()).getMetadataProperties().getNodePartitions();
+ for (String replicaId : replicaIds) {
+ //get replica partitions
+ ClusterPartition[] replicaPatitions = nodePartitions.get(replicaId);
+ for (ClusterPartition partition : replicaPatitions) {
+ requestedPartitions.add(partition.getPartitionId());
+ }
+ }
+
long fromLSN = request.getFromLSN();
long minLocalFirstLSN = asterixAppRuntimeContextProvider.getAppContext().getTransactionSubsystem()
.getRecoveryManager().getLocalMinFirstLSN();
- //get Log read
+ //get Log reader
ILogReader logReader = logManager.getLogReader(true);
try {
if (fromLSN < logManager.getReadableSmallestLSN()) {
@@ -429,25 +467,34 @@
logReader.initializeScan(fromLSN);
ILogRecord logRecord = logReader.next();
+ Set<Integer> requestedPartitionsJobs = new HashSet<>();
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()
- + ReplicationProtocol.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);
- ReplicationProtocol.writeRemoteRecoveryLogRequest(outBuffer, logRecord);
- NetworkingUtil.transferBufferToChannel(socketChannel, outBuffer);
+ //send only logs that belong to the partitions of the request and required for recovery
+ switch (logRecord.getLogType()) {
+ case LogType.UPDATE:
+ case LogType.ENTITY_COMMIT:
+ case LogType.UPSERT_ENTITY_COMMIT:
+ if (requestedPartitions.contains(logRecord.getResourcePartition())) {
+ sendRemoteRecoveryLog(logRecord, socketChannel, outBuffer);
+ requestedPartitionsJobs.add(logRecord.getJobId());
+ }
+ break;
+ case LogType.JOB_COMMIT:
+ if (requestedPartitionsJobs.contains(logRecord.getJobId())) {
+ sendRemoteRecoveryLog(logRecord, socketChannel, outBuffer);
+ requestedPartitionsJobs.remove(logRecord.getJobId());
+ }
+ break;
+ case LogType.ABORT:
+ case LogType.FLUSH:
+ break;
+ default:
+ throw new ACIDException("Unsupported LogType: " + logRecord.getLogType());
}
logRecord = logReader.next();
}
@@ -459,12 +506,6 @@
ReplicationProtocol.sendGoodbye(socketChannel);
}
- private void handleUpdateReplica() throws IOException {
- inBuffer = ReplicationProtocol.readRequest(socketChannel, inBuffer);
- Replica replica = ReplicationProtocol.readReplicaUpdateRequest(inBuffer);
- replicationManager.updateReplicaInfo(replica);
- }
-
private void handleReplicaEvent() throws IOException {
inBuffer = ReplicationProtocol.readRequest(socketChannel, inBuffer);
ReplicaEvent event = ReplicationProtocol.readReplicaEventRequest(inBuffer);
@@ -484,37 +525,45 @@
inBuffer = ReplicationProtocol.readRequest(socketChannel, inBuffer);
//Deserialize log
- remoteLog.readRemoteLog(inBuffer, false, localNodeID);
+ remoteLog.readRemoteLog(inBuffer, false);
remoteLog.setLogSource(LogSource.REMOTE);
- if (remoteLog.getLogType() == LogType.JOB_COMMIT) {
- LogRecord jobCommitLog = new LogRecord();
- TransactionUtil.formJobTerminateLogRecord(jobCommitLog, 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();
- TransactionUtil.formFlushLogRecord(flushLog, 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);
+ switch (remoteLog.getLogType()) {
+ case LogType.UPDATE:
+ case LogType.ENTITY_COMMIT:
+ case LogType.UPSERT_ENTITY_COMMIT:
+ //if the log partition belongs to a partitions hosted on this node, replicate it
+ if (nodeHostedPartitions.contains(remoteLog.getResourcePartition())) {
+ logManager.log(remoteLog);
+ }
+ break;
+ case LogType.JOB_COMMIT:
+ LogRecord jobCommitLog = new LogRecord();
+ TransactionUtil.formJobTerminateLogRecord(jobCommitLog, remoteLog.getJobId(), true);
+ jobCommitLog.setReplicationThread(this);
+ jobCommitLog.setLogSource(LogSource.REMOTE);
+ logManager.log(jobCommitLog);
+ break;
+ case LogType.FLUSH:
+ LogRecord flushLog = new LogRecord();
+ TransactionUtil.formFlushLogRecord(flushLog, 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();
+ }
+ break;
+ default:
+ throw new ACIDException("Unsupported LogType: " + remoteLog.getLogType());
}
}
@@ -649,4 +698,4 @@
}
}
}
-}
\ 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
deleted file mode 100644
index 39130a4..0000000
--- a/asterix-replication/src/main/java/org/apache/asterix/replication/management/ReplicationLifecycleListener.java
+++ /dev/null
@@ -1,77 +0,0 @@
-/*
- * 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
index 5c35df4..7243629 100644
--- 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
@@ -37,6 +37,7 @@
import java.util.HashMap;
import java.util.HashSet;
import java.util.Iterator;
+import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import java.util.Set;
@@ -50,7 +51,9 @@
import java.util.logging.Level;
import java.util.logging.Logger;
+import org.apache.asterix.common.cluster.ClusterPartition;
import org.apache.asterix.common.config.AsterixReplicationProperties;
+import org.apache.asterix.common.config.IAsterixPropertiesProvider;
import org.apache.asterix.common.dataflow.AsterixLSMIndexUtil;
import org.apache.asterix.common.exceptions.ACIDException;
import org.apache.asterix.common.replication.AsterixReplicationJob;
@@ -59,23 +62,25 @@
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.ReplicationProtocol;
-import org.apache.asterix.replication.functions.ReplicationProtocol.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.functions.ReplicationProtocol;
+import org.apache.asterix.replication.functions.ReplicationProtocol.ReplicationRequestType;
import org.apache.asterix.replication.logging.ReplicationLogBuffer;
import org.apache.asterix.replication.logging.ReplicationLogFlusher;
import org.apache.asterix.replication.storage.LSMComponentProperties;
import org.apache.asterix.replication.storage.LSMIndexFileProperties;
import org.apache.asterix.replication.storage.ReplicaResourcesManager;
+import org.apache.asterix.transaction.management.resource.PersistentLocalResourceRepository;
+import org.apache.hyracks.api.application.IClusterLifecycleListener.ClusterEventType;
+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;
@@ -107,6 +112,7 @@
private final IAsterixAppRuntimeContextProvider asterixAppRuntimeContextProvider;
private final AsterixReplicationProperties replicationProperties;
private final Map<String, Replica> replicas;
+ private final Map<String, Set<Integer>> replica2PartitionsMap;
private final AtomicBoolean replicationSuspended;
private AtomicBoolean terminateJobsReplication;
@@ -117,8 +123,8 @@
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);
+ private static final IReplicationJob REPLICATION_JOB_POISON_PILL = new AsterixReplicationJob(
+ ReplicationJobType.METADATA, ReplicationOperation.REPLICATE, ReplicationExecutionType.ASYNC, null);
//used to identify the correct IP address when the node has multiple network interfaces
private String hostIPAddressFirstOctet = null;
@@ -128,6 +134,7 @@
private ReplicationLogFlusher txnlogsReplicator;
private Future<? extends Object> txnLogReplicatorTask;
private Map<String, SocketChannel> logsReplicaSockets = null;
+
//TODO this class needs to be refactored by moving its private classes to separate files
//and possibly using MessageBroker to send/receive remote replicas events.
public ReplicationManager(String nodeId, AsterixReplicationProperties replicationProperties,
@@ -144,7 +151,6 @@
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>();
@@ -156,12 +162,25 @@
replicationJobsProcessor = new ReplicationJobsProccessor();
replicationMonitor = new ReplicasEventsMonitor();
+ Map<String, ClusterPartition[]> nodePartitions = ((IAsterixPropertiesProvider) asterixAppRuntimeContextProvider
+ .getAppContext()).getMetadataProperties().getNodePartitions();
//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);
+ replica2PartitionsMap = new HashMap<>(replicaNodes.size());
+ for (Replica replica : replicaNodes) {
+ replicas.put(replica.getNode().getId(), replica);
+ //for each remote replica, get the list of replication clients
+ Set<String> nodeReplicationClients = replicationProperties.getNodeReplicationClients(replica.getId());
+ //get the partitions of each client
+ List<Integer> clientPartitions = new ArrayList<>();
+ for (String clientId : nodeReplicationClients) {
+ for (ClusterPartition clusterPartition : nodePartitions.get(clientId)) {
+ clientPartitions.add(clusterPartition.getPartitionId());
+ }
}
+ Set<Integer> clientPartitonsSet = new HashSet<>(clientPartitions.size());
+ clientPartitonsSet.addAll(clientPartitions);
+ replica2PartitionsMap.put(replica.getId(), clientPartitonsSet);
}
int numLogBuffers = logManager.getNumLogPages();
emptyLogBuffersQ = new LinkedBlockingQueue<ReplicationLogBuffer>(numLogBuffers);
@@ -173,10 +192,6 @@
}
}
- /**
- * 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) {
@@ -253,24 +268,28 @@
*/
private void processJob(IReplicationJob job, Map<String, SocketChannel> replicasSockets, ByteBuffer requestBuffer)
throws IOException {
- boolean isLSMComponentFile;
- ByteBuffer responseBuffer = null;
- LSMIndexFileProperties asterixFileProperties = new LSMIndexFileProperties();
- 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();
+
+ //all of the job's files belong to a single storage partition.
+ //get any of them to determine the partition from the file path.
+ String jobFile = job.getJobFiles().iterator().next();
+ int jobPartitionId = PersistentLocalResourceRepository.getResourcePartition(jobFile);
+
+ ByteBuffer responseBuffer = null;
+ LSMIndexFileProperties asterixFileProperties = new LSMIndexFileProperties();
+ if (requestBuffer == null) {
+ requestBuffer = ByteBuffer.allocate(INITIAL_BUFFER_SIZE);
}
- int remainingFiles = job.getJobFiles().size();
+ boolean isLSMComponentFile = job.getJobType() == ReplicationJobType.LSM_COMPONENT;
+ try {
+ //if there isn't already a connection, establish a new one
+ if (replicasSockets == null) {
+ replicasSockets = getActiveRemoteReplicasSockets();
+ }
- if (job.getOperation() == ReplicationOperation.REPLICATE) {
- try {
+ int remainingFiles = job.getJobFiles().size();
+ if (job.getOperation() == ReplicationOperation.REPLICATE) {
//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) {
@@ -316,6 +335,10 @@
Iterator<Map.Entry<String, SocketChannel>> iterator = replicasSockets.entrySet().iterator();
while (iterator.hasNext()) {
Map.Entry<String, SocketChannel> entry = iterator.next();
+ //if the remote replica is not interested in this partition, skip it.
+ if (!replica2PartitionsMap.get(entry.getKey()).contains(jobPartitionId)) {
+ continue;
+ }
SocketChannel socketChannel = entry.getValue();
//transfer request header & file
try {
@@ -338,44 +361,52 @@
}
}
}
- } 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,
- IMetaDataPageManager.INVALID_LSN_OFFSET, remainingFiles == 0);
- ReplicationProtocol.writeFileReplicationRequest(requestBuffer, asterixFileProperties,
- ReplicationRequestType.DELETE_FILE);
+ } else if (job.getOperation() == ReplicationOperation.DELETE) {
+ for (String filePath : job.getJobFiles()) {
+ remainingFiles--;
+ asterixFileProperties.initialize(filePath, -1, nodeId, isLSMComponentFile,
+ IMetaDataPageManager.INVALID_LSN_OFFSET, remainingFiles == 0);
+ ReplicationProtocol.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);
+ Iterator<Map.Entry<String, SocketChannel>> iterator = replicasSockets.entrySet().iterator();
+ while (iterator.hasNext()) {
+ Map.Entry<String, SocketChannel> entry = iterator.next();
+ //if the remote replica is not interested in this partition, skip it.
+ if (!replica2PartitionsMap.get(entry.getKey()).contains(jobPartitionId)) {
+ continue;
}
- } catch (IOException e) {
- reportFailedReplica(entry.getKey());
- iterator.remove();
- } finally {
- requestBuffer.position(0);
+ 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);
+ }
}
} finally {
- //if sync, close sockets with replicas since they wont be reused
- if (job.getExecutionType() == ReplicationExecutionType.SYNC) {
- closeReplicaSockets(replicasSockets);
- }
+ exitReplicatedLSMComponent(job);
+ }
+ }
+
+ private static void exitReplicatedLSMComponent(IReplicationJob job) throws HyracksDataException {
+ if (job.getOperation() == ReplicationOperation.REPLICATE && job instanceof ILSMIndexReplicationJob) {
+ //exit the replicated LSM components
+ ILSMIndexReplicationJob aJob = (ILSMIndexReplicationJob) job;
+ aJob.endReplication();
}
}
@@ -398,8 +429,7 @@
}
//read response from remote replicas
- ReplicationRequestType responseFunction = ReplicationProtocol.getRequestType(socketChannel,
- responseBuffer);
+ ReplicationRequestType responseFunction = ReplicationProtocol.getRequestType(socketChannel, responseBuffer);
return responseFunction;
}
@@ -415,17 +445,7 @@
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);
- }
- */
}
/**
@@ -440,7 +460,7 @@
if (force) {
terminateJobsReplication.set(true);
}
- replicationJobsQ.offer(replicationJobPoisonPill);
+ replicationJobsQ.offer(REPLICATION_JOB_POISON_PILL);
//wait until the jobs are suspended
synchronized (jobsReplicationSuspended) {
@@ -504,27 +524,6 @@
}
}
- @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 = ReplicationProtocol.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.
@@ -536,7 +535,7 @@
node.setId(nodeId);
node.setClusterIp(NetworkingUtil.getHostAddress(hostIPAddressFirstOctet));
Replica replica = new Replica(node);
- ReplicaEvent event = new ReplicaEvent(replica, ReplicaEventType.SHUTDOWN);
+ ReplicaEvent event = new ReplicaEvent(replica, ClusterEventType.NODE_SHUTTING_DOWN);
ByteBuffer buffer = ReplicationProtocol.writeReplicaEventRequest(event);
Map<String, SocketChannel> replicaSockets = getActiveRemoteReplicasSockets();
sendRequest(replicaSockets, buffer);
@@ -672,9 +671,6 @@
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--;
@@ -873,7 +869,7 @@
//need to stop processing any new logs or jobs
terminateJobsReplication.set(true);
- ReplicaEvent event = new ReplicaEvent(replica, ReplicaEventType.FAIL);
+ ReplicaEvent event = new ReplicaEvent(replica, ClusterEventType.NODE_FAILURE);
reportReplicaEvent(event);
}
@@ -986,9 +982,10 @@
//Recovery Method
@Override
- public void requestReplicaFiles(String selectedReplicaId, Set<String> replicasDataToRecover) throws IOException {
- ReplicaFilesRequest request = new ReplicaFilesRequest(replicasDataToRecover);
- ReplicationProtocol.writeGetReplicaFilesRequest(dataBuffer, request);
+ public void requestReplicaFiles(String selectedReplicaId, Set<String> replicasDataToRecover,
+ Set<String> existingFiles) throws IOException {
+ ReplicaFilesRequest request = new ReplicaFilesRequest(replicasDataToRecover, existingFiles);
+ dataBuffer = ReplicationProtocol.writeGetReplicaFilesRequest(dataBuffer, request);
try (SocketChannel socketChannel = getReplicaSocket(selectedReplicaId)) {
@@ -997,8 +994,7 @@
String indexPath;
String destFilePath;
- ReplicationRequestType responseFunction = ReplicationProtocol.getRequestType(socketChannel,
- dataBuffer);
+ ReplicationRequestType responseFunction = ReplicationProtocol.getRequestType(socketChannel, dataBuffer);
LSMIndexFileProperties fileProperties;
while (responseFunction != ReplicationRequestType.GOODBYE) {
dataBuffer = ReplicationProtocol.readRequest(socketChannel, dataBuffer);
@@ -1057,11 +1053,10 @@
//Recovery Method
@Override
- public ArrayList<ILogRecord> requestReplicaLogs(String remoteNode, Set<String> nodeIdsToRecoverFor, long fromLSN)
- throws IOException, ACIDException {
+ public void requestReplicaLogs(String remoteNode, Set<String> nodeIdsToRecoverFor, long fromLSN,
+ File recoveryLogsFile) throws IOException, ACIDException {
ReplicaLogsRequest request = new ReplicaLogsRequest(nodeIdsToRecoverFor, fromLSN);
dataBuffer = ReplicationProtocol.writeGetReplicaLogsRequest(dataBuffer, request);
-
try (SocketChannel socketChannel = getReplicaSocket(remoteNode)) {
//transfer request
NetworkingUtil.transferBufferToChannel(socketChannel, dataBuffer);
@@ -1069,28 +1064,55 @@
//read response type
ReplicationRequestType responseType = ReplicationProtocol.getRequestType(socketChannel, dataBuffer);
- ArrayList<ILogRecord> recoveryLogs = new ArrayList<ILogRecord>();
ILogRecord logRecord = new LogRecord();
- while (responseType != ReplicationRequestType.GOODBYE) {
- dataBuffer = ReplicationProtocol.readRequest(socketChannel, dataBuffer);
- logRecord.readRemoteLog(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);
+ Set<Integer> nodePartitions = ((PersistentLocalResourceRepository) asterixAppRuntimeContextProvider
+ .getLocalResourceRepository()).getNodeOrignalPartitions();
+ Set<Integer> nodePartitionsJobs = new HashSet<>();
+ try (RandomAccessFile raf = new RandomAccessFile(recoveryLogsFile, "rw");
+ FileChannel fileChannel = raf.getChannel();) {
+ while (responseType != ReplicationRequestType.GOODBYE) {
+ dataBuffer = ReplicationProtocol.readRequest(socketChannel, dataBuffer);
+ logRecord.readRemoteLog(dataBuffer, true);
+ switch (logRecord.getLogType()) {
+ case LogType.UPDATE:
+ case LogType.ENTITY_COMMIT:
+ case LogType.UPSERT_ENTITY_COMMIT:
+ if (nodePartitions.contains(logRecord.getResourcePartition())) {
+ nodePartitionsJobs.add(logRecord.getJobId());
+ dataBuffer.flip();
+ while (dataBuffer.hasRemaining()) {
+ //store log in temp file to replay it for recovery
+ fileChannel.write(dataBuffer);
+ }
+ } else {
+ //send log to log manager as a remote recovery log
+ logManager.log(logRecord);
+ }
+ break;
+ case LogType.JOB_COMMIT:
+ if (nodePartitionsJobs.contains(logRecord.getJobId())) {
+ nodePartitionsJobs.remove(logRecord.getJobId());
+ dataBuffer.flip();
+ while (dataBuffer.hasRemaining()) {
+ //store log in temp file to replay it for recovery
+ fileChannel.write(dataBuffer);
+ }
+ break;
+ }
+ logManager.log(logRecord);
+ break;
+ case LogType.ABORT:
+ case LogType.FLUSH:
+ break;
+ default:
+ throw new ACIDException("Unsupported LogType: " + logRecord.getLogType());
+ }
+ responseType = ReplicationProtocol.getRequestType(socketChannel, dataBuffer);
}
-
- responseType = ReplicationProtocol.getRequestType(socketChannel, dataBuffer);
}
//send goodbye
ReplicationProtocol.sendGoodbye(socketChannel);
- return recoveryLogs;
}
}
@@ -1108,13 +1130,14 @@
event = replicaEventsQ.take();
switch (event.getEventType()) {
- case FAIL:
+ case NODE_FAILURE:
handleReplicaFailure(event.getReplica().getId());
break;
- case JOIN:
+ case NODE_JOIN:
+ updateReplicaInfo(event.getReplica());
checkReplicaState(event.getReplica().getId(), false, true);
break;
- case SHUTDOWN:
+ case NODE_SHUTTING_DOWN:
handleShutdownEvent(event.getReplica().getId());
break;
default:
@@ -1168,18 +1191,17 @@
}
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 {
+ if (job == REPLICATION_JOB_POISON_PILL) {
terminateJobsReplication.set(true);
continue;
}
+ //if there isn't already a connection, establish a new one
+ if (replicaSockets == null) {
+ replicaSockets = getActiveRemoteReplicasSockets();
+ }
+ processJob(job, replicaSockets, reusableBuffer);
+
//if no more jobs to process, close sockets
if (replicationJobsQ.size() == 0) {
LOGGER.log(Level.INFO, "No pending replication jobs. Closing connections to replicas");
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
index ee987f8..c905add 100644
--- 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
@@ -18,8 +18,11 @@
*/
package org.apache.asterix.replication.recovery;
+import java.io.File;
import java.io.IOException;
-import java.util.ArrayList;
+import java.io.RandomAccessFile;
+import java.nio.channels.FileChannel;
+import java.util.Arrays;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Map;
@@ -30,56 +33,62 @@
import org.apache.asterix.common.api.IAsterixAppRuntimeContext;
import org.apache.asterix.common.api.IDatasetLifecycleManager;
+import org.apache.asterix.common.cluster.ClusterPartition;
import org.apache.asterix.common.config.AsterixReplicationProperties;
+import org.apache.asterix.common.config.IAsterixPropertiesProvider;
import org.apache.asterix.common.exceptions.ACIDException;
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.common.transactions.ILogReader;
+import org.apache.asterix.common.transactions.IRecoveryManager;
import org.apache.asterix.om.util.AsterixClusterProperties;
+import org.apache.asterix.replication.storage.ReplicaResourcesManager;
import org.apache.asterix.transaction.management.resource.PersistentLocalResourceRepository;
+import org.apache.asterix.transaction.management.service.logging.RemoteLogReader;
public class RemoteRecoveryManager implements IRemoteRecoveryManager {
+ private static final String RECOVERY_LOGS_FILE_NAME = "recoveryLogs";
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;
+ private final static int REMOTE_RECOVERY_JOB_ID = -1;
+ private Map<String, Set<String>> failbackRecoveryReplicas;
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() {
- //TODO this method needs to be adapted to perform failback when autoFailover is enabled.
- //Currently we will not allow a node to perform remote recovery since another replica
- //already tookover its workload and might not resync correctly if there are on on-going
- //jobs on the replica.
- if (AsterixClusterProperties.INSTANCE.isAutoFailoverEnabled()) {
- throw new IllegalStateException("Cannot perform remote recovery when auto failover is enabled.");
- }
//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;
+ //Any error happens, we should start the recovery from the start until the recovery is
+ //complete or an illegal state is reached (cannot recover or max attempts exceed).
+ int maxRecoveryAttempts = replicationProperties.getMaxRemoteRecoveryAttempts();
PersistentLocalResourceRepository resourceRepository = (PersistentLocalResourceRepository) runtimeContext
.getLocalResourceRepository();
+ IRecoveryManager recoveryManager = runtimeContext.getTransactionSubsystem().getRecoveryManager();
+ ILogManager logManager = runtimeContext.getTransactionSubsystem().getLogManager();
while (true) {
//start recovery steps
try {
- maxRecoveryAttempts--;
-
- if (maxRecoveryAttempts == 0) {
+ if (maxRecoveryAttempts <= 0) {
//to avoid infinite loop in case of unexpected behavior.
throw new IllegalStateException("Failed to perform remote recovery.");
}
+ //delete any existing recovery files from previous failed recovery attempts
+ recoveryManager.deleteRecoveryTemporaryFiles();
+
+ //create temporary file to store recovery logs
+ File recoveryLogsFile = recoveryManager.createJobRecoveryFile(REMOTE_RECOVERY_JOB_ID,
+ RECOVERY_LOGS_FILE_NAME);
+
/*** Prepare for Recovery ***/
//1. check remote replicas states
replicationManager.initializeReplicasState();
@@ -93,8 +102,9 @@
IDatasetLifecycleManager datasetLifeCycleManager = runtimeContext.getDatasetLifecycleManager();
datasetLifeCycleManager.closeAllDatasets();
- //3. remove any existing storage data
+ //3. remove any existing storage data and initialize storage metadata
resourceRepository.deleteStorageData(true);
+ resourceRepository.initializeNewUniverse(AsterixClusterProperties.INSTANCE.getStorageDirectoryName());
//4. select remote replicas to recover from per lost replica data
Map<String, Set<String>> selectedRemoteReplicas = constructRemoteRecoveryPlan();
@@ -110,47 +120,42 @@
String replicaId = remoteReplica.getKey();
Set<String> replicasDataToRecover = remoteReplica.getValue();
- //1. Request indexes metadata and LSM components
- replicationManager.requestReplicaFiles(replicaId, replicasDataToRecover);
+ //Request indexes metadata and LSM components
+ replicationManager.requestReplicaFiles(replicaId, replicasDataToRecover, new HashSet<String>());
- //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())
- .initializeNewUniverse(AsterixClusterProperties.INSTANCE.getStorageDirectoryName());
- //initialize resource id factor to correct max resource id
- runtimeContext.initializeResourceIdFactory();
- }
-
- //3. Get min LSN to start requesting logs from
+ //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);
+ //Request remote logs from selected remote replicas
+ replicationManager.requestReplicaLogs(replicaId, replicasDataToRecover, minLSN, recoveryLogsFile);
- //5. Replay remote logs using recovery manager
+ //Replay remote logs using recovery manager
if (replicasDataToRecover.contains(logManager.getNodeId())) {
- if (remoteRecoveryLogs.size() > 0) {
- runtimeContext.getTransactionSubsystem().getRecoveryManager()
- .replayRemoteLogs(remoteRecoveryLogs);
+ //replay logs for local partitions only
+ Set<Integer> nodePartitions = resourceRepository.getNodeOrignalPartitions();
+ try (RandomAccessFile raf = new RandomAccessFile(recoveryLogsFile, "r");
+ FileChannel fileChannel = raf.getChannel();) {
+ ILogReader logReader = new RemoteLogReader(fileChannel, fileChannel.size(),
+ logManager.getLogPageSize());
+ recoveryManager.replayPartitionsLogs(nodePartitions, logReader, 0);
}
- 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...");
+ maxRecoveryAttempts--;
}
}
}
private Map<String, Set<String>> constructRemoteRecoveryPlan() {
//1. identify which replicas reside in this node
- String localNodeId = logManager.getNodeId();
- Set<String> nodes = replicationProperties.getNodeReplicasIds(localNodeId);
+ String localNodeId = runtimeContext.getTransactionSubsystem().getId();
+
+ Set<String> nodes = replicationProperties.getNodeReplicationClients(localNodeId);
Map<String, Set<String>> recoveryCandidates = new HashMap<String, Set<String>>();
Map<String, Integer> candidatesScore = new HashMap<String, Integer>();
@@ -187,7 +192,6 @@
//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()) {
@@ -214,12 +218,133 @@
}
@Override
- public void takeoverPartitons(String failedNode, Integer[] partitions) throws IOException, ACIDException {
- long minLSN = runtimeContext.getReplicaResourcesManager().getPartitionsMinLSN(partitions);
- //reply logs > minLSN that belong to these partitions
- //TODO (mhubail) currently we assume the logs for these partitions belong to the failed node
- //this needs to be updated once log formats are updated to include the partition id
- runtimeContext.getTransactionSubsystem().getRecoveryManager().replayPartitionsLogs(partitions, minLSN,
- failedNode);
+ public void takeoverPartitons(Integer[] partitions) throws IOException, ACIDException {
+ /**
+ * TODO even though the takeover is always expected to succeed,
+ * in case of any failure during the takeover, the CC should be
+ * notified that the takeover failed.
+ */
+ Set<Integer> partitionsToTakeover = new HashSet<>(Arrays.asList(partitions));
+ ILogManager logManager = runtimeContext.getTransactionSubsystem().getLogManager();
+
+ long minLSN = runtimeContext.getReplicaResourcesManager().getPartitionsMinLSN(partitionsToTakeover);
+ long readableSmallestLSN = logManager.getReadableSmallestLSN();
+ if (minLSN < readableSmallestLSN) {
+ minLSN = readableSmallestLSN;
+ }
+
+ //replay logs > minLSN that belong to these partitions
+ IRecoveryManager recoveryManager = runtimeContext.getTransactionSubsystem().getRecoveryManager();
+ recoveryManager.replayPartitionsLogs(partitionsToTakeover, logManager.getLogReader(true), minLSN);
+
+ //mark these partitions as active in this node
+ PersistentLocalResourceRepository resourceRepository = (PersistentLocalResourceRepository) runtimeContext
+ .getLocalResourceRepository();
+ for (Integer patitionId : partitions) {
+ resourceRepository.addActivePartition(patitionId);
+ }
+ }
+
+ @Override
+ public void startFailbackProcess() {
+ int maxRecoveryAttempts = replicationProperties.getMaxRemoteRecoveryAttempts();
+ PersistentLocalResourceRepository resourceRepository = (PersistentLocalResourceRepository) runtimeContext
+ .getLocalResourceRepository();
+ IDatasetLifecycleManager datasetLifeCycleManager = runtimeContext.getDatasetLifecycleManager();
+
+ failbackRecoveryReplicas = new HashMap<>();
+ while (true) {
+ //start recovery steps
+ try {
+ 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 perform remote recovery");
+ }
+
+ //2. clean any memory data that could've existed from previous failed recovery attempt
+ datasetLifeCycleManager.closeAllDatasets();
+
+ //3. remove any existing storage data and initialize storage metadata
+ resourceRepository.deleteStorageData(true);
+ resourceRepository.initializeNewUniverse(AsterixClusterProperties.INSTANCE.getStorageDirectoryName());
+
+ //4. select remote replicas to recover from per lost replica data
+ failbackRecoveryReplicas = constructRemoteRecoveryPlan();
+
+ /*** Start Recovery Per Lost Replica ***/
+ for (Entry<String, Set<String>> remoteReplica : failbackRecoveryReplicas.entrySet()) {
+ String replicaId = remoteReplica.getKey();
+ Set<String> partitionsToRecover = remoteReplica.getValue();
+
+ //1. Request indexes metadata and LSM components
+ replicationManager.requestReplicaFiles(replicaId, partitionsToRecover, new HashSet<String>());
+ }
+ break;
+ } catch (IOException e) {
+ e.printStackTrace();
+ LOGGER.log(Level.WARNING, "Failed during remote recovery. Attempting again...");
+ maxRecoveryAttempts--;
+ }
+ }
+ }
+
+ @Override
+ public void completeFailbackProcess() throws IOException {
+ ILogManager logManager = runtimeContext.getTransactionSubsystem().getLogManager();
+ ReplicaResourcesManager replicaResourcesManager = (ReplicaResourcesManager) runtimeContext
+ .getReplicaResourcesManager();
+ Map<String, ClusterPartition[]> nodePartitions = ((IAsterixPropertiesProvider) runtimeContext)
+ .getMetadataProperties().getNodePartitions();
+
+ /**
+ * for each lost partition, get the remaining files from replicas
+ * to complete the failback process.
+ */
+ try {
+ for (Entry<String, Set<String>> remoteReplica : failbackRecoveryReplicas.entrySet()) {
+ String replicaId = remoteReplica.getKey();
+ Set<String> NCsDataToRecover = remoteReplica.getValue();
+ Set<String> existingFiles = new HashSet<>();
+ for (String nodeId : NCsDataToRecover) {
+ //get partitions that will be recovered from this node
+ ClusterPartition[] replicaPartitions = nodePartitions.get(nodeId);
+ for (ClusterPartition partition : replicaPartitions) {
+ existingFiles.addAll(
+ replicaResourcesManager.getPartitionIndexesFiles(partition.getPartitionId(), true));
+ }
+ }
+
+ //Request remaining indexes files
+ replicationManager.requestReplicaFiles(replicaId, NCsDataToRecover, existingFiles);
+ }
+ } catch (IOException e) {
+ /**
+ * in case of failure during failback completion process we need to construct a new plan
+ * and get all the files from the start since the remote replicas will change in the new plan.
+ */
+ e.printStackTrace();
+ startFailbackProcess();
+ }
+
+ //get max LSN from selected remote replicas
+ long maxRemoteLSN = replicationManager.getMaxRemoteLSN(failbackRecoveryReplicas.keySet());
+
+ //6. force LogManager to start from a partition > maxLSN in selected remote replicas
+ logManager.renewLogFilesAndStartFromLSN(maxRemoteLSN);
+
+ //start replication service after failback completed
+ runtimeContext.getReplicationChannel().start();
+ runtimeContext.getReplicationManager().startReplicationThreads();
+
+ failbackRecoveryReplicas = null;
}
}
\ No newline at end of file
diff --git a/asterix-replication/src/main/java/org/apache/asterix/replication/storage/LSMIndexFileProperties.java b/asterix-replication/src/main/java/org/apache/asterix/replication/storage/LSMIndexFileProperties.java
index 890d3a2..031aeb6 100644
--- a/asterix-replication/src/main/java/org/apache/asterix/replication/storage/LSMIndexFileProperties.java
+++ b/asterix-replication/src/main/java/org/apache/asterix/replication/storage/LSMIndexFileProperties.java
@@ -69,11 +69,7 @@
this.fileName = tokens[arraySize - 1];
this.idxName = tokens[arraySize - 2];
this.dataverse = tokens[arraySize - 3];
- this.partition = getPartitonNumFromName(tokens[arraySize - 4]);
- }
-
- private static int getPartitonNumFromName(String name) {
- return Integer.parseInt(name.substring(StoragePathUtil.PARTITION_DIR_PREFIX.length()));
+ this.partition = StoragePathUtil.getPartitonNumFromName(tokens[arraySize - 4]);
}
public void serialize(OutputStream out) throws IOException {
@@ -114,10 +110,6 @@
return nodeId;
}
- public void setNodeId(String nodeId) {
- this.nodeId = nodeId;
- }
-
public String getDataverse() {
return dataverse;
}
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
index b9f7506..55d442d 100644
--- 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
@@ -34,7 +34,6 @@
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;
@@ -55,7 +54,6 @@
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;
private final PersistentLocalResourceRepository localRepository;
private final Map<String, ClusterPartition[]> nodePartitions;
@@ -63,7 +61,6 @@
AsterixMetadataProperties metadataProperties) {
this.localRepository = (PersistentLocalResourceRepository) localRepository;
nodePartitions = metadataProperties.getNodePartitions();
- lastMinRemoteLSN = new AtomicLong(-1);
}
public void deleteIndexFile(LSMIndexFileProperties afp) {
@@ -126,7 +123,6 @@
//update map on disk
updateReplicaIndexLSNMap(lsmComponentProperties.getReplicaComponentPath(this), lsnMap);
-
}
public Set<File> getReplicaIndexes(String replicaId) {
@@ -139,35 +135,7 @@
}
@Override
- public long getMinRemoteLSN(Set<String> replicaIds) {
- if (lastMinRemoteLSN.get() != -1) {
- return lastMinRemoteLSN.get();
- }
- long minRemoteLSN = Long.MAX_VALUE;
- for (String replica : replicaIds) {
- //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;
- }
- }
- }
- lastMinRemoteLSN.set(minRemoteLSN);
- return minRemoteLSN;
- }
-
- @Override
- public long getPartitionsMinLSN(Integer[] partitions) {
+ public long getPartitionsMinLSN(Set<Integer> partitions) {
long minRemoteLSN = Long.MAX_VALUE;
for (Integer partition : partitions) {
//for every index in replica
@@ -219,7 +187,6 @@
remoteIndexMaxLSN = Math.max(remoteIndexMaxLSN, lsn);
}
}
-
return remoteIndexMaxLSN;
}
@@ -271,7 +238,6 @@
ObjectOutputStream oosToFos = new ObjectOutputStream(fos)) {
oosToFos.writeObject(lsnMap);
oosToFos.flush();
- lastMinRemoteLSN.set(-1);
}
}
@@ -293,7 +259,9 @@
File[] indexFileList = dataverseFile.listFiles();
if (indexFileList != null) {
for (File indexFile : indexFileList) {
- partitionIndexes.add(indexFile);
+ if (indexFile.isDirectory()) {
+ partitionIndexes.add(indexFile);
+ }
}
}
}
@@ -307,7 +275,7 @@
* @param partition
* @return Absolute paths to all partition files
*/
- public List<String> getPartitionIndexesFiles(int partition) {
+ public List<String> getPartitionIndexesFiles(int partition, boolean relativePath) {
List<String> partitionFiles = new ArrayList<String>();
Set<File> partitionIndexes = getPartitionIndexes(partition);
for (File indexDir : partitionIndexes) {
@@ -315,7 +283,12 @@
File[] indexFiles = indexDir.listFiles(LSM_INDEX_FILES_FILTER);
if (indexFiles != null) {
for (File file : indexFiles) {
- partitionFiles.add(file.getAbsolutePath());
+ if (!relativePath) {
+ partitionFiles.add(file.getAbsolutePath());
+ } else {
+ partitionFiles.add(
+ PersistentLocalResourceRepository.getResourceRelativePath(file.getAbsolutePath()));
+ }
}
}
}
@@ -324,18 +297,21 @@
}
private static final FilenameFilter LSM_COMPONENTS_MASKS_FILTER = new FilenameFilter() {
+ @Override
public boolean accept(File dir, String name) {
return name.endsWith(LSM_COMPONENT_MASK_SUFFIX);
}
};
private static final FilenameFilter LSM_COMPONENTS_NON_MASKS_FILTER = new FilenameFilter() {
+ @Override
public boolean accept(File dir, String name) {
return !name.endsWith(LSM_COMPONENT_MASK_SUFFIX);
}
};
private static final FilenameFilter LSM_INDEX_FILES_FILTER = new FilenameFilter() {
+ @Override
public boolean accept(File dir, String name) {
return name.equalsIgnoreCase(PersistentLocalResourceRepository.METADATA_FILE_NAME) || !name.startsWith(".");
}
diff --git a/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/AsterixLSMPrimaryUpsertOperatorNodePushable.java b/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/AsterixLSMPrimaryUpsertOperatorNodePushable.java
index f35f4d6..97683d5 100644
--- a/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/AsterixLSMPrimaryUpsertOperatorNodePushable.java
+++ b/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/AsterixLSMPrimaryUpsertOperatorNodePushable.java
@@ -75,7 +75,7 @@
public AsterixLSMPrimaryUpsertOperatorNodePushable(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx,
int partition, int[] fieldPermutation, IRecordDescriptorProvider recordDescProvider, int numOfPrimaryKeys,
- ARecordType recordType, int filterFieldIndex) throws HyracksDataException {
+ ARecordType recordType, int filterFieldIndex) {
super(opDesc, ctx, partition, fieldPermutation, recordDescProvider, IndexOperation.UPSERT);
// initialize nullWriter
this.nullWriter = opDesc.getNullWriterFactory().createNullWriter();
@@ -125,7 +125,8 @@
dos = tb.getDataOutput();
appender = new FrameTupleAppender(new VSizeFrame(ctx), true);
modCallback = opDesc.getModificationOpCallbackFactory().createModificationOperationCallback(
- indexHelper.getResourcePath(), indexHelper.getResourceID(), lsmIndex, ctx);
+ indexHelper.getResourcePath(), indexHelper.getResourceID(), indexHelper.getResourcePartition(),
+ lsmIndex, ctx);
indexAccessor = lsmIndex.createAccessor(modCallback, opDesc.getSearchOpCallbackFactory()
.createSearchOperationCallback(indexHelper.getResourceID(), ctx));
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 3b5630f..65c9a49 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
@@ -45,7 +45,7 @@
protected AbstractIndexModificationOperationCallback(int datasetId, int[] primaryKeyFields,
ITransactionContext txnCtx, ILockManager lockManager, ITransactionSubsystem txnSubsystem, long resourceId,
- byte resourceType, IndexOperation indexOp) {
+ int resourcePartition, byte resourceType, IndexOperation indexOp) {
super(datasetId, primaryKeyFields, txnCtx, lockManager);
this.resourceId = resourceId;
this.resourceType = resourceType;
@@ -58,8 +58,8 @@
logRecord.setJobId(txnCtx.getJobId().getId());
logRecord.setDatasetId(datasetId);
logRecord.setResourceId(resourceId);
+ logRecord.setResourcePartition(resourcePartition);
logRecord.setNewOp((byte) (indexOp.ordinal()));
- logRecord.setNodeId(txnSubsystem.getId());
}
protected void log(int PKHash, ITupleReference newValue) throws ACIDException {
diff --git a/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/PrimaryIndexModificationOperationCallback.java b/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/PrimaryIndexModificationOperationCallback.java
index 3c34153..780f294 100644
--- a/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/PrimaryIndexModificationOperationCallback.java
+++ b/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/PrimaryIndexModificationOperationCallback.java
@@ -37,9 +37,10 @@
implements IModificationOperationCallback {
public PrimaryIndexModificationOperationCallback(int datasetId, int[] primaryKeyFields, ITransactionContext txnCtx,
- ILockManager lockManager, ITransactionSubsystem txnSubsystem, long resourceId, byte resourceType,
- IndexOperation indexOp) {
- super(datasetId, primaryKeyFields, txnCtx, lockManager, txnSubsystem, resourceId, resourceType, indexOp);
+ ILockManager lockManager, ITransactionSubsystem txnSubsystem, long resourceId, int resourcePartition,
+ byte resourceType, IndexOperation indexOp) {
+ super(datasetId, primaryKeyFields, txnCtx, lockManager, txnSubsystem, resourceId, resourcePartition,
+ resourceType, indexOp);
}
@Override
diff --git a/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/PrimaryIndexModificationOperationCallbackFactory.java b/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/PrimaryIndexModificationOperationCallbackFactory.java
index 794f867..db68b26 100644
--- a/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/PrimaryIndexModificationOperationCallbackFactory.java
+++ b/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/PrimaryIndexModificationOperationCallbackFactory.java
@@ -37,8 +37,8 @@
/**
* Assumes LSM-BTrees as primary indexes.
*/
-public class PrimaryIndexModificationOperationCallbackFactory extends AbstractOperationCallbackFactory implements
- IModificationOperationCallbackFactory {
+public class PrimaryIndexModificationOperationCallbackFactory extends AbstractOperationCallbackFactory
+ implements IModificationOperationCallbackFactory {
private static final long serialVersionUID = 1L;
private final IndexOperation indexOp;
@@ -51,7 +51,7 @@
@Override
public IModificationOperationCallback createModificationOperationCallback(String resourcePath, long resourceId,
- Object resource, IHyracksTaskContext ctx) throws HyracksDataException {
+ int resourcePartition, Object resource, IHyracksTaskContext ctx) throws HyracksDataException {
ITransactionSubsystem txnSubsystem = txnSubsystemProvider.getTransactionSubsystem(ctx);
IIndexLifecycleManager indexLifeCycleManager = txnSubsystem.getAsterixAppRuntimeContextProvider()
@@ -64,8 +64,8 @@
try {
ITransactionContext txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(jobId, false);
IModificationOperationCallback modCallback = new PrimaryIndexModificationOperationCallback(datasetId,
- primaryKeyFields, txnCtx, txnSubsystem.getLockManager(), txnSubsystem, resourceId, resourceType,
- indexOp);
+ primaryKeyFields, txnCtx, txnSubsystem.getLockManager(), txnSubsystem, resourceId,
+ resourcePartition, resourceType, indexOp);
txnCtx.registerIndexAndCallback(resourceId, index, (AbstractOperationCallback) modCallback, true);
return modCallback;
} catch (ACIDException e) {
diff --git a/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/SecondaryIndexModificationOperationCallback.java b/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/SecondaryIndexModificationOperationCallback.java
index 250e28d..8044d90 100644
--- a/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/SecondaryIndexModificationOperationCallback.java
+++ b/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/SecondaryIndexModificationOperationCallback.java
@@ -40,8 +40,9 @@
public SecondaryIndexModificationOperationCallback(int datasetId, int[] primaryKeyFields,
ITransactionContext txnCtx, ILockManager lockManager, ITransactionSubsystem txnSubsystem, long resourceId,
- byte resourceType, IndexOperation indexOp) {
- super(datasetId, primaryKeyFields, txnCtx, lockManager, txnSubsystem, resourceId, resourceType, indexOp);
+ int resourcePartition, byte resourceType, IndexOperation indexOp) {
+ super(datasetId, primaryKeyFields, txnCtx, lockManager, txnSubsystem, resourceId, resourcePartition,
+ resourceType, indexOp);
oldOp = (indexOp == IndexOperation.DELETE) ? IndexOperation.INSERT : IndexOperation.DELETE;
}
diff --git a/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/SecondaryIndexModificationOperationCallbackFactory.java b/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/SecondaryIndexModificationOperationCallbackFactory.java
index 06a1957..ef2b498 100644
--- a/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/SecondaryIndexModificationOperationCallbackFactory.java
+++ b/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/SecondaryIndexModificationOperationCallbackFactory.java
@@ -34,8 +34,8 @@
import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
-public class SecondaryIndexModificationOperationCallbackFactory extends AbstractOperationCallbackFactory implements
- IModificationOperationCallbackFactory {
+public class SecondaryIndexModificationOperationCallbackFactory extends AbstractOperationCallbackFactory
+ implements IModificationOperationCallbackFactory {
private static final long serialVersionUID = 1L;
private final IndexOperation indexOp;
@@ -48,7 +48,7 @@
@Override
public IModificationOperationCallback createModificationOperationCallback(String resourcePath, long resourceId,
- Object resource, IHyracksTaskContext ctx) throws HyracksDataException {
+ int resourcePartition, Object resource, IHyracksTaskContext ctx) throws HyracksDataException {
ITransactionSubsystem txnSubsystem = txnSubsystemProvider.getTransactionSubsystem(ctx);
IIndexLifecycleManager indexLifeCycleManager = txnSubsystem.getAsterixAppRuntimeContextProvider()
.getDatasetLifecycleManager();
@@ -60,8 +60,8 @@
try {
ITransactionContext txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(jobId, false);
IModificationOperationCallback modCallback = new SecondaryIndexModificationOperationCallback(datasetId,
- primaryKeyFields, txnCtx, txnSubsystem.getLockManager(), txnSubsystem, resourceId, resourceType,
- indexOp);
+ primaryKeyFields, txnCtx, txnSubsystem.getLockManager(), txnSubsystem, resourceId,
+ resourcePartition, resourceType, indexOp);
txnCtx.registerIndexAndCallback(resourceId, index, (AbstractOperationCallback) modCallback, false);
return modCallback;
} catch (ACIDException e) {
diff --git a/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/TempDatasetIndexModificationOperationCallback.java b/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/TempDatasetIndexModificationOperationCallback.java
index 69aad24..32d3461 100644
--- a/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/TempDatasetIndexModificationOperationCallback.java
+++ b/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/TempDatasetIndexModificationOperationCallback.java
@@ -39,8 +39,9 @@
public TempDatasetIndexModificationOperationCallback(int datasetId, int[] primaryKeyFields,
ITransactionContext txnCtx, ILockManager lockManager, ITransactionSubsystem txnSubsystem, long resourceId,
- byte resourceType, IndexOperation indexOp) {
- super(datasetId, primaryKeyFields, txnCtx, lockManager, txnSubsystem, resourceId, resourceType, indexOp);
+ int resourcePartition, byte resourceType, IndexOperation indexOp) {
+ super(datasetId, primaryKeyFields, txnCtx, lockManager, txnSubsystem, resourceId, resourcePartition,
+ resourceType, indexOp);
}
@Override
diff --git a/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/TempDatasetPrimaryIndexModificationOperationCallbackFactory.java b/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/TempDatasetPrimaryIndexModificationOperationCallbackFactory.java
index f2a6820..b08798c 100644
--- a/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/TempDatasetPrimaryIndexModificationOperationCallbackFactory.java
+++ b/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/TempDatasetPrimaryIndexModificationOperationCallbackFactory.java
@@ -49,7 +49,7 @@
@Override
public IModificationOperationCallback createModificationOperationCallback(String resourcePath, long resourceId,
- Object resource, IHyracksTaskContext ctx) throws HyracksDataException {
+ int resourcePartition, Object resource, IHyracksTaskContext ctx) throws HyracksDataException {
ITransactionSubsystem txnSubsystem = txnSubsystemProvider.getTransactionSubsystem(ctx);
IIndexLifecycleManager indexLifeCycleManager = txnSubsystem.getAsterixAppRuntimeContextProvider()
.getDatasetLifecycleManager();
@@ -61,8 +61,8 @@
try {
ITransactionContext txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(jobId, false);
IModificationOperationCallback modCallback = new TempDatasetIndexModificationOperationCallback(datasetId,
- primaryKeyFields, txnCtx, txnSubsystem.getLockManager(), txnSubsystem, resourceId, resourceType,
- indexOp);
+ primaryKeyFields, txnCtx, txnSubsystem.getLockManager(), txnSubsystem, resourceId,
+ resourcePartition, resourceType, indexOp);
txnCtx.registerIndexAndCallback(resourceId, index, (AbstractOperationCallback) modCallback, true);
return modCallback;
} catch (ACIDException e) {
diff --git a/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/TempDatasetSecondaryIndexModificationOperationCallbackFactory.java b/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/TempDatasetSecondaryIndexModificationOperationCallbackFactory.java
index 8d838a3..403d68d 100644
--- a/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/TempDatasetSecondaryIndexModificationOperationCallbackFactory.java
+++ b/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/TempDatasetSecondaryIndexModificationOperationCallbackFactory.java
@@ -49,7 +49,7 @@
@Override
public IModificationOperationCallback createModificationOperationCallback(String resourcePath, long resourceId,
- Object resource, IHyracksTaskContext ctx) throws HyracksDataException {
+ int resourcePartition, Object resource, IHyracksTaskContext ctx) throws HyracksDataException {
ITransactionSubsystem txnSubsystem = txnSubsystemProvider.getTransactionSubsystem(ctx);
IIndexLifecycleManager indexLifeCycleManager = txnSubsystem.getAsterixAppRuntimeContextProvider()
.getDatasetLifecycleManager();
@@ -61,8 +61,8 @@
try {
ITransactionContext txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(jobId, false);
IModificationOperationCallback modCallback = new TempDatasetIndexModificationOperationCallback(datasetId,
- primaryKeyFields, txnCtx, txnSubsystem.getLockManager(), txnSubsystem, resourceId, resourceType,
- indexOp);
+ primaryKeyFields, txnCtx, txnSubsystem.getLockManager(), txnSubsystem, resourceId,
+ resourcePartition, resourceType, indexOp);
txnCtx.registerIndexAndCallback(resourceId, index, (AbstractOperationCallback) modCallback, false);
return modCallback;
} catch (ACIDException e) {
diff --git a/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/UpsertOperationCallback.java b/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/UpsertOperationCallback.java
index dfc622a..f98083a 100644
--- a/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/UpsertOperationCallback.java
+++ b/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/UpsertOperationCallback.java
@@ -31,9 +31,10 @@
implements IModificationOperationCallback {
public UpsertOperationCallback(int datasetId, int[] primaryKeyFields, ITransactionContext txnCtx,
- ILockManager lockManager, ITransactionSubsystem txnSubsystem, long resourceId, byte resourceType,
- IndexOperation indexOp) {
- super(datasetId, primaryKeyFields, txnCtx, lockManager, txnSubsystem, resourceId, resourceType, indexOp);
+ ILockManager lockManager, ITransactionSubsystem txnSubsystem, long resourceId, int resourcePartition,
+ byte resourceType, IndexOperation indexOp) {
+ super(datasetId, primaryKeyFields, txnCtx, lockManager, txnSubsystem, resourceId, resourcePartition,
+ resourceType, indexOp);
}
@Override
diff --git a/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/UpsertOperationCallbackFactory.java b/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/UpsertOperationCallbackFactory.java
index 0c83ab5..707f986 100644
--- a/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/UpsertOperationCallbackFactory.java
+++ b/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/UpsertOperationCallbackFactory.java
@@ -47,7 +47,7 @@
@Override
public IModificationOperationCallback createModificationOperationCallback(String resourceName, long resourceId,
- Object resource, IHyracksTaskContext ctx) throws HyracksDataException {
+ int resourcePartition, Object resource, IHyracksTaskContext ctx) throws HyracksDataException {
ITransactionSubsystem txnSubsystem = txnSubsystemProvider.getTransactionSubsystem(ctx);
IIndexLifecycleManager indexLifeCycleManager = txnSubsystem.getAsterixAppRuntimeContextProvider()
@@ -60,7 +60,8 @@
try {
ITransactionContext txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(jobId, false);
IModificationOperationCallback modCallback = new UpsertOperationCallback(datasetId, primaryKeyFields,
- txnCtx, txnSubsystem.getLockManager(), txnSubsystem, resourceId, resourceType, indexOp);
+ txnCtx, txnSubsystem.getLockManager(), txnSubsystem, resourceId, resourcePartition, resourceType,
+ indexOp);
txnCtx.registerIndexAndCallback(resourceId, index, (AbstractOperationCallback) modCallback, true);
return modCallback;
} catch (ACIDException e) {
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 655fd2a..9cb456f 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
@@ -28,11 +28,13 @@
import java.nio.file.Files;
import java.nio.file.Path;
import java.nio.file.Paths;
+import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
import java.util.SortedMap;
+import java.util.concurrent.ConcurrentHashMap;
import java.util.logging.Level;
import java.util.logging.Logger;
@@ -40,6 +42,7 @@
import org.apache.asterix.common.config.AsterixMetadataProperties;
import org.apache.asterix.common.replication.AsterixReplicationJob;
import org.apache.asterix.common.replication.IReplicationManager;
+import org.apache.asterix.common.utils.StoragePathUtil;
import org.apache.commons.io.FileUtils;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.api.io.IODeviceHandle;
@@ -67,6 +70,9 @@
private boolean isReplicationEnabled = false;
private Set<String> filesToBeReplicated;
private final SortedMap<Integer, ClusterPartition> clusterPartitions;
+ private final Set<Integer> nodeOriginalPartitions;
+ private final Set<Integer> nodeActivePartitions;
+ private Set<Integer> nodeInactivePartitions;
public PersistentLocalResourceRepository(List<IODeviceHandle> devices, String nodeId,
AsterixMetadataProperties metadataProperties) throws HyracksDataException {
@@ -86,6 +92,15 @@
}
}
resourceCache = CacheBuilder.newBuilder().maximumSize(MAX_CACHED_RESOURCES).build();
+
+ ClusterPartition[] nodePartitions = metadataProperties.getNodePartitions().get(nodeId);
+ //initially the node active partitions are the same as the original partitions
+ nodeOriginalPartitions = new HashSet<>(nodePartitions.length);
+ nodeActivePartitions = new HashSet<>(nodePartitions.length);
+ for (ClusterPartition partition : nodePartitions) {
+ nodeOriginalPartitions.add(partition.getPartitionId());
+ nodeActivePartitions.add(partition.getPartitionId());
+ }
}
private static String getStorageMetadataDirPath(String mountPoint, String nodeId, int ioDeviceId) {
@@ -301,6 +316,7 @@
}
private static final FilenameFilter METADATA_FILES_FILTER = new FilenameFilter() {
+ @Override
public boolean accept(File dir, String name) {
if (name.equalsIgnoreCase(METADATA_FILE_NAME)) {
return true;
@@ -316,6 +332,7 @@
if (isReplicationEnabled) {
filesToBeReplicated = new HashSet<String>();
+ nodeInactivePartitions = ConcurrentHashMap.newKeySet();
}
}
@@ -404,4 +421,43 @@
//currently each partition is replicated on the same IO device number on all NCs.
return mountPoints[clusterPartitions.get(partition).getIODeviceNum()];
}
-}
+
+ public Set<Integer> getActivePartitions() {
+ return Collections.unmodifiableSet(nodeActivePartitions);
+ }
+
+ public Set<Integer> getInactivePartitions() {
+ return Collections.unmodifiableSet(nodeInactivePartitions);
+ }
+
+ public Set<Integer> getNodeOrignalPartitions() {
+ return Collections.unmodifiableSet(nodeOriginalPartitions);
+ }
+
+ public synchronized void addActivePartition(int partitonId) {
+ nodeActivePartitions.add(partitonId);
+ nodeInactivePartitions.remove(partitonId);
+ }
+
+ public synchronized void addInactivePartition(int partitonId) {
+ nodeInactivePartitions.add(partitonId);
+ nodeActivePartitions.remove(partitonId);
+ }
+
+ /**
+ * @param resourceAbsolutePath
+ * @return the resource relative path starting from the partition directory
+ */
+ public static String getResourceRelativePath(String resourceAbsolutePath) {
+ String[] tokens = resourceAbsolutePath.split(File.separator);
+ //partiton/dataverse/idx/fileName
+ return tokens[tokens.length - 4] + File.separator + tokens[tokens.length - 3] + File.separator
+ + tokens[tokens.length - 2] + File.separator + tokens[tokens.length - 1];
+ }
+
+ public static int getResourcePartition(String resourceAbsolutePath) {
+ String[] tokens = resourceAbsolutePath.split(File.separator);
+ //partiton/dataverse/idx/fileName
+ return StoragePathUtil.getPartitonNumFromName(tokens[tokens.length - 4]);
+ }
+}
\ No newline at end of file
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
index 1966c39..5649710 100644
--- 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
@@ -41,6 +41,9 @@
throw new IllegalStateException();
}
+ //only locally generated logs should be replicated
+ logRecord.setReplicated(logRecord.getLogSource() == LogSource.LOCAL);
+
//Remote flush logs do not need to be flushed separately since they may not trigger local flush
if (logRecord.getLogType() == LogType.FLUSH && logRecord.getLogSource() == LogSource.LOCAL) {
flushLogsQ.offer(logRecord);
@@ -54,7 +57,7 @@
protected void appendToLogTail(ILogRecord logRecord) throws ACIDException {
syncAppendToLogTail(logRecord);
- if (logRecord.getLogSource() == LogSource.LOCAL) {
+ if (logRecord.isReplicated()) {
replicationManager.replicateLog(logRecord);
}
diff --git a/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogReader.java b/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogReader.java
index e0cddee..a018dc2 100644
--- a/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogReader.java
+++ b/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogReader.java
@@ -26,11 +26,10 @@
import org.apache.asterix.common.exceptions.ACIDException;
import org.apache.asterix.common.transactions.ILogReader;
import org.apache.asterix.common.transactions.ILogRecord;
+import org.apache.asterix.common.transactions.ILogRecord.RECORD_STATUS;
import org.apache.asterix.common.transactions.LogRecord;
import org.apache.asterix.common.transactions.MutableLong;
-import static org.apache.asterix.common.transactions.LogRecord.*;
-
/**
* NOTE: Many method calls of this class are not thread safe.
* Be very cautious using it in a multithreaded context.
diff --git a/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/RemoteLogReader.java b/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/RemoteLogReader.java
new file mode 100644
index 0000000..8f88321
--- /dev/null
+++ b/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/RemoteLogReader.java
@@ -0,0 +1,138 @@
+/*
+ * 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 java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+
+import org.apache.asterix.common.exceptions.ACIDException;
+import org.apache.asterix.common.transactions.ILogReader;
+import org.apache.asterix.common.transactions.ILogRecord;
+import org.apache.asterix.common.transactions.ILogRecord.RECORD_STATUS;
+import org.apache.asterix.common.transactions.LogRecord;
+
+public class RemoteLogReader implements ILogReader {
+
+ private final FileChannel fileChannel;
+ private final ILogRecord logRecord;
+ private final ByteBuffer readBuffer;
+ private long readLSN;
+ private final int logPageSize;
+
+ public RemoteLogReader(FileChannel fileChannel, long logFileSize, int logPageSize) {
+ this.fileChannel = fileChannel;
+ this.logPageSize = logPageSize;
+ logRecord = new LogRecord();
+ readBuffer = ByteBuffer.allocate(logPageSize);
+ }
+
+ @Override
+ public void initializeScan(long beginLSN) throws ACIDException {
+ readLSN = beginLSN;
+ fillLogReadBuffer();
+ }
+
+ private boolean fillLogReadBuffer() throws ACIDException {
+ int size = 0;
+ int read = 0;
+ readBuffer.position(0);
+ readBuffer.limit(logPageSize);
+ try {
+ fileChannel.position(readLSN);
+ //We loop here because read() may return 0, but this simply means we are waiting on IO.
+ //Therefore we want to break out only when either the buffer is full, or we reach EOF.
+ while (size < logPageSize && read != -1) {
+ read = fileChannel.read(readBuffer);
+ if (read > 0) {
+ size += read;
+ }
+ }
+ } catch (IOException e) {
+ throw new ACIDException(e);
+ }
+ readBuffer.position(0);
+ readBuffer.limit(size);
+ if (size == 0 && read == -1) {
+ return false; //EOF
+ }
+ return true;
+ }
+
+ @Override
+ public ILogRecord read(long LSN) throws ACIDException {
+ throw new UnsupportedOperationException("Random read is not supported.");
+ }
+
+ @Override
+ public ILogRecord next() throws ACIDException {
+ if (readBuffer.position() == readBuffer.limit()) {
+ boolean hasRemaining = fillLogReadBuffer();
+ if (!hasRemaining) {
+ return null;
+ }
+ }
+
+ RECORD_STATUS status = logRecord.readRemoteLog(readBuffer, true);
+ switch (status) {
+ case TRUNCATED: {
+ //we may have just read off the end of the buffer, so try refiling it
+ if (!fillLogReadBuffer()) {
+ return null;
+ }
+ //now see what we have in the refilled buffer
+ status = logRecord.readRemoteLog(readBuffer, true);
+ switch (status) {
+ case TRUNCATED: {
+ return null;
+ }
+ case OK:
+ break;
+ default:
+ break;
+ }
+ //if we have exited the inner switch,
+ // this means status is really "OK" after buffer refill
+ break;
+ }
+ case BAD_CHKSUM: {
+ return null;
+ }
+ case OK:
+ break;
+ }
+
+ readLSN += logRecord.getSerializedLogSize();
+ return logRecord;
+ }
+
+ @Override
+ public void close() throws ACIDException {
+ try {
+ if (fileChannel != null) {
+ if (fileChannel.isOpen()) {
+ fileChannel.close();
+ }
+ }
+ } catch (IOException e) {
+ throw new ACIDException(e);
+ }
+ }
+
+}
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 11dc282..4a17541 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,19 +49,18 @@
import org.apache.asterix.common.api.IDatasetLifecycleManager;
import org.apache.asterix.common.api.ILocalResourceMetadata;
+import org.apache.asterix.common.cluster.ClusterPartition;
+import org.apache.asterix.common.config.AsterixMetadataProperties;
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;
import org.apache.asterix.common.transactions.ILogRecord;
import org.apache.asterix.common.transactions.IRecoveryManager;
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.LogType;
import org.apache.asterix.transaction.management.resource.PersistentLocalResourceRepository;
import org.apache.asterix.transaction.management.service.logging.LogManager;
@@ -71,25 +70,21 @@
import org.apache.commons.io.FileUtils;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.api.lifecycle.ILifeCycleComponent;
-import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
import org.apache.hyracks.storage.am.common.api.IIndex;
import org.apache.hyracks.storage.am.common.impls.NoOpOperationCallback;
import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndex;
-import org.apache.hyracks.storage.common.file.ILocalResourceRepository;
import org.apache.hyracks.storage.common.file.LocalResource;
/**
* This is the Recovery Manager and is responsible for rolling back a
- * transaction as well as doing a system recovery. TODO: Crash Recovery logic is
- * not in place completely. Once we have physical logging implemented, we would
- * add support for crash recovery.
+ * transaction as well as doing a system recovery.
*/
public class RecoveryManager implements IRecoveryManager, ILifeCycleComponent {
- public static final boolean IS_DEBUG_MODE = false;//true
+ public static final boolean IS_DEBUG_MODE = false;
private static final Logger LOGGER = Logger.getLogger(RecoveryManager.class.getName());
private final TransactionSubsystem txnSubsystem;
private final LogManager logMgr;
@@ -101,6 +96,7 @@
private static final long MEGABYTE = 1024L * 1024L;
private Map<Integer, JobEntityCommits> jobId2WinnerEntitiesMap = null;
private static final long MAX_CACHED_ENTITY_COMMITS_PER_JOB_SIZE = 4 * MEGABYTE; //4MB;
+ private final PersistentLocalResourceRepository localResourceRepository;
/**
* A file at a known location that contains the LSN of the last log record
@@ -111,11 +107,13 @@
public RecoveryManager(TransactionSubsystem txnSubsystem) {
this.txnSubsystem = txnSubsystem;
- this.logMgr = (LogManager) txnSubsystem.getLogManager();
- this.checkpointHistory = this.txnSubsystem.getTransactionProperties().getCheckpointHistory();
+ logMgr = (LogManager) txnSubsystem.getLogManager();
+ checkpointHistory = txnSubsystem.getTransactionProperties().getCheckpointHistory();
IAsterixPropertiesProvider propertiesProvider = (IAsterixPropertiesProvider) txnSubsystem
.getAsterixAppRuntimeContextProvider().getAppContext();
- this.replicationEnabled = propertiesProvider.getReplicationProperties().isReplicationEnabled();
+ replicationEnabled = propertiesProvider.getReplicationProperties().isReplicationEnabled();
+ localResourceRepository = (PersistentLocalResourceRepository) txnSubsystem.getAsterixAppRuntimeContextProvider()
+ .getLocalResourceRepository();
}
/**
@@ -160,12 +158,8 @@
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.
- }
+ LOGGER.warning("Some(or all) of transaction log files are lost.");
+ //No choice but continuing when the log files are lost.
}
state = SystemState.HEALTHY;
return state;
@@ -180,73 +174,82 @@
}
}
- //This method is used only when replication is disabled. Therefore, there is no need to check logs node ids
+ //This method is used only when replication is disabled.
@Override
public void startRecovery(boolean synchronous) throws IOException, ACIDException {
- //delete any recovery files from previous failed recovery attempts
- deleteRecoveryTemporaryFiles();
-
- int updateLogCount = 0;
- int entityCommitLogCount = 0;
- int jobCommitLogCount = 0;
- int redoCount = 0;
- int abortLogCount = 0;
- int jobId = -1;
-
state = SystemState.RECOVERING;
- LOGGER.log(Level.INFO, "[RecoveryMgr] starting recovery ...");
+ LOGGER.log(Level.INFO, "starting recovery ...");
- Set<Integer> winnerJobSet = new HashSet<Integer>();
- jobId2WinnerEntitiesMap = new HashMap<>();
-
- TxnId tempKeyTxnId = new TxnId(-1, -1, -1, null, -1, false);
- JobEntityCommits jobEntityWinners = null;
- //#. read checkpoint file and set lowWaterMark where anaylsis and redo start
long readableSmallestLSN = logMgr.getReadableSmallestLSN();
CheckpointObject checkpointObject = readCheckpoint();
long lowWaterMarkLSN = checkpointObject.getMinMCTFirstLsn();
if (lowWaterMarkLSN < readableSmallestLSN) {
lowWaterMarkLSN = readableSmallestLSN;
}
- int maxJobId = checkpointObject.getMaxJobId();
- //-------------------------------------------------------------------------
- // [ analysis phase ]
- // - collect all committed Lsn
- //-------------------------------------------------------------------------
- LOGGER.log(Level.INFO, "[RecoveryMgr] in analysis phase");
+ //delete any recovery files from previous failed recovery attempts
+ deleteRecoveryTemporaryFiles();
- //#. set log reader to the lowWaterMarkLsn
- ILogReader logReader = logMgr.getLogReader(true);
- ILogRecord logRecord = null;
+ //get active partitions on this node
+ Set<Integer> activePartitions = localResourceRepository.getNodeOrignalPartitions();
+ replayPartitionsLogs(activePartitions, logMgr.getLogReader(true), lowWaterMarkLSN);
+ }
+
+ @Override
+ public synchronized void replayPartitionsLogs(Set<Integer> partitions, ILogReader logReader, long lowWaterMarkLSN)
+ throws IOException, ACIDException {
try {
- logReader.initializeScan(lowWaterMarkLSN);
- logRecord = logReader.next();
- while (logRecord != null) {
- if (IS_DEBUG_MODE) {
- LOGGER.info(logRecord.getLogRecordForDisplay());
- }
- //update max jobId
- if (logRecord.getJobId() > maxJobId) {
- maxJobId = logRecord.getJobId();
- }
- switch (logRecord.getLogType()) {
- case LogType.UPDATE:
+ Set<Integer> winnerJobSet = startRecoverysAnalysisPhase(partitions, logReader, lowWaterMarkLSN);
+ startRecoveryRedoPhase(partitions, logReader, lowWaterMarkLSN, winnerJobSet);
+ } finally {
+ logReader.close();
+ deleteRecoveryTemporaryFiles();
+ }
+ }
+
+ private synchronized Set<Integer> startRecoverysAnalysisPhase(Set<Integer> partitions, ILogReader logReader,
+ long lowWaterMarkLSN) throws IOException, ACIDException {
+ int updateLogCount = 0;
+ int entityCommitLogCount = 0;
+ int jobCommitLogCount = 0;
+ int abortLogCount = 0;
+ int jobId = -1;
+
+ Set<Integer> winnerJobSet = new HashSet<Integer>();
+ jobId2WinnerEntitiesMap = new HashMap<>();
+
+ //set log reader to the lowWaterMarkLsn
+ ILogRecord logRecord = null;
+ logReader.initializeScan(lowWaterMarkLSN);
+
+ //collect all committed Lsn
+ JobEntityCommits jobEntityWinners = null;
+
+ logRecord = logReader.next();
+ while (logRecord != null) {
+ if (IS_DEBUG_MODE) {
+ LOGGER.info(logRecord.getLogRecordForDisplay());
+ }
+ switch (logRecord.getLogType()) {
+ case LogType.UPDATE:
+ if (partitions.contains(logRecord.getResourcePartition())) {
updateLogCount++;
- break;
- case LogType.JOB_COMMIT:
- jobId = logRecord.getJobId();
- winnerJobSet.add(jobId);
- if (jobId2WinnerEntitiesMap.containsKey(jobId)) {
- jobEntityWinners = jobId2WinnerEntitiesMap.get(jobId);
- //to delete any spilled files as well
- jobEntityWinners.clear();
- jobId2WinnerEntitiesMap.remove(jobId);
- }
- jobCommitLogCount++;
- break;
- case LogType.ENTITY_COMMIT:
- case LogType.UPSERT_ENTITY_COMMIT:
+ }
+ break;
+ case LogType.JOB_COMMIT:
+ jobId = logRecord.getJobId();
+ winnerJobSet.add(jobId);
+ if (jobId2WinnerEntitiesMap.containsKey(jobId)) {
+ jobEntityWinners = jobId2WinnerEntitiesMap.get(jobId);
+ //to delete any spilled files as well
+ jobEntityWinners.clear();
+ jobId2WinnerEntitiesMap.remove(jobId);
+ }
+ jobCommitLogCount++;
+ break;
+ case LogType.ENTITY_COMMIT:
+ case LogType.UPSERT_ENTITY_COMMIT:
+ if (partitions.contains(logRecord.getResourcePartition())) {
jobId = logRecord.getJobId();
if (!jobId2WinnerEntitiesMap.containsKey(jobId)) {
jobEntityWinners = new JobEntityCommits(jobId);
@@ -261,885 +264,52 @@
}
jobEntityWinners.add(logRecord);
entityCommitLogCount++;
- break;
- case LogType.ABORT:
- abortLogCount++;
- break;
- case LogType.FLUSH:
- break;
- default:
- throw new ACIDException("Unsupported LogType: " + logRecord.getLogType());
- }
- logRecord = logReader.next();
+ }
+ break;
+ case LogType.ABORT:
+ abortLogCount++;
+ break;
+ case LogType.FLUSH:
+ break;
+ default:
+ throw new ACIDException("Unsupported LogType: " + logRecord.getLogType());
}
-
- //prepare winners for search after analysis is done to flush anything remaining in memory to disk.
- for (JobEntityCommits winners : jobId2WinnerEntitiesMap.values()) {
- winners.prepareForSearch();
- }
- //-------------------------------------------------------------------------
- // [ redo phase ]
- // - redo if
- // 1) The TxnId is committed && --> guarantee durability
- // 2) lsn > maxDiskLastLsn of the index --> guarantee idempotence
- //-------------------------------------------------------------------------
- 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;
-
- IAsterixAppRuntimeContextProvider appRuntimeContext = txnSubsystem.getAsterixAppRuntimeContextProvider();
- //get datasetLifeCycleManager
- IDatasetLifecycleManager datasetLifecycleManager = txnSubsystem.getAsterixAppRuntimeContextProvider()
- .getDatasetLifecycleManager();
- ILocalResourceRepository localResourceRepository = appRuntimeContext.getLocalResourceRepository();
- Map<Long, LocalResource> resourcesMap = ((PersistentLocalResourceRepository) localResourceRepository)
- .loadAndGetAllResources();
- //set log reader to the lowWaterMarkLsn again.
- logReader.initializeScan(lowWaterMarkLSN);
logRecord = logReader.next();
- while (logRecord != null) {
- if (IS_DEBUG_MODE) {
- LOGGER.info(logRecord.getLogRecordForDisplay());
- }
- LSN = logRecord.getLSN();
- jobId = logRecord.getJobId();
- foundWinner = false;
- switch (logRecord.getLogType()) {
- case LogType.UPDATE:
- if (winnerJobSet.contains(jobId)) {
- foundWinner = true;
- } else if (jobId2WinnerEntitiesMap.containsKey(jobId)) {
- jobEntityWinners = jobId2WinnerEntitiesMap.get(jobId);
- tempKeyTxnId.setTxnId(jobId, logRecord.getDatasetId(), logRecord.getPKHashValue(),
- logRecord.getPKValue(), logRecord.getPKValueSize());
- if (jobEntityWinners.containsEntityCommitForTxnId(LSN, 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) {
- logRecord = logReader.next();
- continue;
- }
- /*******************************************************************/
-
- //get index instance from IndexLifeCycleManager
- //if index is not registered into IndexLifeCycleManager,
- //create the index using LocalMetadata stored in LocalResourceRepository
- String resourceAbsolutePath = localResource.getResourcePath();
- index = (ILSMIndex) datasetLifecycleManager.getIndex(resourceAbsolutePath);
- if (index == null) {
- //#. create index instance and register to indexLifeCycleManager
- localResourceMetadata = (ILocalResourceMetadata) localResource.getResourceObject();
- index = localResourceMetadata.createIndexInstance(appRuntimeContext,
- resourceAbsolutePath, localResource.getPartition());
- datasetLifecycleManager.register(resourceAbsolutePath, index);
- datasetLifecycleManager.open(resourceAbsolutePath);
-
- //#. get maxDiskLastLSN
- ILSMIndex lsmIndex = index;
- maxDiskLastLsn = ((AbstractLSMIOOperationCallback) lsmIndex.getIOOperationCallback())
- .getComponentLSN(lsmIndex.getImmutableComponents());
-
- //#. set resourceId and maxDiskLastLSN to the map
- resourceId2MaxLSNMap.put(resourceId, maxDiskLastLsn);
- } else {
- maxDiskLastLsn = resourceId2MaxLSNMap.get(resourceId);
- }
-
- if (LSN > maxDiskLastLsn) {
- redo(logRecord, datasetLifecycleManager);
- redoCount++;
- }
- }
- break;
- case LogType.JOB_COMMIT:
- case LogType.ENTITY_COMMIT:
- case LogType.ABORT:
- case LogType.FLUSH:
- case LogType.UPSERT_ENTITY_COMMIT:
- //do nothing
- break;
- default:
- throw new ACIDException("Unsupported LogType: " + logRecord.getLogType());
- }
- logRecord = logReader.next();
- }
-
- //close all indexes
- Set<Long> resourceIdList = resourceId2MaxLSNMap.keySet();
- for (long r : resourceIdList) {
- datasetLifecycleManager.close(resourcesMap.get(r).getResourcePath());
- }
-
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("[RecoveryMgr] recovery is completed.");
- LOGGER.info("[RecoveryMgr's recovery log count] update/entityCommit/jobCommit/abort/redo = "
- + updateLogCount + "/" + entityCommitLogCount + "/" + jobCommitLogCount + "/" + abortLogCount
- + "/" + redoCount);
- }
- } finally {
- logReader.close();
- //delete any recovery files after completing recovery
- deleteRecoveryTemporaryFiles();
}
+
+ //prepare winners for search after analysis is done to flush anything remaining in memory to disk.
+ for (JobEntityCommits winners : jobId2WinnerEntitiesMap.values()) {
+ winners.prepareForSearch();
+ }
+
+ LOGGER.info("Logs analysis phase completed.");
+ LOGGER.info("Analysis log count update/entityCommit/jobCommit/abort = " + updateLogCount + "/"
+ + entityCommitLogCount + "/" + jobCommitLogCount + "/" + abortLogCount);
+
+ return winnerJobSet;
}
- private static boolean needToFreeMemory() {
- return Runtime.getRuntime().freeMemory() < MAX_CACHED_ENTITY_COMMITS_PER_JOB_SIZE;
- }
-
- @Override
- public void replayRemoteLogs(ArrayList<ILogRecord> remoteLogs) throws HyracksDataException, ACIDException {
- int updateLogCount = 0;
- int entityCommitLogCount = 0;
- int jobCommitLogCount = 0;
+ private synchronized void startRecoveryRedoPhase(Set<Integer> partitions, ILogReader logReader, long lowWaterMarkLSN,
+ Set<Integer> winnerJobSet) throws IOException, ACIDException {
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:
- case LogType.UPSERT_ENTITY_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;
+ JobEntityCommits jobEntityWinners = null;
- //#. get indexLifeCycleManager
IAsterixAppRuntimeContextProvider appRuntimeContext = txnSubsystem.getAsterixAppRuntimeContextProvider();
IDatasetLifecycleManager datasetLifecycleManager = appRuntimeContext.getDatasetLifecycleManager();
- PersistentLocalResourceRepository localResourceRepository = (PersistentLocalResourceRepository) appRuntimeContext
- .getLocalResourceRepository();
- Map<Long, LocalResource> resourcesMap = 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
- //get partition path in this node
- String partitionIODevicePath = localResourceRepository
- .getPartitionPath(localResource.getPartition());
- String resourceAbsolutePath = partitionIODevicePath + File.separator
- + localResource.getResourceName();
- localResource.setResourcePath(resourceAbsolutePath);
- index = (ILSMIndex) datasetLifecycleManager.getIndex(resourceAbsolutePath);
- if (index == null) {
- //#. create index instance and register to indexLifeCycleManager
- localResourceMetadata = (ILocalResourceMetadata) localResource.getResourceObject();
- index = localResourceMetadata.createIndexInstance(appRuntimeContext,
- resourceAbsolutePath, localResource.getPartition());
- datasetLifecycleManager.register(resourceAbsolutePath, index);
- datasetLifecycleManager.open(resourceAbsolutePath);
-
- //#. 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:
- case LogType.UPSERT_ENTITY_COMMIT:
-
- //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).getResourcePath());
- }
-
- 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;
- boolean nonSharpCheckpointSucceeded = false;
-
- if (isSharpCheckpoint) {
- LOGGER.log(Level.INFO, "Starting sharp checkpoint ... ");
- }
-
- TransactionManager txnMgr = (TransactionManager) txnSubsystem.getTransactionManager();
- String logDir = logMgr.getLogManagerProperties().getLogDir();
-
- //get the filename of the previous checkpoint files which are about to be deleted
- //right after the new checkpoint file is written.
- File[] prevCheckpointFiles = getPreviousCheckpointFiles();
-
- IDatasetLifecycleManager datasetLifecycleManager = txnSubsystem.getAsterixAppRuntimeContextProvider()
- .getDatasetLifecycleManager();
- //flush all in-memory components if it is the sharp checkpoint
- if (isSharpCheckpoint) {
- datasetLifecycleManager.flushAllDatasets();
- 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) {
- nonSharpCheckpointSucceeded = true;
- } 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(), isSharpCheckpoint);
-
- FileOutputStream fos = null;
- ObjectOutputStream oosToFos = null;
- try {
- String fileName = getCheckpointFileName(logDir, Long.toString(checkpointObject.getTimeStamp()));
- fos = new FileOutputStream(fileName);
- oosToFos = new ObjectOutputStream(fos);
- oosToFos.writeObject(checkpointObject);
- oosToFos.flush();
- } catch (IOException e) {
- throw new ACIDException("Failed to checkpoint", e);
- } finally {
- if (oosToFos != null) {
- try {
- oosToFos.close();
- } catch (IOException e) {
- throw new ACIDException("Failed to checkpoint", e);
- }
- }
- if (oosToFos == null && fos != null) {
- try {
- fos.close();
- } catch (IOException e) {
- throw new ACIDException("Failed to checkpoint", e);
- }
- }
- }
-
- //#. delete the previous checkpoint files
- if (prevCheckpointFiles != null) {
- // sort the filenames lexicographically to keep the latest checkpointHistory files.
- Arrays.sort(prevCheckpointFiles);
- for (int i = 0; i < prevCheckpointFiles.length - this.checkpointHistory; ++i) {
- prevCheckpointFiles[i].delete();
- }
- }
-
- if (isSharpCheckpoint) {
- try {
- if (minMCTFirstLSN == SHARP_CHECKPOINT_LSN) {
- logMgr.renewLogFiles();
- } else {
- logMgr.deleteOldLogFiles(minMCTFirstLSN);
- }
- } catch (IOException e) {
- throw new HyracksDataException(e);
- }
- }
-
- if (nonSharpCheckpointSucceeded) {
- logMgr.deleteOldLogFiles(minMCTFirstLSN);
- }
-
- if (isSharpCheckpoint && LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Completed sharp checkpoint.");
- }
-
- //return the min LSN that was recorded in the checkpoint
- return minMCTFirstLSN;
- }
-
- @Override
- 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;
- }
-
- @Override
- public long getLocalMinFirstLSN() throws HyracksDataException {
- IDatasetLifecycleManager datasetLifecycleManager = txnSubsystem.getAsterixAppRuntimeContextProvider()
- .getDatasetLifecycleManager();
- List<IIndex> openIndexList = datasetLifecycleManager.getOpenIndexes();
- long firstLSN;
- //the min first lsn can only be the current append or smaller
- long minFirstLSN = logMgr.getAppendLSN();
- if (openIndexList.size() > 0) {
- for (IIndex index : openIndexList) {
- AbstractLSMIOOperationCallback ioCallback = (AbstractLSMIOOperationCallback) ((ILSMIndex) index)
- .getIOOperationCallback();
-
- if (!((AbstractLSMIndex) index).isCurrentMutableComponentEmpty() || ioCallback.hasPendingFlush()) {
- firstLSN = ioCallback.getFirstLSN();
- minFirstLSN = Math.min(minFirstLSN, firstLSN);
- }
- }
- }
- 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;
-
- //read all checkpointObjects from the existing checkpoint files
- File[] prevCheckpointFiles = getPreviousCheckpointFiles();
- if (prevCheckpointFiles == null || prevCheckpointFiles.length == 0) {
- throw new FileNotFoundException("Checkpoint file is not found");
- }
-
- List<CheckpointObject> checkpointObjectList = new ArrayList<CheckpointObject>();
- for (File file : prevCheckpointFiles) {
- FileInputStream fis = null;
- ObjectInputStream oisFromFis = null;
-
- try {
- fis = new FileInputStream(file);
- oisFromFis = new ObjectInputStream(fis);
- checkpointObject = (CheckpointObject) oisFromFis.readObject();
- checkpointObjectList.add(checkpointObject);
- } catch (Exception e) {
- throw new ACIDException("Failed to read a checkpoint file", e);
- } finally {
- if (oisFromFis != null) {
- try {
- oisFromFis.close();
- } catch (IOException e) {
- throw new ACIDException("Failed to read a checkpoint file", e);
- }
- }
- if (oisFromFis == null && fis != null) {
- try {
- fis.close();
- } catch (IOException e) {
- throw new ACIDException("Failed to read a checkpoint file", e);
- }
- }
- }
- }
-
- //#. sort checkpointObjects in descending order by timeStamp to find out the most recent one.
- Collections.sort(checkpointObjectList);
-
- //#. return the most recent one (the first one in sorted list)
- return checkpointObjectList.get(0);
- }
-
- private File[] getPreviousCheckpointFiles() {
- String logDir = ((LogManager) txnSubsystem.getLogManager()).getLogManagerProperties().getLogDir();
- File parentDir = new File(logDir);
-
- FilenameFilter filter = new FilenameFilter() {
- @Override
- public boolean accept(File dir, String name) {
- if (name.contains(CHECKPOINT_FILENAME_PREFIX)) {
- return true;
- } else {
- return false;
- }
- }
- };
-
- return parentDir.listFiles(filter);
- }
-
- private static String getCheckpointFileName(String baseDir, String suffix) {
- if (!baseDir.endsWith(System.getProperty("file.separator"))) {
- baseDir += System.getProperty("file.separator");
- }
- return baseDir + CHECKPOINT_FILENAME_PREFIX + suffix;
- }
-
- private File createJobRecoveryFile(int jobId, String fileName) throws IOException {
- String recoveryDirPath = getRecoveryDirPath();
- Path JobRecoveryFolder = Paths.get(recoveryDirPath + File.separator + jobId);
- if (!Files.exists(JobRecoveryFolder)) {
- Files.createDirectories(JobRecoveryFolder);
- }
-
- File jobRecoveryFile = new File(JobRecoveryFolder.toString() + File.separator + fileName);
- if (!jobRecoveryFile.exists()) {
- jobRecoveryFile.createNewFile();
- } else {
- throw new IOException("File: " + fileName + " for job id(" + jobId + ") already exists");
- }
-
- return jobRecoveryFile;
- }
-
- private void deleteRecoveryTemporaryFiles() throws IOException {
- String recoveryDirPath = getRecoveryDirPath();
- Path recoveryFolderPath = Paths.get(recoveryDirPath);
- if (Files.exists(recoveryFolderPath)) {
- FileUtils.deleteDirectory(recoveryFolderPath.toFile());
- }
- }
-
- private String getRecoveryDirPath() {
- String logDir = logMgr.getLogManagerProperties().getLogDir();
- if (!logDir.endsWith(File.separator)) {
- logDir += File.separator;
- }
-
- return logDir + RECOVERY_FILES_DIR_NAME;
- }
-
- private void freeJobsCachedEntities(int requestingJobId) throws IOException {
- if (jobId2WinnerEntitiesMap != null) {
- for (Entry<Integer, JobEntityCommits> jobEntityCommits : jobId2WinnerEntitiesMap.entrySet()) {
- //if the job is not the requester, free its memory
- if (jobEntityCommits.getKey() != requestingJobId) {
- jobEntityCommits.getValue().spillToDiskAndfreeMemory();
- }
- }
- }
- }
-
- /**
- * Rollback a transaction
- *
- * @see org.apache.transaction.management.service.recovery.IRecoveryManager# rollbackTransaction (org.apache.TransactionContext.management.service.transaction .TransactionContext)
- */
- @Override
- public void rollbackTransaction(ITransactionContext txnContext) throws ACIDException {
- int abortedJobId = txnContext.getJobId().getId();
- // Obtain the first/last log record LSNs written by the Job
- long firstLSN = txnContext.getFirstLSN();
- long lastLSN = txnContext.getLastLSN();
-
- LOGGER.log(Level.INFO, "rollbacking transaction log records from " + firstLSN + " to " + lastLSN);
- // check if the transaction actually wrote some logs.
- if (firstLSN == TransactionManagementConstants.LogManagerConstants.TERMINAL_LSN) {
- LOGGER.log(Level.INFO,
- "no need to roll back as there were no operations by the transaction " + txnContext.getJobId());
- return;
- }
-
- // While reading log records from firstLsn to lastLsn, collect uncommitted txn's Lsns
- LOGGER.log(Level.INFO, "collecting loser transaction's LSNs from " + firstLSN + " to " + lastLSN);
-
- Map<TxnId, List<Long>> jobLoserEntity2LSNsMap = new HashMap<TxnId, List<Long>>();
- TxnId tempKeyTxnId = new TxnId(-1, -1, -1, null, -1, false);
- int updateLogCount = 0;
- int entityCommitLogCount = 0;
- int logJobId = -1;
- long currentLSN = -1;
- TxnId loserEntity = null;
- List<Long> undoLSNSet = null;
- String nodeId = logMgr.getNodeId();
- ILogReader logReader = logMgr.getLogReader(false);
- try {
- logReader.initializeScan(firstLSN);
- ILogRecord logRecord = null;
- while (currentLSN < lastLSN) {
- logRecord = logReader.next();
- if (logRecord == null) {
- break;
- } else {
- currentLSN = logRecord.getLSN();
-
- if (IS_DEBUG_MODE) {
- LOGGER.info(logRecord.getLogRecordForDisplay());
- }
- }
- 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:
- case LogType.UPSERT_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) {
- throw new ACIDException("LastLSN mismatch: lastLSN(" + lastLSN + ") vs currentLSN(" + currentLSN
- + ") during abort( " + txnContext.getJobId() + ")");
- }
-
- //undo loserTxn's effect
- LOGGER.log(Level.INFO, "undoing loser transaction's effect");
-
- IDatasetLifecycleManager datasetLifecycleManager = txnSubsystem.getAsterixAppRuntimeContextProvider()
- .getDatasetLifecycleManager();
- //TODO sort loser entities by smallest LSN to undo in one pass.
- Iterator<Entry<TxnId, List<Long>>> iter = jobLoserEntity2LSNsMap.entrySet().iterator();
- int undoCount = 0;
- while (iter.hasNext()) {
- Map.Entry<TxnId, List<Long>> loserEntity2LSNsMap = iter.next();
- undoLSNSet = loserEntity2LSNsMap.getValue();
- // The step below is important since the upsert operations must be done in reverse order.
- Collections.reverse(undoLSNSet);
- for (long undoLSN : undoLSNSet) {
- //here, all the log records are UPDATE type. So, we don't need to check the type again.
- //read the corresponding log record to be undone.
- logRecord = logReader.read(undoLSN);
- if (logRecord == null) {
- throw new ACIDException("IllegalState exception during abort( " + txnContext.getJobId() + ")");
- }
- if (IS_DEBUG_MODE) {
- LOGGER.info(logRecord.getLogRecordForDisplay());
- }
- undo(logRecord, datasetLifecycleManager);
- undoCount++;
- }
- }
-
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("undone loser transaction's effect");
- LOGGER.info("[RecoveryManager's rollback log count] update/entityCommit/undo:" + updateLogCount + "/"
- + entityCommitLogCount + "/" + undoCount);
- }
- } finally {
- logReader.close();
- }
- }
-
- @Override
- public void start() {
- //no op
- }
-
- @Override
- 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
- public void dumpState(OutputStream os) throws IOException {
- // do nothing
- }
-
- private static void undo(ILogRecord logRecord, IDatasetLifecycleManager datasetLifecycleManager) {
- try {
- ILSMIndex index = (ILSMIndex) datasetLifecycleManager.getIndex(logRecord.getDatasetId(),
- logRecord.getResourceId());
- ILSMIndexAccessor indexAccessor = index.createAccessor(NoOpOperationCallback.INSTANCE,
- NoOpOperationCallback.INSTANCE);
- if (logRecord.getNewOp() == IndexOperation.INSERT.ordinal()) {
- indexAccessor.forceDelete(logRecord.getNewValue());
- } else if (logRecord.getNewOp() == IndexOperation.DELETE.ordinal()) {
- indexAccessor.forceInsert(logRecord.getNewValue());
- } else {
- throw new IllegalStateException("Unsupported OperationType: " + logRecord.getNewOp());
- }
- } catch (Exception e) {
- throw new IllegalStateException("Failed to undo", e);
- }
- }
-
- private static void redo(ILogRecord logRecord, IDatasetLifecycleManager datasetLifecycleManager) {
- try {
- ILSMIndex index = (ILSMIndex) datasetLifecycleManager.getIndex(logRecord.getDatasetId(),
- logRecord.getResourceId());
- ILSMIndexAccessor indexAccessor = index.createAccessor(NoOpOperationCallback.INSTANCE,
- NoOpOperationCallback.INSTANCE);
- if (logRecord.getNewOp() == IndexOperation.INSERT.ordinal()) {
- indexAccessor.forceInsert(logRecord.getNewValue());
- } else if (logRecord.getNewOp() == IndexOperation.DELETE.ordinal()) {
- indexAccessor.forceDelete(logRecord.getNewValue());
- } else {
- throw new IllegalStateException("Unsupported OperationType: " + logRecord.getNewOp());
- }
- } catch (Exception e) {
- e.printStackTrace();
- throw new IllegalStateException("Failed to redo", e);
- }
- }
-
- //TODO (mhubail) RecoveryManager has three methods that perform logs REDO based on different parameters.
- //They need to be refactored to use partitions only once the log format includes partition id.
- @Override
- public synchronized void replayPartitionsLogs(Integer[] partitions, long lowWaterMarkLSN, String failedNode)
- throws IOException, ACIDException {
- //delete any recovery files from previous failed recovery attempts
- deleteRecoveryTemporaryFiles();
-
- int updateLogCount = 0;
- int entityCommitLogCount = 0;
- int jobCommitLogCount = 0;
- int redoCount = 0;
- int abortLogCount = 0;
- int jobId = -1;
-
- state = SystemState.RECOVERING;
- LOGGER.log(Level.INFO, "[RecoveryMgr] starting recovery ...");
-
- Set<Integer> winnerJobSet = new HashSet<Integer>();
- jobId2WinnerEntitiesMap = new HashMap<>();
-
- TxnId tempKeyTxnId = new TxnId(-1, -1, -1, null, -1, false);
- JobEntityCommits jobEntityWinners = null;
- //#. read checkpoint file and set lowWaterMark where anaylsis and redo start
- long readableSmallestLSN = logMgr.getReadableSmallestLSN();
- if (lowWaterMarkLSN < readableSmallestLSN) {
- lowWaterMarkLSN = readableSmallestLSN;
- }
- //-------------------------------------------------------------------------
- // [ analysis phase ]
- // - collect all committed Lsn
- //-------------------------------------------------------------------------
- LOGGER.log(Level.INFO, "[RecoveryMgr] in analysis phase");
- IAsterixAppRuntimeContextProvider appRuntimeContext = txnSubsystem.getAsterixAppRuntimeContextProvider();
- //get datasetLifeCycleManager
- IDatasetLifecycleManager datasetLifecycleManager = txnSubsystem.getAsterixAppRuntimeContextProvider()
- .getDatasetLifecycleManager();
- PersistentLocalResourceRepository localResourceRepository = (PersistentLocalResourceRepository) appRuntimeContext
- .getLocalResourceRepository();
Map<Long, LocalResource> resourcesMap = localResourceRepository.loadAndGetAllResources();
Map<Long, Long> resourceId2MaxLSNMap = new HashMap<Long, Long>();
+ TxnId tempKeyTxnId = new TxnId(-1, -1, -1, null, -1, false);
- //#. set log reader to the lowWaterMarkLsn
- ILogReader logReader = logMgr.getLogReader(true);
ILogRecord logRecord = null;
try {
logReader.initializeScan(lowWaterMarkLSN);
@@ -1148,84 +318,12 @@
if (IS_DEBUG_MODE) {
LOGGER.info(logRecord.getLogRecordForDisplay());
}
- //TODO update this partitions once the log format is updated to include partitons
- if (logRecord.getNodeId().equals(failedNode)) {
- switch (logRecord.getLogType()) {
- case LogType.UPDATE:
- updateLogCount++;
- break;
- case LogType.JOB_COMMIT:
- jobId = logRecord.getJobId();
- winnerJobSet.add(jobId);
- if (jobId2WinnerEntitiesMap.containsKey(jobId)) {
- jobEntityWinners = jobId2WinnerEntitiesMap.get(jobId);
- //to delete any spilled files as well
- jobEntityWinners.clear();
- jobId2WinnerEntitiesMap.remove(jobId);
- }
- jobCommitLogCount++;
- break;
- case LogType.ENTITY_COMMIT:
- jobId = logRecord.getJobId();
- if (!jobId2WinnerEntitiesMap.containsKey(jobId)) {
- jobEntityWinners = new JobEntityCommits(jobId);
- if (needToFreeMemory()) {
- //if we don't have enough memory for one more job, we will force all jobs to spill their cached entities to disk.
- //This could happen only when we have many jobs with small number of records and none of them have job commit.
- freeJobsCachedEntities(jobId);
- }
- jobId2WinnerEntitiesMap.put(jobId, jobEntityWinners);
- } else {
- jobEntityWinners = jobId2WinnerEntitiesMap.get(jobId);
- }
- jobEntityWinners.add(logRecord);
- entityCommitLogCount++;
- break;
- case LogType.ABORT:
- abortLogCount++;
- break;
- case LogType.FLUSH:
- break;
- default:
- throw new ACIDException("Unsupported LogType: " + logRecord.getLogType());
- }
- }
- logRecord = logReader.next();
- }
-
- //prepare winners for search after analysis is done to flush anything remaining in memory to disk.
- for (JobEntityCommits winners : jobId2WinnerEntitiesMap.values()) {
- winners.prepareForSearch();
- }
- //-------------------------------------------------------------------------
- // [ redo phase ]
- // - redo if
- // 1) The TxnId is committed && --> guarantee durability
- // 2) lsn > maxDiskLastLsn of the index --> guarantee idempotence
- //-------------------------------------------------------------------------
- LOGGER.info("[RecoveryMgr] in redo phase");
-
- long resourceId;
- long maxDiskLastLsn;
- long LSN = -1;
- ILSMIndex index = null;
- LocalResource localResource = null;
- ILocalResourceMetadata localResourceMetadata = null;
- boolean foundWinner = false;
- //set log reader to the lowWaterMarkLsn again.
- logReader.initializeScan(lowWaterMarkLSN);
- logRecord = logReader.next();
- while (logRecord != null) {
- if (IS_DEBUG_MODE) {
- LOGGER.info(logRecord.getLogRecordForDisplay());
- }
- //TODO update this to check for partitions instead of node id once the log format is updated to include partitions
- if (logRecord.getNodeId().equals(failedNode)) {
- LSN = logRecord.getLSN();
- jobId = logRecord.getJobId();
- foundWinner = false;
- switch (logRecord.getLogType()) {
- case LogType.UPDATE:
+ LSN = logRecord.getLSN();
+ jobId = logRecord.getJobId();
+ foundWinner = false;
+ switch (logRecord.getLogType()) {
+ case LogType.UPDATE:
+ if (partitions.contains(logRecord.getResourcePartition())) {
if (winnerJobSet.contains(jobId)) {
foundWinner = true;
} else if (jobId2WinnerEntitiesMap.containsKey(jobId)) {
@@ -1298,37 +396,479 @@
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());
- }
+ }
+ break;
+ case LogType.JOB_COMMIT:
+ case LogType.ENTITY_COMMIT:
+ case LogType.ABORT:
+ case LogType.FLUSH:
+ case LogType.UPSERT_ENTITY_COMMIT:
+ //do nothing
+ break;
+ default:
+ throw new ACIDException("Unsupported LogType: " + logRecord.getLogType());
}
logRecord = logReader.next();
}
-
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("[RecoveryMgr] recovery is completed.");
- LOGGER.info("[RecoveryMgr's recovery log count] update/entityCommit/jobCommit/abort/redo = "
- + updateLogCount + "/" + entityCommitLogCount + "/" + jobCommitLogCount + "/" + abortLogCount
- + "/" + redoCount);
- }
+ LOGGER.info("Logs REDO phase completed. Redo logs count: " + redoCount);
} finally {
- logReader.close();
-
//close all indexes
Set<Long> resourceIdList = resourceId2MaxLSNMap.keySet();
for (long r : resourceIdList) {
datasetLifecycleManager.close(resourcesMap.get(r).getResourcePath());
}
+ }
+ }
- //delete any recovery files after completing recovery
- deleteRecoveryTemporaryFiles();
+ private static boolean needToFreeMemory() {
+ return Runtime.getRuntime().freeMemory() < MAX_CACHED_ENTITY_COMMITS_PER_JOB_SIZE;
+ }
+
+ @Override
+ public synchronized long checkpoint(boolean isSharpCheckpoint, long nonSharpCheckpointTargetLSN)
+ throws ACIDException, HyracksDataException {
+ long minMCTFirstLSN;
+ boolean nonSharpCheckpointSucceeded = false;
+
+ if (isSharpCheckpoint) {
+ LOGGER.log(Level.INFO, "Starting sharp checkpoint ... ");
+ }
+
+ TransactionManager txnMgr = (TransactionManager) txnSubsystem.getTransactionManager();
+ String logDir = logMgr.getLogManagerProperties().getLogDir();
+
+ //get the filename of the previous checkpoint files which are about to be deleted
+ //right after the new checkpoint file is written.
+ File[] prevCheckpointFiles = getPreviousCheckpointFiles();
+
+ IDatasetLifecycleManager datasetLifecycleManager = txnSubsystem.getAsterixAppRuntimeContextProvider()
+ .getDatasetLifecycleManager();
+ //flush all in-memory components if it is the sharp checkpoint
+ if (isSharpCheckpoint) {
+ datasetLifecycleManager.flushAllDatasets();
+ 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();
+ IAsterixPropertiesProvider propertiesProvider = (IAsterixPropertiesProvider) txnSubsystem
+ .getAsterixAppRuntimeContextProvider().getAppContext();
+ AsterixMetadataProperties metadataProperties = propertiesProvider.getMetadataProperties();
+ Set<Integer> deadReplicasPartitions = new HashSet<>();
+ //get partitions of the dead replicas that are not active on this node
+ for (String deadReplicaId : deadReplicaIds) {
+ ClusterPartition[] nodePartitons = metadataProperties.getNodePartitions()
+ .get(deadReplicaId);
+ for (ClusterPartition partition : nodePartitons) {
+ if (!localResourceRepository.getActivePartitions()
+ .contains(partition.getPartitionId())) {
+ deadReplicasPartitions.add(partition.getPartitionId());
+ }
+ }
+ }
+ minMCTFirstLSN = remoteResourcesManager.getPartitionsMinLSN(deadReplicasPartitions);
+ }
+ } else {
+ //start up complete checkpoint. Avoid deleting remote recovery logs.
+ minMCTFirstLSN = getMinFirstLSN();
+ }
+ }
+ } else {
+ minMCTFirstLSN = getMinFirstLSN();
+ if (minMCTFirstLSN >= nonSharpCheckpointTargetLSN) {
+ nonSharpCheckpointSucceeded = true;
+ } 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(), isSharpCheckpoint);
+
+ String fileName = getCheckpointFileName(logDir, Long.toString(checkpointObject.getTimeStamp()));
+
+ try (FileOutputStream fos = new FileOutputStream(fileName);
+ ObjectOutputStream oosToFos = new ObjectOutputStream(fos)) {
+ oosToFos.writeObject(checkpointObject);
+ oosToFos.flush();
+ } catch (IOException e) {
+ throw new ACIDException("Failed to checkpoint", e);
+ }
+
+ //#. delete the previous checkpoint files
+ if (prevCheckpointFiles != null) {
+ // sort the filenames lexicographically to keep the latest checkpointHistory files.
+ Arrays.sort(prevCheckpointFiles);
+ for (int i = 0; i < prevCheckpointFiles.length - this.checkpointHistory; ++i) {
+ prevCheckpointFiles[i].delete();
+ }
+ }
+
+ if (isSharpCheckpoint) {
+ try {
+ if (minMCTFirstLSN == SHARP_CHECKPOINT_LSN) {
+ logMgr.renewLogFiles();
+ } else {
+ logMgr.deleteOldLogFiles(minMCTFirstLSN);
+ }
+ } catch (IOException e) {
+ throw new HyracksDataException(e);
+ }
+ }
+
+ if (nonSharpCheckpointSucceeded) {
+ logMgr.deleteOldLogFiles(minMCTFirstLSN);
+ }
+
+ if (isSharpCheckpoint) {
+ LOGGER.info("Completed sharp checkpoint.");
+ }
+
+ //return the min LSN that was recorded in the checkpoint
+ return minMCTFirstLSN;
+ }
+
+ @Override
+ 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;
+ }
+
+ @Override
+ public long getLocalMinFirstLSN() throws HyracksDataException {
+ IDatasetLifecycleManager datasetLifecycleManager = txnSubsystem.getAsterixAppRuntimeContextProvider()
+ .getDatasetLifecycleManager();
+ List<IIndex> openIndexList = datasetLifecycleManager.getOpenIndexes();
+ long firstLSN;
+ //the min first lsn can only be the current append or smaller
+ long minFirstLSN = logMgr.getAppendLSN();
+ if (openIndexList.size() > 0) {
+ for (IIndex index : openIndexList) {
+ AbstractLSMIOOperationCallback ioCallback = (AbstractLSMIOOperationCallback) ((ILSMIndex) index)
+ .getIOOperationCallback();
+
+ if (!((AbstractLSMIndex) index).isCurrentMutableComponentEmpty() || ioCallback.hasPendingFlush()) {
+ firstLSN = ioCallback.getFirstLSN();
+ minFirstLSN = Math.min(minFirstLSN, firstLSN);
+ }
+ }
+ }
+ return minFirstLSN;
+ }
+
+ private long getRemoteMinFirstLSN() {
+ IReplicaResourcesManager remoteResourcesManager = txnSubsystem.getAsterixAppRuntimeContextProvider()
+ .getAppContext().getReplicaResourcesManager();
+ long minRemoteLSN = remoteResourcesManager.getPartitionsMinLSN(localResourceRepository.getInactivePartitions());
+ return minRemoteLSN;
+ }
+
+ private CheckpointObject readCheckpoint() throws ACIDException, FileNotFoundException {
+ CheckpointObject checkpointObject = null;
+
+ //read all checkpointObjects from the existing checkpoint files
+ File[] prevCheckpointFiles = getPreviousCheckpointFiles();
+ if (prevCheckpointFiles == null || prevCheckpointFiles.length == 0) {
+ throw new FileNotFoundException("Checkpoint file is not found");
+ }
+
+ List<CheckpointObject> checkpointObjectList = new ArrayList<CheckpointObject>();
+ for (File file : prevCheckpointFiles) {
+ try (FileInputStream fis = new FileInputStream(file);
+ ObjectInputStream oisFromFis = new ObjectInputStream(fis)) {
+ checkpointObject = (CheckpointObject) oisFromFis.readObject();
+ checkpointObjectList.add(checkpointObject);
+ } catch (Exception e) {
+ throw new ACIDException("Failed to read a checkpoint file", e);
+ }
+ }
+
+ //sort checkpointObjects in descending order by timeStamp to find out the most recent one.
+ Collections.sort(checkpointObjectList);
+
+ //return the most recent one (the first one in sorted list)
+ return checkpointObjectList.get(0);
+ }
+
+ private File[] getPreviousCheckpointFiles() {
+ String logDir = ((LogManager) txnSubsystem.getLogManager()).getLogManagerProperties().getLogDir();
+ File parentDir = new File(logDir);
+
+ FilenameFilter filter = new FilenameFilter() {
+ @Override
+ public boolean accept(File dir, String name) {
+ return name.contains(CHECKPOINT_FILENAME_PREFIX);
+ }
+ };
+
+ return parentDir.listFiles(filter);
+ }
+
+ private static String getCheckpointFileName(String baseDir, String suffix) {
+ if (!baseDir.endsWith(System.getProperty("file.separator"))) {
+ baseDir += System.getProperty("file.separator");
+ }
+ return baseDir + CHECKPOINT_FILENAME_PREFIX + suffix;
+ }
+
+ @Override
+ public File createJobRecoveryFile(int jobId, String fileName) throws IOException {
+ String recoveryDirPath = getRecoveryDirPath();
+ Path JobRecoveryFolder = Paths.get(recoveryDirPath + File.separator + jobId);
+ if (!Files.exists(JobRecoveryFolder)) {
+ Files.createDirectories(JobRecoveryFolder);
+ }
+
+ File jobRecoveryFile = new File(JobRecoveryFolder.toString() + File.separator + fileName);
+ if (!jobRecoveryFile.exists()) {
+ jobRecoveryFile.createNewFile();
+ } else {
+ throw new IOException("File: " + fileName + " for job id(" + jobId + ") already exists");
+ }
+
+ return jobRecoveryFile;
+ }
+
+ @Override
+ public void deleteRecoveryTemporaryFiles() {
+ String recoveryDirPath = getRecoveryDirPath();
+ Path recoveryFolderPath = Paths.get(recoveryDirPath);
+ FileUtils.deleteQuietly(recoveryFolderPath.toFile());
+ }
+
+ private String getRecoveryDirPath() {
+ String logDir = logMgr.getLogManagerProperties().getLogDir();
+ if (!logDir.endsWith(File.separator)) {
+ logDir += File.separator;
+ }
+
+ return logDir + RECOVERY_FILES_DIR_NAME;
+ }
+
+ private void freeJobsCachedEntities(int requestingJobId) throws IOException {
+ if (jobId2WinnerEntitiesMap != null) {
+ for (Entry<Integer, JobEntityCommits> jobEntityCommits : jobId2WinnerEntitiesMap.entrySet()) {
+ //if the job is not the requester, free its memory
+ if (jobEntityCommits.getKey() != requestingJobId) {
+ jobEntityCommits.getValue().spillToDiskAndfreeMemory();
+ }
+ }
+ }
+ }
+
+ @Override
+ public void rollbackTransaction(ITransactionContext txnContext) throws ACIDException {
+ int abortedJobId = txnContext.getJobId().getId();
+ // Obtain the first/last log record LSNs written by the Job
+ long firstLSN = txnContext.getFirstLSN();
+ long lastLSN = txnContext.getLastLSN();
+
+ LOGGER.log(Level.INFO, "rollbacking transaction log records from " + firstLSN + " to " + lastLSN);
+ // check if the transaction actually wrote some logs.
+ if (firstLSN == TransactionManagementConstants.LogManagerConstants.TERMINAL_LSN) {
+ LOGGER.log(Level.INFO,
+ "no need to roll back as there were no operations by the transaction " + txnContext.getJobId());
+ return;
+ }
+
+ // While reading log records from firstLsn to lastLsn, collect uncommitted txn's Lsns
+ LOGGER.log(Level.INFO, "collecting loser transaction's LSNs from " + firstLSN + " to " + lastLSN);
+
+ Map<TxnId, List<Long>> jobLoserEntity2LSNsMap = new HashMap<TxnId, List<Long>>();
+ TxnId tempKeyTxnId = new TxnId(-1, -1, -1, null, -1, false);
+ int updateLogCount = 0;
+ int entityCommitLogCount = 0;
+ int logJobId = -1;
+ long currentLSN = -1;
+ TxnId loserEntity = null;
+ List<Long> undoLSNSet = null;
+ //get active partitions on this node
+ Set<Integer> activePartitions = localResourceRepository.getActivePartitions();
+ ILogReader logReader = logMgr.getLogReader(false);
+ try {
+ logReader.initializeScan(firstLSN);
+ ILogRecord logRecord = null;
+ while (currentLSN < lastLSN) {
+ logRecord = logReader.next();
+ if (logRecord == null) {
+ 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:
+ if (activePartitions.contains(logRecord.getResourcePartition())) {
+ 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:
+ case LogType.UPSERT_ENTITY_COMMIT:
+ if (activePartitions.contains(logRecord.getResourcePartition())) {
+ 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) {
+ throw new ACIDException("LastLSN mismatch: lastLSN(" + lastLSN + ") vs currentLSN(" + currentLSN
+ + ") during abort( " + txnContext.getJobId() + ")");
+ }
+
+ //undo loserTxn's effect
+ LOGGER.log(Level.INFO, "undoing loser transaction's effect");
+
+ IDatasetLifecycleManager datasetLifecycleManager = txnSubsystem.getAsterixAppRuntimeContextProvider()
+ .getDatasetLifecycleManager();
+ //TODO sort loser entities by smallest LSN to undo in one pass.
+ Iterator<Entry<TxnId, List<Long>>> iter = jobLoserEntity2LSNsMap.entrySet().iterator();
+ int undoCount = 0;
+ while (iter.hasNext()) {
+ Map.Entry<TxnId, List<Long>> loserEntity2LSNsMap = iter.next();
+ undoLSNSet = loserEntity2LSNsMap.getValue();
+ // The step below is important since the upsert operations must be done in reverse order.
+ Collections.reverse(undoLSNSet);
+ for (long undoLSN : undoLSNSet) {
+ //here, all the log records are UPDATE type. So, we don't need to check the type again.
+ //read the corresponding log record to be undone.
+ logRecord = logReader.read(undoLSN);
+ if (logRecord == null) {
+ throw new ACIDException("IllegalState exception during abort( " + txnContext.getJobId() + ")");
+ }
+ if (IS_DEBUG_MODE) {
+ LOGGER.info(logRecord.getLogRecordForDisplay());
+ }
+ undo(logRecord, datasetLifecycleManager);
+ undoCount++;
+ }
+ }
+
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("undone loser transaction's effect");
+ LOGGER.info("[RecoveryManager's rollback log count] update/entityCommit/undo:" + updateLogCount + "/"
+ + entityCommitLogCount + "/" + undoCount);
+ }
+
+ } finally {
+ logReader.close();
+ }
+ }
+
+ @Override
+ public void start() {
+ //no op
+ }
+
+ @Override
+ 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
+ public void dumpState(OutputStream os) throws IOException {
+ // do nothing
+ }
+
+ private static void undo(ILogRecord logRecord, IDatasetLifecycleManager datasetLifecycleManager) {
+ try {
+ ILSMIndex index = (ILSMIndex) datasetLifecycleManager.getIndex(logRecord.getDatasetId(),
+ logRecord.getResourceId());
+ ILSMIndexAccessor indexAccessor = index.createAccessor(NoOpOperationCallback.INSTANCE,
+ NoOpOperationCallback.INSTANCE);
+ if (logRecord.getNewOp() == IndexOperation.INSERT.ordinal()) {
+ indexAccessor.forceDelete(logRecord.getNewValue());
+ } else if (logRecord.getNewOp() == IndexOperation.DELETE.ordinal()) {
+ indexAccessor.forceInsert(logRecord.getNewValue());
+ } else {
+ throw new IllegalStateException("Unsupported OperationType: " + logRecord.getNewOp());
+ }
+ } catch (Exception e) {
+ throw new IllegalStateException("Failed to undo", e);
+ }
+ }
+
+ private static void redo(ILogRecord logRecord, IDatasetLifecycleManager datasetLifecycleManager) {
+ try {
+ ILSMIndex index = (ILSMIndex) datasetLifecycleManager.getIndex(logRecord.getDatasetId(),
+ logRecord.getResourceId());
+ ILSMIndexAccessor indexAccessor = index.createAccessor(NoOpOperationCallback.INSTANCE,
+ NoOpOperationCallback.INSTANCE);
+ if (logRecord.getNewOp() == IndexOperation.INSERT.ordinal()) {
+ indexAccessor.forceInsert(logRecord.getNewValue());
+ } else if (logRecord.getNewOp() == IndexOperation.DELETE.ordinal()) {
+ indexAccessor.forceDelete(logRecord.getNewValue());
+ } else {
+ throw new IllegalStateException("Unsupported OperationType: " + logRecord.getNewOp());
+ }
+ } catch (Exception e) {
+ e.printStackTrace();
+ throw new IllegalStateException("Failed to redo", e);
}
}
@@ -1499,153 +1039,4 @@
}
}
}
-}
-
-class TxnId {
- public boolean isByteArrayPKValue;
- public int jobId;
- public int datasetId;
- public int pkHashValue;
- public int pkSize;
- public byte[] byteArrayPKValue;
- public ITupleReference tupleReferencePKValue;
-
- public TxnId(int jobId, int datasetId, int pkHashValue, ITupleReference pkValue, int pkSize,
- boolean isByteArrayPKValue) {
- this.jobId = jobId;
- this.datasetId = datasetId;
- this.pkHashValue = pkHashValue;
- this.pkSize = pkSize;
- this.isByteArrayPKValue = isByteArrayPKValue;
- if (isByteArrayPKValue) {
- this.byteArrayPKValue = new byte[pkSize];
- readPKValueIntoByteArray(pkValue, pkSize, byteArrayPKValue);
- } else {
- this.tupleReferencePKValue = pkValue;
- }
- }
-
- public TxnId() {
- }
-
- private static void readPKValueIntoByteArray(ITupleReference pkValue, int pkSize, byte[] byteArrayPKValue) {
- int readOffset = pkValue.getFieldStart(0);
- byte[] readBuffer = pkValue.getFieldData(0);
- for (int i = 0; i < pkSize; i++) {
- byteArrayPKValue[i] = readBuffer[readOffset + i];
- }
- }
-
- public void setTxnId(int jobId, int datasetId, int pkHashValue, ITupleReference pkValue, int pkSize) {
- this.jobId = jobId;
- this.datasetId = datasetId;
- this.pkHashValue = pkHashValue;
- this.tupleReferencePKValue = pkValue;
- this.pkSize = pkSize;
- this.isByteArrayPKValue = false;
- }
-
- @Override
- public String toString() {
- return "[" + jobId + "," + datasetId + "," + pkHashValue + "," + pkSize + "]";
- }
-
- @Override
- public int hashCode() {
- return pkHashValue;
- }
-
- @Override
- public boolean equals(Object o) {
- if (o == this) {
- return true;
- }
- if (!(o instanceof TxnId)) {
- return false;
- }
- TxnId txnId = (TxnId) o;
- return (txnId.pkHashValue == pkHashValue && txnId.datasetId == datasetId && txnId.jobId == jobId
- && pkSize == txnId.pkSize && isEqualTo(txnId));
- }
-
- private boolean isEqualTo(TxnId txnId) {
- if (isByteArrayPKValue && txnId.isByteArrayPKValue) {
- return isEqual(byteArrayPKValue, txnId.byteArrayPKValue, pkSize);
- } else if (isByteArrayPKValue && (!txnId.isByteArrayPKValue)) {
- return isEqual(byteArrayPKValue, txnId.tupleReferencePKValue, pkSize);
- } else if ((!isByteArrayPKValue) && txnId.isByteArrayPKValue) {
- return isEqual(txnId.byteArrayPKValue, tupleReferencePKValue, pkSize);
- } else {
- return isEqual(tupleReferencePKValue, txnId.tupleReferencePKValue, pkSize);
- }
- }
-
- private static boolean isEqual(byte[] a, byte[] b, int size) {
- for (int i = 0; i < size; i++) {
- if (a[i] != b[i]) {
- return false;
- }
- }
- return true;
- }
-
- private static boolean isEqual(byte[] a, ITupleReference b, int size) {
- int readOffset = b.getFieldStart(0);
- byte[] readBuffer = b.getFieldData(0);
- for (int i = 0; i < size; i++) {
- if (a[i] != readBuffer[readOffset + i]) {
- return false;
- }
- }
- return true;
- }
-
- private static boolean isEqual(ITupleReference a, ITupleReference b, int size) {
- int aOffset = a.getFieldStart(0);
- byte[] aBuffer = a.getFieldData(0);
- int bOffset = b.getFieldStart(0);
- byte[] bBuffer = b.getFieldData(0);
- for (int i = 0; i < size; i++) {
- if (aBuffer[aOffset + i] != bBuffer[bOffset + i]) {
- return false;
- }
- }
- return true;
- }
-
- public void serialize(ByteBuffer buffer) {
- buffer.putInt(jobId);
- buffer.putInt(datasetId);
- buffer.putInt(pkHashValue);
- buffer.putInt(pkSize);
- buffer.put((byte) (isByteArrayPKValue ? 1 : 0));
- if (isByteArrayPKValue) {
- buffer.put(byteArrayPKValue);
- }
- }
-
- public static TxnId deserialize(ByteBuffer buffer) {
- TxnId txnId = new TxnId();
- txnId.jobId = buffer.getInt();
- txnId.datasetId = buffer.getInt();
- txnId.pkHashValue = buffer.getInt();
- txnId.pkSize = buffer.getInt();
- txnId.isByteArrayPKValue = (buffer.get() == 1);
- if (txnId.isByteArrayPKValue) {
- byte[] byteArrayPKValue = new byte[txnId.pkSize];
- buffer.get(byteArrayPKValue);
- txnId.byteArrayPKValue = byteArrayPKValue;
- }
- return txnId;
- }
-
- public int getCurrentSize() {
- //job id, dataset id, pkHashValue, arraySize, isByteArrayPKValue
- int size = JobId.BYTES + DatasetId.BYTES + LogRecord.PKHASH_LEN + LogRecord.PKSZ_LEN + Byte.BYTES;
- //byte arraySize
- if (isByteArrayPKValue && byteArrayPKValue != null) {
- size += byteArrayPKValue.length;
- }
- return size;
- }
}
\ No newline at end of file
diff --git a/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/recovery/TxnId.java b/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/recovery/TxnId.java
new file mode 100644
index 0000000..bd4a49a
--- /dev/null
+++ b/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/recovery/TxnId.java
@@ -0,0 +1,175 @@
+/*
+ * 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.recovery;
+
+import java.nio.ByteBuffer;
+
+import org.apache.asterix.common.transactions.DatasetId;
+import org.apache.asterix.common.transactions.JobId;
+import org.apache.asterix.common.transactions.LogRecord;
+import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+
+public class TxnId {
+ public boolean isByteArrayPKValue;
+ public int jobId;
+ public int datasetId;
+ public int pkHashValue;
+ public int pkSize;
+ public byte[] byteArrayPKValue;
+ public ITupleReference tupleReferencePKValue;
+
+ public TxnId(int jobId, int datasetId, int pkHashValue, ITupleReference pkValue, int pkSize,
+ boolean isByteArrayPKValue) {
+ this.jobId = jobId;
+ this.datasetId = datasetId;
+ this.pkHashValue = pkHashValue;
+ this.pkSize = pkSize;
+ this.isByteArrayPKValue = isByteArrayPKValue;
+ if (isByteArrayPKValue) {
+ this.byteArrayPKValue = new byte[pkSize];
+ readPKValueIntoByteArray(pkValue, pkSize, byteArrayPKValue);
+ } else {
+ this.tupleReferencePKValue = pkValue;
+ }
+ }
+
+ public TxnId() {
+ }
+
+ private static void readPKValueIntoByteArray(ITupleReference pkValue, int pkSize, byte[] byteArrayPKValue) {
+ int readOffset = pkValue.getFieldStart(0);
+ byte[] readBuffer = pkValue.getFieldData(0);
+ for (int i = 0; i < pkSize; i++) {
+ byteArrayPKValue[i] = readBuffer[readOffset + i];
+ }
+ }
+
+ public void setTxnId(int jobId, int datasetId, int pkHashValue, ITupleReference pkValue, int pkSize) {
+ this.jobId = jobId;
+ this.datasetId = datasetId;
+ this.pkHashValue = pkHashValue;
+ this.tupleReferencePKValue = pkValue;
+ this.pkSize = pkSize;
+ this.isByteArrayPKValue = false;
+ }
+
+ @Override
+ public String toString() {
+ return "[" + jobId + "," + datasetId + "," + pkHashValue + "," + pkSize + "]";
+ }
+
+ @Override
+ public int hashCode() {
+ return pkHashValue;
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (o == this) {
+ return true;
+ }
+ if (!(o instanceof TxnId)) {
+ return false;
+ }
+ TxnId txnId = (TxnId) o;
+ return (txnId.pkHashValue == pkHashValue && txnId.datasetId == datasetId && txnId.jobId == jobId
+ && pkSize == txnId.pkSize && isEqualTo(txnId));
+ }
+
+ private boolean isEqualTo(TxnId txnId) {
+ if (isByteArrayPKValue && txnId.isByteArrayPKValue) {
+ return isEqual(byteArrayPKValue, txnId.byteArrayPKValue, pkSize);
+ } else if (isByteArrayPKValue && (!txnId.isByteArrayPKValue)) {
+ return isEqual(byteArrayPKValue, txnId.tupleReferencePKValue, pkSize);
+ } else if ((!isByteArrayPKValue) && txnId.isByteArrayPKValue) {
+ return isEqual(txnId.byteArrayPKValue, tupleReferencePKValue, pkSize);
+ } else {
+ return isEqual(tupleReferencePKValue, txnId.tupleReferencePKValue, pkSize);
+ }
+ }
+
+ private static boolean isEqual(byte[] a, byte[] b, int size) {
+ for (int i = 0; i < size; i++) {
+ if (a[i] != b[i]) {
+ return false;
+ }
+ }
+ return true;
+ }
+
+ private static boolean isEqual(byte[] a, ITupleReference b, int size) {
+ int readOffset = b.getFieldStart(0);
+ byte[] readBuffer = b.getFieldData(0);
+ for (int i = 0; i < size; i++) {
+ if (a[i] != readBuffer[readOffset + i]) {
+ return false;
+ }
+ }
+ return true;
+ }
+
+ private static boolean isEqual(ITupleReference a, ITupleReference b, int size) {
+ int aOffset = a.getFieldStart(0);
+ byte[] aBuffer = a.getFieldData(0);
+ int bOffset = b.getFieldStart(0);
+ byte[] bBuffer = b.getFieldData(0);
+ for (int i = 0; i < size; i++) {
+ if (aBuffer[aOffset + i] != bBuffer[bOffset + i]) {
+ return false;
+ }
+ }
+ return true;
+ }
+
+ public void serialize(ByteBuffer buffer) {
+ buffer.putInt(jobId);
+ buffer.putInt(datasetId);
+ buffer.putInt(pkHashValue);
+ buffer.putInt(pkSize);
+ buffer.put((byte) (isByteArrayPKValue ? 1 : 0));
+ if (isByteArrayPKValue) {
+ buffer.put(byteArrayPKValue);
+ }
+ }
+
+ public static TxnId deserialize(ByteBuffer buffer) {
+ TxnId txnId = new TxnId();
+ txnId.jobId = buffer.getInt();
+ txnId.datasetId = buffer.getInt();
+ txnId.pkHashValue = buffer.getInt();
+ txnId.pkSize = buffer.getInt();
+ txnId.isByteArrayPKValue = (buffer.get() == 1);
+ if (txnId.isByteArrayPKValue) {
+ byte[] byteArrayPKValue = new byte[txnId.pkSize];
+ buffer.get(byteArrayPKValue);
+ txnId.byteArrayPKValue = byteArrayPKValue;
+ }
+ return txnId;
+ }
+
+ public int getCurrentSize() {
+ //job id, dataset id, pkHashValue, arraySize, isByteArrayPKValue
+ int size = JobId.BYTES + DatasetId.BYTES + LogRecord.PKHASH_LEN + LogRecord.PKSZ_LEN + Byte.BYTES;
+ //byte arraySize
+ if (isByteArrayPKValue && byteArrayPKValue != null) {
+ size += byteArrayPKValue.length;
+ }
+ return size;
+ }
+}
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 47a92cb..215eb14 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,7 +110,6 @@
primaryIndex = null;
tempResourceIdForRegister = new MutableLong();
logRecord = new LogRecord();
- logRecord.setNodeId(transactionSubsystem.getId());
transactorNumActiveOperations = new AtomicInteger(0);
}