hadoop icon indicating copy to clipboard operation
hadoop copied to clipboard

HADOOP-18487. protobuf 2.5.0 marked as provided.

Open steveloughran opened this issue 3 years ago • 4 comments

The option protobuf.scope defines whether the protobuf 2.5.0 dependency is marked as provided or not.

  • all declarations except those in yarn-csi are updated
  • those modules which don't compile without their own explicit import (hadoop-hdfs-client, hadoop-hdfs-rbf)

It's actually interesting to see where/how that compile fails

hadoop-hdfs-client: ClientNamenodeProtocolTranslatorPB hadoop-hdfs-rbf:RouterAdminProtocolTranslatorPB

both with "class file for com.google.protobuf.ServiceException not found", even though neither class uses it

what they do have is references to ProtobufHelper.getRemoteException(), which is overloaded to both the shaded ServiceException and the original one

Hypothesis: the javac overload resolution needs to look at the entire class hierarchy before it can decide which one to use.

Proposed: add a new method ioe extractException(org.apache.hadoop.thirdparty.protobuf.ServiceException) and move our own code to it. Without the overloading the classes should not be needed

Change-Id: I70354abfe3f1fdc03c418dac88e60f8cc4929a33

How was this patch tested?

local compile, now looking at yetus runs

For code changes:

  • [ ] Does the title or this PR starts with the corresponding JIRA issue id (e.g. 'HADOOP-17799. Your PR title ...')?
  • [ ] Object storage: have the integration tests been executed and the endpoint declared according to the connector-specific documentation?
  • [ ] If adding new dependencies to the code, are these dependencies licensed in a way that is compatible for inclusion under ASF 2.0?
  • [ ] If applicable, have you updated the LICENSE, LICENSE-binary, NOTICE-binary files?

steveloughran avatar Oct 10 '22 16:10 steveloughran

:broken_heart: -1 overall

Vote Subsystem Runtime Logfile Comment
+0 :ok: reexec 1m 3s 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: detsecrets 0m 0s detect-secrets was not available.
+0 :ok: xmllint 0m 0s xmllint was not available.
+1 :green_heart: @author 0m 0s The patch does not contain any @author tags.
-1 :x: test4tests 0m 0s The patch doesn't appear to include any new or modified tests. Please justify why no new tests are needed for this patch. Also please list what manual steps were performed to verify this patch.
_ trunk Compile Tests _
+0 :ok: mvndep 15m 33s Maven dependency ordering for branch
+1 :green_heart: mvninstall 25m 56s trunk passed
+1 :green_heart: compile 23m 26s trunk passed with JDK Ubuntu-11.0.16+8-post-Ubuntu-0ubuntu120.04
+1 :green_heart: compile 20m 54s trunk passed with JDK Private Build-1.8.0_342-8u342-b07-0ubuntu1~20.04-b07
+1 :green_heart: mvnsite 17m 51s trunk passed
+1 :green_heart: javadoc 14m 46s trunk passed with JDK Ubuntu-11.0.16+8-post-Ubuntu-0ubuntu120.04
+1 :green_heart: javadoc 14m 58s trunk passed with JDK Private Build-1.8.0_342-8u342-b07-0ubuntu1~20.04-b07
+1 :green_heart: shadedclient 154m 28s branch has no errors when building and testing our client artifacts.
_ Patch Compile Tests _
+0 :ok: mvndep 0m 34s Maven dependency ordering for patch
+1 :green_heart: mvninstall 10m 22s the patch passed
+1 :green_heart: compile 22m 38s the patch passed with JDK Ubuntu-11.0.16+8-post-Ubuntu-0ubuntu120.04
+1 :green_heart: javac 22m 38s the patch passed
+1 :green_heart: compile 20m 45s the patch passed with JDK Private Build-1.8.0_342-8u342-b07-0ubuntu1~20.04-b07
+1 :green_heart: javac 20m 45s the patch passed
+1 :green_heart: blanks 0m 0s The patch has no blanks issues.
+1 :green_heart: mvnsite 17m 16s the patch passed
+1 :green_heart: javadoc 14m 32s the patch passed with JDK Ubuntu-11.0.16+8-post-Ubuntu-0ubuntu120.04
+1 :green_heart: javadoc 14m 49s the patch passed with JDK Private Build-1.8.0_342-8u342-b07-0ubuntu1~20.04-b07
+1 :green_heart: shadedclient 71m 23s patch has no errors when building and testing our client artifacts.
_ Other Tests _
+1 :green_heart: unit 1m 1s hadoop-project in the patch passed.
+1 :green_heart: unit 18m 42s hadoop-common in the patch passed.
+1 :green_heart: unit 3m 14s hadoop-hdfs-client in the patch passed.
+1 :green_heart: unit 392m 59s hadoop-hdfs in the patch passed.
+1 :green_heart: unit 4m 22s hadoop-hdfs-nfs in the patch passed.
+1 :green_heart: unit 5m 40s hadoop-yarn-common in the patch passed.
+1 :green_heart: unit 4m 1s hadoop-yarn-server-common in the patch passed.
+1 :green_heart: unit 103m 12s hadoop-yarn-server-resourcemanager in the patch passed.
+1 :green_heart: unit 173m 59s hadoop-mapreduce-client in the patch passed.
+1 :green_heart: unit 39m 15s hadoop-hdfs-rbf in the patch passed.
+1 :green_heart: unit 22m 52s hadoop-yarn-services-core in the patch passed.
+1 :green_heart: unit 1m 52s hadoop-yarn-csi in the patch passed.
+1 :green_heart: asflicense 1m 50s The patch does not generate ASF License warnings.
1056m 33s
Subsystem Report/Notes
Docker ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-4996/1/artifact/out/Dockerfile
GITHUB PR https://github.com/apache/hadoop/pull/4996
Optional Tests dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient codespell detsecrets xmllint
uname Linux 91becc1c7c72 4.15.0-191-generic #202-Ubuntu SMP Thu Aug 4 01:49:29 UTC 2022 x86_64 x86_64 x86_64 GNU/Linux
Build tool maven
Personality dev-support/bin/hadoop.sh
git revision trunk / f8353cd97189ce338125c82d8335b4f97434b4c0
Default Java Private Build-1.8.0_342-8u342-b07-0ubuntu1~20.04-b07
Multi-JDK versions /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.16+8-post-Ubuntu-0ubuntu120.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_342-8u342-b07-0ubuntu1~20.04-b07
Test Results https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-4996/1/testReport/
Max. process+thread count 3839 (vs. ulimit of 5500)
modules C: hadoop-project hadoop-common-project/hadoop-common hadoop-hdfs-project/hadoop-hdfs-client hadoop-hdfs-project/hadoop-hdfs hadoop-hdfs-project/hadoop-hdfs-nfs hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager hadoop-mapreduce-project/hadoop-mapreduce-client hadoop-hdfs-project/hadoop-hdfs-rbf hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core hadoop-yarn-project/hadoop-yarn/hadoop-yarn-csi U: .
Console output https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-4996/1/console
versions git=2.25.1 maven=3.6.3
Powered by Apache Yetus 0.14.0 https://yetus.apache.org

This message was automatically generated.

hadoop-yetus avatar Oct 11 '22 10:10 hadoop-yetus

having just seen #2026 i'm not convinced we can remove protobuf2.5 off the classpath of anything using the RPC classes. There's a lot of internal probing for object types and dynamic choice of shaded/unshaded classes.

It might be possible to pull out the code from the shared classes (server, rpcwritable...) so be confident that rpc clients only using the shaded libraries and the RpcEngine2 were safely isolated, but right now I am not convinced that is true. Certainly hdfs, mr and yarn servers and clients MUST export protobuf2, even if hadoop-common downgrades it.

steveloughran avatar Oct 12 '22 17:10 steveloughran

:broken_heart: -1 overall

Vote Subsystem Runtime Logfile Comment
+0 :ok: reexec 1m 8s 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: detsecrets 0m 1s detect-secrets was not available.
+0 :ok: xmllint 0m 1s xmllint 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 3 new or modified test files.
_ trunk Compile Tests _
+0 :ok: mvndep 16m 0s Maven dependency ordering for branch
+1 :green_heart: mvninstall 25m 43s trunk passed
+1 :green_heart: compile 23m 16s trunk passed with JDK Ubuntu-11.0.16+8-post-Ubuntu-0ubuntu120.04
+1 :green_heart: compile 20m 58s trunk passed with JDK Private Build-1.8.0_342-8u342-b07-0ubuntu1~20.04-b07
+1 :green_heart: checkstyle 4m 10s trunk passed
+1 :green_heart: mvnsite 19m 4s trunk passed
+1 :green_heart: javadoc 16m 40s trunk passed with JDK Ubuntu-11.0.16+8-post-Ubuntu-0ubuntu120.04
+1 :green_heart: javadoc 16m 24s trunk passed with JDK Private Build-1.8.0_342-8u342-b07-0ubuntu1~20.04-b07
+0 :ok: spotbugs 1m 18s branch/hadoop-project no spotbugs output file (spotbugsXml.xml)
+1 :green_heart: shadedclient 22m 38s branch has no errors when building and testing our client artifacts.
_ Patch Compile Tests _
+0 :ok: mvndep 0m 28s Maven dependency ordering for patch
+1 :green_heart: mvninstall 11m 0s the patch passed
+1 :green_heart: compile 22m 43s the patch passed with JDK Ubuntu-11.0.16+8-post-Ubuntu-0ubuntu120.04
+1 :green_heart: javac 22m 43s the patch passed
+1 :green_heart: compile 20m 42s the patch passed with JDK Private Build-1.8.0_342-8u342-b07-0ubuntu1~20.04-b07
+1 :green_heart: javac 20m 42s the patch passed
+1 :green_heart: blanks 0m 0s The patch has no blanks issues.
+1 :green_heart: checkstyle 4m 7s root: The patch generated 0 new + 226 unchanged - 4 fixed = 226 total (was 230)
+1 :green_heart: mvnsite 18m 36s the patch passed
+1 :green_heart: javadoc 15m 45s the patch passed with JDK Ubuntu-11.0.16+8-post-Ubuntu-0ubuntu120.04
+1 :green_heart: javadoc 15m 29s the patch passed with JDK Private Build-1.8.0_342-8u342-b07-0ubuntu1~20.04-b07
+0 :ok: spotbugs 0m 50s hadoop-project has no data from spotbugs
+1 :green_heart: shadedclient 22m 4s patch has no errors when building and testing our client artifacts.
_ Other Tests _
+1 :green_heart: unit 0m 52s hadoop-project in the patch passed.
+1 :green_heart: unit 18m 54s hadoop-common in the patch passed.
+1 :green_heart: unit 2m 58s hadoop-hdfs-client in the patch passed.
-1 :x: unit 416m 15s /patch-unit-hadoop-hdfs-project_hadoop-hdfs.txt hadoop-hdfs in the patch passed.
+1 :green_heart: unit 4m 30s hadoop-hdfs-nfs in the patch passed.
-1 :x: unit 1m 31s /patch-unit-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-common.txt hadoop-yarn-common in the patch failed.
+1 :green_heart: unit 4m 1s hadoop-yarn-server-common in the patch passed.
-1 :x: unit 104m 10s /patch-unit-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager.txt hadoop-yarn-server-resourcemanager in the patch passed.
-1 :x: unit 28m 14s /patch-unit-hadoop-mapreduce-project_hadoop-mapreduce-client.txt hadoop-mapreduce-client in the patch failed.
-1 :x: unit 1m 28s /patch-unit-hadoop-mapreduce-project_hadoop-mapreduce-client_hadoop-mapreduce-client-hs.txt hadoop-mapreduce-client-hs in the patch failed.
-1 :x: unit 1m 35s /patch-unit-hadoop-hdfs-project_hadoop-hdfs-rbf.txt hadoop-hdfs-rbf in the patch failed.
+1 :green_heart: unit 22m 46s hadoop-yarn-services-core in the patch passed.
+1 :green_heart: unit 1m 39s hadoop-yarn-csi in the patch passed.
+1 :green_heart: asflicense 1m 52s The patch does not generate ASF License warnings.
974m 32s
Reason Tests
Failed junit tests hadoop.hdfs.server.datanode.fsdataset.impl.TestFsDatasetImpl
hadoop.yarn.server.resourcemanager.reservation.TestCapacityOverTimePolicy
Subsystem Report/Notes
Docker ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-4996/2/artifact/out/Dockerfile
GITHUB PR https://github.com/apache/hadoop/pull/4996
Optional Tests dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient codespell detsecrets xmllint spotbugs checkstyle
uname Linux 557f8b175e7f 4.15.0-191-generic #202-Ubuntu SMP Thu Aug 4 01:49:29 UTC 2022 x86_64 x86_64 x86_64 GNU/Linux
Build tool maven
Personality dev-support/bin/hadoop.sh
git revision trunk / 725e430362497eb50f450c4d0a870a8adb75e621
Default Java Private Build-1.8.0_342-8u342-b07-0ubuntu1~20.04-b07
Multi-JDK versions /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.16+8-post-Ubuntu-0ubuntu120.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_342-8u342-b07-0ubuntu1~20.04-b07
Test Results https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-4996/2/testReport/
Max. process+thread count 2806 (vs. ulimit of 5500)
modules C: hadoop-project hadoop-common-project/hadoop-common hadoop-hdfs-project/hadoop-hdfs-client hadoop-hdfs-project/hadoop-hdfs hadoop-hdfs-project/hadoop-hdfs-nfs hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager hadoop-mapreduce-project/hadoop-mapreduce-client hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs hadoop-hdfs-project/hadoop-hdfs-rbf hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core hadoop-yarn-project/hadoop-yarn/hadoop-yarn-csi U: .
Console output https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-4996/2/console
versions git=2.25.1 maven=3.6.3 spotbugs=4.2.2
Powered by Apache Yetus 0.14.0 https://yetus.apache.org

