[ONOS-5897] (vCore) Create Group Manager and Store

Chages.
1. Virtual Group Manager is added
2. Virtual Group Store interface is added
3. Simple Virtual Group Store is added
4. Virtual Group Manager Test cases are added

Change-Id: I6fc1fae99f5b85fd50437b396f57f78e914ad978
diff --git a/incubator/api/src/main/java/org/onosproject/incubator/net/virtual/VirtualNetworkFlowRuleStore.java b/incubator/api/src/main/java/org/onosproject/incubator/net/virtual/VirtualNetworkFlowRuleStore.java
index 0567083..a5a212a 100644
--- a/incubator/api/src/main/java/org/onosproject/incubator/net/virtual/VirtualNetworkFlowRuleStore.java
+++ b/incubator/api/src/main/java/org/onosproject/incubator/net/virtual/VirtualNetworkFlowRuleStore.java
@@ -27,6 +27,10 @@
 
 import java.util.List;
 
+/**
+ * Manages inventory of flow rules for virtual networks;
+ * not intended for direct use.
+ */
 public interface VirtualNetworkFlowRuleStore
         extends VirtualStore<FlowRuleBatchEvent, FlowRuleStoreDelegate> {
     /**
diff --git a/incubator/api/src/main/java/org/onosproject/incubator/net/virtual/VirtualNetworkGroupStore.java b/incubator/api/src/main/java/org/onosproject/incubator/net/virtual/VirtualNetworkGroupStore.java
new file mode 100644
index 0000000..3cae605
--- /dev/null
+++ b/incubator/api/src/main/java/org/onosproject/incubator/net/virtual/VirtualNetworkGroupStore.java
@@ -0,0 +1,231 @@
+/*
+ * Copyright 2017-present Open Networking Laboratory
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.onosproject.incubator.net.virtual;
+
+import org.onosproject.core.GroupId;
+import org.onosproject.net.DeviceId;
+import org.onosproject.net.group.Group;
+import org.onosproject.net.group.GroupBuckets;
+import org.onosproject.net.group.GroupDescription;
+import org.onosproject.net.group.GroupEvent;
+import org.onosproject.net.group.GroupKey;
+import org.onosproject.net.group.GroupOperation;
+import org.onosproject.net.group.GroupStoreDelegate;
+
+import java.util.Collection;
+
+/**
+ * Manages inventory of groups per virtual network and virtual device;
+ * not intended for direct use.
+ */
+public interface VirtualNetworkGroupStore
+        extends VirtualStore<GroupEvent, GroupStoreDelegate> {
+
+    enum UpdateType {
+        /**
+         * Modify existing group entry by adding provided information.
+         */
+        ADD,
+        /**
+         * Modify existing group by removing provided information from it.
+         */
+        REMOVE,
+        /**
+         * Modify existing group entry by setting the provided information,
+         * overwriting the previous group entry entirely.
+         */
+        SET
+    }
+
+    /**
+     * Returns the number of groups for the specified virtual device in the store.
+     *
+     * @param networkId the virtual network ID
+     * @param deviceId the device ID
+     * @return number of groups for the specified device
+     */
+    int getGroupCount(NetworkId networkId, DeviceId deviceId);
+
+    /**
+     * Returns the groups associated with a virtual device.
+     *
+     * @param networkId the virtual network ID
+     * @param deviceId the device ID
+     * @return the group entries
+     */
+    Iterable<Group> getGroups(NetworkId networkId, DeviceId deviceId);
+
+    /**
+     * Returns the stored group entry in a virtual network.
+     *
+     * @param networkId the virtual network ID
+     * @param deviceId the device ID
+     * @param appCookie the group key
+     * @return a group associated with the key
+     */
+    Group getGroup(NetworkId networkId, DeviceId deviceId, GroupKey appCookie);
+
+    /**
+     * Returns the stored group entry for an id.
+     *
+     * @param networkId the virtual network ID
+     * @param deviceId the device ID
+     * @param groupId the group identifier
+     * @return a group associated with the key
+     */
+    Group getGroup(NetworkId networkId, DeviceId deviceId, GroupId groupId);
+
+    /**
+     * Stores a new group entry using the information from group description
+     * for a virtual network.
+     *
+     * @param networkId the virtual network ID
+     * @param groupDesc group description to be used to store group entry
+     */
+    void storeGroupDescription(NetworkId networkId, GroupDescription groupDesc);
+
+    /**
+     * Updates the existing group entry with the information
+     * from group description.
+     *
+     * @param networkId the virtual network ID
+     * @param deviceId the device ID
+     * @param oldAppCookie the current group key
+     * @param type update type
+     * @param newBuckets group buckets for updates
+     * @param newAppCookie optional new group key
+     */
+    void updateGroupDescription(NetworkId networkId,
+                                DeviceId deviceId,
+                                GroupKey oldAppCookie,
+                                UpdateType type,
+                                GroupBuckets newBuckets,
+                                GroupKey newAppCookie);
+
+    /**
+     * Triggers deleting the existing group entry.
+     *
+     * @param networkId the virtual network ID
+     * @param deviceId the device ID
+     * @param appCookie the group key
+     */
+    void deleteGroupDescription(NetworkId networkId,
+                                DeviceId deviceId,
+                                GroupKey appCookie);
+
+    /**
+     * Stores a new group entry, or updates an existing entry.
+     *
+     * @param networkId the virtual network ID
+     * @param group group entry
+     */
+    void addOrUpdateGroupEntry(NetworkId networkId, Group group);
+
+    /**
+     * Removes the group entry from store.
+     *
+     * @param networkId the virtual network ID
+     * @param group group entry
+     */
+    void removeGroupEntry(NetworkId networkId, Group group);
+
+    /**
+     * Removes all group entries of given device from store.
+     *
+     * @param networkId the virtual network ID
+     * @param deviceId device id
+     */
+    void purgeGroupEntry(NetworkId networkId, DeviceId deviceId);
+
+    /**
+     * Removes all group entries from store.
+     *
+     * @param networkId the virtual network ID
+     */
+    default void purgeGroupEntries(NetworkId networkId) {}
+
+    /**
+     * A group entry that is present in switch but not in the store.
+     *
+     * @param networkId the virtual network ID
+     * @param group group entry
+     */
+    void addOrUpdateExtraneousGroupEntry(NetworkId networkId, Group group);
+
+    /**
+     * Remove the group entry from extraneous database.
+     *
+     * @param networkId the virtual network ID
+     * @param group group entry
+     */
+    void removeExtraneousGroupEntry(NetworkId networkId, Group group);
+
+    /**
+     * Returns the extraneous groups associated with a device.
+     *
+     * @param networkId the virtual network ID
+     * @param deviceId the device ID
+     *
+     * @return the extraneous group entries
+     */
+    Iterable<Group> getExtraneousGroups(NetworkId networkId, DeviceId deviceId);
+
+    /**
+     * Indicates the first group audit is completed.
+     *
+     * @param networkId the virtual network ID
+     * @param deviceId the device ID
+     * @param completed initial audit status
+     */
+    void deviceInitialAuditCompleted(NetworkId networkId, DeviceId deviceId, boolean completed);
+
+    /**
+     * Retrieves the initial group audit status for a device.
+     *
+     * @param networkId the virtual network ID
+     * @param deviceId the device ID
+     *
+     * @return initial group audit status
+     */
+    boolean deviceInitialAuditStatus(NetworkId networkId, DeviceId deviceId);
+
+    /**
+     * Indicates the group operations failed.
+     *
+     * @param networkId the virtual network ID
+     * @param deviceId the device ID
+     * @param operation the group operation failed
+     */
+    void groupOperationFailed(NetworkId networkId, DeviceId deviceId, GroupOperation operation);
+
+    /**
+     * Submits the group metrics to store for a given device ID.
+     *
+     * @param networkId the virtual network ID
+     * @param deviceId the device ID
+     * @param groupEntries the group entries as received from southbound
+     */
+    void pushGroupMetrics(NetworkId networkId, DeviceId deviceId, Collection<Group> groupEntries);
+
+    /**
+     * Indicates failover within a failover group.
+     *
+     * @param networkId the virtual network ID
+     * @param failoverGroups groups to notify
+     */
+    void notifyOfFailovers(NetworkId networkId, Collection<Group> failoverGroups);
+}
diff --git a/incubator/api/src/main/java/org/onosproject/incubator/net/virtual/provider/VirtualGroupProviderService.java b/incubator/api/src/main/java/org/onosproject/incubator/net/virtual/provider/VirtualGroupProviderService.java
index 674bc2e..a6ed87a 100644
--- a/incubator/api/src/main/java/org/onosproject/incubator/net/virtual/provider/VirtualGroupProviderService.java
+++ b/incubator/api/src/main/java/org/onosproject/incubator/net/virtual/provider/VirtualGroupProviderService.java
@@ -16,7 +16,6 @@
 
 package org.onosproject.incubator.net.virtual.provider;
 
-import org.onosproject.incubator.net.virtual.NetworkId;
 import org.onosproject.net.DeviceId;
 import org.onosproject.net.group.Group;
 import org.onosproject.net.group.GroupOperation;
@@ -32,29 +31,24 @@
     /**
      * Notifies core if any failure from data plane during group operations.
      *
-     * @param networkId the identity of the virtual network where this rule applies
      * @param deviceId the device ID
      * @param operation offended group operation
      */
-    void groupOperationFailed(NetworkId networkId, DeviceId deviceId,
-                              GroupOperation operation);
+    void groupOperationFailed(DeviceId deviceId, GroupOperation operation);
 
     /**
      * Pushes the collection of group detected in the data plane along
      * with statistics.
      *
-     * @param networkId the identity of the virtual network where this rule applies
      * @param deviceId device identifier
      * @param groupEntries collection of group entries as seen in data plane
      */
-    void pushGroupMetrics(NetworkId networkId,
-                          DeviceId deviceId, Collection<Group> groupEntries);
+    void pushGroupMetrics(DeviceId deviceId, Collection<Group> groupEntries);
 
     /**
      * Notifies store of group failovers.
      *
-     * @param networkId the identity of the virtual network where this rule applies
      * @param failoverGroups failover groups in which a failover has occurred
      */
-    void notifyOfFailovers(NetworkId networkId, Collection<Group> failoverGroups);
+    void notifyOfFailovers(Collection<Group> failoverGroups);
 }
