hbasecon 2012 | learning hbase internals - lars hofhansl, salesforce

53
HBase Internals Lars Hofhansl Architect @ Salesforce.com HBase Committer

Upload: cloudera-inc

Post on 12-Jun-2015

5.032 views

Category:

Technology


0 download

DESCRIPTION

The strength of an open source project resides entirely in its developer community; a strong democratic culture of participation and hacking makes for a better piece of software. The key requirement is having developers who are not only willing to contribute, but also knowledgeable about the project’s internal structure and architecture. This session will introduce developers to the core internal architectural concepts of HBase, not just “what” it does from the outside, but “how” it works internally, and “why” it does things a certain way. We’ll walk through key sections of code and discuss key concepts like the MVCC implementation and memstore organization. The goal is to convert serious “HBase Users” into HBase Developer Users”, and give voice to some of the deep knowledge locked in the committers’ heads.

TRANSCRIPT

Page 1: HBaseCon 2012 | Learning HBase Internals - Lars Hofhansl, Salesforce

HBase InternalsLars Hofhansl

Architect @ Salesforce.comHBase Committer

Page 2: HBaseCon 2012 | Learning HBase Internals - Lars Hofhansl, Salesforce

HBase InternalsA.K.A. Get ready to:

Page 3: HBaseCon 2012 | Learning HBase Internals - Lars Hofhansl, Salesforce

Agenda

• Overview

• Scanning

• Atomicity/MVCC

• Updates– Put/Delete

• Code!

Page 4: HBaseCon 2012 | Learning HBase Internals - Lars Hofhansl, Salesforce

A sparse, consistent, distributed, multi-dimensional,

persistent, sorted map.

Page 5: HBaseCon 2012 | Learning HBase Internals - Lars Hofhansl, Salesforce
Page 6: HBaseCon 2012 | Learning HBase Internals - Lars Hofhansl, Salesforce

In the end it comes down to a sorting problem.How do you sort 1PB with 16GB of memory?

7 5 6 3 2 9 1 8 4

7 5 6 3 2 9 1 8 4

5 6 7 2 3 9 1 4 8

1 2 3 4 5 6 7 8 9

Page 7: HBaseCon 2012 | Learning HBase Internals - Lars Hofhansl, Salesforce

Overview

• All changes collected/sorted in memory

• Periodically flushed to HDFS– Writes to disk are batched

• HFiles periodically compacted into larger files

• Scanning/Compacting: Merge Sort

Page 8: HBaseCon 2012 | Learning HBase Internals - Lars Hofhansl, Salesforce
Page 9: HBaseCon 2012 | Learning HBase Internals - Lars Hofhansl, Salesforce

HDFS Directory Hierarchy

/hbase  /-ROOT-  /.META.  /.logs  /<table1>    /<region>      [/.recovered_logs]      /<column family>        /<HFile>        /...        /...       /<column family>        /...  /<table2>     ...  

Storage is per CF

Page 10: HBaseCon 2012 | Learning HBase Internals - Lars Hofhansl, Salesforce

Scanning

• "Log Structured Mergetrees"• Multiway mergesort• Efficient scanning/seeking

/Region | | +--/CF  |  |  +--/HFile

Page 11: HBaseCon 2012 | Learning HBase Internals - Lars Hofhansl, Salesforce

KeyValueHeap

• Maintains PriorityQueue of “lower” Scanners

• TopScanner in the Queue has the next KV

Page 12: HBaseCon 2012 | Learning HBase Internals - Lars Hofhansl, Salesforce

Scanning

• "Log Structured Mergetrees"• Multiway mergesort• Efficient scanning/seeking

/Region | | +--/CF  |  |  +--/HFile

Page 13: HBaseCon 2012 | Learning HBase Internals - Lars Hofhansl, Salesforce

Updates

• All changes are written to the WAL first• All changes are written to memory (the "MemStore")• MemStores are flushed to disk (creating a new HFile)• HFiles are periodically and asynchronously compacted into

