flow descriptors should be always stored when addFlow is called
[openflowplugin.git] / openflowplugin-impl / src / main / java / org / opendaylight / openflowplugin / impl / services / SalFlowServiceImpl.java
index dbaf5e9786fe2915a19929324285f08cafda08c2..3912e027980f73e6f9fdedcb1341cb3e7b4cd1dc 100644 (file)
@@ -1,30 +1,38 @@
 /**
  * Copyright (c) 2015 Cisco Systems, Inc. and others.  All rights reserved.
- *
+ * <p/>
  * This program and the accompanying materials are made available under the
  * terms of the Eclipse Public License v1.0 which accompanies this distribution,
  * and is available at http://www.eclipse.org/legal/epl-v10.html
  */
 package org.opendaylight.openflowplugin.impl.services;
 
-import com.google.common.util.concurrent.AsyncFunction;
+import com.google.common.base.Function;
+import com.google.common.util.concurrent.FutureCallback;
 import com.google.common.util.concurrent.Futures;
-import com.google.common.util.concurrent.JdkFutureAdapters;
 import com.google.common.util.concurrent.ListenableFuture;
 import com.google.common.util.concurrent.SettableFuture;
-import java.math.BigInteger;
 import java.util.ArrayList;
-import java.util.Collections;
+import java.util.Collection;
 import java.util.List;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.Future;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
+import org.opendaylight.openflowjava.protocol.api.connection.OutboundQueue;
+import org.opendaylight.openflowplugin.api.openflow.device.DeviceContext;
 import org.opendaylight.openflowplugin.api.openflow.device.RequestContext;
-import org.opendaylight.openflowplugin.api.openflow.rpc.RpcContext;
-import org.opendaylight.openflowplugin.openflow.md.core.sal.OFRpcTaskFactory;
+import org.opendaylight.openflowplugin.api.openflow.device.RequestContextStack;
+import org.opendaylight.openflowplugin.api.openflow.registry.flow.DeviceFlowRegistry;
+import org.opendaylight.openflowplugin.api.openflow.registry.flow.FlowDescriptor;
+import org.opendaylight.openflowplugin.api.openflow.registry.flow.FlowHash;
+import org.opendaylight.openflowplugin.api.openflow.statistics.ofpspecific.MessageSpy;
+import org.opendaylight.openflowplugin.impl.registry.flow.FlowDescriptorFactory;
+import org.opendaylight.openflowplugin.impl.registry.flow.FlowHashFactory;
+import org.opendaylight.openflowplugin.impl.util.FlowUtil;
 import org.opendaylight.openflowplugin.openflow.md.core.sal.convertor.FlowConvertor;
 import org.opendaylight.openflowplugin.openflow.md.util.FlowCreatorUtil;
+import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.inventory.rev130819.FlowId;
+import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.inventory.rev130819.tables.table.Flow;
+import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.inventory.rev130819.tables.table.FlowKey;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.service.rev130819.AddFlowInput;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.service.rev130819.AddFlowInputBuilder;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.service.rev130819.AddFlowOutput;
@@ -36,10 +44,11 @@ import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.service.rev130819.Upda
 import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.service.rev130819.UpdateFlowOutput;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.service.rev130819.flow.update.OriginalFlow;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.flow.service.rev130819.flow.update.UpdatedFlow;
-import org.opendaylight.yang.gen.v1.urn.opendaylight.module.config.rev141015.SetConfigOutput;
+import org.opendaylight.yang.gen.v1.urn.opendaylight.openflow.protocol.rev130731.FlowModInput;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.openflow.protocol.rev130731.FlowModInputBuilder;
-import org.opendaylight.yangtools.yang.binding.DataObject;
+import org.opendaylight.yang.gen.v1.urn.opendaylight.openflow.protocol.rev130731.OfHeader;
 import org.opendaylight.yangtools.yang.common.RpcError;
