HADOOP-16621. [pb-upgrade] Remove Protobuf classes from signatures of Public APIs. Contributed by Vinayakumar B. (#1803)

This commit is contained in:
Vinayakumar B 2020-01-16 23:27:50 +05:30 committed by GitHub
parent a0ff42d761
commit edbbc03ce7
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
6 changed files with 84 additions and 74 deletions

View File

@ -460,4 +460,10 @@
<Method name="save"/>
<Bug pattern="OBL_UNSATISFIED_OBLIGATION"/>
</Match>
<Match>
<Class name="org.apache.hadoop.ipc.ProtobufHelper" />
<Method name="getFixedByteString" />
<Bug pattern="AT_OPERATION_SEQUENCE_ON_CONCURRENT_ABSTRACTION" />
</Match>
</FindBugsFilter>

View File

@ -18,9 +18,15 @@
package org.apache.hadoop.ipc;
import java.io.IOException;
import java.util.concurrent.ConcurrentHashMap;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.security.proto.SecurityProtos.TokenProto;
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.security.token.TokenIdentifier;
import com.google.protobuf.ByteString;
import com.google.protobuf.ServiceException;
/**
@ -46,4 +52,67 @@ public static IOException getRemoteException(ServiceException se) {
}
return e instanceof IOException ? (IOException) e : new IOException(se);
}
/**
* Map used to cache fixed strings to ByteStrings. Since there is no
* automatic expiration policy, only use this for strings from a fixed, small
* set.
* <p/>
* This map should not be accessed directly. Used the getFixedByteString
* methods instead.
*/
private final static ConcurrentHashMap<Object, ByteString>
FIXED_BYTESTRING_CACHE = new ConcurrentHashMap<>();
/**
* Get the ByteString for frequently used fixed and small set strings.
* @param key string
* @return
*/
public static ByteString getFixedByteString(Text key) {
ByteString value = FIXED_BYTESTRING_CACHE.get(key);
if (value == null) {
value = ByteString.copyFromUtf8(key.toString());
FIXED_BYTESTRING_CACHE.put(new Text(key.copyBytes()), value);
}
return value;
}
/**
* Get the ByteString for frequently used fixed and small set strings.
* @param key string
* @return
*/
public static ByteString getFixedByteString(String key) {
ByteString value = FIXED_BYTESTRING_CACHE.get(key);
if (value == null) {
value = ByteString.copyFromUtf8(key);
FIXED_BYTESTRING_CACHE.put(key, value);
}
return value;
}
public static ByteString getByteString(byte[] bytes) {
// return singleton to reduce object allocation
return (bytes.length == 0) ? ByteString.EMPTY : ByteString.copyFrom(bytes);
}
public static Token<? extends TokenIdentifier> tokenFromProto(
TokenProto tokenProto) {
Token<? extends TokenIdentifier> token = new Token<>(
tokenProto.getIdentifier().toByteArray(),
tokenProto.getPassword().toByteArray(), new Text(tokenProto.getKind()),
new Text(tokenProto.getService()));
return token;
}
public static TokenProto protoFromToken(Token<?> tok) {
TokenProto.Builder builder = TokenProto.newBuilder().
setIdentifier(getByteString(tok.getIdentifier())).
setPassword(getByteString(tok.getPassword())).
setKindBytes(getFixedByteString(tok.getKind())).
setServiceBytes(getFixedByteString(tok.getService()));
return builder.build();
}
}

View File

