d445d3c2f6f9e542ec031c2737653317e3952a73
[controller.git] / opendaylight / md-sal / sal-distributed-datastore / src / main / java / org / opendaylight / controller / cluster / databroker / actors / dds / AbstractClientHistory.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 package org.opendaylight.controller.cluster.databroker.actors.dds;
9
10 import static com.google.common.base.Preconditions.checkArgument;
11 import static com.google.common.base.Preconditions.checkState;
12 import static com.google.common.base.Verify.verifyNotNull;
13 import static java.util.Objects.requireNonNull;
14
15 import java.util.Collection;
16 import java.util.HashMap;
17 import java.util.Map;
18 import java.util.concurrent.ConcurrentHashMap;
19 import java.util.concurrent.atomic.AtomicLongFieldUpdater;
20 import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
21 import java.util.concurrent.locks.StampedLock;
22 import org.checkerframework.checker.lock.qual.GuardedBy;
23 import org.checkerframework.checker.lock.qual.Holding;
24 import org.eclipse.jdt.annotation.NonNull;
25 import org.opendaylight.controller.cluster.access.client.AbstractClientConnection;
26 import org.opendaylight.controller.cluster.access.client.ConnectedClientConnection;
27 import org.opendaylight.controller.cluster.access.client.ConnectionEntry;
28 import org.opendaylight.controller.cluster.access.client.InversibleLockException;
29 import org.opendaylight.controller.cluster.access.commands.CreateLocalHistoryRequest;
30 import org.opendaylight.controller.cluster.access.concepts.LocalHistoryIdentifier;
31 import org.opendaylight.controller.cluster.access.concepts.Response;
32 import org.opendaylight.controller.cluster.access.concepts.TransactionIdentifier;
33 import org.opendaylight.mdsal.dom.api.DOMTransactionChainClosedException;
34 import org.opendaylight.yangtools.concepts.Identifiable;
35 import org.opendaylight.yangtools.yang.data.api.YangInstanceIdentifier;
36 import org.slf4j.Logger;
37 import org.slf4j.LoggerFactory;
38
39 /**
40  * Abstract base class for client view of a history. This class has two implementations, one for normal local histories
41  * and the other for single transactions.
42  *
43  * @author Robert Varga
44  */
45 public abstract class AbstractClientHistory extends LocalAbortable implements Identifiable<LocalHistoryIdentifier> {
46     enum State {
47         IDLE,
48         TX_OPEN,
49         CLOSED,
50     }
51
52     private static final Logger LOG = LoggerFactory.getLogger(AbstractClientHistory.class);
53     private static final AtomicLongFieldUpdater<AbstractClientHistory> NEXT_TX_UPDATER =
54             AtomicLongFieldUpdater.newUpdater(AbstractClientHistory.class, "nextTx");
55     private static final AtomicReferenceFieldUpdater<AbstractClientHistory, State> STATE_UPDATER =
56             AtomicReferenceFieldUpdater.newUpdater(AbstractClientHistory.class, State.class, "state");
57
58     @GuardedBy("this")
59     private final Map<TransactionIdentifier, AbstractClientHandle<?>> openTransactions = new HashMap<>();
60     @GuardedBy("this")
61     private final Map<TransactionIdentifier, AbstractTransactionCommitCohort> readyTransactions = new HashMap<>();
62
63     @GuardedBy("lock")
64     private final Map<Long, ProxyHistory> histories = new ConcurrentHashMap<>();
65     private final StampedLock lock = new StampedLock();
66
67     private final @NonNull AbstractDataStoreClientBehavior client;
68     private final @NonNull LocalHistoryIdentifier identifier;
69
70     // Used via NEXT_TX_UPDATER
71     @SuppressWarnings("unused")
72     private volatile long nextTx = 0;
73
74     private volatile State state = State.IDLE;
75
76     AbstractClientHistory(final AbstractDataStoreClientBehavior client, final LocalHistoryIdentifier identifier) {
77         this.client = requireNonNull(client);
78         this.identifier = requireNonNull(identifier);
79         checkArgument(identifier.getCookie() == 0);
80     }
81
82     final State state() {
83         return state;
84     }
85
86     final void updateState(final State expected, final State next) {
87         final boolean success = STATE_UPDATER.compareAndSet(this, expected, next);
88         checkState(success, "Race condition detected, state changed from %s to %s", expected, state);
89         LOG.debug("Client history {} changed state from {} to {}", this, expected, next);
90     }
91
92     final synchronized void doClose() {
93         final State local = state;
94         if (local != State.CLOSED) {
95             checkState(local == State.IDLE, "Local history %s has an open transaction", this);
96             histories.values().forEach(ProxyHistory::close);
97             updateState(local, State.CLOSED);
98         }
99     }
100
101     final synchronized void onProxyDestroyed(final ProxyHistory proxyHistory) {
102         histories.remove(proxyHistory.getIdentifier().getCookie());
103         LOG.debug("{}: removed destroyed proxy {}", this, proxyHistory);
104     }
105
106     @Override
107     public LocalHistoryIdentifier getIdentifier() {
108         return identifier;
109     }
110
111     final long nextTx() {
112         return NEXT_TX_UPDATER.getAndIncrement(this);
113     }
114
115     final Long resolveShardForPath(final YangInstanceIdentifier path) {
116         return client.resolveShardForPath(path);
117     }
118
119     @Override
120     final void localAbort(final Throwable cause) {
121         final State oldState = STATE_UPDATER.getAndSet(this, State.CLOSED);
122         if (oldState != State.CLOSED) {
123             LOG.debug("Force-closing history {}", getIdentifier(), cause);
124
125             synchronized (this) {
126                 for (AbstractClientHandle<?> t : openTransactions.values()) {
127                     t.localAbort(cause);
128                 }
129                 openTransactions.clear();
130                 readyTransactions.clear();
131             }
132         }
133     }
134
135     /**
136      * Create a new history proxy for a given shard.
137      *
138      * @throws InversibleLockException if the shard is being reconnected
139      */
140     @Holding("lock")
141     private ProxyHistory createHistoryProxy(final Long shard) {
142         final AbstractClientConnection<ShardBackendInfo> connection = client.getConnection(shard);
143         final LocalHistoryIdentifier proxyId = new LocalHistoryIdentifier(identifier.getClientId(),
144             identifier.getHistoryId(), shard);
145         LOG.debug("Created proxyId {} for history {} shard {}", proxyId, identifier, shard);
146
147         final ProxyHistory ret = createHistoryProxy(proxyId, connection);
148
149         // Request creation of the history, if it is not the single history
150         if (ret.getIdentifier().getHistoryId() != 0) {
151             connection.sendRequest(new CreateLocalHistoryRequest(ret.getIdentifier(), connection.localActor()),
152                 this::createHistoryCallback);
153         }
154         return ret;
155     }
156
157     abstract ProxyHistory createHistoryProxy(LocalHistoryIdentifier historyId,
158             AbstractClientConnection<ShardBackendInfo> connection);
159
160     private void createHistoryCallback(final Response<?, ?> response) {
161         LOG.debug("Create history response {}", response);
162     }
163
164     private @NonNull ProxyHistory ensureHistoryProxy(final TransactionIdentifier transactionId, final Long shard) {
165         while (true) {
166             try {
167                 // Short-lived lock to ensure exclusion of createHistoryProxy and the lookup phase in startReconnect,
168                 // see comments in startReconnect() for details.
169                 final long stamp = lock.readLock();
170                 try {
171                     return histories.computeIfAbsent(shard, this::createHistoryProxy);
172                 } finally {
173                     lock.unlockRead(stamp);
174                 }
175             } catch (InversibleLockException e) {
176                 LOG.trace("Waiting for transaction {} shard {} connection to resolve", transactionId, shard);
177                 e.awaitResolution();
178                 LOG.trace("Retrying transaction {} shard {} connection", transactionId, shard);
179             }
180         }
181     }
182
183     final @NonNull AbstractProxyTransaction createSnapshotProxy(final TransactionIdentifier transactionId,
184             final Long shard) {
185         return ensureHistoryProxy(transactionId, shard).createTransactionProxy(transactionId, true);
186     }
187
188     final @NonNull AbstractProxyTransaction createTransactionProxy(final TransactionIdentifier transactionId,
189             final Long shard) {
190         return ensureHistoryProxy(transactionId, shard).createTransactionProxy(transactionId, false);
191     }
192
193     private void checkNotClosed() {
194         if (state == State.CLOSED) {
195             throw new DOMTransactionChainClosedException(String.format("Local history %s is closed", identifier));
196         }
197     }
198
199     /**
200      * Allocate a new {@link ClientTransaction}.
201      *
202      * @return A new {@link ClientTransaction}
203      * @throws DOMTransactionChainClosedException if this history is closed
204      * @throws IllegalStateException if a previous dependent transaction has not been closed
205      */
206     public @NonNull ClientTransaction createTransaction() {
207         checkNotClosed();
208
209         synchronized (this) {
210             final ClientTransaction ret = doCreateTransaction();
211             openTransactions.put(ret.getIdentifier(), ret);
212             return ret;
213         }
214     }
215
216     /**
217      * Create a new {@link ClientSnapshot}.
218      *
219      * @return A new {@link ClientSnapshot}
220      * @throws DOMTransactionChainClosedException if this history is closed
221      * @throws IllegalStateException if a previous dependent transaction has not been closed
222      */
223     public ClientSnapshot takeSnapshot() {
224         checkNotClosed();
225
226         synchronized (this) {
227             final ClientSnapshot ret = doCreateSnapshot();
228             openTransactions.put(ret.getIdentifier(), ret);
229             return ret;
230         }
231     }
232
233     @Holding("this")
234     abstract ClientSnapshot doCreateSnapshot();
235
236     @Holding("this")
237     abstract ClientTransaction doCreateTransaction();
238
239     /**
240      * Callback invoked from {@link ClientTransaction} when a child transaction readied for submission.
241      *
242      * @param txId Transaction identifier
243      * @param cohort Transaction commit cohort
244      */
245     synchronized AbstractTransactionCommitCohort onTransactionReady(final ClientTransaction tx,
246             final AbstractTransactionCommitCohort cohort) {
247         final TransactionIdentifier txId = tx.getIdentifier();
248         if (openTransactions.remove(txId) == null) {
249             LOG.warn("Transaction {} not recorded, proceeding with readiness", txId);
250         }
251
252         final AbstractTransactionCommitCohort previous = readyTransactions.putIfAbsent(txId, cohort);
253         checkState(previous == null, "Duplicate cohort %s for transaction %s, already have %s", cohort, txId, previous);
254
255         LOG.debug("Local history {} readied transaction {}", this, txId);
256         return cohort;
257     }
258
259     /**
260      * Callback invoked from {@link ClientTransaction} when a child transaction has been aborted without touching
261      * backend.
262      *
263      * @param snapshot transaction identifier
264      */
265     synchronized void onTransactionAbort(final AbstractClientHandle<?> snapshot) {
266         if (openTransactions.remove(snapshot.getIdentifier()) == null) {
267             LOG.warn("Could not find aborting transaction {}", snapshot.getIdentifier());
268         }
269     }
270
271     /**
272      * Callback invoked from {@link AbstractTransactionCommitCohort} when a child transaction has been completed
273      * and all its state can be removed.
274      *
275      * @param txId transaction identifier
276      */
277     synchronized void onTransactionComplete(final TransactionIdentifier txId) {
278         if (readyTransactions.remove(txId) == null) {
279             LOG.warn("Could not find completed transaction {}", txId);
280         }
281     }
282
283     HistoryReconnectCohort startReconnect(final ConnectedClientConnection<ShardBackendInfo> newConn) {
284         /*
285          * This looks ugly and unusual and there is a reason for that, as the locking involved is in multiple places.
286          *
287          * We need to make sure that a new proxy is not created while we are reconnecting, which is partially satisfied
288          * by client.getConnection() throwing InversibleLockException by the time this method is invoked. That does
289          * not cover the case when createHistoryProxy() has already acquired the connection, but has not yet populated
290          * the history map.
291          *
292          * Hence we need to make sure no potential computation is happening concurrently with us looking at the history
293          * map. Once we have performed that lookup, though, we can release the lock immediately, as all creation
294          * requests are established to happen either before or after the reconnect attempt.
295          */
296         final ProxyHistory oldProxy;
297         final long stamp = lock.writeLock();
298         try {
299             oldProxy = histories.get(newConn.cookie());
300         } finally {
301             lock.unlockWrite(stamp);
302         }
303
304         if (oldProxy == null) {
305             return null;
306         }
307
308         final ProxyReconnectCohort proxy = verifyNotNull(oldProxy.startReconnect(newConn));
309         return new HistoryReconnectCohort() {
310             @Override
311             ProxyReconnectCohort getProxy() {
312                 return proxy;
313             }
314
315             @Override
316             void replayRequests(final Collection<ConnectionEntry> previousEntries) {
317                 proxy.replayRequests(previousEntries);
318             }
319
320             @Override
321             public void close() {
322                 LOG.debug("Client history {} finishing reconnect to {}", AbstractClientHistory.this, newConn);
323                 final ProxyHistory newProxy = proxy.finishReconnect();
324                 if (!histories.replace(newConn.cookie(), oldProxy, newProxy)) {
325                     LOG.warn("Failed to replace proxy {} with {} in {}", oldProxy, newProxy,
326                         AbstractClientHistory.this);
327                 }
328             }
329         };
330     }
331
332 }