Skip to content

Commit bd4e81a

Browse files
committed
Make single constructor and override getters
1 parent 8212985 commit bd4e81a

File tree

3 files changed

+124
-106
lines changed

3 files changed

+124
-106
lines changed

server/gc/src/main/java/org/apache/accumulo/gc/GarbageCollectWriteAheadLogs.java

+42-61
Original file line numberDiff line numberDiff line change
@@ -26,7 +26,6 @@
2626

2727
import java.io.FileNotFoundException;
2828
import java.io.IOException;
29-
import java.util.ArrayList;
3029
import java.util.Collection;
3130
import java.util.HashMap;
3231
import java.util.HashSet;
@@ -36,6 +35,7 @@
3635
import java.util.Set;
3736
import java.util.UUID;
3837
import java.util.concurrent.atomic.AtomicBoolean;
38+
import java.util.stream.Stream;
3939

4040
import org.apache.accumulo.core.gc.thrift.GCStatus;
4141
import org.apache.accumulo.core.gc.thrift.GcCycleStats;
@@ -61,20 +61,18 @@
6161

6262
import com.google.common.annotations.VisibleForTesting;
6363
import com.google.common.base.Preconditions;
64-
import com.google.common.collect.Iterators;
64+
import com.google.common.collect.Streams;
6565

6666
import io.opentelemetry.api.trace.Span;
6767
import io.opentelemetry.context.Scope;
6868

