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.List;
22  
23  import org.apache.commons.logging.Log;
24  import org.apache.commons.logging.LogFactory;
25  import org.apache.hadoop.conf.Configuration;
26  import org.apache.hadoop.fs.Path;
27  import org.apache.hadoop.hbase.classification.InterfaceAudience;
28  import org.apache.hadoop.hbase.regionserver.InternalScanner;
29  import org.apache.hadoop.hbase.regionserver.ScanType;
30  import org.apache.hadoop.hbase.regionserver.Store;
31  import org.apache.hadoop.hbase.regionserver.StoreFileScanner;
32  import org.apache.hadoop.hbase.regionserver.StripeMultiFileWriter;
33  import org.apache.hadoop.hbase.security.User;
34  import org.apache.hadoop.hbase.util.Bytes;
35  
36  /**
37   * This is the placeholder for stripe compactor. The implementation, as well as the proper javadoc,
38   * will be added in HBASE-7967.
39   */
40  @InterfaceAudience.Private
41  public class StripeCompactor extends AbstractMultiOutputCompactor<StripeMultiFileWriter> {
42    private static final Log LOG = LogFactory.getLog(StripeCompactor.class);
43  
44    public StripeCompactor(Configuration conf, Store store) {
45      super(conf, store);
46    }
47  
48    private final class StripeInternalScannerFactory implements InternalScannerFactory {
49  
50      private final byte[] majorRangeFromRow;
51  
52      private final byte[] majorRangeToRow;
53  
54      public StripeInternalScannerFactory(byte[] majorRangeFromRow, byte[] majorRangeToRow) {
55        this.majorRangeFromRow = majorRangeFromRow;
56        this.majorRangeToRow = majorRangeToRow;
57      }
58  
59      @Override
60      public ScanType getScanType(CompactionRequest request) {
61        // If majorRangeFromRow and majorRangeToRow are not null, then we will not use the return
62        // value to create InternalScanner. See the createScanner method below. The return value is
63        // also used when calling coprocessor hooks.
64        return ScanType.COMPACT_RETAIN_DELETES;
65      }
66  
67      @Override
68      public InternalScanner createScanner(List<StoreFileScanner> scanners, ScanType scanType,
69          FileDetails fd, long smallestReadPoint) throws IOException {
70        return (majorRangeFromRow == null)
71            ? StripeCompactor.this.createScanner(store, scanners, scanType, smallestReadPoint,
72              fd.earliestPutTs)
73            : StripeCompactor.this.createScanner(store, scanners, smallestReadPoint, fd.earliestPutTs,
74              majorRangeFromRow, majorRangeToRow);
75      }
76    }
77  
78    public List<Path> compact(CompactionRequest request, final List<byte[]> targetBoundaries,
79        final byte[] majorRangeFromRow, final byte[] majorRangeToRow,
80        CompactionThroughputController throughputController, User user) throws IOException {
81      if (LOG.isDebugEnabled()) {
82        StringBuilder sb = new StringBuilder();
83        sb.append("Executing compaction with " + targetBoundaries.size() + " boundaries:");
84        for (byte[] tb : targetBoundaries) {
85          sb.append(" [").append(Bytes.toString(tb)).append("]");
86        }
87        LOG.debug(sb.toString());
88      }
89      return compact(request, new StripeInternalScannerFactory(majorRangeFromRow, majorRangeToRow),
90        new CellSinkFactory<StripeMultiFileWriter>() {
91  
92          @Override
93          public StripeMultiFileWriter createWriter(InternalScanner scanner, FileDetails fd,
94              boolean shouldDropBehind) throws IOException {
95            StripeMultiFileWriter writer = new StripeMultiFileWriter.BoundaryMultiWriter(
96                store.getComparator(), targetBoundaries, majorRangeFromRow, majorRangeToRow);
97            initMultiWriter(writer, scanner, fd, shouldDropBehind);
98            return writer;
99          }
100       }, throughputController, user);
101   }
102 
103   public List<Path> compact(CompactionRequest request, final int targetCount, final long targetSize,
104       final byte[] left, final byte[] right, byte[] majorRangeFromRow, byte[] majorRangeToRow,
105       CompactionThroughputController throughputController, User user) throws IOException {
106     if (LOG.isDebugEnabled()) {
107       LOG.debug(
108         "Executing compaction with " + targetSize + " target file size, no more than " + targetCount
109             + " files, in [" + Bytes.toString(left) + "] [" + Bytes.toString(right) + "] range");
110     }
111     return compact(request, new StripeInternalScannerFactory(majorRangeFromRow, majorRangeToRow),
112       new CellSinkFactory<StripeMultiFileWriter>() {
113 
114         @Override
115         public StripeMultiFileWriter createWriter(InternalScanner scanner, FileDetails fd,
116             boolean shouldDropBehind) throws IOException {
117           StripeMultiFileWriter writer = new StripeMultiFileWriter.SizeMultiWriter(
118               store.getComparator(), targetCount, targetSize, left, right);
119           initMultiWriter(writer, scanner, fd, shouldDropBehind);
120           return writer;
121         }
122       }, throughputController, user);
123   }
124 
125   @Override
126   protected List<Path> commitWriter(StripeMultiFileWriter writer, FileDetails fd,
127       CompactionRequest request) throws IOException {
128     List<Path> newFiles = writer.commitWriters(fd.maxSeqId, request.isMajor());
129     assert !newFiles.isEmpty() : "Should have produced an empty file to preserve metadata.";
130     return newFiles;
131   }
132 }