This is an automated email from the ASF dual-hosted git repository.

adoroszlai pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/ozone.git


The following commit(s) were added to refs/heads/master by this push:
     new f37fe2a3402 HDDS-14748. Implement ScmGeneratedMessageCodec without 
reflection (#9878)
f37fe2a3402 is described below

commit f37fe2a340245bf7393ee40262f557c74f5cab24
Author: Russole <[email protected]>
AuthorDate: Sun Mar 8 16:46:57 2026 +0800

    HDDS-14748. Implement ScmGeneratedMessageCodec without reflection (#9878)
---
 .../hadoop/hdds/scm/ha/io/ScmCodecFactory.java     | 22 ++++++++--
 .../hdds/scm/ha/io/ScmGeneratedMessageCodec.java   | 50 ----------------------
 .../ha/io/ScmNonShadedGeneratedMessageCodec.java   | 29 ++++++-------
 3 files changed, 33 insertions(+), 68 deletions(-)

diff --git 
a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/ha/io/ScmCodecFactory.java
 
b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/ha/io/ScmCodecFactory.java
index 68e33ebd7b6..828695a179d 100644
--- 
a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/ha/io/ScmCodecFactory.java
+++ 
b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/ha/io/ScmCodecFactory.java
@@ -17,6 +17,7 @@
 
 package org.apache.hadoop.hdds.scm.ha.io;
 
+import com.google.protobuf.Message;
 import com.google.protobuf.ProtocolMessageEnum;
 import java.math.BigInteger;
 import java.security.cert.X509Certificate;
@@ -26,13 +27,18 @@
 import java.util.Map;
 import java.util.function.IntFunction;
 import org.apache.commons.lang3.ClassUtils;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ContainerID;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ContainerInfoProto;
+import 
org.apache.hadoop.hdds.protocol.proto.HddsProtos.DeletedBlocksTransactionSummary;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleEvent;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeType;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos.Pipeline;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos.PipelineID;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos.PipelineState;
+import 
org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.DeletedBlocksTransaction;
 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 ScmCodec} implementation.
@@ -42,8 +48,13 @@ public final class ScmCodecFactory {
   private static Map<Class<?>, ScmCodec<?>> codecs = new HashMap<>();
 
   static {
-    codecs.put(com.google.protobuf.Message.class, new 
ScmNonShadedGeneratedMessageCodec());
-    codecs.put(Message.class, new ScmGeneratedMessageCodec());
+    putProto(ContainerID.getDefaultInstance());
+    putProto(PipelineID.getDefaultInstance());
+    putProto(Pipeline.getDefaultInstance());
+    putProto(ContainerInfoProto.getDefaultInstance());
+    putProto(DeletedBlocksTransaction.getDefaultInstance());
+    putProto(DeletedBlocksTransactionSummary.getDefaultInstance());
+
     codecs.put(List.class, new ScmListCodec());
     codecs.put(Integer.class, new ScmIntegerCodec());
     codecs.put(Long.class, new ScmLongCodec());
@@ -60,6 +71,11 @@ public final class ScmCodecFactory {
     putEnum(NodeType.class, NodeType::forNumber);
   }
 
+  static <T extends Message> void putProto(T proto) {
+    codecs.put(proto.getClass(),
+        new ScmNonShadedGeneratedMessageCodec<>(proto.getParserForType()));
+  }
+
   static <T extends Enum<T> & ProtocolMessageEnum> void putEnum(
       Class<T> enumClass, IntFunction<T> forNumber) {
     codecs.put(enumClass, new ScmEnumCodec<>(enumClass, forNumber));
diff --git 
a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/ha/io/ScmGeneratedMessageCodec.java
 
b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/ha/io/ScmGeneratedMessageCodec.java
deleted file mode 100644
index 44e5efeeef3..00000000000
--- 
a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/ha/io/ScmGeneratedMessageCodec.java
+++ /dev/null
@@ -1,50 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hdds.scm.ha.io;
-
-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.Message;
-
-/**
- * {@link ScmCodec} for {@link Message} objects.
- */
-public class ScmGeneratedMessageCodec implements ScmCodec<Object> {
-
-  @Override
-  public ByteString serialize(Object object) throws 
InvalidProtocolBufferException {
-    return ((Message)object).toByteString();
-  }
-
-  @Override
-  public Message deserialize(Class<?> type, ByteString value)
-      throws InvalidProtocolBufferException {
-    try {
-      return (Message) ReflectionUtil.getMethod(type,
-              "parseFrom", byte[].class)
-          .invoke(null, (Object) value.toByteArray());
-    } catch (NoSuchMethodException | IllegalAccessException
-             | InvocationTargetException ex) {
-      ex.printStackTrace();
-      throw new InvalidProtocolBufferException(
-          "Message cannot be decoded: " + ex.getMessage());
-    }
-  }
-}
diff --git 
a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/ha/io/ScmNonShadedGeneratedMessageCodec.java
 
b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/ha/io/ScmNonShadedGeneratedMessageCodec.java
index 773d447e234..b8de8ccf7d0 100644
--- 
a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/ha/io/ScmNonShadedGeneratedMessageCodec.java
+++ 
b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/ha/io/ScmNonShadedGeneratedMessageCodec.java
@@ -18,8 +18,7 @@
 package org.apache.hadoop.hdds.scm.ha.io;
 
 import com.google.protobuf.Message;
-import java.lang.reflect.InvocationTargetException;
-import org.apache.hadoop.hdds.scm.ha.ReflectionUtil;
+import com.google.protobuf.Parser;
 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;
@@ -28,25 +27,25 @@
  * {@link ScmCodec} implementation for non-shaded
  * {@link com.google.protobuf.Message} objects.
  */
-public class ScmNonShadedGeneratedMessageCodec implements ScmCodec<Object> {
+public class ScmNonShadedGeneratedMessageCodec<T extends Message> implements 
ScmCodec<T> {
+
+  private final Parser<T> parser;
+
+  public ScmNonShadedGeneratedMessageCodec(Parser<T> parser) {
+    this.parser = parser;
+  }
 
   @Override
-  public ByteString serialize(Object object)
-      throws InvalidProtocolBufferException {
-    return UnsafeByteOperations.unsafeWrap(
-        ((Message) object).toByteString().asReadOnlyByteBuffer());
+  public ByteString serialize(T object) throws InvalidProtocolBufferException {
+    return 
UnsafeByteOperations.unsafeWrap(object.toByteString().asReadOnlyByteBuffer());
   }
 
   @Override
-  public Object deserialize(Class<?> type, ByteString value)
-      throws InvalidProtocolBufferException {
+  public T deserialize(Class<?> type, ByteString value) throws 
InvalidProtocolBufferException {
     try {
-      return ReflectionUtil.getMethod(type, "parseFrom", byte[].class)
-          .invoke(null, (Object) value.toByteArray());
-    } catch (NoSuchMethodException | IllegalAccessException
-             | InvocationTargetException ex) {
-      ex.printStackTrace();
-      throw new InvalidProtocolBufferException("Message cannot be decoded: " + 
ex.getMessage());
+      return parser.parseFrom(value.asReadOnlyByteBuffer());
+    } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+      throw new InvalidProtocolBufferException("Message cannot be decoded", e);
     }
   }
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to