This message was automatically generated.

hadoop-yetus avatar Oct 13 '22 07:10 hadoop-yetus

we can't cut an unshaded protobuf of some form without RPC not linking, so hbase/hive/ozone are in trouble here. changes to RPC.java required to somehow add ability to probe a class for being a subclass of com.google.protobuf.Message without having com.google.protobuf.Message on the classpath would be needed.

this does not need to be protobuf 2.5.

I propose

  1. changing some of the imports of hadoop-common to exclude it (cloud components)
  2. changing retention here back to compile that way someone can turn it off if they want to, but it is not something we do in normal asf releases.

steveloughran avatar Oct 13 '22 10:10 steveloughran

:broken_heart: -1 overall

Vote Subsystem Runtime Logfile Comment
+0 :ok: reexec 0m 59s 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: detsecrets 0m 1s detect-secrets was not available.
+0 :ok: xmllint 0m 1s xmllint 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 3 new or modified test files.
_ trunk Compile Tests _
+0 :ok: mvndep 15m 36s Maven dependency ordering for branch
+1 :green_heart: mvninstall 25m 57s trunk passed
+1 :green_heart: compile 23m 22s trunk passed with JDK Ubuntu-11.0.16+8-post-Ubuntu-0ubuntu120.04
+1 :green_heart: compile 20m 55s trunk passed with JDK Private Build-1.8.0_342-8u342-b07-0ubuntu1~20.04-b07
+1 :green_heart: checkstyle 4m 26s trunk passed
+1 :green_heart: mvnsite 18m 58s trunk passed
+1 :green_heart: javadoc 15m 50s trunk passed with JDK Ubuntu-11.0.16+8-post-Ubuntu-0ubuntu120.04
+1 :green_heart: javadoc 15m 53s trunk passed with JDK Private Build-1.8.0_342-8u342-b07-0ubuntu1~20.04-b07
+0 :ok: spotbugs 1m 20s branch/hadoop-project no spotbugs output file (spotbugsXml.xml)
+1 :green_heart: shadedclient 21m 57s branch has no errors when building and testing our client artifacts.
-0 :warning: patch 22m 27s Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.
_ Patch Compile Tests _
+0 :ok: mvndep 0m 34s Maven dependency ordering for patch
+1 :green_heart: mvninstall 10m 43s the patch passed
+1 :green_heart: compile 22m 34s the patch passed with JDK Ubuntu-11.0.16+8-post-Ubuntu-0ubuntu120.04
-1 :x: javac 22m 34s /results-compile-javac-root-jdkUbuntu-11.0.16+8-post-Ubuntu-0ubuntu120.04.txt root-jdkUbuntu-11.0.16+8-post-Ubuntu-0ubuntu120.04 with JDK Ubuntu-11.0.16+8-post-Ubuntu-0ubuntu120.04 generated 1 new + 2821 unchanged - 1 fixed = 2822 total (was 2822)
+1 :green_heart: compile 20m 38s the patch passed with JDK Private Build-1.8.0_342-8u342-b07-0ubuntu1~20.04-b07
-1 :x: javac 20m 38s /results-compile-javac-root-jdkPrivateBuild-1.8.0_342-8u342-b07-0ubuntu1~20.04-b07.txt root-jdkPrivateBuild-1.8.0_342-8u342-b07-0ubuntu1~20.04-b07 with JDK Private Build-1.8.0_342-8u342-b07-0ubuntu1~20.04-b07 generated 1 new + 2614 unchanged - 1 fixed = 2615 total (was 2615)
+1 :green_heart: blanks 0m 0s The patch has no blanks issues.
-0 :warning: checkstyle 4m 1s /results-checkstyle-root.txt root: The patch generated 1 new + 273 unchanged - 5 fixed = 274 total (was 278)
+1 :green_heart: mvnsite 18m 45s the patch passed
+1 :green_heart: javadoc 16m 19s the patch passed with JDK Ubuntu-11.0.16+8-post-Ubuntu-0ubuntu120.04
+1 :green_heart: javadoc 15m 42s the patch passed with JDK Private Build-1.8.0_342-8u342-b07-0ubuntu1~20.04-b07
+0 :ok: spotbugs 0m 52s hadoop-project has no data from spotbugs
-1 :x: spotbugs 3m 4s /new-spotbugs-hadoop-common-project_hadoop-common.html hadoop-common-project/hadoop-common generated 3 new + 0 unchanged - 0 fixed = 3 total (was 0)
-1 :x: shadedclient 21m 47s patch has errors when building and testing our client artifacts.
_ Other Tests _
+1 :green_heart: unit 0m 53s hadoop-project in the patch passed.
+1 :green_heart: unit 18m 42s hadoop-common in the patch passed.
+1 :green_heart: unit 3m 3s hadoop-hdfs-client in the patch passed.
-1 :x: unit 288m 56s /patch-unit-hadoop-hdfs-project_hadoop-hdfs.txt hadoop-hdfs in the patch failed.
+1 :green_heart: unit 4m 53s hadoop-hdfs-nfs in the patch passed.
-1 :x: unit 1m 52s /patch-unit-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-common.txt hadoop-yarn-common in the patch failed.
+1 :green_heart: unit 4m 23s hadoop-yarn-server-common in the patch passed.
-1 :x: unit 2m 0s /patch-unit-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager.txt hadoop-yarn-server-resourcemanager in the patch failed.
-1 :x: unit 28m 16s /patch-unit-hadoop-mapreduce-project_hadoop-mapreduce-client.txt hadoop-mapreduce-client in the patch failed.
-1 :x: unit 1m 41s /patch-unit-hadoop-mapreduce-project_hadoop-mapreduce-client_hadoop-mapreduce-client-hs.txt hadoop-mapreduce-client-hs in the patch failed.
-1 :x: unit 1m 48s /patch-unit-hadoop-hdfs-project_hadoop-hdfs-rbf.txt hadoop-hdfs-rbf in the patch failed.
+1 :green_heart: unit 22m 48s hadoop-yarn-services-core in the patch passed.
+1 :green_heart: unit 2m 4s hadoop-yarn-csi in the patch passed.
+1 :green_heart: asflicense 2m 12s The patch does not generate ASF License warnings.
745m 48s
Reason Tests
SpotBugs module:hadoop-common-project/hadoop-common
Sequence of calls to java.util.concurrent.ConcurrentHashMap may not be atomic in org.apache.hadoop.ipc.internal.ShadedProtobufHelper.getFixedByteString(String) At ShadedProtobufHelper.java:may not be atomic in org.apache.hadoop.ipc.internal.ShadedProtobufHelper.getFixedByteString(String) At ShadedProtobufHelper.java:[line 98]
Sequence of calls to java.util.concurrent.ConcurrentHashMap may not be atomic in org.apache.hadoop.ipc.internal.ShadedProtobufHelper.getFixedByteString(Text) At ShadedProtobufHelper.java:may not be atomic in org.apache.hadoop.ipc.internal.ShadedProtobufHelper.getFixedByteString(Text) At ShadedProtobufHelper.java:[line 84]
There is an apparent infinite recursive loop in org.apache.hadoop.ipc.internal.ShadedProtobufHelper.getFixedByteString(String) At ShadedProtobufHelper.java:recursive loop in org.apache.hadoop.ipc.internal.ShadedProtobufHelper.getFixedByteString(String) At ShadedProtobufHelper.java:[line 100]
Unreaped Processes hadoop-hdfs:2
Failed junit tests hadoop.hdfs.protocolPB.TestPBHelper
hadoop.hdfs.server.blockmanagement.TestPendingDataNodeMessages
hadoop.hdfs.TestDecommission
hadoop.hdfs.TestDFSStripedOutputStream
hadoop.hdfs.TestAppendSnapshotTruncate
hadoop.hdfs.TestDFSStripedOutputStreamUpdatePipeline
hadoop.hdfs.TestWriteConfigurationToDFS
hadoop.hdfs.TestFileCreation
hadoop.hdfs.TestConnCache
hadoop.hdfs.server.datanode.TestDataNodeErasureCodingMetrics
hadoop.hdfs.server.datanode.TestDataNodeVolumeMetrics
hadoop.hdfs.server.balancer.TestBalancerRPCDelay
hadoop.hdfs.TestDistributedFileSystemWithECFileWithRandomECPolicy
hadoop.hdfs.tools.TestStoragePolicySatisfyAdminCommands
hadoop.hdfs.TestMissingBlocksAlert
hadoop.hdfs.TestAppendDifferentChecksum
hadoop.hdfs.server.datanode.fsdataset.impl.TestReplicaCachingGetSpaceUsed
hadoop.hdfs.TestPersistBlocks
hadoop.hdfs.server.datanode.TestTransferRbw
hadoop.hdfs.server.balancer.TestBalancerService
hadoop.hdfs.TestRollingUpgrade
hadoop.hdfs.TestErasureCodeBenchmarkThroughput
hadoop.hdfs.server.datanode.TestTriggerBlockReport
hadoop.hdfs.server.datanode.TestCachingStrategy
hadoop.hdfs.TestBatchedListDirectories
hadoop.hdfs.TestBlockMissingException
hadoop.hdfs.TestFileCorruption
hadoop.hdfs.TestBlocksScheduledCounter
hadoop.hdfs.TestDFSInotifyEventInputStream
hadoop.hdfs.TestReadWhileWriting
hadoop.hdfs.TestStateAlignmentContextWithHA
hadoop.hdfs.TestErasureCodingPoliciesWithRandomECPolicy
hadoop.hdfs.TestAbandonBlock
hadoop.hdfs.TestDecommissionWithBackoffMonitor
hadoop.hdfs.TestStoragePolicyPermissionSettings
hadoop.hdfs.TestAclsEndToEnd
hadoop.hdfs.server.balancer.TestBalancerWithSaslDataTransfer
hadoop.hdfs.TestGetFileChecksum
hadoop.hdfs.server.balancer.TestBalancerWithNodeGroup
hadoop.hdfs.TestErasureCodingPolicyWithSnapshot
hadoop.hdfs.TestDecommissionWithStriped
hadoop.hdfs.TestViewDistributedFileSystemContract
hadoop.hdfs.TestReadStripedFileWithDecodingCorruptData
hadoop.hdfs.TestClientReportBadBlock
hadoop.hdfs.TestEncryptedTransfer
hadoop.hdfs.server.datanode.TestDiskError
hadoop.hdfs.server.datanode.fsdataset.impl.TestLazyPersistLockedMemory
hadoop.hdfs.server.mover.TestStorageMover
hadoop.hdfs.server.datanode.fsdataset.impl.TestProvidedImpl
hadoop.hdfs.server.datanode.fsdataset.impl.TestLazyPersistFiles
hadoop.hdfs.TestDataStream
hadoop.hdfs.tools.TestDFSAdminWithHA
hadoop.hdfs.TestSafeMode
hadoop.hdfs.shortcircuit.TestShortCircuitCache
hadoop.hdfs.server.balancer.TestBalancerWithHANameNodes
hadoop.hdfs.TestDFSShell
hadoop.hdfs.TestDFSStartupVersions
hadoop.hdfs.TestFileLengthOnClusterRestart
hadoop.hdfs.TestReconstructStripedFileWithRandomECPolicy
hadoop.hdfs.server.datanode.fsdataset.impl.TestDatanodeRestart
hadoop.hdfs.TestStripedFileAppend
hadoop.hdfs.TestExternalBlockReader
hadoop.hdfs.TestDatanodeLayoutUpgrade
hadoop.hdfs.tools.offlineImageViewer.TestOfflineImageViewerForStoragePolicy
hadoop.hdfs.TestBlockTokenWrappingQOP
hadoop.hdfs.server.datanode.TestDataNodeMetrics
hadoop.hdfs.tools.offlineEditsViewer.TestOfflineEditsViewer
hadoop.hdfs.TestByteBufferPread
hadoop.hdfs.server.datanode.TestBlockReplacement
hadoop.hdfs.tools.TestWebHDFSStoragePolicyCommands
hadoop.hdfs.server.balancer.TestBalancerLongRunningTasks
hadoop.hdfs.TestDFSStripedInputStream
hadoop.hdfs.TestHAAuxiliaryPort
hadoop.hdfs.server.datanode.TestDeleteBlockPool
hadoop.hdfs.TestFileCreationDelete
hadoop.hdfs.TestTrashWithSecureEncryptionZones
hadoop.hdfs.server.datanode.TestDataNodeVolumeFailureToleration
hadoop.hdfs.TestSnapshotCommands
hadoop.hdfs.TestEncryptionZonesWithKMS
hadoop.hdfs.TestDFSPermission
hadoop.hdfs.TestHDFSFileSystemContract
hadoop.hdfs.TestHDFSTrash
hadoop.hdfs.TestReplication
hadoop.hdfs.TestPread
hadoop.hdfs.TestListFilesInDFS
hadoop.hdfs.TestModTime
hadoop.hdfs.server.datanode.fsdataset.impl.TestLazyPersistReplicaRecovery
hadoop.hdfs.TestDataTransferProtocol
hadoop.hdfs.tools.TestStoragePolicyCommands
hadoop.hdfs.TestFileCreationClient
hadoop.hdfs.TestDatanodeStartupFixesLegacyStorageIDs
hadoop.hdfs.TestHFlush
hadoop.hdfs.TestWriteReadStripedFile
hadoop.hdfs.client.impl.TestBlockReaderLocal
hadoop.hdfs.TestMiniDFSCluster
hadoop.hdfs.TestDeadNodeDetection
hadoop.hdfs.tools.TestDebugAdmin
hadoop.hdfs.TestSetrepIncreasing
hadoop.hdfs.TestErasureCodingExerciseAPIs
hadoop.hdfs.TestDFSStorageStateRecovery
hadoop.hdfs.tools.offlineImageViewer.TestOfflineImageViewerForContentSummary
hadoop.hdfs.TestReplaceDatanodeFailureReplication
hadoop.hdfs.tools.TestViewFSStoragePolicyCommands
hadoop.hdfs.server.datanode.TestCorruptMetadataFile
hadoop.hdfs.TestDFSInputStream
hadoop.hdfs.TestFileAppend3
hadoop.hdfs.qjournal.TestNNWithQJM
hadoop.hdfs.server.datanode.fsdataset.impl.TestLazyWriter
hadoop.hdfs.TestSmallBlock
hadoop.hdfs.TestEncryptionZonesWithHA
hadoop.hdfs.shortcircuit.TestShortCircuitLocalRead
hadoop.hdfs.tools.offlineImageViewer.TestOfflineImageViewerForAcl
hadoop.hdfs.TestDFSMkdirs
hadoop.hdfs.TestDFSRemove
hadoop.hdfs.tools.TestDFSAdmin
hadoop.hdfs.server.datanode.TestBlockHasMultipleReplicasOnSameDN
hadoop.hdfs.server.datanode.fsdataset.impl.TestLazyPersistReplicaPlacement
hadoop.hdfs.TestRestartDFS
hadoop.hdfs.TestFileCreationEmpty
hadoop.hdfs.TestBlockStoragePolicy
hadoop.hdfs.server.datanode.TestDataNodeVolumeFailure
hadoop.hdfs.TestDFSFinalize
hadoop.hdfs.server.balancer.TestBalancerWithMultipleNameNodes
hadoop.hdfs.server.datanode.TestDataNodeRollingUpgrade
hadoop.hdfs.server.datanode.TestDataNodeFaultInjector
hadoop.hdfs.server.datanode.TestBlockRecovery2
hadoop.hdfs.TestDFSStripedOutputStreamWithRandomECPolicy
hadoop.hdfs.TestReconstructStripedFile
hadoop.hdfs.TestReadStripedFileWithDecoding
hadoop.hdfs.TestApplyingStoragePolicy
hadoop.hdfs.TestReadStripedFileWithMissingBlocks
hadoop.hdfs.client.impl.TestBlockReaderFactory
hadoop.hdfs.TestReadStripedFileWithDecodingDeletedData
hadoop.hdfs.TestErasureCodingPolicyWithSnapshotWithRandomECPolicy
hadoop.hdfs.TestErasureCodingPolicies
hadoop.hdfs.server.datanode.TestHSync
hadoop.hdfs.TestInjectionForSimulatedStorage
hadoop.hdfs.TestPipelines
hadoop.hdfs.tools.offlineImageViewer.TestOfflineImageViewerForErasureCodingPolicy
hadoop.hdfs.TestExtendedAcls
hadoop.hdfs.client.impl.TestClientBlockVerification
hadoop.hdfs.TestRenameWhileOpen
hadoop.hdfs.TestFSOutputSummer
hadoop.hdfs.TestFsShellPermission
hadoop.hdfs.TestErasureCodingMultipleRacks
hadoop.hdfs.TestDFSUpgrade
hadoop.hdfs.server.datanode.fsdataset.impl.TestScrLazyPersistFiles
hadoop.hdfs.TestMultiThreadedHflush
hadoop.hdfs.TestMultipleNNPortQOP
hadoop.hdfs.TestLeaseRecovery
hadoop.hdfs.server.datanode.TestDataNodeHotSwapVolumes
hadoop.hdfs.server.datanode.TestBlockRecovery
hadoop.hdfs.server.datanode.fsdataset.impl.TestFsDatasetImpl
hadoop.hdfs.TestFSInputChecker
hadoop.hdfs.TestGetBlocks
hadoop.hdfs.TestRead
hadoop.hdfs.TestDFSShellGenericOptions
hadoop.hdfs.server.datanode.TestIncrementalBrVariations
hadoop.hdfs.TestListFilesInFileContext
hadoop.hdfs.server.datanode.TestBPOfferService
hadoop.hdfs.TestLocalDFS
hadoop.hdfs.server.datanode.TestFsDatasetCacheRevocation
hadoop.hdfs.server.datanode.fsdataset.impl.TestInterDatanodeProtocol
hadoop.hdfs.TestDecommissionWithStripedBackoffMonitor
hadoop.hdfs.server.datanode.TestDataNodeVolumeFailureReporting
hadoop.hdfs.server.blockmanagement.TestSequentialBlockId
hadoop.hdfs.TestLargeBlock
hadoop.hdfs.TestDFSStripedInputStreamWithRandomECPolicy
hadoop.hdfs.TestParallelUnixDomainRead
hadoop.hdfs.TestParallelShortCircuitReadUnCached
hadoop.hdfs.TestFileStatus
hadoop.hdfs.TestViewDistributedFileSystemWithMountLinks
hadoop.hdfs.TestFileConcurrentReader
hadoop.hdfs.server.datanode.TestReadOnlySharedStorage
hadoop.hdfs.server.blockmanagement.TestNodeCount
hadoop.hdfs.TestDataTransferKeepalive
hadoop.hdfs.crypto.TestHdfsCryptoStreams
hadoop.hdfs.security.TestDelegationToken
hadoop.hdfs.server.datanode.TestDataNodeTcpNoDelay
hadoop.hdfs.TestReservedRawPaths
hadoop.hdfs.server.datanode.TestDataXceiverBackwardsCompat
hadoop.hdfs.server.datanode.TestBatchIbr
hadoop.hdfs.tools.offlineImageViewer.TestOfflineImageViewerWithStripedBlocks
hadoop.hdfs.tools.TestDFSZKFailoverController
hadoop.hdfs.TestDFSStripedOutputStreamWithFailureWithRandomECPolicy
hadoop.hdfs.client.impl.TestBlockReaderLocalLegacy
hadoop.hdfs.server.datanode.TestNNHandlesCombinedBlockReport
hadoop.hdfs.TestDFSUpgradeFromImage
hadoop.hdfs.server.datanode.fsdataset.impl.TestLazyPersistPolicy
hadoop.hdfs.security.TestDelegationTokenForProxyUser
hadoop.hdfs.server.datanode.fsdataset.impl.TestFsDatasetCache
hadoop.hdfs.TestDFSClientFailover
hadoop.hdfs.TestReconstructStripedFileWithValidator
hadoop.hdfs.TestParallelShortCircuitLegacyRead
hadoop.hdfs.TestDFSOutputStream
hadoop.hdfs.TestWriteRead
hadoop.hdfs.server.datanode.TestDirectoryScanner
hadoop.hdfs.server.balancer.TestBalancer
hadoop.hdfs.TestDistributedFileSystemWithECFile
hadoop.hdfs.TestClientProtocolForPipelineRecovery
hadoop.hdfs.TestSeekBug
hadoop.hdfs.qjournal.TestSecureNNWithQJM
hadoop.hdfs.TestDFSInputStreamBlockLocations
hadoop.hdfs.TestFetchImage
hadoop.hdfs.TestDisableConnCache
hadoop.hdfs.TestDatanodeReport
hadoop.hdfs.TestClose
hadoop.hdfs.TestEncryptionZones
hadoop.hdfs.client.impl.TestBlockReaderRemote
hadoop.hdfs.TestParallelShortCircuitRead
hadoop.hdfs.server.datanode.TestDataNodeMXBean
hadoop.hdfs.server.datanode.TestNNHandlesBlockReportPerStorage
hadoop.hdfs.TestDatanodeDeath
hadoop.hdfs.TestLeaseRecovery2
hadoop.hdfs.TestSafeModeWithStripedFileWithRandomECPolicy
hadoop.hdfs.tools.offlineImageViewer.TestOfflineImageViewer
hadoop.hdfs.TestQuota
hadoop.hdfs.TestLease
hadoop.hdfs.TestParallelRead
hadoop.hdfs.security.token.block.TestBlockToken
hadoop.hdfs.TestFileStatusWithDefaultECPolicy
hadoop.hdfs.TestSafeModeWithStripedFile
hadoop.hdfs.TestDFSRollback
hadoop.hdfs.server.balancer.TestBalancerWithEncryptedTransfer
hadoop.hdfs.TestLeaseRecoveryStriped
hadoop.hdfs.TestMaintenanceState
hadoop.hdfs.server.datanode.fsdataset.impl.TestFsVolumeList
hadoop.hdfs.TestFileAppend4
hadoop.hdfs.TestCrcCorruption
hadoop.hdfs.server.datanode.TestDnRespectsBlockReportSplitThreshold
hadoop.hdfs.TestReplaceDatanodeOnFailure
hadoop.hdfs.TestQuotaAllowOwner
Subsystem Report/Notes
Docker ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-4996/4/artifact/out/Dockerfile
GITHUB PR https://github.com/apache/hadoop/pull/4996
Optional Tests dupname asflicense codespell detsecrets xmllint compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle
uname Linux dd461af579d0 4.15.0-191-generic #202-Ubuntu SMP Thu Aug 4 01:49:29 UTC 2022 x86_64 x86_64 x86_64 GNU/Linux
Build tool maven
Personality dev-support/bin/hadoop.sh
git revision trunk / 8309ad3ae69e1ba1d744ccd85ba2dbc0bde5d0d3
Default Java Private Build-1.8.0_342-8u342-b07-0ubuntu1~20.04-b07
Multi-JDK versions /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.16+8-post-Ubuntu-0ubuntu120.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_342-8u342-b07-0ubuntu1~20.04-b07
Unreaped Processes Log https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-4996/4/artifact/out/patch-unit-hadoop-hdfs-project_hadoop-hdfs-reaper.txt
Test Results https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-4996/4/testReport/
Max. process+thread count 5371 (vs. ulimit of 5500)
modules C: hadoop-project hadoop-common-project/hadoop-common hadoop-hdfs-project/hadoop-hdfs-client hadoop-hdfs-project/hadoop-hdfs hadoop-hdfs-project/hadoop-hdfs-nfs hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager hadoop-mapreduce-project/hadoop-mapreduce-client hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs hadoop-hdfs-project/hadoop-hdfs-rbf hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core hadoop-yarn-project/hadoop-yarn/hadoop-yarn-csi U: .
Console output https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-4996/4/console
versions git=2.25.1 maven=3.6.3 spotbugs=4.2.2
Powered by Apache Yetus 0.14.0 https://yetus.apache.org

