You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by je...@apache.org on 2015/02/07 03:56:17 UTC
[1/3] hbase git commit: HBASE-11567 Write bulk load COMMIT events to
WAL
Repository: hbase
Updated Branches:
refs/heads/branch-1 a7b811255 -> b0b0a74fe
http://git-wip-us.apache.org/repos/asf/hbase/blob/b0b0a74f/hbase-protocol/src/main/protobuf/WAL.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol/src/main/protobuf/WAL.proto b/hbase-protocol/src/main/protobuf/WAL.proto
index f8a1534..169a9b2 100644
--- a/hbase-protocol/src/main/protobuf/WAL.proto
+++ b/hbase-protocol/src/main/protobuf/WAL.proto
@@ -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;
http://git-wip-us.apache.org/repos/asf/hbase/blob/b0b0a74f/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
index dbb74a6..edc272c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
@@ -62,6 +62,7 @@ import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReentrantReadWriteLock;
import org.apache.commons.lang.RandomStringUtils;
+import com.google.protobuf.ByteString;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
@@ -129,6 +130,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;
@@ -144,6 +146,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;
@@ -224,7 +227,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
@@ -669,7 +672,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();
@@ -758,7 +761,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();
@@ -835,7 +838,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;
@@ -1999,8 +2002,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;
@@ -2263,7 +2266,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()){
@@ -3239,7 +3242,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);
@@ -3596,7 +3599,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) {
@@ -4102,7 +4105,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);
@@ -4112,14 +4125,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));
@@ -4165,7 +4183,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" +
@@ -4173,7 +4191,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
@@ -4197,11 +4214,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) {
@@ -4210,20 +4235,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();
}
}
@@ -5444,8 +5487,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();
}
@@ -5564,8 +5605,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() &&
@@ -5726,8 +5765,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());
}
@@ -6638,7 +6677,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>() {
@@ -6882,7 +6921,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { //
}
/**
- * Explictly sync wal
+ * Explicitly sync wal
* @throws IOException
*/
public void syncWal() throws IOException {
http://git-wip-us.apache.org/repos/asf/hbase/blob/b0b0a74f/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEdit.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEdit.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEdit.java
index 05cead2..39d0536 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEdit.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEdit.java
@@ -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;
+ }
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/hbase/blob/b0b0a74f/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALUtil.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALUtil.java
index 5f00643..94ef072 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALUtil.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALUtil.java
@@ -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;
+ }
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/b0b0a74f/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBulkLoad.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBulkLoad.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBulkLoad.java
new file mode 100644
index 0000000..15dbef5
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBulkLoad.java
@@ -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) {
+
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/hbase/blob/b0b0a74f/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
index b238cd5..27e225d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
@@ -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;
http://git-wip-us.apache.org/repos/asf/hbase/blob/b0b0a74f/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoad.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoad.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoad.java
index d6f4a67..9ebbc59 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoad.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoad.java
@@ -17,28 +17,26 @@
*/
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.client.HTable;
import org.apache.hadoop.hbase.testclassification.LargeTests;
+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,13 +53,26 @@ 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.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.
@@ -288,7 +299,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();
}
@@ -344,5 +359,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;
+ }
+ }
}
+
http://git-wip-us.apache.org/repos/asf/hbase/blob/b0b0a74f/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALActionsListener.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALActionsListener.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALActionsListener.java
index 87fcd2e..46a84f0 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALActionsListener.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALActionsListener.java
@@ -124,7 +124,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;
http://git-wip-us.apache.org/repos/asf/hbase/blob/b0b0a74f/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index d3d96ba..0a72084 100644
--- a/pom.xml
+++ b/pom.xml
@@ -1027,6 +1027,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>
@@ -1545,6 +1546,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 -->
@@ -1568,6 +1581,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 )
[2/3] hbase git commit: HBASE-11567 Write bulk load COMMIT events to
WAL
Posted by je...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/b0b0a74f/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/WALProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/WALProtos.java b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/WALProtos.java
index 977db42..c9fa854 100644
--- a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/WALProtos.java
+++ b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/WALProtos.java
@@ -7604,124 +7604,101 @@ public final class WALProtos {
// @@protoc_insertion_point(class_scope:FlushDescriptor)
}
- public interface RegionEventDescriptorOrBuilder
+ public interface StoreDescriptorOrBuilder
extends com.google.protobuf.MessageOrBuilder {
- // required .RegionEventDescriptor.EventType event_type = 1;
- /**
- * <code>required .RegionEventDescriptor.EventType event_type = 1;</code>
- */
- boolean hasEventType();
- /**
- * <code>required .RegionEventDescriptor.EventType event_type = 1;</code>
- */
- org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor.EventType getEventType();
-
- // required bytes table_name = 2;
+ // required bytes family_name = 1;
/**
- * <code>required bytes table_name = 2;</code>
+ * <code>required bytes family_name = 1;</code>
*/
- boolean hasTableName();
+ boolean hasFamilyName();
/**
- * <code>required bytes table_name = 2;</code>
+ * <code>required bytes family_name = 1;</code>
*/
- com.google.protobuf.ByteString getTableName();
+ com.google.protobuf.ByteString getFamilyName();
- // required bytes encoded_region_name = 3;
- /**
- * <code>required bytes encoded_region_name = 3;</code>
- */
- boolean hasEncodedRegionName();
+ // required string store_home_dir = 2;
/**
- * <code>required bytes encoded_region_name = 3;</code>
+ * <code>required string store_home_dir = 2;</code>
+ *
+ * <pre>
+ *relative to region dir
+ * </pre>
*/
- com.google.protobuf.ByteString getEncodedRegionName();
-
- // optional uint64 log_sequence_number = 4;
+ boolean hasStoreHomeDir();
/**
- * <code>optional uint64 log_sequence_number = 4;</code>
+ * <code>required string store_home_dir = 2;</code>
+ *
+ * <pre>
+ *relative to region dir
+ * </pre>
*/
- boolean hasLogSequenceNumber();
+ java.lang.String getStoreHomeDir();
/**
- * <code>optional uint64 log_sequence_number = 4;</code>
+ * <code>required string store_home_dir = 2;</code>
+ *
+ * <pre>
+ *relative to region dir
+ * </pre>
*/
- long getLogSequenceNumber();
+ com.google.protobuf.ByteString
+ getStoreHomeDirBytes();
- // repeated .RegionEventDescriptor.StoreDescriptor stores = 5;
- /**
- * <code>repeated .RegionEventDescriptor.StoreDescriptor stores = 5;</code>
- */
- java.util.List<org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor.StoreDescriptor>
- getStoresList();
- /**
- * <code>repeated .RegionEventDescriptor.StoreDescriptor stores = 5;</code>
- */
- org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor.StoreDescriptor getStores(int index);
- /**
- * <code>repeated .RegionEventDescriptor.StoreDescriptor stores = 5;</code>
- */
- int getStoresCount();
+ // repeated string store_file = 3;
/**
- * <code>repeated .RegionEventDescriptor.StoreDescriptor stores = 5;</code>
- */
- java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor.StoreDescriptorOrBuilder>
- getStoresOrBuilderList();
- /**
- * <code>repeated .RegionEventDescriptor.StoreDescriptor stores = 5;</code>
+ * <code>repeated string store_file = 3;</code>
+ *
+ * <pre>
+ * relative to store dir
+ * </pre>
*/
- org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor.StoreDescriptorOrBuilder getStoresOrBuilder(
- int index);
-
- // optional .ServerName server = 6;
+ java.util.List<java.lang.String>
+ getStoreFileList();
/**
- * <code>optional .ServerName server = 6;</code>
+ * <code>repeated string store_file = 3;</code>
*
* <pre>
- * Server who opened the region
+ * relative to store dir
* </pre>
*/
- boolean hasServer();
+ int getStoreFileCount();
/**
- * <code>optional .ServerName server = 6;</code>
+ * <code>repeated string store_file = 3;</code>
*
* <pre>
- * Server who opened the region
+ * relative to store dir
* </pre>
*/
- org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName getServer();
+ java.lang.String getStoreFile(int index);
/**
- * <code>optional .ServerName server = 6;</code>
+ * <code>repeated string store_file = 3;</code>
*
* <pre>
- * Server who opened the region
+ * relative to store dir
* </pre>
*/
- org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerNameOrBuilder getServerOrBuilder();
+ com.google.protobuf.ByteString
+ getStoreFileBytes(int index);
}
/**
- * Protobuf type {@code RegionEventDescriptor}
- *
- * <pre>
- **
- * Special WAL entry to hold all related to a region event (open/close).
- * </pre>
+ * Protobuf type {@code StoreDescriptor}
*/
- public static final class RegionEventDescriptor extends
+ public static final class StoreDescriptor extends
com.google.protobuf.GeneratedMessage
- implements RegionEventDescriptorOrBuilder {
- // Use RegionEventDescriptor.newBuilder() to construct.
- private RegionEventDescriptor(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+ implements StoreDescriptorOrBuilder {
+ // Use StoreDescriptor.newBuilder() to construct.
+ private StoreDescriptor(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
super(builder);
this.unknownFields = builder.getUnknownFields();
}
- private RegionEventDescriptor(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+ private StoreDescriptor(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
- private static final RegionEventDescriptor defaultInstance;
- public static RegionEventDescriptor getDefaultInstance() {
+ private static final StoreDescriptor defaultInstance;
+ public static StoreDescriptor getDefaultInstance() {
return defaultInstance;
}
- public RegionEventDescriptor getDefaultInstanceForType() {
+ public StoreDescriptor getDefaultInstanceForType() {
return defaultInstance;
}
@@ -7731,7 +7708,7 @@ public final class WALProtos {
getUnknownFields() {
return this.unknownFields;
}
- private RegionEventDescriptor(
+ private StoreDescriptor(
com.google.protobuf.CodedInputStream input,
com.google.protobuf.ExtensionRegistryLite extensionRegistry)
throws com.google.protobuf.InvalidProtocolBufferException {
@@ -7754,51 +7731,22 @@ public final class WALProtos {
}
break;
}
- case 8: {
- int rawValue = input.readEnum();
- org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor.EventType value = org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor.EventType.valueOf(rawValue);
- if (value == null) {
- unknownFields.mergeVarintField(1, rawValue);
- } else {
- bitField0_ |= 0x00000001;
- eventType_ = value;
- }
+ case 10: {
+ bitField0_ |= 0x00000001;
+ familyName_ = input.readBytes();
break;
}
case 18: {
bitField0_ |= 0x00000002;
- tableName_ = input.readBytes();
+ storeHomeDir_ = input.readBytes();
break;
}
case 26: {
- bitField0_ |= 0x00000004;
- encodedRegionName_ = input.readBytes();
- break;
- }
- case 32: {
- bitField0_ |= 0x00000008;
- logSequenceNumber_ = input.readUInt64();
- break;
- }
- case 42: {
- if (!((mutable_bitField0_ & 0x00000010) == 0x00000010)) {
- stores_ = new java.util.ArrayList<org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor.StoreDescriptor>();
- mutable_bitField0_ |= 0x00000010;
- }
- stores_.add(input.readMessage(org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor.StoreDescriptor.PARSER, extensionRegistry));
- break;
- }
- case 50: {
- org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.Builder subBuilder = null;
- if (((bitField0_ & 0x00000010) == 0x00000010)) {
- subBuilder = server_.toBuilder();
- }
- server_ = input.readMessage(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.PARSER, extensionRegistry);
- if (subBuilder != null) {
- subBuilder.mergeFrom(server_);
- server_ = subBuilder.buildPartial();
+ if (!((mutable_bitField0_ & 0x00000004) == 0x00000004)) {
+ storeFile_ = new com.google.protobuf.LazyStringArrayList();
+ mutable_bitField0_ |= 0x00000004;
}
- bitField0_ |= 0x00000010;
+ storeFile_.add(input.readBytes());
break;
}
}
@@ -7809,8 +7757,8 @@ public final class WALProtos {
throw new com.google.protobuf.InvalidProtocolBufferException(
e.getMessage()).setUnfinishedMessage(this);
} finally {
- if (((mutable_bitField0_ & 0x00000010) == 0x00000010)) {
- stores_ = java.util.Collections.unmodifiableList(stores_);
+ if (((mutable_bitField0_ & 0x00000004) == 0x00000004)) {
+ storeFile_ = new com.google.protobuf.UnmodifiableLazyStringList(storeFile_);
}
this.unknownFields = unknownFields.build();
makeExtensionsImmutable();
@@ -7818,1036 +7766,2238 @@ public final class WALProtos {
}
public static final com.google.protobuf.Descriptors.Descriptor
getDescriptor() {
- return org.apache.hadoop.hbase.protobuf.generated.WALProtos.internal_static_RegionEventDescriptor_descriptor;
+ return org.apache.hadoop.hbase.protobuf.generated.WALProtos.internal_static_StoreDescriptor_descriptor;
}
protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
internalGetFieldAccessorTable() {
- return org.apache.hadoop.hbase.protobuf.generated.WALProtos.internal_static_RegionEventDescriptor_fieldAccessorTable
+ return org.apache.hadoop.hbase.protobuf.generated.WALProtos.internal_static_StoreDescriptor_fieldAccessorTable
.ensureFieldAccessorsInitialized(
- org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor.class, org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor.Builder.class);
+ org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor.class, org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor.Builder.class);
}
- public static com.google.protobuf.Parser<RegionEventDescriptor> PARSER =
- new com.google.protobuf.AbstractParser<RegionEventDescriptor>() {
- public RegionEventDescriptor parsePartialFrom(
+ public static com.google.protobuf.Parser<StoreDescriptor> PARSER =
+ new com.google.protobuf.AbstractParser<StoreDescriptor>() {
+ public StoreDescriptor parsePartialFrom(
com.google.protobuf.CodedInputStream input,
com.google.protobuf.ExtensionRegistryLite extensionRegistry)
throws com.google.protobuf.InvalidProtocolBufferException {
- return new RegionEventDescriptor(input, extensionRegistry);
+ return new StoreDescriptor(input, extensionRegistry);
}
};
@java.lang.Override
- public com.google.protobuf.Parser<RegionEventDescriptor> getParserForType() {
+ public com.google.protobuf.Parser<StoreDescriptor> getParserForType() {
return PARSER;
}
+ private int bitField0_;
+ // required bytes family_name = 1;
+ public static final int FAMILY_NAME_FIELD_NUMBER = 1;
+ private com.google.protobuf.ByteString familyName_;
/**
- * Protobuf enum {@code RegionEventDescriptor.EventType}
+ * <code>required bytes family_name = 1;</code>
*/
- public enum EventType
- implements com.google.protobuf.ProtocolMessageEnum {
- /**
- * <code>REGION_OPEN = 0;</code>
- */
- REGION_OPEN(0, 0),
- /**
- * <code>REGION_CLOSE = 1;</code>
- */
- REGION_CLOSE(1, 1),
- ;
-
- /**
- * <code>REGION_OPEN = 0;</code>
- */
- public static final int REGION_OPEN_VALUE = 0;
- /**
- * <code>REGION_CLOSE = 1;</code>
- */
- public static final int REGION_CLOSE_VALUE = 1;
-
-
- public final int getNumber() { return value; }
+ public boolean hasFamilyName() {
+ return ((bitField0_ & 0x00000001) == 0x00000001);
+ }
+ /**
+ * <code>required bytes family_name = 1;</code>
+ */
+ public com.google.protobuf.ByteString getFamilyName() {
+ return familyName_;
+ }
- public static EventType valueOf(int value) {
- switch (value) {
- case 0: return REGION_OPEN;
- case 1: return REGION_CLOSE;
- default: return null;
+ // required string store_home_dir = 2;
+ public static final int STORE_HOME_DIR_FIELD_NUMBER = 2;
+ private java.lang.Object storeHomeDir_;
+ /**
+ * <code>required string store_home_dir = 2;</code>
+ *
+ * <pre>
+ *relative to region dir
+ * </pre>
+ */
+ public boolean hasStoreHomeDir() {
+ return ((bitField0_ & 0x00000002) == 0x00000002);
+ }
+ /**
+ * <code>required string store_home_dir = 2;</code>
+ *
+ * <pre>
+ *relative to region dir
+ * </pre>
+ */
+ public java.lang.String getStoreHomeDir() {
+ java.lang.Object ref = storeHomeDir_;
+ if (ref instanceof java.lang.String) {
+ return (java.lang.String) ref;
+ } else {
+ com.google.protobuf.ByteString bs =
+ (com.google.protobuf.ByteString) ref;
+ java.lang.String s = bs.toStringUtf8();
+ if (bs.isValidUtf8()) {
+ storeHomeDir_ = s;
}
+ return s;
}
-
- public static com.google.protobuf.Internal.EnumLiteMap<EventType>
- internalGetValueMap() {
- return internalValueMap;
- }
- private static com.google.protobuf.Internal.EnumLiteMap<EventType>
- internalValueMap =
- new com.google.protobuf.Internal.EnumLiteMap<EventType>() {
- public EventType findValueByNumber(int number) {
- return EventType.valueOf(number);
- }
- };
-
- public final com.google.protobuf.Descriptors.EnumValueDescriptor
- getValueDescriptor() {
- return getDescriptor().getValues().get(index);
- }
- public final com.google.protobuf.Descriptors.EnumDescriptor
- getDescriptorForType() {
- return getDescriptor();
- }
- public static final com.google.protobuf.Descriptors.EnumDescriptor
- getDescriptor() {
- return org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor.getDescriptor().getEnumTypes().get(0);
+ }
+ /**
+ * <code>required string store_home_dir = 2;</code>
+ *
+ * <pre>
+ *relative to region dir
+ * </pre>
+ */
+ public com.google.protobuf.ByteString
+ getStoreHomeDirBytes() {
+ java.lang.Object ref = storeHomeDir_;
+ if (ref instanceof java.lang.String) {
+ com.google.protobuf.ByteString b =
+ com.google.protobuf.ByteString.copyFromUtf8(
+ (java.lang.String) ref);
+ storeHomeDir_ = b;
+ return b;
+ } else {
+ return (com.google.protobuf.ByteString) ref;
}
+ }
- private static final EventType[] VALUES = values();
-
- public static EventType valueOf(
- com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
- if (desc.getType() != getDescriptor()) {
- throw new java.lang.IllegalArgumentException(
- "EnumValueDescriptor is not for this type.");
- }
- return VALUES[desc.getIndex()];
- }
-
- private final int index;
- private final int value;
-
- private EventType(int index, int value) {
- this.index = index;
- this.value = value;
- }
-
- // @@protoc_insertion_point(enum_scope:RegionEventDescriptor.EventType)
+ // repeated string store_file = 3;
+ public static final int STORE_FILE_FIELD_NUMBER = 3;
+ private com.google.protobuf.LazyStringList storeFile_;
+ /**
+ * <code>repeated string store_file = 3;</code>
+ *
+ * <pre>
+ * relative to store dir
+ * </pre>
+ */
+ public java.util.List<java.lang.String>
+ getStoreFileList() {
+ return storeFile_;
}
-
- public interface StoreDescriptorOrBuilder
- extends com.google.protobuf.MessageOrBuilder {
-
- // required bytes family_name = 1;
- /**
- * <code>required bytes family_name = 1;</code>
- */
- boolean hasFamilyName();
- /**
- * <code>required bytes family_name = 1;</code>
- */
- com.google.protobuf.ByteString getFamilyName();
-
- // required string store_home_dir = 2;
- /**
- * <code>required string store_home_dir = 2;</code>
- *
- * <pre>
- *relative to region dir
- * </pre>
- */
- boolean hasStoreHomeDir();
- /**
- * <code>required string store_home_dir = 2;</code>
- *
- * <pre>
- *relative to region dir
- * </pre>
- */
- java.lang.String getStoreHomeDir();
- /**
- * <code>required string store_home_dir = 2;</code>
- *
- * <pre>
- *relative to region dir
- * </pre>
- */
- com.google.protobuf.ByteString
- getStoreHomeDirBytes();
-
- // repeated string store_file = 3;
- /**
- * <code>repeated string store_file = 3;</code>
- *
- * <pre>
- * relative to store dir
- * </pre>
- */
- java.util.List<java.lang.String>
- getStoreFileList();
- /**
- * <code>repeated string store_file = 3;</code>
- *
- * <pre>
- * relative to store dir
- * </pre>
- */
- int getStoreFileCount();
- /**
- * <code>repeated string store_file = 3;</code>
- *
- * <pre>
- * relative to store dir
- * </pre>
- */
- java.lang.String getStoreFile(int index);
- /**
- * <code>repeated string store_file = 3;</code>
- *
- * <pre>
- * relative to store dir
- * </pre>
- */
- com.google.protobuf.ByteString
- getStoreFileBytes(int index);
+ /**
+ * <code>repeated string store_file = 3;</code>
+ *
+ * <pre>
+ * relative to store dir
+ * </pre>
+ */
+ public int getStoreFileCount() {
+ return storeFile_.size();
+ }
+ /**
+ * <code>repeated string store_file = 3;</code>
+ *
+ * <pre>
+ * relative to store dir
+ * </pre>
+ */
+ public java.lang.String getStoreFile(int index) {
+ return storeFile_.get(index);
}
/**
- * Protobuf type {@code RegionEventDescriptor.StoreDescriptor}
+ * <code>repeated string store_file = 3;</code>
+ *
+ * <pre>
+ * relative to store dir
+ * </pre>
*/
- public static final class StoreDescriptor extends
- com.google.protobuf.GeneratedMessage
- implements StoreDescriptorOrBuilder {
- // Use StoreDescriptor.newBuilder() to construct.
- private StoreDescriptor(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
- super(builder);
- this.unknownFields = builder.getUnknownFields();
- }
- private StoreDescriptor(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+ public com.google.protobuf.ByteString
+ getStoreFileBytes(int index) {
+ return storeFile_.getByteString(index);
+ }
- private static final StoreDescriptor defaultInstance;
- public static StoreDescriptor getDefaultInstance() {
- return defaultInstance;
- }
+ private void initFields() {
+ familyName_ = com.google.protobuf.ByteString.EMPTY;
+ storeHomeDir_ = "";
+ storeFile_ = com.google.protobuf.LazyStringArrayList.EMPTY;
+ }
+ private byte memoizedIsInitialized = -1;
+ public final boolean isInitialized() {
+ byte isInitialized = memoizedIsInitialized;
+ if (isInitialized != -1) return isInitialized == 1;
- public StoreDescriptor getDefaultInstanceForType() {
- return defaultInstance;
+ if (!hasFamilyName()) {
+ memoizedIsInitialized = 0;
+ return false;
+ }
+ if (!hasStoreHomeDir()) {
+ memoizedIsInitialized = 0;
+ return false;
}
+ memoizedIsInitialized = 1;
+ return true;
+ }
- private final com.google.protobuf.UnknownFieldSet unknownFields;
- @java.lang.Override
- public final com.google.protobuf.UnknownFieldSet
- getUnknownFields() {
- return this.unknownFields;
+ public void writeTo(com.google.protobuf.CodedOutputStream output)
+ throws java.io.IOException {
+ getSerializedSize();
+ if (((bitField0_ & 0x00000001) == 0x00000001)) {
+ output.writeBytes(1, familyName_);
}
- private StoreDescriptor(
- com.google.protobuf.CodedInputStream input,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws com.google.protobuf.InvalidProtocolBufferException {
- initFields();
- int mutable_bitField0_ = 0;
- com.google.protobuf.UnknownFieldSet.Builder unknownFields =
- com.google.protobuf.UnknownFieldSet.newBuilder();
- try {
- boolean done = false;
- while (!done) {
- int tag = input.readTag();
- switch (tag) {
- case 0:
- done = true;
- break;
- default: {
- if (!parseUnknownField(input, unknownFields,
- extensionRegistry, tag)) {
- done = true;
- }
- break;
- }
- case 10: {
- bitField0_ |= 0x00000001;
- familyName_ = input.readBytes();
- break;
- }
- case 18: {
- bitField0_ |= 0x00000002;
- storeHomeDir_ = input.readBytes();
- break;
- }
- case 26: {
- if (!((mutable_bitField0_ & 0x00000004) == 0x00000004)) {
- storeFile_ = new com.google.protobuf.LazyStringArrayList();
- mutable_bitField0_ |= 0x00000004;
- }
- storeFile_.add(input.readBytes());
- break;
- }
- }
- }
- } catch (com.google.protobuf.InvalidProtocolBufferException e) {
- throw e.setUnfinishedMessage(this);
- } catch (java.io.IOException e) {
- throw new com.google.protobuf.InvalidProtocolBufferException(
- e.getMessage()).setUnfinishedMessage(this);
- } finally {
- if (((mutable_bitField0_ & 0x00000004) == 0x00000004)) {
- storeFile_ = new com.google.protobuf.UnmodifiableLazyStringList(storeFile_);
- }
- this.unknownFields = unknownFields.build();
- makeExtensionsImmutable();
- }
+ if (((bitField0_ & 0x00000002) == 0x00000002)) {
+ output.writeBytes(2, getStoreHomeDirBytes());
}
- public static final com.google.protobuf.Descriptors.Descriptor
- getDescriptor() {
- return org.apache.hadoop.hbase.protobuf.generated.WALProtos.internal_static_RegionEventDescriptor_StoreDescriptor_descriptor;
+ for (int i = 0; i < storeFile_.size(); i++) {
+ output.writeBytes(3, storeFile_.getByteString(i));
}
+ getUnknownFields().writeTo(output);
+ }
- protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
- internalGetFieldAccessorTable() {
- return org.apache.hadoop.hbase.protobuf.generated.WALProtos.internal_static_RegionEventDescriptor_StoreDescriptor_fieldAccessorTable
- .ensureFieldAccessorsInitialized(
- org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor.StoreDescriptor.class, org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor.StoreDescriptor.Builder.class);
- }
+ private int memoizedSerializedSize = -1;
+ public int getSerializedSize() {
+ int size = memoizedSerializedSize;
+ if (size != -1) return size;
- public static com.google.protobuf.Parser<StoreDescriptor> PARSER =
- new com.google.protobuf.AbstractParser<StoreDescriptor>() {
- public StoreDescriptor parsePartialFrom(
- com.google.protobuf.CodedInputStream input,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws com.google.protobuf.InvalidProtocolBufferException {
- return new StoreDescriptor(input, extensionRegistry);
+ size = 0;
+ if (((bitField0_ & 0x00000001) == 0x00000001)) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeBytesSize(1, familyName_);
+ }
+ if (((bitField0_ & 0x00000002) == 0x00000002)) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeBytesSize(2, getStoreHomeDirBytes());
+ }
+ {
+ int dataSize = 0;
+ for (int i = 0; i < storeFile_.size(); i++) {
+ dataSize += com.google.protobuf.CodedOutputStream
+ .computeBytesSizeNoTag(storeFile_.getByteString(i));
}
- };
-
- @java.lang.Override
- public com.google.protobuf.Parser<StoreDescriptor> getParserForType() {
- return PARSER;
+ size += dataSize;
+ size += 1 * getStoreFileList().size();
}
+ size += getUnknownFields().getSerializedSize();
+ memoizedSerializedSize = size;
+ return size;
+ }
- private int bitField0_;
- // required bytes family_name = 1;
- public static final int FAMILY_NAME_FIELD_NUMBER = 1;
- private com.google.protobuf.ByteString familyName_;
- /**
- * <code>required bytes family_name = 1;</code>
- */
- public boolean hasFamilyName() {
- return ((bitField0_ & 0x00000001) == 0x00000001);
+ private static final long serialVersionUID = 0L;
+ @java.lang.Override
+ protected java.lang.Object writeReplace()
+ throws java.io.ObjectStreamException {
+ return super.writeReplace();
+ }
+
+ @java.lang.Override
+ public boolean equals(final java.lang.Object obj) {
+ if (obj == this) {
+ return true;
}
- /**
- * <code>required bytes family_name = 1;</code>
- */
- public com.google.protobuf.ByteString getFamilyName() {
- return familyName_;
+ if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor)) {
+ return super.equals(obj);
}
+ org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor other = (org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor) obj;
- // required string store_home_dir = 2;
- public static final int STORE_HOME_DIR_FIELD_NUMBER = 2;
- private java.lang.Object storeHomeDir_;
- /**
- * <code>required string store_home_dir = 2;</code>
- *
- * <pre>
- *relative to region dir
- * </pre>
- */
- public boolean hasStoreHomeDir() {
- return ((bitField0_ & 0x00000002) == 0x00000002);
+ boolean result = true;
+ result = result && (hasFamilyName() == other.hasFamilyName());
+ if (hasFamilyName()) {
+ result = result && getFamilyName()
+ .equals(other.getFamilyName());
}
- /**
- * <code>required string store_home_dir = 2;</code>
- *
- * <pre>
- *relative to region dir
- * </pre>
- */
- public java.lang.String getStoreHomeDir() {
- java.lang.Object ref = storeHomeDir_;
- if (ref instanceof java.lang.String) {
- return (java.lang.String) ref;
- } else {
- com.google.protobuf.ByteString bs =
- (com.google.protobuf.ByteString) ref;
- java.lang.String s = bs.toStringUtf8();
- if (bs.isValidUtf8()) {
- storeHomeDir_ = s;
- }
- return s;
- }
+ result = result && (hasStoreHomeDir() == other.hasStoreHomeDir());
+ if (hasStoreHomeDir()) {
+ result = result && getStoreHomeDir()
+ .equals(other.getStoreHomeDir());
}
- /**
- * <code>required string store_home_dir = 2;</code>
- *
- * <pre>
- *relative to region dir
- * </pre>
- */
- public com.google.protobuf.ByteString
- getStoreHomeDirBytes() {
- java.lang.Object ref = storeHomeDir_;
- if (ref instanceof java.lang.String) {
- com.google.protobuf.ByteString b =
- com.google.protobuf.ByteString.copyFromUtf8(
- (java.lang.String) ref);
- storeHomeDir_ = b;
- return b;
- } else {
- return (com.google.protobuf.ByteString) ref;
- }
+ result = result && getStoreFileList()
+ .equals(other.getStoreFileList());
+ result = result &&
+ getUnknownFields().equals(other.getUnknownFields());
+ return result;
+ }
+
+ private int memoizedHashCode = 0;
+ @java.lang.Override
+ public int hashCode() {
+ if (memoizedHashCode != 0) {
+ return memoizedHashCode;
+ }
+ int hash = 41;
+ hash = (19 * hash) + getDescriptorForType().hashCode();
+ if (hasFamilyName()) {
+ hash = (37 * hash) + FAMILY_NAME_FIELD_NUMBER;
+ hash = (53 * hash) + getFamilyName().hashCode();
+ }
+ if (hasStoreHomeDir()) {
+ hash = (37 * hash) + STORE_HOME_DIR_FIELD_NUMBER;
+ hash = (53 * hash) + getStoreHomeDir().hashCode();
}
+ if (getStoreFileCount() > 0) {
+ hash = (37 * hash) + STORE_FILE_FIELD_NUMBER;
+ hash = (53 * hash) + getStoreFileList().hashCode();
+ }
+ hash = (29 * hash) + getUnknownFields().hashCode();
+ memoizedHashCode = hash;
+ return hash;
+ }
- // repeated string store_file = 3;
- public static final int STORE_FILE_FIELD_NUMBER = 3;
- private com.google.protobuf.LazyStringList storeFile_;
- /**
- * <code>repeated string store_file = 3;</code>
- *
- * <pre>
- * relative to store dir
- * </pre>
- */
- public java.util.List<java.lang.String>
- getStoreFileList() {
- return storeFile_;
+ public static org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor parseFrom(
+ com.google.protobuf.ByteString data)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return PARSER.parseFrom(data);
+ }
+ public static org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor parseFrom(
+ com.google.protobuf.ByteString data,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return PARSER.parseFrom(data, extensionRegistry);
+ }
+ public static org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor parseFrom(byte[] data)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return PARSER.parseFrom(data);
+ }
+ public static org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor parseFrom(
+ byte[] data,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return PARSER.parseFrom(data, extensionRegistry);
+ }
+ public static org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor parseFrom(java.io.InputStream input)
+ throws java.io.IOException {
+ return PARSER.parseFrom(input);
+ }
+ public static org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor parseFrom(
+ java.io.InputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ return PARSER.parseFrom(input, extensionRegistry);
+ }
+ public static org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor parseDelimitedFrom(java.io.InputStream input)
+ throws java.io.IOException {
+ return PARSER.parseDelimitedFrom(input);
+ }
+ public static org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor parseDelimitedFrom(
+ java.io.InputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ return PARSER.parseDelimitedFrom(input, extensionRegistry);
+ }
+ public static org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor parseFrom(
+ com.google.protobuf.CodedInputStream input)
+ throws java.io.IOException {
+ return PARSER.parseFrom(input);
+ }
+ public static org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor parseFrom(
+ com.google.protobuf.CodedInputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ return PARSER.parseFrom(input, extensionRegistry);
+ }
+
+ public static Builder newBuilder() { return Builder.create(); }
+ public Builder newBuilderForType() { return newBuilder(); }
+ public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor prototype) {
+ return newBuilder().mergeFrom(prototype);
+ }
+ public Builder toBuilder() { return newBuilder(this); }
+
+ @java.lang.Override
+ protected Builder newBuilderForType(
+ com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+ Builder builder = new Builder(parent);
+ return builder;
+ }
+ /**
+ * Protobuf type {@code StoreDescriptor}
+ */
+ public static final class Builder extends
+ com.google.protobuf.GeneratedMessage.Builder<Builder>
+ implements org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptorOrBuilder {
+ public static final com.google.protobuf.Descriptors.Descriptor
+ getDescriptor() {
+ return org.apache.hadoop.hbase.protobuf.generated.WALProtos.internal_static_StoreDescriptor_descriptor;
}
- /**
- * <code>repeated string store_file = 3;</code>
- *
- * <pre>
- * relative to store dir
- * </pre>
- */
- public int getStoreFileCount() {
- return storeFile_.size();
+
+ protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+ internalGetFieldAccessorTable() {
+ return org.apache.hadoop.hbase.protobuf.generated.WALProtos.internal_static_StoreDescriptor_fieldAccessorTable
+ .ensureFieldAccessorsInitialized(
+ org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor.class, org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor.Builder.class);
}
- /**
- * <code>repeated string store_file = 3;</code>
- *
- * <pre>
- * relative to store dir
- * </pre>
- */
- public java.lang.String getStoreFile(int index) {
- return storeFile_.get(index);
+
+ // Construct using org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor.newBuilder()
+ private Builder() {
+ maybeForceBuilderInitialization();
}
- /**
- * <code>repeated string store_file = 3;</code>
- *
- * <pre>
- * relative to store dir
- * </pre>
- */
- public com.google.protobuf.ByteString
- getStoreFileBytes(int index) {
- return storeFile_.getByteString(index);
+
+ private Builder(
+ com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+ super(parent);
+ maybeForceBuilderInitialization();
+ }
+ private void maybeForceBuilderInitialization() {
+ if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+ }
+ }
+ private static Builder create() {
+ return new Builder();
}
- private void initFields() {
+ public Builder clear() {
+ super.clear();
familyName_ = com.google.protobuf.ByteString.EMPTY;
+ bitField0_ = (bitField0_ & ~0x00000001);
storeHomeDir_ = "";
+ bitField0_ = (bitField0_ & ~0x00000002);
storeFile_ = com.google.protobuf.LazyStringArrayList.EMPTY;
+ bitField0_ = (bitField0_ & ~0x00000004);
+ return this;
}
- private byte memoizedIsInitialized = -1;
- public final boolean isInitialized() {
- byte isInitialized = memoizedIsInitialized;
- if (isInitialized != -1) return isInitialized == 1;
- if (!hasFamilyName()) {
- memoizedIsInitialized = 0;
- return false;
- }
- if (!hasStoreHomeDir()) {
- memoizedIsInitialized = 0;
- return false;
+ public Builder clone() {
+ return create().mergeFrom(buildPartial());
+ }
+
+ public com.google.protobuf.Descriptors.Descriptor
+ getDescriptorForType() {
+ return org.apache.hadoop.hbase.protobuf.generated.WALProtos.internal_static_StoreDescriptor_descriptor;
+ }
+
+ public org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor getDefaultInstanceForType() {
+ return org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor.getDefaultInstance();
+ }
+
+ public org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor build() {
+ org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor result = buildPartial();
+ if (!result.isInitialized()) {
+ throw newUninitializedMessageException(result);
}
- memoizedIsInitialized = 1;
- return true;
+ return result;
}
- public void writeTo(com.google.protobuf.CodedOutputStream output)
- throws java.io.IOException {
- getSerializedSize();
- if (((bitField0_ & 0x00000001) == 0x00000001)) {
- output.writeBytes(1, familyName_);
+ public org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor buildPartial() {
+ org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor result = new org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor(this);
+ int from_bitField0_ = bitField0_;
+ int to_bitField0_ = 0;
+ if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+ to_bitField0_ |= 0x00000001;
}
- if (((bitField0_ & 0x00000002) == 0x00000002)) {
- output.writeBytes(2, getStoreHomeDirBytes());
+ result.familyName_ = familyName_;
+ if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+ to_bitField0_ |= 0x00000002;
}
- for (int i = 0; i < storeFile_.size(); i++) {
- output.writeBytes(3, storeFile_.getByteString(i));
+ result.storeHomeDir_ = storeHomeDir_;
+ if (((bitField0_ & 0x00000004) == 0x00000004)) {
+ storeFile_ = new com.google.protobuf.UnmodifiableLazyStringList(
+ storeFile_);
+ bitField0_ = (bitField0_ & ~0x00000004);
}
- getUnknownFields().writeTo(output);
+ result.storeFile_ = storeFile_;
+ result.bitField0_ = to_bitField0_;
+ onBuilt();
+ return result;
}
- private int memoizedSerializedSize = -1;
- public int getSerializedSize() {
- int size = memoizedSerializedSize;
- if (size != -1) return size;
-
- size = 0;
- if (((bitField0_ & 0x00000001) == 0x00000001)) {
- size += com.google.protobuf.CodedOutputStream
- .computeBytesSize(1, familyName_);
- }
- if (((bitField0_ & 0x00000002) == 0x00000002)) {
- size += com.google.protobuf.CodedOutputStream
- .computeBytesSize(2, getStoreHomeDirBytes());
- }
- {
- int dataSize = 0;
- for (int i = 0; i < storeFile_.size(); i++) {
- dataSize += com.google.protobuf.CodedOutputStream
- .computeBytesSizeNoTag(storeFile_.getByteString(i));
- }
- size += dataSize;
- size += 1 * getStoreFileList().size();
+ public Builder mergeFrom(com.google.protobuf.Message other) {
+ if (other instanceof org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor) {
+ return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor)other);
+ } else {
+ super.mergeFrom(other);
+ return this;
}
- size += getUnknownFields().getSerializedSize();
- memoizedSerializedSize = size;
- return size;
- }
-
- private static final long serialVersionUID = 0L;
- @java.lang.Override
- protected java.lang.Object writeReplace()
- throws java.io.ObjectStreamException {
- return super.writeReplace();
}
- @java.lang.Override
- public boolean equals(final java.lang.Object obj) {
- if (obj == this) {
- return true;
- }
- if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor.StoreDescriptor)) {
- return super.equals(obj);
+ public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor other) {
+ if (other == org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor.getDefaultInstance()) return this;
+ if (other.hasFamilyName()) {
+ setFamilyName(other.getFamilyName());
}
- org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor.StoreDescriptor other = (org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor.StoreDescriptor) obj;
-
- boolean result = true;
- result = result && (hasFamilyName() == other.hasFamilyName());
- if (hasFamilyName()) {
- result = result && getFamilyName()
- .equals(other.getFamilyName());
+ if (other.hasStoreHomeDir()) {
+ bitField0_ |= 0x00000002;
+ storeHomeDir_ = other.storeHomeDir_;
+ onChanged();
}
- result = result && (hasStoreHomeDir() == other.hasStoreHomeDir());
- if (hasStoreHomeDir()) {
- result = result && getStoreHomeDir()
- .equals(other.getStoreHomeDir());
+ if (!other.storeFile_.isEmpty()) {
+ if (storeFile_.isEmpty()) {
+ storeFile_ = other.storeFile_;
+ bitField0_ = (bitField0_ & ~0x00000004);
+ } else {
+ ensureStoreFileIsMutable();
+ storeFile_.addAll(other.storeFile_);
+ }
+ onChanged();
}
- result = result && getStoreFileList()
- .equals(other.getStoreFileList());
- result = result &&
- getUnknownFields().equals(other.getUnknownFields());
- return result;
+ this.mergeUnknownFields(other.getUnknownFields());
+ return this;
}
- private int memoizedHashCode = 0;
- @java.lang.Override
- public int hashCode() {
- if (memoizedHashCode != 0) {
- return memoizedHashCode;
- }
- int hash = 41;
- hash = (19 * hash) + getDescriptorForType().hashCode();
- if (hasFamilyName()) {
- hash = (37 * hash) + FAMILY_NAME_FIELD_NUMBER;
- hash = (53 * hash) + getFamilyName().hashCode();
- }
- if (hasStoreHomeDir()) {
- hash = (37 * hash) + STORE_HOME_DIR_FIELD_NUMBER;
- hash = (53 * hash) + getStoreHomeDir().hashCode();
+ public final boolean isInitialized() {
+ if (!hasFamilyName()) {
+
+ return false;
}
- if (getStoreFileCount() > 0) {
- hash = (37 * hash) + STORE_FILE_FIELD_NUMBER;
- hash = (53 * hash) + getStoreFileList().hashCode();
+ if (!hasStoreHomeDir()) {
+
+ return false;
}
- hash = (29 * hash) + getUnknownFields().hashCode();
- memoizedHashCode = hash;
- return hash;
+ return true;
}
- public static org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor.StoreDescriptor parseFrom(
- com.google.protobuf.ByteString data)
- throws com.google.protobuf.InvalidProtocolBufferException {
- return PARSER.parseFrom(data);
- }
- public static org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor.StoreDescriptor parseFrom(
- com.google.protobuf.ByteString data,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws com.google.protobuf.InvalidProtocolBufferException {
- return PARSER.parseFrom(data, extensionRegistry);
- }
- public static org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor.StoreDescriptor parseFrom(byte[] data)
- throws com.google.protobuf.InvalidProtocolBufferException {
- return PARSER.parseFrom(data);
- }
- public static org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor.StoreDescriptor parseFrom(
- byte[] data,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws com.google.protobuf.InvalidProtocolBufferException {
- return PARSER.parseFrom(data, extensionRegistry);
- }
- public static org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor.StoreDescriptor parseFrom(java.io.InputStream input)
- throws java.io.IOException {
- return PARSER.parseFrom(input);
- }
- public static org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor.StoreDescriptor parseFrom(
- java.io.InputStream input,
+ public Builder mergeFrom(
+ com.google.protobuf.CodedInputStream input,
com.google.protobuf.ExtensionRegistryLite extensionRegistry)
throws java.io.IOException {
- return PARSER.parseFrom(input, extensionRegistry);
- }
- public static org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor.StoreDescriptor parseDelimitedFrom(java.io.InputStream input)
- throws java.io.IOException {
- return PARSER.parseDelimitedFrom(input);
+ org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor parsedMessage = null;
+ try {
+ parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+ } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+ parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor) e.getUnfinishedMessage();
+ throw e;
+ } finally {
+ if (parsedMessage != null) {
+ mergeFrom(parsedMessage);
+ }
+ }
+ return this;
}
- public static org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor.StoreDescriptor parseDelimitedFrom(
- java.io.InputStream input,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws java.io.IOException {
- return PARSER.parseDelimitedFrom(input, extensionRegistry);
+ private int bitField0_;
+
+ // required bytes family_name = 1;
+ private com.google.protobuf.ByteString familyName_ = com.google.protobuf.ByteString.EMPTY;
+ /**
+ * <code>required bytes family_name = 1;</code>
+ */
+ public boolean hasFamilyName() {
+ return ((bitField0_ & 0x00000001) == 0x00000001);
}
- public static org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor.StoreDescriptor parseFrom(
- com.google.protobuf.CodedInputStream input)
- throws java.io.IOException {
- return PARSER.parseFrom(input);
+ /**
+ * <code>required bytes family_name = 1;</code>
+ */
+ public com.google.protobuf.ByteString getFamilyName() {
+ return familyName_;
}
- public static org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor.StoreDescriptor parseFrom(
- com.google.protobuf.CodedInputStream input,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws java.io.IOException {
- return PARSER.parseFrom(input, extensionRegistry);
+ /**
+ * <code>required bytes family_name = 1;</code>
+ */
+ public Builder setFamilyName(com.google.protobuf.ByteString value) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ bitField0_ |= 0x00000001;
+ familyName_ = value;
+ onChanged();
+ return this;
}
-
- public static Builder newBuilder() { return Builder.create(); }
- public Builder newBuilderForType() { return newBuilder(); }
- public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor.StoreDescriptor prototype) {
- return newBuilder().mergeFrom(prototype);
+ /**
+ * <code>required bytes family_name = 1;</code>
+ */
+ public Builder clearFamilyName() {
+ bitField0_ = (bitField0_ & ~0x00000001);
+ familyName_ = getDefaultInstance().getFamilyName();
+ onChanged();
+ return this;
}
- public Builder toBuilder() { return newBuilder(this); }
- @java.lang.Override
- protected Builder newBuilderForType(
- com.google.protobuf.GeneratedMessage.BuilderParent parent) {
- Builder builder = new Builder(parent);
- return builder;
+ // required string store_home_dir = 2;
+ private java.lang.Object storeHomeDir_ = "";
+ /**
+ * <code>required string store_home_dir = 2;</code>
+ *
+ * <pre>
+ *relative to region dir
+ * </pre>
+ */
+ public boolean hasStoreHomeDir() {
+ return ((bitField0_ & 0x00000002) == 0x00000002);
}
/**
- * Protobuf type {@code RegionEventDescriptor.StoreDescriptor}
+ * <code>required string store_home_dir = 2;</code>
+ *
+ * <pre>
+ *relative to region dir
+ * </pre>
*/
- public static final class Builder extends
- com.google.protobuf.GeneratedMessage.Builder<Builder>
- implements org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor.StoreDescriptorOrBuilder {
- public static final com.google.protobuf.Descriptors.Descriptor
- getDescriptor() {
- return org.apache.hadoop.hbase.protobuf.generated.WALProtos.internal_static_RegionEventDescriptor_StoreDescriptor_descriptor;
- }
-
- protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
- internalGetFieldAccessorTable() {
- return org.apache.hadoop.hbase.protobuf.generated.WALProtos.internal_static_RegionEventDescriptor_StoreDescriptor_fieldAccessorTable
- .ensureFieldAccessorsInitialized(
- org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor.StoreDescriptor.class, org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor.StoreDescriptor.Builder.class);
- }
-
- // Construct using org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor.StoreDescriptor.newBuilder()
- private Builder() {
- maybeForceBuilderInitialization();
- }
-
- private Builder(
- com.google.protobuf.GeneratedMessage.BuilderParent parent) {
- super(parent);
- maybeForceBuilderInitialization();
- }
- private void maybeForceBuilderInitialization() {
- if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
- }
+ public java.lang.String getStoreHomeDir() {
+ java.lang.Object ref = storeHomeDir_;
+ if (!(ref instanceof java.lang.String)) {
+ java.lang.String s = ((com.google.protobuf.ByteString) ref)
+ .toStringUtf8();
+ storeHomeDir_ = s;
+ return s;
+ } else {
+ return (java.lang.String) ref;
}
- private static Builder create() {
- return new Builder();
- }
-
- public Builder clear() {
- super.clear();
- familyName_ = com.google.protobuf.ByteString.EMPTY;
- bitField0_ = (bitField0_ & ~0x00000001);
- storeHomeDir_ = "";
- bitField0_ = (bitField0_ & ~0x00000002);
- storeFile_ = com.google.protobuf.LazyStringArrayList.EMPTY;
- bitField0_ = (bitField0_ & ~0x00000004);
- return this;
- }
-
- public Builder clone() {
- return create().mergeFrom(buildPartial());
+ }
+ /**
+ * <code>required string store_home_dir = 2;</code>
+ *
+ * <pre>
+ *relative to region dir
+ * </pre>
+ */
+ public com.google.protobuf.ByteString
+ getStoreHomeDirBytes() {
+ java.lang.Object ref = storeHomeDir_;
+ if (ref instanceof String) {
+ com.google.protobuf.ByteString b =
+ com.google.protobuf.ByteString.copyFromUtf8(
+ (java.lang.String) ref);
+ storeHomeDir_ = b;
+ return b;
+ } else {
+ return (com.google.protobuf.ByteString) ref;
}
+ }
+ /**
+ * <code>required string store_home_dir = 2;</code>
+ *
+ * <pre>
+ *relative to region dir
+ * </pre>
+ */
+ public Builder setStoreHomeDir(
+ java.lang.String value) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ bitField0_ |= 0x00000002;
+ storeHomeDir_ = value;
+ onChanged();
+ return this;
+ }
+ /**
+ * <code>required string store_home_dir = 2;</code>
+ *
+ * <pre>
+ *relative to region dir
+ * </pre>
+ */
+ public Builder clearStoreHomeDir() {
+ bitField0_ = (bitField0_ & ~0x00000002);
+ storeHomeDir_ = getDefaultInstance().getStoreHomeDir();
+ onChanged();
+ return this;
+ }
+ /**
+ * <code>required string store_home_dir = 2;</code>
+ *
+ * <pre>
+ *relative to region dir
+ * </pre>
+ */
+ public Builder setStoreHomeDirBytes(
+ com.google.protobuf.ByteString value) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ bitField0_ |= 0x00000002;
+ storeHomeDir_ = value;
+ onChanged();
+ return this;
+ }
- public com.google.protobuf.Descriptors.Descriptor
- getDescriptorForType() {
- return org.apache.hadoop.hbase.protobuf.generated.WALProtos.internal_static_RegionEventDescriptor_StoreDescriptor_descriptor;
- }
+ // repeated string store_file = 3;
+ private com.google.protobuf.LazyStringList storeFile_ = com.google.protobuf.LazyStringArrayList.EMPTY;
+ private void ensureStoreFileIsMutable() {
+ if (!((bitField0_ & 0x00000004) == 0x00000004)) {
+ storeFile_ = new com.google.protobuf.LazyStringArrayList(storeFile_);
+ bitField0_ |= 0x00000004;
+ }
+ }
+ /**
+ * <code>repeated string store_file = 3;</code>
+ *
+ * <pre>
+ * relative to store dir
+ * </pre>
+ */
+ public java.util.List<java.lang.String>
+ getStoreFileList() {
+ return java.util.Collections.unmodifiableList(storeFile_);
+ }
+ /**
+ * <code>repeated string store_file = 3;</code>
+ *
+ * <pre>
+ * relative to store dir
+ * </pre>
+ */
+ public int getStoreFileCount() {
+ return storeFile_.size();
+ }
+ /**
+ * <code>repeated string store_file = 3;</code>
+ *
+ * <pre>
+ * relative to store dir
+ * </pre>
+ */
+ public java.lang.String getStoreFile(int index) {
+ return storeFile_.get(index);
+ }
+ /**
+ * <code>repeated string store_file = 3;</code>
+ *
+ * <pre>
+ * relative to store dir
+ * </pre>
+ */
+ public com.google.protobuf.ByteString
+ getStoreFileBytes(int index) {
+ return storeFile_.getByteString(index);
+ }
+ /**
+ * <code>repeated string store_file = 3;</code>
+ *
+ * <pre>
+ * relative to store dir
+ * </pre>
+ */
+ public Builder setStoreFile(
+ int index, java.lang.String value) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ ensureStoreFileIsMutable();
+ storeFile_.set(index, value);
+ onChanged();
+ return this;
+ }
+ /**
+ * <code>repeated string store_file = 3;</code>
+ *
+ * <pre>
+ * relative to store dir
+ * </pre>
+ */
+ public Builder addStoreFile(
+ java.lang.String value) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ ensureStoreFileIsMutable();
+ storeFile_.add(value);
+ onChanged();
+ return this;
+ }
+ /**
+ * <code>repeated string store_file = 3;</code>
+ *
+ * <pre>
+ * relative to store dir
+ * </pre>
+ */
+ public Builder addAllStoreFile(
+ java.lang.Iterable<java.lang.String> values) {
+ ensureStoreFileIsMutable();
+ super.addAll(values, storeFile_);
+ onChanged();
+ return this;
+ }
+ /**
+ * <code>repeated string store_file = 3;</code>
+ *
+ * <pre>
+ * relative to store dir
+ * </pre>
+ */
+ public Builder clearStoreFile() {
+ storeFile_ = com.google.protobuf.LazyStringArrayList.EMPTY;
+ bitField0_ = (bitField0_ & ~0x00000004);
+ onChanged();
+ return this;
+ }
+ /**
+ * <code>repeated string store_file = 3;</code>
+ *
+ * <pre>
+ * relative to store dir
+ * </pre>
+ */
+ public Builder addStoreFileBytes(
+ com.google.protobuf.ByteString value) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ ensureStoreFileIsMutable();
+ storeFile_.add(value);
+ onChanged();
+ return this;
+ }
- public org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor.StoreDescriptor getDefaultInstanceForType() {
- return org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor.StoreDescriptor.getDefaultInstance();
- }
+ // @@protoc_insertion_point(builder_scope:StoreDescriptor)
+ }
- public org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor.StoreDescriptor build() {
- org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor.StoreDescriptor result = buildPartial();
- if (!result.isInitialized()) {
- throw newUninitializedMessageException(result);
- }
- return result;
- }
+ static {
+ defaultInstance = new StoreDescriptor(true);
+ defaultInstance.initFields();
+ }
- public org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor.StoreDescriptor buildPartial() {
- org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor.StoreDescriptor result = new org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor.StoreDescriptor(this);
- int from_bitField0_ = bitField0_;
- int to_bitField0_ = 0;
- if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
- to_bitField0_ |= 0x00000001;
- }
- result.familyName_ = familyName_;
- if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
- to_bitField0_ |= 0x00000002;
- }
- result.storeHomeDir_ = storeHomeDir_;
- if (((bitField0_ & 0x00000004) == 0x00000004)) {
- storeFile_ = new com.google.protobuf.UnmodifiableLazyStringList(
- storeFile_);
- bitField0_ = (bitField0_ & ~0x00000004);
- }
- result.storeFile_ = storeFile_;
- result.bitField0_ = to_bitField0_;
- onBuilt();
- return result;
- }
+ // @@protoc_insertion_point(class_scope:StoreDescriptor)
+ }
- public Builder mergeFrom(com.google.protobuf.Message other) {
- if (other instanceof org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor.StoreDescriptor) {
- return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor.StoreDescriptor)other);
- } else {
- super.mergeFrom(other);
- return this;
- }
- }
+ public interface BulkLoadDescriptorOrBuilder
+ extends com.google.protobuf.MessageOrBuilder {
- public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor.StoreDescriptor other) {
- if (other == org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor.StoreDescriptor.getDefaultInstance()) return this;
- if (other.hasFamilyName()) {
- setFamilyName(other.getFamilyName());
- }
- if (other.hasStoreHomeDir()) {
- bitField0_ |= 0x00000002;
- storeHomeDir_ = other.storeHomeDir_;
- onChanged();
- }
- if (!other.storeFile_.isEmpty()) {
- if (storeFile_.isEmpty()) {
- storeFile_ = other.storeFile_;
- bitField0_ = (bitField0_ & ~0x00000004);
- } else {
- ensureStoreFileIsMutable();
- storeFile_.addAll(other.storeFile_);
- }
- onChanged();
- }
- this.mergeUnknownFields(other.getUnknownFields());
- return this;
- }
+ // required .TableName table_name = 1;
+ /**
+ * <code>required .TableName table_name = 1;</code>
+ */
+ boolean hasTableName();
+ /**
+ * <code>required .TableName table_name = 1;</code>
+ */
+ org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName getTableName();
+ /**
+ * <code>required .TableName table_name = 1;</code>
+ */
+ org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableNameOrBuilder();
- public final boolean isInitialized() {
- if (!hasFamilyName()) {
-
- return false;
- }
- if (!hasStoreHomeDir()) {
-
- return false;
- }
- return true;
- }
+ // required bytes encoded_region_name = 2;
+ /**
+ * <code>required bytes encoded_region_name = 2;</code>
+ */
+ boolean hasEncodedRegionName();
+ /**
+ * <code>required bytes encoded_region_name = 2;</code>
+ */
+ com.google.protobuf.ByteString getEncodedRegionName();
- public Builder mergeFrom(
- com.google.protobuf.CodedInputStream input,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws java.io.IOException {
- org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor.StoreDescriptor parsedMessage = null;
- try {
- parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
- } catch (com.google.protobuf.InvalidProtocolBufferException e) {
- parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor.StoreDescriptor) e.getUnfinishedMessage();
- throw e;
- } finally {
- if (parsedMessage != null) {
- mergeFrom(parsedMessage);
- }
- }
- return this;
- }
- private int bitField0_;
+ // repeated .StoreDescriptor stores = 3;
+ /**
+ * <code>repeated .StoreDescriptor stores = 3;</code>
+ */
+ java.util.List<org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor>
+ getStoresList();
+ /**
+ * <code>repeated .StoreDescriptor stores = 3;</code>
+ */
+ org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor getStores(int index);
+ /**
+ * <code>repeated .StoreDescriptor stores = 3;</code>
+ */
+ int getStoresCount();
+ /**
+ * <code>repeated .StoreDescriptor stores = 3;</code>
+ */
+ java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptorOrBuilder>
+ getStoresOrBuilderList();
+ /**
+ * <code>repeated .StoreDescriptor stores = 3;</code>
+ */
+ org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptorOrBuilder getStoresOrBuilder(
+ int index);
- // required bytes family_name = 1;
- private com.google.protobuf.ByteString familyName_ = com.google.protobuf.ByteString.EMPTY;
- /**
- * <code>required bytes family_name = 1;</code>
- */
- public boolean hasFamilyName() {
- return ((bitField0_ & 0x00000001) == 0x00000001);
- }
- /**
- * <code>required bytes family_name = 1;</code>
- */
- public com.google.protobuf.ByteString getFamilyName() {
- return familyName_;
- }
- /**
- * <code>required bytes family_name = 1;</code>
- */
- public Builder setFamilyName(com.google.protobuf.ByteString value) {
- if (value == null) {
- throw new NullPointerException();
+ // required int64 bulkload_seq_num = 4;
+ /**
+ * <code>required int64 bulkload_seq_num = 4;</code>
+ */
+ boolean hasBulkloadSeqNum();
+ /**
+ * <code>required int64 bulkload_seq_num = 4;</code>
+ */
+ long getBulkloadSeqNum();
}
- bitField0_ |= 0x00000001;
- familyName_ = value;
- onChanged();
- return this;
- }
- /**
- * <code>required bytes family_name = 1;</code>
- */
- public Builder clearFamilyName() {
- bitField0_ = (bitField0_ & ~0x00000001);
- familyName_ = getDefaultInstance().getFamilyName();
- onChanged();
- return this;
- }
+ /**
+ * Protobuf type {@code BulkLoadDescriptor}
+ *
+ * <pre>
+ **
+ * Special WAL entry used for writing bulk load events to WAL
+ * </pre>
+ */
+ public static final class BulkLoadDescriptor extends
+ com.google.protobuf.GeneratedMessage
+ implements BulkLoadDescriptorOrBuilder {
+ // Use BulkLoadDescriptor.newBuilder() to construct.
+ private BulkLoadDescriptor(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+ super(builder);
+ this.unknownFields = builder.getUnknownFields();
+ }
+ private BulkLoadDescriptor(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
- // required string store_home_dir = 2;
- private java.lang.Object storeHomeDir_ = "";
- /**
- * <code>required string store_home_dir = 2;</code>
- *
- * <pre>
- *relative to region dir
- * </pre>
- */
- public boolean hasStoreHomeDir() {
- return ((bitField0_ & 0x00000002) == 0x00000002);
- }
- /**
- * <code>required string store_home_dir = 2;</code>
- *
- * <pre>
- *relative to region dir
- * </pre>
- */
- public java.lang.String getStoreHomeDir() {
- java.lang.Object ref = storeHomeDir_;
- if (!(ref instanceof java.lang.String)) {
- java.lang.String s = ((com.google.protobuf.ByteString) ref)
- .toStringUtf8();
- storeHomeDir_ = s;
- return s;
- } else {
- return (java.lang.String) ref;
- }
- }
- /**
- * <code>required string store_home_dir = 2;</code>
- *
- * <pre>
- *relative to region dir
- * </pre>
- */
- public com.google.protobuf.ByteString
- getStoreHomeDirBytes() {
- java.lang.Object ref = storeHomeDir_;
- if (ref instanceof String) {
- com.google.protobuf.ByteString b =
- com.google.protobuf.ByteString.copyFromUtf8(
- (java.lang.String) ref);
- storeHomeDir_ = b;
- return b;
- } else {
- return (com.google.protobuf.ByteString) ref;
+ private static final BulkLoadDescriptor defaultInstance;
+ public static BulkLoadDescriptor getDefaultInstance() {
+ return defaultInstance;
+ }
+
+ public BulkLoadDescriptor getDefaultInstanceForType() {
+ return defaultInstance;
+ }
+
+ private final com.google.protobuf.UnknownFieldSet unknownFields;
+ @java.lang.Override
+ public final com.google.protobuf.UnknownFieldSet
+ getUnknownFields() {
+ return this.unknownFields;
+ }
+ private BulkLoadDescriptor(
+ com.google.protobuf.CodedInputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ initFields();
+ int mutable_bitField0_ = 0;
+ com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+ com.google.protobuf.UnknownFieldSet.newBuilder();
+ try {
+ boolean done = false;
+ while (!done) {
+ int tag = input.readTag();
+ switch (tag) {
+ case 0:
+ done = true;
+ break;
+ default: {
+ if (!parseUnknownField(input, unknownFields,
+ extensionRegistry, tag)) {
+ done = true;
+ }
+ break;
+ }
+ case 10: {
+ org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder subBuilder = null;
+ if (((bitField0_ & 0x00000001) == 0x00000001)) {
+ subBuilder = tableName_.toBuilder();
+ }
+ tableName_ = input.readMessage(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.PARSER, extensionRegistry);
+ if (subBuilder != null) {
+ subBuilder.mergeFrom(tableName_);
+ tableName_ = subBuilder.buildPartial();
+ }
+ bitField0_ |= 0x00000001;
+ break;
+ }
+ case 18: {
+ bitField0_ |= 0x00000002;
+ encodedRegionName_ = input.readBytes();
+ break;
+ }
+ case 26: {
+ if (!((mutable_bitField0_ & 0x00000004) == 0x00000004)) {
+ stores_ = new java.util.ArrayList<org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor>();
+ mutable_bitField0_ |= 0x00000004;
+ }
+ stores_.add(input.readMessage(org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor.PARSER, extensionRegistry));
+ break;
+ }
+ case 32: {
+ bitField0_ |= 0x00000004;
+ bulkloadSeqNum_ = input.readInt64();
+ break;
+ }
}
}
- /**
- * <code>required string store_home_dir = 2;</code>
- *
- * <pre>
- *relative to region dir
- * </pre>
- */
- public Builder setStoreHomeDir(
- java.lang.String value) {
- if (value == null) {
- throw new NullPointerException();
- }
- bitField0_ |= 0x00000002;
- storeHomeDir_ = value;
- onChanged();
- return this;
+ } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+ throw e.setUnfinishedMessage(this);
+ } catch (java.io.IOException e) {
+ throw new com.google.protobuf.InvalidProtocolBufferException(
+ e.getMessage()).setUnfinishedMessage(this);
+ } finally {
+ if (((mutable_bitField0_ & 0x00000004) == 0x00000004)) {
+ stores_ = java.util.Collections.unmodifiableList(stores_);
}
- /**
- * <code>required string store_home_dir = 2;</code>
- *
- * <pre>
- *relative to region dir
- * </pre>
- */
- public Builder clearStoreHomeDir() {
- bitField0_ = (bitField0_ & ~0x00000002);
- storeHomeDir_ = getDefaultInstance().getStoreHomeDir();
- onChanged();
- return this;
+ this.unknownFields = unknownFields.build();
+ makeExtensionsImmutable();
+ }
+ }
+ public static final com.google.protobuf.Descriptors.Descriptor
+ getDescriptor() {
+ return org.apache.hadoop.hbase.protobuf.generated.WALProtos.internal_static_BulkLoadDescriptor_descriptor;
+ }
+
+ protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+ internalGetFieldAccessorTable() {
+ return org.apache.hadoop.hbase.protobuf.generated.WALProtos.internal_static_BulkLoadDescriptor_fieldAccessorTable
+ .ensureFieldAccessorsInitialized(
+ org.apache.hadoop.hbase.protobuf.generated.WALProtos.BulkLoadDescriptor.class, org.apache.hadoop.hbase.protobuf.generated.WALProtos.BulkLoadDescriptor.Builder.class);
+ }
+
+ public static com.google.protobuf.Parser<BulkLoadDescriptor> PARSER =
+ new com.google.protobuf.AbstractParser<BulkLoadDescriptor>() {
+ public BulkLoadDescriptor parsePartialFrom(
+ com.google.protobuf.CodedInputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return new BulkLoadDescriptor(input, extensionRegistry);
+ }
+ };
+
+ @java.lang.Override
+ public com.google.protobuf.Parser<BulkLoadDescriptor> getParserForType() {
+ return PARSER;
+ }
+
+ private int bitField0_;
+ // required .TableName table_name = 1;
+ public static final int TABLE_NAME_FIELD_NUMBER = 1;
+ private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName tableName_;
+ /**
+ * <code>required .TableName table_name = 1;</code>
+ */
+ public boolean hasTableName() {
+ return ((bitField0_ & 0x00000001) == 0x00000001);
+ }
+ /**
+ * <code>required .TableName table_name = 1;</code>
+ */
+ public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName getTableName() {
+ return tableName_;
+ }
+ /**
+ * <code>required .TableName table_name = 1;</code>
+ */
+ public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableNameOrBuilder() {
+ return tableName_;
+ }
+
+ // required bytes encoded_region_name = 2;
+ public static final int ENCODED_REGION_NAME_FIELD_NUMBER = 2;
+ private com.google.protobuf.ByteString encodedRegionName_;
+ /**
+ * <code>required bytes encoded_region_name = 2;</code>
+ */
+ public boolean hasEncodedRegionName() {
+ return ((bitField0_ & 0x00000002) == 0x00000002);
+ }
+ /**
+ * <code>required bytes encoded_region_name = 2;</code>
+ */
+ public com.google.protobuf.ByteString getEncodedRegionName() {
+ return encodedRegionName_;
+ }
+
+ // repeated .StoreDescriptor stores = 3;
+ public static final int STORES_FIELD_NUMBER = 3;
+ private java.util.List<org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor> stores_;
+ /**
+ * <code>repeated .StoreDescriptor stores = 3;</code>
+ */
+ public java.util.List<org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor> getStoresList() {
+ return stores_;
+ }
+ /**
+ * <code>repeated .StoreDescriptor stores = 3;</code>
+ */
+ public java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptorOrBuilder>
+ getStoresOrBuilderList() {
+ return stores_;
+ }
+ /**
+ * <code>repeated .StoreDescriptor stores = 3;</code>
+ */
+ public int getStoresCount() {
+ return stores_.size();
+ }
+ /**
+ * <code>repeated .StoreDescriptor stores = 3;</code>
+ */
+ public org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor getStores(int index) {
+ return stores_.get(index);
+ }
+ /**
+ * <code>repeated .StoreDescriptor stores = 3;</code>
+ */
+ public org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptorOrBuilder getStoresOrBuilder(
+ int index) {
+ return stores_.get(index);
+ }
+
+ // required int64 bulkload_seq_num = 4;
+ public static final int BULKLOAD_SEQ_NUM_FIELD_NUMBER = 4;
+ private long bulkloadSeqNum_;
+ /**
+ * <code>required int64 bulkload_seq_num = 4;</code>
+ */
+ public boolean hasBulkloadSeqNum() {
+ return ((bitField0_ & 0x00000004) == 0x00000004);
+ }
+ /**
+ * <code>required int64 bulkload_seq_num = 4;</code>
+ */
+ public long getBulkloadSeqNum() {
+ return bulkloadSeqNum_;
+ }
+
+ private void initFields() {
+ tableName_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance();
+ encodedRegionName_ = com.google.protobuf.ByteString.EMPTY;
+ stores_ = java.util.Collections.emptyList();
+ bulkloadSeqNum_ = 0L;
+ }
+ private byte memoizedIsInitialized = -1;
+ public final boolean isInitialized() {
+ byte isInitialized = memoizedIsInitialized;
+ if (isInitialized != -1) return isInitialized == 1;
+
+ if (!hasTableName()) {
+ memoizedIsInitialized = 0;
+ return false;
+ }
+ if (!hasEncodedRegionName()) {
+ memoizedIsInitialized = 0;
+ return false;
+ }
+ if (!hasBulkloadSeqNum()) {
+ memoizedIsInitialized = 0;
+ return false;
+ }
+ if (!getTableName().isInitialized()) {
+ memoizedIsInitialized = 0;
+ return false;
+ }
+ for (int i = 0; i < getStoresCount(); i++) {
+ if (!getStores(i).isInitialized()) {
+ memoizedIsInitialized = 0;
+ return false;
+ }
+ }
+ memoizedIsInitialized = 1;
+ return true;
+ }
+
+ public void writeTo(com.google.protobuf.CodedOutputStream output)
+ throws java.io.IOException {
+ getSerializedSize();
+ if (((bitField0_ & 0x00000001) == 0x00000001)) {
+ output.writeMessage(1, tableName_);
+ }
+ if (((bitField0_ & 0x00000002) == 0x00000002)) {
+ output.writeBytes(2, encodedRegionName_);
+ }
+ for (int i = 0; i < stores_.size(); i++) {
+ output.writeMessage(3, stores_.get(i));
+ }
+ if (((bitField0_ & 0x00000004) == 0x00000004)) {
+ output.writeInt64(4, bulkloadSeqNum_);
+ }
+ getUnknownFields().writeTo(output);
+ }
+
+ private int memoizedSerializedSize = -1;
+ public int getSerializedSize() {
+ int size = memoizedSerializedSize;
+ if (size != -1) return size;
+
+ size = 0;
+ if (((bitField0_ & 0x00000001) == 0x00000001)) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeMessageSize(1, tableName_);
+ }
+ if (((bitField0_ & 0x00000002) == 0x00000002)) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeBytesSize(2, encodedRegionName_);
+ }
+ for (int i = 0; i < stores_.size(); i++) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeMessageSize(3, stores_.get(i));
+ }
+ if (((bitField0_ & 0x00000004) == 0x00000004)) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeInt64Size(4, bulkloadSeqNum_);
+ }
+ size += getUnknownFields().getSerializedSize();
+ memoizedSerializedSize = size;
+ return size;
+ }
+
+ private static final long serialVersionUID = 0L;
+ @java.lang.Override
+ protected java.lang.Object writeReplace()
+ throws java.io.ObjectStreamException {
+ return super.writeReplace();
+ }
+
+ @java.lang.Override
+ public boolean equals(final java.lang.Object obj) {
+ if (obj == this) {
+ return true;
+ }
+ if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.WALProtos.BulkLoadDescriptor)) {
+ return super.equals(obj);
+ }
+ org.apache.hadoop.hbase.protobuf.generated.WALProtos.BulkLoadDescriptor other = (org.apache.hadoop.hbase.protobuf.generated.WALProtos.BulkLoadDescriptor) obj;
+
+ boolean result = true;
+ result = result && (hasTableName() == other.hasTableName());
+ if (hasTableName()) {
+ result = result && getTableName()
+ .equals(other.getTableName());
+ }
+ result = result && (hasEncodedRegionName() == other.hasEncodedRegionName());
+ if (hasEncodedRegionName()) {
+ result = result && getEncodedRegionName()
+ .equals(other.getEncodedRegionName());
+ }
+ result = result && getStoresList()
+ .equals(other.getStoresList());
+ result = result && (hasBulkloadSeqNum() == other.hasBulkloadSeqNum());
+ if (hasBulkloadSeqNum()) {
+ result = result && (getBulkloadSeqNum()
+ == other.getBulkloadSeqNum());
+ }
+ result = result &&
+ getUnknownFields().equals(other.getUnknownFields());
+ return result;
+ }
+
+ private int memoizedHashCode = 0;
+ @java.lang.Override
+ public int hashCode() {
+ if (memoizedHashCode != 0) {
+ return memoizedHashCode;
+ }
+ int hash = 41;
+ hash = (19 * hash) + getDescriptorForType().hashCode();
+ if (hasTableName()) {
+ hash = (37 * hash) + TABLE_NAME_FIELD_NUMBER;
+ hash = (53 * hash) + getTableName().hashCode();
+ }
+ if (hasEncodedRegionName()) {
+ hash = (37 * hash) + ENCODED_REGION_NAME_FIELD_NUMBER;
+ hash = (53 * hash) + getEncodedRegionName().hashCode();
+ }
+ if (getStoresCount() > 0) {
+ hash = (37 * hash) + STORES_FIELD_NUMBER;
+ hash = (53 * hash) + getStoresList().hashCode();
+ }
+ if (hasBulkloadSeqNum()) {
+ hash = (37 * hash) + BULKLOAD_SEQ_NUM_FIELD_NUMBER;
+ hash = (53 * hash) + hashLong(getBulkloadSeqNum());
+ }
+ hash = (29 * hash) + getUnknownFields().hashCode();
+ memoizedHashCode = hash;
+ return hash;
+ }
+
+ public static org.apache.hadoop.hbase.protobuf.generated.WALProtos.BulkLoadDescriptor parseFrom(
+ com.google.protobuf.ByteString data)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return PARSER.parseFrom(data);
+ }
+ public static org.apache.hadoop.hbase.protobuf.generated.WALProtos.BulkLoadDescriptor parseFrom(
+ com.google.protobuf.ByteString data,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return PARSER.parseFrom(data, extensionRegistry);
+ }
+ public static org.apache.hadoop.hbase.protobuf.generated.WALProtos.BulkLoadDescriptor parseFrom(byte[] data)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return PARSER.parseFrom(data);
+ }
+ public static org.apache.hadoop.hbase.protobuf.generated.WALProtos.BulkLoadDescriptor parseFrom(
+ byte[] data,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return PARSER.parseFrom(data, extensionRegistry);
+ }
+ public static org.apache.hadoop.hbase.protobuf.generated.WALProtos.BulkLoadDescriptor parseFrom(java.io.InputStream input)
+ throws java.io.IOException {
+ return PARSER.parseFrom(input);
+ }
+ public static org.apache.hadoop.hbase.protobuf.generated.WALProtos.BulkLoadDescriptor parseFrom(
+ java.io.InputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ return PARSER.parseFrom(input, extensionRegistry);
+ }
+ public static org.apache.hadoop.hbase.protobuf.generated.WALProtos.BulkLoadDescriptor parseDelimitedFrom(java.io.InputStream input)
+ throws java.io.IOException {
+ return PARSER.parseDelimitedFrom(input);
+ }
+ public static org.apache.hadoop.hbase.protobuf.generated.WALProtos.BulkLoadDescriptor parseDelimitedFrom(
+ java.io.InputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ return PARSER.parseDelimitedFrom(input, extensionRegistry);
+ }
+ public static org.apache.hadoop.hbase.protobuf.generated.WALProtos.BulkLoadDescriptor parseFrom(
+ com.google.protobuf.CodedInputStream input)
+ throws java.io.IOException {
+ return PARSER.parseFrom(input);
+ }
+ public static org.apache.hadoop.hbase.protobuf.generated.WALProtos.BulkLoadDescriptor parseFrom(
+ com.google.protobuf.CodedInputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ return PARSER.parseFrom(input, extensionRegistry);
+ }
+
+ public static Builder newBuilder() { return Builder.create(); }
+ public Builder newBuilderForType() { return newBuilder(); }
+ public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.WALProtos.BulkLoadDescriptor prototype) {
+ return newBuilder().mergeFrom(prototype);
+ }
+ public Builder toBuilder() { return newBuilder(this); }
+
+ @java.lang.Override
+ protected Builder newBuilderForType(
+ com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+ Builder builder = new Builder(parent);
+ return builder;
+ }
+ /**
+ * Protobuf type {@code BulkLoadDescriptor}
+ *
+ * <pre>
+ **
+ * Special WAL entry used for writing bulk load events to WAL
+ * </pre>
+ */
+ public static final class Builder extends
+ com.google.protobuf.GeneratedMessage.Builder<Builder>
+ implements org.apache.hadoop.hbase.protobuf.generated.WALProtos.BulkLoadDescriptorOrBuilder {
+ public static final com.google.protobuf.Descriptors.Descriptor
+ getDescriptor() {
+ return org.apache.hadoop.hbase.protobuf.generated.WALProtos.internal_static_BulkLoadDescriptor_descriptor;
+ }
+
+ protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+ internalGetFieldAccessorTable() {
+ return org.apache.hadoop.hbase.protobuf.generated.WALProtos.internal_static_BulkLoadDescriptor_fieldAccessorTable
+ .ensureFieldAccessorsInitialized(
+ org.apache.hadoop.hbase.protobuf.generated.WALProtos.BulkLoadDescriptor.class, org.apache.hadoop.hbase.protobuf.generated.WALProtos.BulkLoadDescriptor.Builder.class);
+ }
+
+ // Construct using org.apache.hadoop.hbase.protobuf.generated.WALProtos.BulkLoadDescriptor.newBuilder()
+ private Builder() {
+ maybeForceBuilderInitialization();
+ }
+
+ private Builder(
+ com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+ super(parent);
+ maybeForceBuilderInitialization();
+ }
+ private void maybeForceBuilderInitialization() {
+ if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+ getTableNameFieldBuilder();
+ getStoresFieldBuild
<TRUNCATED>
[3/3] hbase git commit: HBASE-11567 Write bulk load COMMIT events to
WAL
Posted by je...@apache.org.
HBASE-11567 Write bulk load COMMIT events to WAL
Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/b0b0a74f
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/b0b0a74f
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/b0b0a74f
Branch: refs/heads/branch-1
Commit: b0b0a74fef6382643c6ff8d07167ad90ff0d7c43
Parents: a7b8112
Author: Jeffrey Zhong <je...@apache.org>
Authored: Wed Feb 4 15:52:01 2015 -0800
Committer: Jeffrey Zhong <je...@apache.org>
Committed: Fri Feb 6 18:46:00 2015 -0800
----------------------------------------------------------------------
.../hadoop/hbase/protobuf/ProtobufUtil.java | 35 +-
.../hbase/protobuf/generated/FilterProtos.java | 6 +-
.../generated/VisibilityLabelsProtos.java | 6 +-
.../hbase/protobuf/generated/WALProtos.java | 3441 ++++++++++++------
hbase-protocol/src/main/protobuf/WAL.proto | 23 +-
.../hadoop/hbase/regionserver/HRegion.java | 95 +-
.../hadoop/hbase/regionserver/wal/WALEdit.java | 35 +-
.../hadoop/hbase/regionserver/wal/WALUtil.java | 39 +
.../hadoop/hbase/regionserver/TestBulkLoad.java | 312 ++
.../hadoop/hbase/regionserver/TestHRegion.java | 2 +-
.../regionserver/TestHRegionServerBulkLoad.java | 47 +-
.../wal/TestWALActionsListener.java | 2 +-
pom.xml | 17 +
13 files changed, 2870 insertions(+), 1190 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/hbase/blob/b0b0a74f/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
index 6d82444..b8d8ce6 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
@@ -122,6 +122,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.security.access.Permission;
import org.apache.hadoop.hbase.security.access.TablePermission;
import org.apache.hadoop.hbase.security.access.UserPermission;
@@ -2610,8 +2612,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()) {
@@ -2828,4 +2829,34 @@ public final class ProtobufUtil {
}
return result;
}
+
+ /**
+ * 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();
+ }
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/b0b0a74f/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/FilterProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/FilterProtos.java b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/FilterProtos.java
index 3fb466d..af1f33d 100644
--- a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/FilterProtos.java
+++ b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/FilterProtos.java
@@ -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<
http://git-wip-us.apache.org/repos/asf/hbase/blob/b0b0a74f/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/VisibilityLabelsProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/VisibilityLabelsProtos.java b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/VisibilityLabelsProtos.java
index 294772e..70593b0 100644
--- a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/VisibilityLabelsProtos.java
+++ b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/VisibilityLabelsProtos.java
@@ -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;