Madan Jampani

Bugfixes for DistributedLockManager functionality

Added a method called broadcastIncludeSelf to ClusterCommunicationService.
Cosmetic improvements: added toString methods

Change-Id: I1d58720c29e6f8642f950670c3a6d95a7019a491
...@@ -21,7 +21,9 @@ import static org.slf4j.LoggerFactory.getLogger; ...@@ -21,7 +21,9 @@ import static org.slf4j.LoggerFactory.getLogger;
21 21
22 import java.nio.ByteBuffer; 22 import java.nio.ByteBuffer;
23 import java.util.concurrent.ScheduledExecutorService; 23 import java.util.concurrent.ScheduledExecutorService;
24 -import java.util.concurrent.TimeUnit; 24 +//import java.util.concurrent.TimeUnit;
25 +
26 +
25 27
26 import org.apache.felix.scr.annotations.Activate; 28 import org.apache.felix.scr.annotations.Activate;
27 import org.apache.felix.scr.annotations.Component; 29 import org.apache.felix.scr.annotations.Component;
...@@ -43,6 +45,8 @@ import org.onlab.onos.net.intent.IntentListener; ...@@ -43,6 +45,8 @@ import org.onlab.onos.net.intent.IntentListener;
43 import org.onlab.onos.net.intent.IntentService; 45 import org.onlab.onos.net.intent.IntentService;
44 import org.onlab.onos.store.service.DatabaseAdminService; 46 import org.onlab.onos.store.service.DatabaseAdminService;
45 import org.onlab.onos.store.service.DatabaseService; 47 import org.onlab.onos.store.service.DatabaseService;
48 +import org.onlab.onos.store.service.Lock;
49 +import org.onlab.onos.store.service.LockService;
46 import org.onlab.onos.store.service.VersionedValue; 50 import org.onlab.onos.store.service.VersionedValue;
47 import org.slf4j.Logger; 51 import org.slf4j.Logger;
48 52
...@@ -72,6 +76,9 @@ public class FooComponent { ...@@ -72,6 +76,9 @@ public class FooComponent {
72 @Reference(cardinality = ReferenceCardinality.OPTIONAL_UNARY) 76 @Reference(cardinality = ReferenceCardinality.OPTIONAL_UNARY)
73 protected DatabaseService dbService; 77 protected DatabaseService dbService;
74 78
79 + @Reference(cardinality = ReferenceCardinality.OPTIONAL_UNARY)
80 + protected LockService lockService;
81 +
75 private final ClusterEventListener clusterListener = new InnerClusterListener(); 82 private final ClusterEventListener clusterListener = new InnerClusterListener();
76 private final DeviceListener deviceListener = new InnerDeviceListener(); 83 private final DeviceListener deviceListener = new InnerDeviceListener();
77 private final IntentListener intentListener = new InnerIntentListener(); 84 private final IntentListener intentListener = new InnerIntentListener();
...@@ -92,9 +99,10 @@ public class FooComponent { ...@@ -92,9 +99,10 @@ public class FooComponent {
92 log.info("Couldn't find DB service"); 99 log.info("Couldn't find DB service");
93 } else { 100 } else {
94 log.info("Found DB service"); 101 log.info("Found DB service");
95 - longIncrementor(); 102 + //longIncrementor();
96 - executor.scheduleAtFixedRate(new LongIncrementor(), 1, 10, TimeUnit.SECONDS); 103 + //lockUnlock();
97 - executor.scheduleAtFixedRate(new LongIncrementor(), 1, 10, TimeUnit.SECONDS); 104 + //executor.scheduleAtFixedRate(new LongIncrementor(), 1, 10, TimeUnit.SECONDS);
105 + //executor.scheduleAtFixedRate(new LongIncrementor(), 1, 10, TimeUnit.SECONDS);
98 } 106 }
99 log.info("Started"); 107 log.info("Started");
100 } 108 }
...@@ -152,6 +160,31 @@ public class FooComponent { ...@@ -152,6 +160,31 @@ public class FooComponent {
152 } 160 }
153 } 161 }
154 162
163 + private void lockUnlock() {
164 + try {
165 + final String locksTable = "onos-locks";
166 +
167 + if (!dbAdminService.listTables().contains(locksTable)) {
168 + dbAdminService.createTable(locksTable, 10000);
169 + }
170 + Lock lock = lockService.create("foo-bar");
171 + log.info("Requesting lock");
172 + lock.lock(10000);
173 + //try {
174 + //Thread.sleep(5000);
175 + //} catch (InterruptedException e) {
176 + // TODO Auto-generated catch block
177 + //e.printStackTrace();
178 + //}
179 + log.info("Acquired Lock");
180 + log.info("Do I have the lock: {} ", lock.isLocked());
181 + //lock.unlock();
182 + log.info("Do I have the lock: {} ", lock.isLocked());
183 + } finally {
184 + log.info("Done");
185 + }
186 + }
187 +
155 private void longIncrementor() { 188 private void longIncrementor() {
156 try { 189 try {
157 final String someTable = "admin"; 190 final String someTable = "admin";
......
...@@ -38,6 +38,15 @@ public interface ClusterCommunicationService { ...@@ -38,6 +38,15 @@ public interface ClusterCommunicationService {
38 boolean broadcast(ClusterMessage message) throws IOException; 38 boolean broadcast(ClusterMessage message) throws IOException;
39 39
40 /** 40 /**
41 + * Broadcast a message to all controller nodes including self.
42 + *
43 + * @param message message to send
44 + * @return true if the message was sent successfully to all nodes; false otherwise.
45 + * @throws IOException when I/O exception of some sort has occurred
46 + */
47 + boolean broadcastIncludeSelf(ClusterMessage message) throws IOException;
48 +
49 + /**
41 * Sends a message to the specified controller node. 50 * Sends a message to the specified controller node.
42 * 51 *
43 * @param message message to send 52 * @param message message to send
......
...@@ -2,6 +2,7 @@ package org.onlab.onos.store.service; ...@@ -2,6 +2,7 @@ package org.onlab.onos.store.service;
2 2
3 import java.util.List; 3 import java.util.List;
4 4
5 +import com.google.common.base.MoreObjects;
5 import com.google.common.collect.ImmutableList; 6 import com.google.common.collect.ImmutableList;
6 import com.google.common.collect.Lists; 7 import com.google.common.collect.Lists;
7 8
...@@ -41,6 +42,13 @@ public final class BatchReadRequest { ...@@ -41,6 +42,13 @@ public final class BatchReadRequest {
41 return readRequests; 42 return readRequests;
42 } 43 }
43 44
45 + @Override
46 + public String toString() {
47 + return MoreObjects.toStringHelper(getClass())
48 + .add("readRequests", readRequests)
49 + .toString();
50 + }
51 +
44 /** 52 /**
45 * Builder for BatchReadRequest. 53 * Builder for BatchReadRequest.
46 */ 54 */
......
...@@ -2,6 +2,7 @@ package org.onlab.onos.store.service; ...@@ -2,6 +2,7 @@ package org.onlab.onos.store.service;
2 2
3 import java.util.List; 3 import java.util.List;
4 4
5 +import com.google.common.base.MoreObjects;
5 import com.google.common.collect.ImmutableList; 6 import com.google.common.collect.ImmutableList;
6 7
7 /** 8 /**
...@@ -30,4 +31,11 @@ public class BatchReadResult { ...@@ -30,4 +31,11 @@ public class BatchReadResult {
30 public int batchSize() { 31 public int batchSize() {
31 return readResults.size(); 32 return readResults.size();
32 } 33 }
34 +
35 + @Override
36 + public String toString() {
37 + return MoreObjects.toStringHelper(getClass())
38 + .add("readResults", readResults)
39 + .toString();
40 + }
33 } 41 }
......
...@@ -2,6 +2,7 @@ package org.onlab.onos.store.service; ...@@ -2,6 +2,7 @@ package org.onlab.onos.store.service;
2 2
3 import java.util.List; 3 import java.util.List;
4 4
5 +import com.google.common.base.MoreObjects;
5 import com.google.common.collect.ImmutableList; 6 import com.google.common.collect.ImmutableList;
6 import com.google.common.collect.Lists; 7 import com.google.common.collect.Lists;
7 8
...@@ -41,6 +42,13 @@ public final class BatchWriteRequest { ...@@ -41,6 +42,13 @@ public final class BatchWriteRequest {
41 return writeRequests.size(); 42 return writeRequests.size();
42 } 43 }
43 44
45 + @Override
46 + public String toString() {
47 + return MoreObjects.toStringHelper(getClass())
48 + .add("writeRequests", writeRequests)
49 + .toString();
50 + }
51 +
44 /** 52 /**
45 * Builder for BatchWriteRequest. 53 * Builder for BatchWriteRequest.
46 */ 54 */
......
...@@ -2,6 +2,7 @@ package org.onlab.onos.store.service; ...@@ -2,6 +2,7 @@ package org.onlab.onos.store.service;
2 2
3 import java.util.List; 3 import java.util.List;
4 4
5 +import com.google.common.base.MoreObjects;
5 import com.google.common.collect.ImmutableList; 6 import com.google.common.collect.ImmutableList;
6 7
7 /** 8 /**
...@@ -43,4 +44,11 @@ public class BatchWriteResult { ...@@ -43,4 +44,11 @@ public class BatchWriteResult {
43 public int batchSize() { 44 public int batchSize() {
44 return writeResults.size(); 45 return writeResults.size();
45 } 46 }
47 +
48 + @Override
49 + public String toString() {
50 + return MoreObjects.toStringHelper(getClass())
51 + .add("writeResults", writeResults)
52 + .toString();
53 + }
46 } 54 }
......
1 package org.onlab.onos.store.service; 1 package org.onlab.onos.store.service;
2 2
3 import java.util.Collection; 3 import java.util.Collection;
4 -import java.util.List; 4 +import java.util.Set;
5 5
6 import org.onlab.onos.cluster.ControllerNode; 6 import org.onlab.onos.cluster.ControllerNode;
7 7
...@@ -31,9 +31,9 @@ public interface DatabaseAdminService { ...@@ -31,9 +31,9 @@ public interface DatabaseAdminService {
31 31
32 /** 32 /**
33 * Lists all the tables in the database. 33 * Lists all the tables in the database.
34 - * @return list of table names. 34 + * @return set of table names.
35 */ 35 */
36 - public List<String> listTables(); 36 + public Set<String> listTables();
37 37
38 /** 38 /**
39 * Deletes a table from the database. 39 * Deletes a table from the database.
......
...@@ -64,6 +64,7 @@ public class ReadResult { ...@@ -64,6 +64,7 @@ public class ReadResult {
64 @Override 64 @Override
65 public String toString() { 65 public String toString() {
66 return MoreObjects.toStringHelper(getClass()) 66 return MoreObjects.toStringHelper(getClass())
67 + .add("status", status)
67 .add("tableName", tableName) 68 .add("tableName", tableName)
68 .add("key", key) 69 .add("key", key)
69 .add("value", value) 70 .add("value", value)
......
...@@ -114,7 +114,7 @@ public class WriteRequest { ...@@ -114,7 +114,7 @@ public class WriteRequest {
114 */ 114 */
115 public static WriteRequest removeIfVersionMatches(String tableName, String key, 115 public static WriteRequest removeIfVersionMatches(String tableName, String key,
116 long previousVersion) { 116 long previousVersion) {
117 - return new WriteRequest(REMOVE_IF_VALUE, tableName, key, 117 + return new WriteRequest(REMOVE_IF_VERSION, tableName, key,
118 null, previousVersion, null); 118 null, previousVersion, null);
119 } 119 }
120 120
...@@ -129,7 +129,7 @@ public class WriteRequest { ...@@ -129,7 +129,7 @@ public class WriteRequest {
129 */ 129 */
130 public static WriteRequest removeIfValueMatches(String tableName, String key, 130 public static WriteRequest removeIfValueMatches(String tableName, String key,
131 byte[] oldValue) { 131 byte[] oldValue) {
132 - return new WriteRequest(Type.REMOVE_IF_VALUE, tableName, key, 132 + return new WriteRequest(REMOVE_IF_VALUE, tableName, key,
133 null, ANY_VERSION, checkNotNull(oldValue)); 133 null, ANY_VERSION, checkNotNull(oldValue));
134 } 134 }
135 135
......
...@@ -116,6 +116,15 @@ public class ClusterCommunicationManager ...@@ -116,6 +116,15 @@ public class ClusterCommunicationManager
116 } 116 }
117 117
118 @Override 118 @Override
119 + public boolean broadcastIncludeSelf(ClusterMessage message) throws IOException {
120 + boolean ok = true;
121 + for (ControllerNode node : clusterService.getNodes()) {
122 + ok = unicast(message, node.id()) && ok;
123 + }
124 + return ok;
125 + }
126 +
127 + @Override
119 public boolean multicast(ClusterMessage message, Set<NodeId> nodes) throws IOException { 128 public boolean multicast(ClusterMessage message, Set<NodeId> nodes) throws IOException {
120 boolean ok = true; 129 boolean ok = true;
121 final ControllerNode localNode = clusterService.getLocalNode(); 130 final ControllerNode localNode = clusterService.getLocalNode();
...@@ -209,4 +218,4 @@ public class ClusterCommunicationManager ...@@ -209,4 +218,4 @@ public class ClusterCommunicationManager
209 rawMessage.respond(response); 218 rawMessage.respond(response);
210 } 219 }
211 } 220 }
212 -}
...\ No newline at end of file ...\ No newline at end of file
221 +}
......
...@@ -3,6 +3,7 @@ package org.onlab.onos.store.service.impl; ...@@ -3,6 +3,7 @@ package org.onlab.onos.store.service.impl;
3 import static com.google.common.base.Preconditions.checkNotNull; 3 import static com.google.common.base.Preconditions.checkNotNull;
4 4
5 import java.util.List; 5 import java.util.List;
6 +import java.util.Set;
6 import java.util.concurrent.CompletableFuture; 7 import java.util.concurrent.CompletableFuture;
7 import java.util.concurrent.ExecutionException; 8 import java.util.concurrent.ExecutionException;
8 9
...@@ -37,7 +38,7 @@ public class DatabaseClient { ...@@ -37,7 +38,7 @@ public class DatabaseClient {
37 38
38 public boolean createTable(String tableName, int ttlMillis) { 39 public boolean createTable(String tableName, int ttlMillis) {
39 40
40 - CompletableFuture<Boolean> future = copycat.submit("createTable", tableName, ttlMillis); 41 + CompletableFuture<Boolean> future = copycat.submit("createTableWithExpiration", tableName);
41 try { 42 try {
42 return future.get(); 43 return future.get();
43 } catch (InterruptedException | ExecutionException e) { 44 } catch (InterruptedException | ExecutionException e) {
...@@ -65,9 +66,9 @@ public class DatabaseClient { ...@@ -65,9 +66,9 @@ public class DatabaseClient {
65 } 66 }
66 } 67 }
67 68
68 - public List<String> listTables() { 69 + public Set<String> listTables() {
69 70
70 - CompletableFuture<List<String>> future = copycat.submit("listTables"); 71 + CompletableFuture<Set<String>> future = copycat.submit("listTables");
71 try { 72 try {
72 return future.get(); 73 return future.get();
73 } catch (InterruptedException | ExecutionException e) { 74 } catch (InterruptedException | ExecutionException e) {
......
...@@ -33,7 +33,6 @@ import net.kuujo.copycat.event.LeaderElectEvent; ...@@ -33,7 +33,6 @@ import net.kuujo.copycat.event.LeaderElectEvent;
33 import org.onlab.onos.cluster.ControllerNode; 33 import org.onlab.onos.cluster.ControllerNode;
34 import org.onlab.onos.store.cluster.messaging.ClusterCommunicationService; 34 import org.onlab.onos.store.cluster.messaging.ClusterCommunicationService;
35 import org.onlab.onos.store.cluster.messaging.ClusterMessage; 35 import org.onlab.onos.store.cluster.messaging.ClusterMessage;
36 -import org.onlab.onos.store.cluster.messaging.MessageSubject;
37 import org.onlab.onos.store.service.DatabaseService; 36 import org.onlab.onos.store.service.DatabaseService;
38 import org.onlab.onos.store.service.VersionedValue; 37 import org.onlab.onos.store.service.VersionedValue;
39 import org.onlab.onos.store.service.impl.DatabaseStateMachine.State; 38 import org.onlab.onos.store.service.impl.DatabaseStateMachine.State;
...@@ -51,9 +50,6 @@ public class DatabaseEntryExpirationTracker implements ...@@ -51,9 +50,6 @@ public class DatabaseEntryExpirationTracker implements
51 50
52 private final Logger log = LoggerFactory.getLogger(getClass()); 51 private final Logger log = LoggerFactory.getLogger(getClass());
53 52
54 - public static final MessageSubject DATABASE_UPDATES = new MessageSubject(
55 - "database-update-event");
56 -
57 private final DatabaseService databaseService; 53 private final DatabaseService databaseService;
58 private final ClusterCommunicationService clusterCommunicator; 54 private final ClusterCommunicationService clusterCommunicator;
59 55
...@@ -61,9 +57,9 @@ public class DatabaseEntryExpirationTracker implements ...@@ -61,9 +57,9 @@ public class DatabaseEntryExpirationTracker implements
61 private final ControllerNode localNode; 57 private final ControllerNode localNode;
62 private final AtomicBoolean isLocalMemberLeader = new AtomicBoolean(false); 58 private final AtomicBoolean isLocalMemberLeader = new AtomicBoolean(false);
63 59
64 - private final Map<String, Map<DatabaseRow, VersionedValue>> tableEntryExpirationMap = new HashMap<>(); 60 + private final Map<String, Map<DatabaseRow, Long>> tableEntryExpirationMap = new HashMap<>();
65 61
66 - private final ExpirationListener<DatabaseRow, VersionedValue> expirationObserver = new ExpirationObserver(); 62 + private final ExpirationListener<DatabaseRow, Long> expirationObserver = new ExpirationObserver();
67 63
68 DatabaseEntryExpirationTracker( 64 DatabaseEntryExpirationTracker(
69 Member localMember, 65 Member localMember,
...@@ -78,31 +74,38 @@ public class DatabaseEntryExpirationTracker implements ...@@ -78,31 +74,38 @@ public class DatabaseEntryExpirationTracker implements
78 74
79 @Override 75 @Override
80 public void tableModified(TableModificationEvent event) { 76 public void tableModified(TableModificationEvent event) {
77 + log.debug("Received a table modification event {}", event);
78 +
81 if (!tableEntryExpirationMap.containsKey(event.tableName())) { 79 if (!tableEntryExpirationMap.containsKey(event.tableName())) {
82 return; 80 return;
83 } 81 }
84 82
83 + Map<DatabaseRow, Long> map = tableEntryExpirationMap.get(event.tableName());
85 DatabaseRow row = new DatabaseRow(event.tableName(), event.key()); 84 DatabaseRow row = new DatabaseRow(event.tableName(), event.key());
86 - Map<DatabaseRow, VersionedValue> map = tableEntryExpirationMap 85 + Long eventVersion = event.value().version();
87 - .get(event.tableName());
88 86
89 switch (event.type()) { 87 switch (event.type()) {
90 case ROW_DELETED: 88 case ROW_DELETED:
89 + map.remove(row, eventVersion);
91 if (isLocalMemberLeader.get()) { 90 if (isLocalMemberLeader.get()) {
92 try { 91 try {
92 + // FIXME: The broadcast message should be sent to self.
93 clusterCommunicator.broadcast(new ClusterMessage( 93 clusterCommunicator.broadcast(new ClusterMessage(
94 - localNode.id(), DATABASE_UPDATES, 94 + localNode.id(), DatabaseStateMachine.DATABASE_UPDATE_EVENTS,
95 DatabaseStateMachine.SERIALIZER.encode(event))); 95 DatabaseStateMachine.SERIALIZER.encode(event)));
96 } catch (IOException e) { 96 } catch (IOException e) {
97 - log.error( 97 + log.error("Failed to broadcast a database row deleted event.", e);
98 - "Failed to broadcast a database table modification event.",
99 - e);
100 } 98 }
101 } 99 }
102 break; 100 break;
103 case ROW_ADDED: 101 case ROW_ADDED:
104 case ROW_UPDATED: 102 case ROW_UPDATED:
105 - map.put(row, null); 103 + // To account for potential reordering of notifications,
104 + // check to make sure we are replacing an old version with a new version
105 + Long currentVersion = map.get(row);
106 + if (currentVersion == null || currentVersion < eventVersion) {
107 + map.put(row, eventVersion);
108 + }
106 break; 109 break;
107 default: 110 default:
108 break; 111 break;
...@@ -111,60 +114,56 @@ public class DatabaseEntryExpirationTracker implements ...@@ -111,60 +114,56 @@ public class DatabaseEntryExpirationTracker implements
111 114
112 @Override 115 @Override
113 public void tableCreated(TableMetadata metadata) { 116 public void tableCreated(TableMetadata metadata) {
117 + log.debug("Received a table created event {}", metadata);
114 if (metadata.expireOldEntries()) { 118 if (metadata.expireOldEntries()) {
115 tableEntryExpirationMap.put(metadata.tableName(), ExpiringMap.builder() 119 tableEntryExpirationMap.put(metadata.tableName(), ExpiringMap.builder()
116 - .expiration(metadata.ttlMillis(), TimeUnit.SECONDS) 120 + .expiration(metadata.ttlMillis(), TimeUnit.MILLISECONDS)
117 .expirationListener(expirationObserver) 121 .expirationListener(expirationObserver)
118 - // FIXME: make the expiration policy configurable. 122 + // TODO: make the expiration policy configurable.
123 + // Do we need to support expiration based on last access time?
119 .expirationPolicy(ExpirationPolicy.CREATED).build()); 124 .expirationPolicy(ExpirationPolicy.CREATED).build());
120 } 125 }
121 } 126 }
122 127
123 @Override 128 @Override
124 public void tableDeleted(String tableName) { 129 public void tableDeleted(String tableName) {
130 + log.debug("Received a table deleted event for table ({})", tableName);
125 tableEntryExpirationMap.remove(tableName); 131 tableEntryExpirationMap.remove(tableName);
126 } 132 }
127 133
128 private class ExpirationObserver implements 134 private class ExpirationObserver implements
129 - ExpirationListener<DatabaseRow, VersionedValue> { 135 + ExpirationListener<DatabaseRow, Long> {
130 @Override 136 @Override
131 - public void expired(DatabaseRow key, VersionedValue value) { 137 + public void expired(DatabaseRow row, Long version) {
138 + Map<DatabaseRow, Long> map = tableEntryExpirationMap.get(row.tableName);
132 try { 139 try {
133 if (isLocalMemberLeader.get()) { 140 if (isLocalMemberLeader.get()) {
134 - if (!databaseService.removeIfVersionMatches(key.tableName, 141 + if (!databaseService.removeIfVersionMatches(row.tableName,
135 - key.key, value.version())) { 142 + row.key, version)) {
136 - log.info("Entry in the database changed before right its TTL expiration."); 143 + log.info("Entry in database was updated right before its expiration.");
144 + } else {
145 + log.info("Successfully expired old entry with key ({}) from table ({})",
146 + row.key, row.tableName);
137 } 147 }
138 } else { 148 } else {
139 - // If this node is not the current leader, we should never 149 + // Only the current leader will expire keys from database.
140 - // let the expiring entries drop off 150 + // Everyone else function as standby just in case they need to take over
141 - // Under stable conditions (i.e no leadership switch) the
142 - // current leader will initiate
143 - // a database remove and this instance will get notified
144 - // of a tableModification event causing it to remove from
145 - // the map.
146 - Map<DatabaseRow, VersionedValue> map = tableEntryExpirationMap
147 - .get(key.tableName);
148 if (map != null) { 151 if (map != null) {
149 - map.put(key, value); 152 + map.putIfAbsent(row, version);
150 } 153 }
151 } 154 }
152 155
153 } catch (Exception e) { 156 } catch (Exception e) {
154 - log.warn( 157 + log.warn("Failed to delete entry from the database after ttl "
155 - "Failed to delete entry from the database after ttl expiration. Will retry eviction", 158 + + "expiration. Operation will be retried.", e);
156 - e); 159 + map.putIfAbsent(row, version);
157 - tableEntryExpirationMap.get(key.tableName).put(
158 - new DatabaseRow(key.tableName, key.key), value);
159 } 160 }
160 } 161 }
161 } 162 }
162 163
163 @Override 164 @Override
164 public void handle(LeaderElectEvent event) { 165 public void handle(LeaderElectEvent event) {
165 - if (localMember.equals(event.leader())) { 166 + isLocalMemberLeader.set(localMember.equals(event.leader()));
166 - isLocalMemberLeader.set(true);
167 - }
168 } 167 }
169 168
170 /** 169 /**
...@@ -212,12 +211,12 @@ public class DatabaseEntryExpirationTracker implements ...@@ -212,12 +211,12 @@ public class DatabaseEntryExpirationTracker implements
212 continue; 211 continue;
213 } 212 }
214 213
215 - Map<DatabaseRow, VersionedValue> tableExpirationMap = ExpiringMap.builder() 214 + Map<DatabaseRow, Long> tableExpirationMap = ExpiringMap.builder()
216 .expiration(metadata.ttlMillis(), TimeUnit.MILLISECONDS) 215 .expiration(metadata.ttlMillis(), TimeUnit.MILLISECONDS)
217 .expirationListener(expirationObserver) 216 .expirationListener(expirationObserver)
218 .expirationPolicy(ExpirationPolicy.CREATED).build(); 217 .expirationPolicy(ExpirationPolicy.CREATED).build();
219 for (Map.Entry<String, VersionedValue> entry : state.getTable(tableName).entrySet()) { 218 for (Map.Entry<String, VersionedValue> entry : state.getTable(tableName).entrySet()) {
220 - tableExpirationMap.put(new DatabaseRow(tableName, entry.getKey()), entry.getValue()); 219 + tableExpirationMap.put(new DatabaseRow(tableName, entry.getKey()), entry.getValue().version());
221 } 220 }
222 221
223 tableEntryExpirationMap.put(tableName, tableExpirationMap); 222 tableEntryExpirationMap.put(tableName, tableExpirationMap);
......
...@@ -7,7 +7,6 @@ import java.io.IOException; ...@@ -7,7 +7,6 @@ import java.io.IOException;
7 import java.util.Collection; 7 import java.util.Collection;
8 import java.util.Collections; 8 import java.util.Collections;
9 import java.util.HashSet; 9 import java.util.HashSet;
10 -import java.util.List;
11 import java.util.Map; 10 import java.util.Map;
12 import java.util.Set; 11 import java.util.Set;
13 import java.util.concurrent.CountDownLatch; 12 import java.util.concurrent.CountDownLatch;
...@@ -19,6 +18,7 @@ import net.kuujo.copycat.cluster.Member; ...@@ -19,6 +18,7 @@ import net.kuujo.copycat.cluster.Member;
19 import net.kuujo.copycat.cluster.TcpCluster; 18 import net.kuujo.copycat.cluster.TcpCluster;
20 import net.kuujo.copycat.cluster.TcpClusterConfig; 19 import net.kuujo.copycat.cluster.TcpClusterConfig;
21 import net.kuujo.copycat.cluster.TcpMember; 20 import net.kuujo.copycat.cluster.TcpMember;
21 +import net.kuujo.copycat.event.LeaderElectEvent;
22 import net.kuujo.copycat.log.Log; 22 import net.kuujo.copycat.log.Log;
23 23
24 import org.apache.felix.scr.annotations.Activate; 24 import org.apache.felix.scr.annotations.Activate;
...@@ -160,18 +160,22 @@ public class DatabaseManager implements DatabaseService, DatabaseAdminService { ...@@ -160,18 +160,22 @@ public class DatabaseManager implements DatabaseService, DatabaseAdminService {
160 } 160 }
161 log.info("Starting cluster: {}", cluster); 161 log.info("Starting cluster: {}", cluster);
162 162
163 + DatabaseEntryExpirationTracker expirationTracker =
164 + new DatabaseEntryExpirationTracker(
165 + clusterConfig.getLocalMember(),
166 + clusterService.getLocalNode(),
167 + clusterCommunicator,
168 + this);
163 169
164 DatabaseStateMachine stateMachine = new DatabaseStateMachine(); 170 DatabaseStateMachine stateMachine = new DatabaseStateMachine();
165 - stateMachine.addEventListener( 171 + stateMachine.addEventListener(expirationTracker);
166 - new DatabaseEntryExpirationTracker(
167 - clusterConfig.getLocalMember(),
168 - clusterService.getLocalNode(),
169 - clusterCommunicator,
170 - this));
171 Log consensusLog = new MapDBLog(LOG_FILE_PREFIX + localNode.id(), 172 Log consensusLog = new MapDBLog(LOG_FILE_PREFIX + localNode.id(),
172 ClusterMessagingProtocol.SERIALIZER); 173 ClusterMessagingProtocol.SERIALIZER);
173 174
174 copycat = new Copycat(stateMachine, consensusLog, cluster, copycatMessagingProtocol); 175 copycat = new Copycat(stateMachine, consensusLog, cluster, copycatMessagingProtocol);
176 +
177 + copycat.event(LeaderElectEvent.class).registerHandler(expirationTracker);
178 +
175 copycat.start(); 179 copycat.start();
176 180
177 client = new DatabaseClient(copycat); 181 client = new DatabaseClient(copycat);
...@@ -207,7 +211,7 @@ public class DatabaseManager implements DatabaseService, DatabaseAdminService { ...@@ -207,7 +211,7 @@ public class DatabaseManager implements DatabaseService, DatabaseAdminService {
207 } 211 }
208 212
209 @Override 213 @Override
210 - public List<String> listTables() { 214 + public Set<String> listTables() {
211 return client.listTables(); 215 return client.listTables();
212 } 216 }
213 217
......
...@@ -30,6 +30,7 @@ import org.onlab.onos.store.service.WriteStatus; ...@@ -30,6 +30,7 @@ import org.onlab.onos.store.service.WriteStatus;
30 import org.onlab.util.KryoNamespace; 30 import org.onlab.util.KryoNamespace;
31 import org.slf4j.Logger; 31 import org.slf4j.Logger;
32 32
33 +import com.google.common.base.MoreObjects;
33 import com.google.common.collect.ImmutableSet; 34 import com.google.common.collect.ImmutableSet;
34 import com.google.common.collect.Lists; 35 import com.google.common.collect.Lists;
35 import com.google.common.collect.Maps; 36 import com.google.common.collect.Maps;
...@@ -65,6 +66,7 @@ public class DatabaseStateMachine implements StateMachine { ...@@ -65,6 +66,7 @@ public class DatabaseStateMachine implements StateMachine {
65 .register(WriteStatus.class) 66 .register(WriteStatus.class)
66 // TODO: Move this out ? 67 // TODO: Move this out ?
67 .register(TableModificationEvent.class) 68 .register(TableModificationEvent.class)
69 + .register(TableModificationEvent.Type.class)
68 .register(ClusterMessagingProtocol.COMMON) 70 .register(ClusterMessagingProtocol.COMMON)
69 .build() 71 .build()
70 .populate(1); 72 .populate(1);
...@@ -85,7 +87,8 @@ public class DatabaseStateMachine implements StateMachine { ...@@ -85,7 +87,8 @@ public class DatabaseStateMachine implements StateMachine {
85 } 87 }
86 88
87 @Command 89 @Command
88 - public boolean createTable(String tableName, int ttlMillis) { 90 + public boolean createTableWithExpiration(String tableName) {
91 + int ttlMillis = 10000;
89 TableMetadata metadata = new TableMetadata(tableName, ttlMillis); 92 TableMetadata metadata = new TableMetadata(tableName, ttlMillis);
90 return createTable(metadata); 93 return createTable(metadata);
91 } 94 }
...@@ -266,6 +269,7 @@ public class DatabaseStateMachine implements StateMachine { ...@@ -266,6 +269,7 @@ public class DatabaseStateMachine implements StateMachine {
266 // notify listeners of table mod events. 269 // notify listeners of table mod events.
267 for (DatabaseUpdateEventListener listener : listeners) { 270 for (DatabaseUpdateEventListener listener : listeners) {
268 for (TableModificationEvent tableModificationEvent : tableModificationEvents) { 271 for (TableModificationEvent tableModificationEvent : tableModificationEvents) {
272 + log.info("Publishing table modification event: {}", tableModificationEvent);
269 listener.tableModified(tableModificationEvent); 273 listener.tableModified(tableModificationEvent);
270 } 274 }
271 } 275 }
...@@ -345,6 +349,15 @@ public class DatabaseStateMachine implements StateMachine { ...@@ -345,6 +349,15 @@ public class DatabaseStateMachine implements StateMachine {
345 public int ttlMillis() { 349 public int ttlMillis() {
346 return ttlMillis; 350 return ttlMillis;
347 } 351 }
352 +
353 + @Override
354 + public String toString() {
355 + return MoreObjects.toStringHelper(getClass())
356 + .add("tableName", tableName)
357 + .add("expireOldEntries", expireOldEntries)
358 + .add("ttlMillis", ttlMillis)
359 + .toString();
360 + }
348 } 361 }
349 362
350 @Override 363 @Override
......
...@@ -80,6 +80,7 @@ public class DistributedLock implements Lock { ...@@ -80,6 +80,7 @@ public class DistributedLock implements Lock {
80 return false; 80 return false;
81 } 81 }
82 } 82 }
83 + isLocked.set(true);
83 lockExpirationTime = DateTime.now().plusMillis(leaseDurationMillis); 84 lockExpirationTime = DateTime.now().plusMillis(leaseDurationMillis);
84 return true; 85 return true;
85 } 86 }
...@@ -95,9 +96,11 @@ public class DistributedLock implements Lock { ...@@ -95,9 +96,11 @@ public class DistributedLock implements Lock {
95 if (DateTime.now().isAfter(lockExpirationTime)) { 96 if (DateTime.now().isAfter(lockExpirationTime)) {
96 isLocked.set(false); 97 isLocked.set(false);
97 return false; 98 return false;
99 + } else {
100 + return true;
98 } 101 }
99 } 102 }
100 - return true; 103 + return false;
101 } 104 }
102 105
103 @Override 106 @Override
...@@ -105,6 +108,7 @@ public class DistributedLock implements Lock { ...@@ -105,6 +108,7 @@ public class DistributedLock implements Lock {
105 if (!isLocked()) { 108 if (!isLocked()) {
106 return; 109 return;
107 } else { 110 } else {
111 + isLocked.set(false);
108 databaseService.removeIfValueMatches(DistributedLockManager.ONOS_LOCK_TABLE_NAME, path, lockId); 112 databaseService.removeIfValueMatches(DistributedLockManager.ONOS_LOCK_TABLE_NAME, path, lockId);
109 } 113 }
110 } 114 }
......
...@@ -94,6 +94,8 @@ public class DistributedLockManager implements LockService { ...@@ -94,6 +94,8 @@ public class DistributedLockManager implements LockService {
94 return; 94 return;
95 } 95 }
96 96
97 + log.info("Received a lock available event for path: {}", event.key());
98 +
97 String path = event.key(); 99 String path = event.key();
98 if (!locksToAcquire.containsKey(path)) { 100 if (!locksToAcquire.containsKey(path)) {
99 return; 101 return;
...@@ -159,4 +161,4 @@ public class DistributedLockManager implements LockService { ...@@ -159,4 +161,4 @@ public class DistributedLockManager implements LockService {
159 return future; 161 return future;
160 } 162 }
161 } 163 }
162 -}
...\ No newline at end of file ...\ No newline at end of file
164 +}
......
...@@ -2,6 +2,8 @@ package org.onlab.onos.store.service.impl; ...@@ -2,6 +2,8 @@ package org.onlab.onos.store.service.impl;
2 2
3 import org.onlab.onos.store.service.VersionedValue; 3 import org.onlab.onos.store.service.VersionedValue;
4 4
5 +import com.google.common.base.MoreObjects;
6 +
5 /** 7 /**
6 * A table modification event. 8 * A table modification event.
7 */ 9 */
...@@ -9,7 +11,6 @@ public final class TableModificationEvent { ...@@ -9,7 +11,6 @@ public final class TableModificationEvent {
9 11
10 /** 12 /**
11 * Type of table modification event. 13 * Type of table modification event.
12 - *
13 */ 14 */
14 public enum Type { 15 public enum Type {
15 ROW_ADDED, 16 ROW_ADDED,
...@@ -94,4 +95,14 @@ public final class TableModificationEvent { ...@@ -94,4 +95,14 @@ public final class TableModificationEvent {
94 public Type type() { 95 public Type type() {
95 return type; 96 return type;
96 } 97 }
98 +
99 + @Override
100 + public String toString() {
101 + return MoreObjects.toStringHelper(getClass())
102 + .add("type", type)
103 + .add("tableName", tableName)
104 + .add("key", key)
105 + .add("version", value.version())
106 + .toString();
107 + }
97 } 108 }
......