You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-issues@hadoop.apache.org by GitBox <gi...@apache.org> on 2021/03/12 06:48:34 UTC

[GitHub] [hadoop] vinayakumarb opened a new pull request #2767: HDFS-15790. Make ProtobufRpcEngineProtos and ProtobufRpcEngineProtos2 Co-Exist

vinayakumarb opened a new pull request #2767:
URL: https://github.com/apache/hadoop/pull/2767


   ## Existing ClientSide available Protobuf RPC Engines.
   * ProtobufRpcEngine: Supports existing implementations based in protobuf 2.5.0 in both client side and server side. No code changes required in downstreams use this.
   * ProtobufRpcEngine2: Uses shaded protobuf of 3.7.1 version and supports client side and server side implementations based on shaded protobuf 3.7.1
   
   # Whats Changed?
   
   ## Server side Protobuf RPC Engines
   `ProtobufRpcEngine2` itself will handle both versions (2.5.0 and 3.7.1) of requests for RpcKind.PROTOCOL_BUFFERS. 
   `ProtobufRpcEngine2` will handover the processing to Legacy `ProtobufRpcEngine` if server side protocol implemenation found to be using older version of protobuf (2.5.0).
   
   ## How conflict arised?
   Both `ProtobufRpcEngine` and `ProtobufRpcEngine2` tried to register their corresponding RPC `Server` implementations to handle requests of kind `RpcKind.PROTOCOL_BUFFERS`, although this registering is of no-use in client-JVM. Only Server side this registration is required.
   
   Now, since `ProtobufRpcEngine2` itself can handle both versions of protobuf requests at server side, registering only `ProtobufRpcEngine2` will be sufficient.
   So no conflict is raised for co-existence.
   
   ## How old clients (before 3.3.0) are affected.?
   Since client JVM side registering of `ProtobufRpcEngine` will not affect server-JVM (running with latest version) and server can efficiently handle both type of implementations, there will not be any issues.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] jojochuang commented on a change in pull request #2767: HDFS-15790. Make ProtobufRpcEngineProtos and ProtobufRpcEngineProtos2 Co-Exist

Posted by GitBox <gi...@apache.org>.
jojochuang commented on a change in pull request #2767:
URL: https://github.com/apache/hadoop/pull/2767#discussion_r637485155



