HBASE-18446 Mark StoreFileScanner/StoreFileReader as IA.LimitedPrivate(Phoenix)

This commit is contained in:
zhangduo 2017-09-15 16:35:21 +08:00
parent d90f77ab7d
commit a5c8461ca8
3 changed files with 16 additions and 7 deletions

View File

@ -239,8 +239,7 @@ public interface RegionObserver extends Coprocessor {
* effect in this hook.
* @param c the environment provided by the region server
* @param store the store being compacted
* @param scanners the list {@link org.apache.hadoop.hbase.regionserver.StoreFileScanner}s
* to be read from
* @param scanners the list of store file scanners to be read from
* @param scanType the {@link ScanType} indicating whether this is a major or minor compaction
* @param earliestPutTs timestamp of the earliest put that was found in any of the involved store
* files
@ -1034,7 +1033,9 @@ public interface RegionObserver extends Coprocessor {
* @param reader the base reader, if not {@code null}, from previous RegionObserver in the chain
* @return a Reader instance to use instead of the base reader if overriding
* default behavior, null otherwise
* @deprecated For Phoenix only, StoreFileReader is not a stable interface.
*/
@Deprecated
default StoreFileReader preStoreFileReaderOpen(ObserverContext<RegionCoprocessorEnvironment> ctx,
FileSystem fs, Path p, FSDataInputStreamWrapper in, long size, CacheConfig cacheConf,
Reference r, StoreFileReader reader) throws IOException {
@ -1053,7 +1054,9 @@ public interface RegionObserver extends Coprocessor {
* @param r original reference file. This will be not null only when reading a split file.
* @param reader the base reader instance
* @return The reader to use
* @deprecated For Phoenix only, StoreFileReader is not a stable interface.
*/
@Deprecated
default StoreFileReader postStoreFileReaderOpen(ObserverContext<RegionCoprocessorEnvironment> ctx,
FileSystem fs, Path p, FSDataInputStreamWrapper in, long size, CacheConfig cacheConf,
Reference r, StoreFileReader reader) throws IOException {

View File

@ -18,8 +18,6 @@
*/
package org.apache.hadoop.hbase.regionserver;
import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
import java.io.DataInput;
import java.io.IOException;
import java.util.Map;
@ -34,9 +32,9 @@ import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellComparator;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper;
import org.apache.hadoop.hbase.io.TimeRange;
@ -49,11 +47,16 @@ import org.apache.hadoop.hbase.nio.ByteBuff;
import org.apache.hadoop.hbase.util.BloomFilter;
import org.apache.hadoop.hbase.util.BloomFilterFactory;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.yetus.audience.InterfaceStability;
import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
/**
* Reader for a StoreFile.
*/
@InterfaceAudience.Private
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.PHOENIX)
@InterfaceStability.Evolving
public class StoreFileReader {
private static final Log LOG = LogFactory.getLog(StoreFileReader.class.getName());

View File

@ -32,8 +32,10 @@ import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellComparator;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.HConstants;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.yetus.audience.InterfaceStability;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.io.TimeRange;
import org.apache.hadoop.hbase.io.hfile.HFileScanner;
@ -43,7 +45,8 @@ import org.apache.hadoop.hbase.regionserver.querymatcher.ScanQueryMatcher;
* KeyValueScanner adaptor over the Reader. It also provides hooks into
* bloom filter things.
*/
@InterfaceAudience.LimitedPrivate("Coprocessor")
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.PHOENIX)
@InterfaceStability.Evolving
public class StoreFileScanner implements KeyValueScanner {
// the reader it comes from:
private final StoreFileReader reader;