Madan Jampani
Committed by Yuta HIGUCHI

1. Fixed a synchronization issue with database update processing and expiry tracking.

2. Fixed a synchronization issue with MapDBLog appendEntries method.
3. DatabaseClient now uses ProtocolClient to interact with Raft cluster.
4. Misc javdoc and logging improvements

Change-Id: I147eb5bf859cf9827df452d62ab415d643a00aa4
...@@ -19,4 +19,18 @@ public class DatabaseException extends RuntimeException { ...@@ -19,4 +19,18 @@ public class DatabaseException extends RuntimeException {
19 19
20 public DatabaseException() { 20 public DatabaseException() {
21 }; 21 };
22 +
23 + public static class Timeout extends DatabaseException {
24 + public Timeout(String message, Throwable t) {
25 + super(message, t);
26 + }
27 +
28 + public Timeout(String message) {
29 + super(message);
30 + }
31 +
32 + public Timeout(Throwable t) {
33 + super(t);
34 + }
35 + }
22 } 36 }
...\ No newline at end of file ...\ No newline at end of file
......
1 package org.onlab.onos.store.service.impl; 1 package org.onlab.onos.store.service.impl;
2 2
3 -import static com.google.common.base.Preconditions.checkNotNull;
4 import static org.slf4j.LoggerFactory.getLogger; 3 import static org.slf4j.LoggerFactory.getLogger;
5 4
6 -import java.util.Collections; 5 +import java.util.Arrays;
7 import java.util.List; 6 import java.util.List;
8 import java.util.Map; 7 import java.util.Map;
9 import java.util.Set; 8 import java.util.Set;
9 +import java.util.UUID;
10 import java.util.concurrent.CompletableFuture; 10 import java.util.concurrent.CompletableFuture;
11 -import java.util.concurrent.CountDownLatch;
12 import java.util.concurrent.ExecutionException; 11 import java.util.concurrent.ExecutionException;
13 import java.util.concurrent.TimeUnit; 12 import java.util.concurrent.TimeUnit;
14 import java.util.concurrent.TimeoutException; 13 import java.util.concurrent.TimeoutException;
15 14
16 -import net.kuujo.copycat.Copycat; 15 +import net.kuujo.copycat.cluster.Member;
16 +import net.kuujo.copycat.cluster.TcpMember;
17 import net.kuujo.copycat.event.EventHandler; 17 import net.kuujo.copycat.event.EventHandler;
18 import net.kuujo.copycat.event.LeaderElectEvent; 18 import net.kuujo.copycat.event.LeaderElectEvent;
19 +import net.kuujo.copycat.protocol.SubmitRequest;
20 +import net.kuujo.copycat.protocol.SubmitResponse;
21 +import net.kuujo.copycat.spi.protocol.ProtocolClient;
19 22
20 import org.onlab.onos.store.service.BatchReadRequest; 23 import org.onlab.onos.store.service.BatchReadRequest;
21 import org.onlab.onos.store.service.BatchWriteRequest; 24 import org.onlab.onos.store.service.BatchWriteRequest;
...@@ -28,7 +31,7 @@ import org.slf4j.Logger; ...@@ -28,7 +31,7 @@ import org.slf4j.Logger;
28 /** 31 /**
29 * Client for interacting with the Copycat Raft cluster. 32 * Client for interacting with the Copycat Raft cluster.
30 */ 33 */
31 -public class DatabaseClient { 34 +public class DatabaseClient implements EventHandler<LeaderElectEvent> {
32 35
33 private static final int RETRIES = 5; 36 private static final int RETRIES = 5;
34 37
...@@ -36,137 +39,101 @@ public class DatabaseClient { ...@@ -36,137 +39,101 @@ public class DatabaseClient {
36 39
37 private final Logger log = getLogger(getClass()); 40 private final Logger log = getLogger(getClass());
38 41
39 - private final Copycat copycat; 42 + private final DatabaseProtocolService protocol;
43 + private volatile ProtocolClient copycat = null;
44 + private volatile Member currentLeader = null;
40 45
41 - public DatabaseClient(Copycat copycat) { 46 + public DatabaseClient(DatabaseProtocolService protocol) {
42 - this.copycat = checkNotNull(copycat); 47 + this.protocol = protocol;
48 + }
49 +
50 + @Override
51 + public void handle(LeaderElectEvent event) {
52 + Member newLeader = event.leader();
53 + if (newLeader != null && !newLeader.equals(currentLeader)) {
54 + currentLeader = newLeader;
55 + if (copycat != null) {
56 + copycat.close();
57 + }
58 + copycat = protocol.createClient((TcpMember) currentLeader);
59 + copycat.connect();
60 + }
61 + }
62 +
63 + private String nextRequestId() {
64 + return UUID.randomUUID().toString();
43 } 65 }
44 66
45 public void waitForLeader() { 67 public void waitForLeader() {
46 - if (copycat.leader() != null) { 68 + if (currentLeader != null) {
47 return; 69 return;
48 } 70 }
49 71
50 log.info("No leader in cluster, waiting for election."); 72 log.info("No leader in cluster, waiting for election.");
51 - final CountDownLatch latch = new CountDownLatch(1);
52 - final EventHandler<LeaderElectEvent> leaderLsnr = new EventHandler<LeaderElectEvent>() {
53 73
54 - @Override
55 - public void handle(LeaderElectEvent event) {
56 - log.info("Leader chosen: {}", event);
57 - latch.countDown();
58 - }
59 - };
60 -
61 - copycat.event(LeaderElectEvent.class).registerHandler(leaderLsnr);
62 try { 74 try {
63 - while (copycat.leader() == null) { 75 + while (currentLeader == null) {
64 - latch.await(200, TimeUnit.MILLISECONDS); 76 + Thread.sleep(200);
65 } 77 }
66 - log.info("Leader appeared: {}", copycat.leader()); 78 + log.info("Leader appeared: {}", currentLeader);
67 return; 79 return;
68 } catch (InterruptedException e) { 80 } catch (InterruptedException e) {
69 log.error("Interrupted while waiting for Leader", e); 81 log.error("Interrupted while waiting for Leader", e);
70 Thread.currentThread().interrupt(); 82 Thread.currentThread().interrupt();
71 - } finally {
72 - copycat.event(LeaderElectEvent.class).unregisterHandler(leaderLsnr);
73 } 83 }
74 } 84 }
75 85
76 - public boolean createTable(String tableName) { 86 + private <T> T submit(String operationName, Object... args) {
77 waitForLeader(); 87 waitForLeader();
78 - CompletableFuture<Boolean> future = copycat.submit("createTable", tableName); 88 + if (currentLeader == null) {
89 + throw new DatabaseException("Raft cluster does not have a leader.");
90 + }
91 +
92 + SubmitRequest request =
93 + new SubmitRequest(nextRequestId(), operationName, Arrays.asList(args));
94 +
95 + CompletableFuture<SubmitResponse> submitResponse = copycat.submit(request);
96 +
97 + log.debug("Sent {} to {}", request, currentLeader);
98 +
79 try { 99 try {
80 - return future.get(); 100 + return (T) submitResponse.get(TIMEOUT_MS, TimeUnit.MILLISECONDS).result();
81 - } catch (InterruptedException | ExecutionException e) { 101 + } catch (ExecutionException | InterruptedException e) {
82 throw new DatabaseException(e); 102 throw new DatabaseException(e);
103 + } catch (TimeoutException e) {
104 + throw new DatabaseException.Timeout(e);
83 } 105 }
84 } 106 }
85 107
86 - public boolean createTable(String tableName, int ttlMillis) { 108 + public boolean createTable(String tableName) {
87 - waitForLeader(); 109 + return submit("createTable", tableName);
88 - CompletableFuture<Boolean> future = copycat.submit("createTableWithExpiration", tableName);
89 - try {
90 - return future.get();
91 - } catch (InterruptedException | ExecutionException e) {
92 - throw new DatabaseException(e);
93 } 110 }
111 +
112 + public boolean createTable(String tableName, int ttlMillis) {
113 + return submit("createTable", tableName, ttlMillis);
94 } 114 }
95 115
96 public void dropTable(String tableName) { 116 public void dropTable(String tableName) {
97 - waitForLeader(); 117 + submit("dropTable", tableName);
98 - CompletableFuture<Void> future = copycat.submit("dropTable", tableName);
99 - try {
100 - future.get();
101 - } catch (InterruptedException | ExecutionException e) {
102 - throw new DatabaseException(e);
103 - }
104 } 118 }
105 119
106 public void dropAllTables() { 120 public void dropAllTables() {
107 - waitForLeader(); 121 + submit("dropAllTables");
108 - CompletableFuture<Void> future = copycat.submit("dropAllTables");
109 - try {
110 - future.get();
111 - } catch (InterruptedException | ExecutionException e) {
112 - throw new DatabaseException(e);
113 - }
114 } 122 }
115 123
116 public Set<String> listTables() { 124 public Set<String> listTables() {
117 - waitForLeader(); 125 + return submit("listTables");
118 - try {
119 - for (int i = 0; i < RETRIES; ++i) {
120 - CompletableFuture<Set<String>> future = copycat.submit("listTables");
121 - try {
122 - return future.get(TIMEOUT_MS, TimeUnit.MILLISECONDS);
123 - } catch (TimeoutException e) {
124 - log.debug("Timed out retrying {}", i);
125 - future.cancel(true);
126 - waitForLeader();
127 - }
128 - }
129 - // TODO: proper timeout handling
130 - log.error("Timed out");
131 - return Collections.emptySet();
132 - } catch (InterruptedException | ExecutionException e) {
133 - throw new DatabaseException(e);
134 - }
135 } 126 }
136 127
137 public List<ReadResult> batchRead(BatchReadRequest batchRequest) { 128 public List<ReadResult> batchRead(BatchReadRequest batchRequest) {
138 - waitForLeader(); 129 + return submit("read", batchRequest);
139 - CompletableFuture<List<ReadResult>> future = copycat.submit("read", batchRequest);
140 - try {
141 - return future.get(TIMEOUT_MS, TimeUnit.MILLISECONDS);
142 - } catch (InterruptedException | ExecutionException e) {
143 - throw new DatabaseException(e);
144 - } catch (TimeoutException e) {
145 - throw new DatabaseException(e);
146 - }
147 } 130 }
148 131
149 public List<WriteResult> batchWrite(BatchWriteRequest batchRequest) { 132 public List<WriteResult> batchWrite(BatchWriteRequest batchRequest) {
150 - waitForLeader(); 133 + return submit("write", batchRequest);
151 - CompletableFuture<List<WriteResult>> future = copycat.submit("write", batchRequest);
152 - try {
153 - return future.get(TIMEOUT_MS, TimeUnit.MILLISECONDS);
154 - } catch (InterruptedException | ExecutionException e) {
155 - throw new DatabaseException(e);
156 - } catch (TimeoutException e) {
157 - throw new DatabaseException(e);
158 - }
159 } 134 }
160 135
161 public Map<String, VersionedValue> getAll(String tableName) { 136 public Map<String, VersionedValue> getAll(String tableName) {
162 - waitForLeader(); 137 + return submit("getAll", tableName);
163 - CompletableFuture<Map<String, VersionedValue>> future = copycat.submit("getAll", tableName);
164 - try {
165 - return future.get(TIMEOUT_MS, TimeUnit.MILLISECONDS);
166 - } catch (InterruptedException | ExecutionException e) {
167 - throw new DatabaseException(e);
168 - } catch (TimeoutException e) {
169 - throw new DatabaseException(e);
170 - }
171 } 138 }
172 } 139 }
......
...@@ -16,10 +16,14 @@ ...@@ -16,10 +16,14 @@
16 16
17 package org.onlab.onos.store.service.impl; 17 package org.onlab.onos.store.service.impl;
18 18
19 +import static org.onlab.util.Tools.namedThreads;
20 +
19 import java.io.IOException; 21 import java.io.IOException;
20 import java.util.HashMap; 22 import java.util.HashMap;
21 import java.util.Map; 23 import java.util.Map;
22 import java.util.Objects; 24 import java.util.Objects;
25 +import java.util.concurrent.ExecutorService;
26 +import java.util.concurrent.Executors;
23 import java.util.concurrent.TimeUnit; 27 import java.util.concurrent.TimeUnit;
24 import java.util.concurrent.atomic.AtomicBoolean; 28 import java.util.concurrent.atomic.AtomicBoolean;
25 29
...@@ -40,6 +44,8 @@ import org.onlab.onos.store.service.impl.DatabaseStateMachine.TableMetadata; ...@@ -40,6 +44,8 @@ import org.onlab.onos.store.service.impl.DatabaseStateMachine.TableMetadata;
40 import org.slf4j.Logger; 44 import org.slf4j.Logger;
41 import org.slf4j.LoggerFactory; 45 import org.slf4j.LoggerFactory;
42 46
47 +import com.google.common.base.MoreObjects;
48 +
43 /** 49 /**
44 * Plugs into the database update stream and track the TTL of entries added to 50 * Plugs into the database update stream and track the TTL of entries added to
45 * the database. For tables with pre-configured finite TTL, this class has 51 * the database. For tables with pre-configured finite TTL, this class has
...@@ -48,6 +54,9 @@ import org.slf4j.LoggerFactory; ...@@ -48,6 +54,9 @@ import org.slf4j.LoggerFactory;
48 public class DatabaseEntryExpirationTracker implements 54 public class DatabaseEntryExpirationTracker implements
49 DatabaseUpdateEventListener, EventHandler<LeaderElectEvent> { 55 DatabaseUpdateEventListener, EventHandler<LeaderElectEvent> {
50 56
57 + private static final ExecutorService THREAD_POOL =
58 + Executors.newCachedThreadPool(namedThreads("database-stale-entry-expirer-%d"));
59 +
51 private final Logger log = LoggerFactory.getLogger(getClass()); 60 private final Logger log = LoggerFactory.getLogger(getClass());
52 61
53 private final DatabaseService databaseService; 62 private final DatabaseService databaseService;
...@@ -74,7 +83,7 @@ public class DatabaseEntryExpirationTracker implements ...@@ -74,7 +83,7 @@ public class DatabaseEntryExpirationTracker implements
74 83
75 @Override 84 @Override
76 public void tableModified(TableModificationEvent event) { 85 public void tableModified(TableModificationEvent event) {
77 - log.debug("Received a table modification event {}", event); 86 + log.debug("{}: Received {}", localNode.id(), event);
78 87
79 if (!tableEntryExpirationMap.containsKey(event.tableName())) { 88 if (!tableEntryExpirationMap.containsKey(event.tableName())) {
80 return; 89 return;
...@@ -89,8 +98,8 @@ public class DatabaseEntryExpirationTracker implements ...@@ -89,8 +98,8 @@ public class DatabaseEntryExpirationTracker implements
89 map.remove(row, eventVersion); 98 map.remove(row, eventVersion);
90 if (isLocalMemberLeader.get()) { 99 if (isLocalMemberLeader.get()) {
91 try { 100 try {
92 - // FIXME: The broadcast message should be sent to self. 101 + log.debug("Broadcasting {} to the entire cluster", event);
93 - clusterCommunicator.broadcast(new ClusterMessage( 102 + clusterCommunicator.broadcastIncludeSelf(new ClusterMessage(
94 localNode.id(), DatabaseStateMachine.DATABASE_UPDATE_EVENTS, 103 localNode.id(), DatabaseStateMachine.DATABASE_UPDATE_EVENTS,
95 DatabaseStateMachine.SERIALIZER.encode(event))); 104 DatabaseStateMachine.SERIALIZER.encode(event)));
96 } catch (IOException e) { 105 } catch (IOException e) {
...@@ -119,8 +128,6 @@ public class DatabaseEntryExpirationTracker implements ...@@ -119,8 +128,6 @@ public class DatabaseEntryExpirationTracker implements
119 tableEntryExpirationMap.put(metadata.tableName(), ExpiringMap.builder() 128 tableEntryExpirationMap.put(metadata.tableName(), ExpiringMap.builder()
120 .expiration(metadata.ttlMillis(), TimeUnit.MILLISECONDS) 129 .expiration(metadata.ttlMillis(), TimeUnit.MILLISECONDS)
121 .expirationListener(expirationObserver) 130 .expirationListener(expirationObserver)
122 - // TODO: make the expiration policy configurable.
123 - // Do we need to support expiration based on last access time?
124 .expirationPolicy(ExpirationPolicy.CREATED).build()); 131 .expirationPolicy(ExpirationPolicy.CREATED).build());
125 } 132 }
126 } 133 }
...@@ -135,6 +142,23 @@ public class DatabaseEntryExpirationTracker implements ...@@ -135,6 +142,23 @@ public class DatabaseEntryExpirationTracker implements
135 ExpirationListener<DatabaseRow, Long> { 142 ExpirationListener<DatabaseRow, Long> {
136 @Override 143 @Override
137 public void expired(DatabaseRow row, Long version) { 144 public void expired(DatabaseRow row, Long version) {
145 + THREAD_POOL.submit(new ExpirationTask(row, version));
146 + }
147 + }
148 +
149 + private class ExpirationTask implements Runnable {
150 +
151 + private final DatabaseRow row;
152 + private final Long version;
153 +
154 + public ExpirationTask(DatabaseRow row, Long version) {
155 + this.row = row;
156 + this.version = version;
157 + }
158 +
159 + @Override
160 + public void run() {
161 + log.debug("Received an expiration event for {}, version: {}", row, version);
138 Map<DatabaseRow, Long> map = tableEntryExpirationMap.get(row.tableName); 162 Map<DatabaseRow, Long> map = tableEntryExpirationMap.get(row.tableName);
139 try { 163 try {
140 if (isLocalMemberLeader.get()) { 164 if (isLocalMemberLeader.get()) {
...@@ -142,7 +166,7 @@ public class DatabaseEntryExpirationTracker implements ...@@ -142,7 +166,7 @@ public class DatabaseEntryExpirationTracker implements
142 row.key, version)) { 166 row.key, version)) {
143 log.info("Entry in database was updated right before its expiration."); 167 log.info("Entry in database was updated right before its expiration.");
144 } else { 168 } else {
145 - log.info("Successfully expired old entry with key ({}) from table ({})", 169 + log.debug("Successfully expired old entry with key ({}) from table ({})",
146 row.key, row.tableName); 170 row.key, row.tableName);
147 } 171 }
148 } else { 172 } else {
...@@ -164,6 +188,9 @@ public class DatabaseEntryExpirationTracker implements ...@@ -164,6 +188,9 @@ public class DatabaseEntryExpirationTracker implements
164 @Override 188 @Override
165 public void handle(LeaderElectEvent event) { 189 public void handle(LeaderElectEvent event) {
166 isLocalMemberLeader.set(localMember.equals(event.leader())); 190 isLocalMemberLeader.set(localMember.equals(event.leader()));
191 + if (isLocalMemberLeader.get()) {
192 + log.info("{} is now the leader of Raft cluster", localNode.id());
193 + }
167 } 194 }
168 195
169 /** 196 /**
...@@ -180,6 +207,14 @@ public class DatabaseEntryExpirationTracker implements ...@@ -180,6 +207,14 @@ public class DatabaseEntryExpirationTracker implements
180 } 207 }
181 208
182 @Override 209 @Override
210 + public String toString() {
211 + return MoreObjects.toStringHelper(getClass())
212 + .add("tableName", tableName)
213 + .add("key", key)
214 + .toString();
215 + }
216 +
217 + @Override
183 public boolean equals(Object obj) { 218 public boolean equals(Object obj) {
184 if (this == obj) { 219 if (this == obj) {
185 return true; 220 return true;
...@@ -204,6 +239,7 @@ public class DatabaseEntryExpirationTracker implements ...@@ -204,6 +239,7 @@ public class DatabaseEntryExpirationTracker implements
204 if (!tableEntryExpirationMap.isEmpty()) { 239 if (!tableEntryExpirationMap.isEmpty()) {
205 return; 240 return;
206 } 241 }
242 + log.debug("Received a snapshot installed notification");
207 for (String tableName : state.getTableNames()) { 243 for (String tableName : state.getTableNames()) {
208 244
209 TableMetadata metadata = state.getTableMetadata(tableName); 245 TableMetadata metadata = state.getTableMetadata(tableName);
......
...@@ -173,13 +173,16 @@ public class DatabaseManager implements DatabaseService, DatabaseAdminService { ...@@ -173,13 +173,16 @@ public class DatabaseManager implements DatabaseService, DatabaseAdminService {
173 Log consensusLog = new MapDBLog(LOG_FILE_PREFIX + localNode.id(), 173 Log consensusLog = new MapDBLog(LOG_FILE_PREFIX + localNode.id(),
174 ClusterMessagingProtocol.SERIALIZER); 174 ClusterMessagingProtocol.SERIALIZER);
175 175
176 + client = new DatabaseClient(copycatMessagingProtocol);
177 +
176 copycat = new Copycat(stateMachine, consensusLog, cluster, copycatMessagingProtocol); 178 copycat = new Copycat(stateMachine, consensusLog, cluster, copycatMessagingProtocol);
177 179
180 + copycat.event(LeaderElectEvent.class).registerHandler(client);
178 copycat.event(LeaderElectEvent.class).registerHandler(expirationTracker); 181 copycat.event(LeaderElectEvent.class).registerHandler(expirationTracker);
179 182
180 copycat.start().get(); 183 copycat.start().get();
181 184
182 - client = new DatabaseClient(copycat); 185 + client = new DatabaseClient(copycatMessagingProtocol);
183 client.waitForLeader(); 186 client.waitForLeader();
184 187
185 log.info("Started."); 188 log.info("Started.");
......
1 package org.onlab.onos.store.service.impl; 1 package org.onlab.onos.store.service.impl;
2 2
3 +import static org.onlab.util.Tools.namedThreads;
3 import static org.slf4j.LoggerFactory.getLogger; 4 import static org.slf4j.LoggerFactory.getLogger;
4 5
5 import java.io.ByteArrayInputStream; 6 import java.io.ByteArrayInputStream;
...@@ -9,6 +10,8 @@ import java.util.Arrays; ...@@ -9,6 +10,8 @@ import java.util.Arrays;
9 import java.util.List; 10 import java.util.List;
10 import java.util.Map; 11 import java.util.Map;
11 import java.util.Set; 12 import java.util.Set;
13 +import java.util.concurrent.ExecutorService;
14 +import java.util.concurrent.Executors;
12 import java.util.zip.DeflaterOutputStream; 15 import java.util.zip.DeflaterOutputStream;
13 import java.util.zip.InflaterInputStream; 16 import java.util.zip.InflaterInputStream;
14 17
...@@ -49,6 +52,9 @@ public class DatabaseStateMachine implements StateMachine { ...@@ -49,6 +52,9 @@ public class DatabaseStateMachine implements StateMachine {
49 52
50 private final Logger log = getLogger(getClass()); 53 private final Logger log = getLogger(getClass());
51 54
55 + private final ExecutorService updatesExecutor =
56 + Executors.newSingleThreadExecutor(namedThreads("database-statemachine-updates"));
57 +
52 // message subject for database update notifications. 58 // message subject for database update notifications.
53 public static final MessageSubject DATABASE_UPDATE_EVENTS = 59 public static final MessageSubject DATABASE_UPDATE_EVENTS =
54 new MessageSubject("database-update-events"); 60 new MessageSubject("database-update-events");
...@@ -88,8 +94,7 @@ public class DatabaseStateMachine implements StateMachine { ...@@ -88,8 +94,7 @@ public class DatabaseStateMachine implements StateMachine {
88 } 94 }
89 95
90 @Command 96 @Command
91 - public boolean createTableWithExpiration(String tableName) { 97 + public boolean createTable(String tableName, Integer ttlMillis) {
92 - int ttlMillis = 10000;
93 TableMetadata metadata = new TableMetadata(tableName, ttlMillis); 98 TableMetadata metadata = new TableMetadata(tableName, ttlMillis);
94 return createTable(metadata); 99 return createTable(metadata);
95 } 100 }
...@@ -100,18 +105,32 @@ public class DatabaseStateMachine implements StateMachine { ...@@ -100,18 +105,32 @@ public class DatabaseStateMachine implements StateMachine {
100 return false; 105 return false;
101 } 106 }
102 state.createTable(metadata); 107 state.createTable(metadata);
108 +
109 + updatesExecutor.submit(new Runnable() {
110 + @Override
111 + public void run() {
103 for (DatabaseUpdateEventListener listener : listeners) { 112 for (DatabaseUpdateEventListener listener : listeners) {
104 listener.tableCreated(metadata); 113 listener.tableCreated(metadata);
105 } 114 }
115 + }
116 + });
117 +
106 return true; 118 return true;
107 } 119 }
108 120
109 @Command 121 @Command
110 public boolean dropTable(String tableName) { 122 public boolean dropTable(String tableName) {
111 if (state.removeTable(tableName)) { 123 if (state.removeTable(tableName)) {
124 +
125 + updatesExecutor.submit(new Runnable() {
126 + @Override
127 + public void run() {
112 for (DatabaseUpdateEventListener listener : listeners) { 128 for (DatabaseUpdateEventListener listener : listeners) {
113 listener.tableDeleted(tableName); 129 listener.tableDeleted(tableName);
114 } 130 }
131 + }
132 + });
133 +
115 return true; 134 return true;
116 } 135 }
117 return false; 136 return false;
...@@ -121,11 +140,18 @@ public class DatabaseStateMachine implements StateMachine { ...@@ -121,11 +140,18 @@ public class DatabaseStateMachine implements StateMachine {
121 public boolean dropAllTables() { 140 public boolean dropAllTables() {
122 Set<String> tableNames = state.getTableNames(); 141 Set<String> tableNames = state.getTableNames();
123 state.removeAllTables(); 142 state.removeAllTables();
143 +
144 + updatesExecutor.submit(new Runnable() {
145 + @Override
146 + public void run() {
124 for (DatabaseUpdateEventListener listener : listeners) { 147 for (DatabaseUpdateEventListener listener : listeners) {
125 for (String tableName : tableNames) { 148 for (String tableName : tableNames) {
126 listener.tableDeleted(tableName); 149 listener.tableDeleted(tableName);
127 } 150 }
128 } 151 }
152 + }
153 + });
154 +
129 return true; 155 return true;
130 } 156 }
131 157
...@@ -273,12 +299,18 @@ public class DatabaseStateMachine implements StateMachine { ...@@ -273,12 +299,18 @@ public class DatabaseStateMachine implements StateMachine {
273 } 299 }
274 300
275 // notify listeners of table mod events. 301 // notify listeners of table mod events.
302 +
303 + updatesExecutor.submit(new Runnable() {
304 + @Override
305 + public void run() {
276 for (DatabaseUpdateEventListener listener : listeners) { 306 for (DatabaseUpdateEventListener listener : listeners) {
277 for (TableModificationEvent tableModificationEvent : tableModificationEvents) { 307 for (TableModificationEvent tableModificationEvent : tableModificationEvents) {
278 log.trace("Publishing table modification event: {}", tableModificationEvent); 308 log.trace("Publishing table modification event: {}", tableModificationEvent);
279 listener.tableModified(tableModificationEvent); 309 listener.tableModified(tableModificationEvent);
280 } 310 }
281 } 311 }
312 + }
313 + });
282 314
283 return results; 315 return results;
284 } 316 }
...@@ -397,10 +429,15 @@ public class DatabaseStateMachine implements StateMachine { ...@@ -397,10 +429,15 @@ public class DatabaseStateMachine implements StateMachine {
397 this.state = SERIALIZER.decode(data); 429 this.state = SERIALIZER.decode(data);
398 } 430 }
399 431
400 - // FIXME: synchronize. 432 + updatesExecutor.submit(new Runnable() {
433 + @Override
434 + public void run() {
401 for (DatabaseUpdateEventListener listener : listeners) { 435 for (DatabaseUpdateEventListener listener : listeners) {
402 listener.snapshotInstalled(state); 436 listener.snapshotInstalled(state);
403 } 437 }
438 + }
439 + });
440 +
404 } catch (Exception e) { 441 } catch (Exception e) {
405 log.error("Failed to install from snapshot", e); 442 log.error("Failed to install from snapshot", e);
406 throw new SnapshotException(e); 443 throw new SnapshotException(e);
......
1 package org.onlab.onos.store.service.impl; 1 package org.onlab.onos.store.service.impl;
2 2
3 +import static org.slf4j.LoggerFactory.getLogger;
4 +
3 import java.util.UUID; 5 import java.util.UUID;
4 import java.util.concurrent.CompletableFuture; 6 import java.util.concurrent.CompletableFuture;
5 import java.util.concurrent.ExecutionException; 7 import java.util.concurrent.ExecutionException;
...@@ -11,12 +13,17 @@ import org.joda.time.DateTime; ...@@ -11,12 +13,17 @@ import org.joda.time.DateTime;
11 import org.onlab.onos.cluster.ClusterService; 13 import org.onlab.onos.cluster.ClusterService;
12 import org.onlab.onos.store.service.DatabaseService; 14 import org.onlab.onos.store.service.DatabaseService;
13 import org.onlab.onos.store.service.Lock; 15 import org.onlab.onos.store.service.Lock;
16 +import org.slf4j.Logger;
14 17
15 /** 18 /**
16 * A distributed lock implementation. 19 * A distributed lock implementation.
17 */ 20 */
18 public class DistributedLock implements Lock { 21 public class DistributedLock implements Lock {
19 22
23 + private final Logger log = getLogger(getClass());
24 +
25 + private static final long MAX_WAIT_TIME_MS = 100000000L;
26 +
20 private final DistributedLockManager lockManager; 27 private final DistributedLockManager lockManager;
21 private final DatabaseService databaseService; 28 private final DatabaseService databaseService;
22 private final String path; 29 private final String path;
...@@ -44,54 +51,60 @@ public class DistributedLock implements Lock { ...@@ -44,54 +51,60 @@ public class DistributedLock implements Lock {
44 51
45 @Override 52 @Override
46 public void lock(int leaseDurationMillis) { 53 public void lock(int leaseDurationMillis) {
47 -
48 if (isLocked() && lockExpirationTime.isAfter(DateTime.now().plusMillis(leaseDurationMillis))) { 54 if (isLocked() && lockExpirationTime.isAfter(DateTime.now().plusMillis(leaseDurationMillis))) {
49 // Nothing to do. 55 // Nothing to do.
50 // Current expiration time is beyond what is requested. 56 // Current expiration time is beyond what is requested.
51 return; 57 return;
52 } else { 58 } else {
53 - tryLock(Long.MAX_VALUE, leaseDurationMillis); 59 + tryLock(MAX_WAIT_TIME_MS, leaseDurationMillis);
54 } 60 }
55 } 61 }
56 62
57 @Override 63 @Override
58 public boolean tryLock(int leaseDurationMillis) { 64 public boolean tryLock(int leaseDurationMillis) {
59 - return databaseService.putIfAbsent( 65 + if (databaseService.putIfAbsent(
60 DistributedLockManager.ONOS_LOCK_TABLE_NAME, 66 DistributedLockManager.ONOS_LOCK_TABLE_NAME,
61 path, 67 path,
62 - lockId); 68 + lockId)) {
69 + isLocked.set(true);
70 + lockExpirationTime = DateTime.now().plusMillis(leaseDurationMillis);
71 + return true;
72 + }
73 + return false;
63 } 74 }
64 75
65 @Override 76 @Override
66 public boolean tryLock( 77 public boolean tryLock(
67 long waitTimeMillis, 78 long waitTimeMillis,
68 int leaseDurationMillis) { 79 int leaseDurationMillis) {
69 - if (!tryLock(leaseDurationMillis)) { 80 + if (tryLock(leaseDurationMillis)) {
70 - CompletableFuture<Void> future = 81 + return true;
82 + }
83 +
84 + CompletableFuture<DateTime> future =
71 lockManager.lockIfAvailable(this, waitTimeMillis, leaseDurationMillis); 85 lockManager.lockIfAvailable(this, waitTimeMillis, leaseDurationMillis);
72 try { 86 try {
73 - future.get(waitTimeMillis, TimeUnit.MILLISECONDS); 87 + lockExpirationTime = future.get(waitTimeMillis, TimeUnit.MILLISECONDS);
88 + return true;
74 } catch (ExecutionException | InterruptedException e) { 89 } catch (ExecutionException | InterruptedException e) {
90 + log.error("Encountered an exception trying to acquire lock for " + path, e);
75 // TODO: ExecutionException could indicate something 91 // TODO: ExecutionException could indicate something
76 // wrong with the backing database. 92 // wrong with the backing database.
77 // Throw an exception? 93 // Throw an exception?
78 return false; 94 return false;
79 } catch (TimeoutException e) { 95 } catch (TimeoutException e) {
96 + log.debug("Timed out waiting to acquire lock for {}", path);
80 return false; 97 return false;
81 } 98 }
82 } 99 }
83 - isLocked.set(true);
84 - lockExpirationTime = DateTime.now().plusMillis(leaseDurationMillis);
85 - return true;
86 - }
87 100
88 @Override 101 @Override
89 public boolean isLocked() { 102 public boolean isLocked() {
90 if (isLocked.get()) { 103 if (isLocked.get()) {
91 // We rely on local information to check 104 // We rely on local information to check
92 - // if the expired. 105 + // if the lock expired.
93 // This should should make this call 106 // This should should make this call
94 - // light weight, which still retaining the same 107 + // light weight, while still retaining the
95 // safety guarantees. 108 // safety guarantees.
96 if (DateTime.now().isAfter(lockExpirationTime)) { 109 if (DateTime.now().isAfter(lockExpirationTime)) {
97 isLocked.set(false); 110 isLocked.set(false);
...@@ -108,17 +121,30 @@ public class DistributedLock implements Lock { ...@@ -108,17 +121,30 @@ public class DistributedLock implements Lock {
108 if (!isLocked()) { 121 if (!isLocked()) {
109 return; 122 return;
110 } else { 123 } else {
124 + if (databaseService.removeIfValueMatches(DistributedLockManager.ONOS_LOCK_TABLE_NAME, path, lockId)) {
111 isLocked.set(false); 125 isLocked.set(false);
112 - databaseService.removeIfValueMatches(DistributedLockManager.ONOS_LOCK_TABLE_NAME, path, lockId); 126 + }
113 } 127 }
114 } 128 }
115 129
116 @Override 130 @Override
117 public boolean extendExpiration(int leaseDurationMillis) { 131 public boolean extendExpiration(int leaseDurationMillis) {
118 - if (isLocked() && lockExpirationTime.isAfter(DateTime.now().plusMillis(leaseDurationMillis))) { 132 + if (!isLocked()) {
133 + log.warn("Ignoring request to extend expiration for lock {}."
134 + + " ExtendExpiration must be called for locks that are already acquired.", path);
135 + }
136 +
137 + if (databaseService.putIfValueMatches(
138 + DistributedLockManager.ONOS_LOCK_TABLE_NAME,
139 + path,
140 + lockId,
141 + lockId)) {
142 + lockExpirationTime = DateTime.now().plusMillis(leaseDurationMillis);
143 + log.debug("Succeeded in extending lock {} expiration time to {}", lockExpirationTime);
119 return true; 144 return true;
120 } else { 145 } else {
121 - return tryLock(leaseDurationMillis); 146 + log.info("Failed to extend expiration for {}", path);
147 + return false;
122 } 148 }
123 } 149 }
124 } 150 }
......
1 package org.onlab.onos.store.service.impl; 1 package org.onlab.onos.store.service.impl;
2 2
3 +import static org.onlab.util.Tools.namedThreads;
3 import static org.slf4j.LoggerFactory.getLogger; 4 import static org.slf4j.LoggerFactory.getLogger;
4 5
5 import java.util.Iterator; 6 import java.util.Iterator;
6 import java.util.List; 7 import java.util.List;
7 import java.util.concurrent.CompletableFuture; 8 import java.util.concurrent.CompletableFuture;
9 +import java.util.concurrent.ExecutorService;
10 +import java.util.concurrent.Executors;
8 11
9 import org.apache.felix.scr.annotations.Activate; 12 import org.apache.felix.scr.annotations.Activate;
10 import org.apache.felix.scr.annotations.Component; 13 import org.apache.felix.scr.annotations.Component;
...@@ -23,18 +26,23 @@ import org.onlab.onos.store.service.LockEventListener; ...@@ -23,18 +26,23 @@ import org.onlab.onos.store.service.LockEventListener;
23 import org.onlab.onos.store.service.LockService; 26 import org.onlab.onos.store.service.LockService;
24 import org.slf4j.Logger; 27 import org.slf4j.Logger;
25 28
26 -import com.google.common.collect.ArrayListMultimap; 29 +import com.google.common.collect.LinkedListMultimap;
30 +import com.google.common.collect.ListMultimap;
31 +import com.google.common.collect.Multimaps;
27 32
28 @Component(immediate = true) 33 @Component(immediate = true)
29 @Service 34 @Service
30 public class DistributedLockManager implements LockService { 35 public class DistributedLockManager implements LockService {
31 36
37 + private static final ExecutorService THREAD_POOL =
38 + Executors.newCachedThreadPool(namedThreads("lock-manager-%d"));
39 +
32 private final Logger log = getLogger(getClass()); 40 private final Logger log = getLogger(getClass());
33 41
34 public static final String ONOS_LOCK_TABLE_NAME = "onos-locks"; 42 public static final String ONOS_LOCK_TABLE_NAME = "onos-locks";
35 43
36 - private final ArrayListMultimap<String, LockRequest> locksToAcquire = ArrayListMultimap 44 + private final ListMultimap<String, LockRequest> locksToAcquire =
37 - .create(); 45 + Multimaps.synchronizedListMultimap(LinkedListMultimap.<String, LockRequest>create());
38 46
39 @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY) 47 @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
40 private ClusterCommunicationService clusterCommunicator; 48 private ClusterCommunicationService clusterCommunicator;
...@@ -56,8 +64,9 @@ public class DistributedLockManager implements LockService { ...@@ -56,8 +64,9 @@ public class DistributedLockManager implements LockService {
56 64
57 @Deactivate 65 @Deactivate
58 public void deactivate() { 66 public void deactivate() {
67 + clusterCommunicator.removeSubscriber(DatabaseStateMachine.DATABASE_UPDATE_EVENTS);
59 locksToAcquire.clear(); 68 locksToAcquire.clear();
60 - log.info("Started."); 69 + log.info("Stopped.");
61 } 70 }
62 71
63 @Override 72 @Override
...@@ -77,11 +86,19 @@ public class DistributedLockManager implements LockService { ...@@ -77,11 +86,19 @@ public class DistributedLockManager implements LockService {
77 throw new UnsupportedOperationException(); 86 throw new UnsupportedOperationException();
78 } 87 }
79 88
80 - protected CompletableFuture<Void> lockIfAvailable(Lock lock, 89 + /**
81 - long waitTimeMillis, int leaseDurationMillis) { 90 + * Attempts to acquire the lock as soon as it becomes available.
82 - CompletableFuture<Void> future = new CompletableFuture<>(); 91 + * @param lock lock to acquire.
83 - locksToAcquire.put(lock.path(), new LockRequest(lock, waitTimeMillis, 92 + * @param waitTimeMillis maximum time to wait before giving up.
84 - leaseDurationMillis, future)); 93 + * @param leaseDurationMillis the duration for which to acquire the lock initially.
94 + * @return Future lease expiration date.
95 + */
96 + protected CompletableFuture<DateTime> lockIfAvailable(
97 + Lock lock,
98 + long waitTimeMillis,
99 + int leaseDurationMillis) {
100 + CompletableFuture<DateTime> future = new CompletableFuture<>();
101 + locksToAcquire.put(lock.path(), new LockRequest(lock, waitTimeMillis, leaseDurationMillis, future));
85 return future; 102 return future;
86 } 103 }
87 104
...@@ -90,35 +107,45 @@ public class DistributedLockManager implements LockService { ...@@ -90,35 +107,45 @@ public class DistributedLockManager implements LockService {
90 public void handle(ClusterMessage message) { 107 public void handle(ClusterMessage message) {
91 TableModificationEvent event = DatabaseStateMachine.SERIALIZER 108 TableModificationEvent event = DatabaseStateMachine.SERIALIZER
92 .decode(message.payload()); 109 .decode(message.payload());
93 - if (!event.tableName().equals(ONOS_LOCK_TABLE_NAME)) { 110 + if (event.tableName().equals(ONOS_LOCK_TABLE_NAME) &&
94 - return; 111 + event.type().equals(TableModificationEvent.Type.ROW_DELETED)) {
112 + THREAD_POOL.submit(new RetryLockTask(event.key()));
95 } 113 }
114 + }
115 + }
116 +
117 + private class RetryLockTask implements Runnable {
96 118
97 - log.info("Received a lock available event for path: {}", event.key()); 119 + private final String path;
98 120
99 - String path = event.key(); 121 + public RetryLockTask(String path) {
122 + this.path = path;
123 + }
124 +
125 + @Override
126 + public void run() {
100 if (!locksToAcquire.containsKey(path)) { 127 if (!locksToAcquire.containsKey(path)) {
101 return; 128 return;
102 } 129 }
103 130
104 - if (event.type() == TableModificationEvent.Type.ROW_DELETED) {
105 List<LockRequest> existingRequests = locksToAcquire.get(path); 131 List<LockRequest> existingRequests = locksToAcquire.get(path);
106 - if (existingRequests == null) { 132 + if (existingRequests == null || existingRequests.isEmpty()) {
107 return; 133 return;
108 } 134 }
135 + log.info("Path {} is now available for locking. There are {} outstanding "
136 + + "requests for it.",
137 + path, existingRequests.size());
109 138
110 synchronized (existingRequests) { 139 synchronized (existingRequests) {
111 - 140 + Iterator<LockRequest> existingRequestIterator = existingRequests.iterator();
112 - Iterator<LockRequest> existingRequestIterator = existingRequests
113 - .iterator();
114 while (existingRequestIterator.hasNext()) { 141 while (existingRequestIterator.hasNext()) {
115 LockRequest request = existingRequestIterator.next(); 142 LockRequest request = existingRequestIterator.next();
116 - if (request.expirationTime().isAfter(DateTime.now())) { 143 + if (DateTime.now().isAfter(request.requestExpirationTime())) {
144 + // request expired.
117 existingRequestIterator.remove(); 145 existingRequestIterator.remove();
118 } else { 146 } else {
119 - if (request.lock().tryLock( 147 + if (request.lock().tryLock(request.leaseDurationMillis())) {
120 - request.leaseDurationMillis())) { 148 + request.future().complete(DateTime.now().plusMillis(request.leaseDurationMillis()));
121 - request.future().complete(null);
122 existingRequestIterator.remove(); 149 existingRequestIterator.remove();
123 } 150 }
124 } 151 }
...@@ -126,21 +153,19 @@ public class DistributedLockManager implements LockService { ...@@ -126,21 +153,19 @@ public class DistributedLockManager implements LockService {
126 } 153 }
127 } 154 }
128 } 155 }
129 - }
130 156
131 private class LockRequest { 157 private class LockRequest {
132 158
133 private final Lock lock; 159 private final Lock lock;
134 - private final DateTime expirationTime; 160 + private final DateTime requestExpirationTime;
135 private final int leaseDurationMillis; 161 private final int leaseDurationMillis;
136 - private final CompletableFuture<Void> future; 162 + private final CompletableFuture<DateTime> future;
137 163
138 public LockRequest(Lock lock, long waitTimeMillis, 164 public LockRequest(Lock lock, long waitTimeMillis,
139 - int leaseDurationMillis, CompletableFuture<Void> future) { 165 + int leaseDurationMillis, CompletableFuture<DateTime> future) {
140 166
141 this.lock = lock; 167 this.lock = lock;
142 - this.expirationTime = DateTime.now().plusMillis( 168 + this.requestExpirationTime = DateTime.now().plusMillis((int) waitTimeMillis);
143 - (int) waitTimeMillis);
144 this.leaseDurationMillis = leaseDurationMillis; 169 this.leaseDurationMillis = leaseDurationMillis;
145 this.future = future; 170 this.future = future;
146 } 171 }
...@@ -149,15 +174,15 @@ public class DistributedLockManager implements LockService { ...@@ -149,15 +174,15 @@ public class DistributedLockManager implements LockService {
149 return lock; 174 return lock;
150 } 175 }
151 176
152 - public DateTime expirationTime() { 177 + public DateTime requestExpirationTime() {
153 - return expirationTime; 178 + return requestExpirationTime;
154 } 179 }
155 180
156 public int leaseDurationMillis() { 181 public int leaseDurationMillis() {
157 return leaseDurationMillis; 182 return leaseDurationMillis;
158 } 183 }
159 184
160 - public CompletableFuture<Void> future() { 185 + public CompletableFuture<DateTime> future() {
161 return future; 186 return future;
162 } 187 }
163 } 188 }
......
...@@ -86,7 +86,7 @@ public class MapDBLog implements Log { ...@@ -86,7 +86,7 @@ public class MapDBLog implements Log {
86 } 86 }
87 87
88 @Override 88 @Override
89 - public List<Long> appendEntries(List<Entry> entries) { 89 + public synchronized List<Long> appendEntries(List<Entry> entries) {
90 assertIsOpen(); 90 assertIsOpen();
91 checkArgument(entries != null, "expecting non-null entries"); 91 checkArgument(entries != null, "expecting non-null entries");
92 final List<Long> indices = new ArrayList<>(entries.size()); 92 final List<Long> indices = new ArrayList<>(entries.size());
......