[ 
https://issues.apache.org/jira/browse/HBASE-1460?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12714800#action_12714800
 ] 

Jonathan Gray edited comment on HBASE-1460 at 5/30/09 5:44 PM:
---------------------------------------------------------------

My first go at {{ConcurrentLRUBlockCache}}.

Two new data structures are introduced in this patch, only one of which is used 
in the patch.

I have added {{BoundedPriorityQueue}} which is not being used in the block 
cache but I built early on thinking I might need it.  Java's PriorityQueue is 
unbounded, so it maintains sorted order but always adds new elements.  A 
bounded queue will grow until it reaches the maximum size (specified on 
construction) and then only adds new items that are larger than the current 
smallest.  Though maybe not now, this can be used for the KeyValueHeap where we 
really want a bounded queue not an unbounded queue.  This opens the door to 
some algorithmic improvements.

This patch also adds {{CachedBlockQueue}} which is also a "bounded priority 
queue" but rather than being bounded on the number of total elements, it's 
bounded on total heap size of the elements.  From comments:

{noformat} 
  /**
   * Initialize a memory-bound queue that will grow until an element brings
   * total size >= maxSize.  From then on, only entries that are sorted larger
   * than the smallest current entry will be inserted/replaced.
   * 
   * <p>Use this when you want to find the largest elements (according to their
   * ordering, not their heap size) that consume as close to the specified
   * maxSize as possible.  Default behavior is to grow just above rather than
   * just below specified max.
   * 
   * <p>Object used in this queue must implement {...@link HeapSize} as well as
   * {...@link Comparable}.
   *
   * @param maxSize the target size of elements in the queue
   */
{noformat} 

On to the LRU cache, this implementation contains three levels of block 
priority.  Single access, multiple access, and in-memory.  Single/multiple is 
to provide a level of scan-resistance to the cache, and in-memory is to give 
blocks from in-memory families a separate chunk of the cache to work with.

The cache is instantiated with its total size, in bytes, and each of the three 
priorities are allocated a portion of that total size.  When evicting, those 
priorities whose blocks have grown past their allotted size will have evictions 
and those who have not will have no elements evicted.  However, if any of the 
priorities are not using their entire portion, other priorities will be allowed 
to grow larger to make use of the total size of the cache.  The eviction is 
where this logic exists.

In addition to the max size, there are two other size markers:  acceptable and 
minimum.  They work as such:  If totalSizeInCache > acceptable, evict elements 
until totalSizeInCache < minimum.  The different between max and acceptable 
must be tweaked as such that it leaves sufficient time for the eviction process 
to run.

Eviction happens in a separate thread, so users of this cache are never 
interrupted.  All used structures are from java's concurrent package so there 
is no synchronization within the cache except to ensure only a single eviction 
is processed at any given time.

The two new data structures outside of the LRU in this patch are tested and 
include unit tests.  The LRU itself currently has no unit tests, I'm working on 
that now, just thought I'd get a patch up so others can see what I'm doing here.

(edit just changed comment to noformat)

      was (Author: streamy):
    My first go at {{ConcurrentLRUBlockCache}}.

Two new data structures are introduced in this patch, only one of which is used 
in the patch.

I have added {{BoundedPriorityQueue}} which is not being used in the block 
cache but I built early on thinking I might need it.  Java's PriorityQueue is 
unbounded, so it maintains sorted order but always adds new elements.  A 
bounded queue will grow until it reaches the maximum size (specified on 
construction) and then only adds new items that are larger than the current 
smallest.  Though maybe not now, this can be used for the KeyValueHeap where we 
really want a bounded queue not an unbounded queue.  This opens the door to 
some algorithmic improvements.

This patch also adds {{CachedBlockQueue}} which is also a "bounded priority 
queue" but rather than being bounded on the number of total elements, it's 
bounded on total heap size of the elements.  From comments:

{quote}
  /**
   * Initialize a memory-bound queue that will grow until an element brings
   * total size >= maxSize.  From then on, only entries that are sorted larger
   * than the smallest current entry will be inserted/replaced.
   * 
   * <p>Use this when you want to find the largest elements (according to their
   * ordering, not their heap size) that consume as close to the specified
   * maxSize as possible.  Default behavior is to grow just above rather than
   * just below specified max.
   * 
   * <p>Object used in this queue must implement {...@link HeapSize} as well as
   * {...@link Comparable}.
   *
   * @param maxSize the target size of elements in the queue
   */
{quote}

On to the LRU cache, this implementation contains three levels of block 
priority.  Single access, multiple access, and in-memory.  Single/multiple is 
to provide a level of scan-resistance to the cache, and in-memory is to give 
blocks from in-memory families a separate chunk of the cache to work with.

The cache is instantiated with its total size, in bytes, and each of the three 
priorities are allocated a portion of that total size.  When evicting, those 
priorities whose blocks have grown past their allotted size will have evictions 
and those who have not will have no elements evicted.  However, if any of the 
priorities are not using their entire portion, other priorities will be allowed 
to grow larger to make use of the total size of the cache.  The eviction is 
where this logic exists.

In addition to the max size, there are two other size markers:  acceptable and 
minimum.  They work as such:  If totalSizeInCache > acceptable, evict elements 
until totalSizeInCache < minimum.  The different between max and acceptable 
must be tweaked as such that it leaves sufficient time for the eviction process 
to run.

Eviction happens in a separate thread, so users of this cache are never 
interrupted.  All used structures are from java's concurrent package so there 
is no synchronization within the cache except to ensure only a single eviction 
is processed at any given time.

The two new data structures outside of the LRU in this patch are tested and 
include unit tests.  The LRU itself currently has no unit tests, I'm working on 
that now, just thought I'd get a patch up so others can see what I'm doing here.
  
> Concurrent LRU Block Cache
> --------------------------
>
>                 Key: HBASE-1460
>                 URL: https://issues.apache.org/jira/browse/HBASE-1460
>             Project: Hadoop HBase
>          Issue Type: Improvement
>          Components: io
>            Reporter: Jonathan Gray
>            Assignee: Jonathan Gray
>             Fix For: 0.20.0
>
>         Attachments: HBASE-1460-v1.patch
>
>
> The LRU-based block cache that will be committed in HBASE-1192 is thread-safe 
> but contains a big lock on the hash map.  Under high load, the block cache 
> will be hit very heavily from a number of threads, so it needs to be built to 
> handle massive concurrency.
> This issue aims to implement a new block cache with LRU eviction, but backed 
> by a ConcurrentHashMap and a separate eviction thread.  Influence will be 
> drawn from Solr's ConcurrentLRUCache, however there are major differences 
> because solr treats all cached elements as equal size whereas we are 
> dependent on our HeapSize interface with realistic (though approximate) heap 
> usage.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.

Reply via email to