Skip to content

Commit 8212985

Browse files
committed
Make GCWAL autoclosable, revert back to iterator
1 parent 0ec481a commit 8212985

File tree

3 files changed

+58
-33
lines changed

3 files changed

+58
-33
lines changed

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

+33-14
Original file line numberDiff line numberDiff line change
@@ -26,6 +26,7 @@
2626

2727
import java.io.FileNotFoundException;
2828
import java.io.IOException;
29+
import java.util.ArrayList;
2930
import java.util.Collection;
3031
import java.util.HashMap;
3132
import java.util.HashSet;
@@ -34,7 +35,7 @@
3435
import java.util.Map.Entry;
3536
import java.util.Set;
3637
import java.util.UUID;
37-
import java.util.stream.Stream;
38+
import java.util.concurrent.atomic.AtomicBoolean;
3839

3940
import org.apache.accumulo.core.gc.thrift.GCStatus;
4041
import org.apache.accumulo.core.gc.thrift.GcCycleStats;
@@ -59,19 +60,22 @@
5960
import org.slf4j.LoggerFactory;
6061

6162
import com.google.common.annotations.VisibleForTesting;
62-
import com.google.common.collect.Streams;
63+
import com.google.common.base.Preconditions;
64+
import com.google.common.collect.Iterators;
6365

6466
import io.opentelemetry.api.trace.Span;
6567
import io.opentelemetry.context.Scope;
6668

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

7072
private final ServerContext context;
7173
private final VolumeManager fs;
7274
private final LiveTServerSet liveServers;
7375
private final WalStateManager walMarker;
74-
private final Stream<TabletMetadata> store;
76+
private final Iterable<TabletMetadata> store;
77+
private final List<AutoCloseable> closeables = new ArrayList<>();
78+
private final AtomicBoolean hasCollected;
7579

7680
/**
7781
* Creates a new GC WAL object.
@@ -85,17 +89,20 @@ public class GarbageCollectWriteAheadLogs {
8589
this.fs = fs;
8690
this.liveServers = liveServers;
8791
this.walMarker = new WalStateManager(context);
92+
8893
TabletsMetadata root = context.getAmple().readTablets().forLevel(DataLevel.ROOT)
8994
.filter(new HasWalsFilter()).fetch(LOCATION, LAST, LOGS, PREV_ROW, SUSPEND).build();
9095
TabletsMetadata metadata = context.getAmple().readTablets().forLevel(DataLevel.METADATA)
9196
.filter(new HasWalsFilter()).fetch(LOCATION, LAST, LOGS, PREV_ROW, SUSPEND).build();
9297
TabletsMetadata user = context.getAmple().readTablets().forLevel(DataLevel.USER)
9398
.filter(new HasWalsFilter()).fetch(LOCATION, LAST, LOGS, PREV_ROW, SUSPEND).build();
94-
this.store = Streams.concat(root.stream(), metadata.stream(), user.stream()).onClose(() -> {
95-
root.close();
96-
metadata.close();
97-
user.close();
98-
});
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);
99106
}
100107

101108
/**
@@ -107,15 +114,17 @@ public class GarbageCollectWriteAheadLogs {
107114
*/
108115
@VisibleForTesting
109116
GarbageCollectWriteAheadLogs(ServerContext context, VolumeManager fs,
110-
LiveTServerSet liveTServerSet, WalStateManager walMarker, Stream<TabletMetadata> store) {
117+
LiveTServerSet liveTServerSet, WalStateManager walMarker, Iterable<TabletMetadata> store) {
111118
this.context = context;
112119
this.fs = fs;
113120
this.liveServers = liveTServerSet;
114121
this.walMarker = walMarker;
115122
this.store = store;
123+
this.hasCollected = new AtomicBoolean(false);
116124
}
117125

118126
public void collect(GCStatus status) {
127+
Preconditions.checkState(!hasCollected.get(), "Can only call collect once per object");
119128
try {
120129
long count;
121130
long fileScanStop;
@@ -203,7 +212,7 @@ public void collect(GCStatus status) {
203212
} finally {
204213
span5.end();
205214
}
206-
215+
hasCollected.set(true);
207216
} catch (Exception e) {
208217
log.error("exception occurred while garbage collecting write ahead logs", e);
209218
} finally {
@@ -285,7 +294,7 @@ private Map<UUID,TServerInstance> removeEntriesInUse(Map<TServerInstance,Set<UUI
285294
}
286295

287296
// remove any entries if there's a log reference (recovery hasn't finished)
288-
store.forEach(tabletMetadata -> {
297+
for (TabletMetadata tabletMetadata : store) {
289298
// Tablet is still assigned to a dead server. Manager has moved markers and reassigned it
290299
// Easiest to just ignore all the WALs for the dead server.
291300
if (TabletState.compute(tabletMetadata, liveServers) == TabletState.ASSIGNED_TO_DEAD_SERVER) {
@@ -307,8 +316,7 @@ private Map<UUID,TServerInstance> removeEntriesInUse(Map<TServerInstance,Set<UUI
307316
recoveryLogs.keySet().removeAll(idsToIgnore);
308317
}
309318
}
310-
});
311-
store.close();
319+
}
312320

313321
// Remove OPEN and CLOSED logs for live servers: they are still in use
314322
for (TServerInstance liveServer : liveServers) {
@@ -376,4 +384,15 @@ protected Map<UUID,Path> getSortedWALogs() throws IOException {
376384
}
377385
return result;
378386
}
387+
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+
}
379398
}

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

