adude3141 commented on a change in pull request #15955:
URL: https://github.com/apache/beam/pull/15955#discussion_r759573709
##########
File path:
sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisIO.java
##########
@@ -884,28 +885,81 @@ 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");
+ producerConfiguration(); // verify Kinesis producer configuration can be
build
input.apply(ParDo.of(new KinesisWriterFn(this)));
return PDone.in(input.getPipeline());
}
- private static class KinesisWriterFn extends DoFn<byte[], Void> {
+ private KinesisProducerConfiguration producerConfiguration() {
+ 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.producerConfiguration());
+ producerRefCount.set(0);
+ }
+ }
+ producerRefCount.incrementAndGet();
Review comment:
Sorry for being unclear. I am more concerned about the 'publishing' of
inconsistent state after releasing the lock/synchronized block in line 936,
i.e. after crossing the memory fence. `producerRecCount` is 0, where it should
be 1.
Unlikely (though possible) flow: Two DoFns. First runs, get suspended after
line 936, i.e. producer initialised, refCount == 0. Now the second runs
through. In `tearDown` refCount will be 0, and producer destroyed and set to
null. Now first continues and will throw NPE trying to access producer.
I d say an easy fix could be to simply pull that
`producerRefCount.incrementAndGet()` (line 937) into the synchronised block. As
we aquire the lock/synchronize anyway, this will not hurt. Of course, we could
consider switching to plain int then as a counter, as it is accessed in
`teardown´ also behind the lock.
Alternatively a call `producerRefCount.set(1); return;` in 934 would als
prevent that. We would still have those 2 men fences guarding the reference
counting, but this should be ok in this situation.
Does this make more sense?
Note: If we were to implement a `RefCountingObject` here supporting
retain/release without locking/synchronization the matter would change. This
would probably need more thoughts though, as this 'static reference' would
simply move and we still need that kind of lazy initialised singleton instance
which needs config for initialisation.
--
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]