+import org.opendaylight.yangtools.yang.common.RpcError.ErrorType;
 import org.opendaylight.yangtools.yang.common.RpcResult;
 import org.opendaylight.yangtools.yang.common.RpcResultBuilder;
 import org.slf4j.Logger;
@@ -48,37 +57,88 @@ public class SalFlowServiceImpl extends CommonService implements SalFlowService
 
     private static final Logger LOG = org.slf4j.LoggerFactory.getLogger(SalFlowServiceImpl.class);
 
-    // TODO set cookie somehow from - DeviceContext probably (temporary set to 0 - primary connection)
-    private final BigInteger connectionID = PRIMARY_CONNECTION;
-
-    private interface Function {
-        Future<RpcResult<Void>> apply(final BigInteger IDConnection);
-    }
-
-    public SalFlowServiceImpl(final RpcContext rpcContext) {
-        super(rpcContext);
+    public SalFlowServiceImpl(final RequestContextStack requestContextStack, final DeviceContext deviceContext) {
+        super(requestContextStack, deviceContext);
     }
 
     @Override
     public Future<RpcResult<AddFlowOutput>> addFlow(final AddFlowInput input) {
-        return processFlow(new Function() {
+        getMessageSpy().spyMessage(input.getImplementedInterface(), MessageSpy.STATISTIC_GROUP.TO_SWITCH_ENTERED);
+
+        final List<FlowModInputBuilder> ofFlowModInputs = FlowConvertor.toFlowModInputs(input, getVersion(), getDatapathId());
+        final ListenableFuture<RpcResult<AddFlowOutput>> future = processFlowModInputBuilders(ofFlowModInputs);
+        final FlowId flowId;
+        if (null != input.getFlowRef()) {
+            flowId = input.getFlowRef().getValue().firstKeyOf(Flow.class, FlowKey.class).getId();
+        } else {
+            flowId = FlowUtil.createAlienFlowId(input.getTableId());
+        }
+
+        final DeviceContext deviceContext = getDeviceContext();
+        final FlowHash flowHash = FlowHashFactory.create(input, deviceContext.getPrimaryConnectionContext().getFeatures().getVersion());
+        final FlowDescriptor flowDescriptor = FlowDescriptorFactory.create(input.getTableId(), flowId);
+        deviceContext.getDeviceFlowRegistry().store(flowHash, flowDescriptor);
+        Futures.addCallback(future, new FutureCallback<RpcResult<AddFlowOutput>>() {
+
+
             @Override
-            public ListenableFuture<RpcResult<Void>> apply(final BigInteger IDConnection) {
-                final List<FlowModInputBuilder> ofFlowModInputs = FlowConvertor.toFlowModInputs(input, version,
-                        datapathId);
-                return chainFlowMods(ofFlowModInputs, 0, IDConnection);
+            public void onSuccess(final RpcResult<AddFlowOutput> rpcResult) {
+                if (rpcResult.isSuccessful()) {
+                    LOG.debug("flow add finished without error, id={}", flowId.getValue());
+                } else {
+                    LOG.debug("flow add failed with error, id={}", flowId.getValue());
+                }
+            }
+
+            @Override
+            public void onFailure(final Throwable throwable) {
+                deviceContext.getDeviceFlowRegistry().markToBeremoved(flowHash);
+                LOG.trace("Service call for adding flows failed, id={}.", flowId.getValue(), throwable);
             }
         });
+
+        return future;
     }
 
     @Override
     public Future<RpcResult<RemoveFlowOutput>> removeFlow(final RemoveFlowInput input) {
-        return processFlow(new Function() {
+        LOG.trace("Calling remove flow for flow with ID ={}.", input.getFlowRef());
+        return this.<RemoveFlowOutput, Void>handleServiceCall(new Function<RequestContext<RemoveFlowOutput>, ListenableFuture<RpcResult<Void>>>() {
             @Override
-            public Future<RpcResult<Void>> apply(final BigInteger IDConnection) {
-                final List<FlowModInputBuilder> ofFlowModInputs = FlowConvertor.toFlowModInputs(input, version,
-                        datapathId);
-                return provideConnectionAdapter(IDConnection).flowMod(ofFlowModInputs.get(0).build());
+            public ListenableFuture<RpcResult<Void>> apply(final RequestContext<RemoveFlowOutput> requestContext) {
+                final FlowModInputBuilder ofFlowModInput = FlowConvertor.toFlowModInput(input, getVersion(),
+                        getDatapathId());
+                final ListenableFuture<RpcResult<Void>> future = createResultForFlowMod(requestContext, ofFlowModInput);
+                Futures.addCallback(future, new FutureCallback<RpcResult<Void>>() {
+                    @Override
+                    public void onSuccess(final RpcResult<Void> o) {
+                        final DeviceContext deviceContext = getDeviceContext();
+                        getMessageSpy().spyMessage(input.getImplementedInterface(), MessageSpy.STATISTIC_GROUP.TO_SWITCH_SUBMIT_SUCCESS);
+                        FlowHash flowHash = FlowHashFactory.create(input, deviceContext.getPrimaryConnectionContext().getFeatures().getVersion());
+                        deviceContext.getDeviceFlowRegistry().markToBeremoved(flowHash);
+                    }
+
+                    @Override
+                    public void onFailure(final Throwable throwable) {
+                        getMessageSpy().spyMessage(input.getImplementedInterface(), MessageSpy.STATISTIC_GROUP.TO_SWITCH_SUBMIT_FAILURE);
+                        LOG.trace("Flow modification failed..", throwable);
+                        StringBuilder errors = new StringBuilder();
+                        try {
+                            RpcResult<Void> result = future.get();
+                            Collection<RpcError> rpcErrors = result.getErrors();
+                            if (null != rpcErrors && rpcErrors.size() > 0) {
+                                for (RpcError rpcError : rpcErrors) {
+                                    errors.append(rpcError.getMessage());
+                                }
+                            }
+                        } catch (InterruptedException | ExecutionException e) {
+                            LOG.trace("Flow modification failed. Can't read errors from RpcResult.");
+                        } finally {
+                            LOG.trace("Flow modification failed. Errors : {}", errors.toString());
+                        }
+                    }
+                });
+                return future;
             }
         });
     }
@@ -92,88 +152,105 @@ public class SalFlowServiceImpl extends CommonService implements SalFlowService
         final List<FlowModInputBuilder> allFlowMods = new ArrayList<>();
         List<FlowModInputBuilder> ofFlowModInputs;
 
-        if (!FlowCreatorUtil.canModifyFlow(original, updated, version)) {
+        if (!FlowCreatorUtil.canModifyFlow(original, updated, getVersion())) {
             // We would need to remove original and add updated.
 
             // remove flow
             final RemoveFlowInputBuilder removeflow = new RemoveFlowInputBuilder(original);
             final List<FlowModInputBuilder> ofFlowRemoveInput = FlowConvertor.toFlowModInputs(removeflow.build(),
-                    version, datapathId);
+                    getVersion(), getDatapathId());
             // remove flow should be the first
             allFlowMods.addAll(ofFlowRemoveInput);
             final AddFlowInputBuilder addFlowInputBuilder = new AddFlowInputBuilder(updated);
-            ofFlowModInputs = FlowConvertor.toFlowModInputs(addFlowInputBuilder.build(), version, datapathId);
+            ofFlowModInputs = FlowConvertor.toFlowModInputs(addFlowInputBuilder.build(), getVersion(), getDatapathId());
         } else {
-            ofFlowModInputs = FlowConvertor.toFlowModInputs(updated, version, datapathId);
+            ofFlowModInputs = FlowConvertor.toFlowModInputs(updated, getVersion(), getDatapathId());
         }
 
         allFlowMods.addAll(ofFlowModInputs);
-        LOG.debug("Number of flows to push to switch: {}", allFlowMods.size());
-        Collections.<String> emptyList();
-        return this.<UpdateFlowOutput> processFlow(new Function() {
+        ListenableFuture<RpcResult<UpdateFlowOutput>> future = processFlowModInputBuilders(allFlowMods);
+        Futures.addCallback(future, new FutureCallback<RpcResult<UpdateFlowOutput>>() {
+            @Override
+            public void onSuccess(final RpcResult<UpdateFlowOutput> o) {
+                final DeviceContext deviceContext = getDeviceContext();
+                getMessageSpy().spyMessage(input.getImplementedInterface(), MessageSpy.STATISTIC_GROUP.TO_SWITCH_SUBMIT_SUCCESS);
+                final short version = deviceContext.getPrimaryConnectionContext().getFeatures().getVersion();
+                FlowHash flowHash = FlowHashFactory.create(original, version);
+
+                FlowHash updatedflowHash = FlowHashFactory.create(updated, version);
+                FlowId flowId = input.getFlowRef().getValue().firstKeyOf(Flow.class, FlowKey.class).getId();
+                FlowDescriptor flowDescriptor = FlowDescriptorFactory.create(updated.getTableId(), flowId);
+                final DeviceFlowRegistry deviceFlowRegistry = deviceContext.getDeviceFlowRegistry();
+                deviceFlowRegistry.markToBeremoved(flowHash);
+                deviceFlowRegistry.store(updatedflowHash, flowDescriptor);
+            }
+
             @Override
-            public Future<RpcResult<Void>> apply(final BigInteger cookie) {
-                return chainFlowMods(allFlowMods, 0, cookie);
+            public void onFailure(final Throwable throwable) {
+                getMessageSpy().spyMessage(input.getImplementedInterface(), MessageSpy.STATISTIC_GROUP.TO_SWITCH_SUBMIT_FAILURE);
             }
         });
+        return future;
     }
 
-    private <T extends DataObject> Future<RpcResult<T>> processFlow(final Function function) {
-        LOG.debug("Calling the FlowMod RPC method on MessageDispatchService");
-        // use primary connection
+    private <T> ListenableFuture<RpcResult<T>> processFlowModInputBuilders(final List<FlowModInputBuilder> ofFlowModInputs) {
 
-        final RequestContext requestContext = rpcContext.createRequestContext();
-        final SettableFuture<RpcResult<T>> result = rpcContext.storeOrFail(requestContext);
+        final List<ListenableFuture<RpcResult<T>>> partialFutures = new ArrayList<>();
 
-        if (!result.isDone()) {
-                final Future<RpcResult<Void>> resultFromOFLib = function.apply(connectionID);
+        for (final FlowModInputBuilder flowModInputBuilder : ofFlowModInputs) {
+            ListenableFuture<RpcResult<T>> partialFuture = handleServiceCall(
+                    new Function<RequestContext<T>, ListenableFuture<RpcResult<Void>>>() {
+                        @Override
+                        public ListenableFuture<RpcResult<Void>> apply(final RequestContext<T> requestContext) {
+                            return createResultForFlowMod(requestContext, flowModInputBuilder);
+                        }
+                    });
+            partialFutures.add(partialFuture);
+        }
 
-                RpcResultConvertor<T> rpcResultConvertor = new RpcResultConvertor<>(requestContext);
-                rpcResultConvertor.processResultFromOfJava(resultFromOFLib, getWaitTime());
+        final ListenableFuture<List<RpcResult<T>>> allFutures = Futures.successfulAsList(partialFutures);
+        final SettableFuture<RpcResult<T>> finalFuture = SettableFuture.create();
+        Futures.addCallback(allFutures, new FutureCallback<List<RpcResult<T>>>() {
+            @Override
+            public void onSuccess(final List<RpcResult<T>> results) {
+                RpcResultBuilder<T> rpcResultBuilder = RpcResultBuilder.success();
+                finalFuture.set(rpcResultBuilder.build());
+            }
 
-        } else {
-            requestContext.close();
-        }
-        return result;
-    }
+            @Override
+            public void onFailure(final Throwable t) {
+                RpcResultBuilder<T> rpcResultBuilder = RpcResultBuilder.failed();
+                finalFuture.set(rpcResultBuilder.build());
+            }
+        });
 
-    /**
-     * Recursive helper method for
-     * {@link OFRpcTaskFactory#chainFlowMods(java.util.List, int, org.opendaylight.openflowplugin.openflow.md.core.sal.OFRpcTaskContext, org.opendaylight.openflowplugin.api.openflow.md.core.SwitchConnectionDistinguisher)}
-     * {@link OFRpcTaskFactory#createUpdateFlowTask()} to chain results of multiple flowmods. The next flowmod gets
-     * executed if the earlier one is successful. All the flowmods should have the same xid, in-order to cross-reference
-     * the notification
-     */
-    protected ListenableFuture<RpcResult<Void>> chainFlowMods(final List<FlowModInputBuilder> ofFlowModInputs,
-            final int index, final BigInteger cookie) {
-
-        final Future<RpcResult<Void>> resultFromOFLib = createResultForFlowMod(ofFlowModInputs.get(index), cookie);
-
-        final ListenableFuture<RpcResult<Void>> result = JdkFutureAdapters.listenInPoolThread(resultFromOFLib);
-
-        if (ofFlowModInputs.size() > index + 1) {
-            // there are more flowmods to chain
-            return Futures.transform(result, new AsyncFunction<RpcResult<Void>, RpcResult<Void>>() {
-                @Override
-                public ListenableFuture<RpcResult<Void>> apply(final RpcResult<Void> input) throws Exception {
-                    if (input.isSuccessful()) {
-                        return chainFlowMods(ofFlowModInputs, index + 1, cookie);
-                    } else {
-                        LOG.warn("Flowmod failed. Any chained flowmods are ignored. xid:{}", ofFlowModInputs.get(index)
-                                .getXid());
-                        return Futures.immediateFuture(input);
-                    }
-                }
-            });
-        } else {
-            return result;
-        }
+        return finalFuture;
     }
 
-    protected Future<RpcResult<Void>> createResultForFlowMod(final FlowModInputBuilder flowModInput,
-            final BigInteger cookie) {
-        flowModInput.setXid(deviceContext.getNextXid().getValue());
-        return provideConnectionAdapter(cookie).flowMod(flowModInput.build());
+    protected <T> ListenableFuture<RpcResult<Void>> createResultForFlowMod(final RequestContext<T> requestContext, final FlowModInputBuilder flowModInputBuilder) {
+        final OutboundQueue outboundQueue = getDeviceContext().getPrimaryConnectionContext().getOutboundQueueProvider();
+        final long xid = requestContext.getXid().getValue();
+        flowModInputBuilder.setXid(xid);
+        final FlowModInput flowModInput = flowModInputBuilder.build();
+
+        final SettableFuture<RpcResult<Void>> settableFuture = SettableFuture.create();
+        outboundQueue.commitEntry(xid, flowModInput, new FutureCallback<OfHeader>() {
+            @Override
+            public void onSuccess(final OfHeader ofHeader) {
+                RequestContextUtil.closeRequstContext(requestContext);
+                getMessageSpy().spyMessage(FlowModInput.class, MessageSpy.STATISTIC_GROUP.TO_SWITCH_SUBMIT_SUCCESS);
+
+                settableFuture.set(SUCCESSFUL_RPCRESULT);
+            }
+
+            @Override
+            public void onFailure(final Throwable throwable) {
+                RpcResultBuilder<Void> rpcResultBuilder = RpcResultBuilder.<Void>failed().withError(ErrorType.APPLICATION, throwable.getMessage(), throwable);
+                RequestContextUtil.closeRequstContext(requestContext);
+                settableFuture.set(rpcResultBuilder.build());
+            }
+        });
+        return settableFuture;
     }
 
 }