[ASTERIXDB-2195][REPL] Replace Static Replication
- user model changes: no
- storage format changes: no
- interface changes: yes
- Redesigned all replication interfaces
Details:
- Replace static replication and fault tolerance by
dynamic storage API.
- Remove static based fault tolerance strategies.
- Redesign replication APIs and classes to smaller
maintainable parts.
- Clean up replication properties.
- Unify logic for checkpoints when replication is
enabled.
- Remove static replication test cases.
- Add replication runtime test cases for:
- Bulkload component replication.
- Memory component recovery.
- Flushed component replication.
- Add replication integration test for:
- Resync failed replica.
Change-Id: Ic5c4b0ac199a4530c807e558c8aebb1eb1284048
Reviewed-on: https://asterix-gerrit.ics.uci.edu/2252
Sonar-Qube: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
Integration-Tests: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
Tested-by: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
Contrib: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
Reviewed-by: Michael Blow <mblow@apache.org>
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/NCAppRuntimeContext.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/NCAppRuntimeContext.java
index 425cbe4..9c53c18 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/NCAppRuntimeContext.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/NCAppRuntimeContext.java
@@ -54,11 +54,8 @@
import org.apache.asterix.common.exceptions.ACIDException;
import org.apache.asterix.common.exceptions.AsterixException;
import org.apache.asterix.common.library.ILibraryManager;
-import org.apache.asterix.common.replication.IRemoteRecoveryManager;
-import org.apache.asterix.common.replication.IReplicaResourcesManager;
import org.apache.asterix.common.replication.IReplicationChannel;
import org.apache.asterix.common.replication.IReplicationManager;
-import org.apache.asterix.common.replication.Replica;
import org.apache.asterix.common.storage.IIndexCheckpointManagerProvider;
import org.apache.asterix.common.storage.IReplicaManager;
import org.apache.asterix.common.transactions.IAppRuntimeContextProvider;
@@ -72,10 +69,8 @@
import org.apache.asterix.metadata.api.IAsterixStateProxy;
import org.apache.asterix.metadata.api.IMetadataNode;
import org.apache.asterix.metadata.bootstrap.MetadataBootstrap;
-import org.apache.asterix.replication.management.ReplicationChannel;
import org.apache.asterix.replication.management.ReplicationManager;
-import org.apache.asterix.replication.recovery.RemoteRecoveryManager;
-import org.apache.asterix.replication.storage.ReplicaResourcesManager;
+import org.apache.asterix.replication.management.ReplicationChannel;
import org.apache.asterix.runtime.transaction.GlobalResourceIdFactoryProvider;
import org.apache.asterix.runtime.utils.NoOpCoordinationService;
import org.apache.asterix.transaction.management.resource.PersistentLocalResourceRepository;
@@ -139,8 +134,6 @@
private ActiveManager activeManager;
private IReplicationChannel replicationChannel;
private IReplicationManager replicationManager;
- private IRemoteRecoveryManager remoteRecoveryManager;
- private IReplicaResourcesManager replicaResourcesManager;
private final ILibraryManager libraryManager;
private final NCExtensionManager ncExtensionManager;
private final IStorageComponentProvider componentProvider;
@@ -190,8 +183,7 @@
indexCheckpointManagerProvider = new IndexCheckpointManagerProvider(ioManager);
ILocalResourceRepositoryFactory persistentLocalResourceRepositoryFactory =
- new PersistentLocalResourceRepositoryFactory(ioManager, getServiceContext().getNodeId(),
- metadataProperties, indexCheckpointManagerProvider);
+ new PersistentLocalResourceRepositoryFactory(ioManager, indexCheckpointManagerProvider);
localResourceRepository =
(PersistentLocalResourceRepository) persistentLocalResourceRepositoryFactory.createRepository();
@@ -224,47 +216,21 @@
this.ncServiceContext);
if (replicationProperties.isReplicationEnabled()) {
+ replicationManager = new ReplicationManager(this, replicationProperties);
- replicaResourcesManager = new ReplicaResourcesManager(localResourceRepository, metadataProperties,
- indexCheckpointManagerProvider);
+ //pass replication manager to replication required object
+ //LogManager to replicate logs
+ txnSubsystem.getLogManager().setReplicationManager(replicationManager);
- replicationManager = new ReplicationManager(nodeId, replicationProperties, replicaResourcesManager,
- txnSubsystem.getLogManager(), asterixAppRuntimeContextProvider, ncServiceContext);
+ //PersistentLocalResourceRepository to replicate metadata files and delete backups on drop index
+ localResourceRepository.setReplicationManager(replicationManager);
- if (replicationManager.getReplicationStrategy().isParticipant(getServiceContext().getNodeId())) {
+ //initialize replication channel
+ replicationChannel = new ReplicationChannel(this);
- //pass replication manager to replication required object
- //LogManager to replicate logs
- txnSubsystem.getLogManager().setReplicationManager(replicationManager);
-
- //PersistentLocalResourceRepository to replicate metadata files and delete backups on drop index
- 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> remotePrimaryReplicas = replicationManager.getReplicationStrategy()
- .getRemotePrimaryReplicas(nodeId).stream().map(Replica::getId).collect(Collectors.toSet());
- for (String clientId : remotePrimaryReplicas) {
- //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(),
- replicaResourcesManager, replicationManager, getServiceContext(),
- asterixAppRuntimeContextProvider, replicationManager.getReplicationStrategy());
-
- remoteRecoveryManager = new RemoteRecoveryManager(replicationManager, this, replicationProperties);
-
- bufferCache = new BufferCache(ioManager, prs, pcp, new FileMapManager(),
- storageProperties.getBufferCacheMaxOpenFiles(), getServiceContext().getThreadFactory(),
- replicationManager);
- }
+ bufferCache = new BufferCache(ioManager, prs, pcp, new FileMapManager(),
+ storageProperties.getBufferCacheMaxOpenFiles(), getServiceContext().getThreadFactory(),
+ replicationManager);
} else {
bufferCache = new BufferCache(ioManager, prs, pcp, new FileMapManager(),
storageProperties.getBufferCacheMaxOpenFiles(), getServiceContext().getThreadFactory());
@@ -441,16 +407,6 @@
}
@Override
- public IReplicaResourcesManager getReplicaResourcesManager() {
- return replicaResourcesManager;
- }
-
- @Override
- public IRemoteRecoveryManager getRemoteRecoveryManager() {
- return remoteRecoveryManager;
- }
-
- @Override
public IReplicationManager getReplicationManager() {
return replicationManager;
}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/RecoveryManager.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/RecoveryManager.java
index f458688..080ad48 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/RecoveryManager.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/RecoveryManager.java
@@ -39,6 +39,7 @@
import java.util.Map;
import java.util.Map.Entry;
import java.util.Set;
+import java.util.stream.Collectors;
import org.apache.asterix.common.api.IDatasetLifecycleManager;
import org.apache.asterix.common.api.INcApplicationContext;
@@ -46,7 +47,6 @@
import org.apache.asterix.common.dataflow.DatasetLocalResource;
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.storage.DatasetResourceReference;
import org.apache.asterix.common.storage.IIndexCheckpointManagerProvider;
import org.apache.asterix.common.transactions.Checkpoint;
@@ -96,16 +96,17 @@
private final ICheckpointManager checkpointManager;
private SystemState state;
private final INCServiceContext serviceCtx;
+ private final INcApplicationContext appCtx;
+
public RecoveryManager(ITransactionSubsystem txnSubsystem, INCServiceContext serviceCtx) {
this.serviceCtx = serviceCtx;
this.txnSubsystem = txnSubsystem;
+ this.appCtx = txnSubsystem.getAsterixAppRuntimeContextProvider().getAppContext();
logMgr = (LogManager) txnSubsystem.getLogManager();
- ReplicationProperties repProperties = txnSubsystem.getAsterixAppRuntimeContextProvider().getAppContext()
- .getReplicationProperties();
+ ReplicationProperties repProperties = appCtx.getReplicationProperties();
replicationEnabled = repProperties.isReplicationEnabled();
- localResourceRepository = (PersistentLocalResourceRepository) txnSubsystem.getAsterixAppRuntimeContextProvider()
- .getLocalResourceRepository();
+ localResourceRepository = (PersistentLocalResourceRepository) appCtx.getLocalResourceRepository();
cachedEntityCommitsPerJobSize = txnSubsystem.getTransactionProperties().getJobRecoveryMemorySize();
checkpointManager = txnSubsystem.getCheckpointManager();
}
@@ -129,32 +130,18 @@
LOGGER.info("The checkpoint file doesn't exist: systemState = PERMANENT_DATA_LOSS");
return state;
}
-
- if (replicationEnabled) {
- if (checkpointObject.getMinMCTFirstLsn() == AbstractCheckpointManager.SHARP_CHECKPOINT_LSN) {
- //no logs exist
- state = SystemState.HEALTHY;
- } else if (checkpointObject.getCheckpointLsn() == logMgr.getAppendLSN() && checkpointObject.isSharp()) {
- //only remote logs exist
- state = SystemState.HEALTHY;
- } else {
- //need to perform remote recovery
- state = SystemState.CORRUPTED;
+ long readableSmallestLSN = logMgr.getReadableSmallestLSN();
+ if (logMgr.getAppendLSN() == readableSmallestLSN) {
+ if (checkpointObject.getMinMCTFirstLsn() != AbstractCheckpointManager.SHARP_CHECKPOINT_LSN) {
+ LOGGER.warn("Some(or all) of transaction log files are lost.");
+ //No choice but continuing when the log files are lost.
}
+ state = SystemState.HEALTHY;
+ } else if (checkpointObject.getCheckpointLsn() == logMgr.getAppendLSN()
+ && checkpointObject.getMinMCTFirstLsn() == AbstractCheckpointManager.SHARP_CHECKPOINT_LSN) {
+ state = SystemState.HEALTHY;
} else {
- long readableSmallestLSN = logMgr.getReadableSmallestLSN();
- if (logMgr.getAppendLSN() == readableSmallestLSN) {
- if (checkpointObject.getMinMCTFirstLsn() != AbstractCheckpointManager.SHARP_CHECKPOINT_LSN) {
- LOGGER.warn("Some(or all) of transaction log files are lost.");
- //No choice but continuing when the log files are lost.
- }
- state = SystemState.HEALTHY;
- } else if (checkpointObject.getCheckpointLsn() == logMgr.getAppendLSN()
- && checkpointObject.getMinMCTFirstLsn() == AbstractCheckpointManager.SHARP_CHECKPOINT_LSN) {
- state = SystemState.HEALTHY;
- } else {
- state = SystemState.CORRUPTED;
- }
+ state = SystemState.CORRUPTED;
}
return state;
}
@@ -442,9 +429,47 @@
}
private long getRemoteMinFirstLSN() throws HyracksDataException {
- IReplicaResourcesManager remoteResourcesManager = txnSubsystem.getAsterixAppRuntimeContextProvider()
- .getAppContext().getReplicaResourcesManager();
- return remoteResourcesManager.getPartitionsMinLSN(localResourceRepository.getInactivePartitions());
+ // find the min first lsn of partitions that are replicated on this node
+ final Set<Integer> allPartitions = localResourceRepository.getAllPartitions();
+ final INcApplicationContext appContext = txnSubsystem.getAsterixAppRuntimeContextProvider().getAppContext();
+ final Set<Integer> masterPartitions = appContext.getReplicaManager().getPartitions();
+ allPartitions.removeAll(masterPartitions);
+ return getPartitionsMinLSN(allPartitions);
+ }
+
+ private long getPartitionsMinLSN(Set<Integer> partitions) throws HyracksDataException {
+ final IIndexCheckpointManagerProvider idxCheckpointMgrProvider = appCtx.getIndexCheckpointManagerProvider();
+ long minRemoteLSN = Long.MAX_VALUE;
+ for (Integer partition : partitions) {
+ final List<DatasetResourceReference> partitionResources = localResourceRepository.getResources(resource -> {
+ DatasetLocalResource dsResource = (DatasetLocalResource) resource.getResource();
+ return dsResource.getPartition() == partition;
+ }).values().stream().map(DatasetResourceReference::of).collect(Collectors.toList());
+ for (DatasetResourceReference indexRef : partitionResources) {
+ long remoteIndexMaxLSN = idxCheckpointMgrProvider.get(indexRef).getLowWatermark();
+ minRemoteLSN = Math.min(minRemoteLSN, remoteIndexMaxLSN);
+ }
+ }
+ return minRemoteLSN;
+ }
+
+ @Override
+ public void replayReplicaPartitionLogs(Set<Integer> partitions, boolean flush) throws HyracksDataException {
+ long minLSN = getPartitionsMinLSN(partitions);
+ long readableSmallestLSN = logMgr.getReadableSmallestLSN();
+ if (minLSN < readableSmallestLSN) {
+ minLSN = readableSmallestLSN;
+ }
+
+ //replay logs > minLSN that belong to these partitions
+ try {
+ replayPartitionsLogs(partitions, logMgr.getLogReader(true), minLSN);
+ if (flush) {
+ appCtx.getDatasetLifecycleManager().flushAllDatasets();
+ }
+ } catch (IOException | ACIDException e) {
+ throw HyracksDataException.create(e);
+ }
}
@Override
@@ -535,7 +560,7 @@
TxnEntityId loserEntity;
List<Long> undoLSNSet = null;
//get active partitions on this node
- Set<Integer> activePartitions = localResourceRepository.getActivePartitions();
+ Set<Integer> activePartitions = appCtx.getReplicaManager().getPartitions();
ILogReader logReader = logMgr.getLogReader(false);
try {
logReader.setPosition(firstLSN);
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/ReplicaManager.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/ReplicaManager.java
index bf17a5b..155fa1d 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/ReplicaManager.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/ReplicaManager.java
@@ -29,10 +29,10 @@
import org.apache.asterix.common.api.INcApplicationContext;
import org.apache.asterix.common.replication.IPartitionReplica;
-import org.apache.asterix.common.replication.IRemoteRecoveryManager;
import org.apache.asterix.common.storage.IReplicaManager;
import org.apache.asterix.common.storage.ReplicaIdentifier;
-import org.apache.asterix.replication.storage.PartitionReplica;
+import org.apache.asterix.common.transactions.IRecoveryManager;
+import org.apache.asterix.replication.api.PartitionReplica;
import org.apache.hyracks.api.exceptions.HyracksDataException;
public class ReplicaManager implements IReplicaManager {
@@ -67,7 +67,9 @@
if (!replicas.containsKey(id)) {
throw new IllegalStateException("replica with id(" + id + ") does not exist");
}
- replicas.remove(id);
+ PartitionReplica replica = replicas.remove(id);
+ appCtx.getReplicationManager().unregister(replica);
+
}
@Override
@@ -83,8 +85,8 @@
@Override
public synchronized void promote(int partition) throws HyracksDataException {
- final IRemoteRecoveryManager remoteRecoveryManager = appCtx.getRemoteRecoveryManager();
- remoteRecoveryManager.replayReplicaPartitionLogs(Stream.of(partition).collect(Collectors.toSet()), true);
+ final IRecoveryManager recoveryManager = appCtx.getTransactionSubsystem().getRecoveryManager();
+ recoveryManager.replayReplicaPartitionLogs(Stream.of(partition).collect(Collectors.toSet()), true);
partitions.add(partition);
}
}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/TransactionSubsystem.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/TransactionSubsystem.java
index cd9b617..fae0413 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/TransactionSubsystem.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/TransactionSubsystem.java
@@ -76,7 +76,7 @@
if (LOGGER.isInfoEnabled()) {
LOGGER.log(Level.INFO, "Checkpoint Properties: " + checkpointProperties);
}
- checkpointManager = CheckpointManagerFactory.create(this, checkpointProperties, replicationEnabled);
+ checkpointManager = CheckpointManagerFactory.create(this, checkpointProperties);
final Checkpoint latestCheckpoint = checkpointManager.getLatest();
if (latestCheckpoint != null) {
transactionManager.ensureMaxTxnId(latestCheckpoint.getMaxTxnId());
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/task/RemoteRecoveryTask.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/task/RemoteRecoveryTask.java
deleted file mode 100644
index d330684..0000000
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/task/RemoteRecoveryTask.java
+++ /dev/null
@@ -1,57 +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.app.nc.task;
-
-import java.io.IOException;
-import java.util.Map;
-import java.util.Set;
-
-import org.apache.asterix.common.api.INCLifecycleTask;
-import org.apache.asterix.common.api.INcApplicationContext;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.api.service.IControllerService;
-
-public class RemoteRecoveryTask implements INCLifecycleTask {
-
- private static final long serialVersionUID = 1L;
- private final Map<String, Set<Integer>> recoveryPlan;
-
- public RemoteRecoveryTask(Map<String, Set<Integer>> recoveryPlan) {
- this.recoveryPlan = recoveryPlan;
- }
-
- @Override
- public void perform(IControllerService cs) throws HyracksDataException {
- INcApplicationContext appContext = (INcApplicationContext) cs.getApplicationContext();
- appContext.getRemoteRecoveryManager().doRemoteRecoveryPlan(recoveryPlan);
- }
-
- private void writeObject(java.io.ObjectOutputStream out) throws IOException {
- out.defaultWriteObject();
- }
-
- private void readObject(java.io.ObjectInputStream in) throws IOException, ClassNotFoundException {
- in.defaultReadObject();
- }
-
- @Override
- public String toString() {
- return "{ \"class\" : \"" + getClass().getSimpleName() + "\", \"recovery-plan\" : " + recoveryPlan + " }";
- }
-}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/task/StartFailbackTask.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/task/StartFailbackTask.java
deleted file mode 100644
index ecd93a3..0000000
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/task/StartFailbackTask.java
+++ /dev/null
@@ -1,40 +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.app.nc.task;
-
-import org.apache.asterix.common.api.INCLifecycleTask;
-import org.apache.asterix.common.api.INcApplicationContext;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.api.service.IControllerService;
-
-public class StartFailbackTask implements INCLifecycleTask {
-
- private static final long serialVersionUID = 1L;
-
- @Override
- public void perform(IControllerService cs) throws HyracksDataException {
- INcApplicationContext appContext = (INcApplicationContext) cs.getApplicationContext();
- appContext.getRemoteRecoveryManager().startFailbackProcess();
- }
-
- @Override
- public String toString() {
- return "{ \"class\" : \"" + getClass().getSimpleName() + "\" }";
- }
-}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/task/StartReplicationServiceTask.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/task/StartReplicationServiceTask.java
index 253f121..7071271 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/task/StartReplicationServiceTask.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/task/StartReplicationServiceTask.java
@@ -20,7 +20,6 @@
import org.apache.asterix.common.api.INCLifecycleTask;
import org.apache.asterix.common.api.INcApplicationContext;
-import org.apache.asterix.common.replication.IReplicationManager;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.api.service.IControllerService;
@@ -32,13 +31,8 @@
public void perform(IControllerService cs) throws HyracksDataException {
INcApplicationContext appContext = (INcApplicationContext) cs.getApplicationContext();
try {
- //Open replication channel
+ // open replication channel
appContext.getReplicationChannel().start();
- final IReplicationManager replicationManager = appContext.getReplicationManager();
- //Check the state of remote replicas
- replicationManager.initializeReplicasState();
- //Start replication after the state of remote replicas has been initialized.
- replicationManager.startReplicationThreads();
} catch (Exception e) {
throw HyracksDataException.create(e);
}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/AutoFaultToleranceStrategy.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/AutoFaultToleranceStrategy.java
deleted file mode 100644
index 37b4d4f..0000000
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/AutoFaultToleranceStrategy.java
+++ /dev/null
@@ -1,542 +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.app.replication;
-
-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;
-import java.util.stream.Collectors;
-
-import org.apache.asterix.app.nc.task.BindMetadataNodeTask;
-import org.apache.asterix.app.nc.task.CheckpointTask;
-import org.apache.asterix.app.nc.task.ExternalLibrarySetupTask;
-import org.apache.asterix.app.nc.task.MetadataBootstrapTask;
-import org.apache.asterix.app.nc.task.ReportLocalCountersTask;
-import org.apache.asterix.app.nc.task.StartFailbackTask;
-import org.apache.asterix.app.nc.task.StartLifecycleComponentsTask;
-import org.apache.asterix.app.nc.task.StartReplicationServiceTask;
-import org.apache.asterix.app.replication.NodeFailbackPlan.FailbackPlanState;
-import org.apache.asterix.app.replication.message.CompleteFailbackRequestMessage;
-import org.apache.asterix.app.replication.message.CompleteFailbackResponseMessage;
-import org.apache.asterix.app.replication.message.NCLifecycleTaskReportMessage;
-import org.apache.asterix.app.replication.message.PreparePartitionsFailbackRequestMessage;
-import org.apache.asterix.app.replication.message.PreparePartitionsFailbackResponseMessage;
-import org.apache.asterix.app.replication.message.RegistrationTasksRequestMessage;
-import org.apache.asterix.app.replication.message.RegistrationTasksResponseMessage;
-import org.apache.asterix.app.replication.message.MetadataNodeRequestMessage;
-import org.apache.asterix.app.replication.message.MetadataNodeResponseMessage;
-import org.apache.asterix.app.replication.message.TakeoverPartitionsRequestMessage;
-import org.apache.asterix.app.replication.message.TakeoverPartitionsResponseMessage;
-import org.apache.asterix.common.api.IClusterManagementWork.ClusterState;
-import org.apache.asterix.common.api.INCLifecycleTask;
-import org.apache.asterix.common.cluster.ClusterPartition;
-import org.apache.asterix.common.cluster.IClusterStateManager;
-import org.apache.asterix.common.dataflow.ICcApplicationContext;
-import org.apache.asterix.common.exceptions.ErrorCode;
-import org.apache.asterix.common.exceptions.RuntimeDataException;
-import org.apache.asterix.common.messaging.api.ICCMessageBroker;
-import org.apache.asterix.common.replication.IFaultToleranceStrategy;
-import org.apache.asterix.common.replication.INCLifecycleMessage;
-import org.apache.asterix.common.replication.IReplicationStrategy;
-import org.apache.asterix.common.replication.Replica;
-import org.apache.asterix.common.transactions.IRecoveryManager.SystemState;
-import org.apache.asterix.metadata.MetadataManager;
-import org.apache.asterix.util.FaultToleranceUtil;
-import org.apache.hyracks.api.application.ICCServiceContext;
-import org.apache.hyracks.api.application.IClusterLifecycleListener.ClusterEventType;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.logging.log4j.Level;
-import org.apache.logging.log4j.LogManager;
-import org.apache.logging.log4j.Logger;
-
-public class AutoFaultToleranceStrategy implements IFaultToleranceStrategy {
-
- private static final Logger LOGGER = LogManager.getLogger();
- private long clusterRequestId = 0;
-
- private Set<String> failedNodes = new HashSet<>();
- private LinkedList<NodeFailbackPlan> pendingProcessingFailbackPlans = new LinkedList<>();
- private Map<Long, NodeFailbackPlan> planId2FailbackPlanMap = new HashMap<>();
- private Map<Long, TakeoverPartitionsRequestMessage> pendingTakeoverRequests = new HashMap<>();;
- private String currentMetadataNode;
- private boolean metadataNodeActive = false;
- private IClusterStateManager clusterManager;
- private ICCMessageBroker messageBroker;
- private IReplicationStrategy replicationStrategy;
- private ICCServiceContext serviceCtx;
- private Set<String> pendingStartupCompletionNodes = new HashSet<>();
- private List<String> nodeIds;
- private Map<String, SystemState> startupQueue = new HashMap<>();
-
- @Override
- public void notifyNodeJoin(String nodeId) {
- pendingStartupCompletionNodes.add(nodeId);
- }
-
- @Override
- public void notifyNodeFailure(String nodeId) throws HyracksDataException {
- //if this node was waiting for failback and failed before it completed
- if (failedNodes.contains(nodeId)) {
- notifyFailbackPlansNodeFailure(nodeId);
- revertFailedFailbackPlanEffects();
- return;
- }
- //an active node failed
- failedNodes.add(nodeId);
- clusterManager.updateNodePartitions(nodeId, false);
- if (nodeId.equals(currentMetadataNode)) {
- metadataNodeActive = false;
- clusterManager.updateMetadataNode(nodeId, metadataNodeActive);
- }
- validateClusterState();
- FaultToleranceUtil.notifyImpactedReplicas(nodeId, ClusterEventType.NODE_FAILURE, clusterManager, messageBroker,
- replicationStrategy);
- notifyFailbackPlansNodeFailure(nodeId);
- requestPartitionsTakeover(nodeId);
- }
-
- private synchronized void notifyFailbackPlansNodeFailure(String nodeId) {
- for (NodeFailbackPlan plan : planId2FailbackPlanMap.values()) {
- plan.notifyNodeFailure(nodeId);
- }
- }
-
- 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 requestPartitionsTakeover(String failedNodeId) {
- //replica -> list of partitions to takeover
- Map<String, List<Integer>> partitionRecoveryPlan = new HashMap<>();
- //collect the partitions of the failed NC
- List<ClusterPartition> lostPartitions = getNodeAssignedPartitions(failedNodeId);
- if (!lostPartitions.isEmpty()) {
- for (ClusterPartition partition : lostPartitions) {
- //find replicas for this partitions
- Set<String> partitionReplicas = replicationStrategy.getRemoteReplicas(partition.getNodeId()).stream()
- .map(Replica::getId).collect(Collectors.toSet());
- //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 (addActiveReplica(replica, partition, partitionRecoveryPlan)) {
- break;
- }
- }
- }
-
- if (partitionRecoveryPlan.size() == 0) {
- //no active replicas were found for the failed node
- LOGGER.error("Could not find active replicas for the partitions " + lostPartitions);
- return;
- } else {
- LOGGER.info("Partitions to recover: " + lostPartitions);
- }
- //For each replica, send a request to takeover the assigned partitions
- partitionRecoveryPlan.forEach((replica, value) -> {
- Integer[] partitionsToTakeover = value.toArray(new Integer[value.size()]);
- 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.log(Level.WARN, "Failed to send takeover request: " + takeoverRequest, e);
- }
- });
- }
- }
-
- private boolean addActiveReplica(String replica, ClusterPartition partition,
- Map<String, List<Integer>> partitionRecoveryPlan) {
- final Set<String> participantNodes = clusterManager.getParticipantNodes();
- if (participantNodes.contains(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());
- }
- return true;
- }
- return false;
- }
-
- 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
- Set<String> nodeReplicas = replicationStrategy.getRemoteReplicas(failingBackNodeId).stream().map(Replica::getId)
- .collect(Collectors.toSet());
- clusterManager.getClusterPartitons();
- for (String replicaId : nodeReplicas) {
- ClusterPartition[] nodePartitions = clusterManager.getNodePartitions(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.isInfoEnabled()) {
- LOGGER.info("Prepared Failback plan: " + plan.toString());
- }
-
- processPendingFailbackPlans();
- }
-
- private synchronized void processPendingFailbackPlans() {
- ClusterState state = clusterManager.getState();
- /*
- * 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()) {
- //partition expected to be returned to the failing back node
- clusterManager.updateClusterPartition(partitionId, failbackNode, false);
- }
-
- /*
- * if the returning node is the original metadata node,
- * then metadata node will change after the failback completes
- */
- ICcApplicationContext appCtx = (ICcApplicationContext) serviceCtx.getApplicationContext();
- String originalMetadataNode = appCtx.getMetadataProperties().getMetadataNodeName();
- if (originalMetadataNode.equals(failbackNode)) {
- plan.setNodeToReleaseMetadataManager(currentMetadataNode);
- currentMetadataNode = "";
- metadataNodeActive = false;
- clusterManager.updateMetadataNode(currentMetadataNode, metadataNodeActive);
- }
-
- //force new jobs to wait
- clusterManager.setState(ClusterState.REBALANCING);
- handleFailbackRequests(plan, messageBroker);
- /*
- * 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());
- }
- }
- }
- }
-
- private void handleFailbackRequests(NodeFailbackPlan plan, ICCMessageBroker messageBroker) {
- //send requests to other nodes to complete on-going jobs and prepare partitions for failback
- for (PreparePartitionsFailbackRequestMessage request : plan.getPlanFailbackRequests()) {
- try {
- messageBroker.sendApplicationMessageToNC(request, request.getNodeID());
- plan.addPendingRequest(request);
- } catch (Exception e) {
- LOGGER.log(Level.WARN, "Failed to send failback request to: " + request.getNodeID(), e);
- plan.notifyNodeFailure(request.getNodeID());
- revertFailedFailbackPlanEffects();
- break;
- }
- }
- }
-
- public synchronized List<ClusterPartition> getNodeAssignedPartitions(String nodeId) {
- List<ClusterPartition> nodePartitions = new ArrayList<>();
- ClusterPartition[] clusterPartitons = clusterManager.getClusterPartitons();
- Map<Integer, ClusterPartition> clusterPartitionsMap = new HashMap<>();
- for (ClusterPartition partition : clusterPartitons) {
- clusterPartitionsMap.put(partition.getPartitionId(), partition);
- }
- for (ClusterPartition partition : clusterPartitons) {
- if (nodeId.equals(partition.getActiveNodeId())) {
- nodePartitions.add(partition);
- }
- }
- /*
- * if there is any pending takeover request this node was supposed to handle,
- * it needs to be sent to a different replica
- */
- List<Long> failedTakeoverRequests = new ArrayList<>();
- for (TakeoverPartitionsRequestMessage request : pendingTakeoverRequests.values()) {
- if (request.getNodeId().equals(nodeId)) {
- for (Integer partitionId : request.getPartitions()) {
- nodePartitions.add(clusterPartitionsMap.get(partitionId));
- }
- failedTakeoverRequests.add(request.getRequestId());
- }
- }
-
- //remove failed requests
- for (Long requestId : failedTakeoverRequests) {
- pendingTakeoverRequests.remove(requestId);
- }
- return nodePartitions;
- }
-
- public synchronized void process(TakeoverPartitionsResponseMessage response) throws HyracksDataException {
- for (Integer partitonId : response.getPartitions()) {
- clusterManager.updateClusterPartition(partitonId, response.getNodeId(), true);
- }
- pendingTakeoverRequests.remove(response.getRequestId());
- validateClusterState();
- }
-
- public synchronized void process(MetadataNodeResponseMessage response) throws HyracksDataException {
- currentMetadataNode = response.getNodeId();
- metadataNodeActive = true;
- clusterManager.updateMetadataNode(currentMetadataNode, metadataNodeActive);
- validateClusterState();
- }
-
- private void validateClusterState() throws HyracksDataException {
- clusterManager.refreshState();
- ClusterState newState = clusterManager.getState();
- // PENDING: all partitions are active but metadata node is not
- if (newState == ClusterState.PENDING) {
- requestMetadataNodeTakeover();
- } else if (newState == ClusterState.ACTIVE) {
- processPendingFailbackPlans();
- }
- }
-
- public synchronized void process(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
- try {
- messageBroker.sendApplicationMessageToNC(request, request.getNodeId());
- } catch (Exception e) {
- LOGGER.log(Level.WARN, "Failed to send complete failback request to: " + request.getNodeId(), e);
- notifyFailbackPlansNodeFailure(request.getNodeId());
- revertFailedFailbackPlanEffects();
- }
- } else if (plan.getState() == FailbackPlanState.PENDING_ROLLBACK) {
- revertFailedFailbackPlanEffects();
- }
- }
-
- public synchronized void process(CompleteFailbackResponseMessage response) throws HyracksDataException {
- /*
- * 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(response.getPlanId());
- String nodeId = plan.getNodeId();
- failedNodes.remove(nodeId);
- //notify impacted replicas they can reconnect to this node
- FaultToleranceUtil.notifyImpactedReplicas(nodeId, ClusterEventType.NODE_JOIN, clusterManager, messageBroker,
- replicationStrategy);
- clusterManager.updateNodePartitions(nodeId, true);
- validateClusterState();
- }
-
- private synchronized void requestMetadataNodeTakeover() {
- //need a new node to takeover metadata node
- ICcApplicationContext appCtx = (ICcApplicationContext) serviceCtx.getApplicationContext();
- ClusterPartition metadataPartiton = appCtx.getMetadataProperties().getMetadataPartition();
- //request the metadataPartition node to register itself as the metadata node
- MetadataNodeRequestMessage takeoverRequest = new MetadataNodeRequestMessage(true);
- try {
- messageBroker.sendApplicationMessageToNC(takeoverRequest, metadataPartiton.getActiveNodeId());
- // Since the metadata node will be changed, we need to rebind the proxy object
- MetadataManager.INSTANCE.rebindMetadataNode();
- } 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, a new NC will be assigned to
- * the metadata partition and a new metadata node takeover request will be sent to it.
- */
- LOGGER.log(Level.WARN,
- "Failed to send metadata node takeover request to: " + metadataPartiton.getActiveNodeId(), e);
- }
- }
-
- @Override
- public IFaultToleranceStrategy from(ICCServiceContext serviceCtx, IReplicationStrategy replicationStrategy) {
- AutoFaultToleranceStrategy ft = new AutoFaultToleranceStrategy();
- ft.messageBroker = (ICCMessageBroker) serviceCtx.getMessageBroker();
- ft.replicationStrategy = replicationStrategy;
- ft.serviceCtx = serviceCtx;
- ft.nodeIds = serviceCtx.getAppConfig().getNCNames();
- return ft;
- }
-
- @Override
- public synchronized void process(INCLifecycleMessage message) throws HyracksDataException {
- switch (message.getType()) {
- case REGISTRATION_TASKS_REQUEST:
- process((RegistrationTasksRequestMessage) message);
- break;
- case REGISTRATION_TASKS_RESULT:
- process((NCLifecycleTaskReportMessage) message);
- break;
- case TAKEOVER_PARTITION_RESPONSE:
- process((TakeoverPartitionsResponseMessage) message);
- break;
- case METADATA_NODE_RESPONSE:
- process((MetadataNodeResponseMessage) message);
- break;
- case PREPARE_FAILBACK_RESPONSE:
- process((PreparePartitionsFailbackResponseMessage) message);
- break;
- case COMPLETE_FAILBACK_RESPONSE:
- process((CompleteFailbackResponseMessage) message);
- break;
- default:
- throw new RuntimeDataException(ErrorCode.UNSUPPORTED_MESSAGE_TYPE, message.getType().name());
- }
- }
-
- private synchronized void process(NCLifecycleTaskReportMessage msg) throws HyracksDataException {
- final String nodeId = msg.getNodeId();
- pendingStartupCompletionNodes.remove(nodeId);
- if (msg.isSuccess()) {
- if (failedNodes.contains(nodeId)) {
- prepareFailbackPlan(nodeId);
- return;
- }
- // If this node failed and recovered, notify impacted replicas to reconnect to it
- if (replicationStrategy.isParticipant(nodeId) && failedNodes.remove(nodeId)) {
- FaultToleranceUtil.notifyImpactedReplicas(nodeId, ClusterEventType.NODE_JOIN, clusterManager,
- messageBroker, replicationStrategy);
- }
- clusterManager.updateNodePartitions(nodeId, true);
- if (msg.getNodeId().equals(currentMetadataNode)) {
- clusterManager.updateMetadataNode(currentMetadataNode, true);
- }
- clusterManager.refreshState();
- } else {
- LOGGER.log(Level.ERROR, msg.getNodeId() + " failed to complete startup. ", msg.getException());
- }
- }
-
- @Override
- public void bindTo(IClusterStateManager clusterManager) {
- this.clusterManager = clusterManager;
- currentMetadataNode = clusterManager.getCurrentMetadataNodeId();
- }
-
- private synchronized void process(RegistrationTasksRequestMessage msg) throws HyracksDataException {
- final String nodeId = msg.getNodeId();
- final SystemState state = msg.getState();
- //last node needed to start
- if (startupQueue.keySet().size() == nodeIds.size() - 1) {
- startupQueue.put(nodeId, state);
- for (Map.Entry<String, SystemState> nodeState : startupQueue.entrySet()) {
- List<INCLifecycleTask> tasks = buildStartupSequence(nodeState.getKey());
- RegistrationTasksResponseMessage response = new RegistrationTasksResponseMessage(nodeState.getKey(),
- tasks);
- try {
- messageBroker.sendApplicationMessageToNC(response, nodeState.getKey());
- } catch (Exception e) {
- throw HyracksDataException.create(e);
- }
- }
- } else if (!failedNodes.isEmpty()) {
- List<INCLifecycleTask> tasks = buildFailbackStartupSequence();
- RegistrationTasksResponseMessage response = new RegistrationTasksResponseMessage(nodeId, tasks);
- try {
- messageBroker.sendApplicationMessageToNC(response, msg.getNodeId());
- } catch (Exception e) {
- throw HyracksDataException.create(e);
- }
- } else {
- startupQueue.put(nodeId, state);
- }
- }
-
- private List<INCLifecycleTask> buildFailbackStartupSequence() {
- final List<INCLifecycleTask> tasks = new ArrayList<>();
- tasks.add(new StartFailbackTask());
- tasks.add(new ReportLocalCountersTask());
- tasks.add(new StartLifecycleComponentsTask());
- return tasks;
- }
-
- private List<INCLifecycleTask> buildStartupSequence(String nodeId) {
- final List<INCLifecycleTask> tasks = new ArrayList<>();
- tasks.add(new StartReplicationServiceTask());
- final boolean isMetadataNode = nodeId.equals(currentMetadataNode);
- if (isMetadataNode) {
- tasks.add(new MetadataBootstrapTask());
- }
- tasks.add(new ExternalLibrarySetupTask(isMetadataNode));
- tasks.add(new ReportLocalCountersTask());
- tasks.add(new CheckpointTask());
- tasks.add(new StartLifecycleComponentsTask());
- if (isMetadataNode) {
- tasks.add(new BindMetadataNodeTask(true));
- }
- return tasks;
- }
-}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/FaultToleranceStrategyFactory.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/FaultToleranceStrategyFactory.java
index fe24fca..5a715d6 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/FaultToleranceStrategyFactory.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/FaultToleranceStrategyFactory.java
@@ -18,9 +18,7 @@
*/
package org.apache.asterix.app.replication;
-import org.apache.asterix.common.config.ReplicationProperties;
import org.apache.asterix.common.replication.IFaultToleranceStrategy;
-import org.apache.asterix.common.replication.IReplicationStrategy;
import org.apache.hyracks.api.application.ICCServiceContext;
public class FaultToleranceStrategyFactory {
@@ -29,20 +27,10 @@
throw new AssertionError();
}
- public static final String STRATEGY_NAME = "metadata_only";
-
- public static IFaultToleranceStrategy create(ICCServiceContext serviceCtx, ReplicationProperties repProp,
- IReplicationStrategy strategy) {
- Class<? extends IFaultToleranceStrategy> clazz;
- if (!repProp.isReplicationEnabled()) {
- clazz = NoFaultToleranceStrategy.class;
- } else if (STRATEGY_NAME.equals(repProp.getReplicationStrategy())) {
- clazz = MetadataNodeFaultToleranceStrategy.class;
- } else {
- clazz = AutoFaultToleranceStrategy.class;
- }
+ public static IFaultToleranceStrategy create(ICCServiceContext serviceCtx, boolean replicationEnabled) {
+ Class<? extends IFaultToleranceStrategy> clazz = NoFaultToleranceStrategy.class;
try {
- return clazz.newInstance().from(serviceCtx, strategy);
+ return clazz.newInstance().from(serviceCtx, replicationEnabled);
} catch (InstantiationException | IllegalAccessException e) {
throw new IllegalStateException(e);
}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/MetadataNodeFaultToleranceStrategy.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/MetadataNodeFaultToleranceStrategy.java
deleted file mode 100644
index 02174da..0000000
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/MetadataNodeFaultToleranceStrategy.java
+++ /dev/null
@@ -1,304 +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.app.replication;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.stream.Collectors;
-
-import org.apache.asterix.app.nc.task.BindMetadataNodeTask;
-import org.apache.asterix.app.nc.task.CheckpointTask;
-import org.apache.asterix.app.nc.task.ExternalLibrarySetupTask;
-import org.apache.asterix.app.nc.task.LocalRecoveryTask;
-import org.apache.asterix.app.nc.task.MetadataBootstrapTask;
-import org.apache.asterix.app.nc.task.RemoteRecoveryTask;
-import org.apache.asterix.app.nc.task.ReportLocalCountersTask;
-import org.apache.asterix.app.nc.task.StartLifecycleComponentsTask;
-import org.apache.asterix.app.nc.task.StartReplicationServiceTask;
-import org.apache.asterix.app.replication.message.MetadataNodeRequestMessage;
-import org.apache.asterix.app.replication.message.MetadataNodeResponseMessage;
-import org.apache.asterix.app.replication.message.NCLifecycleTaskReportMessage;
-import org.apache.asterix.app.replication.message.RegistrationTasksRequestMessage;
-import org.apache.asterix.app.replication.message.ReplayPartitionLogsRequestMessage;
-import org.apache.asterix.app.replication.message.ReplayPartitionLogsResponseMessage;
-import org.apache.asterix.app.replication.message.RegistrationTasksResponseMessage;
-import org.apache.asterix.common.api.INCLifecycleTask;
-import org.apache.asterix.common.cluster.ClusterPartition;
-import org.apache.asterix.common.cluster.IClusterStateManager;
-import org.apache.asterix.common.dataflow.ICcApplicationContext;
-import org.apache.asterix.common.exceptions.ErrorCode;
-import org.apache.asterix.common.exceptions.RuntimeDataException;
-import org.apache.asterix.common.messaging.api.ICCMessageBroker;
-import org.apache.asterix.common.replication.IFaultToleranceStrategy;
-import org.apache.asterix.common.replication.INCLifecycleMessage;
-import org.apache.asterix.common.replication.IReplicationStrategy;
-import org.apache.asterix.common.replication.Replica;
-import org.apache.asterix.common.transactions.IRecoveryManager.SystemState;
-import org.apache.asterix.util.FaultToleranceUtil;
-import org.apache.hyracks.api.application.ICCServiceContext;
-import org.apache.hyracks.api.application.IClusterLifecycleListener.ClusterEventType;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.logging.log4j.Level;
-import org.apache.logging.log4j.LogManager;
-import org.apache.logging.log4j.Logger;
-
-public class MetadataNodeFaultToleranceStrategy implements IFaultToleranceStrategy {
-
- private static final Logger LOGGER = LogManager.getLogger();
- private IClusterStateManager clusterManager;
- private String metadataNodeId;
- private IReplicationStrategy replicationStrategy;
- private ICCMessageBroker messageBroker;
- private ICCServiceContext serviceCtx;
- private final Set<String> hotStandbyMetadataReplica = new HashSet<>();
- private final Set<String> failedNodes = new HashSet<>();
- private Set<String> pendingStartupCompletionNodes = new HashSet<>();
-
- @Override
- public synchronized void notifyNodeJoin(String nodeId) throws HyracksDataException {
- pendingStartupCompletionNodes.add(nodeId);
- }
-
- @Override
- public synchronized void notifyNodeFailure(String nodeId) throws HyracksDataException {
- failedNodes.add(nodeId);
- hotStandbyMetadataReplica.remove(nodeId);
- clusterManager.updateNodePartitions(nodeId, false);
- if (nodeId.equals(metadataNodeId)) {
- clusterManager.updateMetadataNode(metadataNodeId, false);
- }
- clusterManager.refreshState();
- if (replicationStrategy.isParticipant(nodeId)) {
- // Notify impacted replica
- FaultToleranceUtil.notifyImpactedReplicas(nodeId, ClusterEventType.NODE_FAILURE, clusterManager,
- messageBroker, replicationStrategy);
- }
- // If the failed node is the metadata node, ask its replicas to replay any committed jobs
- if (nodeId.equals(metadataNodeId)) {
- ICcApplicationContext appCtx = (ICcApplicationContext) serviceCtx.getApplicationContext();
- int metadataPartitionId = appCtx.getMetadataProperties().getMetadataPartition().getPartitionId();
- Set<Integer> metadataPartition = new HashSet<>(Arrays.asList(metadataPartitionId));
- Set<Replica> activeRemoteReplicas = replicationStrategy.getRemoteReplicas(metadataNodeId).stream()
- .filter(replica -> !failedNodes.contains(replica.getId())).collect(Collectors.toSet());
- //TODO Do election to identity the node with latest state
- for (Replica replica : activeRemoteReplicas) {
- ReplayPartitionLogsRequestMessage msg = new ReplayPartitionLogsRequestMessage(metadataPartition);
- try {
- messageBroker.sendApplicationMessageToNC(msg, replica.getId());
- } catch (Exception e) {
- LOGGER.log(Level.WARN, "Failed sending an application message to an NC", e);
- continue;
- }
- }
- }
- }
-
- @Override
- public IFaultToleranceStrategy from(ICCServiceContext serviceCtx, IReplicationStrategy replicationStrategy) {
- MetadataNodeFaultToleranceStrategy ft = new MetadataNodeFaultToleranceStrategy();
- ft.replicationStrategy = replicationStrategy;
- ft.messageBroker = (ICCMessageBroker) serviceCtx.getMessageBroker();
- ft.serviceCtx = serviceCtx;
- return ft;
- }
-
- @Override
- public synchronized void process(INCLifecycleMessage message) throws HyracksDataException {
- switch (message.getType()) {
- case REGISTRATION_TASKS_REQUEST:
- process((RegistrationTasksRequestMessage) message);
- break;
- case REGISTRATION_TASKS_RESULT:
- process((NCLifecycleTaskReportMessage) message);
- break;
- case REPLAY_LOGS_RESPONSE:
- process((ReplayPartitionLogsResponseMessage) message);
- break;
- case METADATA_NODE_RESPONSE:
- process((MetadataNodeResponseMessage) message);
- break;
- default:
- throw new RuntimeDataException(ErrorCode.UNSUPPORTED_MESSAGE_TYPE, message.getType().name());
- }
- }
-
- @Override
- public synchronized void bindTo(IClusterStateManager clusterManager) {
- this.clusterManager = clusterManager;
- this.metadataNodeId = clusterManager.getCurrentMetadataNodeId();
- }
-
- @Override
- public void notifyMetadataNodeChange(String node) throws HyracksDataException {
- if (metadataNodeId.equals(node)) {
- return;
- }
- // if current metadata node is active, we need to unbind its metadata proxy object
- if (clusterManager.isMetadataNodeActive()) {
- MetadataNodeRequestMessage msg = new MetadataNodeRequestMessage(false);
- try {
- messageBroker.sendApplicationMessageToNC(msg, metadataNodeId);
- // when the current node responses, we will bind to the new one
- metadataNodeId = node;
- } catch (Exception e) {
- throw HyracksDataException.create(e);
- }
- } else {
- requestMetadataNodeTakeover(node);
- }
- }
-
- private synchronized void process(ReplayPartitionLogsResponseMessage msg) {
- hotStandbyMetadataReplica.add(msg.getNodeId());
- if (LOGGER.isInfoEnabled()) {
- LOGGER.info("Hot Standby Metadata Replicas: " + hotStandbyMetadataReplica);
- }
- }
-
- private synchronized void process(RegistrationTasksRequestMessage msg) throws HyracksDataException {
- final String nodeId = msg.getNodeId();
- final SystemState state = msg.getState();
- final boolean isParticipant = replicationStrategy.isParticipant(nodeId);
- List<INCLifecycleTask> tasks;
- if (!isParticipant) {
- tasks = buildNonParticipantStartupSequence(nodeId, state);
- } else {
- tasks = buildParticipantStartupSequence(nodeId, state);
- }
- RegistrationTasksResponseMessage response = new RegistrationTasksResponseMessage(nodeId, tasks);
- try {
- messageBroker.sendApplicationMessageToNC(response, msg.getNodeId());
- } catch (Exception e) {
- throw HyracksDataException.create(e);
- }
- }
-
- private synchronized void process(NCLifecycleTaskReportMessage msg) throws HyracksDataException {
- final String nodeId = msg.getNodeId();
- pendingStartupCompletionNodes.remove(nodeId);
- if (msg.isSuccess()) {
- // If this node failed and recovered, notify impacted replicas to reconnect to it
- if (replicationStrategy.isParticipant(nodeId) && failedNodes.remove(nodeId)) {
- FaultToleranceUtil.notifyImpactedReplicas(nodeId, ClusterEventType.NODE_JOIN, clusterManager,
- messageBroker, replicationStrategy);
- }
- clusterManager.updateNodePartitions(msg.getNodeId(), true);
- if (msg.getNodeId().equals(metadataNodeId)) {
- clusterManager.updateMetadataNode(metadataNodeId, true);
- // When metadata node is active, it is the only hot standby replica
- hotStandbyMetadataReplica.clear();
- hotStandbyMetadataReplica.add(metadataNodeId);
- }
- clusterManager.refreshState();
- } else {
- LOGGER.log(Level.ERROR, msg.getNodeId() + " failed to complete startup. ", msg.getException());
- }
- }
-
- private List<INCLifecycleTask> buildNonParticipantStartupSequence(String nodeId, SystemState state) {
- final List<INCLifecycleTask> tasks = new ArrayList<>();
- if (state == SystemState.CORRUPTED) {
- //need to perform local recovery for node partitions
- LocalRecoveryTask rt = new LocalRecoveryTask(Arrays.asList(clusterManager.getNodePartitions(nodeId))
- .stream().map(ClusterPartition::getPartitionId).collect(Collectors.toSet()));
- tasks.add(rt);
- }
- tasks.add(new ExternalLibrarySetupTask(false));
- tasks.add(new ReportLocalCountersTask());
- tasks.add(new CheckpointTask());
- tasks.add(new StartLifecycleComponentsTask());
- return tasks;
- }
-
- private List<INCLifecycleTask> buildParticipantStartupSequence(String nodeId, SystemState state) {
- final List<INCLifecycleTask> tasks = new ArrayList<>();
- switch (state) {
- case PERMANENT_DATA_LOSS:
- if (failedNodes.isEmpty()) { //bootstrap
- break;
- }
- // If the metadata node (or replica) failed and lost its data
- // => Metadata Remote Recovery from standby replica
- tasks.add(getMetadataPartitionRecoveryPlan());
- // Checkpoint after remote recovery to move node to HEALTHY state
- tasks.add(new CheckpointTask());
- break;
- case CORRUPTED:
- // If the metadata node (or replica) failed and started again without losing data => Local Recovery
- LocalRecoveryTask rt = new LocalRecoveryTask(Arrays.asList(clusterManager.getNodePartitions(nodeId))
- .stream().map(ClusterPartition::getPartitionId).collect(Collectors.toSet()));
- tasks.add(rt);
- break;
- case BOOTSTRAPPING:
- case HEALTHY:
- case RECOVERING:
- break;
- default:
- break;
- }
- tasks.add(new StartReplicationServiceTask());
- final boolean isMetadataNode = nodeId.equals(metadataNodeId);
- if (isMetadataNode) {
- tasks.add(new MetadataBootstrapTask());
- }
- tasks.add(new ExternalLibrarySetupTask(isMetadataNode));
- tasks.add(new ReportLocalCountersTask());
- tasks.add(new CheckpointTask());
- tasks.add(new StartLifecycleComponentsTask());
- if (isMetadataNode) {
- tasks.add(new BindMetadataNodeTask(true));
- }
- return tasks;
- }
-
- private RemoteRecoveryTask getMetadataPartitionRecoveryPlan() {
- if (hotStandbyMetadataReplica.isEmpty()) {
- throw new IllegalStateException("No metadata replicas to recover from");
- }
- // Construct recovery plan: Node => Set of partitions to recover from it
- Map<String, Set<Integer>> recoveryPlan = new HashMap<>();
- // Recover metadata partition from any metadata hot standby replica
- ICcApplicationContext appCtx = (ICcApplicationContext) serviceCtx.getApplicationContext();
- int metadataPartitionId = appCtx.getMetadataProperties().getMetadataPartition().getPartitionId();
- Set<Integer> metadataPartition = new HashSet<>(Arrays.asList(metadataPartitionId));
- recoveryPlan.put(hotStandbyMetadataReplica.iterator().next(), metadataPartition);
- return new RemoteRecoveryTask(recoveryPlan);
- }
-
- private void process(MetadataNodeResponseMessage response) throws HyracksDataException {
- clusterManager.updateMetadataNode(response.getNodeId(), response.isExported());
- if (!response.isExported()) {
- requestMetadataNodeTakeover(metadataNodeId);
- }
- }
-
- private void requestMetadataNodeTakeover(String node) throws HyracksDataException {
- MetadataNodeRequestMessage msg = new MetadataNodeRequestMessage(true);
- try {
- messageBroker.sendApplicationMessageToNC(msg, node);
- } catch (Exception e) {
- throw HyracksDataException.create(e);
- }
- }
-}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/NoFaultToleranceStrategy.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/NoFaultToleranceStrategy.java
index 3c5442e..49f4b32 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/NoFaultToleranceStrategy.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/NoFaultToleranceStrategy.java
@@ -32,6 +32,9 @@
import org.apache.asterix.app.nc.task.MetadataBootstrapTask;
import org.apache.asterix.app.nc.task.ReportLocalCountersTask;
import org.apache.asterix.app.nc.task.StartLifecycleComponentsTask;
+import org.apache.asterix.app.nc.task.StartReplicationServiceTask;
+import org.apache.asterix.app.replication.message.MetadataNodeRequestMessage;
+import org.apache.asterix.app.replication.message.MetadataNodeResponseMessage;
import org.apache.asterix.app.replication.message.NCLifecycleTaskReportMessage;
import org.apache.asterix.app.replication.message.RegistrationTasksRequestMessage;
import org.apache.asterix.app.replication.message.RegistrationTasksResponseMessage;
@@ -43,8 +46,8 @@
import org.apache.asterix.common.messaging.api.ICCMessageBroker;
import org.apache.asterix.common.replication.IFaultToleranceStrategy;
import org.apache.asterix.common.replication.INCLifecycleMessage;
-import org.apache.asterix.common.replication.IReplicationStrategy;
import org.apache.asterix.common.transactions.IRecoveryManager.SystemState;
+import org.apache.asterix.metadata.MetadataManager;
import org.apache.hyracks.api.application.ICCServiceContext;
import org.apache.hyracks.api.client.NodeStatus;
import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -59,6 +62,7 @@
private String metadataNodeId;
private Set<String> pendingStartupCompletionNodes = new HashSet<>();
private ICCMessageBroker messageBroker;
+ private boolean replicationEnabled;
@Override
public void notifyNodeJoin(String nodeId) throws HyracksDataException {
@@ -84,15 +88,19 @@
case REGISTRATION_TASKS_RESULT:
process((NCLifecycleTaskReportMessage) message);
break;
+ case METADATA_NODE_RESPONSE:
+ process((MetadataNodeResponseMessage) message);
+ break;
default:
throw new RuntimeDataException(ErrorCode.UNSUPPORTED_MESSAGE_TYPE, message.getType().name());
}
}
@Override
- public IFaultToleranceStrategy from(ICCServiceContext serviceCtx, IReplicationStrategy replicationStrategy) {
+ public IFaultToleranceStrategy from(ICCServiceContext serviceCtx, boolean replicationEnabled) {
NoFaultToleranceStrategy ft = new NoFaultToleranceStrategy();
ft.messageBroker = (ICCMessageBroker) serviceCtx.getMessageBroker();
+ ft.replicationEnabled = replicationEnabled;
return ft;
}
@@ -141,10 +149,14 @@
final List<INCLifecycleTask> tasks = new ArrayList<>();
if (state == SystemState.CORRUPTED) {
//need to perform local recovery for node partitions
- LocalRecoveryTask rt = new LocalRecoveryTask(Arrays.asList(clusterManager.getNodePartitions(nodeId))
- .stream().map(ClusterPartition::getPartitionId).collect(Collectors.toSet()));
+ LocalRecoveryTask rt = new LocalRecoveryTask(
+ Arrays.asList(clusterManager.getNodePartitions(nodeId)).stream()
+ .map(ClusterPartition::getPartitionId).collect(Collectors.toSet()));
tasks.add(rt);
}
+ if (replicationEnabled) {
+ tasks.add(new StartReplicationServiceTask());
+ }
if (isMetadataNode) {
tasks.add(new MetadataBootstrapTask());
}
@@ -168,4 +180,42 @@
tasks.add(new ReportLocalCountersTask());
return tasks;
}
+
+ @Override
+ public void notifyMetadataNodeChange(String node) throws HyracksDataException {
+ if (metadataNodeId.equals(node)) {
+ return;
+ }
+ // if current metadata node is active, we need to unbind its metadata proxy object
+ if (clusterManager.isMetadataNodeActive()) {
+ MetadataNodeRequestMessage msg = new MetadataNodeRequestMessage(false);
+ try {
+ messageBroker.sendApplicationMessageToNC(msg, metadataNodeId);
+ // when the current node responses, we will bind to the new one
+ metadataNodeId = node;
+ } catch (Exception e) {
+ throw HyracksDataException.create(e);
+ }
+ } else {
+ requestMetadataNodeTakeover(node);
+ }
+ }
+
+ private void process(MetadataNodeResponseMessage response) throws HyracksDataException {
+ // rebind metadata node since it might be changing
+ MetadataManager.INSTANCE.rebindMetadataNode();
+ clusterManager.updateMetadataNode(response.getNodeId(), response.isExported());
+ if (!response.isExported()) {
+ requestMetadataNodeTakeover(metadataNodeId);
+ }
+ }
+
+ private void requestMetadataNodeTakeover(String node) throws HyracksDataException {
+ MetadataNodeRequestMessage msg = new MetadataNodeRequestMessage(true);
+ try {
+ messageBroker.sendApplicationMessageToNC(msg, node);
+ } catch (Exception e) {
+ throw HyracksDataException.create(e);
+ }
+ }
}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/NodeFailbackPlan.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/NodeFailbackPlan.java
deleted file mode 100644
index ab938d2..0000000
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/NodeFailbackPlan.java
+++ /dev/null
@@ -1,210 +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.app.replication;
-
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-
-import org.apache.asterix.app.replication.message.CompleteFailbackRequestMessage;
-import org.apache.asterix.app.replication.message.PreparePartitionsFailbackRequestMessage;
-import org.apache.asterix.app.replication.message.PreparePartitionsFailbackResponseMessage;
-
-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.isEmpty()) {
- 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<>();
- partition2nodeMap.forEach((key, value) -> {
- if (value.equals(participant)) {
- partitionToPrepareForFailback.add(key);
- }
- });
- 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/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/CompleteFailbackRequestMessage.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/CompleteFailbackRequestMessage.java
deleted file mode 100644
index ad4afd0..0000000
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/CompleteFailbackRequestMessage.java
+++ /dev/null
@@ -1,94 +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.app.replication.message;
-
-import java.io.IOException;
-import java.util.Set;
-
-import org.apache.asterix.common.api.INcApplicationContext;
-import org.apache.asterix.common.messaging.api.INCMessageBroker;
-import org.apache.asterix.common.messaging.api.INcAddressedMessage;
-import org.apache.asterix.common.replication.IRemoteRecoveryManager;
-import org.apache.asterix.runtime.message.AbstractFailbackPlanMessage;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.logging.log4j.Level;
-import org.apache.logging.log4j.LogManager;
-import org.apache.logging.log4j.Logger;
-
-public class CompleteFailbackRequestMessage extends AbstractFailbackPlanMessage implements INcAddressedMessage {
-
- private static final long serialVersionUID = 1L;
- private static final Logger LOGGER = LogManager.getLogger();
- 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;
- }
-
- public Set<Integer> getPartitions() {
- return partitions;
- }
-
- public String getNodeId() {
- return nodeId;
- }
-
- @Override
- public String toString() {
- StringBuilder sb = new StringBuilder();
- sb.append(CompleteFailbackRequestMessage.class.getSimpleName());
- sb.append(" Plan ID: " + planId);
- sb.append(" Node ID: " + nodeId);
- sb.append(" Partitions: " + partitions);
- return sb.toString();
- }
-
- @Override
- public void handle(INcApplicationContext appContext) throws HyracksDataException, InterruptedException {
- INCMessageBroker broker = (INCMessageBroker) appContext.getServiceContext().getMessageBroker();
- HyracksDataException hde = null;
- try {
- IRemoteRecoveryManager remoteRecoeryManager = appContext.getRemoteRecoveryManager();
- remoteRecoeryManager.completeFailbackProcess();
- } catch (IOException | InterruptedException e) {
- LOGGER.log(Level.ERROR, "Failure during completion of failback process", e);
- hde = HyracksDataException.create(e);
- } finally {
- CompleteFailbackResponseMessage reponse =
- new CompleteFailbackResponseMessage(planId, requestId, partitions);
- try {
- broker.sendMessageToCC(reponse);
- } catch (Exception e) {
- LOGGER.log(Level.ERROR, "Failure sending message to CC", e);
- hde = HyracksDataException.suppress(hde, e);
- }
- }
- if (hde != null) {
- throw hde;
- }
- }
-
- @Override
- public MessageType getType() {
- return MessageType.COMPLETE_FAILBACK_REQUEST;
- }
-}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/CompleteFailbackResponseMessage.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/CompleteFailbackResponseMessage.java
deleted file mode 100644
index 0c5678f..0000000
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/CompleteFailbackResponseMessage.java
+++ /dev/null
@@ -1,61 +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.app.replication.message;
-
-import java.util.Set;
-
-import org.apache.asterix.common.dataflow.ICcApplicationContext;
-import org.apache.asterix.common.messaging.api.ICcAddressedMessage;
-import org.apache.asterix.runtime.message.AbstractFailbackPlanMessage;
-import org.apache.asterix.runtime.utils.CcApplicationContext;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-public class CompleteFailbackResponseMessage extends AbstractFailbackPlanMessage implements ICcAddressedMessage {
-
- 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;
- }
-
- public Set<Integer> getPartitions() {
- return partitions;
- }
-
- @Override
- public String toString() {
- StringBuilder sb = new StringBuilder();
- sb.append(CompleteFailbackResponseMessage.class.getSimpleName());
- sb.append(" Plan ID: " + planId);
- sb.append(" Partitions: " + partitions);
- return sb.toString();
- }
-
- @Override
- public void handle(ICcApplicationContext appCtx) throws HyracksDataException, InterruptedException {
- ((CcApplicationContext) appCtx).getFaultToleranceStrategy().process(this);
- }
-
- @Override
- public MessageType getType() {
- return MessageType.COMPLETE_FAILBACK_RESPONSE;
- }
-}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/PreparePartitionsFailbackRequestMessage.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/PreparePartitionsFailbackRequestMessage.java
deleted file mode 100644
index 6b85050..0000000
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/PreparePartitionsFailbackRequestMessage.java
+++ /dev/null
@@ -1,122 +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.app.replication.message;
-
-import java.rmi.RemoteException;
-import java.util.Set;
-
-import org.apache.asterix.common.api.INcApplicationContext;
-import org.apache.asterix.common.messaging.api.INCMessageBroker;
-import org.apache.asterix.common.messaging.api.INcAddressedMessage;
-import org.apache.asterix.runtime.message.AbstractFailbackPlanMessage;
-import org.apache.asterix.transaction.management.resource.PersistentLocalResourceRepository;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.logging.log4j.Level;
-import org.apache.logging.log4j.LogManager;
-import org.apache.logging.log4j.Logger;
-
-public class PreparePartitionsFailbackRequestMessage extends AbstractFailbackPlanMessage
- implements INcAddressedMessage {
-
- private static final long serialVersionUID = 1L;
- private static final Logger LOGGER = LogManager.getLogger();
- 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;
- }
-
- 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(PreparePartitionsFailbackRequestMessage.class.getSimpleName());
- sb.append(" Plan ID: " + planId);
- sb.append(" Partitions: " + partitions);
- sb.append(" releaseMetadataNode: " + releaseMetadataNode);
- return sb.toString();
- }
-
- @Override
- public void handle(INcApplicationContext appContext) throws HyracksDataException, InterruptedException {
- INCMessageBroker broker = (INCMessageBroker) appContext.getServiceContext().getMessageBroker();
- /**
- * 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 (releaseMetadataNode) {
- appContext.getDatasetLifecycleManager().closeAllDatasets();
- //remove the metadata node stub from RMI registry
- try {
- appContext.unexportMetadataNodeStub();
- } catch (RemoteException e) {
- LOGGER.log(Level.ERROR, "Failed unexporting metadata stub", e);
- throw HyracksDataException.create(e);
- }
- } 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 : partitions) {
- localResourceRepo.addInactivePartition(partitionId);
- }
-
- //send response after partitions prepared for failback
- PreparePartitionsFailbackResponseMessage reponse =
- new PreparePartitionsFailbackResponseMessage(planId, requestId, partitions);
- try {
- broker.sendMessageToCC(reponse);
- } catch (Exception e) {
- LOGGER.log(Level.ERROR, "Failed sending message to cc", e);
- throw HyracksDataException.create(e);
- }
- }
-
- @Override
- public MessageType getType() {
- return MessageType.PREPARE_FAILBACK_REQUEST;
- }
-}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/PreparePartitionsFailbackResponseMessage.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/PreparePartitionsFailbackResponseMessage.java
deleted file mode 100644
index bea1039..0000000
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/PreparePartitionsFailbackResponseMessage.java
+++ /dev/null
@@ -1,58 +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.app.replication.message;
-
-import java.util.Set;
-
-import org.apache.asterix.common.dataflow.ICcApplicationContext;
-import org.apache.asterix.common.messaging.api.ICcAddressedMessage;
-import org.apache.asterix.runtime.message.AbstractFailbackPlanMessage;
-import org.apache.asterix.runtime.utils.CcApplicationContext;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-public class PreparePartitionsFailbackResponseMessage extends AbstractFailbackPlanMessage
- implements ICcAddressedMessage {
-
- 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;
- }
-
- public Set<Integer> getPartitions() {
- return partitions;
- }
-
- @Override
- public void handle(ICcApplicationContext appCtx) throws HyracksDataException, InterruptedException {
- ((CcApplicationContext) appCtx).getFaultToleranceStrategy().process(this);
- }
-
- @Override
- public String toString() {
- return PreparePartitionsFailbackResponseMessage.class.getSimpleName() + " " + partitions.toString();
- }
-
- @Override
- public MessageType getType() {
- return MessageType.PREPARE_FAILBACK_RESPONSE;
- }
-}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/ReplayPartitionLogsRequestMessage.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/ReplayPartitionLogsRequestMessage.java
deleted file mode 100644
index beac2b5..0000000
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/ReplayPartitionLogsRequestMessage.java
+++ /dev/null
@@ -1,63 +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.app.replication.message;
-
-import java.util.Set;
-
-import org.apache.asterix.common.api.INcApplicationContext;
-import org.apache.asterix.common.messaging.api.INCMessageBroker;
-import org.apache.asterix.common.messaging.api.INcAddressedMessage;
-import org.apache.asterix.common.replication.INCLifecycleMessage;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.control.nc.NodeControllerService;
-import org.apache.logging.log4j.Level;
-import org.apache.logging.log4j.LogManager;
-import org.apache.logging.log4j.Logger;
-
-public class ReplayPartitionLogsRequestMessage implements INCLifecycleMessage, INcAddressedMessage {
-
- private static final Logger LOGGER = LogManager.getLogger();
- private static final long serialVersionUID = 1L;
- private final Set<Integer> partitions;
-
- public ReplayPartitionLogsRequestMessage(Set<Integer> partitions) {
- this.partitions = partitions;
- }
-
- @Override
- public void handle(INcApplicationContext appContext) throws HyracksDataException, InterruptedException {
- NodeControllerService ncs = (NodeControllerService) appContext.getServiceContext().getControllerService();
- // Replay the logs for these partitions and flush any impacted dataset
- appContext.getRemoteRecoveryManager().replayReplicaPartitionLogs(partitions, true);
-
- INCMessageBroker broker = (INCMessageBroker) ncs.getContext().getMessageBroker();
- ReplayPartitionLogsResponseMessage reponse = new ReplayPartitionLogsResponseMessage(ncs.getId(), partitions);
- try {
- broker.sendMessageToCC(reponse);
- } catch (Exception e) {
- LOGGER.log(Level.ERROR, "Failed sending message to cc", e);
- throw HyracksDataException.create(e);
- }
- }
-
- @Override
- public MessageType getType() {
- return MessageType.REPLAY_LOGS_REQUEST;
- }
-}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/ReplayPartitionLogsResponseMessage.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/ReplayPartitionLogsResponseMessage.java
deleted file mode 100644
index e05fd47..0000000
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/ReplayPartitionLogsResponseMessage.java
+++ /dev/null
@@ -1,57 +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.app.replication.message;
-
-import java.util.Set;
-
-import org.apache.asterix.common.dataflow.ICcApplicationContext;
-import org.apache.asterix.common.messaging.api.ICcAddressedMessage;
-import org.apache.asterix.common.replication.INCLifecycleMessage;
-import org.apache.asterix.runtime.utils.CcApplicationContext;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-public class ReplayPartitionLogsResponseMessage implements INCLifecycleMessage, ICcAddressedMessage {
-
- private static final long serialVersionUID = 1L;
- private final Set<Integer> partitions;
- private final String nodeId;
-
- public ReplayPartitionLogsResponseMessage(String nodeId, Set<Integer> partitions) {
- this.partitions = partitions;
- this.nodeId = nodeId;
- }
-
- @Override
- public void handle(ICcApplicationContext appCtx) throws HyracksDataException, InterruptedException {
- ((CcApplicationContext) appCtx).getFaultToleranceStrategy().process(this);
- }
-
- public Set<Integer> getPartitions() {
- return partitions;
- }
-
- public String getNodeId() {
- return nodeId;
- }
-
- @Override
- public MessageType getType() {
- return MessageType.REPLAY_LOGS_RESPONSE;
- }
-}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/TakeoverPartitionsRequestMessage.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/TakeoverPartitionsRequestMessage.java
deleted file mode 100644
index 86be516..0000000
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/TakeoverPartitionsRequestMessage.java
+++ /dev/null
@@ -1,108 +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.app.replication.message;
-
-import java.io.IOException;
-
-import org.apache.asterix.common.api.INcApplicationContext;
-import org.apache.asterix.common.exceptions.ACIDException;
-import org.apache.asterix.common.messaging.api.INCMessageBroker;
-import org.apache.asterix.common.messaging.api.INcAddressedMessage;
-import org.apache.asterix.common.replication.INCLifecycleMessage;
-import org.apache.asterix.common.replication.IRemoteRecoveryManager;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.logging.log4j.Level;
-import org.apache.logging.log4j.LogManager;
-import org.apache.logging.log4j.Logger;
-
-public class TakeoverPartitionsRequestMessage implements INCLifecycleMessage, INcAddressedMessage {
-
- private static final long serialVersionUID = 1L;
- private static final Logger LOGGER = LogManager.getLogger();
- private final Integer[] partitions;
- private final long requestId;
- private final String nodeId;
-
- public TakeoverPartitionsRequestMessage(long requestId, String nodeId, Integer[] partitionsToTakeover) {
- this.requestId = requestId;
- this.nodeId = nodeId;
- this.partitions = partitionsToTakeover;
- }
-
- public Integer[] getPartitions() {
- return partitions;
- }
-
- public long getRequestId() {
- return requestId;
- }
-
- public String getNodeId() {
- return nodeId;
- }
-
- @Override
- public String toString() {
- StringBuilder sb = new StringBuilder();
- sb.append(TakeoverPartitionsRequestMessage.class.getSimpleName());
- sb.append(" Request ID: " + requestId);
- sb.append(" Node ID: " + nodeId);
- sb.append(" Partitions: ");
- for (Integer partitionId : partitions) {
- sb.append(partitionId + ",");
- }
- //remove last comma
- sb.charAt(sb.length() - 1);
- return sb.toString();
- }
-
- @Override
- public void handle(INcApplicationContext appContext) throws HyracksDataException, InterruptedException {
- INCMessageBroker broker = (INCMessageBroker) appContext.getServiceContext().getMessageBroker();
- //if the NC is shutting down, it should ignore takeover partitions request
- if (!appContext.isShuttingdown()) {
- HyracksDataException hde = null;
- try {
- IRemoteRecoveryManager remoteRecoeryManager = appContext.getRemoteRecoveryManager();
- remoteRecoeryManager.takeoverPartitons(partitions);
- } catch (IOException | ACIDException e) {
- LOGGER.log(Level.ERROR, "Failure taking over partitions", e);
- hde = HyracksDataException.suppress(hde, e);
- } finally {
- //send response after takeover is completed
- TakeoverPartitionsResponseMessage reponse = new TakeoverPartitionsResponseMessage(requestId,
- appContext.getTransactionSubsystem().getId(), partitions);
- try {
- broker.sendMessageToCC(reponse);
- } catch (Exception e) {
- LOGGER.log(Level.ERROR, "Failure taking over partitions", e);
- hde = HyracksDataException.suppress(hde, e);
- }
- }
- if (hde != null) {
- throw hde;
- }
- }
- }
-
- @Override
- public MessageType getType() {
- return MessageType.TAKEOVER_PARTITION_REQUEST;
- }
-}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/TakeoverPartitionsResponseMessage.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/TakeoverPartitionsResponseMessage.java
deleted file mode 100644
index d9484f9..0000000
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/message/TakeoverPartitionsResponseMessage.java
+++ /dev/null
@@ -1,66 +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.app.replication.message;
-
-import org.apache.asterix.common.dataflow.ICcApplicationContext;
-import org.apache.asterix.common.messaging.api.ICcAddressedMessage;
-import org.apache.asterix.common.replication.INCLifecycleMessage;
-import org.apache.asterix.runtime.utils.CcApplicationContext;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-public class TakeoverPartitionsResponseMessage implements INCLifecycleMessage, ICcAddressedMessage {
-
- private static final long serialVersionUID = 1L;
- private final Integer[] partitions;
- private final String nodeId;
- private final long requestId;
-
- public TakeoverPartitionsResponseMessage(long requestId, String nodeId, Integer[] partitionsToTakeover) {
- this.requestId = requestId;
- this.nodeId = nodeId;
- this.partitions = partitionsToTakeover;
- }
-
- public Integer[] getPartitions() {
- return partitions;
- }
-
- public String getNodeId() {
- return nodeId;
- }
-
- public long getRequestId() {
- return requestId;
- }
-
- @Override
- public void handle(ICcApplicationContext appCtx) throws HyracksDataException, InterruptedException {
- ((CcApplicationContext) appCtx).getFaultToleranceStrategy().process(this);
- }
-
- @Override
- public String toString() {
- return TakeoverPartitionsResponseMessage.class.getSimpleName();
- }
-
- @Override
- public MessageType getType() {
- return MessageType.TAKEOVER_PARTITION_RESPONSE;
- }
-}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplication.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplication.java
index 622e28f..c2a0fc1 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplication.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplication.java
@@ -142,9 +142,8 @@
ILibraryManager libraryManager = new ExternalLibraryManager();
ReplicationProperties repProp = new ReplicationProperties(
PropertiesAccessor.getInstance(ccServiceCtx.getAppConfig()));
- IReplicationStrategy repStrategy = ReplicationStrategyFactory.create(repProp.getReplicationStrategy(), repProp,
- getConfigManager());
- IFaultToleranceStrategy ftStrategy = FaultToleranceStrategyFactory.create(ccServiceCtx, repProp, repStrategy);
+ IFaultToleranceStrategy ftStrategy =
+ FaultToleranceStrategyFactory.create(ccServiceCtx, repProp.isReplicationEnabled());
ExternalLibraryUtils.setUpExternaLibraries(libraryManager, false);
componentProvider = new StorageComponentProvider();
GlobalRecoveryManager globalRecoveryManager = createGlobalRecoveryManager();
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/util/FaultToleranceUtil.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/util/FaultToleranceUtil.java
deleted file mode 100644
index ddd0967..0000000
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/util/FaultToleranceUtil.java
+++ /dev/null
@@ -1,71 +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.util;
-
-import java.util.List;
-import java.util.Map;
-import java.util.stream.Collectors;
-
-import org.apache.asterix.common.cluster.IClusterStateManager;
-import org.apache.asterix.common.messaging.api.ICCMessageBroker;
-import org.apache.asterix.common.replication.IReplicationStrategy;
-import org.apache.asterix.common.replication.Replica;
-import org.apache.asterix.runtime.message.ReplicaEventMessage;
-import org.apache.commons.lang3.StringUtils;
-import org.apache.hyracks.api.application.IClusterLifecycleListener.ClusterEventType;
-import org.apache.hyracks.api.config.IOption;
-import org.apache.hyracks.control.common.controllers.NCConfig;
-import org.apache.logging.log4j.LogManager;
-import org.apache.logging.log4j.Logger;
-
-public class FaultToleranceUtil {
-
- private static final Logger LOGGER = LogManager.getLogger();
-
- private FaultToleranceUtil() {
- throw new AssertionError();
- }
-
- public static void notifyImpactedReplicas(String nodeId, ClusterEventType event,
- IClusterStateManager clusterManager, ICCMessageBroker messageBroker,
- IReplicationStrategy replicationStrategy) {
- List<String> primaryRemoteReplicas = replicationStrategy.getRemotePrimaryReplicas(nodeId).stream()
- .map(Replica::getId).collect(Collectors.toList());
- String nodeIdAddress = StringUtils.EMPTY;
- int nodePort = -1;
- Map<String, Map<IOption, Object>> activeNcConfiguration = clusterManager.getActiveNcConfiguration();
-
- // 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 = (String) activeNcConfiguration.get(nodeId).get(NCConfig.Option.REPLICATION_PUBLIC_ADDRESS);
- nodePort = (int) activeNcConfiguration.get(nodeId).get(NCConfig.Option.REPLICATION_PUBLIC_PORT);
- }
- ReplicaEventMessage msg = new ReplicaEventMessage(nodeId, nodeIdAddress, nodePort, event);
- for (String replica : primaryRemoteReplicas) {
- // If the remote replica is alive, send the event
- if (activeNcConfiguration.containsKey(replica)) {
- try {
- messageBroker.sendApplicationMessageToNC(msg, replica);
- } catch (Exception e) {
- LOGGER.warn("Failed sending an application message to an NC", e);
- }
- }
- }
- }
-}
diff --git a/asterixdb/asterix-app/src/main/resources/cc-rep.conf b/asterixdb/asterix-app/src/main/resources/cc-rep.conf
index 9c093dc..bfca677 100644
--- a/asterixdb/asterix-app/src/main/resources/cc-rep.conf
+++ b/asterixdb/asterix-app/src/main/resources/cc-rep.conf
@@ -48,5 +48,4 @@
[common]
log.level = INFO
replication.enabled=true
-replication.strategy=metadata_only
-replication.factor=2
+replication.strategy=all
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload/bulkload.1.sto.cmd
similarity index 94%
copy from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload/bulkload.1.sto.cmd
index b750c64..1947749 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload/bulkload.1.sto.cmd
@@ -16,4 +16,4 @@
* specific language governing permissions and limitations
* under the License.
*/
-start asterix_nc1
\ No newline at end of file
+nc:asterix_nc2 /addReplica 2 asterix_nc1
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload/bulkload.10.post.http
similarity index 92%
copy from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload/bulkload.10.post.http
index b750c64..a3ea801 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload/bulkload.10.post.http
@@ -16,4 +16,4 @@
* specific language governing permissions and limitations
* under the License.
*/
-start asterix_nc1
\ No newline at end of file
+/admin/cluster/partition/master?partition=3&node=asterix_nc1
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload/bulkload.11.pollget.http
similarity index 93%
copy from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload/bulkload.11.pollget.http
index b750c64..32e2f78 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload/bulkload.11.pollget.http
@@ -16,4 +16,6 @@
* specific language governing permissions and limitations
* under the License.
*/
-start asterix_nc1
\ No newline at end of file
+//polltimeoutsecs=30
+
+/admin/cluster/summary
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload/bulkload.12.query.sqlpp
similarity index 93%
copy from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload/bulkload.12.query.sqlpp
index b750c64..cd777cf 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload/bulkload.12.query.sqlpp
@@ -16,4 +16,7 @@
* specific language governing permissions and limitations
* under the License.
*/
-start asterix_nc1
\ No newline at end of file
+
+use tpch;
+
+select value count(*) from LineItem;
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload/bulkload.13.sto.cmd
similarity index 94%
copy from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload/bulkload.13.sto.cmd
index b750c64..1e192f4 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload/bulkload.13.sto.cmd
@@ -16,4 +16,4 @@
* specific language governing permissions and limitations
* under the License.
*/
-start asterix_nc1
\ No newline at end of file
+nc:asterix_nc2 /removeReplica 2 asterix_nc1
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload/bulkload.14.sto.cmd
similarity index 94%
copy from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload/bulkload.14.sto.cmd
index b750c64..530432f 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload/bulkload.14.sto.cmd
@@ -16,4 +16,4 @@
* specific language governing permissions and limitations
* under the License.
*/
-start asterix_nc1
\ No newline at end of file
+nc:asterix_nc2 /removeReplica 3 asterix_nc1
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload/bulkload.2.sto.cmd
similarity index 94%
copy from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload/bulkload.2.sto.cmd
index b750c64..f3810f8 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload/bulkload.2.sto.cmd
@@ -16,4 +16,4 @@
* specific language governing permissions and limitations
* under the License.
*/
-start asterix_nc1
\ No newline at end of file
+nc:asterix_nc2 /addReplica 3 asterix_nc1
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload/bulkload.3.pollget.http
similarity index 91%
copy from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload/bulkload.3.pollget.http
index b750c64..4ea16d7 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload/bulkload.3.pollget.http
@@ -16,4 +16,6 @@
* specific language governing permissions and limitations
* under the License.
*/
-start asterix_nc1
\ No newline at end of file
+//polltimeoutsecs=30
+
+nc:asterix_nc2 /admin/storage/partition/2
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload/bulkload.4.pollget.http
similarity index 91%
copy from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload/bulkload.4.pollget.http
index b750c64..22558bc 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload/bulkload.4.pollget.http
@@ -16,4 +16,6 @@
* specific language governing permissions and limitations
* under the License.
*/
-start asterix_nc1
\ No newline at end of file
+//polltimeoutsecs=30
+
+nc:asterix_nc2 /admin/storage/partition/3
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload/bulkload.5.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload/bulkload.5.ddl.sqlpp
new file mode 100644
index 0000000..071a40f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload/bulkload.5.ddl.sqlpp
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse tpch if exists;
+create dataverse tpch;
+
+use tpch;
+
+create type tpch.LineItemType as
+ closed {
+ l_orderkey : bigint,
+ l_partkey : bigint,
+ l_suppkey : bigint,
+ l_linenumber : bigint,
+ l_quantity : double,
+ l_extendedprice : double,
+ l_discount : double,
+ l_tax : double,
+ l_returnflag : string,
+ l_linestatus : string,
+ l_shipdate : string,
+ l_commitdate : string,
+ l_receiptdate : string,
+ l_shipinstruct : string,
+ l_shipmode : string,
+ l_comment : string
+};
+
+create dataset LineItem(LineItemType) primary key l_orderkey,l_linenumber;
+
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload/bulkload.6.update.sqlpp
similarity index 82%
copy from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload/bulkload.6.update.sqlpp
index b750c64..e53f462 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload/bulkload.6.update.sqlpp
@@ -16,4 +16,8 @@
* specific language governing permissions and limitations
* under the License.
*/
-start asterix_nc1
\ No newline at end of file
+
+use tpch;
+
+load dataset LineItem using localfs ((`path`=`asterix_nc1://data/tpch0.001/lineitem.tbl`),
+ (`format`=`delimited-text`),(`delimiter`=`|`));
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload/bulkload.7.sto.cmd
similarity index 94%
copy from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload/bulkload.7.sto.cmd
index b750c64..389cf68 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload/bulkload.7.sto.cmd
@@ -16,4 +16,4 @@
* specific language governing permissions and limitations
* under the License.
*/
-start asterix_nc1
\ No newline at end of file
+nc:asterix_nc1 /promote 2 asterix_nc1
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload/bulkload.8.sto.cmd
similarity index 94%
copy from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload/bulkload.8.sto.cmd
index b750c64..257f26a 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload/bulkload.8.sto.cmd
@@ -16,4 +16,4 @@
* specific language governing permissions and limitations
* under the License.
*/
-start asterix_nc1
\ No newline at end of file
+nc:asterix_nc1 /promote 3 asterix_nc1
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload/bulkload.9.post.http
similarity index 92%
copy from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload/bulkload.9.post.http
index b750c64..36e1d00 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload/bulkload.9.post.http
@@ -16,4 +16,4 @@
* specific language governing permissions and limitations
* under the License.
*/
-start asterix_nc1
\ No newline at end of file
+/admin/cluster/partition/master?partition=2&node=asterix_nc1
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component/flushed_component.1.sto.cmd
similarity index 94%
copy from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component/flushed_component.1.sto.cmd
index b750c64..7ddaa20 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component/flushed_component.1.sto.cmd
@@ -16,4 +16,4 @@
* specific language governing permissions and limitations
* under the License.
*/
-start asterix_nc1
\ No newline at end of file
+nc:asterix_nc1 /addReplica 0 asterix_nc2
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component/flushed_component.10.sto.cmd
similarity index 94%
copy from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component/flushed_component.10.sto.cmd
index b750c64..71621ac 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component/flushed_component.10.sto.cmd
@@ -16,4 +16,4 @@
* specific language governing permissions and limitations
* under the License.
*/
-start asterix_nc1
\ No newline at end of file
+nc:asterix_nc1 /removeReplica 0 asterix_nc2
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component/flushed_component.2.pollget.http
similarity index 91%
copy from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component/flushed_component.2.pollget.http
index b750c64..6867a5d 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component/flushed_component.2.pollget.http
@@ -16,4 +16,6 @@
* specific language governing permissions and limitations
* under the License.
*/
-start asterix_nc1
\ No newline at end of file
+//polltimeoutsecs=30
+
+nc:asterix_nc1 /admin/storage/partition/0
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component/flushed_component.3.ddl.sqlpp
similarity index 89%
copy from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component/flushed_component.3.ddl.sqlpp
index b750c64..15bc3c5 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component/flushed_component.3.ddl.sqlpp
@@ -16,4 +16,8 @@
* specific language governing permissions and limitations
* under the License.
*/
-start asterix_nc1
\ No newline at end of file
+CREATE TYPE MyType AS {
+ id : int
+};
+
+CREATE DATASET ds_1(MyType) PRIMARY KEY id;
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component/flushed_component.4.get.http
similarity index 92%
copy from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component/flushed_component.4.get.http
index b750c64..09ddf42 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component/flushed_component.4.get.http
@@ -16,4 +16,4 @@
* specific language governing permissions and limitations
* under the License.
*/
-start asterix_nc1
\ No newline at end of file
+/connector?dataverseName=Metadata&datasetName=Dataset
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component/flushed_component.5.sto.cmd
similarity index 94%
copy from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component/flushed_component.5.sto.cmd
index b750c64..a5753f0 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component/flushed_component.5.sto.cmd
@@ -16,4 +16,4 @@
* specific language governing permissions and limitations
* under the License.
*/
-start asterix_nc1
\ No newline at end of file
+nc:asterix_nc2 /promote 0 asterix_nc2
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component/flushed_component.6.post.http
similarity index 92%
copy from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component/flushed_component.6.post.http
index b750c64..2e8fc63 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component/flushed_component.6.post.http
@@ -16,4 +16,4 @@
* specific language governing permissions and limitations
* under the License.
*/
-start asterix_nc1
\ No newline at end of file
+/admin/cluster/partition/master?partition=0&node=asterix_nc2
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component/flushed_component.7.post.http
similarity index 93%
copy from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component/flushed_component.7.post.http
index b750c64..e8dca0b 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component/flushed_component.7.post.http
@@ -16,4 +16,4 @@
* specific language governing permissions and limitations
* under the License.
*/
-start asterix_nc1
\ No newline at end of file
+/admin/cluster/metadataNode?node=asterix_nc2
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component/flushed_component.8.pollget.http
similarity index 93%
copy from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component/flushed_component.8.pollget.http
index b750c64..32e2f78 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component/flushed_component.8.pollget.http
@@ -16,4 +16,6 @@
* specific language governing permissions and limitations
* under the License.
*/
-start asterix_nc1
\ No newline at end of file
+//polltimeoutsecs=30
+
+/admin/cluster/summary
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component/flushed_component.9.query.sqlpp
similarity index 90%
copy from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component/flushed_component.9.query.sqlpp
index b750c64..a612cbb 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component/flushed_component.9.query.sqlpp
@@ -16,4 +16,6 @@
* specific language governing permissions and limitations
* under the License.
*/
-start asterix_nc1
\ No newline at end of file
+SELECT value count(*)
+FROM Metadata.`Dataset`
+WHERE DatasetName = 'ds_1';
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/mem_component_recovery/mem_component_recovery.1.sto.cmd
similarity index 94%
copy from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/mem_component_recovery/mem_component_recovery.1.sto.cmd
index b750c64..1947749 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/mem_component_recovery/mem_component_recovery.1.sto.cmd
@@ -16,4 +16,4 @@
* specific language governing permissions and limitations
* under the License.
*/
-start asterix_nc1
\ No newline at end of file
+nc:asterix_nc2 /addReplica 2 asterix_nc1
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/mem_component_recovery/mem_component_recovery.10.post.http
similarity index 92%
copy from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/mem_component_recovery/mem_component_recovery.10.post.http
index b750c64..a3ea801 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/mem_component_recovery/mem_component_recovery.10.post.http
@@ -16,4 +16,4 @@
* specific language governing permissions and limitations
* under the License.
*/
-start asterix_nc1
\ No newline at end of file
+/admin/cluster/partition/master?partition=3&node=asterix_nc1
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/mem_component_recovery/mem_component_recovery.11.pollget.http
similarity index 93%
copy from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/mem_component_recovery/mem_component_recovery.11.pollget.http
index b750c64..32e2f78 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/mem_component_recovery/mem_component_recovery.11.pollget.http
@@ -16,4 +16,6 @@
* specific language governing permissions and limitations
* under the License.
*/
-start asterix_nc1
\ No newline at end of file
+//polltimeoutsecs=30
+
+/admin/cluster/summary
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/mem_component_recovery/mem_component_recovery.12.query.sqlpp
similarity index 92%
copy from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/mem_component_recovery/mem_component_recovery.12.query.sqlpp
index b750c64..4951292 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/mem_component_recovery/mem_component_recovery.12.query.sqlpp
@@ -16,4 +16,7 @@
* specific language governing permissions and limitations
* under the License.
*/
-start asterix_nc1
\ No newline at end of file
+
+use tpch;
+
+select value count(*) from InMemoryLineItem;
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/mem_component_recovery/mem_component_recovery.13.sto.cmd
similarity index 94%
copy from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/mem_component_recovery/mem_component_recovery.13.sto.cmd
index b750c64..1e192f4 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/mem_component_recovery/mem_component_recovery.13.sto.cmd
@@ -16,4 +16,4 @@
* specific language governing permissions and limitations
* under the License.
*/
-start asterix_nc1
\ No newline at end of file
+nc:asterix_nc2 /removeReplica 2 asterix_nc1
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/mem_component_recovery/mem_component_recovery.14.sto.cmd
similarity index 94%
copy from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/mem_component_recovery/mem_component_recovery.14.sto.cmd
index b750c64..530432f 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/mem_component_recovery/mem_component_recovery.14.sto.cmd
@@ -16,4 +16,4 @@
* specific language governing permissions and limitations
* under the License.
*/
-start asterix_nc1
\ No newline at end of file
+nc:asterix_nc2 /removeReplica 3 asterix_nc1
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/mem_component_recovery/mem_component_recovery.2.sto.cmd
similarity index 94%
copy from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/mem_component_recovery/mem_component_recovery.2.sto.cmd
index b750c64..f3810f8 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/mem_component_recovery/mem_component_recovery.2.sto.cmd
@@ -16,4 +16,4 @@
* specific language governing permissions and limitations
* under the License.
*/
-start asterix_nc1
\ No newline at end of file
+nc:asterix_nc2 /addReplica 3 asterix_nc1
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/mem_component_recovery/mem_component_recovery.3.pollget.http
similarity index 91%
copy from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/mem_component_recovery/mem_component_recovery.3.pollget.http
index b750c64..4ea16d7 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/mem_component_recovery/mem_component_recovery.3.pollget.http
@@ -16,4 +16,6 @@
* specific language governing permissions and limitations
* under the License.
*/
-start asterix_nc1
\ No newline at end of file
+//polltimeoutsecs=30
+
+nc:asterix_nc2 /admin/storage/partition/2
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/mem_component_recovery/mem_component_recovery.4.pollget.http
similarity index 91%
copy from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/mem_component_recovery/mem_component_recovery.4.pollget.http
index b750c64..22558bc 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/mem_component_recovery/mem_component_recovery.4.pollget.http
@@ -16,4 +16,6 @@
* specific language governing permissions and limitations
* under the License.
*/
-start asterix_nc1
\ No newline at end of file
+//polltimeoutsecs=30
+
+nc:asterix_nc2 /admin/storage/partition/3
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/mem_component_recovery/mem_component_recovery.5.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/mem_component_recovery/mem_component_recovery.5.ddl.sqlpp
new file mode 100644
index 0000000..4dbf9e3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/mem_component_recovery/mem_component_recovery.5.ddl.sqlpp
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse tpch if exists;
+create dataverse tpch;
+
+use tpch;
+
+create type tpch.LineItemType as
+ closed {
+ l_orderkey : bigint,
+ l_partkey : bigint,
+ l_suppkey : bigint,
+ l_linenumber : bigint,
+ l_quantity : double,
+ l_extendedprice : double,
+ l_discount : double,
+ l_tax : double,
+ l_returnflag : string,
+ l_linestatus : string,
+ l_shipdate : string,
+ l_commitdate : string,
+ l_receiptdate : string,
+ l_shipinstruct : string,
+ l_shipmode : string,
+ l_comment : string
+};
+
+create dataset LineItem(LineItemType) primary key l_orderkey,l_linenumber;
+create dataset InMemoryLineItem(LineItemType) primary key l_orderkey,l_linenumber;
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/mem_component_recovery/mem_component_recovery.6.update.sqlpp
similarity index 77%
copy from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/mem_component_recovery/mem_component_recovery.6.update.sqlpp
index b750c64..8962e6d 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/mem_component_recovery/mem_component_recovery.6.update.sqlpp
@@ -16,4 +16,10 @@
* specific language governing permissions and limitations
* under the License.
*/
-start asterix_nc1
\ No newline at end of file
+
+use tpch;
+
+load dataset LineItem using localfs ((`path`=`asterix_nc1://data/tpch0.001/lineitem.tbl`),
+ (`format`=`delimited-text`),(`delimiter`=`|`));
+
+insert into InMemoryLineItem (select value ls from LineItem ls);
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/mem_component_recovery/mem_component_recovery.7.sto.cmd
similarity index 94%
copy from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/mem_component_recovery/mem_component_recovery.7.sto.cmd
index b750c64..389cf68 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/mem_component_recovery/mem_component_recovery.7.sto.cmd
@@ -16,4 +16,4 @@
* specific language governing permissions and limitations
* under the License.
*/
-start asterix_nc1
\ No newline at end of file
+nc:asterix_nc1 /promote 2 asterix_nc1
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/mem_component_recovery/mem_component_recovery.8.sto.cmd
similarity index 94%
copy from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/mem_component_recovery/mem_component_recovery.8.sto.cmd
index b750c64..257f26a 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/mem_component_recovery/mem_component_recovery.8.sto.cmd
@@ -16,4 +16,4 @@
* specific language governing permissions and limitations
* under the License.
*/
-start asterix_nc1
\ No newline at end of file
+nc:asterix_nc1 /promote 3 asterix_nc1
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/mem_component_recovery/mem_component_recovery.9.post.http
similarity index 92%
copy from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/mem_component_recovery/mem_component_recovery.9.post.http
index b750c64..36e1d00 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/mem_component_recovery/mem_component_recovery.9.post.http
@@ -16,4 +16,4 @@
* specific language governing permissions and limitations
* under the License.
*/
-start asterix_nc1
\ No newline at end of file
+/admin/cluster/partition/master?partition=2&node=asterix_nc1
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/metadata_failover/metadata_failover.12.sto.cmd
similarity index 94%
copy from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/metadata_failover/metadata_failover.12.sto.cmd
index b750c64..71621ac 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/metadata_failover/metadata_failover.12.sto.cmd
@@ -16,4 +16,4 @@
* specific language governing permissions and limitations
* under the License.
*/
-start asterix_nc1
\ No newline at end of file
+nc:asterix_nc1 /removeReplica 0 asterix_nc2
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/replication.xml b/asterixdb/asterix-app/src/test/resources/runtimets/replication.xml
index c3dfb3c..79c9f5c 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/replication.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/replication.xml
@@ -29,5 +29,20 @@
<output-dir compare="Text">metadata_failover</output-dir>
</compilation-unit>
</test-case>
+ <test-case FilePath="replication">
+ <compilation-unit name="bulkload">
+ <output-dir compare="Text">bulkload</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="replication">
+ <compilation-unit name="mem_component_recovery">
+ <output-dir compare="Text">mem_component_recovery</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="replication">
+ <compilation-unit name="flushed_component">
+ <output-dir compare="Text">flushed_component</output-dir>
+ </compilation-unit>
+ </test-case>
</test-group>
</test-suite>
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1/cluster_state_1.1.regexadm b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1/cluster_state_1.1.regexadm
index 53d6563..85b121e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1/cluster_state_1.1.regexadm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1/cluster_state_1.1.regexadm
@@ -22,14 +22,12 @@
"metadata\.node" : "asterix_nc1",
"metadata\.registration\.timeout\.secs" : 60,
"replication\.enabled" : false,
- "replication\.factor" : 3,
+ "replication\.factor" : 2,
"replication\.log\.batchsize" : 4096,
"replication\.log\.buffer\.numpages" : 8,
"replication\.log\.buffer\.pagesize" : 131072,
- "replication\.max\.remote\.recovery\.attempts" : 5,
- "replication\.port" : 2000,
- "replication\.strategy" : "chained_declustering",
- "replication\.timeout" : 15,
+ "replication\.strategy" : "none",
+ "replication\.timeout" : 30,
"txn\.commitprofiler\.enabled" : false,
"txn\.commitprofiler\.reportinterval" : 5,
"txn\.job\.recovery\.memorysize" : 67108864,
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_full/cluster_state_1_full.1.regexadm b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_full/cluster_state_1_full.1.regexadm
index 1160758..7017b06 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_full/cluster_state_1_full.1.regexadm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_full/cluster_state_1_full.1.regexadm
@@ -22,14 +22,12 @@
"metadata\.node" : "asterix_nc1",
"metadata\.registration\.timeout\.secs" : 60,
"replication\.enabled" : false,
- "replication\.factor" : 3,
+ "replication\.factor" : 2,
"replication\.log\.batchsize" : 4096,
"replication\.log\.buffer\.numpages" : 8,
"replication\.log\.buffer\.pagesize" : 131072,
- "replication\.max\.remote\.recovery\.attempts" : 5,
- "replication\.port" : 2000,
- "replication\.strategy" : "chained_declustering",
- "replication\.timeout" : 15,
+ "replication\.strategy" : "none",
+ "replication\.timeout" : 30,
"txn\.commitprofiler\.enabled" : false,
"txn\.commitprofiler\.reportinterval" : 5,
"txn\.job\.recovery\.memorysize" : 67108864,
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_less/cluster_state_1_less.1.regexadm b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_less/cluster_state_1_less.1.regexadm
index d1c9fe0..00e4d67 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_less/cluster_state_1_less.1.regexadm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_less/cluster_state_1_less.1.regexadm
@@ -22,14 +22,12 @@
"metadata\.node" : "asterix_nc1",
"metadata\.registration\.timeout\.secs" : 60,
"replication\.enabled" : false,
- "replication\.factor" : 3,
+ "replication\.factor" : 2,
"replication\.log\.batchsize" : 4096,
"replication\.log\.buffer\.numpages" : 8,
"replication\.log\.buffer\.pagesize" : 131072,
- "replication\.max\.remote\.recovery\.attempts" : 5,
- "replication\.port" : 2000,
- "replication\.strategy" : "chained_declustering",
- "replication\.timeout" : 15,
+ "replication\.strategy" : "none",
+ "replication\.timeout" : 30,
"txn\.commitprofiler\.enabled" : false,
"txn\.commitprofiler\.reportinterval" : 5,
"txn\.job\.recovery\.memorysize" : 67108864,
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/bulkload/bulkload.10.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/bulkload/bulkload.10.adm
new file mode 100644
index 0000000..3a8dfe0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/bulkload/bulkload.10.adm
@@ -0,0 +1 @@
+6005
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/bulkload/bulkload.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/bulkload/bulkload.3.adm
new file mode 100644
index 0000000..a6afbb4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/bulkload/bulkload.3.adm
@@ -0,0 +1,7 @@
+[ {
+ "partition" : 2,
+ "replicas" : [ {
+ "location" : "127.0.0.1:2001",
+ "status" : "IN_SYNC"
+ } ]
+} ]
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/bulkload/bulkload.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/bulkload/bulkload.4.adm
new file mode 100644
index 0000000..53648a2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/bulkload/bulkload.4.adm
@@ -0,0 +1,7 @@
+[ {
+ "partition" : 3,
+ "replicas" : [ {
+ "location" : "127.0.0.1:2001",
+ "status" : "IN_SYNC"
+ } ]
+} ]
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/bulkload/bulkload.7.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/bulkload/bulkload.7.adm
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/bulkload/bulkload.7.adm
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/bulkload/bulkload.8.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/bulkload/bulkload.8.adm
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/bulkload/bulkload.8.adm
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/results/metadata_recovery/metadata_node_recovery/metadata_node_recovery.cluster_state.7.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/bulkload/bulkload.9.adm
similarity index 90%
rename from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/results/metadata_recovery/metadata_node_recovery/metadata_node_recovery.cluster_state.7.adm
rename to asterixdb/asterix-app/src/test/resources/runtimets/results/replication/bulkload/bulkload.9.adm
index c0697b7..c382703 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/results/metadata_recovery/metadata_node_recovery/metadata_node_recovery.cluster_state.7.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/bulkload/bulkload.9.adm
@@ -19,7 +19,7 @@
},
"2" : {
"active" : true,
- "activeNodeId" : "asterix_nc2",
+ "activeNodeId" : "asterix_nc1",
"iodeviceNum" : 0,
"nodeId" : "asterix_nc2",
"partitionId" : 2,
@@ -27,7 +27,7 @@
},
"3" : {
"active" : true,
- "activeNodeId" : "asterix_nc2",
+ "activeNodeId" : "asterix_nc1",
"iodeviceNum" : 1,
"nodeId" : "asterix_nc2",
"partitionId" : 3,
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/flushed_component/flushed_component.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/flushed_component/flushed_component.2.adm
new file mode 100644
index 0000000..7e92c87
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/flushed_component/flushed_component.2.adm
@@ -0,0 +1,7 @@
+[ {
+ "partition" : 0,
+ "replicas" : [ {
+ "location" : "127.0.0.1:2002",
+ "status" : "IN_SYNC"
+ } ]
+} ]
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/flushed_component/flushed_component.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/flushed_component/flushed_component.4.adm
new file mode 100644
index 0000000..f84d77a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/flushed_component/flushed_component.4.adm
@@ -0,0 +1 @@
+{"keys":"DataverseName,DatasetName","type":{"type":"org.apache.asterix.om.types.ARecordType","name":"DatasetRecordType","open":true,"fields":[{"DataverseName":{"type":"AString"}},{"DatasetName":{"type":"AString"}},{"DatatypeDataverseName":{"type":"AString"}},{"DatatypeName":{"type":"AString"}},{"DatasetType":{"type":"AString"}},{"GroupName":{"type":"AString"}},{"CompactionPolicy":{"type":"AString"}},{"CompactionPolicyProperties":{"type":"org.apache.asterix.om.types.AOrderedListType","item-type":{"type":"org.apache.asterix.om.types.ARecordType","name":"DatasetRecordType_CompactionPolicyProperties_Item","open":true,"fields":[{"Name":{"type":"AString"}},{"Value":{"type":"AString"}}]}}},{"InternalDetails":{"type":"org.apache.asterix.om.types.AUnionType","fields":[{"type":"org.apache.asterix.om.types.ARecordType","name":"DatasetRecordType_InternalDetails","open":true,"fields":[{"FileStructure":{"type":"AString"}},{"PartitioningStrategy":{"type":"AString"}},{"PartitioningKey":{"type":"org.apache.asterix.om.types.AOrderedListType","item-type":{"type":"org.apache.asterix.om.types.AOrderedListType","item-type":{"type":"AString"}}}},{"PrimaryKey":{"type":"org.apache.asterix.om.types.AOrderedListType","item-type":{"type":"org.apache.asterix.om.types.AOrderedListType","item-type":{"type":"AString"}}}},{"Autogenerated":{"type":"ABoolean"}}]},{"type":"ANULL"},{"type":"AMISSING"}]}},{"ExternalDetails":{"type":"org.apache.asterix.om.types.AUnionType","fields":[{"type":"org.apache.asterix.om.types.ARecordType","name":"DatasetRecordType_ExternalDetails","open":true,"fields":[{"DatasourceAdapter":{"type":"AString"}},{"Properties":{"type":"org.apache.asterix.om.types.AOrderedListType","item-type":{"type":"org.apache.asterix.om.types.ARecordType","name":"DatasetRecordType_ExternalDetails_Properties_Item","open":true,"fields":[{"Name":{"type":"AString"}},{"Value":{"type":"AString"}}]}}},{"LastRefreshTime":{"type":"ADateTime"}},{"TransactionState":{"type":"AInt32"}}]},{"type":"ANULL"},{"type":"AMISSING"}]}},{"Hints":{"type":"org.apache.asterix.om.types.AUnorderedListType","item-type":{"type":"org.apache.asterix.om.types.ARecordType","name":"DatasetRecordType_Hints_Item","open":true,"fields":[{"Name":{"type":"AString"}},{"Value":{"type":"AString"}}]}}},{"Timestamp":{"type":"AString"}},{"DatasetId":{"type":"AInt32"}},{"PendingOp":{"type":"AInt32"}}]},"splits":[{"ip":"127.0.0.1","path":"storage/partition_0/Metadata/Dataset/0/Dataset"}]}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/flushed_component/flushed_component.6.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/flushed_component/flushed_component.6.adm
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/flushed_component/flushed_component.6.adm
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/flushed_component/flushed_component.7.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/flushed_component/flushed_component.7.adm
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/flushed_component/flushed_component.7.adm
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failback/node_failback/node_failback.cluster_state.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/flushed_component/flushed_component.8.adm
similarity index 94%
rename from asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failback/node_failback/node_failback.cluster_state.5.adm
rename to asterixdb/asterix-app/src/test/resources/runtimets/results/replication/flushed_component/flushed_component.8.adm
index 15c200c..fa5cfb4 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failback/node_failback/node_failback.cluster_state.5.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/flushed_component/flushed_component.8.adm
@@ -11,7 +11,7 @@
},
"1" : {
"active" : true,
- "activeNodeId" : "asterix_nc2",
+ "activeNodeId" : "asterix_nc1",
"iodeviceNum" : 1,
"nodeId" : "asterix_nc1",
"partitionId" : 1,
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failover/metadata_node/metadata_node.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/flushed_component/flushed_component.9.adm
similarity index 100%
copy from asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failover/metadata_node/metadata_node.4.adm
copy to asterixdb/asterix-app/src/test/resources/runtimets/results/replication/flushed_component/flushed_component.9.adm
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/mem_component_recovery/mem_component_recovery.10.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/mem_component_recovery/mem_component_recovery.10.adm
new file mode 100644
index 0000000..3a8dfe0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/mem_component_recovery/mem_component_recovery.10.adm
@@ -0,0 +1 @@
+6005
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/mem_component_recovery/mem_component_recovery.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/mem_component_recovery/mem_component_recovery.3.adm
new file mode 100644
index 0000000..a6afbb4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/mem_component_recovery/mem_component_recovery.3.adm
@@ -0,0 +1,7 @@
+[ {
+ "partition" : 2,
+ "replicas" : [ {
+ "location" : "127.0.0.1:2001",
+ "status" : "IN_SYNC"
+ } ]
+} ]
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/mem_component_recovery/mem_component_recovery.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/mem_component_recovery/mem_component_recovery.4.adm
new file mode 100644
index 0000000..53648a2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/mem_component_recovery/mem_component_recovery.4.adm
@@ -0,0 +1,7 @@
+[ {
+ "partition" : 3,
+ "replicas" : [ {
+ "location" : "127.0.0.1:2001",
+ "status" : "IN_SYNC"
+ } ]
+} ]
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/mem_component_recovery/mem_component_recovery.7.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/mem_component_recovery/mem_component_recovery.7.adm
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/mem_component_recovery/mem_component_recovery.7.adm
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/mem_component_recovery/mem_component_recovery.8.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/mem_component_recovery/mem_component_recovery.8.adm
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/mem_component_recovery/mem_component_recovery.8.adm
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/results/metadata_recovery/metadata_node_recovery/metadata_node_recovery.cluster_state.7.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/mem_component_recovery/mem_component_recovery.9.adm
similarity index 90%
copy from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/results/metadata_recovery/metadata_node_recovery/metadata_node_recovery.cluster_state.7.adm
copy to asterixdb/asterix-app/src/test/resources/runtimets/results/replication/mem_component_recovery/mem_component_recovery.9.adm
index c0697b7..c382703 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/results/metadata_recovery/metadata_node_recovery/metadata_node_recovery.cluster_state.7.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/mem_component_recovery/mem_component_recovery.9.adm
@@ -19,7 +19,7 @@
},
"2" : {
"active" : true,
- "activeNodeId" : "asterix_nc2",
+ "activeNodeId" : "asterix_nc1",
"iodeviceNum" : 0,
"nodeId" : "asterix_nc2",
"partitionId" : 2,
@@ -27,7 +27,7 @@
},
"3" : {
"active" : true,
- "activeNodeId" : "asterix_nc2",
+ "activeNodeId" : "asterix_nc1",
"iodeviceNum" : 1,
"nodeId" : "asterix_nc2",
"partitionId" : 3,
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/metadata_failover/metadata_failover.11.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/metadata_failover/metadata_failover.11.adm
index 56a6051..d0138cb 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/metadata_failover/metadata_failover.11.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/replication/metadata_failover/metadata_failover.11.adm
@@ -1 +1,3 @@
-1
\ No newline at end of file
+1
+
+
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/INcApplicationContext.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/INcApplicationContext.java
index 28be6fa..8a83c7b 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/INcApplicationContext.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/INcApplicationContext.java
@@ -24,8 +24,6 @@
import org.apache.asterix.common.context.IStorageComponentProvider;
import org.apache.asterix.common.exceptions.ACIDException;
-import org.apache.asterix.common.replication.IRemoteRecoveryManager;
-import org.apache.asterix.common.replication.IReplicaResourcesManager;
import org.apache.asterix.common.replication.IReplicationChannel;
import org.apache.asterix.common.replication.IReplicationManager;
import org.apache.asterix.common.storage.IIndexCheckpointManagerProvider;
@@ -80,10 +78,6 @@
Object getActiveManager();
- IRemoteRecoveryManager getRemoteRecoveryManager();
-
- IReplicaResourcesManager getReplicaResourcesManager();
-
IReplicationManager getReplicationManager();
IReplicationChannel getReplicationChannel();
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/ReplicationProperties.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/ReplicationProperties.java
index 6e7d58b..3455774 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/ReplicationProperties.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/ReplicationProperties.java
@@ -18,43 +18,34 @@
*/
package org.apache.asterix.common.config;
-import java.util.List;
-import org.apache.hyracks.api.config.IApplicationConfig;
+import static org.apache.hyracks.control.common.config.OptionTypes.BOOLEAN;
+import static org.apache.hyracks.control.common.config.OptionTypes.INTEGER;
+import static org.apache.hyracks.control.common.config.OptionTypes.INTEGER_BYTE_UNIT;
+import static org.apache.hyracks.control.common.config.OptionTypes.LONG;
+import static org.apache.hyracks.control.common.config.OptionTypes.STRING;
+
+import java.util.concurrent.TimeUnit;
+
import org.apache.hyracks.api.config.IOption;
import org.apache.hyracks.api.config.IOptionType;
import org.apache.hyracks.api.config.Section;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.control.common.controllers.NCConfig;
import org.apache.hyracks.util.StorageUtil;
import org.apache.hyracks.util.StorageUtil.StorageUnit;
-import static org.apache.hyracks.control.common.config.OptionTypes.*;
-
public class ReplicationProperties extends AbstractProperties {
public enum Option implements IOption {
- REPLICATION_MAX_REMOTE_RECOVERY_ATTEMPTS(
- INTEGER,
- 5,
- "The maximum number of times to attempt to recover from a replica on failure before giving up"),
- REPLICATION_LOG_BUFFER_PAGESIZE(
- INTEGER_BYTE_UNIT,
- StorageUtil.getIntSizeInBytes(128, StorageUnit.KILOBYTE),
+ REPLICATION_LOG_BUFFER_PAGESIZE(INTEGER_BYTE_UNIT, StorageUtil.getIntSizeInBytes(128, StorageUnit.KILOBYTE),
"The size in bytes of each log buffer page"),
REPLICATION_LOG_BUFFER_NUMPAGES(INTEGER, 8, "The number of log buffer pages"),
- REPLICATION_LOG_BATCHSIZE(
- INTEGER_BYTE_UNIT,
- StorageUtil.getIntSizeInBytes(4, StorageUnit.KILOBYTE),
+ REPLICATION_LOG_BATCHSIZE(INTEGER_BYTE_UNIT, StorageUtil.getIntSizeInBytes(4, StorageUnit.KILOBYTE),
"The size in bytes to replicate in each batch"),
- REPLICATION_TIMEOUT(
- INTEGER,
- REPLICATION_TIME_OUT_DEFAULT,
- "The time in seconds to timeout when trying to contact a replica, before assuming it is dead"),
-
+ REPLICATION_TIMEOUT(LONG, TimeUnit.SECONDS.toSeconds(30),
+ "The time in seconds to timeout waiting for master or replica to ack"),
REPLICATION_ENABLED(BOOLEAN, false, "Whether or not data replication is enabled"),
- REPLICATION_FACTOR(INTEGER, 3, "Number of node controller faults to tolerate with replication"),
- REPLICATION_STRATEGY(STRING, "chained_declustering", "Replication strategy to choose"),
- REPLICATION_PORT(INTEGER, 2000, "port on which to run replication related communications"),;
+ REPLICATION_FACTOR(INTEGER, 2, "Number of replicas (backups) to maintain per master replica"),
+ REPLICATION_STRATEGY(STRING, "none", "Replication strategy to choose");
private final IOptionType type;
private final Object defaultValue;
@@ -85,40 +76,16 @@
public Object defaultValue() {
return defaultValue;
}
-
- @Override
- public Object get(IApplicationConfig config) {
- switch (this) {
- case REPLICATION_TIMEOUT:
- return REPLICATION_TIME_OUT_DEFAULT;
- default:
- return config.getStatic(this);
- }
- }
}
public boolean isReplicationEnabled() {
return accessor.getBoolean(Option.REPLICATION_ENABLED);
}
- private static final int REPLICATION_TIME_OUT_DEFAULT = 15;
-
- public ReplicationProperties(PropertiesAccessor accessor) throws HyracksDataException {
+ public ReplicationProperties(PropertiesAccessor accessor) {
super(accessor);
}
- public int getMaxRemoteRecoveryAttempts() {
- return accessor.getInt(Option.REPLICATION_MAX_REMOTE_RECOVERY_ATTEMPTS);
- }
-
- public int getReplicationFactor() {
- return accessor.getInt(Option.REPLICATION_FACTOR);
- }
-
- public List<String> getNodeIds() {
- return accessor.getNCNames();
- }
-
public int getLogBufferPageSize() {
return accessor.getInt(Option.REPLICATION_LOG_BUFFER_PAGESIZE);
}
@@ -131,20 +98,23 @@
return accessor.getInt(Option.REPLICATION_LOG_BATCHSIZE);
}
- public String getNodeIpFromId(String id) {
- return accessor.getNCEffectiveConfig(id).getString(NCConfig.Option.PUBLIC_ADDRESS);
+ public String getReplicationAddress() {
+ return accessor.getString(NCConfig.Option.REPLICATION_LISTEN_ADDRESS);
+ }
+
+ public int getReplicationPort() {
+ return accessor.getInt(NCConfig.Option.REPLICATION_LISTEN_PORT);
}
public String getReplicationStrategy() {
return accessor.getString(Option.REPLICATION_STRATEGY);
}
- public int getReplicationTimeOut() {
- return accessor.getInt(Option.REPLICATION_TIMEOUT);
+ public long getReplicationTimeOut() {
+ return accessor.getLong(Option.REPLICATION_TIMEOUT);
}
- public MetadataProperties getMetadataProperties() {
- return new MetadataProperties(accessor);
+ public int getReplicationFactor() {
+ return accessor.getInt(Option.REPLICATION_FACTOR);
}
-
}
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/AllDatasetsReplicationStrategy.java
similarity index 78%
copy from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
copy to asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/AllDatasetsReplicationStrategy.java
index b750c64..9f9294b 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/AllDatasetsReplicationStrategy.java
@@ -16,4 +16,12 @@
* specific language governing permissions and limitations
* under the License.
*/
-start asterix_nc1
\ No newline at end of file
+package org.apache.asterix.common.replication;
+
+public class AllDatasetsReplicationStrategy implements IReplicationStrategy {
+
+ @Override
+ public boolean isMatch(int datasetId) {
+ return true;
+ }
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/ChainedDeclusteringReplicationStrategy.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/ChainedDeclusteringReplicationStrategy.java
deleted file mode 100644
index 6f66599..0000000
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/ChainedDeclusteringReplicationStrategy.java
+++ /dev/null
@@ -1,105 +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.common.replication;
-
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.Set;
-
-import org.apache.asterix.common.config.ReplicationProperties;
-import org.apache.hyracks.api.config.IConfigManager;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.control.common.config.ConfigManager;
-import org.apache.hyracks.control.common.controllers.NCConfig;
-import org.apache.logging.log4j.LogManager;
-import org.apache.logging.log4j.Logger;
-
-public class ChainedDeclusteringReplicationStrategy implements IReplicationStrategy {
-
- private static final Logger LOGGER = LogManager.getLogger();
- private int replicationFactor;
- private ReplicationProperties repProp;
- private ConfigManager configManager;
-
- @Override
- public boolean isMatch(int datasetId) {
- return true;
- }
-
- @Override
- public Set<Replica> getRemoteReplicas(String nodeId) {
- Set<Replica> remoteReplicas = new HashSet<>();
- int numberOfRemoteReplicas = replicationFactor - 1;
- int nodeIndex = repProp.getNodeIds().indexOf(nodeId);
-
- if (nodeIndex == -1) {
- if (LOGGER.isWarnEnabled()) {
- LOGGER.warn("Could not find node " + nodeId + " in cluster configurations");
- }
- return Collections.emptySet();
- }
-
- //find nodes to the right of this node
- while (remoteReplicas.size() != numberOfRemoteReplicas) {
- String replica = repProp.getNodeIds().get(++nodeIndex % repProp.getNodeIds().size());
- remoteReplicas.add(new Replica(replica,
- configManager.getNodeEffectiveConfig(replica).getString(NCConfig.Option.REPLICATION_LISTEN_ADDRESS),
- configManager.getNodeEffectiveConfig(replica).getInt(NCConfig.Option.REPLICATION_LISTEN_PORT)));
- }
-
- return remoteReplicas;
- }
-
- @Override
- public Set<Replica> getRemoteReplicasAndSelf(String nodeId) {
- Set<Replica> replicas = getRemoteReplicas(nodeId);
- replicas.add(new Replica(nodeId,
- configManager.getNodeEffectiveConfig(nodeId).getString(NCConfig.Option.REPLICATION_LISTEN_ADDRESS),
- configManager.getNodeEffectiveConfig(nodeId).getInt(NCConfig.Option.REPLICATION_LISTEN_PORT)));
- return replicas;
-
- }
-
- @Override
- public Set<Replica> getRemotePrimaryReplicas(String nodeId) {
- Set<Replica> clientReplicas = new HashSet<>();
- final int remotePrimaryReplicasCount = replicationFactor - 1;
- int nodeIndex = repProp.getNodeIds().indexOf(nodeId);
-
- //find nodes to the left of this node
- while (clientReplicas.size() != remotePrimaryReplicasCount) {
- String replica = repProp.getNodeIds().get(Math.abs(--nodeIndex % repProp.getNodeIds().size()));
- clientReplicas.add(new Replica(replica,
- configManager.getNodeEffectiveConfig(replica).getString(NCConfig.Option.REPLICATION_LISTEN_ADDRESS),
- configManager.getNodeEffectiveConfig(replica).getInt(NCConfig.Option.REPLICATION_LISTEN_PORT)));
- }
-
- return clientReplicas;
- }
-
- @Override
- public ChainedDeclusteringReplicationStrategy from(ReplicationProperties repProp, IConfigManager configManager)
- throws HyracksDataException {
- ChainedDeclusteringReplicationStrategy cd = new ChainedDeclusteringReplicationStrategy();
- cd.repProp = repProp;
- cd.replicationFactor = repProp.getReplicationFactor();
- cd.configManager = (ConfigManager) configManager;
- return cd;
- }
-}
\ No newline at end of file
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/IFaultToleranceStrategy.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/IFaultToleranceStrategy.java
index e871374..bf4cac9 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/IFaultToleranceStrategy.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/IFaultToleranceStrategy.java
@@ -58,11 +58,11 @@
/**
* Constructs a fault tolerance strategy.
*
- * @param replicationStrategy
- * @param messageBroker
- * @return
+ * @param serviceCtx
+ * @param replicationEnabled
+ * @return the fault tolerance strategy
*/
- IFaultToleranceStrategy from(ICCServiceContext serviceCtx, IReplicationStrategy replicationStrategy);
+ IFaultToleranceStrategy from(ICCServiceContext serviceCtx, boolean replicationEnabled);
/**
* Performs the required steps to change the metadata node to {@code node}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/INCLifecycleMessage.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/INCLifecycleMessage.java
index 372a88a..69432e9 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/INCLifecycleMessage.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/INCLifecycleMessage.java
@@ -23,17 +23,9 @@
public interface INCLifecycleMessage extends IMessage {
enum MessageType {
- REPLAY_LOGS_REQUEST,
- REPLAY_LOGS_RESPONSE,
- PREPARE_FAILBACK_REQUEST,
- PREPARE_FAILBACK_RESPONSE,
- COMPLETE_FAILBACK_REQUEST,
- COMPLETE_FAILBACK_RESPONSE,
REGISTRATION_TASKS_REQUEST,
REGISTRATION_TASKS_RESPONSE,
REGISTRATION_TASKS_RESULT,
- TAKEOVER_PARTITION_REQUEST,
- TAKEOVER_PARTITION_RESPONSE,
METADATA_NODE_REQUEST,
METADATA_NODE_RESPONSE
}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/IPartitionReplica.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/IPartitionReplica.java
index 5a9dc3f..761b2c6 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/IPartitionReplica.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/IPartitionReplica.java
@@ -44,4 +44,11 @@
* @return The identifier
*/
ReplicaIdentifier getIdentifier();
+
+ /**
+ * Notifies that failure {@code failure} occurred on this replica
+ *
+ * @param failure
+ */
+ void notifyFailure(Exception failure);
}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/IRemoteRecoveryManager.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/IRemoteRecoveryManager.java
deleted file mode 100644
index 51b826b..0000000
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/IRemoteRecoveryManager.java
+++ /dev/null
@@ -1,71 +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.common.replication;
-
-import java.io.IOException;
-import java.util.Map;
-import java.util.Set;
-
-import org.apache.asterix.common.exceptions.ACIDException;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-public interface IRemoteRecoveryManager {
-
- /**
- * Performs the partitions takeover process from the {@code failedNode}
- *
- * @param failedNode
- * @param partitions
- * @throws IOException
- * @throws 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
- * @throws InterruptedException
- */
- public void completeFailbackProcess() throws IOException, InterruptedException;
-
- /**
- * Replays all committed jobs logs for {@code partitions}. Optionally, flushes all datasets
- * to convert the replayed logs into LSM Components.
- *
- * @param partitions
- * @param flush
- * @throws HyracksDataException
- */
- void replayReplicaPartitionLogs(Set<Integer> partitions, boolean flush) throws HyracksDataException;
-
- /**
- * Performs the remote recovery plan by requesting data from each specified node
- * for each partitions specified.
- *
- * @param recoveryPlan
- * @throws HyracksDataException
- */
- void doRemoteRecoveryPlan(Map<String, Set<Integer>> recoveryPlan) throws HyracksDataException;
-}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/IReplicaResourcesManager.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/IReplicaResourcesManager.java
deleted file mode 100644
index 72a7f9d..0000000
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/IReplicaResourcesManager.java
+++ /dev/null
@@ -1,36 +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.common.replication;
-
-import java.util.List;
-import java.util.Set;
-
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-public interface IReplicaResourcesManager {
-
- /**
- * @param partitions
- * @return the minimum LSN of all indexes that belong to {@code partitions}.
- * @throws HyracksDataException
- */
- long getPartitionsMinLSN(Set<Integer> partitions) throws HyracksDataException;
-
- List<String> getPartitionIndexesFiles(int partition, boolean relativePath) throws HyracksDataException;
-}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/IReplicationDestination.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/IReplicationDestination.java
new file mode 100644
index 0000000..2fe9de8
--- /dev/null
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/IReplicationDestination.java
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.common.replication;
+
+import java.net.InetSocketAddress;
+import java.util.Set;
+
+public interface IReplicationDestination {
+
+ /**
+ * Adds {@code replica} to this {@link IReplicationDestination}
+ *
+ * @param replica
+ */
+ void add(IPartitionReplica replica);
+
+ /**
+ * Removes {@code replica} from this {@link IReplicationDestination}
+ *
+ * @param replica
+ */
+ void remove(IPartitionReplica replica);
+
+ /**
+ * Notifies that failure {@code failure} occurred on this {@link IReplicationDestination}
+ *
+ * @param failure
+ */
+ void notifyFailure(Exception failure);
+
+ /**
+ * Gets the list of replicas on this {@link IReplicationDestination}
+ *
+ * @return the list of replicas
+ */
+ Set<IPartitionReplica> getReplicas();
+
+ /**
+ * Gets the location of this {@link IReplicationDestination}
+ *
+ * @return the location
+ */
+ InetSocketAddress getLocation();
+}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/IReplicationManager.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/IReplicationManager.java
index 07076cf..1da2980 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/IReplicationManager.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/IReplicationManager.java
@@ -18,122 +18,44 @@
*/
package org.apache.asterix.common.replication;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.Set;
-
import org.apache.asterix.common.transactions.ILogRecord;
import org.apache.hyracks.api.replication.IIOReplicationManager;
public interface IReplicationManager extends IIOReplicationManager {
/**
- * Asynchronously sends a serialized version of the record to remote replicas.
- *
- * @param logRecord
- * The log record to be replicated,
- * @throws InterruptedException
- */
- public void replicateLog(ILogRecord logRecord) throws InterruptedException;
-
- /**
- * Checks whether a log record has been replicated
- *
- * @param logRecord
- * the log to check for.
- * @return true, if all ACKs were received from remote replicas.
- */
- public boolean hasBeenReplicated(ILogRecord logRecord);
-
- /**
- * Requests LSM components files from a remote replica.
- *
- * @param remoteReplicaId
- * The replica id to send the request to.
- * @param partitionsToRecover
- * Get files that belong to those partitions.
- * @param existingFiles
- * a list of already existing files on the requester
- * @throws IOException
- */
- public void requestReplicaFiles(String remoteReplicaId, Set<Integer> partitionsToRecover, Set<String> existingFiles)
- throws IOException;
-
- /**
- * Requests current maximum LSN from remote replicas.
- *
- * @param remoteReplicaIds
- * remote replicas to send the request to.
- * @return The maximum of the received maximum LSNs.
- * @throws IOException
- */
- public long getMaxRemoteLSN(Set<String> remoteReplicaIds) throws IOException;
-
- /**
- * @return The number of remote replicas that are in ACTIVE state.
- */
- public int getActiveReplicasCount();
-
- /**
- * @return The IDs of the remote replicas that are in DEAD state.
- */
- public Set<String> getDeadReplicasIds();
-
- /**
- * Starts processing of ASYNC replication jobs as well as Txn logs.
- *
- * @throws InterruptedException
- */
- public void startReplicationThreads() throws InterruptedException;
-
- /**
- * Checks and sets each remote replica state.
- */
- public void initializeReplicasState();
-
- /**
- * Updates remote replica (in-memory) information.
- *
- * @param replica
- * the replica to update.
- */
- public void updateReplicaInfo(Replica replica);
-
- /**
- * @return The IDs of the remote replicas that are in ACTIVE state.
- */
- public Set<String> getActiveReplicasIds();
-
- /**
- * Submits a ReplicaEvent to ReplicationEventsMonitor thread.
- *
- * @param event
- */
- public void reportReplicaEvent(ReplicaEvent event);
-
- /**
- * Sends a request to remote replicas to flush indexes that have LSN less than nonSharpCheckpointTargetLSN
- *
- * @param nonSharpCheckpointTargetLSN
- * @throws IOException
- */
- public void requestFlushLaggingReplicaIndexes(long nonSharpCheckpointTargetLSN) throws IOException;
-
- /**
- * Transfers the contents of the {@code buffer} to active remote replicas.
- * The transfer starts from the {@code buffer} current position to its limit.
- * After the transfer, the {@code buffer} position will be its limit.
- *
- * @param buffer
- */
- public void replicateTxnLogBatch(ByteBuffer buffer);
-
- IReplicationStrategy getReplicationStrategy();
-
- /**
* Registers {@code replica}. After registration, the replica will be included in all replication events
*
* @param replica
*/
void register(IPartitionReplica replica);
+
+ /**
+ * Unregisters {@code replica} from replication events.
+ *
+ * @param replica
+ */
+ void unregister(IPartitionReplica replica);
+
+ /**
+ * Notifies that failure {@code failure} occurred on {@code dest}
+ *
+ * @param dest
+ * @param failure
+ */
+ void notifyFailure(IReplicationDestination dest, Exception failure);
+
+ /**
+ * Asynchronously sends a serialized version of the record to remote replicas.
+ *
+ * @param logRecord The log record to be replicated
+ * @throws InterruptedException
+ */
+ void replicate(ILogRecord logRecord) throws InterruptedException;
+
+ /**
+ * @return the replication strategy
+ */
+ IReplicationStrategy getReplicationStrategy();
+
}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/IReplicationStrategy.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/IReplicationStrategy.java
index 93ccfbe..0ad8be2 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/IReplicationStrategy.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/IReplicationStrategy.java
@@ -18,12 +18,6 @@
*/
package org.apache.asterix.common.replication;
-import java.util.Set;
-
-import org.apache.asterix.common.config.ReplicationProperties;
-import org.apache.hyracks.api.config.IConfigManager;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-
public interface IReplicationStrategy {
/**
@@ -31,32 +25,4 @@
* @return True, if the dataset should be replicated. Otherwise false.
*/
boolean isMatch(int datasetId);
-
- Set<Replica> getRemoteReplicasAndSelf(String nodeId);
-
- /**
- * @param nodeId
- * @return The set of nodes that replicate data on {@code nodeId}.
- */
- Set<Replica> getRemotePrimaryReplicas(String nodeId);
-
- /**
- * @param node
- * @return The set of nodes that {@code nodeId} replicates data to.
- */
- Set<Replica> getRemoteReplicas(String node);
-
- /**
- * @param nodeId
- * @return true if {@code nodeId} has any remote primary replica or remote replica. Otherwise false.
- */
- default boolean isParticipant(String nodeId) {
- return !getRemoteReplicas(nodeId).isEmpty() || !getRemotePrimaryReplicas(nodeId).isEmpty();
- }
-
- /**
- * @param p
- * @return A replication strategy based on the passed configurations.
- */
- IReplicationStrategy from(ReplicationProperties p, IConfigManager configManager) throws HyracksDataException;
}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/MetadataOnlyReplicationStrategy.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/MetadataOnlyReplicationStrategy.java
index 2262437..5733a60 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/MetadataOnlyReplicationStrategy.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/MetadataOnlyReplicationStrategy.java
@@ -18,91 +18,12 @@
*/
package org.apache.asterix.common.replication;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.Set;
-import java.util.stream.Collectors;
-
-import org.apache.asterix.common.config.MetadataProperties;
-import org.apache.asterix.common.config.ReplicationProperties;
import org.apache.asterix.common.metadata.MetadataIndexImmutableProperties;
-import org.apache.hyracks.api.config.IConfigManager;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.control.common.config.ConfigManager;
-import org.apache.hyracks.control.common.controllers.NCConfig;
public class MetadataOnlyReplicationStrategy implements IReplicationStrategy {
- private String metadataPrimaryReplicaId;
- private Replica metadataPrimaryReplica;
- private Set<Replica> metadataNodeReplicas;
- MetadataProperties metadataProperties;
-
@Override
public boolean isMatch(int datasetId) {
return datasetId < MetadataIndexImmutableProperties.FIRST_AVAILABLE_USER_DATASET_ID && datasetId >= 0;
}
-
- @Override
- public Set<Replica> getRemoteReplicas(String nodeId) {
- if (nodeId.equals(metadataPrimaryReplicaId)) {
- return metadataNodeReplicas;
- }
- return Collections.emptySet();
- }
-
- @Override
- public Set<Replica> getRemoteReplicasAndSelf(String nodeId) {
-
- if (nodeId.equals(metadataPrimaryReplicaId)) {
- Set<Replica> replicasAndSelf = new HashSet<>();
- replicasAndSelf.addAll(metadataNodeReplicas);
- replicasAndSelf.add(metadataPrimaryReplica);
- return replicasAndSelf;
- }
- return Collections.emptySet();
- }
-
- @Override
- public Set<Replica> getRemotePrimaryReplicas(String nodeId) {
- if (metadataNodeReplicas.stream().map(Replica::getId).filter(replicaId -> replicaId.equals(nodeId))
- .count() != 0) {
- return new HashSet<>(Arrays.asList(metadataPrimaryReplica));
- }
- return Collections.emptySet();
- }
-
- @Override
- public MetadataOnlyReplicationStrategy from(ReplicationProperties p, IConfigManager configManager)
- throws HyracksDataException {
- MetadataOnlyReplicationStrategy st = new MetadataOnlyReplicationStrategy();
- st.metadataProperties = p.getMetadataProperties();
- st.metadataPrimaryReplicaId = st.metadataProperties.getMetadataNodeName();
- st.metadataPrimaryReplica = new Replica(st.metadataPrimaryReplicaId,
- ((ConfigManager) configManager).getNodeEffectiveConfig(st.metadataPrimaryReplicaId)
- .getString(NCConfig.Option.REPLICATION_LISTEN_ADDRESS),
- ((ConfigManager) configManager).getNodeEffectiveConfig(st.metadataPrimaryReplicaId)
- .getInt(NCConfig.Option.REPLICATION_LISTEN_PORT));
- final Set<Replica> replicas = new HashSet<>();
- Set<String> candidateSet = new HashSet<>();
- candidateSet.addAll(((ConfigManager) (configManager)).getNodeNames());
- candidateSet.remove(st.metadataPrimaryReplicaId);
- String[] candidateAry = new String[candidateSet.size()];
- candidateSet.toArray(candidateAry);
- for (int i = 0; i < candidateAry.length && i < p.getReplicationFactor(); i++) {
- replicas.add(new Replica(candidateAry[i],
- ((ConfigManager) configManager).getNodeEffectiveConfig(candidateAry[i])
- .getString(NCConfig.Option.REPLICATION_LISTEN_ADDRESS),
- ((ConfigManager) configManager).getNodeEffectiveConfig(candidateAry[i])
- .getInt(NCConfig.Option.REPLICATION_LISTEN_PORT)));
- }
- st.metadataNodeReplicas = replicas;
- return st;
- }
-
- @Override
- public boolean isParticipant(String nodeId) {
- return true;
- }
}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/NoReplicationStrategy.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/NoReplicationStrategy.java
index c32ee3c..c3d9ced 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/NoReplicationStrategy.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/NoReplicationStrategy.java
@@ -18,42 +18,10 @@
*/
package org.apache.asterix.common.replication;
-import java.util.Collections;
-import java.util.Set;
-
-import org.apache.asterix.common.config.ReplicationProperties;
-import org.apache.hyracks.api.config.IConfigManager;
-import org.apache.hyracks.control.common.config.ConfigManager;
-import org.apache.hyracks.control.common.controllers.NCConfig;
-
public class NoReplicationStrategy implements IReplicationStrategy {
@Override
public boolean isMatch(int datasetId) {
return false;
}
-
- @Override
- public boolean isParticipant(String nodeId) {
- return false;
- }
-
- @Override
- public Set<Replica> getRemotePrimaryReplicas(String nodeId) {
- return Collections.emptySet();
- }
-
- @Override
- public Set<Replica> getRemoteReplicas(String node) {
- return Collections.emptySet();
- }
-
- public Set<Replica> getRemoteReplicasAndSelf(String nodeId) {
- return Collections.emptySet();
- }
-
- @Override
- public NoReplicationStrategy from(ReplicationProperties p, IConfigManager configManager) {
- return new NoReplicationStrategy();
- }
}
\ No newline at end of file
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/Replica.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/Replica.java
deleted file mode 100644
index 52dbd25..0000000
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/Replica.java
+++ /dev/null
@@ -1,112 +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.common.replication;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.io.OutputStream;
-
-public class Replica {
-
- public enum ReplicaState {
- ACTIVE,
- DEAD,
- UNKNOWN
- }
-
- private ReplicaState state = ReplicaState.UNKNOWN;
- String nodeId;
- String ipAddr;
- int port;
-
- public Replica(String id, String ip, int port) {
- nodeId = id;
- ipAddr = ip;
- this.port = port;
- }
-
- public ReplicaState getState() {
- return state;
- }
-
- public void setState(ReplicaState state) {
- this.state = state;
- }
-
- public static Replica create(DataInput input) throws IOException {
- Replica replica = new Replica(null, null, -1);
- replica.readFields(input);
- return replica;
- }
-
- public String getId() {
- return nodeId;
- }
-
- public void writeFields(DataOutput output) throws IOException {
- output.writeUTF(nodeId);
- output.writeUTF(ipAddr);
- output.writeInt(port);
- output.writeInt(state.ordinal());
- }
-
- public void readFields(DataInput input) throws IOException {
- this.nodeId = input.readUTF();
- this.ipAddr = input.readUTF();
- this.port = input.readInt();
- this.state = ReplicaState.values()[input.readInt()];
- }
-
- public String getClusterIp() {
- return ipAddr;
- }
-
- public void setClusterIp(String ip) {
- ipAddr = ip;
- }
-
- public int getPort() {
- return port;
- }
-
- public void setPort(int port) {
- this.port = port;
- }
-
- public void serialize(OutputStream out) throws IOException {
- DataOutputStream dos = new DataOutputStream(out);
- writeFields(dos);
- }
-
- @Override
- public boolean equals(Object o) {
- if (!(o instanceof Replica)) {
- return false;
- }
- Replica other = (Replica) o;
- return nodeId.equals(other.getId());
- }
-
- @Override
- public int hashCode() {
- return nodeId.hashCode();
- }
-}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/ReplicaEvent.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/ReplicaEvent.java
deleted file mode 100644
index ae02ca9..0000000
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/ReplicaEvent.java
+++ /dev/null
@@ -1,67 +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.common.replication;
-
-import java.io.DataInput;
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.io.OutputStream;
-
-import org.apache.hyracks.api.application.IClusterLifecycleListener.ClusterEventType;
-
-public class ReplicaEvent {
-
- Replica replica;
- ClusterEventType eventType;
-
- public ReplicaEvent(Replica replica, ClusterEventType eventType) {
- this.replica = replica;
- this.eventType = eventType;
- }
-
- public Replica getReplica() {
- return replica;
- }
-
- public void setReplica(Replica replica) {
- this.replica = replica;
- }
-
- public ClusterEventType getEventType() {
- return eventType;
- }
-
- public void setEventType(ClusterEventType eventType) {
- this.eventType = eventType;
- }
-
- public void serialize(OutputStream out) throws IOException {
- DataOutputStream dos = new DataOutputStream(out);
- replica.writeFields(dos);
- dos.writeInt(eventType.ordinal());
- }
-
- public static ReplicaEvent create(DataInput input) throws IOException {
- Replica replica = Replica.create(input);
- ClusterEventType eventType = ClusterEventType.values()[input.readInt()];
- ReplicaEvent event = new ReplicaEvent(replica, eventType);
- return event;
- }
-
-}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/ReplicationStrategyFactory.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/ReplicationStrategyFactory.java
index e6b6445..8d51a99 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/ReplicationStrategyFactory.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/ReplicationStrategyFactory.java
@@ -21,39 +21,31 @@
import java.util.HashMap;
import java.util.Map;
-import org.apache.asterix.common.config.ReplicationProperties;
-import org.apache.asterix.common.exceptions.ErrorCode;
-import org.apache.asterix.common.exceptions.RuntimeDataException;
-import org.apache.hyracks.api.config.IConfigManager;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-
public class ReplicationStrategyFactory {
private static final Map<String, Class<? extends IReplicationStrategy>> BUILT_IN_REPLICATION_STRATEGY =
new HashMap<>();
static {
- BUILT_IN_REPLICATION_STRATEGY.put("no_replication", NoReplicationStrategy.class);
- BUILT_IN_REPLICATION_STRATEGY.put("chained_declustering", ChainedDeclusteringReplicationStrategy.class);
- BUILT_IN_REPLICATION_STRATEGY.put("metadata_only", MetadataOnlyReplicationStrategy.class);
+ BUILT_IN_REPLICATION_STRATEGY.put("none", NoReplicationStrategy.class);
+ BUILT_IN_REPLICATION_STRATEGY.put("all", AllDatasetsReplicationStrategy.class);
+ BUILT_IN_REPLICATION_STRATEGY.put("metadata", MetadataOnlyReplicationStrategy.class);
}
private ReplicationStrategyFactory() {
throw new AssertionError();
}
- public static IReplicationStrategy create(String name, ReplicationProperties repProp, IConfigManager ncConfig)
- throws HyracksDataException {
+ public static IReplicationStrategy create(String name) {
String strategyName = name.toLowerCase();
if (!BUILT_IN_REPLICATION_STRATEGY.containsKey(strategyName)) {
- throw new RuntimeDataException(ErrorCode.UNSUPPORTED_REPLICATION_STRATEGY, String.format(
- "%s. Available strategies: %s", strategyName, BUILT_IN_REPLICATION_STRATEGY.keySet().toString()));
+ throw new IllegalStateException("Couldn't find strategy with name: " + name);
}
Class<? extends IReplicationStrategy> clazz = BUILT_IN_REPLICATION_STRATEGY.get(strategyName);
try {
- return clazz.newInstance().from(repProp, ncConfig);
+ return clazz.newInstance();
} catch (InstantiationException | IllegalAccessException e) {
- throw new RuntimeDataException(ErrorCode.INSTANTIATION_ERROR, e, clazz.getName());
+ throw new IllegalStateException("Couldn't instantiated replication strategy: " + name, e);
}
}
}
\ No newline at end of file
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/storage/ResourceReference.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/storage/ResourceReference.java
index 8cc11a8..c7de7a3 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/storage/ResourceReference.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/storage/ResourceReference.java
@@ -91,6 +91,10 @@
ref.root = tokens[--offset];
}
+ public int getPartitionNum() {
+ return Integer.parseInt(partition.substring(StorageConstants.PARTITION_DIR_PREFIX.length()));
+ }
+
protected static void parseLegacyPath(ResourceReference ref, String path) {
// old format: root/partition/dataverse/datasetName_idx_IndexName/fileName
final String[] tokens = path.split(File.separator);
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/ILogRecord.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/ILogRecord.java
index 4090b65..3274849 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/ILogRecord.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/ILogRecord.java
@@ -20,7 +20,6 @@
import java.nio.ByteBuffer;
-import org.apache.asterix.common.replication.IReplicationThread;
import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
public interface ILogRecord {
@@ -133,8 +132,6 @@
public void readRemoteLog(ByteBuffer buffer);
- public void setReplicationThread(IReplicationThread replicationThread);
-
public void setLogSource(byte logSource);
public byte getLogSource();
@@ -150,7 +147,7 @@
public void setReplicated(boolean replicated);
/**
- * @return a flag indicating whether the log record should be sent to remote replicas
+ * @return a flag indicating whether the log was replicated
*/
public boolean isReplicated();
@@ -169,4 +166,18 @@
public void logAppended(long lsn);
public long getPreviousMarkerLSN();
+
+ /**
+ * Sets flag indicating if this log should be replicated or not
+ *
+ * @param replicate
+ */
+ void setReplicate(boolean replicate);
+
+ /**
+ * Gets a flag indicating if this log should be replicated or not
+ *
+ * @return the flag
+ */
+ boolean isReplicate();
}
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/ILogRequester.java
similarity index 72%
copy from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
copy to asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/ILogRequester.java
index b750c64..8e874a2 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/ILogRequester.java
@@ -16,4 +16,14 @@
* specific language governing permissions and limitations
* under the License.
*/
-start asterix_nc1
\ No newline at end of file
+package org.apache.asterix.common.transactions;
+
+public interface ILogRequester {
+
+ /**
+ * Sends a notification to the log requester that {@code logRecord} has been flushed.
+ *
+ * @param logRecord The log that has been flushed.
+ */
+ void notifyFlushed(ILogRecord logRecord);
+}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/IRecoveryManager.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/IRecoveryManager.java
index dea7a67..bfe7963 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/IRecoveryManager.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/IRecoveryManager.java
@@ -119,4 +119,15 @@
* @throws ACIDException
*/
void startLocalRecovery(Set<Integer> partitions) throws IOException, ACIDException;
+
+ /**
+ * Replay the commited transactions' logs belonging to {@code partitions}. if {@code flush} is true,
+ * all datasets are flushed after the logs are replayed.
+ *
+ * @param partitions
+ * @param flush
+ * @throws HyracksDataException
+ */
+ void replayReplicaPartitionLogs(Set<Integer> partitions, boolean flush) throws HyracksDataException;
+
}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/LogRecord.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/LogRecord.java
index 68afb2a..6b38aa1 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/LogRecord.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/LogRecord.java
@@ -24,7 +24,6 @@
import java.util.zip.CRC32;
import org.apache.asterix.common.context.PrimaryIndexOperationTracker;
-import org.apache.asterix.common.replication.IReplicationThread;
import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
import org.apache.hyracks.storage.am.common.tuples.SimpleTupleReference;
import org.apache.hyracks.storage.am.common.tuples.SimpleTupleWriter;
@@ -94,18 +93,21 @@
private final CRC32 checksumGen;
private int[] PKFields;
private PrimaryIndexOperationTracker opTracker;
- private IReplicationThread replicationThread;
+
/**
* The fields (numOfFlushedIndexes and nodeId) are used for remote 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;
+ private final AtomicBoolean replicated;
+ private boolean replicate = false;
+ private ILogRequester requester;
public LogRecord(ILogMarkerCallback callback) {
this.callback = callback;
isFlushed = new AtomicBoolean(false);
+ replicated = new AtomicBoolean(false);
readPKValue = new PrimaryKeyTupleReference();
readNewValue = SimpleTupleWriter.INSTANCE.createTupleReference();
readOldValue = SimpleTupleWriter.INSTANCE.createTupleReference();
@@ -638,15 +640,6 @@
this.nodeId = nodeId;
}
- public IReplicationThread getReplicationThread() {
- return replicationThread;
- }
-
- @Override
- public void setReplicationThread(IReplicationThread replicationThread) {
- this.replicationThread = replicationThread;
- }
-
@Override
public void setLogSource(byte logSource) {
this.logSource = logSource;
@@ -684,13 +677,13 @@
}
@Override
- public void setReplicated(boolean replicate) {
- this.replicated = replicate;
+ public void setReplicated(boolean replicated) {
+ this.replicated.set(replicated);
}
@Override
public boolean isReplicated() {
- return replicated;
+ return replicated.get();
}
@Override
@@ -732,4 +725,22 @@
public ByteBuffer getMarker() {
return marker;
}
+
+ @Override
+ public void setReplicate(boolean replicate) {
+ this.replicate = replicate;
+ }
+
+ @Override
+ public boolean isReplicate() {
+ return replicate;
+ }
+
+ public ILogRequester getRequester() {
+ return requester;
+ }
+
+ public void setRequester(ILogRequester requester) {
+ this.requester = requester;
+ }
}
diff --git a/asterixdb/asterix-replication/pom.xml b/asterixdb/asterix-replication/pom.xml
index 7072e48..bb983ad 100644
--- a/asterixdb/asterix-replication/pom.xml
+++ b/asterixdb/asterix-replication/pom.xml
@@ -51,10 +51,6 @@
<artifactId>hyracks-util</artifactId>
</dependency>
<dependency>
- <groupId>commons-io</groupId>
- <artifactId>commons-io</artifactId>
- </dependency>
- <dependency>
<groupId>org.apache.hyracks</groupId>
<artifactId>hyracks-storage-am-lsm-common</artifactId>
</dependency>
@@ -72,14 +68,6 @@
<version>${project.version}</version>
</dependency>
<dependency>
- <groupId>org.apache.hyracks</groupId>
- <artifactId>hyracks-control-nc</artifactId>
- </dependency>
- <dependency>
- <groupId>org.apache.hyracks</groupId>
- <artifactId>hyracks-control-common</artifactId>
- </dependency>
- <dependency>
<groupId>com.fasterxml.jackson.core</groupId>
<artifactId>jackson-core</artifactId>
</dependency>
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/api/IReplicaTask.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/api/IReplicaTask.java
index 001d41f..807ae5f 100644
--- a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/api/IReplicaTask.java
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/api/IReplicaTask.java
@@ -19,7 +19,6 @@
package org.apache.asterix.replication.api;
import org.apache.asterix.common.api.INcApplicationContext;
-import org.apache.asterix.common.replication.IReplicationThread;
import org.apache.hyracks.api.exceptions.HyracksDataException;
public interface IReplicaTask extends IReplicationMessage {
@@ -31,5 +30,5 @@
* @param worker
* @throws HyracksDataException
*/
- void perform(INcApplicationContext appCtx, IReplicationThread worker) throws HyracksDataException;
+ void perform(INcApplicationContext appCtx, IReplicationWorker worker) throws HyracksDataException;
}
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/api/IReplicationMessage.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/api/IReplicationMessage.java
index 2e1cb8a..bc6e87b 100644
--- a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/api/IReplicationMessage.java
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/api/IReplicationMessage.java
@@ -20,7 +20,7 @@
import java.io.OutputStream;
-import org.apache.asterix.replication.functions.ReplicationProtocol;
+import org.apache.asterix.replication.messaging.ReplicationProtocol;
import org.apache.hyracks.api.exceptions.HyracksDataException;
public interface IReplicationMessage {
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/IReplicationThread.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/api/IReplicationWorker.java
similarity index 78%
rename from asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/IReplicationThread.java
rename to asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/api/IReplicationWorker.java
index 5b9d4fa..8840c3f 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/IReplicationThread.java
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/api/IReplicationWorker.java
@@ -16,21 +16,14 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.asterix.common.replication;
+package org.apache.asterix.replication.api;
import java.nio.ByteBuffer;
import java.nio.channels.SocketChannel;
import org.apache.asterix.common.transactions.LogRecord;
-public interface IReplicationThread extends Runnable {
-
- /**
- * Sends a notification to this thread that logRecord has been flushed.
- *
- * @param logRecord The log that has been flushed.
- */
- void notifyLogReplicationRequester(LogRecord logRecord);
+public interface IReplicationWorker extends Runnable {
/**
* @return The replication socket channel.
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/storage/PartitionReplica.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/api/PartitionReplica.java
similarity index 90%
rename from asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/storage/PartitionReplica.java
rename to asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/api/PartitionReplica.java
index b7fa49d..f752745 100644
--- a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/storage/PartitionReplica.java
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/api/PartitionReplica.java
@@ -16,7 +16,7 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.asterix.replication.storage;
+package org.apache.asterix.replication.api;
import static org.apache.asterix.common.replication.IPartitionReplica.PartitionReplicaStatus.CATCHING_UP;
import static org.apache.asterix.common.replication.IPartitionReplica.PartitionReplicaStatus.DISCONNECTED;
@@ -30,8 +30,8 @@
import org.apache.asterix.common.exceptions.ReplicationException;
import org.apache.asterix.common.replication.IPartitionReplica;
import org.apache.asterix.common.storage.ReplicaIdentifier;
-import org.apache.asterix.replication.functions.ReplicationProtocol;
-import org.apache.asterix.replication.recovery.ReplicaSynchronizer;
+import org.apache.asterix.replication.messaging.ReplicationProtocol;
+import org.apache.asterix.replication.sync.ReplicaSynchronizer;
import org.apache.hyracks.util.JSONUtil;
import org.apache.hyracks.util.StorageUtil;
import org.apache.hyracks.util.annotations.ThreadSafe;
@@ -70,6 +70,11 @@
return id;
}
+ @Override
+ public synchronized void notifyFailure(Exception failure) {
+ setStatus(DISCONNECTED);
+ }
+
public synchronized void sync() {
if (status == IN_SYNC || status == CATCHING_UP) {
return;
@@ -81,7 +86,7 @@
setStatus(IN_SYNC);
} catch (Exception e) {
LOGGER.error(() -> "Failed to sync replica " + this, e);
- setStatus(DISCONNECTED);
+ notifyFailure(e);
} finally {
close();
}
@@ -109,11 +114,11 @@
sc = null;
}
} catch (IOException e) {
- throw new ReplicationException(e);
+ LOGGER.warn("Failed to close channel", e);
}
}
- public synchronized ByteBuffer gerReusableBuffer() {
+ public synchronized ByteBuffer getReusableBuffer() {
if (reusbaleBuf == null) {
reusbaleBuf = ByteBuffer.allocate(INITIAL_BUFFER_SIZE);
}
@@ -154,6 +159,9 @@
}
private synchronized void setStatus(PartitionReplicaStatus status) {
+ if (this.status == status) {
+ return;
+ }
LOGGER.info(() -> "Replica " + this + " status changing: " + this.status + " -> " + status);
this.status = status;
}
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/api/ReplicationDestination.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/api/ReplicationDestination.java
new file mode 100644
index 0000000..a092322
--- /dev/null
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/api/ReplicationDestination.java
@@ -0,0 +1,128 @@
+/*
+ * 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.api;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.nio.channels.SocketChannel;
+import java.util.HashSet;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+
+import org.apache.asterix.common.exceptions.ReplicationException;
+import org.apache.asterix.common.replication.IPartitionReplica;
+import org.apache.asterix.common.replication.IReplicationDestination;
+import org.apache.asterix.replication.messaging.ReplicationProtocol;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+public class ReplicationDestination implements IReplicationDestination {
+
+ private static final Logger LOGGER = LogManager.getLogger();
+ private final Set<IPartitionReplica> replicas = new HashSet<>();
+ private final InetSocketAddress location;
+ private SocketChannel logRepChannel;
+
+ private ReplicationDestination(InetSocketAddress location) {
+ this.location = location;
+ }
+
+ public static ReplicationDestination at(InetSocketAddress location) {
+ return new ReplicationDestination(location);
+ }
+
+ @Override
+ public synchronized void add(IPartitionReplica replica) {
+ replicas.add(replica);
+ }
+
+ @Override
+ public synchronized void remove(IPartitionReplica replica) {
+ replicas.remove(replica);
+ }
+
+ @Override
+ public synchronized void notifyFailure(Exception failure) {
+ replicas.forEach(replica -> replica.notifyFailure(failure));
+ closeLogReplicationChannel();
+ }
+
+ @Override
+ public Set<IPartitionReplica> getReplicas() {
+ return new HashSet<>(replicas);
+ }
+
+ public synchronized Optional<IPartitionReplica> getPartitionReplica(int partition) {
+ return replicas.stream().filter(replica -> replica.getIdentifier().getPartition() == partition
+ && replica.getStatus() == IPartitionReplica.PartitionReplicaStatus.IN_SYNC).findAny();
+ }
+
+ public synchronized SocketChannel getLogReplicationChannel() {
+ try {
+ if (logRepChannel == null || !logRepChannel.isOpen() || !logRepChannel.isConnected()) {
+ logRepChannel = SocketChannel.open();
+ logRepChannel.configureBlocking(true);
+ logRepChannel.connect(location);
+ }
+ return logRepChannel;
+ } catch (IOException e) {
+ throw new ReplicationException(e);
+ }
+ }
+
+ private synchronized void closeLogReplicationChannel() {
+ try {
+ if (logRepChannel != null && logRepChannel.isOpen()) {
+ ReplicationProtocol.sendGoodbye(logRepChannel);
+ logRepChannel.close();
+ logRepChannel = null;
+ }
+ } catch (IOException e) {
+ LOGGER.warn("Exception while closing socket", e);
+ }
+ }
+
+ @Override
+ public InetSocketAddress getLocation() {
+ return location;
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+ ReplicationDestination that = (ReplicationDestination) o;
+ return Objects.equals(location, that.location);
+ }
+
+ @Override
+ public String toString() {
+ return location.toString();
+ }
+
+ @Override
+ public int hashCode() {
+ return Objects.hash(location);
+ }
+}
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/functions/ReplicaFilesRequest.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/functions/ReplicaFilesRequest.java
deleted file mode 100644
index f5b2378..0000000
--- a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/functions/ReplicaFilesRequest.java
+++ /dev/null
@@ -1,70 +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.functions;
-
-import java.io.DataInput;
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.io.OutputStream;
-import java.util.HashSet;
-import java.util.Set;
-
-public class ReplicaFilesRequest {
- private final Set<Integer> partitionIds;
- private final Set<String> existingFiles;
-
- public ReplicaFilesRequest(Set<Integer> partitionIds, Set<String> existingFiles) {
- this.partitionIds = partitionIds;
- this.existingFiles = existingFiles;
- }
-
- public void serialize(OutputStream out) throws IOException {
- DataOutputStream dos = new DataOutputStream(out);
- dos.writeInt(partitionIds.size());
- for (Integer partitionId : partitionIds) {
- dos.writeInt(partitionId);
- }
- dos.writeInt(existingFiles.size());
- for (String fileName : existingFiles) {
- dos.writeUTF(fileName);
- }
- }
-
- public static ReplicaFilesRequest create(DataInput input) throws IOException {
- int size = input.readInt();
- Set<Integer> partitionIds = new HashSet<>(size);
- for (int i = 0; i < size; i++) {
- partitionIds.add(input.readInt());
- }
- int filesCount = input.readInt();
- Set<String> existingFiles = new HashSet<>(filesCount);
- for (int i = 0; i < filesCount; i++) {
- existingFiles.add(input.readUTF());
- }
- return new ReplicaFilesRequest(partitionIds, existingFiles);
- }
-
- public Set<Integer> getPartitionIds() {
- return partitionIds;
- }
-
- public Set<String> getExistingFiles() {
- return existingFiles;
- }
-}
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/functions/ReplicaIndexFlushRequest.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/functions/ReplicaIndexFlushRequest.java
deleted file mode 100644
index a3c269d..0000000
--- a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/functions/ReplicaIndexFlushRequest.java
+++ /dev/null
@@ -1,60 +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.functions;
-
-import java.io.DataInput;
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.io.OutputStream;
-import java.util.HashSet;
-import java.util.Set;
-
-public class ReplicaIndexFlushRequest {
- Set<Long> laggingRescouresIds;
-
- public ReplicaIndexFlushRequest(Set<Long> laggingRescouresIds) {
- this.laggingRescouresIds = laggingRescouresIds;
- }
-
- public void serialize(OutputStream out) throws IOException {
- DataOutputStream dos = new DataOutputStream(out);
- dos.writeInt(laggingRescouresIds.size());
- for (Long resourceId : laggingRescouresIds) {
- dos.writeLong(resourceId);
- }
- }
-
- public static ReplicaIndexFlushRequest create(DataInput input) throws IOException {
- int numOfResources = input.readInt();
- Set<Long> laggingRescouresIds = new HashSet<Long>(numOfResources);
- for (int i = 0; i < numOfResources; i++) {
- laggingRescouresIds.add(input.readLong());
- }
- return new ReplicaIndexFlushRequest(laggingRescouresIds);
- }
-
- public Set<Long> getLaggingRescouresIds() {
- return laggingRescouresIds;
- }
-
- public void setLaggingRescouresIds(Set<Long> laggingRescouresIds) {
- this.laggingRescouresIds = laggingRescouresIds;
- }
-
-}
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/functions/ReplicaLogsRequest.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/functions/ReplicaLogsRequest.java
deleted file mode 100644
index 90df1e7..0000000
--- a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/functions/ReplicaLogsRequest.java
+++ /dev/null
@@ -1,71 +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.functions;
-
-import java.io.DataInput;
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.io.OutputStream;
-import java.util.HashSet;
-import java.util.Set;
-
-public class ReplicaLogsRequest {
- Set<String> replicaIds;
- long fromLSN;
-
- public ReplicaLogsRequest(Set<String> replicaIds, long fromLSN) {
- this.replicaIds = replicaIds;
- this.fromLSN = fromLSN;
- }
-
- public void serialize(OutputStream out) throws IOException {
- DataOutputStream dos = new DataOutputStream(out);
- dos.writeInt(replicaIds.size());
- for (String replicaId : replicaIds) {
- dos.writeUTF(replicaId);
- }
- dos.writeLong(fromLSN);
- }
-
- public static ReplicaLogsRequest create(DataInput input) throws IOException {
- int size = input.readInt();
- Set<String> replicaIds = new HashSet<String>(size);
- for (int i = 0; i < size; i++) {
- replicaIds.add(input.readUTF());
- }
- long fromLSN = input.readLong();
- return new ReplicaLogsRequest(replicaIds, fromLSN);
- }
-
- public Set<String> getReplicaIds() {
- return replicaIds;
- }
-
- public void setReplicaIds(Set<String> replicaIds) {
- this.replicaIds = replicaIds;
- }
-
- public long getFromLSN() {
- return fromLSN;
- }
-
- public void setFromLSN(long fromLSN) {
- this.fromLSN = fromLSN;
- }
-}
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/functions/ReplicationProtocol.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/functions/ReplicationProtocol.java
deleted file mode 100644
index 8094548..0000000
--- a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/functions/ReplicationProtocol.java
+++ /dev/null
@@ -1,354 +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.functions;
-
-import java.io.ByteArrayInputStream;
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.nio.channels.SocketChannel;
-
-import org.apache.asterix.common.exceptions.ReplicationException;
-import org.apache.asterix.common.replication.ReplicaEvent;
-import org.apache.asterix.replication.api.IReplicationMessage;
-import org.apache.asterix.replication.management.NetworkingUtil;
-import org.apache.asterix.replication.messaging.CheckpointPartitionIndexesTask;
-import org.apache.asterix.replication.messaging.DeleteFileTask;
-import org.apache.asterix.replication.messaging.PartitionResourcesListResponse;
-import org.apache.asterix.replication.messaging.PartitionResourcesListTask;
-import org.apache.asterix.replication.messaging.ReplicateFileTask;
-import org.apache.asterix.replication.storage.LSMComponentProperties;
-import org.apache.asterix.replication.storage.LSMIndexFileProperties;
-import org.apache.asterix.replication.storage.PartitionReplica;
-import org.apache.hyracks.data.std.util.ExtendedByteArrayOutputStream;
-
-public class ReplicationProtocol {
-
- /**
- * All replication messages start with ReplicationRequestType (4 bytes), then the length of the request in bytes
- */
- public static final String JOB_REPLICATION_ACK = "$";
-
- public static final int REPLICATION_REQUEST_TYPE_SIZE = Integer.BYTES;
- private static final int REPLICATION_REQUEST_HEADER_SIZE = REPLICATION_REQUEST_TYPE_SIZE + Integer.BYTES;
-
- /*
- * ReplicationRequestType:
- * REPLICATE_LOG: txn log replication
- * REPLICATE_FILE: replicate a file(s)
- * DELETE_FILE: delete a file(s)
- * GET_REPLICA_FILES: used during remote recovery to request lost LSM Components
- * GET_REPLICA_MAX_LSN: used during remote recovery initialize a log manager LSN
- * GOODBYE: used to notify replicas that the replication request has been completed
- * REPLICA_EVENT: used to notify replicas about a remote replica split/merge.
- * LSM_COMPONENT_PROPERTIES: used to send the properties of an LSM Component before its physical files are sent
- * ACK: used to notify the requesting replica that the request has been completed successfully
- * FLUSH_INDEX: request remote replica to flush an LSM component
- */
- public enum ReplicationRequestType {
- REPLICATE_LOG,
- REPLICATE_FILE,
- DELETE_FILE,
- GET_REPLICA_FILES,
- GET_REPLICA_MAX_LSN,
- GOODBYE,
- REPLICA_EVENT,
- LSM_COMPONENT_PROPERTIES,
- ACK,
- FLUSH_INDEX,
- PARTITION_RESOURCES_REQUEST,
- PARTITION_RESOURCES_RESPONSE,
- REPLICATE_RESOURCE_FILE,
- DELETE_RESOURCE_FILE,
- CHECKPOINT_PARTITION
- }
-
- public static ByteBuffer readRequest(SocketChannel socketChannel, ByteBuffer dataBuffer) throws IOException {
- //read request size
- NetworkingUtil.readBytes(socketChannel, dataBuffer, Integer.BYTES);
- int requestSize = dataBuffer.getInt();
-
- if (dataBuffer.capacity() < requestSize) {
- dataBuffer = ByteBuffer.allocate(requestSize);
- }
-
- //read request
- NetworkingUtil.readBytes(socketChannel, dataBuffer, requestSize);
-
- return dataBuffer;
- }
-
- public static ByteBuffer writeLSMComponentPropertiesRequest(LSMComponentProperties lsmCompProp, ByteBuffer buffer)
- throws IOException {
- ExtendedByteArrayOutputStream outputStream = new ExtendedByteArrayOutputStream();
- try (DataOutputStream oos = new DataOutputStream(outputStream)) {
- lsmCompProp.serialize(oos);
- int requestSize = REPLICATION_REQUEST_HEADER_SIZE + oos.size();
- if (buffer.capacity() < requestSize) {
- buffer = ByteBuffer.allocate(requestSize);
- } else {
- buffer.clear();
- }
- buffer.putInt(ReplicationRequestType.LSM_COMPONENT_PROPERTIES.ordinal());
- buffer.putInt(oos.size());
- buffer.put(outputStream.getByteArray(), 0, outputStream.getLength());
- buffer.flip();
- return buffer;
- }
- }
-
- public static ReplicationRequestType getRequestType(SocketChannel socketChannel, ByteBuffer byteBuffer)
- throws IOException {
- //read replication request type
- NetworkingUtil.readBytes(socketChannel, byteBuffer, REPLICATION_REQUEST_TYPE_SIZE);
-
- ReplicationRequestType requestType = ReplicationProtocol.ReplicationRequestType.values()[byteBuffer.getInt()];
- return requestType;
- }
-
- public static LSMComponentProperties readLSMPropertiesRequest(ByteBuffer buffer) throws IOException {
- ByteArrayInputStream bais = new ByteArrayInputStream(buffer.array(), buffer.position(), buffer.limit());
- DataInputStream dis = new DataInputStream(bais);
- return LSMComponentProperties.create(dis);
- }
-
- public static ByteBuffer getGoodbyeBuffer() {
- ByteBuffer bb = ByteBuffer.allocate(REPLICATION_REQUEST_TYPE_SIZE);
- bb.putInt(ReplicationRequestType.GOODBYE.ordinal());
- bb.flip();
- return bb;
- }
-
- public static ByteBuffer getAckBuffer() {
- ByteBuffer bb = ByteBuffer.allocate(REPLICATION_REQUEST_TYPE_SIZE);
- bb.putInt(ReplicationRequestType.ACK.ordinal());
- bb.flip();
- return bb;
- }
-
- public static ByteBuffer writeFileReplicationRequest(ByteBuffer requestBuffer, LSMIndexFileProperties afp,
- ReplicationRequestType requestType) throws IOException {
- ExtendedByteArrayOutputStream outputStream = new ExtendedByteArrayOutputStream();
- try (DataOutputStream oos = new DataOutputStream(outputStream)) {
- afp.serialize(oos);
- int requestSize = REPLICATION_REQUEST_HEADER_SIZE + oos.size();
- if (requestBuffer.capacity() < requestSize) {
- requestBuffer = ByteBuffer.allocate(requestSize);
- } else {
- requestBuffer.clear();
- }
- requestBuffer.putInt(requestType.ordinal());
- requestBuffer.putInt(oos.size());
- requestBuffer.put(outputStream.getByteArray(), 0, outputStream.getLength());
- requestBuffer.flip();
- return requestBuffer;
- }
- }
-
- public static LSMIndexFileProperties readFileReplicationRequest(ByteBuffer buffer) throws IOException {
- ByteArrayInputStream bais = new ByteArrayInputStream(buffer.array(), buffer.position(), buffer.limit());
- DataInputStream dis = new DataInputStream(bais);
- return LSMIndexFileProperties.create(dis);
- }
-
- public static ByteBuffer writeReplicaEventRequest(ReplicaEvent event) throws IOException {
- ExtendedByteArrayOutputStream outputStream = new ExtendedByteArrayOutputStream();
- try (DataOutputStream oos = new DataOutputStream(outputStream)) {
- event.serialize(oos);
- ByteBuffer buffer = ByteBuffer.allocate(REPLICATION_REQUEST_HEADER_SIZE + oos.size());
- buffer.putInt(ReplicationRequestType.REPLICA_EVENT.ordinal());
- buffer.putInt(oos.size());
- buffer.put(outputStream.getByteArray(), 0, outputStream.getLength());
- buffer.flip();
- return buffer;
- }
- }
-
- public static ReplicaEvent readReplicaEventRequest(ByteBuffer buffer) throws IOException {
- ByteArrayInputStream bais = new ByteArrayInputStream(buffer.array(), buffer.position(), buffer.limit());
- DataInputStream dis = new DataInputStream(bais);
-
- return ReplicaEvent.create(dis);
- }
-
- public static ByteBuffer writeGetReplicaFilesRequest(ByteBuffer buffer, ReplicaFilesRequest request)
- throws IOException {
- ExtendedByteArrayOutputStream outputStream = new ExtendedByteArrayOutputStream();
- 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();
- }
- buffer.putInt(ReplicationRequestType.GET_REPLICA_FILES.ordinal());
- buffer.putInt(oos.size());
- buffer.put(outputStream.getByteArray(), 0, outputStream.getLength());
- buffer.flip();
- return buffer;
- }
- }
-
- public static ByteBuffer writeGetReplicaIndexFlushRequest(ByteBuffer buffer, ReplicaIndexFlushRequest request)
- throws IOException {
- ExtendedByteArrayOutputStream outputStream = new ExtendedByteArrayOutputStream();
- 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();
- }
- buffer.putInt(ReplicationRequestType.FLUSH_INDEX.ordinal());
- buffer.putInt(oos.size());
- buffer.put(outputStream.getByteArray(), 0, outputStream.getLength());
- buffer.flip();
- return buffer;
- }
- }
-
- public static ReplicaFilesRequest readReplicaFileRequest(ByteBuffer buffer) throws IOException {
- ByteArrayInputStream bais = new ByteArrayInputStream(buffer.array(), buffer.position(), buffer.limit());
- DataInputStream dis = new DataInputStream(bais);
- return ReplicaFilesRequest.create(dis);
- }
-
- public static ReplicaIndexFlushRequest readReplicaIndexFlushRequest(ByteBuffer buffer) throws IOException {
- ByteArrayInputStream bais = new ByteArrayInputStream(buffer.array(), buffer.position(), buffer.limit());
- DataInputStream dis = new DataInputStream(bais);
- return ReplicaIndexFlushRequest.create(dis);
- }
-
- public static void writeGetReplicaMaxLSNRequest(ByteBuffer requestBuffer) {
- requestBuffer.clear();
- requestBuffer.putInt(ReplicationRequestType.GET_REPLICA_MAX_LSN.ordinal());
- requestBuffer.flip();
- }
-
- public static int getJobIdFromLogAckMessage(String msg) {
- return Integer.parseInt(msg.substring((msg.indexOf(JOB_REPLICATION_ACK) + 1)));
- }
-
- public static String getNodeIdFromLogAckMessage(String msg) {
- return msg.substring(0, msg.indexOf(JOB_REPLICATION_ACK));
- }
-
- /**
- * Sends a goodbye request to a remote replica indicating the end of a replication request.
- *
- * @param socketChannel
- * the remote replica socket.
- * @throws IOException
- */
- public static void sendGoodbye(SocketChannel socketChannel) throws IOException {
- ByteBuffer goodbyeBuffer = ReplicationProtocol.getGoodbyeBuffer();
- NetworkingUtil.transferBufferToChannel(socketChannel, goodbyeBuffer);
- }
-
- public static void sendAck(SocketChannel socketChannel) throws IOException {
- ByteBuffer ackBuffer = ReplicationProtocol.getAckBuffer();
- NetworkingUtil.transferBufferToChannel(socketChannel, ackBuffer);
- }
-
- public static void sendAck(SocketChannel socketChannel, ByteBuffer buf) {
- try {
- buf.clear();
- buf.putInt(ReplicationRequestType.ACK.ordinal());
- buf.flip();
- NetworkingUtil.transferBufferToChannel(socketChannel, buf);
- } catch (IOException e) {
- throw new ReplicationException(e);
- }
- }
-
- public static void waitForAck(PartitionReplica replica) throws IOException {
- final SocketChannel channel = replica.getChannel();
- final ByteBuffer buf = replica.gerReusableBuffer();
- ReplicationRequestType responseFunction = ReplicationProtocol.getRequestType(channel, buf);
- if (responseFunction != ReplicationRequestType.ACK) {
- throw new IllegalStateException("Unexpected response while waiting for ack.");
- }
- }
-
- public static void sendTo(PartitionReplica replica, IReplicationMessage task) {
- final SocketChannel channel = replica.getChannel();
- final ByteBuffer buf = replica.gerReusableBuffer();
- sendTo(channel, task, buf);
- }
-
- public static void sendTo(SocketChannel channel, IReplicationMessage task, ByteBuffer buf) {
- ExtendedByteArrayOutputStream outputStream = new ExtendedByteArrayOutputStream();
- try (DataOutputStream oos = new DataOutputStream(outputStream)) {
- task.serialize(oos);
- final int requestSize = REPLICATION_REQUEST_HEADER_SIZE + oos.size();
- final ByteBuffer requestBuffer = ensureSize(buf, requestSize);
- requestBuffer.putInt(task.getMessageType().ordinal());
- requestBuffer.putInt(oos.size());
- requestBuffer.put(outputStream.getByteArray(), 0, outputStream.getLength());
- requestBuffer.flip();
- NetworkingUtil.transferBufferToChannel(channel, requestBuffer);
- } catch (IOException e) {
- throw new ReplicationException(e);
- }
- }
-
- public static IReplicationMessage read(SocketChannel socketChannel, ByteBuffer buffer) throws IOException {
- final ReplicationRequestType type = getRequestType(socketChannel, buffer);
- return readMessage(type, socketChannel, buffer);
- }
-
- public static IReplicationMessage readMessage(ReplicationRequestType type, SocketChannel socketChannel,
- ByteBuffer buffer) {
- try {
- ReplicationProtocol.readRequest(socketChannel, buffer);
- final ByteArrayInputStream input =
- new ByteArrayInputStream(buffer.array(), buffer.position(), buffer.limit());
- try (DataInputStream dis = new DataInputStream(input)) {
- switch (type) {
- case PARTITION_RESOURCES_REQUEST:
- return PartitionResourcesListTask.create(dis);
- case PARTITION_RESOURCES_RESPONSE:
- return PartitionResourcesListResponse.create(dis);
- case REPLICATE_RESOURCE_FILE:
- return ReplicateFileTask.create(dis);
- case DELETE_RESOURCE_FILE:
- return DeleteFileTask.create(dis);
- case CHECKPOINT_PARTITION:
- return CheckpointPartitionIndexesTask.create(dis);
- default:
- throw new IllegalStateException("Unrecognized replication message");
- }
- }
- } catch (IOException e) {
- throw new ReplicationException(e);
- }
- }
-
- private static ByteBuffer ensureSize(ByteBuffer buffer, int size) {
- if (buffer.capacity() < size) {
- return ByteBuffer.allocate(size);
- }
- buffer.clear();
- return buffer;
- }
-}
\ No newline at end of file
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/logging/RemoteLogMapping.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/logging/RemoteLogMapping.java
deleted file mode 100644
index 0444952..0000000
--- a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/logging/RemoteLogMapping.java
+++ /dev/null
@@ -1,62 +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.logging;
-
-import java.util.concurrent.atomic.AtomicInteger;
-
-public class RemoteLogMapping {
-
- private String remoteNodeID;
- private long remoteLSN;
- private long localLSN;
- public AtomicInteger numOfFlushedIndexes = new AtomicInteger();
-
- public void setRemoteNodeID(String remoteNodeID) {
- this.remoteNodeID = remoteNodeID;
- }
-
- public long getRemoteLSN() {
- return remoteLSN;
- }
-
- public void setRemoteLSN(long remoteLSN) {
- this.remoteLSN = remoteLSN;
- }
-
- public long getLocalLSN() {
- return localLSN;
- }
-
- public void setLocalLSN(long localLSN) {
- this.localLSN = localLSN;
- }
-
- @Override
- public String toString() {
- StringBuilder sb = new StringBuilder();
- sb.append("Remote Node: " + remoteNodeID);
- sb.append(" Remote LSN: " + remoteLSN);
- sb.append(" Local LSN: " + localLSN);
- return sb.toString();
- }
-
- public String getNodeUniqueLSN() {
- return TxnLogUtil.getNodeUniqueLSN(remoteNodeID, remoteLSN);
- }
-}
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/logging/RemoteLogRecord.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/logging/RemoteLogRecord.java
new file mode 100644
index 0000000..0b496b8
--- /dev/null
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/logging/RemoteLogRecord.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.replication.logging;
+
+import org.apache.asterix.replication.api.IReplicationWorker;
+import org.apache.asterix.common.transactions.LogRecord;
+
+public class RemoteLogRecord extends LogRecord {
+
+ private long masterLsn;
+ private IReplicationWorker replicationWorker;
+
+ public long getMasterLsn() {
+ return masterLsn;
+ }
+
+ public void setMasterLsn(long masterLsn) {
+ this.masterLsn = masterLsn;
+ }
+
+ public IReplicationWorker getReplicationWorker() {
+ return replicationWorker;
+ }
+
+ public void setReplicationWorker(IReplicationWorker replicationWorker) {
+ this.replicationWorker = replicationWorker;
+ }
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/logging/RemoteLogsNotifier.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/logging/RemoteLogsNotifier.java
new file mode 100644
index 0000000..d63496f
--- /dev/null
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/logging/RemoteLogsNotifier.java
@@ -0,0 +1,102 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.replication.logging;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.function.Predicate;
+import java.util.stream.Collectors;
+
+import org.apache.asterix.common.api.INcApplicationContext;
+import org.apache.asterix.common.dataflow.DatasetLocalResource;
+import org.apache.asterix.common.storage.DatasetResourceReference;
+import org.apache.asterix.common.storage.IIndexCheckpointManager;
+import org.apache.asterix.common.storage.IIndexCheckpointManagerProvider;
+import org.apache.asterix.common.transactions.LogType;
+import org.apache.asterix.replication.messaging.ReplicationProtocol;
+import org.apache.asterix.transaction.management.resource.PersistentLocalResourceRepository;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.common.LocalResource;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+class RemoteLogsNotifier implements Runnable {
+
+ private static final Logger LOGGER = LogManager.getLogger();
+ private final PersistentLocalResourceRepository localResourceRep;
+ private final IIndexCheckpointManagerProvider indexCheckpointManagerProvider;
+ private final LinkedBlockingQueue<RemoteLogRecord> remoteLogsQ;
+ private final INcApplicationContext appCtx;
+
+ public RemoteLogsNotifier(INcApplicationContext appCtx, LinkedBlockingQueue<RemoteLogRecord> remoteLogsQ) {
+ this.appCtx = appCtx;
+ this.remoteLogsQ = remoteLogsQ;
+ localResourceRep = (PersistentLocalResourceRepository) appCtx.getLocalResourceRepository();
+ indexCheckpointManagerProvider = appCtx.getIndexCheckpointManagerProvider();
+ }
+
+ @Override
+ public void run() {
+ final String nodeId = appCtx.getServiceContext().getNodeId();
+ Thread.currentThread().setName(nodeId + RemoteLogsNotifier.class.getSimpleName());
+ while (!Thread.currentThread().isInterrupted()) {
+ try {
+ final RemoteLogRecord logRecord = remoteLogsQ.take();
+ switch (logRecord.getLogType()) {
+ case LogType.JOB_COMMIT:
+ case LogType.ABORT:
+ // send ACK to requester
+ logRecord.getReplicationWorker().getChannel().socket().getOutputStream()
+ .write((nodeId + ReplicationProtocol.LOG_REPLICATION_ACK + logRecord.getTxnId() + System
+ .lineSeparator()).getBytes());
+ break;
+ case LogType.FLUSH:
+ checkpointReplicaIndexes(logRecord, logRecord.getDatasetId());
+ break;
+ default:
+ throw new IllegalStateException("Unexpected log type: " + logRecord.getLogType());
+ }
+ } catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
+ } catch (IOException e) {
+ LOGGER.error("Failed to process replicated log", e);
+ }
+ }
+ }
+
+ private void checkpointReplicaIndexes(RemoteLogRecord remoteLogMapping, int datasetId) throws HyracksDataException {
+ final Set<Integer> masterPartitions = appCtx.getReplicaManager().getPartitions();
+ final Predicate<LocalResource> replicaIndexesPredicate = lr -> {
+ DatasetLocalResource dls = (DatasetLocalResource) lr.getResource();
+ return dls.getDatasetId() == datasetId && !masterPartitions.contains(dls.getPartition());
+ };
+ final Map<Long, LocalResource> resources = localResourceRep.getResources(replicaIndexesPredicate);
+ final List<DatasetResourceReference> replicaIndexesRef =
+ resources.values().stream().map(DatasetResourceReference::of).collect(Collectors.toList());
+ for (DatasetResourceReference replicaIndexRef : replicaIndexesRef) {
+ final IIndexCheckpointManager indexCheckpointManager = indexCheckpointManagerProvider.get(replicaIndexRef);
+ synchronized (indexCheckpointManager) {
+ indexCheckpointManager.masterFlush(remoteLogMapping.getMasterLsn(), remoteLogMapping.getLSN());
+ }
+ }
+ }
+}
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/logging/RemoteLogsProcessor.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/logging/RemoteLogsProcessor.java
new file mode 100644
index 0000000..c054e6c
--- /dev/null
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/logging/RemoteLogsProcessor.java
@@ -0,0 +1,88 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.replication.logging;
+
+import java.nio.ByteBuffer;
+import java.util.concurrent.LinkedBlockingQueue;
+
+import org.apache.asterix.common.api.INcApplicationContext;
+import org.apache.asterix.common.transactions.ILogManager;
+import org.apache.asterix.common.transactions.ILogRecord;
+import org.apache.asterix.common.transactions.ILogRequester;
+import org.apache.asterix.common.transactions.LogSource;
+import org.apache.asterix.common.transactions.LogType;
+import org.apache.asterix.common.utils.TransactionUtil;
+import org.apache.asterix.replication.api.IReplicationWorker;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+public class RemoteLogsProcessor implements ILogRequester {
+
+ private static final Logger LOGGER = LogManager.getLogger();
+ private final LinkedBlockingQueue<RemoteLogRecord> remoteLogsQ;
+ private final ILogManager logManager;
+
+ public RemoteLogsProcessor(INcApplicationContext appCtx) {
+ logManager = appCtx.getTransactionSubsystem().getLogManager();
+ remoteLogsQ = new LinkedBlockingQueue<>();
+ appCtx.getThreadExecutor().execute(new RemoteLogsNotifier(appCtx, remoteLogsQ));
+ }
+
+ public void process(ByteBuffer logsBatch, RemoteLogRecord reusableLog, IReplicationWorker worker) {
+ while (logsBatch.hasRemaining()) {
+ // get rid of log size
+ logsBatch.getInt();
+ reusableLog.readRemoteLog(logsBatch);
+ reusableLog.setLogSource(LogSource.REMOTE);
+ switch (reusableLog.getLogType()) {
+ case LogType.UPDATE:
+ case LogType.ENTITY_COMMIT:
+ logManager.log(reusableLog);
+ break;
+ case LogType.JOB_COMMIT:
+ case LogType.ABORT:
+ RemoteLogRecord jobTerminationLog = new RemoteLogRecord();
+ TransactionUtil.formJobTerminateLogRecord(jobTerminationLog, reusableLog.getTxnId(),
+ reusableLog.getLogType() == LogType.JOB_COMMIT);
+ jobTerminationLog.setRequester(this);
+ jobTerminationLog.setReplicationWorker(worker);
+ jobTerminationLog.setLogSource(LogSource.REMOTE);
+ logManager.log(jobTerminationLog);
+ break;
+ case LogType.FLUSH:
+ RemoteLogRecord flushLog = new RemoteLogRecord();
+ TransactionUtil
+ .formFlushLogRecord(flushLog, reusableLog.getDatasetId(), null, reusableLog.getNodeId(),
+ reusableLog.getNumOfFlushedIndexes());
+ flushLog.setRequester(this);
+ flushLog.setLogSource(LogSource.REMOTE);
+ flushLog.setMasterLsn(reusableLog.getLSN());
+ logManager.log(flushLog);
+ break;
+ default:
+ LOGGER.error(() -> "Unsupported LogType: " + reusableLog.getLogType());
+ }
+ }
+ }
+
+ @Override
+ public void notifyFlushed(ILogRecord logRecord) {
+ remoteLogsQ.add((RemoteLogRecord) logRecord);
+ }
+}
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/logging/ReplicationLogBuffer.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/logging/ReplicationLogBuffer.java
index 283f69f..a94f073 100644
--- a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/logging/ReplicationLogBuffer.java
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/logging/ReplicationLogBuffer.java
@@ -22,9 +22,10 @@
import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.asterix.common.transactions.ILogRecord;
-import org.apache.asterix.replication.management.ReplicationManager;
+import org.apache.asterix.replication.management.LogReplicationManager;
public class ReplicationLogBuffer {
+
private final int logBufferSize;
private final AtomicBoolean full;
private int appendOffset;
@@ -32,10 +33,10 @@
private final ByteBuffer appendBuffer;
private final ByteBuffer replicationBuffer;
private boolean stop;
- private ReplicationManager replicationManager;
+ private final LogReplicationManager replicationManager;
private final int batchSize;
- public ReplicationLogBuffer(ReplicationManager replicationManager, int logBufferSize, int batchSize) {
+ public ReplicationLogBuffer(LogReplicationManager replicationManager, int logBufferSize, int batchSize) {
this.replicationManager = replicationManager;
this.logBufferSize = logBufferSize;
this.batchSize = batchSize;
@@ -116,7 +117,7 @@
private void transferBuffer(ByteBuffer buffer) {
if (buffer.remaining() <= batchSize) {
//the current batch can be sent as it is
- replicationManager.replicateTxnLogBatch(buffer);
+ replicationManager.transferBatch(buffer);
return;
}
/**
@@ -141,7 +142,7 @@
//return to the beginning of the batch position
buffer.reset();
}
- replicationManager.replicateTxnLogBatch(buffer);
+ replicationManager.transferBatch(buffer);
//return the original limit to check the new remaining size
buffer.limit(totalTransferLimit);
}
@@ -159,7 +160,7 @@
return logBufferSize;
}
- public ReplicationManager getReplicationManager() {
+ public LogReplicationManager getReplicationManager() {
return replicationManager;
}
}
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/logging/TxnAck.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/logging/TxnAck.java
new file mode 100644
index 0000000..1b0ac23
--- /dev/null
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/logging/TxnAck.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.replication.logging;
+
+import java.util.Set;
+
+import org.apache.asterix.common.replication.IReplicationDestination;
+import org.apache.asterix.common.transactions.ILogRecord;
+
+public class TxnAck {
+
+ private final Set<IReplicationDestination> pendingAcks;
+ private final ILogRecord logRecord;
+
+ public TxnAck(ILogRecord logRecord, Set<IReplicationDestination> pendingAcks) {
+ this.logRecord = logRecord;
+ this.pendingAcks = pendingAcks;
+ }
+
+ public synchronized void ack(IReplicationDestination dest) {
+ pendingAcks.remove(dest);
+ if (allAcked()) {
+ synchronized (logRecord) {
+ logRecord.setReplicated(true);
+ logRecord.notifyAll();
+ }
+ }
+ }
+
+ public synchronized boolean allAcked() {
+ return pendingAcks.isEmpty();
+ }
+}
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/logging/TxnAckTracker.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/logging/TxnAckTracker.java
new file mode 100644
index 0000000..23e6f3c
--- /dev/null
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/logging/TxnAckTracker.java
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.replication.logging;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.asterix.common.replication.IReplicationDestination;
+import org.apache.asterix.common.transactions.ILogRecord;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+public class TxnAckTracker {
+
+ private static final Logger LOGGER = LogManager.getLogger();
+ private final Map<Long, TxnAck> txnsAcks = new HashMap<>();
+
+ public synchronized void track(ILogRecord logRecord, Set<IReplicationDestination> replicas) {
+ if (replicas.isEmpty()) {
+ logRecord.setReplicated(true);
+ return;
+ }
+ final long txnId = logRecord.getTxnId();
+ //TODO use LSN instead of txnId when all logs have LSN
+ txnsAcks.put(txnId, new TxnAck(logRecord, replicas));
+ }
+
+ public synchronized void ack(long txnId, IReplicationDestination replica) {
+ if (!txnsAcks.containsKey(txnId)) {
+ LOGGER.warn("Received ack for unknown txn {}", txnId);
+ return;
+ }
+ TxnAck txnAcks = txnsAcks.get(txnId);
+ txnAcks.ack(replica);
+ if (txnAcks.allAcked()) {
+ txnsAcks.remove(txnId);
+ }
+ }
+
+ public synchronized void unregister(IReplicationDestination replica) {
+ // assume the ack was received from leaving replicas
+ final HashSet<Long> pendingTxn = new HashSet<>(txnsAcks.keySet());
+ pendingTxn.forEach(txnId -> ack(txnId, replica));
+ }
+}
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/logging/TxnLogUtil.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/logging/TxnLogUtil.java
deleted file mode 100644
index f51a64d..0000000
--- a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/logging/TxnLogUtil.java
+++ /dev/null
@@ -1,35 +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.logging;
-
-public class TxnLogUtil {
-
- private TxnLogUtil() {
- //prevent util class construction
- }
-
- /**
- * @param nodeId
- * @param LSN
- * @return Concatenation of nodeId and LSN
- */
- public static String getNodeUniqueLSN(String nodeId, long LSN) {
- return nodeId + LSN;
- }
-}
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/management/IndexReplicationManager.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/management/IndexReplicationManager.java
new file mode 100644
index 0000000..b6f752f
--- /dev/null
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/management/IndexReplicationManager.java
@@ -0,0 +1,184 @@
+/*
+ * 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.HashSet;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.LinkedBlockingQueue;
+
+import org.apache.asterix.common.api.INcApplicationContext;
+import org.apache.asterix.common.exceptions.ReplicationException;
+import org.apache.asterix.common.replication.IPartitionReplica;
+import org.apache.asterix.common.replication.IReplicationDestination;
+import org.apache.asterix.common.replication.IReplicationManager;
+import org.apache.asterix.common.replication.IReplicationStrategy;
+import org.apache.asterix.common.storage.DatasetResourceReference;
+import org.apache.asterix.common.storage.ResourceReference;
+import org.apache.asterix.replication.api.PartitionReplica;
+import org.apache.asterix.replication.api.ReplicationDestination;
+import org.apache.asterix.replication.sync.IndexSynchronizer;
+import org.apache.asterix.transaction.management.resource.PersistentLocalResourceRepository;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.replication.IReplicationJob;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexReplicationJob;
+import org.apache.logging.log4j.Level;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+public class IndexReplicationManager {
+
+ private static final Logger LOGGER = LogManager.getLogger();
+ private final IReplicationManager replicationManager;
+ private final Set<ReplicationDestination> destinations = new HashSet<>();
+ private final LinkedBlockingQueue<IReplicationJob> replicationJobsQ = new LinkedBlockingQueue<>();
+ private final IReplicationStrategy replicationStrategy;
+ private final PersistentLocalResourceRepository resourceRepository;
+ private final INcApplicationContext appCtx;
+ private final Object transferLock = new Object();
+ private final Set<ReplicationDestination> failedDest = new HashSet<>();
+
+ public IndexReplicationManager(INcApplicationContext appCtx, IReplicationManager replicationManager) {
+ this.appCtx = appCtx;
+ this.replicationManager = replicationManager;
+ this.resourceRepository = (PersistentLocalResourceRepository) appCtx.getLocalResourceRepository();
+ replicationStrategy = replicationManager.getReplicationStrategy();
+ appCtx.getThreadExecutor().execute(new ReplicationJobsProcessor());
+ }
+
+ public void register(ReplicationDestination dest) {
+ synchronized (transferLock) {
+ LOGGER.info(() -> "register " + dest);
+ destinations.add(dest);
+ failedDest.remove(dest);
+ }
+ }
+
+ public void unregister(IReplicationDestination dest) {
+ synchronized (transferLock) {
+ LOGGER.info(() -> "unregister " + dest);
+ destinations.remove(dest);
+ failedDest.remove(dest);
+ }
+ }
+
+ private void handleFailure(ReplicationDestination dest, Exception e) {
+ synchronized (transferLock) {
+ if (failedDest.contains(dest)) {
+ return;
+ }
+ LOGGER.error("Replica failed", e);
+ if (destinations.contains(dest)) {
+ failedDest.add(dest);
+ }
+ replicationManager.notifyFailure(dest, e);
+ }
+ }
+
+ public void accept(IReplicationJob job) {
+ if (job.getExecutionType() == IReplicationJob.ReplicationExecutionType.ASYNC) {
+ replicationJobsQ.add(job);
+ return;
+ }
+ process(job);
+ }
+
+ private void process(IReplicationJob job) {
+ try {
+ if (skip(job)) {
+ return;
+ }
+ synchronized (transferLock) {
+ if (destinations.isEmpty()) {
+ return;
+ }
+ final IndexSynchronizer synchronizer = new IndexSynchronizer(job, appCtx);
+ final int indexPartition = getJobPartition(job);
+ for (ReplicationDestination dest : destinations) {
+ try {
+ Optional<IPartitionReplica> partitionReplica = dest.getPartitionReplica(indexPartition);
+ if (!partitionReplica.isPresent()) {
+ continue;
+ }
+ PartitionReplica replica = (PartitionReplica) partitionReplica.get();
+ synchronizer.sync(replica);
+ } catch (Exception e) {
+ handleFailure(dest, e);
+ }
+ }
+ closeChannels();
+ }
+ } finally {
+ afterReplication(job);
+ }
+ }
+
+ private boolean skip(IReplicationJob job) {
+ try {
+ final DatasetResourceReference indexFileRef =
+ resourceRepository.getLocalResourceReference(job.getAnyFile());
+ return !replicationStrategy.isMatch(indexFileRef.getDatasetId());
+ } catch (HyracksDataException e) {
+ throw new IllegalStateException("Couldn't find resource for " + job.getAnyFile(), e);
+ }
+ }
+
+ private int getJobPartition(IReplicationJob job) {
+ return ResourceReference.of(job.getAnyFile()).getPartitionNum();
+ }
+
+ private void closeChannels() {
+ if (!replicationJobsQ.isEmpty()) {
+ return;
+ }
+ LOGGER.log(Level.INFO, "No pending replication jobs. Closing connections to replicas");
+ for (ReplicationDestination dest : destinations) {
+ dest.getReplicas().stream().map(PartitionReplica.class::cast).forEach(PartitionReplica::close);
+ }
+ }
+
+ private static void afterReplication(IReplicationJob job) {
+ try {
+ if (job.getOperation() == IReplicationJob.ReplicationOperation.REPLICATE
+ && job instanceof ILSMIndexReplicationJob) {
+ ((ILSMIndexReplicationJob) job).endReplication();
+ }
+ } catch (HyracksDataException e) {
+ throw new ReplicationException(e);
+ }
+ }
+
+ private class ReplicationJobsProcessor implements Runnable {
+
+ @Override
+ public void run() {
+ Thread.currentThread().setName(ReplicationJobsProcessor.class.getSimpleName());
+ while (!Thread.currentThread().isInterrupted()) {
+ try {
+ final IReplicationJob job = replicationJobsQ.take();
+ process(job);
+ } catch (InterruptedException e) {
+ LOGGER.warn(() -> ReplicationJobsProcessor.class.getSimpleName() + " interrupted.", e);
+ Thread.currentThread().interrupt();
+ }
+ }
+ LOGGER.warn("{} stopped.", ReplicationJobsProcessor.class.getSimpleName());
+ }
+ }
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/management/LogReplicationManager.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/management/LogReplicationManager.java
new file mode 100644
index 0000000..b28c2f7
--- /dev/null
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/management/LogReplicationManager.java
@@ -0,0 +1,258 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.replication.management;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.nio.ByteBuffer;
+import java.nio.channels.AsynchronousCloseException;
+import java.nio.channels.SocketChannel;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.Executor;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.LinkedBlockingQueue;
+
+import org.apache.asterix.common.api.INcApplicationContext;
+import org.apache.asterix.common.config.ReplicationProperties;
+import org.apache.asterix.common.exceptions.ReplicationException;
+import org.apache.asterix.common.replication.IReplicationDestination;
+import org.apache.asterix.common.replication.IReplicationManager;
+import org.apache.asterix.common.transactions.ILogRecord;
+import org.apache.asterix.common.transactions.LogType;
+import org.apache.asterix.replication.api.ReplicationDestination;
+import org.apache.asterix.replication.logging.ReplicationLogBuffer;
+import org.apache.asterix.replication.logging.TxnAckTracker;
+import org.apache.asterix.replication.logging.TxnLogReplicator;
+import org.apache.asterix.replication.messaging.ReplicateLogsTask;
+import org.apache.asterix.replication.messaging.ReplicationProtocol;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+public class LogReplicationManager {
+
+ private static final Logger LOGGER = LogManager.getLogger();
+ private final LinkedBlockingQueue<ReplicationLogBuffer> emptyLogBuffersQ;
+ private final LinkedBlockingQueue<ReplicationLogBuffer> pendingFlushLogBuffersQ;
+ private final ByteBuffer txnLogsBatchSizeBuffer = ByteBuffer.allocate(Integer.BYTES);
+ private final Map<ReplicationDestination, SocketChannel> destinations = new HashMap<>();
+ private final IReplicationManager replicationManager;
+ private final Executor executor;
+ private final TxnAckTracker ackTracker = new TxnAckTracker();
+ private final Set<SocketChannel> failedSockets = new HashSet<>();
+ private final Object transferLock = new Object();
+ private final INcApplicationContext appCtx;
+ private final int logPageSize;
+ private final int logBatchSize;
+ private ReplicationLogBuffer currentTxnLogBuffer;
+ private SocketChannel[] destSockets;
+
+ public LogReplicationManager(INcApplicationContext appCtx, IReplicationManager replicationManager) {
+ this.appCtx = appCtx;
+ this.replicationManager = replicationManager;
+ final ReplicationProperties replicationProperties = appCtx.getReplicationProperties();
+ logPageSize = replicationProperties.getLogBufferPageSize();
+ logBatchSize = replicationProperties.getLogBatchSize();
+ executor = appCtx.getThreadExecutor();
+ emptyLogBuffersQ = new LinkedBlockingQueue<>();
+ pendingFlushLogBuffersQ = new LinkedBlockingQueue<>();
+ initBuffers(replicationProperties.getLogBufferNumOfPages());
+ TxnLogReplicator txnlogReplicator = new TxnLogReplicator(emptyLogBuffersQ, pendingFlushLogBuffersQ);
+ ((ExecutorService) executor).submit(txnlogReplicator);
+ }
+
+ private void initBuffers(int buffers) {
+ for (int i = 0; i < buffers; i++) {
+ emptyLogBuffersQ.add(new ReplicationLogBuffer(this, logPageSize, logBatchSize));
+ }
+ try {
+ getAndInitNewPage();
+ } catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
+ throw new ReplicationException(e);
+ }
+ }
+
+ public void register(ReplicationDestination dest) {
+ synchronized (transferLock) {
+ synchronized (destinations) {
+ if (destinations.containsKey(dest)) {
+ return;
+ }
+ LOGGER.info(() -> "register " + dest);
+ SocketChannel socketChannel = dest.getLogReplicationChannel();
+ handshake(dest, socketChannel);
+ destinations.put(dest, socketChannel);
+ failedSockets.remove(socketChannel);
+ destSockets = destinations.values().toArray(new SocketChannel[destinations.size()]);
+ }
+ }
+ }
+
+ public void unregister(IReplicationDestination dest) {
+ synchronized (transferLock) {
+ synchronized (destinations) {
+ if (!destinations.containsKey(dest)) {
+ return;
+ }
+ LOGGER.info(() -> "unregister " + dest);
+ ackTracker.unregister(dest);
+ SocketChannel destSocket = destinations.remove(dest);
+ failedSockets.remove(destSocket);
+ destSockets = destinations.values().toArray(new SocketChannel[destinations.size()]);
+ endReplication(destSocket);
+ }
+ }
+ }
+
+ public void replicate(ILogRecord logRecord) throws InterruptedException {
+ if (logRecord.getLogType() == LogType.JOB_COMMIT || logRecord.getLogType() == LogType.ABORT) {
+ synchronized (destinations) {
+ ackTracker.track(logRecord, new HashSet<>(destinations.keySet()));
+ }
+ }
+ appendToLogBuffer(logRecord);
+ }
+
+ public void transferBatch(final ByteBuffer buffer) {
+ // prepare the batch size buffer
+ txnLogsBatchSizeBuffer.clear();
+ txnLogsBatchSizeBuffer.putInt(buffer.remaining());
+ txnLogsBatchSizeBuffer.flip();
+
+ buffer.mark();
+ synchronized (transferLock) {
+ if (destSockets != null) {
+ for (SocketChannel replicaSocket : destSockets) {
+ try {
+ // send batch size then the batch itself
+ NetworkingUtil.transferBufferToChannel(replicaSocket, txnLogsBatchSizeBuffer);
+ NetworkingUtil.transferBufferToChannel(replicaSocket, buffer);
+ } catch (IOException e) {
+ handleFailure(replicaSocket, e);
+ } finally {
+ txnLogsBatchSizeBuffer.position(0);
+ buffer.reset();
+ }
+ }
+ }
+ }
+ // move the buffer position to the sent limit
+ buffer.position(buffer.limit());
+ }
+
+ public int getLogPageSize() {
+ return logPageSize;
+ }
+
+ private synchronized void appendToLogBuffer(ILogRecord logRecord) throws InterruptedException {
+ if (!currentTxnLogBuffer.hasSpace(logRecord)) {
+ currentTxnLogBuffer.isFull(true);
+ if (logRecord.getLogSize() > logPageSize) {
+ getAndInitNewLargePage(logRecord.getLogSize());
+ } else {
+ getAndInitNewPage();
+ }
+ }
+ currentTxnLogBuffer.append(logRecord);
+ }
+
+ private void getAndInitNewPage() throws InterruptedException {
+ currentTxnLogBuffer = null;
+ while (currentTxnLogBuffer == null) {
+ currentTxnLogBuffer = emptyLogBuffersQ.take();
+ }
+ currentTxnLogBuffer.reset();
+ pendingFlushLogBuffersQ.add(currentTxnLogBuffer);
+ }
+
+ private void getAndInitNewLargePage(int pageSize) {
+ // for now, alloc a new buffer for each large page
+ currentTxnLogBuffer = new ReplicationLogBuffer(this, pageSize, logBatchSize);
+ pendingFlushLogBuffersQ.add(currentTxnLogBuffer);
+ }
+
+ private void handshake(ReplicationDestination dest, SocketChannel socketChannel) {
+ final String nodeId = appCtx.getServiceContext().getNodeId();
+ final ReplicateLogsTask task = new ReplicateLogsTask(nodeId);
+ ReplicationProtocol.sendTo(socketChannel, task, null);
+ executor.execute(new TxnAckListener(dest, socketChannel));
+ }
+
+ private void endReplication(SocketChannel socketChannel) {
+ if (socketChannel.isConnected()) {
+ // end log replication (by sending a dummy log with a single byte)
+ final ByteBuffer endLogRepBuffer = ReplicationProtocol.getEndLogReplicationBuffer();
+ try {
+ NetworkingUtil.transferBufferToChannel(socketChannel, endLogRepBuffer);
+ } catch (IOException e) {
+ LOGGER.warn("Failed to end txn log", e);
+ }
+ }
+ }
+
+ private synchronized void handleFailure(SocketChannel replicaSocket, IOException e) {
+ if (failedSockets.contains(replicaSocket)) {
+ return;
+ }
+ LOGGER.error("Replica failed", e);
+ failedSockets.add(replicaSocket);
+ Optional<ReplicationDestination> socketDest =
+ destinations.entrySet().stream().filter(entry -> entry.getValue().equals(replicaSocket))
+ .map(Map.Entry::getKey).findFirst();
+ socketDest.ifPresent(dest -> replicationManager.notifyFailure(dest, e));
+ }
+
+ private class TxnAckListener implements Runnable {
+ private final ReplicationDestination dest;
+ private final SocketChannel replicaSocket;
+
+ TxnAckListener(ReplicationDestination dest, SocketChannel replicaSocket) {
+ this.dest = dest;
+ this.replicaSocket = replicaSocket;
+ }
+
+ @Override
+ public void run() {
+ Thread.currentThread().setName("TxnAckListener (" + dest + ")");
+ LOGGER.info("Started listening on socket: {}", dest);
+ try (BufferedReader incomingResponse = new BufferedReader(
+ new InputStreamReader(replicaSocket.socket().getInputStream()))) {
+ while (true) {
+ final String response = incomingResponse.readLine();
+ if (response == null) {
+ handleFailure(replicaSocket, new IOException("Unexpected response from replica " + dest));
+ break;
+ }
+ // read ACK
+ final int txnId = ReplicationProtocol.getTxnIdFromLogAckMessage(response);
+ ackTracker.ack(txnId, dest);
+ }
+ } catch (AsynchronousCloseException e) {
+ LOGGER.debug(() -> "Stopped listening on socket:" + dest, e);
+ } catch (IOException e) {
+ handleFailure(replicaSocket, e);
+ }
+ }
+ }
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/management/ReplicaStateChecker.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/management/ReplicaStateChecker.java
deleted file mode 100644
index e260de5..0000000
--- a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/management/ReplicaStateChecker.java
+++ /dev/null
@@ -1,76 +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.concurrent.Callable;
-
-import org.apache.asterix.common.replication.Replica;
-import org.apache.asterix.common.replication.Replica.ReplicaState;
-import org.apache.asterix.replication.functions.ReplicationProtocol;
-
-public class ReplicaStateChecker implements Callable<Void> {
-
- private final int WAIT_TIME = 2000;
- private final Replica replica;
- private final int replicationTimeOut;
- private final ReplicationManager replicationManager;
- private final boolean suspendReplication;
-
- public ReplicaStateChecker(Replica replica, int replicationTimeOut, ReplicationManager replicationManager,
- boolean suspendReplication) {
- this.replica = replica;
- this.replicationTimeOut = replicationTimeOut;
- this.replicationManager = replicationManager;
- this.suspendReplication = suspendReplication;
- }
-
- @Override
- public Void call() throws Exception {
- Thread.currentThread().setName("ReplicaConnector Thread");
-
- long startTime = System.currentTimeMillis();
- InetSocketAddress replicaAddress = InetSocketAddress.createUnresolved(replica.getClusterIp(),
- replica.getPort());
-
- while (true) {
- try (SocketChannel connection = SocketChannel.open()) {
- connection.configureBlocking(true);
- connection.connect(new InetSocketAddress(replicaAddress.getHostString(), replicaAddress.getPort()));
- ByteBuffer buffer = ReplicationProtocol.getGoodbyeBuffer();
- connection.write(buffer);
- replicationManager.updateReplicaState(replica.getId(), ReplicaState.ACTIVE, suspendReplication);
- return null;
- } catch (IOException | UnresolvedAddressException e) {
- Thread.sleep(WAIT_TIME);
-
- //check if connection to replica timed out
- if (((System.currentTimeMillis() - startTime) / 1000) >= replicationTimeOut) {
- replicationManager.updateReplicaState(replica.getId(), ReplicaState.DEAD, suspendReplication);
- return null;
- }
- }
- }
- }
-
-}
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/management/ReplicationChannel.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/management/ReplicationChannel.java
index c0863e2..f19f1cd 100644
--- a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/management/ReplicationChannel.java
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/management/ReplicationChannel.java
@@ -18,79 +18,23 @@
*/
package org.apache.asterix.replication.management;
-import java.io.File;
import java.io.IOException;
-import java.io.RandomAccessFile;
import java.net.InetAddress;
import java.net.InetSocketAddress;
import java.nio.ByteBuffer;
import java.nio.channels.AsynchronousCloseException;
-import java.nio.channels.FileChannel;
import java.nio.channels.ServerSocketChannel;
import java.nio.channels.SocketChannel;
-import java.util.ArrayList;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.function.Predicate;
-import java.util.stream.Collectors;
-import org.apache.asterix.common.api.IDatasetLifecycleManager;
import org.apache.asterix.common.api.INcApplicationContext;
-import org.apache.asterix.common.cluster.ClusterPartition;
import org.apache.asterix.common.config.ReplicationProperties;
-import org.apache.asterix.common.context.IndexInfo;
-import org.apache.asterix.common.dataflow.DatasetLocalResource;
-import org.apache.asterix.common.exceptions.ACIDException;
-import org.apache.asterix.common.ioopcallbacks.AbstractLSMIOOperationCallback;
-import org.apache.asterix.common.replication.IReplicaResourcesManager;
import org.apache.asterix.common.replication.IReplicationChannel;
-import org.apache.asterix.common.replication.IReplicationManager;
-import org.apache.asterix.common.replication.IReplicationStrategy;
-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.storage.DatasetResourceReference;
-import org.apache.asterix.common.storage.IIndexCheckpointManager;
-import org.apache.asterix.common.storage.IIndexCheckpointManagerProvider;
-import org.apache.asterix.common.storage.ResourceReference;
-import org.apache.asterix.common.transactions.IAppRuntimeContextProvider;
-import org.apache.asterix.common.transactions.ILogManager;
-import org.apache.asterix.common.transactions.LogRecord;
-import org.apache.asterix.common.transactions.LogSource;
-import org.apache.asterix.common.transactions.LogType;
-import org.apache.asterix.common.utils.StoragePathUtil;
-import org.apache.asterix.common.utils.TransactionUtil;
-import org.apache.asterix.replication.functions.ReplicaFilesRequest;
-import org.apache.asterix.replication.functions.ReplicaIndexFlushRequest;
-import org.apache.asterix.replication.functions.ReplicationProtocol;
-import org.apache.asterix.replication.functions.ReplicationProtocol.ReplicationRequestType;
-import org.apache.asterix.replication.logging.RemoteLogMapping;
-import org.apache.asterix.replication.messaging.CheckpointPartitionIndexesTask;
-import org.apache.asterix.replication.messaging.DeleteFileTask;
-import org.apache.asterix.replication.messaging.PartitionResourcesListTask;
-import org.apache.asterix.replication.messaging.ReplicateFileTask;
-import org.apache.asterix.replication.storage.LSMComponentLSNSyncTask;
-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.asterix.transaction.management.service.logging.LogBuffer;
-import org.apache.hyracks.api.application.INCServiceContext;
-import org.apache.hyracks.control.common.controllers.NCConfig;
-import org.apache.hyracks.control.nc.NodeControllerService;
+import org.apache.asterix.replication.api.IReplicaTask;
+import org.apache.asterix.replication.api.IReplicationWorker;
+import org.apache.asterix.replication.logging.RemoteLogsProcessor;
+import org.apache.asterix.replication.messaging.ReplicationProtocol;
+import org.apache.asterix.replication.messaging.ReplicationProtocol.ReplicationRequestType;
import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.storage.am.lsm.common.api.LSMOperationType;
-import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndex;
-import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndexFileManager;
-import org.apache.hyracks.storage.common.LocalResource;
-import org.apache.hyracks.util.StorageUtil;
-import org.apache.hyracks.util.StorageUtil.StorageUnit;
import org.apache.logging.log4j.Level;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
@@ -101,200 +45,74 @@
public class ReplicationChannel extends Thread implements IReplicationChannel {
private static final Logger LOGGER = LogManager.getLogger();
- private static final int LOG_REPLICATION_END_HANKSHAKE_LOG_SIZE = 1;
- private final ExecutorService replicationThreads;
- private final String localNodeID;
- private final ILogManager logManager;
- private final ReplicaResourcesManager replicaResourcesManager;
private ServerSocketChannel serverSocketChannel = null;
- private final IReplicationManager replicationManager;
- private final ReplicationProperties replicationProperties;
- private final IAppRuntimeContextProvider appContextProvider;
- private static final int INTIAL_BUFFER_SIZE = StorageUtil.getIntSizeInBytes(4, StorageUnit.KILOBYTE);
- private final LinkedBlockingQueue<LSMComponentLSNSyncTask> lsmComponentRemoteLSN2LocalLSNMappingTaskQ;
- private final LinkedBlockingQueue<LogRecord> pendingNotificationRemoteLogsQ;
- private final Map<String, LSMComponentProperties> lsmComponentId2PropertiesMap;
- private final Map<Long, RemoteLogMapping> localLsn2RemoteMapping;
- private final Map<String, RemoteLogMapping> replicaUniqueLSN2RemoteMapping;
- private final LSMComponentsSyncService lsmComponentLSNMappingService;
- private final ReplicationNotifier replicationNotifier;
- private final Object flushLogslock = new Object();
- private final IDatasetLifecycleManager dsLifecycleManager;
- private final PersistentLocalResourceRepository localResourceRep;
- private final IReplicationStrategy replicationStrategy;
- private final NCConfig ncConfig;
- private Set nodeHostedPartitions;
- private final IIndexCheckpointManagerProvider indexCheckpointManagerProvider;
private final INcApplicationContext appCtx;
+ private final RemoteLogsProcessor logsProcessor;
- public ReplicationChannel(String nodeId, ReplicationProperties replicationProperties, ILogManager logManager,
- IReplicaResourcesManager replicaResoucesManager, IReplicationManager replicationManager,
- INCServiceContext ncServiceContext, IAppRuntimeContextProvider asterixAppRuntimeContextProvider,
- IReplicationStrategy replicationStrategy) {
- this.logManager = logManager;
- this.localNodeID = nodeId;
- this.replicaResourcesManager = (ReplicaResourcesManager) replicaResoucesManager;
- this.replicationManager = replicationManager;
- this.replicationProperties = replicationProperties;
- this.appContextProvider = asterixAppRuntimeContextProvider;
- this.dsLifecycleManager = asterixAppRuntimeContextProvider.getDatasetLifecycleManager();
- this.localResourceRep = (PersistentLocalResourceRepository) asterixAppRuntimeContextProvider
- .getLocalResourceRepository();
- this.replicationStrategy = replicationStrategy;
- this.ncConfig = ((NodeControllerService) ncServiceContext.getControllerService()).getConfiguration();
- lsmComponentRemoteLSN2LocalLSNMappingTaskQ = new LinkedBlockingQueue<>();
- pendingNotificationRemoteLogsQ = new LinkedBlockingQueue<>();
- lsmComponentId2PropertiesMap = new ConcurrentHashMap<>();
- replicaUniqueLSN2RemoteMapping = new ConcurrentHashMap<>();
- localLsn2RemoteMapping = new ConcurrentHashMap<>();
- lsmComponentLSNMappingService = new LSMComponentsSyncService();
- replicationNotifier = new ReplicationNotifier();
- replicationThreads = Executors.newCachedThreadPool(ncServiceContext.getThreadFactory());
- Map<String, ClusterPartition[]> nodePartitions = asterixAppRuntimeContextProvider.getAppContext()
- .getMetadataProperties().getNodePartitions();
- Set<String> nodeReplicationClients = replicationStrategy.getRemotePrimaryReplicas(nodeId).stream()
- .map(Replica::getId).collect(Collectors.toSet());
- 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);
- this.indexCheckpointManagerProvider =
- ((INcApplicationContext) ncServiceContext.getApplicationContext()).getIndexCheckpointManagerProvider();
- this.appCtx = (INcApplicationContext) ncServiceContext.getApplicationContext();
+ public ReplicationChannel(INcApplicationContext appCtx) {
+ this.appCtx = appCtx;
+ logsProcessor = new RemoteLogsProcessor(appCtx);
}
@Override
public void run() {
- Thread.currentThread().setName("Replication Channel Thread");
-
- String nodeIP = replicationProperties.getNodeIpFromId(localNodeID);
- int dataPort = ncConfig.getReplicationPublicPort();
+ final String nodeId = appCtx.getServiceContext().getNodeId();
+ Thread.currentThread().setName(nodeId + " Replication Channel Thread");
+ final ReplicationProperties replicationProperties = appCtx.getReplicationProperties();
+ final String nodeIP = replicationProperties.getReplicationAddress();
+ final int dataPort = replicationProperties.getReplicationPort();
try {
serverSocketChannel = ServerSocketChannel.open();
serverSocketChannel.configureBlocking(true);
- InetSocketAddress replicationChannelAddress = new InetSocketAddress(InetAddress.getByName(nodeIP),
- dataPort);
+ InetSocketAddress replicationChannelAddress =
+ new InetSocketAddress(InetAddress.getByName(nodeIP), dataPort);
serverSocketChannel.socket().bind(replicationChannelAddress);
- lsmComponentLSNMappingService.start();
- replicationNotifier.start();
LOGGER.log(Level.INFO, "opened Replication Channel @ IP Address: " + nodeIP + ":" + dataPort);
-
- //start accepting replication requests
while (serverSocketChannel.isOpen()) {
SocketChannel socketChannel = serverSocketChannel.accept();
socketChannel.configureBlocking(true);
//start a new thread to handle the request
- replicationThreads.execute(new ReplicationThread(socketChannel));
+ appCtx.getThreadExecutor().execute(new ReplicationWorker(socketChannel));
}
} catch (AsynchronousCloseException e) {
- LOGGER.warn("Replication channel closed", e);
+ LOGGER.debug("Replication channel closed", e);
} catch (IOException e) {
- throw new IllegalStateException(
- "Could not open replication channel @ IP Address: " + nodeIP + ":" + dataPort, e);
+ throw new IllegalStateException("Failed to bind replication channel @ " + nodeIP + ":" + dataPort, e);
}
}
- private void updateLSMComponentRemainingFiles(String lsmComponentId) throws IOException {
- LSMComponentProperties lsmCompProp = lsmComponentId2PropertiesMap.get(lsmComponentId);
- int remainingFile = lsmCompProp.markFileComplete();
-
- //clean up when all the LSM component files have been received.
- if (remainingFile == 0) {
- if (lsmCompProp.getOpType() == LSMOperationType.FLUSH && lsmCompProp.getReplicaLSN() != null
- && replicaUniqueLSN2RemoteMapping.containsKey(lsmCompProp.getNodeUniqueLSN())) {
- int remainingIndexes = replicaUniqueLSN2RemoteMapping
- .get(lsmCompProp.getNodeUniqueLSN()).numOfFlushedIndexes.decrementAndGet();
- if (remainingIndexes == 0) {
- /**
- * Note: there is a chance that this will never be removed because some
- * index in the dataset was not flushed because it is empty. This could
- * be solved by passing only the number of successfully flushed indexes.
- */
- replicaUniqueLSN2RemoteMapping.remove(lsmCompProp.getNodeUniqueLSN());
- }
- }
-
- //delete mask to indicate that this component is now valid.
- replicaResourcesManager.markLSMComponentReplicaAsValid(lsmCompProp);
- lsmComponentId2PropertiesMap.remove(lsmComponentId);
- LOGGER.log(Level.INFO, "Completed LSMComponent " + lsmComponentId + " Replication.");
- }
+ public RemoteLogsProcessor getRemoteLogsProcessor() {
+ return logsProcessor;
}
@Override
public void close() throws IOException {
- serverSocketChannel.close();
- LOGGER.log(Level.INFO, "Replication channel closed.");
+ if (serverSocketChannel != null) {
+ serverSocketChannel.close();
+ LOGGER.info("Replication channel closed.");
+ }
}
- /**
- * A replication thread is created per received replication request.
- */
- private class ReplicationThread implements IReplicationThread {
+ private class ReplicationWorker implements IReplicationWorker {
private final SocketChannel socketChannel;
- private final LogRecord remoteLog;
- private ByteBuffer inBuffer;
- private ByteBuffer outBuffer;
+ private final ByteBuffer inBuffer;
+ private final ByteBuffer outBuffer;
- public ReplicationThread(SocketChannel socketChannel) {
+ public ReplicationWorker(SocketChannel socketChannel) {
this.socketChannel = socketChannel;
- inBuffer = ByteBuffer.allocate(INTIAL_BUFFER_SIZE);
- outBuffer = ByteBuffer.allocate(INTIAL_BUFFER_SIZE);
- remoteLog = new LogRecord();
+ inBuffer = ByteBuffer.allocate(ReplicationProtocol.INITIAL_BUFFER_SIZE);
+ outBuffer = ByteBuffer.allocate(ReplicationProtocol.INITIAL_BUFFER_SIZE);
}
@Override
public void run() {
- Thread.currentThread().setName("Replication Thread");
+ final String oldName = Thread.currentThread().getName();
+ Thread.currentThread().setName("Replication WorkerÙ‹");
try {
- ReplicationRequestType replicationFunction = ReplicationProtocol.getRequestType(socketChannel,
- inBuffer);
- while (replicationFunction != ReplicationRequestType.GOODBYE) {
- switch (replicationFunction) {
- case REPLICATE_LOG:
- handleLogReplication();
- break;
- case LSM_COMPONENT_PROPERTIES:
- handleLSMComponentProperties();
- break;
- case REPLICATE_FILE:
- handleReplicateFile();
- break;
- case DELETE_FILE:
- handleDeleteFile();
- break;
- case REPLICA_EVENT:
- handleReplicaEvent();
- break;
- case GET_REPLICA_MAX_LSN:
- handleGetReplicaMaxLSN();
- break;
- case GET_REPLICA_FILES:
- handleGetReplicaFiles();
- break;
- case FLUSH_INDEX:
- handleFlushIndex();
- break;
- case PARTITION_RESOURCES_REQUEST:
- handleGetPartitionResources();
- break;
- case REPLICATE_RESOURCE_FILE:
- handleReplicateResourceFile();
- break;
- case DELETE_RESOURCE_FILE:
- handleDeleteResourceFile();
- break;
- case CHECKPOINT_PARTITION:
- handleCheckpointPartition();
- break;
- default:
- throw new IllegalStateException("Unknown replication request");
- }
- replicationFunction = ReplicationProtocol.getRequestType(socketChannel, inBuffer);
+ ReplicationRequestType requestType = ReplicationProtocol.getRequestType(socketChannel, inBuffer);
+ while (requestType != ReplicationRequestType.GOODBYE) {
+ handle(requestType);
+ requestType = ReplicationProtocol.getRequestType(socketChannel, inBuffer);
}
} catch (Exception e) {
LOGGER.warn("Unexpectedly error during replication.", e);
@@ -303,261 +121,10 @@
try {
socketChannel.close();
} catch (IOException e) {
- LOGGER.warn("Filed to close replication socket.", e);
+ LOGGER.warn("Failed to close replication socket.", e);
}
}
- }
- }
-
- private void handleFlushIndex() throws IOException {
- inBuffer = ReplicationProtocol.readRequest(socketChannel, inBuffer);
- //read which indexes are requested to be flushed from remote replica
- ReplicaIndexFlushRequest request = ReplicationProtocol.readReplicaIndexFlushRequest(inBuffer);
- Set<Long> requestedIndexesToBeFlushed = request.getLaggingRescouresIds();
-
- /**
- * check which indexes can be flushed (open indexes) and which cannot be
- * flushed (closed or have empty memory component).
- */
- IDatasetLifecycleManager datasetLifeCycleManager = appContextProvider.getDatasetLifecycleManager();
- List<IndexInfo> openIndexesInfo = datasetLifeCycleManager.getOpenIndexesInfo();
- Set<Integer> datasetsToForceFlush = new HashSet<>();
- for (IndexInfo iInfo : openIndexesInfo) {
- if (requestedIndexesToBeFlushed.contains(iInfo.getResourceId())) {
- AbstractLSMIOOperationCallback ioCallback = (AbstractLSMIOOperationCallback) iInfo.getIndex()
- .getIOOperationCallback();
- //if an index has a pending flush, then the request to flush it will succeed.
- if (ioCallback.hasPendingFlush()) {
- //remove index to indicate that it will be flushed
- requestedIndexesToBeFlushed.remove(iInfo.getResourceId());
- } else if (!((AbstractLSMIndex) iInfo.getIndex()).isCurrentMutableComponentEmpty()) {
- /**
- * if an index has something to be flushed, then the request to flush it
- * will succeed and we need to schedule it to be flushed.
- */
- datasetsToForceFlush.add(iInfo.getDatasetId());
- //remove index to indicate that it will be flushed
- requestedIndexesToBeFlushed.remove(iInfo.getResourceId());
- }
- }
- }
-
- //schedule flush for datasets requested to be flushed
- for (int datasetId : datasetsToForceFlush) {
- datasetLifeCycleManager.flushDataset(datasetId, true);
- }
-
- //the remaining indexes in the requested set are those which cannot be flushed.
- //respond back to the requester that those indexes cannot be flushed
- ReplicaIndexFlushRequest laggingIndexesResponse = new ReplicaIndexFlushRequest(requestedIndexesToBeFlushed);
- outBuffer = ReplicationProtocol.writeGetReplicaIndexFlushRequest(outBuffer, laggingIndexesResponse);
- NetworkingUtil.transferBufferToChannel(socketChannel, outBuffer);
- }
-
- private void handleLSMComponentProperties() throws IOException {
- inBuffer = ReplicationProtocol.readRequest(socketChannel, inBuffer);
- LSMComponentProperties lsmCompProp = ReplicationProtocol.readLSMPropertiesRequest(inBuffer);
- //create mask to indicate that this component is not valid yet
- replicaResourcesManager.createRemoteLSMComponentMask(lsmCompProp);
- lsmComponentId2PropertiesMap.put(lsmCompProp.getComponentId(), lsmCompProp);
- }
-
- private void handleReplicateFile() throws IOException {
- inBuffer = ReplicationProtocol.readRequest(socketChannel, inBuffer);
- LSMIndexFileProperties afp = ReplicationProtocol.readFileReplicationRequest(inBuffer);
-
- //get index path
- String indexPath = replicaResourcesManager.getIndexPath(afp);
- String replicaFilePath = indexPath + File.separator + afp.getFileName();
-
- //create file
- File destFile = new File(replicaFilePath);
- destFile.createNewFile();
-
- try (RandomAccessFile fileOutputStream = new RandomAccessFile(destFile, "rw");
- FileChannel fileChannel = fileOutputStream.getChannel()) {
- fileOutputStream.setLength(afp.getFileSize());
- NetworkingUtil.downloadFile(fileChannel, socketChannel);
- fileChannel.force(true);
-
- if (afp.requiresAck()) {
- ReplicationProtocol.sendAck(socketChannel);
- }
- if (afp.isLSMComponentFile()) {
- String componentId = LSMComponentProperties.getLSMComponentID(afp.getFilePath());
- final LSMComponentProperties lsmComponentProperties = lsmComponentId2PropertiesMap.get(componentId);
- // merge operations do not generate flush logs
- if (afp.requiresAck() && lsmComponentProperties.getOpType() == LSMOperationType.FLUSH) {
- LSMComponentLSNSyncTask syncTask =
- new LSMComponentLSNSyncTask(componentId, destFile.getAbsolutePath());
- lsmComponentRemoteLSN2LocalLSNMappingTaskQ.offer(syncTask);
- } else {
- updateLSMComponentRemainingFiles(componentId);
- }
- } else {
- //index metadata file
- final ResourceReference indexRef = ResourceReference.of(destFile.getAbsolutePath());
- indexCheckpointManagerProvider.get(indexRef).init(logManager.getAppendLSN());
- }
- }
- }
-
- private void handleGetReplicaMaxLSN() throws IOException {
- long maxLNS = logManager.getAppendLSN();
- outBuffer.clear();
- outBuffer.putLong(maxLNS);
- outBuffer.flip();
- NetworkingUtil.transferBufferToChannel(socketChannel, outBuffer);
- }
-
- private void handleGetReplicaFiles() throws IOException {
- inBuffer = ReplicationProtocol.readRequest(socketChannel, inBuffer);
- ReplicaFilesRequest request = ReplicationProtocol.readReplicaFileRequest(inBuffer);
-
- LSMIndexFileProperties fileProperties = new LSMIndexFileProperties();
-
- List<String> filesList;
- Set<Integer> partitionIds = request.getPartitionIds();
- Set<String> requesterExistingFiles = request.getExistingFiles();
- Map<Integer, ClusterPartition> clusterPartitions = appContextProvider.getAppContext()
- .getMetadataProperties().getClusterPartitions();
-
- // Flush replicated datasets to generate the latest LSM components
- dsLifecycleManager.flushDataset(replicationStrategy);
- for (Integer partitionId : partitionIds) {
- ClusterPartition partition = clusterPartitions.get(partitionId);
- filesList = replicaResourcesManager.getPartitionIndexesFiles(partition.getPartitionId(), false);
- //start sending files
- for (String filePath : filesList) {
- // Send only files of datasets that are replciated.
- DatasetResourceReference indexFileRef = localResourceRep.getLocalResourceReference(filePath);
- if (!replicationStrategy.isMatch(indexFileRef.getDatasetId())) {
- continue;
- }
- String relativeFilePath = StoragePathUtil.getIndexFileRelativePath(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, partition.getNodeId(), false, false);
- outBuffer = ReplicationProtocol.writeFileReplicationRequest(outBuffer, fileProperties,
- ReplicationRequestType.REPLICATE_FILE);
-
- //send file info
- NetworkingUtil.transferBufferToChannel(socketChannel, outBuffer);
-
- //transfer file
- NetworkingUtil.sendFile(fileChannel, socketChannel);
- }
- }
- }
- }
-
- //send goodbye (end of files)
- ReplicationProtocol.sendGoodbye(socketChannel);
- }
-
- private void handleReplicaEvent() throws IOException {
- inBuffer = ReplicationProtocol.readRequest(socketChannel, inBuffer);
- ReplicaEvent event = ReplicationProtocol.readReplicaEventRequest(inBuffer);
- replicationManager.reportReplicaEvent(event);
- }
-
- private void handleDeleteFile() throws IOException {
- inBuffer = ReplicationProtocol.readRequest(socketChannel, inBuffer);
- LSMIndexFileProperties fileProp = ReplicationProtocol.readFileReplicationRequest(inBuffer);
- replicaResourcesManager.deleteIndexFile(fileProp);
- if (fileProp.requiresAck()) {
- ReplicationProtocol.sendAck(socketChannel);
- }
- }
-
- private void handleLogReplication() throws IOException, ACIDException {
- //set initial buffer size to a log buffer page size
- inBuffer = ByteBuffer.allocate(logManager.getLogPageSize());
- while (true) {
- //read a batch of logs
- inBuffer = ReplicationProtocol.readRequest(socketChannel, inBuffer);
- //check if it is end of handshake (a single byte log)
- if (inBuffer.remaining() == LOG_REPLICATION_END_HANKSHAKE_LOG_SIZE) {
- break;
- }
-
- processLogsBatch(inBuffer);
- }
- }
-
- private void processLogsBatch(ByteBuffer buffer) throws ACIDException {
- while (buffer.hasRemaining()) {
- //get rid of log size
- inBuffer.getInt();
- //Deserialize log
- remoteLog.readRemoteLog(inBuffer);
- remoteLog.setLogSource(LogSource.REMOTE);
-
- switch (remoteLog.getLogType()) {
- case LogType.UPDATE:
- case LogType.ENTITY_COMMIT:
- logManager.log(remoteLog);
- break;
- case LogType.JOB_COMMIT:
- case LogType.ABORT:
- LogRecord jobTerminationLog = new LogRecord();
- TransactionUtil.formJobTerminateLogRecord(jobTerminationLog, remoteLog.getTxnId(),
- remoteLog.getLogType() == LogType.JOB_COMMIT);
- jobTerminationLog.setReplicationThread(this);
- jobTerminationLog.setLogSource(LogSource.REMOTE);
- logManager.log(jobTerminationLog);
- break;
- case LogType.FLUSH:
- //store mapping information for flush logs to use them in incoming LSM components.
- RemoteLogMapping flushLogMap = new RemoteLogMapping();
- LogRecord flushLog = new LogRecord();
- TransactionUtil.formFlushLogRecord(flushLog, remoteLog.getDatasetId(), null,
- remoteLog.getNodeId(), remoteLog.getNumOfFlushedIndexes());
- flushLog.setReplicationThread(this);
- flushLog.setLogSource(LogSource.REMOTE);
- flushLogMap.setRemoteNodeID(remoteLog.getNodeId());
- flushLogMap.setRemoteLSN(remoteLog.getLSN());
- synchronized (localLsn2RemoteMapping) {
- logManager.log(flushLog);
- //the log LSN value is updated by logManager.log(.) to a local value
- flushLogMap.setLocalLSN(flushLog.getLSN());
- flushLogMap.numOfFlushedIndexes.set(flushLog.getNumOfFlushedIndexes());
- replicaUniqueLSN2RemoteMapping.put(flushLogMap.getNodeUniqueLSN(), flushLogMap);
- localLsn2RemoteMapping.put(flushLog.getLSN(), flushLogMap);
- }
- synchronized (flushLogslock) {
- flushLogslock.notify();
- }
- break;
- default:
- LOGGER.error("Unsupported LogType: " + remoteLog.getLogType());
- }
- }
- }
-
- /**
- * this method is called sequentially by {@link LogBuffer#notifyReplicationTermination()}
- * for JOB_COMMIT, JOB_ABORT, and FLUSH log types.
- */
- @Override
- public void notifyLogReplicationRequester(LogRecord logRecord) {
- switch (logRecord.getLogType()) {
- case LogType.JOB_COMMIT:
- case LogType.ABORT:
- pendingNotificationRemoteLogsQ.offer(logRecord);
- break;
- case LogType.FLUSH:
- final RemoteLogMapping remoteLogMapping;
- synchronized (localLsn2RemoteMapping) {
- remoteLogMapping = localLsn2RemoteMapping.remove(logRecord.getLSN());
- }
- checkpointReplicaIndexes(remoteLogMapping, logRecord.getDatasetId());
- break;
- default:
- throw new IllegalStateException("Unexpected log type: " + logRecord.getLogType());
+ Thread.currentThread().setName(oldName);
}
}
@@ -571,115 +138,10 @@
return outBuffer;
}
- private void checkpointReplicaIndexes(RemoteLogMapping remoteLogMapping, int datasetId) {
- try {
- Predicate<LocalResource> replicaIndexesPredicate = lr -> {
- DatasetLocalResource dls = (DatasetLocalResource) lr.getResource();
- return dls.getDatasetId() == datasetId && !localResourceRep.getActivePartitions()
- .contains(dls.getPartition());
- };
- final Map<Long, LocalResource> resources = localResourceRep.getResources(replicaIndexesPredicate);
- final List<DatasetResourceReference> replicaIndexesRef =
- resources.values().stream().map(DatasetResourceReference::of).collect(Collectors.toList());
- for (DatasetResourceReference replicaIndexRef : replicaIndexesRef) {
- final IIndexCheckpointManager indexCheckpointManager =
- indexCheckpointManagerProvider.get(replicaIndexRef);
- synchronized (indexCheckpointManager) {
- indexCheckpointManager
- .masterFlush(remoteLogMapping.getRemoteLSN(), remoteLogMapping.getLocalLSN());
- }
- }
- } catch (Exception e) {
- LOGGER.error("Failed to checkpoint replica indexes", e);
- }
- }
-
- private void handleGetPartitionResources() throws IOException {
- final PartitionResourcesListTask task = (PartitionResourcesListTask) ReplicationProtocol
- .readMessage(ReplicationRequestType.PARTITION_RESOURCES_REQUEST, socketChannel, inBuffer);
+ private void handle(ReplicationRequestType requestType) throws HyracksDataException {
+ final IReplicaTask task =
+ (IReplicaTask) ReplicationProtocol.readMessage(requestType, socketChannel, inBuffer);
task.perform(appCtx, this);
}
-
- private void handleReplicateResourceFile() throws HyracksDataException {
- ReplicateFileTask task = (ReplicateFileTask) ReplicationProtocol
- .readMessage(ReplicationRequestType.REPLICATE_RESOURCE_FILE, socketChannel, inBuffer);
- task.perform(appCtx, this);
- }
-
- private void handleDeleteResourceFile() throws HyracksDataException {
- DeleteFileTask task = (DeleteFileTask) ReplicationProtocol
- .readMessage(ReplicationRequestType.DELETE_RESOURCE_FILE, socketChannel, inBuffer);
- task.perform(appCtx, this);
- }
-
- private void handleCheckpointPartition() throws HyracksDataException {
- CheckpointPartitionIndexesTask task = (CheckpointPartitionIndexesTask) ReplicationProtocol
- .readMessage(ReplicationRequestType.CHECKPOINT_PARTITION, socketChannel, inBuffer);
- task.perform(appCtx, this);
- }
- }
-
- /**
- * This thread is responsible for sending JOB_COMMIT/ABORT ACKs to replication clients.
- */
- private class ReplicationNotifier extends Thread {
- @Override
- public void run() {
- Thread.currentThread().setName("ReplicationNotifier Thread");
- while (true) {
- try {
- LogRecord logRecord = pendingNotificationRemoteLogsQ.take();
- //send ACK to requester
- logRecord.getReplicationThread().getChannel().socket().getOutputStream()
- .write((localNodeID + ReplicationProtocol.JOB_REPLICATION_ACK + logRecord.getTxnId()
- + System.lineSeparator()).getBytes());
- } catch (InterruptedException e) {
- Thread.currentThread().interrupt();
- } catch (IOException e) {
- LOGGER.warn("Failed to send job replication ACK", e);
- }
- }
- }
- }
-
- /**
- * This thread is responsible for synchronizing the LSN of
- * the received LSM components to a local LSN.
- */
- private class LSMComponentsSyncService extends Thread {
-
- @Override
- public void run() {
- Thread.currentThread().setName("LSMComponentsSyncService Thread");
-
- while (true) {
- try {
- LSMComponentLSNSyncTask syncTask = lsmComponentRemoteLSN2LocalLSNMappingTaskQ.take();
- LSMComponentProperties lsmCompProp = lsmComponentId2PropertiesMap.get(syncTask.getComponentId());
- syncLSMComponentFlushLSN(lsmCompProp, syncTask);
- updateLSMComponentRemainingFiles(lsmCompProp.getComponentId());
- } catch (InterruptedException e) {
- Thread.currentThread().interrupt();
- } catch (Exception e) {
- LOGGER.error("Unexpected exception during LSN synchronization", e);
- }
- }
- }
-
- private void syncLSMComponentFlushLSN(LSMComponentProperties lsmCompProp, LSMComponentLSNSyncTask syncTask)
- throws InterruptedException, IOException {
- final String componentFilePath = syncTask.getComponentFilePath();
- final ResourceReference indexRef = ResourceReference.of(componentFilePath);
- final IIndexCheckpointManager indexCheckpointManager = indexCheckpointManagerProvider.get(indexRef);
- synchronized (indexCheckpointManager) {
- long masterLsn = lsmCompProp.getOriginalLSN();
- // wait until the lsn mapping is flushed to disk
- while (!indexCheckpointManager.isFlushed(masterLsn)) {
- indexCheckpointManager.wait();
- }
- indexCheckpointManager
- .replicated(AbstractLSMIndexFileManager.getComponentEndTime(indexRef.getName()), masterLsn);
- }
- }
}
}
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/management/ReplicationManager.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/management/ReplicationManager.java
index 6445345..727a379 100644
--- a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/management/ReplicationManager.java
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/management/ReplicationManager.java
@@ -18,1342 +18,120 @@
*/
package org.apache.asterix.replication.management;
-import java.io.BufferedReader;
-import java.io.File;
import java.io.IOException;
-import java.io.InputStreamReader;
import java.io.OutputStream;
-import java.io.RandomAccessFile;
import java.net.InetSocketAddress;
-import java.nio.ByteBuffer;
-import java.nio.channels.AsynchronousCloseException;
-import java.nio.channels.FileChannel;
-import java.nio.channels.SocketChannel;
-import java.nio.file.Files;
-import java.nio.file.Path;
-import java.nio.file.Paths;
-import java.util.ArrayList;
-import java.util.Collections;
import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Optional;
-import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.Future;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.stream.Collectors;
-import org.apache.asterix.common.cluster.ClusterPartition;
+import org.apache.asterix.common.api.INcApplicationContext;
import org.apache.asterix.common.config.ReplicationProperties;
import org.apache.asterix.common.replication.IPartitionReplica;
-import org.apache.asterix.common.replication.IReplicaResourcesManager;
+import org.apache.asterix.common.replication.IReplicationDestination;
import org.apache.asterix.common.replication.IReplicationManager;
import org.apache.asterix.common.replication.IReplicationStrategy;
-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.ReplicationJob;
import org.apache.asterix.common.replication.ReplicationStrategyFactory;
-import org.apache.asterix.common.storage.DatasetResourceReference;
-import org.apache.asterix.common.storage.IIndexCheckpointManagerProvider;
-import org.apache.asterix.common.storage.ResourceReference;
-import org.apache.asterix.common.transactions.IAppRuntimeContextProvider;
-import org.apache.asterix.common.transactions.ILogManager;
import org.apache.asterix.common.transactions.ILogRecord;
-import org.apache.asterix.common.transactions.LogType;
-import org.apache.asterix.replication.functions.ReplicaFilesRequest;
-import org.apache.asterix.replication.functions.ReplicaIndexFlushRequest;
-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.TxnLogReplicator;
-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.application.INCServiceContext;
-import org.apache.hyracks.api.config.IApplicationConfig;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.asterix.replication.api.ReplicationDestination;
import org.apache.hyracks.api.replication.IReplicationJob;
-import org.apache.hyracks.api.replication.IReplicationJob.ReplicationExecutionType;
-import org.apache.hyracks.api.replication.IReplicationJob.ReplicationJobType;
-import org.apache.hyracks.api.replication.IReplicationJob.ReplicationOperation;
-import org.apache.hyracks.control.common.controllers.NCConfig;
-import org.apache.hyracks.control.nc.NodeControllerService;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexReplicationJob;
-import org.apache.hyracks.util.StorageUtil;
-import org.apache.hyracks.util.StorageUtil.StorageUnit;
-import org.apache.logging.log4j.Level;
+import org.apache.hyracks.util.annotations.ThreadSafe;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
-/**
- * This class is used to process replication jobs and maintain remote replicas states
- */
+@ThreadSafe
public class ReplicationManager implements IReplicationManager {
private static final Logger LOGGER = LogManager.getLogger();
- private static final int INITIAL_REPLICATION_FACTOR = 1;
- private static final int MAX_JOB_COMMIT_ACK_WAIT = 10000;
- private final String nodeId;
- private ExecutorService replicationListenerThreads;
- private final Map<Long, Set<String>> txnCommitAcks;
- private final Map<Long, ILogRecord> replicationTxnsPendingAcks;
- private ByteBuffer dataBuffer;
- private final LinkedBlockingQueue<IReplicationJob> replicationJobsQ;
- private final LinkedBlockingQueue<ReplicaEvent> replicaEventsQ;
-
- private int replicationFactor = 1;
- private final ReplicaResourcesManager replicaResourcesManager;
- private final ILogManager logManager;
- private final IAppRuntimeContextProvider asterixAppRuntimeContextProvider;
+ private final Map<InetSocketAddress, ReplicationDestination> dests = new HashMap<>();
private final ReplicationProperties replicationProperties;
- private final Map<String, Replica> replicas;
- private final Map<String, Set<Integer>> replica2PartitionsMap;
+ private final IReplicationStrategy strategy;
+ private final INcApplicationContext appCtx;
+ private final LogReplicationManager logReplicationManager;
+ private final IndexReplicationManager lsnIndexReplicationManager;
- private final AtomicBoolean replicationSuspended;
- private AtomicBoolean terminateJobsReplication;
- private AtomicBoolean jobsReplicationSuspended;
- private static final int INITIAL_BUFFER_SIZE = StorageUtil.getIntSizeInBytes(4, StorageUnit.KILOBYTE);
- private final Set<String> shuttingDownReplicaIds;
- //replication threads
- private ReplicationJobsProccessor replicationJobsProcessor;
- private final ReplicasEventsMonitor replicationMonitor;
- //dummy job used to stop ReplicationJobsProccessor thread.
- private static final IReplicationJob REPLICATION_JOB_POISON_PILL = new ReplicationJob(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;
-
- private LinkedBlockingQueue<ReplicationLogBuffer> emptyLogBuffersQ;
- private LinkedBlockingQueue<ReplicationLogBuffer> pendingFlushLogBuffersQ;
- protected ReplicationLogBuffer currentTxnLogBuffer;
- private TxnLogReplicator txnlogReplicator;
- private Future<? extends Object> txnLogReplicatorTask;
- private SocketChannel[] logsRepSockets;
- private final ByteBuffer txnLogsBatchSizeBuffer = ByteBuffer.allocate(Integer.BYTES);
- private IReplicationStrategy replicationStrategy;
- private final PersistentLocalResourceRepository localResourceRepo;
- private NCConfig ncConfig;
- private final IIndexCheckpointManagerProvider indexCheckpointManagerProvider;
-
- //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, ReplicationProperties replicationProperties,
- IReplicaResourcesManager remoteResoucesManager, ILogManager logManager,
- IAppRuntimeContextProvider asterixAppRuntimeContextProvider, INCServiceContext ncServiceContext) {
- this.nodeId = nodeId;
- this.ncConfig = ((NodeControllerService) ncServiceContext.getControllerService()).getConfiguration();
+ public ReplicationManager(INcApplicationContext appCtx, ReplicationProperties replicationProperties) {
this.replicationProperties = replicationProperties;
- try {
- replicationStrategy = ReplicationStrategyFactory.create(replicationProperties.getReplicationStrategy(),
- replicationProperties, ncConfig.getConfigManager());
- } catch (HyracksDataException e) {
- LOGGER.log(Level.WARN, "Couldn't initialize replication strategy", e);
- }
- this.replicaResourcesManager = (ReplicaResourcesManager) remoteResoucesManager;
- this.asterixAppRuntimeContextProvider = asterixAppRuntimeContextProvider;
- this.logManager = logManager;
- localResourceRepo =
- (PersistentLocalResourceRepository) asterixAppRuntimeContextProvider.getLocalResourceRepository();
- this.hostIPAddressFirstOctet = ncConfig.getPublicAddress().substring(0, 3);
- this.indexCheckpointManagerProvider =
- asterixAppRuntimeContextProvider.getAppContext().getIndexCheckpointManagerProvider();
- replicas = new HashMap<>();
- replicationJobsQ = new LinkedBlockingQueue<>();
- replicaEventsQ = new LinkedBlockingQueue<>();
- terminateJobsReplication = new AtomicBoolean(false);
- jobsReplicationSuspended = new AtomicBoolean(true);
- replicationSuspended = new AtomicBoolean(true);
- txnCommitAcks = new ConcurrentHashMap<>();
- replicationTxnsPendingAcks = new ConcurrentHashMap<>();
- shuttingDownReplicaIds = new HashSet<>();
- dataBuffer = ByteBuffer.allocate(INITIAL_BUFFER_SIZE);
- replicationMonitor = new ReplicasEventsMonitor();
- //add list of replicas from configurations (To be read from another source e.g. Zookeeper)
- Set<Replica> replicaNodes = replicationStrategy.getRemoteReplicas(nodeId);
-
- //Used as async listeners from replicas
- replicationListenerThreads = Executors.newCachedThreadPool();
- replicationJobsProcessor = new ReplicationJobsProccessor();
-
- Map<String, ClusterPartition[]> nodePartitions =
- asterixAppRuntimeContextProvider.getAppContext().getMetadataProperties().getNodePartitions();
- replica2PartitionsMap = new HashMap<>(replicaNodes.size());
- for (Replica replica : replicaNodes) {
- replicas.put(replica.getId(), replica);
- //for each remote replica, get the list of replication clients
- Set<Replica> nodeReplicationClients = replicationStrategy.getRemotePrimaryReplicas(replica.getId());
- //get the partitions of each client
- List<Integer> clientPartitions = new ArrayList<>();
- for (Replica client : nodeReplicationClients) {
- for (ClusterPartition clusterPartition : nodePartitions.get(client.getId())) {
- clientPartitions.add(clusterPartition.getPartitionId());
- }
- }
- Set<Integer> clientPartitonsSet = new HashSet<>(clientPartitions.size());
- clientPartitonsSet.addAll(clientPartitions);
- replica2PartitionsMap.put(replica.getId(), clientPartitonsSet);
- }
- int numLogBuffers = replicationProperties.getLogBufferNumOfPages();
- emptyLogBuffersQ = new LinkedBlockingQueue<>(numLogBuffers);
- pendingFlushLogBuffersQ = new LinkedBlockingQueue<>(numLogBuffers);
-
- int logBufferSize = replicationProperties.getLogBufferPageSize();
- for (int i = 0; i < numLogBuffers; i++) {
- emptyLogBuffersQ
- .offer(new ReplicationLogBuffer(this, logBufferSize, replicationProperties.getLogBatchSize()));
- }
- }
-
- @Override
- public void submitJob(IReplicationJob job) throws IOException {
- if (job.getExecutionType() == ReplicationExecutionType.ASYNC) {
- replicationJobsQ.offer(job);
- } else {
- //wait until replication is resumed
- while (replicationSuspended.get()) {
- synchronized (replicationSuspended) {
- try {
- replicationSuspended.wait();
- } catch (InterruptedException e) {
- Thread.currentThread().interrupt();
- }
- }
- }
- processJob(job, null, null);
- }
- }
-
- @Override
- public void replicateLog(ILogRecord logRecord) throws InterruptedException {
- if (logRecord.getLogType() == LogType.JOB_COMMIT || logRecord.getLogType() == LogType.ABORT) {
- //if replication is suspended, wait until it is resumed.
- while (replicationSuspended.get()) {
- synchronized (replicationSuspended) {
- replicationSuspended.wait();
- }
- }
- Set<String> replicaIds = Collections.synchronizedSet(new HashSet<String>());
- replicaIds.add(nodeId);
- txnCommitAcks.put(logRecord.getTxnId(), replicaIds);
- }
-
- appendToLogBuffer(logRecord);
- }
-
- protected void getAndInitNewLargePage(int pageSize) {
- // for now, alloc a new buffer for each large page
- // TODO: consider pooling large pages
- currentTxnLogBuffer = new ReplicationLogBuffer(this, pageSize, replicationProperties.getLogBufferPageSize());
- pendingFlushLogBuffersQ.offer(currentTxnLogBuffer);
- }
-
- protected void getAndInitNewPage() throws InterruptedException {
- currentTxnLogBuffer = null;
- while (currentTxnLogBuffer == null) {
- currentTxnLogBuffer = emptyLogBuffersQ.take();
- }
- currentTxnLogBuffer.reset();
- pendingFlushLogBuffersQ.offer(currentTxnLogBuffer);
- }
-
- private synchronized void appendToLogBuffer(ILogRecord logRecord) throws InterruptedException {
- if (!currentTxnLogBuffer.hasSpace(logRecord)) {
- currentTxnLogBuffer.isFull(true);
- if (logRecord.getLogSize() > getLogPageSize()) {
- getAndInitNewLargePage(logRecord.getLogSize());
- } else {
- getAndInitNewPage();
- }
- }
- currentTxnLogBuffer.append(logRecord);
- }
-
- /**
- * Processes the replication job based on its specifications
- *
- * @param job
- * The replication job
- * @param replicasSockets
- * The remote replicas sockets to send the request to.
- * @param requestBuffer
- * The buffer to use to send the request.
- * @throws IOException
- */
- private void processJob(IReplicationJob job, Map<String, SocketChannel> replicasSockets, ByteBuffer requestBuffer)
- throws IOException {
- try {
-
- //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();
- DatasetResourceReference indexFileRef = localResourceRepo.getLocalResourceReference(jobFile);
- if (!replicationStrategy.isMatch(indexFileRef.getDatasetId())) {
- return;
- }
- int jobPartitionId = indexFileRef.getPartitionId();
-
- ByteBuffer responseBuffer = null;
- LSMIndexFileProperties asterixFileProperties = new LSMIndexFileProperties();
- if (requestBuffer == null) {
- requestBuffer = ByteBuffer.allocate(INITIAL_BUFFER_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();
- }
-
- 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) {
- //send LSMComponent properties
- LSMComponentJob = (ILSMIndexReplicationJob) job;
- LSMComponentProperties lsmCompProp = new LSMComponentProperties(LSMComponentJob, nodeId);
- requestBuffer = ReplicationProtocol.writeLSMComponentPropertiesRequest(lsmCompProp, //NOSONAR
- requestBuffer);
- sendRequest(replicasSockets, requestBuffer);
- }
-
- for (String filePath : job.getJobFiles()) {
- remainingFiles--;
- Path path = Paths.get(filePath);
- if (Files.notExists(path)) {
- LOGGER.log(Level.ERROR, "File deleted before replication: " + filePath);
- continue;
- }
-
- LOGGER.log(Level.INFO, "Replicating file: " + filePath);
- //open file for reading
- try (RandomAccessFile fromFile = new RandomAccessFile(filePath, "r");
- FileChannel fileChannel = fromFile.getChannel();) {
-
- long fileSize = fileChannel.size();
- asterixFileProperties.initialize(filePath, fileSize, nodeId, isLSMComponentFile,
- remainingFiles == 0);
- requestBuffer = ReplicationProtocol.writeFileReplicationRequest(requestBuffer,
- asterixFileProperties, ReplicationRequestType.REPLICATE_FILE);
- Iterator<Map.Entry<String, SocketChannel>> iterator = replicasSockets.entrySet().iterator();
- while (iterator.hasNext()) {
- Map.Entry<String, SocketChannel> entry = iterator.next();
- //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 {
- NetworkingUtil.transferBufferToChannel(socketChannel, requestBuffer);
- NetworkingUtil.sendFile(fileChannel, socketChannel);
- if (asterixFileProperties.requiresAck()) {
- ReplicationRequestType responseType =
- waitForResponse(socketChannel, responseBuffer);
- if (responseType != ReplicationRequestType.ACK) {
- throw new IOException(
- "Could not receive ACK from replica " + entry.getKey());
- }
- }
- } catch (IOException e) {
- handleReplicationFailure(socketChannel, e);
- iterator.remove();
- } finally {
- requestBuffer.position(0);
- }
- }
- }
- }
- } else if (job.getOperation() == ReplicationOperation.DELETE) {
- for (String filePath : job.getJobFiles()) {
- remainingFiles--;
- asterixFileProperties.initialize(filePath, -1, nodeId, isLSMComponentFile, 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();
- //if the remote replica is not interested in this partition, skip it.
- if (!replica2PartitionsMap.get(entry.getKey()).contains(jobPartitionId)) {
- continue;
- }
- SocketChannel socketChannel = entry.getValue();
- try {
- sendRequest(replicasSockets, requestBuffer);
- if (asterixFileProperties.requiresAck()) {
- waitForResponse(socketChannel, responseBuffer);
- }
- } catch (IOException e) {
- handleReplicationFailure(socketChannel, e);
- 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 {
- 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();
- }
- }
-
- /**
- * Waits and reads a response from a remote replica
- *
- * @param socketChannel
- * The socket to read the response from
- * @param responseBuffer
- * The response buffer to read the response to.
- * @return The response type.
- * @throws IOException
- */
- private static ReplicationRequestType waitForResponse(SocketChannel socketChannel, ByteBuffer responseBuffer)
- throws IOException {
- if (responseBuffer == null) {
- responseBuffer = ByteBuffer.allocate(ReplicationProtocol.REPLICATION_REQUEST_TYPE_SIZE);
- } else {
- responseBuffer.clear();
- }
-
- //read response from remote replicas
- ReplicationRequestType responseFunction = ReplicationProtocol.getRequestType(socketChannel, responseBuffer);
- return responseFunction;
- }
-
- @Override
- public boolean isReplicationEnabled() {
- return replicationStrategy.isParticipant(nodeId);
- }
-
- @Override
- public synchronized void updateReplicaInfo(Replica replicaNode) {
- Replica replica = replicas.get(replicaNode.getId());
- //should not update the info of an active replica
- if (replica.getState() == ReplicaState.ACTIVE) {
- return;
- }
- replica.setClusterIp(replicaNode.getClusterIp());
- }
-
- /**
- * Suspends processing replication jobs/logs.
- *
- * @param force
- * a flag indicates if replication should be suspended right away or when the pending jobs are completed.
- */
- private void suspendReplication(boolean force) {
- //suspend replication jobs processing
- if (replicationJobsProcessor != null && replicationJobsProcessor.isAlive()) {
- if (force) {
- terminateJobsReplication.set(true);
- }
- replicationJobsQ.offer(REPLICATION_JOB_POISON_PILL);
-
- //wait until the jobs are suspended
- synchronized (jobsReplicationSuspended) {
- while (!jobsReplicationSuspended.get()) {
- try {
- jobsReplicationSuspended.wait();
- } catch (InterruptedException e) {
- Thread.currentThread().interrupt();
- }
- }
- }
- }
-
- //suspend logs replication
- if (txnlogReplicator != null) {
- endTxnLogReplicationHandshake();
- }
- }
-
- /**
- * Opens a new connection with Active remote replicas and starts a listen thread per connection.
- */
- private void establishTxnLogReplicationHandshake() {
- Map<String, SocketChannel> activeRemoteReplicasSockets = getActiveRemoteReplicasSockets();
- logsRepSockets = new SocketChannel[activeRemoteReplicasSockets.size()];
- int i = 0;
- //start a listener thread per connection
- for (Entry<String, SocketChannel> entry : activeRemoteReplicasSockets.entrySet()) {
- logsRepSockets[i] = entry.getValue();
- replicationListenerThreads
- .execute(new TxnLogsReplicationResponseListener(entry.getKey(), entry.getValue()));
- i++;
- }
-
- /*
- * establish log replication handshake
- */
- ByteBuffer handshakeBuffer = ByteBuffer.allocate(ReplicationProtocol.REPLICATION_REQUEST_TYPE_SIZE)
- .putInt(ReplicationProtocol.ReplicationRequestType.REPLICATE_LOG.ordinal());
- handshakeBuffer.flip();
- //send handshake request
- for (SocketChannel replicaSocket : logsRepSockets) {
- try {
- NetworkingUtil.transferBufferToChannel(replicaSocket, handshakeBuffer);
- } catch (IOException e) {
- handleReplicationFailure(replicaSocket, e);
- } finally {
- handshakeBuffer.position(0);
- }
- }
- }
-
- private void handleReplicationFailure(SocketChannel socketChannel, Throwable t) {
- if (LOGGER.isWarnEnabled()) {
- LOGGER.log(Level.WARN, "Could not complete replication request.", t);
- }
- if (socketChannel.isOpen()) {
- try {
- socketChannel.close();
- } catch (IOException e) {
- LOGGER.log(Level.WARN, "Could not close socket.", e);
- }
- }
- reportFailedReplica(getReplicaIdBySocket(socketChannel));
- }
-
- /**
- * Stops TxnLogReplicator and closes the sockets used to replicate logs.
- */
- private void endTxnLogReplicationHandshake() {
- LOGGER.info("Terminating TxnLogReplicator thread ...");
- txnlogReplicator.terminate();
- try {
- txnLogReplicatorTask.get();
- } catch (ExecutionException | InterruptedException e) {
- LOGGER.error("TxnLogReplicator thread terminated abnormally", e);
- }
- LOGGER.info("TxnLogReplicator thread was terminated.");
-
- /*
- * End log replication handshake (by sending a dummy log with a single byte)
- */
- ByteBuffer endLogRepHandshake = ByteBuffer.allocate(Integer.SIZE + 1).putInt(1).put((byte) 0);
- endLogRepHandshake.flip();
- for (SocketChannel replicaSocket : logsRepSockets) {
- try {
- NetworkingUtil.transferBufferToChannel(replicaSocket, endLogRepHandshake);
- } catch (IOException e) {
- handleReplicationFailure(replicaSocket, e);
- } finally {
- endLogRepHandshake.position(0);
- }
- }
-
- //wait for any ACK to arrive before closing sockets.
- if (logsRepSockets != null) {
- synchronized (txnCommitAcks) {
- try {
- long waitStartTime = System.currentTimeMillis();
- while (!txnCommitAcks.isEmpty()) {
- txnCommitAcks.wait(1000);
- long waitDuration = System.currentTimeMillis() - waitStartTime;
- if (waitDuration > MAX_JOB_COMMIT_ACK_WAIT) {
- LOGGER.log(Level.ERROR,
- "Timeout before receving all job ACKs from replicas. Pending txns ("
- + txnCommitAcks.keySet().toString() + ")");
- break;
- }
- }
- } catch (InterruptedException e) {
- LOGGER.error("Interrupted while waiting for jobs ACK", e);
- Thread.currentThread().interrupt();
- }
- }
- }
-
- /*
- * Close log replication sockets
- */
- ByteBuffer goodbyeBuffer = ReplicationProtocol.getGoodbyeBuffer();
- for (SocketChannel replicaSocket : logsRepSockets) {
- try {
- //send goodbye to remote replica
- NetworkingUtil.transferBufferToChannel(replicaSocket, goodbyeBuffer);
- replicaSocket.close();
- } catch (IOException e) {
- handleReplicationFailure(replicaSocket, e);
- } finally {
- goodbyeBuffer.position(0);
- }
- }
- logsRepSockets = null;
- }
-
- /**
- * Sends a shutdown event to remote replicas notifying them
- * no more logs/files will be sent from this local replica.
- *
- * @throws IOException
- */
- private void sendShutdownNotifiction() throws IOException {
- Replica replica = new Replica(nodeId, NetworkingUtil.getHostAddress(hostIPAddressFirstOctet),
- ncConfig.getReplicationPublicPort());
- ReplicaEvent event = new ReplicaEvent(replica, ClusterEventType.NODE_SHUTTING_DOWN);
- ByteBuffer buffer = ReplicationProtocol.writeReplicaEventRequest(event);
- Map<String, SocketChannel> replicaSockets = getActiveRemoteReplicasSockets();
- sendRequest(replicaSockets, buffer);
- closeReplicaSockets(replicaSockets);
- }
-
- /**
- * Sends a request to remote replicas
- *
- * @param replicaSockets
- * The sockets to send the request to.
- * @param requestBuffer
- * The buffer that contains the request.
- */
- private void sendRequest(Map<String, SocketChannel> replicaSockets, ByteBuffer requestBuffer) {
- Iterator<Map.Entry<String, SocketChannel>> iterator = replicaSockets.entrySet().iterator();
- while (iterator.hasNext()) {
- Entry<String, SocketChannel> replicaSocket = iterator.next();
- SocketChannel clientSocket = replicaSocket.getValue();
- try {
- NetworkingUtil.transferBufferToChannel(clientSocket, requestBuffer);
- } catch (IOException e) {
- handleReplicationFailure(clientSocket, e);
- iterator.remove();
- } finally {
- requestBuffer.position(0);
- }
- }
- }
-
- /**
- * Closes the passed replication sockets by sending GOODBYE request to remote replicas.
- *
- * @param replicaSockets
- */
- private void closeReplicaSockets(Map<String, SocketChannel> replicaSockets) {
- //send goodbye
- ByteBuffer goodbyeBuffer = ReplicationProtocol.getGoodbyeBuffer();
- sendRequest(replicaSockets, goodbyeBuffer);
-
- Iterator<Map.Entry<String, SocketChannel>> iterator = replicaSockets.entrySet().iterator();
- while (iterator.hasNext()) {
- Entry<String, SocketChannel> replicaSocket = iterator.next();
- SocketChannel clientSocket = replicaSocket.getValue();
- if (clientSocket.isOpen()) {
- try {
- clientSocket.close();
- } catch (IOException e) {
- handleReplicationFailure(clientSocket, e);
- }
- }
- }
- }
-
- @Override
- public void initializeReplicasState() {
- for (Replica replica : replicas.values()) {
- checkReplicaState(replica.getId(), false, false);
- }
- }
-
- /**
- * Checks the state of a remote replica by trying to ping it.
- *
- * @param replicaId
- * The replica to check the state for.
- * @param async
- * a flag indicating whether to wait for the result or not.
- * @param suspendReplication
- * a flag indicating whether to suspend replication on replica state change or not.
- */
- private void checkReplicaState(String replicaId, boolean async, boolean suspendReplication) {
- Replica replica = replicas.get(replicaId);
-
- ReplicaStateChecker connector = new ReplicaStateChecker(replica, replicationProperties.getReplicationTimeOut(),
- this, suspendReplication);
- Future<? extends Object> ft = asterixAppRuntimeContextProvider.getThreadExecutor().submit(connector);
-
- if (!async) {
- //wait until task is done
- while (!ft.isDone()) {
- try {
- Thread.sleep(1000);
- } catch (InterruptedException e) {
- Thread.currentThread().interrupt();
- }
- }
- }
- }
-
- /**
- * Updates the state of a remote replica.
- *
- * @param replicaId
- * The replica id to update.
- * @param newState
- * The new state of the replica.
- * @param suspendReplication
- * a flag indicating whether to suspend replication on state change or not.
- * @throws InterruptedException
- */
- public synchronized void updateReplicaState(String replicaId, ReplicaState newState, boolean suspendReplication)
- throws InterruptedException {
- Replica replica = replicas.get(replicaId);
-
- if (replica.getState() == newState) {
- return;
- }
-
- if (suspendReplication) {
- //prevent new jobs/logs from coming in
- replicationSuspended.set(true);
-
- if (newState == ReplicaState.DEAD) {
- //assume the dead replica ACK has been received for all pending jobs
- synchronized (txnCommitAcks) {
- for (Long txnId : txnCommitAcks.keySet()) {
- addAckToJob(txnId, replicaId);
- }
- }
- }
-
- //force replication threads to stop in order to change the replication factor
- suspendReplication(true);
- }
-
- replica.setState(newState);
-
- if (newState == ReplicaState.ACTIVE) {
- replicationFactor++;
- } else if (newState == ReplicaState.DEAD && replicationFactor > INITIAL_REPLICATION_FACTOR) {
- replicationFactor--;
- }
-
- if (LOGGER.isWarnEnabled()) {
- LOGGER.warn("Replica " + replicaId + " state changed to: " + newState.name()
- + ". Replication factor changed to: " + replicationFactor);
- }
-
- if (suspendReplication) {
- startReplicationThreads();
- }
- }
-
- /**
- * When an ACK for a JOB_COMMIT is received, it is added to the corresponding job.
- *
- * @param txnId
- * @param replicaId
- * The remote replica id the ACK received from.
- */
- private void addAckToJob(long txnId, String replicaId) {
- synchronized (txnCommitAcks) {
- //add ACK to the job
- if (txnCommitAcks.containsKey(txnId)) {
- Set<String> replicaIds = txnCommitAcks.get(txnId);
- replicaIds.add(replicaId);
- } else {
- if (LOGGER.isWarnEnabled()) {
- LOGGER.warn("Invalid job replication ACK received for txnId(" + txnId + ")");
- }
- return;
- }
-
- //if got ACKs from all remote replicas, notify pending jobs if any
-
- if (txnCommitAcks.get(txnId).size() == replicationFactor && replicationTxnsPendingAcks.containsKey(txnId)) {
- ILogRecord pendingLog = replicationTxnsPendingAcks.get(txnId);
- synchronized (pendingLog) {
- pendingLog.notifyAll();
- }
- }
- }
- }
-
- @Override
- public boolean hasBeenReplicated(ILogRecord logRecord) {
- long txnId = logRecord.getTxnId();
- if (txnCommitAcks.containsKey(txnId)) {
- synchronized (txnCommitAcks) {
- //check if all ACKs have been received
- if (txnCommitAcks.get(txnId).size() == replicationFactor) {
- txnCommitAcks.remove(txnId);
-
- //remove from pending jobs if exists
- replicationTxnsPendingAcks.remove(txnId);
-
- //notify any threads waiting for all jobs to finish
- if (txnCommitAcks.size() == 0) {
- txnCommitAcks.notifyAll();
- }
- return true;
- } else {
- replicationTxnsPendingAcks.putIfAbsent(txnId, logRecord);
- return false;
- }
- }
- }
- //presume replicated
- return true;
- }
-
- private Map<String, SocketChannel> getActiveRemoteReplicasSockets() {
- Map<String, SocketChannel> replicaNodesSockets = new HashMap<>();
- for (Replica replica : replicas.values()) {
- if (replica.getState() == ReplicaState.ACTIVE) {
- try {
- SocketChannel sc = getReplicaSocket(replica.getId());
- replicaNodesSockets.put(replica.getId(), sc);
- } catch (IOException e) {
- if (LOGGER.isWarnEnabled()) {
- LOGGER.log(Level.WARN, "Could not get replica socket", e);
- }
- reportFailedReplica(replica.getId());
- }
- }
- }
- return replicaNodesSockets;
- }
-
- /**
- * Establishes a connection with a remote replica.
- *
- * @param replicaId
- * The replica to connect to.
- * @return The socket of the remote replica
- * @throws IOException
- */
- private SocketChannel getReplicaSocket(String replicaId) throws IOException {
- SocketChannel sc = SocketChannel.open();
- sc.configureBlocking(true);
- IApplicationConfig config = ncConfig.getConfigManager().getNodeEffectiveConfig(replicaId);
- sc.connect(new InetSocketAddress(config.getString(NCConfig.Option.REPLICATION_LISTEN_ADDRESS),
- config.getInt(NCConfig.Option.REPLICATION_LISTEN_PORT)));
- return sc;
- }
-
- @Override
- public Set<String> getDeadReplicasIds() {
- Set<String> replicasIds = new HashSet<>();
- for (Replica replica : replicas.values()) {
- if (replica.getState() == ReplicaState.DEAD) {
- replicasIds.add(replica.getId());
- }
- }
- return replicasIds;
- }
-
- @Override
- public Set<String> getActiveReplicasIds() {
- Set<String> replicasIds = new HashSet<>();
- for (Replica replica : replicas.values()) {
- if (replica.getState() == ReplicaState.ACTIVE) {
- replicasIds.add(replica.getId());
- }
- }
- return replicasIds;
- }
-
- @Override
- public int getActiveReplicasCount() {
- return getActiveReplicasIds().size();
- }
-
- @Override
- public void start() {
- //do nothing
- }
-
- @Override
- public void dumpState(OutputStream os) throws IOException {
- //do nothing
- }
-
- /**
- * Called during NC shutdown to notify remote replicas about the shutdown
- * and wait for remote replicas shutdown notification then closes the local
- * replication channel.
- */
- @Override
- public void stop(boolean dumpState, OutputStream ouputStream) throws IOException {
- //stop replication thread afters all jobs/logs have been processed
- suspendReplication(false);
-
- /*
- * If this node has any remote replicas, it needs to inform them
- * that it is shutting down.
- */
- if (!replicationStrategy.getRemoteReplicas(nodeId).isEmpty()) {
- //send shutdown event to remote replicas
- sendShutdownNotifiction();
- }
-
- /*
- * If this node has any remote primary replicas, then it needs to wait
- * until all of them send the shutdown notification.
- */
- // find active remote primary replicas
- Set<String> activeRemotePrimaryReplicas = replicationStrategy.getRemotePrimaryReplicas(nodeId).stream()
- .map(Replica::getId).filter(getActiveReplicasIds()::contains).collect(Collectors.toSet());
-
- if (!activeRemotePrimaryReplicas.isEmpty()) {
- //wait until all shutdown events come from all remote primary replicas
- synchronized (shuttingDownReplicaIds) {
- while (!shuttingDownReplicaIds.containsAll(activeRemotePrimaryReplicas)) {
- try {
- shuttingDownReplicaIds.wait();
- } catch (InterruptedException e) {
- Thread.currentThread().interrupt();
- }
- }
- }
- }
-
- LOGGER.log(Level.INFO, "Got shutdown notification from all remote replicas");
- //close replication channel
- asterixAppRuntimeContextProvider.getAppContext().getReplicationChannel().close();
-
- LOGGER.log(Level.INFO, "Replication manager stopped.");
- }
-
- @Override
- public void reportReplicaEvent(ReplicaEvent event) {
- replicaEventsQ.offer(event);
- }
-
- /**
- * Suspends replications and sends a remote replica failure event to ReplicasEventsMonitor.
- *
- * @param replicaId
- * the failed replica id.
- */
- public void reportFailedReplica(String replicaId) {
- Replica replica = replicas.get(replicaId);
- if (replica == null) {
- return;
- }
- if (replica.getState() == ReplicaState.DEAD) {
- return;
- }
-
- //need to stop processing any new logs or jobs
- terminateJobsReplication.set(true);
-
- ReplicaEvent event = new ReplicaEvent(replica, ClusterEventType.NODE_FAILURE);
- reportReplicaEvent(event);
- }
-
- private String getReplicaIdBySocket(SocketChannel socketChannel) {
- InetSocketAddress socketAddress = NetworkingUtil.getSocketAddress(socketChannel);
- for (Replica replica : replicas.values()) {
- if (replica.getClusterIp().equals(socketAddress.getHostName())
- && ncConfig.getReplicationPublicPort() == socketAddress.getPort()) {
- return replica.getId();
- }
- }
- return null;
- }
-
- @Override
- public void startReplicationThreads() throws InterruptedException {
- replicationJobsProcessor = new ReplicationJobsProccessor();
-
- //start/continue processing jobs/logs
- if (logsRepSockets == null) {
- establishTxnLogReplicationHandshake();
- getAndInitNewPage();
- txnlogReplicator = new TxnLogReplicator(emptyLogBuffersQ, pendingFlushLogBuffersQ);
- txnLogReplicatorTask = asterixAppRuntimeContextProvider.getThreadExecutor().submit(txnlogReplicator);
- }
-
- replicationJobsProcessor.start();
-
- if (!replicationMonitor.isAlive()) {
- replicationMonitor.start();
- }
-
- //notify any waiting threads that replication has been resumed
- synchronized (replicationSuspended) {
- LOGGER.log(Level.INFO, "Replication started/resumed");
- replicationSuspended.set(false);
- replicationSuspended.notifyAll();
- }
- }
-
- @Override
- public void requestFlushLaggingReplicaIndexes(long nonSharpCheckpointTargetLSN) throws IOException {
- long startLSN = logManager.getAppendLSN();
- Set<String> replicaIds = getActiveReplicasIds();
- if (replicaIds.isEmpty()) {
- return;
- }
- ByteBuffer requestBuffer = ByteBuffer.allocate(INITIAL_BUFFER_SIZE);
- for (String replicaId : replicaIds) {
- //1. identify replica indexes with LSN less than nonSharpCheckpointTargetLSN.
- Map<Long, DatasetResourceReference> laggingIndexes =
- replicaResourcesManager.getLaggingReplicaIndexesId2PathMap(replicaId, nonSharpCheckpointTargetLSN);
-
- if (!laggingIndexes.isEmpty()) {
- //2. send request to remote replicas that have lagging indexes.
- ReplicaIndexFlushRequest laggingIndexesResponse = null;
- try (SocketChannel socketChannel = getReplicaSocket(replicaId)) {
- ReplicaIndexFlushRequest laggingIndexesRequest =
- new ReplicaIndexFlushRequest(laggingIndexes.keySet());
- requestBuffer =
- ReplicationProtocol.writeGetReplicaIndexFlushRequest(requestBuffer, laggingIndexesRequest);
- NetworkingUtil.transferBufferToChannel(socketChannel, requestBuffer);
-
- //3. remote replicas will respond with indexes that were not flushed.
- ReplicationRequestType responseFunction = waitForResponse(socketChannel, requestBuffer);
-
- if (responseFunction == ReplicationRequestType.FLUSH_INDEX) {
- requestBuffer = ReplicationProtocol.readRequest(socketChannel, requestBuffer);
- //returning the indexes that were not flushed
- laggingIndexesResponse = ReplicationProtocol.readReplicaIndexFlushRequest(requestBuffer);
- }
- //send goodbye
- ReplicationProtocol.sendGoodbye(socketChannel);
- }
-
- /*
- * 4. update checkpoints for indexes that were not flushed
- * to the current append LSN to indicate no operations happened
- * since the checkpoint start.
- */
- if (laggingIndexesResponse != null) {
- for (Long resouceId : laggingIndexesResponse.getLaggingRescouresIds()) {
- final DatasetResourceReference datasetResourceReference = laggingIndexes.get(resouceId);
- indexCheckpointManagerProvider.get(datasetResourceReference).advanceLowWatermark(startLSN);
- }
- }
- }
- }
- }
-
- //Recovery Method
- @Override
- public long getMaxRemoteLSN(Set<String> remoteReplicas) throws IOException {
- long maxRemoteLSN = 0;
-
- ReplicationProtocol.writeGetReplicaMaxLSNRequest(dataBuffer);
- Map<String, SocketChannel> replicaSockets = new HashMap<>();
- try {
- for (String replicaId : remoteReplicas) {
- replicaSockets.put(replicaId, getReplicaSocket(replicaId));
- }
-
- //send request
- Iterator<Map.Entry<String, SocketChannel>> iterator = replicaSockets.entrySet().iterator();
- while (iterator.hasNext()) {
- Entry<String, SocketChannel> replicaSocket = iterator.next();
- SocketChannel clientSocket = replicaSocket.getValue();
- NetworkingUtil.transferBufferToChannel(clientSocket, dataBuffer);
- dataBuffer.position(0);
- }
-
- iterator = replicaSockets.entrySet().iterator();
- while (iterator.hasNext()) {
- Entry<String, SocketChannel> replicaSocket = iterator.next();
- SocketChannel clientSocket = replicaSocket.getValue();
- //read response
- NetworkingUtil.readBytes(clientSocket, dataBuffer, Long.BYTES);
- maxRemoteLSN = Math.max(maxRemoteLSN, dataBuffer.getLong());
- }
- } finally {
- closeReplicaSockets(replicaSockets);
- }
-
- return maxRemoteLSN;
- }
-
- //Recovery Method
- @Override
- public void requestReplicaFiles(String selectedReplicaId, Set<Integer> partitionsToRecover,
- Set<String> existingFiles) throws IOException {
- ReplicaFilesRequest request = new ReplicaFilesRequest(partitionsToRecover, existingFiles);
- dataBuffer = ReplicationProtocol.writeGetReplicaFilesRequest(dataBuffer, request);
-
- try (SocketChannel socketChannel = getReplicaSocket(selectedReplicaId)) {
-
- //transfer request
- NetworkingUtil.transferBufferToChannel(socketChannel, dataBuffer);
-
- String indexPath;
- String destFilePath;
- ReplicationRequestType responseFunction = ReplicationProtocol.getRequestType(socketChannel, dataBuffer);
- LSMIndexFileProperties fileProperties;
- while (responseFunction != ReplicationRequestType.GOODBYE) {
- dataBuffer = ReplicationProtocol.readRequest(socketChannel, dataBuffer);
-
- fileProperties = ReplicationProtocol.readFileReplicationRequest(dataBuffer);
-
- //get index path
- indexPath = replicaResourcesManager.getIndexPath(fileProperties);
- destFilePath = indexPath + File.separator + fileProperties.getFileName();
-
- //create file
- File destFile = new File(destFilePath);
- destFile.createNewFile();
-
- try (RandomAccessFile fileOutputStream = new RandomAccessFile(destFile, "rw");
- FileChannel fileChannel = fileOutputStream.getChannel();) {
- fileOutputStream.setLength(fileProperties.getFileSize());
-
- NetworkingUtil.downloadFile(fileChannel, socketChannel);
- fileChannel.force(true);
- }
-
- //we need to create initial map for .metadata files that belong to remote replicas
- if (!fileProperties.isLSMComponentFile() && !fileProperties.getNodeId().equals(nodeId)) {
- final ResourceReference indexRef = ResourceReference.of(destFile.getAbsolutePath());
- indexCheckpointManagerProvider.get(indexRef).init(logManager.getAppendLSN());
- }
- responseFunction = ReplicationProtocol.getRequestType(socketChannel, dataBuffer);
- }
-
- //send goodbye
- ReplicationProtocol.sendGoodbye(socketChannel);
- }
- }
-
- public int getLogPageSize() {
- return replicationProperties.getLogBufferPageSize();
- }
-
- @Override
- public void replicateTxnLogBatch(final ByteBuffer buffer) {
- //if replication is suspended, wait until it is resumed
- try {
- while (replicationSuspended.get()) {
- synchronized (replicationSuspended) {
- replicationSuspended.wait();
- }
- }
- } catch (InterruptedException e) {
- Thread.currentThread().interrupt();
- }
- //prepare the batch size buffer
- txnLogsBatchSizeBuffer.clear();
- txnLogsBatchSizeBuffer.putInt(buffer.remaining());
- txnLogsBatchSizeBuffer.flip();
-
- buffer.mark();
- for (SocketChannel replicaSocket : logsRepSockets) {
- try {
- //send batch size
- NetworkingUtil.transferBufferToChannel(replicaSocket, txnLogsBatchSizeBuffer);
- //send log
- NetworkingUtil.transferBufferToChannel(replicaSocket, buffer);
- } catch (IOException e) {
- handleReplicationFailure(replicaSocket, e);
- } finally {
- txnLogsBatchSizeBuffer.position(0);
- buffer.reset();
- }
- }
- //move the bufeer position to the sent limit
- buffer.position(buffer.limit());
+ this.appCtx = appCtx;
+ strategy = ReplicationStrategyFactory.create(replicationProperties.getReplicationStrategy());
+ logReplicationManager = new LogReplicationManager(appCtx, this);
+ lsnIndexReplicationManager = new IndexReplicationManager(appCtx, this);
}
@Override
public void register(IPartitionReplica replica) {
- // find the replica node based on ip and replication port
- Optional<Replica> replicaNode = replicationStrategy.getRemoteReplicasAndSelf(nodeId).stream()
- .filter(node -> node.getClusterIp().equals(replica.getIdentifier().getLocation().getHostString())
- && node.getPort() == replica.getIdentifier().getLocation().getPort())
- .findAny();
- if (!replicaNode.isPresent()) {
- throw new IllegalStateException("Couldn't find node for replica: " + replica);
+ synchronized (dests) {
+ final InetSocketAddress location = replica.getIdentifier().getLocation();
+ final ReplicationDestination replicationDest = dests.computeIfAbsent(location, ReplicationDestination::at);
+ replicationDest.add(replica);
+ logReplicationManager.register(replicationDest);
+ lsnIndexReplicationManager.register(replicationDest);
}
- Replica replicaRef = replicaNode.get();
- final String replicaId = replicaRef.getId();
- replicas.putIfAbsent(replicaId, replicaRef);
- replica2PartitionsMap.computeIfAbsent(replicaId, k -> new HashSet<>());
- replica2PartitionsMap.get(replicaId).add(replica.getIdentifier().getPartition());
- updateReplicaInfo(replicaRef);
- checkReplicaState(replicaId, false, true);
}
- //supporting classes
- /**
- * This class is responsible for processing replica events.
- */
- private class ReplicasEventsMonitor extends Thread {
- ReplicaEvent event;
-
- @Override
- public void run() {
- while (true) {
- try {
- event = replicaEventsQ.take();
-
- switch (event.getEventType()) {
- case NODE_FAILURE:
- handleReplicaFailure(event.getReplica().getId());
- break;
- case NODE_JOIN:
- updateReplicaInfo(event.getReplica());
- checkReplicaState(event.getReplica().getId(), false, true);
- break;
- case NODE_SHUTTING_DOWN:
- handleShutdownEvent(event.getReplica().getId());
- break;
- default:
- break;
- }
- } catch (InterruptedException e) {
- Thread.currentThread().interrupt();
- }
- }
- }
-
- public void handleReplicaFailure(String replicaId) throws InterruptedException {
- Replica replica = replicas.get(replicaId);
-
- if (replica.getState() == ReplicaState.DEAD) {
+ @Override
+ public void unregister(IPartitionReplica replica) {
+ synchronized (dests) {
+ final InetSocketAddress location = replica.getIdentifier().getLocation();
+ final ReplicationDestination dest = dests.get(location);
+ if (dest == null) {
+ LOGGER.warn(() -> "Asked to unregister unknown replica " + replica);
return;
}
-
- updateReplicaState(replicaId, ReplicaState.DEAD, true);
-
- //delete any invalid LSMComponents for this replica
- replicaResourcesManager.cleanInvalidLSMComponents(replicaId);
- }
-
- public void handleShutdownEvent(String replicaId) {
- synchronized (shuttingDownReplicaIds) {
- shuttingDownReplicaIds.add(replicaId);
- shuttingDownReplicaIds.notifyAll();
- }
- }
- }
-
- /**
- * This class process is responsible for processing ASYNC replication job.
- */
- private class ReplicationJobsProccessor extends Thread {
- Map<String, SocketChannel> replicaSockets;
- ByteBuffer reusableBuffer = ByteBuffer.allocate(INITIAL_BUFFER_SIZE);
-
- @Override
- public void run() {
- Thread.currentThread().setName("ReplicationJobsProccessor Thread");
- terminateJobsReplication.set(false);
- jobsReplicationSuspended.set(false);
-
- while (true) {
- try {
- if (terminateJobsReplication.get()) {
- closeSockets();
- break;
- }
-
- IReplicationJob job = replicationJobsQ.take();
- if (job == 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.isEmpty()) {
- LOGGER.log(Level.INFO, "No pending replication jobs. Closing connections to replicas");
- closeSockets();
- }
- } catch (InterruptedException e) {
- Thread.currentThread().interrupt();
- } catch (IOException e) {
- LOGGER.warn("Couldn't complete processing replication job", e);
- }
- }
-
- synchronized (jobsReplicationSuspended) {
- jobsReplicationSuspended.set(true);
- jobsReplicationSuspended.notifyAll();
- }
- LOGGER.log(Level.INFO, "ReplicationJobsProccessor stopped. ");
- }
-
- private void closeSockets() {
- if (replicaSockets != null) {
- closeReplicaSockets(replicaSockets);
- replicaSockets.clear();
- replicaSockets = null;
- }
- }
- }
-
- /**
- * This class is responsible for listening on sockets that belong to TxnLogsReplicator.
- */
- private class TxnLogsReplicationResponseListener implements Runnable {
- final SocketChannel replicaSocket;
- final String replicaId;
-
- public TxnLogsReplicationResponseListener(String replicaId, SocketChannel replicaSocket) {
- this.replicaId = replicaId;
- this.replicaSocket = replicaSocket;
- }
-
- @Override
- public void run() {
- Thread.currentThread().setName("TxnLogs Replication Listener Thread");
- LOGGER.log(Level.INFO, "Started listening on socket: " + replicaSocket.socket().getRemoteSocketAddress());
-
- try (BufferedReader incomingResponse =
- new BufferedReader(new InputStreamReader(replicaSocket.socket().getInputStream()))) {
- while (true) {
- String responseLine = incomingResponse.readLine();
- if (responseLine == null) {
- break;
- }
- //read ACK for job commit log
- String ackFrom = ReplicationProtocol.getNodeIdFromLogAckMessage(responseLine);
- int jobId = ReplicationProtocol.getJobIdFromLogAckMessage(responseLine);
- addAckToJob(jobId, ackFrom);
- }
- } catch (AsynchronousCloseException e) {
- if (LOGGER.isInfoEnabled()) {
- LOGGER.log(Level.INFO, "Replication listener stopped for remote replica: " + replicaId, e);
- }
- } catch (IOException e) {
- handleReplicationFailure(replicaSocket, e);
+ LOGGER.info(() -> "unregister " + replica);
+ dest.remove(replica);
+ if (dest.getReplicas().isEmpty()) {
+ LOGGER.info(() -> "Removing destination with no replicas " + dest);
+ logReplicationManager.unregister(dest);
+ lsnIndexReplicationManager.unregister(dest);
+ dests.remove(location);
}
}
}
@Override
- public IReplicationStrategy getReplicationStrategy() {
- return replicationStrategy;
+ public void notifyFailure(IReplicationDestination dest, Exception failure) {
+ LOGGER.info(() -> "processing failure for " + dest);
+ appCtx.getThreadExecutor().execute(() -> {
+ logReplicationManager.unregister(dest);
+ lsnIndexReplicationManager.unregister(dest);
+ dest.notifyFailure(failure);
+ });
}
-}
+
+ @Override
+ public void replicate(ILogRecord logRecord) throws InterruptedException {
+ logReplicationManager.replicate(logRecord);
+ }
+
+ @Override
+ public IReplicationStrategy getReplicationStrategy() {
+ return strategy;
+ }
+
+ @Override
+ public void submitJob(IReplicationJob job) {
+ lsnIndexReplicationManager.accept(job);
+ }
+
+ @Override
+ public boolean isReplicationEnabled() {
+ return replicationProperties.isReplicationEnabled();
+ }
+
+ @Override
+ public void start() {
+ // no op
+ }
+
+ @Override
+ public void dumpState(OutputStream os) {
+ // no op
+ }
+
+ @Override
+ public void stop(boolean dumpState, OutputStream ouputStream) throws IOException {
+ LOGGER.info("Closing replication channel");
+ appCtx.getReplicationChannel().close();
+ LOGGER.info("Replication manager stopped");
+ }
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/CheckpointPartitionIndexesTask.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/CheckpointPartitionIndexesTask.java
index 2c1937b..84922cd 100644
--- a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/CheckpointPartitionIndexesTask.java
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/CheckpointPartitionIndexesTask.java
@@ -25,12 +25,11 @@
import java.util.Collection;
import org.apache.asterix.common.api.INcApplicationContext;
-import org.apache.asterix.common.replication.IReplicationThread;
+import org.apache.asterix.replication.api.IReplicationWorker;
import org.apache.asterix.common.storage.DatasetResourceReference;
import org.apache.asterix.common.storage.IIndexCheckpointManager;
import org.apache.asterix.common.storage.IIndexCheckpointManagerProvider;
import org.apache.asterix.replication.api.IReplicaTask;
-import org.apache.asterix.replication.functions.ReplicationProtocol;
import org.apache.asterix.transaction.management.resource.PersistentLocalResourceRepository;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.storage.common.LocalResource;
@@ -47,7 +46,7 @@
}
@Override
- public void perform(INcApplicationContext appCtx, IReplicationThread worker) throws HyracksDataException {
+ public void perform(INcApplicationContext appCtx, IReplicationWorker worker) throws HyracksDataException {
final IIndexCheckpointManagerProvider indexCheckpointManagerProvider =
appCtx.getIndexCheckpointManagerProvider();
PersistentLocalResourceRepository resRepo =
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/ComponentMaskTask.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/ComponentMaskTask.java
new file mode 100644
index 0000000..26c9577
--- /dev/null
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/ComponentMaskTask.java
@@ -0,0 +1,92 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.replication.messaging;
+
+import java.io.DataInput;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+
+import org.apache.asterix.common.api.INcApplicationContext;
+import org.apache.asterix.common.exceptions.ReplicationException;
+import org.apache.asterix.replication.api.IReplicationWorker;
+import org.apache.asterix.common.utils.StorageConstants;
+import org.apache.asterix.replication.api.IReplicaTask;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.FileReference;
+import org.apache.hyracks.api.io.IIOManager;
+
+/**
+ * A task to create a mask file for an incoming lsm component from master
+ */
+public class ComponentMaskTask implements IReplicaTask {
+
+ private static final String COMPONENT_MASK_FILE_PREFIX = StorageConstants.MASK_FILE_PREFIX + "C_";
+ private final String file;
+ private final String componentId;
+
+ public ComponentMaskTask(String file, String componentId) {
+ this.file = file;
+ this.componentId = componentId;
+ }
+
+ @Override
+ public void perform(INcApplicationContext appCtx, IReplicationWorker worker) {
+ try {
+ // create mask
+ final Path maskPath = getComponentMaskPath(appCtx, file);
+ Files.createFile(maskPath);
+ ReplicationProtocol.sendAck(worker.getChannel(), worker.getReusableBuffer());
+ } catch (IOException e) {
+ throw new ReplicationException(e);
+ }
+ }
+
+ public static Path getComponentMaskPath(INcApplicationContext appCtx, String file) throws IOException {
+ final IIOManager ioManager = appCtx.getIoManager();
+ final FileReference localPath = ioManager.resolve(file);
+ final Path resourceDir = Files.createDirectories(localPath.getFile().getParentFile().toPath());
+ return Paths.get(resourceDir.toString(), COMPONENT_MASK_FILE_PREFIX + localPath.getFile().getName());
+ }
+
+ @Override
+ public ReplicationProtocol.ReplicationRequestType getMessageType() {
+ return ReplicationProtocol.ReplicationRequestType.LSM_COMPONENT_MASK;
+ }
+
+ @Override
+ public void serialize(OutputStream out) throws HyracksDataException {
+ try {
+ final DataOutputStream dos = new DataOutputStream(out);
+ dos.writeUTF(file);
+ dos.writeUTF(componentId);
+ } catch (IOException e) {
+ throw HyracksDataException.create(e);
+ }
+ }
+
+ public static ComponentMaskTask create(DataInput input) throws IOException {
+ String indexFile = input.readUTF();
+ String componentId = input.readUTF();
+ return new ComponentMaskTask(indexFile, componentId);
+ }
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/DeleteFileTask.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/DeleteFileTask.java
index d4de3b7..1b5470d 100644
--- a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/DeleteFileTask.java
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/DeleteFileTask.java
@@ -27,9 +27,8 @@
import org.apache.asterix.common.api.INcApplicationContext;
import org.apache.asterix.common.exceptions.ReplicationException;
-import org.apache.asterix.common.replication.IReplicationThread;
+import org.apache.asterix.replication.api.IReplicationWorker;
import org.apache.asterix.replication.api.IReplicaTask;
-import org.apache.asterix.replication.functions.ReplicationProtocol;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.api.io.IIOManager;
import org.apache.logging.log4j.LogManager;
@@ -48,7 +47,7 @@
}
@Override
- public void perform(INcApplicationContext appCtx, IReplicationThread worker) {
+ public void perform(INcApplicationContext appCtx, IReplicationWorker worker) {
try {
final IIOManager ioManager = appCtx.getIoManager();
final File localFile = ioManager.resolve(file).getFile();
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/DropIndexTask.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/DropIndexTask.java
new file mode 100644
index 0000000..b7f0985
--- /dev/null
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/DropIndexTask.java
@@ -0,0 +1,84 @@
+/*
+ * 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.messaging;
+
+import java.io.DataInput;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.logging.Logger;
+
+import org.apache.asterix.common.api.INcApplicationContext;
+import org.apache.asterix.common.exceptions.ReplicationException;
+import org.apache.asterix.replication.api.IReplicationWorker;
+import org.apache.asterix.replication.api.IReplicaTask;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.IIOManager;
+import org.apache.hyracks.api.util.IoUtil;
+
+/**
+ * A task to drop an index that was dropped on master
+ */
+public class DropIndexTask implements IReplicaTask {
+
+ private static final Logger LOGGER = Logger.getLogger(DeleteFileTask.class.getName());
+ private final String file;
+
+ public DropIndexTask(String file) {
+ this.file = file;
+ }
+
+ @Override
+ public void perform(INcApplicationContext appCtx, IReplicationWorker worker) {
+ try {
+ final IIOManager ioManager = appCtx.getIoManager();
+ final File indexFile = ioManager.resolve(file).getFile();
+ if (indexFile.exists()) {
+ File indexDir = indexFile.getParentFile();
+ IoUtil.deleteDirectory(indexDir);
+ LOGGER.info(() -> "Deleted index: " + indexFile.getAbsolutePath());
+ } else {
+ LOGGER.warning(() -> "Requested to delete a non-existing index: " + indexFile.getAbsolutePath());
+ }
+ ReplicationProtocol.sendAck(worker.getChannel(), worker.getReusableBuffer());
+ } catch (IOException e) {
+ throw new ReplicationException(e);
+ }
+ }
+
+ @Override
+ public ReplicationProtocol.ReplicationRequestType getMessageType() {
+ return ReplicationProtocol.ReplicationRequestType.REPLICATE_RESOURCE_FILE;
+ }
+
+ @Override
+ public void serialize(OutputStream out) throws HyracksDataException {
+ try {
+ DataOutputStream dos = new DataOutputStream(out);
+ dos.writeUTF(file);
+ } catch (IOException e) {
+ throw HyracksDataException.create(e);
+ }
+ }
+
+ public static DropIndexTask create(DataInput input) throws IOException {
+ return new DropIndexTask(input.readUTF());
+ }
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/MarkComponentValidTask.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/MarkComponentValidTask.java
new file mode 100644
index 0000000..b581321
--- /dev/null
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/MarkComponentValidTask.java
@@ -0,0 +1,110 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.replication.messaging;
+
+import java.io.DataInput;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+import org.apache.asterix.common.api.INcApplicationContext;
+import org.apache.asterix.common.exceptions.ReplicationException;
+import org.apache.asterix.replication.api.IReplicationWorker;
+import org.apache.asterix.common.storage.IIndexCheckpointManager;
+import org.apache.asterix.common.storage.IIndexCheckpointManagerProvider;
+import org.apache.asterix.common.storage.ResourceReference;
+import org.apache.asterix.replication.api.IReplicaTask;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndexFileManager;
+
+/**
+ * A task to mark a replicated LSM component as valid
+ */
+public class MarkComponentValidTask implements IReplicaTask {
+
+ private final long masterLsn;
+ private final String file;
+
+ public MarkComponentValidTask(String file, long masterLsn) {
+ this.file = file;
+ this.masterLsn = masterLsn;
+ }
+
+ @Override
+ public void perform(INcApplicationContext appCtx, IReplicationWorker worker) {
+ try {
+ if (masterLsn > 0) {
+ ensureComponentLsnFlushed(appCtx);
+ }
+ // delete mask
+ final Path maskPath = ComponentMaskTask.getComponentMaskPath(appCtx, file);
+ Files.delete(maskPath);
+ ReplicationProtocol.sendAck(worker.getChannel(), worker.getReusableBuffer());
+ } catch (IOException | InterruptedException e) {
+ throw new ReplicationException(e);
+ }
+ }
+
+ private void ensureComponentLsnFlushed(INcApplicationContext appCtx)
+ throws HyracksDataException, InterruptedException {
+ final ResourceReference indexRef = ResourceReference.of(file);
+ final IIndexCheckpointManagerProvider checkpointManagerProvider = appCtx.getIndexCheckpointManagerProvider();
+ final IIndexCheckpointManager indexCheckpointManager = checkpointManagerProvider.get(indexRef);
+ long replicationTimeOut = TimeUnit.SECONDS.toMillis(appCtx.getReplicationProperties().getReplicationTimeOut());
+ final long startTime = System.nanoTime();
+ synchronized (indexCheckpointManager) {
+ // wait until the lsn mapping is flushed to disk
+ while (!indexCheckpointManager.isFlushed(masterLsn)) {
+ if (replicationTimeOut <= 0) {
+ throw new ReplicationException(new TimeoutException("Couldn't receive flush lsn from master"));
+ }
+ indexCheckpointManager.wait(replicationTimeOut);
+ replicationTimeOut -= TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTime);
+ }
+ final String componentEndTime = AbstractLSMIndexFileManager.getComponentEndTime(indexRef.getName());
+ indexCheckpointManager.replicated(componentEndTime, masterLsn);
+ }
+ }
+
+ @Override
+ public ReplicationProtocol.ReplicationRequestType getMessageType() {
+ return ReplicationProtocol.ReplicationRequestType.MARK_COMPONENT_VALID;
+ }
+
+ @Override
+ public void serialize(OutputStream out) throws HyracksDataException {
+ try {
+ final DataOutputStream dos = new DataOutputStream(out);
+ dos.writeUTF(file);
+ dos.writeLong(masterLsn);
+ } catch (IOException e) {
+ throw HyracksDataException.create(e);
+ }
+ }
+
+ public static MarkComponentValidTask create(DataInput input) throws IOException {
+ final String indexFile = input.readUTF();
+ final long lsn = input.readLong();
+ return new MarkComponentValidTask(indexFile, lsn);
+ }
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/PartitionResourcesListResponse.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/PartitionResourcesListResponse.java
index 85b7bb9..561a6bf 100644
--- a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/PartitionResourcesListResponse.java
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/PartitionResourcesListResponse.java
@@ -26,7 +26,6 @@
import java.util.List;
import org.apache.asterix.replication.api.IReplicationMessage;
-import org.apache.asterix.replication.functions.ReplicationProtocol;
import org.apache.hyracks.api.exceptions.HyracksDataException;
public class PartitionResourcesListResponse implements IReplicationMessage {
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/PartitionResourcesListTask.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/PartitionResourcesListTask.java
index b2b8ac6..b972f32 100644
--- a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/PartitionResourcesListTask.java
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/PartitionResourcesListTask.java
@@ -26,10 +26,10 @@
import java.util.stream.Collectors;
import org.apache.asterix.common.api.INcApplicationContext;
-import org.apache.asterix.common.replication.IReplicationThread;
+import org.apache.asterix.replication.api.IReplicationWorker;
import org.apache.asterix.common.utils.StoragePathUtil;
import org.apache.asterix.replication.api.IReplicaTask;
-import org.apache.asterix.replication.functions.ReplicationProtocol;
+import org.apache.asterix.transaction.management.resource.PersistentLocalResourceRepository;
import org.apache.hyracks.api.exceptions.HyracksDataException;
/**
@@ -44,11 +44,12 @@
}
@Override
- public void perform(INcApplicationContext appCtx, IReplicationThread worker) throws HyracksDataException {
+ public void perform(INcApplicationContext appCtx, IReplicationWorker worker) throws HyracksDataException {
//TODO delete any invalid files with masks
- final List<String> partitionResources =
- appCtx.getReplicaResourcesManager().getPartitionIndexesFiles(partition, false).stream()
- .map(StoragePathUtil::getFileRelativePath).collect(Collectors.toList());
+ final PersistentLocalResourceRepository localResourceRepository =
+ (PersistentLocalResourceRepository) appCtx.getLocalResourceRepository();
+ final List<String> partitionResources = localResourceRepository.getPartitionIndexesFiles(partition).stream()
+ .map(StoragePathUtil::getFileRelativePath).collect(Collectors.toList());
final PartitionResourcesListResponse response =
new PartitionResourcesListResponse(partition, partitionResources);
ReplicationProtocol.sendTo(worker.getChannel(), response, worker.getReusableBuffer());
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/ReplicateFileTask.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/ReplicateFileTask.java
index 45d8971..99c7256 100644
--- a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/ReplicateFileTask.java
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/ReplicateFileTask.java
@@ -30,10 +30,12 @@
import org.apache.asterix.common.api.INcApplicationContext;
import org.apache.asterix.common.exceptions.ReplicationException;
-import org.apache.asterix.common.replication.IReplicationThread;
+import org.apache.asterix.replication.api.IReplicationWorker;
+import org.apache.asterix.common.storage.IIndexCheckpointManager;
+import org.apache.asterix.common.storage.IIndexCheckpointManagerProvider;
+import org.apache.asterix.common.storage.ResourceReference;
import org.apache.asterix.common.utils.StorageConstants;
import org.apache.asterix.replication.api.IReplicaTask;
-import org.apache.asterix.replication.functions.ReplicationProtocol;
import org.apache.asterix.replication.management.NetworkingUtil;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.api.io.FileReference;
@@ -49,14 +51,16 @@
private static final Logger LOGGER = LogManager.getLogger();
private final String file;
private final long size;
+ private final boolean indexMetadata;
- public ReplicateFileTask(String file, long size) {
+ public ReplicateFileTask(String file, long size, boolean indexMetadata) {
this.file = file;
this.size = size;
+ this.indexMetadata = indexMetadata;
}
@Override
- public void perform(INcApplicationContext appCtx, IReplicationThread worker) throws HyracksDataException {
+ public void perform(INcApplicationContext appCtx, IReplicationWorker worker) {
try {
final IIOManager ioManager = appCtx.getIoManager();
// resolve path
@@ -76,6 +80,9 @@
NetworkingUtil.downloadFile(fileChannel, worker.getChannel());
fileChannel.force(true);
}
+ if (indexMetadata) {
+ initIndexCheckpoint(appCtx);
+ }
//delete mask
Files.delete(maskPath);
LOGGER.info(() -> "Replicated file: " + localPath);
@@ -85,6 +92,16 @@
}
}
+ private void initIndexCheckpoint(INcApplicationContext appCtx) throws HyracksDataException {
+ final ResourceReference indexRef = ResourceReference.of(file);
+ final IIndexCheckpointManagerProvider checkpointManagerProvider = appCtx.getIndexCheckpointManagerProvider();
+ final IIndexCheckpointManager indexCheckpointManager = checkpointManagerProvider.get(indexRef);
+ final long currentLSN = appCtx.getTransactionSubsystem().getLogManager().getAppendLSN();
+ indexCheckpointManager.delete();
+ indexCheckpointManager.init(currentLSN);
+ LOGGER.info(() -> "Checkpoint index: " + indexRef);
+ }
+
@Override
public ReplicationProtocol.ReplicationRequestType getMessageType() {
return ReplicationProtocol.ReplicationRequestType.REPLICATE_RESOURCE_FILE;
@@ -96,6 +113,7 @@
DataOutputStream dos = new DataOutputStream(out);
dos.writeUTF(file);
dos.writeLong(size);
+ dos.writeBoolean(indexMetadata);
} catch (IOException e) {
throw HyracksDataException.create(e);
}
@@ -104,6 +122,7 @@
public static ReplicateFileTask create(DataInput input) throws IOException {
final String s = input.readUTF();
final long i = input.readLong();
- return new ReplicateFileTask(s, i);
+ final boolean isMetadata = input.readBoolean();
+ return new ReplicateFileTask(s, i, isMetadata);
}
}
\ No newline at end of file
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/ReplicateLogsTask.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/ReplicateLogsTask.java
new file mode 100644
index 0000000..b71f4b8
--- /dev/null
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/ReplicateLogsTask.java
@@ -0,0 +1,92 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.replication.messaging;
+
+import java.io.DataInput;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.ByteBuffer;
+import java.nio.channels.SocketChannel;
+
+import org.apache.asterix.common.api.INcApplicationContext;
+import org.apache.asterix.common.exceptions.ReplicationException;
+import org.apache.asterix.replication.api.IReplicationWorker;
+import org.apache.asterix.common.transactions.ILogManager;
+import org.apache.asterix.replication.api.IReplicaTask;
+import org.apache.asterix.replication.logging.RemoteLogRecord;
+import org.apache.asterix.replication.logging.RemoteLogsProcessor;
+import org.apache.asterix.replication.management.ReplicationChannel;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+/**
+ * A task to replicate transaction logs from master replica
+ */
+public class ReplicateLogsTask implements IReplicaTask {
+
+ public static final int END_REPLICATION_LOG_SIZE = 1;
+ private final String nodeId;
+
+ public ReplicateLogsTask(String nodeId) {
+ this.nodeId = nodeId;
+ }
+
+ @Override
+ public void perform(INcApplicationContext appCtx, IReplicationWorker worker) {
+ final ReplicationChannel replicationChannel = (ReplicationChannel) appCtx.getReplicationChannel();
+ final RemoteLogsProcessor logsProcessor = replicationChannel.getRemoteLogsProcessor();
+ final ILogManager logManager = appCtx.getTransactionSubsystem().getLogManager();
+ final RemoteLogRecord reusableLog = new RemoteLogRecord();
+ final SocketChannel channel = worker.getChannel();
+ ByteBuffer logsBuffer = ByteBuffer.allocate(logManager.getLogPageSize());
+ try {
+ while (true) {
+ // read a batch of logs
+ logsBuffer = ReplicationProtocol.readRequest(channel, logsBuffer);
+ // check if it is end of handshake
+ if (logsBuffer.remaining() == END_REPLICATION_LOG_SIZE) {
+ break;
+ }
+ logsProcessor.process(logsBuffer, reusableLog, worker);
+ }
+ } catch (IOException e) {
+ throw new ReplicationException(e);
+ }
+ }
+
+ @Override
+ public ReplicationProtocol.ReplicationRequestType getMessageType() {
+ return ReplicationProtocol.ReplicationRequestType.REPLICATE_LOGS;
+ }
+
+ @Override
+ public void serialize(OutputStream out) throws HyracksDataException {
+ try {
+ DataOutputStream dos = new DataOutputStream(out);
+ dos.writeUTF(nodeId);
+ } catch (IOException e) {
+ throw HyracksDataException.create(e);
+ }
+ }
+
+ public static ReplicateLogsTask create(DataInput input) throws IOException {
+ final String node = input.readUTF();
+ return new ReplicateLogsTask(node);
+ }
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/ReplicationProtocol.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/ReplicationProtocol.java
new file mode 100644
index 0000000..280a2d4
--- /dev/null
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/ReplicationProtocol.java
@@ -0,0 +1,200 @@
+/*
+ * 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.messaging;
+
+import java.io.ByteArrayInputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.SocketChannel;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.stream.Stream;
+
+import org.apache.asterix.common.exceptions.ReplicationException;
+import org.apache.asterix.replication.api.IReplicationMessage;
+import org.apache.asterix.replication.api.PartitionReplica;
+import org.apache.asterix.replication.management.NetworkingUtil;
+import org.apache.hyracks.data.std.util.ExtendedByteArrayOutputStream;
+import org.apache.hyracks.util.StorageUtil;
+
+public class ReplicationProtocol {
+
+ /**
+ * All replication messages start with ReplicationRequestType (4 bytes), then the length of the request in bytes
+ */
+ public static final String LOG_REPLICATION_ACK = "$";
+ public static final int INITIAL_BUFFER_SIZE = StorageUtil.getIntSizeInBytes(4, StorageUtil.StorageUnit.KILOBYTE);
+ private static final int REPLICATION_REQUEST_TYPE_SIZE = Integer.BYTES;
+ private static final int REPLICATION_REQUEST_HEADER_SIZE = REPLICATION_REQUEST_TYPE_SIZE + Integer.BYTES;
+
+ public enum ReplicationRequestType {
+ GOODBYE,
+ ACK,
+ PARTITION_RESOURCES_REQUEST,
+ PARTITION_RESOURCES_RESPONSE,
+ REPLICATE_RESOURCE_FILE,
+ DELETE_RESOURCE_FILE,
+ CHECKPOINT_PARTITION,
+ LSM_COMPONENT_MASK,
+ MARK_COMPONENT_VALID,
+ DROP_INDEX,
+ REPLICATE_LOGS
+ }
+
+ private static final Map<Integer, ReplicationRequestType> TYPES = new HashMap<>();
+
+ static {
+ Stream.of(ReplicationRequestType.values()).forEach(type -> TYPES.put(type.ordinal(), type));
+ }
+
+ public static ByteBuffer readRequest(SocketChannel socketChannel, ByteBuffer dataBuffer) throws IOException {
+ // read request size
+ NetworkingUtil.readBytes(socketChannel, dataBuffer, Integer.BYTES);
+ final int requestSize = dataBuffer.getInt();
+ final ByteBuffer buf = ensureSize(dataBuffer, requestSize);
+ // read request
+ NetworkingUtil.readBytes(socketChannel, buf, requestSize);
+ return dataBuffer;
+ }
+
+ public static ReplicationRequestType getRequestType(SocketChannel socketChannel, ByteBuffer byteBuffer)
+ throws IOException {
+ // read replication request type
+ NetworkingUtil.readBytes(socketChannel, byteBuffer, REPLICATION_REQUEST_TYPE_SIZE);
+ return TYPES.get(byteBuffer.getInt());
+ }
+
+ private static ByteBuffer getGoodbyeBuffer() {
+ ByteBuffer bb = ByteBuffer.allocate(REPLICATION_REQUEST_TYPE_SIZE);
+ bb.putInt(ReplicationRequestType.GOODBYE.ordinal());
+ bb.flip();
+ return bb;
+ }
+
+ public static int getTxnIdFromLogAckMessage(String msg) {
+ return Integer.parseInt(msg.substring(msg.indexOf(LOG_REPLICATION_ACK) + 1));
+ }
+
+ public static void sendGoodbye(SocketChannel socketChannel) throws IOException {
+ ByteBuffer goodbyeBuffer = ReplicationProtocol.getGoodbyeBuffer();
+ NetworkingUtil.transferBufferToChannel(socketChannel, goodbyeBuffer);
+ }
+
+ public static void sendAck(SocketChannel socketChannel, ByteBuffer buf) {
+ try {
+ buf.clear();
+ buf.putInt(ReplicationRequestType.ACK.ordinal());
+ buf.flip();
+ NetworkingUtil.transferBufferToChannel(socketChannel, buf);
+ } catch (IOException e) {
+ throw new ReplicationException(e);
+ }
+ }
+
+ public static void waitForAck(PartitionReplica replica) throws IOException {
+ final SocketChannel channel = replica.getChannel();
+ final ByteBuffer buf = replica.getReusableBuffer();
+ ReplicationRequestType responseFunction = ReplicationProtocol.getRequestType(channel, buf);
+ if (responseFunction != ReplicationRequestType.ACK) {
+ throw new IllegalStateException("Unexpected response while waiting for ack.");
+ }
+ }
+
+ public static void sendTo(PartitionReplica replica, IReplicationMessage task) {
+ final SocketChannel channel = replica.getChannel();
+ final ByteBuffer buf = replica.getReusableBuffer();
+ sendTo(channel, task, buf);
+ }
+
+ public static void sendTo(SocketChannel channel, IReplicationMessage task, ByteBuffer buf) {
+ ExtendedByteArrayOutputStream outputStream = new ExtendedByteArrayOutputStream();
+ try (DataOutputStream oos = new DataOutputStream(outputStream)) {
+ task.serialize(oos);
+ final int requestSize = REPLICATION_REQUEST_HEADER_SIZE + oos.size();
+ final ByteBuffer requestBuffer = ensureSize(buf, requestSize);
+ requestBuffer.putInt(task.getMessageType().ordinal());
+ requestBuffer.putInt(oos.size());
+ requestBuffer.put(outputStream.getByteArray(), 0, outputStream.getLength());
+ requestBuffer.flip();
+ NetworkingUtil.transferBufferToChannel(channel, requestBuffer);
+ } catch (IOException e) {
+ throw new ReplicationException(e);
+ }
+ }
+
+ public static IReplicationMessage read(SocketChannel socketChannel, ByteBuffer buffer) throws IOException {
+ final ReplicationRequestType type = getRequestType(socketChannel, buffer);
+ return readMessage(type, socketChannel, buffer);
+ }
+
+ public static IReplicationMessage readMessage(ReplicationRequestType type, SocketChannel socketChannel,
+ ByteBuffer buffer) {
+ try {
+ ReplicationProtocol.readRequest(socketChannel, buffer);
+ final ByteArrayInputStream input =
+ new ByteArrayInputStream(buffer.array(), buffer.position(), buffer.limit());
+ try (DataInputStream dis = new DataInputStream(input)) {
+ switch (type) {
+ case PARTITION_RESOURCES_REQUEST:
+ return PartitionResourcesListTask.create(dis);
+ case PARTITION_RESOURCES_RESPONSE:
+ return PartitionResourcesListResponse.create(dis);
+ case REPLICATE_RESOURCE_FILE:
+ return ReplicateFileTask.create(dis);
+ case DELETE_RESOURCE_FILE:
+ return DeleteFileTask.create(dis);
+ case CHECKPOINT_PARTITION:
+ return CheckpointPartitionIndexesTask.create(dis);
+ case LSM_COMPONENT_MASK:
+ return ComponentMaskTask.create(dis);
+ case DROP_INDEX:
+ return DropIndexTask.create(dis);
+ case MARK_COMPONENT_VALID:
+ return MarkComponentValidTask.create(dis);
+ case REPLICATE_LOGS:
+ return ReplicateLogsTask.create(dis);
+ default:
+ throw new IllegalStateException("Unrecognized replication message");
+ }
+ }
+ } catch (IOException e) {
+ throw new ReplicationException(e);
+ }
+ }
+
+ public static ByteBuffer getEndLogReplicationBuffer() {
+ final int logsBatchSize = 1;
+ final ByteBuffer endLogRepBuffer =
+ ByteBuffer.allocate(Integer.BYTES + ReplicateLogsTask.END_REPLICATION_LOG_SIZE);
+ endLogRepBuffer.putInt(logsBatchSize);
+ endLogRepBuffer.put((byte) 0);
+ endLogRepBuffer.flip();
+ return endLogRepBuffer;
+ }
+
+ private static ByteBuffer ensureSize(ByteBuffer buffer, int size) {
+ if (buffer == null || buffer.capacity() < size) {
+ return ByteBuffer.allocate(size);
+ }
+ buffer.clear();
+ return buffer;
+ }
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/recovery/RemoteRecoveryManager.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/recovery/RemoteRecoveryManager.java
deleted file mode 100644
index 5d044b4..0000000
--- a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/recovery/RemoteRecoveryManager.java
+++ /dev/null
@@ -1,320 +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.recovery;
-
-import java.io.IOException;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-import java.util.stream.Collectors;
-
-import org.apache.asterix.common.api.IDatasetLifecycleManager;
-import org.apache.asterix.common.api.INcApplicationContext;
-import org.apache.asterix.common.cluster.ClusterPartition;
-import org.apache.asterix.common.config.ReplicationProperties;
-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.replication.IReplicationStrategy;
-import org.apache.asterix.common.replication.Replica;
-import org.apache.asterix.common.transactions.ILogManager;
-import org.apache.asterix.common.transactions.IRecoveryManager;
-import org.apache.asterix.replication.storage.ReplicaResourcesManager;
-import org.apache.asterix.transaction.management.resource.PersistentLocalResourceRepository;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.logging.log4j.LogManager;
-import org.apache.logging.log4j.Logger;
-
-public class RemoteRecoveryManager implements IRemoteRecoveryManager {
-
- private final IReplicationManager replicationManager;
- private static final Logger LOGGER = LogManager.getLogger();
- private final INcApplicationContext runtimeContext;
- private final ReplicationProperties replicationProperties;
- private Map<String, Set<String>> failbackRecoveryReplicas;
- private IReplicationStrategy replicationStrategy;
-
- public RemoteRecoveryManager(IReplicationManager replicationManager, INcApplicationContext runtimeContext,
- ReplicationProperties replicationProperties) {
- this.replicationManager = replicationManager;
- this.runtimeContext = runtimeContext;
- this.replicationProperties = replicationProperties;
- this.replicationStrategy = replicationManager.getReplicationStrategy();
- }
-
- private Map<String, Set<String>> constructRemoteRecoveryPlan() {
- //1. identify which replicas reside in this node
- String localNodeId = runtimeContext.getTransactionSubsystem().getId();
-
- Set<Replica> replicas = replicationStrategy.getRemoteReplicasAndSelf(localNodeId);
- Map<String, Set<String>> recoveryCandidates = new HashMap<>();
- Map<String, Integer> candidatesScore = new HashMap<>();
-
- //2. identify which nodes has backup per lost node data
- for (Replica node : replicas) {
- Set<Replica> locations = replicationStrategy.getRemoteReplicasAndSelf(node.getId());
-
- //since the local node just started, remove it from candidates
- locations.remove(new Replica(localNodeId, "", -1));
-
- //remove any dead replicas
- Set<String> deadReplicas = replicationManager.getDeadReplicasIds();
- for (String deadReplica : deadReplicas) {
- locations.remove(new Replica(deadReplica, "", -1));
- }
-
- //no active replicas to recover from
- if (locations.isEmpty()) {
- throw new IllegalStateException("Could not find any ACTIVE replica to recover " + node + " data.");
- }
-
- for (Replica locationRep : locations) {
- String location = locationRep.getId();
- if (candidatesScore.containsKey(location)) {
- candidatesScore.put(location, candidatesScore.get(location) + 1);
- } else {
- candidatesScore.put(location, 1);
- }
- }
- recoveryCandidates.put(node.getId(), locations.stream().map(Replica::getId).collect(Collectors.toSet()));
- }
-
- Map<String, Set<String>> recoveryList = new HashMap<>();
-
- //3. find best candidate to recover from per lost replica data
- recoveryCandidates.forEach((key, value) -> {
- int winnerScore = -1;
- String winner = "";
- for (String node : value) {
-
- int nodeScore = candidatesScore.get(node);
-
- if (nodeScore > winnerScore) {
- winnerScore = nodeScore;
- winner = node;
- }
- }
-
- if (recoveryList.containsKey(winner)) {
- recoveryList.get(winner).add(key);
- } else {
- Set<String> nodesToRecover = new HashSet<>();
- nodesToRecover.add(key);
- recoveryList.put(winner, nodesToRecover);
- }
-
- });
-
- return recoveryList;
- }
-
- @Override
- public void replayReplicaPartitionLogs(Set<Integer> partitions, boolean flush) throws HyracksDataException {
- ILogManager logManager = runtimeContext.getTransactionSubsystem().getLogManager();
- long minLSN = runtimeContext.getReplicaResourcesManager().getPartitionsMinLSN(partitions);
- long readableSmallestLSN = logManager.getReadableSmallestLSN();
- if (minLSN < readableSmallestLSN) {
- minLSN = readableSmallestLSN;
- }
-
- //replay logs > minLSN that belong to these partitions
- IRecoveryManager recoveryManager = runtimeContext.getTransactionSubsystem().getRecoveryManager();
- try {
- recoveryManager.replayPartitionsLogs(partitions, logManager.getLogReader(true), minLSN);
- if (flush) {
- runtimeContext.getDatasetLifecycleManager().flushAllDatasets();
- }
- } catch (IOException | ACIDException e) {
- throw new HyracksDataException(e);
- }
- }
-
- @Override
- 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));
- replayReplicaPartitionLogs(partitionsToTakeover, false);
-
- //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();
- Map<String, ClusterPartition[]> nodePartitions = runtimeContext.getMetadataProperties().getNodePartitions();
-
- 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();
-
- //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> ncsToRecoverFor = remoteReplica.getValue();
- Set<Integer> partitionsIds = new HashSet<>();
- for (String node : ncsToRecoverFor) {
- partitionsIds.addAll((Arrays.asList(nodePartitions.get(node))).stream()
- .map(ClusterPartition::getPartitionId).collect(Collectors.toList()));
- }
-
- //1. Request indexes metadata and LSM components
- replicationManager.requestReplicaFiles(replicaId, partitionsIds, new HashSet<String>());
- }
- break;
- } catch (IOException e) {
- LOGGER.warn("Failed during remote recovery. Attempting again...", e);
- maxRecoveryAttempts--;
- }
- }
- }
-
- @Override
- public void completeFailbackProcess() throws IOException, InterruptedException {
- ILogManager logManager = runtimeContext.getTransactionSubsystem().getLogManager();
- ReplicaResourcesManager replicaResourcesManager = (ReplicaResourcesManager) runtimeContext
- .getReplicaResourcesManager();
- Map<String, ClusterPartition[]> nodePartitions = 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<>();
- Set<Integer> partitionsToRecover = 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));
- partitionsToRecover.add(partition.getPartitionId());
- }
- }
-
- //Request remaining indexes files
- replicationManager.requestReplicaFiles(replicaId, partitionsToRecover, 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.
- */
- LOGGER.warn("Failed during completing failback. Restarting failback process...", e);
- 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;
- }
-
- //TODO refactor common code between remote recovery and failback process
- @Override
- public void doRemoteRecoveryPlan(Map<String, Set<Integer>> recoveryPlan) throws HyracksDataException {
- int maxRecoveryAttempts = replicationProperties.getMaxRemoteRecoveryAttempts();
- PersistentLocalResourceRepository resourceRepository = (PersistentLocalResourceRepository) runtimeContext
- .getLocalResourceRepository();
- IDatasetLifecycleManager datasetLifeCycleManager = runtimeContext.getDatasetLifecycleManager();
- ILogManager logManager = runtimeContext.getTransactionSubsystem().getLogManager();
- 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. clean any memory data that could've existed from previous failed recovery attempt
- datasetLifeCycleManager.closeAllDatasets();
-
- //2. remove any existing storage data and initialize storage metadata
- resourceRepository.deleteStorageData();
-
- /*** Start Recovery Per Lost Replica ***/
- for (Entry<String, Set<Integer>> remoteReplica : recoveryPlan.entrySet()) {
- String replicaId = remoteReplica.getKey();
- Set<Integer> partitionsToRecover = remoteReplica.getValue();
-
- //Request indexes metadata and LSM components
- replicationManager.requestReplicaFiles(replicaId, partitionsToRecover, new HashSet<String>());
- }
-
- //get max LSN from selected remote replicas
- long maxRemoteLSN = replicationManager.getMaxRemoteLSN(recoveryPlan.keySet());
-
- //6. force LogManager to start from a partition > maxLSN in selected remote replicas
- logManager.renewLogFilesAndStartFromLSN(maxRemoteLSN);
- break;
- } catch (IOException e) {
- LOGGER.warn("Failed during remote recovery. Attempting again...", e);
- maxRecoveryAttempts--;
- }
- }
- }
-}
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/storage/LSMComponentLSNSyncTask.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/storage/LSMComponentLSNSyncTask.java
deleted file mode 100644
index 08c0ec7..0000000
--- a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/storage/LSMComponentLSNSyncTask.java
+++ /dev/null
@@ -1,39 +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.storage;
-
-public class LSMComponentLSNSyncTask {
-
- private String componentFilePath;
- private String componentId;
-
- public LSMComponentLSNSyncTask(String componentId, String componentFilePath) {
- this.componentId = componentId;
- this.componentFilePath = componentFilePath;
- }
-
- public String getComponentFilePath() {
- return componentFilePath;
- }
-
- public String getComponentId() {
- return componentId;
- }
-
-}
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/storage/LSMComponentProperties.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/storage/LSMComponentProperties.java
deleted file mode 100644
index bf987d0..0000000
--- a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/storage/LSMComponentProperties.java
+++ /dev/null
@@ -1,159 +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.storage;
-
-import java.io.DataInput;
-import java.io.DataOutputStream;
-import java.io.File;
-import java.io.IOException;
-import java.io.OutputStream;
-import java.nio.file.Paths;
-import java.util.concurrent.atomic.AtomicInteger;
-
-import org.apache.asterix.common.ioopcallbacks.AbstractLSMIOOperationCallback;
-import org.apache.asterix.common.storage.ResourceReference;
-import org.apache.asterix.replication.logging.TxnLogUtil;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexReplicationJob;
-import org.apache.hyracks.storage.am.lsm.common.api.LSMOperationType;
-import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndex;
-import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndexFileManager;
-
-public class LSMComponentProperties {
-
- private AtomicInteger numberOfFiles;
- private String componentId;
- private long lsnOffset;
- private long originalLSN;
- private String nodeId;
- private Long replicaLSN;
- private String maskPath = null;
- private String replicaPath = null;
- private LSMOperationType opType;
-
- public LSMComponentProperties(ILSMIndexReplicationJob job, String nodeId) {
- this.nodeId = nodeId;
- componentId = LSMComponentProperties.getLSMComponentID((String) job.getJobFiles().toArray()[0]);
- numberOfFiles = new AtomicInteger(job.getJobFiles().size());
- opType = job.getLSMOpType();
- originalLSN = opType == LSMOperationType.FLUSH ?
- LSMComponentProperties.getLSMComponentLSN((AbstractLSMIndex) job.getLSMIndex(),
- job.getLSMIndexOperationContext()) : 0;
- }
-
- public LSMComponentProperties() {
- }
-
- public static long getLSMComponentLSN(AbstractLSMIndex lsmIndex, ILSMIndexOperationContext ctx) {
- long componentLSN = -1;
- try {
- componentLSN = ((AbstractLSMIOOperationCallback) lsmIndex.getIOOperationCallback())
- .getComponentLSN(ctx.getComponentsToBeReplicated());
- } catch (HyracksDataException e) {
- e.printStackTrace();
- }
- if (componentLSN < 0) {
- componentLSN = 0;
- }
- return componentLSN;
- }
-
- public void serialize(OutputStream out) throws IOException {
- DataOutputStream dos = new DataOutputStream(out);
- dos.writeUTF(componentId);
- dos.writeUTF(nodeId);
- dos.writeInt(numberOfFiles.get());
- dos.writeLong(originalLSN);
- dos.writeLong(lsnOffset);
- dos.writeInt(opType.ordinal());
- }
-
- public static LSMComponentProperties create(DataInput input) throws IOException {
- LSMComponentProperties lsmCompProp = new LSMComponentProperties();
- lsmCompProp.componentId = input.readUTF();
- lsmCompProp.nodeId = input.readUTF();
- lsmCompProp.numberOfFiles = new AtomicInteger(input.readInt());
- lsmCompProp.originalLSN = input.readLong();
- lsmCompProp.lsnOffset = input.readLong();
- lsmCompProp.opType = LSMOperationType.values()[input.readInt()];
- return lsmCompProp;
- }
-
- public String getMaskPath(ReplicaResourcesManager resourceManager) throws HyracksDataException {
- if (maskPath == null) {
- LSMIndexFileProperties afp = new LSMIndexFileProperties(this);
- maskPath = getReplicaComponentPath(resourceManager) + File.separator + afp.getFileName()
- + ReplicaResourcesManager.LSM_COMPONENT_MASK_SUFFIX;
- }
- return maskPath;
- }
-
- public String getReplicaComponentPath(ReplicaResourcesManager resourceManager) throws HyracksDataException {
- if (replicaPath == null) {
- LSMIndexFileProperties afp = new LSMIndexFileProperties(this);
- replicaPath = resourceManager.getIndexPath(afp);
- }
- return replicaPath;
- }
-
- /***
- * @param filePath
- * any file of the LSM component
- * @return a unique id based on the timestamp of the component
- */
- public static String getLSMComponentID(String filePath) {
- final ResourceReference ref = ResourceReference.of(filePath);
- final String fileUniqueTimestamp =
- ref.getName().substring(0, ref.getName().lastIndexOf(AbstractLSMIndexFileManager.DELIMITER));
- return Paths.get(ref.getRelativePath().toString(), fileUniqueTimestamp).toString();
- }
-
- public String getComponentId() {
- return componentId;
- }
-
- public long getOriginalLSN() {
- return originalLSN;
- }
-
- public String getNodeId() {
- return nodeId;
- }
-
- public int markFileComplete() {
- return numberOfFiles.decrementAndGet();
- }
-
- public Long getReplicaLSN() {
- return replicaLSN;
- }
-
- public void setReplicaLSN(Long replicaLSN) {
- this.replicaLSN = replicaLSN;
- }
-
- public LSMOperationType getOpType() {
- return opType;
- }
-
- public String getNodeUniqueLSN() {
- return TxnLogUtil.getNodeUniqueLSN(nodeId, originalLSN);
- }
-}
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/storage/LSMIndexFileProperties.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/storage/LSMIndexFileProperties.java
deleted file mode 100644
index 2ebf2cb..0000000
--- a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/storage/LSMIndexFileProperties.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.storage;
-
-import java.io.DataInput;
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.io.OutputStream;
-import java.nio.file.Paths;
-
-public class LSMIndexFileProperties {
-
- private long fileSize;
- private String nodeId;
- private boolean lsmComponentFile;
- private String filePath;
- private boolean requiresAck = false;
-
- public LSMIndexFileProperties() {
- }
-
- public LSMIndexFileProperties(String filePath, long fileSize, String nodeId, boolean lsmComponentFile,
- boolean requiresAck) {
- initialize(filePath, fileSize, nodeId, lsmComponentFile, requiresAck);
- }
-
- public LSMIndexFileProperties(LSMComponentProperties lsmComponentProperties) {
- initialize(lsmComponentProperties.getComponentId(), -1, lsmComponentProperties.getNodeId(), false, false);
- }
-
- public void initialize(String filePath, long fileSize, String nodeId, boolean lsmComponentFile,
- boolean requiresAck) {
- this.filePath = filePath;
- this.fileSize = fileSize;
- this.nodeId = nodeId;
- this.lsmComponentFile = lsmComponentFile;
- this.requiresAck = requiresAck;
- }
-
- public void serialize(OutputStream out) throws IOException {
- DataOutputStream dos = new DataOutputStream(out);
- dos.writeUTF(nodeId);
- dos.writeUTF(filePath);
- dos.writeLong(fileSize);
- dos.writeBoolean(lsmComponentFile);
- dos.writeBoolean(requiresAck);
- }
-
- public static LSMIndexFileProperties create(DataInput input) throws IOException {
- String nodeId = input.readUTF();
- String filePath = input.readUTF();
- long fileSize = input.readLong();
- boolean lsmComponentFile = input.readBoolean();
- boolean requiresAck = input.readBoolean();
- LSMIndexFileProperties fileProp =
- new LSMIndexFileProperties(filePath, fileSize, nodeId, lsmComponentFile, requiresAck);
- return fileProp;
- }
-
- public String getFilePath() {
- return filePath;
- }
-
- public long getFileSize() {
- return fileSize;
- }
-
- public String getNodeId() {
- return nodeId;
- }
-
- public boolean isLSMComponentFile() {
- return lsmComponentFile;
- }
-
- public boolean requiresAck() {
- return requiresAck;
- }
-
- public String getFileName() {
- return Paths.get(filePath).toFile().getName();
- }
-
- @Override
- public String toString() {
- StringBuilder sb = new StringBuilder();
- sb.append("File Path: " + filePath + " ");
- sb.append("File Size: " + fileSize + " ");
- sb.append("Node ID: " + nodeId + " ");
- sb.append("isLSMComponentFile : " + lsmComponentFile + " ");
- return sb.toString();
- }
-}
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/storage/ReplicaResourcesManager.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/storage/ReplicaResourcesManager.java
deleted file mode 100644
index 398f97d..0000000
--- a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/storage/ReplicaResourcesManager.java
+++ /dev/null
@@ -1,226 +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.storage;
-
-import java.io.File;
-import java.io.FilenameFilter;
-import java.io.IOException;
-import java.nio.file.Files;
-import java.nio.file.Path;
-import java.nio.file.Paths;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.stream.Collectors;
-
-import org.apache.asterix.common.cluster.ClusterPartition;
-import org.apache.asterix.common.config.MetadataProperties;
-import org.apache.asterix.common.dataflow.DatasetLocalResource;
-import org.apache.asterix.common.replication.IReplicaResourcesManager;
-import org.apache.asterix.common.storage.DatasetResourceReference;
-import org.apache.asterix.common.storage.IIndexCheckpointManagerProvider;
-import org.apache.asterix.common.utils.StorageConstants;
-import org.apache.asterix.common.utils.StoragePathUtil;
-import org.apache.asterix.transaction.management.resource.PersistentLocalResourceRepository;
-import org.apache.commons.io.FileUtils;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.api.io.FileReference;
-import org.apache.hyracks.storage.common.ILocalResourceRepository;
-import org.apache.hyracks.storage.common.LocalResource;
-
-public class ReplicaResourcesManager implements IReplicaResourcesManager {
- public static final String LSM_COMPONENT_MASK_SUFFIX = "_mask";
- private final PersistentLocalResourceRepository localRepository;
- private final Map<String, ClusterPartition[]> nodePartitions;
- private final IIndexCheckpointManagerProvider indexCheckpointManagerProvider;
-
- public ReplicaResourcesManager(ILocalResourceRepository localRepository, MetadataProperties metadataProperties,
- IIndexCheckpointManagerProvider indexCheckpointManagerProvider) {
- this.localRepository = (PersistentLocalResourceRepository) localRepository;
- this.indexCheckpointManagerProvider = indexCheckpointManagerProvider;
- nodePartitions = metadataProperties.getNodePartitions();
- }
-
- public void deleteIndexFile(LSMIndexFileProperties afp) throws HyracksDataException {
- String indexPath = getIndexPath(afp);
- if (indexPath != null) {
- if (afp.isLSMComponentFile()) {
- //delete index file
- String indexFilePath = indexPath + File.separator + afp.getFileName();
- File destFile = new File(indexFilePath);
- FileUtils.deleteQuietly(destFile);
- } else {
- //delete index directory
- FileUtils.deleteQuietly(new File(indexPath));
- }
- }
- }
-
- public String getIndexPath(LSMIndexFileProperties fileProperties) throws HyracksDataException {
- final FileReference indexPath = localRepository.getIndexPath(Paths.get(fileProperties.getFilePath()));
- if (!indexPath.getFile().exists()) {
- indexPath.getFile().mkdirs();
- }
- return indexPath.toString();
- }
-
- public void createRemoteLSMComponentMask(LSMComponentProperties lsmComponentProperties) throws IOException {
- String maskPath = lsmComponentProperties.getMaskPath(this);
- Path path = Paths.get(maskPath);
- if (!Files.exists(path)) {
- File maskFile = new File(maskPath);
- maskFile.createNewFile();
- }
- }
-
- public void markLSMComponentReplicaAsValid(LSMComponentProperties lsmComponentProperties) throws IOException {
- //remove mask to mark component as valid
- String maskPath = lsmComponentProperties.getMaskPath(this);
- Path path = Paths.get(maskPath);
- Files.deleteIfExists(path);
- }
-
- public Set<File> getReplicaIndexes(String replicaId) throws HyracksDataException {
- Set<File> remoteIndexesPaths = new HashSet<File>();
- ClusterPartition[] partitions = nodePartitions.get(replicaId);
- for (ClusterPartition partition : partitions) {
- remoteIndexesPaths.addAll(localRepository.getPartitionIndexes(partition.getPartitionId()));
- }
- return remoteIndexesPaths;
- }
-
- @Override
- public long getPartitionsMinLSN(Set<Integer> partitions) throws HyracksDataException {
- long minRemoteLSN = Long.MAX_VALUE;
- for (Integer partition : partitions) {
- final List<DatasetResourceReference> partitionResources = localRepository.getResources(resource -> {
- DatasetLocalResource dsResource = (DatasetLocalResource) resource.getResource();
- return dsResource.getPartition() == partition;
- }).values().stream().map(DatasetResourceReference::of).collect(Collectors.toList());
- for (DatasetResourceReference indexRef : partitionResources) {
- long remoteIndexMaxLSN = indexCheckpointManagerProvider.get(indexRef).getLowWatermark();
- minRemoteLSN = Math.min(minRemoteLSN, remoteIndexMaxLSN);
- }
- }
- return minRemoteLSN;
- }
-
- public Map<Long, DatasetResourceReference> getLaggingReplicaIndexesId2PathMap(String replicaId, long targetLSN)
- throws HyracksDataException {
- Map<Long, DatasetResourceReference> laggingReplicaIndexes = new HashMap<>();
- final List<Integer> replicaPartitions =
- Arrays.stream(nodePartitions.get(replicaId)).map(ClusterPartition::getPartitionId)
- .collect(Collectors.toList());
- for (int patition : replicaPartitions) {
- final Map<Long, LocalResource> partitionResources = localRepository.getPartitionResources(patition);
- final List<DatasetResourceReference> indexesRefs =
- partitionResources.values().stream().map(DatasetResourceReference::of).collect(Collectors.toList());
- for (DatasetResourceReference ref : indexesRefs) {
- if (indexCheckpointManagerProvider.get(ref).getLowWatermark() < targetLSN) {
- laggingReplicaIndexes.put(ref.getResourceId(), ref);
- }
- }
- }
- return laggingReplicaIndexes;
- }
-
- public void cleanInvalidLSMComponents(String replicaId) {
- //for every index in replica
- Set<File> remoteIndexes = null;
- try {
- remoteIndexes = getReplicaIndexes(replicaId);
- } catch (HyracksDataException e) {
- throw new IllegalStateException(e);
- }
- for (File remoteIndexFile : remoteIndexes) {
- //search for any mask
- File[] masks = remoteIndexFile.listFiles(LSM_COMPONENTS_MASKS_FILTER);
-
- for (File mask : masks) {
- //delete all files belonging to this mask
- deleteLSMComponentFilesForMask(mask);
- //delete the mask itself
- mask.delete();
- }
- }
- }
-
- private static void deleteLSMComponentFilesForMask(File maskFile) {
- String lsmComponentTimeStamp = maskFile.getName().substring(0,
- maskFile.getName().length() - LSM_COMPONENT_MASK_SUFFIX.length());
- File indexFolder = maskFile.getParentFile();
- File[] lsmComponentsFiles = indexFolder.listFiles(LSM_COMPONENTS_NON_MASKS_FILTER);
- for (File lsmComponentFile : lsmComponentsFiles) {
- if (lsmComponentFile.getName().contains(lsmComponentTimeStamp)) {
- //match based on time stamp
- lsmComponentFile.delete();
- }
- }
- }
-
- /**
- * @param partition
- * @return Absolute paths to all partition files
- */
- @Override
- public List<String> getPartitionIndexesFiles(int partition, boolean relativePath) throws HyracksDataException {
- List<String> partitionFiles = new ArrayList<String>();
- Set<File> partitionIndexes = localRepository.getPartitionIndexes(partition);
- for (File indexDir : partitionIndexes) {
- if (indexDir.isDirectory()) {
- File[] indexFiles = indexDir.listFiles(LSM_INDEX_FILES_FILTER);
- if (indexFiles != null) {
- for (File file : indexFiles) {
- if (!relativePath) {
- partitionFiles.add(file.getAbsolutePath());
- } else {
- partitionFiles.add(StoragePathUtil.getIndexFileRelativePath(file.getAbsolutePath()));
- }
- }
- }
- }
- }
- return partitionFiles;
- }
-
- 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(StorageConstants.METADATA_FILE_NAME) || !name.startsWith(".");
- }
- };
-}
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/recovery/FileSynchronizer.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/sync/FileSynchronizer.java
similarity index 89%
rename from asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/recovery/FileSynchronizer.java
rename to asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/sync/FileSynchronizer.java
index 8aa4487..e1649b3 100644
--- a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/recovery/FileSynchronizer.java
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/sync/FileSynchronizer.java
@@ -16,7 +16,7 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.asterix.replication.recovery;
+package org.apache.asterix.replication.sync;
import java.io.IOException;
import java.io.RandomAccessFile;
@@ -25,11 +25,11 @@
import org.apache.asterix.common.api.INcApplicationContext;
import org.apache.asterix.common.exceptions.ReplicationException;
-import org.apache.asterix.replication.functions.ReplicationProtocol;
+import org.apache.asterix.replication.messaging.ReplicationProtocol;
import org.apache.asterix.replication.management.NetworkingUtil;
import org.apache.asterix.replication.messaging.DeleteFileTask;
import org.apache.asterix.replication.messaging.ReplicateFileTask;
-import org.apache.asterix.replication.storage.PartitionReplica;
+import org.apache.asterix.replication.api.PartitionReplica;
import org.apache.hyracks.api.io.FileReference;
import org.apache.hyracks.api.io.IIOManager;
@@ -44,11 +44,15 @@
}
public void replicate(String file) {
+ replicate(file, false);
+ }
+
+ public void replicate(String file, boolean metadata) {
try {
final IIOManager ioManager = appCtx.getIoManager();
final SocketChannel channel = replica.getChannel();
final FileReference filePath = ioManager.resolve(file);
- ReplicateFileTask task = new ReplicateFileTask(file, filePath.getFile().length());
+ ReplicateFileTask task = new ReplicateFileTask(file, filePath.getFile().length(), metadata);
ReplicationProtocol.sendTo(replica, task);
// send the file itself
try (RandomAccessFile fromFile = new RandomAccessFile(filePath.getFile(),
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/sync/IndexSynchronizer.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/sync/IndexSynchronizer.java
new file mode 100644
index 0000000..74f38e2
--- /dev/null
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/sync/IndexSynchronizer.java
@@ -0,0 +1,140 @@
+/*
+ * 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.sync;
+
+import static org.apache.hyracks.api.replication.IReplicationJob.ReplicationOperation.DELETE;
+import static org.apache.hyracks.api.replication.IReplicationJob.ReplicationOperation.REPLICATE;
+
+import java.io.IOException;
+import java.nio.file.Paths;
+
+import org.apache.asterix.common.api.INcApplicationContext;
+import org.apache.asterix.common.ioopcallbacks.AbstractLSMIOOperationCallback;
+import org.apache.asterix.common.storage.ResourceReference;
+import org.apache.asterix.common.utils.StoragePathUtil;
+import org.apache.asterix.replication.api.PartitionReplica;
+import org.apache.asterix.replication.messaging.ComponentMaskTask;
+import org.apache.asterix.replication.messaging.DropIndexTask;
+import org.apache.asterix.replication.messaging.MarkComponentValidTask;
+import org.apache.asterix.replication.messaging.ReplicationProtocol;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.replication.IReplicationJob;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexReplicationJob;
+import org.apache.hyracks.storage.am.lsm.common.api.LSMOperationType;
+import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndexFileManager;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+public class IndexSynchronizer {
+
+ private static final Logger LOGGER = LogManager.getLogger();
+ private final IReplicationJob job;
+ private final INcApplicationContext appCtx;
+
+ public IndexSynchronizer(IReplicationJob job, INcApplicationContext appCtx) {
+ this.job = job;
+ this.appCtx = appCtx;
+ }
+
+ public void sync(PartitionReplica replica) throws IOException {
+ switch (job.getJobType()) {
+ case LSM_COMPONENT:
+ syncComponent(replica);
+ break;
+ case METADATA:
+ syncMetadata(replica);
+ break;
+ default:
+ throw new IllegalStateException("unrecognized job type: " + job.getJobType().name());
+ }
+ }
+
+ private void syncComponent(PartitionReplica replica) throws IOException {
+ if (job.getOperation() == REPLICATE) {
+ replicateComponent(replica);
+ } else if (job.getOperation() == DELETE) {
+ deleteComponent(replica);
+ }
+ }
+
+ private void syncMetadata(PartitionReplica replica) throws IOException {
+ if (job.getOperation() == REPLICATE) {
+ replicateIndexMetadata(replica);
+ } else if (job.getOperation() == DELETE) {
+ deleteIndexMetadata(replica);
+ }
+ }
+
+ private void replicateComponent(PartitionReplica replica) throws IOException {
+ // send component header
+ final String anyFile = job.getAnyFile();
+ final String lsmComponentID = getComponentId(anyFile);
+ final String indexFile = StoragePathUtil.getFileRelativePath(anyFile);
+ final ComponentMaskTask maskTask = new ComponentMaskTask(indexFile, lsmComponentID);
+ ReplicationProtocol.sendTo(replica, maskTask);
+ ReplicationProtocol.waitForAck(replica);
+ // send component files
+ final FileSynchronizer fileSynchronizer = new FileSynchronizer(appCtx, replica);
+ job.getJobFiles().stream().map(StoragePathUtil::getFileRelativePath).forEach(fileSynchronizer::replicate);
+ // send mark component valid
+ MarkComponentValidTask markValidTask = new MarkComponentValidTask(indexFile, getReplicatedComponentLsn());
+ ReplicationProtocol.sendTo(replica, markValidTask);
+ ReplicationProtocol.waitForAck(replica);
+ LOGGER.debug("Replicated component ({}) to replica {}", indexFile, replica);
+ }
+
+ private void deleteComponent(PartitionReplica replica) {
+ FileSynchronizer fileSynchronizer = new FileSynchronizer(appCtx, replica);
+ job.getJobFiles().stream().map(StoragePathUtil::getFileRelativePath).forEach(fileSynchronizer::delete);
+ }
+
+ private void replicateIndexMetadata(PartitionReplica replica) {
+ // send the index metadata file
+ final FileSynchronizer fileSynchronizer = new FileSynchronizer(appCtx, replica);
+ job.getJobFiles().stream().map(StoragePathUtil::getFileRelativePath)
+ .forEach(file -> fileSynchronizer.replicate(file, true));
+ }
+
+ private void deleteIndexMetadata(PartitionReplica replica) throws IOException {
+ final String file = StoragePathUtil.getFileRelativePath(job.getAnyFile());
+ final DropIndexTask task = new DropIndexTask(file);
+ ReplicationProtocol.sendTo(replica, task);
+ ReplicationProtocol.waitForAck(replica);
+ }
+
+ private long getReplicatedComponentLsn() throws HyracksDataException {
+ final ILSMIndexReplicationJob indexReplJob = (ILSMIndexReplicationJob) job;
+ if (indexReplJob.getLSMOpType() != LSMOperationType.FLUSH) {
+ return AbstractLSMIOOperationCallback.INVALID;
+ }
+ final ILSMIndex lsmIndex = indexReplJob.getLSMIndex();
+ final ILSMIndexOperationContext ctx = indexReplJob.getLSMIndexOperationContext();
+ return ((AbstractLSMIOOperationCallback) lsmIndex.getIOOperationCallback())
+ .getComponentLSN(ctx.getComponentsToBeReplicated());
+ }
+
+ private static String getComponentId(String filePath) {
+ final ResourceReference ref = ResourceReference.of(filePath);
+ final String fileUniqueTimestamp =
+ ref.getName().substring(0, ref.getName().lastIndexOf(AbstractLSMIndexFileManager.DELIMITER));
+ return Paths.get(ref.getRelativePath().toString(), fileUniqueTimestamp).toString();
+ }
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/recovery/ReplicaFilesSynchronizer.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/sync/ReplicaFilesSynchronizer.java
similarity index 82%
rename from asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/recovery/ReplicaFilesSynchronizer.java
rename to asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/sync/ReplicaFilesSynchronizer.java
index 2021cee..5658779 100644
--- a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/recovery/ReplicaFilesSynchronizer.java
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/sync/ReplicaFilesSynchronizer.java
@@ -16,7 +16,7 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.asterix.replication.recovery;
+package org.apache.asterix.replication.sync;
import java.io.IOException;
import java.nio.ByteBuffer;
@@ -28,10 +28,11 @@
import org.apache.asterix.common.api.INcApplicationContext;
import org.apache.asterix.common.utils.StoragePathUtil;
-import org.apache.asterix.replication.functions.ReplicationProtocol;
+import org.apache.asterix.replication.api.PartitionReplica;
import org.apache.asterix.replication.messaging.PartitionResourcesListResponse;
import org.apache.asterix.replication.messaging.PartitionResourcesListTask;
-import org.apache.asterix.replication.storage.PartitionReplica;
+import org.apache.asterix.replication.messaging.ReplicationProtocol;
+import org.apache.asterix.transaction.management.resource.PersistentLocalResourceRepository;
/**
* Ensures that the files between master and a replica are synchronized
@@ -49,9 +50,10 @@
public void sync() throws IOException {
final int partition = replica.getIdentifier().getPartition();
final Set<String> replicaFiles = getReplicaFiles(partition);
- final Set<String> masterFiles =
- appCtx.getReplicaResourcesManager().getPartitionIndexesFiles(partition, false).stream()
- .map(StoragePathUtil::getFileRelativePath).collect(Collectors.toSet());
+ final PersistentLocalResourceRepository localResourceRepository =
+ (PersistentLocalResourceRepository) appCtx.getLocalResourceRepository();
+ final Set<String> masterFiles = localResourceRepository.getPartitionIndexesFiles(partition).stream()
+ .map(StoragePathUtil::getFileRelativePath).collect(Collectors.toSet());
// find files on master and not on replica
final List<String> replicaMissingFiles =
masterFiles.stream().filter(file -> !replicaFiles.contains(file)).collect(Collectors.toList());
@@ -65,7 +67,7 @@
private Set<String> getReplicaFiles(int partition) throws IOException {
final PartitionResourcesListTask replicaFilesRequest = new PartitionResourcesListTask(partition);
final SocketChannel channel = replica.getChannel();
- final ByteBuffer reusableBuffer = replica.gerReusableBuffer();
+ final ByteBuffer reusableBuffer = replica.getReusableBuffer();
ReplicationProtocol.sendTo(replica, replicaFilesRequest);
final PartitionResourcesListResponse response =
(PartitionResourcesListResponse) ReplicationProtocol.read(channel, reusableBuffer);
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/recovery/ReplicaSynchronizer.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/sync/ReplicaSynchronizer.java
similarity index 90%
rename from asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/recovery/ReplicaSynchronizer.java
rename to asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/sync/ReplicaSynchronizer.java
index 1fa3246..9f397d2 100644
--- a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/recovery/ReplicaSynchronizer.java
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/sync/ReplicaSynchronizer.java
@@ -16,16 +16,15 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.asterix.replication.recovery;
+package org.apache.asterix.replication.sync;
import java.io.IOException;
import org.apache.asterix.common.api.INcApplicationContext;
-import org.apache.asterix.common.config.ReplicationProperties;
import org.apache.asterix.common.replication.IReplicationStrategy;
-import org.apache.asterix.replication.functions.ReplicationProtocol;
+import org.apache.asterix.replication.messaging.ReplicationProtocol;
import org.apache.asterix.replication.messaging.CheckpointPartitionIndexesTask;
-import org.apache.asterix.replication.storage.PartitionReplica;
+import org.apache.asterix.replication.api.PartitionReplica;
/**
* Performs the steps required to ensure any newly added replica
diff --git a/asterixdb/asterix-replication/src/test/resources/data/fbu.adm b/asterixdb/asterix-replication/src/test/resources/data/fbu.adm
deleted file mode 100644
index 7e99ea4..0000000
--- a/asterixdb/asterix-replication/src/test/resources/data/fbu.adm
+++ /dev/null
@@ -1,10 +0,0 @@
-{"id":1,"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")}]}
-{"id":2,"alias":"Isbel","name":"IsbelDull","user-since":datetime("2011-01-22T10:10:00"),"friend-ids":{{1,4}},"employment":[{"organization-name":"Hexviafind","start-date":date("2010-04-27")}]}
-{"id":3,"alias":"Emory","name":"EmoryUnk","user-since":datetime("2012-07-10T10:10:00"),"friend-ids":{{1,5,8,9}},"employment":[{"organization-name":"geomedia","start-date":date("2010-06-17"),"end-date":date("2010-01-26")}]}
-{"id":4,"alias":"Nicholas","name":"NicholasStroh","user-since":datetime("2010-12-27T10:10:00"),"friend-ids":{{2}},"employment":[{"organization-name":"Zamcorporation","start-date":date("2010-06-08")}]}
-{"id":5,"alias":"Von","name":"VonKemble","user-since":datetime("2010-01-05T10:10:00"),"friend-ids":{{3,6,10}},"employment":[{"organization-name":"Kongreen","start-date":date("2010-11-27")}]}
-{"id":6,"alias":"Willis","name":"WillisWynne","user-since":datetime("2005-01-17T10:10:00"),"friend-ids":{{1,3,7}},"employment":[{"organization-name":"jaydax","start-date":date("2009-05-15")}]}
-{"id":7,"alias":"Suzanna","name":"SuzannaTillson","user-since":datetime("2012-08-07T10:10:00"),"friend-ids":{{6}},"employment":[{"organization-name":"Labzatron","start-date":date("2011-04-19")}]}
-{"id":8,"alias":"Nila","name":"NilaMilliron","user-since":datetime("2008-01-01T10:10:00"),"friend-ids":{{3}},"employment":[{"organization-name":"Plexlane","start-date":date("2010-02-28")}]}
-{"id":9,"alias":"Woodrow","name":"WoodrowNehling","user-since":datetime("2005-09-20T10:10:00"),"friend-ids":{{3,10}},"employment":[{"organization-name":"Zuncan","start-date":date("2003-04-22"),"end-date":date("2009-12-13")}]}
-{"id":10,"alias":"Bram","name":"BramHatch","user-since":datetime("2010-10-16T10:10:00"),"friend-ids":{{1,5,9}},"employment":[{"organization-name":"physcane","start-date":date("2007-06-05"),"end-date":date("2011-11-05")}]}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/message/ReplicaEventMessage.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/message/ReplicaEventMessage.java
deleted file mode 100644
index 8ae1bd7..0000000
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/message/ReplicaEventMessage.java
+++ /dev/null
@@ -1,65 +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.runtime.message;
-
-import org.apache.asterix.common.api.INcApplicationContext;
-import org.apache.asterix.common.messaging.api.INcAddressedMessage;
-import org.apache.asterix.common.replication.Replica;
-import org.apache.asterix.common.replication.ReplicaEvent;
-import org.apache.hyracks.api.application.IClusterLifecycleListener.ClusterEventType;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-public class ReplicaEventMessage implements INcAddressedMessage {
-
- private static final long serialVersionUID = 1L;
- private final String nodeId;
- private final ClusterEventType event;
- private final String nodeIPAddress;
- private final int nodePort;
-
- public ReplicaEventMessage(String nodeId, String nodeIPAddress, int nodePort, ClusterEventType event) {
- this.nodeId = nodeId;
- this.nodeIPAddress = nodeIPAddress;
- this.nodePort = nodePort;
- this.event = event;
- }
-
- public String getNodeId() {
- return nodeId;
- }
-
- public ClusterEventType getEvent() {
- return event;
- }
-
- public String getNodeIPAddress() {
- return nodeIPAddress;
- }
-
- @Override
- public void handle(INcApplicationContext appContext) throws HyracksDataException, InterruptedException {
- Replica replica = new Replica(nodeId, nodeIPAddress, nodePort);
- appContext.getReplicationManager().reportReplicaEvent(new ReplicaEvent(replica, event));
- }
-
- @Override
- public String toString() {
- return ReplicaEventMessage.class.getSimpleName();
- }
-}
diff --git a/asterixdb/asterix-server/src/test/java/org/apache/asterix/server/test/MetadataReplicationIT.java b/asterixdb/asterix-server/src/test/java/org/apache/asterix/server/test/MetadataReplicationIT.java
deleted file mode 100644
index 5ae6843..0000000
--- a/asterixdb/asterix-server/src/test/java/org/apache/asterix/server/test/MetadataReplicationIT.java
+++ /dev/null
@@ -1,168 +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.server.test;
-
-import java.io.File;
-import java.nio.file.Paths;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Map;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.asterix.test.base.RetainLogsRule;
-import org.apache.asterix.test.common.TestExecutor;
-import org.apache.asterix.testframework.context.TestCaseContext;
-import org.apache.commons.io.FileUtils;
-import org.apache.commons.lang3.StringUtils;
-import org.apache.hyracks.server.process.HyracksCCProcess;
-import org.apache.hyracks.server.process.HyracksNCServiceProcess;
-import org.apache.hyracks.server.process.HyracksVirtualCluster;
-import org.apache.logging.log4j.LogManager;
-import org.apache.logging.log4j.Logger;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TestRule;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
-@RunWith(Parameterized.class)
-public class MetadataReplicationIT {
-
- // Important paths and files for this test.
-
- // The "target" subdirectory of asterix-server. All outputs go here.
- public static final String TARGET_DIR = StringUtils.join(new String[] { "../asterix-server/target" },
- File.separator);
-
- // Directory where the NCs create and store all data, as configured by
- // src/test/resources/NCServiceExecutionIT/cc.conf.
- public static final String INSTANCE_DIR = StringUtils.join(new String[] { TARGET_DIR, "tmp" }, File.separator);
-
- // The log directory, where all CC, NCService, and NC logs are written. CC and
- // NCService logs are configured on the HyracksVirtualCluster below. NC logs
- // are configured in src/test/resources/NCServiceExecutionIT/ncservice*.conf.
- public static final String LOG_DIR = StringUtils.join(new String[] { TARGET_DIR, "failsafe-reports" },
- File.separator);
-
- // Directory where *.conf files are located.
- public static final String CONF_DIR = StringUtils
- .join(new String[] { TARGET_DIR, "test-classes", "MetadataReplicationIT" }, File.separator);
-
- // The app.home specified for HyracksVirtualCluster. The NCService expects
- // to find the NC startup script in ${app.home}/bin.
- public static final String APP_HOME = StringUtils.join(new String[] { TARGET_DIR, "appassembler" }, File.separator);
-
- // Path to the asterix-app directory. This is used as the current working
- // directory for the CC and NCService processes, which allows relative file
- // paths in "load" statements in test queries to find the right data. It is
- // also used for HDFSCluster.
- public static final String ASTERIX_APP_DIR = StringUtils.join(new String[] { "..", "asterix-app" }, File.separator);
-
- // Path to the actual AQL test files, which we borrow from asterix-app. This is
- // passed to TestExecutor.
- protected static final String TESTS_DIR = StringUtils
- .join(new String[] { ASTERIX_APP_DIR, "src", "test", "resources", "runtimets" }, File.separator);
-
- // Path that actual results are written to. We create and clean this directory
- // here, and also pass it to TestExecutor which writes the test output there.
- public static final String ACTUAL_RESULTS_DIR = StringUtils.join(new String[] { TARGET_DIR, "ittest" },
- File.separator);
- private static final String PATH_BASE = Paths
- .get("src", "test", "resources", "integrationts", "metadata_only_replication").toString() + File.separator;
- private static final String PATH_ACTUAL = "target" + File.separator + "ittest" + File.separator;
-
- private static final Logger LOGGER = LogManager.getLogger();
- private static String reportPath = new File(
- StringUtils.join(new String[] { "target", "failsafe-reports" }, File.separator)).getAbsolutePath();
-
- private final TestExecutor testExecutor = new TestExecutor();
- private TestCaseContext tcCtx;
- private static String scriptHomePath;
- private static File asterixInstallerPath;
- private static ProcessBuilder pb;
- private static Map<String, String> env;
-
- private static HyracksCCProcess cc;
- private static HyracksNCServiceProcess nc1;
- private static HyracksNCServiceProcess nc2;
- private static HyracksVirtualCluster cluster;
-
- public MetadataReplicationIT(TestCaseContext tcCtx) {
- this.tcCtx = tcCtx;
- }
-
- @Rule
- public TestRule retainLogs = new RetainLogsRule(NCServiceExecutionIT.ASTERIX_APP_DIR, reportPath, this);
-
- @Before
- public void before() throws Exception {
- LOGGER.info("Creating new instance...");
- File instanceDir = new File(INSTANCE_DIR);
- if (instanceDir.isDirectory()) {
- FileUtils.deleteDirectory(instanceDir);
- }
-
- // HDFSCluster requires the input directory to end with a file separator.
-
- cluster = new HyracksVirtualCluster(new File(APP_HOME), new File(ASTERIX_APP_DIR));
- nc1 = cluster.addNCService(new File(CONF_DIR, "ncservice1.conf"), new File(LOG_DIR, "ncservice1.log"));
-
- nc2 = cluster.addNCService(new File(CONF_DIR, "ncservice2.conf"), new File(LOG_DIR, "ncservice2.log"));
-
- // Start CC
- cc = cluster.start(new File(CONF_DIR, "cc.conf"), new File(LOG_DIR, "cc.log"));
-
- LOGGER.info("Instance created.");
- testExecutor.waitForClusterActive(30, TimeUnit.SECONDS);
- LOGGER.info("Instance is in ACTIVE state.");
-
- }
-
- @After
- public void after() throws Exception {
- LOGGER.info("Destroying instance...");
- cluster.stop();
- LOGGER.info("Instance destroyed.");
- }
-
- @Test
- public void test() throws Exception {
- testExecutor.executeTest(PATH_ACTUAL, tcCtx, null, false);
- }
-
- @Parameterized.Parameters(name = "MetadataReplicationIT {index}: {0}")
- public static Collection<Object[]> tests() throws Exception {
- Collection<Object[]> testArgs = buildTestsInXml(TestCaseContext.DEFAULT_TESTSUITE_XML_NAME);
- if (testArgs.size() == 0) {
- testArgs = buildTestsInXml(TestCaseContext.DEFAULT_TESTSUITE_XML_NAME);
- }
- return testArgs;
- }
-
- protected static Collection<Object[]> buildTestsInXml(String xmlfile) throws Exception {
- Collection<Object[]> testArgs = new ArrayList<>();
- TestCaseContext.Builder b = new TestCaseContext.Builder();
- for (TestCaseContext ctx : b.build(new File(PATH_BASE), xmlfile)) {
- testArgs.add(new Object[] { ctx });
- }
- return testArgs;
- }
-}
diff --git a/asterixdb/asterix-server/src/test/java/org/apache/asterix/server/test/ReplicationIT.java b/asterixdb/asterix-server/src/test/java/org/apache/asterix/server/test/ReplicationIT.java
index aa2d97d..289dbf2 100644
--- a/asterixdb/asterix-server/src/test/java/org/apache/asterix/server/test/ReplicationIT.java
+++ b/asterixdb/asterix-server/src/test/java/org/apache/asterix/server/test/ReplicationIT.java
@@ -18,9 +18,19 @@
*/
package org.apache.asterix.server.test;
+import static org.apache.asterix.server.test.NCServiceExecutionIT.APP_HOME;
+import static org.apache.asterix.server.test.NCServiceExecutionIT.ASTERIX_APP_DIR;
+import static org.apache.asterix.server.test.NCServiceExecutionIT.INSTANCE_DIR;
+import static org.apache.asterix.server.test.NCServiceExecutionIT.LOG_DIR;
+import static org.apache.asterix.server.test.NCServiceExecutionIT.TARGET_DIR;
+
import java.io.File;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
import java.util.ArrayList;
import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
import java.util.concurrent.TimeUnit;
import org.apache.asterix.test.base.RetainLogsRule;
@@ -28,8 +38,6 @@
import org.apache.asterix.testframework.context.TestCaseContext;
import org.apache.commons.io.FileUtils;
import org.apache.commons.lang3.StringUtils;
-import org.apache.hyracks.server.process.HyracksCCProcess;
-import org.apache.hyracks.server.process.HyracksNCServiceProcess;
import org.apache.hyracks.server.process.HyracksVirtualCluster;
import org.apache.hyracks.util.file.FileUtil;
import org.apache.logging.log4j.LogManager;
@@ -42,38 +50,38 @@
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
-import static org.apache.asterix.server.test.NCServiceExecutionIT.APP_HOME;
-import static org.apache.asterix.server.test.NCServiceExecutionIT.INSTANCE_DIR;
-import static org.apache.asterix.server.test.NCServiceExecutionIT.TARGET_DIR;
-import static org.apache.asterix.server.test.NCServiceExecutionIT.ASTERIX_APP_DIR;
-import static org.apache.asterix.server.test.NCServiceExecutionIT.LOG_DIR;
-
@RunWith(Parameterized.class)
public class ReplicationIT {
- private static final String PATH_BASE = FileUtil.joinPath("src", "test", "resources", "integrationts",
- "replication");
- public static final String CONF_DIR = StringUtils.join(new String[] { TARGET_DIR, "test-classes", "ReplicationIT" },
- File.separator);
+ private static final String PATH_BASE =
+ FileUtil.joinPath("src", "test", "resources", "integrationts", "replication");
+ private static final String CONF_DIR =
+ StringUtils.join(new String[] { TARGET_DIR, "test-classes", "ReplicationIT" }, File.separator);
private static final String PATH_ACTUAL = FileUtil.joinPath("target", "ittest");
private static final Logger LOGGER = LogManager.getLogger();
private static String reportPath = new File(FileUtil.joinPath("target", "failsafe-reports")).getAbsolutePath();
-
- private final TestExecutor testExecutor = new TestExecutor();
- private TestCaseContext tcCtx;
- private static ProcessBuilder pb;
-
- private static HyracksCCProcess cc;
- private static HyracksNCServiceProcess nc1;
- private static HyracksNCServiceProcess nc2;
+ private static final TestExecutor testExecutor = new TestExecutor();
private static HyracksVirtualCluster cluster;
+ static {
+ final Map<String, InetSocketAddress> ncEndPoints = new HashMap<>();
+ final Map<String, InetSocketAddress> replicationAddress = new HashMap<>();
+ final String ip = InetAddress.getLoopbackAddress().getHostAddress();
+ ncEndPoints.put("asterix_nc1", InetSocketAddress.createUnresolved(ip, 19004));
+ ncEndPoints.put("asterix_nc2", InetSocketAddress.createUnresolved(ip, 19005));
+ replicationAddress.put("asterix_nc1", InetSocketAddress.createUnresolved(ip, 2001));
+ replicationAddress.put("asterix_nc2", InetSocketAddress.createUnresolved(ip, 2002));
+ testExecutor.setNcEndPoints(ncEndPoints);
+ testExecutor.setNcReplicationAddress(replicationAddress);
+ }
+
+ private TestCaseContext tcCtx;
+
public ReplicationIT(TestCaseContext tcCtx) {
this.tcCtx = tcCtx;
}
- @Rule
- public TestRule retainLogs = new RetainLogsRule(NCServiceExecutionIT.ASTERIX_APP_DIR, reportPath, this);
+ @Rule public TestRule retainLogs = new RetainLogsRule(NCServiceExecutionIT.ASTERIX_APP_DIR, reportPath, this);
@Before
public void before() throws Exception {
@@ -83,24 +91,19 @@
FileUtils.deleteDirectory(instanceDir);
}
- // HDFSCluster requires the input directory to end with a file separator.
-
cluster = new HyracksVirtualCluster(new File(APP_HOME), new File(ASTERIX_APP_DIR));
- nc1 = cluster.addNCService(new File(CONF_DIR, "ncservice1.conf"), new File(LOG_DIR, "ncservice1.log"));
-
- nc2 = cluster.addNCService(new File(CONF_DIR, "ncservice2.conf"), new File(LOG_DIR, "ncservice2.log"));
+ cluster.addNCService(new File(CONF_DIR, "ncservice1.conf"), new File(LOG_DIR, "ncservice1.log"));
+ cluster.addNCService(new File(CONF_DIR, "ncservice2.conf"), new File(LOG_DIR, "ncservice2.log"));
// Start CC
- cc = cluster.start(new File(CONF_DIR, "cc.conf"), new File(LOG_DIR, "cc.log"));
-
+ cluster.start(new File(CONF_DIR, "cc.conf"), new File(LOG_DIR, "cc.log"));
LOGGER.info("Instance created.");
testExecutor.waitForClusterActive(30, TimeUnit.SECONDS);
LOGGER.info("Instance is in ACTIVE state.");
-
}
@After
- public void after() throws Exception {
+ public void after() {
LOGGER.info("Destroying instance...");
cluster.stop();
LOGGER.info("Instance destroyed.");
@@ -120,7 +123,7 @@
return testArgs;
}
- protected static Collection<Object[]> buildTestsInXml(String xmlfile) throws Exception {
+ private static Collection<Object[]> buildTestsInXml(String xmlfile) throws Exception {
Collection<Object[]> testArgs = new ArrayList<>();
TestCaseContext.Builder b = new TestCaseContext.Builder();
for (TestCaseContext ctx : b.build(new File(PATH_BASE), xmlfile)) {
diff --git a/asterixdb/asterix-server/src/test/resources/MetadataReplicationIT/cc.conf b/asterixdb/asterix-server/src/test/resources/MetadataReplicationIT/cc.conf
deleted file mode 100644
index 32e38ae..0000000
--- a/asterixdb/asterix-server/src/test/resources/MetadataReplicationIT/cc.conf
+++ /dev/null
@@ -1,52 +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.
-
-[nc/asterix_nc1]
-txn.log.dir=../asterix-server/target/tmp/asterix_nc1/txnlog
-core.dump.dir=../asterix-server/target/tmp/asterix_nc1/coredump
-iodevices=../asterix-server/target/tmp/asterix_nc1/iodevice1,../asterix-server/target/tmp/asterix_nc1/iodevice2
-jvm.args=-agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=5006
-replication.listen.port=2001
-nc.api.port=19004
-
-[nc/asterix_nc2]
-ncservice.port=9091
-txn.log.dir=../asterix-server/target/tmp/asterix_nc2/txnlog
-core.dump.dir=../asterix-server/target/tmp/asterix_nc2/coredump
-iodevices=../asterix-server/target/tmp/asterix_nc2/iodevice1,../asterix-server/target/tmp/asterix_nc2/iodevice2
-jvm.args=-agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=5007
-replication.listen.port=2002
-nc.api.port=19005
-
-[nc]
-address=127.0.0.1
-command=asterixnc
-app.class=org.apache.asterix.hyracks.bootstrap.NCApplication
-jvm.args=-Xmx4096m -Dnode.Resolver="org.apache.asterix.external.util.IdentitiyResolverFactory"
-storage.subdir=test_storage
-storage.memorycomponent.globalbudget = 1073741824
-
-[cc]
-address = 127.0.0.1
-app.class=org.apache.asterix.hyracks.bootstrap.CCApplication
-heartbeat.period=2000
-
-[common]
-log.level = INFO
-replication.enabled=true
-replication.factor=2
-replication.strategy=metadata_only
diff --git a/asterixdb/asterix-server/src/test/resources/MetadataReplicationIT/ncservice1.conf b/asterixdb/asterix-server/src/test/resources/MetadataReplicationIT/ncservice1.conf
deleted file mode 100644
index ba10142..0000000
--- a/asterixdb/asterix-server/src/test/resources/MetadataReplicationIT/ncservice1.conf
+++ /dev/null
@@ -1,20 +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.
-
-[ncservice]
-logdir=../asterix-server/target/failsafe-reports
-
diff --git a/asterixdb/asterix-server/src/test/resources/MetadataReplicationIT/ncservice2.conf b/asterixdb/asterix-server/src/test/resources/MetadataReplicationIT/ncservice2.conf
deleted file mode 100644
index 2036584..0000000
--- a/asterixdb/asterix-server/src/test/resources/MetadataReplicationIT/ncservice2.conf
+++ /dev/null
@@ -1,21 +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.
-
-[ncservice]
-logdir=../asterix-server/target/failsafe-reports
-port=9091
-
diff --git a/asterixdb/asterix-server/src/test/resources/ReplicationIT/cc.conf b/asterixdb/asterix-server/src/test/resources/ReplicationIT/cc.conf
index 776a89a..2b68dd2 100644
--- a/asterixdb/asterix-server/src/test/resources/ReplicationIT/cc.conf
+++ b/asterixdb/asterix-server/src/test/resources/ReplicationIT/cc.conf
@@ -48,4 +48,5 @@
[common]
log.level = INFO
replication.enabled=true
+replication.strategy=all
replication.factor=2
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.1.ddl.aql b/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.1.ddl.aql
deleted file mode 100644
index 725ed61..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.1.ddl.aql
+++ /dev/null
@@ -1,50 +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.
- */
-/*
- * Test case Name : metadata_node_recovery.aql
- * Description : Make sure metadata only replication completes as expected.
- The test goes as follows:
- start metadata node and a replica, update metadata, kill metadata node,
- start metadata node again, validate metadata, update metadata, kill replica,
- start replica, update metadata, kill metadata node, start metadata node,
- validate metadata.
- * Expected Result : Success
- * Date : January 10 2017
- */
-drop dataverse Social if exists;
-create dataverse Social;
-use dataverse Social;
-
-create type EmploymentType as open {
- organization-name: string,
- start-date: date,
- end-date: date?
-}
-
-create type UserType as closed {
- id: int,
- alias: string,
- name: string,
- user-since: datetime,
- friend-ids: {{ int32 }},
- employment: [EmploymentType]
-}
-
-create dataset Users(UserType)
-primary key id;
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.10.node.aql b/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.10.node.aql
deleted file mode 100644
index f4685df..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.10.node.aql
+++ /dev/null
@@ -1,30 +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.
- */
-/*
- * Test case Name : metadata_node_recovery.aql
- * Description : Make sure metadata only replication completes as expected.
- The test goes as follows:
- start metadata node and a replica, update metadata, kill metadata node,
- start metadata node again, validate metadata, update metadata, kill replica,
- start replica, update metadata, kill metadata node, start metadata node,
- validate metadata.
- * Expected Result : Success
- * Date : January 10 2017
- */
-kill asterix_nc2
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.11.sleep.aql b/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.11.sleep.aql
deleted file mode 100644
index 9c50bf1..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.11.sleep.aql
+++ /dev/null
@@ -1,19 +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.
-#
-60000
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.13.sleep.aql b/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.13.sleep.aql
deleted file mode 100644
index dd83b4b..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.13.sleep.aql
+++ /dev/null
@@ -1,19 +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.
-#
-20000
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.14.ddl.aql b/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.14.ddl.aql
deleted file mode 100644
index eee7423..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.14.ddl.aql
+++ /dev/null
@@ -1,33 +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.
- */
-/*
- * Test case Name : metadata_node_recovery.aql
- * Description : Make sure metadata only replication completes as expected.
- The test goes as follows:
- start metadata node and a replica, update metadata, kill metadata node,
- start metadata node again, validate metadata, update metadata, kill replica,
- start replica, update metadata, kill metadata node, start metadata node,
- validate metadata.
- * Expected Result : Success
- * Date : January 10 2017
- */
-use dataverse Social;
-
-create dataset UsersAfterReplicaRecovery(UserType)
-primary key id;
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.15.node.aql b/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.15.node.aql
deleted file mode 100644
index 720f33e..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.15.node.aql
+++ /dev/null
@@ -1,30 +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.
- */
-/*
- * Test case Name : metadata_node_recovery.aql
- * Description : Make sure metadata only replication completes as expected.
- The test goes as follows:
- start metadata node and a replica, update metadata, kill metadata node,
- start metadata node again, validate metadata, update metadata, kill replica,
- start replica, update metadata, kill metadata node, start metadata node,
- validate metadata.
- * Expected Result : Success
- * Date : January 10 2017
- */
-kill asterix_nc1
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.16.sleep.aql b/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.16.sleep.aql
deleted file mode 100644
index 9c50bf1..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.16.sleep.aql
+++ /dev/null
@@ -1,19 +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.
-#
-60000
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.18.sleep.aql b/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.18.sleep.aql
deleted file mode 100644
index dd83b4b..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.18.sleep.aql
+++ /dev/null
@@ -1,19 +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.
-#
-20000
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.19.query.aql b/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.19.query.aql
deleted file mode 100644
index 5da6c59..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.19.query.aql
+++ /dev/null
@@ -1,35 +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.
- */
-/*
- * Test case Name : metadata_node_recovery.aql
- * Description : Make sure metadata only replication completes as expected.
- The test goes as follows:
- start metadata node and a replica, update metadata, kill metadata node,
- start metadata node again, validate metadata, update metadata, kill replica,
- start replica, update metadata, kill metadata node, start metadata node,
- validate metadata.
- * Expected Result : Success
- * Date : January 10 2017
- */
-count(
-for $x in dataset Metadata.Dataset
-where $x.DatasetName ='UsersAfterRecovery'
-or $x.DatasetName ='UsersAfterReplicaRecovery'
-return $x.DatasetName
-);
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.2.node.aql b/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.2.node.aql
deleted file mode 100644
index 720f33e..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.2.node.aql
+++ /dev/null
@@ -1,30 +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.
- */
-/*
- * Test case Name : metadata_node_recovery.aql
- * Description : Make sure metadata only replication completes as expected.
- The test goes as follows:
- start metadata node and a replica, update metadata, kill metadata node,
- start metadata node again, validate metadata, update metadata, kill replica,
- start replica, update metadata, kill metadata node, start metadata node,
- validate metadata.
- * Expected Result : Success
- * Date : January 10 2017
- */
-kill asterix_nc1
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.3.sleep.aql b/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.3.sleep.aql
deleted file mode 100644
index 9c50bf1..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.3.sleep.aql
+++ /dev/null
@@ -1,19 +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.
-#
-60000
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.4.get.http b/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.4.get.http
deleted file mode 100644
index fdacfd0..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.4.get.http
+++ /dev/null
@@ -1,30 +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.
- */
-/*
- * Test case Name : metadata_node_recovery.aql
- * Description : Make sure metadata only replication completes as expected.
- The test goes as follows:
- start metadata node and a replica, update metadata, kill metadata node,
- start metadata node again, validate metadata, update metadata, kill replica,
- start replica, update metadata, kill metadata node, start metadata node,
- validate metadata.
- * Expected Result : Success
- * Date : January 10 2017
- */
-/admin/cluster/summary
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.5.node.aql b/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.5.node.aql
deleted file mode 100644
index b750c64..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.5.node.aql
+++ /dev/null
@@ -1,19 +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.
- */
-start asterix_nc1
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.6.sleep.aql b/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.6.sleep.aql
deleted file mode 100644
index dd83b4b..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.6.sleep.aql
+++ /dev/null
@@ -1,19 +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.
-#
-20000
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.7.get.http b/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.7.get.http
deleted file mode 100644
index fdacfd0..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.7.get.http
+++ /dev/null
@@ -1,30 +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.
- */
-/*
- * Test case Name : metadata_node_recovery.aql
- * Description : Make sure metadata only replication completes as expected.
- The test goes as follows:
- start metadata node and a replica, update metadata, kill metadata node,
- start metadata node again, validate metadata, update metadata, kill replica,
- start replica, update metadata, kill metadata node, start metadata node,
- validate metadata.
- * Expected Result : Success
- * Date : January 10 2017
- */
-/admin/cluster/summary
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.8.query.aql b/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.8.query.aql
deleted file mode 100644
index 778e9f3..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.8.query.aql
+++ /dev/null
@@ -1,34 +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.
- */
-/*
- * Test case Name : metadata_node_recovery.aql
- * Description : Make sure metadata only replication completes as expected.
- The test goes as follows:
- start metadata node and a replica, update metadata, kill metadata node,
- start metadata node again, validate metadata, update metadata, kill replica,
- start replica, update metadata, kill metadata node, start metadata node,
- validate metadata.
- * Expected Result : Success
- * Date : January 10 2017
- */
-count(
-for $x in dataset Metadata.Dataset
-where $x.DatasetName ='Users'
-return $x.DatasetName
-);
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.9.ddl.aql b/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.9.ddl.aql
deleted file mode 100644
index 376df70..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.9.ddl.aql
+++ /dev/null
@@ -1,33 +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.
- */
-/*
- * Test case Name : metadata_node_recovery.aql
- * Description : Make sure metadata only replication completes as expected.
- The test goes as follows:
- start metadata node and a replica, update metadata, kill metadata node,
- start metadata node again, validate metadata, update metadata, kill replica,
- start replica, update metadata, kill metadata node, start metadata node,
- validate metadata.
- * Expected Result : Success
- * Date : January 10 2017
- */
-use dataverse Social;
-
-create dataset UsersAfterRecovery(UserType)
-primary key id;
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/results/metadata_recovery/metadata_node_recovery/metadata_node_recovery.query.19.adm b/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/results/metadata_recovery/metadata_node_recovery/metadata_node_recovery.query.19.adm
deleted file mode 100644
index d8263ee..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/results/metadata_recovery/metadata_node_recovery/metadata_node_recovery.query.19.adm
+++ /dev/null
@@ -1 +0,0 @@
-2
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/results/metadata_recovery/metadata_node_recovery/metadata_node_recovery.query.8.adm b/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/results/metadata_recovery/metadata_node_recovery/metadata_node_recovery.query.8.adm
deleted file mode 100644
index 56a6051..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/results/metadata_recovery/metadata_node_recovery/metadata_node_recovery.query.8.adm
+++ /dev/null
@@ -1 +0,0 @@
-1
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/testsuite.xml b/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/testsuite.xml
deleted file mode 100644
index d0ac325..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/testsuite.xml
+++ /dev/null
@@ -1,27 +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.
- !-->
-<test-suite xmlns="urn:xml.testframework.asterix.apache.org" ResultOffsetPath="results" QueryOffsetPath="queries" QueryFileExtension=".aql">
- <test-group name="metadata_recovery">
- <test-case FilePath="metadata_recovery">
- <compilation-unit name="metadata_node_recovery">
- <output-dir compare="Text">metadata_node_recovery</output-dir>
- </compilation-unit>
- </test-case>
- </test-group>
-</test-suite>
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.1.ddl.aql b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.1.ddl.aql
deleted file mode 100644
index 2c49a01..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.1.ddl.aql
+++ /dev/null
@@ -1,59 +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.
- */
-/*
- * 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/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.10.get.http b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.10.get.http
deleted file mode 100644
index 3faa945..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.10.get.http
+++ /dev/null
@@ -1,30 +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.
- */
-/*
- * 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
- */
-/admin/cluster/summary
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.11.query.aql b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.11.query.aql
deleted file mode 100644
index b09c3d3..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.11.query.aql
+++ /dev/null
@@ -1,33 +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.
- */
-/*
- * 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/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.2.update.aql b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.2.update.aql
deleted file mode 100644
index 377e097..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.2.update.aql
+++ /dev/null
@@ -1,36 +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.
- */
-/*
- * 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://../asterix-server/src/test/resources/integrationts/replication/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/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.3.node.aql b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.3.node.aql
deleted file mode 100644
index 9d5dc9b..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.3.node.aql
+++ /dev/null
@@ -1,28 +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.
- */
-/*
- * Test case Name : bulkload.aql
- * Description : Check that Bulkload LSM component are replicated correclty.
- The test goes as follows:
- start 2 nodes, bulkload a dataset, query data, kill one node
- and wait until the failover complete, query the data again.
- * Expected Result : Success
- * Date : January 6 2016
- */
-kill asterix_nc1
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.4.sleep.aql b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.4.sleep.aql
deleted file mode 100644
index 9c50bf1..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.4.sleep.aql
+++ /dev/null
@@ -1,19 +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.
-#
-60000
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.5.get.http b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.5.get.http
deleted file mode 100644
index 3faa945..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.5.get.http
+++ /dev/null
@@ -1,30 +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.
- */
-/*
- * 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
- */
-/admin/cluster/summary
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.6.query.aql b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.6.query.aql
deleted file mode 100644
index b09c3d3..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.6.query.aql
+++ /dev/null
@@ -1,33 +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.
- */
-/*
- * 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/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.7.update.aql b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.7.update.aql
deleted file mode 100644
index 56a88a2..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.7.update.aql
+++ /dev/null
@@ -1,51 +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.
- */
-/*
- * 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/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.8.node.aql b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.8.node.aql
deleted file mode 100644
index 98e2761..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.8.node.aql
+++ /dev/null
@@ -1,28 +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.
- */
-/*
- * Test case Name : bulkload.aql
- * Description : Check that Bulkload LSM component are replicated correclty.
- The test goes as follows:
- start 2 nodes, bulkload a dataset, query data, kill one node
- and wait until the failover complete, query the data again.
- * Expected Result : Success
- * Date : January 6 2016
- */
-start asterix_nc1
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.9.sleep.aql b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.9.sleep.aql
deleted file mode 100644
index dd83b4b..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failback/node_failback/node_failback.9.sleep.aql
+++ /dev/null
@@ -1,19 +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.
-#
-20000
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/bulkload/bulkload.1.ddl.aql b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/bulkload/bulkload.1.ddl.aql
deleted file mode 100644
index 6a2441e..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/bulkload/bulkload.1.ddl.aql
+++ /dev/null
@@ -1,54 +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.
- */
-/*
- * Test case Name : bulkload.aql
- * Description : Check that Bulkload LSM component are replicated correclty.
- The test goes as follows:
- start 2 nodes, bulkload a dataset, query data, kill one node
- and wait until the failover complete, query the data again.
- * Expected Result : Success
- * Date : January 6 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;
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/bulkload/bulkload.2.update.aql b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/bulkload/bulkload.2.update.aql
deleted file mode 100644
index 3cb5e80..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/bulkload/bulkload.2.update.aql
+++ /dev/null
@@ -1,33 +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.
- */
-/*
- * Test case Name : bulkload.aql
- * Description : Check that Bulkload LSM component are replicated correclty.
- The test goes as follows:
- start 2 nodes, bulkload a dataset, query data, kill one node
- and wait until the failover complete, query the data again.
- * Expected Result : Success
- * Date : January 6 2016
- */
-
-use dataverse TinySocial;
-
-load dataset FacebookUsers using localfs
-(("path"="asterix_nc1://../asterix-server/src/test/resources/integrationts/replication/data/fbu.adm"),
-("format"="adm"));
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/bulkload/bulkload.3.node.aql b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/bulkload/bulkload.3.node.aql
deleted file mode 100644
index 118b7e8..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/bulkload/bulkload.3.node.aql
+++ /dev/null
@@ -1,28 +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.
- */
-/*
- * Test case Name : bulkload.aql
- * Description : Check that Bulkload LSM component are replicated correclty.
- The test goes as follows:
- start 2 nodes, bulkload a dataset, query data, kill one node
- and wait until the failover complete, query the data again.
- * Expected Result : Success
- * Date : January 6 2016
- */
-kill asterix_nc2
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/bulkload/bulkload.4.sleep.aql b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/bulkload/bulkload.4.sleep.aql
deleted file mode 100644
index 9c50bf1..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/bulkload/bulkload.4.sleep.aql
+++ /dev/null
@@ -1,19 +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.
-#
-60000
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/bulkload/bulkload.5.query.aql b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/bulkload/bulkload.5.query.aql
deleted file mode 100644
index 9c8cb96..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/bulkload/bulkload.5.query.aql
+++ /dev/null
@@ -1,31 +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.
- */
-/*
- * Test case Name : bulkload.aql
- * Description : Check that Bulkload LSM component are replicated correclty.
- The test goes as follows:
- start 2 nodes, bulkload a dataset, query data, kill one node
- and wait until the failover complete, query the data again.
- * Expected Result : Success
- * Date : January 6 2016
- */
-
-use dataverse TinySocial;
-
-count (for $x in dataset FacebookUsers return $x);
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/mem_component_recovery/mem_component_recovery.1.ddl.aql b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/mem_component_recovery/mem_component_recovery.1.ddl.aql
deleted file mode 100644
index 8de2067..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/mem_component_recovery/mem_component_recovery.1.ddl.aql
+++ /dev/null
@@ -1,58 +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.
- */
-/*
- * Test case Name : mem_component_recovery.aql
- * Description : Check that Memory LSM component are replicated and recovered correclty.
- The test goes as follows:
- start 2 nodes, bulkload a dataset, copy it to in-memory dataset, query
- data from memory, kill one node and wait until the failover complete,
- query the data again.
- * Expected Result : Success
- * Date : January 6 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/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/mem_component_recovery/mem_component_recovery.2.update.aql b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/mem_component_recovery/mem_component_recovery.2.update.aql
deleted file mode 100644
index 6dd2cea..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/mem_component_recovery/mem_component_recovery.2.update.aql
+++ /dev/null
@@ -1,34 +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.
- */
-/*
- * Test case Name : mem_component_recovery.aql
- * Description : Check that Memory LSM component are replicated and recovered correclty.
- The test goes as follows:
- start 2 nodes, bulkload a dataset, copy it to in-memory dataset, query
- data from memory, kill one node and wait until the failover complete,
- query the data again.
- * Expected Result : Success
- * Date : January 6 2016
- */
-use dataverse TinySocial;
-
-load dataset FacebookUsers using localfs
-(("path"="asterix_nc1://../asterix-server/src/test/resources/integrationts/replication/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/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/mem_component_recovery/mem_component_recovery.3.node.aql b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/mem_component_recovery/mem_component_recovery.3.node.aql
deleted file mode 100644
index 118b7e8..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/mem_component_recovery/mem_component_recovery.3.node.aql
+++ /dev/null
@@ -1,28 +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.
- */
-/*
- * Test case Name : bulkload.aql
- * Description : Check that Bulkload LSM component are replicated correclty.
- The test goes as follows:
- start 2 nodes, bulkload a dataset, query data, kill one node
- and wait until the failover complete, query the data again.
- * Expected Result : Success
- * Date : January 6 2016
- */
-kill asterix_nc2
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/mem_component_recovery/mem_component_recovery.4.sleep.aql b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/mem_component_recovery/mem_component_recovery.4.sleep.aql
deleted file mode 100644
index 9c50bf1..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/mem_component_recovery/mem_component_recovery.4.sleep.aql
+++ /dev/null
@@ -1,19 +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.
-#
-60000
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/mem_component_recovery/mem_component_recovery.5.query.aql b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/mem_component_recovery/mem_component_recovery.5.query.aql
deleted file mode 100644
index e25e409..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/mem_component_recovery/mem_component_recovery.5.query.aql
+++ /dev/null
@@ -1,32 +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.
- */
-/*
- * Test case Name : mem_component_recovery.aql
- * Description : Check that Memory LSM component are replicated and recovered correclty.
- The test goes as follows:
- start 2 nodes, bulkload a dataset, copy it to in-memory dataset, query
- data from memory, kill one node and wait until the failover complete,
- query the data again.
- * Expected Result : Success
- * Date : January 6 2016
- */
-
-use dataverse TinySocial;
-
-count (for $x in dataset FacebookUsersInMemory return $x);
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/metadata_node/metadata_node.1.ddl.aql b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/metadata_node/metadata_node.1.ddl.aql
deleted file mode 100644
index 113d144..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/metadata_node/metadata_node.1.ddl.aql
+++ /dev/null
@@ -1,55 +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.
- */
-/*
- * Test case Name : metadata_node_recovery.aql
- * Description : Check that metadata node failover is done correctly.
- The test goes as follows:
- start 2 nodes, create a dataset, kill metadata node
- and wait until the failover complete, verify the
- dataset still exists.
- * Expected Result : Success
- * Date : January 6 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;
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/metadata_node/metadata_node.2.node.aql b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/metadata_node/metadata_node.2.node.aql
deleted file mode 100644
index 9d5dc9b..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/metadata_node/metadata_node.2.node.aql
+++ /dev/null
@@ -1,28 +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.
- */
-/*
- * Test case Name : bulkload.aql
- * Description : Check that Bulkload LSM component are replicated correclty.
- The test goes as follows:
- start 2 nodes, bulkload a dataset, query data, kill one node
- and wait until the failover complete, query the data again.
- * Expected Result : Success
- * Date : January 6 2016
- */
-kill asterix_nc1
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/metadata_node/metadata_node.3.sleep.aql b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/metadata_node/metadata_node.3.sleep.aql
deleted file mode 100644
index 9c50bf1..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/metadata_node/metadata_node.3.sleep.aql
+++ /dev/null
@@ -1,19 +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.
-#
-60000
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/metadata_node/metadata_node.4.query.aql b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/metadata_node/metadata_node.4.query.aql
deleted file mode 100644
index 3d525f6..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/metadata_node/metadata_node.4.query.aql
+++ /dev/null
@@ -1,36 +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.
- */
-/*
- * Test case Name : metadata_node_recovery.aql
- * Description : Check that metadata node failover is done correctly.
- The test goes as follows:
- start 2 nodes, create a dataset, kill metadata node
- and wait until the failover complete, verify the
- dataset still exists.
- * Expected Result : Success
- * Date : January 6 2016
- */
-
-use dataverse TinySocial;
-
-count(
-for $x in dataset Metadata.Dataset
-where $x.DatasetName ='FacebookUsers'
-return $x.DatasetName
-);
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/resync_failed_replica/resync_failed_replica.1.sto.cmd
similarity index 94%
copy from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
copy to asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/resync_failed_replica/resync_failed_replica.1.sto.cmd
index b750c64..7ddaa20 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/resync_failed_replica/resync_failed_replica.1.sto.cmd
@@ -16,4 +16,4 @@
* specific language governing permissions and limitations
* under the License.
*/
-start asterix_nc1
\ No newline at end of file
+nc:asterix_nc1 /addReplica 0 asterix_nc2
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/resync_failed_replica/resync_failed_replica.10.pollget.http
similarity index 91%
copy from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
copy to asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/resync_failed_replica/resync_failed_replica.10.pollget.http
index b750c64..6867a5d 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/resync_failed_replica/resync_failed_replica.10.pollget.http
@@ -16,4 +16,6 @@
* specific language governing permissions and limitations
* under the License.
*/
-start asterix_nc1
\ No newline at end of file
+//polltimeoutsecs=30
+
+nc:asterix_nc1 /admin/storage/partition/0
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/resync_failed_replica/resync_failed_replica.11.post.http
similarity index 92%
copy from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
copy to asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/resync_failed_replica/resync_failed_replica.11.post.http
index b750c64..2e8fc63 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/resync_failed_replica/resync_failed_replica.11.post.http
@@ -16,4 +16,4 @@
* specific language governing permissions and limitations
* under the License.
*/
-start asterix_nc1
\ No newline at end of file
+/admin/cluster/partition/master?partition=0&node=asterix_nc2
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/resync_failed_replica/resync_failed_replica.12.post.http
similarity index 93%
copy from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
copy to asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/resync_failed_replica/resync_failed_replica.12.post.http
index b750c64..e8dca0b 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/resync_failed_replica/resync_failed_replica.12.post.http
@@ -16,4 +16,4 @@
* specific language governing permissions and limitations
* under the License.
*/
-start asterix_nc1
\ No newline at end of file
+/admin/cluster/metadataNode?node=asterix_nc2
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/resync_failed_replica/resync_failed_replica.13.pollget.http
similarity index 93%
copy from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
copy to asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/resync_failed_replica/resync_failed_replica.13.pollget.http
index b750c64..32e2f78 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/resync_failed_replica/resync_failed_replica.13.pollget.http
@@ -16,4 +16,6 @@
* specific language governing permissions and limitations
* under the License.
*/
-start asterix_nc1
\ No newline at end of file
+//polltimeoutsecs=30
+
+/admin/cluster/summary
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/resync_failed_replica/resync_failed_replica.14.query.sqlpp
similarity index 90%
copy from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
copy to asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/resync_failed_replica/resync_failed_replica.14.query.sqlpp
index b750c64..a612cbb 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/resync_failed_replica/resync_failed_replica.14.query.sqlpp
@@ -16,4 +16,6 @@
* specific language governing permissions and limitations
* under the License.
*/
-start asterix_nc1
\ No newline at end of file
+SELECT value count(*)
+FROM Metadata.`Dataset`
+WHERE DatasetName = 'ds_1';
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/resync_failed_replica/resync_failed_replica.15.ddl.sqlpp
similarity index 94%
copy from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
copy to asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/resync_failed_replica/resync_failed_replica.15.ddl.sqlpp
index b750c64..f96d5a8 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/resync_failed_replica/resync_failed_replica.15.ddl.sqlpp
@@ -16,4 +16,4 @@
* specific language governing permissions and limitations
* under the License.
*/
-start asterix_nc1
\ No newline at end of file
+CREATE DATASET ds_2(MyType) PRIMARY KEY id;
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/resync_failed_replica/resync_failed_replica.16.query.sqlpp
similarity index 90%
copy from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
copy to asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/resync_failed_replica/resync_failed_replica.16.query.sqlpp
index b750c64..555954d 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/resync_failed_replica/resync_failed_replica.16.query.sqlpp
@@ -16,4 +16,6 @@
* specific language governing permissions and limitations
* under the License.
*/
-start asterix_nc1
\ No newline at end of file
+SELECT value count(*)
+FROM Metadata.`Dataset`
+WHERE DatasetName = 'ds_2';
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/resync_failed_replica/resync_failed_replica.17.sto.cmd
similarity index 94%
copy from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
copy to asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/resync_failed_replica/resync_failed_replica.17.sto.cmd
index b750c64..71621ac 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/resync_failed_replica/resync_failed_replica.17.sto.cmd
@@ -16,4 +16,4 @@
* specific language governing permissions and limitations
* under the License.
*/
-start asterix_nc1
\ No newline at end of file
+nc:asterix_nc1 /removeReplica 0 asterix_nc2
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/resync_failed_replica/resync_failed_replica.2.pollget.http
similarity index 91%
copy from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
copy to asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/resync_failed_replica/resync_failed_replica.2.pollget.http
index b750c64..6867a5d 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/resync_failed_replica/resync_failed_replica.2.pollget.http
@@ -16,4 +16,6 @@
* specific language governing permissions and limitations
* under the License.
*/
-start asterix_nc1
\ No newline at end of file
+//polltimeoutsecs=30
+
+nc:asterix_nc1 /admin/storage/partition/0
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/resync_failed_replica/resync_failed_replica.3.ddl.sqlpp
similarity index 89%
copy from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
copy to asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/resync_failed_replica/resync_failed_replica.3.ddl.sqlpp
index b750c64..15bc3c5 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/resync_failed_replica/resync_failed_replica.3.ddl.sqlpp
@@ -16,4 +16,8 @@
* specific language governing permissions and limitations
* under the License.
*/
-start asterix_nc1
\ No newline at end of file
+CREATE TYPE MyType AS {
+ id : int
+};
+
+CREATE DATASET ds_1(MyType) PRIMARY KEY id;
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/resync_failed_replica/resync_failed_replica.4.node.cmd
similarity index 97%
rename from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
rename to asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/resync_failed_replica/resync_failed_replica.4.node.cmd
index b750c64..a04c093 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/resync_failed_replica/resync_failed_replica.4.node.cmd
@@ -16,4 +16,4 @@
* specific language governing permissions and limitations
* under the License.
*/
-start asterix_nc1
\ No newline at end of file
+kill asterix_nc2
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/resync_failed_replica/resync_failed_replica.5.pollget.http
similarity index 93%
copy from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
copy to asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/resync_failed_replica/resync_failed_replica.5.pollget.http
index b750c64..777e3dd 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/resync_failed_replica/resync_failed_replica.5.pollget.http
@@ -16,4 +16,6 @@
* specific language governing permissions and limitations
* under the License.
*/
-start asterix_nc1
\ No newline at end of file
+//polltimeoutsecs=60
+
+/admin/cluster/summary
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/resync_failed_replica/resync_failed_replica.6.pollget.http
similarity index 91%
copy from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
copy to asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/resync_failed_replica/resync_failed_replica.6.pollget.http
index b750c64..6867a5d 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/resync_failed_replica/resync_failed_replica.6.pollget.http
@@ -16,4 +16,6 @@
* specific language governing permissions and limitations
* under the License.
*/
-start asterix_nc1
\ No newline at end of file
+//polltimeoutsecs=30
+
+nc:asterix_nc1 /admin/storage/partition/0
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.12.node.aql b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/resync_failed_replica/resync_failed_replica.7.node.cmd
similarity index 100%
rename from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.12.node.aql
rename to asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/resync_failed_replica/resync_failed_replica.7.node.cmd
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/resync_failed_replica/resync_failed_replica.8.pollget.http
similarity index 93%
copy from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
copy to asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/resync_failed_replica/resync_failed_replica.8.pollget.http
index b750c64..777e3dd 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/resync_failed_replica/resync_failed_replica.8.pollget.http
@@ -16,4 +16,6 @@
* specific language governing permissions and limitations
* under the License.
*/
-start asterix_nc1
\ No newline at end of file
+//polltimeoutsecs=60
+
+/admin/cluster/summary
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/resync_failed_replica/resync_failed_replica.9.sto.cmd
similarity index 94%
copy from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
copy to asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/resync_failed_replica/resync_failed_replica.9.sto.cmd
index b750c64..7ddaa20 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/queries/metadata_recovery/metadata_node_recovery/metadata_node_recovery.17.node.aql
+++ b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/resync_failed_replica/resync_failed_replica.9.sto.cmd
@@ -16,4 +16,4 @@
* specific language governing permissions and limitations
* under the License.
*/
-start asterix_nc1
\ No newline at end of file
+nc:asterix_nc1 /addReplica 0 asterix_nc2
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failback/node_failback/node_failback.query.11.adm b/asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failback/node_failback/node_failback.query.11.adm
deleted file mode 100644
index 2edeafb..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failback/node_failback/node_failback.query.11.adm
+++ /dev/null
@@ -1 +0,0 @@
-20
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failback/node_failback/node_failback.query.6.adm b/asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failback/node_failback/node_failback.query.6.adm
deleted file mode 100644
index 9a03714..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failback/node_failback/node_failback.query.6.adm
+++ /dev/null
@@ -1 +0,0 @@
-10
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failover/bulkload/bulkload.5.adm b/asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failover/bulkload/bulkload.5.adm
deleted file mode 100644
index 9a03714..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failover/bulkload/bulkload.5.adm
+++ /dev/null
@@ -1 +0,0 @@
-10
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failover/mem_component_recovery/mem_component_recovery.5.adm b/asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failover/mem_component_recovery/mem_component_recovery.5.adm
deleted file mode 100644
index 9a03714..0000000
--- a/asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failover/mem_component_recovery/mem_component_recovery.5.adm
+++ /dev/null
@@ -1 +0,0 @@
-10
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failover/resync_failed_replica/resync_failed_replica.10.adm b/asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failover/resync_failed_replica/resync_failed_replica.10.adm
new file mode 100644
index 0000000..7e92c87
--- /dev/null
+++ b/asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failover/resync_failed_replica/resync_failed_replica.10.adm
@@ -0,0 +1,7 @@
+[ {
+ "partition" : 0,
+ "replicas" : [ {
+ "location" : "127.0.0.1:2002",
+ "status" : "IN_SYNC"
+ } ]
+} ]
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failover/resync_failed_replica/resync_failed_replica.11.adm b/asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failover/resync_failed_replica/resync_failed_replica.11.adm
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failover/resync_failed_replica/resync_failed_replica.11.adm
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failover/resync_failed_replica/resync_failed_replica.12.adm b/asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failover/resync_failed_replica/resync_failed_replica.12.adm
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failover/resync_failed_replica/resync_failed_replica.12.adm
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failback/node_failback/node_failback.cluster_state.5.adm b/asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failover/resync_failed_replica/resync_failed_replica.13.adm
similarity index 94%
copy from asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failback/node_failback/node_failback.cluster_state.5.adm
copy to asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failover/resync_failed_replica/resync_failed_replica.13.adm
index 15c200c..fa5cfb4 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failback/node_failback/node_failback.cluster_state.5.adm
+++ b/asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failover/resync_failed_replica/resync_failed_replica.13.adm
@@ -11,7 +11,7 @@
},
"1" : {
"active" : true,
- "activeNodeId" : "asterix_nc2",
+ "activeNodeId" : "asterix_nc1",
"iodeviceNum" : 1,
"nodeId" : "asterix_nc1",
"partitionId" : 1,
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failover/metadata_node/metadata_node.4.adm b/asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failover/resync_failed_replica/resync_failed_replica.14.adm
similarity index 100%
rename from asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failover/metadata_node/metadata_node.4.adm
rename to asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failover/resync_failed_replica/resync_failed_replica.14.adm
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failover/resync_failed_replica/resync_failed_replica.16.adm b/asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failover/resync_failed_replica/resync_failed_replica.16.adm
new file mode 100644
index 0000000..d0138cb
--- /dev/null
+++ b/asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failover/resync_failed_replica/resync_failed_replica.16.adm
@@ -0,0 +1,3 @@
+1
+
+
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failover/resync_failed_replica/resync_failed_replica.2.adm b/asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failover/resync_failed_replica/resync_failed_replica.2.adm
new file mode 100644
index 0000000..7e92c87
--- /dev/null
+++ b/asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failover/resync_failed_replica/resync_failed_replica.2.adm
@@ -0,0 +1,7 @@
+[ {
+ "partition" : 0,
+ "replicas" : [ {
+ "location" : "127.0.0.1:2002",
+ "status" : "IN_SYNC"
+ } ]
+} ]
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/results/metadata_recovery/metadata_node_recovery/metadata_node_recovery.cluster_state.4.adm b/asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failover/resync_failed_replica/resync_failed_replica.5.adm
similarity index 99%
rename from asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/results/metadata_recovery/metadata_node_recovery/metadata_node_recovery.cluster_state.4.adm
rename to asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failover/resync_failed_replica/resync_failed_replica.5.adm
index c327ecf..8d5d123 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/metadata_only_replication/results/metadata_recovery/metadata_node_recovery/metadata_node_recovery.cluster_state.4.adm
+++ b/asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failover/resync_failed_replica/resync_failed_replica.5.adm
@@ -2,7 +2,7 @@
"metadata_node" : "asterix_nc1",
"partitions" : {
"0" : {
- "active" : false,
+ "active" : true,
"activeNodeId" : "asterix_nc1",
"iodeviceNum" : 0,
"nodeId" : "asterix_nc1",
@@ -10,7 +10,7 @@
"pendingActivation" : false
},
"1" : {
- "active" : false,
+ "active" : true,
"activeNodeId" : "asterix_nc1",
"iodeviceNum" : 1,
"nodeId" : "asterix_nc1",
@@ -18,7 +18,7 @@
"pendingActivation" : false
},
"2" : {
- "active" : true,
+ "active" : false,
"activeNodeId" : "asterix_nc2",
"iodeviceNum" : 0,
"nodeId" : "asterix_nc2",
@@ -26,7 +26,7 @@
"pendingActivation" : false
},
"3" : {
- "active" : true,
+ "active" : false,
"activeNodeId" : "asterix_nc2",
"iodeviceNum" : 1,
"nodeId" : "asterix_nc2",
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failover/resync_failed_replica/resync_failed_replica.6.adm b/asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failover/resync_failed_replica/resync_failed_replica.6.adm
new file mode 100644
index 0000000..4da3d42
--- /dev/null
+++ b/asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failover/resync_failed_replica/resync_failed_replica.6.adm
@@ -0,0 +1,7 @@
+[ {
+ "partition" : 0,
+ "replicas" : [ {
+ "location" : "127.0.0.1:2002",
+ "status" : "DISCONNECTED"
+ } ]
+} ]
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failback/node_failback/node_failback.cluster_state.10.adm b/asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failover/resync_failed_replica/resync_failed_replica.8.adm
similarity index 100%
rename from asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failback/node_failback/node_failback.cluster_state.10.adm
rename to asterixdb/asterix-server/src/test/resources/integrationts/replication/results/failover/resync_failed_replica/resync_failed_replica.8.adm
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/replication/testsuite.xml b/asterixdb/asterix-server/src/test/resources/integrationts/replication/testsuite.xml
index 0f6b528..df5dbac 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/replication/testsuite.xml
+++ b/asterixdb/asterix-server/src/test/resources/integrationts/replication/testsuite.xml
@@ -19,25 +19,8 @@
<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="failback">
- <test-case FilePath="failback">
- <compilation-unit name="node_failback">
- <output-dir compare="Text">node_failback</output-dir>
+ <compilation-unit name="resync_failed_replica">
+ <output-dir compare="Text">resync_failed_replica</output-dir>
</compilation-unit>
</test-case>
</test-group>
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepository.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepository.java
index 601dec3..54d6268 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepository.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepository.java
@@ -18,37 +18,37 @@
*/
package org.apache.asterix.transaction.management.resource;
+import static org.apache.asterix.common.utils.StorageConstants.INDEX_CHECKPOINT_FILE_PREFIX;
import static org.apache.hyracks.api.exceptions.ErrorCode.CANNOT_CREATE_FILE;
import java.io.File;
import java.io.FileInputStream;
import java.io.FileOutputStream;
+import java.io.FilenameFilter;
import java.io.IOException;
import java.io.ObjectInputStream;
import java.io.ObjectOutputStream;
import java.nio.file.Files;
import java.nio.file.Path;
import java.nio.file.Paths;
+import java.util.ArrayList;
import java.util.Collection;
-import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
import java.util.function.Predicate;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
-import org.apache.asterix.common.cluster.ClusterPartition;
-import org.apache.asterix.common.config.MetadataProperties;
import org.apache.asterix.common.dataflow.DatasetLocalResource;
import org.apache.asterix.common.exceptions.AsterixException;
import org.apache.asterix.common.replication.IReplicationManager;
import org.apache.asterix.common.replication.ReplicationJob;
import org.apache.asterix.common.storage.DatasetResourceReference;
import org.apache.asterix.common.storage.IIndexCheckpointManagerProvider;
-import org.apache.asterix.common.storage.ResourceReference;
import org.apache.asterix.common.utils.StorageConstants;
import org.apache.asterix.common.utils.StoragePathUtil;
import org.apache.commons.io.FileUtils;
@@ -71,7 +71,8 @@
public class PersistentLocalResourceRepository implements ILocalResourceRepository {
public static final Predicate<Path> INDEX_COMPONENTS = path -> !path.endsWith(StorageConstants.METADATA_FILE_NAME);
- // Private constants
+ private static final FilenameFilter LSM_INDEX_FILES_FILTER =
+ (dir, name) -> !name.startsWith(INDEX_CHECKPOINT_FILE_PREFIX);
private static final int MAX_CACHED_RESOURCES = 1000;
private static final IOFileFilter METADATA_FILES_FILTER = new IOFileFilter() {
@Override
@@ -100,17 +101,14 @@
// Finals
private final IIOManager ioManager;
private final Cache<String, LocalResource> resourceCache;
- private final Set<Integer> nodeOriginalPartitions;
- private final Set<Integer> nodeActivePartitions;
// Mutables
private boolean isReplicationEnabled = false;
private Set<String> filesToBeReplicated;
private IReplicationManager replicationManager;
- private Set<Integer> nodeInactivePartitions;
private final Path[] storageRoots;
private final IIndexCheckpointManagerProvider indexCheckpointManagerProvider;
- public PersistentLocalResourceRepository(IIOManager ioManager, String nodeId, MetadataProperties metadataProperties,
+ public PersistentLocalResourceRepository(IIOManager ioManager,
IIndexCheckpointManagerProvider indexCheckpointManagerProvider) {
this.ioManager = ioManager;
this.indexCheckpointManagerProvider = indexCheckpointManagerProvider;
@@ -122,15 +120,6 @@
}
createStorageRoots();
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);
- nodeInactivePartitions = new HashSet<>(nodePartitions.length);
- for (ClusterPartition partition : nodePartitions) {
- nodeOriginalPartitions.add(partition.getPartitionId());
- nodeActivePartitions.add(partition.getPartitionId());
- }
}
@Override
@@ -267,7 +256,6 @@
if (isReplicationEnabled) {
filesToBeReplicated = new HashSet<>();
- nodeInactivePartitions = ConcurrentHashMap.newKeySet();
}
}
@@ -299,26 +287,13 @@
createStorageRoots();
}
- public Set<Integer> getActivePartitions() {
- return Collections.unmodifiableSet(nodeActivePartitions);
- }
-
- public Set<Integer> getInactivePartitions() {
- return Collections.unmodifiableSet(nodeInactivePartitions);
- }
-
- public synchronized void addActivePartition(int partitonId) {
- nodeActivePartitions.add(partitonId);
- nodeInactivePartitions.remove(partitonId);
- }
-
- public synchronized void addInactivePartition(int partitonId) {
- nodeInactivePartitions.add(partitonId);
- nodeActivePartitions.remove(partitonId);
+ public Set<Integer> getAllPartitions() throws HyracksDataException {
+ return loadAndGetAllResources().values().stream().map(LocalResource::getResource)
+ .map(DatasetLocalResource.class::cast).map(DatasetLocalResource::getPartition)
+ .collect(Collectors.toSet());
}
public DatasetResourceReference getLocalResourceReference(String absoluteFilePath) throws HyracksDataException {
- //TODO pass relative path
final String localResourcePath = StoragePathUtil.getIndexFileRelativePath(absoluteFilePath);
final LocalResource lr = get(localResourcePath);
return DatasetResourceReference.of(lr);
@@ -351,17 +326,18 @@
});
}
- /**
- * Given any index file, an absolute {@link FileReference} is returned which points to where the index of
- * {@code indexFile} is located.
- *
- * @param indexFile
- * @return
- * @throws HyracksDataException
- */
- public FileReference getIndexPath(Path indexFile) throws HyracksDataException {
- final ResourceReference ref = ResourceReference.of(indexFile.toString());
- return ioManager.resolve(ref.getRelativePath().toString());
+ public List<String> getPartitionIndexesFiles(int partition) throws HyracksDataException {
+ List<String> partitionFiles = new ArrayList<>();
+ Set<File> partitionIndexes = getPartitionIndexes(partition);
+ for (File indexDir : partitionIndexes) {
+ if (indexDir.isDirectory()) {
+ File[] indexFiles = indexDir.listFiles(LSM_INDEX_FILES_FILTER);
+ if (indexFiles != null) {
+ Stream.of(indexFiles).map(File::getAbsolutePath).forEach(partitionFiles::add);
+ }
+ }
+ }
+ return partitionFiles;
}
private void createStorageRoots() {
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepositoryFactory.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepositoryFactory.java
index 33c6260..d15e6ff 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepositoryFactory.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepositoryFactory.java
@@ -18,30 +18,23 @@
*/
package org.apache.asterix.transaction.management.resource;
-import org.apache.asterix.common.config.MetadataProperties;
import org.apache.asterix.common.storage.IIndexCheckpointManagerProvider;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.api.io.IIOManager;
import org.apache.hyracks.storage.common.ILocalResourceRepository;
import org.apache.hyracks.storage.common.file.ILocalResourceRepositoryFactory;
public class PersistentLocalResourceRepositoryFactory implements ILocalResourceRepositoryFactory {
private final IIOManager ioManager;
- private final String nodeId;
- private final MetadataProperties metadataProperties;
private final IIndexCheckpointManagerProvider indexCheckpointManagerProvider;
- public PersistentLocalResourceRepositoryFactory(IIOManager ioManager, String nodeId,
- MetadataProperties metadataProperties, IIndexCheckpointManagerProvider indexCheckpointManagerProvider) {
+ public PersistentLocalResourceRepositoryFactory(IIOManager ioManager,
+ IIndexCheckpointManagerProvider indexCheckpointManagerProvider) {
this.ioManager = ioManager;
- this.nodeId = nodeId;
- this.metadataProperties = metadataProperties;
this.indexCheckpointManagerProvider = indexCheckpointManagerProvider;
}
@Override
- public ILocalResourceRepository createRepository() throws HyracksDataException {
- return new PersistentLocalResourceRepository(ioManager, nodeId, metadataProperties,
- indexCheckpointManagerProvider);
+ public ILocalResourceRepository createRepository() {
+ return new PersistentLocalResourceRepository(ioManager, indexCheckpointManagerProvider);
}
}
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogBuffer.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogBuffer.java
index b67da80..011d2a1 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogBuffer.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogBuffer.java
@@ -26,10 +26,10 @@
import org.apache.asterix.common.context.PrimaryIndexOperationTracker;
import org.apache.asterix.common.exceptions.ACIDException;
-import org.apache.asterix.common.replication.IReplicationThread;
import org.apache.asterix.common.transactions.DatasetId;
import org.apache.asterix.common.transactions.ILogBuffer;
import org.apache.asterix.common.transactions.ILogRecord;
+import org.apache.asterix.common.transactions.ILogRequester;
import org.apache.asterix.common.transactions.ITransactionContext;
import org.apache.asterix.common.transactions.ITransactionSubsystem;
import org.apache.asterix.common.transactions.LogRecord;
@@ -318,10 +318,9 @@
}
}
logRecord.isFlushed(true);
- IReplicationThread replicationThread = logRecord.getReplicationThread();
-
- if (replicationThread != null) {
- replicationThread.notifyLogReplicationRequester(logRecord);
+ final ILogRequester logRequester = logRecord.getRequester();
+ if (logRequester != null) {
+ logRequester.notifyFlushed(logRecord);
}
}
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogManagerWithReplication.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogManagerWithReplication.java
index 9b6a4f9..53cd038 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogManagerWithReplication.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogManagerWithReplication.java
@@ -61,7 +61,7 @@
shouldReplicate = false;
}
}
- logRecord.setReplicated(shouldReplicate);
+ logRecord.setReplicate(shouldReplicate);
//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) {
@@ -76,9 +76,9 @@
protected void appendToLogTail(ILogRecord logRecord) {
syncAppendToLogTail(logRecord);
- if (logRecord.isReplicated()) {
+ if (logRecord.isReplicate()) {
try {
- replicationManager.replicateLog(logRecord);
+ replicationManager.replicate(logRecord);
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
throw new ACIDException(e);
@@ -92,9 +92,9 @@
logRecord.wait();
}
//wait for job Commit/Abort ACK from replicas
- if (logRecord.isReplicated() && (logRecord.getLogType() == LogType.JOB_COMMIT
+ if (logRecord.isReplicate() && (logRecord.getLogType() == LogType.JOB_COMMIT
|| logRecord.getLogType() == LogType.ABORT)) {
- while (!replicationManager.hasBeenReplicated(logRecord)) {
+ while (!logRecord.isReplicated()) {
logRecord.wait();
}
}
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/recovery/CheckpointManagerFactory.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/recovery/CheckpointManagerFactory.java
index 68c5ce1..ca7d37a 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/recovery/CheckpointManagerFactory.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/recovery/CheckpointManagerFactory.java
@@ -29,11 +29,7 @@
}
public static ICheckpointManager create(ITransactionSubsystem txnSubsystem,
- CheckpointProperties checkpointProperties, boolean replicationEnabled) {
- if (!replicationEnabled) {
- return new CheckpointManager(txnSubsystem, checkpointProperties);
- } else {
- return new ReplicationCheckpointManager(txnSubsystem, checkpointProperties);
- }
+ CheckpointProperties checkpointProperties) {
+ return new CheckpointManager(txnSubsystem, checkpointProperties);
}
}
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/recovery/ReplicationCheckpointManager.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/recovery/ReplicationCheckpointManager.java
deleted file mode 100644
index 4bbcabe..0000000
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/recovery/ReplicationCheckpointManager.java
+++ /dev/null
@@ -1,144 +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.transaction.management.service.recovery;
-
-import java.io.IOException;
-import java.util.HashSet;
-import java.util.Set;
-
-import org.apache.asterix.common.api.IApplicationContext;
-import org.apache.asterix.common.api.IDatasetLifecycleManager;
-import org.apache.asterix.common.cluster.ClusterPartition;
-import org.apache.asterix.common.config.MetadataProperties;
-import org.apache.asterix.common.replication.IReplicaResourcesManager;
-import org.apache.asterix.common.replication.IReplicationManager;
-import org.apache.asterix.common.transactions.CheckpointProperties;
-import org.apache.asterix.common.transactions.ICheckpointManager;
-import org.apache.asterix.common.transactions.ITransactionSubsystem;
-import org.apache.asterix.transaction.management.resource.PersistentLocalResourceRepository;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.logging.log4j.LogManager;
-import org.apache.logging.log4j.Logger;
-
-/**
- * An implementation of {@link ICheckpointManager} that defines the logic
- * of checkpoints when replication is enabled..
- */
-public class ReplicationCheckpointManager extends AbstractCheckpointManager {
-
- private static final Logger LOGGER = LogManager.getLogger();
-
- public ReplicationCheckpointManager(ITransactionSubsystem txnSubsystem, CheckpointProperties checkpointProperties) {
- super(txnSubsystem, checkpointProperties);
- }
-
- /**
- * Performs a sharp checkpoint. All datasets are flushed and all transaction
- * log files are deleted except the files that are needed for dead replicas.
- */
- @Override
- public synchronized void doSharpCheckpoint() throws HyracksDataException {
- LOGGER.info("Starting sharp checkpoint...");
- final IDatasetLifecycleManager datasetLifecycleManager =
- txnSubsystem.getAsterixAppRuntimeContextProvider().getDatasetLifecycleManager();
- datasetLifecycleManager.flushAllDatasets();
- long minFirstLSN;
- // If shutting down, need to check if we need to keep any remote logs for dead replicas
- if (txnSubsystem.getAsterixAppRuntimeContextProvider().getAppContext().isShuttingdown()) {
- final Set<String> deadReplicaIds = txnSubsystem.getAsterixAppRuntimeContextProvider().getAppContext()
- .getReplicationManager().getDeadReplicasIds();
- if (deadReplicaIds.isEmpty()) {
- // No dead replicas => no need to keep any log
- minFirstLSN = SHARP_CHECKPOINT_LSN;
- } else {
- // Get min LSN of dead replicas remote resources
- minFirstLSN = getDeadReplicasMinFirstLSN(deadReplicaIds);
- }
- } else {
- // Start up complete checkpoint. Avoid deleting remote recovery logs.
- minFirstLSN = txnSubsystem.getRecoveryManager().getMinFirstLSN();
- }
- capture(minFirstLSN, true);
- if (minFirstLSN == SHARP_CHECKPOINT_LSN) {
- // No need to keep any logs
- txnSubsystem.getLogManager().renewLogFiles();
- } else {
- // Delete only log files with LSNs < any dead replica partition minimum LSN
- txnSubsystem.getLogManager().deleteOldLogFiles(minFirstLSN);
- }
- LOGGER.info("Completed sharp checkpoint.");
- }
-
- /***
- * Attempts to perform a soft checkpoint at the specified {@code checkpointTargetLSN}.
- * If a checkpoint cannot be captured due to datasets having LSN < {@code checkpointTargetLSN},
- * an asynchronous flush is triggered on them. If the checkpoint fails due to a replica index,
- * a request is sent to the primary replica of the index to flush it.
- * When a checkpoint is successful, all transaction log files that end with
- * LSN < {@code checkpointTargetLSN} are deleted.
- */
- @Override
- public synchronized long tryCheckpoint(long checkpointTargetLSN) throws HyracksDataException {
- LOGGER.info("Attemping soft checkpoint...");
- final long minFirstLSN = txnSubsystem.getRecoveryManager().getMinFirstLSN();
- boolean checkpointSucceeded = minFirstLSN >= checkpointTargetLSN;
- if (!checkpointSucceeded) {
- // Flush datasets with indexes behind target checkpoint LSN
- final IDatasetLifecycleManager datasetLifecycleManager =
- txnSubsystem.getAsterixAppRuntimeContextProvider().getDatasetLifecycleManager();
- datasetLifecycleManager.scheduleAsyncFlushForLaggingDatasets(checkpointTargetLSN);
- // Request remote replicas to flush lagging indexes
- final IReplicationManager replicationManager =
- txnSubsystem.getAsterixAppRuntimeContextProvider().getAppContext().getReplicationManager();
- try {
- replicationManager.requestFlushLaggingReplicaIndexes(checkpointTargetLSN);
- } catch (IOException e) {
- throw new HyracksDataException(e);
- }
- }
- capture(minFirstLSN, false);
- if (checkpointSucceeded) {
- txnSubsystem.getLogManager().deleteOldLogFiles(minFirstLSN);
- LOGGER.info(String.format("soft checkpoint succeeded with at LSN(%s)", minFirstLSN));
- }
- return minFirstLSN;
- }
-
- private long getDeadReplicasMinFirstLSN(Set<String> deadReplicaIds) throws HyracksDataException {
- final IReplicaResourcesManager remoteResourcesManager =
- txnSubsystem.getAsterixAppRuntimeContextProvider().getAppContext().getReplicaResourcesManager();
- final IApplicationContext propertiesProvider =
- txnSubsystem.getAsterixAppRuntimeContextProvider().getAppContext();
- final MetadataProperties metadataProperties = propertiesProvider.getMetadataProperties();
- final PersistentLocalResourceRepository localResourceRepository =
- (PersistentLocalResourceRepository) txnSubsystem.getAsterixAppRuntimeContextProvider()
- .getLocalResourceRepository();
- // Get partitions of the dead replicas that are not active on this node
- final Set<Integer> deadReplicasPartitions = new HashSet<>();
- for (String deadReplicaId : deadReplicaIds) {
- final ClusterPartition[] nodePartitons = metadataProperties.getNodePartitions().get(deadReplicaId);
- for (ClusterPartition partition : nodePartitons) {
- if (!localResourceRepository.getActivePartitions().contains(partition.getPartitionId())) {
- deadReplicasPartitions.add(partition.getPartitionId());
- }
- }
- }
- return remoteResourcesManager.getPartitionsMinLSN(deadReplicasPartitions);
- }
-}
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/replication/IReplicationJob.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/replication/IReplicationJob.java
index cbe6d1a..33c9c96 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/replication/IReplicationJob.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/replication/IReplicationJob.java
@@ -45,4 +45,8 @@
public Set<String> getJobFiles();
+ default String getAnyFile() {
+ return getJobFiles().stream().findAny()
+ .orElseThrow(() -> new IllegalStateException("Replication job without any files"));
+ }
}