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;
19  
20  import java.io.IOException;
21  import java.util.Collection;
22  import java.util.List;
23  import java.util.NavigableSet;
24  
25  import org.apache.hadoop.hbase.classification.InterfaceAudience;
26  import org.apache.hadoop.hbase.classification.InterfaceStability;
27  import org.apache.hadoop.fs.FileSystem;
28  import org.apache.hadoop.fs.Path;
29  import org.apache.hadoop.hbase.Cell;
30  import org.apache.hadoop.hbase.HBaseInterfaceAudience;
31  import org.apache.hadoop.hbase.HColumnDescriptor;
32  import org.apache.hadoop.hbase.HRegionInfo;
33  import org.apache.hadoop.hbase.KeyValue;
34  import org.apache.hadoop.hbase.TableName;
35  import org.apache.hadoop.hbase.client.Scan;
36  import org.apache.hadoop.hbase.conf.PropagatingConfigurationObserver;
37  import org.apache.hadoop.hbase.io.HeapSize;
38  import org.apache.hadoop.hbase.io.compress.Compression;
39  import org.apache.hadoop.hbase.io.hfile.CacheConfig;
40  import org.apache.hadoop.hbase.io.hfile.HFileDataBlockEncoder;
41  import org.apache.hadoop.hbase.protobuf.generated.WALProtos.CompactionDescriptor;
42  import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
43  import org.apache.hadoop.hbase.regionserver.compactions.CompactionProgress;
44  import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
45  import org.apache.hadoop.hbase.regionserver.compactions.CompactionThroughputController;
46  import org.apache.hadoop.hbase.security.User;
47  import org.apache.hadoop.hbase.util.Pair;
48  
49  /**
50   * Interface for objects that hold a column family in a Region. Its a memstore and a set of zero or
51   * more StoreFiles, which stretch backwards over time.
52   */
53  @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC)
54  @InterfaceStability.Evolving
55  public interface Store extends HeapSize, StoreConfigInformation, PropagatingConfigurationObserver {
56  
57    /* The default priority for user-specified compaction requests.
58     * The user gets top priority unless we have blocking compactions. (Pri <= 0)
59     */ int PRIORITY_USER = 1;
60    int NO_PRIORITY = Integer.MIN_VALUE;
61  
62    // General Accessors
63    KeyValue.KVComparator getComparator();
64  
65    Collection<StoreFile> getStorefiles();
66  
67    /**
68     * Close all the readers We don't need to worry about subsequent requests because the Region
69     * holds a write lock that will prevent any more reads or writes.
70     * @return the {@link StoreFile StoreFiles} that were previously being used.
71     * @throws IOException on failure
72     */
73    Collection<StoreFile> close() throws IOException;
74  
75    /**
76     * Return a scanner for both the memstore and the HStore files. Assumes we are not in a
77     * compaction.
78     * @param scan Scan to apply when scanning the stores
79     * @param targetCols columns to scan
80     * @return a scanner over the current key values
81     * @throws IOException on failure
82     */
83    KeyValueScanner getScanner(Scan scan, final NavigableSet<byte[]> targetCols, long readPt)
84        throws IOException;
85  
86    /**
87     * Get all scanners with no filtering based on TTL (that happens further down
88     * the line).
89     * @param cacheBlocks
90     * @param isGet
91     * @param usePread
92     * @param isCompaction
93     * @param matcher
94     * @param startRow
95     * @param stopRow
96     * @param readPt
97     * @return all scanners for this store
98     */
99    List<KeyValueScanner> getScanners(
100     boolean cacheBlocks,
101     boolean isGet,
102     boolean usePread,
103     boolean isCompaction,
104     ScanQueryMatcher matcher,
105     byte[] startRow,
106     byte[] stopRow,
107     long readPt
108   ) throws IOException;
109 
110   ScanInfo getScanInfo();
111 
112   /**
113    * Adds or replaces the specified KeyValues.
114    * <p>
115    * For each KeyValue specified, if a cell with the same row, family, and qualifier exists in
116    * MemStore, it will be replaced. Otherwise, it will just be inserted to MemStore.
117    * <p>
118    * This operation is atomic on each KeyValue (row/family/qualifier) but not necessarily atomic
119    * across all of them.
120    * @param cells
121    * @param readpoint readpoint below which we can safely remove duplicate KVs
122    * @return memstore size delta
123    * @throws IOException
124    */
125   long upsert(Iterable<Cell> cells, long readpoint) throws IOException;
126 
127   /**
128    * Adds a value to the memstore
129    * @param cell
130    * @return memstore size delta & newly added KV which maybe different than the passed in KV
131    */
132   Pair<Long, Cell> add(Cell cell);
133 
134   /**
135    * When was the last edit done in the memstore
136    */
137   long timeOfOldestEdit();
138 
139   /**
140    * Removes a Cell from the memstore. The Cell is removed only if its key & memstoreTS match the
141    * key & memstoreTS value of the cell parameter.
142    * @param cell
143    */
144   void rollback(final Cell cell);
145 
146   /**
147    * Find the key that matches <i>row</i> exactly, or the one that immediately precedes it. WARNING:
148    * Only use this method on a table where writes occur with strictly increasing timestamps. This
149    * method assumes this pattern of writes in order to make it reasonably performant. Also our
150    * search is dependent on the axiom that deletes are for cells that are in the container that
151    * follows whether a memstore snapshot or a storefile, not for the current container: i.e. we'll
152    * see deletes before we come across cells we are to delete. Presumption is that the
153    * memstore#kvset is processed before memstore#snapshot and so on.
154    * @param row The row key of the targeted row.
155    * @return Found Cell or null if none found.
156    * @throws IOException
157    */
158   Cell getRowKeyAtOrBefore(final byte[] row) throws IOException;
159 
160   FileSystem getFileSystem();
161 
162   /*
163    * @param maxKeyCount
164    * @param compression Compression algorithm to use
165    * @param isCompaction whether we are creating a new file in a compaction
166    * @param includeMVCCReadpoint whether we should out the MVCC readpoint
167    * @return Writer for a new StoreFile in the tmp dir.
168    */
169   StoreFile.Writer createWriterInTmp(
170     long maxKeyCount,
171     Compression.Algorithm compression,
172     boolean isCompaction,
173     boolean includeMVCCReadpoint,
174     boolean includesTags
175   ) throws IOException;
176 
177   // Compaction oriented methods
178 
179   boolean throttleCompaction(long compactionSize);
180 
181   /**
182    * getter for CompactionProgress object
183    * @return CompactionProgress object; can be null
184    */
185   CompactionProgress getCompactionProgress();
186 
187   CompactionContext requestCompaction() throws IOException;
188 
189   /**
190    * @deprecated see requestCompaction(int, CompactionRequest, User)
191    */
192   @Deprecated
193   CompactionContext requestCompaction(int priority, CompactionRequest baseRequest)
194       throws IOException;
195 
196   CompactionContext requestCompaction(int priority, CompactionRequest baseRequest, User user)
197       throws IOException;
198 
199   void cancelRequestedCompaction(CompactionContext compaction);
200 
201   /**
202    * @deprecated see compact(CompactionContext, CompactionThroughputController, User)
203    */
204   @Deprecated
205   List<StoreFile> compact(CompactionContext compaction,
206       CompactionThroughputController throughputController) throws IOException;
207 
208   List<StoreFile> compact(CompactionContext compaction,
209     CompactionThroughputController throughputController, User user) throws IOException;
210 
211   /**
212    * @return true if we should run a major compaction.
213    */
214   boolean isMajorCompaction() throws IOException;
215 
216   void triggerMajorCompaction();
217 
218   /**
219    * See if there's too much store files in this store
220    * @return true if number of store files is greater than the number defined in minFilesToCompact
221    */
222   boolean needsCompaction();
223 
224   int getCompactPriority();
225 
226   StoreFlushContext createFlushContext(long cacheFlushId);
227 
228   /**
229    * Call to complete a compaction. Its for the case where we find in the WAL a compaction
230    * that was not finished.  We could find one recovering a WAL after a regionserver crash.
231    * See HBASE-2331.
232    * @param compaction the descriptor for compaction
233    * @param pickCompactionFiles whether or not pick up the new compaction output files and
234    * add it to the store
235    * @param removeFiles whether to remove/archive files from filesystem
236    */
237   void replayCompactionMarker(CompactionDescriptor compaction, boolean pickCompactionFiles,
238       boolean removeFiles)
239       throws IOException;
240 
241   // Split oriented methods
242 
243   boolean canSplit();
244 
245   /**
246    * Determines if Store should be split
247    * @return byte[] if store should be split, null otherwise.
248    */
249   byte[] getSplitPoint();
250 
251   // Bulk Load methods
252 
253   /**
254    * This throws a WrongRegionException if the HFile does not fit in this region, or an
255    * InvalidHFileException if the HFile is not valid.
256    */
257   void assertBulkLoadHFileOk(Path srcPath) throws IOException;
258 
259   /**
260    * This method should only be called from Region. It is assumed that the ranges of values in the
261    * HFile fit within the stores assigned region. (assertBulkLoadHFileOk checks this)
262    *
263    * @param srcPathStr
264    * @param sequenceId sequence Id associated with the HFile
265    */
266   Path bulkLoadHFile(String srcPathStr, long sequenceId) throws IOException;
267 
268   // General accessors into the state of the store
269   // TODO abstract some of this out into a metrics class
270 
271   /**
272    * @return <tt>true</tt> if the store has any underlying reference files to older HFiles
273    */
274   boolean hasReferences();
275 
276   /**
277    * @return The size of this store's memstore, in bytes
278    */
279   long getMemStoreSize();
280 
281   /**
282    * @return The amount of memory we could flush from this memstore; usually this is equal to
283    * {@link #getMemStoreSize()} unless we are carrying snapshots and then it will be the size of
284    * outstanding snapshots.
285    */
286   long getFlushableSize();
287 
288   /**
289    * Returns the memstore snapshot size
290    * @return size of the memstore snapshot
291    */
292   long getSnapshotSize();
293 
294   HColumnDescriptor getFamily();
295 
296   /**
297    * @return The maximum sequence id in all store files.
298    */
299   long getMaxSequenceId();
300 
301   /**
302    * @return The maximum memstoreTS in all store files.
303    */
304   long getMaxMemstoreTS();
305 
306   /**
307    * @return the data block encoder
308    */
309   HFileDataBlockEncoder getDataBlockEncoder();
310 
311   /** @return aggregate size of all HStores used in the last compaction */
312   long getLastCompactSize();
313 
314   /** @return aggregate size of HStore */
315   long getSize();
316 
317   /**
318    * @return Count of store files
319    */
320   int getStorefilesCount();
321 
322   /**
323    * @return Max age of store files in this store
324    */
325   long getMaxStoreFileAge();
326 
327   /**
328    * @return Min age of store files in this store
329    */
330   long getMinStoreFileAge();
331 
332   /**
333    *  @return Average age of store files in this store, 0 if no store files
334    */
335   long getAvgStoreFileAge();
336 
337   /**
338    *  @return Number of reference files in this store
339    */
340   long getNumReferenceFiles();
341 
342   /**
343    *  @return Number of HFiles in this store
344    */
345   long getNumHFiles();
346 
347   /**
348    * @return The size of the store files, in bytes, uncompressed.
349    */
350   long getStoreSizeUncompressed();
351 
352   /**
353    * @return The size of the store files, in bytes.
354    */
355   long getStorefilesSize();
356 
357   /**
358    * @return The size of the store file indexes, in bytes.
359    */
360   long getStorefilesIndexSize();
361 
362   /**
363    * Returns the total size of all index blocks in the data block indexes, including the root level,
364    * intermediate levels, and the leaf level for multi-level indexes, or just the root level for
365    * single-level indexes.
366    * @return the total size of block indexes in the store
367    */
368   long getTotalStaticIndexSize();
369 
370   /**
371    * Returns the total byte size of all Bloom filter bit arrays. For compound Bloom filters even the
372    * Bloom blocks currently not loaded into the block cache are counted.
373    * @return the total size of all Bloom filters in the store
374    */
375   long getTotalStaticBloomSize();
376 
377   // Test-helper methods
378 
379   /**
380    * Used for tests.
381    * @return cache configuration for this Store.
382    */
383   CacheConfig getCacheConfig();
384 
385   /**
386    * @return the parent region info hosting this store
387    */
388   HRegionInfo getRegionInfo();
389 
390   RegionCoprocessorHost getCoprocessorHost();
391 
392   boolean areWritesEnabled();
393 
394   /**
395    * @return The smallest mvcc readPoint across all the scanners in this
396    * region. Writes older than this readPoint, are included  in every
397    * read operation.
398    */
399   long getSmallestReadPoint();
400 
401   String getColumnFamilyName();
402 
403   TableName getTableName();
404 
405   /**
406    * @return The number of cells flushed to disk
407    */
408   long getFlushedCellsCount();
409 
410   /**
411    * @return The total size of data flushed to disk, in bytes
412    */
413   long getFlushedCellsSize();
414 
415   /**
416    * @return The total size of out output files on disk, in bytes
417    */
418   long getFlushedOutputFileSize();
419 
420   /**
421    * @return The number of cells processed during minor compactions
422    */
423   long getCompactedCellsCount();
424 
425   /**
426    * @return The total amount of data processed during minor compactions, in bytes
427    */
428   long getCompactedCellsSize();
429 
430   /**
431    * @return The number of cells processed during major compactions
432    */
433   long getMajorCompactedCellsCount();
434 
435   /**
436    * @return The total amount of data processed during major compactions, in bytes
437    */
438   long getMajorCompactedCellsSize();
439 
440   /*
441    * @param o Observer who wants to know about changes in set of Readers
442    */
443   void addChangedReaderObserver(ChangedReadersObserver o);
444 
445   /*
446    * @param o Observer no longer interested in changes in set of Readers.
447    */
448   void deleteChangedReaderObserver(ChangedReadersObserver o);
449 
450   /**
451    * @return Whether this store has too many store files.
452    */
453   boolean hasTooManyStoreFiles();
454 
455   /**
456    * Checks the underlying store files, and opens the files that  have not
457    * been opened, and removes the store file readers for store files no longer
458    * available. Mainly used by secondary region replicas to keep up to date with
459    * the primary region files.
460    * @throws IOException
461    */
462   void refreshStoreFiles() throws IOException;
463 
464   /**
465    * This value can represent the degree of emergency of compaction for this store. It should be
466    * greater than or equal to 0.0, any value greater than 1.0 means we have too many store files.
467    * <ul>
468    * <li>if getStorefilesCount &lt;= getMinFilesToCompact, return 0.0</li>
469    * <li>return (getStorefilesCount - getMinFilesToCompact) / (blockingFileCount -
470    * getMinFilesToCompact)</li>
471    * </ul>
472    * <p>
473    * And for striped stores, we should calculate this value by the files in each stripe separately
474    * and return the maximum value.
475    * <p>
476    * It is similar to {@link #getCompactPriority()} except that it is more suitable to use in a
477    * linear formula.
478    */
479   double getCompactionPressure();
480 
481    /**
482     * Replaces the store files that the store has with the given files. Mainly used by
483     * secondary region replicas to keep up to date with
484     * the primary region files.
485     * @throws IOException
486     */
487   void refreshStoreFiles(Collection<String> newFiles) throws IOException;
488 
489   void bulkLoadHFile(StoreFileInfo fileInfo) throws IOException;
490 
491   boolean isPrimaryReplicaStore();
492 }