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

pulsar connector accepts auth params #22028

Closed
wants to merge 2 commits into from
Closed
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 @@ -51,6 +51,10 @@ public abstract static class Read extends PTransform<PBegin, PCollection<PulsarM

abstract @Nullable String getAdminUrl();

abstract @Nullable String getAuthPluginClassName();

abstract @Nullable String getAuthParameters();

abstract @Nullable String getTopic();

abstract @Nullable Long getStartTimestamp();
Expand All @@ -71,6 +75,10 @@ abstract static class Builder {

abstract Builder setAdminUrl(String url);

abstract Builder setAuthPluginClassName(String authPluginClassName);

abstract Builder setAuthParameters(String authParameters);

abstract Builder setTopic(String topic);

abstract Builder setStartTimestamp(Long timestamp);
Expand All @@ -95,6 +103,14 @@ public Read withClientUrl(String url) {
return builder().setClientUrl(url).build();
}

public Read withAuthPluginClassName(String pluginClassName) {
return builder().setAuthPluginClassName(pluginClassName).build();
}

public Read withAuthParameters(String parameters) {
return builder().setAuthParameters(parameters).build();
}

public Read withTopic(String topic) {
return builder().setTopic(topic).build();
}
Expand Down Expand Up @@ -138,7 +154,9 @@ public PCollection<PulsarMessage> expand(PBegin input) {
getEndTimestamp(),
getEndMessageId(),
getClientUrl(),
getAdminUrl())))
getAdminUrl(),
getAuthPluginClassName(),
getAuthParameters())))
.apply(ParDo.of(new ReadFromPulsarDoFn(this)))
.setCoder(PulsarMessageCoder.of());
}
Expand All @@ -156,6 +174,10 @@ public abstract static class Write extends PTransform<PCollection<byte[]>, PDone

abstract String getClientUrl();

abstract @Nullable String getAuthPluginClassName();

abstract @Nullable String getAuthParameters();

abstract Builder builder();

@AutoValue.Builder
Expand All @@ -164,6 +186,10 @@ abstract static class Builder {

abstract Builder setClientUrl(String clientUrl);

abstract Builder setAuthPluginClassName(String authPluginClassName);

abstract Builder setAuthParameters(String authParameters);

abstract Write build();
}

Expand All @@ -175,6 +201,14 @@ public Write withClientUrl(String clientUrl) {
return builder().setClientUrl(clientUrl).build();
}

public Write withAuthPluginClassName(String pluginClassName) {
return builder().setAuthPluginClassName(pluginClassName).build();
}

public Write withAuthParameters(String parameters) {
return builder().setAuthParameters(parameters).build();
}