@ -46,6 +46,7 @@
import org.apache.hadoop.io.Text;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.io.WritableUtils;
import org.apache.hadoop.ipc.ProtobufHelper;
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.security.token.TokenIdentifier;
import org.apache.hadoop.security.proto.SecurityProtos.CredentialsKVProto;
@ -368,7 +369,7 @@ void writeProto(DataOutput out) throws IOException {
CredentialsKVProto.Builder kv = CredentialsKVProto.newBuilder().
setAliasBytes(ByteString.copyFrom(
e.getKey().getBytes(), 0, e.getKey().getLength())).
setToken(e.getValue().toTokenProto());
setToken(ProtobufHelper.protoFromToken(e.getValue()));
storage.addTokens(kv.build());
}
@ -390,7 +391,7 @@ void readProto(DataInput in) throws IOException {
CredentialsProto storage = CredentialsProto.parseDelimitedFrom((DataInputStream)in);
for (CredentialsKVProto kv : storage.getTokensList()) {
addToken(new Text(kv.getAliasBytes().toByteArray()),
(Token<? extends TokenIdentifier>) new Token(kv.getToken()));
ProtobufHelper.tokenFromProto(kv.getToken()));
}
for (CredentialsKVProto kv : storage.getSecretsList()) {
addSecretKey(new Text(kv.getAliasBytes().toByteArray()),

View File

@ -19,7 +19,6 @@
package org.apache.hadoop.security.token;
import com.google.common.collect.Maps;
import com.google.protobuf.ByteString;
import com.google.common.primitives.Bytes;
import org.apache.commons.codec.binary.Base64;
@ -28,7 +27,6 @@
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.io.*;
import org.apache.hadoop.security.proto.SecurityProtos.TokenProto;
import org.apache.hadoop.util.ReflectionUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -117,32 +115,6 @@ public Token<T> copyToken() {
return new Token<T>(this);
}
/**
* Construct a Token from a TokenProto.
* @param tokenPB the TokenProto object
*/
public Token(TokenProto tokenPB) {
this.identifier = tokenPB.getIdentifier().toByteArray();
this.password = tokenPB.getPassword().toByteArray();
this.kind = new Text(tokenPB.getKindBytes().toByteArray());
this.service = new Text(tokenPB.getServiceBytes().toByteArray());
}
/**
* Construct a TokenProto from this Token instance.
* @return a new TokenProto object holding copies of data in this instance
*/
public TokenProto toTokenProto() {
return TokenProto.newBuilder().
setIdentifier(ByteString.copyFrom(this.getIdentifier())).
setPassword(ByteString.copyFrom(this.getPassword())).
setKindBytes(ByteString.copyFrom(
this.getKind().getBytes(), 0, this.getKind().getLength())).
setServiceBytes(ByteString.copyFrom(
this.getService().getBytes(), 0, this.getService().getLength())).
build();
}
/**
* Get the token identifier's byte representation.
* @return the token identifier's byte representation

View File

@ -92,10 +92,5 @@
<Method name="getSymlinkInBytes" />
<Bug pattern="EI_EXPOSE_REP" />
</Match>
<Match>
<Class name="org.apache.hadoop.hdfs.protocolPB.PBHelperClient" />
<Method name="getFixedByteString" />
<Bug pattern="AT_OPERATION_SEQUENCE_ON_CONCURRENT_ABSTRACTION" />
</Match>
</FindBugsFilter>

View File

@ -27,7 +27,6 @@
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import com.google.common.base.Preconditions;
import com.google.common.cache.CacheBuilder;
@ -207,6 +206,7 @@
import org.apache.hadoop.io.EnumSetWritable;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.io.erasurecode.ECSchema;
import org.apache.hadoop.ipc.ProtobufHelper;
import org.apache.hadoop.security.proto.SecurityProtos.TokenProto;
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.util.ChunkedArrayList;
@ -232,33 +232,8 @@ public class PBHelperClient {
private static final FsAction[] FSACTION_VALUES =
FsAction.values();
/**
* Map used to cache fixed strings to ByteStrings. Since there is no
* automatic expiration policy, only use this for strings from a fixed, small
* set.
* <p/>
* This map should not be accessed directly. Used the getFixedByteString
* methods instead.
*/
private static ConcurrentHashMap<Object, ByteString> fixedByteStringCache =
new ConcurrentHashMap<>();
private static ByteString getFixedByteString(Text key) {
ByteString value = fixedByteStringCache.get(key);
if (value == null) {
value = ByteString.copyFromUtf8(key.toString());
fixedByteStringCache.put(new Text(key.copyBytes()), value);
}
return value;
}
private static ByteString getFixedByteString(String key) {
ByteString value = fixedByteStringCache.get(key);
if (value == null) {
value = ByteString.copyFromUtf8(key);
fixedByteStringCache.put(key, value);
}
return value;
return ProtobufHelper.getFixedByteString(key);
}
/**
@ -281,7 +256,7 @@ private PBHelperClient() {
public static ByteString getByteString(byte[] bytes) {
// return singleton to reduce object allocation
return (bytes.length == 0) ? ByteString.EMPTY : ByteString.copyFrom(bytes);
return ProtobufHelper.getByteString(bytes);
}
public static ShmId convert(ShortCircuitShmIdProto shmId) {
@ -349,12 +324,7 @@ public static ExtendedBlockProto convert(final ExtendedBlock b) {
}
public static TokenProto convert(Token<?> tok) {
TokenProto.Builder builder = TokenProto.newBuilder().
setIdentifier(getByteString(tok.getIdentifier())).
setPassword(getByteString(tok.getPassword())).
setKindBytes(getFixedByteString(tok.getKind())).
setServiceBytes(getFixedByteString(tok.getService()));
return builder.build();
return ProtobufHelper.protoFromToken(tok);
}
public static ShortCircuitShmIdProto convert(ShmId shmId) {
@ -832,11 +802,8 @@ public static StorageType[] convertStorageTypes(
public static Token<BlockTokenIdentifier> convert(
TokenProto blockToken) {
Token<BlockTokenIdentifier> token =
new Token<>(blockToken.getIdentifier()
.toByteArray(), blockToken.getPassword().toByteArray(), new Text(
blockToken.getKind()), new Text(blockToken.getService()));
return token;
return (Token<BlockTokenIdentifier>) ProtobufHelper
.tokenFromProto(blockToken);
}
// DatanodeId