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