BUG-5626: remove CompositeModification(ByteString)Payload
[controller.git] / opendaylight / md-sal / sal-distributed-datastore / src / main / java / org / opendaylight / controller / cluster / datastore / ShardRecoveryCoordinator.java
index 50528575e77123ce433bf0b4e6ab73f0077c39c5..dae3383a2e10eb78a83798b2a5dd52de7000f891 100644 (file)
@@ -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<DOMStoreWriteTransaction> 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<Object> 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<DOMStoreWriteTransaction> 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<Object> logEntries;
-
-        LogRecoveryTask(List<Object> 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;
     }
 }