HBASE-11567 Write bulk load COMMIT events to WAL (Alex Newman, Jeffrey Zhong)
This commit is contained in:
parent
7f4146bf4d
commit
3f4427739d
|
@ -125,6 +125,8 @@ import org.apache.hadoop.hbase.protobuf.generated.WALProtos.FlushDescriptor;
|
|||
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.FlushDescriptor.FlushAction;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor.EventType;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.BulkLoadDescriptor;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor;
|
||||
import org.apache.hadoop.hbase.quotas.QuotaScope;
|
||||
import org.apache.hadoop.hbase.quotas.QuotaType;
|
||||
import org.apache.hadoop.hbase.quotas.ThrottleType;
|
||||
|
@ -2608,8 +2610,7 @@ public final class ProtobufUtil {
|
|||
.setServer(toServerName(server));
|
||||
|
||||
for (Map.Entry<byte[], List<Path>> entry : storeFiles.entrySet()) {
|
||||
RegionEventDescriptor.StoreDescriptor.Builder builder
|
||||
= RegionEventDescriptor.StoreDescriptor.newBuilder()
|
||||
StoreDescriptor.Builder builder = StoreDescriptor.newBuilder()
|
||||
.setFamilyName(ByteStringer.wrap(entry.getKey()))
|
||||
.setStoreHomeDir(Bytes.toString(entry.getKey()));
|
||||
for (Path path : entry.getValue()) {
|
||||
|
@ -2963,4 +2964,34 @@ public final class ProtobufUtil {
|
|||
.setScope(toProtoQuotaScope(scope))
|
||||
.build();
|
||||
}
|
||||
|
||||
/**
|
||||
* Generates a marker for the WAL so that we propagate the notion of a bulk region load
|
||||
* throughout the WAL.
|
||||
*
|
||||
* @param tableName The tableName into which the bulk load is being imported into.
|
||||
* @param encodedRegionName Encoded region name of the region which is being bulk loaded.
|
||||
* @param storeFiles A set of store files of a column family are bulk loaded.
|
||||
* @param bulkloadSeqId sequence ID (by a force flush) used to create bulk load hfile
|
||||
* name
|
||||
* @return The WAL log marker for bulk loads.
|
||||
*/
|
||||
public static WALProtos.BulkLoadDescriptor toBulkLoadDescriptor(TableName tableName,
|
||||
ByteString encodedRegionName, Map<byte[], List<Path>> storeFiles, long bulkloadSeqId) {
|
||||
BulkLoadDescriptor.Builder desc = BulkLoadDescriptor.newBuilder()
|
||||
.setTableName(ProtobufUtil.toProtoTableName(tableName))
|
||||
.setEncodedRegionName(encodedRegionName).setBulkloadSeqNum(bulkloadSeqId);
|
||||
|
||||
for (Map.Entry<byte[], List<Path>> entry : storeFiles.entrySet()) {
|
||||
WALProtos.StoreDescriptor.Builder builder = StoreDescriptor.newBuilder()
|
||||
.setFamilyName(ByteStringer.wrap(entry.getKey()))
|
||||
.setStoreHomeDir(Bytes.toString(entry.getKey())); // relative to region
|
||||
for (Path path : entry.getValue()) {
|
||||
builder.addStoreFile(path.getName());
|
||||
}
|
||||
desc.addStores(builder);
|
||||
}
|
||||
|
||||
return desc.build();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -16611,7 +16611,7 @@ public final class FilterProtos {
|
|||
/**
|
||||
* <code>repeated .RowRange row_range_list = 1;</code>
|
||||
*/
|
||||
java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowRangeOrBuilder>
|
||||
java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowRangeOrBuilder>
|
||||
getRowRangeListOrBuilderList();
|
||||
/**
|
||||
* <code>repeated .RowRange row_range_list = 1;</code>
|
||||
|
@ -17270,12 +17270,12 @@ public final class FilterProtos {
|
|||
/**
|
||||
* <code>repeated .RowRange row_range_list = 1;</code>
|
||||
*/
|
||||
public java.util.List<org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowRange.Builder>
|
||||
public java.util.List<org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowRange.Builder>
|
||||
getRowRangeListBuilderList() {
|
||||
return getRowRangeListFieldBuilder().getBuilderList();
|
||||
}
|
||||
private com.google.protobuf.RepeatedFieldBuilder<
|
||||
org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowRange, org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowRange.Builder, org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowRangeOrBuilder>
|
||||
org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowRange, org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowRange.Builder, org.apache.hadoop.hbase.protobuf.generated.FilterProtos.RowRangeOrBuilder>
|
||||
getRowRangeListFieldBuilder() {
|
||||
if (rowRangeListBuilder_ == null) {
|
||||
rowRangeListBuilder_ = new com.google.protobuf.RepeatedFieldBuilder<
|
||||
|
|
|
@ -5092,7 +5092,7 @@ public final class VisibilityLabelsProtos {
|
|||
if (ref instanceof java.lang.String) {
|
||||
return (java.lang.String) ref;
|
||||
} else {
|
||||
com.google.protobuf.ByteString bs =
|
||||
com.google.protobuf.ByteString bs =
|
||||
(com.google.protobuf.ByteString) ref;
|
||||
java.lang.String s = bs.toStringUtf8();
|
||||
if (bs.isValidUtf8()) {
|
||||
|
@ -5108,7 +5108,7 @@ public final class VisibilityLabelsProtos {
|
|||
getRegexBytes() {
|
||||
java.lang.Object ref = regex_;
|
||||
if (ref instanceof java.lang.String) {
|
||||
com.google.protobuf.ByteString b =
|
||||
com.google.protobuf.ByteString b =
|
||||
com.google.protobuf.ByteString.copyFromUtf8(
|
||||
(java.lang.String) ref);
|
||||
regex_ = b;
|
||||
|
@ -5414,7 +5414,7 @@ public final class VisibilityLabelsProtos {
|
|||
getRegexBytes() {
|
||||
java.lang.Object ref = regex_;
|
||||
if (ref instanceof String) {
|
||||
com.google.protobuf.ByteString b =
|
||||
com.google.protobuf.ByteString b =
|
||||
com.google.protobuf.ByteString.copyFromUtf8(
|
||||
(java.lang.String) ref);
|
||||
regex_ = b;
|
||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -22,6 +22,7 @@ option java_generate_equals_and_hash = true;
|
|||
option optimize_for = SPEED;
|
||||
|
||||
import "HBase.proto";
|
||||
import "Client.proto";
|
||||
|
||||
message WALHeader {
|
||||
optional bool has_compression = 1;
|
||||
|
@ -123,6 +124,22 @@ message FlushDescriptor {
|
|||
repeated StoreFlushDescriptor store_flushes = 5;
|
||||
}
|
||||
|
||||
message StoreDescriptor {
|
||||
required bytes family_name = 1;
|
||||
required string store_home_dir = 2; //relative to region dir
|
||||
repeated string store_file = 3; // relative to store dir
|
||||
}
|
||||
|
||||
/**
|
||||
* Special WAL entry used for writing bulk load events to WAL
|
||||
*/
|
||||
message BulkLoadDescriptor {
|
||||
required TableName table_name = 1;
|
||||
required bytes encoded_region_name = 2;
|
||||
repeated StoreDescriptor stores = 3;
|
||||
required int64 bulkload_seq_num = 4;
|
||||
}
|
||||
|
||||
/**
|
||||
* Special WAL entry to hold all related to a region event (open/close).
|
||||
*/
|
||||
|
@ -132,12 +149,6 @@ message RegionEventDescriptor {
|
|||
REGION_CLOSE = 1;
|
||||
}
|
||||
|
||||
message StoreDescriptor {
|
||||
required bytes family_name = 1;
|
||||
required string store_home_dir = 2; //relative to region dir
|
||||
repeated string store_file = 3; // relative to store dir
|
||||
}
|
||||
|
||||
required EventType event_type = 1;
|
||||
required bytes table_name = 2;
|
||||
required bytes encoded_region_name = 3;
|
||||
|
|
|
@ -61,6 +61,7 @@ import java.util.concurrent.atomic.AtomicLong;
|
|||
import java.util.concurrent.locks.Lock;
|
||||
import java.util.concurrent.locks.ReentrantReadWriteLock;
|
||||
|
||||
import com.google.protobuf.ByteString;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
|
@ -128,6 +129,7 @@ import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetRegionInfoRespo
|
|||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCall;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.WALProtos;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.CompactionDescriptor;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.FlushDescriptor;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.FlushDescriptor.FlushAction;
|
||||
|
@ -141,6 +143,7 @@ import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
|
|||
import org.apache.hadoop.hbase.regionserver.wal.WALUtil;
|
||||
import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
|
||||
import org.apache.hadoop.hbase.snapshot.SnapshotManifest;
|
||||
import org.apache.hadoop.hbase.util.ByteStringer;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.CancelableProgressable;
|
||||
import org.apache.hadoop.hbase.util.ClassSize;
|
||||
|
@ -221,7 +224,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { //
|
|||
* This is the global default value for durability. All tables/mutations not
|
||||
* defining a durability or using USE_DEFAULT will default to this value.
|
||||
*/
|
||||
private static final Durability DEFAULT_DURABLITY = Durability.SYNC_WAL;
|
||||
private static final Durability DEFAULT_DURABILITY = Durability.SYNC_WAL;
|
||||
|
||||
final AtomicBoolean closed = new AtomicBoolean(false);
|
||||
/* Closing can take some time; use the closing flag if there is stuff we don't
|
||||
|
@ -664,7 +667,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { //
|
|||
this.rowProcessorTimeout = conf.getLong(
|
||||
"hbase.hregion.row.processor.timeout", DEFAULT_ROW_PROCESSOR_TIMEOUT);
|
||||
this.durability = htd.getDurability() == Durability.USE_DEFAULT
|
||||
? DEFAULT_DURABLITY
|
||||
? DEFAULT_DURABILITY
|
||||
: htd.getDurability();
|
||||
if (rsServices != null) {
|
||||
this.rsAccounting = this.rsServices.getRegionServerAccounting();
|
||||
|
@ -753,7 +756,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { //
|
|||
}
|
||||
|
||||
private long initializeRegionInternals(final CancelableProgressable reporter,
|
||||
final MonitoredTask status) throws IOException, UnsupportedEncodingException {
|
||||
final MonitoredTask status) throws IOException {
|
||||
if (coprocessorHost != null) {
|
||||
status.setStatus("Running coprocessor pre-open hook");
|
||||
coprocessorHost.preOpen();
|
||||
|
@ -830,7 +833,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { //
|
|||
}
|
||||
|
||||
private long initializeRegionStores(final CancelableProgressable reporter, MonitoredTask status)
|
||||
throws IOException, UnsupportedEncodingException {
|
||||
throws IOException {
|
||||
// Load in all the HStores.
|
||||
|
||||
long maxSeqId = -1;
|
||||
|
@ -2001,8 +2004,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { //
|
|||
long oldestUnflushedSeqId = wal.getEarliestMemstoreSeqNum(encodedRegionName);
|
||||
// no oldestUnflushedSeqId means we flushed all stores.
|
||||
// or the unflushed stores are all empty.
|
||||
flushedSeqId =
|
||||
oldestUnflushedSeqId == HConstants.NO_SEQNUM ? flushOpSeqId : oldestUnflushedSeqId - 1;
|
||||
flushedSeqId = (oldestUnflushedSeqId == HConstants.NO_SEQNUM) ? flushOpSeqId
|
||||
: oldestUnflushedSeqId - 1;
|
||||
} else {
|
||||
// use the provided sequence Id as WAL is not being used for this flush.
|
||||
flushedSeqId = flushOpSeqId = myseqid;
|
||||
|
@ -2265,7 +2268,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { //
|
|||
return getScanner(scan, null);
|
||||
}
|
||||
|
||||
void prepareScanner(Scan scan) throws IOException {
|
||||
void prepareScanner(Scan scan) {
|
||||
if(!scan.hasFamilies()) {
|
||||
// Adding all families to scanner
|
||||
for(byte[] family: this.htableDescriptor.getFamiliesKeys()){
|
||||
|
@ -3241,7 +3244,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { //
|
|||
closeRegionOperation();
|
||||
}
|
||||
}
|
||||
private void doBatchMutate(Mutation mutation) throws IOException, DoNotRetryIOException {
|
||||
private void doBatchMutate(Mutation mutation) throws IOException {
|
||||
// Currently this is only called for puts and deletes, so no nonces.
|
||||
OperationStatus[] batchMutate = this.batchMutate(new Mutation[] { mutation },
|
||||
HConstants.NO_NONCE, HConstants.NO_NONCE);
|
||||
|
@ -3595,7 +3598,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { //
|
|||
protected long replayRecoveredEditsIfAny(final Path regiondir,
|
||||
Map<byte[], Long> maxSeqIdInStores,
|
||||
final CancelableProgressable reporter, final MonitoredTask status)
|
||||
throws UnsupportedEncodingException, IOException {
|
||||
throws IOException {
|
||||
long minSeqIdForTheRegion = -1;
|
||||
for (Long maxSeqIdInStore : maxSeqIdInStores.values()) {
|
||||
if (maxSeqIdInStore < minSeqIdForTheRegion || minSeqIdForTheRegion == -1) {
|
||||
|
@ -4101,7 +4104,17 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { //
|
|||
return multipleFamilies;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Bulk load a/many HFiles into this region
|
||||
*
|
||||
* @param familyPaths A list which maps column families to the location of the HFile to load
|
||||
* into that column family region.
|
||||
* @param assignSeqId Force a flush, get it's sequenceId to preserve the guarantee that all the
|
||||
* edits lower than the highest sequential ID from all the HFiles are flushed
|
||||
* on disk.
|
||||
* @return true if successful, false if failed recoverably
|
||||
* @throws IOException if failed unrecoverably.
|
||||
*/
|
||||
public boolean bulkLoadHFiles(List<Pair<byte[], String>> familyPaths,
|
||||
boolean assignSeqId) throws IOException {
|
||||
return bulkLoadHFiles(familyPaths, assignSeqId, null);
|
||||
|
@ -4111,14 +4124,19 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { //
|
|||
* Attempts to atomically load a group of hfiles. This is critical for loading
|
||||
* rows with multiple column families atomically.
|
||||
*
|
||||
* @param familyPaths List of Pair<byte[] column family, String hfilePath>
|
||||
* @param familyPaths List of Pair<byte[] column family, String hfilePath>
|
||||
* @param bulkLoadListener Internal hooks enabling massaging/preparation of a
|
||||
* file about to be bulk loaded
|
||||
* file about to be bulk loaded
|
||||
* @param assignSeqId Force a flush, get it's sequenceId to preserve the guarantee that
|
||||
* all the edits lower than the highest sequential ID from all the
|
||||
* HFiles are flushed on disk.
|
||||
* @return true if successful, false if failed recoverably
|
||||
* @throws IOException if failed unrecoverably.
|
||||
*/
|
||||
public boolean bulkLoadHFiles(List<Pair<byte[], String>> familyPaths, boolean assignSeqId,
|
||||
BulkLoadListener bulkLoadListener) throws IOException {
|
||||
long seqId = -1;
|
||||
Map<byte[], List<Path>> storeFiles = new TreeMap<byte[], List<Path>>(Bytes.BYTES_COMPARATOR);
|
||||
Preconditions.checkNotNull(familyPaths);
|
||||
// we need writeLock for multi-family bulk load
|
||||
startBulkRegionOperation(hasMultipleColumnFamilies(familyPaths));
|
||||
|
@ -4164,7 +4182,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { //
|
|||
StringBuilder list = new StringBuilder();
|
||||
for (Pair<byte[], String> p : failures) {
|
||||
list.append("\n").append(Bytes.toString(p.getFirst())).append(" : ")
|
||||
.append(p.getSecond());
|
||||
.append(p.getSecond());
|
||||
}
|
||||
// problem when validating
|
||||
LOG.warn("There was a recoverable bulk load failure likely due to a" +
|
||||
|
@ -4172,7 +4190,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { //
|
|||
return false;
|
||||
}
|
||||
|
||||
long seqId = -1;
|
||||
// We need to assign a sequential ID that's in between two memstores in order to preserve
|
||||
// the guarantee that all the edits lower than the highest sequential ID from all the
|
||||
// HFiles are flushed on disk. See HBASE-10958. The sequence id returned when we flush is
|
||||
|
@ -4196,11 +4213,19 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { //
|
|||
Store store = getStore(familyName);
|
||||
try {
|
||||
String finalPath = path;
|
||||
if(bulkLoadListener != null) {
|
||||
if (bulkLoadListener != null) {
|
||||
finalPath = bulkLoadListener.prepareBulkLoad(familyName, path);
|
||||
}
|
||||
store.bulkLoadHFile(finalPath, seqId);
|
||||
if(bulkLoadListener != null) {
|
||||
|
||||
if(storeFiles.containsKey(familyName)) {
|
||||
storeFiles.get(familyName).add(new Path(finalPath));
|
||||
} else {
|
||||
List<Path> storeFileNames = new ArrayList<Path>();
|
||||
storeFileNames.add(new Path(finalPath));
|
||||
storeFiles.put(familyName, storeFileNames);
|
||||
}
|
||||
if (bulkLoadListener != null) {
|
||||
bulkLoadListener.doneBulkLoad(familyName, path);
|
||||
}
|
||||
} catch (IOException ioe) {
|
||||
|
@ -4209,20 +4234,38 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { //
|
|||
|
||||
// TODO Need a better story for reverting partial failures due to HDFS.
|
||||
LOG.error("There was a partial failure due to IO when attempting to" +
|
||||
" load " + Bytes.toString(p.getFirst()) + " : "+ p.getSecond(), ioe);
|
||||
if(bulkLoadListener != null) {
|
||||
" load " + Bytes.toString(p.getFirst()) + " : " + p.getSecond(), ioe);
|
||||
if (bulkLoadListener != null) {
|
||||
try {
|
||||
bulkLoadListener.failedBulkLoad(familyName, path);
|
||||
} catch (Exception ex) {
|
||||
LOG.error("Error while calling failedBulkLoad for family "+
|
||||
Bytes.toString(familyName)+" with path "+path, ex);
|
||||
LOG.error("Error while calling failedBulkLoad for family " +
|
||||
Bytes.toString(familyName) + " with path " + path, ex);
|
||||
}
|
||||
}
|
||||
throw ioe;
|
||||
}
|
||||
}
|
||||
|
||||
return true;
|
||||
} finally {
|
||||
if (wal != null && !storeFiles.isEmpty()) {
|
||||
// write a bulk load event when not all hfiles are loaded
|
||||
try {
|
||||
WALProtos.BulkLoadDescriptor loadDescriptor = ProtobufUtil.toBulkLoadDescriptor(
|
||||
this.getRegionInfo().getTable(),
|
||||
ByteStringer.wrap(this.getRegionInfo().getEncodedNameAsBytes()), storeFiles, seqId);
|
||||
WALUtil.writeBulkLoadMarkerAndSync(wal, this.htableDescriptor, getRegionInfo(),
|
||||
loadDescriptor, sequenceId);
|
||||
} catch (IOException ioe) {
|
||||
if (this.rsServices != null) {
|
||||
// Have to abort region server because some hfiles has been loaded but we can't write
|
||||
// the event into WAL
|
||||
this.rsServices.abort("Failed to write bulk load event into WAL.", ioe);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
closeBulkRegionOperation();
|
||||
}
|
||||
}
|
||||
|
@ -5340,8 +5383,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { //
|
|||
doProcessRowWithTimeout(
|
||||
processor, now, this, null, null, timeout);
|
||||
processor.postProcess(this, walEdit, true);
|
||||
} catch (IOException e) {
|
||||
throw e;
|
||||
} finally {
|
||||
closeRegionOperation();
|
||||
}
|
||||
|
@ -5460,8 +5501,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { //
|
|||
// 14. Run post-process hook
|
||||
processor.postProcess(this, walEdit, walSyncSuccessful);
|
||||
|
||||
} catch (IOException e) {
|
||||
throw e;
|
||||
} finally {
|
||||
closeRegionOperation();
|
||||
if (!mutations.isEmpty() &&
|
||||
|
@ -5621,8 +5660,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { //
|
|||
}
|
||||
}
|
||||
if (cell.getTagsLength() > 0) {
|
||||
Iterator<Tag> i = CellUtil.tagsIterator(cell.getTagsArray(), cell.getTagsOffset(),
|
||||
cell.getTagsLength());
|
||||
Iterator<Tag> i = CellUtil.tagsIterator(cell.getTagsArray(),
|
||||
cell.getTagsOffset(), cell.getTagsLength());
|
||||
while (i.hasNext()) {
|
||||
newTags.add(i.next());
|
||||
}
|
||||
|
@ -6534,7 +6573,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { //
|
|||
}
|
||||
|
||||
/**
|
||||
* A mocked list implementaion - discards all updates.
|
||||
* A mocked list implementation - discards all updates.
|
||||
*/
|
||||
private static final List<Cell> MOCKED_LIST = new AbstractList<Cell>() {
|
||||
|
||||
|
@ -6785,7 +6824,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { //
|
|||
}
|
||||
|
||||
/**
|
||||
* Explictly sync wal
|
||||
* Explicitly sync wal
|
||||
* @throws IOException
|
||||
*/
|
||||
public void syncWal() throws IOException {
|
||||
|
|
|
@ -36,6 +36,7 @@ import org.apache.hadoop.hbase.KeyValue;
|
|||
import org.apache.hadoop.hbase.KeyValueUtil;
|
||||
import org.apache.hadoop.hbase.codec.Codec;
|
||||
import org.apache.hadoop.hbase.io.HeapSize;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.WALProtos;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.CompactionDescriptor;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.FlushDescriptor;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor;
|
||||
|
@ -90,6 +91,7 @@ public class WALEdit implements Writable, HeapSize {
|
|||
static final byte[] COMPACTION = Bytes.toBytes("HBASE::COMPACTION");
|
||||
static final byte [] FLUSH = Bytes.toBytes("HBASE::FLUSH");
|
||||
static final byte [] REGION_EVENT = Bytes.toBytes("HBASE::REGION_EVENT");
|
||||
public static final byte [] BULK_LOAD = Bytes.toBytes("HBASE::BULK_LOAD");
|
||||
|
||||
private final int VERSION_2 = -1;
|
||||
private final boolean isReplay;
|
||||
|
@ -294,7 +296,7 @@ public class WALEdit implements Writable, HeapSize {
|
|||
}
|
||||
|
||||
/**
|
||||
* Create a compacion WALEdit
|
||||
* Create a compaction WALEdit
|
||||
* @param c
|
||||
* @return A WALEdit that has <code>c</code> serialized as its value
|
||||
*/
|
||||
|
@ -326,4 +328,33 @@ public class WALEdit implements Writable, HeapSize {
|
|||
}
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a bulk loader WALEdit
|
||||
*
|
||||
* @param hri The HRegionInfo for the region in which we are bulk loading
|
||||
* @param bulkLoadDescriptor The descriptor for the Bulk Loader
|
||||
* @return The WALEdit for the BulkLoad
|
||||
*/
|
||||
public static WALEdit createBulkLoadEvent(HRegionInfo hri,
|
||||
WALProtos.BulkLoadDescriptor bulkLoadDescriptor) {
|
||||
KeyValue kv = new KeyValue(getRowForRegion(hri),
|
||||
METAFAMILY,
|
||||
BULK_LOAD,
|
||||
EnvironmentEdgeManager.currentTime(),
|
||||
bulkLoadDescriptor.toByteArray());
|
||||
return new WALEdit().add(kv);
|
||||
}
|
||||
|
||||
/**
|
||||
* Deserialized and returns a BulkLoadDescriptor from the passed in Cell
|
||||
* @param cell the key value
|
||||
* @return deserialized BulkLoadDescriptor or null.
|
||||
*/
|
||||
public static WALProtos.BulkLoadDescriptor getBulkLoadDescriptor(Cell cell) throws IOException {
|
||||
if (CellUtil.matchingColumn(cell, METAFAMILY, BULK_LOAD)) {
|
||||
return WALProtos.BulkLoadDescriptor.parseFrom(cell.getValue());
|
||||
}
|
||||
return null;
|
||||
}
|
||||
}
|
|
@ -20,14 +20,17 @@
|
|||
package org.apache.hadoop.hbase.regionserver.wal;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.WALProtos;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.CompactionDescriptor;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.FlushDescriptor;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor;
|
||||
|
@ -97,5 +100,41 @@ public class WALUtil {
|
|||
}
|
||||
return trx;
|
||||
}
|
||||
|
||||
/**
|
||||
* Write a log marker that a bulk load has succeeded and is about to be committed.
|
||||
*
|
||||
* @param wal The log to write into.
|
||||
* @param htd A description of the table that we are bulk loading into.
|
||||
* @param info A description of the region in the table that we are bulk loading into.
|
||||
* @param descriptor A protocol buffers based description of the client's bulk loading request
|
||||
* @param sequenceId The current sequenceId in the log at the time when we were to write the
|
||||
* bulk load marker.
|
||||
* @return txid of this transaction or if nothing to do, the last txid
|
||||
* @throws IOException We will throw an IOException if we can not append to the HLog.
|
||||
*/
|
||||
public static long writeBulkLoadMarkerAndSync(final WAL wal,
|
||||
final HTableDescriptor htd,
|
||||
final HRegionInfo info,
|
||||
final WALProtos.BulkLoadDescriptor descriptor,
|
||||
final AtomicLong sequenceId) throws IOException {
|
||||
TableName tn = info.getTable();
|
||||
WALKey key = new HLogKey(info.getEncodedNameAsBytes(), tn);
|
||||
|
||||
// Add it to the log but the false specifies that we don't need to add it to the memstore
|
||||
long trx = wal.append(htd,
|
||||
info,
|
||||
key,
|
||||
WALEdit.createBulkLoadEvent(info, descriptor),
|
||||
sequenceId,
|
||||
false,
|
||||
new ArrayList<Cell>());
|
||||
wal.sync(trx);
|
||||
|
||||
if (LOG.isTraceEnabled()) {
|
||||
LOG.trace("Appended Bulk Load marker " + TextFormat.shortDebugString(descriptor));
|
||||
}
|
||||
return trx;
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -0,0 +1,312 @@
|
|||
/**
|
||||
* 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 org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.DoNotRetryIOException;
|
||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.testclassification.SmallTests;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.io.hfile.HFile;
|
||||
import org.apache.hadoop.hbase.io.hfile.HFileContext;
|
||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.BulkLoadDescriptor;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.Pair;
|
||||
import org.apache.hadoop.hbase.wal.WAL;
|
||||
import org.apache.hadoop.hbase.wal.WALKey;
|
||||
import org.hamcrest.Description;
|
||||
import org.hamcrest.Matcher;
|
||||
import org.hamcrest.TypeSafeMatcher;
|
||||
import org.jmock.Expectations;
|
||||
import org.jmock.integration.junit4.JUnitRuleMockery;
|
||||
import org.jmock.lib.concurrent.Synchroniser;
|
||||
import org.junit.Before;
|
||||
import org.junit.ClassRule;
|
||||
import org.junit.Rule;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
import org.junit.rules.TemporaryFolder;
|
||||
import org.junit.rules.TestName;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.FileNotFoundException;
|
||||
import java.io.FileOutputStream;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.Random;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
||||
import static java.util.Arrays.asList;
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertNotNull;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
/**
|
||||
* This class attempts to unit test bulk HLog loading.
|
||||
*/
|
||||
@Category(SmallTests.class)
|
||||
public class TestBulkLoad {
|
||||
|
||||
@ClassRule
|
||||
public static TemporaryFolder testFolder = new TemporaryFolder();
|
||||
@Rule
|
||||
public final JUnitRuleMockery context = new JUnitRuleMockery() {{
|
||||
setThreadingPolicy(new Synchroniser());
|
||||
}};
|
||||
private final WAL log = context.mock(WAL.class);
|
||||
private final Configuration conf = HBaseConfiguration.create();
|
||||
private final Random random = new Random();
|
||||
private final byte[] randomBytes = new byte[100];
|
||||
private final byte[] family1 = Bytes.toBytes("family1");
|
||||
private final byte[] family2 = Bytes.toBytes("family2");
|
||||
private final Expectations callOnce;
|
||||
@Rule
|
||||
public TestName name = new TestName();
|
||||
|
||||
public TestBulkLoad() throws IOException {
|
||||
callOnce = new Expectations() {
|
||||
{
|
||||
oneOf(log).append(with(any(HTableDescriptor.class)), with(any(HRegionInfo.class)),
|
||||
with(any(WALKey.class)), with(bulkLogWalEditType(WALEdit.BULK_LOAD)),
|
||||
with(any(AtomicLong.class)), with(any(boolean.class)), with(any(List.class)));
|
||||
will(returnValue(0l));
|
||||
oneOf(log).sync(with(any(long.class)));
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Before
|
||||
public void before() throws IOException {
|
||||
random.nextBytes(randomBytes);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void verifyBulkLoadEvent() throws IOException {
|
||||
TableName tableName = TableName.valueOf("test", "test");
|
||||
List<Pair<byte[], String>> familyPaths = withFamilyPathsFor(family1);
|
||||
byte[] familyName = familyPaths.get(0).getFirst();
|
||||
String storeFileName = familyPaths.get(0).getSecond();
|
||||
storeFileName = (new Path(storeFileName)).getName();
|
||||
List<String> storeFileNames = new ArrayList<String>();
|
||||
storeFileNames.add(storeFileName);
|
||||
final Matcher<WALEdit> bulkEventMatcher = bulkLogWalEdit(WALEdit.BULK_LOAD,
|
||||
tableName.toBytes(), familyName, storeFileNames);
|
||||
Expectations expection = new Expectations() {
|
||||
{
|
||||
oneOf(log).append(with(any(HTableDescriptor.class)), with(any(HRegionInfo.class)),
|
||||
with(any(WALKey.class)), with(bulkEventMatcher),
|
||||
with(any(AtomicLong.class)), with(any(boolean.class)), with(any(List.class)));
|
||||
will(returnValue(0l));
|
||||
oneOf(log).sync(with(any(long.class)));
|
||||
}
|
||||
};
|
||||
context.checking(expection);
|
||||
testRegionWithFamiliesAndSpecifiedTableName(tableName, family1)
|
||||
.bulkLoadHFiles(familyPaths, false);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void bulkHLogShouldThrowNoErrorAndWriteMarkerWithBlankInput() throws IOException {
|
||||
testRegionWithFamilies(family1).bulkLoadHFiles(new ArrayList<Pair<byte[], String>>(), false);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void shouldBulkLoadSingleFamilyHLog() throws IOException {
|
||||
context.checking(callOnce);
|
||||
testRegionWithFamilies(family1).bulkLoadHFiles(withFamilyPathsFor(family1), false);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void shouldBulkLoadManyFamilyHLog() throws IOException {
|
||||
context.checking(callOnce);
|
||||
testRegionWithFamilies(family1, family2).bulkLoadHFiles(withFamilyPathsFor(family1, family2),
|
||||
false);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void shouldBulkLoadManyFamilyHLogEvenWhenTableNameNamespaceSpecified() throws IOException {
|
||||
context.checking(callOnce);
|
||||
TableName tableName = TableName.valueOf("test", "test");
|
||||
testRegionWithFamiliesAndSpecifiedTableName(tableName, family1, family2)
|
||||
.bulkLoadHFiles(withFamilyPathsFor(family1, family2), false);
|
||||
}
|
||||
|
||||
@Test(expected = DoNotRetryIOException.class)
|
||||
public void shouldCrashIfBulkLoadFamiliesNotInTable() throws IOException {
|
||||
testRegionWithFamilies(family1).bulkLoadHFiles(withFamilyPathsFor(family1, family2), false);
|
||||
}
|
||||
|
||||
@Test(expected = DoNotRetryIOException.class)
|
||||
public void bulkHLogShouldThrowErrorWhenFamilySpecifiedAndHFileExistsButNotInTableDescriptor()
|
||||
throws IOException {
|
||||
testRegionWithFamilies().bulkLoadHFiles(withFamilyPathsFor(family1), false);
|
||||
}
|
||||
|
||||
@Test(expected = DoNotRetryIOException.class)
|
||||
public void shouldThrowErrorIfBadFamilySpecifiedAsFamilyPath() throws IOException {
|
||||
testRegionWithFamilies()
|
||||
.bulkLoadHFiles(asList(withInvalidColumnFamilyButProperHFileLocation(family1)),
|
||||
false);
|
||||
}
|
||||
|
||||
@Test(expected = FileNotFoundException.class)
|
||||
public void shouldThrowErrorIfHFileDoesNotExist() throws IOException {
|
||||
List<Pair<byte[], String>> list = asList(withMissingHFileForFamily(family1));
|
||||
testRegionWithFamilies(family1).bulkLoadHFiles(list, false);
|
||||
}
|
||||
|
||||
private Pair<byte[], String> withMissingHFileForFamily(byte[] family) {
|
||||
return new Pair<byte[], String>(family, "/tmp/does_not_exist");
|
||||
}
|
||||
|
||||
private Pair<byte[], String> withInvalidColumnFamilyButProperHFileLocation(byte[] family)
|
||||
throws IOException {
|
||||
createHFileForFamilies(family);
|
||||
return new Pair<byte[], String>(new byte[]{0x00, 0x01, 0x02}, "/tmp/does_not_exist");
|
||||
}
|
||||
|
||||
|
||||
private HRegion testRegionWithFamiliesAndSpecifiedTableName(TableName tableName,
|
||||
byte[]... families)
|
||||
throws IOException {
|
||||
HRegionInfo hRegionInfo = new HRegionInfo(tableName);
|
||||
HTableDescriptor hTableDescriptor = new HTableDescriptor(tableName);
|
||||
for (byte[] family : families) {
|
||||
hTableDescriptor.addFamily(new HColumnDescriptor(family));
|
||||
}
|
||||
|
||||
// TODO We need a way to do this without creating files
|
||||
return HRegion.createHRegion(hRegionInfo,
|
||||
new Path(testFolder.newFolder().toURI()),
|
||||
conf,
|
||||
hTableDescriptor,
|
||||
log);
|
||||
|
||||
}
|
||||
|
||||
private HRegion testRegionWithFamilies(byte[]... families) throws IOException {
|
||||
TableName tableName = TableName.valueOf(name.getMethodName());
|
||||
return testRegionWithFamiliesAndSpecifiedTableName(tableName, families);
|
||||
}
|
||||
|
||||
private List<Pair<byte[], String>> getBlankFamilyPaths(){
|
||||
return new ArrayList<Pair<byte[], String>>();
|
||||
}
|
||||
|
||||
private List<Pair<byte[], String>> withFamilyPathsFor(byte[]... families) throws IOException {
|
||||
List<Pair<byte[], String>> familyPaths = getBlankFamilyPaths();
|
||||
for (byte[] family : families) {
|
||||
familyPaths.add(new Pair<byte[], String>(family, createHFileForFamilies(family)));
|
||||
}
|
||||
return familyPaths;
|
||||
}
|
||||
|
||||
private String createHFileForFamilies(byte[] family) throws IOException {
|
||||
HFile.WriterFactory hFileFactory = HFile.getWriterFactoryNoCache(conf);
|
||||
// TODO We need a way to do this without creating files
|
||||
File hFileLocation = testFolder.newFile();
|
||||
hFileFactory.withOutputStream(new FSDataOutputStream(new FileOutputStream(hFileLocation)));
|
||||
hFileFactory.withFileContext(new HFileContext());
|
||||
HFile.Writer writer = hFileFactory.create();
|
||||
|
||||
writer.append(new KeyValue(CellUtil.createCell(randomBytes,
|
||||
family,
|
||||
randomBytes,
|
||||
0l,
|
||||
KeyValue.Type.Put.getCode(),
|
||||
randomBytes)));
|
||||
writer.close();
|
||||
return hFileLocation.getAbsoluteFile().getAbsolutePath();
|
||||
}
|
||||
|
||||
private static Matcher<WALEdit> bulkLogWalEditType(byte[] typeBytes) {
|
||||
return new WalMatcher(typeBytes);
|
||||
}
|
||||
|
||||
private static Matcher<WALEdit> bulkLogWalEdit(byte[] typeBytes, byte[] tableName,
|
||||
byte[] familyName, List<String> storeFileNames) {
|
||||
return new WalMatcher(typeBytes, tableName, familyName, storeFileNames);
|
||||
}
|
||||
|
||||
private static class WalMatcher extends TypeSafeMatcher<WALEdit> {
|
||||
private final byte[] typeBytes;
|
||||
private final byte[] tableName;
|
||||
private final byte[] familyName;
|
||||
private final List<String> storeFileNames;
|
||||
|
||||
public WalMatcher(byte[] typeBytes) {
|
||||
this(typeBytes, null, null, null);
|
||||
}
|
||||
|
||||
public WalMatcher(byte[] typeBytes, byte[] tableName, byte[] familyName,
|
||||
List<String> storeFileNames) {
|
||||
this.typeBytes = typeBytes;
|
||||
this.tableName = tableName;
|
||||
this.familyName = familyName;
|
||||
this.storeFileNames = storeFileNames;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean matchesSafely(WALEdit item) {
|
||||
assertTrue(Arrays.equals(item.getCells().get(0).getQualifier(), typeBytes));
|
||||
BulkLoadDescriptor desc;
|
||||
try {
|
||||
desc = WALEdit.getBulkLoadDescriptor(item.getCells().get(0));
|
||||
} catch (IOException e) {
|
||||
return false;
|
||||
}
|
||||
assertNotNull(desc);
|
||||
|
||||
if (tableName != null) {
|
||||
assertTrue(Bytes.equals(ProtobufUtil.toTableName(desc.getTableName()).getName(),
|
||||
tableName));
|
||||
}
|
||||
|
||||
if(storeFileNames != null) {
|
||||
int index=0;
|
||||
StoreDescriptor store = desc.getStores(0);
|
||||
assertTrue(Bytes.equals(store.getFamilyName().toByteArray(), familyName));
|
||||
assertTrue(Bytes.equals(Bytes.toBytes(store.getStoreHomeDir()), familyName));
|
||||
assertEquals(storeFileNames.size(), store.getStoreFileCount());
|
||||
for (String storeFile : store.getStoreFileList()) {
|
||||
assertTrue(storeFile.equals(storeFileNames.get(index++)));
|
||||
}
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void describeTo(Description description) {
|
||||
|
||||
}
|
||||
}
|
||||
}
|
|
@ -124,7 +124,7 @@ import org.apache.hadoop.hbase.protobuf.generated.WALProtos.FlushDescriptor;
|
|||
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.FlushDescriptor.FlushAction;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.FlushDescriptor.StoreFlushDescriptor;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor.StoreDescriptor;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegion.RegionScannerImpl;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegion.RowLock;
|
||||
import org.apache.hadoop.hbase.regionserver.TestStore.FaultyFileSystem;
|
||||
|
|
|
@ -17,27 +17,24 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.regionserver;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
||||
import com.google.common.collect.Lists;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
||||
import org.apache.hadoop.hbase.MultithreadedTestUtil.RepeatingTestThread;
|
||||
import org.apache.hadoop.hbase.MultithreadedTestUtil.TestContext;
|
||||
import org.apache.hadoop.hbase.TableExistsException;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.client.HConnection;
|
||||
import org.apache.hadoop.hbase.client.HTable;
|
||||
import org.apache.hadoop.hbase.client.RegionServerCallable;
|
||||
import org.apache.hadoop.hbase.client.Result;
|
||||
import org.apache.hadoop.hbase.client.ResultScanner;
|
||||
|
@ -55,15 +52,28 @@ import org.apache.hadoop.hbase.protobuf.RequestConverter;
|
|||
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CompactRegionRequest;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.BulkLoadHFileRequest;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.TestWALActionsListener;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
|
||||
import org.apache.hadoop.hbase.testclassification.LargeTests;
|
||||
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.Pair;
|
||||
import org.apache.hadoop.hbase.wal.WAL;
|
||||
import org.apache.hadoop.hbase.wal.WALKey;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
|
||||
import com.google.common.collect.Lists;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
||||
import static org.hamcrest.core.Is.is;
|
||||
import static org.junit.Assert.assertThat;
|
||||
|
||||
/**
|
||||
* Tests bulk loading of HFiles and shows the atomicity or lack of atomicity of
|
||||
* the region server's bullkLoad functionality.
|
||||
|
@ -290,7 +300,11 @@ public class TestHRegionServerBulkLoad {
|
|||
|
||||
UTIL.startMiniCluster(1);
|
||||
try {
|
||||
WAL log = UTIL.getHBaseCluster().getRegionServer(0).getWAL(null);
|
||||
FindBulkHBaseListener listener = new FindBulkHBaseListener();
|
||||
log.registerWALActionsListener(listener);
|
||||
runAtomicBulkloadTest(TABLE_NAME, millisToRun, numScanners);
|
||||
assertThat(listener.isFound(), is(true));
|
||||
} finally {
|
||||
UTIL.shutdownMiniCluster();
|
||||
}
|
||||
|
@ -346,5 +360,25 @@ public class TestHRegionServerBulkLoad {
|
|||
UTIL = new HBaseTestingUtility(c);
|
||||
}
|
||||
|
||||
static class FindBulkHBaseListener extends TestWALActionsListener.DummyWALActionsListener {
|
||||
private boolean found = false;
|
||||
|
||||
@Override
|
||||
public void visitLogEntryBeforeWrite(HTableDescriptor htd, WALKey logKey, WALEdit logEdit) {
|
||||
for (Cell cell : logEdit.getCells()) {
|
||||
KeyValue kv = KeyValueUtil.ensureKeyValue(cell);
|
||||
for (Map.Entry entry : kv.toStringMap().entrySet()) {
|
||||
if (entry.getValue().equals(Bytes.toString(WALEdit.BULK_LOAD))) {
|
||||
found = true;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public boolean isFound() {
|
||||
return found;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
|
|
|
@ -125,7 +125,7 @@ public class TestWALActionsListener {
|
|||
/**
|
||||
* Just counts when methods are called
|
||||
*/
|
||||
static class DummyWALActionsListener extends WALActionsListener.Base {
|
||||
public static class DummyWALActionsListener extends WALActionsListener.Base {
|
||||
public int preLogRollCounter = 0;
|
||||
public int postLogRollCounter = 0;
|
||||
public int closedCount = 0;
|
||||
|
|
17
pom.xml
17
pom.xml
|
@ -1034,6 +1034,7 @@
|
|||
<jetty.version>6.1.26</jetty.version>
|
||||
<jetty.jspapi.version>6.1.14</jetty.jspapi.version>
|
||||
<jersey.version>1.9</jersey.version>
|
||||
<jmock-junit4.version>2.6.0</jmock-junit4.version>
|
||||
<jruby.version>1.6.8</jruby.version>
|
||||
<junit.version>4.11</junit.version>
|
||||
<hamcrest.version>1.3</hamcrest.version>
|
||||
|
@ -1540,6 +1541,18 @@
|
|||
<artifactId>disruptor</artifactId>
|
||||
<version>${disruptor.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.jmock</groupId>
|
||||
<artifactId>jmock-junit4</artifactId>
|
||||
<version>${jmock-junit4.version}</version>
|
||||
<scope>test</scope>
|
||||
<exclusions>
|
||||
<exclusion>
|
||||
<artifactId>junit-dep</artifactId>
|
||||
<groupId>junit</groupId>
|
||||
</exclusion>
|
||||
</exclusions>
|
||||
</dependency>
|
||||
</dependencies>
|
||||
</dependencyManagement>
|
||||
<!-- Dependencies needed by subprojects -->
|
||||
|
@ -1563,6 +1576,10 @@
|
|||
<groupId>org.mockito</groupId>
|
||||
<artifactId>mockito-all</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.jmock</groupId>
|
||||
<artifactId>jmock-junit4</artifactId>
|
||||
</dependency>
|
||||
</dependencies>
|
||||
<!--
|
||||
To publish, use the following settings.xml file ( placed in ~/.m2/settings.xml )
|
||||
|
|
Loading…
Reference in New Issue