@Override
public PDone expand(PCollection<byte[]> input) {
input.apply(ParDo.of(new WriteToPulsarDoFn(this)));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -50,14 +50,31 @@ public abstract class PulsarSourceDescriptor implements Serializable {
@SchemaFieldName("admin_url")
abstract String getAdminUrl();

@SchemaFieldName("authPluginClassName")
@Nullable
abstract String getAuthPluginClassName();

@SchemaFieldName("authParams")
@Nullable
abstract String getAuthParams();

public static PulsarSourceDescriptor of(
String topic,
Long startOffsetTimestamp,
Long endOffsetTimestamp,
MessageId endMessageId,
String clientUrl,
String adminUrl) {
String adminUrl,
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Since this is a public method then it would be better to provide another an overloaded method with the same name and additional arguments.

If this class and method are not supposed to be used by user then it should be private or package private to avoid potential breaking changes like this.

String authPluginClassName,
String authParams) {
return new AutoValue_PulsarSourceDescriptor(
topic, startOffsetTimestamp, endOffsetTimestamp, endMessageId, clientUrl, adminUrl);
topic,
startOffsetTimestamp,
endOffsetTimestamp,
endMessageId,
clientUrl,
adminUrl,
authPluginClassName,
authParams);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -53,18 +53,23 @@ public class ReadFromPulsarDoFn extends DoFn<PulsarSourceDescriptor, PulsarMessa
private PulsarAdmin admin;
private String clientUrl;
private String adminUrl;
private String authPluginClassName;
private String authParameters;

private final SerializableFunction<Message<byte[]>, Instant> extractOutputTimestampFn;

public ReadFromPulsarDoFn(PulsarIO.Read transform) {
this.extractOutputTimestampFn = transform.getExtractOutputTimestampFn();
this.clientUrl = transform.getClientUrl();
this.adminUrl = transform.getAdminUrl();
this.authPluginClassName = transform.getAuthPluginClassName();
this.authParameters = transform.getAuthParameters();
this.pulsarClientSerializableFunction = transform.getPulsarClient();
}

// Open connection to Pulsar clients
@Setup
// TODO add auth related
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

What is TODO here?

public void initPulsarClients() throws Exception {
if (this.clientUrl == null) {
this.clientUrl = PulsarIOUtils.SERVICE_URL;
Expand All @@ -76,14 +81,19 @@ public void initPulsarClients() throws Exception {
if (this.client == null) {
this.client = pulsarClientSerializableFunction.apply(this.clientUrl);
if (this.client == null) {
this.client = PulsarClient.builder().serviceUrl(clientUrl).build();
this.client =
PulsarClient.builder()
.serviceUrl(clientUrl)
.authentication(authPluginClassName, authParameters)
.build();
}
}

if (this.admin == null) {
this.admin =
PulsarAdmin.builder()
.serviceHttpUrl(adminUrl)
.authentication(authPluginClassName, authParameters)
.tlsTrustCertsFilePath(null)
.allowTlsInsecureConnection(false)
.build();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,15 +31,23 @@ public class WriteToPulsarDoFn extends DoFn<byte[], Void> {
private PulsarClient client;
private String clientUrl;
private String topic;
private String authPluginClassName;
private String authParameters;

WriteToPulsarDoFn(PulsarIO.Write transform) {
this.clientUrl = transform.getClientUrl();
this.topic = transform.getTopic();
this.authPluginClassName = transform.getAuthPluginClassName();
this.authParameters = transform.getAuthParameters();
}

@Setup
public void setup() throws PulsarClientException {
client = PulsarClient.builder().serviceUrl(clientUrl).build();
client =
PulsarClient.builder()
.serviceUrl(clientUrl)
.authentication(authPluginClassName, authParameters)
.build();
producer = client.newProducer().topic(topic).compressionType(CompressionType.LZ4).create();
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -77,7 +77,7 @@ public void testInitialRestrictionWhenHasStartOffset() throws Exception {
OffsetRange result =
dofnInstance.getInitialRestriction(
PulsarSourceDescriptor.of(
TOPIC, expectedStartOffset, null, null, SERVICE_URL, ADMIN_URL));
TOPIC, expectedStartOffset, null, null, SERVICE_URL, ADMIN_URL, null, null));
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Please, add the tests with non-null auth values.

assertEquals(new OffsetRange(expectedStartOffset, Long.MAX_VALUE), result);
}

Expand All @@ -87,7 +87,7 @@ public void testInitialRestrictionWithConsumerPosition() throws Exception {
OffsetRange result =
dofnInstance.getInitialRestriction(
PulsarSourceDescriptor.of(
TOPIC, expectedStartOffset, null, null, SERVICE_URL, ADMIN_URL));
TOPIC, expectedStartOffset, null, null, SERVICE_URL, ADMIN_URL, null, null));
assertEquals(new OffsetRange(expectedStartOffset, Long.MAX_VALUE), result);
}

Expand All @@ -97,7 +97,8 @@ public void testInitialRestrictionWithConsumerEndPosition() throws Exception {
long endOffset = fakePulsarReader.getEndTimestamp();
OffsetRange result =
dofnInstance.getInitialRestriction(
PulsarSourceDescriptor.of(TOPIC, startOffset, endOffset, null, SERVICE_URL, ADMIN_URL));
PulsarSourceDescriptor.of(
TOPIC, startOffset, endOffset, null, SERVICE_URL, ADMIN_URL, null, null));
assertEquals(new OffsetRange(startOffset, endOffset), result);
}

Expand All @@ -108,7 +109,8 @@ public void testProcessElement() throws Exception {
long endOffset = fakePulsarReader.getEndTimestamp();
OffsetRangeTracker tracker = new OffsetRangeTracker(new OffsetRange(startOffset, endOffset));
PulsarSourceDescriptor descriptor =
PulsarSourceDescriptor.of(TOPIC, startOffset, endOffset, null, SERVICE_URL, ADMIN_URL);
PulsarSourceDescriptor.of(
TOPIC, startOffset, endOffset, null, SERVICE_URL, ADMIN_URL, null, null);
DoFn.ProcessContinuation result =
dofnInstance.processElement(descriptor, tracker, null, (DoFn.OutputReceiver) receiver);
int expectedResultWithoutCountingLastOffset = NUMBEROFMESSAGES - 1;
Expand All @@ -123,7 +125,8 @@ public void testProcessElementWhenEndMessageIdIsDefined() throws Exception {
MessageId endMessageId = DefaultImplementation.newMessageId(50L, 50L, 50);
DoFn.ProcessContinuation result =
dofnInstance.processElement(
PulsarSourceDescriptor.of(TOPIC, null, null, endMessageId, SERVICE_URL, ADMIN_URL),
PulsarSourceDescriptor.of(
TOPIC, null, null, endMessageId, SERVICE_URL, ADMIN_URL, null, null),
tracker,
null,
(DoFn.OutputReceiver) receiver);
Expand All @@ -138,7 +141,7 @@ public void testProcessElementWithEmptyRecords() throws Exception {
OffsetRangeTracker tracker = new OffsetRangeTracker(new OffsetRange(0L, Long.MAX_VALUE));
DoFn.ProcessContinuation result =
dofnInstance.processElement(
PulsarSourceDescriptor.of(TOPIC, null, null, null, SERVICE_URL, ADMIN_URL),
PulsarSourceDescriptor.of(TOPIC, null, null, null, SERVICE_URL, ADMIN_URL, null, null),
tracker,
null,
(DoFn.OutputReceiver) receiver);
Expand All @@ -153,7 +156,7 @@ public void testProcessElementWhenHasReachedEndTopic() throws Exception {
OffsetRangeTracker tracker = new OffsetRangeTracker(new OffsetRange(0L, Long.MAX_VALUE));
DoFn.ProcessContinuation result =
dofnInstance.processElement(
PulsarSourceDescriptor.of(TOPIC, null, null, null, SERVICE_URL, ADMIN_URL),
PulsarSourceDescriptor.of(TOPIC, null, null, null, SERVICE_URL, ADMIN_URL, null, null),
tracker,
null,
(DoFn.OutputReceiver) receiver);
Expand Down