/
HBaseQueueRegionObserver.java
353 lines (313 loc) · 15.4 KB
/
HBaseQueueRegionObserver.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
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
/*
* Copyright © 2016 Cask Data, Inc.
*
* Licensed 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 co.cask.cdap.data2.transaction.queue.coprocessor.hbase12cdh570;
import co.cask.cdap.common.conf.Constants;
import co.cask.cdap.common.queue.QueueName;
import co.cask.cdap.data2.transaction.coprocessor.DefaultTransactionStateCacheSupplier;
import co.cask.cdap.data2.transaction.queue.ConsumerEntryState;
import co.cask.cdap.data2.transaction.queue.QueueEntryRow;
import co.cask.cdap.data2.transaction.queue.hbase.HBaseQueueAdmin;
import co.cask.cdap.data2.transaction.queue.hbase.SaltedHBaseQueueStrategy;
import co.cask.cdap.data2.transaction.queue.hbase.coprocessor.CConfigurationReader;
import co.cask.cdap.data2.transaction.queue.hbase.coprocessor.ConsumerConfigCache;
import co.cask.cdap.data2.transaction.queue.hbase.coprocessor.ConsumerInstance;
import co.cask.cdap.data2.transaction.queue.hbase.coprocessor.QueueConsumerConfig;
import co.cask.cdap.data2.util.TableId;
import co.cask.cdap.data2.util.hbase.HTable12CDH570NameConverter;
import com.google.common.base.Supplier;
import com.google.common.io.InputSupplier;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CoprocessorEnvironment;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.HTableInterface;
import org.apache.hadoop.hbase.coprocessor.BaseRegionObserver;
import org.apache.hadoop.hbase.coprocessor.ObserverContext;
import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
import org.apache.hadoop.hbase.regionserver.InternalScanner;
import org.apache.hadoop.hbase.regionserver.ScanType;
import org.apache.hadoop.hbase.regionserver.ScannerContext;
import org.apache.hadoop.hbase.regionserver.Store;
import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.tephra.coprocessor.TransactionStateCache;
import org.apache.tephra.persist.TransactionVisibilityState;
import java.io.IOException;
import java.util.Iterator;
import java.util.List;
/**
* RegionObserver for queue table. This class should only have JSE and HBase classes dependencies only.
* It can also has dependencies on CDAP classes provided that all the transitive dependencies stay within
* the mentioned scope.
*
* This region observer does queue eviction during flush time and compact time by using queue consumer state
* information to determine if a queue entry row can be omitted during flush/compact.
*/
public final class HBaseQueueRegionObserver extends BaseRegionObserver {
private static final Log LOG = LogFactory.getLog(HBaseQueueRegionObserver.class);
private TableName configTableName;
private CConfigurationReader cConfReader;
TransactionStateCache txStateCache;
private Supplier<TransactionVisibilityState> txSnapshotSupplier;
private ConsumerConfigCache configCache;
private int prefixBytes;
private String namespaceId;
private String appName;
private String flowName;
@Override
public void start(CoprocessorEnvironment env) {
if (env instanceof RegionCoprocessorEnvironment) {
HTableDescriptor tableDesc = ((RegionCoprocessorEnvironment) env).getRegion().getTableDesc();
String hTableName = tableDesc.getNameAsString();
String prefixBytes = tableDesc.getValue(HBaseQueueAdmin.PROPERTY_PREFIX_BYTES);
try {
// Default to SALT_BYTES for the older salted queue implementation.
this.prefixBytes = prefixBytes == null ? SaltedHBaseQueueStrategy.SALT_BYTES : Integer.parseInt(prefixBytes);
} catch (NumberFormatException e) {
// Shouldn't happen for table created by cdap.
LOG.error("Unable to parse value of '" + HBaseQueueAdmin.PROPERTY_PREFIX_BYTES + "' property. " +
"Default to " + SaltedHBaseQueueStrategy.SALT_BYTES, e);
this.prefixBytes = SaltedHBaseQueueStrategy.SALT_BYTES;
}
HTable12CDH570NameConverter nameConverter = new HTable12CDH570NameConverter();
namespaceId = nameConverter.from(tableDesc).getNamespace();
appName = HBaseQueueAdmin.getApplicationName(hTableName);
flowName = HBaseQueueAdmin.getFlowName(hTableName);
Configuration conf = env.getConfiguration();
String hbaseNamespacePrefix = tableDesc.getValue(Constants.Dataset.TABLE_PREFIX);
final String sysConfigTablePrefix = nameConverter.getSysConfigTablePrefix(hbaseNamespacePrefix);
txStateCache = new DefaultTransactionStateCacheSupplier(sysConfigTablePrefix, conf).get();
txSnapshotSupplier = new Supplier<TransactionVisibilityState>() {
@Override
public TransactionVisibilityState get() {
return txStateCache.getLatestState();
}
};
String queueConfigTableId = HBaseQueueAdmin.getConfigTableName();
configTableName = nameConverter.toTableName(hbaseNamespacePrefix, TableId.from(namespaceId, queueConfigTableId));
cConfReader = new CConfigurationReader(conf, sysConfigTablePrefix);
configCache = createConfigCache(env);
}
}
@Override
public InternalScanner preFlush(ObserverContext<RegionCoprocessorEnvironment> e,
Store store, InternalScanner scanner) throws IOException {
if (!e.getEnvironment().getRegion().isAvailable()) {
return scanner;
}
LOG.info("preFlush, creates EvictionInternalScanner");
return new EvictionInternalScanner("flush", e.getEnvironment(), scanner);
}
@Override
public InternalScanner preCompact(ObserverContext<RegionCoprocessorEnvironment> e, Store store,
InternalScanner scanner, ScanType type,
CompactionRequest request) throws IOException {
if (!e.getEnvironment().getRegion().isAvailable()) {
return scanner;
}
LOG.info("preCompact, creates EvictionInternalScanner");
return new EvictionInternalScanner("compaction", e.getEnvironment(), scanner);
}
// needed for queue unit-test
@SuppressWarnings("unused")
private void updateCache() throws IOException {
ConsumerConfigCache configCache = this.configCache;
if (configCache != null) {
configCache.updateCache();
}
}
private ConsumerConfigCache getConfigCache(CoprocessorEnvironment env) {
if (!configCache.isAlive()) {
configCache = createConfigCache(env);
}
return configCache;
}
private ConsumerConfigCache createConfigCache(final CoprocessorEnvironment env) {
return ConsumerConfigCache.getInstance(configTableName, cConfReader,
txSnapshotSupplier, new InputSupplier<HTableInterface>() {
@Override
public HTableInterface getInput() throws IOException {
return env.getTable(configTableName);
}
});
}
// need for queue unit-test
private TransactionStateCache getTxStateCache() {
return txStateCache;
}
/**
* An {@link InternalScanner} that will skip queue entries that are safe to be evicted.
*/
private final class EvictionInternalScanner implements InternalScanner {
private final String triggeringAction;
private final RegionCoprocessorEnvironment env;
private final InternalScanner scanner;
// This is just for object reused to reduce objects creation.
private final ConsumerInstance consumerInstance;
private byte[] currentQueue;
private byte[] currentQueueRowPrefix;
private QueueConsumerConfig consumerConfig;
private long totalRows = 0;
private long rowsEvicted = 0;
// couldn't be evicted due to incomplete view of row
private long skippedIncomplete = 0;
private EvictionInternalScanner(String action, RegionCoprocessorEnvironment env, InternalScanner scanner) {
this.triggeringAction = action;
this.env = env;
this.scanner = scanner;
this.consumerInstance = new ConsumerInstance(0, 0);
}
@Override
public boolean next(List<Cell> results) throws IOException {
return next(results, ScannerContext.newBuilder().setBatchLimit(-1).build());
}
@Override
public boolean next(List<Cell> results, ScannerContext scannerContext) throws IOException {
boolean hasNext = scanner.next(results, scannerContext);
while (!results.isEmpty()) {
totalRows++;
// Check if it is eligible for eviction.
Cell cell = results.get(0);
// If current queue is unknown or the row is not a queue entry of current queue,
// it either because it scans into next queue entry or simply current queue is not known.
// Hence needs to find the currentQueue
if (currentQueue == null || !QueueEntryRow.isQueueEntry(currentQueueRowPrefix, prefixBytes, cell.getRowArray(),
cell.getRowOffset(), cell.getRowLength())) {
// If not eligible, it either because it scans into next queue entry or simply current queue is not known.
currentQueue = null;
}
// This row is a queue entry. If currentQueue is null, meaning it's a new queue encountered during scan.
if (currentQueue == null) {
QueueName queueName = QueueEntryRow.getQueueName(namespaceId, appName, flowName, prefixBytes,
cell.getRowArray(), cell.getRowOffset(),
cell.getRowLength());
currentQueue = queueName.toBytes();
currentQueueRowPrefix = QueueEntryRow.getQueueRowPrefix(queueName);
consumerConfig = getConfigCache(env).getConsumerConfig(currentQueue);
}
if (consumerConfig == null) {
// no config is present yet, so cannot evict
return hasNext;
}
if (canEvict(consumerConfig, results)) {
rowsEvicted++;
results.clear();
hasNext = scanner.next(results, scannerContext);
} else {
break;
}
}
return hasNext;
}
@Override
public void close() throws IOException {
LOG.info("Region " + env.getRegionInfo().getRegionNameAsString() + " " + triggeringAction +
", rows evicted: " + rowsEvicted + " / " + totalRows + ", skipped incomplete: " + skippedIncomplete);
scanner.close();
}
/**
* Determines the given queue entry row can be evicted.
* @param result All KeyValues of a queue entry row.
* @return true if it can be evicted, false otherwise.
*/
private boolean canEvict(QueueConsumerConfig consumerConfig, List<Cell> result) {
// If no consumer group, this queue is dead, should be ok to evict.
if (consumerConfig.getNumGroups() == 0) {
return true;
}
// If unknown consumer config (due to error), keep the queue.
if (consumerConfig.getNumGroups() < 0) {
return false;
}
// TODO (terence): Right now we can only evict if we see all the data columns.
// It's because it's possible that in some previous flush, only the data columns are flush,
// then consumer writes the state columns. In the next flush, it'll only see the state columns and those
// should not be evicted otherwise the entry might get reprocessed, depending on the consumer start row state.
// This logic is not perfect as if flush happens after enqueue and before dequeue, that entry may never get
// evicted (depends on when the next compaction happens, whether the queue configuration has been change or not).
// There are two data columns, "d" and "m".
// If the size == 2, it should not be evicted as well,
// as state columns (dequeue) always happen after data columns (enqueue).
if (result.size() <= 2) {
skippedIncomplete++;
return false;
}
// "d" and "m" columns always comes before the state columns, prefixed with "s".
Iterator<Cell> iterator = result.iterator();
Cell cell = iterator.next();
if (!QueueEntryRow.isDataColumn(cell.getQualifierArray(), cell.getQualifierOffset())) {
skippedIncomplete++;
return false;
}
cell = iterator.next();
if (!QueueEntryRow.isMetaColumn(cell.getQualifierArray(), cell.getQualifierOffset())) {
skippedIncomplete++;
return false;
}
// Need to determine if this row can be evicted iff all consumer groups have committed process this row.
int consumedGroups = 0;
// Inspect each state column
while (iterator.hasNext()) {
cell = iterator.next();
if (!QueueEntryRow.isStateColumn(cell.getQualifierArray(), cell.getQualifierOffset())) {
continue;
}
// If any consumer has a state != PROCESSED, it should not be evicted
if (!isProcessed(cell, consumerInstance)) {
break;
}
// If it is PROCESSED, check if this row is smaller than the consumer instance startRow.
// Essentially a loose check of committed PROCESSED.
byte[] startRow = consumerConfig.getStartRow(consumerInstance);
if (startRow != null && compareRowKey(cell, startRow) < 0) {
consumedGroups++;
}
}
// It can be evicted if from the state columns, it's been processed by all consumer groups
// Otherwise, this row has to be less than smallest among all current consumers.
// The second condition is for handling consumer being removed after it consumed some entries.
// However, the second condition alone is not good enough as it's possible that in hash partitioning,
// only one consumer is keep consuming when the other consumer never proceed.
return consumedGroups == consumerConfig.getNumGroups()
|| compareRowKey(result.get(0), consumerConfig.getSmallestStartRow()) < 0;
}
private int compareRowKey(Cell cell, byte[] row) {
return Bytes.compareTo(cell.getRowArray(), cell.getRowOffset() + prefixBytes,
cell.getRowLength() - prefixBytes, row, 0, row.length);
}
/**
* Returns {@code true} if the given {@link KeyValue} has a {@link ConsumerEntryState#PROCESSED} state and
* also put the consumer information into the given {@link ConsumerInstance}.
* Otherwise, returns {@code false} and the {@link ConsumerInstance} is left untouched.
*/
private boolean isProcessed(Cell cell, ConsumerInstance consumerInstance) {
int stateIdx = cell.getValueOffset() + cell.getValueLength() - 1;
boolean processed = cell.getValueArray()[stateIdx] == ConsumerEntryState.PROCESSED.getState();
if (processed) {
// Column is "s<groupId>"
long groupId = Bytes.toLong(cell.getQualifierArray(), cell.getQualifierOffset() + 1);
// Value is "<writePointer><instanceId><state>"
int instanceId = Bytes.toInt(cell.getValueArray(), cell.getValueOffset() + Bytes.SIZEOF_LONG);
consumerInstance.setGroupInstance(groupId, instanceId);
}
return processed;
}
}
}