Uploaded image for project: 'HBase'
  1. HBase
  2. HBASE-28522

UNASSIGN proc indefinitely stuck on dead rs

Details

    • Reviewed

    Description

      One scenario we noticed in production -

      we had DisableTableProc and SCP almost triggered at similar time

      2024-03-16 17:59:23,014 INFO [PEWorker-11] procedure.DisableTableProcedure -
      Set <TABLE_NAME> to state=DISABLING

      2024-03-16 17:59:15,243 INFO [PEWorker-26] procedure.ServerCrashProcedure -
      Start pid=21592440, state=RUNNABLE:SERVER_CRASH_START, locked=true; ServerCrashProcedure
      <regionserver>, splitWal=true, meta=false

      DisabeTableProc creates unassign procs, and at this time ASSIGNs of SCP is not completed

      2024-03-16 17:59:23,003 DEBUG [PEWorker-40] procedure2.ProcedureExecutor - LOCK_EVENT_WAIT pid=21594220, ppid=21592440, state=RUNNABLE:REGION_STATE_TRANSITION_GET_ASSIGN_CANDIDATE; TransitRegionStateProcedure table=<TABLE_NAME>, region=<regionhash>, ASSIGN

      UNASSIGN created by DisableTableProc is stuck on the dead regionserver and we had to manually bypass unassign of DisableTableProc and then do ASSIGN.

      If we can break the loop for UNASSIGN procedure to not retry if there is scp for that server, we do not need manual intervention?, at least the DisableTableProc can go to a rollback state?

      Attachments

        1. timeline.jpg
          84 kB
          Prathyusha

        Issue Links

          Activity

            zhangduo Duo Zhang added a comment -

            The flow by design is SCP will interrupt the TRSP to assign the region first, and then unassign it.

            Bypassing the unassign TRSP may cause data loss, as disabling a table does not always mean we want to drop it, it could be enabled later…

            zhangduo Duo Zhang added a comment - The flow by design is SCP will interrupt the TRSP to assign the region first, and then unassign it. Bypassing the unassign TRSP may cause data loss, as disabling a table does not always mean we want to drop it, it could be enabled later…

            I think the problem here might be that:- TRSP, the child of DisabeTableProc took the lock and SCP wasn't able to proceed further. DisableTableProc is also unable to complete as UNASSIGN is not possible on dead RS, thus creating a deadlock. prathyu6 can confirm it.

            zhangduo does SCP also get the lock? make the TRSP submit the lock? 

            umesh9414 Umesh Kumar Kumawat added a comment - I think the problem here might be that:- TRSP, the child of DisabeTableProc took the lock and SCP wasn't able to proceed further. DisableTableProc is also unable to complete as UNASSIGN is not possible on dead RS, thus creating a deadlock. prathyu6 can confirm it. zhangduo does SCP also get the lock? make the TRSP submit the lock? 
            prathyu6 Prathyusha added a comment - - edited

            >The flow by design is SCP will interrupt the TRSP to assign the region first, and then unassign it.

            True, from my understanding this code path should take care of it
            SCP#assingRegions

            if (regionNode.getProcedure() != null) {

            LOG.info("{} found RIT {}; {}", this, regionNode.getProcedure(), regionNode);

            regionNode.getProcedure().serverCrashed(env, regionNode, getServerName(),

            !retainAssignment);

            continue;

            }

            if (

            env.getMasterServices().getTableStateManager().isTableState(regionNode.getTable(),

            TableState.State.DISABLING)

            ) {

            // We need to change the state here otherwise the TRSP scheduled by DTP will try to

            // close the region from a dead server and will never succeed. Please see HBASE-23636

            // for more details.

            env.getAssignmentManager().regionClosedAbnormally(regionNode);

            LOG.info("{} found table disabling for region {}, set it state to ABNORMALLY_CLOSED.",

            this, regionNode);

            continue;

            }

            if (

            env.getMasterServices().getTableStateManager().isTableState(regionNode.getTable(),

            TableState.State.DISABLED)

            ) {

            // This should not happen, table disabled but has regions on server.

            LOG.warn("Found table disabled for region {}, procDetails: {}", regionNode, this);

            continue;

            }

            TransitRegionStateProcedure proc =

            TransitRegionStateProcedure.assign(env, region, !retainAssignment, null);

            regionNode.setProcedure(proc);
            addChildProcedure(proc);

             

            ---------------------------------------------------------------------------

            but we did not see "found RIT" log lines and SCP was triggered a bit before DisableTableProc set the table state to DISABLING.
            So it has set the ASSIGN proc in regionNode, before DisableTableProc has triggered forceCreateUnssignProcedure and this essentially again is overriding the current proc of regionNode (which should be the child assign of TRSP)

            if (regionNode.getProcedure() != null) {

            regionNode.unsetProcedure(regionNode.getProcedure());

            }

            return regionNode.setProcedure(TransitRegionStateProcedure.unassign(getProcedureEnvironment(),

            regionNode.getRegionInfo()));

             ---------------------------------------------------------------------------

            Now the Assign proc of SCP also was waiting on the shared Table lock, but DisableTableProc must have taken the table exclusive lock blocking ASSIGN of SCP.

            2024-03-16 17:59:23,003 DEBUG [PEWorker-40] procedure2.ProcedureExecutor - LOCK_EVENT_WAIT pid=21594220, ppid=21592440, state=RUNNABLE:REGION_STATE_TRANSITION_GET_ASSIGN_CANDIDATE; TransitRegionStateProcedure table=<TABLE_NAME>, region=<regionhash>, ASSIGN

            It looks like if the SCP was triggered a bit later, it would have interrupted current child UNASSIGN of DisableTableProc.

            zhangduo umesh9414 

            prathyu6 Prathyusha added a comment - - edited >The flow by design is SCP will interrupt the TRSP to assign the region first, and then unassign it. True, from my understanding this code path should take care of it SCP#assingRegions if ( regionNode .getProcedure() != null ) { LOG .info( "{} found RIT {}; {}" , this , regionNode .getProcedure(), regionNode ); regionNode .getProcedure(). serverCrashed( env , regionNode , getServerName(), ! retainAssignment ) ; continue ; } if ( env .getMasterServices().getTableStateManager().isTableState( regionNode .getTable(), TableState.State. DISABLING ) ) { // We need to change the state here otherwise the TRSP scheduled by DTP will try to // close the region from a dead server and will never succeed. Please see HBASE-23636 // for more details. env .getAssignmentManager().regionClosedAbnormally( regionNode ); LOG .info( "{} found table disabling for region {}, set it state to ABNORMALLY_CLOSED." , this , regionNode ); continue ; } if ( env .getMasterServices().getTableStateManager().isTableState( regionNode .getTable(), TableState.State. DISABLED ) ) { // This should not happen, table disabled but has regions on server. LOG .warn( "Found table disabled for region {}, procDetails: {}" , regionNode , this ); continue ; } TransitRegionStateProcedure proc = TransitRegionStateProcedure. assign ( env , region , ! retainAssignment , null ); regionNode .setProcedure( proc ); addChildProcedure( proc );   --------------------------------------------------------------------------- but we did not see "found RIT" log lines and SCP was triggered a bit before DisableTableProc set the table state to DISABLING. So it has set the ASSIGN proc in regionNode, before DisableTableProc has triggered forceCreateUnssignProcedure  and this essentially again is overriding the current proc of regionNode (which should be the child assign of TRSP) if ( regionNode .getProcedure() != null ) { regionNode .unsetProcedure( regionNode .getProcedure()); } return regionNode .setProcedure(TransitRegionStateProcedure. unassign (getProcedureEnvironment(), regionNode .getRegionInfo()));  --------------------------------------------------------------------------- Now the Assign proc of SCP also was waiting on the shared Table lock, but DisableTableProc must have taken the table exclusive lock blocking ASSIGN of SCP. 2024-03-16 17:59:23,003 DEBUG  [PEWorker-40]  procedure2.ProcedureExecutor - LOCK_EVENT_WAIT pid=21594220, ppid=21592440, state=RUNNABLE:REGION_STATE_TRANSITION_GET_ASSIGN_CANDIDATE; TransitRegionStateProcedure table=<TABLE_NAME>, region=<regionhash>, ASSIGN It looks like if the SCP was triggered a bit later, it would have interrupted current child UNASSIGN of DisableTableProc. zhangduo umesh9414  
            vjasani Viraj Jasani added a comment -
            if (
              env.getMasterServices().getTableStateManager().isTableState(regionNode.getTable(),
                TableState.State.DISABLING)
            ) {
              // We need to change the state here otherwise the TRSP scheduled by DTP will try to
              // close the region from a dead server and will never succeed. Please see HBASE-23636
              // for more details.
              env.getAssignmentManager().regionClosedAbnormally(regionNode);
              LOG.info("{} found table disabling for region {}, set it state to ABNORMALLY_CLOSED.",
                this, regionNode);
              continue;
            } 

            zhangduo why don't we mark the region state to CLOSED here, given that DTP is a special case where TRSP is used to only unassign and close the regions?

            vjasani Viraj Jasani added a comment - if ( env.getMasterServices().getTableStateManager().isTableState(regionNode.getTable(), TableState.State.DISABLING) ) { // We need to change the state here otherwise the TRSP scheduled by DTP will try to // close the region from a dead server and will never succeed. Please see HBASE-23636 // for more details. env.getAssignmentManager().regionClosedAbnormally(regionNode); LOG.info( "{} found table disabling for region {}, set it state to ABNORMALLY_CLOSED." , this , regionNode); continue ; } zhangduo why don't we mark the region state to CLOSED here, given that DTP is a special case where TRSP is used to only unassign and close the regions?
            zhangduo Duo Zhang added a comment -

            Because we can not make sure it is closed normally...

            The region server may crash before finishing unassign the region right?

            zhangduo Duo Zhang added a comment - Because we can not make sure it is closed normally... The region server may crash before finishing unassign the region right?
            zhangduo Duo Zhang added a comment -

            After checking the code, I think a possible race is that, in SCP, we have schedule a TRSP to assign the region, but at the same time, DisableTableProcedure has unset the procedure and scheduled a new one, but actually, the new TRSP can not be executed because the target server is already dead, so the unassign will fail and wait for SCP to interrupte it, but at the same time, SCP is waiting the old TRSP(scheduled by SCP) to finish, so it has no change to interrupte the new TRSP(and even if it can execute, in the current logic it will just finish itself without interrupting any other TRSPs).

            I think the root cause is here, in forceCreateUnssignProcedure

                regionNode.lock();
                try {
                  if (regionNode.isInState(State.OFFLINE, State.CLOSED, State.SPLIT)) {
                    return null;
                  }
                  // in general, a split parent should be in CLOSED or SPLIT state, but anyway, let's check it
                  // here for safety
                  if (regionNode.getRegionInfo().isSplit()) {
                    LOG.warn("{} is a split parent but not in CLOSED or SPLIT state", regionNode);
                    return null;
                  }
                  // As in DisableTableProcedure or ModifyTableProcedure, we will hold the xlock for table, so
                  // we can make sure that this procedure has not been executed yet, as TRSP will hold the
                  // shared lock for table all the time. So here we will unset it and when it is actually
                  // executed, it will find that the attach procedure is not itself and quit immediately.
                  if (regionNode.getProcedure() != null) {
                    regionNode.unsetProcedure(regionNode.getProcedure());
                  }
                  return regionNode.setProcedure(TransitRegionStateProcedure.unassign(getProcedureEnvironment(),
                    regionNode.getRegionInfo()));
                } finally {
                  regionNode.unlock();
                }
            

            We should reuse the same TRSP, instead of unsetting it and scheduling a new one.

            Let me think if this is possible.

            zhangduo Duo Zhang added a comment - After checking the code, I think a possible race is that, in SCP, we have schedule a TRSP to assign the region, but at the same time, DisableTableProcedure has unset the procedure and scheduled a new one, but actually, the new TRSP can not be executed because the target server is already dead, so the unassign will fail and wait for SCP to interrupte it, but at the same time, SCP is waiting the old TRSP(scheduled by SCP) to finish, so it has no change to interrupte the new TRSP(and even if it can execute, in the current logic it will just finish itself without interrupting any other TRSPs). I think the root cause is here, in forceCreateUnssignProcedure regionNode.lock(); try { if (regionNode.isInState(State.OFFLINE, State.CLOSED, State.SPLIT)) { return null ; } // in general, a split parent should be in CLOSED or SPLIT state, but anyway, let's check it // here for safety if (regionNode.getRegionInfo().isSplit()) { LOG.warn( "{} is a split parent but not in CLOSED or SPLIT state" , regionNode); return null ; } // As in DisableTableProcedure or ModifyTableProcedure, we will hold the xlock for table, so // we can make sure that this procedure has not been executed yet, as TRSP will hold the // shared lock for table all the time. So here we will unset it and when it is actually // executed, it will find that the attach procedure is not itself and quit immediately. if (regionNode.getProcedure() != null ) { regionNode.unsetProcedure(regionNode.getProcedure()); } return regionNode.setProcedure(TransitRegionStateProcedure.unassign(getProcedureEnvironment(), regionNode.getRegionInfo())); } finally { regionNode.unlock(); } We should reuse the same TRSP, instead of unsetting it and scheduling a new one. Let me think if this is possible.
            zhangduo Duo Zhang added a comment -

            Oh, reuse seems impossible, as the old TRSP can not be executed since DisableTableProcedure will hold the xlock of this table all the time...

            zhangduo Duo Zhang added a comment - Oh, reuse seems impossible, as the old TRSP can not be executed since DisableTableProcedure will hold the xlock of this table all the time...
            zhangduo Duo Zhang added a comment -

            ModifyTableProcedure may have the same problem when reducing region replica count, and the implementation is totally wrong as ModifyTableProcedure does not hold exclusive lock all the time...

            Let file an issue to fix it too.

            The fix is DTP is more difficult, as for DTP, we want it to hold the exclusive lock all the time so it will not skew with other table procedures...

            zhangduo Duo Zhang added a comment - ModifyTableProcedure may have the same problem when reducing region replica count, and the implementation is totally wrong as ModifyTableProcedure does not hold exclusive lock all the time... Let file an issue to fix it too. The fix is DTP is more difficult, as for DTP, we want it to hold the exclusive lock all the time so it will not skew with other table procedures...
            prathyu6 Prathyusha added a comment - - edited

            zhangduo yes, this is exactly the condition which I was trying to describe in my comment above (sorry if I was unclear a bit),  here is the below sequence of events happend, ending in a state of stuck procedures and bypass was the only way out.
            fyi apurtell vjasani 

            prathyu6 Prathyusha added a comment - - edited zhangduo yes, this is exactly the condition which I was trying to describe in my comment above (sorry if I was unclear a bit),  here is the below sequence of events happend, ending in a state of stuck procedures and bypass was the only way out. fyi apurtell vjasani  
            prathyu6 Prathyusha added a comment -

            The problem is currently SCP interrupts only procedures which are already set in RegionStateNodes while the for loop is in progress
            this will need to do more than that, because that doesn’t fully capture the set of all procedures that need to be interrupted.
            apurtell vjasani zhangduo 

            And once above happens, we had to bypass the stuck UNASSIGN proc on dead regionserver, but now the table state is DISABLING, mark it DISABLED manually. But then table still has couple of regions in OPEN state! And then we had to drop the table manually as well.
            So once this hits, lot of manual steps required to bring it back to the original state. You think we can add something to make it better?

            prathyu6 Prathyusha added a comment - The problem is currently SCP interrupts only procedures which are already set in RegionStateNodes while the for loop is in progress this will need to do more than that, because that doesn’t fully capture the set of all procedures that need to be interrupted. apurtell vjasani zhangduo   And once above happens, we had to bypass the stuck UNASSIGN proc on dead regionserver, but now the table state is DISABLING, mark it DISABLED manually. But then table still has couple of regions in OPEN state! And then we had to drop the table manually as well. So once this hits, lot of manual steps required to bring it back to the original state. You think we can add something to make it better?
            zhangduo Duo Zhang added a comment -

            The PR for HBASE-28582 is ready, I also created a test for it to show that it will wait for RIT to finish before continue. PTAL.

            In general, we can use the same solution for DisableTableProcedure, but we need to change holdLock from true to false, which makes me a bit uncomfortable.

            I will try to see if we do something like draining before starting to schedule TRSP in DisableTableProcedure, so we can still keep holdLock to true in later processing to simplify the logic. If not, let's change to use the same solution in HBASE-28582, i.e, introduce a special CloseTableRegionsProcedure, to close all regions for a table.

            Thanks.

            zhangduo Duo Zhang added a comment - The PR for HBASE-28582 is ready, I also created a test for it to show that it will wait for RIT to finish before continue. PTAL. In general, we can use the same solution for DisableTableProcedure, but we need to change holdLock from true to false, which makes me a bit uncomfortable. I will try to see if we do something like draining before starting to schedule TRSP in DisableTableProcedure, so we can still keep holdLock to true in later processing to simplify the logic. If not, let's change to use the same solution in HBASE-28582 , i.e, introduce a special CloseTableRegionsProcedure, to close all regions for a table. Thanks.
            prathyu6 Prathyusha added a comment - - edited

            zhangduo Even if we introduce a procedure like CloseTableRegionsProcedure in HBASE-28582 here, even though we put a logic to wait for the current rit(instead of creating a new child UNASSIGN directly), every TRSP(which just tried to start execute) will be blocked on trying to get the shared lock on Table (DTP holding exclusive lock) so they wont finish right? 
            Or you mean if we go via this approach
            >If not, let's change to use the same solution in HBASE-28582, i.e, introduce a special
            >CloseTableRegionsProcedure, to close all regions for a table.
            we have the holdLock as false for Table?

            An orthogonal thought - can we somehow add them(current RIT TRSPs) also as child procs to this? so that they can get the shared lock to table? cause CloseTableRegionsProcedure is anyway waiting on them to finish. 
            Or if not child procs, another field like dependent procedures and those also have access to shared lock of the resources it holds

            prathyu6 Prathyusha added a comment - - edited zhangduo Even if we introduce a procedure like CloseTableRegionsProcedure in HBASE-28582 here, even though we put a logic to wait for the current rit(instead of creating a new child UNASSIGN directly), every TRSP(which just tried to start execute) will be blocked on trying to get the shared lock on Table (DTP holding exclusive lock) so they wont finish right?  Or you mean if we go via this approach >If not, let's change to use the same solution in  HBASE-28582 , i.e, introduce a special >CloseTableRegionsProcedure, to close all regions for a table. we have the holdLock as false for Table? An orthogonal thought - can we somehow add them(current RIT TRSPs) also as child procs to this? so that they can get the shared lock to table? cause CloseTableRegionsProcedure is anyway waiting on them to finish.  Or if not child procs, another field like dependent procedures and those also have access to shared lock of the resources it holds
            zhangduo Duo Zhang added a comment -

            When DisableTableProcedure is scheduled, actually we do not need to schedule any TRSPs at all, as finally all the regions will be offlined.

            We do have special logic in SCP to not schedule TRSP for disabling tables but there are still races which could cause dead lock.

            Please give me some more time for thinking this.

            If we change the holdLock from true to false, I'm afraid we may introduce skew between CloseTableProcedure and other procedures like ModifyTableProcedure, which may cause more unexpected behavior...

            Or we may introduce another mechanism to make sure that there is only one table procedure could be executed for a table at the same time.

            Thanks.

            zhangduo Duo Zhang added a comment - When DisableTableProcedure is scheduled, actually we do not need to schedule any TRSPs at all, as finally all the regions will be offlined. We do have special logic in SCP to not schedule TRSP for disabling tables but there are still races which could cause dead lock. Please give me some more time for thinking this. If we change the holdLock from true to false, I'm afraid we may introduce skew between CloseTableProcedure and other procedures like ModifyTableProcedure, which may cause more unexpected behavior... Or we may introduce another mechanism to make sure that there is only one table procedure could be executed for a table at the same time. Thanks.
            zhangduo Duo Zhang added a comment -

            I've opened a PR with a possible fix, PTAL.

            Let me think how to write a UT here...

            zhangduo Duo Zhang added a comment - I've opened a PR with a possible fix, PTAL. Let me think how to write a UT here...
            zhangduo Duo Zhang added a comment -

            I've added a UT, and verified that on master branch it will hang forever.

            PTAL. prathyu6vjasani

            Thanks.

            zhangduo Duo Zhang added a comment - I've added a UT, and verified that on master branch it will hang forever. PTAL. prathyu6 vjasani Thanks.
            comnetwork chenglei added a comment -

            Can we acquire the table shared lock before checking DISABLED or DISABLING state of the table in ServerCrashProcedure.assignRegions to avoid the race ?

            comnetwork chenglei added a comment - Can we acquire the table shared lock before checking DISABLED or DISABLING state of the table in ServerCrashProcedure.assignRegions to avoid the race ?
            zhangduo Duo Zhang added a comment -

            The PR for fixing this issue is ready for reviewing.

            PTAL if you have interest.

            Thanks.

            zhangduo Duo Zhang added a comment - The PR for fixing this issue is ready for reviewing. PTAL if you have interest. Thanks.
            zhangduo Duo Zhang added a comment -

            Pushed to all active branches.

            Thanks all for helping and reviewing!

            zhangduo Duo Zhang added a comment - Pushed to all active branches. Thanks all for helping and reviewing!
            hudson Hudson added a comment -

            Results for branch branch-2.6
            build #176 on builds.a.o: -1 overall


            details (if available):

            +1 general checks
            – For more information see general report

            +1 jdk8 hadoop2 checks
            – For more information see jdk8 (hadoop2) report

            -1 jdk8 hadoop3 checks
            – For more information see jdk8 (hadoop3) report

            +1 jdk11 hadoop3 checks
            – For more information see jdk11 report

            +1 jdk17 hadoop3 checks
            – For more information see jdk17 report

            +1 source release artifact
            – See build output for details.

            +1 client integration test

            hudson Hudson added a comment - Results for branch branch-2.6 build #176 on builds.a.o : -1 overall details (if available): +1 general checks – For more information see general report +1 jdk8 hadoop2 checks – For more information see jdk8 (hadoop2) report -1 jdk8 hadoop3 checks – For more information see jdk8 (hadoop3) report +1 jdk11 hadoop3 checks – For more information see jdk11 report +1 jdk17 hadoop3 checks – For more information see jdk17 report +1 source release artifact – See build output for details. +1 client integration test
            hudson Hudson added a comment -

            Results for branch branch-2
            build #1117 on builds.a.o: -1 overall


            details (if available):

            +1 general checks
            – For more information see general report

            +1 jdk8 hadoop2 checks
            – For more information see jdk8 (hadoop2) report

            -1 jdk8 hadoop3 checks
            – For more information see jdk8 (hadoop3) report

            +1 jdk11 hadoop3 checks
            – For more information see jdk11 report

            +1 jdk17 hadoop3 checks
            – For more information see jdk17 report

            +1 source release artifact
            – See build output for details.

            +1 client integration test

            hudson Hudson added a comment - Results for branch branch-2 build #1117 on builds.a.o : -1 overall details (if available): +1 general checks – For more information see general report +1 jdk8 hadoop2 checks – For more information see jdk8 (hadoop2) report -1 jdk8 hadoop3 checks – For more information see jdk8 (hadoop3) report +1 jdk11 hadoop3 checks – For more information see jdk11 report +1 jdk17 hadoop3 checks – For more information see jdk17 report +1 source release artifact – See build output for details. +1 client integration test
            hudson Hudson added a comment -

            Results for branch branch-3
            build #261 on builds.a.o: -1 overall


            details (if available):

            +1 general checks
            – For more information see general report

            -1 jdk17 hadoop3 checks
            – For more information see jdk17 report

            +1 source release artifact
            – See build output for details.

            -1 client integration test
            – Something went wrong with this stage, check relevant console output.

            hudson Hudson added a comment - Results for branch branch-3 build #261 on builds.a.o : -1 overall details (if available): +1 general checks – For more information see general report -1 jdk17 hadoop3 checks – For more information see jdk17 report +1 source release artifact – See build output for details. -1 client integration test – Something went wrong with this stage, check relevant console output .
            hudson Hudson added a comment -

            Results for branch master
            build #1132 on builds.a.o: -1 overall


            details (if available):

            +1 general checks
            – For more information see general report

            +1 jdk17 hadoop3 checks
            – For more information see jdk17 report

            +1 source release artifact
            – See build output for details.

            -1 client integration test
            – Something went wrong with this stage, check relevant console output.

            hudson Hudson added a comment - Results for branch master build #1132 on builds.a.o : -1 overall details (if available): +1 general checks – For more information see general report +1 jdk17 hadoop3 checks – For more information see jdk17 report +1 source release artifact – See build output for details. -1 client integration test – Something went wrong with this stage, check relevant console output .
            hudson Hudson added a comment -

            Results for branch branch-2.5
            build #579 on builds.a.o: +1 overall


            details (if available):

            +1 general checks
            – For more information see general report

            +1 jdk8 hadoop2 checks
            – For more information see jdk8 (hadoop2) report

            +1 jdk8 hadoop3 checks
            – For more information see jdk8 (hadoop3) report

            +1 jdk11 hadoop3 checks
            – For more information see jdk11 report

            +1 jdk17 hadoop3 checks
            – For more information see jdk17 report

            +1 source release artifact
            – See build output for details.

            +1 client integration test

            hudson Hudson added a comment - Results for branch branch-2.5 build #579 on builds.a.o : +1 overall details (if available): +1 general checks – For more information see general report +1 jdk8 hadoop2 checks – For more information see jdk8 (hadoop2) report +1 jdk8 hadoop3 checks – For more information see jdk8 (hadoop3) report +1 jdk11 hadoop3 checks – For more information see jdk11 report +1 jdk17 hadoop3 checks – For more information see jdk17 report +1 source release artifact – See build output for details. +1 client integration test

            People

              zhangduo Duo Zhang
              prathyu6 Prathyusha
              Votes:
              1 Vote for this issue
              Watchers:
              13 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: