|
@@ -27,7 +27,6 @@ import java.util.HashMap;
|
|
|
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.hdfs.shortcircuit.ShortCircuitShm.SlotId;
|
|
|
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 @@ public class 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 class PBHelperClient {
|
|
|
}
|
|
|
|
|
|
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 class PBHelperClient {
|
|
|
|
|
|
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
|