HBASE-13739 Remove KeyValueUtil.ensureKeyValue(cell) from MOB code.(Jingcheng)
This commit is contained in:
parent
5098f365b4
commit
132f65ea1f
|
@ -31,7 +31,6 @@ import org.apache.hadoop.fs.Path;
|
||||||
import org.apache.hadoop.hbase.Cell;
|
import org.apache.hadoop.hbase.Cell;
|
||||||
import org.apache.hadoop.hbase.HConstants;
|
import org.apache.hadoop.hbase.HConstants;
|
||||||
import org.apache.hadoop.hbase.KeyValue;
|
import org.apache.hadoop.hbase.KeyValue;
|
||||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
|
||||||
import org.apache.hadoop.hbase.Tag;
|
import org.apache.hadoop.hbase.Tag;
|
||||||
import org.apache.hadoop.hbase.TagType;
|
import org.apache.hadoop.hbase.TagType;
|
||||||
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
|
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
|
||||||
|
@ -175,19 +174,18 @@ public class DefaultMobStoreFlusher extends DefaultStoreFlusher {
|
||||||
// If we know that this KV is going to be included always, then let us
|
// If we know that this KV is going to be included always, then let us
|
||||||
// set its memstoreTS to 0. This will help us save space when writing to
|
// set its memstoreTS to 0. This will help us save space when writing to
|
||||||
// disk.
|
// disk.
|
||||||
KeyValue kv = KeyValueUtil.ensureKeyValue(c);
|
if (c.getValueLength() <= mobCellValueSizeThreshold || MobUtils.isMobReferenceCell(c)
|
||||||
if (kv.getValueLength() <= mobCellValueSizeThreshold || MobUtils.isMobReferenceCell(kv)
|
|| c.getTypeByte() != KeyValue.Type.Put.getCode()) {
|
||||||
|| kv.getTypeByte() != KeyValue.Type.Put.getCode()) {
|
writer.append(c);
|
||||||
writer.append(kv);
|
|
||||||
} else {
|
} else {
|
||||||
// append the original keyValue in the mob file.
|
// append the original keyValue in the mob file.
|
||||||
mobFileWriter.append(kv);
|
mobFileWriter.append(c);
|
||||||
mobSize += kv.getValueLength();
|
mobSize += c.getValueLength();
|
||||||
mobCount++;
|
mobCount++;
|
||||||
|
|
||||||
// append the tags to the KeyValue.
|
// append the tags to the KeyValue.
|
||||||
// The key is same, the value is the filename of the mob file
|
// The key is same, the value is the filename of the mob file
|
||||||
KeyValue reference = MobUtils.createMobRefKeyValue(kv, fileName, tableNameTag);
|
KeyValue reference = MobUtils.createMobRefKeyValue(c, fileName, tableNameTag);
|
||||||
writer.append(reference);
|
writer.append(reference);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -361,12 +361,10 @@ public class PartitionedMobFileCompactor extends MobFileCompactor {
|
||||||
do {
|
do {
|
||||||
hasMore = scanner.next(cells, scannerContext);
|
hasMore = scanner.next(cells, scannerContext);
|
||||||
for (Cell cell : cells) {
|
for (Cell cell : cells) {
|
||||||
// TODO remove this after the new code are introduced.
|
|
||||||
KeyValue kv = KeyValueUtil.ensureKeyValue(cell);
|
|
||||||
// write the mob cell to the mob file.
|
// write the mob cell to the mob file.
|
||||||
writer.append(kv);
|
writer.append(cell);
|
||||||
// write the new reference cell to the store file.
|
// write the new reference cell to the store file.
|
||||||
KeyValue reference = MobUtils.createMobRefKeyValue(kv, fileName, tableNameTag);
|
KeyValue reference = MobUtils.createMobRefKeyValue(cell, fileName, tableNameTag);
|
||||||
refFileWriter.append(reference);
|
refFileWriter.append(reference);
|
||||||
mobCells++;
|
mobCells++;
|
||||||
}
|
}
|
||||||
|
@ -468,9 +466,7 @@ public class PartitionedMobFileCompactor extends MobFileCompactor {
|
||||||
do {
|
do {
|
||||||
hasMore = scanner.next(cells, scannerContext);
|
hasMore = scanner.next(cells, scannerContext);
|
||||||
for (Cell cell : cells) {
|
for (Cell cell : cells) {
|
||||||
// TODO remove this after the new code are introduced.
|
writer.append(cell);
|
||||||
KeyValue kv = KeyValueUtil.ensureKeyValue(cell);
|
|
||||||
writer.append(kv);
|
|
||||||
}
|
}
|
||||||
cells.clear();
|
cells.clear();
|
||||||
} while (hasMore);
|
} while (hasMore);
|
||||||
|
|
|
@ -34,9 +34,7 @@ import org.apache.hadoop.hbase.KeyValueUtil;
|
||||||
import org.apache.hadoop.hbase.Tag;
|
import org.apache.hadoop.hbase.Tag;
|
||||||
import org.apache.hadoop.hbase.TagType;
|
import org.apache.hadoop.hbase.TagType;
|
||||||
import org.apache.hadoop.hbase.client.BufferedMutator;
|
import org.apache.hadoop.hbase.client.BufferedMutator;
|
||||||
import org.apache.hadoop.hbase.client.HTable;
|
|
||||||
import org.apache.hadoop.hbase.client.Put;
|
import org.apache.hadoop.hbase.client.Put;
|
||||||
import org.apache.hadoop.hbase.client.Table;
|
|
||||||
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
|
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
|
||||||
import org.apache.hadoop.hbase.mob.MobConstants;
|
import org.apache.hadoop.hbase.mob.MobConstants;
|
||||||
import org.apache.hadoop.hbase.mob.MobUtils;
|
import org.apache.hadoop.hbase.mob.MobUtils;
|
||||||
|
@ -140,8 +138,7 @@ public class MemStoreWrapper {
|
||||||
KeyValueScanner scanner = snapshot.getScanner();
|
KeyValueScanner scanner = snapshot.getScanner();
|
||||||
Cell cell = null;
|
Cell cell = null;
|
||||||
while (null != (cell = scanner.next())) {
|
while (null != (cell = scanner.next())) {
|
||||||
KeyValue kv = KeyValueUtil.ensureKeyValue(cell);
|
mobFileWriter.append(cell);
|
||||||
mobFileWriter.append(kv);
|
|
||||||
}
|
}
|
||||||
scanner.close();
|
scanner.close();
|
||||||
// Write out the log sequence number that corresponds to this output
|
// Write out the log sequence number that corresponds to this output
|
||||||
|
|
|
@ -73,8 +73,8 @@ import org.apache.zookeeper.KeeperException;
|
||||||
* This reducer merges the small mob files into bigger ones, and write visited
|
* This reducer merges the small mob files into bigger ones, and write visited
|
||||||
* names of mob files to a sequence file which is used by the sweep job to delete
|
* names of mob files to a sequence file which is used by the sweep job to delete
|
||||||
* the unused mob files.
|
* the unused mob files.
|
||||||
* The key of the input is a file name, the value is a collection of KeyValue where
|
* The key of the input is a file name, the value is a collection of KeyValues
|
||||||
* the KeyValue is the actual cell (its format is valueLength + fileName) in HBase.
|
* (the value format of KeyValue is valueLength + fileName) in HBase.
|
||||||
* In this reducer, we could know how many cells exist in HBase for a mob file.
|
* In this reducer, we could know how many cells exist in HBase for a mob file.
|
||||||
* If the existCellSize/mobFileSize < compactionRatio, this mob
|
* If the existCellSize/mobFileSize < compactionRatio, this mob
|
||||||
* file needs to be merged.
|
* file needs to be merged.
|
||||||
|
|
Loading…
Reference in New Issue