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.List;
22  
23  import org.apache.commons.logging.Log;
24  import org.apache.commons.logging.LogFactory;
25  import org.apache.hadoop.conf.Configuration;
26  import org.apache.hadoop.fs.Path;
27  import org.apache.hadoop.hbase.classification.InterfaceAudience;
28  import org.apache.hadoop.hbase.regionserver.DateTieredMultiFileWriter;
29  import org.apache.hadoop.hbase.regionserver.InternalScanner;
30  import org.apache.hadoop.hbase.regionserver.ScanType;
31  import org.apache.hadoop.hbase.regionserver.Store;
32  import org.apache.hadoop.hbase.regionserver.StoreFile;
33  import org.apache.hadoop.hbase.regionserver.StoreFileScanner;
34  import org.apache.hadoop.hbase.security.User;
35  
36  /**
37   * This compactor will generate StoreFile for different time ranges.
38   */
39  @InterfaceAudience.Private
40  public class DateTieredCompactor extends AbstractMultiOutputCompactor<DateTieredMultiFileWriter> {
41  
42    private static final Log LOG = LogFactory.getLog(DateTieredCompactor.class);
43  
44    public DateTieredCompactor(Configuration conf, Store store) {
45      super(conf, store);
46    }
47  
48    private boolean needEmptyFile(CompactionRequest request) {
49      // if we are going to compact the last N files, then we need to emit an empty file to retain the
50      // maxSeqId if we haven't written out anything.
51      return StoreFile.getMaxSequenceIdInList(request.getFiles()) == store.getMaxSequenceId();
52    }
53  
54    public List<Path> compact(final CompactionRequest request, final List<Long> lowerBoundaries,
55      CompactionThroughputController throughputController, User user) throws IOException {
56      if (LOG.isDebugEnabled()) {
57        LOG.debug("Executing compaction with " + lowerBoundaries.size()
58            + "windows, lower boundaries: " + lowerBoundaries);
59      }
60  
61      return compact(request, defaultScannerFactory,
62        new CellSinkFactory<DateTieredMultiFileWriter>() {
63  
64          @Override
65          public DateTieredMultiFileWriter createWriter(InternalScanner scanner, FileDetails fd,
66              boolean shouldDropBehind) throws IOException {
67            DateTieredMultiFileWriter writer = new DateTieredMultiFileWriter(lowerBoundaries,
68                needEmptyFile(request));
69            initMultiWriter(writer, scanner, fd, shouldDropBehind);
70            return writer;
71          }
72        }, throughputController, user);
73    }
74  
75    @Override
76    protected List<Path> commitWriter(DateTieredMultiFileWriter writer, FileDetails fd,
77        CompactionRequest request) throws IOException {
78      return writer.commitWriters(fd.maxSeqId, request.isAllFiles());
79    }
80  }