2424
2525import java .util .ArrayList ;
2626import java .util .List ;
27+ import java .util .concurrent .TimeUnit ;
2728import org .apache .beam .sdk .fn .splittabledofn .RestrictionTrackers .ClaimObserver ;
2829import org .apache .beam .sdk .transforms .splittabledofn .RestrictionTracker ;
2930import org .apache .beam .sdk .transforms .splittabledofn .RestrictionTracker .HasProgress ;
3031import org .apache .beam .sdk .transforms .splittabledofn .SplitResult ;
32+ import org .junit .Rule ;
3133import org .junit .Test ;
34+ import org .junit .rules .Timeout ;
3235import org .junit .runner .RunWith ;
3336import org .junit .runners .JUnit4 ;
3437
3841 "rawtypes" , // TODO(https://github.com/apache/beam/issues/20447)
3942})
4043public class RestrictionTrackersTest {
44+ @ Rule public Timeout timeout = new Timeout (1 , TimeUnit .MINUTES );
45+
4146 @ Test
4247 public void testObservingClaims () {
4348 RestrictionTracker <String , String > observedTracker =
@@ -95,14 +100,37 @@ public void onClaimFailed(String position) {
95100
96101 private static class RestrictionTrackerWithProgress extends RestrictionTracker <Object , Object >
97102 implements HasProgress {
103+ private boolean blockTryClaim ;
104+ private boolean blockTrySplit ;
105+ private boolean isBlocked ;
106+ public static final Progress REPORT_PROGRESS = Progress .from (2.0 , 3.0 );
107+
108+ public RestrictionTrackerWithProgress () {
109+ this (false , false );
110+ }
111+
112+ public RestrictionTrackerWithProgress (boolean blockTryClaim , boolean blockTrySplit ) {
113+ this .blockTryClaim = blockTryClaim ;
114+ this .blockTrySplit = blockTrySplit ;
115+ this .isBlocked = false ;
116+ }
98117
99118 @ Override
100119 public Progress getProgress () {
101- return RestrictionTracker . Progress . from ( 2.0 , 3.0 ) ;
120+ return REPORT_PROGRESS ;
102121 }
103122
104123 @ Override
105- public boolean tryClaim (Object position ) {
124+ public synchronized boolean tryClaim (Object position ) {
125+ while (blockTryClaim ) {
126+ isBlocked = true ;
127+ try {
128+ wait ();
129+ } catch (InterruptedException e ) {
130+ Thread .currentThread ().interrupt ();
131+ }
132+ }
133+ isBlocked = false ;
106134 return false ;
107135 }
108136
@@ -112,7 +140,16 @@ public Object currentRestriction() {
112140 }
113141
114142 @ Override
115- public SplitResult <Object > trySplit (double fractionOfRemainder ) {
143+ public synchronized SplitResult <Object > trySplit (double fractionOfRemainder ) {
144+ while (blockTrySplit ) {
145+ isBlocked = true ;
146+ try {
147+ wait ();
148+ } catch (InterruptedException e ) {
149+ Thread .currentThread ().interrupt ();
150+ }
151+ }
152+ isBlocked = false ;
116153 return null ;
117154 }
118155
@@ -123,6 +160,19 @@ public void checkDone() throws IllegalStateException {}
123160 public IsBounded isBounded () {
124161 return IsBounded .BOUNDED ;
125162 }
163+
164+ public synchronized void releaseLock () {
165+ blockTrySplit = false ;
166+ blockTryClaim = false ;
167+ notifyAll ();
168+ }
169+
170+ /** Wait until RestrictionTracker becomes blocking or unblocking. */
171+ public void waitUntilBlocking (boolean blocking ) throws InterruptedException {
172+ while (isBlocked != blocking ) {
173+ Thread .sleep (1 );
174+ }
175+ }
126176 }
127177
128178 @ Test
@@ -131,4 +181,38 @@ public void testClaimObserversMaintainBacklogInterfaces() {
131181 RestrictionTrackers .observe (new RestrictionTrackerWithProgress (), null );
132182 assertThat (hasSize , instanceOf (HasProgress .class ));
133183 }
184+
185+ @ Test
186+ public void testClaimObserversProgressNonBlockingOnTryClaim () throws InterruptedException {
187+ RestrictionTrackerWithProgress withProgress = new RestrictionTrackerWithProgress (true , false );
188+ RestrictionTracker <Object , Object > tracker =
189+ RestrictionTrackers .observe (withProgress , new RestrictionTrackers .NoopClaimObserver <>());
190+ Thread blocking = new Thread (() -> tracker .tryClaim (new Object ()));
191+ blocking .start ();
192+ withProgress .waitUntilBlocking (true );
193+ RestrictionTracker .Progress progress =
194+ ((RestrictionTrackers .RestrictionTrackerObserverWithProgress ) tracker ).getProgress (1 );
195+ assertEquals (RestrictionTracker .Progress .NONE , progress );
196+ withProgress .releaseLock ();
197+ withProgress .waitUntilBlocking (false );
198+ progress = ((HasProgress ) tracker ).getProgress ();
199+ assertEquals (RestrictionTrackerWithProgress .REPORT_PROGRESS , progress );
200+ }
201+
202+ @ Test
203+ public void testClaimObserversProgressNonBlockingOnTrySplit () throws InterruptedException {
204+ RestrictionTrackerWithProgress withProgress = new RestrictionTrackerWithProgress (false , true );
205+ RestrictionTracker <Object , Object > tracker =
206+ RestrictionTrackers .observe (withProgress , new RestrictionTrackers .NoopClaimObserver <>());
207+ Thread blocking = new Thread (() -> tracker .trySplit (0.5 ));
208+ blocking .start ();
209+ withProgress .waitUntilBlocking (true );
210+ RestrictionTracker .Progress progress =
211+ ((RestrictionTrackers .RestrictionTrackerObserverWithProgress ) tracker ).getProgress (1 );
212+ assertEquals (RestrictionTracker .Progress .NONE , progress );
213+ withProgress .releaseLock ();
214+ withProgress .waitUntilBlocking (false );
215+ progress = ((HasProgress ) tracker ).getProgress ();
216+ assertEquals (RestrictionTrackerWithProgress .REPORT_PROGRESS , progress );
217+ }
134218}
0 commit comments