fewer files.• HFiles are immutable

Page 14: HBaseCon 2012 | Learning HBase Internals - Lars Hofhansl, Salesforce

ROW Atomicity

• Snapshot isolation and locking (per row)o Row is locked while memory is changedo Each row-update "creates" a new snapshoto Each row-read sees exactly one snapshot

• Done with MultiVersionConcurrencyControl (MVCC)

Page 15: HBaseCon 2012 | Learning HBase Internals - Lars Hofhansl, Salesforce

Locking

• All KVs for a “row” are co-located in a region

• Locks are per row

• Stored in-memory at region server

Page 16: HBaseCon 2012 | Learning HBase Internals - Lars Hofhansl, Salesforce

MultiVersionConcurrencyControl

Wikipedia: "implement updates not by deleting an old piece of data and overwriting it with a new one, but instead by marking the old data as obsolete and adding the newer version"

Note that HBase calls it MultiVersionConsistencyControl

Page 17: HBaseCon 2012 | Learning HBase Internals - Lars Hofhansl, Salesforce

MVCC writing

• Each write gets a monotonically increasing "writenumber“

• Each KV written is tagged with this writenumber(called memstoreTS in HBase)

• "Committing" a writenumber:– wait for all prior writes to finish– set the current readpoint to the writenumber

Page 18: HBaseCon 2012 | Learning HBase Internals - Lars Hofhansl, Salesforce

MVCC reading

• Each read reads as of a "readpoint“– Filters KVs with a newer memstoreTS

• This is per regionserver (cheap in memory data structures)– if regionserver dies, current read is lost anyway

• but... "writenumbers" are persisted to disk  – for scanners that outlive a Memstore flush

Page 19: HBaseCon 2012 | Learning HBase Internals - Lars Hofhansl, Salesforce
Page 20: HBaseCon 2012 | Learning HBase Internals - Lars Hofhansl, Salesforce

MVCC

• Reader do not lock

• Transaction are committed strictly serially

• HBase has no client demarcated transactions– A transaction does not outlive an RPC

Page 21: HBaseCon 2012 | Learning HBase Internals - Lars Hofhansl, Salesforce

Anatomy of an "update"

1. Acquire a MVCC "writenumber"2. Lock the "row" (the row-key) 3. Tag all KVs with the writenumber4. Write change to the WAL and sync to file system5. Apply update in memory ("Memstore")6. Unlock "row" 7. Roll MVCC forward -> now change is visible8. If Memstore reaches configurable size, take a snapshot, flush it

to disk (asynchronously)

Page 22: HBaseCon 2012 | Learning HBase Internals - Lars Hofhansl, Salesforce

Anatomy of an "update"

1. Acquire a MVCC "writenumber"2. Lock the "row" (the row-key) 3. Tag all KVs with the writenumber4. Write change to the WAL and sync to file system5. Apply update in memory ("Memstore")6. Unlock "row" 

o 5.5 sync WAL to HDFS without the row lock held If that fails undo changes in Memstore that works because changes are not visible, yet

7. Roll MVCC forward -> now change is visible8. If Memstore reaches configurable size, take a snapshot, flush it

to disk (asynchronously)

Puts are optimized

Page 23: HBaseCon 2012 | Learning HBase Internals - Lars Hofhansl, Salesforce

Anatomy of an "update"

1. Acquire a MVCC "writenumber"2. lock as many "rows" as possible3. Tag all KVs with the writenumber4. write all changes to the WAL and sync to file system5. apply all updates in memory ("Memstore")6. unlock all "rows" 

o 5.5 sync WAL to HDFS without the row locks held If that fails undo changes in Memstore that works because changes are not visible, yet

7. roll MVCC forward -> now changes are visible8. if Memstore reaches configurable size, take a snapshot, flush it

to disk (asynchronously)

Puts are optimized, and batched

