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/bbe29eb9 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/bbe29eb9 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/bbe29eb9 Branch: refs/heads/master Commit: bbe29eb93cc819fbf0287aa2cb343649b72783bf Parents: fe50c6d Author: Enis Soztutar <e...@apache.org> Authored: Tue Jul 15 14:47:19 2014 -0700 Committer: Enis Soztutar <e...@apache.org> Committed: Tue Jul 15 14:47:19 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/bbe29eb9/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