This message was automatically generated.

hadoop-yetus avatar Oct 21 '22 02:10 hadoop-yetus

:broken_heart: -1 overall

Vote Subsystem Runtime Logfile Comment
+0 :ok: reexec 0m 51s 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: detsecrets 0m 1s detect-secrets was not available.
+0 :ok: xmllint 0m 1s xmllint 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 3 new or modified test files.
_ trunk Compile Tests _
+0 :ok: mvndep 15m 30s Maven dependency ordering for branch
+1 :green_heart: mvninstall 25m 43s trunk passed
+1 :green_heart: compile 23m 16s trunk passed with JDK Ubuntu-11.0.16+8-post-Ubuntu-0ubuntu120.04
+1 :green_heart: compile 20m 56s trunk passed with JDK Private Build-1.8.0_342-8u342-b07-0ubuntu1~20.04-b07
+1 :green_heart: checkstyle 4m 7s trunk passed
+1 :green_heart: mvnsite 18m 37s trunk passed
+1 :green_heart: javadoc 15m 20s trunk passed with JDK Ubuntu-11.0.16+8-post-Ubuntu-0ubuntu120.04
+1 :green_heart: javadoc 14m 55s trunk passed with JDK Private Build-1.8.0_342-8u342-b07-0ubuntu1~20.04-b07
+0 :ok: spotbugs 1m 5s branch/hadoop-project no spotbugs output file (spotbugsXml.xml)
+1 :green_heart: shadedclient 21m 52s branch has no errors when building and testing our client artifacts.
-0 :warning: patch 22m 21s Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.
_ Patch Compile Tests _
+0 :ok: mvndep 0m 28s Maven dependency ordering for patch
+1 :green_heart: mvninstall 10m 50s the patch passed
+1 :green_heart: compile 22m 43s the patch passed with JDK Ubuntu-11.0.16+8-post-Ubuntu-0ubuntu120.04
-1 :x: javac 22m 43s /results-compile-javac-root-jdkUbuntu-11.0.16+8-post-Ubuntu-0ubuntu120.04.txt root-jdkUbuntu-11.0.16+8-post-Ubuntu-0ubuntu120.04 with JDK Ubuntu-11.0.16+8-post-Ubuntu-0ubuntu120.04 generated 1 new + 2821 unchanged - 1 fixed = 2822 total (was 2822)
+1 :green_heart: compile 20m 52s the patch passed with JDK Private Build-1.8.0_342-8u342-b07-0ubuntu1~20.04-b07
-1 :x: javac 20m 52s /results-compile-javac-root-jdkPrivateBuild-1.8.0_342-8u342-b07-0ubuntu1~20.04-b07.txt root-jdkPrivateBuild-1.8.0_342-8u342-b07-0ubuntu1~20.04-b07 with JDK Private Build-1.8.0_342-8u342-b07-0ubuntu1~20.04-b07 generated 1 new + 2614 unchanged - 1 fixed = 2615 total (was 2615)
+1 :green_heart: blanks 0m 0s The patch has no blanks issues.
-0 :warning: checkstyle 4m 6s /results-checkstyle-root.txt root: The patch generated 3 new + 273 unchanged - 5 fixed = 276 total (was 278)
+1 :green_heart: mvnsite 18m 57s the patch passed
+1 :green_heart: javadoc 16m 1s the patch passed with JDK Ubuntu-11.0.16+8-post-Ubuntu-0ubuntu120.04
+1 :green_heart: javadoc 15m 55s the patch passed with JDK Private Build-1.8.0_342-8u342-b07-0ubuntu1~20.04-b07
+0 :ok: spotbugs 0m 54s hadoop-project has no data from spotbugs
-1 :x: spotbugs 2m 53s /new-spotbugs-hadoop-common-project_hadoop-common.html hadoop-common-project/hadoop-common generated 2 new + 0 unchanged - 0 fixed = 2 total (was 0)
+1 :green_heart: shadedclient 22m 14s patch has no errors when building and testing our client artifacts.
_ Other Tests _
+1 :green_heart: unit 1m 3s hadoop-project in the patch passed.
+1 :green_heart: unit 18m 49s hadoop-common in the patch passed.
+1 :green_heart: unit 3m 15s hadoop-hdfs-client in the patch passed.
+1 :green_heart: unit 374m 22s hadoop-hdfs in the patch passed.
+1 :green_heart: unit 4m 1s hadoop-hdfs-nfs in the patch passed.
+1 :green_heart: unit 5m 47s hadoop-yarn-common in the patch passed.
+1 :green_heart: unit 4m 1s hadoop-yarn-server-common in the patch passed.
+1 :green_heart: unit 102m 46s hadoop-yarn-server-resourcemanager in the patch passed.
-1 :x: unit 27m 55s /patch-unit-hadoop-mapreduce-project_hadoop-mapreduce-client.txt hadoop-mapreduce-client in the patch failed.
-1 :x: unit 1m 27s /patch-unit-hadoop-mapreduce-project_hadoop-mapreduce-client_hadoop-mapreduce-client-hs.txt hadoop-mapreduce-client-hs in the patch failed.
-1 :x: unit 1m 35s /patch-unit-hadoop-hdfs-project_hadoop-hdfs-rbf.txt hadoop-hdfs-rbf in the patch failed.
+1 :green_heart: unit 22m 24s hadoop-yarn-services-core in the patch passed.
+1 :green_heart: unit 1m 36s hadoop-yarn-csi in the patch passed.
+1 :green_heart: asflicense 1m 38s The patch does not generate ASF License warnings.
930m 54s
Reason Tests
SpotBugs module:hadoop-common-project/hadoop-common
Sequence of calls to java.util.concurrent.ConcurrentHashMap may not be atomic in org.apache.hadoop.ipc.internal.ShadedProtobufHelper.getFixedByteString(String) At ShadedProtobufHelper.java:may not be atomic in org.apache.hadoop.ipc.internal.ShadedProtobufHelper.getFixedByteString(String) At ShadedProtobufHelper.java:[line 98]
Sequence of calls to java.util.concurrent.ConcurrentHashMap may not be atomic in org.apache.hadoop.ipc.internal.ShadedProtobufHelper.getFixedByteString(Text) At ShadedProtobufHelper.java:may not be atomic in org.apache.hadoop.ipc.internal.ShadedProtobufHelper.getFixedByteString(Text) At ShadedProtobufHelper.java:[line 84]
Subsystem Report/Notes
Docker ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-4996/6/artifact/out/Dockerfile
GITHUB PR https://github.com/apache/hadoop/pull/4996
Optional Tests dupname asflicense codespell detsecrets xmllint compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle
uname Linux a9356ec540fb 4.15.0-191-generic #202-Ubuntu SMP Thu Aug 4 01:49:29 UTC 2022 x86_64 x86_64 x86_64 GNU/Linux
Build tool maven
Personality dev-support/bin/hadoop.sh
git revision trunk / b10066e03c8f9d7e96ac0566a266454463b265ad
Default Java Private Build-1.8.0_342-8u342-b07-0ubuntu1~20.04-b07
Multi-JDK versions /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.16+8-post-Ubuntu-0ubuntu120.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_342-8u342-b07-0ubuntu1~20.04-b07
Test Results https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-4996/6/testReport/
Max. process+thread count 2950 (vs. ulimit of 5500)
modules C: hadoop-project hadoop-common-project/hadoop-common hadoop-hdfs-project/hadoop-hdfs-client hadoop-hdfs-project/hadoop-hdfs hadoop-hdfs-project/hadoop-hdfs-nfs hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager hadoop-mapreduce-project/hadoop-mapreduce-client hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs hadoop-hdfs-project/hadoop-hdfs-rbf hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core hadoop-yarn-project/hadoop-yarn/hadoop-yarn-csi U: .
Console output https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-4996/6/console
versions git=2.25.1 maven=3.6.3 spotbugs=4.2.2
Powered by Apache Yetus 0.14.0 https://yetus.apache.org

