Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[INLONG-7940][Sort] Use pulsar subscriptions and specify subscription offset in pulsar connector #7943

Merged
merged 2 commits into from
May 8, 2023
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -50,7 +50,7 @@ public Node getTestObject() {
format,
"earliest",
null,
null,
null);
"subscription",
"earliest");
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -1006,6 +1006,16 @@ public Map<TopicRange, MessageId> offsetForEachTopic(
}
}
return specificOffsets;
case EXTERNAL_SUBSCRIPTION:
Map<TopicRange, MessageId> offsetsFromSubs = new HashMap<>();
for (TopicRange topic : topics) {
offsetsFromSubs.put(
topic,
metadataReader.getPositionFromSubscription(
topic, subscriptionPosition));
}
log.info("offset for each topic: {}", offsetsFromSubs);
return offsetsFromSubs;
}
return null;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,9 @@

package org.apache.inlong.sort.pulsar.internal;

import java.io.UnsupportedEncodingException;
import java.net.URLEncoder;
import java.nio.charset.StandardCharsets;
import org.apache.flink.configuration.ConfigOption;
import org.apache.flink.configuration.ConfigOptions;
import org.apache.flink.streaming.connectors.pulsar.internal.PulsarClientUtils;
Expand All @@ -33,9 +36,13 @@
import org.apache.pulsar.client.api.PulsarClient;
import org.apache.pulsar.client.api.PulsarClientException;
import org.apache.pulsar.client.api.Range;
import org.apache.pulsar.client.impl.MessageIdImpl;
import org.apache.pulsar.client.impl.PulsarClientImpl;
import org.apache.pulsar.client.impl.conf.ClientConfigurationData;
import org.apache.pulsar.common.naming.TopicName;
import org.apache.pulsar.common.policies.data.PersistentTopicInternalStats;
import org.apache.pulsar.common.policies.data.SubscriptionStats;
import org.apache.pulsar.common.policies.data.TopicStats;
import org.apache.pulsar.shade.com.google.common.collect.Sets;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
Expand Down Expand Up @@ -260,6 +267,44 @@ public void commitOffsetToCursor(Map<TopicRange, MessageId> offset) {
}
}

public MessageId getPositionFromSubscription(TopicRange topic, MessageId defaultPosition) {
try {
String subscriptionName = subscriptionNameFrom(topic);
TopicStats topicStats = admin.topics().getStats(topic.getTopic());
if (topicStats.getSubscriptions().containsKey(subscriptionName)) {
SubscriptionStats subStats = topicStats.getSubscriptions().get(subscriptionName);
if (subStats.getConsumers().size() != 0) {
throw new IllegalStateException(
"Subscription been actively used by other consumers, "
+ "in this situation, the exactly-once semantics cannot be guaranteed.");
} else {
String encodedSubName =
URLEncoder.encode(subscriptionName, StandardCharsets.UTF_8.toString());
PersistentTopicInternalStats.CursorStats c =
admin.topics()
.getInternalStats(topic.getTopic()).cursors
.get(encodedSubName);
String[] ids = c.markDeletePosition.split(":", 2);
long ledgerId = Long.parseLong(ids[0]);
long entryIdInMarkDelete = Long.parseLong(ids[1]);
// we are getting the next mid from sub position, if the entryId is -1,
// it denotes we haven't read data from the ledger before,
// therefore no need to skip the current entry for the next position
long entryId = entryIdInMarkDelete == -1 ? -1 : entryIdInMarkDelete + 1;
int partitionIdx = TopicName.getPartitionIndex(topic.getTopic());
return new MessageIdImpl(ledgerId, entryId, partitionIdx);
}
} else {
// create sub on topic
admin.topics()
.createSubscription(topic.getTopic(), subscriptionName, defaultPosition);
return defaultPosition;
}
} catch (PulsarAdminException | UnsupportedEncodingException e) {
throw new IllegalStateException("Failed to get stats for topic " + topic, e);
}
}

/**
* Designate the close of the metadata reader.
*/
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -74,8 +74,8 @@ public PulsarExtractNode buildPulsarExtractNode() {
format,
"earliest",
null,
null,
null);
"test",
"earliest");
}

private NodeRelation buildNodeRelation(List<Node> inputs, List<Node> outputs) {
Expand Down