diff --git a/incubator/net/src/main/java/org/onosproject/incubator/net/virtual/impl/VirtualNetworkGroupManager.java b/incubator/net/src/main/java/org/onosproject/incubator/net/virtual/impl/VirtualNetworkGroupManager.java
new file mode 100644
index 0000000..fee36bc
--- /dev/null
+++ b/incubator/net/src/main/java/org/onosproject/incubator/net/virtual/impl/VirtualNetworkGroupManager.java
@@ -0,0 +1,273 @@
+/*
+ * Copyright 2017-present Open Networking Laboratory
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.onosproject.incubator.net.virtual.impl;
+
+import org.onosproject.core.ApplicationId;
+import org.onosproject.incubator.net.virtual.NetworkId;
+import org.onosproject.incubator.net.virtual.VirtualNetworkGroupStore;
+import org.onosproject.incubator.net.virtual.VirtualNetworkService;
+import org.onosproject.incubator.net.virtual.event.AbstractVirtualListenerManager;
+import org.onosproject.incubator.net.virtual.provider.AbstractVirtualProviderService;
+import org.onosproject.incubator.net.virtual.provider.VirtualGroupProvider;
+import org.onosproject.incubator.net.virtual.provider.VirtualGroupProviderService;
+import org.onosproject.incubator.net.virtual.provider.VirtualProviderRegistryService;
+import org.onosproject.net.DeviceId;
+import org.onosproject.net.device.DeviceEvent;
+import org.onosproject.net.device.DeviceListener;
+import org.onosproject.net.device.DeviceService;
+import org.onosproject.net.group.Group;
+import org.onosproject.net.group.GroupBuckets;
+import org.onosproject.net.group.GroupDescription;
+import org.onosproject.net.group.GroupEvent;
+import org.onosproject.net.group.GroupKey;
+import org.onosproject.net.group.GroupListener;
+import org.onosproject.net.group.GroupOperation;
+import org.onosproject.net.group.GroupOperations;
+import org.onosproject.net.group.GroupService;
+import org.onosproject.net.group.GroupStoreDelegate;
+import org.onosproject.net.provider.ProviderId;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Set;
+
+/**
+ * Group service implementation built on the virtual network service.
+ */
+public class VirtualNetworkGroupManager
+        extends AbstractVirtualListenerManager<GroupEvent, GroupListener>
+        implements GroupService {
+
+    private final Logger log = LoggerFactory.getLogger(getClass());
+
+    private final VirtualNetworkGroupStore store;
+
+    private VirtualProviderRegistryService providerRegistryService = null;
+    private VirtualGroupProviderService innerProviderService;
+    private InternalStoreDelegate storeDelegate;
+    private DeviceService deviceService;
+
+    //TODO: make this configurable
+    private boolean purgeOnDisconnection = false;
+
+    public VirtualNetworkGroupManager(VirtualNetworkService manager, NetworkId networkId) {
+        super(manager, networkId);
+
+        store = serviceDirectory.get(VirtualNetworkGroupStore.class);
+        deviceService = manager.get(networkId, DeviceService.class);
+
+        providerRegistryService =
+                serviceDirectory.get(VirtualProviderRegistryService.class);
+        innerProviderService = new InternalGroupProviderService();
+        providerRegistryService.registerProviderService(networkId(), innerProviderService);
+
+        this.storeDelegate = new InternalStoreDelegate();
+        store.setDelegate(networkId, this.storeDelegate);
+
+        log.info("Started");
+    }
+
+    @Override
+    public void addGroup(GroupDescription groupDesc) {
+        store.storeGroupDescription(networkId(), groupDesc);
+    }
+
+    @Override
+    public Group getGroup(DeviceId deviceId, GroupKey appCookie) {
+        return store.getGroup(networkId(), deviceId, appCookie);
+    }
+
+    @Override
+    public void addBucketsToGroup(DeviceId deviceId, GroupKey oldCookie, GroupBuckets buckets,
+                                  GroupKey newCookie, ApplicationId appId) {
+        store.updateGroupDescription(networkId(),
+                                     deviceId,
+                                     oldCookie,
+                                     VirtualNetworkGroupStore.UpdateType.ADD,
+                                     buckets,
+                                     newCookie);
+    }
+
+    @Override
+    public void removeBucketsFromGroup(DeviceId deviceId, GroupKey oldCookie,
+                                       GroupBuckets buckets, GroupKey newCookie,
+                                       ApplicationId appId) {
+        store.updateGroupDescription(networkId(),
+                                     deviceId,
+                                     oldCookie,
+                                     VirtualNetworkGroupStore.UpdateType.REMOVE,
+                                     buckets,
+                                     newCookie);
+
+    }
+
+    @Override
+    public void setBucketsForGroup(DeviceId deviceId,
+                                   GroupKey oldCookie,
+                                   GroupBuckets buckets,
+                                   GroupKey newCookie,
+                                   ApplicationId appId) {
+        store.updateGroupDescription(networkId(),
+                                     deviceId,
+                                     oldCookie,
+                                     VirtualNetworkGroupStore.UpdateType.SET,
+                                     buckets,
+                                     newCookie);
+    }
+
+    @Override
+    public void purgeGroupEntries(DeviceId deviceId) {
+        store.purgeGroupEntry(networkId(), deviceId);
+    }
+
+    @Override
+    public void purgeGroupEntries() {
+        store.purgeGroupEntries(networkId());
+    }
+
+    @Override
+    public void removeGroup(DeviceId deviceId, GroupKey appCookie, ApplicationId appId) {
+        store.deleteGroupDescription(networkId(), deviceId, appCookie);
+    }
+
+    @Override
+    public Iterable<Group> getGroups(DeviceId deviceId, ApplicationId appId) {
+        return store.getGroups(networkId(), deviceId);
+    }
+
+    @Override
+    public Iterable<Group> getGroups(DeviceId deviceId) {
+        return store.getGroups(networkId(), deviceId);
+    }
+
+    private class InternalGroupProviderService
+            extends AbstractVirtualProviderService<VirtualGroupProvider>
+            implements VirtualGroupProviderService {
+
+        protected InternalGroupProviderService() {
+            Set<ProviderId> providerIds =
+                    providerRegistryService.getProvidersByService(this);
+            ProviderId providerId = providerIds.stream().findFirst().get();
+            VirtualGroupProvider provider = (VirtualGroupProvider)
+                    providerRegistryService.getProvider(providerId);
+            setProvider(provider);
+        }
+
+        @Override
+        public void groupOperationFailed(DeviceId deviceId,
+                                         GroupOperation operation) {
+            store.groupOperationFailed(networkId(), deviceId, operation);
+        }
+
+        @Override
+        public void pushGroupMetrics(DeviceId deviceId, Collection<Group> groupEntries) {
+            log.trace("Received group metrics from device {}", deviceId);
+            checkValidity();
+            store.pushGroupMetrics(networkId(), deviceId, groupEntries);
+        }
+
+        @Override
+        public void notifyOfFailovers(Collection<Group> failoverGroups) {
+            store.notifyOfFailovers(networkId(), failoverGroups);
+        }
+    }
+
+    private class InternalStoreDelegate implements GroupStoreDelegate {
+        @Override
+        public void notify(GroupEvent event) {
+            final Group group = event.subject();
+            VirtualGroupProvider groupProvider = innerProviderService.provider();
+            GroupOperations groupOps = null;
+            switch (event.type()) {
+                case GROUP_ADD_REQUESTED:
+                    log.debug("GROUP_ADD_REQUESTED for Group {} on device {}",
+                              group.id(), group.deviceId());
+                    GroupOperation groupAddOp = GroupOperation.
+                            createAddGroupOperation(group.id(),
+                                                    group.type(),
+                                                    group.buckets());
+                    groupOps = new GroupOperations(
+                            Collections.singletonList(groupAddOp));
+                    groupProvider.performGroupOperation(networkId(), group.deviceId(),
+                                                        groupOps);
+                    break;
+
+                case GROUP_UPDATE_REQUESTED:
+                    log.debug("GROUP_UPDATE_REQUESTED for Group {} on device {}",
+                              group.id(), group.deviceId());
+                    GroupOperation groupModifyOp = GroupOperation.
+                            createModifyGroupOperation(group.id(),
+                                                       group.type(),
+                                                       group.buckets());
+                    groupOps = new GroupOperations(
+                            Collections.singletonList(groupModifyOp));
+                    groupProvider.performGroupOperation(networkId(), group.deviceId(),
+                                                        groupOps);
+                    break;
+
+                case GROUP_REMOVE_REQUESTED:
+                    log.debug("GROUP_REMOVE_REQUESTED for Group {} on device {}",
+                              group.id(), group.deviceId());
+                    GroupOperation groupDeleteOp = GroupOperation.
+                            createDeleteGroupOperation(group.id(),
+                                                       group.type());
+                    groupOps = new GroupOperations(
+                            Collections.singletonList(groupDeleteOp));
+                    groupProvider.performGroupOperation(networkId(), group.deviceId(),
+                                                        groupOps);
+                    break;
+
+                case GROUP_ADDED:
+                case GROUP_UPDATED:
+                case GROUP_REMOVED:
+                case GROUP_ADD_FAILED:
+                case GROUP_UPDATE_FAILED:
+                case GROUP_REMOVE_FAILED:
+                case GROUP_BUCKET_FAILOVER:
+                    post(event);
+                    break;
+                default:
+                    break;
+            }
+        }
+    }
+
+    private class InternalDeviceListener implements DeviceListener {
+        @Override
+        public void event(DeviceEvent event) {
+            switch (event.type()) {
+                case DEVICE_REMOVED:
+                case DEVICE_AVAILABILITY_CHANGED:
+                    DeviceId deviceId = event.subject().id();
+                    if (!deviceService.isAvailable(deviceId)) {
+                        log.debug("Device {} became un available; clearing initial audit status",
+                                  event.type(), event.subject().id());
+                        store.deviceInitialAuditCompleted(networkId(), event.subject().id(), false);
+
+                        if (purgeOnDisconnection) {
+                            store.purgeGroupEntry(networkId(), deviceId);
+                        }
+                    }
+                    break;
+                default:
+                    break;
+            }
+        }
+    }
+}
diff --git a/incubator/net/src/main/java/org/onosproject/incubator/net/virtual/impl/VirtualNetworkManager.java b/incubator/net/src/main/java/org/onosproject/incubator/net/virtual/impl/VirtualNetworkManager.java
index 84589c8..e9ce122 100644
--- a/incubator/net/src/main/java/org/onosproject/incubator/net/virtual/impl/VirtualNetworkManager.java
+++ b/incubator/net/src/main/java/org/onosproject/incubator/net/virtual/impl/VirtualNetworkManager.java
@@ -57,6 +57,7 @@
 import org.onosproject.net.PortNumber;
 import org.onosproject.net.device.DeviceService;
 import org.onosproject.net.flow.FlowRuleService;
