BUG 2138: Introduce DistributedShardedDOMDataTree
[controller.git] / opendaylight / md-sal / sal-distributed-datastore / src / main / java / org / opendaylight / controller / cluster / sharding / DistributedShardedDOMDataTree.java
1 /*
2  * Copyright (c) 2016 Cisco Systems, Inc. and others.  All rights reserved.
3  *
4  * This program and the accompanying materials are made available under the
5  * terms of the Eclipse Public License v1.0 which accompanies this distribution,
6  * and is available at http://www.eclipse.org/legal/epl-v10.html
7  */
8
9 package org.opendaylight.controller.cluster.sharding;
10
11 import static akka.actor.ActorRef.noSender;
12
13 import akka.actor.ActorRef;
14 import akka.actor.ActorSystem;
15 import akka.actor.PoisonPill;
16 import akka.actor.Props;
17 import akka.util.Timeout;
18 import com.google.common.base.Preconditions;
19 import com.google.common.base.Throwables;
20 import com.google.common.collect.ForwardingObject;
21 import com.google.common.util.concurrent.Uninterruptibles;
22 import java.util.Collection;
23 import java.util.Collections;
24 import java.util.concurrent.CompletionException;
25 import java.util.concurrent.TimeUnit;
26 import javax.annotation.Nonnull;
27 import org.opendaylight.controller.cluster.access.concepts.MemberName;
28 import org.opendaylight.controller.cluster.databroker.actors.dds.DataStoreClient;
29 import org.opendaylight.controller.cluster.databroker.actors.dds.SimpleDataStoreClientActor;
30 import org.opendaylight.controller.cluster.datastore.DistributedDataStore;
31 import org.opendaylight.controller.cluster.datastore.Shard;
32 import org.opendaylight.controller.cluster.datastore.config.PrefixShardConfiguration;
33 import org.opendaylight.controller.cluster.datastore.messages.CreatePrefixedShard;
34 import org.opendaylight.controller.cluster.datastore.utils.ActorContext;
35 import org.opendaylight.controller.cluster.datastore.utils.ClusterUtils;
36 import org.opendaylight.controller.cluster.sharding.ShardedDataTreeActor.ShardedDataTreeActorCreator;
37 import org.opendaylight.controller.cluster.sharding.messages.PrefixShardCreated;
38 import org.opendaylight.controller.cluster.sharding.messages.PrefixShardRemoved;
39 import org.opendaylight.controller.cluster.sharding.messages.ProducerCreated;
40 import org.opendaylight.controller.cluster.sharding.messages.ProducerRemoved;
41 import org.opendaylight.mdsal.dom.api.DOMDataTreeCursorAwareTransaction;
42 import org.opendaylight.mdsal.dom.api.DOMDataTreeIdentifier;
43 import org.opendaylight.mdsal.dom.api.DOMDataTreeListener;
44 import org.opendaylight.mdsal.dom.api.DOMDataTreeLoopException;
45 import org.opendaylight.mdsal.dom.api.DOMDataTreeProducer;
46 import org.opendaylight.mdsal.dom.api.DOMDataTreeProducerException;
47 import org.opendaylight.mdsal.dom.api.DOMDataTreeService;
48 import org.opendaylight.mdsal.dom.api.DOMDataTreeShard;
49 import org.opendaylight.mdsal.dom.api.DOMDataTreeShardingConflictException;
50 import org.opendaylight.mdsal.dom.api.DOMDataTreeShardingService;
51 import org.opendaylight.mdsal.dom.broker.ShardedDOMDataTree;
52 import org.opendaylight.yangtools.concepts.ListenerRegistration;
53 import org.slf4j.Logger;
54 import org.slf4j.LoggerFactory;
55 import scala.concurrent.Await;
56 import scala.concurrent.Future;
57
58 /**
59  * A layer on top of DOMDataTreeService that distributes producer/shard registrations to remote nodes via
60  * {@link ShardedDataTreeActor}. Also provides QoL method for addition of prefix based clustered shard into the system.
61  */
62 public class DistributedShardedDOMDataTree implements DOMDataTreeService, DOMDataTreeShardingService,
63         DistributedShardFactory {
64
65     private static final Logger LOG = LoggerFactory.getLogger(DistributedShardedDOMDataTree.class);
66
67     private static final Timeout DEFAULT_ASK_TIMEOUT = new Timeout(15, TimeUnit.SECONDS);
68     private static final int MAX_ACTOR_CREATION_RETRIES = 100;
69     private static final int ACTOR_RETRY_DELAY = 100;
70     private static final TimeUnit ACTOR_RETRY_TIME_UNIT = TimeUnit.MILLISECONDS;
71
72     static final String ACTOR_ID = "ShardedDOMDataTreeFrontend";
73
74     private final ShardedDOMDataTree shardedDOMDataTree;
75     private final ActorSystem actorSystem;
76     private final DistributedDataStore distributedOperDatastore;
77     private final DistributedDataStore distributedConfigDatastore;
78
79     private final ActorRef shardedDataTreeActor;
80     private final MemberName memberName;
81
82     public DistributedShardedDOMDataTree(final ActorSystem actorSystem,
83                                          final DistributedDataStore distributedOperDatastore,
84                                          final DistributedDataStore distributedConfigDatastore) {
85         this.actorSystem = Preconditions.checkNotNull(actorSystem);
86         this.distributedOperDatastore = Preconditions.checkNotNull(distributedOperDatastore);
87         this.distributedConfigDatastore = Preconditions.checkNotNull(distributedConfigDatastore);
88         shardedDOMDataTree = new ShardedDOMDataTree();
89
90         shardedDataTreeActor = createShardedDataTreeActor(actorSystem,
91                 new ShardedDataTreeActorCreator()
92                         .setDataTreeService(shardedDOMDataTree)
93                         .setShardingService(shardedDOMDataTree)
94                         .setActorSystem(actorSystem)
95                         .setClusterWrapper(distributedConfigDatastore.getActorContext().getClusterWrapper())
96                         .setDistributedConfigDatastore(distributedConfigDatastore)
97                         .setDistributedOperDatastore(distributedOperDatastore),
98                 ACTOR_ID);
99
100         this.memberName = distributedConfigDatastore.getActorContext().getCurrentMemberName();
101     }
102
103     @Nonnull
104     @Override
105     public <T extends DOMDataTreeListener> ListenerRegistration<T> registerListener(
106             final T listener, final Collection<DOMDataTreeIdentifier> subtrees,
107             final boolean allowRxMerges, final Collection<DOMDataTreeProducer> producers)
108             throws DOMDataTreeLoopException {
109
110         throw new UnsupportedOperationException("Not implemented");
111     }
112
113     @Nonnull
114     @Override
115     public DOMDataTreeProducer createProducer(@Nonnull final Collection<DOMDataTreeIdentifier> subtrees) {
116         LOG.debug("Creating producer for {}", subtrees);
117         final DOMDataTreeProducer producer = shardedDOMDataTree.createProducer(subtrees);
118
119         final Object response = distributedConfigDatastore.getActorContext()
120                 .executeOperation(shardedDataTreeActor, new ProducerCreated(subtrees));
121         if (response == null) {
122             LOG.debug("Received success from remote nodes, creating producer:{}", subtrees);
123             return new ProxyProducer(producer, subtrees, shardedDataTreeActor,
124                     distributedConfigDatastore.getActorContext());
125         } else if (response instanceof Exception) {
126             closeProducer(producer);
127             throw Throwables.propagate((Exception) response);
128         } else {
129             closeProducer(producer);
130             throw new RuntimeException("Unexpected response to create producer received." + response);
131         }
132     }
133
134     @Override
135     @SuppressWarnings("checkstyle:IllegalCatch")
136     public DistributedShardRegistration createDistributedShard(
137             final DOMDataTreeIdentifier prefix, final Collection<MemberName> replicaMembers)
138             throws DOMDataTreeShardingConflictException, DOMDataTreeProducerException,
139             DOMDataTreeShardCreationFailedException {
140
141         final String shardName = ClusterUtils.getCleanShardName(prefix.getRootIdentifier());
142         final DistributedDataStore distributedDataStore =
143                 prefix.getDatastoreType().equals(org.opendaylight.mdsal.common.api.LogicalDatastoreType.CONFIGURATION)
144                         ? distributedConfigDatastore : distributedOperDatastore;
145
146         final PrefixShardConfiguration config = new PrefixShardConfiguration(prefix, "prefix", replicaMembers);
147         if (replicaMembers.contains(memberName)) {
148             // spawn the backend shard and have the shard Manager create all replicas
149             final ActorRef shardManager = distributedDataStore.getActorContext().getShardManager();
150
151             shardManager.tell(new CreatePrefixedShard(config, null, Shard.builder()), noSender());
152         }
153
154         LOG.debug("Creating distributed datastore client for shard {}", shardName);
155         final Props distributedDataStoreClientProps =
156                 SimpleDataStoreClientActor
157                         .props(memberName, "Shard-" + shardName, distributedDataStore.getActorContext(), shardName);
158
159         final ActorRef clientActor = actorSystem.actorOf(distributedDataStoreClientProps);
160         final DataStoreClient client;
161         try {
162             client = SimpleDataStoreClientActor.getDistributedDataStoreClient(clientActor, 30, TimeUnit.SECONDS);
163         } catch (final Exception e) {
164             LOG.error("Failed to get actor for {}", distributedDataStoreClientProps, e);
165             clientActor.tell(PoisonPill.getInstance(), noSender());
166             throw new DOMDataTreeProducerException("Unable to create producer", e);
167         }
168
169         // register the frontend into the sharding service and let the actor distribute this onto the other nodes
170         final ListenerRegistration<ShardFrontend> shardFrontendRegistration;
171         try (DOMDataTreeProducer producer = createProducer(Collections.singletonList(prefix))) {
172             shardFrontendRegistration = shardedDOMDataTree
173                     .registerDataTreeShard(prefix,
174                             new ShardFrontend(client, prefix),
175                             ((ProxyProducer) producer).delegate());
176         }
177
178         final Future<Object> future = distributedDataStore.getActorContext()
179                 .executeOperationAsync(shardedDataTreeActor, new PrefixShardCreated(config), DEFAULT_ASK_TIMEOUT);
180         try {
181             final Object result = Await.result(future, DEFAULT_ASK_TIMEOUT.duration());
182             if (result != null) {
183                 throw new DOMDataTreeShardCreationFailedException("Received unexpected response to PrefixShardCreated"
184                         + result);
185             }
186
187             return new DistributedShardRegistrationImpl(shardFrontendRegistration, prefix, shardedDataTreeActor);
188         } catch (final CompletionException e) {
189             shardedDataTreeActor.tell(new PrefixShardRemoved(prefix), noSender());
190             clientActor.tell(PoisonPill.getInstance(), noSender());
191
192             final Throwable cause = e.getCause();
193             if (cause instanceof DOMDataTreeShardingConflictException) {
194                 throw (DOMDataTreeShardingConflictException) cause;
195             }
196
197             throw new DOMDataTreeShardCreationFailedException("Shard creation failed.", e.getCause());
198         } catch (final Exception e) {
199             shardedDataTreeActor.tell(new PrefixShardRemoved(prefix), noSender());
200             clientActor.tell(PoisonPill.getInstance(), noSender());
201
202             throw new DOMDataTreeShardCreationFailedException("Shard creation failed.", e);
203         }
204     }
205
206     @Nonnull
207     @Override
208     public <T extends DOMDataTreeShard> ListenerRegistration<T> registerDataTreeShard(
209             @Nonnull final DOMDataTreeIdentifier prefix,
210             @Nonnull final T shard,
211             @Nonnull final DOMDataTreeProducer producer)
212             throws DOMDataTreeShardingConflictException {
213
214         LOG.debug("Registering shard[{}] at prefix: {}", shard, prefix);
215
216         return shardedDOMDataTree.registerDataTreeShard(prefix, shard, producer);
217     }
218
219     private static void closeProducer(final DOMDataTreeProducer producer) {
220         try {
221             producer.close();
222         } catch (final DOMDataTreeProducerException e) {
223             LOG.error("Unable to close producer", e);
224         }
225     }
226
227     @SuppressWarnings("checkstyle:IllegalCatch")
228     private static ActorRef createShardedDataTreeActor(final ActorSystem actorSystem,
229                                                        final ShardedDataTreeActorCreator creator,
230                                                        final String shardDataTreeActorId) {
231         Exception lastException = null;
232
233         for (int i = 0; i < MAX_ACTOR_CREATION_RETRIES; i++) {
234             try {
235                 return actorSystem.actorOf(creator.props(), shardDataTreeActorId);
236             } catch (final Exception e) {
237                 lastException = e;
238                 Uninterruptibles.sleepUninterruptibly(ACTOR_RETRY_DELAY, ACTOR_RETRY_TIME_UNIT);
239                 LOG.debug("Could not create actor {} because of {} -"
240                                 + " waiting for sometime before retrying (retry count = {})",
241                         shardDataTreeActorId, e.getMessage(), i);
242             }
243         }
244
245         throw new IllegalStateException("Failed to create actor for ShardedDOMDataTree", lastException);
246     }
247
248     private static class DistributedShardRegistrationImpl implements DistributedShardRegistration {
249         private final ListenerRegistration<ShardFrontend> registration;
250         private final DOMDataTreeIdentifier prefix;
251         private final ActorRef shardedDataTreeActor;
252
253         DistributedShardRegistrationImpl(final ListenerRegistration<ShardFrontend> registration,
254                                          final DOMDataTreeIdentifier prefix,
255                                          final ActorRef shardedDataTreeActor) {
256             this.registration = registration;
257             this.prefix = prefix;
258             this.shardedDataTreeActor = shardedDataTreeActor;
259         }
260
261         @Override
262         public void close() {
263             // TODO send the correct messages to ShardManager to destroy the shard
264             // maybe we could provide replica removal mechanisms also?
265             shardedDataTreeActor.tell(new PrefixShardRemoved(prefix), noSender());
266             registration.close();
267         }
268     }
269
270     private static final class ProxyProducer extends ForwardingObject implements DOMDataTreeProducer {
271
272         private final DOMDataTreeProducer delegate;
273         private final Collection<DOMDataTreeIdentifier> subtrees;
274         private final ActorRef shardDataTreeActor;
275         private final ActorContext actorContext;
276
277         ProxyProducer(final DOMDataTreeProducer delegate,
278                       final Collection<DOMDataTreeIdentifier> subtrees,
279                       final ActorRef shardDataTreeActor,
280                       final ActorContext actorContext) {
281             this.delegate = Preconditions.checkNotNull(delegate);
282             this.subtrees = Preconditions.checkNotNull(subtrees);
283             this.shardDataTreeActor = Preconditions.checkNotNull(shardDataTreeActor);
284             this.actorContext = Preconditions.checkNotNull(actorContext);
285         }
286
287         @Nonnull
288         @Override
289         public DOMDataTreeCursorAwareTransaction createTransaction(final boolean isolated) {
290             return delegate.createTransaction(isolated);
291         }
292
293         @Nonnull
294         @Override
295         public DOMDataTreeProducer createProducer(@Nonnull final Collection<DOMDataTreeIdentifier> subtrees) {
296             // TODO we probably don't need to distribute this on the remote nodes since once we have this producer
297             // open we surely have the rights to all the subtrees.
298             return delegate.createProducer(subtrees);
299         }
300
301         @Override
302         public void close() throws DOMDataTreeProducerException {
303             delegate.close();
304
305             final Object o = actorContext.executeOperation(shardDataTreeActor, new ProducerRemoved(subtrees));
306             if (o instanceof DOMDataTreeProducerException) {
307                 throw ((DOMDataTreeProducerException) o);
308             } else if (o instanceof Throwable) {
309                 throw new DOMDataTreeProducerException("Unable to close producer", (Throwable) o);
310             }
311         }
312
313         @Override
314         protected DOMDataTreeProducer delegate() {
315             return delegate;
316         }
317     }
318 }