/
DefaultMemStore.java
222 lines (203 loc) · 7.68 KB
/
DefaultMemStore.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
/**
* 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.hadoop.hbase.regionserver;
import java.io.IOException;
import java.lang.management.ManagementFactory;
import java.lang.management.RuntimeMXBean;
import java.util.ArrayList;
import java.util.List;
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.CellComparator;
import org.apache.hadoop.hbase.CellComparatorImpl;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.ClassSize;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
/**
* The MemStore holds in-memory modifications to the Store. Modifications
* are {@link Cell}s. When asked to flush, current memstore is moved
* to snapshot and is cleared. We continue to serve edits out of new memstore
* and backing snapshot until flusher reports in that the flush succeeded. At
* this point we let the snapshot go.
* <p>
* The MemStore functions should not be called in parallel. Callers should hold
* write and read locks. This is done in {@link HStore}.
* </p>
*
* TODO: Adjust size of the memstore when we remove items because they have
* been deleted.
* TODO: With new KVSLS, need to make sure we update HeapSize with difference
* in KV size.
*/
@InterfaceAudience.Private
public class DefaultMemStore extends AbstractMemStore {
private static final Log LOG = LogFactory.getLog(DefaultMemStore.class);
public final static long DEEP_OVERHEAD = ClassSize.align(AbstractMemStore.DEEP_OVERHEAD);
public final static long FIXED_OVERHEAD = ClassSize.align(AbstractMemStore.FIXED_OVERHEAD);
/**
* Default constructor. Used for tests.
*/
public DefaultMemStore() {
this(HBaseConfiguration.create(), CellComparatorImpl.COMPARATOR);
}
/**
* Constructor.
* @param c Comparator
*/
public DefaultMemStore(final Configuration conf, final CellComparator c) {
super(conf, c);
}
/**
* Creates a snapshot of the current memstore.
* Snapshot must be cleared by call to {@link #clearSnapshot(long)}
*/
@Override
public MemStoreSnapshot snapshot() {
// If snapshot currently has entries, then flusher failed or didn't call
// cleanup. Log a warning.
if (!this.snapshot.isEmpty()) {
LOG.warn("Snapshot called again without clearing previous. " +
"Doing nothing. Another ongoing flush or did we fail last attempt?");
} else {
this.snapshotId = EnvironmentEdgeManager.currentTime();
if (!this.active.isEmpty()) {
ImmutableSegment immutableSegment = SegmentFactory.instance().
createImmutableSegment(this.active);
this.snapshot = immutableSegment;
resetActive();
}
}
return new MemStoreSnapshot(this.snapshotId, this.snapshot);
}
/**
* On flush, how much memory we will clear from the active cell set.
*
* @return size of data that is going to be flushed from active set
*/
@Override
public MemStoreSize getFlushableSize() {
MemStoreSize snapshotSize = getSnapshotSize();
return snapshotSize.getDataSize() > 0 ? snapshotSize
: new MemStoreSize(keySize(), heapSize());
}
@Override
protected long keySize() {
return this.active.keySize();
}
@Override
protected long heapSize() {
return this.active.heapSize();
}
@Override
/*
* Scanners are ordered from 0 (oldest) to newest in increasing order.
*/
public List<KeyValueScanner> getScanners(long readPt) throws IOException {
List<KeyValueScanner> list = new ArrayList<>();
long order = snapshot.getNumOfSegments();
order = addToScanners(active, readPt, order, list);
addToScanners(snapshot.getAllSegments(), readPt, order, list);
return list;
}
@Override
protected List<Segment> getSegments() throws IOException {
List<Segment> list = new ArrayList<>(2);
list.add(this.active);
list.add(this.snapshot);
return list;
}
/**
* @param cell Find the row that comes after this one. If null, we return the
* first.
* @return Next row or null if none found.
*/
Cell getNextRow(final Cell cell) {
return getLowest(
getNextRow(cell, this.active.getCellSet()),
getNextRow(cell, this.snapshot.getCellSet()));
}
@Override public void updateLowestUnflushedSequenceIdInWAL(boolean onlyIfMoreRecent) {
}
@Override
public MemStoreSize size() {
return new MemStoreSize(this.active.keySize(), this.active.heapSize());
}
/**
* Check whether anything need to be done based on the current active set size
* Nothing need to be done for the DefaultMemStore
*/
@Override
protected void checkActiveSize() {
return;
}
@Override
public long preFlushSeqIDEstimation() {
return HConstants.NO_SEQNUM;
}
@Override public boolean isSloppy() {
return false;
}
/**
* Code to help figure if our approximation of object heap sizes is close
* enough. See hbase-900. Fills memstores then waits so user can heap
* dump and bring up resultant hprof in something like jprofiler which
* allows you get 'deep size' on objects.
* @param args main args
*/
public static void main(String [] args) {
RuntimeMXBean runtime = ManagementFactory.getRuntimeMXBean();
LOG.info("vmName=" + runtime.getVmName() + ", vmVendor=" +
runtime.getVmVendor() + ", vmVersion=" + runtime.getVmVersion());
LOG.info("vmInputArguments=" + runtime.getInputArguments());
DefaultMemStore memstore1 = new DefaultMemStore();
// TODO: x32 vs x64
final int count = 10000;
byte [] fam = Bytes.toBytes("col");
byte [] qf = Bytes.toBytes("umn");
byte [] empty = new byte[0];
MemStoreSizing memstoreSizing = new MemStoreSizing();
for (int i = 0; i < count; i++) {
// Give each its own ts
memstore1.add(new KeyValue(Bytes.toBytes(i), fam, qf, i, empty), memstoreSizing);
}
LOG.info("memstore1 estimated size="
+ (memstoreSizing.getDataSize() + memstoreSizing.getHeapSize()));
for (int i = 0; i < count; i++) {
memstore1.add(new KeyValue(Bytes.toBytes(i), fam, qf, i, empty), memstoreSizing);
}
LOG.info("memstore1 estimated size (2nd loading of same data)="
+ (memstoreSizing.getDataSize() + memstoreSizing.getHeapSize()));
// Make a variably sized memstore.
DefaultMemStore memstore2 = new DefaultMemStore();
memstoreSizing = new MemStoreSizing();
for (int i = 0; i < count; i++) {
memstore2.add(new KeyValue(Bytes.toBytes(i), fam, qf, i, new byte[i]), memstoreSizing);
}
LOG.info("memstore2 estimated size="
+ (memstoreSizing.getDataSize() + memstoreSizing.getHeapSize()));
final int seconds = 30;
LOG.info("Waiting " + seconds + " seconds while heap dump is taken");
LOG.info("Exiting.");
}
}