bug 8029 added ref counts for physical locators. 35/61535/10
authorK.V Suneelu Verma <k.v.suneelu.verma@ericsson.com>
Fri, 11 Aug 2017 09:20:36 +0000 (14:50 +0530)
committerAnil Vishnoi <vishnoianil@gmail.com>
Tue, 3 Oct 2017 09:08:05 +0000 (09:08 +0000)
remote mcast and ucast macs refer to physical locators

whenever a new mac is added or updated, update their refcount for that
tep.

When the ref count of the tep goes to zero , the device may delete that
tep after the current transaction.
In that case mark the locator as in transit.

The next transaction which depends on this locator will not use this
locator uuid and will wait for the locator in transit state to be
resolved in dependency queue.

Eventually controller will get locator delete update command from the
device and proceed the transactions which are waiting for this locator
state to be changed from in transit state.

Added onSuccess and onFailure methods to transaction command.
Clear the locator refs for the succeeded delete mcast macs immediately
upon the success of the device transaction.

Clearing the locator ref count immediately upon success of delete mcast
is needed so that the next transaction will not use the possibly stale locator.

Clear the intransit status of the failed commands upon failure.

Change-Id: I84d64f7deb20f801f14e524f88431807afac3a99
Signed-off-by: K.V Suneelu Verma <k.v.suneelu.verma@ericsson.com>
18 files changed:
hwvtepsouthbound/hwvtepsouthbound-impl/src/main/java/org/opendaylight/ovsdb/hwvtepsouthbound/HwvtepDeviceInfo.java
hwvtepsouthbound/hwvtepsouthbound-impl/src/main/java/org/opendaylight/ovsdb/hwvtepsouthbound/HwvtepSouthboundConstants.java
hwvtepsouthbound/hwvtepsouthbound-impl/src/main/java/org/opendaylight/ovsdb/hwvtepsouthbound/transact/AbstractTransactCommand.java
hwvtepsouthbound/hwvtepsouthbound-impl/src/main/java/org/opendaylight/ovsdb/hwvtepsouthbound/transact/HwvtepOperationalState.java
hwvtepsouthbound/hwvtepsouthbound-impl/src/main/java/org/opendaylight/ovsdb/hwvtepsouthbound/transact/LogicalRouterRemoveCommand.java
hwvtepsouthbound/hwvtepsouthbound-impl/src/main/java/org/opendaylight/ovsdb/hwvtepsouthbound/transact/LogicalSwitchRemoveCommand.java
hwvtepsouthbound/hwvtepsouthbound-impl/src/main/java/org/opendaylight/ovsdb/hwvtepsouthbound/transact/LogicalSwitchUpdateCommand.java
hwvtepsouthbound/hwvtepsouthbound-impl/src/main/java/org/opendaylight/ovsdb/hwvtepsouthbound/transact/McastMacsRemoteRemoveCommand.java
hwvtepsouthbound/hwvtepsouthbound-impl/src/main/java/org/opendaylight/ovsdb/hwvtepsouthbound/transact/McastMacsRemoteUpdateCommand.java
hwvtepsouthbound/hwvtepsouthbound-impl/src/main/java/org/opendaylight/ovsdb/hwvtepsouthbound/transact/MdsalUpdate.java [new file with mode: 0644]
hwvtepsouthbound/hwvtepsouthbound-impl/src/main/java/org/opendaylight/ovsdb/hwvtepsouthbound/transact/PhysicalSwitchUpdateCommand.java
hwvtepsouthbound/hwvtepsouthbound-impl/src/main/java/org/opendaylight/ovsdb/hwvtepsouthbound/transact/TransactCommand.java
hwvtepsouthbound/hwvtepsouthbound-impl/src/main/java/org/opendaylight/ovsdb/hwvtepsouthbound/transact/TransactCommandAggregator.java
hwvtepsouthbound/hwvtepsouthbound-impl/src/main/java/org/opendaylight/ovsdb/hwvtepsouthbound/transact/TransactInvokerImpl.java
hwvtepsouthbound/hwvtepsouthbound-impl/src/main/java/org/opendaylight/ovsdb/hwvtepsouthbound/transact/TransactUtils.java
hwvtepsouthbound/hwvtepsouthbound-impl/src/main/java/org/opendaylight/ovsdb/hwvtepsouthbound/transact/UcastMacsRemoteRemoveCommand.java
hwvtepsouthbound/hwvtepsouthbound-impl/src/main/java/org/opendaylight/ovsdb/hwvtepsouthbound/transact/UcastMacsRemoteUpdateCommand.java
hwvtepsouthbound/hwvtepsouthbound-impl/src/main/java/org/opendaylight/ovsdb/hwvtepsouthbound/transactions/md/AbstractTransactionCommand.java

index 4817cbd922b0685fb095138bd36b174ba6edb9d4..ec637a98abb77b83fb76330c3dd6140b2eba2cfd 100644 (file)
@@ -8,6 +8,7 @@
 
 package org.opendaylight.ovsdb.hwvtepsouthbound;
 
+import com.google.common.collect.Sets;
 import org.opendaylight.ovsdb.hwvtepsouthbound.transact.DependencyQueue;
 import org.opendaylight.ovsdb.hwvtepsouthbound.transact.DependentJob;
 import org.opendaylight.ovsdb.hwvtepsouthbound.transact.TransactCommand;
@@ -16,7 +17,11 @@ import org.opendaylight.ovsdb.schema.hardwarevtep.LogicalSwitch;
 import org.opendaylight.ovsdb.schema.hardwarevtep.PhysicalLocator;
 import org.opendaylight.ovsdb.schema.hardwarevtep.PhysicalSwitch;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.ovsdb.hwvtep.rev150901.hwvtep.global.attributes.LogicalSwitches;
+import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.ovsdb.hwvtep.rev150901.hwvtep.global.attributes.RemoteMcastMacs;
+import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.ovsdb.hwvtep.rev150901.hwvtep.global.attributes.RemoteUcastMacs;
+import org.opendaylight.yang.gen.v1.urn.tbd.params.xml.ns.yang.network.topology.rev131021.network.topology.topology.Node;
 import org.opendaylight.yang.gen.v1.urn.tbd.params.xml.ns.yang.network.topology.rev131021.network.topology.topology.node.TerminationPoint;
+import org.opendaylight.yangtools.yang.binding.DataObject;
 import org.opendaylight.yangtools.yang.binding.Identifiable;
 import org.opendaylight.yangtools.yang.binding.InstanceIdentifier;
 import org.slf4j.Logger;
@@ -25,6 +30,7 @@ import org.slf4j.LoggerFactory;
 import java.util.HashMap;
 import java.util.Iterator;
 import java.util.Map;
