forked from apache/asterixdb
-
Notifications
You must be signed in to change notification settings - Fork 6
/
PrimaryIndexOperationTracker.java
282 lines (258 loc) · 12.9 KB
/
PrimaryIndexOperationTracker.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
/*
* 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.asterix.common.context;
import java.util.ArrayList;
import java.util.Collection;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.asterix.common.exceptions.ACIDException;
import org.apache.asterix.common.ioopcallbacks.LSMIOOperationCallback;
import org.apache.asterix.common.metadata.MetadataIndexImmutableProperties;
import org.apache.asterix.common.transactions.AbstractOperationCallback;
import org.apache.asterix.common.transactions.ILogManager;
import org.apache.asterix.common.transactions.LogRecord;
import org.apache.asterix.common.utils.TransactionUtil;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.storage.am.common.impls.NoOpIndexAccessParameters;
import org.apache.hyracks.storage.am.common.impls.NoOpOperationCallback;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent.ComponentState;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentId;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentIdGenerator;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMMemoryComponent;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
import org.apache.hyracks.storage.am.lsm.common.api.IoOperationCompleteListener;
import org.apache.hyracks.storage.am.lsm.common.api.LSMOperationType;
import org.apache.hyracks.storage.am.lsm.common.impls.FlushOperation;
import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentId;
import org.apache.hyracks.storage.common.IModificationOperationCallback;
import org.apache.hyracks.storage.common.ISearchOperationCallback;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
public class PrimaryIndexOperationTracker extends BaseOperationTracker implements IoOperationCompleteListener {
private static final Logger LOGGER = LogManager.getLogger();
private final int partition;
// Number of active operations on an ILSMIndex instance.
private final AtomicInteger numActiveOperations;
private final ILogManager logManager;
private final ILSMComponentIdGenerator idGenerator;
private boolean flushOnExit = false;
private boolean flushLogCreated = false;
private final Map<String, FlushOperation> scheduledFlushes = new HashMap<>();
public PrimaryIndexOperationTracker(int datasetID, int partition, ILogManager logManager, DatasetInfo dsInfo,
ILSMComponentIdGenerator idGenerator) {
super(datasetID, dsInfo);
this.partition = partition;
this.logManager = logManager;
this.numActiveOperations = new AtomicInteger();
this.idGenerator = idGenerator;
}
@Override
public void beforeOperation(ILSMIndex index, LSMOperationType opType, ISearchOperationCallback searchCallback,
IModificationOperationCallback modificationCallback) throws HyracksDataException {
super.beforeOperation(index, opType, searchCallback, modificationCallback);
if (opType == LSMOperationType.MODIFICATION || opType == LSMOperationType.FORCE_MODIFICATION) {
incrementNumActiveOperations(modificationCallback);
}
}
@Override
public synchronized void completeOperation(ILSMIndex index, LSMOperationType opType,
ISearchOperationCallback searchCallback, IModificationOperationCallback modificationCallback)
throws HyracksDataException {
super.completeOperation(index, opType, searchCallback, modificationCallback);
if (opType == LSMOperationType.MODIFICATION || opType == LSMOperationType.FORCE_MODIFICATION) {
decrementNumActiveOperations(modificationCallback);
flushIfNeeded();
}
}
public synchronized void flushIfNeeded() throws HyracksDataException {
if (canSafelyFlush()) {
flushIfRequested();
}
}
public void flushIfRequested() throws HyracksDataException {
// If we need a flush, and this is the last completing operation, then schedule the flush,
// or if there is a flush scheduled by the checkpoint (flushOnExit), then schedule it
boolean needsFlush = false;
Set<ILSMIndex> indexes = dsInfo.getDatasetPartitionOpenIndexes(partition);
if (!flushOnExit) {
for (ILSMIndex lsmIndex : indexes) {
if (lsmIndex.hasFlushRequestForCurrentMutableComponent()) {
needsFlush = true;
break;
}
}
}
ILSMIndex primaryLsmIndex = null;
if (needsFlush || flushOnExit) {
flushOnExit = false;
// make the current mutable components READABLE_UNWRITABLE to stop coming modify operations from entering
// them until the current flush is scheduled.
LSMComponentId primaryId = null;
//Double check that the primary index has been modified
synchronized (this) {
if (numActiveOperations.get() > 0) {
throw new IllegalStateException(
"Can't request a flush on an index with active operations: " + numActiveOperations.get());
}
for (ILSMIndex lsmIndex : indexes) {
if (lsmIndex.isPrimaryIndex()) {
if (lsmIndex.isCurrentMutableComponentEmpty()) {
LOGGER.info("Primary index on dataset {} and partition {} is empty... skipping flush",
dsInfo.getDatasetID(), partition);
return;
}
primaryLsmIndex = lsmIndex;
break;
}
}
}
if (primaryLsmIndex == null) {
throw new IllegalStateException(
"Primary index not found in dataset " + dsInfo.getDatasetID() + " and partition " + partition);
}
for (ILSMIndex lsmIndex : indexes) {
ILSMOperationTracker opTracker = lsmIndex.getOperationTracker();
synchronized (opTracker) {
ILSMMemoryComponent memComponent = lsmIndex.getCurrentMemoryComponent();
if (memComponent.getWriterCount() > 0) {
throw new IllegalStateException(
"Can't request a flush on a component with writers inside: Index:" + lsmIndex
+ " Component:" + memComponent);
}
if (memComponent.getState() == ComponentState.READABLE_WRITABLE && memComponent.isModified()) {
memComponent.setUnwritable();
}
if (lsmIndex.isPrimaryIndex()) {
primaryId = (LSMComponentId) memComponent.getId();
}
}
}
if (primaryId == null) {
throw new IllegalStateException("Primary index found in dataset " + dsInfo.getDatasetID()
+ " and partition " + partition + " and is modified but its component id is null");
}
LogRecord logRecord = new LogRecord();
if (dsInfo.isDurable()) {
/*
* Generate a FLUSH log.
* Flush will be triggered when the log is written to disk by LogFlusher.
*/
TransactionUtil.formFlushLogRecord(logRecord, datasetID, partition, primaryId.getMinId(),
primaryId.getMaxId(), this);
try {
logManager.log(logRecord);
} catch (ACIDException e) {
throw new IllegalStateException("could not write flush log", e);
}
flushLogCreated = true;
} else {
//trigger flush for temporary indexes without generating a FLUSH log.
triggerScheduleFlush(logRecord);
}
}
}
//This method is called sequentially by LogPage.notifyFlushTerminator in the sequence flushes were scheduled.
public synchronized void triggerScheduleFlush(LogRecord logRecord) throws HyracksDataException {
try {
if (!canSafelyFlush()) {
// if a force modification operation started before the flush is scheduled, this flush will fail
// and a next attempt will be made when that operation completes. This is only expected for metadata
// datasets since they always use force modification
if (MetadataIndexImmutableProperties.isMetadataDataset(datasetID)) {
return;
}
throw new IllegalStateException("Operation started while index was pending scheduling a flush");
}
idGenerator.refresh();
long flushLsn = logRecord.getLSN();
if (flushLsn == 0) {
LOGGER.warn("flushing an index with LSN 0. Flush log record: {}", logRecord::getLogRecordForDisplay);
}
ILSMComponentId nextComponentId = idGenerator.getId();
Map<String, Object> flushMap = new HashMap<>();
flushMap.put(LSMIOOperationCallback.KEY_FLUSH_LOG_LSN, flushLsn);
flushMap.put(LSMIOOperationCallback.KEY_NEXT_COMPONENT_ID, nextComponentId);
synchronized (scheduledFlushes) {
for (ILSMIndex lsmIndex : dsInfo.getDatasetPartitionOpenIndexes(partition)) {
ILSMIndexAccessor accessor = lsmIndex.createAccessor(NoOpIndexAccessParameters.INSTANCE);
accessor.getOpContext().setParameters(flushMap);
ILSMIOOperation flush = accessor.scheduleFlush();
scheduledFlushes.put(flush.getTarget().getRelativePath(), (FlushOperation) flush);
flush.addCompleteListener(this);
}
}
} finally {
flushLogCreated = false;
}
}
@Override
public void completed(ILSMIOOperation operation) {
synchronized (scheduledFlushes) {
scheduledFlushes.remove(operation.getTarget().getRelativePath());
}
}
public List<FlushOperation> getScheduledFlushes() {
synchronized (scheduledFlushes) {
Collection<FlushOperation> scheduled = scheduledFlushes.values();
List<FlushOperation> flushes = new ArrayList<FlushOperation>(scheduled.size());
flushes.addAll(scheduled);
return flushes;
}
}
public int getNumActiveOperations() {
return numActiveOperations.get();
}
private void incrementNumActiveOperations(IModificationOperationCallback modificationCallback) {
//modificationCallback can be NoOpOperationCallback when redo/undo operations are executed.
if (modificationCallback != NoOpOperationCallback.INSTANCE) {
numActiveOperations.incrementAndGet();
((AbstractOperationCallback) modificationCallback).beforeOperation();
}
}
private void decrementNumActiveOperations(IModificationOperationCallback modificationCallback) {
//modificationCallback can be NoOpOperationCallback when redo/undo operations are executed.
if (modificationCallback != NoOpOperationCallback.INSTANCE) {
if (numActiveOperations.decrementAndGet() < 0) {
throw new IllegalStateException("The number of active operations cannot be negative!");
}
((AbstractOperationCallback) modificationCallback).afterOperation();
}
}
public boolean isFlushOnExit() {
return flushOnExit;
}
public void setFlushOnExit(boolean flushOnExit) {
this.flushOnExit = flushOnExit;
}
public boolean isFlushLogCreated() {
return flushLogCreated;
}
public int getPartition() {
return partition;
}
private boolean canSafelyFlush() {
return numActiveOperations.get() == 0;
}
}