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