Page 24: HBaseCon 2012 | Learning HBase Internals - Lars Hofhansl, Salesforce

Undo

• In-memory only

• Changes are not visible until MVCC is rolled

• Changes are tagged with the writenumber

• Undo removes KVs tagged with the writenumber

Page 25: HBaseCon 2012 | Learning HBase Internals - Lars Hofhansl, Salesforce

Deletes

• Nothing is deleted in place.• a Delete sets a "tombstone marker" with a timestamp• upon compaction deleted KeyValues are removed• upon major compactions the tombstones are removed• Three different tombstone marker scopes (all within a row)

o version - mark a specific version of a column as deletedo column - mark all versions of a column as deletedo column family - mark all versions of all columns of a column

family as deleted

Page 26: HBaseCon 2012 | Learning HBase Internals - Lars Hofhansl, Salesforce

Deletes, cont

• Delete markers always sort before KVs

• A scanner remembers markers it encounters

• Each KV is checked against remembered markers

• Only one-pass is required for scanning

• Deletes are just KVs stored in HFiles

Page 27: HBaseCon 2012 | Learning HBase Internals - Lars Hofhansl, Salesforce

Let’s look at some code!

Page 28: HBaseCon 2012 | Learning HBase Internals - Lars Hofhansl, Salesforce

MVCC

Page 29: HBaseCon 2012 | Learning HBase Internals - Lars Hofhansl, Salesforce

MultiVersionConsistencyControl.java

  public WriteEntry beginMemstoreInsert() {    synchronized (writeQueue) {      long nextWriteNumber = ++memstoreWrite;      WriteEntry e = new WriteEntry(nextWriteNumber);      writeQueue.add(e);      return e;    }  }  public void completeMemstoreInsert(WriteEntry e) {    advanceMemstore(e);    waitForRead(e);  } 

Acquire a new Writenumber

Roll forward the readpoint

Wait for prior transactions to finish

Page 30: HBaseCon 2012 | Learning HBase Internals - Lars Hofhansl, Salesforce

MultiVersionConsistencyControl.java

  boolean advanceMemstore(WriteEntry e) {    synchronized (writeQueue) {      e.markCompleted();

      long nextReadValue = -1;      while (!writeQueue.isEmpty()) {        WriteEntry queueFirst = writeQueue.getFirst();        ...        if (queueFirst.isCompleted()) {          nextReadValue = queueFirst.getWriteNumber();          writeQueue.removeFirst();        } else {          break;        }      }      ...

Roll forward completedtransactions. Ordering is preserved.

Page 31: HBaseCon 2012 | Learning HBase Internals - Lars Hofhansl, Salesforce

MultiVersionConsistencyControl.java

        ...      if (nextReadValue > 0) {        synchronized (readWaiters) {          memstoreRead = nextReadValue;          readWaiters.notifyAll();        }      }      if (memstoreRead >= e.getWriteNumber()) {        return true;      }      return false;    }  }

Notify later transactions

Page 32: HBaseCon 2012 | Learning HBase Internals - Lars Hofhansl, Salesforce

MultiVersionConsistencyControl.java

  public void waitForRead(WriteEntry e) {    boolean interrupted = false;    synchronized (readWaiters) {      while (memstoreRead < e.getWriteNumber()) {        try {          readWaiters.wait(0);        } catch (InterruptedException ie) {          interrupted = true;        }      }    }    if (interrupted) Thread.currentThread().interrupt();  }

Wait until write entry was applied.

Page 33: HBaseCon 2012 | Learning HBase Internals - Lars Hofhansl, Salesforce

Scanning

Page 34: HBaseCon 2012 | Learning HBase Internals - Lars Hofhansl, Salesforce