+5-4
Original file line numberDiff line numberDiff line change
@@ -232,10 +232,11 @@ 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-
GarbageCollectWriteAheadLogs walogCollector =
236-
new GarbageCollectWriteAheadLogs(getContext(), fs, liveTServerSet);
237-
log.info("Beginning garbage collection of write-ahead logs");
238-
walogCollector.collect(status);
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+
}
239240
gcCycleMetrics.setLastWalCollect(status.lastLog);
240241
} catch (Exception e) {
241242
TraceUtil.setException(walSpan, e, false);

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

+20-15
Original file line numberDiff line numberDiff line change
@@ -26,7 +26,6 @@
2626
import java.util.List;
2727
import java.util.Map;
2828
import java.util.UUID;
29-
import java.util.stream.Stream;
3029

3130
import org.apache.accumulo.core.client.admin.TabletAvailability;
3231
import org.apache.accumulo.core.dataImpl.KeyExtent;
@@ -75,8 +74,10 @@ public class GarbageCollectWriteAheadLogsTest {
7574
}
7675
}
7776

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

8182
@Test
8283
public void testRemoveUnusedLog() throws Exception {
@@ -98,14 +99,15 @@ public void testRemoveUnusedLog() throws Exception {
9899
marker.removeWalMarker(server1, id);
99100
EasyMock.expectLastCall().once();
100101
EasyMock.replay(context, fs, marker, tserverSet);
101-
GarbageCollectWriteAheadLogs gc =
102+
try (GarbageCollectWriteAheadLogs gc =
102103
new GarbageCollectWriteAheadLogs(context, fs, tserverSet, marker, tabletOnServer1List) {
103104
@Override
104105
protected Map<UUID,Path> getSortedWALogs() {
105106
return Collections.emptyMap();
106107
}
107-
};
108-
gc.collect(status);
108+
}) {
109+
gc.collect(status);
110+
}
109111
EasyMock.verify(context, fs, marker, tserverSet);
110112
}
111113

@@ -125,14 +127,15 @@ public void testKeepClosedLog() throws Exception {
125127
EasyMock.expect(marker.getAllMarkers()).andReturn(markers).once();
126128
EasyMock.expect(marker.state(server1, id)).andReturn(new Pair<>(WalState.CLOSED, path));
127129
EasyMock.replay(context, marker, tserverSet, fs);
128-
GarbageCollectWriteAheadLogs gc =
130+
try (GarbageCollectWriteAheadLogs gc =
129131
new GarbageCollectWriteAheadLogs(context, fs, tserverSet, marker, tabletOnServer1List) {
130132
@Override
131133
protected Map<UUID,Path> getSortedWALogs() {
132134
return Collections.emptyMap();
133135
}
134-
};
135-
gc.collect(status);
136+
}) {
137+
gc.collect(status);
138+
}
136139
EasyMock.verify(context, marker, tserverSet, fs);
137140
}
138141

@@ -159,14 +162,15 @@ public void deleteUnreferenceLogOnDeadServer() throws Exception {
159162
marker.forget(server2);
160163
EasyMock.expectLastCall().once();
161164
EasyMock.replay(context, fs, marker, tserverSet);
162-
GarbageCollectWriteAheadLogs gc =
165+
try (GarbageCollectWriteAheadLogs gc =
163166
new GarbageCollectWriteAheadLogs(context, fs, tserverSet, marker, tabletOnServer1List) {
164167
@Override
165168
protected Map<UUID,Path> getSortedWALogs() {
166169
return Collections.emptyMap();
167170
}
168-
};
169-
gc.collect(status);
171+
}) {
172+
gc.collect(status);
173+
}
170174
EasyMock.verify(context, fs, marker, tserverSet);
171175
}
172176

@@ -187,14 +191,15 @@ public void ignoreReferenceLogOnDeadServer() throws Exception {
187191
EasyMock.expect(marker.state(server2, id)).andReturn(new Pair<>(WalState.OPEN, path));
188192

189193
EasyMock.replay(context, fs, marker, tserverSet);
190-
GarbageCollectWriteAheadLogs gc =
194+
try (GarbageCollectWriteAheadLogs gc =
191195
new GarbageCollectWriteAheadLogs(context, fs, tserverSet, marker, tabletOnServer2List) {
192196
@Override
193197
protected Map<UUID,Path> getSortedWALogs() {
194198
return Collections.emptyMap();
195199
}
196-
};
197-
gc.collect(status);
200+
}) {
201+
gc.collect(status);
202+
}
198203
EasyMock.verify(context, fs, marker, tserverSet);
199204
}
200205

0 commit comments

Comments
 (0)