Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
2 changes: 2 additions & 0 deletions hadoop-hdds/interface-server/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -83,6 +83,7 @@
<includes>
<include>InterSCMProtocol.proto</include>
<include>SCMUpdateProtocol.proto</include>
<include>SCMRatisProtocol.proto</include>
</includes>
<outputDirectory>target/generated-sources/proto-java-for-ratis</outputDirectory>
<clearOutputDirectory>false</clearOutputDirectory>
Expand All @@ -101,6 +102,7 @@
<excludes>
<exclude>InterSCMProtocol.proto</exclude>
<exclude>SCMUpdateProtocol.proto</exclude>
<exclude>SCMRatisProtocol.proto</exclude>
</excludes>
<outputDirectory>target/generated-sources/proto-java-for-protobuf-${protobuf.version}</outputDirectory>
<clearOutputDirectory>false</clearOutputDirectory>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,8 +18,6 @@
package org.apache.hadoop.hdds.scm.ha;

import com.google.common.base.Preconditions;
import com.google.protobuf.InvalidProtocolBufferException;
import com.google.protobuf.TextFormat;
import java.util.ArrayList;
import java.util.List;
import org.apache.hadoop.hdds.protocol.proto.SCMRatisProtocol.Method;
Expand All @@ -29,6 +27,8 @@
import org.apache.hadoop.hdds.scm.ha.io.CodecFactory;
import org.apache.ratis.proto.RaftProtos.StateMachineLogEntryProto;
import org.apache.ratis.protocol.Message;
import org.apache.ratis.thirdparty.com.google.protobuf.InvalidProtocolBufferException;
import org.apache.ratis.thirdparty.com.google.protobuf.TextFormat;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

Expand Down Expand Up @@ -111,9 +111,8 @@ public Message encode() throws InvalidProtocolBufferException {
}
methodBuilder.addAllArgs(args);
requestProtoBuilder.setMethod(methodBuilder.build());
return Message.valueOf(
org.apache.ratis.thirdparty.com.google.protobuf.ByteString.copyFrom(
requestProtoBuilder.build().toByteArray()));
final SCMRatisRequestProto requestProto = requestProtoBuilder.build();
return Message.valueOf(requestProto.toByteString());
}

