Class WALEdit

java.lang.Object
org.apache.hadoop.hbase.wal.WALEdit
All Implemented Interfaces:
HeapSize

@LimitedPrivate({"Replication","Coprocesssor"}) public class WALEdit extends Object implements HeapSize
Used in HBase's transaction log (WAL) to represent a collection of edits (Cell/KeyValue objects) that came in as a single transaction. All the edits for a given transaction are written out as a single record, in PB format, followed (optionally) by Cells written via the WALCellEncoder.

This class is LimitedPrivate for CPs to read-only. The add(org.apache.hadoop.hbase.Cell, byte[]) methods are classified as private methods, not for use by CPs.

A particular WALEdit 'type' is the 'meta' type used to mark key operational events in the WAL such as compaction, flush, or region open. These meta types do not traverse hbase memstores. They are edits made by the hbase system rather than edit data submitted by clients. They only show in the WAL. These 'Meta' types have not been formally specified (or made into an explicit class type). They evolved organically. HBASE-8457 suggests codifying a WALEdit 'type' by adding a type field to WALEdit that gets serialized into the WAL. TODO. Would have to work on the consumption-side. Reading WALs on replay we seem to consume a Cell-at-a-time rather than by WALEdit. We are already in the below going out of our way to figure particular types -- e.g. if a compaction, replay, or close meta Marker -- during normal processing so would make sense to do this. Current system is an awkward marking of Cell columnfamily as METAFAMILY and then setting qualifier based off meta edit type. For replay-time where we read Cell-at-a-time, there are utility methods below for figuring meta type. See also createBulkLoadEvent(RegionInfo, WALProtos.BulkLoadDescriptor), etc., for where we create meta WALEdit instances.

WALEdit will accumulate a Set of all column family names referenced by the Cells add(Cell)'d. This is an optimization. Usually when loading a WALEdit, we have the column family name to-hand.. just shove it into the WALEdit if available. Doing this, we can save on a parse of each Cell to figure column family down the line when we go to add the WALEdit to the WAL file. See the hand-off in FSWALEntry Constructor.

