Improve segmented journal actor metrics
[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     @VisibleForTesting
135     protected AbstractShardManagerCreator<?> getShardManagerCreator() {
136         return new ShardManagerCreator();
137     }
138
139     protected final DataStoreClient getClient() {
140         return client;
141     }
142
143     public void setCloseable(final AutoCloseable closeable) {
144         this.closeable = closeable;
145     }
146
147     @Override
148     public final Registration registerTreeChangeListener(final YangInstanceIdentifier treeId,
149             final DOMDataTreeChangeListener listener) {
150         return registerTreeChangeListener(treeId, listener, true);
151     }
152
153     private @NonNull Registration registerTreeChangeListener(final YangInstanceIdentifier treeId,
154             final DOMDataTreeChangeListener listener, final boolean clustered) {
155         requireNonNull(treeId, "treeId should not be null");
156         requireNonNull(listener, "listener should not be null");
157
158         /*
159          * We need to potentially deal with multi-shard composition for registration targeting the root of the data
160          * store. If that is the case, we delegate to a more complicated setup invol
161          */
162         if (treeId.isEmpty()) {
163             // User is targeting root of the datastore. If there is more than one shard, we have to register with them
164             // all and perform data composition.
165             final var shardNames = actorUtils.getConfiguration().getAllShardNames();
166             if (shardNames.size() > 1) {
167                 if (!clustered) {
168                     throw new IllegalArgumentException(
169                         "Cannot listen on root without non-clustered listener " + listener);
170                 }
171                 return new RootDataTreeChangeListenerProxy<>(actorUtils, listener, shardNames);
172             }
173         }
174
175         final var shardName = actorUtils.getShardStrategyFactory().getStrategy(treeId).findShard(treeId);
176         LOG.debug("Registering tree listener: {} for tree: {} shard: {}", listener, treeId, shardName);
177
178         return DataTreeChangeListenerProxy.of(actorUtils, listener, treeId, clustered, shardName);
179     }
180
181     @Override
182     @Deprecated(since = "9.0.0", forRemoval = true)
183     public final Registration registerLegacyTreeChangeListener(final YangInstanceIdentifier treeId,
184             final DOMDataTreeChangeListener listener) {
185         return registerTreeChangeListener(treeId, listener, false);
186     }
187
188     @Override
189     // Non-final for testing
190     public Registration registerCommitCohort(final DOMDataTreeIdentifier subtree,
191             final DOMDataTreeCommitCohort cohort) {
192         YangInstanceIdentifier treeId = requireNonNull(subtree, "subtree should not be null").path();
193         requireNonNull(cohort, "listener should not be null");
194
195
196         final String shardName = actorUtils.getShardStrategyFactory().getStrategy(treeId).findShard(treeId);
197         LOG.debug("Registering cohort: {} for tree: {} shard: {}", cohort, treeId, shardName);
198
199         final var cohortProxy = new DataTreeCohortRegistrationProxy<>(actorUtils, subtree, cohort);
200         cohortProxy.init(shardName);
201         return cohortProxy;
202     }
203
204     public void onModelContextUpdated(final EffectiveModelContext newModelContext) {
205         actorUtils.setSchemaContext(newModelContext);
206     }
207
208     @Override
209     public final void onDatastoreContextUpdated(final DatastoreContextFactory contextFactory) {
210         LOG.info("DatastoreContext updated for data store {}", actorUtils.getDataStoreName());
211
212         actorUtils.setDatastoreContext(contextFactory);
213         datastoreConfigMXBean.setContext(contextFactory.getBaseDatastoreContext());
214     }
215
216     @Override
217     @SuppressWarnings("checkstyle:IllegalCatch")
218     public final void close() {
219         LOG.info("Closing data store {}", identifier);
220
221         if (datastoreConfigMXBean != null) {
222             datastoreConfigMXBean.unregisterMBean();
223         }
224         if (datastoreInfoMXBean != null) {
225             datastoreInfoMXBean.unregisterMBean();
226         }
227
228         if (closeable != null) {
229             try {
230                 closeable.close();
231             } catch (Exception e) {
232                 LOG.debug("Error closing instance", e);
233             }
234         }
235
236         actorUtils.shutdown();
237
238         if (client != null) {
239             client.close();
240         }
241     }
242
243     @Override
244     public final ActorUtils getActorUtils() {
245         return actorUtils;
246     }
247
248     // TODO: consider removing this in favor of awaitReadiness()
249     @Deprecated
250     public final void waitTillReady() {
251         LOG.info("Beginning to wait for data store to become ready : {}", identifier);
252
253         final Duration toWait = initialSettleTime();
254         try {
255             if (!awaitReadiness(toWait)) {
256                 LOG.error("Shard leaders failed to settle in {}, giving up", toWait);
257                 return;
258             }
259         } catch (InterruptedException e) {
260             LOG.error("Interrupted while waiting for shards to settle", e);
261             return;
262         }
263
264         LOG.debug("Data store {} is now ready", identifier);
265     }
266
267     @Beta
268     @Deprecated
269     public final boolean awaitReadiness() throws InterruptedException {
270         return awaitReadiness(initialSettleTime());
271     }
272
273     @Beta
274     @Deprecated
275     public final boolean awaitReadiness(final Duration toWait) throws InterruptedException {
276         try {
277             if (toWait.isFinite()) {
278                 try {
279                     readinessFuture.get(toWait.toNanos(), TimeUnit.NANOSECONDS);
280                 } catch (TimeoutException e) {
281                     LOG.debug("Timed out waiting for shards to settle", e);
282                     return false;
283                 }
284             } else {
285                 readinessFuture.get();
286             }
287         } catch (ExecutionException e) {
288             LOG.warn("Unexpected readiness failure, assuming convergence", e);
289         }
290
291         return true;
292     }
293
294     @Beta
295     @Deprecated
296     public final void awaitReadiness(final long timeout, final TimeUnit unit)
297             throws InterruptedException, TimeoutException {
298         if (!awaitReadiness(Duration.create(timeout, unit))) {
299             throw new TimeoutException("Shard leaders failed to settle");
300         }
301     }
302
303     @SuppressWarnings("checkstyle:IllegalCatch")
304     private static ActorRef createShardManager(final ActorSystem actorSystem,
305             final AbstractShardManagerCreator<?> creator, final String shardDispatcher,
306             final String shardManagerId) {
307         Exception lastException = null;
308
309         for (int i = 0; i < 100; i++) {
310             try {
311                 return actorSystem.actorOf(creator.props().withDispatcher(shardDispatcher), shardManagerId);
312             } catch (Exception e) {
313                 lastException = e;
314                 Uninterruptibles.sleepUninterruptibly(100, TimeUnit.MILLISECONDS);
315                 LOG.debug("Could not create actor {} because of {} - waiting for sometime before retrying "
316                         + "(retry count = {})", shardManagerId, e.getMessage(), i);
317             }
318         }
319
320         throw new IllegalStateException("Failed to create Shard Manager", lastException);
321     }
322
323     /**
324      * Future which completes when all shards settle for the first time.
325      *
326      * @return A Listenable future.
327      */
328     public final ListenableFuture<?> initialSettleFuture() {
329         return readinessFuture;
330     }
331
332     @VisibleForTesting
333     public final SettableFuture<Empty> readinessFuture() {
334         return readinessFuture;
335     }
336
337     @Override
338     public final Registration registerProxyListener(final YangInstanceIdentifier shardLookup,
339             final YangInstanceIdentifier insideShard, final DOMDataTreeChangeListener delegate) {
340         requireNonNull(shardLookup, "shardLookup should not be null");
341         requireNonNull(insideShard, "insideShard should not be null");
342         requireNonNull(delegate, "delegate should not be null");
343
344         final var shardName = actorUtils.getShardStrategyFactory().getStrategy(shardLookup).findShard(shardLookup);
345         LOG.debug("Registering tree listener: {} for tree: {} shard: {}, path inside shard: {}", delegate, shardLookup,
346             shardName, insideShard);
347
348         return DataTreeChangeListenerProxy.of(actorUtils, new DOMDataTreeChangeListener() {
349             @Override
350             public void onDataTreeChanged(final List<DataTreeCandidate> changes) {
351                 delegate.onDataTreeChanged(changes);
352             }
353
354             @Override
355             public void onInitialData() {
356                 delegate.onInitialData();
357             }
358         }, insideShard, true, shardName);
359     }
360
361     private Duration initialSettleTime() {
362         final DatastoreContext context = actorUtils.getDatastoreContext();
363         final int multiplier = context.getInitialSettleTimeoutMultiplier();
364         return multiplier == 0 ? Duration.Inf() : context.getShardLeaderElectionTimeout().duration().$times(multiplier);
365     }
366 }