adude3141 commented on a change in pull request #15955:
URL: https://github.com/apache/beam/pull/15955#discussion_r758658621
##########
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:
Wouldn't we have 2 memory fences here? Although I can not come up with a
code path here, I could imagine someone adding some code in setup before
calling into setupSharedProducer. If that throws, we could end up with setting
refCount to 0 in the synchronised block here, get suspended before line 937,
teardown on that other instance will called, resulting in refCount to be -1.
From there on we will be off by one. Does this make any sense?
--
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]