Bug 2103: Make sure InventoryManager survives Transaction failure.
[controller.git] / opendaylight / md-sal / inventory-manager / src / main / java / org / opendaylight / controller / md / inventory / manager / FlowCapableInventoryProvider.java
index 7e4190f1df4b7246da4a175d2efabb103282383a..d63f2bef86d551280b9815e896ae9c07da96eace 100644 (file)
  */
 package org.opendaylight.controller.md.inventory.manager;
 
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingDeque;
+
+import org.opendaylight.controller.md.sal.binding.api.BindingTransactionChain;
+import org.opendaylight.controller.md.sal.binding.api.DataBroker;
+import org.opendaylight.controller.md.sal.binding.api.ReadWriteTransaction;
+import org.opendaylight.controller.md.sal.common.api.data.AsyncTransaction;
+import org.opendaylight.controller.md.sal.common.api.data.TransactionChain;
+import org.opendaylight.controller.md.sal.common.api.data.TransactionChainListener;
+import org.opendaylight.controller.md.sal.common.api.data.TransactionCommitFailedException;
 import org.opendaylight.controller.sal.binding.api.NotificationProviderService;
-import org.opendaylight.controller.sal.binding.api.data.DataModificationTransaction;
-import org.opendaylight.controller.sal.binding.api.data.DataProviderService;
-import org.opendaylight.yangtools.concepts.Registration;
-import org.opendaylight.yangtools.yang.binding.NotificationListener;
+import org.opendaylight.yangtools.concepts.ListenerRegistration;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-public class FlowCapableInventoryProvider implements AutoCloseable {
+import com.google.common.base.Preconditions;
+import com.google.common.util.concurrent.CheckedFuture;
+import com.google.common.util.concurrent.FutureCallback;
+import com.google.common.util.concurrent.Futures;
+
+class FlowCapableInventoryProvider implements AutoCloseable, Runnable, TransactionChainListener {
+    private static final Logger LOG = LoggerFactory.getLogger(FlowCapableInventoryProvider.class);
+    private static final int QUEUE_DEPTH = 500;
+    private static final int MAX_BATCH = 100;
+
+    private final BlockingQueue<InventoryOperation> queue = new LinkedBlockingDeque<>(QUEUE_DEPTH);
+    private final NotificationProviderService notificationService;
+
+    private final DataBroker dataBroker;
+    private BindingTransactionChain txChain;
+    private ListenerRegistration<?> listenerRegistration;
+    private Thread thread;
 
-    private final static Logger LOG = LoggerFactory.getLogger(FlowCapableInventoryProvider.class);
+    FlowCapableInventoryProvider(final DataBroker dataBroker, final NotificationProviderService notificationService) {
+        this.dataBroker = Preconditions.checkNotNull(dataBroker);
+        this.notificationService = Preconditions.checkNotNull(notificationService);
+    }
+
+    void start() {
+        final NodeChangeCommiter changeCommiter = new NodeChangeCommiter(FlowCapableInventoryProvider.this);
+        this.listenerRegistration = this.notificationService.registerNotificationListener(changeCommiter);
 
-    private DataProviderService dataService;
-    private NotificationProviderService notificationService;
-    private Registration<NotificationListener> listenerRegistration;
-    private final NodeChangeCommiter changeCommiter = new NodeChangeCommiter(FlowCapableInventoryProvider.this);
+        this.txChain = dataBroker.createTransactionChain(this);
+        thread = new Thread(this);
+        thread.setDaemon(true);
+        thread.setName("FlowCapableInventoryProvider");
+        thread.start();
 
-    public void start() {
-        this.listenerRegistration = this.notificationService.registerNotificationListener(this.changeCommiter);
         LOG.info("Flow Capable Inventory Provider started.");
     }
 
-    protected DataModificationTransaction startChange() {
-        DataProviderService _dataService = this.dataService;
-        return _dataService.beginTransaction();
+    void enqueue(final InventoryOperation op) {
+        try {
+            queue.put(op);
+        } catch (final InterruptedException e) {
+            LOG.warn("Failed to enqueue operation {}", op, e);
+        }
     }
 
     @Override
-    public void close() {
-        try {
-            LOG.info("Flow Capable Inventory Provider stopped.");
-            if (this.listenerRegistration != null) {
+    public void close() throws InterruptedException {
+        LOG.info("Flow Capable Inventory Provider stopped.");
+        if (this.listenerRegistration != null) {
+            try {
                 this.listenerRegistration.close();
+            } catch (final Exception e) {
+                LOG.error("Failed to stop inventory provider", e);
             }
-        } catch (Exception e) {
-            String errMsg = "Error by stop Flow Capable Inventory Provider.";
-            LOG.error(errMsg, e);
-            throw new RuntimeException(errMsg, e);
+            listenerRegistration = null;
         }
-    }
 
-    public DataProviderService getDataService() {
-        return this.dataService;
+        if (thread != null) {
+            thread.interrupt();
+            thread.join();
+            thread = null;
+        }
+        if (txChain != null) {
+            txChain.close();
+            txChain = null;
+        }
+
+
     }
 
-    public void setDataService(final DataProviderService dataService) {
-        this.dataService = dataService;
+    @Override
+    public void run() {
+        try {
+            for (; ; ) {
+                InventoryOperation op = queue.take();
+
+                ReadWriteTransaction tx;
+                try {
+                    tx = txChain.newReadWriteTransaction();
+                } catch (final IllegalStateException e) {
+                    txChain.close();
+                    txChain = dataBroker.createTransactionChain(this);
+                    tx = txChain.newReadWriteTransaction();
+                }
+                LOG.debug("New operations available, starting transaction {}", tx.getIdentifier());
+
+                int ops = 0;
+                do {
+                    op.applyOperation(tx);
+
+                    ops++;
+                    if (ops < MAX_BATCH) {
+                        op = queue.poll();
+                    } else {
+                        op = null;
+                    }
+                } while (op != null);
+
+                LOG.debug("Processed {} operations, submitting transaction {}", ops, tx.getIdentifier());
+
+                final CheckedFuture<Void, TransactionCommitFailedException> result = tx.submit();
+                final Object ident = tx.getIdentifier();
+                Futures.addCallback(result, new FutureCallback<Void>() {
+                    @Override
+                    public void onSuccess(final Void aVoid) {
+                        //NOOP
+                    }
+
+                    @Override
+                    public void onFailure(final Throwable throwable) {
+                        LOG.error("Transaction {} failed.", ident, throwable);
+                    }
+                });
+            }
+        } catch (final InterruptedException e) {
+            LOG.info("Processing interrupted, terminating", e);
+        }
+
+        // Drain all events, making sure any blocked threads are unblocked
+        while (!queue.isEmpty()) {
+            queue.poll();
+        }
     }
 
-    public NotificationProviderService getNotificationService() {
-        return this.notificationService;
+    @Override
+    public void onTransactionChainFailed(final TransactionChain<?, ?> chain, final AsyncTransaction<?, ?> transaction,
+                                         final Throwable cause) {
+        LOG.error("Failed to export Flow Capable Inventory, Transaction {} failed.", transaction.getIdentifier(), cause);
+
     }
 
-    public void setNotificationService(
-            final NotificationProviderService notificationService) {
-        this.notificationService = notificationService;
+    @Override
+    public void onTransactionChainSuccessful(final TransactionChain<?, ?> chain) {
+        // NOOP
     }
 }