[ 
https://issues.apache.org/jira/browse/FLINK-7554?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16428541#comment-16428541
 ] 

buptljy commented on FLINK-7554:
--------------------------------

[~twalthr] I find that it seems to be a bit harder for testing streaming 
functions like "CoProcessFunction", in which has a Context, because I can't 
create a Context directly. My idea is to create a testingfunction for it, like:

 
{code:java}
public class TestingCoProcessFunction extends CoProcessFunction<Object, Object, 
Object> {

   private TestingRuntimeContext ctx;

   private Context context = new Context() {
      @Override
      public Long timestamp() {
         return -1L;
      }

      @Override
      public TimerService timerService() {
         return new TestingTimerService();
      }

      @Override
      public <X> void output(OutputTag<X> outputTag, X value) {
         ctx.addSideOutput(outputTag, value);
      }
   };

   public TestingCoProcessFunction(TestingRuntimeContext ctx) {
      this.ctx = ctx;
   }

   public Context getContext() {
      return context;
   }

   @Override
   public void processElement1(Object value, Context ctx, Collector<Object> 
out) throws Exception {}

   @Override
   public void processElement2(Object value, Context ctx, Collector<Object> 
out) throws Exception {}
}

{code}
And the test will be like this:
{code:java}
@Test
public void testEnrichmentFunction() throws Exception {
   TestingRuntimeContext ctx = new TestingRuntimeContext(true);
   EnrichmentFunction func = new EnrichmentFunction();
   func.setRuntimeContext(ctx);
   CoProcessFunction.Context context = new 
TestingCoProcessFunction(ctx).getContext();
   ValueStateDescriptor<TaxiRide> rideStateDesc = new 
ValueStateDescriptor<>("saved ride", TaxiRide.class);
   ValueStateDescriptor<TaxiFare> fareStateDesc = new 
ValueStateDescriptor<>("saved fare", TaxiFare.class);
   ctx.setState(rideStateDesc, new TestingValueState<TaxiRide>(null));
   ctx.setState(fareStateDesc, new TestingValueState<TaxiFare>(null));
   func.open(new Configuration());

   TaxiRide ride1 = new TaxiRide(1);
   func.processElement1(ride1, context, ctx.getCollector());
   Assert.assertEquals(ctx.getState(rideStateDesc).value(), ride1);
}
{code}
And in this way, I will create many testingfunctions for 
"BroadcastProcessFunction", "CoProcessFunction", etc. I can do this but I 
wonder if you have any better ideas ?

 

> Add a testing RuntimeContext to test utilities
> ----------------------------------------------
>
>                 Key: FLINK-7554
>                 URL: https://issues.apache.org/jira/browse/FLINK-7554
>             Project: Flink
>          Issue Type: New Feature
>          Components: Tests
>            Reporter: Timo Walther
>            Assignee: buptljy
>            Priority: Major
>              Labels: starter
>
> When unit testing user-defined functions it would be useful to have an 
> official testing {{RuntimeContext}} that uses Java collections for storing 
> state, metrics, etc.
> After executing the business logic, the user could then verify how the state 
> of the UDF changed or which metrics have been collected.
> This issue includes documentation for the "Testing" section.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to