BUG-2138: Create DistributedShardFrontend
[controller.git] / opendaylight / md-sal / sal-distributed-datastore / src / main / java / org / opendaylight / controller / cluster / sharding / DistributedShardedDOMDataTree.java
index 91b479dfbe7ebe5e9e8b50ece8c60fd2b1a3c552..557543803769413e0baaebb73c1eae218293ba1e 100644 (file)
@@ -14,30 +14,36 @@ import akka.actor.ActorRef;
 import akka.actor.ActorSystem;
 import akka.actor.PoisonPill;
 import akka.actor.Props;
+import akka.cluster.Cluster;
+import akka.cluster.Member;
 import akka.util.Timeout;
 import com.google.common.base.Preconditions;
 import com.google.common.base.Throwables;
+import com.google.common.collect.Collections2;
 import com.google.common.collect.ForwardingObject;
 import com.google.common.util.concurrent.Uninterruptibles;
+import java.util.AbstractMap.SimpleEntry;
+import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
-import java.util.concurrent.CompletionException;
+import java.util.EnumMap;
+import java.util.Map.Entry;
+import java.util.Set;
 import java.util.concurrent.TimeUnit;
 import javax.annotation.Nonnull;
 import org.opendaylight.controller.cluster.access.concepts.MemberName;
 import org.opendaylight.controller.cluster.databroker.actors.dds.DataStoreClient;
 import org.opendaylight.controller.cluster.databroker.actors.dds.SimpleDataStoreClientActor;
 import org.opendaylight.controller.cluster.datastore.DistributedDataStore;
-import org.opendaylight.controller.cluster.datastore.Shard;
 import org.opendaylight.controller.cluster.datastore.config.PrefixShardConfiguration;
-import org.opendaylight.controller.cluster.datastore.messages.CreatePrefixedShard;
 import org.opendaylight.controller.cluster.datastore.utils.ActorContext;
 import org.opendaylight.controller.cluster.datastore.utils.ClusterUtils;
 import org.opendaylight.controller.cluster.sharding.ShardedDataTreeActor.ShardedDataTreeActorCreator;
-import org.opendaylight.controller.cluster.sharding.messages.PrefixShardCreated;
-import org.opendaylight.controller.cluster.sharding.messages.PrefixShardRemoved;
+import org.opendaylight.controller.cluster.sharding.messages.CreatePrefixShard;
 import org.opendaylight.controller.cluster.sharding.messages.ProducerCreated;
 import org.opendaylight.controller.cluster.sharding.messages.ProducerRemoved;
+import org.opendaylight.controller.cluster.sharding.messages.RemovePrefixShard;
+import org.opendaylight.mdsal.common.api.LogicalDatastoreType;
 import org.opendaylight.mdsal.dom.api.DOMDataTreeCursorAwareTransaction;
 import org.opendaylight.mdsal.dom.api.DOMDataTreeIdentifier;
 import org.opendaylight.mdsal.dom.api.DOMDataTreeListener;
@@ -48,12 +54,15 @@ import org.opendaylight.mdsal.dom.api.DOMDataTreeService;
 import org.opendaylight.mdsal.dom.api.DOMDataTreeShard;
 import org.opendaylight.mdsal.dom.api.DOMDataTreeShardingConflictException;
 import org.opendaylight.mdsal.dom.api.DOMDataTreeShardingService;
+import org.opendaylight.mdsal.dom.broker.DOMDataTreeShardRegistration;
 import org.opendaylight.mdsal.dom.broker.ShardedDOMDataTree;
+import org.opendaylight.mdsal.dom.spi.DOMDataTreePrefixTable;
+import org.opendaylight.mdsal.dom.spi.DOMDataTreePrefixTableEntry;
 import org.opendaylight.yangtools.concepts.ListenerRegistration;