/**
Expand All @@ -122,7 +121,7 @@ public Message encode() throws InvalidProtocolBufferException {
public static SCMRatisRequest decode(Message message)
throws InvalidProtocolBufferException {
final SCMRatisRequestProto requestProto =
SCMRatisRequestProto.parseFrom(message.getContent().toByteArray());
SCMRatisRequestProto.parseFrom(message.getContent().asReadOnlyByteBuffer());

// proto2 required-equivalent checks
if (!requestProto.hasType()) {
Expand Down Expand Up @@ -173,7 +172,7 @@ public static String smProtoToString(StateMachineLogEntryProto proto) {
StringBuilder builder = new StringBuilder();
try {
builder.append(TextFormat.shortDebugString(
SCMRatisRequestProto.parseFrom(proto.getLogData().toByteArray())));
SCMRatisRequestProto.parseFrom(proto.getLogData().asReadOnlyByteBuffer())));
} catch (Throwable ex) {
LOG.error("smProtoToString failed", ex);
builder.append("smProtoToString failed with");
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,12 +17,12 @@

package org.apache.hadoop.hdds.scm.ha;

import com.google.protobuf.InvalidProtocolBufferException;
import org.apache.hadoop.hdds.protocol.proto.SCMRatisProtocol.SCMRatisResponseProto;
import org.apache.hadoop.hdds.scm.ha.io.CodecFactory;
import org.apache.ratis.protocol.Message;
import org.apache.ratis.protocol.RaftClientReply;
import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
import org.apache.ratis.thirdparty.com.google.protobuf.InvalidProtocolBufferException;
import org.apache.ratis.thirdparty.com.google.protobuf.UnsafeByteOperations;

/**
Expand Down Expand Up @@ -92,7 +92,7 @@ public static SCMRatisResponse decode(RaftClientReply reply)
return new SCMRatisResponse();
}

final SCMRatisResponseProto responseProto = SCMRatisResponseProto.parseFrom(response.toByteArray());
final SCMRatisResponseProto responseProto = SCMRatisResponseProto.parseFrom(response.asReadOnlyByteBuffer());

// proto2 required-equivalent checks
if (!responseProto.hasType()) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,9 +17,9 @@

package org.apache.hadoop.hdds.scm.ha.io;

import com.google.protobuf.ByteString;
import com.google.protobuf.ProtoUtils;
import java.math.BigInteger;
import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
import org.apache.ratis.thirdparty.com.google.protobuf.UnsafeByteOperations;

/**
* Codec for type BigInteger.
Expand All @@ -29,7 +29,7 @@ public class BigIntegerCodec implements Codec {
@Override
public ByteString serialize(Object object) {
// BigInteger returns a new byte[].
return ProtoUtils.unsafeByteString(((BigInteger)object).toByteArray());
return UnsafeByteOperations.unsafeWrap(((BigInteger) object).toByteArray());
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,7 @@

package org.apache.hadoop.hdds.scm.ha.io;

import com.google.protobuf.ByteString;
import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;

/**
* {@link Codec} for {@code Boolean} objects.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,23 +17,27 @@

package org.apache.hadoop.hdds.scm.ha.io;

import com.google.protobuf.ByteString;
import com.google.protobuf.InvalidProtocolBufferException;
import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
import org.apache.ratis.thirdparty.com.google.protobuf.InvalidProtocolBufferException;
import org.apache.ratis.thirdparty.com.google.protobuf.UnsafeByteOperations;

/**
* A dummy codec that serializes a ByteString object to ByteString.
* {@link Codec} implementation for non-shaded
* {@link com.google.protobuf.ByteString} objects.
*/
public class ByteStringCodec implements Codec {

@Override
public ByteString serialize(Object object)
throws InvalidProtocolBufferException {
return (ByteString) object;
return UnsafeByteOperations.unsafeWrap(
((com.google.protobuf.ByteString) object).asReadOnlyByteBuffer());
}

@Override
public Object deserialize(Class<?> type, ByteString value)
throws InvalidProtocolBufferException {
return value;
return com.google.protobuf.UnsafeByteOperations.
unsafeWrap(value.asReadOnlyByteBuffer());
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -17,8 +17,8 @@

package org.apache.hadoop.hdds.scm.ha.io;

import com.google.protobuf.ByteString;
import com.google.protobuf.InvalidProtocolBufferException;
import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
import org.apache.ratis.thirdparty.com.google.protobuf.InvalidProtocolBufferException;

/**
* Codec interface to marshall/unmarshall data to/from {@link ByteString}.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,9 +17,6 @@

package org.apache.hadoop.hdds.scm.ha.io;

import com.google.protobuf.ByteString;
import com.google.protobuf.InvalidProtocolBufferException;
import com.google.protobuf.Message;
import com.google.protobuf.ProtocolMessageEnum;
import java.math.BigInteger;
import java.security.cert.X509Certificate;
Expand All @@ -29,6 +26,9 @@
import java.util.Map;
import org.apache.commons.lang3.ClassUtils;
import org.apache.hadoop.hdds.security.symmetric.ManagedSecretKey;
import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
import org.apache.ratis.thirdparty.com.google.protobuf.InvalidProtocolBufferException;
import org.apache.ratis.thirdparty.com.google.protobuf.Message;

/**
* Maps types to the corresponding {@link Codec} implementation.
Expand All @@ -38,7 +38,8 @@ public final class CodecFactory {
private static Map<Class<?>, Codec> codecs = new HashMap<>();

static {
codecs.put(Message.class, new GeneratedMessageCodec());
codecs.put(com.google.protobuf.Message.class, new GeneratedMessageCodec());
codecs.put(Message.class, new ScmGeneratedMessageCodec());
codecs.put(ProtocolMessageEnum.class, new EnumCodec());
codecs.put(List.class, new ListCodec());
codecs.put(Integer.class, new IntegerCodec());
Expand All @@ -47,7 +48,8 @@ public final class CodecFactory {
codecs.put(Boolean.class, new BooleanCodec());
codecs.put(BigInteger.class, new BigIntegerCodec());
codecs.put(X509Certificate.class, new X509CertificateCodec());
codecs.put(ByteString.class, new ByteStringCodec());
codecs.put(com.google.protobuf.ByteString.class, new ByteStringCodec());
codecs.put(ByteString.class, new ScmByteStringCodec());
codecs.put(ManagedSecretKey.class, new ManagedSecretKeyCodec());
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,12 +18,12 @@
package org.apache.hadoop.hdds.scm.ha.io;

import com.google.common.primitives.Ints;
import com.google.protobuf.ByteString;
import com.google.protobuf.InvalidProtocolBufferException;
import com.google.protobuf.ProtoUtils;
import com.google.protobuf.ProtocolMessageEnum;
import java.lang.reflect.InvocationTargetException;
import org.apache.hadoop.hdds.scm.ha.ReflectionUtil;
import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
import org.apache.ratis.thirdparty.com.google.protobuf.InvalidProtocolBufferException;
import org.apache.ratis.thirdparty.com.google.protobuf.UnsafeByteOperations;

/**
* {@link Codec} for {@link ProtocolMessageEnum} objects.
Expand All @@ -34,8 +34,7 @@ public class EnumCodec implements Codec {
public ByteString serialize(Object object)
throws InvalidProtocolBufferException {
// toByteArray returns a new array
return ProtoUtils.unsafeByteString(Ints.toByteArray(
((ProtocolMessageEnum) object).getNumber()));
return UnsafeByteOperations.unsafeWrap(Ints.toByteArray(((ProtocolMessageEnum) object).getNumber()));
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

FYI, filed HDDS-14623 for removing ProtoUtils.

}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,34 +17,36 @@

package org.apache.hadoop.hdds.scm.ha.io;

import com.google.protobuf.ByteString;
import com.google.protobuf.InvalidProtocolBufferException;
import com.google.protobuf.Message;
import java.lang.reflect.InvocationTargetException;
import org.apache.hadoop.hdds.scm.ha.ReflectionUtil;
import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
import org.apache.ratis.thirdparty.com.google.protobuf.InvalidProtocolBufferException;
import org.apache.ratis.thirdparty.com.google.protobuf.UnsafeByteOperations;

/**
* {@link Codec} for {@link Message} objects.
* {@link Codec} implementation for non-shaded
* {@link com.google.protobuf.Message} objects.
*/
public class GeneratedMessageCodec implements Codec {

@Override
public ByteString serialize(Object object) {
return ((Message)object).toByteString();
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Do we need to support both com.google.protobuf.Message and org.apache.ratis.thirdparty.com.google.protobuf.Message?

If yes, use two GeneratedMessageCodec classes instead of using one class. Also, we need to put them to CodecFactory.

//CodecFactory
    codecs.put(com.google.protobuf.Message.class, new GeneratedMessageCodec());
    codecs.put(Message.class, new ScmGeneratedMessageCodec());

public ByteString serialize(Object object)
throws InvalidProtocolBufferException {
return UnsafeByteOperations.unsafeWrap(
((Message) object).toByteString().asReadOnlyByteBuffer());
}

@Override
public Message deserialize(Class<?> type, ByteString value)
public Object deserialize(Class<?> type, ByteString value)
throws InvalidProtocolBufferException {
try {
return (Message) ReflectionUtil.getMethod(type,
"parseFrom", byte[].class)
return ReflectionUtil.getMethod(type, "parseFrom", byte[].class)
.invoke(null, (Object) value.toByteArray());
} catch (NoSuchMethodException | IllegalAccessException
| InvocationTargetException ex) {
| InvocationTargetException ex) {
ex.printStackTrace();
throw new InvalidProtocolBufferException(
"Message cannot be decoded: " + ex.getMessage());
throw new InvalidProtocolBufferException("Message cannot be decoded: " + ex.getMessage());
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -18,9 +18,9 @@
package org.apache.hadoop.hdds.scm.ha.io;

import com.google.common.primitives.Ints;
import com.google.protobuf.ByteString;
import com.google.protobuf.InvalidProtocolBufferException;
import com.google.protobuf.ProtoUtils;
import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
import org.apache.ratis.thirdparty.com.google.protobuf.InvalidProtocolBufferException;
import org.apache.ratis.thirdparty.com.google.protobuf.UnsafeByteOperations;

/**
* Encodes/decodes an integer to a byte string.
Expand All @@ -30,7 +30,7 @@ public class IntegerCodec implements Codec {
public ByteString serialize(Object object)
throws InvalidProtocolBufferException {
// toByteArray returns a new array
return ProtoUtils.unsafeByteString(Ints.toByteArray((Integer) object));
return UnsafeByteOperations.unsafeWrap(Ints.toByteArray((Integer) object));
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,13 +17,13 @@

package org.apache.hadoop.hdds.scm.ha.io;

import com.google.protobuf.ByteString;
import com.google.protobuf.InvalidProtocolBufferException;
import java.lang.reflect.InvocationTargetException;
import java.util.ArrayList;
import java.util.List;
import org.apache.hadoop.hdds.protocol.proto.SCMRatisProtocol.ListArgument;
import org.apache.hadoop.hdds.scm.ha.ReflectionUtil;
import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
import org.apache.ratis.thirdparty.com.google.protobuf.InvalidProtocolBufferException;

/**
* {@link Codec} for {@link List} objects.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,9 +18,9 @@
package org.apache.hadoop.hdds.scm.ha.io;

import com.google.common.primitives.Longs;
import com.google.protobuf.ByteString;
import com.google.protobuf.InvalidProtocolBufferException;
import com.google.protobuf.ProtoUtils;
import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
import org.apache.ratis.thirdparty.com.google.protobuf.InvalidProtocolBufferException;
import org.apache.ratis.thirdparty.com.google.protobuf.UnsafeByteOperations;

/**
* {@link Codec} for {@code Long} objects.
Expand All @@ -31,7 +31,7 @@ public class LongCodec implements Codec {
public ByteString serialize(Object object)
throws InvalidProtocolBufferException {
// toByteArray returns a new array
return ProtoUtils.unsafeByteString(Longs.toByteArray((Long) object));
return UnsafeByteOperations.unsafeWrap(Longs.toByteArray((Long) object));
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,10 +17,11 @@

package org.apache.hadoop.hdds.scm.ha.io;

import com.google.protobuf.ByteString;
import com.google.protobuf.InvalidProtocolBufferException;
import org.apache.hadoop.hdds.protocol.proto.SCMSecretKeyProtocolProtos;
import org.apache.hadoop.hdds.security.symmetric.ManagedSecretKey;
import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
import org.apache.ratis.thirdparty.com.google.protobuf.InvalidProtocolBufferException;
import org.apache.ratis.thirdparty.com.google.protobuf.UnsafeByteOperations;

/**
* A codec for {@link ManagedSecretKey} objects.
Expand All @@ -30,14 +31,20 @@ public class ManagedSecretKeyCodec implements Codec {
public ByteString serialize(Object object)
throws InvalidProtocolBufferException {
ManagedSecretKey secretKey = (ManagedSecretKey) object;
return secretKey.toProtobuf().toByteString();
return UnsafeByteOperations.unsafeWrap(
secretKey.toProtobuf().toByteString().asReadOnlyByteBuffer());
}

@Override
public Object deserialize(Class<?> type, ByteString value)
throws InvalidProtocolBufferException {
SCMSecretKeyProtocolProtos.ManagedSecretKey message =
SCMSecretKeyProtocolProtos.ManagedSecretKey.parseFrom(value);
return ManagedSecretKey.fromProtobuf(message);
try {
SCMSecretKeyProtocolProtos.ManagedSecretKey message =
SCMSecretKeyProtocolProtos.ManagedSecretKey.parseFrom(
value.asReadOnlyByteBuffer());
return ManagedSecretKey.fromProtobuf(message);
} catch (com.google.protobuf.InvalidProtocolBufferException e) {
throw new InvalidProtocolBufferException("Failed to deserialize value for " + type, e);
}
}
}
Loading