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.mob;
19  
20  import java.io.IOException;
21  import java.util.ArrayList;
22  import java.util.Date;
23  import java.util.List;
24  
25  import org.apache.commons.logging.Log;
26  import org.apache.commons.logging.LogFactory;
27  import org.apache.hadoop.hbase.classification.InterfaceAudience;
28  import org.apache.hadoop.conf.Configuration;
29  import org.apache.hadoop.fs.Path;
30  import org.apache.hadoop.hbase.Cell;
31  import org.apache.hadoop.hbase.CellUtil;
32  import org.apache.hadoop.hbase.KeyValue;
33  import org.apache.hadoop.hbase.KeyValueUtil;
34  import org.apache.hadoop.hbase.Tag;
35  import org.apache.hadoop.hbase.TagType;
36  import org.apache.hadoop.hbase.client.Scan;
37  import org.apache.hadoop.hbase.regionserver.*;
38  import org.apache.hadoop.hbase.regionserver.StoreFile.Writer;
39  import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
40  import org.apache.hadoop.hbase.regionserver.compactions.CompactionThroughputController;
41  import org.apache.hadoop.hbase.regionserver.compactions.DefaultCompactor;
42  import org.apache.hadoop.hbase.security.User;
43  import org.apache.hadoop.hbase.util.Bytes;
44  
45  /**
46   * Compact passed set of files in the mob-enabled column family.
47   */
48  @InterfaceAudience.Private
49  public class DefaultMobStoreCompactor extends DefaultCompactor {
50  
51    private static final Log LOG = LogFactory.getLog(DefaultMobStoreCompactor.class);
52    private long mobSizeThreshold;
53    private HMobStore mobStore;
54  
55    private final InternalScannerFactory scannerFactory = new InternalScannerFactory() {
56  
57      @Override
58      public ScanType getScanType(CompactionRequest request) {
59        return request.isRetainDeleteMarkers() ? ScanType.COMPACT_RETAIN_DELETES
60            : ScanType.COMPACT_DROP_DELETES;
61      }
62  
63      @Override
64      public InternalScanner createScanner(List<StoreFileScanner> scanners,
65          ScanType scanType, FileDetails fd, long smallestReadPoint) throws IOException {
66        Scan scan = new Scan();
67        scan.setMaxVersions(store.getFamily().getMaxVersions());
68        if (scanType == ScanType.COMPACT_DROP_DELETES) {
69          // In major compaction, we need to write the delete markers to del files, so we have to
70          // retain the them in scanning.
71          scanType = ScanType.COMPACT_RETAIN_DELETES;
72          return new MobCompactionStoreScanner(store, store.getScanInfo(), scan, scanners,
73              scanType, smallestReadPoint, fd.earliestPutTs, true);
74        } else {
75          return new MobCompactionStoreScanner(store, store.getScanInfo(), scan, scanners,
76              scanType, smallestReadPoint, fd.earliestPutTs, false);
77        }
78      }
79    };
80  
81    private final CellSinkFactory<Writer> writerFactory = new CellSinkFactory<Writer>() {
82  
83      @Override
84      public Writer createWriter(InternalScanner scanner,
85          org.apache.hadoop.hbase.regionserver.compactions.Compactor.FileDetails fd,
86          boolean shouldDropBehind) throws IOException {
87        // make this writer with tags always because of possible new cells with tags.
88        return store.createWriterInTmp(fd.maxKeyCount, compactionCompression, true, true, true);
89      }
90    };
91  
92    public DefaultMobStoreCompactor(Configuration conf, Store store) {
93      super(conf, store);
94      // The mob cells reside in the mob-enabled column family which is held by HMobStore.
95      // During the compaction, the compactor reads the cells from the mob files and
96      // probably creates new mob files. All of these operations are included in HMobStore,
97      // so we need to cast the Store to HMobStore.
98      if (!(store instanceof HMobStore)) {
99        throw new IllegalArgumentException("The store " + store + " is not a HMobStore");
100     }
101     mobStore = (HMobStore) store;
102     mobSizeThreshold = store.getFamily().getMobThreshold();
103   }
104 
105   @Override
106   public List<Path> compact(CompactionRequest request,
107     CompactionThroughputController throughputController, User user) throws IOException {
108     return compact(request, scannerFactory, writerFactory, throughputController, user);
109   }
110 
111   // TODO refactor to take advantage of the throughput controller.
112 
113   /**
114    * Performs compaction on a column family with the mob flag enabled.
115    * This is for when the mob threshold size has changed or if the mob
116    * column family mode has been toggled via an alter table statement.
117    * Compacts the files by the following rules.
118    * 1. If the cell has a mob reference tag, the cell's value is the path of the mob file.
119    * <ol>
120    * <li>
121    * If the value size of a cell is larger than the threshold, this cell is regarded as a mob,
122    * directly copy the (with mob tag) cell into the new store file.
123    * </li>
124    * <li>
125    * Otherwise, retrieve the mob cell from the mob file, and writes a copy of the cell into
126    * the new store file.
127    * </li>
128    * </ol>
129    * 2. If the cell doesn't have a reference tag.
130    * <ol>
131    * <li>
132    * If the value size of a cell is larger than the threshold, this cell is regarded as a mob,
133    * write this cell to a mob file, and write the path of this mob file to the store file.
134    * </li>
135    * <li>
136    * Otherwise, directly write this cell into the store file.
137    * </li>
138    * </ol>
139    * In the mob compaction, the {@link MobCompactionStoreScanner} is used as a scanner
140    * which could output the normal cells and delete markers together when required.
141    * After the major compaction on the normal hfiles, we have a guarantee that we have purged all
142    * deleted or old version mob refs, and the delete markers are written to a del file with the
143    * suffix _del. Because of this, it is safe to use the del file in the mob compaction.
144    * The mob compaction doesn't take place in the normal hfiles, it occurs directly in the
145    * mob files. When the small mob files are merged into bigger ones, the del file is added into
146    * the scanner to filter the deleted cells.
147    * @param fd File details
148    * @param scanner Where to read from.
149    * @param writer Where to write to.
150    * @param smallestReadPoint Smallest read point.
151    * @param cleanSeqId When true, remove seqId(used to be mvcc) value which is <= smallestReadPoint
152    * @param throughputController The compaction throughput controller.
153    * @param major Is a major compaction.
154    * @return Whether compaction ended; false if it was interrupted for any reason.
155    */
156   @Override
157   protected boolean performCompaction(FileDetails fd, InternalScanner scanner, CellSink writer,
158       long smallestReadPoint, boolean cleanSeqId,
159       CompactionThroughputController throughputController,  boolean major) throws IOException {
160     if (!(scanner instanceof MobCompactionStoreScanner)) {
161       throw new IllegalArgumentException(
162           "The scanner should be an instance of MobCompactionStoreScanner");
163     }
164     MobCompactionStoreScanner compactionScanner = (MobCompactionStoreScanner) scanner;
165     int bytesWritten = 0;
166     // Since scanner.next() can return 'false' but still be delivering data,
167     // we have to use a do/while loop.
168     List<Cell> cells = new ArrayList<Cell>();
169     // Limit to "hbase.hstore.compaction.kv.max" (default 10) to avoid OOME
170     int closeCheckInterval = HStore.getCloseCheckInterval();
171     boolean hasMore;
172     Path path = MobUtils.getMobFamilyPath(conf, store.getTableName(), store.getColumnFamilyName());
173     byte[] fileName = null;
174     Writer mobFileWriter = null, delFileWriter = null;
175     long mobCells = 0;
176     long deleteMarkersCount = 0;
177     Tag tableNameTag = new Tag(TagType.MOB_TABLE_NAME_TAG_TYPE, store.getTableName()
178             .getName());
179     long cellsCountCompactedToMob = 0;
180     long cellsCountCompactedFromMob = 0;
181     long cellsSizeCompactedToMob = 0;
182     long cellsSizeCompactedFromMob = 0;
183     try {
184       try {
185         // If the mob file writer could not be created, directly write the cell to the store file.
186         mobFileWriter = mobStore.createWriterInTmp(new Date(fd.latestPutTs), fd.maxKeyCount,
187             store.getFamily().getCompression(), store.getRegionInfo().getStartKey());
188         fileName = Bytes.toBytes(mobFileWriter.getPath().getName());
189       } catch (IOException e) {
190         LOG.error(
191             "Failed to create mob writer, "
192                 + "we will continue the compaction by writing MOB cells directly in store files",
193             e);
194       }
195       delFileWriter = mobStore.createDelFileWriterInTmp(new Date(fd.latestPutTs), fd.maxKeyCount,
196           store.getFamily().getCompression(), store.getRegionInfo().getStartKey());
197       ScannerContext scannerContext =
198               ScannerContext.newBuilder().setBatchLimit(compactionKVMax).build();
199       do {
200         hasMore = compactionScanner.next(cells, scannerContext);
201         // output to writer:
202         for (Cell c : cells) {
203           if (compactionScanner.isOutputDeleteMarkers() && CellUtil.isDelete(c)) {
204             delFileWriter.append(c);
205             deleteMarkersCount++;
206           } else if (mobFileWriter == null || c.getTypeByte() != KeyValue.Type.Put.getCode()) {
207             // If the mob file writer is null or the kv type is not put, directly write the cell
208             // to the store file.
209             writer.append(c);
210           } else if (MobUtils.isMobReferenceCell(c)) {
211             if (MobUtils.hasValidMobRefCellValue(c)) {
212               int size = MobUtils.getMobValueLength(c);
213               if (size > mobSizeThreshold) {
214                 // If the value size is larger than the threshold, it's regarded as a mob. Since
215                 // its value is already in the mob file, directly write this cell to the store file
216                 writer.append(c);
217               } else {
218                 // If the value is not larger than the threshold, it's not regarded a mob. Retrieve
219                 // the mob cell from the mob file, and write it back to the store file.
220                 Cell mobCell = mobStore.resolve(c, false);
221                 if (mobCell.getValueLength() != 0) {
222                   // put the mob data back to the store file
223                   CellUtil.setSequenceId(mobCell, c.getSequenceId());
224                   writer.append(mobCell);
225                   cellsCountCompactedFromMob++;
226                   cellsSizeCompactedFromMob += mobCell.getValueLength();
227                 } else {
228                   // If the value of a file is empty, there might be issues when retrieving,
229                   // directly write the cell to the store file, and leave it to be handled by the
230                   // next compaction.
231                   writer.append(c);
232                 }
233               }
234             } else {
235               LOG.warn("The value format of the KeyValue " + c
236                   + " is wrong, its length is less than " + Bytes.SIZEOF_INT);
237               writer.append(c);
238             }
239           } else if (c.getValueLength() <= mobSizeThreshold) {
240             // If the value size of a cell is not larger than the threshold, directly write it to
241             // the store file.
242             writer.append(c);
243           } else {
244             // If the value size of a cell is larger than the threshold, it's regarded as a mob,
245             // write this cell to a mob file, and write the path to the store file.
246             mobCells++;
247             // append the original keyValue in the mob file.
248             mobFileWriter.append(c);
249             KeyValue reference = MobUtils.createMobRefKeyValue(c, fileName, tableNameTag);
250             // write the cell whose value is the path of a mob file to the store file.
251             writer.append(reference);
252             cellsCountCompactedToMob++;
253             cellsSizeCompactedToMob += c.getValueLength();
254           }
255           ++progress.currentCompactedKVs;
256 
257           // check periodically to see if a system stop is requested
258           if (closeCheckInterval > 0) {
259             bytesWritten += KeyValueUtil.length(c);
260             if (bytesWritten > closeCheckInterval) {
261               bytesWritten = 0;
262               if (!store.areWritesEnabled()) {
263                 progress.cancel();
264                 return false;
265               }
266             }
267           }
268         }
269         cells.clear();
270       } while (hasMore);
271     } finally {
272       if (mobFileWriter != null) {
273         mobFileWriter.appendMetadata(fd.maxSeqId, major, mobCells);
274         mobFileWriter.close();
275       }
276       if (delFileWriter != null) {
277         delFileWriter.appendMetadata(fd.maxSeqId, major, deleteMarkersCount);
278         delFileWriter.close();
279       }
280     }
281     if (mobFileWriter != null) {
282       if (mobCells > 0) {
283         // If the mob file is not empty, commit it.
284         mobStore.commitFile(mobFileWriter.getPath(), path);
285       } else {
286         try {
287           // If the mob file is empty, delete it instead of committing.
288           store.getFileSystem().delete(mobFileWriter.getPath(), true);
289         } catch (IOException e) {
290           LOG.error("Failed to delete the temp mob file", e);
291         }
292       }
293     }
294     if (delFileWriter != null) {
295       if (deleteMarkersCount > 0) {
296         // If the del file is not empty, commit it.
297         // If the commit fails, the compaction is re-performed again.
298         mobStore.commitFile(delFileWriter.getPath(), path);
299       } else {
300         try {
301           // If the del file is empty, delete it instead of committing.
302           store.getFileSystem().delete(delFileWriter.getPath(), true);
303         } catch (IOException e) {
304           LOG.error("Failed to delete the temp del file", e);
305         }
306       }
307     }
308     mobStore.updateCellsCountCompactedFromMob(cellsCountCompactedFromMob);
309     mobStore.updateCellsCountCompactedToMob(cellsCountCompactedToMob);
310     mobStore.updateCellsSizeCompactedFromMob(cellsSizeCompactedFromMob);
311     mobStore.updateCellsSizeCompactedToMob(cellsSizeCompactedToMob);
312     progress.complete();
313     return true;
314   }
315 }