+import org.onosproject.net.group.GroupService;
 import org.onosproject.net.host.HostService;
 import org.onosproject.net.intent.IntentEvent;
 import org.onosproject.net.intent.IntentListener;
@@ -421,6 +422,8 @@
             service = new VirtualNetworkFlowRuleManager(this, network.id());
         } else if (serviceKey.serviceClass.equals(PacketService.class)) {
             service = new VirtualNetworkPacketManager(this, network.id());
+        } else if (serviceKey.serviceClass.equals(GroupService.class)) {
+            service = new VirtualNetworkGroupManager(this, network.id());
         } else {
             return null;
         }
diff --git a/incubator/net/src/test/java/org/onosproject/incubator/net/virtual/impl/VirtualNetworkGroupManagerTest.java b/incubator/net/src/test/java/org/onosproject/incubator/net/virtual/impl/VirtualNetworkGroupManagerTest.java
new file mode 100644
index 0000000..8366fee
--- /dev/null
+++ b/incubator/net/src/test/java/org/onosproject/incubator/net/virtual/impl/VirtualNetworkGroupManagerTest.java
@@ -0,0 +1,712 @@
+/*
+ * Copyright 2017-present Open Networking Laboratory
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.onosproject.incubator.net.virtual.impl;
+
+import com.google.common.collect.Iterables;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.onlab.junit.TestUtils;
+import org.onlab.osgi.ServiceDirectory;
+import org.onlab.osgi.TestServiceDirectory;
+import org.onlab.packet.MacAddress;
+import org.onlab.packet.MplsLabel;
+import org.onosproject.TestApplicationId;
+import org.onosproject.common.event.impl.TestEventDispatcher;
+import org.onosproject.core.ApplicationId;
+import org.onosproject.core.CoreService;
+import org.onosproject.core.DefaultGroupId;
+import org.onosproject.core.GroupId;
+import org.onosproject.event.EventDeliveryService;
+import org.onosproject.incubator.net.virtual.NetworkId;
+import org.onosproject.incubator.net.virtual.VirtualNetwork;
+import org.onosproject.incubator.net.virtual.VirtualNetworkGroupStore;
+import org.onosproject.incubator.net.virtual.VirtualNetworkStore;
+import org.onosproject.incubator.net.virtual.event.VirtualEvent;
+import org.onosproject.incubator.net.virtual.event.VirtualListenerRegistryManager;
+import org.onosproject.incubator.net.virtual.impl.provider.VirtualProviderManager;
+import org.onosproject.incubator.net.virtual.provider.AbstractVirtualProvider;
+import org.onosproject.incubator.net.virtual.provider.VirtualGroupProvider;
+import org.onosproject.incubator.net.virtual.provider.VirtualGroupProviderService;
+import org.onosproject.incubator.net.virtual.provider.VirtualProviderRegistryService;
+import org.onosproject.incubator.store.virtual.impl.DistributedVirtualNetworkStore;
+import org.onosproject.incubator.store.virtual.impl.SimpleVirtualGroupStore;
+import org.onosproject.net.DeviceId;
+import org.onosproject.net.PortNumber;
+import org.onosproject.net.flow.DefaultTrafficTreatment;
+import org.onosproject.net.flow.TrafficTreatment;
+import org.onosproject.net.group.DefaultGroup;
+import org.onosproject.net.group.DefaultGroupBucket;
+import org.onosproject.net.group.DefaultGroupDescription;
+import org.onosproject.net.group.DefaultGroupKey;
+import org.onosproject.net.group.Group;
+import org.onosproject.net.group.GroupBucket;
+import org.onosproject.net.group.GroupBuckets;
+import org.onosproject.net.group.GroupDescription;
+import org.onosproject.net.group.GroupEvent;
+import org.onosproject.net.group.GroupKey;
+import org.onosproject.net.group.GroupListener;
+import org.onosproject.net.group.GroupOperation;
+import org.onosproject.net.group.GroupOperations;
+import org.onosproject.net.group.StoredGroupEntry;
+import org.onosproject.net.intent.FakeIntentManager;
+import org.onosproject.net.intent.TestableIntentService;
+import org.onosproject.net.provider.ProviderId;
+import org.onosproject.store.service.TestStorageService;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+
+import static org.junit.Assert.*;
+import static org.onosproject.incubator.net.virtual.impl.VirtualNetworkTestUtil.*;
+import static org.onosproject.net.NetTestTools.injectEventDispatcher;
+
+/**
+ * Test codifying the virtual group service & group provider service contracts.
+ */
+public class VirtualNetworkGroupManagerTest {
+
+    private VirtualNetworkManager manager;
+    private DistributedVirtualNetworkStore virtualNetworkManagerStore;
+    private TestableIntentService intentService = new FakeIntentManager();
+    private ServiceDirectory testDirectory;
+    private VirtualProviderManager providerRegistryService;
+
+    private EventDeliveryService eventDeliveryService;
+    VirtualListenerRegistryManager listenerRegistryManager =
+            VirtualListenerRegistryManager.getInstance();
+
+    private VirtualNetworkGroupManager groupManager1;
+    private VirtualNetworkGroupManager groupManager2;
+
+    private VirtualNetworkGroupStore groupStore;
+
+    private TestGroupProvider provider = new TestGroupProvider();
+    private VirtualGroupProviderService providerService1;
+    private VirtualGroupProviderService providerService2;
+
+    protected TestGroupListener listener1 = new TestGroupListener();
+    protected TestGroupListener listener2 = new TestGroupListener();
+
+    private VirtualNetwork vnet1;
+    private VirtualNetwork vnet2;
+
+    private ApplicationId appId;
+
+    @Before
+    public void setUp() throws Exception {
+        virtualNetworkManagerStore = new DistributedVirtualNetworkStore();
+
+        CoreService coreService = new TestCoreService();
+        TestUtils.setField(virtualNetworkManagerStore, "coreService", coreService);
+        TestUtils.setField(virtualNetworkManagerStore, "storageService", new TestStorageService());
+        virtualNetworkManagerStore.activate();
+
+        groupStore = new SimpleVirtualGroupStore();
+
+        providerRegistryService = new VirtualProviderManager();
+        providerRegistryService.registerProvider(provider);
+
+        manager = new VirtualNetworkManager();
+        manager.store = virtualNetworkManagerStore;
+        manager.intentService = intentService;
+        TestUtils.setField(manager, "coreService", coreService);
+
+        eventDeliveryService = new TestEventDispatcher();
+        injectEventDispatcher(manager, eventDeliveryService);
+        eventDeliveryService.addSink(VirtualEvent.class, listenerRegistryManager);
+
+        appId = new TestApplicationId("VirtualGroupManagerTest");
+
+        testDirectory = new TestServiceDirectory()
+                .add(VirtualNetworkStore.class, virtualNetworkManagerStore)
+                .add(CoreService.class, coreService)
+                .add(VirtualProviderRegistryService.class, providerRegistryService)
+                .add(EventDeliveryService.class, eventDeliveryService)
+                .add(VirtualNetworkGroupStore.class, groupStore);
+        TestUtils.setField(manager, "serviceDirectory", testDirectory);
+
+        manager.activate();
+
+        vnet1 = setupVirtualNetworkTopology(manager, TID1);
+        vnet2 = setupVirtualNetworkTopology(manager, TID2);
+
+        groupManager1 = new VirtualNetworkGroupManager(manager, vnet1.id());
+        groupManager2 = new VirtualNetworkGroupManager(manager, vnet2.id());
+        groupManager1.addListener(listener1);
+        groupManager2.addListener(listener2);
+
+        providerService1 = (VirtualGroupProviderService)
+                providerRegistryService.getProviderService(vnet1.id(),
+                                                           VirtualGroupProvider.class);
+        providerService2 = (VirtualGroupProviderService)
+                providerRegistryService.getProviderService(vnet2.id(),
+                                                           VirtualGroupProvider.class);
+    }
+
+    @After
+    public void tearDown() {
+        providerRegistryService.unregisterProvider(provider);
+        assertFalse("provider should not be registered",
+                    providerRegistryService.getProviders().contains(provider.id()));
+        groupManager1.removeListener(listener1);
+        groupManager2.removeListener(listener2);
+
+        manager.deactivate();
+        virtualNetworkManagerStore.deactivate();
+    }
+
+    /**
+     * Tests group creation before the device group AUDIT completes.
+     */
+    @Test
+    public void testGroupServiceBasics() {
+        // Test Group creation before AUDIT process
+        testGroupCreationBeforeAudit(vnet1.id(), VDID1);
+        testGroupCreationBeforeAudit(vnet2.id(), VDID1);
+    }
+
+    /**
+     * Tests initial device group AUDIT process.
+     */
+    @Test
+    public void testGroupServiceInitialAudit() {
+        // Test Group creation before AUDIT process
+        testGroupCreationBeforeAudit(vnet1.id(), VDID1);
+        testGroupCreationBeforeAudit(vnet2.id(), VDID1);
+        // Test initial group audit process
+        testInitialAuditWithPendingGroupRequests(vnet1.id(), VDID1);
+        testInitialAuditWithPendingGroupRequests(vnet2.id(), VDID1);
+    }
+
+    /**
+     * Tests deletion process of any extraneous groups.
+     */
+    @Test
+    public void testGroupServiceAuditExtraneous() {
+        // Test Group creation before AUDIT process
+        testGroupCreationBeforeAudit(vnet1.id(), VDID1);
+        testGroupCreationBeforeAudit(vnet2.id(), VDID1);
+
+        // Test audit with extraneous and missing groups
+        testAuditWithExtraneousMissingGroups(vnet1.id(), VDID1);
+        testAuditWithExtraneousMissingGroups(vnet2.id(), VDID1);
+    }
+
+    /**
+     * Tests re-apply process of any missing groups tests execution of
+     * any pending group creation request after the device group AUDIT completes
+     * and tests event notifications after receiving confirmation for any
+     * operations from data plane.
+     */
+    @Test
+    public void testGroupServiceAuditConfirmed() {
+        // Test Group creation before AUDIT process
+        testGroupCreationBeforeAudit(vnet1.id(), VDID1);
+        testGroupCreationBeforeAudit(vnet2.id(), VDID1);
+
+        // Test audit with extraneous and missing groups
+        testAuditWithExtraneousMissingGroups(vnet1.id(), VDID1);
+        testAuditWithExtraneousMissingGroups(vnet2.id(), VDID1);
+
+        // Test audit with confirmed groups
+        testAuditWithConfirmedGroups(vnet1.id(), VDID1);
+        testAuditWithConfirmedGroups(vnet2.id(), VDID1);
+    }
+
+    /**
+     * Tests group Purge Operation.
+     */
+    @Test
+    public void testPurgeGroups() {
+        // Tests for virtual network 1
+        // Test Group creation before AUDIT process
+        testGroupCreationBeforeAudit(vnet1.id(), VDID1);
+        testAuditWithExtraneousMissingGroups(vnet1.id(), VDID1);
+        // Test group add bucket operations
+        testAddBuckets(vnet1.id(), VDID1);
+        // Test group Purge operations
+        testPurgeGroupEntry(vnet1.id(), VDID1);
+
+        // Tests for virtual network 2
+        // Test Group creation before AUDIT process
+        testGroupCreationBeforeAudit(vnet2.id(), VDID1);
+        testAuditWithExtraneousMissingGroups(vnet2.id(), VDID1);
+        // Test group add bucket operations
+        testAddBuckets(vnet2.id(), VDID1);
+        // Test group Purge operations
+        testPurgeGroupEntry(vnet2.id(), VDID1);
+    }
+
+    /**
+     * Tests group bucket modifications (additions and deletions) and
+     * Tests group deletion.
+     */
+    @Test
+    public void testGroupServiceBuckets() {
+        // Tests for virtual network 1
+        // Test Group creation before AUDIT process
+        testGroupCreationBeforeAudit(vnet1.id(), VDID1);
+
+        testAuditWithExtraneousMissingGroups(vnet1.id(), VDID1);
+        // Test group add bucket operations
+        testAddBuckets(vnet1.id(), VDID1);
+
+        // Test group remove bucket operations
+        testRemoveBuckets(vnet1.id(), VDID1);
+
+        // Test group remove operations
+        testRemoveGroup(vnet1.id(), VDID1);
+
+        // Tests for virtual network 2
+        // Test Group creation before AUDIT process
+        testGroupCreationBeforeAudit(vnet2.id(), VDID1);
+
+        testAuditWithExtraneousMissingGroups(vnet2.id(), VDID1);
+        // Test group add bucket operations
+        testAddBuckets(vnet2.id(), VDID1);
+
+        // Test group remove bucket operations
+        testRemoveBuckets(vnet2.id(), VDID1);
+
+        // Test group remove operations
+        testRemoveGroup(vnet2.id(), VDID1);
+    }
+
+    /**
+     * Tests group creation before the device group AUDIT completes with fallback
+     * provider.
+     */
+    @Test
+    public void testGroupServiceFallbackBasics() {
+        // Test Group creation before AUDIT process
+        testGroupCreationBeforeAudit(vnet1.id(), VDID2);
+        testGroupCreationBeforeAudit(vnet2.id(), VDID2);
+    }
+
+    // Test Group creation before AUDIT process
+    private void testGroupCreationBeforeAudit(NetworkId networkId, DeviceId deviceId) {
+        PortNumber[] ports1 = {PortNumber.portNumber(31),
+                PortNumber.portNumber(32)};
+        PortNumber[] ports2 = {PortNumber.portNumber(41),
+                PortNumber.portNumber(42)};
+        GroupKey key = new DefaultGroupKey("group1BeforeAudit".getBytes());
+        List<GroupBucket> buckets = new ArrayList<>();
+        List<PortNumber> outPorts = new ArrayList<>();
+        outPorts.addAll(Arrays.asList(ports1));
+        outPorts.addAll(Arrays.asList(ports2));
+        for (PortNumber portNumber : outPorts) {
+            TrafficTreatment.Builder tBuilder = DefaultTrafficTreatment.builder();
+            tBuilder.setOutput(portNumber)
+                    .setEthDst(MacAddress.valueOf("00:00:00:00:00:02"))
+                    .setEthSrc(MacAddress.valueOf("00:00:00:00:00:01"))
+                    .pushMpls()
+                    .setMpls(MplsLabel.mplsLabel(106));
+            buckets.add(DefaultGroupBucket.createSelectGroupBucket(
+                    tBuilder.build()));
+        }
+        GroupBuckets groupBuckets = new GroupBuckets(buckets);
+        GroupDescription newGroupDesc = new DefaultGroupDescription(deviceId,
+                                                                    Group.Type.SELECT,
+                                                                    groupBuckets,
+                                                                    key,
+                                                                    null,
+                                                                    appId);
+        VirtualNetworkGroupManager groupManager;
+        if (networkId.id() == 1) {
+            groupManager = groupManager1;
+        } else {
+            groupManager = groupManager2;
+        }
+
+        groupManager.addGroup(newGroupDesc);
+        assertEquals(null, groupManager.getGroup(deviceId, key));
+        assertEquals(0, Iterables.size(groupManager.getGroups(deviceId, appId)));
+    }
+
+
+    // Test initial AUDIT process with pending group requests
+    private void testInitialAuditWithPendingGroupRequests(NetworkId networkId,
+                                                          DeviceId deviceId) {
+        VirtualNetworkGroupManager groupManager;
+        VirtualGroupProviderService providerService;
+        if (networkId.id() == 1) {
+            groupManager = groupManager1;
+            providerService = providerService1;
+        } else {
+            groupManager = groupManager2;
+            providerService = providerService2;
+        }
+
+        PortNumber[] ports1 = {PortNumber.portNumber(31),
+                PortNumber.portNumber(32)};
+        PortNumber[] ports2 = {PortNumber.portNumber(41),
+                PortNumber.portNumber(42)};
+        GroupId gId1 = new DefaultGroupId(1);
+        Group group1 = createSouthboundGroupEntry(gId1,
+                                                  Arrays.asList(ports1),
+                                                  0, deviceId);
+        GroupId gId2 = new DefaultGroupId(2);
+        // Non zero reference count will make the group manager to queue
+        // the extraneous groups until reference count is zero.
+        Group group2 = createSouthboundGroupEntry(gId2,
+                                                  Arrays.asList(ports2),
+                                                  2, deviceId);
+        List<Group> groupEntries = Arrays.asList(group1, group2);
+        providerService.pushGroupMetrics(deviceId, groupEntries);
+        // First group metrics would trigger the device audit completion
+        // post which all pending group requests are also executed.
+        GroupKey key = new DefaultGroupKey("group1BeforeAudit".getBytes());
+        Group createdGroup = groupManager.getGroup(deviceId, key);
+        int createdGroupId = createdGroup.id().id();
+        assertNotEquals(gId1.id().intValue(), createdGroupId);
+        assertNotEquals(gId2.id().intValue(), createdGroupId);
+
+        List<GroupOperation> expectedGroupOps = Arrays.asList(
+                GroupOperation.createDeleteGroupOperation(gId1,
+                                                          Group.Type.SELECT),
+                GroupOperation.createAddGroupOperation(
+                        createdGroup.id(),
+                        Group.Type.SELECT,
+                        createdGroup.buckets()));
+        if (deviceId.equals(VDID1)) {
+            provider.validate(networkId, deviceId, expectedGroupOps);
+        }
+    }
+
+    // Test AUDIT process with extraneous groups and missing groups
+    private void testAuditWithExtraneousMissingGroups(NetworkId networkId,
+                                                      DeviceId deviceId) {
+        VirtualNetworkGroupManager groupManager;
+        VirtualGroupProviderService providerService;
+        if (networkId.id() == 1) {
+            groupManager = groupManager1;
+            providerService = providerService1;
+        } else {
+            groupManager = groupManager2;
+            providerService = providerService2;
+        }
+
+        PortNumber[] ports1 = {PortNumber.portNumber(31),
+                PortNumber.portNumber(32)};
+        PortNumber[] ports2 = {PortNumber.portNumber(41),
+                PortNumber.portNumber(42)};
+        GroupId gId1 = new DefaultGroupId(1);
+        Group group1 = createSouthboundGroupEntry(gId1,
+                                                  Arrays.asList(ports1),
+                                                  0, deviceId);
+        GroupId gId2 = new DefaultGroupId(2);
+        Group group2 = createSouthboundGroupEntry(gId2,
+                                                  Arrays.asList(ports2),
+                                                  0, deviceId);
+        List<Group> groupEntries = Arrays.asList(group1, group2);
+        providerService.pushGroupMetrics(deviceId, groupEntries);
+        GroupKey key = new DefaultGroupKey("group1BeforeAudit".getBytes());
+        Group createdGroup = groupManager.getGroup(deviceId, key);
+        List<GroupOperation> expectedGroupOps = Arrays.asList(
+                GroupOperation.createDeleteGroupOperation(gId1,
+                                                          Group.Type.SELECT),
+                GroupOperation.createDeleteGroupOperation(gId2,
+                                                          Group.Type.SELECT),
+                GroupOperation.createAddGroupOperation(createdGroup.id(),
+                                                       Group.Type.SELECT,
+                                                       createdGroup.buckets()));
+        if (deviceId.equals(VDID1)) {
+            provider.validate(networkId, deviceId, expectedGroupOps);
+        }
+    }
+
+    // Test AUDIT with confirmed groups
+    private void testAuditWithConfirmedGroups(NetworkId networkId,
+                                              DeviceId deviceId) {
+        VirtualNetworkGroupManager groupManager;
+        VirtualGroupProviderService providerService;
+        TestGroupListener listener;
+
+        if (networkId.id() == 1) {
+            groupManager = groupManager1;
+            providerService = providerService1;
+            listener = listener1;
+        } else {
+            groupManager = groupManager2;
+            providerService = providerService2;
+            listener = listener2;
+        }
+
+        GroupKey key = new DefaultGroupKey("group1BeforeAudit".getBytes());
+        Group createdGroup = groupManager.getGroup(deviceId, key);
+        createdGroup = new DefaultGroup(createdGroup.id(),
+                                        deviceId,
+                                        Group.Type.SELECT,
+                                        createdGroup.buckets());
+        List<Group> groupEntries = Collections.singletonList(createdGroup);
+        providerService.pushGroupMetrics(deviceId, groupEntries);
+        listener.validateEvent(Collections.singletonList(GroupEvent.Type.GROUP_ADDED));
+    }
+
+    private Group createSouthboundGroupEntry(GroupId gId,
+                                             List<PortNumber> ports,
+                                             long referenceCount, DeviceId deviceId) {
+        List<PortNumber> outPorts = new ArrayList<>();
+        outPorts.addAll(ports);
+
+        List<GroupBucket> buckets = new ArrayList<>();
+        for (PortNumber portNumber : outPorts) {
+            TrafficTreatment.Builder tBuilder = DefaultTrafficTreatment.builder();
+            tBuilder.setOutput(portNumber)
+                    .setEthDst(MacAddress.valueOf("00:00:00:00:00:02"))
+                    .setEthSrc(MacAddress.valueOf("00:00:00:00:00:01"))
+                    .pushMpls()
+                    .setMpls(MplsLabel.mplsLabel(106));
+            buckets.add(DefaultGroupBucket.createSelectGroupBucket(
+                    tBuilder.build()));
+        }
+        GroupBuckets groupBuckets = new GroupBuckets(buckets);
+        StoredGroupEntry group = new DefaultGroup(
+                gId, deviceId, Group.Type.SELECT, groupBuckets);
+        group.setReferenceCount(referenceCount);
+        return group;
+    }
+
+    // Test group add bucket operations
+    private void testAddBuckets(NetworkId networkId, DeviceId deviceId) {
+        VirtualNetworkGroupManager groupManager;
+        VirtualGroupProviderService providerService;
+        TestGroupListener listener;
+
+        if (networkId.id() == 1) {
+            groupManager = groupManager1;
+            providerService = providerService1;
+            listener = listener1;
+        } else {
+            groupManager = groupManager2;
+            providerService = providerService2;
+            listener = listener2;
+        }
+
+        GroupKey addKey = new DefaultGroupKey("group1AddBuckets".getBytes());
+
+        GroupKey prevKey = new DefaultGroupKey("group1BeforeAudit".getBytes());
+        Group createdGroup = groupManager.getGroup(deviceId, prevKey);
+        List<GroupBucket> buckets = new ArrayList<>();
+        buckets.addAll(createdGroup.buckets().buckets());
+
+        PortNumber[] addPorts = {PortNumber.portNumber(51),
+                PortNumber.portNumber(52)};
+        List<PortNumber> outPorts;
+        outPorts = new ArrayList<>();
+        outPorts.addAll(Arrays.asList(addPorts));
+        List<GroupBucket> addBuckets;
+        addBuckets = new ArrayList<>();
+        for (PortNumber portNumber : outPorts) {
+            TrafficTreatment.Builder tBuilder = DefaultTrafficTreatment.builder();
+            tBuilder.setOutput(portNumber)
+                    .setEthDst(MacAddress.valueOf("00:00:00:00:00:02"))
+                    .setEthSrc(MacAddress.valueOf("00:00:00:00:00:01"))
+                    .pushMpls()
+                    .setMpls(MplsLabel.mplsLabel(106));
+            addBuckets.add(DefaultGroupBucket.createSelectGroupBucket(
+                    tBuilder.build()));
+            buckets.add(DefaultGroupBucket.createSelectGroupBucket(
+                    tBuilder.build()));
+        }
+        GroupBuckets groupAddBuckets = new GroupBuckets(addBuckets);
+        groupManager.addBucketsToGroup(deviceId,
+                                       prevKey,
+                                       groupAddBuckets,
+                                       addKey,
+                                       appId);
+        GroupBuckets updatedBuckets = new GroupBuckets(buckets);
+        List<GroupOperation> expectedGroupOps = Collections.singletonList(
+                GroupOperation.createModifyGroupOperation(createdGroup.id(),
+                                                          Group.Type.SELECT,
+                                                          updatedBuckets));
+        if (deviceId.equals(VDID1)) {
+            provider.validate(networkId, deviceId, expectedGroupOps);
+        }
+
+        Group existingGroup = groupManager.getGroup(deviceId, addKey);
+        List<Group> groupEntries = Collections.singletonList(existingGroup);
+        providerService.pushGroupMetrics(deviceId, groupEntries);
+        listener.validateEvent(Collections.singletonList(GroupEvent.Type.GROUP_UPDATED));
+    }
+
+    // Test purge group entry operations
+    private void testPurgeGroupEntry(NetworkId networkId, DeviceId deviceId) {
+        VirtualNetworkGroupManager groupManager;
+        if (networkId.id() == 1) {
+            groupManager = groupManager1;
+        } else {
+            groupManager = groupManager2;
+        }
+
+        assertEquals(1, Iterables.size(groupManager.getGroups(deviceId, appId)));
+        groupManager.purgeGroupEntries(deviceId);
+        assertEquals(0, Iterables.size(groupManager.getGroups(deviceId, appId)));
+    }
+
+    // Test group remove bucket operations
+    private void testRemoveBuckets(NetworkId networkId, DeviceId deviceId) {
+        VirtualNetworkGroupManager groupManager;
+        VirtualGroupProviderService providerService;
+        TestGroupListener listener;
+
+        if (networkId.id() == 1) {
+            groupManager = groupManager1;
+            providerService = providerService1;
+            listener = listener1;
+        } else {
+            groupManager = groupManager2;
+            providerService = providerService2;
+            listener = listener2;
+        }
+
+        GroupKey removeKey = new DefaultGroupKey("group1RemoveBuckets".getBytes());
+
+        GroupKey prevKey = new DefaultGroupKey("group1AddBuckets".getBytes());
+        Group createdGroup = groupManager.getGroup(deviceId, prevKey);
+        List<GroupBucket> buckets = new ArrayList<>();
+        buckets.addAll(createdGroup.buckets().buckets());
+
+        PortNumber[] removePorts = {PortNumber.portNumber(31),
+                PortNumber.portNumber(32)};
+        List<PortNumber> outPorts = new ArrayList<>();
+        outPorts.addAll(Arrays.asList(removePorts));
+        List<GroupBucket> removeBuckets = new ArrayList<>();
+        for (PortNumber portNumber : outPorts) {
+            TrafficTreatment.Builder tBuilder = DefaultTrafficTreatment.builder();
+            tBuilder.setOutput(portNumber)
+                    .setEthDst(MacAddress.valueOf("00:00:00:00:00:02"))
+                    .setEthSrc(MacAddress.valueOf("00:00:00:00:00:01"))
+                    .pushMpls()
+                    .setMpls(MplsLabel.mplsLabel(106));
+            removeBuckets.add(DefaultGroupBucket.createSelectGroupBucket(
+                    tBuilder.build()));
+            buckets.remove(DefaultGroupBucket.createSelectGroupBucket(
+                    tBuilder.build()));
+        }
+        GroupBuckets groupRemoveBuckets = new GroupBuckets(removeBuckets);
+        groupManager.removeBucketsFromGroup(deviceId,
+                                            prevKey,
+                                            groupRemoveBuckets,
+                                            removeKey,
+                                            appId);
+        GroupBuckets updatedBuckets = new GroupBuckets(buckets);
+        List<GroupOperation> expectedGroupOps = Collections.singletonList(
+                GroupOperation.createModifyGroupOperation(createdGroup.id(),
+                                                          Group.Type.SELECT,
+                                                          updatedBuckets));
+        if (deviceId.equals(VDID1)) {
+            provider.validate(networkId, deviceId, expectedGroupOps);
+        }
+
+        Group existingGroup = groupManager.getGroup(deviceId, removeKey);
+        List<Group> groupEntries = Collections.singletonList(existingGroup);
+        providerService.pushGroupMetrics(deviceId, groupEntries);
+        listener.validateEvent(Collections.singletonList(GroupEvent.Type.GROUP_UPDATED));
+    }
+
+    // Test group remove operations
+    private void testRemoveGroup(NetworkId networkId, DeviceId deviceId) {
+        VirtualNetworkGroupManager groupManager;
+        VirtualGroupProviderService providerService;
+        TestGroupListener listener;
+
+        if (networkId.id() == 1) {
+            groupManager = groupManager1;
+            providerService = providerService1;
+            listener = listener1;
+        } else {
+            groupManager = groupManager2;
+            providerService = providerService2;
+            listener = listener2;
+        }
+
+        GroupKey currKey = new DefaultGroupKey("group1RemoveBuckets".getBytes());
+        Group existingGroup = groupManager.getGroup(deviceId, currKey);
+        groupManager.removeGroup(deviceId, currKey, appId);
+        List<GroupOperation> expectedGroupOps = Collections.singletonList(
+                GroupOperation.createDeleteGroupOperation(existingGroup.id(),
+                                                          Group.Type.SELECT));
+        if (deviceId.equals(VDID1)) {
+            provider.validate(networkId, deviceId, expectedGroupOps);
+        }
+
+        List<Group> groupEntries = Collections.emptyList();
+        providerService.pushGroupMetrics(deviceId, groupEntries);
+        listener.validateEvent(Collections.singletonList(GroupEvent.Type.GROUP_REMOVED));
+    }
+
+    private class TestGroupProvider extends AbstractVirtualProvider
+            implements VirtualGroupProvider {
+        NetworkId lastNetworkId;
+        DeviceId lastDeviceId;
+        List<GroupOperation> groupOperations = new ArrayList<>();
+
+        protected TestGroupProvider() {
+            super(new ProviderId("test", "org.onosproject.virtual.testprovider"));
+        }
+
+        @Override
+        public void performGroupOperation(NetworkId networkId, DeviceId deviceId,
+                                          GroupOperations groupOps) {
+            lastNetworkId = networkId;
+            lastDeviceId = deviceId;
+            groupOperations.addAll(groupOps.operations());
+        }
+
+        public void validate(NetworkId expectedNetworkId, DeviceId expectedDeviceId,
+                             List<GroupOperation> expectedGroupOps) {
+            if (expectedGroupOps == null) {
+                assertTrue("events generated", groupOperations.isEmpty());
+                return;
+            }
+
+            assertEquals(lastNetworkId, expectedNetworkId);
+            assertEquals(lastDeviceId, expectedDeviceId);
+            assertTrue((this.groupOperations.containsAll(expectedGroupOps) &&
+                    expectedGroupOps.containsAll(groupOperations)));
+
+            groupOperations.clear();
+            lastDeviceId = null;
+            lastNetworkId = null;
+        }
+    }
+
+    private static class TestGroupListener implements GroupListener {
+        final List<GroupEvent> events = new ArrayList<>();
+
+        @Override
+        public void event(GroupEvent event) {
+            events.add(event);
+        }
+
+        public void validateEvent(List<GroupEvent.Type> expectedEvents) {
+            int i = 0;
+            System.err.println("events :" + events);
+            for (GroupEvent e : events) {
+                assertEquals("unexpected event", expectedEvents.get(i), e.type());
+                i++;
+            }
+            assertEquals("mispredicted number of events",
+                         expectedEvents.size(), events.size());
+            events.clear();
+        }
+    }
+}
\ No newline at end of file
diff --git a/incubator/store/src/main/java/org/onosproject/incubator/store/virtual/impl/SimpleVirtualFlowRuleStore.java b/incubator/store/src/main/java/org/onosproject/incubator/store/virtual/impl/SimpleVirtualFlowRuleStore.java
index ff13ac2..412ffd1 100644
--- a/incubator/store/src/main/java/org/onosproject/incubator/store/virtual/impl/SimpleVirtualFlowRuleStore.java
+++ b/incubator/store/src/main/java/org/onosproject/incubator/store/virtual/impl/SimpleVirtualFlowRuleStore.java
@@ -67,8 +67,11 @@
 import static org.onosproject.net.flow.FlowRuleEvent.Type.RULE_REMOVED;
 import static org.slf4j.LoggerFactory.getLogger;
 