+import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import scala.concurrent.Await;
-import scala.concurrent.Future;
+import scala.collection.JavaConverters;
 
 /**
  * A layer on top of DOMDataTreeService that distributes producer/shard registrations to remote nodes via
@@ -79,6 +88,12 @@ public class DistributedShardedDOMDataTree implements DOMDataTreeService, DOMDat
     private final ActorRef shardedDataTreeActor;
     private final MemberName memberName;
 
+    private final DOMDataTreePrefixTable<DOMDataTreeShardRegistration<DOMDataTreeShard>> shards =
+            DOMDataTreePrefixTable.create();
+
+    private final EnumMap<LogicalDatastoreType, DistributedShardRegistration> defaultShardRegistrations =
+            new EnumMap<>(LogicalDatastoreType.class);
+
     public DistributedShardedDOMDataTree(final ActorSystem actorSystem,
                                          final DistributedDataStore distributedOperDatastore,
                                          final DistributedDataStore distributedConfigDatastore) {
@@ -89,8 +104,7 @@ public class DistributedShardedDOMDataTree implements DOMDataTreeService, DOMDat
 
         shardedDataTreeActor = createShardedDataTreeActor(actorSystem,
                 new ShardedDataTreeActorCreator()
-                        .setDataTreeService(shardedDOMDataTree)
-                        .setShardingService(shardedDOMDataTree)
+                        .setShardingService(this)
                         .setActorSystem(actorSystem)
                         .setClusterWrapper(distributedConfigDatastore.getActorContext().getClusterWrapper())
                         .setDistributedConfigDatastore(distributedConfigDatastore)
@@ -98,6 +112,21 @@ public class DistributedShardedDOMDataTree implements DOMDataTreeService, DOMDat
                 ACTOR_ID);
 
         this.memberName = distributedConfigDatastore.getActorContext().getCurrentMemberName();
+
+        //create shard registration for DEFAULT_SHARD
+        try {
+            defaultShardRegistrations.put(LogicalDatastoreType.CONFIGURATION,
+                    initDefaultShard(LogicalDatastoreType.CONFIGURATION));
+        } catch (final DOMDataTreeProducerException | DOMDataTreeShardingConflictException e) {
+            LOG.error("Unable to create default shard frontend for config shard", e);
+        }
+
+        try {
+            defaultShardRegistrations.put(LogicalDatastoreType.OPERATIONAL,
+                    initDefaultShard(LogicalDatastoreType.OPERATIONAL));
+        } catch (final DOMDataTreeProducerException | DOMDataTreeShardingConflictException e) {
+            LOG.error("Unable to create default shard frontend for operational shard", e);
+        }
     }
 
     @Nonnull
@@ -113,13 +142,15 @@ public class DistributedShardedDOMDataTree implements DOMDataTreeService, DOMDat
     @Nonnull
     @Override
     public DOMDataTreeProducer createProducer(@Nonnull final Collection<DOMDataTreeIdentifier> subtrees) {
-        LOG.debug("Creating producer for {}", subtrees);
+        LOG.debug("{} - Creating producer for {}",
+                distributedConfigDatastore.getActorContext().getClusterWrapper().getCurrentMemberName(), subtrees);
         final DOMDataTreeProducer producer = shardedDOMDataTree.createProducer(subtrees);
 
         final Object response = distributedConfigDatastore.getActorContext()
                 .executeOperation(shardedDataTreeActor, new ProducerCreated(subtrees));
         if (response == null) {
-            LOG.debug("Received success from remote nodes, creating producer:{}", subtrees);
+            LOG.debug("{} - Received success from remote nodes, creating producer:{}",
+                    distributedConfigDatastore.getActorContext().getClusterWrapper().getCurrentMemberName(), subtrees);
             return new ProxyProducer(producer, subtrees, shardedDataTreeActor,
                     distributedConfigDatastore.getActorContext());
         } else if (response instanceof Exception) {
@@ -133,74 +164,92 @@ public class DistributedShardedDOMDataTree implements DOMDataTreeService, DOMDat
 
     @Override
     @SuppressWarnings("checkstyle:IllegalCatch")
+    //TODO: it would be better to block here until the message is processed by the actor
     public DistributedShardRegistration createDistributedShard(
             final DOMDataTreeIdentifier prefix, final Collection<MemberName> replicaMembers)
-            throws DOMDataTreeShardingConflictException, DOMDataTreeProducerException,
-            DOMDataTreeShardCreationFailedException {
+            throws DOMDataTreeShardingConflictException, DOMDataTreeProducerException {
+        final DOMDataTreePrefixTableEntry<DOMDataTreeShardRegistration<DOMDataTreeShard>> lookup =
+                shards.lookup(prefix);
+        if (lookup != null && lookup.getValue().getPrefix().equals(prefix)) {
+            throw new DOMDataTreeShardingConflictException(
+                    "Prefix " + prefix + " is already occupied by another shard.");
+        }
+
+        PrefixShardConfiguration config = new PrefixShardConfiguration(prefix, "prefix", replicaMembers);
+        shardedDataTreeActor.tell(new CreatePrefixShard(config), noSender());
+
+        return new DistributedShardRegistrationImpl(prefix, shardedDataTreeActor, this);
+    }
+
+    void resolveShardAdditions(final Set<DOMDataTreeIdentifier> additions) {
+        LOG.debug("Member {}: Resolving additions : {}", memberName, additions);
+        final ArrayList<DOMDataTreeIdentifier> list = new ArrayList<>(additions);
+        // we need to register the shards from top to bottom, so we need to atleast make sure the ordering reflects that
+        Collections.sort(list, (o1, o2) -> {
+            if (o1.getRootIdentifier().getPathArguments().size() < o2.getRootIdentifier().getPathArguments().size()) {
+                return -1;
+            } else if (o1.getRootIdentifier().getPathArguments().size()
+                    == o2.getRootIdentifier().getPathArguments().size()) {
+                return 0;
+            } else {
+                return 1;
+            }
+        });
+        list.forEach(this::createShardFrontend);
+    }
 
+    void resolveShardRemovals(final Set<DOMDataTreeIdentifier> removals) {
+        LOG.debug("Member {}: Resolving removals : {}", memberName, removals);
+
+        // do we need to go from bottom to top?
+        removals.forEach(this::despawnShardFrontend);
+    }
+
+    private void createShardFrontend(final DOMDataTreeIdentifier prefix) {
+        LOG.debug("Member {}: Creating CDS shard for prefix: {}", memberName, prefix);
         final String shardName = ClusterUtils.getCleanShardName(prefix.getRootIdentifier());
         final DistributedDataStore distributedDataStore =
                 prefix.getDatastoreType().equals(org.opendaylight.mdsal.common.api.LogicalDatastoreType.CONFIGURATION)
                         ? distributedConfigDatastore : distributedOperDatastore;
 
-        final PrefixShardConfiguration config = new PrefixShardConfiguration(prefix, "prefix", replicaMembers);
-        if (replicaMembers.contains(memberName)) {
-            // spawn the backend shard and have the shard Manager create all replicas
-            final ActorRef shardManager = distributedDataStore.getActorContext().getShardManager();
-
-            shardManager.tell(new CreatePrefixedShard(config, null, Shard.builder()), noSender());
-        }
-
-        LOG.debug("Creating distributed datastore client for shard {}", shardName);
-        final Props distributedDataStoreClientProps =
-                SimpleDataStoreClientActor
-                        .props(memberName, "Shard-" + shardName, distributedDataStore.getActorContext(), shardName);
+        try (final DOMDataTreeProducer producer = localCreateProducer(Collections.singletonList(prefix))) {
+            final Entry<DataStoreClient, ActorRef> entry =
+                    createDatastoreClient(shardName, distributedDataStore.getActorContext());
 
-        final ActorRef clientActor = actorSystem.actorOf(distributedDataStoreClientProps);
-        final DataStoreClient client;
-        try {
-            client = SimpleDataStoreClientActor.getDistributedDataStoreClient(clientActor, 30, TimeUnit.SECONDS);
-        } catch (final Exception e) {
-            LOG.error("Failed to get actor for {}", distributedDataStoreClientProps, e);
-            clientActor.tell(PoisonPill.getInstance(), noSender());
-            throw new DOMDataTreeProducerException("Unable to create producer", e);
-        }
+            final DistributedShardFrontend shard =
+                    new DistributedShardFrontend(distributedDataStore, entry.getKey(), prefix);
 
-        // register the frontend into the sharding service and let the actor distribute this onto the other nodes
-        final ListenerRegistration<ShardFrontend> shardFrontendRegistration;
-        try (DOMDataTreeProducer producer = createProducer(Collections.singletonList(prefix))) {
-            shardFrontendRegistration = shardedDOMDataTree
-                    .registerDataTreeShard(prefix,
-                            new ShardFrontend(client, prefix),
-                            ((ProxyProducer) producer).delegate());
+            @SuppressWarnings("unchecked")
+            final DOMDataTreeShardRegistration<DOMDataTreeShard> reg =
+                    (DOMDataTreeShardRegistration) shardedDOMDataTree.registerDataTreeShard(prefix, shard, producer);
+            shards.store(prefix, reg);
+        } catch (final DOMDataTreeShardingConflictException e) {
+            LOG.error("Prefix {} is already occupied by another shard", prefix, e);
+        } catch (DOMDataTreeProducerException e) {
+            LOG.error("Unable to close producer", e);
+        } catch (DOMDataTreeShardCreationFailedException e) {
+            LOG.error("Unable to create datastore client for shard {}", prefix, e);
         }
+    }
 
-        final Future<Object> future = distributedDataStore.getActorContext()
-                .executeOperationAsync(shardedDataTreeActor, new PrefixShardCreated(config), DEFAULT_ASK_TIMEOUT);
-        try {
-            final Object result = Await.result(future, DEFAULT_ASK_TIMEOUT.duration());
-            if (result != null) {
-                throw new DOMDataTreeShardCreationFailedException("Received unexpected response to PrefixShardCreated"
-                        + result);
-            }
-
-            return new DistributedShardRegistrationImpl(shardFrontendRegistration, prefix, shardedDataTreeActor);
-        } catch (final CompletionException e) {
-            shardedDataTreeActor.tell(new PrefixShardRemoved(prefix), noSender());
-            clientActor.tell(PoisonPill.getInstance(), noSender());
+    private void despawnShardFrontend(final DOMDataTreeIdentifier prefix) {
+        LOG.debug("Member {}: Removing CDS shard for prefix: {}", memberName, prefix);
+        final DOMDataTreePrefixTableEntry<DOMDataTreeShardRegistration<DOMDataTreeShard>> lookup =
+                shards.lookup(prefix);
 
-            final Throwable cause = e.getCause();
-            if (cause instanceof DOMDataTreeShardingConflictException) {
-                throw (DOMDataTreeShardingConflictException) cause;
-            }
+        if (lookup == null || !lookup.getValue().getPrefix().equals(prefix)) {
+            LOG.debug("Member {}: Received despawn for non-existing CDS shard frontend, prefix: {}, ignoring..",
+                    memberName, prefix);
+            return;
+        }
 
-            throw new DOMDataTreeShardCreationFailedException("Shard creation failed.", e.getCause());
-        } catch (final Exception e) {
-            shardedDataTreeActor.tell(new PrefixShardRemoved(prefix), noSender());
-            clientActor.tell(PoisonPill.getInstance(), noSender());
+        lookup.getValue().close();
+        // need to remove from our local table thats used for tracking
+        shards.remove(prefix);
+    }
 
-            throw new DOMDataTreeShardCreationFailedException("Shard creation failed.", e);
-        }
+    DOMDataTreeProducer localCreateProducer(final Collection<DOMDataTreeIdentifier> prefix) {
+        return shardedDOMDataTree.createProducer(prefix);
     }
 
     @Nonnull
@@ -216,6 +265,38 @@ public class DistributedShardedDOMDataTree implements DOMDataTreeService, DOMDat
         return shardedDOMDataTree.registerDataTreeShard(prefix, shard, producer);
     }
 
+    @SuppressWarnings("checkstyle:IllegalCatch")
+    private Entry<DataStoreClient, ActorRef> createDatastoreClient(
+            final String shardName, final ActorContext actorContext)
+            throws DOMDataTreeShardCreationFailedException {
+
+        LOG.debug("Creating distributed datastore client for shard {}", shardName);
+        final Props distributedDataStoreClientProps =
+                SimpleDataStoreClientActor.props(memberName, "Shard-" + shardName, actorContext, shardName);
+
+        final ActorRef clientActor = actorSystem.actorOf(distributedDataStoreClientProps);
+        try {
+            return new SimpleEntry<>(SimpleDataStoreClientActor
+                    .getDistributedDataStoreClient(clientActor, 30, TimeUnit.SECONDS), clientActor);
+        } catch (final Exception e) {
+            LOG.error("Failed to get actor for {}", distributedDataStoreClientProps, e);
+            clientActor.tell(PoisonPill.getInstance(), noSender());
+            throw new DOMDataTreeShardCreationFailedException(
+                    "Unable to create datastore client for shard{" + shardName + "}", e);
+        }
+    }
+
+    private DistributedShardRegistration initDefaultShard(final LogicalDatastoreType logicalDatastoreType)
+            throws DOMDataTreeProducerException, DOMDataTreeShardingConflictException {
+        final Collection<Member> members = JavaConverters.asJavaCollectionConverter(
+            Cluster.get(actorSystem).state().members()).asJavaCollection();
+        final Collection<MemberName> names = Collections2.transform(members,
+            m -> MemberName.forName(m.roles().iterator().next()));
+
+        return createDistributedShard(
+                new DOMDataTreeIdentifier(logicalDatastoreType, YangInstanceIdentifier.EMPTY), names);
+    }
+
     private static void closeProducer(final DOMDataTreeProducer producer) {
         try {
             producer.close();
@@ -246,24 +327,25 @@ public class DistributedShardedDOMDataTree implements DOMDataTreeService, DOMDat
     }
 
     private static class DistributedShardRegistrationImpl implements DistributedShardRegistration {
-        private final ListenerRegistration<ShardFrontend> registration;
+
         private final DOMDataTreeIdentifier prefix;
         private final ActorRef shardedDataTreeActor;
+        private final DistributedShardedDOMDataTree distributedShardedDOMDataTree;
 
-        DistributedShardRegistrationImpl(final ListenerRegistration<ShardFrontend> registration,
-                                         final DOMDataTreeIdentifier prefix,
-                                         final ActorRef shardedDataTreeActor) {
-            this.registration = registration;
+        DistributedShardRegistrationImpl(final DOMDataTreeIdentifier prefix,
+                                         final ActorRef shardedDataTreeActor,
+                                         final DistributedShardedDOMDataTree distributedShardedDOMDataTree) {
             this.prefix = prefix;
             this.shardedDataTreeActor = shardedDataTreeActor;
+            this.distributedShardedDOMDataTree = distributedShardedDOMDataTree;
         }
 
         @Override
         public void close() {
-            // TODO send the correct messages to ShardManager to destroy the shard
-            // maybe we could provide replica removal mechanisms also?
-            shardedDataTreeActor.tell(new PrefixShardRemoved(prefix), noSender());
-            registration.close();
+            // first despawn on the local node
+            distributedShardedDOMDataTree.despawnShardFrontend(prefix);
+            // update the config so the remote nodes are updated
+            shardedDataTreeActor.tell(new RemovePrefixShard(prefix), noSender());
         }
     }