/** * Invocation of {@link #signalSuccessWhen(Coder, SerializableFunction, SerializableFunction)} * with {@link Object#toString} as the formatter. */ public <T> PTransform<PCollection<? extends T>, POutput> signalSuccessWhen( Coder<T> coder, SerializableFunction<Set<T>, Boolean> successPredicate) { return signalSuccessWhen(coder, T::toString, successPredicate); }
/** * Creates an instance of this rule. * * <p>Loads GCP configuration from {@link TestPipelineOptions}. */ public static TestPubsubSignal create() { TestPubsubOptions options = TestPipeline.testingPipelineOptions().as(TestPubsubOptions.class); return new TestPubsubSignal(options); }
/** Wait for a success signal for {@code duration}. */ public void waitForSuccess(Duration duration) throws IOException { SubscriptionPath resultSubscriptionPath = PubsubClient.subscriptionPathFromName( pipelineOptions.getProject(), "result-subscription-" + String.valueOf(ThreadLocalRandom.current().nextLong())); pubsub.createSubscription( resultTopicPath, resultSubscriptionPath, (int) duration.getStandardSeconds()); String result = pollForResultForDuration(resultSubscriptionPath, duration); if (!RESULT_SUCCESS_MESSAGE.equals(result)) { throw new AssertionError(result); } }
@Test public void testReadPublicData() throws Exception { // The pipeline will never terminate on its own pipeline.getOptions().as(DirectOptions.class).setBlockOnRun(false); PCollection<String> messages = pipeline.apply( PubsubIO.readStrings() .fromTopic("projects/pubsub-public-data/topics/taxirides-realtime")); messages.apply( "waitForAnyMessage", signal.signalSuccessWhen(messages.getCoder(), anyMessages -> true)); Supplier<Void> start = signal.waitForStart(Duration.standardMinutes(5)); pipeline.apply(signal.signalStart()); PipelineResult job = pipeline.run(); start.get(); signal.waitForSuccess(Duration.standardSeconds(30)); // A runner may not support cancel try { job.cancel(); } catch (UnsupportedOperationException exc) { // noop } } }
private String pollForResultForDuration( SubscriptionPath signalSubscriptionPath, Duration duration) throws IOException { List<PubsubClient.IncomingMessage> signal = null; DateTime endPolling = DateTime.now().plus(duration.getMillis()); do { try { signal = pubsub.pull(DateTime.now().getMillis(), signalSubscriptionPath, 1, false); pubsub.acknowledge( signalSubscriptionPath, signal.stream().map(m -> m.ackId).collect(toList())); break; } catch (StatusRuntimeException e) { if (!Status.DEADLINE_EXCEEDED.equals(e.getStatus())) { LOG.warn( "(Will retry) Error while polling {} for signal: {}", signalSubscriptionPath, e.getStatus()); } sleep(500); } } while (DateTime.now().isBefore(endPolling)); if (signal == null) { throw new AssertionError( String.format( "Did not receive signal on %s in %ss", signalSubscriptionPath, duration.getStandardSeconds())); } return new String(signal.get(0).elementBytes, UTF_8); }
resultSignal.signalSuccessWhen( SchemaCoder.of( PAYLOAD_SCHEMA, SerializableFunctions.identity(), SerializableFunctions.identity()), Supplier<Void> start = resultSignal.waitForStart(Duration.standardMinutes(5)); pipeline.begin().apply(resultSignal.signalStart()); resultSignal.waitForSuccess(Duration.standardSeconds(60));
resultSignal.signalSuccessWhen( SchemaCoder.of( PAYLOAD_SCHEMA, SerializableFunctions.identity(), SerializableFunctions.identity()), Supplier<Void> start = resultSignal.waitForStart(Duration.standardMinutes(5)); pipeline.begin().apply("signal query results started", resultSignal.signalStart()); dlqSignal.signalSuccessWhen( PubsubMessageWithAttributesCoder.of(), dlqMessages -> Supplier<Void> startDlq = dlqSignal.waitForStart(Duration.standardMinutes(5)); pipeline.begin().apply("signal DLQ started", dlqSignal.signalStart()); resultSignal.waitForSuccess(Duration.standardMinutes(2)); dlqSignal.waitForSuccess(Duration.standardMinutes(2));
/** * Future that waits for a start signal for {@code duration}. * * <p>This future must be created before running the pipeline. A subscription must exist prior to * the start signal being published, which occurs immediately upon pipeline startup. */ public Supplier<Void> waitForStart(Duration duration) throws IOException { SubscriptionPath startSubscriptionPath = PubsubClient.subscriptionPathFromName( pipelineOptions.getProject(), "start-subscription-" + String.valueOf(ThreadLocalRandom.current().nextLong())); pubsub.createSubscription( startTopicPath, startSubscriptionPath, (int) duration.getStandardSeconds()); return Suppliers.memoize( () -> { try { String result = pollForResultForDuration(startSubscriptionPath, duration); checkState(START_SIGNAL_MESSAGE.equals(result)); return null; } catch (IOException e) { throw new RuntimeException(e); } }); }