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.List;
25  
26  import org.apache.commons.logging.Log;
27  import org.apache.commons.logging.LogFactory;
28  import org.apache.hadoop.conf.Configuration;
29  import org.apache.hadoop.fs.Path;
30  import org.apache.hadoop.hbase.classification.InterfaceAudience;
31  import org.apache.hadoop.hbase.io.compress.Compression;
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.StoreFile.Writer;
37  import org.apache.hadoop.hbase.regionserver.StoreFileScanner;
38  import org.apache.hadoop.hbase.regionserver.StoreScanner;
39  import org.apache.hadoop.hbase.regionserver.StripeMultiFileWriter;
40  import org.apache.hadoop.hbase.util.Bytes;
41  
42  /**
43   * This is the placeholder for stripe compactor. The implementation,
44   * as well as the proper javadoc, will be added in HBASE-7967.
45   */
46  @InterfaceAudience.Private
47  public class StripeCompactor extends Compactor {
48    private static final Log LOG = LogFactory.getLog(StripeCompactor.class);
49    public StripeCompactor(Configuration conf, Store store) {
50      super(conf, store);
51    }
52  
53    public List<Path> compact(CompactionRequest request, List<byte[]> targetBoundaries,
54        byte[] majorRangeFromRow, byte[] majorRangeToRow,
55        CompactionThroughputController throughputController) throws IOException {
56      if (LOG.isDebugEnabled()) {
57        StringBuilder sb = new StringBuilder();
58        sb.append("Executing compaction with " + targetBoundaries.size() + " boundaries:");
59        for (byte[] tb : targetBoundaries) {
60          sb.append(" [").append(Bytes.toString(tb)).append("]");
61        }
62        LOG.debug(sb.toString());
63      }
64      StripeMultiFileWriter writer = new StripeMultiFileWriter.BoundaryMultiWriter(
65          targetBoundaries, majorRangeFromRow, majorRangeToRow);
66      return compactInternal(writer, request, majorRangeFromRow, majorRangeToRow,
67        throughputController);
68    }
69  
70    public List<Path> compact(CompactionRequest request, int targetCount, long targetSize,
71        byte[] left, byte[] right, byte[] majorRangeFromRow, byte[] majorRangeToRow,
72        CompactionThroughputController throughputController) throws IOException {
73      if (LOG.isDebugEnabled()) {
74        LOG.debug("Executing compaction with " + targetSize
75            + " target file size, no more than " + targetCount + " files, in ["
76            + Bytes.toString(left) + "] [" + Bytes.toString(right) + "] range");
77      }
78      StripeMultiFileWriter writer = new StripeMultiFileWriter.SizeMultiWriter(
79          targetCount, targetSize, left, right);
80      return compactInternal(writer, request, majorRangeFromRow, majorRangeToRow,
81        throughputController);
82    }
83  
84    private List<Path> compactInternal(StripeMultiFileWriter mw, CompactionRequest request,
85        byte[] majorRangeFromRow, byte[] majorRangeToRow,
86        CompactionThroughputController throughputController) throws IOException {
87      final Collection<StoreFile> filesToCompact = request.getFiles();
88      final FileDetails fd = getFileDetails(filesToCompact, request.isMajor());
89      this.progress = new CompactionProgress(fd.maxKeyCount);
90  
91      long smallestReadPoint = getSmallestReadPoint();
92      List<StoreFileScanner> scanners = createFileScanners(filesToCompact, smallestReadPoint);
93  
94      boolean finished = false;
95      InternalScanner scanner = null;
96      try {
97        // Get scanner to use.
98        ScanType coprocScanType = ScanType.COMPACT_RETAIN_DELETES;
99        scanner = preCreateCoprocScanner(request, coprocScanType, fd.earliestPutTs, scanners);
100       if (scanner == null) {
101         scanner = (majorRangeFromRow == null)
102             ? createScanner(store, scanners,
103                 ScanType.COMPACT_RETAIN_DELETES, smallestReadPoint, fd.earliestPutTs)
104             : createScanner(store, scanners,
105                 smallestReadPoint, fd.earliestPutTs, majorRangeFromRow, majorRangeToRow);
106       }
107       scanner = postCreateCoprocScanner(request, coprocScanType, scanner);
108       if (scanner == null) {
109         // NULL scanner returned from coprocessor hooks means skip normal processing.
110         return new ArrayList<Path>();
111       }
112 
113       // Create the writer factory for compactions.
114       final boolean needMvcc = fd.maxMVCCReadpoint >= smallestReadPoint;
115       final Compression.Algorithm compression = store.getFamily().getCompactionCompression();
116       StripeMultiFileWriter.WriterFactory factory = new StripeMultiFileWriter.WriterFactory() {
117         @Override
118         public Writer createWriter() throws IOException {
119           return store.createWriterInTmp(
120               fd.maxKeyCount, compression, true, needMvcc, fd.maxTagsLength > 0);
121         }
122       };
123 
124       // Prepare multi-writer, and perform the compaction using scanner and writer.
125       // It is ok here if storeScanner is null.
126       StoreScanner storeScanner = (scanner instanceof StoreScanner) ? (StoreScanner)scanner : null;
127       mw.init(storeScanner, factory, store.getComparator());
128       finished = performCompaction(scanner, mw, smallestReadPoint, throughputController);
129       if (!finished) {
130         throw new InterruptedIOException( "Aborting compaction of store " + store +
131             " in region " + store.getRegionInfo().getRegionNameAsString() +
132             " because it was interrupted.");
133       }
134     } finally {
135       if (scanner != null) {
136         try {
137           scanner.close();
138         } catch (Throwable t) {
139           // Don't fail the compaction if this fails.
140           LOG.error("Failed to close scanner after compaction.", t);
141         }
142       }
143       if (!finished) {
144         for (Path leftoverFile : mw.abortWriters()) {
145           try {
146             store.getFileSystem().delete(leftoverFile, false);
147           } catch (Exception ex) {
148             LOG.error("Failed to delete the leftover file after an unfinished compaction.", ex);
149           }
150         }
151       }
152     }
153 
154     assert finished : "We should have exited the method on all error paths";
155     List<Path> newFiles = mw.commitWriters(fd.maxSeqId, request.isMajor());
156     assert !newFiles.isEmpty() : "Should have produced an empty file to preserve metadata.";
157     return newFiles;
158   }
159 }