-
Notifications
You must be signed in to change notification settings - Fork 3.6k
/
HintsStore.java
328 lines (280 loc) · 10 KB
/
HintsStore.java
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.cassandra.hints;
import java.io.IOException;
import java.util.*;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentLinkedDeque;
import java.util.concurrent.ConcurrentLinkedQueue;
import java.util.function.Predicate;
import javax.annotation.Nullable;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Iterables;
import org.apache.cassandra.io.util.File;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.cassandra.gms.FailureDetector;
import org.apache.cassandra.io.FSWriteError;
import org.apache.cassandra.locator.InetAddressAndPort;
import org.apache.cassandra.schema.Schema;
import org.apache.cassandra.service.StorageService;
import org.apache.cassandra.utils.SyncUtil;
import static org.apache.cassandra.utils.Clock.Global.currentTimeMillis;
/**
* Encapsulates the state of a peer's hints: the queue of hints files for dispatch, and the current writer (if any).
*
* The queue for dispatch is multi-threading safe.
*
* The writer MUST only be accessed by {@link HintsWriteExecutor}.
*/
final class HintsStore
{
private static final Logger logger = LoggerFactory.getLogger(HintsStore.class);
public final UUID hostId;
private final File hintsDirectory;
private final ImmutableMap<String, Object> writerParams;
private final Map<HintsDescriptor, InputPosition> dispatchPositions;
private final Deque<HintsDescriptor> dispatchDequeue;
private final Queue<HintsDescriptor> corruptedFiles;
private final Map<HintsDescriptor, Long> hintsExpirations;
// last timestamp used in a descriptor; make sure to not reuse the same timestamp for new descriptors.
private volatile long lastUsedTimestamp;
private volatile HintsWriter hintsWriter;
private HintsStore(UUID hostId, File hintsDirectory, ImmutableMap<String, Object> writerParams, List<HintsDescriptor> descriptors)
{
this.hostId = hostId;
this.hintsDirectory = hintsDirectory;
this.writerParams = writerParams;
dispatchPositions = new ConcurrentHashMap<>();
dispatchDequeue = new ConcurrentLinkedDeque<>(descriptors);
corruptedFiles = new ConcurrentLinkedQueue<>();
hintsExpirations = new ConcurrentHashMap<>();
//noinspection resource
lastUsedTimestamp = descriptors.stream().mapToLong(d -> d.timestamp).max().orElse(0L);
}
static HintsStore create(UUID hostId, File hintsDirectory, ImmutableMap<String, Object> writerParams, List<HintsDescriptor> descriptors)
{
descriptors.sort((d1, d2) -> Long.compare(d1.timestamp, d2.timestamp));
return new HintsStore(hostId, hintsDirectory, writerParams, descriptors);
}
@VisibleForTesting
int getDispatchQueueSize()
{
return dispatchDequeue.size();
}
@VisibleForTesting
int getHintsExpirationsMapSize()
{
return hintsExpirations.size();
}
InetAddressAndPort address()
{
return StorageService.instance.getEndpointForHostId(hostId);
}
@Nullable
PendingHintsInfo getPendingHintsInfo()
{
Iterator<HintsDescriptor> descriptors = dispatchDequeue.iterator();
int queueSize = 0;
long minTimestamp = Long.MAX_VALUE;
long maxTimestamp = Long.MIN_VALUE;
while (descriptors.hasNext())
{
HintsDescriptor descriptor = descriptors.next();
minTimestamp = Math.min(minTimestamp, descriptor.timestamp);
maxTimestamp = Math.max(maxTimestamp, descriptor.timestamp);
queueSize++;
}
if (queueSize == 0)
return null;
return new PendingHintsInfo(hostId, queueSize, minTimestamp, maxTimestamp);
}
boolean isLive()
{
InetAddressAndPort address = address();
return address != null && FailureDetector.instance.isAlive(address);
}
HintsDescriptor poll()
{
return dispatchDequeue.poll();
}
void offerFirst(HintsDescriptor descriptor)
{
dispatchDequeue.offerFirst(descriptor);
}
void offerLast(HintsDescriptor descriptor)
{
dispatchDequeue.offerLast(descriptor);
}
void deleteAllHints()
{
HintsDescriptor descriptor;
while ((descriptor = poll()) != null)
{
cleanUp(descriptor);
delete(descriptor);
}
while ((descriptor = corruptedFiles.poll()) != null)
{
cleanUp(descriptor);
delete(descriptor);
}
}
void deleteExpiredHints(long now)
{
deleteHints(it -> hasExpired(it, now));
}
private boolean hasExpired(HintsDescriptor descriptor, long now)
{
Long cachedExpiresAt = hintsExpirations.get(descriptor);
if (null != cachedExpiresAt)
return cachedExpiresAt <= now;
File hintFile = new File(hintsDirectory, descriptor.fileName());
// the file does not exist or if an I/O error occurs
if (!hintFile.exists() || hintFile.lastModified() == 0)
return false;
// 'lastModified' can be considered as the upper bound of the hint creation time.
// So the TTL upper bound of all hints in the file can be estimated by lastModified + maxGcgs of all tables
long ttl = hintFile.lastModified() + Schema.instance.largestGcgs();
hintsExpirations.put(descriptor, ttl);
return ttl <= now;
}
private void deleteHints(Predicate<HintsDescriptor> predicate)
{
Set<HintsDescriptor> removeSet = new HashSet<>();
try
{
for (HintsDescriptor descriptor : Iterables.concat(dispatchDequeue, corruptedFiles))
{
if (predicate.test(descriptor))
{
cleanUp(descriptor);
removeSet.add(descriptor);
delete(descriptor);
}
}
}
finally // remove the already deleted hints from internal queues in case of exception
{
dispatchDequeue.removeAll(removeSet);
corruptedFiles.removeAll(removeSet);
}
}
void delete(HintsDescriptor descriptor)
{
File hintsFile = descriptor.file(hintsDirectory);
if (hintsFile.tryDelete())
logger.info("Deleted hint file {}", descriptor.fileName());
else if (hintsFile.exists())
logger.error("Failed to delete hint file {}", descriptor.fileName());
else
logger.info("Already deleted hint file {}", descriptor.fileName());
//noinspection ResultOfMethodCallIgnored
descriptor.checksumFile(hintsDirectory).tryDelete();
}
boolean hasFiles()
{
return !dispatchDequeue.isEmpty();
}
InputPosition getDispatchOffset(HintsDescriptor descriptor)
{
return dispatchPositions.get(descriptor);
}
void markDispatchOffset(HintsDescriptor descriptor, InputPosition inputPosition)
{
dispatchPositions.put(descriptor, inputPosition);
}
/**
* @return the total size of all files belonging to the hints store, in bytes.
*/
@SuppressWarnings({ "resource" })
long getTotalFileSize()
{
long total = 0;
for (HintsDescriptor descriptor : Iterables.concat(dispatchDequeue, corruptedFiles))
total += descriptor.hintsFileSize(hintsDirectory);
HintsWriter currentWriter = getWriter();
if (null != currentWriter)
total += currentWriter.descriptor().hintsFileSize(hintsDirectory);
return total;
}
void cleanUp(HintsDescriptor descriptor)
{
dispatchPositions.remove(descriptor);
hintsExpirations.remove(descriptor);
}
void markCorrupted(HintsDescriptor descriptor)
{
corruptedFiles.add(descriptor);
}
/**
* @return a copy of the first {@link HintsDescriptor} in the queue for dispatch or {@code null} if queue is empty.
*/
HintsDescriptor getFirstDescriptor()
{
return dispatchDequeue.peekFirst();
}
/*
* Methods dealing with HintsWriter.
*
* All of these, with the exception of isWriting(), are for exclusively single-threaded use by HintsWriteExecutor.
*/
boolean isWriting()
{
return hintsWriter != null;
}
HintsWriter getOrOpenWriter()
{
if (hintsWriter == null)
hintsWriter = openWriter();
return hintsWriter;
}
HintsWriter getWriter()
{
return hintsWriter;
}
private HintsWriter openWriter()
{
lastUsedTimestamp = Math.max(currentTimeMillis(), lastUsedTimestamp + 1);
HintsDescriptor descriptor = new HintsDescriptor(hostId, lastUsedTimestamp, writerParams);
try
{
return HintsWriter.create(hintsDirectory, descriptor);
}
catch (IOException e)
{
throw new FSWriteError(e, descriptor.fileName());
}
}
void closeWriter()
{
if (hintsWriter != null)
{
hintsWriter.close();
offerLast(hintsWriter.descriptor());
hintsWriter = null;
SyncUtil.trySyncDir(hintsDirectory);
}
}
void fsyncWriter()
{
if (hintsWriter != null)
hintsWriter.fsync();
}
}