Nicolas: Thanks for your insight. Can you point Mikael to a few of the JIRAs where algorithm mentioned in #1 was implemented ?
On Mon, Jan 9, 2012 at 10:55 AM, Nicolas Spiegelberg <nspiegelb...@fb.com>wrote: > Mikael, > > Hi, I wrote the current compaction algorithm, so I should be able to > answer most questions that you have about the feature. It sounds like > you're creating quite a task list of work to do, but I don't understand > what your use case is so a lot of that work may be not be critical and you > can leverage existing functionality. A better description of your system > requirements is a must to getting a good solution. > > 1. Major compactions are triggered by 3 methods: user issued, timed, and > size-based. You are probably hitting size-based compactions where your > config is disabling time-based compactions. Minor compactions are issued > on a size-based threshold. The algorithm sees if sum(file[0:i] * ratio) > > file[i+1] and includes file[0:i+1] if so. This is a reverse iteration, so > the highest 'i' value is used. If all files match, then you can remove > delete markers [which is the difference between a major and minor > compaction]. Major compactions aren't a bad or time-intensive thing, it's > just delete marker removal. > > As a note, we use timed majors in an OLTP production environment. They > are less useful if you're doing bulk imports or have an OLAP environment > where you're either running a read-intensive test or the cluster is idle. > In that case, it's definitely best to disable compactions and run them > when you're not using the cluster very much. > > 2. See HBASE-4418 for showing all configuration options in the Web UI. > This is in 0.92 however. > > 4. The compaction queue shows compactions that are waiting to happen. If > you invoke a compaction and the queue is empty, the thread will > immediately pick up your request and the queue will remain empty. > > 8. A patch for pluggable compactions had been thrown up in the past. It > was not well-tested and the compaction algorithm was undergoing major > design changes at the time that clashed with the patch. I think it's been > a low priority because there are many other ways to get big performance > wins from HBase outside of pluggable compactions. Most people don't > understand how to optimize the current algorithm, which is well-known > (very similar to BigTable's). I think bigger wins can come from correctly > laying out a good schema and understanding the config knobs currently at > our disposal. > > > > On 1/8/12 7:25 AM, "Mikael Sitruk" <mikael.sit...@gmail.com> wrote: > > >Hi > > > > > > > >I have some concern regarding major compactions below... > > > > > > 1. According to best practices from the mailing list and from the book, > > automatic major compaction should be disabled. This can be done by > >setting > > the property Œhbase.hregion.majorcompaction¹ to Œ0¹. Neverhteless even > > after having doing this I STILL see ³major compaction² messages in > >logs. > > therefore it is unclear how can I manage major compactions. (The > >system has > > heavy insert - uniformly on the cluster, and major compaction affect > >the > > performance of the system). > > If I'm not wrong it seems from the code that: even if not requested and > > even if the indicator is set to '0' (no automatic major compaction), > >major > > compaction can be triggered by the code in case all store files are > > candidate for a compaction (from Store.compact(final boolean > >forceMajor)). > > Shouldn't the code add a condition that automatic major compaction is > > disabled?? > > > > 2. I tried to check the parameter Œhbase.hregion.majorcompaction¹ at > > runtime using several approaches - to validate that the server indeed > > loaded the parameter. > > > >a. Using a connection created from local config > > > >*conn = (HConnection) HConnectionManager.getConnection(m_hbConfig);* > > > >*conn.getConfiguration().getString(³hbase.hregion.majorcompaction²)* > > > >returns the parameter from local config and not from cluster. Is it a bug? > >If I set the property via the configuration shouldn¹t all the cluster be > >aware of? (supposing that the connection indeed connected to the cluster) > > > >b. fetching the property from the table descriptor > > > >*HTableDescriptor hTableDescriptor = > >conn.getHTableDescriptor(Bytes.toBytes("my table"));* > > > >*hTableDescriptor.getValue("hbase.hregion.majorcompaction")* > > > >This will returns the default parameter value (1 day) not the parameter > >from the configuration (on the cluster). It seems to be a bug, isn¹t it? > >(the parameter from the config, should be the default if not set at the > >table level) > > > >c. The only way I could set the parameter to 0 and really see it is via > >the > >Admin API, updating the table descriptor or the column descriptor. Now I > >could see the parameter on the web UI. So is it the only way to set > >correctly the parameter? If setting the parameter via the configuration > >file, shouldn¹t the webUI show this on any table created? > > > >d. I tried also to setup the parameter via hbase shell but setting such > >properties is not supported. (do you plan to add such support via the > >shell?) > > > >e. Generally is it possible to get via API the configuration used by the > >servers? (at cluster/server level) > > > > 3. I ran both major compaction requests from the shell or from API > >but since both are async there is no progress indication. Neither the JMX > >nor the Web will help here since you don¹t know if a compaction task is > >running. Tailling the logs is not an efficient way to do this neither. The > >point is that I would like to automate the process and avoid compaction > >storm. So I want to do that region, region, but if I don¹t know when a > >compaction started/ended I can¹t automate it. > > > >4. In case there is no compaction files in queue (but still you have > >more than 1 storefile per store e.g. minor compaction just finished) then > >invoking major_compact will indeed decrease the number of store files, but > >the compaction queue will remain to 0 during the compaction task > >(shouldn¹t > >the compaction queue increase by the number of file to compact and be > >reduced when the task ended?) > > > > > >5. I saw already HBASE-3965 for getting status of major compaction, > >nevertheless it has be removed from 0.92, is it possible to put it back? > >Even sooner than 0.92? > > > >6. In case a compaction (major) is running it seems there is no way > >to stop-it. Do you plan to add such feature? > > > >7. Do you plan to add functionality via JMX (starting/stopping > >compaction, splitting....) > > > >8. Finally there were some request for allowing custom compaction, > >part of this was given via the RegionObserver in HBASE-2001, nevertheless > >do you consider adding support for custom compaction (providing real > >pluggable compaction stategy not just observer)? > > > > > >Regards, > >Mikael.S > >