pabloem commented on code in PR #24713:
URL: https://github.com/apache/beam/pull/24713#discussion_r1085740778
##########
sdks/java/io/debezium/src/main/java/org/apache/beam/io/debezium/KafkaSourceConsumerFn.java:
##########
@@ -217,26 +239,44 @@ private static Instant
ensureTimestampWithinBounds(Instant timestamp) {
public ProcessContinuation process(
@Element Map<String, String> element,
RestrictionTracker<OffsetHolder, Map<String, Object>> tracker,
- OutputReceiver<T> receiver)
- throws Exception {
- Map<String, String> configuration = new HashMap<>(element);
-
+ OutputReceiver<T> receiver) {
// Adding the current restriction to the class object to be found by the
database history
register(tracker);
+ Map<String, String> configuration = new HashMap<>(element);
Review Comment:
done thanks
##########
sdks/java/io/debezium/src/main/java/org/apache/beam/io/debezium/KafkaSourceConsumerFn.java:
##########
@@ -217,26 +239,44 @@ private static Instant
ensureTimestampWithinBounds(Instant timestamp) {
public ProcessContinuation process(
@Element Map<String, String> element,
RestrictionTracker<OffsetHolder, Map<String, Object>> tracker,
- OutputReceiver<T> receiver)
- throws Exception {
- Map<String, String> configuration = new HashMap<>(element);
-
+ OutputReceiver<T> receiver) {
// Adding the current restriction to the class object to be found by the
database history
register(tracker);
+ Map<String, String> configuration = new HashMap<>(element);
+ String cacheKey =
+ configuration.entrySet().stream()
Review Comment:
removed thanks
##########
sdks/java/io/debezium/src/main/java/org/apache/beam/io/debezium/KafkaSourceConsumerFn.java:
##########
@@ -217,26 +239,44 @@ private static Instant
ensureTimestampWithinBounds(Instant timestamp) {
public ProcessContinuation process(
@Element Map<String, String> element,
RestrictionTracker<OffsetHolder, Map<String, Object>> tracker,
- OutputReceiver<T> receiver)
- throws Exception {
- Map<String, String> configuration = new HashMap<>(element);
-
+ OutputReceiver<T> receiver) {
// Adding the current restriction to the class object to be found by the
database history
register(tracker);
+ Map<String, String> configuration = new HashMap<>(element);
+ String cacheKey =
+ configuration.entrySet().stream()
+ .map(entry -> String.format("(%s,%s)", entry.getKey(),
entry.getValue()))
+ .sorted()
+ .collect(Collectors.joining(","));
configuration.put(BEAM_INSTANCE_PROPERTY, this.getHashCode());
+ SourceTask task;
- SourceConnector connector =
connectorClass.getDeclaredConstructor().newInstance();
- connector.start(configuration);
-
- SourceTask task = (SourceTask)
connector.taskClass().getDeclaredConstructor().newInstance();
-
- try {
+ if (CONNECTOR_CACHE.getIfPresent(cacheKey) == null) {
Review Comment:
done
##########
sdks/java/io/debezium/src/main/java/org/apache/beam/io/debezium/KafkaSourceConsumerFn.java:
##########
@@ -217,26 +239,44 @@ private static Instant
ensureTimestampWithinBounds(Instant timestamp) {
public ProcessContinuation process(
@Element Map<String, String> element,
RestrictionTracker<OffsetHolder, Map<String, Object>> tracker,
- OutputReceiver<T> receiver)
- throws Exception {
- Map<String, String> configuration = new HashMap<>(element);
-
+ OutputReceiver<T> receiver) {
// Adding the current restriction to the class object to be found by the
database history
register(tracker);
+ Map<String, String> configuration = new HashMap<>(element);
+ String cacheKey =
+ configuration.entrySet().stream()
+ .map(entry -> String.format("(%s,%s)", entry.getKey(),
entry.getValue()))
+ .sorted()
+ .collect(Collectors.joining(","));
configuration.put(BEAM_INSTANCE_PROPERTY, this.getHashCode());
+ SourceTask task;
- SourceConnector connector =
connectorClass.getDeclaredConstructor().newInstance();
- connector.start(configuration);
-
- SourceTask task = (SourceTask)
connector.taskClass().getDeclaredConstructor().newInstance();
-
- try {
+ if (CONNECTOR_CACHE.getIfPresent(cacheKey) == null) {
+ SourceConnector connector = null;
+ try {
+ connector = connectorClass.getDeclaredConstructor().newInstance();
+ connector.start(configuration);
+ task = (SourceTask)
connector.taskClass().getDeclaredConstructor().newInstance();
+ } catch (InstantiationException
+ | IllegalAccessException
+ | InvocationTargetException
+ | NoSuchMethodException e) {
+ throw new RuntimeException("Unable to initialize connector instance
for Debezium", e);
+ }
Map<String, ?> consumerOffset = tracker.currentRestriction().offset;
- LOG.debug("--------- Consumer offset from Debezium Tracker: {}",
consumerOffset);
+ LOG.debug("--------- Created new Debezium task with offset: {}",
consumerOffset);
task.initialize(new
BeamSourceTaskContext(tracker.currentRestriction().offset));
task.start(connector.taskConfigs(1).get(0));
+ CONNECTOR_CACHE.put(cacheKey, task);
+ } else {
+ task = CONNECTOR_CACHE.getIfPresent(cacheKey);
Review Comment:
done
##########
sdks/java/io/debezium/src/test/java/org/apache/beam/io/debezium/KafkaSourceConsumerFnTest.java:
##########
@@ -103,6 +103,7 @@ public void testStoppableKafkaSourceConsumerFn() {
.setCoder(VarIntCoder.of());
pipeline.run().waitUntilFinish();
+ // Since we're now caching calls
Review Comment:
not really important.
--
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]