##########
File path: hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine2.java
##########
@@ -495,6 +524,7 @@ private static ProtoClassProtoImpl getProtocolImpl(RPC.Server server,
        * it is.</li>
        * </ol>
        */
+      @SuppressWarnings("deprecation")

Review comment:
       It would help a lot for the applications using the new ProtobufRpcEngine2 what API replaces this deprecated API.
   Maybe it can be written in the javadoc; or in the release note.

##########
File path: hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java
##########
@@ -443,144 +430,52 @@ public Server(Class<?> protocolClass, Object protocolImpl,
         SecretManager<? extends TokenIdentifier> secretManager, 
         String portRangeConfig, AlignmentContext alignmentContext)
         throws IOException {
-      super(bindAddress, port, null, numHandlers,
-          numReaders, queueSizePerHandler, conf,
-          serverNameFromClass(protocolImpl.getClass()), secretManager,
-          portRangeConfig);
-      setAlignmentContext(alignmentContext);
-      this.verbose = verbose;  
-      registerProtocolAndImpl(RPC.RpcKind.RPC_PROTOCOL_BUFFER, protocolClass,
-          protocolImpl);
+      super(protocolClass, protocolImpl, conf, bindAddress, port, numHandlers,
+          numReaders, queueSizePerHandler, verbose, secretManager,
+          portRangeConfig, alignmentContext);
     }
 
-    @Override
-    protected RpcInvoker getServerRpcInvoker(RpcKind rpcKind) {
-      if (rpcKind == RpcKind.RPC_PROTOCOL_BUFFER) {
-        return RPC_INVOKER;
-      }
-      return super.getServerRpcInvoker(rpcKind);
-    }
-
-    /**
-     * Protobuf invoker for {@link RpcInvoker}
-     */
-    static class ProtoBufRpcInvoker implements RpcInvoker {
-      private static ProtoClassProtoImpl getProtocolImpl(RPC.Server server,
-          String protoName, long clientVersion) throws RpcServerException {
-        ProtoNameVer pv = new ProtoNameVer(protoName, clientVersion);
-        ProtoClassProtoImpl impl = 
-            server.getProtocolImplMap(RPC.RpcKind.RPC_PROTOCOL_BUFFER).get(pv);
-        if (impl == null) { // no match for Protocol AND Version
-          VerProtocolImpl highest = 
-              server.getHighestSupportedProtocol(RPC.RpcKind.RPC_PROTOCOL_BUFFER, 
-                  protoName);
-          if (highest == null) {
-            throw new RpcNoSuchProtocolException(
-                "Unknown protocol: " + protoName);
-          }
-          // protocol supported but not the version that client wants
-          throw new RPC.VersionMismatch(protoName, clientVersion,
-              highest.version);
-        }
-        return impl;
+    static RpcWritable processCall(RPC.Server server,

Review comment:
       can you add a comment here that this is practically the same as ProtobufRpccEngine2.call() except the Message class, and that if this method is modified, the other method should be updated as well? (Or add the comment in the ProtobufRpccEngine2.call())

##########
File path: hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RPC.java
##########
@@ -937,11 +937,18 @@ public int hashCode() {
     */
    static class ProtoClassProtoImpl {
      final Class<?> protocolClass;
-     final Object protocolImpl; 
+      final Object protocolImpl;
+      private final boolean newPBImpl;
+
      ProtoClassProtoImpl(Class<?> protocolClass, Object protocolImpl) {
        this.protocolClass = protocolClass;
        this.protocolImpl = protocolImpl;
+        this.newPBImpl = protocolImpl instanceof BlockingService;
      }
+
+      public boolean isNewPBImpl() {

Review comment:
       Might be easier to understand to call it "isShadedPBImpl()" instead




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] vinayakumarb commented on a change in pull request #2767: HDFS-15790. Make ProtobufRpcEngineProtos and ProtobufRpcEngineProtos2 Co-Exist

Posted by GitBox <gi...@apache.org>.
vinayakumarb commented on a change in pull request #2767:
URL: https://github.com/apache/hadoop/pull/2767#discussion_r619602510



##########
File path: hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestProtoBufRpc.java
##########
@@ -179,10 +281,41 @@ public void testProtoBufRpc2() throws Exception {
     MetricsRecordBuilder rpcDetailedMetrics = 
         getMetrics(server.getRpcDetailedMetrics().name());
     assertCounterGt("Echo2NumOps", 0L, rpcDetailedMetrics);
+
+    if (testWithLegacy) {
+      testProtobufLegacy();
+    }
+  }
+
+  private void testProtobufLegacy()
+      throws IOException, com.google.protobuf.ServiceException {
+    TestRpcService2Legacy client = getClientLegacy();
+
+    // Test ping method
+    client.ping2(null, TestProtosLegacy.EmptyRequestProto.newBuilder().build());
+
+    // Test echo method
+    TestProtosLegacy.EchoResponseProto echoResponse = client.echo2(null,
+        TestProtosLegacy.EchoRequestProto.newBuilder().setMessage("hello")
+            .build());
+    assertThat(echoResponse.getMessage()).isEqualTo("hello");
+
+    // Ensure RPC metrics are updated
+    MetricsRecordBuilder rpcMetrics = getMetrics(server.getRpcMetrics().name());
+    assertCounterGt("RpcQueueTimeNumOps", 0L, rpcMetrics);
+    assertCounterGt("RpcProcessingTimeNumOps", 0L, rpcMetrics);
+
+    MetricsRecordBuilder rpcDetailedMetrics =
+        getMetrics(server.getRpcDetailedMetrics().name());
+    assertCounterGt("Echo2NumOps", 0L, rpcDetailedMetrics);
   }
 
   @Test (timeout=5000)
   public void testProtoBufRandomException() throws Exception {
+    if (testWithLegacy) {
+      //No test with legacy
+      return;
+    }

Review comment:
       Updated. Please check.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] vinayakumarb commented on a change in pull request #2767: HDFS-15790. Make ProtobufRpcEngineProtos and ProtobufRpcEngineProtos2 Co-Exist

Posted by GitBox <gi...@apache.org>.
vinayakumarb commented on a change in pull request #2767:
URL: https://github.com/apache/hadoop/pull/2767#discussion_r637551571



##########
File path: hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RPC.java
##########
@@ -937,11 +937,18 @@ public int hashCode() {
     */
    static class ProtoClassProtoImpl {
      final Class<?> protocolClass;
-     final Object protocolImpl; 
+      final Object protocolImpl;
+      private final boolean newPBImpl;
+
      ProtoClassProtoImpl(Class<?> protocolClass, Object protocolImpl) {
        this.protocolClass = protocolClass;
        this.protocolImpl = protocolImpl;
+        this.newPBImpl = protocolImpl instanceof BlockingService;
      }
+
+      public boolean isNewPBImpl() {

Review comment:
       Done

##########
File path: hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java
##########
@@ -443,144 +430,52 @@ public Server(Class<?> protocolClass, Object protocolImpl,
         SecretManager<? extends TokenIdentifier> secretManager, 
         String portRangeConfig, AlignmentContext alignmentContext)
         throws IOException {
-      super(bindAddress, port, null, numHandlers,
-          numReaders, queueSizePerHandler, conf,
-          serverNameFromClass(protocolImpl.getClass()), secretManager,
-          portRangeConfig);
-      setAlignmentContext(alignmentContext);
-      this.verbose = verbose;  
-      registerProtocolAndImpl(RPC.RpcKind.RPC_PROTOCOL_BUFFER, protocolClass,
-          protocolImpl);
+      super(protocolClass, protocolImpl, conf, bindAddress, port, numHandlers,
+          numReaders, queueSizePerHandler, verbose, secretManager,
+          portRangeConfig, alignmentContext);
     }
 
-    @Override
-    protected RpcInvoker getServerRpcInvoker(RpcKind rpcKind) {
-      if (rpcKind == RpcKind.RPC_PROTOCOL_BUFFER) {
-        return RPC_INVOKER;
-      }
-      return super.getServerRpcInvoker(rpcKind);
-    }
-
-    /**
-     * Protobuf invoker for {@link RpcInvoker}
-     */
-    static class ProtoBufRpcInvoker implements RpcInvoker {
-      private static ProtoClassProtoImpl getProtocolImpl(RPC.Server server,
-          String protoName, long clientVersion) throws RpcServerException {
-        ProtoNameVer pv = new ProtoNameVer(protoName, clientVersion);
-        ProtoClassProtoImpl impl = 
-            server.getProtocolImplMap(RPC.RpcKind.RPC_PROTOCOL_BUFFER).get(pv);
-        if (impl == null) { // no match for Protocol AND Version
-          VerProtocolImpl highest = 
-              server.getHighestSupportedProtocol(RPC.RpcKind.RPC_PROTOCOL_BUFFER, 
-                  protoName);
-          if (highest == null) {
-            throw new RpcNoSuchProtocolException(
-                "Unknown protocol: " + protoName);
-          }
-          // protocol supported but not the version that client wants
-          throw new RPC.VersionMismatch(protoName, clientVersion,
-              highest.version);
-        }
-        return impl;
+    static RpcWritable processCall(RPC.Server server,

Review comment:
       Done.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] vinayakumarb commented on pull request #2767: HDFS-15790. Make ProtobufRpcEngineProtos and ProtobufRpcEngineProtos2 Co-Exist

Posted by GitBox <gi...@apache.org>.
vinayakumarb commented on pull request #2767:
URL: https://github.com/apache/hadoop/pull/2767#issuecomment-797277002


   Generated test code (for 2.5.0 protobuf) has been added for 'arm' since 2.5.0 protobuf executable is not available for aarch64 platform.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] jojochuang commented on pull request #2767: HDFS-15790. Make ProtobufRpcEngineProtos and ProtobufRpcEngineProtos2 Co-Exist

Posted by GitBox <gi...@apache.org>.
jojochuang commented on pull request #2767:
URL: https://github.com/apache/hadoop/pull/2767#issuecomment-840536895


   Please rebase the PR to incorporate the changes made in HADOOP-17680.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] aajisaka commented on pull request #2767: HDFS-15790. Make ProtobufRpcEngineProtos and ProtobufRpcEngineProtos2 Co-Exist

Posted by GitBox <gi...@apache.org>.
aajisaka commented on pull request #2767:
URL: https://github.com/apache/hadoop/pull/2767#issuecomment-816447348


   Thank you @vinayakumarb. Mostly looks good to me.
   
   Hi @belugabehr do you have any comments?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] hadoop-yetus commented on pull request #2767: HDFS-15790. Make ProtobufRpcEngineProtos and ProtobufRpcEngineProtos2 Co-Exist

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on pull request #2767:
URL: https://github.com/apache/hadoop/pull/2767#issuecomment-846593536


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   0m 44s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files found.  |
   | +0 :ok: |  codespell  |   0m  0s |  |  codespell was not available.  |
   | +0 :ok: |  buf  |   0m  0s |  |  buf was not available.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  |  The patch appears to include 6 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  33m 58s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |  20m 50s |  |  trunk passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  compile  |  18m  0s |  |  trunk passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  checkstyle  |   1m  9s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   1m 34s |  |  trunk passed  |
   | +1 :green_heart: |  javadoc  |   1m  7s |  |  trunk passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javadoc  |   1m 42s |  |  trunk passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  spotbugs  |   2m 22s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  15m 36s |  |  branch has no errors when building and testing our client artifacts.  |
   |||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 52s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |  20m  8s |  |  the patch passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | -1 :x: |  cc  |  20m  8s | [/results-compile-cc-root-jdkUbuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2767/8/artifact/out/results-compile-cc-root-jdkUbuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04.txt) |  root-jdkUbuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04 with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04 generated 22 new + 305 unchanged - 22 fixed = 327 total (was 327)  |
   | +1 :green_heart: |  javac  |  20m  8s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |  18m 11s |  |  the patch passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | -1 :x: |  cc  |  18m 11s | [/results-compile-cc-root-jdkPrivateBuild-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2767/8/artifact/out/results-compile-cc-root-jdkPrivateBuild-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08.txt) |  root-jdkPrivateBuild-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08 with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08 generated 34 new + 293 unchanged - 34 fixed = 327 total (was 327)  |
   | +1 :green_heart: |  javac  |  18m 11s |  |  the patch passed  |
   | +1 :green_heart: |  blanks  |   0m  0s |  |  The patch has no blanks issues.  |
   | -0 :warning: |  checkstyle  |   1m  8s | [/results-checkstyle-hadoop-common-project_hadoop-common.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2767/8/artifact/out/results-checkstyle-hadoop-common-project_hadoop-common.txt) |  hadoop-common-project/hadoop-common: The patch generated 2 new + 210 unchanged - 7 fixed = 212 total (was 217)  |
   | +1 :green_heart: |  mvnsite  |   1m 34s |  |  the patch passed  |
   | +1 :green_heart: |  xml  |   0m  2s |  |  The patch has no ill-formed XML file.  |
   | +1 :green_heart: |  javadoc  |   1m  5s |  |  the patch passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javadoc  |   1m 39s |  |  the patch passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  spotbugs  |   2m 32s |  |  the patch passed  |
   | +1 :green_heart: |  shadedclient  |  15m 52s |  |  patch has no errors when building and testing our client artifacts.  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |  17m  8s |  |  hadoop-common in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 59s |  |  The patch does not generate ASF License warnings.  |
   |  |   | 179m  1s |  |  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2767/8/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2767 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient codespell xml spotbugs checkstyle cc buflint bufcompat |
   | uname | Linux 31674546d4ed 4.15.0-112-generic #113-Ubuntu SMP Thu Jul 9 23:41:39 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / 540ab7c8bdcab6eaef40b83546a912a0c66194c3 |
   | Default Java | Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2767/8/testReport/ |
   | Max. process+thread count | 1266 (vs. ulimit of 5500) |
   | modules | C: hadoop-common-project/hadoop-common U: hadoop-common-project/hadoop-common |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2767/8/console |
   | versions | git=2.25.1 maven=3.6.3 spotbugs=4.2.2 |
   | Powered by | Apache Yetus 0.14.0-SNAPSHOT https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] jojochuang merged pull request #2767: HDFS-15790. Make ProtobufRpcEngineProtos and ProtobufRpcEngineProtos2 Co-Exist

Posted by GitBox <gi...@apache.org>.
jojochuang merged pull request #2767:
URL: https://github.com/apache/hadoop/pull/2767


   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] hadoop-yetus commented on pull request #2767: HDFS-15790. Make ProtobufRpcEngineProtos and ProtobufRpcEngineProtos2 Co-Exist

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on pull request #2767:
URL: https://github.com/apache/hadoop/pull/2767#issuecomment-826069768


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |  13m 30s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files found.  |
   | +0 :ok: |  codespell  |   0m  0s |  |  codespell was not available.  |
   | +0 :ok: |  buf  |   0m  0s |  |  buf was not available.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  |  The patch appears to include 6 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  34m 13s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |  20m 45s |  |  trunk passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  compile  |  17m 59s |  |  trunk passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  checkstyle  |   1m  6s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   1m 31s |  |  trunk passed  |
   | +1 :green_heart: |  javadoc  |   1m  4s |  |  trunk passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javadoc  |   1m 36s |  |  trunk passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  spotbugs  |   2m 20s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  15m 45s |  |  branch has no errors when building and testing our client artifacts.  |
   |||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 52s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |  20m 10s |  |  the patch passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | -1 :x: |  cc  |  20m 10s | [/results-compile-cc-root-jdkUbuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2767/5/artifact/out/results-compile-cc-root-jdkUbuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04.txt) |  root-jdkUbuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04 with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04 generated 26 new + 301 unchanged - 26 fixed = 327 total (was 327)  |
   | +1 :green_heart: |  javac  |  20m 10s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |  17m 58s |  |  the patch passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | -1 :x: |  cc  |  17m 58s | [/results-compile-cc-root-jdkPrivateBuild-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2767/5/artifact/out/results-compile-cc-root-jdkPrivateBuild-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08.txt) |  root-jdkPrivateBuild-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08 with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08 generated 9 new + 318 unchanged - 9 fixed = 327 total (was 327)  |
   | +1 :green_heart: |  javac  |  17m 58s |  |  the patch passed  |
   | +1 :green_heart: |  blanks  |   0m  0s |  |  The patch has no blanks issues.  |
   | -0 :warning: |  checkstyle  |   1m  5s | [/results-checkstyle-hadoop-common-project_hadoop-common.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2767/5/artifact/out/results-checkstyle-hadoop-common-project_hadoop-common.txt) |  hadoop-common-project/hadoop-common: The patch generated 1 new + 211 unchanged - 7 fixed = 212 total (was 218)  |
   | +1 :green_heart: |  mvnsite  |   1m 30s |  |  the patch passed  |
   | +1 :green_heart: |  xml  |   0m  1s |  |  The patch has no ill-formed XML file.  |
   | +1 :green_heart: |  javadoc  |   1m  2s |  |  the patch passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javadoc  |   1m 36s |  |  the patch passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  spotbugs  |   2m 33s |  |  the patch passed  |
   | +1 :green_heart: |  shadedclient  |  15m 46s |  |  patch has no errors when building and testing our client artifacts.  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |  17m 15s |  |  hadoop-common in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 55s |  |  The patch does not generate ASF License warnings.  |
   |  |   | 190m 58s |  |  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2767/5/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2767 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient codespell xml spotbugs checkstyle cc buflint bufcompat |
   | uname | Linux 0d2a9bec18f7 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / e19b1e0f5736eb1913a82516f0c85c4ef72cdd71 |
   | Default Java | Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2767/5/testReport/ |
   | Max. process+thread count | 2249 (vs. ulimit of 5500) |
   | modules | C: hadoop-common-project/hadoop-common U: hadoop-common-project/hadoop-common |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2767/5/console |
   | versions | git=2.25.1 maven=3.6.3 spotbugs=4.2.2 |
   | Powered by | Apache Yetus 0.14.0-SNAPSHOT https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] hadoop-yetus commented on pull request #2767: HDFS-15790. Make ProtobufRpcEngineProtos and ProtobufRpcEngineProtos2 Co-Exist

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on pull request #2767:
URL: https://github.com/apache/hadoop/pull/2767#issuecomment-799629642


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   0m 39s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  1s |  |  No case conflicting files found.  |
   | +0 :ok: |  codespell  |   0m  1s |  |  codespell was not available.  |
   | +0 :ok: |  buf  |   0m  1s |  |  buf was not available.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  |  The patch appears to include 6 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  33m  5s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |  20m 41s |  |  trunk passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  compile  |  18m  1s |  |  trunk passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  checkstyle  |   1m  7s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   1m 33s |  |  trunk passed  |
   | +1 :green_heart: |  javadoc  |   1m  7s |  |  trunk passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javadoc  |   1m 35s |  |  trunk passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  spotbugs  |   2m 21s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  15m 28s |  |  branch has no errors when building and testing our client artifacts.  |
   |||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 56s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |  20m 12s |  |  the patch passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | -1 :x: |  cc  |  20m 12s | [/results-compile-cc-root-jdkUbuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2767/3/artifact/out/results-compile-cc-root-jdkUbuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04.txt) |  root-jdkUbuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04 with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04 generated 40 new + 372 unchanged - 40 fixed = 412 total (was 412)  |
   | -1 :x: |  javac  |  20m 12s | [/results-compile-javac-root-jdkUbuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2767/3/artifact/out/results-compile-javac-root-jdkUbuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04.txt) |  root-jdkUbuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04 with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04 generated 2 new + 1955 unchanged - 0 fixed = 1957 total (was 1955)  |
   | +1 :green_heart: |  compile  |  18m  0s |  |  the patch passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | -1 :x: |  cc  |  18m  0s | [/results-compile-cc-root-jdkPrivateBuild-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2767/3/artifact/out/results-compile-cc-root-jdkPrivateBuild-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08.txt) |  root-jdkPrivateBuild-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08 with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08 generated 23 new + 389 unchanged - 23 fixed = 412 total (was 412)  |
   | -1 :x: |  javac  |  18m  0s | [/results-compile-javac-root-jdkPrivateBuild-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2767/3/artifact/out/results-compile-javac-root-jdkPrivateBuild-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08.txt) |  root-jdkPrivateBuild-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08 with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08 generated 2 new + 1847 unchanged - 0 fixed = 1849 total (was 1847)  |
   | +1 :green_heart: |  blanks  |   0m  0s |  |  The patch has no blanks issues.  |
   | -0 :warning: |  checkstyle  |   1m  8s | [/results-checkstyle-hadoop-common-project_hadoop-common.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2767/3/artifact/out/results-checkstyle-hadoop-common-project_hadoop-common.txt) |  hadoop-common-project/hadoop-common: The patch generated 10 new + 211 unchanged - 7 fixed = 221 total (was 218)  |
   | +1 :green_heart: |  mvnsite  |   1m 31s |  |  the patch passed  |
   | +1 :green_heart: |  xml  |   0m  1s |  |  The patch has no ill-formed XML file.  |
   | +1 :green_heart: |  javadoc  |   1m  3s |  |  the patch passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javadoc  |   1m 41s |  |  the patch passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  spotbugs  |   2m 31s |  |  the patch passed  |
   | +1 :green_heart: |  shadedclient  |  15m 44s |  |  patch has no errors when building and testing our client artifacts.  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |  17m 22s |  |  hadoop-common in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 55s |  |  The patch does not generate ASF License warnings.  |
   |  |   | 177m 19s |  |  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2767/3/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2767 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient codespell xml spotbugs checkstyle cc buflint bufcompat |
   | uname | Linux dff2f2452617 4.15.0-112-generic #113-Ubuntu SMP Thu Jul 9 23:41:39 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / 8b5b31e10911a70c0db743b5bbf156853bd2b44e |
   | Default Java | Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2767/3/testReport/ |
   | Max. process+thread count | 1266 (vs. ulimit of 5500) |
   | modules | C: hadoop-common-project/hadoop-common U: hadoop-common-project/hadoop-common |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2767/3/console |
   | versions | git=2.25.1 maven=3.6.3 spotbugs=4.2.2 |
   | Powered by | Apache Yetus 0.14.0-SNAPSHOT https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] vinayakumarb commented on a change in pull request #2767: HDFS-15790. Make ProtobufRpcEngineProtos and ProtobufRpcEngineProtos2 Co-Exist

Posted by GitBox <gi...@apache.org>.
vinayakumarb commented on a change in pull request #2767:
URL: https://github.com/apache/hadoop/pull/2767#discussion_r618930701



##########
File path: hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestProtoBufRpc.java
##########
@@ -179,10 +281,41 @@ public void testProtoBufRpc2() throws Exception {
     MetricsRecordBuilder rpcDetailedMetrics = 
         getMetrics(server.getRpcDetailedMetrics().name());
     assertCounterGt("Echo2NumOps", 0L, rpcDetailedMetrics);
+
+    if (testWithLegacy) {
+      testProtobufLegacy();
+    }
+  }
+
+  private void testProtobufLegacy()
+      throws IOException, com.google.protobuf.ServiceException {
+    TestRpcService2Legacy client = getClientLegacy();
+
+    // Test ping method
+    client.ping2(null, TestProtosLegacy.EmptyRequestProto.newBuilder().build());
+
+    // Test echo method
+    TestProtosLegacy.EchoResponseProto echoResponse = client.echo2(null,
+        TestProtosLegacy.EchoRequestProto.newBuilder().setMessage("hello")
+            .build());
+    assertThat(echoResponse.getMessage()).isEqualTo("hello");
+
+    // Ensure RPC metrics are updated
+    MetricsRecordBuilder rpcMetrics = getMetrics(server.getRpcMetrics().name());
+    assertCounterGt("RpcQueueTimeNumOps", 0L, rpcMetrics);
+    assertCounterGt("RpcProcessingTimeNumOps", 0L, rpcMetrics);
+
+    MetricsRecordBuilder rpcDetailedMetrics =
+        getMetrics(server.getRpcDetailedMetrics().name());
+    assertCounterGt("Echo2NumOps", 0L, rpcDetailedMetrics);
   }
 
   @Test (timeout=5000)
   public void testProtoBufRandomException() throws Exception {
+    if (testWithLegacy) {
+      //No test with legacy
+      return;
+    }

Review comment:
       Thanks @aajisaka Will update soon.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] vinayakumarb commented on a change in pull request #2767: HDFS-15790. Make ProtobufRpcEngineProtos and ProtobufRpcEngineProtos2 Co-Exist

Posted by GitBox <gi...@apache.org>.
vinayakumarb commented on a change in pull request #2767:
URL: https://github.com/apache/hadoop/pull/2767#discussion_r619602510



##########
File path: hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestProtoBufRpc.java
##########
@@ -179,10 +281,41 @@ public void testProtoBufRpc2() throws Exception {
     MetricsRecordBuilder rpcDetailedMetrics = 
         getMetrics(server.getRpcDetailedMetrics().name());
     assertCounterGt("Echo2NumOps", 0L, rpcDetailedMetrics);
+
+    if (testWithLegacy) {
+      testProtobufLegacy();
+    }
+  }
+
+  private void testProtobufLegacy()
+      throws IOException, com.google.protobuf.ServiceException {
+    TestRpcService2Legacy client = getClientLegacy();
+
+    // Test ping method
+    client.ping2(null, TestProtosLegacy.EmptyRequestProto.newBuilder().build());
+
+    // Test echo method
+    TestProtosLegacy.EchoResponseProto echoResponse = client.echo2(null,
+        TestProtosLegacy.EchoRequestProto.newBuilder().setMessage("hello")
+            .build());
+    assertThat(echoResponse.getMessage()).isEqualTo("hello");
+
+    // Ensure RPC metrics are updated
+    MetricsRecordBuilder rpcMetrics = getMetrics(server.getRpcMetrics().name());
+    assertCounterGt("RpcQueueTimeNumOps", 0L, rpcMetrics);
+    assertCounterGt("RpcProcessingTimeNumOps", 0L, rpcMetrics);
+
+    MetricsRecordBuilder rpcDetailedMetrics =
+        getMetrics(server.getRpcDetailedMetrics().name());
+    assertCounterGt("Echo2NumOps", 0L, rpcDetailedMetrics);
   }
 
   @Test (timeout=5000)
   public void testProtoBufRandomException() throws Exception {
+    if (testWithLegacy) {
+      //No test with legacy
+      return;
+    }

Review comment:
       Updated. Please check.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] hadoop-yetus commented on pull request #2767: HDFS-15790. Make ProtobufRpcEngineProtos and ProtobufRpcEngineProtos2 Co-Exist

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on pull request #2767:
URL: https://github.com/apache/hadoop/pull/2767#issuecomment-826072516


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |  21m 16s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files found.  |
   | +0 :ok: |  codespell  |   0m  1s |  |  codespell was not available.  |
   | +0 :ok: |  buf  |   0m  1s |  |  buf was not available.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  |  The patch appears to include 6 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  36m  9s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |  22m 17s |  |  trunk passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  compile  |  18m 59s |  |  trunk passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  checkstyle  |   0m 59s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   1m 28s |  |  trunk passed  |
   | +1 :green_heart: |  javadoc  |   0m 56s |  |  trunk passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javadoc  |   1m 32s |  |  trunk passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  spotbugs  |   2m 22s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  18m 11s |  |  branch has no errors when building and testing our client artifacts.  |
   |||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 55s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |  21m 37s |  |  the patch passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | -1 :x: |  cc  |  21m 37s | [/results-compile-cc-root-jdkUbuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2767/6/artifact/out/results-compile-cc-root-jdkUbuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04.txt) |  root-jdkUbuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04 with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04 generated 29 new + 298 unchanged - 29 fixed = 327 total (was 327)  |
   | +1 :green_heart: |  javac  |  21m 37s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |  19m  0s |  |  the patch passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | -1 :x: |  cc  |  19m  0s | [/results-compile-cc-root-jdkPrivateBuild-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2767/6/artifact/out/results-compile-cc-root-jdkPrivateBuild-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08.txt) |  root-jdkPrivateBuild-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08 with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08 generated 21 new + 306 unchanged - 21 fixed = 327 total (was 327)  |
   | +1 :green_heart: |  javac  |  19m  0s |  |  the patch passed  |
   | +1 :green_heart: |  blanks  |   0m  0s |  |  The patch has no blanks issues.  |
   | -0 :warning: |  checkstyle  |   1m  0s | [/results-checkstyle-hadoop-common-project_hadoop-common.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2767/6/artifact/out/results-checkstyle-hadoop-common-project_hadoop-common.txt) |  hadoop-common-project/hadoop-common: The patch generated 1 new + 211 unchanged - 7 fixed = 212 total (was 218)  |
   | +1 :green_heart: |  mvnsite  |   1m 26s |  |  the patch passed  |
   | +1 :green_heart: |  xml  |   0m  1s |  |  The patch has no ill-formed XML file.  |
   | +1 :green_heart: |  javadoc  |   0m 57s |  |  the patch passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javadoc  |   1m 28s |  |  the patch passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  spotbugs  |   2m 30s |  |  the patch passed  |
   | +1 :green_heart: |  shadedclient  |  18m 13s |  |  patch has no errors when building and testing our client artifacts.  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |  17m 24s |  |  hadoop-common in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 46s |  |  The patch does not generate ASF License warnings.  |
   |  |   | 209m 15s |  |  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2767/6/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2767 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient codespell xml spotbugs checkstyle cc buflint bufcompat |
   | uname | Linux 45b53b91785e 4.15.0-126-generic #129-Ubuntu SMP Mon Nov 23 18:53:38 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / e19b1e0f5736eb1913a82516f0c85c4ef72cdd71 |
   | Default Java | Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2767/6/testReport/ |
   | Max. process+thread count | 2289 (vs. ulimit of 5500) |
   | modules | C: hadoop-common-project/hadoop-common U: hadoop-common-project/hadoop-common |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2767/6/console |
   | versions | git=2.25.1 maven=3.6.3 spotbugs=4.2.2 |
   | Powered by | Apache Yetus 0.14.0-SNAPSHOT https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] hadoop-yetus commented on pull request #2767: HDFS-15790. Make ProtobufRpcEngineProtos and ProtobufRpcEngineProtos2 Co-Exist

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on pull request #2767:
URL: https://github.com/apache/hadoop/pull/2767#issuecomment-845223697


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   0m 34s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files found.  |
   | +0 :ok: |  codespell  |   0m  1s |  |  codespell was not available.  |
   | +0 :ok: |  buf  |   0m  1s |  |  buf was not available.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  |  The patch appears to include 6 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  34m  2s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |  20m 52s |  |  trunk passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  compile  |  18m 14s |  |  trunk passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  checkstyle  |   1m  8s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   1m 32s |  |  trunk passed  |
   | +1 :green_heart: |  javadoc  |   1m  6s |  |  trunk passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javadoc  |   1m 39s |  |  trunk passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  spotbugs  |   2m 21s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  15m 33s |  |  branch has no errors when building and testing our client artifacts.  |
   |||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 53s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |  20m 15s |  |  the patch passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | -1 :x: |  cc  |  20m 15s | [/results-compile-cc-root-jdkUbuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2767/7/artifact/out/results-compile-cc-root-jdkUbuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04.txt) |  root-jdkUbuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04 with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04 generated 38 new + 289 unchanged - 38 fixed = 327 total (was 327)  |
   | -1 :x: |  javac  |  20m 15s | [/results-compile-javac-root-jdkUbuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2767/7/artifact/out/results-compile-javac-root-jdkUbuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04.txt) |  root-jdkUbuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04 with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04 generated 1 new + 1987 unchanged - 0 fixed = 1988 total (was 1987)  |
   | +1 :green_heart: |  compile  |  18m 13s |  |  the patch passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | -1 :x: |  cc  |  18m 13s | [/results-compile-cc-root-jdkPrivateBuild-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2767/7/artifact/out/results-compile-cc-root-jdkPrivateBuild-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08.txt) |  root-jdkPrivateBuild-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08 with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08 generated 29 new + 298 unchanged - 29 fixed = 327 total (was 327)  |
   | -1 :x: |  javac  |  18m 13s | [/results-compile-javac-root-jdkPrivateBuild-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2767/7/artifact/out/results-compile-javac-root-jdkPrivateBuild-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08.txt) |  root-jdkPrivateBuild-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08 with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08 generated 1 new + 1890 unchanged - 0 fixed = 1891 total (was 1890)  |
   | +1 :green_heart: |  blanks  |   0m  0s |  |  The patch has no blanks issues.  |
   | -0 :warning: |  checkstyle  |   1m  6s | [/results-checkstyle-hadoop-common-project_hadoop-common.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2767/7/artifact/out/results-checkstyle-hadoop-common-project_hadoop-common.txt) |  hadoop-common-project/hadoop-common: The patch generated 1 new + 210 unchanged - 7 fixed = 211 total (was 217)  |
   | +1 :green_heart: |  mvnsite  |   1m 30s |  |  the patch passed  |
   | +1 :green_heart: |  xml  |   0m  2s |  |  The patch has no ill-formed XML file.  |
   | +1 :green_heart: |  javadoc  |   1m  4s |  |  the patch passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javadoc  |   1m 39s |  |  the patch passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  spotbugs  |   2m 29s |  |  the patch passed  |
   | +1 :green_heart: |  shadedclient  |  16m 58s |  |  patch has no errors when building and testing our client artifacts.  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |  17m  9s |  |  hadoop-common in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 51s |  |  The patch does not generate ASF License warnings.  |
   |  |   | 179m 51s |  |  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2767/7/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2767 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient codespell xml spotbugs checkstyle cc buflint bufcompat |
   | uname | Linux 26833a6aab00 4.15.0-112-generic #113-Ubuntu SMP Thu Jul 9 23:41:39 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / 5f8642d516ecb54b9041197e53c5e4de32f30b57 |
   | Default Java | Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2767/7/testReport/ |
   | Max. process+thread count | 1266 (vs. ulimit of 5500) |
   | modules | C: hadoop-common-project/hadoop-common U: hadoop-common-project/hadoop-common |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2767/7/console |
   | versions | git=2.25.1 maven=3.6.3 spotbugs=4.2.2 |
   | Powered by | Apache Yetus 0.14.0-SNAPSHOT https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] hadoop-yetus commented on pull request #2767: HDFS-15790. Make ProtobufRpcEngineProtos and ProtobufRpcEngineProtos2 Co-Exist

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on pull request #2767:
URL: https://github.com/apache/hadoop/pull/2767#issuecomment-800282693


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   0m 58s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files found.  |
   | +0 :ok: |  codespell  |   0m  2s |  |  codespell was not available.  |
   | +0 :ok: |  buf  |   0m  2s |  |  buf was not available.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  |  The patch appears to include 6 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  35m  9s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |  22m 48s |  |  trunk passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  compile  |  19m 11s |  |  trunk passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  checkstyle  |   1m  0s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   1m 30s |  |  trunk passed  |
   | +1 :green_heart: |  javadoc  |   1m  1s |  |  trunk passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javadoc  |   1m 31s |  |  trunk passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  spotbugs  |   2m 21s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  18m 11s |  |  branch has no errors when building and testing our client artifacts.  |
   |||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 55s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |  21m 49s |  |  the patch passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | -1 :x: |  cc  |  21m 49s | [/results-compile-cc-root-jdkUbuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2767/4/artifact/out/results-compile-cc-root-jdkUbuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04.txt) |  root-jdkUbuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04 with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04 generated 17 new + 395 unchanged - 17 fixed = 412 total (was 412)  |
   | +1 :green_heart: |  javac  |  21m 49s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |  19m 20s |  |  the patch passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | -1 :x: |  cc  |  19m 20s | [/results-compile-cc-root-jdkPrivateBuild-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2767/4/artifact/out/results-compile-cc-root-jdkPrivateBuild-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08.txt) |  root-jdkPrivateBuild-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08 with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08 generated 30 new + 382 unchanged - 30 fixed = 412 total (was 412)  |
   | +1 :green_heart: |  javac  |  19m 20s |  |  the patch passed  |
   | +1 :green_heart: |  blanks  |   0m  0s |  |  The patch has no blanks issues.  |
   | -0 :warning: |  checkstyle  |   1m  2s | [/results-checkstyle-hadoop-common-project_hadoop-common.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2767/4/artifact/out/results-checkstyle-hadoop-common-project_hadoop-common.txt) |  hadoop-common-project/hadoop-common: The patch generated 1 new + 211 unchanged - 7 fixed = 212 total (was 218)  |
   | +1 :green_heart: |  mvnsite  |   1m 30s |  |  the patch passed  |
   | +1 :green_heart: |  xml  |   0m  1s |  |  The patch has no ill-formed XML file.  |
   | +1 :green_heart: |  javadoc  |   0m 57s |  |  the patch passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javadoc  |   1m 32s |  |  the patch passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  spotbugs  |   2m 30s |  |  the patch passed  |
   | +1 :green_heart: |  shadedclient  |  17m 57s |  |  patch has no errors when building and testing our client artifacts.  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |  17m  6s |  |  hadoop-common in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 48s |  |  The patch does not generate ASF License warnings.  |
   |  |   | 189m 13s |  |  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2767/4/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2767 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient codespell xml spotbugs checkstyle cc buflint bufcompat |
   | uname | Linux a44c5fa3fb28 4.15.0-101-generic #102-Ubuntu SMP Mon May 11 10:07:26 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / f0f74f6353c5b32dc169ba6ae3716adf0f97677d |
   | Default Java | Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2767/4/testReport/ |
   | Max. process+thread count | 3143 (vs. ulimit of 5500) |
   | modules | C: hadoop-common-project/hadoop-common U: hadoop-common-project/hadoop-common |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2767/4/console |
   | versions | git=2.25.1 maven=3.6.3 spotbugs=4.2.2 |
   | Powered by | Apache Yetus 0.14.0-SNAPSHOT https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] aajisaka commented on a change in pull request #2767: HDFS-15790. Make ProtobufRpcEngineProtos and ProtobufRpcEngineProtos2 Co-Exist

Posted by GitBox <gi...@apache.org>.
aajisaka commented on a change in pull request #2767:
URL: https://github.com/apache/hadoop/pull/2767#discussion_r610378108



##########
File path: hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestProtoBufRpc.java
##########
@@ -179,10 +281,41 @@ public void testProtoBufRpc2() throws Exception {
     MetricsRecordBuilder rpcDetailedMetrics = 
         getMetrics(server.getRpcDetailedMetrics().name());
     assertCounterGt("Echo2NumOps", 0L, rpcDetailedMetrics);
+
+    if (testWithLegacy) {
+      testProtobufLegacy();
+    }
+  }
+
+  private void testProtobufLegacy()
+      throws IOException, com.google.protobuf.ServiceException {
+    TestRpcService2Legacy client = getClientLegacy();
+
+    // Test ping method
+    client.ping2(null, TestProtosLegacy.EmptyRequestProto.newBuilder().build());
+
+    // Test echo method
+    TestProtosLegacy.EchoResponseProto echoResponse = client.echo2(null,
+        TestProtosLegacy.EchoRequestProto.newBuilder().setMessage("hello")
+            .build());
+    assertThat(echoResponse.getMessage()).isEqualTo("hello");
+
+    // Ensure RPC metrics are updated
+    MetricsRecordBuilder rpcMetrics = getMetrics(server.getRpcMetrics().name());
+    assertCounterGt("RpcQueueTimeNumOps", 0L, rpcMetrics);
+    assertCounterGt("RpcProcessingTimeNumOps", 0L, rpcMetrics);
+
+    MetricsRecordBuilder rpcDetailedMetrics =
+        getMetrics(server.getRpcDetailedMetrics().name());
+    assertCounterGt("Echo2NumOps", 0L, rpcDetailedMetrics);
   }
 
   @Test (timeout=5000)
   public void testProtoBufRandomException() throws Exception {
+    if (testWithLegacy) {
+      //No test with legacy
+      return;
+    }

Review comment:
       `assumeFalse(testWithLegacy)` can be used instead.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] vinayakumarb commented on a change in pull request #2767: HDFS-15790. Make ProtobufRpcEngineProtos and ProtobufRpcEngineProtos2 Co-Exist

Posted by GitBox <gi...@apache.org>.
vinayakumarb commented on a change in pull request #2767:
URL: https://github.com/apache/hadoop/pull/2767#discussion_r637551551



##########
File path: hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine2.java
##########
@@ -495,6 +524,7 @@ private static ProtoClassProtoImpl getProtocolImpl(RPC.Server server,
        * it is.</li>
        * </ol>
        */
+      @SuppressWarnings("deprecation")

Review comment:
       This was to suppress javac warnings due to usage of {{ProtobufRpcEngine}} in {{ProtobufRpcEngine2}}. I have moved to new method, where this exact usage is present.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] hadoop-yetus commented on pull request #2767: HDFS-15790. Make ProtobufRpcEngineProtos and ProtobufRpcEngineProtos2 Co-Exist

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on pull request #2767:
URL: https://github.com/apache/hadoop/pull/2767#issuecomment-826069768






-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


[GitHub] [hadoop] jojochuang commented on pull request #2767: HDFS-15790. Make ProtobufRpcEngineProtos and ProtobufRpcEngineProtos2 Co-Exist

Posted by GitBox <gi...@apache.org>.
jojochuang commented on pull request #2767:
URL: https://github.com/apache/hadoop/pull/2767#issuecomment-846926246


   merged to trunk. thanks @vinayakumarb and @aajisaka .
   branch-3.3 backport is under way. There are just a few trivial import conflicts. I'll resolve them and push the commits later.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org