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  
22  import org.apache.commons.logging.Log;
23  import org.apache.commons.logging.LogFactory;
24  import org.apache.hadoop.conf.Configuration;
25  import org.apache.hadoop.fs.FileSystem;
26  import org.apache.hadoop.fs.Path;
27  import org.apache.hadoop.hbase.classification.InterfaceAudience;
28  import org.apache.hadoop.hbase.regionserver.AbstractMultiFileWriter;
29  import org.apache.hadoop.hbase.regionserver.AbstractMultiFileWriter.WriterFactory;
30  import org.apache.hadoop.hbase.regionserver.InternalScanner;
31  import org.apache.hadoop.hbase.regionserver.Store;
32  import org.apache.hadoop.hbase.regionserver.StoreFile.Writer;
33  import org.apache.hadoop.hbase.regionserver.StoreScanner;
34  
35  /**
36   * Base class for implementing a Compactor which will generate multiple output files after
37   * compaction.
38   */
39  @InterfaceAudience.Private
40  public abstract class AbstractMultiOutputCompactor<T extends AbstractMultiFileWriter>
41      extends Compactor<T> {
42  
43    private static final Log LOG = LogFactory.getLog(AbstractMultiOutputCompactor.class);
44  
45    public AbstractMultiOutputCompactor(Configuration conf, Store store) {
46      super(conf, store);
47    }
48  
49    protected void initMultiWriter(AbstractMultiFileWriter writer, InternalScanner scanner,
50        final FileDetails fd, final boolean shouldDropBehind) {
51      WriterFactory writerFactory = new WriterFactory() {
52        @Override
53        public Writer createWriter() throws IOException {
54          return createTmpWriter(fd, shouldDropBehind);
55        }
56      };
57      // Prepare multi-writer, and perform the compaction using scanner and writer.
58      // It is ok here if storeScanner is null.
59      StoreScanner storeScanner = (scanner instanceof StoreScanner) ? (StoreScanner) scanner : null;
60      writer.init(storeScanner, writerFactory);
61    }
62  
63    @Override
64    protected void abortWriter(T writer) throws IOException {
65      FileSystem fs = store.getFileSystem();
66      for (Path leftoverFile : writer.abortWriters()) {
67        try {
68          fs.delete(leftoverFile, false);
69        } catch (IOException e) {
70          LOG.warn(
71            "Failed to delete the leftover file " + leftoverFile + " after an unfinished compaction.",
72            e);
73        }
74      }
75    }
76  }