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;