X-Git-Url: https://git.opendaylight.org/gerrit/gitweb?p=controller.git;a=blobdiff_plain;f=opendaylight%2Fmd-sal%2Fsal-distributed-datastore%2Fsrc%2Fmain%2Fjava%2Forg%2Fopendaylight%2Fcontroller%2Fcluster%2Fdatastore%2FShardRecoveryCoordinator.java;h=dae3383a2e10eb78a83798b2a5dd52de7000f891;hp=50528575e77123ce433bf0b4e6ab73f0077c39c5;hb=8e6e0aeb5ca474cb292aab6ca581e921b0a56489;hpb=c31509c7a6630e54a9f9749a643fed5e1a1ad380 diff --git a/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/ShardRecoveryCoordinator.java b/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/ShardRecoveryCoordinator.java index 50528575e7..dae3383a2e 100644 --- a/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/ShardRecoveryCoordinator.java +++ b/opendaylight/md-sal/sal-distributed-datastore/src/main/java/org/opendaylight/controller/cluster/datastore/ShardRecoveryCoordinator.java @@ -7,19 +7,16 @@ */ package org.opendaylight.controller.cluster.datastore; -import com.google.common.collect.Lists; -import com.google.common.util.concurrent.ThreadFactoryBuilder; -import java.util.Collection; -import java.util.Collections; -import java.util.List; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.TimeUnit; -import org.opendaylight.controller.cluster.datastore.modification.MutableCompositeModification; +import com.google.common.base.Preconditions; +import java.io.IOException; +import org.opendaylight.controller.cluster.datastore.utils.PruningDataTreeModification; import org.opendaylight.controller.cluster.datastore.utils.SerializationUtils; -import org.opendaylight.controller.sal.core.spi.data.DOMStoreWriteTransaction; +import org.opendaylight.controller.cluster.raft.RaftActorRecoveryCohort; +import org.opendaylight.controller.cluster.raft.protobuff.client.messages.Payload; import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier; import org.opendaylight.yangtools.yang.data.api.schema.NormalizedNode; +import org.opendaylight.yangtools.yang.data.api.schema.tree.DataTreeCandidates; +import org.opendaylight.yangtools.yang.data.api.schema.tree.DataValidationFailedException; import org.opendaylight.yangtools.yang.model.api.SchemaContext; import org.slf4j.Logger; @@ -30,119 +27,91 @@ import org.slf4j.Logger; * committed to the data store in the order the corresponding snapshot or log batch are received * to preserve data store integrity. * - * @author Thomas Panetelis + * @author Thomas Pantelis */ -class ShardRecoveryCoordinator { - - private static final int TIME_OUT = 10; - - private final List resultingTxList = Lists.newArrayList(); - private final SchemaContext schemaContext; +class ShardRecoveryCoordinator implements RaftActorRecoveryCohort { + private final ShardDataTree store; private final String shardName; - private final ExecutorService executor; private final Logger log; - private final String name; - - ShardRecoveryCoordinator(String shardName, SchemaContext schemaContext, Logger log, - String name) { - this.schemaContext = schemaContext; + private final SchemaContext schemaContext; + private PruningDataTreeModification transaction; + private int size; + private final byte[] restoreFromSnapshot; + + ShardRecoveryCoordinator(ShardDataTree store, SchemaContext schemaContext, byte[] restoreFromSnapshot, + String shardName, Logger log) { + this.store = Preconditions.checkNotNull(store); + this.restoreFromSnapshot = restoreFromSnapshot; this.shardName = shardName; this.log = log; - this.name = name; - - executor = Executors.newFixedThreadPool(Runtime.getRuntime().availableProcessors(), - new ThreadFactoryBuilder().setDaemon(true) - .setNameFormat("ShardRecovery-" + shardName + "-%d").build()); - } - - /** - * Submits a batch of journal log entries. - * - * @param logEntries the serialized journal log entries - * @param resultingTx the write Tx to which to apply the entries - */ - void submit(List logEntries, DOMStoreWriteTransaction resultingTx) { - LogRecoveryTask task = new LogRecoveryTask(logEntries, resultingTx); - resultingTxList.add(resultingTx); - executor.execute(task); + this.schemaContext = schemaContext; } - /** - * Submits a snapshot. - * - * @param snapshotBytes the serialized snapshot - * @param resultingTx the write Tx to which to apply the entries - */ - void submit(byte[] snapshotBytes, DOMStoreWriteTransaction resultingTx) { - SnapshotRecoveryTask task = new SnapshotRecoveryTask(snapshotBytes, resultingTx); - resultingTxList.add(resultingTx); - executor.execute(task); + @Override + public void startLogRecoveryBatch(int maxBatchSize) { + log.debug("{}: starting log recovery batch with max size {}", shardName, maxBatchSize); + transaction = new PruningDataTreeModification(store.newModification(), store.getDataTree(), schemaContext); + size = 0; } - Collection getTransactions() { - // Shutdown the executor and wait for task completion. - executor.shutdown(); + @Override + public void appendRecoveredLogEntry(Payload payload) { + Preconditions.checkState(transaction != null, "call startLogRecovery before calling appendRecoveredLogEntry"); try { - if(executor.awaitTermination(TIME_OUT, TimeUnit.MINUTES)) { - return resultingTxList; + if (payload instanceof DataTreeCandidatePayload) { + DataTreeCandidates.applyToModification(transaction, ((DataTreeCandidatePayload)payload).getCandidate()); + size++; } else { - log.error("{}: Recovery for shard {} timed out after {} minutes", name, shardName, TIME_OUT); + log.error("{}: Unknown payload {} received during recovery", shardName, payload); } - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); + } catch (IOException e) { + log.error("{}: Error extracting ModificationPayload", shardName, e); } - - return Collections.emptyList(); } - private static abstract class ShardRecoveryTask implements Runnable { - - final DOMStoreWriteTransaction resultingTx; - - ShardRecoveryTask(DOMStoreWriteTransaction resultingTx) { - this.resultingTx = resultingTx; - } + private void commitTransaction(PruningDataTreeModification tx) throws DataValidationFailedException { + store.commit(tx.getResultingModification()); } - private class LogRecoveryTask extends ShardRecoveryTask { - - private final List logEntries; - - LogRecoveryTask(List logEntries, DOMStoreWriteTransaction resultingTx) { - super(resultingTx); - this.logEntries = logEntries; - } + /** + * Applies the current batched log entries to the data store. + */ + @Override + public void applyCurrentLogRecoveryBatch() { + Preconditions.checkState(transaction != null, "call startLogRecovery before calling applyCurrentLogRecoveryBatch"); - @Override - public void run() { - for(int i = 0; i < logEntries.size(); i++) { - MutableCompositeModification.fromSerializable( - logEntries.get(i)).apply(resultingTx); - // Null out to GC quicker. - logEntries.set(i, null); - } + log.debug("{}: Applying current log recovery batch with size {}", shardName, size); + try { + commitTransaction(transaction); + } catch (DataValidationFailedException e) { + log.error("{}: Failed to apply recovery batch", shardName, e); } + transaction = null; } - private class SnapshotRecoveryTask extends ShardRecoveryTask { - - private final byte[] snapshotBytes; - - SnapshotRecoveryTask(byte[] snapshotBytes, DOMStoreWriteTransaction resultingTx) { - super(resultingTx); - this.snapshotBytes = snapshotBytes; + /** + * Applies a recovered snapshot to the data store. + * + * @param snapshotBytes the serialized snapshot + */ + @Override + public void applyRecoverySnapshot(final byte[] snapshotBytes) { + log.debug("{}: Applying recovered snapshot", shardName); + + final NormalizedNode node = SerializationUtils.deserializeNormalizedNode(snapshotBytes); + final PruningDataTreeModification tx = new PruningDataTreeModification(store.newModification(), + store.getDataTree(), schemaContext); + tx.write(YangInstanceIdentifier.EMPTY, node); + try { + commitTransaction(tx); + } catch (DataValidationFailedException e) { + log.error("{}: Failed to apply recovery snapshot", shardName, e); } + } - @Override - public void run() { - NormalizedNode node = SerializationUtils.deserializeNormalizedNode(snapshotBytes); - - // delete everything first - resultingTx.delete(YangInstanceIdentifier.builder().build()); - - // Add everything from the remote node back - resultingTx.write(YangInstanceIdentifier.builder().build(), node); - } + @Override + public byte[] getRestoreFromSnapshot() { + return restoreFromSnapshot; } }