This message was automatically generated.

hadoop-yetus avatar Oct 25 '22 02:10 hadoop-yetus

:broken_heart: -1 overall

Vote Subsystem Runtime Logfile Comment
+0 :ok: reexec 1m 51s 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: detsecrets 0m 1s detect-secrets was not available.
+0 :ok: xmllint 0m 1s xmllint 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 3 new or modified test files.
_ trunk Compile Tests _
+0 :ok: mvndep 15m 59s Maven dependency ordering for branch
+1 :green_heart: mvninstall 26m 6s trunk passed
+1 :green_heart: compile 23m 18s trunk passed with JDK Ubuntu-11.0.16+8-post-Ubuntu-0ubuntu120.04
+1 :green_heart: compile 21m 21s trunk passed with JDK Private Build-1.8.0_342-8u342-b07-0ubuntu1~20.04-b07
+1 :green_heart: checkstyle 4m 17s trunk passed
+1 :green_heart: mvnsite 18m 52s trunk passed
+1 :green_heart: javadoc 14m 41s trunk passed with JDK Ubuntu-11.0.16+8-post-Ubuntu-0ubuntu120.04
+1 :green_heart: javadoc 15m 39s trunk passed with JDK Private Build-1.8.0_342-8u342-b07-0ubuntu1~20.04-b07
+0 :ok: spotbugs 1m 7s branch/hadoop-project no spotbugs output file (spotbugsXml.xml)
+1 :green_heart: shadedclient 23m 47s branch has no errors when building and testing our client artifacts.
-0 :warning: patch 24m 17s Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.
_ Patch Compile Tests _
+0 :ok: mvndep 0m 32s Maven dependency ordering for patch
-1 :x: mvninstall 0m 21s /patch-mvninstall-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-csi.txt hadoop-yarn-csi in the patch failed.
-1 :x: compile 20m 56s /patch-compile-root-jdkUbuntu-11.0.16+8-post-Ubuntu-0ubuntu120.04.txt root in the patch failed with JDK Ubuntu-11.0.16+8-post-Ubuntu-0ubuntu120.04.
-1 :x: javac 20m 56s /patch-compile-root-jdkUbuntu-11.0.16+8-post-Ubuntu-0ubuntu120.04.txt root in the patch failed with JDK Ubuntu-11.0.16+8-post-Ubuntu-0ubuntu120.04.
-1 :x: compile 18m 58s /patch-compile-root-jdkPrivateBuild-1.8.0_342-8u342-b07-0ubuntu1~20.04-b07.txt root in the patch failed with JDK Private Build-1.8.0_342-8u342-b07-0ubuntu1~20.04-b07.
-1 :x: javac 18m 58s /patch-compile-root-jdkPrivateBuild-1.8.0_342-8u342-b07-0ubuntu1~20.04-b07.txt root in the patch failed with JDK Private Build-1.8.0_342-8u342-b07-0ubuntu1~20.04-b07.
+1 :green_heart: blanks 0m 0s The patch has no blanks issues.
+1 :green_heart: checkstyle 4m 19s root: The patch generated 0 new + 273 unchanged - 5 fixed = 273 total (was 278)
-1 :x: mvnsite 0m 45s /patch-mvnsite-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-csi.txt hadoop-yarn-csi in the patch failed.
-1 :x: javadoc 0m 43s /patch-javadoc-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-csi-jdkUbuntu-11.0.16+8-post-Ubuntu-0ubuntu120.04.txt hadoop-yarn-csi in the patch failed with JDK Ubuntu-11.0.16+8-post-Ubuntu-0ubuntu120.04.
-1 :x: javadoc 0m 49s /patch-javadoc-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-csi-jdkPrivateBuild-1.8.0_342-8u342-b07-0ubuntu1~20.04-b07.txt hadoop-yarn-csi in the patch failed with JDK Private Build-1.8.0_342-8u342-b07-0ubuntu1~20.04-b07.
+0 :ok: spotbugs 0m 49s hadoop-project has no data from spotbugs
-1 :x: spotbugs 3m 35s /new-spotbugs-hadoop-common-project_hadoop-common.html hadoop-common-project/hadoop-common generated 2 new + 0 unchanged - 0 fixed = 2 total (was 0)
-1 :x: spotbugs 0m 51s /patch-spotbugs-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-csi.txt hadoop-yarn-csi in the patch failed.
+1 :green_heart: shadedclient 23m 21s patch has no errors when building and testing our client artifacts.
_ Other Tests _
+1 :green_heart: unit 0m 53s hadoop-project in the patch passed.
+1 :green_heart: unit 19m 26s hadoop-common in the patch passed.
+1 :green_heart: unit 3m 16s hadoop-hdfs-client in the patch passed.
-1 :x: unit 446m 40s /patch-unit-hadoop-hdfs-project_hadoop-hdfs.txt hadoop-hdfs in the patch passed.
+1 :green_heart: unit 4m 58s hadoop-hdfs-nfs in the patch passed.
-1 :x: unit 1m 33s /patch-unit-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-common.txt hadoop-yarn-common in the patch failed.
+1 :green_heart: unit 4m 26s hadoop-yarn-server-common in the patch passed.
-1 :x: unit 113m 27s /patch-unit-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-server_hadoop-yarn-server-resourcemanager.txt hadoop-yarn-server-resourcemanager in the patch passed.
-1 :x: unit 29m 0s /patch-unit-hadoop-mapreduce-project_hadoop-mapreduce-client.txt hadoop-mapreduce-client in the patch failed.
-1 :x: unit 1m 18s /patch-unit-hadoop-mapreduce-project_hadoop-mapreduce-client_hadoop-mapreduce-client-hs.txt hadoop-mapreduce-client-hs in the patch failed.
-1 :x: unit 1m 13s /patch-unit-hadoop-hdfs-project_hadoop-hdfs-rbf.txt hadoop-hdfs-rbf in the patch failed.
+1 :green_heart: unit 22m 32s hadoop-yarn-services-core in the patch passed.
-1 :x: unit 1m 5s /patch-unit-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-csi.txt hadoop-yarn-csi in the patch failed.
+1 :green_heart: asflicense 1m 36s The patch does not generate ASF License warnings.
1013m 5s
Reason Tests
SpotBugs module:hadoop-common-project/hadoop-common
Sequence of calls to java.util.concurrent.ConcurrentHashMap may not be atomic in org.apache.hadoop.ipc.internal.ShadedProtobufHelper.getFixedByteString(String) At ShadedProtobufHelper.java:may not be atomic in org.apache.hadoop.ipc.internal.ShadedProtobufHelper.getFixedByteString(String) At ShadedProtobufHelper.java:[line 98]
Sequence of calls to java.util.concurrent.ConcurrentHashMap may not be atomic in org.apache.hadoop.ipc.internal.ShadedProtobufHelper.getFixedByteString(Text) At ShadedProtobufHelper.java:may not be atomic in org.apache.hadoop.ipc.internal.ShadedProtobufHelper.getFixedByteString(Text) At ShadedProtobufHelper.java:[line 84]
Failed junit tests hadoop.hdfs.server.namenode.ha.TestSeveralNameNodes
hadoop.hdfs.server.namenode.ha.TestObserverNode
hadoop.yarn.server.resourcemanager.recovery.TestFSRMStateStore
Subsystem Report/Notes
Docker ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-4996/9/artifact/out/Dockerfile
GITHUB PR https://github.com/apache/hadoop/pull/4996
Optional Tests dupname asflicense codespell detsecrets xmllint compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle
uname Linux ce6cf05b6110 4.15.0-191-generic #202-Ubuntu SMP Thu Aug 4 01:49:29 UTC 2022 x86_64 x86_64 x86_64 GNU/Linux
Build tool maven
Personality dev-support/bin/hadoop.sh
git revision trunk / dad1ddb8da48833067704cd006c32c33eb4f8a4b
Default Java Private Build-1.8.0_342-8u342-b07-0ubuntu1~20.04-b07
Multi-JDK versions /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.16+8-post-Ubuntu-0ubuntu120.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_342-8u342-b07-0ubuntu1~20.04-b07
Test Results https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-4996/9/testReport/
Max. process+thread count 2852 (vs. ulimit of 5500)
modules C: hadoop-project hadoop-common-project/hadoop-common hadoop-hdfs-project/hadoop-hdfs-client hadoop-hdfs-project/hadoop-hdfs hadoop-hdfs-project/hadoop-hdfs-nfs hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager hadoop-mapreduce-project/hadoop-mapreduce-client hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs hadoop-hdfs-project/hadoop-hdfs-rbf hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core hadoop-yarn-project/hadoop-yarn/hadoop-yarn-csi U: .
Console output https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-4996/9/console
versions git=2.25.1 maven=3.6.3 spotbugs=4.2.2
Powered by Apache Yetus 0.14.0 https://yetus.apache.org

