Skip to content

Commit 9560fe1

Browse files
authored
SolaceIO: refactor to allow inheritance of BasicAuthSempClient (#32400)
* Refactored to allow inheritance and overriding of BasicAuthSempClient * Fix docs and use Map#computeIfAbsent with a lambda. * Fix integration test * Remove 'serializable' * Revert 'Remove 'serializable''
1 parent 9696201 commit 9560fe1

File tree

6 files changed

+403
-29
lines changed

6 files changed

+403
-29
lines changed

sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/BasicAuthSempClient.java

Lines changed: 7 additions & 24 deletions
Original file line numberDiff line numberDiff line change
@@ -17,14 +17,10 @@
1717
*/
1818
package org.apache.beam.sdk.io.solace.broker;
1919

20-
import com.fasterxml.jackson.core.JsonProcessingException;
21-
import com.fasterxml.jackson.databind.DeserializationFeature;
22-
import com.fasterxml.jackson.databind.ObjectMapper;
2320
import com.google.api.client.http.HttpRequestFactory;
2421
import com.solacesystems.jcsmp.JCSMPFactory;
2522
import java.io.IOException;
2623
import org.apache.beam.sdk.annotations.Internal;
27-
import org.apache.beam.sdk.io.solace.data.Semp.Queue;
2824
import org.apache.beam.sdk.util.SerializableSupplier;
2925
import org.slf4j.Logger;
3026
import org.slf4j.LoggerFactory;
@@ -40,8 +36,6 @@
4036
@Internal
4137
public class BasicAuthSempClient implements SempClient {
4238
private static final Logger LOG = LoggerFactory.getLogger(BasicAuthSempClient.class);
43-
private final ObjectMapper objectMapper =
44-
new ObjectMapper().configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
4539

4640
private final SempBasicAuthClientExecutor sempBasicAuthClientExecutor;
4741

@@ -58,13 +52,12 @@ public BasicAuthSempClient(
5852

5953
@Override
6054
public boolean isQueueNonExclusive(String queueName) throws IOException {
61-
LOG.info("SolaceIO.Read: SempOperations: query SEMP if queue {} is nonExclusive", queueName);
62-
BrokerResponse response = sempBasicAuthClientExecutor.getQueueResponse(queueName);
63-
if (response.content == null) {
64-
throw new IOException("SolaceIO: response from SEMP is empty!");
65-
}
66-
Queue q = mapJsonToClass(response.content, Queue.class);
67-
return q.data().accessType().equals("non-exclusive");
55+
boolean queueNonExclusive = sempBasicAuthClientExecutor.isQueueNonExclusive(queueName);
56+
LOG.info(
57+
"SolaceIO.Read: SempOperations: queried SEMP if queue {} is non-exclusive: {}",
58+
queueName,
59+
queueNonExclusive);
60+
return queueNonExclusive;
6861
}
6962

7063
@Override
@@ -77,12 +70,7 @@ public com.solacesystems.jcsmp.Queue createQueueForTopic(String queueName, Strin
7770

7871
@Override
7972
public long getBacklogBytes(String queueName) throws IOException {
80-
BrokerResponse response = sempBasicAuthClientExecutor.getQueueResponse(queueName);
81-
if (response.content == null) {
82-
throw new IOException("SolaceIO: response from SEMP is empty!");
83-
}
84-
Queue q = mapJsonToClass(response.content, Queue.class);
85-
return q.data().msgSpoolUsage();
73+
return sempBasicAuthClientExecutor.getBacklogBytes(queueName);
8674
}
8775

8876
private void createQueue(String queueName) throws IOException {
@@ -94,9 +82,4 @@ private void createSubscription(String queueName, String topicName) throws IOExc
9482
LOG.info("SolaceIO.Read: Creating new subscription {} for topic {}.", queueName, topicName);
9583
sempBasicAuthClientExecutor.createSubscriptionResponse(queueName, topicName);
9684
}
97-
98-
private <T> T mapJsonToClass(String content, Class<T> mapSuccessToClass)
99-
throws JsonProcessingException {
100-
return objectMapper.readValue(content, mapSuccessToClass);
101-
}
10285
}

sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SempBasicAuthClientExecutor.java

Lines changed: 32 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -19,6 +19,9 @@
1919

2020
import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull;
2121

22+
import com.fasterxml.jackson.core.JsonProcessingException;
23+
import com.fasterxml.jackson.databind.DeserializationFeature;
24+
import com.fasterxml.jackson.databind.ObjectMapper;
2225
import com.google.api.client.http.GenericUrl;
2326
import com.google.api.client.http.HttpContent;
2427
import com.google.api.client.http.HttpHeaders;
@@ -40,6 +43,7 @@
4043
import java.util.Objects;
4144
import java.util.concurrent.ConcurrentHashMap;
4245
import java.util.stream.Collectors;
46+
import org.apache.beam.sdk.io.solace.data.Semp.Queue;
4347
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap;
4448
import org.checkerframework.checker.nullness.qual.Nullable;
4549

@@ -52,7 +56,7 @@
5256
* response is 401 Unauthorized, the client will execute an additional request with Basic Auth
5357
* header to refresh the token.
5458
*/
55-
class SempBasicAuthClientExecutor implements Serializable {
59+
public class SempBasicAuthClientExecutor implements Serializable {
5660
// Every request will be repeated 2 times in case of abnormal connection failures.
5761
private static final int REQUEST_NUM_RETRIES = 2;
5862
private static final Map<CookieManagerKey, CookieManager> COOKIE_MANAGER_MAP =
@@ -65,8 +69,10 @@ class SempBasicAuthClientExecutor implements Serializable {
6569
private final String password;
6670
private final CookieManagerKey cookieManagerKey;
6771
private final transient HttpRequestFactory requestFactory;
72+
private final ObjectMapper objectMapper =
73+
new ObjectMapper().configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
6874

69-
SempBasicAuthClientExecutor(
75+
public SempBasicAuthClientExecutor(
7076
String host,
7177
String username,
7278
String password,
@@ -78,7 +84,16 @@ class SempBasicAuthClientExecutor implements Serializable {
7884
this.password = password;
7985
this.requestFactory = httpRequestFactory;
8086
this.cookieManagerKey = new CookieManagerKey(this.baseUrl, this.username);
81-
COOKIE_MANAGER_MAP.putIfAbsent(this.cookieManagerKey, new CookieManager());
87+
COOKIE_MANAGER_MAP.computeIfAbsent(this.cookieManagerKey, key -> new CookieManager());
88+
}
89+
90+
public boolean isQueueNonExclusive(String queueName) throws IOException {
91+
BrokerResponse response = getQueueResponse(queueName);
92+
if (response.content == null) {
93+
throw new IOException("SolaceIO: response from SEMP is empty!");
94+
}
95+
Queue q = mapJsonToClass(response.content, Queue.class);
96+
return q.data().accessType().equals("non-exclusive");
8297
}
8398

8499
private static String getQueueEndpoint(String messageVpn, String queueName)
@@ -199,6 +214,20 @@ private static String urlEncode(String queueName) throws UnsupportedEncodingExce
199214
return URLEncoder.encode(queueName, StandardCharsets.UTF_8.name());
200215
}
201216

217+
private <T> T mapJsonToClass(String content, Class<T> mapSuccessToClass)
218+
throws JsonProcessingException {
219+
return objectMapper.readValue(content, mapSuccessToClass);
220+
}
221+
222+
public long getBacklogBytes(String queueName) throws IOException {
223+
BrokerResponse response = getQueueResponse(queueName);
224+
if (response.content == null) {
225+
throw new IOException("SolaceIO: response from SEMP is empty!");
226+
}
227+
Queue q = mapJsonToClass(response.content, Queue.class);
228+
return q.data().msgSpoolUsage();
229+
}
230+
202231
private static class CookieManagerKey implements Serializable {
203232
private final String baseUrl;
204233
private final String username;

sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/write/AddShardKeyDoFn.java

Lines changed: 3 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -23,8 +23,9 @@
2323
import org.apache.beam.sdk.values.KV;
2424

2525
/**
26-
* This class a pseudo-key with a given cardinality. The downstream steps will use state {@literal
27-
* &} timers to distribute the data and control for the number of parallel workers used for writing.
26+
* This class adds pseudo-key with a given cardinality. The downstream steps will use state
27+
* {@literal &} timers to distribute the data and control for the number of parallel workers used
28+
* for writing.
2829
*/
2930
@Internal
3031
public class AddShardKeyDoFn extends DoFn<Solace.Record, KV<Integer, Solace.Record>> {
Lines changed: 62 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,62 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one
3+
* or more contributor license agreements. See the NOTICE file
4+
* distributed with this work for additional information
5+
* regarding copyright ownership. The ASF licenses this file
6+
* to you under the Apache License, Version 2.0 (the
7+
* "License"); you may not use this file except in compliance
8+
* with the License. You may obtain a copy of the License at
9+
*
10+
* http://www.apache.org/licenses/LICENSE-2.0
11+
*
12+
* Unless required by applicable law or agreed to in writing, software
13+
* distributed under the License is distributed on an "AS IS" BASIS,
14+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15+
* See the License for the specific language governing permissions and
16+
* limitations under the License.
17+
*/
18+
package org.apache.beam.sdk.io.solace.it;
19+
20+
import com.google.api.client.http.HttpRequestFactory;
21+
import java.io.IOException;
22+
import java.util.List;
23+
import java.util.stream.Collectors;
24+
import org.apache.beam.sdk.io.solace.broker.BasicAuthSempClient;
25+
import org.apache.beam.sdk.io.solace.broker.SempBasicAuthClientExecutor;
26+
import org.apache.beam.sdk.util.SerializableSupplier;
27+
28+
/**
29+
* Example class showing how the {@link BasicAuthSempClient} can be extended or have functionalities
30+
* overridden. In this case, the modified method is {@link
31+
* BasicAuthSempClient#getBacklogBytes(String)}, which queries multiple SEMP endpoints to collect
32+
* accurate backlog metrics. For usage, see {@link SolaceIOMultipleSempIT}.
33+
*/
34+
public class BasicAuthMultipleSempClient extends BasicAuthSempClient {
35+
private final List<SempBasicAuthClientExecutor> sempBacklogBasicAuthClientExecutors;
36+
37+
public BasicAuthMultipleSempClient(
38+
String mainHost,
39+
List<String> backlogHosts,
40+
String username,
41+
String password,
42+
String vpnName,
43+
SerializableSupplier<HttpRequestFactory> httpRequestFactorySupplier) {
44+
super(mainHost, username, password, vpnName, httpRequestFactorySupplier);
45+
sempBacklogBasicAuthClientExecutors =
46+
backlogHosts.stream()
47+
.map(
48+
host ->
49+
new SempBasicAuthClientExecutor(
50+
host, username, password, vpnName, httpRequestFactorySupplier.get()))
51+
.collect(Collectors.toList());
52+
}
53+
54+
@Override
55+
public long getBacklogBytes(String queueName) throws IOException {
56+
long backlog = 0;
57+
for (SempBasicAuthClientExecutor client : sempBacklogBasicAuthClientExecutors) {
58+
backlog += client.getBacklogBytes(queueName);
59+
}
60+
return backlog;
61+
}
62+
}
Lines changed: 92 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,92 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one
3+
* or more contributor license agreements. See the NOTICE file
4+
* distributed with this work for additional information
5+
* regarding copyright ownership. The ASF licenses this file
6+
* to you under the Apache License, Version 2.0 (the
7+
* "License"); you may not use this file except in compliance
8+
* with the License. You may obtain a copy of the License at
9+
*
10+
* http://www.apache.org/licenses/LICENSE-2.0
11+
*
12+
* Unless required by applicable law or agreed to in writing, software
13+
* distributed under the License is distributed on an "AS IS" BASIS,
14+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15+
* See the License for the specific language governing permissions and
16+
* limitations under the License.
17+
*/
18+
package org.apache.beam.sdk.io.solace.it;
19+
20+
import com.google.api.client.http.HttpRequestFactory;
21+
import com.google.api.client.http.javanet.NetHttpTransport;
22+
import com.google.auto.value.AutoValue;
23+
import java.util.List;
24+
import org.apache.beam.sdk.io.solace.broker.SempClient;
25+
import org.apache.beam.sdk.io.solace.broker.SempClientFactory;
26+
import org.apache.beam.sdk.util.SerializableSupplier;
27+
import org.checkerframework.checker.nullness.qual.NonNull;
28+
import org.checkerframework.checker.nullness.qual.Nullable;
29+
30+
/**
31+
* Example class showing how to implement a custom {@link SempClientFactory} with custom client. For
32+
* usage, see {@link SolaceIOMultipleSempIT}.
33+
*/
34+
@AutoValue
35+
public abstract class BasicAuthMultipleSempClientFactory implements SempClientFactory {
36+
37+
public abstract String mainHost();
38+
39+
public abstract List<String> backlogHosts();
40+
41+
public abstract String username();
42+
43+
public abstract String password();
44+
45+
public abstract String vpnName();
46+
47+
public abstract @Nullable SerializableSupplier<HttpRequestFactory> httpRequestFactorySupplier();
48+
49+
public static Builder builder() {
50+
return new AutoValue_BasicAuthMultipleSempClientFactory.Builder();
51+
}
52+
53+
@AutoValue.Builder
54+
public abstract static class Builder {
55+
/** Set Solace host, format: [Protocol://]Host[:Port]. */
56+
public abstract Builder mainHost(String host);
57+
58+
public abstract Builder backlogHosts(List<String> hosts);
59+
60+
/** Set Solace username. */
61+
public abstract Builder username(String username);
62+
/** Set Solace password. */
63+
public abstract Builder password(String password);
64+
65+
/** Set Solace vpn name. */
66+
public abstract Builder vpnName(String vpnName);
67+
68+
abstract Builder httpRequestFactorySupplier(
69+
SerializableSupplier<HttpRequestFactory> httpRequestFactorySupplier);
70+
71+
public abstract BasicAuthMultipleSempClientFactory build();
72+
}
73+
74+
@Override
75+
public SempClient create() {
76+
return new BasicAuthMultipleSempClient(
77+
mainHost(),
78+
backlogHosts(),
79+
username(),
80+
password(),
81+
vpnName(),
82+
getHttpRequestFactorySupplier());
83+
}
84+
85+
@SuppressWarnings("return")
86+
private @NonNull SerializableSupplier<HttpRequestFactory> getHttpRequestFactorySupplier() {
87+
SerializableSupplier<HttpRequestFactory> httpRequestSupplier = httpRequestFactorySupplier();
88+
return httpRequestSupplier != null
89+
? httpRequestSupplier
90+
: () -> new NetHttpTransport().createRequestFactory();
91+
}
92+
}

0 commit comments

Comments
 (0)