Bump upstreams
[controller.git] / opendaylight / md-sal / sal-distributed-datastore / src / main / java / org / opendaylight / controller / cluster / datastore / AbstractDataStore.java
1 /*
2  * Copyright (c) 2014 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 package org.opendaylight.controller.cluster.datastore;
9
10 import static java.util.Objects.requireNonNull;
11
12 import akka.actor.ActorRef;
13 import akka.actor.ActorSystem;
14 import akka.actor.PoisonPill;
15 import akka.actor.Props;
16 import com.google.common.annotations.Beta;
17 import com.google.common.annotations.VisibleForTesting;
18 import com.google.common.base.Throwables;
19 import com.google.common.util.concurrent.ListenableFuture;
20 import com.google.common.util.concurrent.SettableFuture;
21 import com.google.common.util.concurrent.Uninterruptibles;
22 import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
23 import java.util.List;
24 import java.util.concurrent.ExecutionException;
25 import java.util.concurrent.TimeUnit;
26 import java.util.concurrent.TimeoutException;
27 import org.eclipse.jdt.annotation.NonNull;
28 import org.opendaylight.controller.cluster.access.concepts.ClientIdentifier;
29 import org.opendaylight.controller.cluster.common.actor.Dispatchers;
30 import org.opendaylight.controller.cluster.databroker.actors.dds.DataStoreClient;
31 import org.opendaylight.controller.cluster.databroker.actors.dds.DistributedDataStoreClientActor;
32 import org.opendaylight.controller.cluster.datastore.config.Configuration;
33 import org.opendaylight.controller.cluster.datastore.identifiers.ShardManagerIdentifier;
34 import org.opendaylight.controller.cluster.datastore.persisted.DatastoreSnapshot;
35 import org.opendaylight.controller.cluster.datastore.shardmanager.AbstractShardManagerCreator;
36 import org.opendaylight.controller.cluster.datastore.shardmanager.ShardManagerCreator;
37 import org.opendaylight.controller.cluster.datastore.utils.ActorUtils;
38 import org.opendaylight.controller.cluster.datastore.utils.PrimaryShardInfoFutureCache;
39 import org.opendaylight.mdsal.dom.api.DOMDataBroker.CommitCohortExtension;
40 import org.opendaylight.mdsal.dom.api.DOMDataTreeChangeListener;
41 import org.opendaylight.mdsal.dom.api.DOMDataTreeCommitCohort;
42 import org.opendaylight.mdsal.dom.api.DOMDataTreeIdentifier;
43 import org.opendaylight.mdsal.dom.spi.store.DOMStoreTreeChangePublisher;
44 import org.opendaylight.yangtools.concepts.Registration;
45 import org.opendaylight.yangtools.yang.common.Empty;
46 import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier;
47 import org.opendaylight.yangtools.yang.data.tree.api.DataTreeCandidate;
48 import org.opendaylight.yangtools.yang.model.api.EffectiveModelContext;
49 import org.slf4j.Logger;
50 import org.slf4j.LoggerFactory;
51 import scala.concurrent.duration.Duration;
52
53 /**
54  * Base implementation of a distributed DOMStore.
55  */
56 public abstract class AbstractDataStore implements DistributedDataStoreInterface,
57         DatastoreContextPropertiesUpdater.Listener, DOMStoreTreeChangePublisher, CommitCohortExtension,
58         AutoCloseable {
59     private static final Logger LOG = LoggerFactory.getLogger(AbstractDataStore.class);
60
61     private final SettableFuture<Empty> readinessFuture = SettableFuture.create();
62     private final ClientIdentifier identifier;
63     private final DataStoreClient client;
64     private final ActorUtils actorUtils;
65
66     private AutoCloseable closeable;
67     private DatastoreConfigurationMXBeanImpl datastoreConfigMXBean;
68     private DatastoreInfoMXBeanImpl datastoreInfoMXBean;
69
70     @SuppressWarnings("checkstyle:IllegalCatch")
71     @SuppressFBWarnings(value = "MC_OVERRIDABLE_METHOD_CALL_IN_CONSTRUCTOR", justification = "Testing overrides")
72     protected AbstractDataStore(final ActorSystem actorSystem, final ClusterWrapper cluster,
73             final Configuration configuration, final DatastoreContextFactory datastoreContextFactory,
74             final DatastoreSnapshot restoreFromSnapshot) {
75         requireNonNull(actorSystem, "actorSystem should not be null");
76         requireNonNull(cluster, "cluster should not be null");
77         requireNonNull(configuration, "configuration should not be null");
78         requireNonNull(datastoreContextFactory, "datastoreContextFactory should not be null");
79
80         String shardManagerId = ShardManagerIdentifier.builder()
81                 .type(datastoreContextFactory.getBaseDatastoreContext().getDataStoreName()).build().toString();
82
83         LOG.info("Creating ShardManager : {}", shardManagerId);
84
85         String shardDispatcher =
86                 new Dispatchers(actorSystem.dispatchers()).getDispatcherPath(Dispatchers.DispatcherType.Shard);
87
88         PrimaryShardInfoFutureCache primaryShardInfoCache = new PrimaryShardInfoFutureCache();
89
90         AbstractShardManagerCreator<?> creator = getShardManagerCreator().cluster(cluster).configuration(configuration)
91                 .datastoreContextFactory(datastoreContextFactory)
92                 .readinessFuture(readinessFuture)
93                 .primaryShardInfoCache(primaryShardInfoCache)
94                 .restoreFromSnapshot(restoreFromSnapshot)
95                 .distributedDataStore(this);
96
97         actorUtils = new ActorUtils(actorSystem, createShardManager(actorSystem, creator, shardDispatcher,
98                 shardManagerId), cluster, configuration, datastoreContextFactory.getBaseDatastoreContext(),
99                 primaryShardInfoCache);
100
101         final Props clientProps = DistributedDataStoreClientActor.props(cluster.getCurrentMemberName(),
102             datastoreContextFactory.getBaseDatastoreContext().getDataStoreName(), actorUtils);
103         final ActorRef clientActor = actorSystem.actorOf(clientProps);
104         try {
105             client = DistributedDataStoreClientActor.getDistributedDataStoreClient(clientActor, 30, TimeUnit.SECONDS);
106         } catch (Exception e) {
107             LOG.error("Failed to get actor for {}", clientProps, e);
108             clientActor.tell(PoisonPill.getInstance(), ActorRef.noSender());
109             Throwables.throwIfUnchecked(e);
110             throw new IllegalStateException(e);
111         }
112
113         identifier = client.getIdentifier();
114         LOG.debug("Distributed data store client {} started", identifier);
115
116         datastoreConfigMXBean = new DatastoreConfigurationMXBeanImpl(
117                 datastoreContextFactory.getBaseDatastoreContext().getDataStoreMXBeanType());
118         datastoreConfigMXBean.setContext(datastoreContextFactory.getBaseDatastoreContext());
119         datastoreConfigMXBean.registerMBean();
120
121         datastoreInfoMXBean = new DatastoreInfoMXBeanImpl(datastoreContextFactory.getBaseDatastoreContext()
122                 .getDataStoreMXBeanType(), actorUtils);
123         datastoreInfoMXBean.registerMBean();
124     }
125
126     @VisibleForTesting
127     protected AbstractDataStore(final ActorUtils actorUtils, final ClientIdentifier identifier,
128                                 final DataStoreClient clientActor) {
129         this.actorUtils = requireNonNull(actorUtils, "actorContext should not be null");
130         client = clientActor;
131         this.identifier = requireNonNull(identifier);
132     }
133
134     protected AbstractShardManagerCreator<?> getShardManagerCreator() {
135         return new ShardManagerCreator();
136     }
137
138     protected final DataStoreClient getClient() {
139         return client;
140     }
141
142     final ClientIdentifier getIdentifier() {
143         return identifier;
144     }
145
146     public void setCloseable(final AutoCloseable closeable) {
147         this.closeable = closeable;
148     }
149
150     @Override
151     public Registration registerTreeChangeListener(final YangInstanceIdentifier treeId,
152             final DOMDataTreeChangeListener listener) {
153         return registerTreeChangeListener(treeId, listener, true);
154     }
155
156     private @NonNull Registration registerTreeChangeListener(final YangInstanceIdentifier treeId,
157             final DOMDataTreeChangeListener listener, final boolean clustered) {
158         requireNonNull(treeId, "treeId should not be null");
159         requireNonNull(listener, "listener should not be null");
160
161         /*
162          * We need to potentially deal with multi-shard composition for registration targeting the root of the data
163          * store. If that is the case, we delegate to a more complicated setup invol
164          */
165         if (treeId.isEmpty()) {
166             // User is targeting root of the datastore. If there is more than one shard, we have to register with them
167             // all and perform data composition.
168             final var shardNames = actorUtils.getConfiguration().getAllShardNames();
169             if (shardNames.size() > 1) {
170                 if (!clustered) {
171                     throw new IllegalArgumentException(
172                         "Cannot listen on root without non-clustered listener " + listener);
173                 }
174                 return new RootDataTreeChangeListenerProxy<>(actorUtils, listener, shardNames);
175             }
176         }
177
178         final var shardName = actorUtils.getShardStrategyFactory().getStrategy(treeId).findShard(treeId);
179         LOG.debug("Registering tree listener: {} for tree: {} shard: {}", listener, treeId, shardName);
180
181         return DataTreeChangeListenerProxy.of(actorUtils, listener, treeId, clustered, shardName);
182     }
183
184     @Override
185     @Deprecated(since = "9.0.0", forRemoval = true)
186     public Registration registerLegacyTreeChangeListener(final YangInstanceIdentifier treeId,
187             final DOMDataTreeChangeListener listener) {
188         return registerTreeChangeListener(treeId, listener, false);
189     }
190
191     @Override
192     public Registration registerCommitCohort(final DOMDataTreeIdentifier subtree,
193             final DOMDataTreeCommitCohort cohort) {
194         YangInstanceIdentifier treeId = requireNonNull(subtree, "subtree should not be null").path();
195         requireNonNull(cohort, "listener should not be null");
196
197
198         final String shardName = actorUtils.getShardStrategyFactory().getStrategy(treeId).findShard(treeId);
199         LOG.debug("Registering cohort: {} for tree: {} shard: {}", cohort, treeId, shardName);
200
201         final var cohortProxy = new DataTreeCohortRegistrationProxy<>(actorUtils, subtree, cohort);
202         cohortProxy.init(shardName);
203         return cohortProxy;
204     }
205
206     public void onModelContextUpdated(final EffectiveModelContext newModelContext) {
207         actorUtils.setSchemaContext(newModelContext);
208     }
209
210     @Override
211     public void onDatastoreContextUpdated(final DatastoreContextFactory contextFactory) {
212         LOG.info("DatastoreContext updated for data store {}", actorUtils.getDataStoreName());
213
214         actorUtils.setDatastoreContext(contextFactory);
215         datastoreConfigMXBean.setContext(contextFactory.getBaseDatastoreContext());
216     }
217
218     @Override
219     @SuppressWarnings("checkstyle:IllegalCatch")
220     public void close() {
221         LOG.info("Closing data store {}", identifier);
222
223         if (datastoreConfigMXBean != null) {
224             datastoreConfigMXBean.unregisterMBean();
225         }
226         if (datastoreInfoMXBean != null) {
227             datastoreInfoMXBean.unregisterMBean();
228         }
229
230         if (closeable != null) {
231             try {
232                 closeable.close();
233             } catch (Exception e) {
234                 LOG.debug("Error closing instance", e);
235             }
236         }
237
238         actorUtils.shutdown();
239
240         if (client != null) {
241             client.close();
242         }
243     }
244
245     @Override
246     public final ActorUtils getActorUtils() {
247         return actorUtils;
248     }
249
250     // TODO: consider removing this in favor of awaitReadiness()
251     @Deprecated
252     public void waitTillReady() {
253         LOG.info("Beginning to wait for data store to become ready : {}", identifier);
254
255         final Duration toWait = initialSettleTime();
256         try {
257             if (!awaitReadiness(toWait)) {
258                 LOG.error("Shard leaders failed to settle in {}, giving up", toWait);
259                 return;
260             }
261         } catch (InterruptedException e) {
262             LOG.error("Interrupted while waiting for shards to settle", e);
263             return;
264         }
265
266         LOG.debug("Data store {} is now ready", identifier);
267     }
268
269     @Beta
270     @Deprecated
271     public boolean awaitReadiness() throws InterruptedException {
272         return awaitReadiness(initialSettleTime());
273     }
274
275     @Beta
276     @Deprecated
277     public boolean awaitReadiness(final Duration toWait) throws InterruptedException {
278         try {
279             if (toWait.isFinite()) {
280                 try {
281                     readinessFuture.get(toWait.toNanos(), TimeUnit.NANOSECONDS);
282                 } catch (TimeoutException e) {
283                     LOG.debug("Timed out waiting for shards to settle", e);
284                     return false;
285                 }
286             } else {
287                 readinessFuture.get();
288             }
289         } catch (ExecutionException e) {
290             LOG.warn("Unexpected readiness failure, assuming convergence", e);
291         }
292
293         return true;
294     }
295
296     @Beta
297     @Deprecated
298     public void awaitReadiness(final long timeout, final TimeUnit unit) throws InterruptedException, TimeoutException {
299         if (!awaitReadiness(Duration.create(timeout, unit))) {
300             throw new TimeoutException("Shard leaders failed to settle");
301         }
302     }
303
304     @SuppressWarnings("checkstyle:IllegalCatch")
305     private static ActorRef createShardManager(final ActorSystem actorSystem,
306             final AbstractShardManagerCreator<?> creator, final String shardDispatcher,
307             final String shardManagerId) {
308         Exception lastException = null;
309
310         for (int i = 0; i < 100; i++) {
311             try {
312                 return actorSystem.actorOf(creator.props().withDispatcher(shardDispatcher), shardManagerId);
313             } catch (Exception e) {
314                 lastException = e;
315                 Uninterruptibles.sleepUninterruptibly(100, TimeUnit.MILLISECONDS);
316                 LOG.debug("Could not create actor {} because of {} - waiting for sometime before retrying "
317                         + "(retry count = {})", shardManagerId, e.getMessage(), i);
318             }
319         }
320
321         throw new IllegalStateException("Failed to create Shard Manager", lastException);
322     }
323
324     /**
325      * Future which completes when all shards settle for the first time.
326      *
327      * @return A Listenable future.
328      */
329     public final ListenableFuture<?> initialSettleFuture() {
330         return readinessFuture;
331     }
332
333     @VisibleForTesting
334     public final SettableFuture<Empty> readinessFuture() {
335         return readinessFuture;
336     }
337
338     @Override
339     public Registration registerProxyListener(final YangInstanceIdentifier shardLookup,
340             final YangInstanceIdentifier insideShard, final DOMDataTreeChangeListener delegate) {
341         requireNonNull(shardLookup, "shardLookup should not be null");
342         requireNonNull(insideShard, "insideShard should not be null");
343         requireNonNull(delegate, "delegate should not be null");
344
345         final var shardName = actorUtils.getShardStrategyFactory().getStrategy(shardLookup).findShard(shardLookup);
346         LOG.debug("Registering tree listener: {} for tree: {} shard: {}, path inside shard: {}", delegate, shardLookup,
347             shardName, insideShard);
348
349         return DataTreeChangeListenerProxy.of(actorUtils, new DOMDataTreeChangeListener() {
350             @Override
351             public void onDataTreeChanged(final List<DataTreeCandidate> changes) {
352                 delegate.onDataTreeChanged(changes);
353             }
354
355             @Override
356             public void onInitialData() {
357                 delegate.onInitialData();
358             }
359         }, insideShard, true, shardName);
360     }
361
362     private Duration initialSettleTime() {
363         final DatastoreContext context = actorUtils.getDatastoreContext();
364         final int multiplier = context.getInitialSettleTimeoutMultiplier();
365         return multiplier == 0 ? Duration.Inf() : context.getShardLeaderElectionTimeout().duration().$times(multiplier);
366     }
367 }