This message was automatically generated.

hadoop-yetus avatar Oct 27 '22 04:10 hadoop-yetus

:broken_heart: -1 overall

Vote Subsystem Runtime Logfile Comment
+0 :ok: reexec 1m 13s 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: detsecrets 0m 1s detect-secrets was not available.
+0 :ok: xmllint 0m 1s xmllint 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 3 new or modified test files.
_ trunk Compile Tests _
+0 :ok: mvndep 15m 43s Maven dependency ordering for branch
+1 :green_heart: mvninstall 28m 40s trunk passed
+1 :green_heart: compile 26m 8s trunk passed with JDK Ubuntu-11.0.16+8-post-Ubuntu-0ubuntu120.04
+1 :green_heart: compile 22m 50s trunk passed with JDK Private Build-1.8.0_342-8u342-b07-0ubuntu1~20.04-b07
+1 :green_heart: checkstyle 3m 57s trunk passed
+1 :green_heart: mvnsite 18m 14s trunk passed
+1 :green_heart: javadoc 16m 23s trunk passed with JDK Ubuntu-11.0.16+8-post-Ubuntu-0ubuntu120.04
+1 :green_heart: javadoc 15m 14s trunk passed with JDK Private Build-1.8.0_342-8u342-b07-0ubuntu1~20.04-b07
+0 :ok: spotbugs 1m 11s branch/hadoop-project no spotbugs output file (spotbugsXml.xml)
+1 :green_heart: shadedclient 24m 51s branch has no errors when building and testing our client artifacts.
-0 :warning: patch 25m 19s Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.
_ Patch Compile Tests _
+0 :ok: mvndep 0m 33s Maven dependency ordering for patch
+1 :green_heart: mvninstall 11m 27s the patch passed
+1 :green_heart: compile 25m 37s the patch passed with JDK Ubuntu-11.0.16+8-post-Ubuntu-0ubuntu120.04
-1 :x: javac 25m 37s /results-compile-javac-root-jdkUbuntu-11.0.16+8-post-Ubuntu-0ubuntu120.04.txt root-jdkUbuntu-11.0.16+8-post-Ubuntu-0ubuntu120.04 with JDK Ubuntu-11.0.16+8-post-Ubuntu-0ubuntu120.04 generated 1 new + 2824 unchanged - 1 fixed = 2825 total (was 2825)
+1 :green_heart: compile 22m 8s the patch passed with JDK Private Build-1.8.0_342-8u342-b07-0ubuntu1~20.04-b07
-1 :x: javac 22m 8s /results-compile-javac-root-jdkPrivateBuild-1.8.0_342-8u342-b07-0ubuntu1~20.04-b07.txt root-jdkPrivateBuild-1.8.0_342-8u342-b07-0ubuntu1~20.04-b07 with JDK Private Build-1.8.0_342-8u342-b07-0ubuntu1~20.04-b07 generated 1 new + 2616 unchanged - 1 fixed = 2617 total (was 2617)
+1 :green_heart: blanks 0m 0s The patch has no blanks issues.
+1 :green_heart: checkstyle 4m 17s root: The patch generated 0 new + 273 unchanged - 5 fixed = 273 total (was 278)
+1 :green_heart: mvnsite 18m 9s the patch passed
+1 :green_heart: javadoc 16m 14s the patch passed with JDK Ubuntu-11.0.16+8-post-Ubuntu-0ubuntu120.04
+1 :green_heart: javadoc 15m 59s the patch passed with JDK Private Build-1.8.0_342-8u342-b07-0ubuntu1~20.04-b07
+0 :ok: spotbugs 0m 54s hadoop-project has no data from spotbugs
-1 :x: spotbugs 3m 8s /new-spotbugs-hadoop-common-project_hadoop-common.html hadoop-common-project/hadoop-common generated 2 new + 0 unchanged - 0 fixed = 2 total (was 0)
+1 :green_heart: shadedclient 24m 5s patch has no errors when building and testing our client artifacts.
_ Other Tests _
+1 :green_heart: unit 0m 55s hadoop-project in the patch passed.
+1 :green_heart: unit 19m 45s hadoop-common in the patch passed.
+1 :green_heart: unit 3m 11s hadoop-hdfs-client in the patch passed.
-1 :x: unit 444m 14s /patch-unit-hadoop-hdfs-project_hadoop-hdfs.txt hadoop-hdfs in the patch passed.
+1 :green_heart: unit 4m 26s hadoop-hdfs-nfs in the patch passed.
-1 :x: unit 1m 26s /patch-unit-hadoop-yarn-project_hadoop-yarn_hadoop-yarn-common.txt hadoop-yarn-common in the patch failed.
+1 :green_heart: unit 4m 9s hadoop-yarn-server-common in the patch passed.
+1 :green_heart: unit 104m 24s hadoop-yarn-server-resourcemanager in the patch passed.
-1 :x: unit 27m 51s /patch-unit-hadoop-mapreduce-project_hadoop-mapreduce-client.txt hadoop-mapreduce-client in the patch failed.
-1 :x: unit 1m 19s /patch-unit-hadoop-mapreduce-project_hadoop-mapreduce-client_hadoop-mapreduce-client-hs.txt hadoop-mapreduce-client-hs in the patch failed.
-1 :x: unit 1m 26s /patch-unit-hadoop-hdfs-project_hadoop-hdfs-rbf.txt hadoop-hdfs-rbf in the patch failed.
+1 :green_heart: unit 22m 40s hadoop-yarn-services-core in the patch passed.
+1 :green_heart: unit 1m 38s hadoop-yarn-csi in the patch passed.
+1 :green_heart: asflicense 1m 38s The patch does not generate ASF License warnings.
1021m 39s
Reason Tests
SpotBugs module:hadoop-common-project/hadoop-common
Sequence of calls to java.util.concurrent.ConcurrentHashMap may not be atomic in org.apache.hadoop.ipc.internal.ShadedProtobufHelper.getFixedByteString(String) At ShadedProtobufHelper.java:may not be atomic in org.apache.hadoop.ipc.internal.ShadedProtobufHelper.getFixedByteString(String) At ShadedProtobufHelper.java:[line 98]
Sequence of calls to java.util.concurrent.ConcurrentHashMap may not be atomic in org.apache.hadoop.ipc.internal.ShadedProtobufHelper.getFixedByteString(Text) At ShadedProtobufHelper.java:may not be atomic in org.apache.hadoop.ipc.internal.ShadedProtobufHelper.getFixedByteString(Text) At ShadedProtobufHelper.java:[line 84]
Failed junit tests hadoop.hdfs.server.diskbalancer.command.TestDiskBalancerCommand
Subsystem Report/Notes
Docker ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-4996/10/artifact/out/Dockerfile
GITHUB PR https://github.com/apache/hadoop/pull/4996
Optional Tests dupname asflicense codespell detsecrets xmllint compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle
uname Linux 3f322bacee52 4.15.0-191-generic #202-Ubuntu SMP Thu Aug 4 01:49:29 UTC 2022 x86_64 x86_64 x86_64 GNU/Linux
Build tool maven
Personality dev-support/bin/hadoop.sh
git revision trunk / 79fd97ab302790a96eb839d2623025f411624212
Default Java Private Build-1.8.0_342-8u342-b07-0ubuntu1~20.04-b07
Multi-JDK versions /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.16+8-post-Ubuntu-0ubuntu120.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_342-8u342-b07-0ubuntu1~20.04-b07
Test Results https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-4996/10/testReport/
Max. process+thread count 3152 (vs. ulimit of 5500)
modules C: hadoop-project hadoop-common-project/hadoop-common hadoop-hdfs-project/hadoop-hdfs-client hadoop-hdfs-project/hadoop-hdfs hadoop-hdfs-project/hadoop-hdfs-nfs hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager hadoop-mapreduce-project/hadoop-mapreduce-client hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs hadoop-hdfs-project/hadoop-hdfs-rbf hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core hadoop-yarn-project/hadoop-yarn/hadoop-yarn-csi U: .
Console output https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-4996/10/console
versions git=2.25.1 maven=3.6.3 spotbugs=4.2.2
Powered by Apache Yetus 0.14.0 https://yetus.apache.org

This message was automatically generated.

hadoop-yetus avatar Oct 27 '22 06:10 hadoop-yetus

...not targeting 3.3.5 for this; not a blocker

steveloughran avatar Nov 01 '22 18:11 steveloughran

spotbugs still unhappy. will have to explicitly exclude

steveloughran avatar Mar 08 '23 15:03 steveloughran

:broken_heart: -1 overall

