HDFS-14149. [SBN read] Fix annotations on new interfaces/classes for SBN reads. Contributed by Chao Sun.

This commit is contained in:
Chao Sun 2018-12-17 16:53:43 -08:00 committed by Chen Liang
parent 7a650b018c
commit 8f36bdc3b9
8 changed files with 22 additions and 4 deletions

View File

@ -34,7 +34,7 @@ import org.apache.hadoop.ipc.protobuf.RpcHeaderProtos.RpcResponseHeaderProto;
* to client.
*/
@InterfaceAudience.Private
@InterfaceStability.Stable
@InterfaceStability.Evolving
public interface AlignmentContext {
/**

View File

@ -17,6 +17,7 @@
*/
package org.apache.hadoop.ipc;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import java.io.IOException;
@ -26,6 +27,7 @@ import java.io.IOException;
* client should retry active namenode directly (instead of retry other
* ObserverNodes).
*/
@InterfaceAudience.Private
@InterfaceStability.Evolving
public class ObserverRetryOnActiveException extends IOException {
static final long serialVersionUID = 1L;

View File

@ -34,7 +34,7 @@ import java.util.concurrent.atomic.LongAccumulator;
* state alignment info from server(s).
*/
@InterfaceAudience.Private
@InterfaceStability.Stable
@InterfaceStability.Evolving
public class ClientGSIContext implements AlignmentContext {
private final LongAccumulator lastSeenStateId =

View File

@ -26,6 +26,8 @@ import java.lang.reflect.Proxy;
import java.net.URI;
import java.util.List;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
import org.apache.hadoop.hdfs.ClientGSIContext;
@ -58,6 +60,8 @@ import com.google.common.annotations.VisibleForTesting;
* Read and write requests will still be sent to active NN if reading from
* observer is turned off.
*/
@InterfaceAudience.Private
@InterfaceStability.Evolving
public class ObserverReadProxyProvider<T extends ClientProtocol>
extends AbstractNNFailoverProxyProvider<T> {
private static final Logger LOG = LoggerFactory.getLogger(

View File

@ -21,6 +21,9 @@ import java.net.InetSocketAddress;
import java.net.URI;
import java.util.Collections;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
import org.apache.hadoop.hdfs.HAUtilClient;
@ -74,7 +77,8 @@ import org.slf4j.LoggerFactory;
* nn01-ha1 and nn01-ha2, used for read/write RPC calls, but for the failover,
* it relies on the virtual address nn01.com
*/
@InterfaceAudience.Private
@InterfaceStability.Evolving
public class ObserverReadProxyProviderWithIPFailover<T extends ClientProtocol>
extends ObserverReadProxyProvider<T> {
private static final Logger LOG = LoggerFactory.getLogger(

View File

@ -22,6 +22,8 @@ import java.lang.annotation.Inherited;
import java.lang.annotation.Retention;
import java.lang.annotation.RetentionPolicy;
import java.lang.annotation.Target;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
/**
@ -30,6 +32,7 @@ import org.apache.hadoop.classification.InterfaceStability;
@Inherited
@Retention(RetentionPolicy.RUNTIME)
@Target(ElementType.METHOD)
@InterfaceAudience.Private
@InterfaceStability.Evolving
public @interface ReadOnly {
/**

View File

@ -31,6 +31,9 @@ import java.util.NavigableMap;
import java.util.TreeMap;
import java.util.concurrent.locks.ReadWriteLock;
import java.util.concurrent.locks.ReentrantReadWriteLock;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.server.namenode.EditLogFileOutputStream;
@ -67,6 +70,8 @@ import org.apache.hadoop.util.AutoCloseableLock;
* cache misses occurred, and how many more transactions would have been
* needed in the cache to serve the request.
*/
@InterfaceAudience.Private
@InterfaceStability.Evolving
class JournaledEditsCache {
private static final int INVALID_LAYOUT_VERSION = 0;

View File

@ -37,7 +37,7 @@ import org.apache.hadoop.ipc.protobuf.RpcHeaderProtos.RpcResponseHeaderProto;
* state alignment info to clients.
*/
@InterfaceAudience.Private
@InterfaceStability.Stable
@InterfaceStability.Evolving
class GlobalStateIdContext implements AlignmentContext {
/**
* Estimated number of journal transactions a typical NameNode can execute