.apply(MapElements.via(getFormatFn())) .apply( new PubsubUnboundedSink( FACTORY, NestedValueProvider.of(getTopicProvider(), new TopicPathTranslator()),
StepTranslationContext stepContext = context.addStep(transform, "ParallelWrite"); stepContext.addInput(PropertyNames.FORMAT, "pubsub"); if (overriddenTransform.getTopicProvider().isAccessible()) { stepContext.addInput( PropertyNames.PUBSUB_TOPIC, overriddenTransform.getTopic().getV1Beta1Path()); } else { stepContext.addInput( PropertyNames.PUBSUB_TOPIC_OVERRIDE, ((NestedValueProvider) overriddenTransform.getTopicProvider()).propertyName()); if (overriddenTransform.getTimestampAttribute() != null) { stepContext.addInput( PropertyNames.PUBSUB_TIMESTAMP_ATTRIBUTE, overriddenTransform.getTimestampAttribute()); if (overriddenTransform.getIdAttribute() != null) { stepContext.addInput( PropertyNames.PUBSUB_ID_ATTRIBUTE, overriddenTransform.getIdAttribute());
PubsubTestClient.createFactoryForPublish(TOPIC, outgoing, ImmutableList.of())) { PubsubUnboundedSink sink = new PubsubUnboundedSink( factory, StaticValueProvider.of(TOPIC),
PubsubTestClient.createFactoryForPublish(TOPIC, outgoing, ImmutableList.of())) { PubsubUnboundedSink sink = new PubsubUnboundedSink( factory, StaticValueProvider.of(TOPIC),
@Test public void sendOneMessageWithoutAttributes() throws IOException { List<OutgoingMessage> outgoing = ImmutableList.of( new OutgoingMessage( DATA.getBytes(StandardCharsets.UTF_8), null /* attributes */, TIMESTAMP, getRecordId(DATA))); try (PubsubTestClientFactory factory = PubsubTestClient.createFactoryForPublish(TOPIC, outgoing, ImmutableList.of())) { PubsubUnboundedSink sink = new PubsubUnboundedSink( factory, StaticValueProvider.of(TOPIC), TIMESTAMP_ATTRIBUTE, ID_ATTRIBUTE, NUM_SHARDS, 1 /* batchSize */, 1 /* batchBytes */, Duration.standardSeconds(2), RecordIdMethod.DETERMINISTIC); p.apply(Create.of(ImmutableList.of(DATA))) .apply(ParDo.of(new Stamp(null /* attributes */))) .apply(sink); p.run(); } // The PubsubTestClientFactory will assert fail on close if the actual published // message does not match the expected publish message. }
@Test public void sendOneMessage() throws IOException { List<OutgoingMessage> outgoing = ImmutableList.of( new OutgoingMessage( DATA.getBytes(StandardCharsets.UTF_8), ATTRIBUTES, TIMESTAMP, getRecordId(DATA))); int batchSize = 1; int batchBytes = 1; try (PubsubTestClientFactory factory = PubsubTestClient.createFactoryForPublish(TOPIC, outgoing, ImmutableList.of())) { PubsubUnboundedSink sink = new PubsubUnboundedSink( factory, StaticValueProvider.of(TOPIC), TIMESTAMP_ATTRIBUTE, ID_ATTRIBUTE, NUM_SHARDS, batchSize, batchBytes, Duration.standardSeconds(2), RecordIdMethod.DETERMINISTIC); p.apply(Create.of(ImmutableList.of(DATA))).apply(ParDo.of(new Stamp(ATTRIBUTES))).apply(sink); p.run(); } // The PubsubTestClientFactory will assert fail on close if the actual published // message does not match the expected publish message. }