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.Collection;
22  import java.util.List;
23  
24  import org.apache.commons.logging.Log;
25  import org.apache.commons.logging.LogFactory;
26  import org.apache.hadoop.conf.Configuration;
27  import org.apache.hadoop.fs.Path;
28  import org.apache.hadoop.hbase.classification.InterfaceAudience;
29  import org.apache.hadoop.hbase.regionserver.InternalScanner;
30  import org.apache.hadoop.hbase.regionserver.Store;
31  import org.apache.hadoop.hbase.regionserver.StoreFile;
32  import org.apache.hadoop.hbase.regionserver.StoreFile.Writer;
33  import org.apache.hadoop.hbase.security.User;
34  
35  import com.google.common.collect.Lists;
36  
37  /**
38   * Compact passed set of files. Create an instance and then call
39   * {@link #compact(CompactionRequest, CompactionThroughputController, User)}
40   */
41  @InterfaceAudience.Private
42  public class DefaultCompactor extends Compactor<Writer> {
43    private static final Log LOG = LogFactory.getLog(DefaultCompactor.class);
44  
45    public DefaultCompactor(final Configuration conf, final Store store) {
46      super(conf, store);
47    }
48  
49    private final CellSinkFactory<Writer> writerFactory = new CellSinkFactory<Writer>() {
50  
51      @Override
52      public Writer createWriter(InternalScanner scanner,
53          org.apache.hadoop.hbase.regionserver.compactions.Compactor.FileDetails fd,
54          boolean shouldDropBehind) throws IOException {
55        return createTmpWriter(fd, shouldDropBehind);
56      }
57    };
58  
59    /**
60     * Do a minor/major compaction on an explicit set of storefiles from a Store.
61     */
62    public List<Path> compact(final CompactionRequest request,
63        CompactionThroughputController throughputController, User user) throws IOException {
64      return compact(request, defaultScannerFactory, writerFactory, throughputController, user);
65    }
66  
67    /**
68     * Compact a list of files for testing. Creates a fake {@link CompactionRequest} to pass to
69     * {@link #compact(CompactionRequest, CompactionThroughputController, User)};
70     * @param filesToCompact the files to compact. These are used as the compactionSelection for
71     *          the generated {@link CompactionRequest}.
72     * @param isMajor true to major compact (prune all deletes, max versions, etc)
73     * @return Product of compaction or an empty list if all cells expired or deleted and nothing \
74     *         made it through the compaction.
75     * @throws IOException
76     */
77    public List<Path> compactForTesting(final Collection<StoreFile> filesToCompact, boolean isMajor)
78        throws IOException {
79      CompactionRequest cr = new CompactionRequest(filesToCompact);
80      cr.setIsMajor(isMajor);
81      return this.compact(cr, NoLimitCompactionThroughputController.INSTANCE, null);
82    }
83  
84    @Override
85    protected List<Path> commitWriter(Writer writer, FileDetails fd,
86        CompactionRequest request) throws IOException {
87      List<Path> newFiles = Lists.newArrayList(writer.getPath());
88      writer.appendMetadata(fd.maxSeqId, request.isMajor());
89      writer.close();
90      return newFiles;
91    }
92  
93    @Override
94    protected void abortWriter(Writer writer) throws IOException {
95      Path leftoverFile = writer.getPath();
96      try {
97        writer.close();
98      } catch (IOException e) {
99        LOG.warn("Failed to close the writer after an unfinished compaction.", e);
100     }
101     try {
102       store.getFileSystem().delete(leftoverFile, false);
103     } catch (IOException e) {
104       LOG.warn(
105         "Failed to delete the leftover file " + leftoverFile + " after an unfinished compaction.",
106         e);
107     }
108   }
109 }