mosche commented on a change in pull request #15955:
URL: https://github.com/apache/beam/pull/15955#discussion_r759112753
##########
File path:
sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisIO.java
##########
@@ -884,28 +885,82 @@ public PDone expand(PCollection<byte[]> input) {
getPartitionKey() == null || (getPartitioner() == null),
"only one of either withPartitionKey() or withPartitioner() is
possible");
checkArgument(getAWSClientsProvider() != null, "withAWSClientsProvider()
is required");
+ createProducerConfiguration(); // verify Kinesis producer configuration
can be built
input.apply(ParDo.of(new KinesisWriterFn(this)));
return PDone.in(input.getPipeline());
}
- private static class KinesisWriterFn extends DoFn<byte[], Void> {
+ private KinesisProducerConfiguration createProducerConfiguration() {
+ Properties props = getProducerProperties();
+ if (props == null) {
+ props = new Properties();
+ }
+ return KinesisProducerConfiguration.fromProperties(props);
+ }
+ private static class KinesisWriterFn extends DoFn<byte[], Void> {
private static final int MAX_NUM_FAILURES = 10;
+ /** Usage count of static, shared Kinesis producer. */
+ private static final AtomicInteger producerRefCount = new
AtomicInteger();
+
+ /** Static, shared Kinesis producer. */
+ private static IKinesisProducer producer;
private final KinesisIO.Write spec;
- private static transient IKinesisProducer producer;
+
private transient KinesisPartitioner partitioner;
private transient LinkedBlockingDeque<KinesisWriteException> failures;
private transient List<Future<UserRecordResult>> putFutures;
KinesisWriterFn(KinesisIO.Write spec) {
this.spec = spec;
- initKinesisProducer();
+ }
+
+ /**
+ * Initialize statically shared Kinesis producer if required and count
usage.
+ *
+ * <p>NOTE: If there is, for whatever reasons, another instance of a
{@link KinesisWriterFn}
+ * with different producer properties or even a different implementation
of {@link
+ * AWSClientsProvider}, these changes will be silently discarded in
favor of an existing
+ * producer instance.
+ */
+ @SuppressFBWarnings("JLM_JSR166_UTILCONCURRENT_MONITORENTER")
+ private void setupSharedProducer() {
+ synchronized (producerRefCount) {
+ if (producer == null) {
+ producer =
+ spec.getAWSClientsProvider()
+ .createKinesisProducer(spec.createProducerConfiguration());
+ producerRefCount.set(0);
+ }
+ }
+ producerRefCount.incrementAndGet();
+ }
+
+ /**
+ * Discard statically shared producer if it is not used anymore
according to the usage count.
+ */
+ @SuppressFBWarnings("JLM_JSR166_UTILCONCURRENT_MONITORENTER")
+ private void teardownSharedProducer() {
+ synchronized (producerRefCount) {
+ if (producerRefCount.decrementAndGet() == 0) {
+ // the following checks should never be true, but just in case
+ if (producer == null) {
+ return;
+ }
+ if (producer.getOutstandingRecordsCount() > 0) {
+ producer.flushSync();
Review comment:
`finishBundle` contains a different blocking version of flush that only
blocks on the records emitted by this bundle. It would actually be harmful to
do that: Using `producer.flushSync()` in `finishBundle` would likely block that
dofn instance "forever" because the producer is shared and other instances keep
writing to the producer. `flushSync` basically loops indefinitely until there's
no more pending records in the producer.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]