Vote Subsystem Runtime Logfile Comment
+0 :ok: reexec 0m 43s Docker mode activated.
_ Prechecks _
+1 :green_heart: dupname 0m 2s No case conflicting files found.
+0 :ok: codespell 0m 2s codespell was not available.
+0 :ok: detsecrets 0m 2s detect-secrets was not available.
+0 :ok: xmllint 0m 2s xmllint 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 3 new or modified test files.
_ trunk Compile Tests _
+0 :ok: mvndep 18m 25s Maven dependency ordering for branch
+1 :green_heart: mvninstall 37m 6s trunk passed
+1 :green_heart: compile 18m 47s trunk passed with JDK Ubuntu-11.0.19+7-post-Ubuntu-0ubuntu120.04.1
+1 :green_heart: compile 17m 22s trunk passed with JDK Private Build-1.8.0_362-8u372-ga~us1-0ubuntu1~20.04-b09
+1 :green_heart: checkstyle 4m 37s trunk passed
+1 :green_heart: mvnsite 15m 2s trunk passed
+1 :green_heart: javadoc 13m 16s trunk passed with JDK Ubuntu-11.0.19+7-post-Ubuntu-0ubuntu120.04.1
+1 :green_heart: javadoc 13m 18s trunk passed with JDK Private Build-1.8.0_362-8u372-ga~us1-0ubuntu1~20.04-b09
+0 :ok: spotbugs 0m 52s branch/hadoop-project no spotbugs output file (spotbugsXml.xml)
+1 :green_heart: shadedclient 38m 1s branch has no errors when building and testing our client artifacts.
-0 :warning: patch 38m 30s Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.
_ Patch Compile Tests _
+0 :ok: mvndep 0m 34s Maven dependency ordering for patch
-1 :x: mvninstall 0m 32s /patch-mvninstall-hadoop-hdfs-project_hadoop-hdfs-rbf.txt hadoop-hdfs-rbf in the patch failed.
-1 :x: compile 11m 41s /patch-compile-root-jdkUbuntu-11.0.19+7-post-Ubuntu-0ubuntu120.04.1.txt root in the patch failed with JDK Ubuntu-11.0.19+7-post-Ubuntu-0ubuntu120.04.1.
-1 :x: javac 11m 41s /patch-compile-root-jdkUbuntu-11.0.19+7-post-Ubuntu-0ubuntu120.04.1.txt root in the patch failed with JDK Ubuntu-11.0.19+7-post-Ubuntu-0ubuntu120.04.1.
-1 :x: compile 10m 10s /patch-compile-root-jdkPrivateBuild-1.8.0_362-8u372-ga~us1-0ubuntu1~20.04-b09.txt root in the patch failed with JDK Private Build-1.8.0_362-8u372-ga~us1-0ubuntu1~20.04-b09.
-1 :x: javac 10m 10s /patch-compile-root-jdkPrivateBuild-1.8.0_362-8u372-ga~us1-0ubuntu1~20.04-b09.txt root in the patch failed with JDK Private Build-1.8.0_362-8u372-ga~us1-0ubuntu1~20.04-b09.
+1 :green_heart: blanks 0m 0s The patch has no blanks issues.
+1 :green_heart: checkstyle 4m 21s root: The patch generated 0 new + 272 unchanged - 5 fixed = 272 total (was 277)
-1 :x: mvnsite 0m 39s /patch-mvnsite-hadoop-hdfs-project_hadoop-hdfs-rbf.txt hadoop-hdfs-rbf in the patch failed.
+1 :green_heart: javadoc 9m 41s the patch passed with JDK Ubuntu-11.0.19+7-post-Ubuntu-0ubuntu120.04.1
+1 :green_heart: javadoc 8m 57s the patch passed with JDK Private Build-1.8.0_362-8u372-ga~us1-0ubuntu1~20.04-b09
+0 :ok: spotbugs 0m 23s hadoop-project has no data from spotbugs
-1 :x: spotbugs 0m 37s /patch-spotbugs-hadoop-hdfs-project_hadoop-hdfs-rbf.txt hadoop-hdfs-rbf in the patch failed.
+1 :green_heart: shadedclient 37m 45s patch has no errors when building and testing our client artifacts.
_ Other Tests _
+1 :green_heart: unit 0m 25s hadoop-project in the patch passed.
+1 :green_heart: unit 19m 33s hadoop-common in the patch passed.
+1 :green_heart: unit 2m 33s hadoop-hdfs-client in the patch passed.
+1 :green_heart: unit 214m 42s hadoop-hdfs in the patch passed.
+1 :green_heart: unit 3m 8s hadoop-hdfs-nfs in the patch passed.
+1 :green_heart: unit 5m 53s hadoop-yarn-common in the patch passed.
+1 :green_heart: unit 3m 39s hadoop-yarn-server-common in the patch passed.
+1 :green_heart: unit 99m 31s hadoop-yarn-server-resourcemanager in the patch passed.
-1 :x: unit 177m 27s /patch-unit-hadoop-mapreduce-project_hadoop-mapreduce-client.txt hadoop-mapreduce-client in the patch passed.
+1 :green_heart: unit 4m 37s hadoop-mapreduce-client-hs in the patch passed.
-1 :x: unit 1m 1s /patch-unit-hadoop-hdfs-project_hadoop-hdfs-rbf.txt hadoop-hdfs-rbf in the patch failed.
+1 :green_heart: unit 21m 36s hadoop-yarn-services-core in the patch passed.
+1 :green_heart: unit 1m 5s hadoop-yarn-csi in the patch passed.
+1 :green_heart: asflicense 1m 5s The patch does not generate ASF License warnings.
894m 10s
Reason Tests
Failed junit tests hadoop.mapreduce.v2.TestUberAM
hadoop.mapreduce.v2.TestMRJobsWithProfiler
hadoop.mapreduce.v2.TestMRJobs
Subsystem Report/Notes
Docker ClientAPI=1.43 ServerAPI=1.43 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-4996/15/artifact/out/Dockerfile
GITHUB PR https://github.com/apache/hadoop/pull/4996
Optional Tests dupname asflicense codespell detsecrets xmllint compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle
uname Linux df3b96528f0a 4.15.0-212-generic #223-Ubuntu SMP Tue May 23 13:09:22 UTC 2023 x86_64 x86_64 x86_64 GNU/Linux
Build tool maven
Personality dev-support/bin/hadoop.sh
git revision trunk / ca82091343af29c68ba9a6bf98d524e534d89361
Default Java Private Build-1.8.0_362-8u372-ga~us1-0ubuntu1~20.04-b09
Multi-JDK versions /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.19+7-post-Ubuntu-0ubuntu120.04.1 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_362-8u372-ga~us1-0ubuntu1~20.04-b09
Test Results https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-4996/15/testReport/
Max. process+thread count 3433 (vs. ulimit of 5500)
modules C: hadoop-project hadoop-common-project/hadoop-common hadoop-hdfs-project/hadoop-hdfs-client hadoop-hdfs-project/hadoop-hdfs hadoop-hdfs-project/hadoop-hdfs-nfs hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager hadoop-mapreduce-project/hadoop-mapreduce-client hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs hadoop-hdfs-project/hadoop-hdfs-rbf hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core hadoop-yarn-project/hadoop-yarn/hadoop-yarn-csi U: .
Console output https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-4996/15/console
versions git=2.25.1 maven=3.6.3 spotbugs=4.2.2
Powered by Apache Yetus 0.14.0 https://yetus.apache.org

This message was automatically generated.

hadoop-yetus avatar Jul 18 '23 04:07 hadoop-yetus

test Failures are all in TestMRJobsWithProfiler, covered exactly in MAPREDUCE-7436 and therefore unrelated.

steveloughran avatar Jul 18 '23 13:07 steveloughran

rbf compile failure is very much related

rbf: [ERROR] COMPILATION ERROR : 
[INFO] -------------------------------------------------------------
[ERROR] /home/jenkins/jenkins-home/workspace/hadoop-multibranch_PR-4996/ubuntu-focal/src/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/protocolPB/RouterAdminProtocolTranslatorPB.java:[174,29] cannot find symbol
  symbol:   variable ProtobufHelper
  location: class org.apache.hadoop.hdfs.protocolPB.RouterAdminProtocolTranslatorPB
[INFO] 1 error
[INFO] ---------------------------------

steveloughran avatar Jul 18 '23 13:07 steveloughran

:broken_heart: -1 overall

Vote Subsystem Runtime Logfile Comment
+0 :ok: reexec 0m 54s 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: detsecrets 0m 1s detect-secrets was not available.
+0 :ok: xmllint 0m 1s xmllint 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 3 new or modified test files.
_ trunk Compile Tests _
+0 :ok: mvndep 15m 6s Maven dependency ordering for branch
+1 :green_heart: mvninstall 32m 7s trunk passed
+1 :green_heart: compile 17m 34s trunk passed with JDK Ubuntu-11.0.20+8-post-Ubuntu-1ubuntu120.04
+1 :green_heart: compile 16m 3s trunk passed with JDK Private Build-1.8.0_382-8u382-ga-1~20.04.1-b05
+1 :green_heart: checkstyle 4m 23s trunk passed
+1 :green_heart: mvnsite 15m 28s trunk passed
+1 :green_heart: javadoc 13m 54s trunk passed with JDK Ubuntu-11.0.20+8-post-Ubuntu-1ubuntu120.04
+1 :green_heart: javadoc 13m 23s trunk passed with JDK Private Build-1.8.0_382-8u382-ga-1~20.04.1-b05
+0 :ok: spotbugs 0m 51s branch/hadoop-project no spotbugs output file (spotbugsXml.xml)
+1 :green_heart: shadedclient 35m 5s branch has no errors when building and testing our client artifacts.
-0 :warning: patch 35m 35s Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.
_ Patch Compile Tests _
+0 :ok: mvndep 0m 33s Maven dependency ordering for patch
+1 :green_heart: mvninstall 8m 58s the patch passed
+1 :green_heart: compile 16m 44s the patch passed with JDK Ubuntu-11.0.20+8-post-Ubuntu-1ubuntu120.04
+1 :green_heart: javac 16m 44s the patch passed
+1 :green_heart: compile 16m 28s the patch passed with JDK Private Build-1.8.0_382-8u382-ga-1~20.04.1-b05
+1 :green_heart: javac 16m 28s the patch passed
+1 :green_heart: blanks 0m 0s The patch has no blanks issues.
+1 :green_heart: checkstyle 4m 28s root: The patch generated 0 new + 272 unchanged - 5 fixed = 272 total (was 277)
+1 :green_heart: mvnsite 15m 31s the patch passed
+1 :green_heart: javadoc 13m 43s the patch passed with JDK Ubuntu-11.0.20+8-post-Ubuntu-1ubuntu120.04
+1 :green_heart: javadoc 13m 24s the patch passed with JDK Private Build-1.8.0_382-8u382-ga-1~20.04.1-b05
+0 :ok: spotbugs 0m 44s hadoop-project has no data from spotbugs
+1 :green_heart: shadedclient 35m 3s patch has no errors when building and testing our client artifacts.
_ Other Tests _
+1 :green_heart: unit 0m 46s hadoop-project in the patch passed.
+1 :green_heart: unit 19m 18s hadoop-common in the patch passed.
+1 :green_heart: unit 2m 55s hadoop-hdfs-client in the patch passed.
-1 :x: unit 250m 14s /patch-unit-hadoop-hdfs-project_hadoop-hdfs.txt hadoop-hdfs in the patch passed.
+1 :green_heart: unit 3m 33s hadoop-hdfs-nfs in the patch passed.
+1 :green_heart: unit 6m 16s hadoop-yarn-common in the patch passed.
+1 :green_heart: unit 4m 7s hadoop-yarn-server-common in the patch passed.
+1 :green_heart: unit 105m 23s hadoop-yarn-server-resourcemanager in the patch passed.
-1 :x: unit 177m 52s /patch-unit-hadoop-mapreduce-project_hadoop-mapreduce-client.txt hadoop-mapreduce-client in the patch passed.
+1 :green_heart: unit 4m 57s hadoop-mapreduce-client-hs in the patch passed.
+1 :green_heart: unit 23m 23s hadoop-hdfs-rbf in the patch passed.
+1 :green_heart: unit 22m 10s hadoop-yarn-services-core in the patch passed.
+1 :green_heart: unit 1m 28s hadoop-yarn-csi in the patch passed.
+1 :green_heart: asflicense 1m 38s The patch does not generate ASF License warnings.
976m 48s
Reason Tests
Failed junit tests hadoop.hdfs.server.datanode.TestDirectoryScanner
hadoop.mapreduce.v2.TestMRJobsWithProfiler
hadoop.mapreduce.v2.TestMRJobs
hadoop.mapreduce.v2.TestUberAM
Subsystem Report/Notes
Docker ClientAPI=1.43 ServerAPI=1.43 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-4996/16/artifact/out/Dockerfile
GITHUB PR https://github.com/apache/hadoop/pull/4996
Optional Tests dupname asflicense codespell detsecrets xmllint compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle
uname Linux 7895f6d96b24 4.15.0-212-generic #223-Ubuntu SMP Tue May 23 13:09:22 UTC 2023 x86_64 x86_64 x86_64 GNU/Linux
Build tool maven
Personality dev-support/bin/hadoop.sh
git revision trunk / 6ce5309126cbcb947e1b178ee2f2e8f45efeebe1
Default Java Private Build-1.8.0_382-8u382-ga-1~20.04.1-b05
Multi-JDK versions /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.20+8-post-Ubuntu-1ubuntu120.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_382-8u382-ga-1~20.04.1-b05
Test Results https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-4996/16/testReport/
Max. process+thread count 3155 (vs. ulimit of 5500)
modules C: hadoop-project hadoop-common-project/hadoop-common hadoop-hdfs-project/hadoop-hdfs-client hadoop-hdfs-project/hadoop-hdfs hadoop-hdfs-project/hadoop-hdfs-nfs hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager hadoop-mapreduce-project/hadoop-mapreduce-client hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs hadoop-hdfs-project/hadoop-hdfs-rbf hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core hadoop-yarn-project/hadoop-yarn/hadoop-yarn-csi U: .
Console output https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-4996/16/console
versions git=2.25.1 maven=3.6.3 spotbugs=4.2.2
Powered by Apache Yetus 0.14.0 https://yetus.apache.org

This message was automatically generated.

hadoop-yetus avatar Aug 15 '23 11:08 hadoop-yetus

the failing tests are the usual "tests that fail"

steveloughran avatar Aug 15 '23 18:08 steveloughran

tests are unrelated.

this pr is ready to go in, followed by one switching protobuf 2.5 to provided

steveloughran avatar Aug 18 '23 15:08 steveloughran

@pjfanning thanks -reviewed all the javadocs and corrected any issues I could see, made sure everything had a doc and used MUST NOT USE in the class javadoc.

steveloughran avatar Aug 21 '23 15:08 steveloughran

:broken_heart: -1 overall

