[ https://issues.apache.org/jira/browse/HBASE-14468?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15049310#comment-15049310 ]
stack commented on HBASE-14468: ------------------------------- This test hung just now in a 1.3 build: kalashnikov:hbase.git stack$ !520 python ./dev-support/findHangingTests.py https://builds.apache.org/job/HBase-1.3/jdk=latest1.8,label=Hadoop/425/consoleText Fetching https://builds.apache.org/job/HBase-1.3/jdk=latest1.8,label=Hadoop/425/consoleText Building remotely on H1 (Mapreduce Hadoop Pig Hdfs) in workspace /home/jenkins/jenkins-slave/workspace/HBase-1.3/jdk/latest1.8/label/Hadoop Printing hanging tests Hanging test : org.apache.hadoop.hbase.regionserver.compactions.TestFIFOCompactionPolicy It looks like it is stuck waiting on a server to show up. https://builds.apache.org/job/HBase-1.3/jdk=latest1.8,label=Hadoop/425/artifact/hbase-server/target/surefire-reports/org.apache.hadoop.hbase.regionserver.compactions.TestFIFOCompactionPolicy-output.txt Please take a look see when you get a chance. Thanks. > Compaction improvements: FIFO compaction policy > ----------------------------------------------- > > Key: HBASE-14468 > URL: https://issues.apache.org/jira/browse/HBASE-14468 > Project: HBase > Issue Type: Improvement > Reporter: Vladimir Rodionov > Assignee: Vladimir Rodionov > Fix For: 2.0.0, 1.2.0, 1.3.0 > > Attachments: HBASE-14468-v1.patch, HBASE-14468-v10.patch, > HBASE-14468-v2.patch, HBASE-14468-v3.patch, HBASE-14468-v4.patch, > HBASE-14468-v5.patch, HBASE-14468-v6.patch, HBASE-14468-v7.patch, > HBASE-14468-v8.patch, HBASE-14468-v9.patch > > > h2. FIFO Compaction > h3. Introduction > FIFO compaction policy selects only files which have all cells expired. The > column family MUST have non-default TTL. > Essentially, FIFO compactor does only one job: collects expired store files. > These are some applications which could benefit the most: > # Use it for very high volume raw data which has low TTL and which is the > source of another data (after additional processing). Example: Raw > time-series vs. time-based rollup aggregates and compacted time-series. We > collect raw time-series and store them into CF with FIFO compaction policy, > periodically we run task which creates rollup aggregates and compacts > time-series, the original raw data can be discarded after that. > # Use it for data which can be kept entirely in a a block cache (RAM/SSD). > Say we have local SSD (1TB) which we can use as a block cache. No need for > compaction of a raw data at all. > Because we do not do any real compaction, we do not use CPU and IO (disk and > network), we do not evict hot data from a block cache. The result: improved > throughput and latency both write and read. > See: https://github.com/facebook/rocksdb/wiki/FIFO-compaction-style > h3. To enable FIFO compaction policy > For table: > {code} > HTableDescriptor desc = new HTableDescriptor(tableName); > > desc.setConfiguration(DefaultStoreEngine.DEFAULT_COMPACTION_POLICY_CLASS_KEY, > FIFOCompactionPolicy.class.getName()); > {code} > For CF: > {code} > HColumnDescriptor desc = new HColumnDescriptor(family); > > desc.setConfiguration(DefaultStoreEngine.DEFAULT_COMPACTION_POLICY_CLASS_KEY, > FIFOCompactionPolicy.class.getName()); > {code} > Although region splitting is supported, for optimal performance it should be > disabled, either by setting explicitly DisabledRegionSplitPolicy or by > setting ConstantSizeRegionSplitPolicy and very large max region size. You > will have to increase to a very large number store's blocking file number : > *hbase.hstore.blockingStoreFiles* as well. > > h3. Limitations > Do not use FIFO compaction if : > * Table/CF has MIN_VERSION > 0 > * Table/CF has TTL = FOREVER (HColumnDescriptor.DEFAULT_TTL) -- This message was sent by Atlassian JIRA (v6.3.4#6332)