hbase-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From e...@apache.org
Subject [3/3] git commit: HBASE-11511 Write flush events to WAL
Date Tue, 15 Jul 2014 21:58:56 GMT
HBASE-11511 Write flush 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/4fa0e327
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/4fa0e327
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/4fa0e327

Branch: refs/heads/branch-1
Commit: 4fa0e3274fbade789ae5801bf34ee532972a2ec4
Parents: 063247b
Author: Enis Soztutar <enis@apache.org>
Authored: Tue Jul 15 14:47:19 2014 -0700
Committer: Enis Soztutar <enis@apache.org>
Committed: Tue Jul 15 14:48:30 2014 -0700

----------------------------------------------------------------------
 .../hadoop/hbase/protobuf/ProtobufUtil.java     |   26 +
 .../hbase/protobuf/generated/WALProtos.java     | 2456 +++++++++++++++++-
 hbase-protocol/src/main/protobuf/WAL.proto      |   27 +-
 .../hadoop/hbase/regionserver/HRegion.java      |   66 +-
 .../hadoop/hbase/regionserver/HStore.java       |   12 +-
 .../hbase/regionserver/StoreFlushContext.java   |    8 +
 .../hadoop/hbase/regionserver/wal/HLogUtil.java |   16 +
 .../hadoop/hbase/regionserver/wal/WALEdit.java  |   23 +-
 .../hadoop/hbase/regionserver/TestHRegion.java  |  228 ++
 9 files changed, 2775 insertions(+), 87 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/4fa0e327/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 a70b639..f7531ee 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
@@ -87,6 +87,7 @@ import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SplitRegionRequest
 import org.apache.hadoop.hbase.protobuf.generated.AuthenticationProtos;
 import org.apache.hadoop.hbase.protobuf.generated.CellProtos;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
+import org.apache.hadoop.hbase.protobuf.generated.WALProtos;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.BulkLoadHFileRequest;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.BulkLoadHFileResponse;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ClientService;
@@ -117,6 +118,8 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MasterService;
 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerReportRequest;
 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerStartupRequest;
 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;
 import org.apache.hadoop.hbase.security.access.Permission;
 import org.apache.hadoop.hbase.security.access.TablePermission;
 import org.apache.hadoop.hbase.security.access.UserPermission;
@@ -2499,6 +2502,29 @@ public final class ProtobufUtil {
     return builder.build();
   }
 
+  public static FlushDescriptor toFlushDescriptor(FlushAction action, HRegionInfo hri,
+      long flushSeqId, Map<byte[], List<Path>> committedFiles) {
+    FlushDescriptor.Builder desc = FlushDescriptor.newBuilder()
+        .setAction(action)
+        .setEncodedRegionName(ByteStringer.wrap(hri.getEncodedNameAsBytes()))
+        .setFlushSequenceNumber(flushSeqId)
+        .setTableName(ByteStringer.wrap(hri.getTable().getName()));
+
+    for (Map.Entry<byte[], List<Path>> entry : committedFiles.entrySet()) {
+      WALProtos.FlushDescriptor.StoreFlushDescriptor.Builder builder =
+          WALProtos.FlushDescriptor.StoreFlushDescriptor.newBuilder()
+          .setFamilyName(ByteStringer.wrap(entry.getKey()))
+          .setStoreHomeDir(Bytes.toString(entry.getKey())); //relative to region
+      if (entry.getValue() != null) {
+        for (Path path : entry.getValue()) {
+          builder.addFlushOutput(path.getName());
+        }
+      }
+      desc.addStoreFlushes(builder);
+    }
+    return desc.build();
+  }
+
   /**
    * Return short version of Message toString'd, shorter than TextFormat#shortDebugString.
    * Tries to NOT print out data both because it can be big but also so we do not have data
in our


Mime
View raw message