Vote Subsystem Runtime Logfile Comment
+0 :ok: reexec 0m 53s Docker mode activated.
_ Prechecks _
+1 :green_heart: dupname 0m 2s No case conflicting files found.
+0 :ok: codespell 0m 2s codespell was not available.
+0 :ok: detsecrets 0m 2s detect-secrets was not available.
+0 :ok: xmllint 0m 2s xmllint 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 3 new or modified test files.
_ trunk Compile Tests _
+0 :ok: mvndep 15m 4s Maven dependency ordering for branch
+1 :green_heart: mvninstall 59m 17s trunk passed
+1 :green_heart: compile 17m 39s trunk passed with JDK Ubuntu-11.0.20+8-post-Ubuntu-1ubuntu120.04
+1 :green_heart: compile 16m 9s trunk passed with JDK Private Build-1.8.0_382-8u382-ga-1~20.04.1-b05
+1 :green_heart: checkstyle 4m 25s trunk passed
+1 :green_heart: mvnsite 15m 20s trunk passed
+1 :green_heart: javadoc 13m 56s trunk passed with JDK Ubuntu-11.0.20+8-post-Ubuntu-1ubuntu120.04
+1 :green_heart: javadoc 13m 22s trunk passed with JDK Private Build-1.8.0_382-8u382-ga-1~20.04.1-b05
+0 :ok: spotbugs 0m 51s branch/hadoop-project no spotbugs output file (spotbugsXml.xml)
+1 :green_heart: shadedclient 34m 55s branch has no errors when building and testing our client artifacts.
-0 :warning: patch 35m 24s Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.
_ Patch Compile Tests _
+0 :ok: mvndep 0m 33s Maven dependency ordering for patch
+1 :green_heart: mvninstall 9m 0s the patch passed
+1 :green_heart: compile 16m 32s the patch passed with JDK Ubuntu-11.0.20+8-post-Ubuntu-1ubuntu120.04
+1 :green_heart: javac 16m 32s the patch passed
+1 :green_heart: compile 16m 2s the patch passed with JDK Private Build-1.8.0_382-8u382-ga-1~20.04.1-b05
+1 :green_heart: javac 16m 2s the patch passed
+1 :green_heart: blanks 0m 0s The patch has no blanks issues.
+1 :green_heart: checkstyle 4m 22s root: The patch generated 0 new + 272 unchanged - 5 fixed = 272 total (was 277)
+1 :green_heart: mvnsite 15m 21s the patch passed
+1 :green_heart: javadoc 13m 56s the patch passed with JDK Ubuntu-11.0.20+8-post-Ubuntu-1ubuntu120.04
+1 :green_heart: javadoc 13m 19s the patch passed with JDK Private Build-1.8.0_382-8u382-ga-1~20.04.1-b05
+0 :ok: spotbugs 0m 44s hadoop-project has no data from spotbugs
+1 :green_heart: shadedclient 35m 19s patch has no errors when building and testing our client artifacts.
_ Other Tests _
+1 :green_heart: unit 0m 46s hadoop-project in the patch passed.
+1 :green_heart: unit 19m 21s hadoop-common in the patch passed.
+1 :green_heart: unit 2m 53s hadoop-hdfs-client in the patch passed.
+1 :green_heart: unit 248m 1s hadoop-hdfs in the patch passed.
+1 :green_heart: unit 3m 46s hadoop-hdfs-nfs in the patch passed.
+1 :green_heart: unit 6m 18s hadoop-yarn-common in the patch passed.
+1 :green_heart: unit 4m 20s hadoop-yarn-server-common in the patch passed.
+1 :green_heart: unit 119m 47s hadoop-yarn-server-resourcemanager in the patch passed.
-1 :x: unit 176m 50s /patch-unit-hadoop-mapreduce-project_hadoop-mapreduce-client.txt hadoop-mapreduce-client in the patch passed.
+1 :green_heart: unit 4m 59s hadoop-mapreduce-client-hs in the patch passed.
+1 :green_heart: unit 23m 25s hadoop-hdfs-rbf in the patch passed.
+1 :green_heart: unit 22m 8s hadoop-yarn-services-core in the patch passed.
+1 :green_heart: unit 1m 31s hadoop-yarn-csi in the patch passed.
+1 :green_heart: asflicense 1m 37s The patch does not generate ASF License warnings.
1014m 16s
Reason Tests
Failed junit tests hadoop.mapreduce.v2.TestMRJobsWithProfiler
hadoop.mapreduce.v2.TestMRJobs
hadoop.mapreduce.v2.TestUberAM
Subsystem Report/Notes
Docker ClientAPI=1.43 ServerAPI=1.43 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-4996/17/artifact/out/Dockerfile
GITHUB PR https://github.com/apache/hadoop/pull/4996
Optional Tests dupname asflicense codespell detsecrets xmllint compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle
uname Linux 9019b70f446b 4.15.0-212-generic #223-Ubuntu SMP Tue May 23 13:09:22 UTC 2023 x86_64 x86_64 x86_64 GNU/Linux
Build tool maven
Personality dev-support/bin/hadoop.sh
git revision trunk / 74b2a60b4a9f11238014cb72608316df2e22247f
Default Java Private Build-1.8.0_382-8u382-ga-1~20.04.1-b05
Multi-JDK versions /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.20+8-post-Ubuntu-1ubuntu120.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_382-8u382-ga-1~20.04.1-b05
Test Results https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-4996/17/testReport/
Max. process+thread count 3152 (vs. ulimit of 5500)
modules C: hadoop-project hadoop-common-project/hadoop-common hadoop-hdfs-project/hadoop-hdfs-client hadoop-hdfs-project/hadoop-hdfs hadoop-hdfs-project/hadoop-hdfs-nfs hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager hadoop-mapreduce-project/hadoop-mapreduce-client hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs hadoop-hdfs-project/hadoop-hdfs-rbf hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core hadoop-yarn-project/hadoop-yarn/hadoop-yarn-csi U: .
Console output https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-4996/17/console
versions git=2.25.1 maven=3.6.3 spotbugs=4.2.2
Powered by Apache Yetus 0.14.0 https://yetus.apache.org

This message was automatically generated.

hadoop-yetus avatar Aug 22 '23 08:08 hadoop-yetus