RegionScanner, creation

      RegionScannerImpl(Scan scan,      List<KeyValueScanner> additionalScanners) {      ...      IsolationLevel isolationLevel = scan.getIsolationLevel();      synchronized(scannerReadPoints) {        if (isolationLevel == IsolationLevel.READ_UNCOMMITTED) {          // This scan can read even uncommitted transactions          this.readPt = Long.MAX_VALUE;          MVCC.setThreadReadPoint(this.readPt);        } else {          this.readPt = MVCC.resetThreadReadPoint(mvcc);        }        scannerReadPoints.put(this, this.readPt);      }      ...

MVCC protocol

Page 35: HBaseCon 2012 | Learning HBase Internals - Lars Hofhansl, Salesforce

RegionScanner

      ...      for (Map.Entry<byte[], NavigableSet<byte[]>> entry :          scan.getFamilyMap().entrySet()) {        Store store = stores.get(entry.getKey());        StoreScanner scanner = store.getScanner(...);        scanners.add(scanner);      }      this.storeHeap = new KeyValueHeap(scanners, comparator);    }

Get all StoreScanners

Heap of StoreScanners

Page 36: HBaseCon 2012 | Learning HBase Internals - Lars Hofhansl, Salesforce

RegionScanner, cont.

    public synchronized boolean next(        List<KeyValue> outResults, int limit) {        ...        MVCC.setThreadReadPoint(this.readPt);        boolean returnResult = nextInternal(limit);        ...     }    private boolean nextInternal(int limit) throws IOException {        if (isStopRow(currentRow)) {          ...          return false;        } else {          byte [] nextRow;          do {            this.storeHeap.next(results, limit - results.size())          } while (Bytes.equals(currentRow, nextRow =peekRow()));

          final boolean stopRow = isStopRow(nextRow);      }      ...    }

MVCC

Get next KV from StoreScanners

Page 37: HBaseCon 2012 | Learning HBase Internals - Lars Hofhansl, Salesforce

StoreScanner  public synchronized boolean next(…) {    ...    LOOP: while((kv = this.heap.peek()) != null) {      ScanQueryMatcher.MatchCode qcode = matcher.match(kv);      switch(qcode) {        case INCLUDE:          results.add(kv);          this.heap.next();          ...          continue;        case DONE_SCAN: ...        case SEEK_NEXT_ROW:          reseek(matcher.getKeyForNextRow(kv));          break;        case SEEK_NEXT_COL: ...        case SKIP:          this.heap.next();          break;        case SEEK_NEXT_USING_HINT:          KeyValue nextKV = matcher.getNextKeyHint(kv);          ...          reseek(nextKV);          break;        ...      }    }  }

Heap of Memstore/StoreFile scanners

What to do with the KV:•Versions•TTL•Deletes

Page 38: HBaseCon 2012 | Learning HBase Internals - Lars Hofhansl, Salesforce

Seek Hints

• Allow skipping many KVs without “touching”

• Seek-to(KV) instead of skip, skip, skip, …

• Used internally (for deletes, skipping older versions, TTL)

• Used by filters

Page 39: HBaseCon 2012 | Learning HBase Internals - Lars Hofhansl, Salesforce

Memstore Scanner

    public synchronized KeyValue next() {      if (theNext == null) {          return null;      }

      final KeyValue ret = theNext;

      // Advance one of the iterators      if (theNext == kvsetNextRow) {        kvsetNextRow = getNext(kvsetIt);      } else {        snapshotNextRow = getNext(snapshotIt);      }

      // Calculate the next value      theNext = getLowest(kvsetNextRow, snapshotNextRow);

      return ret;    }

Snapshot duringflushes

KeyValueSkipListSet.iterator()

Page 40: HBaseCon 2012 | Learning HBase Internals - Lars Hofhansl, Salesforce

Memstore Scanner

protected KeyValue getNext(Iterator<KeyValue> it) {

long readPoint = MVCC.getThreadReadPoint();

while (it.hasNext()) {

KeyValue v = it.next();

if (v.getMemstoreTS() <= readPoint) {

return v;

}

}

return null;

}

MVCC

Page 41: HBaseCon 2012 | Learning HBase Internals - Lars Hofhansl, Salesforce

Memstore Scanner

    @Override    public synchronized boolean seek(KeyValue key) {      ...      // kvset and snapshot will never be null.      // if tailSet can't find anything, SortedSet is empty (not null).      kvTail = kvsetAtCreation.tailSet(key);      snapshotTail = snapshotAtCreation.tailSet(key);

      return seekInSubLists(key);    }

For seek find the right tailSet

seekInSubLists() almost identical to next()

Page 42: HBaseCon 2012 | Learning HBase Internals - Lars Hofhansl, Salesforce

StoreFileScanner

  private final HFileScanner hfs;  private KeyValue cur = null;  ...   public KeyValue next() throws IOException {    KeyValue retKey = cur;

    try {      // only seek if we aren't at the end.

// cur == null implies 'end'.      if (cur != null) {        hfs.next();        cur = hfs.getKeyValue();        skipKVsNewerThanReadpoint();      }    } catch(IOException e) {      throw new IOException("Could not iterate " + this, e);    }    return retKey;  }

Page 43: HBaseCon 2012 | Learning HBase Internals - Lars Hofhansl, Salesforce

HFileScanner/Reader

    @Override    public boolean next() throws IOException {      ...      blockBuffer.position(...);      ...      if (blockBuffer.remaining() <= 0) {        long lastDataBlockOffset = reader.getTrailer().getLastDataBlockOffset();

        // read the next block        HFileBlock nextBlock = readNextDataBlock();        if (nextBlock == null) {          return false;        }

        updateCurrBlock(nextBlock);        return true;      }

      // We are still in the same block.      readKeyValueLen();      return true;    }

Still on current block?

If not, read the next block

Mark the next KV in the buffer

Page 44: HBaseCon 2012 | Learning HBase Internals - Lars Hofhansl, Salesforce

Puts

Page 45: HBaseCon 2012 | Learning HBase Internals - Lars Hofhansl, Salesforce

Batch Put•   private long doMiniBatchPut(BatchOperationInProgress<…> batchOp){

    WALEdit walEdit = new WALEdit();    ...    MultiVersionConsistencyControl.WriteEntry w = null;    ...    try { // STEP 1. Try to acquire as many locks as we can

// STEP 2. Update any LATEST_TIMESTAMP timestamps

 

    // Acquire the latest mvcc number      w = mvcc.beginMemstoreInsert();

      // STEP 3. Write back to memstore      for (int i = firstIndex; i < lastIndexExclusive; i++) {

        addedSize += applyFamilyMapToMemstore(familyMaps[i], w);      }

      // STEP 4. Build WAL edit      for (int i = firstIndex; i < lastIndexExclusive; i++) {        addFamilyMapToWALEdit(familyMaps[i], walEdit);      }

     

Begin transaction

Page 46: HBaseCon 2012 | Learning HBase Internals - Lars Hofhansl, Salesforce

Batch Put, cont.      // STEP 5. Append the edit to WAL. Do not sync wal.      txid = this.log.appendNoSync(regionInfo, …, walEdit, …);

      // STEP 6. Release row locks, etc.      this.updatesLock.readLock().unlock();      for (Integer toRelease : acquiredLocks) {         releaseRowLock(toRelease);      } // STEP 7. Sync wal.      this.log.sync(txid);

      walSyncSuccessful = true;      // STEP 8. Advance mvcc. // This will make this put visible to scanners and getters.      if (w != null) {        mvcc.completeMemstoreInsert(w);        w = null;      }      ...      return addedSize;    }

Write WAL recordbut don’t sync!

Sync after locks are released

Commit

Guard against concurrent flushes

Page 47: HBaseCon 2012 | Learning HBase Internals - Lars Hofhansl, Salesforce

Batch Put, something went wrong

} finally {      if (!walSyncSuccessful) {        rollbackMemstore(batchOp, familyMaps, firstIndex, lastIndexExclusive);      }      if (w != null) mvcc.completeMemstoreInsert(w);

      if (locked) {        this.updatesLock.readLock().unlock();      }

      for (Integer toRelease : acquiredLocks) {        releaseRowLock(toRelease);      }     ...    }

Always completethe transaction!

Page 48: HBaseCon 2012 | Learning HBase Internals - Lars Hofhansl, Salesforce

Memstore changes

  private long applyFamilyMapToMemstore(    Map<byte[], List<KeyValue>> familyMap,    WriteEntry localizedWriteEntry) {    long size = 0;    boolean freemvcc = false;

    try {      if (localizedWriteEntry == null) {        localizedWriteEntry = mvcc.beginMemstoreInsert();        freemvcc = true;      }      for (Map.Entry<…> e : familyMap.entrySet()) {        byte[] family = e.getKey();        List<KeyValue> edits = e.getValue();        ...

Can pass a write entrythat spans mutliple calls

This begins thetransaction(MVCC)

Page 49: HBaseCon 2012 | Learning HBase Internals - Lars Hofhansl, Salesforce

Memstore changes

        ...        Store store = getStore(family);        for (KeyValue kv: edits) {          kv.setMemstoreTS(localizedWriteEntry.getWriteNumber());          size += store.add(kv);        }      }    } finally {      if (freemvcc) {        mvcc.completeMemstoreInsert(localizedWriteEntry);      }    }     return size;   }

Tag KV with write number (MVCC)

This makes the changes visible

Page 50: HBaseCon 2012 | Learning HBase Internals - Lars Hofhansl, Salesforce

Deletes

Page 51: HBaseCon 2012 | Learning HBase Internals - Lars Hofhansl, Salesforce

ScanDeleteTracker (checking markers)  public void add(buffer, qualifierOffset, qualifierLength, ts, type) {    if (!hasFamilyStamp || ts > familyStamp) {      if (type == KeyValue.Type.DeleteFamily.getCode()) {        hasFamilyStamp = true;        familyStamp = ts;        return;      }

      if (deleteBuffer != null && type < deleteType) {        // same column, so ignore less specific delete        if (Bytes.equals(deleteBuffer, deleteOffset, deleteLength,            buffer, qualifierOffset, qualifierLength)){          return;        }      }      // new column, or more general delete type      deleteBuffer = buffer;      deleteOffset = qualifierOffset;      deleteLength = qualifierLength;      deleteType = type;      deleteTimestamp = ts;    }      }

Only remember TS for family deletes

A version delete marker can be ignored if there is a column marker already.

Remember the KV

Page 52: HBaseCon 2012 | Learning HBase Internals - Lars Hofhansl, Salesforce

ScanDeleteTracker (checking markers)  public DeleteResult isDeleted(buffer, qualifierOffset,qualifierLength, timestamp) {    if (hasFamilyStamp && timestamp <= familyStamp) {      return DeleteResult.FAMILY_DELETED;    }    if (deleteBuffer != null) {      int ret = Bytes.compareTo(deleteBuffer, deleteOffset, deleteLength,          buffer, qualifierOffset, qualifierLength);      if (ret == 0) {        if (deleteType == KeyValue.Type.DeleteColumn.getCode()) {          return DeleteResult.COLUMN_DELETED;        }        // If the timestamp is the same, keep this one        if (timestamp == deleteTimestamp) {          return DeleteResult.VERSION_DELETED;        }        // different timestamp, let's clear the buffer.        deleteBuffer = null;      } else if(ret < 0){        // Next column case.        deleteBuffer = null;      } else {        throw new IllegalStateException(...);      }    }    return DeleteResult.NOT_DELETED;  }

Family marker case

Column marker case

Version marker case

HFiles scanned newest TS first

Page 53: HBaseCon 2012 | Learning HBase Internals - Lars Hofhansl, Salesforce

Questions?Comments?

More details on http://hadoop-hbase.blogspot.com