69-
public class GarbageCollectWriteAheadLogs implements AutoCloseable {
69+
public class GarbageCollectWriteAheadLogs {
7070
private static final Logger log = LoggerFactory.getLogger(GarbageCollectWriteAheadLogs.class);
7171

7272
private final ServerContext context;
7373
private final VolumeManager fs;
7474
private final LiveTServerSet liveServers;
7575
private final WalStateManager walMarker;
76-
private final Iterable<TabletMetadata> store;
77-
private final List<AutoCloseable> closeables = new ArrayList<>();
7876
private final AtomicBoolean hasCollected;
7977

8078
/**
@@ -88,39 +86,28 @@ public class GarbageCollectWriteAheadLogs implements AutoCloseable {
8886
this.context = context;
8987
this.fs = fs;
9088
this.liveServers = liveServers;
91-
this.walMarker = new WalStateManager(context);
89+
this.walMarker = getWalStateManager();
90+
this.hasCollected = new AtomicBoolean(false);
91+
}
9292

93+
@VisibleForTesting
94+
WalStateManager getWalStateManager() {
95+
return new WalStateManager(context);
96+
}
97+
98+
@VisibleForTesting
99+
Stream<TabletMetadata> getStore() {
93100
TabletsMetadata root = context.getAmple().readTablets().forLevel(DataLevel.ROOT)
94101
.filter(new HasWalsFilter()).fetch(LOCATION, LAST, LOGS, PREV_ROW, SUSPEND).build();
95102
TabletsMetadata metadata = context.getAmple().readTablets().forLevel(DataLevel.METADATA)
96103
.filter(new HasWalsFilter()).fetch(LOCATION, LAST, LOGS, PREV_ROW, SUSPEND).build();
97104
TabletsMetadata user = context.getAmple().readTablets().forLevel(DataLevel.USER)
98105
.filter(new HasWalsFilter()).fetch(LOCATION, LAST, LOGS, PREV_ROW, SUSPEND).build();
99-
100-
closeables.add(root);
101-
closeables.add(metadata);
102-
closeables.add(user);
103-
104-
this.store = () -> Iterators.concat(root.iterator(), metadata.iterator(), user.iterator());
105-
this.hasCollected = new AtomicBoolean(false);
106-
}
107-
108-
/**
109-
* Creates a new GC WAL object. Meant for testing -- allows mocked objects.
110-
*
111-
* @param context the collection server's context
112-
* @param fs volume manager to use
113-
* @param liveTServerSet a started LiveTServerSet instance
114-
*/
115-
@VisibleForTesting
116-
GarbageCollectWriteAheadLogs(ServerContext context, VolumeManager fs,
117-
LiveTServerSet liveTServerSet, WalStateManager walMarker, Iterable<TabletMetadata> store) {
118-
this.context = context;
119-
this.fs = fs;
120-
this.liveServers = liveTServerSet;
121-
this.walMarker = walMarker;
122-
this.store = store;
123-
this.hasCollected = new AtomicBoolean(false);
106+
return Streams.concat(root.stream(), metadata.stream(), user.stream()).onClose(() -> {
107+
root.close();
108+
metadata.close();
109+
user.close();
110+
});
124111
}
125112

126113
public void collect(GCStatus status) {
@@ -294,28 +281,32 @@ private Map<UUID,TServerInstance> removeEntriesInUse(Map<TServerInstance,Set<UUI
294281
}
295282

296283
// remove any entries if there's a log reference (recovery hasn't finished)
297-
for (TabletMetadata tabletMetadata : store) {
298-
// Tablet is still assigned to a dead server. Manager has moved markers and reassigned it
299-
// Easiest to just ignore all the WALs for the dead server.
300-
if (TabletState.compute(tabletMetadata, liveServers) == TabletState.ASSIGNED_TO_DEAD_SERVER) {
301-
Set<UUID> idsToIgnore = candidates.remove(tabletMetadata.getLocation().getServerInstance());
302-
if (idsToIgnore != null) {
303-
result.keySet().removeAll(idsToIgnore);
304-
recoveryLogs.keySet().removeAll(idsToIgnore);
284+
try (Stream<TabletMetadata> store = getStore()) {
285+
store.forEach(tabletMetadata -> {
286+
// Tablet is still assigned to a dead server. Manager has moved markers and reassigned it
287+
// Easiest to just ignore all the WALs for the dead server.
288+
if (TabletState.compute(tabletMetadata, liveServers)
289+
== TabletState.ASSIGNED_TO_DEAD_SERVER) {
290+
Set<UUID> idsToIgnore =
291+
candidates.remove(tabletMetadata.getLocation().getServerInstance());
292+
if (idsToIgnore != null) {
293+
result.keySet().removeAll(idsToIgnore);
294+
recoveryLogs.keySet().removeAll(idsToIgnore);
295+
}
305296
}
306-
}
307-
// Tablet is being recovered and has WAL references, remove all the WALs for the dead server
308-
// that made the WALs.
309-
for (LogEntry wal : tabletMetadata.getLogs()) {
310-
UUID walUUID = wal.getUniqueID();
311-
TServerInstance dead = result.get(walUUID);
312-
// There's a reference to a log file, so skip that server's logs
313-
Set<UUID> idsToIgnore = candidates.remove(dead);
314-
if (idsToIgnore != null) {
315-
result.keySet().removeAll(idsToIgnore);
316-
recoveryLogs.keySet().removeAll(idsToIgnore);
297+
// Tablet is being recovered and has WAL references, remove all the WALs for the dead server
298+
// that made the WALs.
299+
for (LogEntry wal : tabletMetadata.getLogs()) {
300+
UUID walUUID = wal.getUniqueID();
301+
TServerInstance dead = result.get(walUUID);
302+
// There's a reference to a log file, so skip that server's logs
303+
Set<UUID> idsToIgnore = candidates.remove(dead);
304+
if (idsToIgnore != null) {
305+
result.keySet().removeAll(idsToIgnore);
306+
recoveryLogs.keySet().removeAll(idsToIgnore);
307+
}
317308
}
318-
}
309+
});
319310
}
320311

321312
// Remove OPEN and CLOSED logs for live servers: they are still in use
@@ -385,14 +376,4 @@ protected Map<UUID,Path> getSortedWALogs() throws IOException {
385376
return result;
386377
}
387378

388-
@Override
389-
public void close() throws Exception {
390-
for (AutoCloseable closable : closeables) {
391-
try {
392-
closable.close();
393-
} catch (Exception e) {
394-
throw new RuntimeException(e);
395-
}
396-
}
397-
}
398379
}

server/gc/src/main/java/org/apache/accumulo/gc/SimpleGarbageCollector.java

+4-5
Original file line numberDiff line numberDiff line change
@@ -232,11 +232,10 @@ public void run() {
232232
// Clean up any unused write-ahead logs
233233
Span walSpan = TraceUtil.startSpan(this.getClass(), "walogs");
234234
try (Scope walScope = walSpan.makeCurrent()) {
235-
try (GarbageCollectWriteAheadLogs walogCollector =
236-
new GarbageCollectWriteAheadLogs(getContext(), fs, liveTServerSet)) {
237-
log.info("Beginning garbage collection of write-ahead logs");
238-
walogCollector.collect(status);
239-
}
235+
GarbageCollectWriteAheadLogs walogCollector =
236+
new GarbageCollectWriteAheadLogs(getContext(), fs, liveTServerSet);
237+
log.info("Beginning garbage collection of write-ahead logs");
238+
walogCollector.collect(status);
240239
gcCycleMetrics.setLastWalCollect(status.lastLog);
241240
} catch (Exception e) {
242241
TraceUtil.setException(walSpan, e, false);

server/gc/src/test/java/org/apache/accumulo/gc/GarbageCollectWriteAheadLogsTest.java

+78-40
Original file line numberDiff line numberDiff line change
@@ -21,11 +21,13 @@
2121
import static org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType.LAST;
2222
import static org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType.LOGS;
2323
import static org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType.SUSPEND;
24+
import static org.junit.jupiter.api.Assertions.assertThrows;
2425

2526
import java.util.Collections;
2627
import java.util.List;
2728
import java.util.Map;
2829
import java.util.UUID;
30+
import java.util.stream.Stream;
2931

3032
import org.apache.accumulo.core.client.admin.TabletAvailability;
3133
import org.apache.accumulo.core.dataImpl.KeyExtent;
@@ -74,10 +76,8 @@ public class GarbageCollectWriteAheadLogsTest {
7476
}
7577
}
7678

77-
private final Iterable<TabletMetadata> tabletOnServer1List =
78-
Collections.singletonList(tabletAssignedToServer1);
79-
private final Iterable<TabletMetadata> tabletOnServer2List =
80-
Collections.singletonList(tabletAssignedToServer2);
79+
private final Stream<TabletMetadata> tabletOnServer1List = Stream.of(tabletAssignedToServer1);
80+
private final Stream<TabletMetadata> tabletOnServer2List = Stream.of(tabletAssignedToServer2);
8181

8282
@Test
8383
public void testRemoveUnusedLog() throws Exception {
@@ -99,15 +99,26 @@ public void testRemoveUnusedLog() throws Exception {
9999
marker.removeWalMarker(server1, id);
100100
EasyMock.expectLastCall().once();
101101
EasyMock.replay(context, fs, marker, tserverSet);
102-
try (GarbageCollectWriteAheadLogs gc =
103-
new GarbageCollectWriteAheadLogs(context, fs, tserverSet, marker, tabletOnServer1List) {
104-
@Override
105-
protected Map<UUID,Path> getSortedWALogs() {
106-
return Collections.emptyMap();
107-
}
108-
}) {
109-
gc.collect(status);
110-
}
102+
GarbageCollectWriteAheadLogs gc = new GarbageCollectWriteAheadLogs(context, fs, tserverSet) {
103+
@Override
104+
protected Map<UUID,Path> getSortedWALogs() {
105+
return Collections.emptyMap();
106+
}
107+
108+
@Override
109+
WalStateManager getWalStateManager() {
110+
return marker;
111+
}
112+
113+
@Override
114+
Stream<TabletMetadata> getStore() {
115+
return tabletOnServer1List;
116+
}
117+
};
118+
gc.collect(status);
119+
assertThrows(IllegalStateException.class, () -> gc.collect(status),
120+
"Should only be able to call collect once");
121+
111122
EasyMock.verify(context, fs, marker, tserverSet);
112123
}
113124

@@ -127,15 +138,24 @@ public void testKeepClosedLog() throws Exception {
127138
EasyMock.expect(marker.getAllMarkers()).andReturn(markers).once();
128139
EasyMock.expect(marker.state(server1, id)).andReturn(new Pair<>(WalState.CLOSED, path));
129140
EasyMock.replay(context, marker, tserverSet, fs);
130-
try (GarbageCollectWriteAheadLogs gc =
131-
new GarbageCollectWriteAheadLogs(context, fs, tserverSet, marker, tabletOnServer1List) {
132-
@Override
133-
protected Map<UUID,Path> getSortedWALogs() {
134-
return Collections.emptyMap();
135-
}
136-
}) {
137-
gc.collect(status);
138-
}
141+
GarbageCollectWriteAheadLogs gc = new GarbageCollectWriteAheadLogs(context, fs, tserverSet) {
142+
@Override
143+
protected Map<UUID,Path> getSortedWALogs() {
144+
return Collections.emptyMap();
145+
}
146+
147+
@Override
148+
WalStateManager getWalStateManager() {
149+
return marker;
150+
}
151+
152+
@Override
153+
Stream<TabletMetadata> getStore() {
154+
return tabletOnServer1List;
155+
}
156+
};
157+
gc.collect(status);
158+
139159
EasyMock.verify(context, marker, tserverSet, fs);
140160
}
141161

@@ -162,15 +182,24 @@ public void deleteUnreferenceLogOnDeadServer() throws Exception {
162182
marker.forget(server2);
163183
EasyMock.expectLastCall().once();
164184
EasyMock.replay(context, fs, marker, tserverSet);
165-
try (GarbageCollectWriteAheadLogs gc =
166-
new GarbageCollectWriteAheadLogs(context, fs, tserverSet, marker, tabletOnServer1List) {
167-
@Override
168-
protected Map<UUID,Path> getSortedWALogs() {
169-
return Collections.emptyMap();
170-
}
171-
}) {
172-
gc.collect(status);
173-
}
185+
GarbageCollectWriteAheadLogs gc = new GarbageCollectWriteAheadLogs(context, fs, tserverSet) {
186+
@Override
187+
protected Map<UUID,Path> getSortedWALogs() {
188+
return Collections.emptyMap();
189+
}
190+
191+
@Override
192+
WalStateManager getWalStateManager() {
193+
return marker;
194+
}
195+
196+
@Override
197+
Stream<TabletMetadata> getStore() {
198+
return tabletOnServer1List;
199+
}
200+
};
201+
gc.collect(status);
202+
174203
EasyMock.verify(context, fs, marker, tserverSet);
175204
}
176205

@@ -191,15 +220,24 @@ public void ignoreReferenceLogOnDeadServer() throws Exception {
191220
EasyMock.expect(marker.state(server2, id)).andReturn(new Pair<>(WalState.OPEN, path));
192221

193222
EasyMock.replay(context, fs, marker, tserverSet);
194-
try (GarbageCollectWriteAheadLogs gc =
195-
new GarbageCollectWriteAheadLogs(context, fs, tserverSet, marker, tabletOnServer2List) {
196-
@Override
197-
protected Map<UUID,Path> getSortedWALogs() {
198-
return Collections.emptyMap();
199-
}
200-
}) {
201-
gc.collect(status);
202-
}
223+
GarbageCollectWriteAheadLogs gc = new GarbageCollectWriteAheadLogs(context, fs, tserverSet) {
224+
@Override
225+
protected Map<UUID,Path> getSortedWALogs() {
226+
return Collections.emptyMap();
227+
}
228+
229+
@Override
230+
WalStateManager getWalStateManager() {
231+
return marker;
232+
}
233+
234+
@Override
235+
Stream<TabletMetadata> getStore() {
236+
return tabletOnServer2List;
237+
}
238+
};
239+
gc.collect(status);
240+
203241
EasyMock.verify(context, fs, marker, tserverSet);
204242
}
205243

0 commit comments

Comments
 (0)