From 03daf83720d37c3967f7fc5478b58d853a234e2d Mon Sep 17 00:00:00 2001 From: Clara Date: Sat, 5 Mar 2016 23:32:31 -0800 Subject: [PATCH] HBASE-15400 Using Multiple Output for Date Tiered Compaction --- .../regionserver/AbstractMultiFileWriter.java | 128 +++++++++ .../regionserver/DateTieredMultiFileWriter.java | 82 ++++++ .../hbase/regionserver/DateTieredStoreEngine.java | 89 ++++++ .../hadoop/hbase/regionserver/StoreFile.java | 21 ++ .../hbase/regionserver/StripeMultiFileWriter.java | 135 ++++----- .../hbase/regionserver/StripeStoreFlusher.java | 2 +- .../compactions/DateTieredCompactionPolicy.java | 311 ++++++++++++++++----- .../compactions/DateTieredCompactionRequest.java | 35 +++ .../compactions/DateTieredCompactor.java | 145 ++++++++++ .../compactions/RatioBasedCompactionPolicy.java | 4 +- .../regionserver/TestDateTieredCompaction.java | 199 ++----------- .../TestDateTieredCompactionPolicy.java | 272 ++++++++++++++++++ 12 files changed, 1104 insertions(+), 319 deletions(-) create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/AbstractMultiFileWriter.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DateTieredMultiFileWriter.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DateTieredStoreEngine.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DateTieredCompactionRequest.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DateTieredCompactor.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDateTieredCompactionPolicy.java diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/AbstractMultiFileWriter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/AbstractMultiFileWriter.java new file mode 100644 index 0000000..9d4bdf9 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/AbstractMultiFileWriter.java @@ -0,0 +1,128 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.regionserver; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.Map; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.CellComparator; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.regionserver.compactions.Compactor.CellSink; + +import com.google.common.collect.Iterators; +import com.google.common.collect.PeekingIterator; + +/** + * Base class for cell sink that separates the provided cells into multiple files. + */ +@InterfaceAudience.Private +public abstract class AbstractMultiFileWriter implements CellSink { + + private static final Log LOG = LogFactory.getLog(StripeMultiFileWriter.class); + + /** Factory that is used to produce single StoreFile.Writer-s */ + protected WriterFactory writerFactory; + protected CellComparator comparator; + + /** Source scanner that is tracking KV count; may be null if source is not StoreScanner */ + protected StoreScanner sourceScanner; + + /** Whether to write stripe metadata */ + private boolean doWriteExtraMetadata = true; + + public interface WriterFactory { + public StoreFile.Writer createWriter() throws IOException; + } + + /** + * Initializes multi-writer before usage. + * @param sourceScanner Optional store scanner to obtain the information about read progress. + * @param factory Factory used to produce individual file writers. + * @param comparator Comparator used to compare rows. + */ + public void init(StoreScanner sourceScanner, WriterFactory factory, CellComparator comparator) + throws IOException { + this.writerFactory = factory; + this.sourceScanner = sourceScanner; + this.comparator = comparator; + } + + public void setNoExtraMetadata() { + this.doWriteExtraMetadata = false; + } + + public List commitWriters(long maxSeqId, boolean isMajor) throws IOException { + commitWritersInternal(); + LOG.debug((this.doWriteExtraMetadata ? "W" : "Not w") + "riting out metadata for " + + this.writers().size() + " writers"); + List paths = new ArrayList(); + for (PeekingIterator> iter = Iterators + .peekingIterator(boundaryAndWriters().iterator());;) { + Map.Entry entry = iter.next(); + if (!iter.hasNext()) { + // this is the last boundary, just break + break; + } + StoreFile.Writer writer = entry.getValue(); + if (writer == null) { + continue; + } + if (doWriteExtraMetadata) { + writeExtraMetadata(writer, entry.getKey(), iter.peek().getKey()); + } + writer.appendMetadata(maxSeqId, isMajor); + paths.add(writer.getPath()); + writer.close(); + } + return paths; + } + + public List abortWriters() { + List paths = new ArrayList(); + for (StoreFile.Writer writer : writers()) { + try { + if (writer != null) { + paths.add(writer.getPath()); + writer.close(); + } + } catch (Exception ex) { + LOG.error("Failed to close the writer after an unfinished compaction.", ex); + } + } + return paths; + } + + protected abstract Iterable> boundaryAndWriters(); + + protected abstract Collection writers(); + + /** + * Subclasses override this method to be called at the end of a successful sequence of append; all + * appends are processed before this method is called. + */ + protected abstract void commitWritersInternal() throws IOException; + + protected abstract void writeExtraMetadata(StoreFile.Writer writer, K lowerBoundary, + K higherBoundary) throws IOException; +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DateTieredMultiFileWriter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DateTieredMultiFileWriter.java new file mode 100644 index 0000000..38d81f6 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DateTieredMultiFileWriter.java @@ -0,0 +1,82 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.regionserver; + +import java.io.IOException; +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.NavigableMap; + +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.regionserver.StoreFile.Writer; + +/** + * class for cell sink that separates the provided cells into multiple files for date tiered + * compaction. + */ +@InterfaceAudience.Private +public class DateTieredMultiFileWriter extends AbstractMultiFileWriter { + /** + * Usually the boundaries.size = writers.size + 1, so use {@link NavigableMap#floorEntry(Object)} + * to get writer. And when commit, you should know that the last value must be null. + */ + protected NavigableMap boundary2Writer; + + public DateTieredMultiFileWriter(List boundaries) { + for (Long boundary: boundaries) { + boundary2Writer.put(boundary, null); + } + } + + @Override + public void append(Cell cell) throws IOException { + Map.Entry entry = boundary2Writer.floorEntry(cell.getTimestamp()); + if (entry == null || entry.getKey().equals(boundary2Writer.lastKey())) { + throw new IndexOutOfBoundsException(entry + " is out of range, should in range [" + + boundary2Writer.firstKey() + ", " + boundary2Writer.lastKey() + ")"); + } + StoreFile.Writer writer = entry.getValue(); + if (writer == null) { + writer = writerFactory.createWriter(); + entry.setValue(writer); + } + writer.append(cell); + } + + @Override + protected Iterable> boundaryAndWriters() { + return boundary2Writer.entrySet(); + } + + @Override + protected Collection writers() { + return boundary2Writer.headMap(boundary2Writer.lastKey(), false).values(); + } + + @Override + protected void commitWritersInternal() throws IOException { + } + + @Override + protected void writeExtraMetadata(Writer writer, Long lowerBoundary, Long higherBoundary) + throws IOException { + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DateTieredStoreEngine.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DateTieredStoreEngine.java new file mode 100644 index 0000000..bd80752 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DateTieredStoreEngine.java @@ -0,0 +1,89 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.regionserver; + +import java.io.IOException; +import java.util.List; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.CellComparator; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext; +import org.apache.hadoop.hbase.regionserver.compactions.DateTieredCompactionPolicy; +import org.apache.hadoop.hbase.regionserver.compactions.DateTieredCompactionRequest; +import org.apache.hadoop.hbase.regionserver.compactions.DateTieredCompactor; +import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController; +import org.apache.hadoop.hbase.security.User; + +/** + * + */ +@InterfaceAudience.Private +public class DateTieredStoreEngine extends + StoreEngine { + + @Override + public boolean needsCompaction(List filesCompacting) { + return compactionPolicy.needsCompaction(storeFileManager.getStorefiles(), filesCompacting); + } + + @Override + public CompactionContext createCompaction() throws IOException { + return new DateTieredCompactionContext(); + } + + @Override + protected void createComponents(Configuration conf, Store store, CellComparator kvComparator) + throws IOException { + this.compactionPolicy = new DateTieredCompactionPolicy(conf, store); + this.storeFileManager = new DefaultStoreFileManager(kvComparator, conf, + compactionPolicy.getConf()); + this.storeFlusher = new DefaultStoreFlusher(conf, store); + this.compactor = new DateTieredCompactor(conf, store); + } + + private final class DateTieredCompactionContext extends CompactionContext { + + @Override + public List preSelect(List filesCompacting) { + return compactionPolicy.preSelectCompactionForCoprocessor(storeFileManager.getStorefiles(), + filesCompacting); + } + + @Override + public boolean select(List filesCompacting, boolean isUserCompaction, + boolean mayUseOffPeak, boolean forceMajor) throws IOException { + request = compactionPolicy.selectCompaction(storeFileManager.getStorefiles(), filesCompacting, + isUserCompaction, mayUseOffPeak, forceMajor); + return request != null; + } + + @Override + public List compact(ThroughputController throughputController) throws IOException { + return compact(throughputController, null); + } + + @Override + public List compact(ThroughputController throughputController, User user) + throws IOException { + return compactor.compact(request, ((DateTieredCompactionRequest) request).getBoundaries(), + throughputController, user); + } + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java index 61eb9b8..97e497e 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java @@ -1765,11 +1765,25 @@ public class StoreFile { public static final Comparator SEQ_ID = Ordering.compound(ImmutableList.of( Ordering.natural().onResultOf(new GetSeqId()), + Ordering.natural().onResultOf(new GetMaxTimestamp()), Ordering.natural().onResultOf(new GetFileSize()).reverse(), Ordering.natural().onResultOf(new GetBulkTime()), Ordering.natural().onResultOf(new GetPathName()) )); + /** + * Comparator for time-aware compaction. SeqId is still the first + * ording criterion to maintain MVCC. + */ + public static final Comparator SEQ_ID_MAX_TIMESTAMP = + Ordering.compound(ImmutableList.of( + Ordering.natural().onResultOf(new GetSeqId()), + Ordering.natural().onResultOf(new GetMaxTimestamp()), + Ordering.natural().onResultOf(new GetFileSize()).reverse(), + Ordering.natural().onResultOf(new GetBulkTime()), + Ordering.natural().onResultOf(new GetPathName()) + )); + private static class GetSeqId implements Function { @Override public Long apply(StoreFile sf) { @@ -1804,5 +1818,12 @@ public class StoreFile { return sf.getPath().getName(); } } + + private static class GetMaxTimestamp implements Function { + @Override + public Long apply(StoreFile sf) { + return sf.getMaximumTimestamp(); + } + } } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeMultiFileWriter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeMultiFileWriter.java index 651b863..b251a21 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeMultiFileWriter.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeMultiFileWriter.java @@ -20,94 +20,95 @@ package org.apache.hadoop.hbase.regionserver; import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collection; import java.util.Collections; +import java.util.Iterator; import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.NoSuchElementException; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hbase.classification.InterfaceAudience; -import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.Cell; -import org.apache.hadoop.hbase.CellComparator; import org.apache.hadoop.hbase.CellUtil; -import org.apache.hadoop.hbase.regionserver.compactions.Compactor; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.regionserver.StoreFile.Writer; import org.apache.hadoop.hbase.util.Bytes; /** - * Base class for cell sink that separates the provided cells into multiple files. + * Base class for cell sink that separates the provided cells into multiple files for stripe + * compaction. */ @InterfaceAudience.Private -public abstract class StripeMultiFileWriter implements Compactor.CellSink { - private static final Log LOG = LogFactory.getLog(StripeMultiFileWriter.class); +public abstract class StripeMultiFileWriter extends AbstractMultiFileWriter { - /** Factory that is used to produce single StoreFile.Writer-s */ - protected WriterFactory writerFactory; - protected CellComparator comparator; + private static final Log LOG = LogFactory.getLog(StripeMultiFileWriter.class); protected List existingWriters; protected List boundaries; - /** Source scanner that is tracking KV count; may be null if source is not StoreScanner */ - protected StoreScanner sourceScanner; - - /** Whether to write stripe metadata */ - private boolean doWriteStripeMetadata = true; - - public interface WriterFactory { - public StoreFile.Writer createWriter() throws IOException; - } - - /** - * Initializes multi-writer before usage. - * @param sourceScanner Optional store scanner to obtain the information about read progress. - * @param factory Factory used to produce individual file writers. - * @param comparator Comparator used to compare rows. - */ - public void init(StoreScanner sourceScanner, WriterFactory factory, CellComparator comparator) - throws IOException { - this.writerFactory = factory; - this.sourceScanner = sourceScanner; - this.comparator = comparator; - } - public void setNoStripeMetadata() { - this.doWriteStripeMetadata = false; + @Override + protected Iterable> boundaryAndWriters() { + return new Iterable>() { + + @Override + public Iterator> iterator() { + return new Iterator>() { + + private int index; + + @Override + public boolean hasNext() { + return index < boundaries.size() - 1; + } + + @Override + public Entry next() { + if (!hasNext()) { + throw new NoSuchElementException(); + } + final byte[] boundary = boundaries.get(index); + final StoreFile.Writer writer = index < existingWriters.size() + ? existingWriters.get(index) : null; + return new Entry() { + + @Override + public Writer setValue(Writer value) { + throw new UnsupportedOperationException(); + } + + @Override + public Writer getValue() { + return writer; + } + + @Override + public byte[] getKey() { + return boundary; + } + }; + } + + @Override + public void remove() { + throw new UnsupportedOperationException(); + } + }; + } + }; } - public List commitWriters(long maxSeqId, boolean isMajor) throws IOException { - assert this.existingWriters != null; - commitWritersInternal(); - assert this.boundaries.size() == (this.existingWriters.size() + 1); - LOG.debug((this.doWriteStripeMetadata ? "W" : "Not w") - + "riting out metadata for " + this.existingWriters.size() + " writers"); - List paths = new ArrayList(); - for (int i = 0; i < this.existingWriters.size(); ++i) { - StoreFile.Writer writer = this.existingWriters.get(i); - if (writer == null) continue; // writer was skipped due to 0 KVs - if (doWriteStripeMetadata) { - writer.appendFileInfo(StripeStoreFileManager.STRIPE_START_KEY, this.boundaries.get(i)); - writer.appendFileInfo(StripeStoreFileManager.STRIPE_END_KEY, this.boundaries.get(i + 1)); - } - writer.appendMetadata(maxSeqId, isMajor); - paths.add(writer.getPath()); - writer.close(); - } - this.existingWriters = null; - return paths; + @Override + protected Collection writers() { + return existingWriters; } - public List abortWriters() { - assert this.existingWriters != null; - List paths = new ArrayList(); - for (StoreFile.Writer writer : this.existingWriters) { - try { - paths.add(writer.getPath()); - writer.close(); - } catch (Exception ex) { - LOG.error("Failed to close the writer after an unfinished compaction.", ex); - } - } - this.existingWriters = null; - return paths; + @Override + protected void writeExtraMetadata(Writer writer, byte[] lowerBoundary, byte[] higherBoundary) + throws IOException { + writer.appendFileInfo(StripeStoreFileManager.STRIPE_START_KEY, lowerBoundary); + writer.appendFileInfo(StripeStoreFileManager.STRIPE_END_KEY, higherBoundary); } /** diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeStoreFlusher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeStoreFlusher.java index 9a06a88..d8eab98 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeStoreFlusher.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeStoreFlusher.java @@ -127,7 +127,7 @@ public class StripeStoreFlusher extends StoreFlusher { public StripeMultiFileWriter createWriter() throws IOException { StripeMultiFileWriter writer = new StripeMultiFileWriter.SizeMultiWriter(1, Long.MAX_VALUE, OPEN_KEY, OPEN_KEY); - writer.setNoStripeMetadata(); + writer.setNoExtraMetadata(); return writer; } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DateTieredCompactionPolicy.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DateTieredCompactionPolicy.java index 9f65e6e..7e09000 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DateTieredCompactionPolicy.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DateTieredCompactionPolicy.java @@ -1,4 +1,5 @@ /** + * * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -38,6 +39,8 @@ import org.apache.hadoop.hbase.HBaseInterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.regionserver.StoreConfigInformation; import org.apache.hadoop.hbase.regionserver.StoreFile; +import org.apache.hadoop.hbase.regionserver.StoreUtils; +import org.apache.hadoop.hbase.regionserver.TimeRangeTracker; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.hbase.util.ReflectionUtils; @@ -48,12 +51,12 @@ import org.apache.hadoop.hbase.util.ReflectionUtils; * 1. Improve date-range-based scan by structuring store files in date-based tiered layout. * 2. Reduce compaction overhead. * 3. Improve TTL efficiency. - * Perfect fit for the use cases that: + * Perfect fit for the use cases that: * 1. has mostly date-based data write and scan and a focus on the most recent data. - * 2. never or rarely deletes data. Out-of-order writes are handled gracefully. Time range - * overlapping among store files is tolerated and the performance impact is minimized. Configuration - * can be set at hbase-site or overriden at per-table or per-column-famly level by hbase shell. - * Design spec is at + * 2. never or rarely deletes data. + * Out-of-order writes are handled gracefully. Time range overlapping among store files is + * tolerated and the performance impact is minimized. Configuration can be set at hbase-site + * or overriden at per-table or per-column-famly level by hbase shell. Design spec is at * https://docs.google.com/document/d/1_AmlNb2N8Us1xICsTeGDLKIqL6T-oHoRLZ323MG_uy8/ */ @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG) @@ -67,25 +70,19 @@ public class DateTieredCompactionPolicy extends RatioBasedCompactionPolicy { super(conf, storeConfigInfo); try { compactionPolicyPerWindow = - ReflectionUtils.instantiateWithCustomCtor(comConf.getCompactionPolicyForTieredWindow(), - new Class[] { Configuration.class, StoreConfigInformation.class }, new Object[] { conf, - storeConfigInfo }); + ReflectionUtils.instantiateWithCustomCtor(comConf.getCompactionPolicyForTieredWindow(), + new Class[] { Configuration.class, StoreConfigInformation.class }, new Object[] { conf, + storeConfigInfo }); } catch (Exception e) { throw new IOException("Unable to load configured compaction policy '" + comConf.getCompactionPolicyForTieredWindow() + "'", e); } } - @Override - public boolean isMajorCompaction(Collection filesToCompact) throws IOException { - // Never do major compaction unless forced - return false; - } - - @Override /** * Heuristics for guessing whether we need compaction. */ + @Override public boolean needsCompaction(final Collection storeFiles, final List filesCompacting) { return needsCompaction(storeFiles, filesCompacting, EnvironmentEdgeManager.currentTime()); @@ -94,15 +91,9 @@ public class DateTieredCompactionPolicy extends RatioBasedCompactionPolicy { @VisibleForTesting public boolean needsCompaction(final Collection storeFiles, final List filesCompacting, long now) { - if (!super.needsCompaction(storeFiles, filesCompacting)) { - return false; - } - ArrayList candidates = new ArrayList(storeFiles); - candidates = filterBulk(candidates); - candidates = skipLargeFiles(candidates, true); try { - candidates = applyCompactionPolicy(candidates, true, false, now); + candidates = applyCompactionPolicy(candidates, true, now).storeFileList; } catch (Exception e) { LOG.error("Can not check for compaction: ", e); return false; @@ -112,69 +103,186 @@ public class DateTieredCompactionPolicy extends RatioBasedCompactionPolicy { } /** - * Could return null if no candidates are found + * @param candidateFiles candidate files, ordered from oldest to newest by seqId. We rely on + * DefaultStoreFileManager to sort the files by seqId to guarantee contiguous compaction + * based on seqId for data consistency. + * @return subset copy of candidate list that meets compaction criteria + * @throws java.io.IOException for error */ @Override - public ArrayList applyCompactionPolicy(ArrayList candidates, - boolean mayUseOffPeak, boolean mayBeStuck) throws IOException { - return applyCompactionPolicy(candidates, mayUseOffPeak, mayBeStuck, - EnvironmentEdgeManager.currentTime()); + public CompactionRequest selectCompaction(Collection candidateFiles, + final List filesCompacting, final boolean isUserCompaction, + final boolean mayUseOffPeak, final boolean forceMajor) throws IOException { + return selectCompaction(candidateFiles, filesCompacting, isUserCompaction, + mayUseOffPeak, forceMajor, EnvironmentEdgeManager.currentTime()); + } + + @VisibleForTesting + public CompactionRequest selectCompaction(Collection candidateFiles, + final List filesCompacting, final boolean isUserCompaction, + final boolean mayUseOffPeak, final boolean forceMajor, long now) throws IOException { + // Preliminary compaction subject to filters + ArrayList candidateSelection = new ArrayList(candidateFiles); + candidateSelection = getCurrentEligibleFiles(candidateSelection, filesCompacting); + LOG.debug("Selecting compaction from " + candidateFiles.size() + " store files, " + + filesCompacting.size() + " compacting, " + candidateSelection.size() + " eligible, " + + storeConfigInfo.getBlockingFileCount() + " blocking"); + + // Force a major compaction if this is a user-requested major compaction, + // or if we do not have too many files to compact and this was requested + // as a major compaction. + // Or, if there are any references among the candidates. + boolean majorCompaction = + ((forceMajor && isUserCompaction) + || ((forceMajor || isMajorCompaction(candidateSelection)) && + (candidateSelection.size() < comConf + .getMaxFilesToCompact())) || StoreUtils.hasReferences(candidateSelection)); + + List boundaries; + if (!majorCompaction) { + // we're doing a minor compaction, let's see what files are applicable and + // determine the boundaries for output + Bucket bucket = applyCompactionPolicy(candidateSelection, mayUseOffPeak, now); + if (bucket == null) { + return null; + } + candidateSelection = bucket.storeFileList; + boundaries = getCompactionBoundariesForMinor(bucket, now); + } else { + // We will output files along all tiers for major compaction + boundaries = getCompactBoundariesForMajor(candidateSelection, now); + } + + DateTieredCompactionRequest result = new DateTieredCompactionRequest(candidateSelection, + boundaries); + result.setOffPeak(!candidateSelection.isEmpty() && !majorCompaction && mayUseOffPeak); + return result; } /** * Input candidates are sorted from oldest to newest by seqId. Could return null if no candidates * are found. */ - @VisibleForTesting - public ArrayList applyCompactionPolicy(ArrayList candidates, - boolean mayUseOffPeak, boolean mayBeStuck, long now) throws IOException { + private Bucket applyCompactionPolicy(ArrayList candidates, boolean mayUseOffPeak, + long now) throws IOException { Iterable candidatesInWindow = filterOldStoreFiles(Lists.newArrayList(candidates), comConf.getMaxStoreFileAgeMillis(), now); + long oldestToCompact = getOldestToCompact(comConf.getMaxStoreFileAgeMillis(), now); - List> buckets = + List buckets = partitionFilesToBuckets(candidatesInWindow, comConf.getBaseWindowMillis(), - comConf.getWindowsPerTier(), now); + comConf.getWindowsPerTier(), oldestToCompact, now); LOG.debug("Compaction buckets are: " + buckets); - if (buckets.size() >= storeConfigInfo.getBlockingFileCount()) { - LOG.warn("Number of compaction buckets:" + buckets.size() - + ", exceeds blocking file count setting: " - + storeConfigInfo.getBlockingFileCount() - + ", either increase hbase.hstore.blockingStoreFiles or " - + "reduce the number of tiered compaction windows"); + if (buckets.size() >= storeConfigInfo.getBlockingFileCount()) { + LOG.warn("Number of compaction buckets:" + buckets.size() + + ", exceeds blocking file count setting: " + storeConfigInfo.getBlockingFileCount() + + ", either increase hbase.hstore.blockingStoreFiles or " + + "reduce the number of tiered compaction windows"); } - return newestBucket(buckets, comConf.getIncomingWindowMin(), now, - comConf.getBaseWindowMillis(), mayUseOffPeak); - } - - /** - * @param buckets the list of buckets, sorted from newest to oldest, from which to return the - * newest bucket within thresholds. - * @param incomingWindowThreshold minimum number of storeFiles in a bucket to qualify. - * @param maxThreshold maximum number of storeFiles to compact at once (the returned bucket will - * be trimmed down to this). - * @return a bucket (a list of store files within a window to be compacted). - * @throws IOException error - */ - private ArrayList newestBucket(List> buckets, - int incomingWindowThreshold, long now, long baseWindowMillis, boolean mayUseOffPeak) - throws IOException { - Window incomingWindow = getInitialWindow(now, baseWindowMillis); - for (ArrayList bucket : buckets) { + Window incomingWindow = getInitialWindow(now, comConf.getBaseWindowMillis()); + for (Bucket bucket : buckets) { int minThreshold = - incomingWindow.compareToTimestamp(bucket.get(0).getMaximumTimestamp()) <= 0 ? comConf + isIncomingWindow(incomingWindow, bucket) ? comConf .getIncomingWindowMin() : comConf.getMinFilesToCompact(); compactionPolicyPerWindow.setMinThreshold(minThreshold); - ArrayList candidates = - compactionPolicyPerWindow.applyCompactionPolicy(bucket, mayUseOffPeak, false); + + // The files has to be in ascending order for ratio-based compaction to work right + // and removeExcessFile to exclude youngest files. + Collections.reverse(bucket.storeFileList); + + candidates = + compactionPolicyPerWindow.applyCompactionPolicy(bucket.storeFileList, mayUseOffPeak, + false); if (candidates != null && !candidates.isEmpty()) { - return candidates; + // If there is any file in the window excluded from compaction, + // only one file will be output from compaction. + TimeRangeTracker timeRange = candidates.size() == bucket.storeFileList.size() ? + bucket.windowRange + : new TimeRangeTracker(Long.MIN_VALUE, Long.MAX_VALUE); + removeExcessFiles(candidates, false, false); + return new Bucket(timeRange.getMinimumTimestamp(), timeRange.getMaximumTimestamp(), + candidates); } } return null; } /** + * @param filesToCompact list of files to compact + * @param now current time + * @return a list of boundaries for multiple compaction output + * from minTimestamp to maxTimestamp. + */ + public List getCompactBoundariesForMajor(Collection filesToCompact,long now) { + TimeRangeTracker timeRange = getTimeRange(filesToCompact); + long oldestToCompact = getOldestToCompact(comConf.getMaxStoreFileAgeMillis(), now); + List boundaries = new ArrayList(); + Window window; + + // locate max timestamp first + for (window = getInitialWindow(now, comConf.getBaseWindowMillis());; window = + window.nextWindow(comConf.getWindowsPerTier(), oldestToCompact)) { + if (window.compareToTimestamp(timeRange.getMaximumTimestamp()) <= 0) { + boundaries.add(timeRange.getMaximumTimestamp() + 1); + boundaries.add(window.startMillis()); + break; + } + } + assert !boundaries.isEmpty(); + for (window = window.nextWindow(comConf.getWindowsPerTier(), oldestToCompact);; + window = window.nextWindow(comConf.getWindowsPerTier(), oldestToCompact)) { + if (window.compareToTimestamp(timeRange.getMinimumTimestamp()) > 0) { + boundaries.add(window.startMillis()); + } else { + long startMillis = window.startMillis(); + if (startMillis > timeRange.getMinimumTimestamp()) { + boundaries.add(startMillis); + } + boundaries.add(timeRange.getMinimumTimestamp()); + break; + } + } + Collections.reverse(boundaries); + return boundaries; + } + + /** + * @param bucket the bucket for minor compaction + * @return a list of boundaries for multiple compaction output + * from minTimestamp to maxTimestamp. + */ + public List getCompactionBoundariesForMinor(Bucket bucket, long now) { + List boundaries = new ArrayList(); + TimeRangeTracker timeRange = getTimeRange(bucket.storeFileList); + boundaries.add(timeRange.getMinimumTimestamp()); + + if (bucket.windowRange.getMinimumTimestamp() != Long.MIN_VALUE + && bucket.windowRange.getMaximumTimestamp() != Long.MAX_VALUE) { + boundaries.add(bucket.windowRange.getMinimumTimestamp()); + Window incomingWindow = getInitialWindow(now, comConf.getBaseWindowMillis()); + if (isIncomingWindow(incomingWindow, bucket) + && bucket.windowRange.getMaximumTimestamp() + <= timeRange.getMaximumTimestamp()) { + boundaries.add(bucket.windowRange.getMaximumTimestamp()); + } + } + + boundaries.add(timeRange.getMaximumTimestamp() + 1); + return boundaries; + } + + private static TimeRangeTracker getTimeRange(Collection storeFiles) { + long maxTimestamp = Long.MIN_VALUE; + long minTimestamp = Long.MAX_VALUE; + for (StoreFile file : storeFiles) { + maxTimestamp = Math.max(maxTimestamp, file.getMaximumTimestamp()); + minTimestamp = Math.min(minTimestamp, file.getMinimumTimestamp()); + } + return new TimeRangeTracker(minTimestamp, maxTimestamp); + } + + /** * We receive store files sorted in ascending order by seqId then scan the list of files. If the * current file has a maxTimestamp older than last known maximum, treat this file as it carries * the last known maximum. This way both seqId and timestamp are in the same order. If files carry @@ -182,15 +290,19 @@ public class DateTieredCompactionPolicy extends RatioBasedCompactionPolicy { * by seqId and maxTimestamp in decending order and build the time windows. All the out-of-order * data into the same compaction windows, guaranteeing contiguous compaction based on sequence id. */ - private static List> partitionFilesToBuckets(Iterable storeFiles, - long baseWindowSizeMillis, int windowsPerTier, long now) { - List> buckets = Lists.newArrayList(); + private static List partitionFilesToBuckets(Iterable storeFiles, + long baseWindowSizeMillis, int windowsPerTier, long minToCompactMillis, long now) { + List buckets = Lists.newArrayList(); Window window = getInitialWindow(now, baseWindowSizeMillis); + // Make sure the store files is sorted by SeqId then maxTimestamp + List storeFileList = Lists.newArrayList(storeFiles); + Collections.sort(storeFileList, StoreFile.Comparators.SEQ_ID_MAX_TIMESTAMP); + List> storefileMaxTimestampPairs = - Lists.newArrayListWithCapacity(Iterables.size(storeFiles)); + Lists.newArrayListWithCapacity(Iterables.size(storeFileList)); long maxTimestampSeen = Long.MIN_VALUE; - for (StoreFile storeFile : storeFiles) { + for (StoreFile storeFile : storeFileList) { // if there is out-of-order data, // we put them in the same window as the last file in increasing order maxTimestampSeen = Math.max(maxTimestampSeen, storeFile.getMaximumTimestamp()); @@ -205,17 +317,18 @@ public class DateTieredCompactionPolicy extends RatioBasedCompactionPolicy { int compResult = window.compareToTimestamp(it.peek().getSecond()); if (compResult > 0) { // If the file is too old for the window, switch to the next window - window = window.nextWindow(windowsPerTier); + window = window.nextWindow(windowsPerTier, + getOldestToCompact(minToCompactMillis, now)); } else { // The file is within the target window - ArrayList bucket = Lists.newArrayList(); + ArrayList fileList = Lists.newArrayList(); // Add all files in the same window to current bucket. For incoming window // we tolerate files with future data although it is sub-optimal while (it.hasNext() && window.compareToTimestamp(it.peek().getSecond()) <= 0) { - bucket.add(it.next().getFirst()); + fileList.add(it.next().getFirst()); } - if (!bucket.isEmpty()) { - buckets.add(bucket); + if (!fileList.isEmpty()) { + buckets.add(new Bucket(window.startMillis(), window.endMillis(), fileList)); } } } @@ -248,10 +361,26 @@ public class DateTieredCompactionPolicy extends RatioBasedCompactionPolicy { }); } - private static Window getInitialWindow(long now, long timeUnit) { - return new Window(timeUnit, now / timeUnit); + /** + * @param now current time + * @param baseWindowMillis base window size in milliseconds + * @param bucket the bucket for compaction + * @return whether the bucket is in incoming window + */ + private static boolean isIncomingWindow(Window incomingWindow, Bucket bucket) { + return incomingWindow.compareToTimestamp( + bucket.storeFileList.get(0).getMaximumTimestamp()) <= 0 ; + } + + private static Window getInitialWindow(long now, long baseWindowMillis) { + return new Window(baseWindowMillis, now / baseWindowMillis); } + private static long getOldestToCompact(long maxAgeMillis, long now) { + maxAgeMillis = Math.min(maxAgeMillis, now); + return now - maxAgeMillis; + } + /** * This is the class we use to partition from epoch time to now into tiers of exponential sizes of * windows. @@ -268,10 +397,10 @@ public class DateTieredCompactionPolicy extends RatioBasedCompactionPolicy { private final long divPosition; private Window(long baseWindowMillis, long divPosition) { - this.windowMillis = baseWindowMillis; + windowMillis = baseWindowMillis; this.divPosition = divPosition; } - + /** * Compares the window to a timestamp. * @param timestamp the timestamp to compare. @@ -290,12 +419,42 @@ public class DateTieredCompactionPolicy extends RatioBasedCompactionPolicy { * following those will be tierBase times as big. * @return The next window */ - public Window nextWindow(int windowsPerTier) { - if (divPosition % windowsPerTier > 0) { + public Window nextWindow(int windowsPerTier, long oldestToCompact) { + // Don't promote to the next tier if there is not even 1 window at current tier + // or if the next window crosses the max age. + if (divPosition % windowsPerTier > 0 || + startMillis() - windowMillis * windowsPerTier < oldestToCompact) { return new Window(windowMillis, divPosition - 1); } else { return new Window(windowMillis * windowsPerTier, divPosition / windowsPerTier - 1); } } + + /** + * Inclusive upper bound + */ + public long startMillis() { + return windowMillis * divPosition; + } + + /** + * Exclusive lower bound + */ + public long endMillis() { + return windowMillis * (divPosition + 1); + } + } + + /** + * This is the data for all the files in a window and the time bound of the window + */ + private static final class Bucket { + private TimeRangeTracker windowRange; + private ArrayList storeFileList; + + private Bucket(long startMillis, long endMillis, ArrayList storeFiles) { + storeFileList = storeFiles; + windowRange = new TimeRangeTracker(startMillis, endMillis); + } } } \ No newline at end of file diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DateTieredCompactionRequest.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DateTieredCompactionRequest.java new file mode 100644 index 0000000..245665b --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DateTieredCompactionRequest.java @@ -0,0 +1,35 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.regionserver.compactions; + +import java.util.Collection; +import java.util.List; + +import org.apache.hadoop.hbase.regionserver.StoreFile; + +public class DateTieredCompactionRequest extends CompactionRequest { + private static List boundaries; + + public DateTieredCompactionRequest(Collection files, List boundaryList) { + super(files); + boundaries = boundaryList; + } + public List getBoundaries() { + return boundaries; + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DateTieredCompactor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DateTieredCompactor.java new file mode 100644 index 0000000..0ca62e7 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DateTieredCompactor.java @@ -0,0 +1,145 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.regionserver.compactions; + +import java.io.IOException; +import java.io.InterruptedIOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.List; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.io.compress.Compression; +import org.apache.hadoop.hbase.regionserver.DateTieredMultiFileWriter; +import org.apache.hadoop.hbase.regionserver.InternalScanner; +import org.apache.hadoop.hbase.regionserver.ScanType; +import org.apache.hadoop.hbase.regionserver.Store; +import org.apache.hadoop.hbase.regionserver.StoreFile; +import org.apache.hadoop.hbase.regionserver.StoreFile.Writer; +import org.apache.hadoop.hbase.regionserver.StoreFileScanner; +import org.apache.hadoop.hbase.regionserver.StoreScanner; +import org.apache.hadoop.hbase.regionserver.StripeMultiFileWriter; +import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController; +import org.apache.hadoop.hbase.security.User; + +import com.google.common.io.Closeables; + +/** + * This compactor will generate StoreFile for different time ranges. + */ +@InterfaceAudience.Private +public class DateTieredCompactor extends Compactor { + + private static final Log LOG = LogFactory.getLog(DateTieredCompactor.class); + + public DateTieredCompactor(Configuration conf, Store store) { + super(conf, store); + } + + public List compact(final CompactionRequest request, List boundaries, + ThroughputController throughputController, User user) throws IOException { + if (LOG.isDebugEnabled()) { + LOG.debug("Executing compaction with " + boundaries.size() + " boundaries: " + boundaries); + } + final FileDetails fd = getFileDetails(request.getFiles(), request.isMajor()); + this.progress = new CompactionProgress(fd.maxKeyCount); + + // Find the smallest read point across all the Scanners. + long smallestReadPoint = getSmallestReadPoint(); + + List scanners; + Collection readersToClose; + boolean cleanSeqId = false; + if (this.conf.getBoolean("hbase.regionserver.compaction.private.readers", true)) { + // clone all StoreFiles, so we'll do the compaction on a independent copy of StoreFiles, + // HFileFiles, and their readers + readersToClose = new ArrayList(request.getFiles().size()); + for (StoreFile f : request.getFiles()) { + readersToClose.add(new StoreFile(f)); + } + scanners = createFileScanners(readersToClose, smallestReadPoint, + store.throttleCompaction(request.getSize())); + } else { + readersToClose = Collections.emptyList(); + scanners = createFileScanners(request.getFiles(), smallestReadPoint, + store.throttleCompaction(request.getSize())); + } + InternalScanner scanner = null; + DateTieredMultiFileWriter writer = new DateTieredMultiFileWriter(boundaries); + boolean finished = false; + try { + /* Include deletes, unless we are doing a major compaction */ + ScanType scanType = request.isMajor() ? ScanType.COMPACT_DROP_DELETES + : ScanType.COMPACT_RETAIN_DELETES; + scanner = preCreateCoprocScanner(request, scanType, fd.earliestPutTs, scanners); + if (scanner == null) { + scanner = createScanner(store, scanners, scanType, smallestReadPoint, fd.earliestPutTs); + } + scanner = postCreateCoprocScanner(request, scanType, scanner, user); + if (scanner == null) { + // NULL scanner returned from coprocessor hooks means skip normal processing. + return new ArrayList(); + } + if (fd.minSeqIdToKeep > 0) { + smallestReadPoint = Math.min(fd.minSeqIdToKeep, smallestReadPoint); + cleanSeqId = true; + } + // Create the writer factory for compactions. + final boolean needMvcc = fd.maxMVCCReadpoint >= smallestReadPoint; + final Compression.Algorithm compression = store.getFamily().getCompactionCompressionType(); + DateTieredMultiFileWriter.WriterFactory factory = new StripeMultiFileWriter.WriterFactory() { + @Override + public Writer createWriter() throws IOException { + return store.createWriterInTmp(fd.maxKeyCount, compression, true, needMvcc, + fd.maxTagsLength > 0, store.throttleCompaction(request.getSize())); + } + }; + // Prepare multi-writer, and perform the compaction using scanner and writer. + // It is ok here if storeScanner is null. + StoreScanner storeScanner = (scanner instanceof StoreScanner) ? (StoreScanner) scanner : null; + writer.init(storeScanner, factory, store.getComparator()); + finished = performCompaction(fd, scanner, writer, smallestReadPoint, cleanSeqId, + throughputController, request.isMajor()); + if (!finished) { + throw new InterruptedIOException("Aborting compaction of store " + store + " in region " + + store.getRegionInfo().getRegionNameAsString() + " because it was interrupted."); + } + } finally { + Closeables.close(scanner, true); + if (!finished) { + FileSystem fs = store.getFileSystem(); + for (Path leftoverFile : writer.abortWriters()) { + try { + fs.delete(leftoverFile, false); + } catch (Exception e) { + LOG.error("Failed to delete the leftover file " + leftoverFile + + " after an unfinished compaction.", + e); + } + } + } + } + return writer.commitWriters(fd.maxSeqId, request.isMajor()); + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/RatioBasedCompactionPolicy.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/RatioBasedCompactionPolicy.java index 4533a9c..7162e87 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/RatioBasedCompactionPolicy.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/RatioBasedCompactionPolicy.java @@ -53,7 +53,7 @@ public class RatioBasedCompactionPolicy extends CompactionPolicy { super(conf, storeConfigInfo); } - private ArrayList getCurrentEligibleFiles( + protected ArrayList getCurrentEligibleFiles( ArrayList candidateFiles, final List filesCompacting) { // candidates = all storefiles not already in compaction queue if (!filesCompacting.isEmpty()) { @@ -166,7 +166,7 @@ public class RatioBasedCompactionPolicy extends CompactionPolicy { * @return filtered subset * take upto maxFilesToCompact from the start */ - private ArrayList removeExcessFiles(ArrayList candidates, + protected ArrayList removeExcessFiles(ArrayList candidates, boolean isUserCompaction, boolean isMajorCompaction) { int excess = candidates.size() - comConf.getMaxFilesToCompact(); if (excess > 0) { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDateTieredCompaction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDateTieredCompaction.java index cfb54b7..e2d29fc 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDateTieredCompaction.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDateTieredCompaction.java @@ -1,5 +1,4 @@ /** - * * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -18,194 +17,48 @@ */ package org.apache.hadoop.hbase.regionserver; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.Lists; - import java.io.IOException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; -import org.apache.hadoop.hbase.regionserver.compactions.CompactionConfiguration; -import org.apache.hadoop.hbase.regionserver.compactions.DateTieredCompactionPolicy; -import org.apache.hadoop.hbase.testclassification.SmallTests; -import org.junit.Assert; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.HColumnDescriptor; +import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.apache.hadoop.hbase.testclassification.RegionServerTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.junit.After; +import org.junit.Before; import org.junit.Test; import org.junit.experimental.categories.Category; -@Category(SmallTests.class) -public class TestDateTieredCompaction extends TestCompactionPolicy { - ArrayList sfCreate(long[] minTimestamps, long[] maxTimestamps, long[] sizes) - throws IOException { - ArrayList ageInDisk = new ArrayList(); - for (int i = 0; i < sizes.length; i++) { - ageInDisk.add(0L); - } - - ArrayList ret = Lists.newArrayList(); - for (int i = 0; i < sizes.length; i++) { - MockStoreFile msf = - new MockStoreFile(TEST_UTIL, TEST_FILE, sizes[i], ageInDisk.get(i), false, i); - msf.setTimeRangeTracker(new TimeRangeTracker(minTimestamps[i], maxTimestamps[i])); - ret.add(msf); - } - return ret; - } - - @Override - protected void config() { - super.config(); - - // Set up policy - conf.setLong(CompactionConfiguration.MAX_AGE_MILLIS_KEY, 100); - conf.setLong(CompactionConfiguration.INCOMING_WINDOW_MIN_KEY, 3); - conf.setLong(CompactionConfiguration.BASE_WINDOW_MILLIS_KEY, 6); - conf.setInt(CompactionConfiguration.WINDOWS_PER_TIER_KEY, 4); - conf.set(DefaultStoreEngine.DEFAULT_COMPACTION_POLICY_CLASS_KEY, - DateTieredCompactionPolicy.class.getName()); - - // Special settings for compaction policy per window - this.conf.setInt(CompactionConfiguration.HBASE_HSTORE_COMPACTION_MIN_KEY, 2); - this.conf.setInt(CompactionConfiguration.HBASE_HSTORE_COMPACTION_MAX_KEY, 12); - this.conf.setFloat(CompactionConfiguration.HBASE_HSTORE_COMPACTION_RATIO_KEY, 1.2F); - } - - void compactEquals(long now, ArrayList candidates, long... expected) - throws IOException { - Assert.assertTrue(((DateTieredCompactionPolicy) store.storeEngine.getCompactionPolicy()) - .needsCompaction(candidates, ImmutableList. of(), now)); - - List actual = - ((DateTieredCompactionPolicy) store.storeEngine.getCompactionPolicy()) - .applyCompactionPolicy(candidates, false, false, now); +@Category({ RegionServerTests.class, MediumTests.class }) +public class TestDateTieredCompaction { - Assert.assertEquals(Arrays.toString(expected), Arrays.toString(getSizes(actual))); - } + private static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); - /** - * Test for incoming window - * @throws IOException with error - */ - @Test - public void incomingWindow() throws IOException { - long[] minTimestamps = new long[] { 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0 }; - long[] maxTimestamps = new long[] { 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15 }; - long[] sizes = new long[] { 30, 31, 32, 33, 34, 20, 21, 22, 23, 24, 25, 10, 11, 12, 13 }; + private static final TableName TN = TableName.valueOf("dtc"); - compactEquals(16, sfCreate(minTimestamps, maxTimestamps, sizes), 13, 12, 11, 10); - } - - /** - * Not enough files in incoming window - * @throws IOException with error - */ - @Test - public void NotIncomingWindow() throws IOException { - long[] minTimestamps = new long[] { 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0 }; - long[] maxTimestamps = new long[] { 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13 }; - long[] sizes = new long[] { 30, 31, 32, 33, 34, 20, 21, 22, 23, 24, 25, 10, 11 }; + private static final byte[] FAMILY = Bytes.toBytes("f"); - compactEquals(16, sfCreate(minTimestamps, maxTimestamps, sizes), 25, 24, 23, 22, 21, 20); - } + private static final byte[] QUALIFIER = Bytes.toBytes("q"); - /** - * Test for file newer than incoming window - * @throws IOException with error - */ - @Test - public void NewerThanIncomingWindow() throws IOException { - long[] minTimestamps = new long[] { 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0 }; - long[] maxTimestamps = new long[] { 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 18 }; - long[] sizes = new long[] { 30, 31, 32, 33, 34, 20, 21, 22, 23, 24, 25, 10, 11, 12, 13 }; + private HRegion region; - compactEquals(16, sfCreate(minTimestamps, maxTimestamps, sizes), 13, 12, 11, 10); + @Before + public void setUp() throws IOException { + HTableDescriptor htd = new HTableDescriptor(TN).addFamily(new HColumnDescriptor(FAMILY)); + htd.setConfiguration(StoreEngine.STORE_ENGINE_CLASS_KEY, DateTieredStoreEngine.class.getName()); + region = UTIL.createLocalHRegion(htd, null, null); } - /** - * If there is no T1 window, we don't build 2 - * @throws IOException with error - */ - @Test - public void NoT2() throws IOException { - long[] minTimestamps = new long[] { 0, 0, 0, 0, 0, 0, 0, 0, 0, 0 }; - long[] maxTimestamps = new long[] { 44, 60, 61, 92, 95, 100 }; - long[] sizes = new long[] { 0, 20, 21, 22, 23, 1 }; - - compactEquals(100, sfCreate(minTimestamps, maxTimestamps, sizes), 23, 22); - } - - @Test - public void T1() throws IOException { - long[] minTimestamps = new long[] { 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0 }; - long[] maxTimestamps = new long[] { 44, 60, 61, 96, 100, 104, 120, 124, 143, 145, 157 }; - long[] sizes = new long[] { 0, 50, 51, 40, 41, 42, 30, 31, 32, 2, 1 }; - - compactEquals(161, sfCreate(minTimestamps, maxTimestamps, sizes), 32, 31, 30); - } - - /** - * Apply exploring logic on non-incoming window - * @throws IOException with error - */ - @Test - public void RatioT0() throws IOException { - long[] minTimestamps = new long[] { 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0 }; - long[] maxTimestamps = new long[] { 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12 }; - long[] sizes = new long[] { 30, 31, 32, 33, 34, 20, 21, 22, 280, 23, 24, 1 }; - - compactEquals(16, sfCreate(minTimestamps, maxTimestamps, sizes), 22, 21, 20); - } - - /** - * Also apply ratio-based logic on t2 window - * @throws IOException with error - */ - @Test - public void RatioT2() throws IOException { - long[] minTimestamps = new long[] { 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0 }; - long[] maxTimestamps = new long[] { 44, 60, 61, 96, 100, 104, 120, 124, 143, 145, 157 }; - long[] sizes = new long[] { 0, 50, 51, 40, 41, 42, 350, 30, 31, 2, 1 }; - - compactEquals(161, sfCreate(minTimestamps, maxTimestamps, sizes), 31, 30); - } - - /** - * The next compaction call after testTieredCompactionRatioT0 is compacted - * @throws IOException with error - */ - @Test - public void RatioT0Next() throws IOException { - long[] minTimestamps = new long[] { 0, 0, 0, 0, 0, 0, 0, 0, 0, 0 }; - long[] maxTimestamps = new long[] { 1, 2, 3, 4, 5, 8, 9, 10, 11, 12 }; - long[] sizes = new long[] { 30, 31, 32, 33, 34, 22, 280, 23, 24, 1 }; - - compactEquals(16, sfCreate(minTimestamps, maxTimestamps, sizes), 24, 23); - } - - /** - * Older than now(161) - maxAge(100) - * @throws IOException with error - */ - @Test - public void olderThanMaxAge() throws IOException { - long[] minTimestamps = new long[] { 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0 }; - long[] maxTimestamps = new long[] { 44, 60, 61, 96, 100, 104, 105, 106, 113, 145, 157 }; - long[] sizes = new long[] { 0, 50, 51, 40, 41, 42, 33, 30, 31, 2, 1 }; - - compactEquals(161, sfCreate(minTimestamps, maxTimestamps, sizes), 31, 30, 33, 42, 41, 40); + @After + public void tearDown() throws IOException { + HBaseTestingUtility.closeRegionAndWAL(region); + UTIL.cleanupTestDir(); } - /** - * Out-of-order data - * @throws IOException with error - */ @Test - public void OutOfOrder() throws IOException { - long[] minTimestamps = new long[] { 0, 0, 0, 0, 0, 0, 0, 0, 0, 0 }; - long[] maxTimestamps = new long[] { 0, 13, 3, 10, 11, 1, 2, 12, 14, 15 }; - long[] sizes = new long[] { 30, 31, 32, 33, 34, 22, 28, 23, 24, 1 }; + public void test() { - compactEquals(16, sfCreate(minTimestamps, maxTimestamps, sizes), 1, 24, 23, 28, 22, 34, 33, 32, - 31); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDateTieredCompactionPolicy.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDateTieredCompactionPolicy.java new file mode 100644 index 0000000..6e70588 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDateTieredCompactionPolicy.java @@ -0,0 +1,272 @@ +/** + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.regionserver; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +import org.apache.hadoop.hbase.regionserver.compactions.CompactionConfiguration; +import org.apache.hadoop.hbase.regionserver.compactions.DateTieredCompactionPolicy; +import org.apache.hadoop.hbase.regionserver.compactions.DateTieredCompactionRequest; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.Assert; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +@Category(SmallTests.class) +public class TestDateTieredCompactionPolicy extends TestCompactionPolicy { + ArrayList sfCreate(long[] minTimestamps, long[] maxTimestamps, long[] sizes) + throws IOException { + ArrayList ageInDisk = new ArrayList(); + for (int i = 0; i < sizes.length; i++) { + ageInDisk.add(0L); + } + + ArrayList ret = Lists.newArrayList(); + for (int i = 0; i < sizes.length; i++) { + MockStoreFile msf = + new MockStoreFile(TEST_UTIL, TEST_FILE, sizes[i], ageInDisk.get(i), false, i); + msf.setTimeRangeTracker(new TimeRangeTracker(minTimestamps[i], maxTimestamps[i])); + ret.add(msf); + } + return ret; + } + + @Override + protected void config() { + super.config(); + + // Set up policy + conf.setLong(CompactionConfiguration.MAX_AGE_MILLIS_KEY, 100); + conf.setLong(CompactionConfiguration.INCOMING_WINDOW_MIN_KEY, 3); + conf.setLong(CompactionConfiguration.BASE_WINDOW_MILLIS_KEY, 6); + conf.setInt(CompactionConfiguration.WINDOWS_PER_TIER_KEY, 4); + conf.set(DefaultStoreEngine.DEFAULT_COMPACTION_POLICY_CLASS_KEY, + DateTieredCompactionPolicy.class.getName()); + + // Special settings for compaction policy per window + conf.setInt(CompactionConfiguration.HBASE_HSTORE_COMPACTION_MIN_KEY, 2); + conf.setInt(CompactionConfiguration.HBASE_HSTORE_COMPACTION_MAX_KEY, 12); + conf.setFloat(CompactionConfiguration.HBASE_HSTORE_COMPACTION_RATIO_KEY, 1.2F); + } + + void compactMinorEquals(long now, ArrayList candidates, + long[] expectedFileSizes, long[] expectedBoundaries) + throws IOException { + Assert.assertTrue(((DateTieredCompactionPolicy) store.storeEngine.getCompactionPolicy()) + .needsCompaction(candidates, ImmutableList. of(), now)); + + DateTieredCompactionRequest request = + (DateTieredCompactionRequest) ((DateTieredCompactionPolicy) store.storeEngine + .getCompactionPolicy()) + .selectCompaction(candidates, ImmutableList.of(), false, false, false, now); + List actual = Lists.newArrayList(request.getFiles()); + Assert.assertEquals(Arrays.toString(expectedFileSizes), Arrays.toString(getSizes(actual))); + Assert.assertEquals(Arrays.toString(expectedBoundaries), + Arrays.toString(request.getBoundaries().toArray())); + } + + void compactMajorEquals(long now, ArrayList candidates, + long[] expectedFileSizes, long[] expectedBoundaries) + throws IOException { + + DateTieredCompactionRequest request = + (DateTieredCompactionRequest) ((DateTieredCompactionPolicy) store.storeEngine + .getCompactionPolicy()) + .selectCompaction(candidates, ImmutableList.of(), true, false, true, now); + List actual = Lists.newArrayList(request.getFiles()); + Assert.assertEquals(Arrays.toString(expectedFileSizes), Arrays.toString(getSizes(actual))); + Assert.assertEquals(Arrays.toString(expectedBoundaries), + Arrays.toString(request.getBoundaries().toArray())); + } + + /** + * Test for incoming window + * @throws IOException with error + */ + @Test + public void incomingWindow() throws IOException { + long[] minTimestamps = new long[] { 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0 }; + long[] maxTimestamps = new long[] { 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15 }; + long[] sizes = new long[] { 30, 31, 32, 33, 34, 20, 21, 22, 23, 24, 25, 10, 11, 12, 13 }; + + compactMinorEquals(16, sfCreate(minTimestamps, maxTimestamps, sizes), + new long[] { 10, 11, 12, 13 }, new long [] { 0, 12, 16 }); + } + + /** + * Not enough files in incoming window + * @throws IOException with error + */ + @Test + public void NotIncomingWindow() throws IOException { + long[] minTimestamps = new long[] { 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0 }; + long[] maxTimestamps = new long[] { 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13 }; + long[] sizes = new long[] { 30, 31, 32, 33, 34, 20, 21, 22, 23, 24, 25, 10, 11 }; + + compactMinorEquals(16, sfCreate(minTimestamps, maxTimestamps, sizes), + new long[] { 20, 21, 22, 23, 24, 25 }, + new long[] { 0, 6, 12 }); + } + + /** + * Test for file on the upper bound of incoming window + * @throws IOException with error + */ + @Test + public void OnUpperBoundOfIncomingWindow() throws IOException { + long[] minTimestamps = new long[] { 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0 }; + long[] maxTimestamps = new long[] { 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 18 }; + long[] sizes = new long[] { 30, 31, 32, 33, 34, 20, 21, 22, 23, 24, 25, 10, 11, 12, 13 }; + + compactMinorEquals(16, sfCreate(minTimestamps, maxTimestamps, sizes), + new long[] { 10, 11, 12, 13 }, + new long[] { 0, 12, 18, 19 }); + } + + /** + * Test for file newer than incoming window + * @throws IOException with error + */ + @Test + public void NewerThanIncomingWindow() throws IOException { + long[] minTimestamps = new long[] { 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0 }; + long[] maxTimestamps = new long[] { 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 19 }; + long[] sizes = new long[] { 30, 31, 32, 33, 34, 20, 21, 22, 23, 24, 25, 10, 11, 12, 13 }; + + compactMinorEquals(16, sfCreate(minTimestamps, maxTimestamps, sizes), + new long[] { 10, 11, 12, 13 }, + new long[] { 0, 12, 18, 20 }); + } + /** + * If there is no T1 window, we don't build T2 + * @throws IOException with error + */ + @Test + public void NoT2() throws IOException { + long[] minTimestamps = new long[] { 0, 0, 0, 0, 0, 0, 0, 0, 0, 0 }; + long[] maxTimestamps = new long[] { 44, 60, 61, 97, 100, 193 }; + long[] sizes = new long[] { 0, 20, 21, 22, 23, 1 }; + + compactMinorEquals(194, sfCreate(minTimestamps, maxTimestamps, sizes), + new long[] { 22, 23 }, new long[] { 0, 96, 101}); + } + + @Test + public void T1() throws IOException { + long[] minTimestamps = new long[] { 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0 }; + long[] maxTimestamps = new long[] { 44, 60, 61, 96, 100, 104, 120, 124, 143, 145, 157 }; + long[] sizes = new long[] { 0, 50, 51, 40, 41, 42, 30, 31, 32, 2, 1 }; + + compactMinorEquals(161, sfCreate(minTimestamps, maxTimestamps, sizes), + new long[] { 30, 31, 32 }, new long[] { 0, 120, 144 }); + } + + /** + * Apply exploring logic on non-incoming window + * @throws IOException with error + */ + @Test + public void RatioT0() throws IOException { + long[] minTimestamps = new long[] { 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0 }; + long[] maxTimestamps = new long[] { 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12 }; + long[] sizes = new long[] { 30, 31, 32, 33, 34, 20, 21, 22, 280, 23, 24, 1 }; + + compactMinorEquals(16, sfCreate(minTimestamps, maxTimestamps, sizes), + new long[] { 20, 21, 22 }, new long[] { 0, 9 }); + } + + /** + * Also apply ratio-based logic on t2 window + * @throws IOException with error + */ + @Test + public void RatioT2() throws IOException { + long[] minTimestamps = new long[] { 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0 }; + long[] maxTimestamps = new long[] { 44, 60, 61, 96, 100, 104, 120, 124, 143, 145, 157 }; + long[] sizes = new long[] { 0, 50, 51, 40, 41, 42, 350, 30, 31, 2, 1 }; + + compactMinorEquals(161, sfCreate(minTimestamps, maxTimestamps, sizes), + new long[] { 30, 31 }, new long[] { 0, 144 }); + } + + /** + * The next compaction call after testTieredCompactionRatioT0 is compacted + * @throws IOException with error + */ + @Test + public void RatioT0Next() throws IOException { + long[] minTimestamps = new long[] { 0, 0, 0, 0, 0, 0, 0, 0, 0, 0 }; + long[] maxTimestamps = new long[] { 1, 2, 3, 4, 5, 8, 9, 10, 11, 12 }; + long[] sizes = new long[] { 30, 31, 32, 33, 34, 22, 280, 23, 24, 1 }; + + compactMinorEquals(16, sfCreate(minTimestamps, maxTimestamps, sizes), + new long[] { 23, 24 }, new long[] { 0, 12 }); + } + + /** + * Older than now(161) - maxAge(100) + * @throws IOException with error + */ + @Test + public void olderThanMaxAge() throws IOException { + long[] minTimestamps = new long[] { 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0 }; + long[] maxTimestamps = new long[] { 44, 60, 61, 96, 100, 104, 105, 106, 113, 145, 157 }; + long[] sizes = new long[] { 0, 50, 51, 40, 41, 42, 33, 30, 31, 2, 1 }; + + compactMinorEquals(161, sfCreate(minTimestamps, maxTimestamps, sizes), + new long[] { 40, 41, 42, 33, 30, 31 }, + new long[] { 0, 96, 114 }); + } + + /** + * Out-of-order data + * @throws IOException with error + */ + @Test + public void outOfOrder() throws IOException { + long[] minTimestamps = new long[] { 0, 0, 0, 0, 0, 0, 0, 0, 0, 0 }; + long[] maxTimestamps = new long[] { 0, 13, 3, 10, 11, 1, 2, 12, 14, 15 }; + long[] sizes = new long[] { 30, 31, 32, 33, 34, 22, 28, 23, 24, 1 }; + + compactMinorEquals(16, sfCreate(minTimestamps, maxTimestamps, sizes), + new long[] { 31, 32, 33, 34, 22, 28, 23, 24, 1 }, + new long[] { 0, 12, 16 }); + } + + /** + * Major compaction + * @throws IOException with error + */ + @Test + public void majorCompation() throws IOException { + long[] minTimestamps = new long[] { 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0 }; + long[] maxTimestamps = new long[] { 44, 60, 61, 96, 100, 104, 105, 106, 113, 145, 157 }; + long[] sizes = new long[] { 0, 50, 51, 40, 41, 42, 33, 30, 31, 2, 1 }; + + compactMajorEquals(161, sfCreate(minTimestamps, maxTimestamps, sizes), + new long[] { 0, 50, 51, 40, 41, 42, 33, 30, 31, 2, 1 }, + new long[] { 0, 24, 48, 72, 96, 120, 144, 150, 156, 158 }); + } +} -- 1.9.3 (Apple Git-50)