Skip to content

Commit 7ca2362

Browse files
committed
tests
1 parent 2542f18 commit 7ca2362

2 files changed

Lines changed: 36 additions & 3 deletions

File tree

sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignatures.java

Lines changed: 4 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -1362,7 +1362,10 @@ private static Parameter analyzeExtraParameter(
13621362
return Parameter.keyT(paramT);
13631363
} else if (rawType.equals(TimeDomain.class)) {
13641364
return Parameter.timeDomainParameter();
1365-
} else if (rawType.equals(CausedByDrain.class)) {
1365+
} else if (CausedByDrain.class.isAssignableFrom(rawType)) {
1366+
methodErrors.checkArgument(
1367+
rawType.equals(CausedByDrain.class),
1368+
"CausedByDrain argument must have type org.apache.beam.sdk.values.CausedByDrain.");
13661369
return Parameter.causedByDrainParameter();
13671370
} else if (hasAnnotation(DoFn.SideInput.class, param.getAnnotations())) {
13681371
String sideInputId = getSideInputId(param.getAnnotations());

sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnSignaturesTest.java

Lines changed: 32 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -56,6 +56,7 @@
5656
import org.apache.beam.sdk.transforms.Sum;
5757
import org.apache.beam.sdk.transforms.reflect.DoFnSignature.Parameter;
5858
import org.apache.beam.sdk.transforms.reflect.DoFnSignature.Parameter.BundleFinalizerParameter;
59+
import org.apache.beam.sdk.transforms.reflect.DoFnSignature.Parameter.CausedByDrainParameter;
5960
import org.apache.beam.sdk.transforms.reflect.DoFnSignature.Parameter.ElementParameter;
6061
import org.apache.beam.sdk.transforms.reflect.DoFnSignature.Parameter.FinishBundleContextParameter;
6162
import org.apache.beam.sdk.transforms.reflect.DoFnSignature.Parameter.OutputReceiverParameter;
@@ -78,6 +79,7 @@
7879
import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
7980
import org.apache.beam.sdk.transforms.windowing.PaneInfo;
8081
import org.apache.beam.sdk.transforms.windowing.TimestampCombiner;
82+
import org.apache.beam.sdk.values.CausedByDrain;
8183
import org.apache.beam.sdk.values.KV;
8284
import org.apache.beam.sdk.values.Row;
8385
import org.apache.beam.sdk.values.TypeDescriptor;
@@ -130,10 +132,11 @@ public void process(
130132
PipelineOptions options,
131133
@SideInput("tag1") String input1,
132134
@SideInput("tag2") Integer input2,
133-
BundleFinalizer bundleFinalizer) {}
135+
BundleFinalizer bundleFinalizer,
136+
CausedByDrain causedByDrain) {}
134137
}.getClass());
135138

136-
assertThat(sig.processElement().extraParameters().size(), equalTo(9));
139+
assertThat(sig.processElement().extraParameters().size(), equalTo(10));
137140
assertThat(sig.processElement().extraParameters().get(0), instanceOf(ElementParameter.class));
138141
assertThat(sig.processElement().extraParameters().get(1), instanceOf(TimestampParameter.class));
139142
assertThat(sig.processElement().extraParameters().get(2), instanceOf(WindowParameter.class));
@@ -146,6 +149,8 @@ public void process(
146149
assertThat(sig.processElement().extraParameters().get(7), instanceOf(SideInputParameter.class));
147150
assertThat(
148151
sig.processElement().extraParameters().get(8), instanceOf(BundleFinalizerParameter.class));
152+
assertThat(
153+
sig.processElement().extraParameters().get(9), instanceOf(CausedByDrainParameter.class));
149154
}
150155

151156
@Test
@@ -585,6 +590,31 @@ public void onTimer(BoundedWindow w) {}
585590
instanceOf(WindowParameter.class));
586591
}
587592

593+
@Test
594+
public void testCausedByDrainOnTimer() throws Exception {
595+
final String timerId = "some-timer-id";
596+
final String timerDeclarationId = TimerDeclaration.PREFIX + timerId;
597+
598+
DoFnSignature sig =
599+
DoFnSignatures.getSignature(
600+
new DoFn<String, String>() {
601+
602+
@TimerId(timerId)
603+
private final TimerSpec myfield1 = TimerSpecs.timer(TimeDomain.EVENT_TIME);
604+
605+
@ProcessElement
606+
public void process(ProcessContext c) {}
607+
608+
@OnTimer(timerId)
609+
public void onTimer(CausedByDrain causedByDrain) {}
610+
}.getClass());
611+
612+
assertThat(sig.onTimerMethods().get(timerDeclarationId).extraParameters().size(), equalTo(1));
613+
assertThat(
614+
sig.onTimerMethods().get(timerDeclarationId).extraParameters().get(0),
615+
instanceOf(CausedByDrainParameter.class));
616+
}
617+
588618
@Test
589619
public void testAllParamsOnTimer() throws Exception {
590620
final String timerId = "some-timer-id";

0 commit comments

Comments
 (0)