See Also:
  • Field Details

    • METAFAMILY

      public static final byte[] METAFAMILY
    • METAROW

      @Deprecated public static final byte[] METAROW
      Deprecated.
      Since 2.3.0. Not used.
    • COMPACTION

      @Deprecated @Private public static final byte[] COMPACTION
      Deprecated.
      Since 2.3.0. Make it protected, internal-use only. Use isCompactionMarker(Cell)
    • FLUSH

      @Deprecated @Private public static final byte[] FLUSH
      Deprecated.
      Since 2.3.0. Make it protected, internal-use only.
    • REGION_EVENT_STR

      private static final String REGION_EVENT_STR
      Qualifier for region event meta 'Marker' WALEdits start with the REGION_EVENT_PREFIX prefix ('HBASE::REGION_EVENT::'). After the prefix, we note the type of the event which we get from the RegionEventDescriptor protobuf instance type (A RegionEventDescriptor protobuf instance is written as the meta Marker Cell value). Adding a type suffix means we do not have to deserialize the protobuf to figure out what type of event this is.. .just read the qualifier suffix. For example, a close region event descriptor will have a qualifier of HBASE::REGION_EVENT::REGION_CLOSE. See WAL.proto and the EventType in RegionEventDescriptor protos for all possible event types.
      See Also:
    • REGION_EVENT_PREFIX_STR

      private static final String REGION_EVENT_PREFIX_STR
      See Also:
    • REGION_EVENT_PREFIX

      private static final byte[] REGION_EVENT_PREFIX
    • REGION_EVENT

      @Deprecated public static final byte[] REGION_EVENT
      Deprecated.
      Since 2.3.0. Remove. Not for external use. Not used.
    • REGION_EVENT_CLOSE

      private static final byte[] REGION_EVENT_CLOSE
      We use this define figuring if we are carrying a close event.
    • BULK_LOAD

      @Private public static final byte[] BULK_LOAD
    • REPLICATION_MARKER

      @Private public static final byte[] REPLICATION_MARKER
      Periodically ReplicationMarkerChore will create marker edits with family as METAFAMILY and REPLICATION_MARKER as qualifier and an empty value. org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceWALReader will populate the Replication Marker edit with region_server_name, wal_name and wal_offset encoded in WALProtos.ReplicationMarkerDescriptor object. Replication will change the REPLICATION_SCOPE for this edit to GLOBAL so that it can replicate. On the sink cluster, ReplicationSink will convert the ReplicationMarkerDescriptor into a Put mutation to REPLICATION_SINK_TRACKER_TABLE_NAME_STR table.
    • replay

      private final transient boolean replay
    • cells

      private ArrayList<Cell> cells
    • families

      private Set<byte[]> families
      All the Cell families in cells. Updated by add(Cell) and add(Map). This Set is passed to the FSWALEntry so it does not have to recalculate the Set of families in a transaction; makes for a bunch of CPU savings.
  • Constructor Details

  • Method Details

    • getOrCreateFamilies

      private Set<byte[]> getOrCreateFamilies()
    • getFamilies

      public Set<byte[]> getFamilies()
      For use by FSWALEntry ONLY. An optimization.
      Returns:
      All families in getCells(); may be null.
    • isMetaEditFamily

      @Deprecated public static boolean isMetaEditFamily(byte[] f)
      Deprecated.
      Since 2.3.0. Do not expose. Make protected.
      Returns:
      True is f is METAFAMILY
    • isMetaEditFamily

      public static boolean isMetaEditFamily(Cell cell)
      Replaying WALs can read Cell-at-a-time so need this method in those cases.
    • isMetaEdit

      public boolean isMetaEdit()
      Returns:
      True if this is a meta edit; has one edit only and its columnfamily is METAFAMILY.
    • isReplay

      public boolean isReplay()
      Returns:
      True when current WALEdit is created by log replay. Replication skips WALEdits from replay.
    • add

      @Private public WALEdit add(Cell cell, byte[] family)
    • add

      @Private public WALEdit add(Cell cell)
    • isEmpty

      public boolean isEmpty()
    • size

      public int size()
    • getCells

      public ArrayList<Cell> getCells()
    • setCells

      @Private public void setCells(ArrayList<Cell> cells)
      This is not thread safe. This will change the WALEdit and shouldn't be used unless you are sure that nothing else depends on the contents being immutable.
      Parameters:
      cells - the list of cells that this WALEdit now contains.
    • readFromCells

      public int readFromCells(Codec.Decoder cellDecoder, int expectedCount) throws IOException
      Reads WALEdit from cells.
      Parameters:
      cellDecoder - Cell decoder.
      expectedCount - Expected cell count.
      Returns:
      Number of KVs read.
      Throws:
      IOException
    • heapSize

      public long heapSize()
      Description copied from interface: HeapSize
      Return the approximate 'exclusive deep size' of implementing object. Includes count of payload and hosting object sizings.
      Specified by:
      heapSize in interface HeapSize
    • estimatedSerializedSizeOf

    • toString

      public String toString()
      Overrides:
      toString in class Object
    • createFlushWALEdit

      public static WALEdit createFlushWALEdit(RegionInfo hri, org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor f)
    • getFlushDescriptor

      public static org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor getFlushDescriptor(Cell cell) throws IOException
      Throws:
      IOException
    • createRegionEventWALEdit

      public static WALEdit createRegionEventWALEdit(RegionInfo hri, org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.RegionEventDescriptor regionEventDesc)
      Returns:
      A meta Marker WALEdit that has a single Cell whose value is the passed in regionEventDesc serialized and whose row is this region, columnfamily is METAFAMILY and qualifier is REGION_EVENT_PREFIX + WALProtos.RegionEventDescriptor.getEventType(); for example HBASE::REGION_EVENT::REGION_CLOSE.
    • createRegionEventWALEdit

      @Private public static WALEdit createRegionEventWALEdit(byte[] rowForRegion, org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.RegionEventDescriptor regionEventDesc)
    • createRegionEventDescriptorQualifier

      @Private public static byte[] createRegionEventDescriptorQualifier(org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.RegionEventDescriptor.EventType t)
      Returns:
      Cell qualifier for the passed in RegionEventDescriptor Type; e.g. we'll return something like a byte array with HBASE::REGION_EVENT::REGION_OPEN in it.
    • isRegionCloseMarker

      public boolean isRegionCloseMarker()
      Public so can be accessed from regionserver.wal package.
      Returns:
      True if this is a Marker Edit and it is a RegionClose type.
    • getRegionEventDescriptor

      public static org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.RegionEventDescriptor getRegionEventDescriptor(Cell cell) throws IOException
      Returns:
      Returns a RegionEventDescriptor made by deserializing the content of the passed in cell, IFF the cell is a RegionEventDescriptor type WALEdit.
      Throws:
      IOException
    • createCompaction

      public static WALEdit createCompaction(RegionInfo hri, org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.CompactionDescriptor c)
      Returns A Marker WALEdit that has c serialized as its value
    • getRowForRegion

      public static byte[] getRowForRegion(RegionInfo hri)
    • getCompaction

      public static org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.CompactionDescriptor getCompaction(Cell kv) throws IOException
      Deserialized and returns a CompactionDescriptor is the KeyValue contains one.
      Parameters:
      kv - the key value
      Returns:
      deserialized CompactionDescriptor or null.
      Throws:
      IOException
    • isCompactionMarker

      public static boolean isCompactionMarker(Cell cell)
      Returns true if the given cell is a serialized WALProtos.CompactionDescriptor
      See Also:
    • createBulkLoadEvent

      public static WALEdit createBulkLoadEvent(RegionInfo hri, org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.BulkLoadDescriptor bulkLoadDescriptor)
      Create a bulk loader WALEdit
      Parameters:
      hri - The RegionInfo for the region in which we are bulk loading
      bulkLoadDescriptor - The descriptor for the Bulk Loader
      Returns:
      The WALEdit for the BulkLoad
    • getBulkLoadDescriptor

      public static org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.BulkLoadDescriptor getBulkLoadDescriptor(Cell cell) throws IOException
      Deserialized and returns a BulkLoadDescriptor from the passed in Cell
      Parameters:
      cell - the key value
      Returns:
      deserialized BulkLoadDescriptor or null.
      Throws:
      IOException
    • add

      public void add(Map<byte[],List<Cell>> familyMap)
      Append the given map of family->edits to a WALEdit data structure. This does not write to the WAL itself. Note that as an optimization, we will stamp the Set of column families into the WALEdit to save on our having to calculate column families subsequently down in the actual WAL writing.
      Parameters:
      familyMap - map of family->edits
    • addFamily

      private void addFamily(byte[] family)
    • addCell

      private WALEdit addCell(Cell cell)
    • createReplicationMarkerEdit

      public static WALEdit createReplicationMarkerEdit(byte[] rowKey, long timestamp)
      Creates a replication tracker edit with METAFAMILY family and REPLICATION_MARKER qualifier and has null value.
      Parameters:
      rowKey - rowkey
      timestamp - timestamp
    • isReplicationMarkerEdit

      public static boolean isReplicationMarkerEdit(WALEdit edit)
      Checks whether this edit is a replication marker edit.
      Parameters:
      edit - edit
      Returns:
      true if the cell within an edit has column = METAFAMILY and qualifier = REPLICATION_MARKER, false otherwise