Return-Path: Delivered-To: apmail-hadoop-hbase-dev-archive@minotaur.apache.org Received: (qmail 59301 invoked from network); 31 May 2009 00:46:21 -0000 Received: from hermes.apache.org (HELO mail.apache.org) (140.211.11.3) by minotaur.apache.org with SMTP; 31 May 2009 00:46:21 -0000 Received: (qmail 82494 invoked by uid 500); 31 May 2009 00:46:33 -0000 Delivered-To: apmail-hadoop-hbase-dev-archive@hadoop.apache.org Received: (qmail 82426 invoked by uid 500); 31 May 2009 00:46:33 -0000 Mailing-List: contact hbase-dev-help@hadoop.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: hbase-dev@hadoop.apache.org Delivered-To: mailing list hbase-dev@hadoop.apache.org Received: (qmail 82414 invoked by uid 99); 31 May 2009 00:46:33 -0000 Received: from nike.apache.org (HELO nike.apache.org) (192.87.106.230) by apache.org (qpsmtpd/0.29) with ESMTP; Sun, 31 May 2009 00:46:33 +0000 X-ASF-Spam-Status: No, hits=-2000.0 required=10.0 tests=ALL_TRUSTED X-Spam-Check-By: apache.org Received: from [140.211.11.140] (HELO brutus.apache.org) (140.211.11.140) by apache.org (qpsmtpd/0.29) with ESMTP; Sun, 31 May 2009 00:46:29 +0000 Received: from brutus (localhost [127.0.0.1]) by brutus.apache.org (Postfix) with ESMTP id 1FE93234C004 for ; Sat, 30 May 2009 17:46:08 -0700 (PDT) Message-ID: <1682335684.1243730768117.JavaMail.jira@brutus> Date: Sat, 30 May 2009 17:46:08 -0700 (PDT) From: "Jonathan Gray (JIRA)" To: hbase-dev@hadoop.apache.org Subject: [jira] Issue Comment Edited: (HBASE-1460) Concurrent LRU Block Cache In-Reply-To: <228517078.1243729807357.JavaMail.jira@brutus> MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: 7bit X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 X-Virus-Checked: Checked by ClamAV on apache.org [ 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. * *

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. * *

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. * *

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. * *

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.