-//TODO: support distributed flowrule store for virtual networks
+/**
+ * Manages inventory of virtual flow rules using trivial in-memory implementation.
+ */
 
+//TODO: support distributed flowrule store for virtual networks
 @Component(immediate = true)
 @Service
 public class SimpleVirtualFlowRuleStore
diff --git a/incubator/store/src/main/java/org/onosproject/incubator/store/virtual/impl/SimpleVirtualGroupStore.java b/incubator/store/src/main/java/org/onosproject/incubator/store/virtual/impl/SimpleVirtualGroupStore.java
new file mode 100644
index 0000000..d1ba607
--- /dev/null
+++ b/incubator/store/src/main/java/org/onosproject/incubator/store/virtual/impl/SimpleVirtualGroupStore.java
@@ -0,0 +1,764 @@
+/*
+ * Copyright 2017-present Open Networking Laboratory
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.onosproject.incubator.store.virtual.impl;
+
+import com.google.common.collect.FluentIterable;
+import com.google.common.collect.Sets;
+import org.apache.felix.scr.annotations.Activate;
+import org.apache.felix.scr.annotations.Component;
+import org.apache.felix.scr.annotations.Deactivate;
+import org.apache.felix.scr.annotations.Service;
+import org.onosproject.core.DefaultGroupId;
+import org.onosproject.core.GroupId;
+import org.onosproject.incubator.net.virtual.NetworkId;
+import org.onosproject.incubator.net.virtual.VirtualNetworkGroupStore;
+import org.onosproject.net.DeviceId;
+import org.onosproject.net.group.DefaultGroup;
+import org.onosproject.net.group.DefaultGroupDescription;
+import org.onosproject.net.group.Group;
+import org.onosproject.net.group.GroupBucket;
+import org.onosproject.net.group.GroupBuckets;
+import org.onosproject.net.group.GroupDescription;
+import org.onosproject.net.group.GroupEvent;
+import org.onosproject.net.group.GroupKey;
+import org.onosproject.net.group.GroupOperation;
+import org.onosproject.net.group.GroupStoreDelegate;
+import org.onosproject.net.group.StoredGroupBucketEntry;
+import org.onosproject.net.group.StoredGroupEntry;
+import org.slf4j.Logger;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static org.slf4j.LoggerFactory.getLogger;
+
+/**
+ * Manages inventory of virtual group entries using trivial in-memory implementation.
+ */
+@Component(immediate = true)
+@Service
+public class SimpleVirtualGroupStore
+        extends AbstractVirtualStore<GroupEvent, GroupStoreDelegate>
+        implements VirtualNetworkGroupStore {
+
+    private final Logger log = getLogger(getClass());
+
+    private final int dummyId = 0xffffffff;
+    private final GroupId dummyGroupId = new DefaultGroupId(dummyId);
+
+    // inner Map is per device group table
+    private final ConcurrentMap<NetworkId,
+            ConcurrentMap<DeviceId, ConcurrentMap<GroupKey, StoredGroupEntry>>>
+            groupEntriesByKey = new ConcurrentHashMap<>();
+
+    private final ConcurrentMap<NetworkId,
+            ConcurrentMap<DeviceId, ConcurrentMap<GroupId, StoredGroupEntry>>>
+            groupEntriesById = new ConcurrentHashMap<>();
+
+    private final ConcurrentMap<NetworkId,
+            ConcurrentMap<DeviceId, ConcurrentMap<GroupKey, StoredGroupEntry>>>
+            pendingGroupEntriesByKey = new ConcurrentHashMap<>();
+
+    private final ConcurrentMap<NetworkId,
+            ConcurrentMap<DeviceId, ConcurrentMap<GroupId, Group>>>
+            extraneousGroupEntriesById = new ConcurrentHashMap<>();
+
+    private final ConcurrentMap<NetworkId, HashMap<DeviceId, Boolean>>
+            deviceAuditStatus = new ConcurrentHashMap<>();
+
+    private final AtomicInteger groupIdGen = new AtomicInteger();
+
+    @Activate
+    public void activate() {
+        log.info("Started");
+    }
+
+    @Deactivate
+    public void deactivate() {
+        groupEntriesByKey.clear();
+        groupEntriesById.clear();
+        log.info("Stopped");
+    }
+
+    /**
+     * Returns the group key table for specified device.
+     *
+     * @param networkId identifier of the virtual network
+     * @param deviceId identifier of the device
+     * @return Map representing group key table of given device.
+     */
+    private ConcurrentMap<GroupKey, StoredGroupEntry>
+    getGroupKeyTable(NetworkId networkId, DeviceId deviceId) {
+        groupEntriesByKey.computeIfAbsent(networkId, n -> new ConcurrentHashMap<>());
+        return groupEntriesByKey.get(networkId)
+                .computeIfAbsent(deviceId, k -> new ConcurrentHashMap<>());
+    }
+
+    /**
+     * Returns the group id table for specified device.
+     *
+     * @param networkId identifier of the virtual network
+     * @param deviceId identifier of the device
+     * @return Map representing group key table of given device.
+     */
+    private ConcurrentMap<GroupId, StoredGroupEntry>
+    getGroupIdTable(NetworkId networkId, DeviceId deviceId) {
+        groupEntriesById.computeIfAbsent(networkId, n -> new ConcurrentHashMap<>());
+        return groupEntriesById.get(networkId)
+                .computeIfAbsent(deviceId, k -> new ConcurrentHashMap<>());
+    }
+
+    /**
+     * Returns the pending group key table for specified device.
+     *
+     * @param networkId identifier of the virtual network
+     * @param deviceId identifier of the device
+     * @return Map representing group key table of given device.
+     */
+    private ConcurrentMap<GroupKey, StoredGroupEntry>
+    getPendingGroupKeyTable(NetworkId networkId, DeviceId deviceId) {
+        pendingGroupEntriesByKey.computeIfAbsent(networkId, n -> new ConcurrentHashMap<>());
+        return pendingGroupEntriesByKey.get(networkId)
+                .computeIfAbsent(deviceId, k -> new ConcurrentHashMap<>());
+    }
+
+    /**
+     * Returns the extraneous group id table for specified device.
+     *
+     * @param networkId identifier of the virtual network
+     * @param deviceId identifier of the device
+     * @return Map representing group key table of given device.
+     */
+    private ConcurrentMap<GroupId, Group>
+    getExtraneousGroupIdTable(NetworkId networkId, DeviceId deviceId) {
+        extraneousGroupEntriesById.computeIfAbsent(networkId, n -> new ConcurrentHashMap<>());
+        return extraneousGroupEntriesById.get(networkId)
+                .computeIfAbsent(deviceId, k -> new ConcurrentHashMap<>());
+    }
+
+    @Override
+    public int getGroupCount(NetworkId networkId, DeviceId deviceId) {
+        return (groupEntriesByKey.get(networkId).get(deviceId) != null) ?
+                groupEntriesByKey.get(networkId).get(deviceId).size() : 0;
+    }
+
+    @Override
+    public Iterable<Group> getGroups(NetworkId networkId, DeviceId deviceId) {
+        // flatten and make iterator unmodifiable
+        return FluentIterable.from(getGroupKeyTable(networkId, deviceId).values())
+                .transform(input -> input);
+    }
+
+    @Override
+    public Group getGroup(NetworkId networkId, DeviceId deviceId, GroupKey appCookie) {
+        if (groupEntriesByKey.get(networkId) != null &&
+                groupEntriesByKey.get(networkId).get(deviceId) != null) {
+            return groupEntriesByKey.get(networkId).get(deviceId).get(appCookie);
+        }
+        return null;
+    }
+
+    @Override
+    public Group getGroup(NetworkId networkId, DeviceId deviceId, GroupId groupId) {
+        if (groupEntriesById.get(networkId) != null &&
+                groupEntriesById.get(networkId).get(deviceId) != null) {
+            return groupEntriesById.get(networkId).get(deviceId).get(groupId);
+        }
+        return null;
+    }
+
+    private int getFreeGroupIdValue(NetworkId networkId, DeviceId deviceId) {
+        int freeId = groupIdGen.incrementAndGet();
+
+        while (true) {
+            Group existing = null;
+            if (groupEntriesById.get(networkId) != null &&
+                    groupEntriesById.get(networkId).get(deviceId) != null) {
+                existing = groupEntriesById.get(networkId).get(deviceId)
+                                .get(new DefaultGroupId(freeId));
+            }
+
+            if (existing == null) {
+                if (extraneousGroupEntriesById.get(networkId) != null &&
+                        extraneousGroupEntriesById.get(networkId).get(deviceId) != null) {
+                    existing = extraneousGroupEntriesById.get(networkId).get(deviceId)
+                                    .get(new DefaultGroupId(freeId));
+                }
+            }
+
+            if (existing != null) {
+                freeId = groupIdGen.incrementAndGet();
+            } else {
+                break;
+            }
+        }
+        return freeId;
+    }
+
+    @Override
+    public void storeGroupDescription(NetworkId networkId, GroupDescription groupDesc) {
+        // Check if a group is existing with the same key
+        if (getGroup(networkId, groupDesc.deviceId(), groupDesc.appCookie()) != null) {
+            return;
+        }
+
+        if (deviceAuditStatus.get(networkId) == null ||
+                deviceAuditStatus.get(networkId).get(groupDesc.deviceId()) == null) {
+            // Device group audit has not completed yet
+            // Add this group description to pending group key table
+            // Create a group entry object with Dummy Group ID
+            StoredGroupEntry group = new DefaultGroup(dummyGroupId, groupDesc);
+            group.setState(Group.GroupState.WAITING_AUDIT_COMPLETE);
+            ConcurrentMap<GroupKey, StoredGroupEntry> pendingKeyTable =
+                    getPendingGroupKeyTable(networkId, groupDesc.deviceId());
+            pendingKeyTable.put(groupDesc.appCookie(), group);
+            return;
+        }
+
+        storeGroupDescriptionInternal(networkId, groupDesc);
+    }
+
+    private void storeGroupDescriptionInternal(NetworkId networkId,
+                                               GroupDescription groupDesc) {
+        // Check if a group is existing with the same key
+        if (getGroup(networkId, groupDesc.deviceId(), groupDesc.appCookie()) != null) {
+            return;
+        }
+
+        GroupId id = null;
+        if (groupDesc.givenGroupId() == null) {
+            // Get a new group identifier
+            id = new DefaultGroupId(getFreeGroupIdValue(networkId, groupDesc.deviceId()));
+        } else {
+            id = new DefaultGroupId(groupDesc.givenGroupId());
+        }
+        // Create a group entry object
+        StoredGroupEntry group = new DefaultGroup(id, groupDesc);
+        // Insert the newly created group entry into concurrent key and id maps
+        ConcurrentMap<GroupKey, StoredGroupEntry> keyTable =
+                getGroupKeyTable(networkId, groupDesc.deviceId());
+        keyTable.put(groupDesc.appCookie(), group);
+        ConcurrentMap<GroupId, StoredGroupEntry> idTable =
+                getGroupIdTable(networkId, groupDesc.deviceId());
+        idTable.put(id, group);
+        notifyDelegate(networkId, new GroupEvent(GroupEvent.Type.GROUP_ADD_REQUESTED,
+                                      group));
+    }
+
+    @Override
+    public void updateGroupDescription(NetworkId networkId, DeviceId deviceId,
+                                       GroupKey oldAppCookie, UpdateType type,
+                                       GroupBuckets newBuckets, GroupKey newAppCookie) {
+        // Check if a group is existing with the provided key
+        Group oldGroup = getGroup(networkId, deviceId, oldAppCookie);
+        if (oldGroup == null) {
+            return;
+        }
+
+        List<GroupBucket> newBucketList = getUpdatedBucketList(oldGroup,
+                                                               type,
+                                                               newBuckets);
+        if (newBucketList != null) {
+            // Create a new group object from the old group
+            GroupBuckets updatedBuckets = new GroupBuckets(newBucketList);
+            GroupKey newCookie = (newAppCookie != null) ? newAppCookie : oldAppCookie;
+            GroupDescription updatedGroupDesc = new DefaultGroupDescription(
+                    oldGroup.deviceId(),
+                    oldGroup.type(),
+                    updatedBuckets,
+                    newCookie,
+                    oldGroup.givenGroupId(),
+                    oldGroup.appId());
+            StoredGroupEntry newGroup = new DefaultGroup(oldGroup.id(),
+                                                         updatedGroupDesc);
+            newGroup.setState(Group.GroupState.PENDING_UPDATE);
+            newGroup.setLife(oldGroup.life());
+            newGroup.setPackets(oldGroup.packets());
+            newGroup.setBytes(oldGroup.bytes());
+
+            // Remove the old entry from maps and add new entry using new key
+            ConcurrentMap<GroupKey, StoredGroupEntry> keyTable =
+                    getGroupKeyTable(networkId, oldGroup.deviceId());
+            ConcurrentMap<GroupId, StoredGroupEntry> idTable =
+                    getGroupIdTable(networkId, oldGroup.deviceId());
+            keyTable.remove(oldGroup.appCookie());
+            idTable.remove(oldGroup.id());
+            keyTable.put(newGroup.appCookie(), newGroup);
+            idTable.put(newGroup.id(), newGroup);
+            notifyDelegate(networkId,
+                           new GroupEvent(GroupEvent.Type.GROUP_UPDATE_REQUESTED,
+                                          newGroup));
+        }
+
+    }
+
+    private List<GroupBucket> getUpdatedBucketList(Group oldGroup,
+                                                   UpdateType type,
+                                                   GroupBuckets buckets) {
+        if (type == UpdateType.SET) {
+            return buckets.buckets();
+        }
+
+        List<GroupBucket> oldBuckets = oldGroup.buckets().buckets();
+        List<GroupBucket> updatedBucketList = new ArrayList<>();
+        boolean groupDescUpdated = false;
+
+        if (type == UpdateType.ADD) {
+            List<GroupBucket> newBuckets = buckets.buckets();
+
+            // Add old buckets that will not be updated and check if any will be updated.
+            for (GroupBucket oldBucket : oldBuckets) {
+                int newBucketIndex = newBuckets.indexOf(oldBucket);
+
+                if (newBucketIndex != -1) {
+                    GroupBucket newBucket = newBuckets.get(newBucketIndex);
+                    if (!newBucket.hasSameParameters(oldBucket)) {
+                        // Bucket will be updated
+                        groupDescUpdated = true;
+                    }
+                } else {
+                    // Old bucket will remain the same - add it.
+                    updatedBucketList.add(oldBucket);
+                }
+            }
+
+            // Add all new buckets
+            updatedBucketList.addAll(newBuckets);
+            if (!oldBuckets.containsAll(newBuckets)) {
+                groupDescUpdated = true;
+            }
+
+        } else if (type == UpdateType.REMOVE) {
+            List<GroupBucket> bucketsToRemove = buckets.buckets();
+
+            // Check which old buckets should remain
+            for (GroupBucket oldBucket : oldBuckets) {
+                if (!bucketsToRemove.contains(oldBucket)) {
+                    updatedBucketList.add(oldBucket);
+                } else {
+                    groupDescUpdated = true;
+                }
+            }
+        }
+
+        if (groupDescUpdated) {
+            return updatedBucketList;
+        } else {
+            return null;
+        }
+    }
+
+    @Override
+    public void deleteGroupDescription(NetworkId networkId, DeviceId deviceId,
+                                       GroupKey appCookie) {
+        // Check if a group is existing with the provided key
+        StoredGroupEntry existing = null;
+        if (groupEntriesByKey.get(networkId) != null &&
+                groupEntriesByKey.get(networkId).get(deviceId) != null) {
+            existing = groupEntriesByKey.get(networkId).get(deviceId).get(appCookie);
+        }
+
+        if (existing == null) {
+            return;
+        }
+
+        synchronized (existing) {
+            existing.setState(Group.GroupState.PENDING_DELETE);
+        }
+        notifyDelegate(networkId,
+                       new GroupEvent(GroupEvent.Type.GROUP_REMOVE_REQUESTED, existing));
+    }
+
+    @Override
+    public void addOrUpdateGroupEntry(NetworkId networkId, Group group) {
+        // check if this new entry is an update to an existing entry
+        StoredGroupEntry existing = null;
+
+        if (groupEntriesById.get(networkId) != null &&
+                groupEntriesById.get(networkId).get(group.deviceId()) != null) {
+            existing = groupEntriesById
+                    .get(networkId)
+                    .get(group.deviceId())
+                    .get(group.id());
+        }
+
+        GroupEvent event = null;
+
+        if (existing != null) {
+            synchronized (existing) {
+                for (GroupBucket bucket:group.buckets().buckets()) {
+                    Optional<GroupBucket> matchingBucket =
+                            existing.buckets().buckets()
+                                    .stream()
+                                    .filter((existingBucket)->(existingBucket.equals(bucket)))
+                                    .findFirst();
+                    if (matchingBucket.isPresent()) {
+                        ((StoredGroupBucketEntry) matchingBucket.
+                                get()).setPackets(bucket.packets());
+                        ((StoredGroupBucketEntry) matchingBucket.
+                                get()).setBytes(bucket.bytes());
+                    } else {
+                        log.warn("addOrUpdateGroupEntry: No matching "
+                                         + "buckets to update stats");
+                    }
+                }
+                existing.setLife(group.life());
+                existing.setPackets(group.packets());
+                existing.setBytes(group.bytes());
+                if (existing.state() == Group.GroupState.PENDING_ADD) {
+                    existing.setState(Group.GroupState.ADDED);
+                    event = new GroupEvent(GroupEvent.Type.GROUP_ADDED, existing);
+                } else {
+                    if (existing.state() == Group.GroupState.PENDING_UPDATE) {
+                        existing.setState(Group.GroupState.ADDED);
+                    }
+                    event = new GroupEvent(GroupEvent.Type.GROUP_UPDATED, existing);
+                }
+            }
+        }
+
+        if (event != null) {
+            notifyDelegate(networkId, event);
+        }
+    }
+
+    @Override
+    public void removeGroupEntry(NetworkId networkId, Group group) {
+        StoredGroupEntry existing = null;
+        if (groupEntriesById.get(networkId) != null
+                && groupEntriesById.get(networkId).get(group.deviceId()) != null) {
+           existing = groupEntriesById
+                   .get(networkId).get(group.deviceId()).get(group.id());
+        }
+
+        if (existing != null) {
+            ConcurrentMap<GroupKey, StoredGroupEntry> keyTable =
+                    getGroupKeyTable(networkId, existing.deviceId());
+            ConcurrentMap<GroupId, StoredGroupEntry> idTable =
+                    getGroupIdTable(networkId, existing.deviceId());
+            idTable.remove(existing.id());
+            keyTable.remove(existing.appCookie());
+            notifyDelegate(networkId,
+                           new GroupEvent(GroupEvent.Type.GROUP_REMOVED, existing));
+        }
+    }
+
+    @Override
+    public void purgeGroupEntry(NetworkId networkId, DeviceId deviceId) {
+        if (groupEntriesById.get(networkId) != null) {
+            Set<Map.Entry<GroupId, StoredGroupEntry>> entryPendingRemove =
+                    groupEntriesById.get(networkId).get(deviceId).entrySet();
+            groupEntriesById.get(networkId).remove(deviceId);
+            groupEntriesByKey.get(networkId).remove(deviceId);
+
+            entryPendingRemove.forEach(entry -> {
+                notifyDelegate(networkId,
+                               new GroupEvent(GroupEvent.Type.GROUP_REMOVED,
+                                              entry.getValue()));
+            });
+        }
+    }
+
+    @Override
+    public void purgeGroupEntries(NetworkId networkId) {
+        if (groupEntriesById.get(networkId) != null) {
+            groupEntriesById.get((networkId)).values().forEach(groupEntries -> {
+                groupEntries.entrySet().forEach(entry -> {
+                    notifyDelegate(networkId,
+                                   new GroupEvent(GroupEvent.Type.GROUP_REMOVED,
+                                                  entry.getValue()));
+                });
+            });
+
+            groupEntriesById.get(networkId).clear();
+            groupEntriesByKey.get(networkId).clear();
+        }
+    }
+
+    @Override
+    public void addOrUpdateExtraneousGroupEntry(NetworkId networkId, Group group) {
+        ConcurrentMap<GroupId, Group> extraneousIdTable =
+                getExtraneousGroupIdTable(networkId, group.deviceId());
+        extraneousIdTable.put(group.id(), group);
+        // Check the reference counter
+        if (group.referenceCount() == 0) {
+            notifyDelegate(networkId,
+                           new GroupEvent(GroupEvent.Type.GROUP_REMOVE_REQUESTED, group));
+        }
+    }
+
+    @Override
+    public void removeExtraneousGroupEntry(NetworkId networkId, Group group) {
+        ConcurrentMap<GroupId, Group> extraneousIdTable =
+                getExtraneousGroupIdTable(networkId, group.deviceId());
+        extraneousIdTable.remove(group.id());
+    }
+
+    @Override
+    public Iterable<Group> getExtraneousGroups(NetworkId networkId, DeviceId deviceId) {
+        // flatten and make iterator unmodifiable
+        return FluentIterable.from(
+                getExtraneousGroupIdTable(networkId, deviceId).values());
+    }
+
+    @Override
+    public void deviceInitialAuditCompleted(NetworkId networkId, DeviceId deviceId,
+                                            boolean completed) {
+        deviceAuditStatus.computeIfAbsent(networkId, k -> new HashMap<>());
+
+        HashMap<DeviceId, Boolean> deviceAuditStatusByNetwork =
+                deviceAuditStatus.get(networkId);
+
+        synchronized (deviceAuditStatusByNetwork) {
+            if (completed) {
+                log.debug("deviceInitialAuditCompleted: AUDIT "
+                                  + "completed for device {}", deviceId);
+                deviceAuditStatusByNetwork.put(deviceId, true);
+                // Execute all pending group requests
+                ConcurrentMap<GroupKey, StoredGroupEntry> pendingGroupRequests =
+                        getPendingGroupKeyTable(networkId, deviceId);
+                for (Group group:pendingGroupRequests.values()) {
+                    GroupDescription tmp = new DefaultGroupDescription(
+                            group.deviceId(),
+                            group.type(),
+                            group.buckets(),
+                            group.appCookie(),
+                            group.givenGroupId(),
+                            group.appId());
+                    storeGroupDescriptionInternal(networkId, tmp);
+                }
+                getPendingGroupKeyTable(networkId, deviceId).clear();
+            } else {
+                if (deviceAuditStatusByNetwork.get(deviceId)) {
+                    log.debug("deviceInitialAuditCompleted: Clearing AUDIT "
+                                      + "status for device {}", deviceId);
+                    deviceAuditStatusByNetwork.put(deviceId, false);
+                }
+            }
+        }
+    }
+
+    @Override
+    public boolean deviceInitialAuditStatus(NetworkId networkId, DeviceId deviceId) {
+        deviceAuditStatus.computeIfAbsent(networkId, k -> new HashMap<>());
+
+        HashMap<DeviceId, Boolean> deviceAuditStatusByNetwork =
+                deviceAuditStatus.get(networkId);
+
+        synchronized (deviceAuditStatusByNetwork) {
+            return (deviceAuditStatusByNetwork.get(deviceId) != null)
+                    ? deviceAuditStatusByNetwork.get(deviceId) : false;
+        }
+    }
+
+    @Override
+    public void groupOperationFailed(NetworkId networkId, DeviceId deviceId,
+                                     GroupOperation operation) {
+
+        StoredGroupEntry existing = null;
+        if (groupEntriesById.get(networkId) != null &&
+                groupEntriesById.get(networkId).get(deviceId) != null) {
+            existing = groupEntriesById.get(networkId).get(deviceId)
+                    .get(operation.groupId());
+        }
+
+        if (existing == null) {
+            log.warn("No group entry with ID {} found ", operation.groupId());
+            return;
+        }
+
+        switch (operation.opType()) {
+            case ADD:
+                notifyDelegate(networkId,
+                               new GroupEvent(GroupEvent.Type.GROUP_ADD_FAILED,
+                                              existing));
+                break;
+            case MODIFY:
+                notifyDelegate(networkId,
+                               new GroupEvent(GroupEvent.Type.GROUP_UPDATE_FAILED,
+                                              existing));
+                break;
+            case DELETE:
+                notifyDelegate(networkId,
+                               new GroupEvent(GroupEvent.Type.GROUP_REMOVE_FAILED,
+                                              existing));
+                break;
+            default:
+                log.warn("Unknown group operation type {}", operation.opType());
+        }
+
+        ConcurrentMap<GroupKey, StoredGroupEntry> keyTable =
+                getGroupKeyTable(networkId, existing.deviceId());
+        ConcurrentMap<GroupId, StoredGroupEntry> idTable =
+                getGroupIdTable(networkId, existing.deviceId());
+        idTable.remove(existing.id());
+        keyTable.remove(existing.appCookie());
+    }
+
+    @Override
+    public void pushGroupMetrics(NetworkId networkId, DeviceId deviceId,
+                                 Collection<Group> groupEntries) {
+        boolean deviceInitialAuditStatus =
+                deviceInitialAuditStatus(networkId, deviceId);
+        Set<Group> southboundGroupEntries =
+                Sets.newHashSet(groupEntries);
+        Set<Group> storedGroupEntries =
+                Sets.newHashSet(getGroups(networkId, deviceId));
+        Set<Group> extraneousStoredEntries =
+                Sets.newHashSet(getExtraneousGroups(networkId, deviceId));
+
+        if (log.isTraceEnabled()) {
+            log.trace("pushGroupMetrics: Displaying all ({}) "
+                              + "southboundGroupEntries for device {}",
+                      southboundGroupEntries.size(),
+                      deviceId);
+            for (Group group : southboundGroupEntries) {
+                log.trace("Group {} in device {}", group, deviceId);
+            }
+
+            log.trace("Displaying all ({}) stored group entries for device {}",
+                      storedGroupEntries.size(),
+                      deviceId);
+            for (Group group : storedGroupEntries) {
+                log.trace("Stored Group {} for device {}", group, deviceId);
+            }
+        }
+
+        for (Iterator<Group> it2 = southboundGroupEntries.iterator(); it2.hasNext();) {
+            Group group = it2.next();
+            if (storedGroupEntries.remove(group)) {
+                // we both have the group, let's update some info then.
+                log.trace("Group AUDIT: group {} exists "
+                                  + "in both planes for device {}",
+                          group.id(), deviceId);
+                groupAdded(networkId, group);
+                it2.remove();
+            }
+        }
+        for (Group group : southboundGroupEntries) {
+            if (getGroup(networkId, group.deviceId(), group.id()) != null) {
+                // There is a group existing with the same id
+                // It is possible that group update is
+                // in progress while we got a stale info from switch
+                if (!storedGroupEntries.remove(getGroup(
+                        networkId, group.deviceId(), group.id()))) {
+                    log.warn("Group AUDIT: Inconsistent state:"
+                                     + "Group exists in ID based table while "
+                                     + "not present in key based table");
+                }
+            } else {
+                // there are groups in the switch that aren't in the store
+                log.trace("Group AUDIT: extraneous group {} exists "
+                                  + "in data plane for device {}",
+                          group.id(), deviceId);
+                extraneousStoredEntries.remove(group);
+                extraneousGroup(networkId, group);
+            }
+        }
+        for (Group group : storedGroupEntries) {
+            // there are groups in the store that aren't in the switch
+            log.trace("Group AUDIT: group {} missing "
+                              + "in data plane for device {}",
+                      group.id(), deviceId);
+            groupMissing(networkId, group);
+        }
+        for (Group group : extraneousStoredEntries) {
+            // there are groups in the extraneous store that
+            // aren't in the switch
+            log.trace("Group AUDIT: clearing extransoeus group {} "
+                              + "from store for device {}",
+                      group.id(), deviceId);
+            removeExtraneousGroupEntry(networkId, group);
+        }
+
+        if (!deviceInitialAuditStatus) {
+            log.debug("Group AUDIT: Setting device {} initial "
+                              + "AUDIT completed", deviceId);
+            deviceInitialAuditCompleted(networkId, deviceId, true);
+        }
+    }
+
+    @Override
+    public void notifyOfFailovers(NetworkId networkId, Collection<Group> failoverGroups) {
+        List<GroupEvent> failoverEvents = new ArrayList<>();
+        failoverGroups.forEach(group -> {
+            if (group.type() == Group.Type.FAILOVER) {
+                failoverEvents.add(new GroupEvent(GroupEvent.Type.GROUP_BUCKET_FAILOVER, group));
+            }
+        });
+        notifyDelegate(networkId, failoverEvents);
+    }
+
+    private void groupMissing(NetworkId networkId, Group group) {
+        switch (group.state()) {
+            case PENDING_DELETE:
+                log.debug("Group {} delete confirmation from device {} " +
+                                  "of virtaual network {}",
+                          group, group.deviceId(), networkId);
+                removeGroupEntry(networkId, group);
+                break;
+            case ADDED:
+            case PENDING_ADD:
+            case PENDING_UPDATE:
+                log.debug("Group {} is in store but not on device {}",
+                          group, group.deviceId());
+                StoredGroupEntry existing = null;
+                if (groupEntriesById.get(networkId) != null &&
+                        groupEntriesById.get(networkId).get(group.deviceId()) != null) {
+
+                    existing = groupEntriesById.get(networkId)
+                            .get(group.deviceId()).get(group.id());
+                }
+
+                log.trace("groupMissing: group "
+                                  + "entry {} in device {} moving "
+                                  + "from {} to PENDING_ADD",
+                          existing.id(),
+                          existing.deviceId(),
+                          existing.state());
+                existing.setState(Group.GroupState.PENDING_ADD);
+                notifyDelegate(networkId, new GroupEvent(GroupEvent.Type.GROUP_ADD_REQUESTED,
+                                              group));
+                break;
+            default:
+                log.debug("Virtual network {} : Group {} has not been installed.",
+                          networkId, group);
+                break;
+        }
+    }
+
+    private void extraneousGroup(NetworkId networkId, Group group) {
+        log.debug("Group {} is on device {} of virtual network{}, but not in store.",
+                  group, group.deviceId(), networkId);
+        addOrUpdateExtraneousGroupEntry(networkId, group);
+    }
+
+    private void groupAdded(NetworkId networkId, Group group) {
+        log.trace("Group {} Added or Updated in device {} of virtual network {}",
+                  group, group.deviceId(), networkId);
+        addOrUpdateGroupEntry(networkId, group);
+    }
+}
diff --git a/incubator/store/src/main/java/org/onosproject/incubator/store/virtual/impl/SimpleVirtualPacketStore.java b/incubator/store/src/main/java/org/onosproject/incubator/store/virtual/impl/SimpleVirtualPacketStore.java
index 5dde804..93fc7d8 100644
--- a/incubator/store/src/main/java/org/onosproject/incubator/store/virtual/impl/SimpleVirtualPacketStore.java
+++ b/incubator/store/src/main/java/org/onosproject/incubator/store/virtual/impl/SimpleVirtualPacketStore.java
@@ -33,6 +33,9 @@
 import java.util.Map;
 import java.util.Set;
 
+/**
+ * Simple single instance implementation of the virtual packet store.
+ */
 public class SimpleVirtualPacketStore
         extends AbstractVirtualStore<PacketEvent, PacketStoreDelegate>
         implements VirtualNetworkPacketStore {