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.io.InterruptedIOException;
22  import java.util.ArrayList;
23  import java.util.Collection;
24  import java.util.Collections;
25  import java.util.List;
26  
27  import org.apache.commons.logging.Log;
28  import org.apache.commons.logging.LogFactory;
29  import org.apache.hadoop.hbase.classification.InterfaceAudience;
30  import org.apache.hadoop.conf.Configuration;
31  import org.apache.hadoop.fs.Path;
32  import org.apache.hadoop.hbase.regionserver.InternalScanner;
33  import org.apache.hadoop.hbase.regionserver.ScanType;
34  import org.apache.hadoop.hbase.regionserver.Store;
35  import org.apache.hadoop.hbase.regionserver.StoreFile;
36  import org.apache.hadoop.hbase.regionserver.StoreFileScanner;
37  
38  /**
39   * Compact passed set of files. Create an instance and then call
40   * {@link #compact(CompactionRequest, CompactionThroughputController)}
41   */
42  @InterfaceAudience.Private
43  public class DefaultCompactor extends Compactor {
44    private static final Log LOG = LogFactory.getLog(DefaultCompactor.class);
45  
46    public DefaultCompactor(final Configuration conf, final Store store) {
47      super(conf, store);
48    }
49  
50    /**
51     * Do a minor/major compaction on an explicit set of storefiles from a Store.
52     */
53    public List<Path> compact(final CompactionRequest request,
54        CompactionThroughputController throughputController) throws IOException {
55      FileDetails fd = getFileDetails(request.getFiles(), request.isAllFiles());
56      this.progress = new CompactionProgress(fd.maxKeyCount);
57  
58      // Find the smallest read point across all the Scanners.
59      long smallestReadPoint = getSmallestReadPoint();
60  
61      List<StoreFileScanner> scanners;
62      Collection<StoreFile> readersToClose;
63      if (this.conf.getBoolean("hbase.regionserver.compaction.private.readers", false)) {
64        // clone all StoreFiles, so we'll do the compaction on a independent copy of StoreFiles,
65        // HFileFiles, and their readers
66        readersToClose = new ArrayList<StoreFile>(request.getFiles().size());
67        for (StoreFile f : request.getFiles()) {
68          readersToClose.add(new StoreFile(f));
69        }
70        scanners = createFileScanners(readersToClose, smallestReadPoint);
71      } else {
72        readersToClose = Collections.emptyList();
73        scanners = createFileScanners(request.getFiles(), smallestReadPoint);
74      }
75  
76      StoreFile.Writer writer = null;
77      List<Path> newFiles = new ArrayList<Path>();
78      boolean cleanSeqId = false;
79      IOException e = null;
80      try {
81        InternalScanner scanner = null;
82        try {
83          /* Include deletes, unless we are doing a compaction of all files */
84          ScanType scanType =
85              request.isAllFiles() ? ScanType.COMPACT_DROP_DELETES : ScanType.COMPACT_RETAIN_DELETES;
86          scanner = preCreateCoprocScanner(request, scanType, fd.earliestPutTs, scanners);
87          if (scanner == null) {
88            scanner = createScanner(store, scanners, scanType, smallestReadPoint, fd.earliestPutTs);
89          }
90          scanner = postCreateCoprocScanner(request, scanType, scanner);
91          if (scanner == null) {
92            // NULL scanner returned from coprocessor hooks means skip normal processing.
93            return newFiles;
94          }
95          // Create the writer even if no kv(Empty store file is also ok),
96          // because we need record the max seq id for the store file, see HBASE-6059
97          if(fd.minSeqIdToKeep > 0) {
98            smallestReadPoint = Math.min(fd.minSeqIdToKeep, smallestReadPoint);
99            cleanSeqId = true;
100         }
101 
102         // When all MVCC readpoints are 0, don't write them.
103         // See HBASE-8166, HBASE-12600, and HBASE-13389.
104         writer = store.createWriterInTmp(fd.maxKeyCount, this.compactionCompression, true,
105           fd.maxMVCCReadpoint > 0, fd.maxTagsLength > 0);
106         boolean finished =
107             performCompaction(scanner, writer, smallestReadPoint, cleanSeqId, throughputController);
108         if (!finished) {
109           writer.close();
110           store.getFileSystem().delete(writer.getPath(), false);
111           writer = null;
112           throw new InterruptedIOException( "Aborting compaction of store " + store +
113               " in region " + store.getRegionInfo().getRegionNameAsString() +
114               " because it was interrupted.");
115          }
116        } finally {
117          if (scanner != null) {
118            scanner.close();
119          }
120       }
121     } catch (IOException ioe) {
122       e = ioe;
123       // Throw the exception
124       throw ioe;
125     }
126     finally {
127       try {
128         if (writer != null) {
129           if (e != null) {
130             writer.close();
131           } else {
132             writer.appendMetadata(fd.maxSeqId, request.isAllFiles());
133             writer.close();
134             newFiles.add(writer.getPath());
135           }
136         }
137       } finally {
138         for (StoreFile f : readersToClose) {
139           try {
140             f.closeReader(true);
141           } catch (IOException ioe) {
142             LOG.warn("Exception closing " + f, ioe);
143           }
144         }
145       }
146     }
147     return newFiles;
148   }
149 
150   /**
151    * Compact a list of files for testing. Creates a fake {@link CompactionRequest} to pass to
152    * {@link #compact(CompactionRequest, CompactionThroughputController)};
153    * @param filesToCompact the files to compact. These are used as the compactionSelection for
154    *          the generated {@link CompactionRequest}.
155    * @param isMajor true to major compact (prune all deletes, max versions, etc)
156    * @return Product of compaction or an empty list if all cells expired or deleted and nothing \
157    *         made it through the compaction.
158    * @throws IOException
159    */
160   public List<Path> compactForTesting(final Collection<StoreFile> filesToCompact, boolean isMajor)
161       throws IOException {
162     CompactionRequest cr = new CompactionRequest(filesToCompact);
163     cr.setIsMajor(isMajor, isMajor);
164     return this.compact(cr, NoLimitCompactionThroughputController.INSTANCE);
165   }
166 }