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