001/**
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *     http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018package org.apache.hadoop.hbase.regionserver;
019
020import java.io.IOException;
021import java.util.Collection;
022import java.util.OptionalDouble;
023import java.util.OptionalLong;
024
025import org.apache.hadoop.fs.FileSystem;
026import org.apache.hadoop.hbase.CellComparator;
027import org.apache.hadoop.hbase.HBaseInterfaceAudience;
028import org.apache.hadoop.hbase.TableName;
029import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
030import org.apache.hadoop.hbase.client.RegionInfo;
031import org.apache.yetus.audience.InterfaceAudience;
032import org.apache.yetus.audience.InterfaceStability;
033
034/**
035 * Interface for objects that hold a column family in a Region. Its a memstore and a set of zero or
036 * more StoreFiles, which stretch backwards over time.
037 */
038@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC)
039@InterfaceStability.Evolving
040public interface Store {
041
042  /**
043   * The default priority for user-specified compaction requests.
044   * The user gets top priority unless we have blocking compactions. (Pri <= 0)
045   */
046  int PRIORITY_USER = 1;
047  int NO_PRIORITY = Integer.MIN_VALUE;
048
049  // General Accessors
050  CellComparator getComparator();
051
052  Collection<? extends StoreFile> getStorefiles();
053
054  Collection<? extends StoreFile> getCompactedFiles();
055
056  /**
057   * When was the last edit done in the memstore
058   */
059  long timeOfOldestEdit();
060
061  FileSystem getFileSystem();
062
063  /**
064   * Tests whether we should run a major compaction. For example, if the configured major compaction
065   * interval is reached.
066   * @return true if we should run a major compaction.
067   */
068  boolean shouldPerformMajorCompaction() throws IOException;
069
070  /**
071   * See if there's too much store files in this store
072   * @return <code>true</code> if number of store files is greater than the number defined in
073   *         minFilesToCompact
074   */
075  boolean needsCompaction();
076
077  int getCompactPriority();
078
079  /**
080   * Returns whether this store is splittable, i.e., no reference file in this store.
081   */
082  boolean canSplit();
083
084  /**
085   * @return <code>true</code> if the store has any underlying reference files to older HFiles
086   */
087  boolean hasReferences();
088
089  /**
090   * @return The size of this store's memstore.
091   */
092  MemStoreSize getMemStoreSize();
093
094  /**
095   * @return The amount of memory we could flush from this memstore; usually this is equal to
096   * {@link #getMemStoreSize()} unless we are carrying snapshots and then it will be the size of
097   * outstanding snapshots.
098   */
099  MemStoreSize getFlushableSize();
100
101  /**
102   * @return size of the memstore snapshot
103   */
104  MemStoreSize getSnapshotSize();
105
106  ColumnFamilyDescriptor getColumnFamilyDescriptor();
107
108  /**
109   * @return The maximum sequence id in all store files.
110   */
111  OptionalLong getMaxSequenceId();
112
113  /**
114   * @return The maximum memstoreTS in all store files.
115   */
116  OptionalLong getMaxMemStoreTS();
117
118  /** @return aggregate size of all HStores used in the last compaction */
119  long getLastCompactSize();
120
121  /** @return aggregate size of HStore */
122  long getSize();
123
124  /**
125   * @return Count of store files
126   */
127  int getStorefilesCount();
128
129  /**
130   * @return Count of compacted store files
131   */
132  int getCompactedFilesCount();
133
134  /**
135   * @return Max age of store files in this store
136   */
137  OptionalLong getMaxStoreFileAge();
138
139  /**
140   * @return Min age of store files in this store
141   */
142  OptionalLong getMinStoreFileAge();
143
144  /**
145   *  @return Average age of store files in this store
146   */
147  OptionalDouble getAvgStoreFileAge();
148
149  /**
150   *  @return Number of reference files in this store
151   */
152  long getNumReferenceFiles();
153
154  /**
155   *  @return Number of HFiles in this store
156   */
157  long getNumHFiles();
158
159  /**
160   * @return The size of the store files, in bytes, uncompressed.
161   */
162  long getStoreSizeUncompressed();
163
164  /**
165   * @return The size of the store files, in bytes.
166   */
167  long getStorefilesSize();
168
169  /**
170   * @return The size of only the store files which are HFiles, in bytes.
171   */
172  long getHFilesSize();
173
174  /**
175   * @return The size of the store file root-level indexes, in bytes.
176   */
177  long getStorefilesRootLevelIndexSize();
178
179  /**
180   * Returns the total size of all index blocks in the data block indexes, including the root level,
181   * intermediate levels, and the leaf level for multi-level indexes, or just the root level for
182   * single-level indexes.
183   * @return the total size of block indexes in the store
184   */
185  long getTotalStaticIndexSize();
186
187  /**
188   * Returns the total byte size of all Bloom filter bit arrays. For compound Bloom filters even the
189   * Bloom blocks currently not loaded into the block cache are counted.
190   * @return the total size of all Bloom filters in the store
191   */
192  long getTotalStaticBloomSize();
193
194  /**
195   * @return the parent region info hosting this store
196   */
197  RegionInfo getRegionInfo();
198
199  boolean areWritesEnabled();
200
201  /**
202   * @return The smallest mvcc readPoint across all the scanners in this
203   * region. Writes older than this readPoint, are included  in every
204   * read operation.
205   */
206  long getSmallestReadPoint();
207
208  String getColumnFamilyName();
209
210  TableName getTableName();
211
212  /**
213   * @return The number of cells flushed to disk
214   */
215  long getFlushedCellsCount();
216
217  /**
218   * @return The total size of data flushed to disk, in bytes
219   */
220  long getFlushedCellsSize();
221
222  /**
223   * @return The total size of out output files on disk, in bytes
224   */
225  long getFlushedOutputFileSize();
226
227  /**
228   * @return The number of cells processed during minor compactions
229   */
230  long getCompactedCellsCount();
231
232  /**
233   * @return The total amount of data processed during minor compactions, in bytes
234   */
235  long getCompactedCellsSize();
236
237  /**
238   * @return The number of cells processed during major compactions
239   */
240  long getMajorCompactedCellsCount();
241
242  /**
243   * @return The total amount of data processed during major compactions, in bytes
244   */
245  long getMajorCompactedCellsSize();
246
247  /**
248   * @return Whether this store has too many store files.
249   */
250  boolean hasTooManyStoreFiles();
251
252  /**
253   * Checks the underlying store files, and opens the files that have not been opened, and removes
254   * the store file readers for store files no longer available. Mainly used by secondary region
255   * replicas to keep up to date with the primary region files.
256   * @throws IOException
257   */
258  void refreshStoreFiles() throws IOException;
259
260  /**
261   * This value can represent the degree of emergency of compaction for this store. It should be
262   * greater than or equal to 0.0, any value greater than 1.0 means we have too many store files.
263   * <ul>
264   * <li>if getStorefilesCount &lt;= getMinFilesToCompact, return 0.0</li>
265   * <li>return (getStorefilesCount - getMinFilesToCompact) / (blockingFileCount -
266   * getMinFilesToCompact)</li>
267   * </ul>
268   * <p>
269   * And for striped stores, we should calculate this value by the files in each stripe separately
270   * and return the maximum value.
271   * <p>
272   * It is similar to {@link #getCompactPriority()} except that it is more suitable to use in a
273   * linear formula.
274   */
275  double getCompactionPressure();
276
277  boolean isPrimaryReplicaStore();
278
279  /**
280   * @return true if the memstore may need some extra memory space
281   */
282  boolean isSloppyMemStore();
283
284  int getCurrentParallelPutCount();
285}