+import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 
 /*
@@ -60,12 +66,16 @@ public class HwvtepDeviceInfo {
         private final UUID uuid;
         private final Object data;
         private final DeviceDataStatus status;
+        private long intransitTimeStamp;
 
         DeviceData(InstanceIdentifier key, UUID uuid, Object data, DeviceDataStatus status) {
             this.data = data;
             this.key = key;
             this.status = status;
             this.uuid = uuid;
+            if (status == DeviceDataStatus.IN_TRANSIT) {
+                intransitTimeStamp = System.currentTimeMillis();
+            }
         }
 
         public Object getData() {
@@ -79,9 +89,24 @@ public class HwvtepDeviceInfo {
         public UUID getUuid() {
             return uuid;
         }
+
+        public InstanceIdentifier getKey() {
+            return key;
+        }
+
+        public boolean isIntransitTimeExpired() {
+            return System.currentTimeMillis()
+                    > intransitTimeStamp + HwvtepSouthboundConstants.IN_TRANSIT_STATE_EXPIRY_TIME_MILLIS;
+        }
+
+        public boolean isInTransitState() {
+            return status == DeviceDataStatus.IN_TRANSIT;
+        }
     }
 
-    //TODO remove this
+    private Map<InstanceIdentifier, Set<InstanceIdentifier>> tepIdReferences;
+    private Map<InstanceIdentifier<LogicalSwitches>, Map<InstanceIdentifier<RemoteUcastMacs>, RemoteUcastMacs>> logicalSwitchVsUcasts;
+    private Map<InstanceIdentifier<LogicalSwitches>, Map<InstanceIdentifier<RemoteMcastMacs>, RemoteMcastMacs>> logicalSwitchVsMcasts;
     private Map<UUID, LogicalSwitch> logicalSwitches = null;
     private Map<UUID, PhysicalSwitch> physicalSwitches = null;
     private Map<UUID, PhysicalLocator> physicalLocators = null;
@@ -96,10 +121,13 @@ public class HwvtepDeviceInfo {
 
     public HwvtepDeviceInfo(HwvtepConnectionInstance hwvtepConnectionInstance) {
         this.connectionInstance = hwvtepConnectionInstance;
-        this.logicalSwitches = new HashMap<>();
-        this.physicalSwitches = new HashMap<>();
-        this.physicalLocators = new HashMap<>();
-        this.mapTunnelToPhysicalSwitch = new HashMap<>();
+        this.logicalSwitches = new ConcurrentHashMap<>();
+        this.physicalSwitches = new ConcurrentHashMap<>();
+        this.physicalLocators = new ConcurrentHashMap<>();
+        this.mapTunnelToPhysicalSwitch = new ConcurrentHashMap<>();
+        this.tepIdReferences = new ConcurrentHashMap<>();
+        this.logicalSwitchVsUcasts = new ConcurrentHashMap<>();
+        this.logicalSwitchVsMcasts = new ConcurrentHashMap<>();
         this.dependencyQueue = new DependencyQueue(this);
     }
 
@@ -139,7 +167,7 @@ public class HwvtepDeviceInfo {
     }
 
     public Map<UUID, PhysicalLocator> getPhysicalLocators() {
-        Map<UUID, Object> locators = uuidVsData.get(LogicalSwitches.class);
+        Map<UUID, Object> locators = uuidVsData.get(TerminationPoint.class);
         Map<UUID, PhysicalLocator> result = new HashMap<>();
         if (locators != null) {
             for (Map.Entry<UUID, Object> entry : locators.entrySet()) {
@@ -180,7 +208,11 @@ public class HwvtepDeviceInfo {
     }
 
     public Object getConfigData(Class<? extends Identifiable> cls, InstanceIdentifier key) {
-        return HwvtepSouthboundUtil.getData(configKeyVsData, cls, key);
+        DeviceData deviceData = HwvtepSouthboundUtil.getData(configKeyVsData, cls, key);
+        if (deviceData != null) {
+            return deviceData.getData();
+        }
+        return null;
     }
 
     public void clearConfigData(Class<? extends Identifiable> cls, InstanceIdentifier key) {
@@ -239,7 +271,7 @@ public class HwvtepDeviceInfo {
         dependencyQueue.processReadyJobsFromConfigQueue(connectionInstance);
     }
 
-    public void onOperDataAvailable() {
+    public synchronized void onOperDataAvailable() {
         dependencyQueue.processReadyJobsFromOpQueue(connectionInstance);
     }
 
@@ -247,17 +279,120 @@ public class HwvtepDeviceInfo {
         dependencyQueue.submit(() -> connectionInstance.transact(transactCommand));
     }
 
-    public void clearInTransitData() {
-        //TODO restore old data
-        for (Map<InstanceIdentifier, DeviceData> map : opKeyVsData.values()) {
-            Iterator<Map.Entry<InstanceIdentifier, DeviceData>> iterator = map.entrySet().iterator();
-            while ( iterator.hasNext() ) {
-                Map.Entry<InstanceIdentifier, DeviceData> entry = iterator.next();
-                if (entry.getValue().getStatus() == DeviceDataStatus.IN_TRANSIT) {
-                    iterator.remove();
+    public void clearDeviceOperData(Class<? extends Identifiable> cls) {
+        Map<InstanceIdentifier, DeviceData> iids = opKeyVsData.get(cls);
+        if (iids != null && !iids.isEmpty()) {
+            Iterator<Map.Entry<InstanceIdentifier, DeviceData>> it = iids.entrySet().iterator();
+            while (it.hasNext()) {
+                Map.Entry<InstanceIdentifier, DeviceData> entry = it.next();
+                DeviceData deviceData = entry.getValue();
+                if (deviceData != null && deviceData.getStatus() != DeviceDataStatus.IN_TRANSIT) {
+                    it.remove();
                 }
             }
         }
-        onOperDataAvailable();
+    }
+
+    public void clearInTransit(Class<? extends Identifiable> cls, InstanceIdentifier key) {
+        DeviceData deviceData = getDeviceOperData(cls, key);
+        if (deviceData != null && deviceData.isInTransitState()) {
+            if (deviceData.getData() != null) {
+                HwvtepSouthboundUtil.updateData(opKeyVsData, cls, key,
+                        new DeviceData(key, deviceData.getUuid(), deviceData.getData(), DeviceDataStatus.AVAILABLE));
+            } else {
+                clearDeviceOperData(cls, key);
+            }
+        }
+    }
+
+    public Map<InstanceIdentifier, DeviceData> getDeviceOperData(Class<? extends Identifiable> cls) {
+        return opKeyVsData.get(cls);
+    }
+
+    public void incRefCount(InstanceIdentifier reference, InstanceIdentifier tep) {
+        if (reference == null || tep == null) {
+            return;
+        }
+        tepIdReferences.computeIfAbsent(tep, (tepId) -> Sets.newConcurrentHashSet());
+        tepIdReferences.get(tep).add(reference);
+    }
+
+    public int getRefCount(InstanceIdentifier tep) {
+        return tepIdReferences.containsKey(tep) ? tepIdReferences.get(tep).size() : 0;
+    }
+
+    public Set<InstanceIdentifier> getRefCounts(InstanceIdentifier tep) {
+        return tepIdReferences.get(tep);
+    }
+
+    public void decRefCount(InstanceIdentifier reference, InstanceIdentifier tep) {
+        if (reference == null || tep == null || !tepIdReferences.containsKey(tep)) {
+            return;
+        }
+        //synchronize to make sure that no two parallel deletes puts the key in transit state twice
+        synchronized (this) {
+            boolean removed = tepIdReferences.get(tep).remove(reference);
+            if (removed && tepIdReferences.get(tep).isEmpty()) {
+                LOG.debug("Marking the termination point as in transit ref count zero {} ", tep);
+                markKeyAsInTransit(TerminationPoint.class, tep);
+            }
+        }
+    }
+
+    public void clearLogicalSwitchRefs(InstanceIdentifier<LogicalSwitches> logicalSwitchKey) {
+        Map<InstanceIdentifier<RemoteMcastMacs>, RemoteMcastMacs> mcasts = logicalSwitchVsMcasts.get(logicalSwitchKey);
+        if (mcasts != null ) {
+            mcasts.entrySet().forEach( (entry) -> removeRemoteMcast(logicalSwitchKey, entry.getKey()));
+        }
+        Map<InstanceIdentifier<RemoteUcastMacs>, RemoteUcastMacs> ucasts = logicalSwitchVsUcasts.get(logicalSwitchKey);
+        if (ucasts != null ) {
+            ucasts.entrySet().forEach( (entry) -> removeRemoteUcast(logicalSwitchKey, entry.getKey()));
+        }
+        markKeyAsInTransit(LogicalSwitches.class, logicalSwitchKey);
+    }
+
+    public  void updateRemoteMcast(InstanceIdentifier<LogicalSwitches> lsIid,
+                                   InstanceIdentifier<RemoteMcastMacs> mcastIid,
+                                   RemoteMcastMacs mac) {
+        logicalSwitchVsMcasts.computeIfAbsent(lsIid, (lsKey) -> new ConcurrentHashMap<>());
+        logicalSwitchVsMcasts.get(lsIid).put(mcastIid, mac);
+        if (mac.getLocatorSet() != null) {
+            mac.getLocatorSet().forEach( (iid) -> incRefCount(mcastIid, iid.getLocatorRef().getValue()));
+        }
+    }
+
+    public  void updateRemoteUcast(InstanceIdentifier<LogicalSwitches> lsIid,
+                                   InstanceIdentifier<RemoteUcastMacs> ucastIid,
+                                   RemoteUcastMacs mac) {
+        logicalSwitchVsUcasts.computeIfAbsent(lsIid, (lsKey) -> new ConcurrentHashMap<>());
+        logicalSwitchVsUcasts.get(lsIid).put(ucastIid, mac);
+        incRefCount(ucastIid, mac.getLocatorRef().getValue());
+    }
+
+    public  void removeRemoteMcast(InstanceIdentifier<LogicalSwitches> lsIid, InstanceIdentifier<RemoteMcastMacs> mcastIid) {
+        if (!logicalSwitchVsMcasts.containsKey(lsIid)) {
+            return;
+        }
+        RemoteMcastMacs mac = logicalSwitchVsMcasts.get(lsIid).remove(mcastIid);
+        if (mac != null && mac.getLocatorSet() != null) {
+            mac.getLocatorSet().forEach((iid) -> decRefCount(mcastIid, iid.getLocatorRef().getValue()));
+        }
+        markKeyAsInTransit(RemoteMcastMacs.class, mcastIid);
+    }
+
+    public void removeRemoteUcast(InstanceIdentifier<LogicalSwitches> lsIid,
+                                   InstanceIdentifier<RemoteUcastMacs> ucastIid) {
+        if (!logicalSwitchVsUcasts.containsKey(lsIid)) {
+            return;
+        }
+        RemoteUcastMacs mac = logicalSwitchVsUcasts.get(lsIid).remove(ucastIid);
+        if (mac != null) {
+            decRefCount(ucastIid, mac.getLocatorRef().getValue());
+        }
+        markKeyAsInTransit(RemoteUcastMacs.class, ucastIid);
+    }
+
+    public HwvtepConnectionInstance getConnectionInstance() {
+        return connectionInstance;
     }
 }
index 0198493d90e2f9dee3a061046889fbb7d866dee5..de47d3023d7b96e945923964c1b9b98a285ec1db 100644 (file)
@@ -41,11 +41,16 @@ public class HwvtepSouthboundConstants {
             = new ImmutableMap.Builder<String,String>()
             .put("Logical_Binding_Stats", "Update callback registration for Logical_Binding_Stats Table is skipped")
             .build();
-
+    public static final String VERSION_COLUMN = "_version";
     public static final ImmutableMap<String,List<String>> SKIP_COLUMN_FROM_HWVTEP_TABLE
             = new ImmutableMap.Builder<String,List<String>>()
-            .put("Manager", Arrays.asList("_version", "status"))
+            .put("Manager", Arrays.asList(VERSION_COLUMN, "status"))
             .build();
-    public static final int WAITING_QUEUE_CAPACITY = 1000;
-    public static final long WAITING_JOB_EXPIRY_TIME_MILLIS = 90000;
+    public static final int WAITING_QUEUE_CAPACITY = Integer.getInteger("hwvtep.wait.queue.capacity", 1000);
+    public static final long WAITING_JOB_EXPIRY_TIME_MILLIS = Integer.getInteger(
+            "hwvtep.wait.job.expiry.time.millis", 90000);
+    public static final long IN_TRANSIT_STATE_EXPIRY_TIME_MILLIS = Integer.getInteger(
+            "hwvtep.intransit.job.expiry.time.millis", 10000);
+    public static final long IN_TRANSIT_STATE_CHECK_PERIOD_MILLIS = Integer.getInteger(
+            "hwvtep.intransit.job.check.period.millis", 30000);
 }
index 32c07889ffeb4d18493e1501ba85c10399745b1d..8a997b136cef1ca0eae2936899d3ecf866e8783b 100644 (file)
@@ -19,36 +19,45 @@ import java.util.List;
 import java.util.Map;
 import java.util.Objects;
 import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
 
 import org.opendaylight.controller.md.sal.binding.api.DataObjectModification;
 import org.opendaylight.controller.md.sal.binding.api.DataTreeModification;
+import org.opendaylight.controller.md.sal.common.api.data.LogicalDatastoreType;
 import org.opendaylight.ovsdb.hwvtepsouthbound.HwvtepDeviceInfo;
 import org.opendaylight.ovsdb.hwvtepsouthbound.HwvtepSouthboundUtil;
 import org.opendaylight.ovsdb.lib.notation.UUID;
 import org.opendaylight.ovsdb.lib.operations.TransactionBuilder;
+import org.opendaylight.ovsdb.utils.mdsal.utils.MdsalUtils;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.ovsdb.hwvtep.rev150901.hwvtep.global.attributes.LogicalSwitches;
 import org.opendaylight.yang.gen.v1.urn.tbd.params.xml.ns.yang.network.topology.rev131021.network.topology.topology.Node;
 import org.opendaylight.yang.gen.v1.urn.tbd.params.xml.ns.yang.network.topology.rev131021.network.topology.topology.node.TerminationPoint;
 import org.opendaylight.yangtools.yang.binding.Augmentation;
 import org.opendaylight.yangtools.yang.binding.Identifiable;
 import org.opendaylight.yangtools.yang.binding.InstanceIdentifier;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public abstract class AbstractTransactCommand<T extends Identifiable, Aug extends Augmentation<Node>> implements TransactCommand<T> {
 
-    private HwvtepOperationalState operationalState;
+    private static final Logger LOG = LoggerFactory.getLogger(AbstractTransactCommand.class);
+    protected static final UUID TXUUID = new UUID("TXUUID");
+    protected ThreadLocal<HwvtepOperationalState> threadLocalOperationalState = new ThreadLocal<>();
+    protected ThreadLocal<TransactionBuilder> threadLocalDeviceTransaction = new ThreadLocal<>();
     private Collection<DataTreeModification<Node>> changes;
+    protected Map<TransactionBuilder, List<MdsalUpdate<T>>> updates = new ConcurrentHashMap<>();
 
     protected AbstractTransactCommand() {
         // NO OP
     }
 
     public AbstractTransactCommand(HwvtepOperationalState state, Collection<DataTreeModification<Node>> changes) {
-        this.operationalState = state;
+        this.threadLocalOperationalState.set(state);
         this.changes = changes;
     }
 
     public HwvtepOperationalState getOperationalState() {
-        return operationalState;
+        return threadLocalOperationalState.get();
     }
 
     public Collection<DataTreeModification<Node>> getChanges() {
@@ -56,13 +65,26 @@ public abstract class AbstractTransactCommand<T extends Identifiable, Aug extend
     }
 
     void updateCurrentTxDeleteData(Class<? extends Identifiable> cls, InstanceIdentifier key, T data) {
-        operationalState.updateCurrentTxDeleteData(cls, key);
-        operationalState.getDeviceInfo().clearConfigData(cls, key);
+        getOperationalState().getDeviceInfo().markKeyAsInTransit(cls, key);
+        addToUpdates(key, data);
+        getOperationalState().getDeviceInfo().clearConfigData(cls, key);
     }
 
-    void updateCurrentTxData(Class<? extends Identifiable> cls, InstanceIdentifier key, UUID uuid, Object data) {
-        operationalState.updateCurrentTxData(cls, key, uuid);
-        operationalState.getDeviceInfo().updateConfigData(cls, key, data);
+    void updateCurrentTxData(Class<? extends Identifiable> cls, InstanceIdentifier key, UUID uuid, T data) {
+        getOperationalState().getDeviceInfo().markKeyAsInTransit(cls, key);
+        addToUpdates(key, data);
+        getOperationalState().getDeviceInfo().updateConfigData(cls, key, data);
+    }
+
+    void addToUpdates(InstanceIdentifier key, T data) {
+        T oldData = null;
+        Type type = getClass().getGenericSuperclass();
+        Type classType = ((ParameterizedType) type).getActualTypeArguments()[0];
+        if (getDeviceInfo().getConfigData((Class<? extends Identifiable>) classType, key) != null) {
+            oldData = (T) getDeviceInfo().getConfigData((Class<? extends Identifiable>) classType, key);
+        }
+        updates.putIfAbsent(getDeviceTransaction(), new ArrayList<MdsalUpdate<T>>());
+        updates.get(getDeviceTransaction()).add(new MdsalUpdate<T>(key, data, oldData));
     }
 
     void processDependencies(final UnMetDependencyGetter<T> unMetDependencyGetter,
@@ -71,13 +93,14 @@ public abstract class AbstractTransactCommand<T extends Identifiable, Aug extend
                              final InstanceIdentifier key,
                              final T data, final Object... extraData) {
 
-        HwvtepDeviceInfo deviceInfo = operationalState.getDeviceInfo();
-        Map inTransitDependencies = Collections.EMPTY_MAP;
-        Map confingDependencies = Collections.EMPTY_MAP;
+        this.threadLocalDeviceTransaction.set(transaction);
+        HwvtepDeviceInfo deviceInfo = getOperationalState().getDeviceInfo();
+        Map inTransitDependencies = new HashMap<>();
+        Map confingDependencies = new HashMap<>();
 
         if (!isRemoveCommand() && unMetDependencyGetter != null) {
-            inTransitDependencies = unMetDependencyGetter.getInTransitDependencies(operationalState, data);
-            confingDependencies = unMetDependencyGetter.getUnMetConfigDependencies(operationalState, data);
+            inTransitDependencies = unMetDependencyGetter.getInTransitDependencies(getOperationalState(), data);
+            confingDependencies = unMetDependencyGetter.getUnMetConfigDependencies(getOperationalState(), data);
             //we can skip the config termination point dependency as we can create them in device as part of this tx
             confingDependencies.remove(TerminationPoint.class);
         }
@@ -92,7 +115,11 @@ public abstract class AbstractTransactCommand<T extends Identifiable, Aug extend
 
         if (HwvtepSouthboundUtil.isEmptyMap(confingDependencies) && HwvtepSouthboundUtil.isEmptyMap(inTransitDependencies)) {
             doDeviceTransaction(transaction, nodeIid, data, key, extraData);
-            updateCurrentTxData((Class<? extends Identifiable>) classType, key, new UUID("uuid"), data);
+            if (isRemoveCommand()) {
+                getDeviceInfo().clearConfigData((Class<? extends Identifiable>) classType, key);
+            } else {
+                getDeviceInfo().updateConfigData((Class<? extends Identifiable>) classType, key, data);
+            }
         }
         if (!HwvtepSouthboundUtil.isEmptyMap(confingDependencies)) {
             DependentJob<T> configWaitingJob = new DependentJob.ConfigWaitingJob(
@@ -101,9 +128,18 @@ public abstract class AbstractTransactCommand<T extends Identifiable, Aug extend
                 @Override
                 public void onDependencyResolved(HwvtepOperationalState operationalState,
                                                  TransactionBuilder transactionBuilder) {
-                    AbstractTransactCommand.this.operationalState = operationalState;
+                    AbstractTransactCommand.this.threadLocalOperationalState.set(operationalState);
+                    AbstractTransactCommand.this.threadLocalDeviceTransaction.set(transactionBuilder);
                     onConfigUpdate(transactionBuilder, nodeIid, data, key, extraData);
                 }
+
+                public void onFailure() {
+                    AbstractTransactCommand.this.onFailure(getDeviceTransaction());
+                }
+
+                public void onSuccess() {
+                    AbstractTransactCommand.this.onSuccess(getDeviceTransaction());
+                }
             };
             deviceInfo.addJobToQueue(configWaitingJob);
         }
@@ -115,8 +151,24 @@ public abstract class AbstractTransactCommand<T extends Identifiable, Aug extend
                 @Override
                 public void onDependencyResolved(HwvtepOperationalState operationalState,
                                                  TransactionBuilder transactionBuilder) {
-                    AbstractTransactCommand.this.operationalState = operationalState;
-                    onConfigUpdate(transactionBuilder, nodeIid, data, key, extraData);
+                    //data would have got deleted by , push the data only if it is still in configds
+                    threadLocalOperationalState.set(operationalState);
+                    threadLocalDeviceTransaction.set(transactionBuilder);
+                    T data = (T)new MdsalUtils(operationalState.getDataBroker()).read(
+                            LogicalDatastoreType.CONFIGURATION, key);
+                    if (data != null) {
+                        onConfigUpdate(transactionBuilder, nodeIid, data, key, extraData);
+                    } else {
+                        LOG.warn("Skipping add of key: {} as it is not present txId: {}", key);
+                    }
+                }
+
+                public void onFailure() {
+                    AbstractTransactCommand.this.onFailure(getDeviceTransaction());
+                }
+
+                public void onSuccess() {
+                    AbstractTransactCommand.this.onSuccess(getDeviceTransaction());
                 }
             };
             deviceInfo.addJobToQueue(opWaitingJob);
@@ -166,10 +218,10 @@ public abstract class AbstractTransactCommand<T extends Identifiable, Aug extend
                 final InstanceIdentifier<Node> key = change.getRootPath().getRootIdentifier();
                 Class<? extends Identifiable> classType = (Class<? extends Identifiable>) getClassType();
                 List<T> removed;
-                if (operationalState.isInReconciliation()) {
+                if (getOperationalState().isInReconciliation()) {
                     removed = getRemoved(change);
                 } else {
-                    removed = (List<T>) operationalState.getDeletedData(key, classType);
+                    removed = (List<T>) getOperationalState().getDeletedData(key, classType);
                 }
                 removed.addAll(getCascadeDeleteData(change));
                 result.put(key, removed);
@@ -186,10 +238,10 @@ public abstract class AbstractTransactCommand<T extends Identifiable, Aug extend
                 InstanceIdentifier<Node> key = change.getRootPath().getRootIdentifier();
                 Class<? extends Identifiable> classType = (Class<? extends Identifiable>) getClassType();
                 List<T> updated = null;
-                if (operationalState.isInReconciliation()) {
+                if (getOperationalState().isInReconciliation()) {
                     updated = getUpdated(change);
                 } else {
-                    updated = (List<T>) operationalState.getUpdatedData(key, classType);
+                    updated = (List<T>) getOperationalState().getUpdatedData(key, classType);
                 }
                 result.put(key, updated);
             }
@@ -306,4 +358,35 @@ public abstract class AbstractTransactCommand<T extends Identifiable, Aug extend
     protected boolean isRemoveCommand() {
         return false;
     }
+
+    protected HwvtepDeviceInfo getDeviceInfo() {
+        return getOperationalState().getDeviceInfo();
+    }
+
+    protected TransactionBuilder getDeviceTransaction() {
+        return threadLocalDeviceTransaction.get();
+    }
+
+    public void onSuccess(TransactionBuilder deviceTransaction) {
+        if (deviceTransaction == null || !updates.containsKey(deviceTransaction)) {
+            return;
+        }
+        onCommandSucceeded();
+    }
+
+    public void onFailure(TransactionBuilder deviceTransaction) {
+        if (deviceTransaction == null || !updates.containsKey(deviceTransaction)) {
+            return;
+        }
+        for (MdsalUpdate mdsalUpdate : updates.get(deviceTransaction)) {
+            getDeviceInfo().clearInTransit((Class<? extends Identifiable>)mdsalUpdate.getClass(), mdsalUpdate.getKey());
+        }
+        onCommandFailed();
+    }
+
+    protected void onCommandSucceeded() {
+    }
+
+    protected void onCommandFailed() {
+    }
 }
index 4f6b09c15d673ba9e9d26a90fc117f1bca6ed29f..ea7dee8a183ee10b168e8938d240b5cb908603bc 100644 (file)
@@ -472,4 +472,21 @@ public class HwvtepOperationalState {
     public void setInReconciliation(boolean inReconciliation) {
         this.inReconciliation = inReconciliation;
     }
+
+    public DataBroker getDataBroker() {
+        return db;
+    }
+
+
+    public void clearIntransitKeys() {
+        currentTxUUIDs.forEach( (cls, map) -> {
+            map.forEach( (iid, uuid) -> deviceInfo.clearInTransit(cls, iid));
+        });
+        currentTxDeletedKeys.forEach( (cls, map) -> {
+            map.forEach( (iid, val) -> deviceInfo.clearInTransit(cls, iid));
+        });
+        currentTxUUIDs.clear();
+        currentTxDeletedKeys.clear();
+        deviceInfo.onOperDataAvailable();
+    }
 }
index b4a785718ddd41d967aac3e124839226d579326f..139334bfafa715294856c22020bb8ff4959b0f51 100644 (file)
@@ -12,10 +12,12 @@ import static org.opendaylight.ovsdb.lib.operations.Operations.op;
 
 import com.google.common.base.Optional;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 import org.opendaylight.controller.md.sal.binding.api.DataTreeModification;
+import org.opendaylight.ovsdb.hwvtepsouthbound.HwvtepConnectionInstance;
 import org.opendaylight.ovsdb.hwvtepsouthbound.HwvtepSouthboundUtil;
 import org.opendaylight.ovsdb.lib.notation.UUID;
 import org.opendaylight.ovsdb.lib.operations.TransactionBuilder;
@@ -44,26 +46,27 @@ public class LogicalRouterRemoveCommand extends AbstractTransactCommand<LogicalR
       if (removed != null) {
           for (Entry<InstanceIdentifier<Node>, List<LogicalRouters>> created: removed.entrySet()) {
               if (!HwvtepSouthboundUtil.isEmpty(created.getValue())) {
-                  for (LogicalRouters lRouter : created.getValue()) {
-                      InstanceIdentifier<LogicalRouters> lsKey = created.getKey().augmentation(
-                              HwvtepGlobalAugmentation.class).child(LogicalRouters.class, lRouter.getKey());
-                      updateCurrentTxDeleteData(LogicalRouters.class, lsKey, lRouter);
-                  }
                   getOperationalState().getDeviceInfo().scheduleTransaction(new TransactCommand() {
                       @Override
                       public void execute(TransactionBuilder transactionBuilder) {
+                          HwvtepConnectionInstance connectionInstance = getDeviceInfo().getConnectionInstance();
+                          HwvtepOperationalState operState = new HwvtepOperationalState(
+                                  connectionInstance.getDataBroker(), connectionInstance, Collections.EMPTY_LIST);
+                          threadLocalOperationalState.set(operState);
+                          threadLocalDeviceTransaction.set(transactionBuilder);
                           LOG.debug("Running delete logical router in seperate tx {}", created.getKey());
                           removeLogicalRouter(transactionBuilder, created.getKey(), created.getValue());
                       }
 
+
                       @Override
-                      public void onConfigUpdate(TransactionBuilder transaction, InstanceIdentifier nodeIid,
-                                                 Identifiable data, InstanceIdentifier key, Object... extraData) {
+                      public void onSuccess(TransactionBuilder deviceTransaction) {
+                          LogicalRouterRemoveCommand.this.onSuccess(deviceTransaction);
                       }
 
                       @Override
-                      public void doDeviceTransaction(TransactionBuilder transaction, InstanceIdentifier nodeIid,
-                                                      Identifiable data, InstanceIdentifier key, Object... extraData) {
+                      public void onFailure(TransactionBuilder deviceTransaction) {
+                          LogicalRouterRemoveCommand.this.onFailure(deviceTransaction);
                       }
                   });
               }
index 7ba2d7b1555dbaafde7d1c7cbee7954d6de66d4e..c1c3cdbaacd4002662569c8c7ee300bd3bbb5a68 100644 (file)
@@ -11,12 +11,14 @@ package org.opendaylight.ovsdb.hwvtepsouthbound.transact;
 import static org.opendaylight.ovsdb.lib.operations.Operations.op;
 
 import java.util.Collection;
+import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Objects;
 
 import org.opendaylight.controller.md.sal.binding.api.DataTreeModification;
+import org.opendaylight.ovsdb.hwvtepsouthbound.HwvtepConnectionInstance;
 import org.opendaylight.ovsdb.hwvtepsouthbound.HwvtepDeviceInfo;
 import org.opendaylight.ovsdb.hwvtepsouthbound.HwvtepSouthboundUtil;
 import org.opendaylight.ovsdb.lib.notation.UUID;
@@ -53,26 +55,26 @@ public class LogicalSwitchRemoveCommand extends AbstractTransactCommand<LogicalS
         if (removeds != null) {
             for (Entry<InstanceIdentifier<Node>, List<LogicalSwitches>> created: removeds.entrySet()) {
                 if (!HwvtepSouthboundUtil.isEmpty(created.getValue())) {
-                    for (LogicalSwitches lswitch : created.getValue()) {
-                        InstanceIdentifier<LogicalSwitches> lsKey = created.getKey().augmentation(
-                                HwvtepGlobalAugmentation.class).child(LogicalSwitches.class, lswitch.getKey());
-                        updateCurrentTxDeleteData(LogicalSwitches.class, lsKey, lswitch);
-                    }
-                    getOperationalState().getDeviceInfo().scheduleTransaction(new TransactCommand() {
+                    HwvtepConnectionInstance connectionInstance = getDeviceInfo().getConnectionInstance();
+                    getDeviceInfo().scheduleTransaction(new TransactCommand() {
                         @Override
                         public void execute(TransactionBuilder transactionBuilder) {
+                            HwvtepOperationalState operState = new HwvtepOperationalState(
+                                    connectionInstance.getDataBroker(), connectionInstance, Collections.EMPTY_LIST);
+                            threadLocalOperationalState.set(operState);
+                            threadLocalDeviceTransaction.set(transactionBuilder);
                             LOG.debug("Running delete logical switch in seperate tx {}", created.getKey());
                             removeLogicalSwitch(transactionBuilder, created.getKey(), created.getValue());
                         }
 
                         @Override
-                        public void onConfigUpdate(TransactionBuilder transaction, InstanceIdentifier nodeIid,
-                                                   Identifiable data, InstanceIdentifier key, Object... extraData) {
+                        public void onSuccess(TransactionBuilder deviceTransaction) {
+                            LogicalSwitchRemoveCommand.this.onSuccess(deviceTransaction);
                         }
 
                         @Override
-                        public void doDeviceTransaction(TransactionBuilder transaction, InstanceIdentifier nodeIid,
-                                                        Identifiable data, InstanceIdentifier key, Object... extraData) {
+                        public void onFailure(TransactionBuilder deviceTransaction) {
+                            LogicalSwitchRemoveCommand.this.onFailure(deviceTransaction);
                         }
                     });
                 }
@@ -133,7 +135,7 @@ public class LogicalSwitchRemoveCommand extends AbstractTransactCommand<LogicalS
                         McastMacsLocal.class, null);
                 transaction.add(op.delete(mcastMacsLocal.getSchema())
                         .where(mcastMacsLocal.getLogicalSwitchColumn().getSchema().opEqual(logicalSwitchUuid)).build());
-                getOperationalState().getDeviceInfo().markKeyAsInTransit(RemoteMcastMacs.class, lsKey);
+                updateCurrentTxDeleteData(LogicalSwitches.class, lsKey, lswitch);
             } else {
                 LOG.warn("Unable to delete logical switch {} because it was not found in the operational store",
                         lswitch.getHwvtepNodeName().getValue());
@@ -154,4 +156,14 @@ public class LogicalSwitchRemoveCommand extends AbstractTransactCommand<LogicalS
     protected boolean isRemoveCommand() {
         return true;
     }
+
+    @Override
+    public void onCommandSucceeded() {
+        if (getDeviceTransaction() == null || !updates.containsKey(getDeviceTransaction())) {
+            return;
+        }
+        for (MdsalUpdate mdsalUpdate : updates.get(getDeviceTransaction())) {
+            getDeviceInfo().clearLogicalSwitchRefs((InstanceIdentifier<LogicalSwitches>) mdsalUpdate.getKey());
+        }
+    }
 }
index 018c0910124ce1614eb5a59e821bded0b7cc906d..072800d6633495063648759ac8f5fdafa334f6e6 100644 (file)
@@ -20,6 +20,7 @@ import java.util.Objects;
 import java.util.Set;
 
 import org.opendaylight.controller.md.sal.binding.api.DataTreeModification;
+import org.opendaylight.ovsdb.hwvtepsouthbound.HwvtepDeviceInfo;
 import org.opendaylight.ovsdb.lib.notation.UUID;
 import org.opendaylight.ovsdb.lib.operations.TransactionBuilder;
 import org.opendaylight.ovsdb.lib.schema.typed.TyperUtils;
@@ -55,7 +56,7 @@ public class LogicalSwitchUpdateCommand extends AbstractTransactCommand<LogicalS
         }
     }
 
-    private void updateLogicalSwitch(final TransactionBuilder transaction,
+    public void updateLogicalSwitch(final TransactionBuilder transaction,
                                      final InstanceIdentifier<Node> nodeIid, final List<LogicalSwitches> lswitchList) {
         for (LogicalSwitches lswitch: lswitchList) {
             InstanceIdentifier<LogicalSwitches> lsKey = nodeIid.augmentation(HwvtepGlobalAugmentation.class).
@@ -80,22 +81,21 @@ public class LogicalSwitchUpdateCommand extends AbstractTransactCommand<LogicalS
                                     final InstanceIdentifier lsKey,
                                     final Object... extraData) {
             LOG.debug("Creating logical switch named: {}", lswitch.getHwvtepNodeName());
-            Optional<LogicalSwitches> operationalSwitchOptional =
-                    getOperationalState().getLogicalSwitches(instanceIdentifier, lswitch.getKey());
+            HwvtepDeviceInfo.DeviceData operationalSwitchOptional =
+                    getDeviceInfo().getDeviceOperData(LogicalSwitches.class, lsKey);
             LogicalSwitch logicalSwitch = TyperUtils.getTypedRowWrapper(transaction.getDatabaseSchema(), LogicalSwitch.class);
             setDescription(logicalSwitch, lswitch);
             setTunnelKey(logicalSwitch, lswitch);
             setReplicationMode(logicalSwitch, lswitch);
-            if (!operationalSwitchOptional.isPresent()) {
-                setName(logicalSwitch, lswitch, operationalSwitchOptional);
+            if (operationalSwitchOptional == null) {
+                setName(logicalSwitch, lswitch);
                 LOG.trace("execute: creating LogicalSwitch entry: {}", logicalSwitch);
                 transaction.add(op.insert(logicalSwitch).withId(TransactUtils.getLogicalSwitchId(lswitch)));
                 transaction.add(op.comment("Logical Switch: Creating " + lswitch.getHwvtepNodeName().getValue()));
                 UUID lsUuid = new UUID(TransactUtils.getLogicalSwitchId(lswitch));
-                getOperationalState().getDeviceInfo().markKeyAsInTransit(RemoteMcastMacs.class, lsKey);
+                updateCurrentTxData(LogicalSwitches.class, lsKey, lsUuid, lswitch);
             } else {
-                LogicalSwitches updatedLSwitch = operationalSwitchOptional.get();
-                String existingLogicalSwitchName = updatedLSwitch.getHwvtepNodeName().getValue();
+                String existingLogicalSwitchName = lswitch.getHwvtepNodeName().getValue();
                 // Name is immutable, and so we *can't* update it.  So we use extraBridge for the schema stuff
                 LogicalSwitch extraLogicalSwitch = TyperUtils.getTypedRowWrapper(transaction.getDatabaseSchema(), LogicalSwitch.class);
                 extraLogicalSwitch.setName("");
@@ -113,12 +113,9 @@ public class LogicalSwitchUpdateCommand extends AbstractTransactCommand<LogicalS
         }
     }
 
-    private void setName(LogicalSwitch logicalSwitch, LogicalSwitches inputSwitch,
-            Optional<LogicalSwitches> inputSwitchOptional) {
+    private void setName(LogicalSwitch logicalSwitch, LogicalSwitches inputSwitch) {
         if (inputSwitch.getHwvtepNodeName() != null) {
             logicalSwitch.setName(inputSwitch.getHwvtepNodeName().getValue());
-        } else if (inputSwitchOptional.isPresent() && inputSwitchOptional.get().getHwvtepNodeName() != null) {
-            logicalSwitch.setName(inputSwitchOptional.get().getHwvtepNodeName().getValue());
         }
     }
 
index 8aeff6a790b651d2030bd450c73c172d9bc0cb19..68e2abeb1c754de1d79bbcab7968d56933ebddcb 100644 (file)
@@ -18,20 +18,20 @@ import java.util.Map.Entry;
 import java.util.Objects;
 
 import org.opendaylight.controller.md.sal.binding.api.DataTreeModification;
+import org.opendaylight.ovsdb.hwvtepsouthbound.HwvtepDeviceInfo;
 import org.opendaylight.ovsdb.hwvtepsouthbound.HwvtepSouthboundUtil;
 import org.opendaylight.ovsdb.lib.notation.UUID;
 import org.opendaylight.ovsdb.lib.operations.TransactionBuilder;
 import org.opendaylight.ovsdb.lib.schema.typed.TyperUtils;
 import org.opendaylight.ovsdb.schema.hardwarevtep.McastMacsRemote;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.ovsdb.hwvtep.rev150901.HwvtepGlobalAugmentation;
+import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.ovsdb.hwvtep.rev150901.hwvtep.global.attributes.LogicalSwitches;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.ovsdb.hwvtep.rev150901.hwvtep.global.attributes.RemoteMcastMacs;
 import org.opendaylight.yang.gen.v1.urn.tbd.params.xml.ns.yang.network.topology.rev131021.network.topology.topology.Node;
 import org.opendaylight.yangtools.yang.binding.InstanceIdentifier;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.google.common.base.Optional;
-
 public class McastMacsRemoteRemoveCommand extends AbstractTransactCommand<RemoteMcastMacs, HwvtepGlobalAugmentation> {
     private static final Logger LOG = LoggerFactory.getLogger(McastMacsRemoteRemoveCommand.class);
 
@@ -95,24 +95,23 @@ public class McastMacsRemoteRemoveCommand extends AbstractTransactCommand<Remote
                                     final InstanceIdentifier macIid,
                                     final Object... extraData) {
             LOG.debug("Removing remoteMcastMacs, mac address: {}", mac.getMacEntryKey().getValue());
-            Optional<RemoteMcastMacs> operationalMacOptional =
-                    getOperationalState().getRemoteMcastMacs(instanceIdentifier, mac.getKey());
+            HwvtepDeviceInfo.DeviceData operationalMacOptional =
+                    getDeviceInfo().getDeviceOperData(RemoteMcastMacs.class, macIid);
             McastMacsRemote mcastMacsRemote = TyperUtils.getTypedRowWrapper(transaction.getDatabaseSchema(),
                     McastMacsRemote.class, null);
-            if (operationalMacOptional.isPresent() && operationalMacOptional.get().getMacEntryUuid() != null) {
+            if (operationalMacOptional != null && operationalMacOptional.getUuid() != null) {
                 //when mac entry is deleted, its referenced locator set and locators are deleted automatically.
                 //TODO: locator in config DS is not deleted
-                UUID macEntryUUID = new UUID(operationalMacOptional.get().getMacEntryUuid().getValue());
+                UUID macEntryUUID = operationalMacOptional.getUuid();
                 mcastMacsRemote.getUuidColumn().setData(macEntryUUID);
                 transaction.add(op.delete(mcastMacsRemote.getSchema()).
                         where(mcastMacsRemote.getUuidColumn().getSchema().opEqual(macEntryUUID)).build());
                 transaction.add(op.comment("McastMacRemote: Deleting " + mac.getMacEntryKey().getValue()));
-                getOperationalState().getDeviceInfo().markKeyAsInTransit(RemoteMcastMacs.class, macIid);
+                updateCurrentTxDeleteData(RemoteMcastMacs.class, macIid, mac);
             } else {
                 LOG.warn("Unable to delete remoteMcastMacs {} because it was not found in the operational store",
                         mac.getMacEntryKey().getValue());
             }
-            updateCurrentTxDeleteData(RemoteMcastMacs.class, macIid, mac);
     }
 
     @Override
@@ -129,4 +128,16 @@ public class McastMacsRemoteRemoveCommand extends AbstractTransactCommand<Remote
     protected boolean isRemoveCommand() {
         return true;
     }
+
+
+    @Override
+    public void onCommandSucceeded() {
+        //remove the refcounts of the deleted macs
+        for (MdsalUpdate mdsalUpdate : updates.get(getDeviceTransaction())) {
+            RemoteMcastMacs deletedMac = (RemoteMcastMacs) mdsalUpdate.getNewData();
+            InstanceIdentifier<RemoteMcastMacs> macIid = mdsalUpdate.getKey();
+            getDeviceInfo().removeRemoteMcast(
+                    (InstanceIdentifier<LogicalSwitches>) deletedMac.getLogicalSwitchRef().getValue(), macIid);
+        }
+    }
 }
index 6d6538faca9cbce7a4371c903f054c8c7785cde7..246aa286570946e240dab16aa1c5738ce42b28bc 100644 (file)
@@ -8,8 +8,6 @@
 
 package org.opendaylight.ovsdb.hwvtepsouthbound.transact;
 
-import com.google.common.base.Optional;
-
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
@@ -19,13 +17,13 @@ import java.util.Map.Entry;
 import java.util.Objects;
 
 import org.opendaylight.controller.md.sal.binding.api.DataTreeModification;
+import org.opendaylight.ovsdb.hwvtepsouthbound.HwvtepDeviceInfo;
 import org.opendaylight.ovsdb.hwvtepsouthbound.HwvtepSouthboundConstants;
 import org.opendaylight.ovsdb.hwvtepsouthbound.HwvtepSouthboundUtil;
 import org.opendaylight.ovsdb.lib.notation.UUID;
 import org.opendaylight.ovsdb.lib.operations.TransactionBuilder;
 import org.opendaylight.ovsdb.lib.schema.typed.TyperUtils;
 import org.opendaylight.ovsdb.schema.hardwarevtep.McastMacsRemote;
-import org.opendaylight.yang.gen.v1.urn.ietf.params.xml.ns.yang.ietf.yang.types.rev130715.Uuid;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.ovsdb.hwvtep.rev150901.HwvtepGlobalAugmentation;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.ovsdb.hwvtep.rev150901.hwvtep.global.attributes.LogicalSwitches;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.ovsdb.hwvtep.rev150901.hwvtep.global.attributes.RemoteMcastMacs;
@@ -86,20 +84,20 @@ public class McastMacsRemoteUpdateCommand extends AbstractTransactCommand<Remote
                                        InstanceIdentifier macKey,
                                        Object... extraData) {
             LOG.debug("Creating remoteMcastMacs, mac address: {}", mac.getMacEntryKey().getValue());
-            Optional<RemoteMcastMacs> operationalMacOptional =
-                    getOperationalState().getRemoteMcastMacs(instanceIdentifier, mac.getKey());
+        HwvtepDeviceInfo.DeviceData operationalMacOptional =
+                    getDeviceInfo().getDeviceOperData(RemoteMcastMacs.class, macKey);
             McastMacsRemote mcastMacsRemote = TyperUtils.getTypedRowWrapper(transaction.getDatabaseSchema(), McastMacsRemote.class);
             setIpAddress(mcastMacsRemote, mac);
             setLocatorSet(transaction, mcastMacsRemote, mac);
-            setLogicalSwitch(mcastMacsRemote, mac);
-            if (!operationalMacOptional.isPresent()) {
-                setMac(mcastMacsRemote, mac, operationalMacOptional);
+            setLogicalSwitch(transaction, mcastMacsRemote, mac);
+            if (operationalMacOptional == null) {
+                setMac(mcastMacsRemote, mac);
                 LOG.trace("execute: create RemoteMcastMac entry: {}", mcastMacsRemote);
                 transaction.add(op.insert(mcastMacsRemote));
                 transaction.add(op.comment("McastMacRemote: Creating " + mac.getMacEntryKey().getValue()));
-                getOperationalState().getDeviceInfo().markKeyAsInTransit(RemoteMcastMacs.class, macKey);
-            } else if (operationalMacOptional.get().getMacEntryUuid() != null) {
-                UUID macEntryUUID = new UUID(operationalMacOptional.get().getMacEntryUuid().getValue());
+                updateCurrentTxData(RemoteMcastMacs.class, macKey, TXUUID, mac);
+            } else if (operationalMacOptional.getUuid() != null) {
+                UUID macEntryUUID = operationalMacOptional.getUuid();
                 McastMacsRemote extraMac = TyperUtils.getTypedRowWrapper(transaction.getDatabaseSchema(),
                                 McastMacsRemote.class, null);
                 extraMac.getUuidColumn().setData(macEntryUUID);
@@ -108,25 +106,19 @@ public class McastMacsRemoteUpdateCommand extends AbstractTransactCommand<Remote
                         .where(extraMac.getUuidColumn().getSchema().opEqual(macEntryUUID))
                         .build());
                 transaction.add(op.comment("McastMacRemote: Updating " + macEntryUUID));
+                //add to updates so that tep ref counts can be updated upon success
+                addToUpdates(macKey, mac);
             } else {
                 LOG.warn("Unable to update remoteMcastMacs {} because uuid not found in the operational store",
                                 mac.getMacEntryKey().getValue());
             }
     }
 
-    private void setLogicalSwitch(McastMacsRemote mcastMacsRemote, RemoteMcastMacs inputMac) {
+    private void setLogicalSwitch(final TransactionBuilder transaction, final McastMacsRemote mcastMacsRemote, final RemoteMcastMacs inputMac) {
         if (inputMac.getLogicalSwitchRef() != null) {
             @SuppressWarnings("unchecked")
             InstanceIdentifier<LogicalSwitches> lswitchIid = (InstanceIdentifier<LogicalSwitches>) inputMac.getLogicalSwitchRef().getValue();
-            Optional<LogicalSwitches> operationalSwitchOptional =
-                    getOperationalState().getLogicalSwitches(lswitchIid);
-            if (operationalSwitchOptional.isPresent()) {
-                Uuid logicalSwitchUuid = operationalSwitchOptional.get().getLogicalSwitchUuid();
-                UUID logicalSwitchUUID = new UUID(logicalSwitchUuid.getValue());
-                mcastMacsRemote.setLogicalSwitch(logicalSwitchUUID);
-            } else {
-                mcastMacsRemote.setLogicalSwitch(TransactUtils.getLogicalSwitchUUID(lswitchIid));
-            }
+            mcastMacsRemote.setLogicalSwitch(TransactUtils.getLogicalSwitchUUID(transaction, getOperationalState(), lswitchIid));
         }
     }
 
@@ -143,16 +135,13 @@ public class McastMacsRemoteUpdateCommand extends AbstractTransactCommand<Remote
         }
     }
 
-    private void setMac(McastMacsRemote mcastMacsRemote, RemoteMcastMacs inputMac,
-            Optional<RemoteMcastMacs> inputSwitchOptional) {
+    private void setMac(McastMacsRemote mcastMacsRemote, RemoteMcastMacs inputMac) {
         if (inputMac.getMacEntryKey() != null) {
             if (inputMac.getMacEntryKey().equals(HwvtepSouthboundConstants.UNKNOWN_DST_MAC)) {
                 mcastMacsRemote.setMac(HwvtepSouthboundConstants.UNKNOWN_DST_STRING);
             } else {
                 mcastMacsRemote.setMac(inputMac.getMacEntryKey().getValue());
             }
-        } else if (inputSwitchOptional.isPresent() && inputSwitchOptional.get().getMacEntryKey() != null) {
-            mcastMacsRemote.setMac(inputSwitchOptional.get().getMacEntryKey().getValue());
         }
     }
 
@@ -186,4 +175,33 @@ public class McastMacsRemoteUpdateCommand extends AbstractTransactCommand<Remote
             return locators;
         }
     }
+
+    private void updateLocatorRefCounts(MdsalUpdate mdsalUpdate) {
+        //decrement the refcounts from old mcast mac
+        //increment the refcounts for new mcast mac
+        RemoteMcastMacs newMac = (RemoteMcastMacs) mdsalUpdate.getNewData();
+        RemoteMcastMacs oldMac = (RemoteMcastMacs) mdsalUpdate.getOldData();
+        InstanceIdentifier<RemoteMcastMacs> macIid = mdsalUpdate.getKey();
+
+        if (oldMac != null && !oldMac.equals(newMac)) {
+            if (oldMac.getLocatorSet() != null) {
+                List<LocatorSet> removedLocators = new ArrayList(oldMac.getLocatorSet());
+                if (newMac.getLocatorSet() != null) {
+                    removedLocators.removeAll(newMac.getLocatorSet());
+                }
+                removedLocators.forEach( (iid) -> getDeviceInfo().decRefCount(macIid, iid.getLocatorRef().getValue()));
+            }
+        }
+    }
+
+    @Override
+    protected void onCommandSucceeded() {
+        for (MdsalUpdate mdsalUpdate : updates.get(getDeviceTransaction())) {
+            updateLocatorRefCounts(mdsalUpdate);
+            RemoteMcastMacs mac = (RemoteMcastMacs) mdsalUpdate.getNewData();
+            InstanceIdentifier<RemoteMcastMacs> macIid = mdsalUpdate.getKey();
+            getDeviceInfo().updateRemoteMcast(
+                    (InstanceIdentifier<LogicalSwitches>) mac.getLogicalSwitchRef().getValue(), macIid, mac);
+        }
+    }
 }
diff --git a/hwvtepsouthbound/hwvtepsouthbound-impl/src/main/java/org/opendaylight/ovsdb/hwvtepsouthbound/transact/MdsalUpdate.java b/hwvtepsouthbound/hwvtepsouthbound-impl/src/main/java/org/opendaylight/ovsdb/hwvtepsouthbound/transact/MdsalUpdate.java
new file mode 100644 (file)
index 0000000..eb3dcda
--- /dev/null
@@ -0,0 +1,37 @@
+/*
+ * Copyright (c) 2017 Ericsson India Global Services Pvt Ltd. and others.  All rights reserved.
+ *
+ * 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.ovsdb.hwvtepsouthbound.transact;
+
+import org.opendaylight.yangtools.yang.binding.Identifiable;
+import org.opendaylight.yangtools.yang.binding.InstanceIdentifier;
+
+public class MdsalUpdate<T extends Identifiable> {
+
+    private InstanceIdentifier key;
+    private T newData;
+    private T oldData;
+
+    public MdsalUpdate(InstanceIdentifier key, T newData, T oldData) {
+        this.key = key;
+        this.newData = newData;
+        this.oldData = oldData;
+    }
+
+    public InstanceIdentifier getKey() {
+        return key;
+    }
+
+    public T getNewData() {
+        return newData;
+    }
+
+    public T getOldData() {
+        return oldData;
+    }
+}
\ No newline at end of file
index 27683a853f2c98b428d4f511ff6f17d8ce5d9000..89849d7f81bb859c9162e73ee734acec3c6503c8 100644 (file)
@@ -289,7 +289,7 @@ public class PhysicalSwitchUpdateCommand extends AbstractTransactCommand {
             if (confLocOptional.isPresent()) {
                 HwvtepPhysicalLocatorAugmentation locatorAugmentation =
                                 confLocOptional.get().getAugmentation(HwvtepPhysicalLocatorAugmentation.class);
-                locatorUUID = TransactUtils.createPhysicalLocator(transaction, locatorAugmentation);
+                locatorUUID = TransactUtils.createPhysicalLocator(transaction, getOperationalState(), iid);
             } else {
                 LOG.warn("Unable to find endpoint for tunnel. Endpoint indentifier is {}", iid);
             }
index 65e9be6e95b476157c23577efbb768ae0edb84db..cad1998f8b38adf3199a8f1b6f6fe21f6e455274 100644 (file)
@@ -17,9 +17,17 @@ public interface TransactCommand<T extends Identifiable> {
 
     void execute(TransactionBuilder transaction);
 
-    void onConfigUpdate(TransactionBuilder transaction, InstanceIdentifier<Node> nodeIid, T data,
-                        InstanceIdentifier key, Object... extraData);
+    default void onConfigUpdate(TransactionBuilder transaction, InstanceIdentifier<Node> nodeIid, T data,
+                                InstanceIdentifier key, Object... extraData) {
+    }
 
-    void doDeviceTransaction(TransactionBuilder transaction, InstanceIdentifier<Node> nodeIid, T data,
-                             InstanceIdentifier key, Object... extraData);
+    default void doDeviceTransaction(TransactionBuilder transaction, InstanceIdentifier<Node> nodeIid, T data,
+                                     InstanceIdentifier key, Object... extraData) {
+    }
+
+    default void onSuccess(TransactionBuilder deviceTransaction) {
+    }
+
+    default void onFailure(TransactionBuilder deviceTransaction) {
+    }
 }
index 7f6c937e3d894fce33bb5c5f3c7112db0193ae11..7ce7d38329c639fbaf63acf2f853b84d0ed86cab 100644 (file)
@@ -194,4 +194,15 @@ public class TransactCommandAggregator implements TransactCommand {
         }
         return null;
     }
+
+    @Override
+    public void onFailure(TransactionBuilder deviceTransaction) {
+        commands.forEach( cmd -> cmd.onFailure(deviceTransaction));
+        operationalState.clearIntransitKeys();
+    }
+
+    @Override
+    public void onSuccess(TransactionBuilder deviceTransaction) {
+        commands.forEach( cmd -> cmd.onSuccess(deviceTransaction));
+    }
 }
index 30d74d129bb68b94e46ec0e1b22de68ce8d3cd3f..f3f2917cc7b133ab512e00411744301d31fbe41c 100644 (file)
@@ -56,8 +56,10 @@ public class TransactInvokerImpl implements TransactInvoker {
                     }
                 }
                 if (errorOccured) {
-                    connectionInstance.getDeviceInfo().clearInTransitData();
                     printError(tb);
+                    command.onFailure(tb);
+                } else {
+                    command.onSuccess(tb);
                 }
             } catch (Exception e) {
                 LOG.warn("Transact execution exception: ", e);
index 76cdce1977dc2fb2ec17a6a0c28ff6c85d698457..5b6c855be2f115baccda6b932938aced7d676e0f 100644 (file)
@@ -9,7 +9,11 @@ package org.opendaylight.ovsdb.hwvtepsouthbound.transact;
 
 import static org.opendaylight.ovsdb.lib.operations.Operations.op;
 
+import com.google.common.base.Optional;
+import com.google.common.collect.Lists;
+
 import java.util.Collection;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
@@ -30,6 +34,7 @@ import org.opendaylight.ovsdb.lib.operations.TransactionBuilder;
 import org.opendaylight.ovsdb.lib.schema.typed.TyperUtils;
 import org.opendaylight.ovsdb.schema.hardwarevtep.PhysicalLocator;
 import org.opendaylight.ovsdb.schema.hardwarevtep.PhysicalLocatorSet;
+import org.opendaylight.ovsdb.utils.mdsal.utils.MdsalUtils;
 import org.opendaylight.yang.gen.v1.urn.ietf.params.xml.ns.yang.ietf.inet.types.rev130715.IpAddress;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.ovsdb.hwvtep.rev150901.EncapsulationTypeVxlanOverIpv4;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.ovsdb.hwvtep.rev150901.HwvtepNodeName;
@@ -45,8 +50,6 @@ import org.opendaylight.yangtools.yang.binding.InstanceIdentifier;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.google.common.base.Optional;
-
 public class TransactUtils {
     private static final Logger LOG = LoggerFactory.getLogger(TransactUtils.class);
 
@@ -223,6 +226,32 @@ public class TransactUtils {
                 sanitizeUUID(lswitchIid.firstKeyOf(LogicalSwitches.class).getHwvtepNodeName()));
     }
 
+    public static UUID getLogicalSwitchUUID(final TransactionBuilder transaction,
+                                            final HwvtepOperationalState operationalState,
+                                            final InstanceIdentifier<LogicalSwitches> lswitchIid) {
+        HwvtepDeviceInfo hwvtepDeviceInfo = operationalState.getDeviceInfo();
+        HwvtepDeviceInfo.DeviceData lsData = hwvtepDeviceInfo.getDeviceOperData(LogicalSwitches.class, lswitchIid);
+        if (lsData != null) {
+            if (lsData.getUuid() != null) {
+                return lsData.getUuid();
+            }
+            if (lsData.isInTransitState()) {
+                return getLogicalSwitchUUID(lswitchIid);
+            }
+            return null;
+        }
+        LogicalSwitchUpdateCommand cmd = new LogicalSwitchUpdateCommand(operationalState, Collections.EMPTY_LIST);
+        MdsalUtils mdsalUtils = new MdsalUtils(operationalState.getDataBroker());
+        LogicalSwitches ls = mdsalUtils.read(LogicalDatastoreType.CONFIGURATION, lswitchIid);
+        if (ls != null) {
+            cmd.updateLogicalSwitch(transaction, lswitchIid.firstIdentifierOf(Node.class), Lists.newArrayList(ls));
+        } else {
+            LOG.error("Could not find logical switch in config ds {}", lswitchIid);
+            return null;
+        }
+        return getLogicalSwitchUUID(lswitchIid);
+    }
+
     public static String getLogicalRouterId(final LogicalRouters lrouter){
         return HwvtepSouthboundConstants.LOGICALROUTER_UUID_PREFIX + sanitizeUUID(lrouter.getHwvtepNodeName());
     }
index 2492e7cbb431936cd3b215ff3494c8d66bf9cb38..0aec3f6a742e3a53f14811d656e257956c131cd4 100644 (file)
@@ -22,6 +22,7 @@ import org.opendaylight.ovsdb.lib.operations.TransactionBuilder;
 import org.opendaylight.ovsdb.lib.schema.typed.TyperUtils;
 import org.opendaylight.ovsdb.schema.hardwarevtep.UcastMacsRemote;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.ovsdb.hwvtep.rev150901.HwvtepGlobalAugmentation;
+import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.ovsdb.hwvtep.rev150901.hwvtep.global.attributes.LogicalSwitches;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.ovsdb.hwvtep.rev150901.hwvtep.global.attributes.RemoteUcastMacs;
 import org.opendaylight.yang.gen.v1.urn.tbd.params.xml.ns.yang.network.topology.rev131021.network.topology.topology.Node;
 import org.opendaylight.yangtools.yang.binding.InstanceIdentifier;
@@ -89,12 +90,11 @@ public class UcastMacsRemoteRemoveCommand extends AbstractTransactCommand<Remote
                 transaction.add(op.delete(ucastMacsRemote.getSchema()).
                         where(ucastMacsRemote.getUuidColumn().getSchema().opEqual(macEntryUUID)).build());
                 transaction.add(op.comment("UcastMacRemote: Deleting " + mac.getMacEntryKey().getValue()));
-                getOperationalState().getDeviceInfo().markKeyAsInTransit(RemoteUcastMacs.class, macKey);
+                updateCurrentTxDeleteData(RemoteUcastMacs.class, macKey, mac);
             } else {
                 LOG.warn("Unable to delete remoteUcastMacs {} because it was not found in the operational store",
                         mac.getMacEntryKey().getValue());
             }
-            updateCurrentTxDeleteData(RemoteUcastMacs.class, macIid, mac);
     }
 
     protected List<RemoteUcastMacs> getData(HwvtepGlobalAugmentation augmentation) {
@@ -105,4 +105,14 @@ public class UcastMacsRemoteRemoveCommand extends AbstractTransactCommand<Remote
     protected boolean isRemoveCommand() {
         return true;
     }
+
+    @Override
+    public void onCommandSucceeded() {
+        for (MdsalUpdate mdsalUpdate : updates.get(getDeviceTransaction())) {
+            RemoteUcastMacs deletedMac = (RemoteUcastMacs) mdsalUpdate.getNewData();
+            InstanceIdentifier<RemoteUcastMacs> macIid = mdsalUpdate.getKey();
+            getDeviceInfo().removeRemoteUcast(
+                    (InstanceIdentifier<LogicalSwitches>) deletedMac.getLogicalSwitchRef().getValue(), macIid);
+        }
+    }
 }
index 13665e2b8f5d4cadf08bca999f7e0ed37791a83d..f1c741f4fb37d97b611caa92a409f640af7425d1 100644 (file)
@@ -16,6 +16,7 @@ import org.opendaylight.ovsdb.lib.notation.UUID;
 import org.opendaylight.ovsdb.lib.operations.TransactionBuilder;
 import org.opendaylight.ovsdb.lib.schema.typed.TyperUtils;
 import org.opendaylight.ovsdb.schema.hardwarevtep.UcastMacsRemote;
+import org.opendaylight.ovsdb.utils.mdsal.utils.MdsalUtils;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.ovsdb.hwvtep.rev150901.HwvtepGlobalAugmentation;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.ovsdb.hwvtep.rev150901.hwvtep.global.attributes.LogicalSwitches;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.ovsdb.hwvtep.rev150901.hwvtep.global.attributes.RemoteUcastMacs;
@@ -87,7 +88,7 @@ public class UcastMacsRemoteUpdateCommand extends AbstractTransactCommand<Remote
             UcastMacsRemote ucastMacsRemote = TyperUtils.getTypedRowWrapper(transaction.getDatabaseSchema(), UcastMacsRemote.class);
             setIpAddress(ucastMacsRemote, remoteUcastMac);
             setLocator(transaction, ucastMacsRemote, remoteUcastMac);
-            setLogicalSwitch(ucastMacsRemote, remoteUcastMac);
+            setLogicalSwitch(transaction, ucastMacsRemote, remoteUcastMac);
             if (deviceData == null) {
                 setMac(ucastMacsRemote, remoteUcastMac);
                 LOG.trace("doDeviceTransaction: creating RemotUcastMac entry: {}", ucastMacsRemote);
@@ -108,18 +109,12 @@ public class UcastMacsRemoteUpdateCommand extends AbstractTransactCommand<Remote
             }
     }
 
-    private void setLogicalSwitch(UcastMacsRemote ucastMacsRemote, RemoteUcastMacs inputMac) {
+    private void setLogicalSwitch(final TransactionBuilder transaction, final UcastMacsRemote ucastMacsRemote, final RemoteUcastMacs inputMac) {
         if (inputMac.getLogicalSwitchRef() != null) {
             @SuppressWarnings("unchecked")
             InstanceIdentifier<LogicalSwitches> lswitchIid =
                     (InstanceIdentifier<LogicalSwitches>) inputMac.getLogicalSwitchRef().getValue();
-            HwvtepDeviceInfo.DeviceData deviceData = getOperationalState().getDeviceInfo().getDeviceOperData(
-                    LogicalSwitches.class, lswitchIid);
-            if (deviceData != null && deviceData.getUuid() != null) {
-                ucastMacsRemote.setLogicalSwitch(deviceData.getUuid());
-            } else {
-                ucastMacsRemote.setLogicalSwitch(TransactUtils.getLogicalSwitchUUID(lswitchIid));
-            }
+            ucastMacsRemote.setLogicalSwitch(TransactUtils.getLogicalSwitchUUID(transaction, getOperationalState(), lswitchIid));
         }
     }
 
@@ -168,4 +163,18 @@ public class UcastMacsRemoteUpdateCommand extends AbstractTransactCommand<Remote
             return Collections.singletonList(data.getLocatorRef().getValue());
         }
     }
+
+    @Override
+    public void onCommandSucceeded() {
+        for (MdsalUpdate mdsalUpdate : updates.get(getDeviceTransaction())) {
+            RemoteUcastMacs newMac = (RemoteUcastMacs) mdsalUpdate.getNewData();
+            InstanceIdentifier<RemoteUcastMacs> macIid = mdsalUpdate.getKey();
+            RemoteUcastMacs oldMac = (RemoteUcastMacs) mdsalUpdate.getOldData();
+            if (oldMac != null && !oldMac.equals(newMac)) {
+                getDeviceInfo().decRefCount(macIid, oldMac.getLocatorRef().getValue());
+            }
+            getDeviceInfo().updateRemoteUcast(
+                    (InstanceIdentifier<LogicalSwitches>) newMac.getLogicalSwitchRef().getValue(), macIid, newMac);
+        }
+    }
 }
\ No newline at end of file
index c6d7e4bf5df098472f56365b69b71dd04e3bf87b..8e513e8bb2a1637dfa292991d021948689a2b7ad 100644 (file)
@@ -9,10 +9,14 @@
 package org.opendaylight.ovsdb.hwvtepsouthbound.transactions.md;
 
 import org.opendaylight.ovsdb.hwvtepsouthbound.HwvtepConnectionInstance;
+import org.opendaylight.ovsdb.hwvtepsouthbound.HwvtepDeviceInfo;
 import org.opendaylight.ovsdb.lib.message.TableUpdates;
+import org.opendaylight.ovsdb.lib.notation.UUID;
 import org.opendaylight.ovsdb.lib.schema.DatabaseSchema;
 import org.opendaylight.yang.gen.v1.urn.opendaylight.params.xml.ns.yang.ovsdb.hwvtep.rev150901.hwvtep.global.attributes.ConnectionInfo;
 import org.opendaylight.yangtools.yang.binding.DataObject;
+import org.opendaylight.yangtools.yang.binding.Identifiable;
+import org.opendaylight.yangtools.yang.binding.InstanceIdentifier;
 
 public abstract class AbstractTransactionCommand<T extends DataObject> implements TransactionCommand{
 
@@ -42,4 +46,7 @@ public abstract class AbstractTransactionCommand<T extends DataObject> implement
         this.key = key;
     }
 
+    public HwvtepDeviceInfo getDeviceInfo() {
+        return key.getDeviceInfo();
+    }
 }