Ohh, I forgot some: Additional to previous ones I think you should update the Building.txt file (https://github.com/apache/hadoop/blob/trunk/BUILDING.txt) and mention about this. Second question: Do you want to keep providing scope optional or should we wrap this up under a profile? Third: We still expect protobuf-2.5.0 to be packaged even if the scope is provided, right?

ayushtkn avatar Aug 25 '23 19:08 ayushtkn

Do you want to keep providing scope optional or should we wrap this up under a profile?

  1. the scope in this PR doesn't actually change protobuf 2.5 being exported
  2. i had a goal of a followup pr which did that, something isolated so easy to revert
  3. but yes, a protobuf-2.5.scope variable would let people switch from provided to compile on the command line.

now, if we make it a switch, should i do it in this pr or that followup I was thinking about

steveloughran avatar Aug 26 '23 12:08 steveloughran

anything is good with me, can do in the followup pr as well

ayushtkn avatar Aug 26 '23 14:08 ayushtkn

we can't cut an unshaded protobuf of some form without RPC not linking, so hbase/hive/ozone are in trouble here.

HBase branch-3 and trunk are fine, we use our own shaded protobuf everywhere. The major version increment allowed it.

For HBase branch-2, all the 2.x releases, and Apache Phoenix, the com.google.protobuf API classes are part of the RPC side of the Coprocessor API so indeed these would all be in trouble in theory. Internally even in 2.x releases we use protobuf 3 for actual RPC. The PB 2.5 classes are only required due to interface compatibility constraints and therefore its on us to ensure the dependencies are imported. We should not (and do not) depend on Hadoop providing an explicit export of PB 2.5 anyway.

When I made a similar change where I work in an internal fork of Hadoop 3 I removed the legacy RPC classes (ProtobufRpcEngine and such) so linkage without any PB 2.5 imports at the Hadoop level in any scope (compile, test, provided, whatever) was not an issue. I couldn't find any users of the legacy RPC classes in any of our downstream code. The only users were some Hadoop test classes, which could either be dropped as moot or modified to use ProtobufRpcEngine2. This is a much more isolated ecosystem than the public Hadoop one so YMMV, Spark is there but not Hive, and so on. Switching PB 2.5 to 'provided' would be a good careful initial step.

apurtell avatar Aug 28 '23 18:08 apurtell

updated pr tries to address reviews, including building.txt details.

regarding @apurtell's comment about static vs qualified import of getRemoteException() I started to convert, but got fed up with the needless repetition. So upgraded the RPC invocations to java 8 language level (!!) by adding a new operation ipc() to take a lambda expression and do the invocation, translating the result

  public static <T> T ipc(IpcCall<T> call) throws IOException {
    try {
      return call.call();
    } catch (ServiceException e) {
      throw getRemoteException(e);
    }
  }

this lets us have far simpler invocations in the code

    status = ipc(() -> rpcProxy.getServiceStatus(NULL_CONTROLLER,
        GET_SERVICE_STATUS_REQ));

I've done this for hadoop-common; if all are happy then I will do for the rest of the modules

steveloughran avatar Sep 11 '23 10:09 steveloughran

:broken_heart: -1 overall

Vote Subsystem Runtime Logfile Comment
+0 :ok: reexec 0m 53s 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: detsecrets 0m 1s detect-secrets was not available.
+0 :ok: xmllint 0m 1s xmllint 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 3 new or modified test files.
_ trunk Compile Tests _
+0 :ok: mvndep 14m 29s Maven dependency ordering for branch
+1 :green_heart: mvninstall 31m 18s trunk passed
+1 :green_heart: compile 17m 33s trunk passed with JDK Ubuntu-11.0.20+8-post-Ubuntu-1ubuntu120.04
+1 :green_heart: compile 15m 58s trunk passed with JDK Private Build-1.8.0_382-8u382-ga-1~20.04.1-b05
+1 :green_heart: checkstyle 4m 28s trunk passed
+1 :green_heart: mvnsite 19m 58s trunk passed
+1 :green_heart: javadoc 8m 42s trunk passed with JDK Ubuntu-11.0.20+8-post-Ubuntu-1ubuntu120.04
+1 :green_heart: javadoc 7m 39s trunk passed with JDK Private Build-1.8.0_382-8u382-ga-1~20.04.1-b05
+0 :ok: spotbugs 0m 24s branch/hadoop-project no spotbugs output file (spotbugsXml.xml)
+1 :green_heart: shadedclient 61m 45s branch has no errors when building and testing our client artifacts.
-0 :warning: patch 62m 11s Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.
_ Patch Compile Tests _
+0 :ok: mvndep 0m 33s Maven dependency ordering for patch
+1 :green_heart: mvninstall 38m 42s the patch passed
+1 :green_heart: compile 16m 43s the patch passed with JDK Ubuntu-11.0.20+8-post-Ubuntu-1ubuntu120.04
+1 :green_heart: javac 16m 43s the patch passed
+1 :green_heart: compile 16m 3s the patch passed with JDK Private Build-1.8.0_382-8u382-ga-1~20.04.1-b05
+1 :green_heart: javac 16m 3s the patch passed
+1 :green_heart: blanks 0m 0s The patch has no blanks issues.
-0 :warning: checkstyle 4m 41s /results-checkstyle-root.txt root: The patch generated 6 new + 269 unchanged - 8 fixed = 275 total (was 277)
+1 :green_heart: mvnsite 15m 39s the patch passed
-1 :x: javadoc 1m 28s /patch-javadoc-root-jdkUbuntu-11.0.20+8-post-Ubuntu-1ubuntu120.04.txt root in the patch failed with JDK Ubuntu-11.0.20+8-post-Ubuntu-1ubuntu120.04.
+1 :green_heart: javadoc 8m 4s the patch passed with JDK Private Build-1.8.0_382-8u382-ga-1~20.04.1-b05
+0 :ok: spotbugs 0m 20s hadoop-project has no data from spotbugs
+1 :green_heart: shadedclient 66m 15s patch has no errors when building and testing our client artifacts.
_ Other Tests _
-1 :x: unit 819m 58s /patch-unit-root.txt root in the patch passed.
+1 :green_heart: asflicense 1m 56s The patch does not generate ASF License warnings.
1229m 9s
Reason Tests
Failed junit tests hadoop.yarn.server.resourcemanager.reservation.TestCapacityOverTimePolicy
hadoop.mapreduce.v2.TestMRJobsWithProfiler
hadoop.mapreduce.v2.TestMRJobs
hadoop.mapreduce.v2.TestUberAM
Subsystem Report/Notes
Docker ClientAPI=1.43 ServerAPI=1.43 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-4996/18/artifact/out/Dockerfile
GITHUB PR https://github.com/apache/hadoop/pull/4996
Optional Tests dupname asflicense codespell detsecrets xmllint compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle
uname Linux 10fc48a083be 4.15.0-212-generic #223-Ubuntu SMP Tue May 23 13:09:22 UTC 2023 x86_64 x86_64 x86_64 GNU/Linux
Build tool maven
Personality dev-support/bin/hadoop.sh
git revision trunk / 8f3ffd14f3f4a6232e35a468cb5c0ce2822bbbfc
Default Java Private Build-1.8.0_382-8u382-ga-1~20.04.1-b05
Multi-JDK versions /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.20+8-post-Ubuntu-1ubuntu120.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_382-8u382-ga-1~20.04.1-b05
Test Results https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-4996/18/testReport/
Max. process+thread count 3152 (vs. ulimit of 5500)
modules C: hadoop-project hadoop-common-project/hadoop-common hadoop-hdfs-project/hadoop-hdfs-client hadoop-hdfs-project/hadoop-hdfs hadoop-hdfs-project/hadoop-hdfs-nfs hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager hadoop-mapreduce-project/hadoop-mapreduce-client hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs hadoop-hdfs-project/hadoop-hdfs-rbf hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core hadoop-yarn-project/hadoop-yarn/hadoop-yarn-csi . U: .
Console output https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-4996/18/console
versions git=2.25.1 maven=3.6.3 spotbugs=4.2.2
Powered by Apache Yetus 0.14.0 https://yetus.apache.org

This message was automatically generated.

hadoop-yetus avatar Sep 12 '23 07:09 hadoop-yetus

@ayushtkn thanks. i will do it everywhere; its the same as the s3a invoker stuff and it is more elegant, especially for anyone writing new rpc stuff

steveloughran avatar Sep 14 '23 17:09 steveloughran

:broken_heart: -1 overall

Vote Subsystem Runtime Logfile Comment
+0 :ok: reexec 0m 0s Docker mode activated.
-1 :x: patch 0m 23s https://github.com/apache/hadoop/pull/4996 does not apply to trunk. Rebase required? Wrong Branch? See https://cwiki.apache.org/confluence/display/HADOOP/How+To+Contribute for help.
Subsystem Report/Notes
GITHUB PR https://github.com/apache/hadoop/pull/4996
Console output https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-4996/19/console
versions git=2.17.1
Powered by Apache Yetus 0.14.0 https://yetus.apache.org

This message was automatically generated.

hadoop-yetus avatar Oct 02 '23 14:10 hadoop-yetus

rebased for a retest, then will merge.

steveloughran avatar Oct 06 '23 10:10 steveloughran

:broken_heart: -1 overall

Vote Subsystem Runtime Logfile Comment
+0 :ok: reexec 0m 53s Docker mode activated.
_ Prechecks _
+1 :green_heart: dupname 0m 2s No case conflicting files found.
+0 :ok: codespell 0m 2s codespell was not available.
+0 :ok: detsecrets 0m 2s detect-secrets was not available.
+0 :ok: xmllint 0m 2s xmllint 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 3 new or modified test files.
_ trunk Compile Tests _
+0 :ok: mvndep 14m 9s Maven dependency ordering for branch
+1 :green_heart: mvninstall 31m 57s trunk passed
+1 :green_heart: compile 17m 22s trunk passed with JDK Ubuntu-11.0.20+8-post-Ubuntu-1ubuntu120.04
+1 :green_heart: compile 16m 2s trunk passed with JDK Private Build-1.8.0_382-8u382-ga-1~20.04.1-b05
+1 :green_heart: checkstyle 4m 25s trunk passed
+1 :green_heart: mvnsite 19m 20s trunk passed
+1 :green_heart: javadoc 8m 55s trunk passed with JDK Ubuntu-11.0.20+8-post-Ubuntu-1ubuntu120.04
+1 :green_heart: javadoc 7m 32s trunk passed with JDK Private Build-1.8.0_382-8u382-ga-1~20.04.1-b05
+0 :ok: spotbugs 0m 23s branch/hadoop-project no spotbugs output file (spotbugsXml.xml)
+1 :green_heart: shadedclient 62m 25s branch has no errors when building and testing our client artifacts.
_ Patch Compile Tests _
+0 :ok: mvndep 0m 33s Maven dependency ordering for patch
+1 :green_heart: mvninstall 38m 11s the patch passed
+1 :green_heart: compile 16m 50s the patch passed with JDK Ubuntu-11.0.20+8-post-Ubuntu-1ubuntu120.04
+1 :green_heart: javac 16m 50s the patch passed
+1 :green_heart: compile 16m 9s the patch passed with JDK Private Build-1.8.0_382-8u382-ga-1~20.04.1-b05
+1 :green_heart: javac 16m 9s the patch passed
+1 :green_heart: blanks 0m 0s The patch has no blanks issues.
-0 :warning: checkstyle 4m 19s /results-checkstyle-root.txt root: The patch generated 7 new + 269 unchanged - 8 fixed = 276 total (was 277)
+1 :green_heart: mvnsite 15m 3s the patch passed
-1 :x: javadoc 1m 26s /patch-javadoc-root-jdkUbuntu-11.0.20+8-post-Ubuntu-1ubuntu120.04.txt root in the patch failed with JDK Ubuntu-11.0.20+8-post-Ubuntu-1ubuntu120.04.
+1 :green_heart: javadoc 7m 36s the patch passed with JDK Private Build-1.8.0_382-8u382-ga-1~20.04.1-b05
+0 :ok: spotbugs 0m 23s hadoop-project has no data from spotbugs
+1 :green_heart: shadedclient 61m 54s patch has no errors when building and testing our client artifacts.
_ Other Tests _
-1 :x: unit 787m 19s /patch-unit-root.txt root in the patch passed.
+1 :green_heart: asflicense 1m 58s The patch does not generate ASF License warnings.
1190m 50s
Reason Tests
Failed junit tests hadoop.yarn.server.resourcemanager.reservation.TestCapacityOverTimePolicy
Subsystem Report/Notes
Docker ClientAPI=1.43 ServerAPI=1.43 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-4996/20/artifact/out/Dockerfile
GITHUB PR https://github.com/apache/hadoop/pull/4996
Optional Tests dupname asflicense codespell detsecrets xmllint compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle
uname Linux c983f319195e 4.15.0-213-generic #224-Ubuntu SMP Mon Jun 19 13:30:12 UTC 2023 x86_64 x86_64 x86_64 GNU/Linux
Build tool maven
Personality dev-support/bin/hadoop.sh
git revision trunk / a70ceffc923398eb71933707b020f14ac51267df
Default Java Private Build-1.8.0_382-8u382-ga-1~20.04.1-b05
Multi-JDK versions /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.20+8-post-Ubuntu-1ubuntu120.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_382-8u382-ga-1~20.04.1-b05
Test Results https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-4996/20/testReport/
Max. process+thread count 3162 (vs. ulimit of 5500)
modules C: hadoop-project hadoop-common-project/hadoop-common hadoop-hdfs-project/hadoop-hdfs-client hadoop-hdfs-project/hadoop-hdfs hadoop-hdfs-project/hadoop-hdfs-nfs hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager hadoop-mapreduce-project/hadoop-mapreduce-client hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs hadoop-hdfs-project/hadoop-hdfs-rbf hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core hadoop-yarn-project/hadoop-yarn/hadoop-yarn-csi . U: .
Console output https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-4996/20/console
versions git=2.25.1 maven=3.6.3 spotbugs=4.2.2
Powered by Apache Yetus 0.14.0 https://yetus.apache.org

This message was automatically generated.

hadoop-yetus avatar Oct 07 '23 06:10 hadoop-yetus

test failure seems unrelated. style

./hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/protocolPB/GenericRefreshProtocolClientSideTranslatorPB.java:62:      .setIdentifier(identifier): '.' has incorrect indentation level 6, expected level should be 8. [Indentation]
./hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/protocolPB/GenericRefreshProtocolClientSideTranslatorPB.java:63:      .addAllArgs(argList): '.' has incorrect indentation level 6, expected level should be 8. [Indentation]
./hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/protocolPB/GenericRefreshProtocolClientSideTranslatorPB.java:64:      .build();: '.' has incorrect indentation level 6, expected level should be 8. [Indentation]
./hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/protocolPB/GenericRefreshProtocolClientSideTranslatorPB.java:66:    GenericRefreshResponseCollectionProto resp = ipc(() -> rpcProxy.refresh(NULL_CONTROLLER, request));: Line is longer than 100 characters (found 103). [LineLength]
./hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tools/protocolPB/GetUserMappingsProtocolClientSideTranslatorPB.java:27:import org.apache.hadoop.ipc.internal.ShadedProtobufHelper;:8: Unused import - org.apache.hadoop.ipc.internal.ShadedProtobufHelper. [UnusedImports]
./hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tools/protocolPB/GetUserMappingsProtocolClientSideTranslatorPB.java:33:import org.apache.hadoop.thirdparty.protobuf.ServiceException;:8: Unused import - org.apache.hadoop.thirdparty.protobuf.ServiceException. [UnusedImports]
./hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolTranslatorPB.java:243:      ipc(() -> rpcProxy.shutdownDatanode(NULL_CONTROLLER, request));: 'method def' child has incorrect indentation level 6, expected level should be 4. [Indentation]

javadocs

[ERROR] in your comments are valid in HTML5, and remove the -html4 option.
[ERROR] /home/jenkins/jenkins-home/workspace/hadoop-multibranch_PR-4996/ubuntu-focal/src/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufHelper.java:39: error: unknown tag: Deprecated:
[ERROR]  * @Deprecated: hadoop code MUST use {@link ShadedProtobufHelper}.

maybe its the :

steveloughran avatar Oct 09 '23 15:10 steveloughran

:broken_heart: -1 overall

Vote Subsystem Runtime Logfile Comment
+0 :ok: reexec 0m 55s 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: detsecrets 0m 1s detect-secrets was not available.
+0 :ok: xmllint 0m 1s xmllint 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 3 new or modified test files.
_ trunk Compile Tests _
+0 :ok: mvndep 14m 31s Maven dependency ordering for branch
+1 :green_heart: mvninstall 31m 40s trunk passed
+1 :green_heart: compile 16m 32s trunk passed with JDK Ubuntu-11.0.20+8-post-Ubuntu-1ubuntu120.04
+1 :green_heart: compile 15m 25s trunk passed with JDK Private Build-1.8.0_382-8u382-ga-1~20.04.1-b05
+1 :green_heart: checkstyle 4m 36s trunk passed
+1 :green_heart: mvnsite 18m 53s trunk passed
+1 :green_heart: javadoc 8m 42s trunk passed with JDK Ubuntu-11.0.20+8-post-Ubuntu-1ubuntu120.04
+1 :green_heart: javadoc 7m 32s trunk passed with JDK Private Build-1.8.0_382-8u382-ga-1~20.04.1-b05
+0 :ok: spotbugs 0m 23s branch/hadoop-project no spotbugs output file (spotbugsXml.xml)
+1 :green_heart: shadedclient 64m 9s branch has no errors when building and testing our client artifacts.
_ Patch Compile Tests _
+0 :ok: mvndep 0m 33s Maven dependency ordering for patch
+1 :green_heart: mvninstall 38m 11s the patch passed
+1 :green_heart: compile 16m 24s the patch passed with JDK Ubuntu-11.0.20+8-post-Ubuntu-1ubuntu120.04
+1 :green_heart: javac 16m 24s the patch passed
+1 :green_heart: compile 16m 12s the patch passed with JDK Private Build-1.8.0_382-8u382-ga-1~20.04.1-b05
+1 :green_heart: javac 16m 12s the patch passed
+1 :green_heart: blanks 0m 0s The patch has no blanks issues.
+1 :green_heart: checkstyle 4m 58s root: The patch generated 0 new + 269 unchanged - 8 fixed = 269 total (was 277)
+1 :green_heart: mvnsite 15m 22s the patch passed
+1 :green_heart: javadoc 8m 33s the patch passed with JDK Ubuntu-11.0.20+8-post-Ubuntu-1ubuntu120.04
+1 :green_heart: javadoc 7m 29s the patch passed with JDK Private Build-1.8.0_382-8u382-ga-1~20.04.1-b05
+0 :ok: spotbugs 0m 23s hadoop-project has no data from spotbugs
+1 :green_heart: shadedclient 61m 38s patch has no errors when building and testing our client artifacts.
_ Other Tests _
-1 :x: unit 800m 7s /patch-unit-root.txt root in the patch passed.
+1 :green_heart: asflicense 1m 53s The patch does not generate ASF License warnings.
1210m 5s
Reason Tests
Failed junit tests hadoop.hdfs.TestRollingUpgrade
Subsystem Report/Notes
Docker ClientAPI=1.43 ServerAPI=1.43 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-4996/21/artifact/out/Dockerfile
GITHUB PR https://github.com/apache/hadoop/pull/4996
Optional Tests dupname asflicense codespell detsecrets xmllint compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle
uname Linux d5886f541f8c 4.15.0-213-generic #224-Ubuntu SMP Mon Jun 19 13:30:12 UTC 2023 x86_64 x86_64 x86_64 GNU/Linux
Build tool maven
Personality dev-support/bin/hadoop.sh
git revision trunk / 06b4b12d9d56d4b7d942ac641696b1e715265e96
Default Java Private Build-1.8.0_382-8u382-ga-1~20.04.1-b05
Multi-JDK versions /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.20+8-post-Ubuntu-1ubuntu120.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_382-8u382-ga-1~20.04.1-b05
Test Results https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-4996/21/testReport/
Max. process+thread count 3071 (vs. ulimit of 5500)
modules C: hadoop-project hadoop-common-project/hadoop-common hadoop-hdfs-project/hadoop-hdfs-client hadoop-hdfs-project/hadoop-hdfs hadoop-hdfs-project/hadoop-hdfs-nfs hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager hadoop-mapreduce-project/hadoop-mapreduce-client hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs hadoop-hdfs-project/hadoop-hdfs-rbf hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core hadoop-yarn-project/hadoop-yarn/hadoop-yarn-csi . U: .
Console output https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-4996/21/console
versions git=2.25.1 maven=3.6.3 spotbugs=4.2.2
Powered by Apache Yetus 0.14.0 https://yetus.apache.org

This message was automatically generated.

hadoop-yetus avatar Oct 10 '23 15:10 hadoop-yetus

dfs upgrade test is flaky (and has a history), created HDFS-17224 to cover the new failure and verified it works standalone

steveloughran avatar Oct 13 '23 12:10 steveloughran