View Javadoc

1   /**
2    * Licensed to the Apache Software Foundation (ASF) under one
3    * or more contributor license agreements.  See the NOTICE file
4    * distributed with this work for additional information
5    * regarding copyright ownership.  The ASF licenses this file
6    * to you under the Apache License, Version 2.0 (the
7    * "License"); you may not use this file except in compliance
8    * with the License.  You may obtain a copy of the License at
9    *
10   *     http://www.apache.org/licenses/LICENSE-2.0
11   *
12   * Unless required by applicable law or agreed to in writing, software
13   * distributed under the License is distributed on an "AS IS" BASIS,
14   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15   * See the License for the specific language governing permissions and
16   * limitations under the License.
17   */
18  package org.apache.hadoop.hbase.regionserver.compactions;
19  
20  import java.io.IOException;
21  import java.util.ArrayList;
22  import java.util.Collection;
23  import java.util.List;
24  import java.util.Map;
25  
26  import org.apache.commons.logging.Log;
27  import org.apache.commons.logging.LogFactory;
28  import org.apache.hadoop.classification.InterfaceAudience;
29  import org.apache.hadoop.conf.Configuration;
30  import org.apache.hadoop.hbase.Cell;
31  import org.apache.hadoop.hbase.HConstants;
32  import org.apache.hadoop.hbase.KeyValue;
33  import org.apache.hadoop.hbase.KeyValueUtil;
34  import org.apache.hadoop.hbase.client.Scan;
35  import org.apache.hadoop.hbase.io.compress.Compression;
36  import org.apache.hadoop.hbase.io.hfile.HFile.FileInfo;
37  import org.apache.hadoop.hbase.io.hfile.HFileWriterV2;
38  import org.apache.hadoop.hbase.regionserver.HStore;
39  import org.apache.hadoop.hbase.regionserver.InternalScanner;
40  import org.apache.hadoop.hbase.regionserver.ScanType;
41  import org.apache.hadoop.hbase.regionserver.Store;
42  import org.apache.hadoop.hbase.regionserver.StoreFile;
43  import org.apache.hadoop.hbase.regionserver.StoreFileScanner;
44  import org.apache.hadoop.hbase.regionserver.StoreScanner;
45  import org.apache.hadoop.hbase.util.Bytes;
46  import org.apache.hadoop.util.StringUtils;
47  
48  /**
49   * A compactor is a compaction algorithm associated a given policy. Base class also contains
50   * reusable parts for implementing compactors (what is common and what isn't is evolving).
51   */
52  @InterfaceAudience.Private
53  public abstract class Compactor {
54    private static final Log LOG = LogFactory.getLog(Compactor.class);
55    protected CompactionProgress progress;
56    protected Configuration conf;
57    protected Store store;
58  
59    private int compactionKVMax;
60    protected Compression.Algorithm compactionCompression;
61    
62    /** specify how many days to keep MVCC values during major compaction **/ 
63    protected int keepSeqIdPeriod;
64  
65    //TODO: depending on Store is not good but, realistically, all compactors currently do.
66    Compactor(final Configuration conf, final Store store) {
67      this.conf = conf;
68      this.store = store;
69      this.compactionKVMax =
70        this.conf.getInt(HConstants.COMPACTION_KV_MAX, HConstants.COMPACTION_KV_MAX_DEFAULT);
71      this.compactionCompression = (this.store.getFamily() == null) ?
72          Compression.Algorithm.NONE : this.store.getFamily().getCompactionCompression();
73      this.keepSeqIdPeriod = Math.max(this.conf.getInt(HConstants.KEEP_SEQID_PERIOD, 
74        HConstants.MIN_KEEP_SEQID_PERIOD), HConstants.MIN_KEEP_SEQID_PERIOD);
75    }
76  
77    /**
78     * TODO: Replace this with CellOutputStream when StoreFile.Writer uses cells.
79     */
80    public interface CellSink {
81      void append(KeyValue kv) throws IOException;
82    }
83  
84    public CompactionProgress getProgress() {
85      return this.progress;
86    }
87  
88    /** The sole reason this class exists is that java has no ref/out/pointer parameters. */
89    protected static class FileDetails {
90      /** Maximum key count after compaction (for blooms) */
91      public long maxKeyCount = 0;
92      /** Earliest put timestamp if major compaction */
93      public long earliestPutTs = HConstants.LATEST_TIMESTAMP;
94      /** The last key in the files we're compacting. */
95      public long maxSeqId = 0;
96      /** Latest memstore read point found in any of the involved files */
97      public long maxMVCCReadpoint = 0;
98      /** Max tags length**/
99      public int maxTagsLength = 0;
100     /** Min SeqId to keep during a major compaction **/
101     public long minSeqIdToKeep = 0;
102   }
103 
104   /**
105    * Extracts some details about the files to compact that are commonly needed by compactors.
106    * @param filesToCompact Files.
107    * @param allFiles Whether all files are included for compaction
108    * @return The result.
109    */
110   protected FileDetails getFileDetails(
111       Collection<StoreFile> filesToCompact, boolean allFiles) throws IOException {
112     FileDetails fd = new FileDetails();
113     long oldestHFileTimeStampToKeepMVCC = System.currentTimeMillis() - 
114       (1000L * 60 * 60 * 24 * this.keepSeqIdPeriod);  
115 
116     for (StoreFile file : filesToCompact) {
117       if(allFiles && (file.getModificationTimeStamp() < oldestHFileTimeStampToKeepMVCC)) {
118         // when isAllFiles is true, all files are compacted so we can calculate the smallest 
119         // MVCC value to keep
120         if(fd.minSeqIdToKeep < file.getMaxMemstoreTS()) {
121           fd.minSeqIdToKeep = file.getMaxMemstoreTS();
122         }
123       }
124       long seqNum = file.getMaxSequenceId();
125       fd.maxSeqId = Math.max(fd.maxSeqId, seqNum);
126       StoreFile.Reader r = file.getReader();
127       if (r == null) {
128         LOG.warn("Null reader for " + file.getPath());
129         continue;
130       }
131       // NOTE: getFilterEntries could cause under-sized blooms if the user
132       // switches bloom type (e.g. from ROW to ROWCOL)
133       long keyCount = (r.getBloomFilterType() == store.getFamily().getBloomFilterType())
134           ? r.getFilterEntries() : r.getEntries();
135       fd.maxKeyCount += keyCount;
136       // calculate the latest MVCC readpoint in any of the involved store files
137       Map<byte[], byte[]> fileInfo = r.loadFileInfo();
138       byte tmp[] = fileInfo.get(HFileWriterV2.MAX_MEMSTORE_TS_KEY);
139       if (tmp != null) {
140         fd.maxMVCCReadpoint = Math.max(fd.maxMVCCReadpoint, Bytes.toLong(tmp));
141       }
142       tmp = fileInfo.get(FileInfo.MAX_TAGS_LEN);
143       if (tmp != null) {
144         fd.maxTagsLength = Math.max(fd.maxTagsLength, Bytes.toInt(tmp));
145       }
146       // If required, calculate the earliest put timestamp of all involved storefiles.
147       // This is used to remove family delete marker during compaction.
148       long earliestPutTs = 0;
149       if (allFiles) {
150         tmp = fileInfo.get(StoreFile.EARLIEST_PUT_TS);
151         if (tmp == null) {
152           // There's a file with no information, must be an old one
153           // assume we have very old puts
154           fd.earliestPutTs = earliestPutTs = HConstants.OLDEST_TIMESTAMP;
155         } else {
156           earliestPutTs = Bytes.toLong(tmp);
157           fd.earliestPutTs = Math.min(fd.earliestPutTs, earliestPutTs);
158         }
159       }
160       if (LOG.isDebugEnabled()) {
161         LOG.debug("Compacting " + file +
162           ", keycount=" + keyCount +
163           ", bloomtype=" + r.getBloomFilterType().toString() +
164           ", size=" + StringUtils.humanReadableInt(r.length()) +
165           ", encoding=" + r.getHFileReader().getDataBlockEncoding() +
166           ", seqNum=" + seqNum +
167           (allFiles ? ", earliestPutTs=" + earliestPutTs: ""));
168       }
169     }
170     return fd;
171   }
172 
173   /**
174    * Creates file scanners for compaction.
175    * @param filesToCompact Files.
176    * @return Scanners.
177    */
178   protected List<StoreFileScanner> createFileScanners(
179       final Collection<StoreFile> filesToCompact, long smallestReadPoint) throws IOException {
180     return StoreFileScanner.getScannersForStoreFiles(filesToCompact, false, false, true,
181       smallestReadPoint);
182   }
183 
184   protected long getSmallestReadPoint() {
185     return store.getSmallestReadPoint();
186   }
187 
188   /**
189    * Calls coprocessor, if any, to create compaction scanner - before normal scanner creation.
190    * @param request Compaction request.
191    * @param scanType Scan type.
192    * @param earliestPutTs Earliest put ts.
193    * @param scanners File scanners for compaction files.
194    * @return Scanner override by coprocessor; null if not overriding.
195    */
196   protected InternalScanner preCreateCoprocScanner(final CompactionRequest request,
197       ScanType scanType, long earliestPutTs,  List<StoreFileScanner> scanners) throws IOException {
198     if (store.getCoprocessorHost() == null) return null;
199     return store.getCoprocessorHost()
200         .preCompactScannerOpen(store, scanners, scanType, earliestPutTs, request);
201   }
202 
203   /**
204    * Calls coprocessor, if any, to create scanners - after normal scanner creation.
205    * @param request Compaction request.
206    * @param scanType Scan type.
207    * @param scanner The default scanner created for compaction.
208    * @return Scanner scanner to use (usually the default); null if compaction should not proceed.
209    */
210    protected InternalScanner postCreateCoprocScanner(final CompactionRequest request,
211       ScanType scanType, InternalScanner scanner) throws IOException {
212     if (store.getCoprocessorHost() == null) return scanner;
213     return store.getCoprocessorHost().preCompact(store, scanner, scanType, request);
214   }
215 
216   /**
217    * Performs the compaction.
218    * @param scanner Where to read from.
219    * @param writer Where to write to.
220    * @param smallestReadPoint Smallest read point.
221    * @param cleanSeqId When true, remove seqId(used to be mvcc) value which is <= smallestReadPoint
222    * @return Whether compaction ended; false if it was interrupted for some reason.
223    */
224   protected boolean performCompaction(InternalScanner scanner,
225       CellSink writer, long smallestReadPoint, boolean cleanSeqId) throws IOException {
226     int bytesWritten = 0;
227     // Since scanner.next() can return 'false' but still be delivering data,
228     // we have to use a do/while loop.
229     List<Cell> kvs = new ArrayList<Cell>();
230     int closeCheckInterval = HStore.getCloseCheckInterval();
231     long lastMillis;
232     if (LOG.isDebugEnabled()) {
233       lastMillis = System.currentTimeMillis();
234     } else {
235       lastMillis = 0;
236     }
237     boolean hasMore;
238     do {
239       hasMore = scanner.next(kvs, compactionKVMax);
240       // output to writer:
241       for (Cell c : kvs) {
242         KeyValue kv = KeyValueUtil.ensureKeyValue(c);
243         if (cleanSeqId && kv.getSequenceId() <= smallestReadPoint) {
244           kv.setSequenceId(0);
245         }
246         writer.append(kv);
247         ++progress.currentCompactedKVs;
248         progress.totalCompactedSize += kv.getLength();
249 
250         // check periodically to see if a system stop is requested
251         if (closeCheckInterval > 0) {
252           bytesWritten += kv.getLength();
253           if (bytesWritten > closeCheckInterval) {
254             // Log the progress of long running compactions every minute if
255             // logging at DEBUG level
256             if (LOG.isDebugEnabled()) {
257               long now = System.currentTimeMillis();
258               if ((now - lastMillis) >= 60 * 1000) {
259                 LOG.debug("Compaction progress: " + progress + String.format(", rate=%.2f kB/sec",
260                   (bytesWritten / 1024.0) / ((now - lastMillis) / 1000.0)));
261                 lastMillis = now;
262               }
263             }
264             bytesWritten = 0;
265             if (!store.areWritesEnabled()) {
266               progress.cancel();
267               return false;
268             }
269           }
270         }
271       }
272       kvs.clear();
273     } while (hasMore);
274     progress.complete();
275     return true;
276   }
277 
278   /**
279    * @param store store
280    * @param scanners Store file scanners.
281    * @param scanType Scan type.
282    * @param smallestReadPoint Smallest MVCC read point.
283    * @param earliestPutTs Earliest put across all files.
284    * @return A compaction scanner.
285    */
286   protected InternalScanner createScanner(Store store, List<StoreFileScanner> scanners,
287       ScanType scanType, long smallestReadPoint, long earliestPutTs) throws IOException {
288     Scan scan = new Scan();
289     scan.setMaxVersions(store.getFamily().getMaxVersions());
290     return new StoreScanner(store, store.getScanInfo(), scan, scanners,
291         scanType, smallestReadPoint, earliestPutTs);
292   }
293 
294   /**
295    * @param store The store.
296    * @param scanners Store file scanners.
297    * @param smallestReadPoint Smallest MVCC read point.
298    * @param earliestPutTs Earliest put across all files.
299    * @param dropDeletesFromRow Drop deletes starting with this row, inclusive. Can be null.
300    * @param dropDeletesToRow Drop deletes ending with this row, exclusive. Can be null.
301    * @return A compaction scanner.
302    */
303   protected InternalScanner createScanner(Store store, List<StoreFileScanner> scanners,
304      long smallestReadPoint, long earliestPutTs, byte[] dropDeletesFromRow,
305      byte[] dropDeletesToRow) throws IOException {
306     Scan scan = new Scan();
307     scan.setMaxVersions(store.getFamily().getMaxVersions());
308     return new StoreScanner(store, store.getScanInfo(), scan, scanners, smallestReadPoint,
309         earliestPutTs, dropDeletesFromRow, dropDeletesToRow);
310   }
311 }