From f404eecc5544aadc2e298767a7ec8679d8ae0027 Mon Sep 17 00:00:00 2001 From: bzablocki Date: Tue, 16 Apr 2024 12:12:23 +0200 Subject: [PATCH 01/41] wip solace connector --- sdks/java/io/solace/build.gradle | 45 + .../sdk/io/solace/RetryCallableManager.java | 113 ++ .../sdk/io/solace/SerializableSupplier.java | 23 + .../apache/beam/sdk/io/solace/SolaceIO.java | 979 ++++++++++++++++++ .../broker/BasicAuthJcsmpSessionService.java | 111 ++ .../BasicAuthJcsmpSessionServiceFactory.java | 75 ++ .../io/solace/broker/BasicAuthSempClient.java | 251 +++++ .../broker/BasicAuthSempClientFactory.java | 123 +++ .../sdk/io/solace/broker/BrokerResponse.java | 63 ++ .../sdk/io/solace/broker/MessageReceiver.java | 35 + .../broker/SempBasicAuthClientExecutor.java | 170 +++ .../beam/sdk/io/solace/broker/SempClient.java | 29 + .../io/solace/broker/SempClientFactory.java | 22 + .../sdk/io/solace/broker/SessionService.java | 28 + .../solace/broker/SessionServiceFactory.java | 29 + .../solace/broker/SolaceMessageReceiver.java | 59 ++ .../beam/sdk/io/solace/data/Solace.java | 340 ++++++ .../sdk/io/solace/data/SolaceRecordCoder.java | 93 ++ .../io/solace/read/SolaceCheckpointMark.java | 90 ++ .../io/solace/read/UnboundedSolaceReader.java | 181 ++++ .../io/solace/read/UnboundedSolaceSource.java | 148 +++ .../io/solace/read/WatermarkParameters.java | 88 ++ .../sdk/io/solace/read/WatermarkPolicy.java | 66 ++ .../beam/sdk/io/solace/MockSempClient.java | 86 ++ .../sdk/io/solace/MockSempClientFactory.java | 33 + .../sdk/io/solace/MockSessionService.java | 87 ++ .../io/solace/MockSessionServiceFactory.java | 32 + .../io/solace/RetryCallableManagerTest.java | 175 ++++ .../beam/sdk/io/solace/SolaceDataUtils.java | 774 ++++++++++++++ .../beam/sdk/io/solace/SolaceIOTest.java | 607 +++++++++++ .../SempBasicAuthClientExecutorTest.java | 174 ++++ .../beam/sdk/io/solace/data/SolaceTest.java | 217 ++++ .../io/solace/it/SolaceContainerManager.java | 166 +++ .../beam/sdk/io/solace/it/SolaceIOIT.java | 273 +++++ 34 files changed, 5785 insertions(+) create mode 100644 sdks/java/io/solace/build.gradle create mode 100644 sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/RetryCallableManager.java create mode 100644 sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/SerializableSupplier.java create mode 100644 sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/SolaceIO.java create mode 100644 sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/BasicAuthJcsmpSessionService.java create mode 100644 sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/BasicAuthJcsmpSessionServiceFactory.java create mode 100644 sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/BasicAuthSempClient.java create mode 100644 sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/BasicAuthSempClientFactory.java create mode 100644 sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/BrokerResponse.java create mode 100644 sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/MessageReceiver.java create mode 100644 sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SempBasicAuthClientExecutor.java create mode 100644 sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SempClient.java create mode 100644 sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SempClientFactory.java create mode 100644 sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SessionService.java create mode 100644 sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SessionServiceFactory.java create mode 100644 sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SolaceMessageReceiver.java create mode 100644 sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/data/Solace.java create mode 100644 sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/data/SolaceRecordCoder.java create mode 100644 sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/SolaceCheckpointMark.java create mode 100644 sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/UnboundedSolaceReader.java create mode 100644 sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/UnboundedSolaceSource.java create mode 100644 sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/WatermarkParameters.java create mode 100644 sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/WatermarkPolicy.java create mode 100644 sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/MockSempClient.java create mode 100644 sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/MockSempClientFactory.java create mode 100644 sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/MockSessionService.java create mode 100644 sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/MockSessionServiceFactory.java create mode 100644 sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/RetryCallableManagerTest.java create mode 100644 sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/SolaceDataUtils.java create mode 100644 sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/SolaceIOTest.java create mode 100644 sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/broker/SempBasicAuthClientExecutorTest.java create mode 100644 sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/data/SolaceTest.java create mode 100644 sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/it/SolaceContainerManager.java create mode 100644 sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/it/SolaceIOIT.java diff --git a/sdks/java/io/solace/build.gradle b/sdks/java/io/solace/build.gradle new file mode 100644 index 000000000000..4814d82358e4 --- /dev/null +++ b/sdks/java/io/solace/build.gradle @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * License); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an AS IS BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +plugins { id 'org.apache.beam.module' } +applyJavaNature( + automaticModuleName: 'org.apache.beam.sdk.io.solace', +) +provideIntegrationTestingDependencies() +enableJavaPerformanceTesting() + +description = "Apache Beam :: SDKs :: Java :: IO :: Solace" +ext.summary = """IO to read and write to Solace destinations (queues and topics).""" + +dependencies { + implementation library.java.vendored_guava_32_1_2_jre + implementation project(path: ":sdks:java:core", configuration: "shadow") + implementation library.java.slf4j_api + implementation library.java.joda_time + testImplementation library.java.activemq_amqp + testImplementation library.java.activemq_broker + testImplementation library.java.activemq_jaas + testImplementation library.java.activemq_kahadb_store + testImplementation library.java.activemq_client + testImplementation library.java.hamcrest + testImplementation library.java.junit + testImplementation project(path: ":sdks:java:io:common", configuration: "testRuntimeMigration") + testImplementation project(path: ":sdks:java:testing:test-utils", configuration: "testRuntimeMigration") + testRuntimeOnly library.java.slf4j_jdk14 + testRuntimeOnly project(path: ":runners:direct-java", configuration: "shadow") +} diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/RetryCallableManager.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/RetryCallableManager.java new file mode 100644 index 000000000000..ceea3c597986 --- /dev/null +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/RetryCallableManager.java @@ -0,0 +1,113 @@ +/* + * Copyright 2024 Google. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.google.cloud.dataflow.dce.io.solace; + +import com.google.api.core.NanoClock; +import com.google.api.gax.retrying.RetrySettings; +import com.google.auto.value.AutoValue; +import com.google.cloud.ExceptionHandler; +import com.google.cloud.ExceptionHandler.Interceptor; +import com.google.cloud.RetryHelper; +import java.io.Serializable; +import java.util.Set; +import java.util.concurrent.Callable; +import org.apache.beam.sdk.annotations.Internal; + +@Internal +@AutoValue +public abstract class RetryCallableManager implements Serializable { + + private static final int NUMBER_OF_RETRIES = 4; + private static final int RETRY_INTERVAL_SECONDS = 1; + private static final int RETRY_MULTIPLIER = 2; + private static final int MAX_DELAY = + NUMBER_OF_RETRIES * RETRY_MULTIPLIER * RETRY_INTERVAL_SECONDS + 1; + + public static RetryCallableManager create() { + return builder().build(); + } + /** + * Method that executes and repeats the execution of the callable argument, if it throws one of + * the exceptions from the exceptionsToIntercept Set. + */ + public V retryCallable( + Callable callable, Set> exceptionsToIntercept) { + return RetryHelper.runWithRetries( + callable, + getRetrySettings(), + getExceptionHandlerForExceptions(exceptionsToIntercept), + NanoClock.getDefaultClock()); + } + + private ExceptionHandler getExceptionHandlerForExceptions( + Set> exceptionsToIntercept) { + return ExceptionHandler.newBuilder() + .abortOn(RuntimeException.class) + .addInterceptors(new ExceptionSetInterceptor(Set.copyOf(exceptionsToIntercept))) + .build(); + } + + abstract RetrySettings getRetrySettings(); + + abstract Builder toBuilder(); + + static Builder builder() { + return new AutoValue_RetryCallableManager.Builder() + .setRetrySettings( + RetrySettings.newBuilder() + .setInitialRetryDelay( + org.threeten.bp.Duration.ofSeconds(RETRY_INTERVAL_SECONDS)) + .setMaxAttempts(NUMBER_OF_RETRIES) + .setMaxRetryDelay(org.threeten.bp.Duration.ofSeconds(MAX_DELAY)) + .setRetryDelayMultiplier(RETRY_MULTIPLIER) + .build()); + } + + @AutoValue.Builder + abstract static class Builder { + abstract Builder setRetrySettings(RetrySettings retrySettings); + + abstract RetryCallableManager build(); + } + + private static class ExceptionSetInterceptor implements Interceptor { + private static final long serialVersionUID = -8429573586820467828L; + private final Set> exceptionsToIntercept; + + public ExceptionSetInterceptor(Set> exceptionsToIntercept) { + this.exceptionsToIntercept = exceptionsToIntercept; + } + + @Override + public RetryResult afterEval(Exception exception, RetryResult retryResult) { + return Interceptor.RetryResult.CONTINUE_EVALUATION; + } + + @Override + public RetryResult beforeEval(Exception exceptionToEvaluate) { + for (Class exceptionToIntercept : exceptionsToIntercept) { + if (isOf(exceptionToIntercept, exceptionToEvaluate)) { + return Interceptor.RetryResult.RETRY; + } + } + return Interceptor.RetryResult.CONTINUE_EVALUATION; + } + + private boolean isOf(Class clazz, Object obj) { + return clazz.isInstance(obj); + } + } +} diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/SerializableSupplier.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/SerializableSupplier.java new file mode 100644 index 000000000000..249598d8cf53 --- /dev/null +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/SerializableSupplier.java @@ -0,0 +1,23 @@ +/* + * Copyright 2024 Google. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.google.cloud.dataflow.dce.io.solace; + +import java.io.Serializable; + +@FunctionalInterface +public interface SerializableSupplier extends Serializable { + OutputT get(); +} diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/SolaceIO.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/SolaceIO.java new file mode 100644 index 000000000000..17e778605a25 --- /dev/null +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/SolaceIO.java @@ -0,0 +1,979 @@ +/* + * Copyright 2023 Google. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.google.cloud.dataflow.dce.io.solace; + +import com.google.auto.value.AutoValue; +import com.google.cloud.dataflow.dce.io.solace.broker.SempClientFactory; +import com.google.cloud.dataflow.dce.io.solace.broker.SessionService; +import com.google.cloud.dataflow.dce.io.solace.broker.SessionServiceFactory; +import com.google.cloud.dataflow.dce.io.solace.data.Solace; +import com.google.cloud.dataflow.dce.io.solace.data.Solace.SolaceRecordMapper; +import com.google.cloud.dataflow.dce.io.solace.data.SolaceRecordCoder; +import com.google.cloud.dataflow.dce.io.solace.read.UnboundedSolaceSource; +import com.google.cloud.dataflow.dce.io.solace.write.SolacePublishResult; +import com.google.cloud.dataflow.dce.io.solace.write.UnboundedBatchedSolaceWriter; +import com.google.cloud.dataflow.dce.io.solace.write.UnboundedSolaceWriter; +import com.google.cloud.dataflow.dce.io.solace.write.UnboundedStreamingSolaceWriter; +import com.google.cloud.dataflow.dce.io.solace.write.properties.BasicAuthenticationProvider; +import com.google.cloud.dataflow.dce.io.solace.write.properties.GoogleCloudSecretProvider; +import com.google.cloud.dataflow.dce.io.solace.write.properties.SessionPropertiesProvider; +import com.google.common.base.Preconditions; +import com.solacesystems.jcsmp.BytesXMLMessage; +import com.solacesystems.jcsmp.DeliveryMode; +import com.solacesystems.jcsmp.Destination; +import com.solacesystems.jcsmp.JCSMPFactory; +import com.solacesystems.jcsmp.Queue; +import com.solacesystems.jcsmp.Topic; +import java.io.IOException; +import java.util.Objects; +import org.apache.arrow.util.VisibleForTesting; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.coders.CannotProvideCoderException; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.schemas.NoSuchSchemaException; +import org.apache.beam.sdk.transforms.MapElements; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.transforms.SerializableFunction; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.transforms.windowing.GlobalWindows; +import org.apache.beam.sdk.transforms.windowing.Window; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PBegin; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionTuple; +import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.TupleTagList; +import org.apache.beam.sdk.values.TypeDescriptor; +import org.apache.beam.sdk.values.WindowingStrategy; +import org.checkerframework.checker.nullness.qual.Nullable; +import org.joda.time.Instant; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * A {@link PTransform} to read and write from/to Solace. + * + *

Authentication

+ * + * TODO: Auth for the read connector + * + *

The writer connector uses a JCSMP session properties provider, where all the authentication + * details must be set. See {@link Write#withSessionPropertiesProvider(SessionPropertiesProvider)}. + * For convenience, the connector provides a provider for basic authentication ({@link + * BasicAuthenticationProvider}) and another one to retrieve the password from Google Cloud Secret + * Manager ({@link GoogleCloudSecretProvider} + * + *

Reading

+ * + * TODO + * + *

Writing

+ * + * To write to Solace, use {@link #write()}. The default VPN name is "default". + * + *

Coders and schemas support + * + *

The connector expects a {@link Solace.Record} as input, so you need to transform your data to + * this format before writing. {@link Solace.Record} uses Beam schemas by default, but you can + * override and set {@link SolaceRecordCoder} as custom coder if you prefer. + * + *

Writing to a topic of a queue + * + *

The connector uses the Solace JCSMP API. + * The clients will write to a SMF + * topic to the port 55555 of the host. If you want to use a different port, specify it in the + * host property with the format "X.X.X.X:PORT". + * + *

Once you have a {@link PCollection} of {@link Solace.Record}, you can write to Solace using + * + *

{@code
+ * PCollection solaceRecs = ...;
+ *
+ * PCollection results =
+ *         solaceRecs.apply(
+ *                 "Write to Solace",
+ *                 SolaceIO.write()
+ *                         .to(SolaceIO.topicFromName("some-topic"))
+ *                         .withSessionProvider(
+ *                            BasicAuthenticationProvider.builder()
+ *                              .username("username")
+ *                              .password("password")
+ *                              .host("host:port")
+ *                              .build()));
+ * }
+ * + * The above code snippet will write to the VPN named "default", using 4 clients per worker (VM in + * Dataflow), and a maximum of 20 workers/VMs for writing (default values). You can change the + * default VPN name by setting the required JCSMP property in the session properties provider (in + * this case, with {@link BasicAuthenticationProvider#vpnName()}), the number of clients per worker + * with {@link Write#withNumberOfClientsPerWorker(int)} and the number of parallel write clients + * using {@link Write#withMaxNumOfUsedWorkers(int)}. + * + *

Direct and persistent messages, and latency metrics

+ * + *

The connector can write either direct or persistent messages. The default mode is DIRECT. + * + *

The connector returns a {@link PCollection} of {@link Solace.PublishResult}, that you can use + * to get a confirmation of messages that have been published, or rejected, but only if it is + * publishing persistent messages. + * + *

If you are publishing persistent messages, then you can have some feedback about whether the + * messages have been published, and some publishing latency metrics. If the message has been + * published, {@link Solace.PublishResult#getPublished()} will be true. If it is false, it means + * that the message could not be published, and {@link Solace.PublishResult#getError()} will contain + * more details about why the message could not be published. To get latency metrics as well as the + * results, set the property {@link Write#publishLatencyMetrics()}. + * + *

Throughput and latency + * + *

This connector can work in two main modes: high latency or high throughput. The default mode + * favors high throughput over high latency. You can control this behavior with the methods {@link + * Write#withSubmissionMode(SubmissionMode)} and {@link Write#withWriterType(WriterType)}. + * + *

The default mode works like the following options: + * + *

{@code
+ * PCollection solaceRecs = ...;
+ *
+ * PCollection results =
+ *         solaceRecs.apply(
+ *                 "Write to Solace",
+ *                 SolaceIO.write()
+ *                         .to(SolaceIO.topicFromName("some-topic"))
+ *                         .withSessionProvider(
+ *                            BasicAuthenticationProvider.builder()
+ *                              .username("username")
+ *                              .password("password")
+ *                              .host("host:port")
+ *                              .build())
+ *                         .withSubmissionMode(SubmissionMode.HIGHER_THROUGHPUT)
+ *                         .withWriterType(WriterType.BATCHED));
+ * }
+ * + *

{@link SubmissionMode#HIGHER_THROUGHPUT} and {@link WriterType#BATCHED} are the default + * values, and offer the higher possible throughput, and the lowest usage of resources in the runner + * side (due to the lower backpressure). + * + *

This connector writes bundles of 50 messages, using a bulk publish JCSMP method. This will + * increase the latency, since a message needs to "wait" until 50 messages are accumulated, before + * they are submitted to Solace. + * + *

For the lowest latency possible, use {@link SubmissionMode#LOWER_LATENCY} and {@link + * WriterType#STREAMING}. + * + *

{@code
+ * PCollection results =
+ *         solaceRecs.apply(
+ *                 "Write to Solace",
+ *                 SolaceIO.write()
+ *                         .to(SolaceIO.topicFromName("some-topic"))
+ *                         .withSessionProvider(
+ *                            BasicAuthenticationProvider.builder()
+ *                              .username("username")
+ *                              .password("password")
+ *                              .host("host:port")
+ *                              .build())
+ *                         .withSubmissionMode(SubmissionMode.LOWER_LATENCY)
+ *                         .withWriterType(WriterType.STREAMING));
+ * }
+ * + * The streaming connector publishes each message individually, without holding up or batching + * before the message is sent to Solace. This will ensure the lowest possible latency, but it will + * offer a much lower throughput. The streaming connector does not use state & timers. + * + *

Both connectors uses state & timers to control the level of parallelism. If you are using + * Cloud Dataflow, it is recommended that you enable Streaming Engine to use this + * connector. + * + *

Connector retries + * + *

When the worker using the connector is created, the connector will attempt to connect to + * Solace. + * + *

If the client cannot connect to Solace for whatever reason, the connector will retry the + * connections using the following strategy. There will be a maximum of 4 retries. The first retry + * will be attempted 1 second after the first connection attempt. Every subsequent retry will + * multiply that time by a factor of two, with a maximum of 10 seconds. + * + *

If after those retries the client is still unable to connect to Solace, the connector will + * attempt to reconnect using the same strategy repeated for every single incoming message. If for + * some reason, there is a persistent issue that prevents the connection (e.g. client quota + * exhausted), you will need to stop your job manually, or the connector will keep retrying. + * + *

This strategy is applied to all the remote calls sent to Solace, either to connect, pull + * messages, push messages, etc. + */ +public class SolaceIO { + + public static final Logger LOG = LoggerFactory.getLogger(SolaceIO.class); + public static final SerializableFunction SENDER_TIMESTAMP_FUNCTION = + (record) -> new Instant(record.getSenderTimestamp()); + public static final int DEFAULT_MAX_NUMBER_OF_WORKERS = 20; + public static final int DEFAULT_CLIENTS_PER_WORKER = 4; + public static final Boolean DEFAULT_PUBLISH_LATENCY_METRICS = false; + public static final SubmissionMode DEFAULT_SUBMISSION_MODE = SubmissionMode.HIGHER_THROUGHPUT; + public static final DeliveryMode DEFAULT_DELIVERY_MODE = DeliveryMode.DIRECT; + public static final WriterType DEFAULT_WRITER_TYPE = WriterType.BATCHED; + private static final boolean DEFAULT_DEDUPLICATE_RECORDS = true; + + public enum SubmissionMode { + HIGHER_THROUGHPUT, + LOWER_LATENCY + } + + public enum WriterType { + STREAMING, + BATCHED + } + + /** Get a {@link Topic} object from the topic name. */ + static Topic topicFromName(String topicName) { + return JCSMPFactory.onlyInstance().createTopic(topicName); + } + + /** Get a {@link Queue} object from the queue name. */ + static Queue queueFromName(String queueName) { + return JCSMPFactory.onlyInstance().createQueue(queueName); + } + + /** + * Convert to a JCSMP destination from a schema-enabled {@link + * com.google.cloud.dataflow.dce.io.solace.data.Solace.Destination}. + * + *

This method returns a {@link Destination}, which may be either a {@link Topic} or a {@link + * Queue} + */ + public static Destination convertToJcsmpDestination(Solace.Destination destination) { + if (destination.getType().equals(Solace.DestinationType.TOPIC)) { + return topicFromName(destination.getName()); + } else if (destination.getType().equals(Solace.DestinationType.QUEUE)) { + return queueFromName(destination.getName()); + } else { + throw new IllegalArgumentException( + "SolaceIO.Write: Unknown destination type: " + destination.getType()); + } + } + + /** Create a {@link Write} transform, to write to Solace with a custom type. */ + public static Write write() { + return Write.builder().build(); + } + + /** Create a {@link Write} transform, to write to Solace using {@link Solace.Record} objects. */ + public static Write writeSolaceRecords() { + return Write.builder().build(); + } + + /** + * Create a {@link Read} transform, to read from Solace. The ingested records will be mapped to + * the {@link Solace.Record} objects. + */ + public static Read read() { + return Read.builder() + .setTypeDescriptor(TypeDescriptor.of(Solace.Record.class)) + .setParseFn(SolaceRecordMapper::map) + .setTimestampFn(SENDER_TIMESTAMP_FUNCTION) + .build(); + } + /** + * Create a {@link Read} transform, to read from Solace. Specify a {@link SerializableFunction} + * to map incoming {@link BytesXMLMessage} records, to the object of your choice. You also need + * to specify a {@link TypeDescriptor} for your class and the timestamp function which returns + * an {@link Instant} from the record. + * + *

The type descriptor will be used to infer a coder from CoderRegistry or Schema Registry. + * You can initialize a new TypeDescriptor in the following manner: + * + *

{@code
+     *   TypeDescriptor typeDescriptor = TypeDescriptor.of(YourOutputType.class);
+     * }
+     */
+    public static  Read read(
+            TypeDescriptor typeDescriptor,
+            SerializableFunction parseFn,
+            SerializableFunction timestampFn) {
+        return Read.builder()
+                .setTypeDescriptor(typeDescriptor)
+                .setParseFn(parseFn)
+                .setTimestampFn(timestampFn)
+                .build();
+    }
+
+    @AutoValue
+    public abstract static class Write extends PTransform, SolacePublishResult> {
+
+        private static final Logger LOG = LoggerFactory.getLogger(Write.class);
+
+        public static final TupleTag FAILED_PUBLISH_TAG = new TupleTag<>() {};
+        public static final TupleTag SUCCESSFUL_PUBLISH_TAG =
+                new TupleTag<>() {};
+
+        /**
+         * Write to a Solace topic.
+         *
+         * 

The topic does not need to exist before launching the pipeline. + * + *

This will write all records to the same topic, ignoring their destination field. + * + *

Optional. If not specified, the connector will use dynamic destinations based on the + * destination field of {@link Solace.Record}. + */ + public Write to(Solace.Topic topic) { + return toBuilder().setDestination(topicFromName(topic.getName())).build(); + } + + /** + * Write to a Solace queue. + * + *

The queue must exist prior to launching the pipeline. + * + *

This will write all records to the same queue, ignoring their destination field. + * + *

Optional. If not specified, the connector will use dynamic destinations based on the + * destination field of {@link Solace.Record}. + */ + public Write to(Solace.Queue queue) { + return toBuilder().setDestination(queueFromName(queue.getName())).build(); + } + + /** + * The number of workers used by the job to write to Solace. + * + *

This is optional, the default value is 20. + * + *

This is the maximum value that the job would use, but depending on the amount of data, + * the actual number of writers may be lower than this value. With the Dataflow runner, the + * connector will as maximum this number of VMs in the job (but the job itself may use more + * VMs). + * + *

Set this number taking into account the limits in the number of clients in your Solace + * cluster, and the need for performance when writing to Solace (more workers will achieve + * higher throughput). + */ + public Write withMaxNumOfUsedWorkers(int maxNumOfUsedWorkers) { + return toBuilder().setMaxNumOfUsedWorkers(maxNumOfUsedWorkers).build(); + } + + /** + * The number of clients that each worker will create. + * + *

This is optional, the default number is 4. + * + *

The number of clients is per worker. If there are more than one worker, the number of + * clients will be multiplied by the number of workers. With the Dataflow runner, this will + * be the number of clients created per VM. The clients will be re-used across different + * threads in the same worker. + * + *

Set this number in combination with {@link #withMaxNumOfUsedWorkers}, to ensure that + * the limit for number of clients in your Solace cluster is not exceeded. + * + *

Normally, using a higher number of clients with fewer workers will achieve better + * throughput at a lower cost, since the workers are better utilized. A good rule of thumb + * to use is setting as many clients per worker as vCPUs the worker has. + */ + public Write withNumberOfClientsPerWorker(int numberOfClientsPerWorker) { + return toBuilder().setNumberOfClientsPerWorker(numberOfClientsPerWorker).build(); + } + + /** + * Set the delivery mode. This is optional, the default value is DIRECT. + * + *

For more details, see https://docs.solace.com/API/API-Developer-Guide/Message-Delivery-Modes.htm + */ + public Write withDeliveryMode(DeliveryMode deliveryMode) { + return toBuilder().setDeliveryMode(deliveryMode).build(); + } + + /** + * Publish latency metrics using Beam metrics. + * + *

Latency metrics are only available if {@link #withDeliveryMode(DeliveryMode)} is set + * to PERSISTENT. In that mode, latency is measured for each single message, as the time + * difference between the message creation and the reception of the publishing confirmation. + * + *

For the batched writer, the creation time is set for every message in a batch shortly + * before the batch is submitted. So the latency is very close to the actual publishing + * latency, and it does not take into account the time spent waiting for the batch to be + * submitted. + * + *

This is optional, the default value is false (don't publish latency metrics). + */ + public Write publishLatencyMetrics() { + return toBuilder().setPublishLatencyMetrics(true).build(); + } + + /** + * This setting controls the JCSMP property MESSAGE_CALLBACK_ON_REACTOR. Optional. + * + *

For full details, please check https://docs.solace.com/API/API-Developer-Guide/Java-API-Best-Practices.htm. + * + *

The Solace JCSMP client libraries can dispatch messages using two different modes: + * + *

One of the modes dispatches messages directly from the same thread that is doing the + * rest of I/O work. This mode favors lower latency but lower throughput. Set this to + * LOWER_LATENCY to use that mode (MESSAGE_CALLBACK_ON_REACTOR set to True). + * + *

The other mode uses a parallel thread to accumulate and dispatch messages. This mode + * favors higher throughput but also has higher latency. Set this to HIGHER_THROUGHPUT to + * use that mode. This is the default mode (MESSAGE_CALLBACK_ON_REACTOR set to False). + * + *

This is optional, the default value is HIGHER_THROUGHPUT. + */ + public Write withSubmissionMode(SubmissionMode submissionMode) { + return toBuilder().setDispatchMode(submissionMode).build(); + } + + /** + * Set the type of writer used by the connector. Optional. + * + *

The Solace writer can either use the JCSMP modes in streaming or batched. + * + *

In streaming mode, the publishing latency will be lower, but the throughput will also + * be lower. + * + *

With the batched mode, messages are accumulated until a batch size of 50 is reached, + * or 5 seconds have elapsed since the first message in the batch was received. The 50 + * messages are sent to Solace in a single batch. This writer offers higher throughput but + * higher publishing latency, as messages can be held up for up to 5 seconds until they are + * published. + * + *

Notice that this is the message publishing latency, not the end-to-end latency. For + * very large scale pipelines, you will probably prefer to use the HIGHER_THROUGHPUT mode, + * as with lower throughput messages will accumulate in the pipeline, and the end-to-end + * latency may actually be higher. + * + *

This is optional, the default is the BATCHED writer. + */ + public Write withWriterType(WriterType writerType) { + return toBuilder().setWriterType(writerType).build(); + } + + /** + * Set the format function for your custom data type, and/or for dynamic destinations. + * + *

If you are using a custom data class, this function should return a {@link + * Solace.Record} corresponding to your custom data class instance. + * + *

If you are using this formatting function with dynamic destinations, you must ensure + * that you set the right value in the destination value of the {@link Solace.Record} + * messages. + * + *

In any other case, this format function is optional. + */ + public Write withFormatFunction(SerializableFunction formatFunction) { + return toBuilder().setFormatFunction(formatFunction).build(); + } + + /** + * Set the provider used to obtain the properties to initialize a new session in the broker. + * + *

This provider should define the destination host where the broker is listening, and + * all the properties related to authentication (base auth, client certificate, etc.). + */ + public Write withSessionPropertiesProvider(SessionPropertiesProvider provider) { + return toBuilder().setSessionPropertiesProvider(provider).build(); + } + + abstract int getMaxNumOfUsedWorkers(); + + abstract int getNumberOfClientsPerWorker(); + + abstract @Nullable Destination getDestination(); + + abstract DeliveryMode getDeliveryMode(); + + abstract boolean getPublishLatencyMetrics(); + + abstract SubmissionMode getDispatchMode(); + + abstract WriterType getWriterType(); + + abstract @Nullable SerializableFunction getFormatFunction(); + + abstract @Nullable SessionPropertiesProvider getSessionPropertiesProvider(); + + static Builder builder() { + return new AutoValue_SolaceIO_Write.Builder() + .setDeliveryMode(DEFAULT_DELIVERY_MODE) + .setMaxNumOfUsedWorkers(DEFAULT_MAX_NUMBER_OF_WORKERS) + .setNumberOfClientsPerWorker(DEFAULT_CLIENTS_PER_WORKER) + .setPublishLatencyMetrics(DEFAULT_PUBLISH_LATENCY_METRICS) + .setDispatchMode(DEFAULT_SUBMISSION_MODE) + .setWriterType(DEFAULT_WRITER_TYPE); + } + + abstract Builder toBuilder(); + + @AutoValue.Builder + abstract static class Builder { + abstract Builder setMaxNumOfUsedWorkers(int maxNumOfUsedWorkers); + + abstract Builder setNumberOfClientsPerWorker(int numberOfClientsPerWorker); + + abstract Builder setDestination(Destination topicOrQueue); + + abstract Builder setDeliveryMode(DeliveryMode deliveryMode); + + abstract Builder setPublishLatencyMetrics(Boolean publishLatencyMetrics); + + abstract Builder setDispatchMode(SubmissionMode submissionMode); + + abstract Builder setWriterType(WriterType writerType); + + abstract Builder setFormatFunction( + SerializableFunction formatFunction); + + abstract Builder setSessionPropertiesProvider( + SessionPropertiesProvider propertiesProvider); + + abstract Write build(); + } + + @Override + public SolacePublishResult expand(PCollection input) { + Class pcollClass = input.getTypeDescriptor().getRawType(); + boolean usingSolaceRecord = pcollClass.isAssignableFrom(Solace.Record.class); + // todo definitely don't merge. Seems like a bug! + // usingSolaceRecord = true; + // System.out.println(pcollClass + " " + + // pcollClass.isAssignableFrom(Solace.Record.class) ); + validate(usingSolaceRecord); + + // Register schema for subtypes needed by Record + + boolean usingDynamicDestinations = getDestination() == null; + SerializableFunction destinationFn; + if (usingDynamicDestinations) { + destinationFn = x -> SolaceIO.convertToJcsmpDestination(x.getDestination()); + } else { + // Constant destination for all messages (same topic or queue) + destinationFn = x -> getDestination(); + } + + @SuppressWarnings("unchecked") + PCollection records = + getFormatFunction() == null + ? (PCollection) input + : input.apply( + "Format records", + MapElements.into(TypeDescriptor.of(Solace.Record.class)) + .via(getFormatFunction())); + + // Store the current window used by the input + PCollection captureWindow = + records.apply( + "Capture window", + ParDo.of(new UnboundedSolaceWriter.RecordToPublishResultDoFn())); + + @SuppressWarnings("unchecked") + WindowingStrategy windowingStrategy = + (WindowingStrategy) + captureWindow.getWindowingStrategy(); + + PCollection> withShardKeys = + records.apply( + "Add shard key", + ParDo.of( + new UnboundedSolaceWriter.AddShardKeyDoFn( + getMaxNumOfUsedWorkers()))); + + PCollection> withGlobalWindow = + withShardKeys.apply("Global window", Window.into(new GlobalWindows())); + + String label = + getWriterType() == WriterType.STREAMING + ? "Publish (streaming)" + : "Publish (batched)"; + + PCollectionTuple solaceOutput = + withGlobalWindow.apply(label, getWriterTransform(destinationFn)); + + SolacePublishResult output; + if (getDeliveryMode() == DeliveryMode.PERSISTENT) { + PCollection failedPublish = + solaceOutput.get(FAILED_PUBLISH_TAG); + PCollection successfulPublish = + solaceOutput.get(SUCCESSFUL_PUBLISH_TAG); + output = + rewindow( + SolacePublishResult.in( + input.getPipeline(), failedPublish, successfulPublish), + windowingStrategy); + } else { + LOG.info( + String.format( + "Solace.Write: omitting writer output because delivery mode is %s", + getDeliveryMode())); + output = SolacePublishResult.in(input.getPipeline(), null, null); + } + + return output; + } + + private ParDo.MultiOutput, Solace.PublishResult> + getWriterTransform(SerializableFunction destinationFn) { + + ParDo.SingleOutput, Solace.PublishResult> writer = + ParDo.of( + getWriterType() == WriterType.STREAMING + ? new UnboundedStreamingSolaceWriter.WriterDoFn( + destinationFn, + getSessionPropertiesProvider(), + getDeliveryMode(), + getDispatchMode(), + getNumberOfClientsPerWorker(), + getPublishLatencyMetrics()) + : new UnboundedBatchedSolaceWriter.WriterDoFn( + destinationFn, + getSessionPropertiesProvider(), + getDeliveryMode(), + getDispatchMode(), + getNumberOfClientsPerWorker(), + getPublishLatencyMetrics())); + + return writer.withOutputTags( + FAILED_PUBLISH_TAG, TupleTagList.of(SUCCESSFUL_PUBLISH_TAG)); + } + + private SolacePublishResult rewindow( + SolacePublishResult solacePublishResult, + WindowingStrategy strategy) { + PCollection correct = solacePublishResult.getSuccessfulPublish(); + PCollection failed = solacePublishResult.getFailedPublish(); + + PCollection correctWithWindow = null; + PCollection failedWithWindow = null; + + if (correct != null) { + correctWithWindow = applyOriginalWindow(correct, strategy, "Rewindow correct"); + } + + if (failed != null) { + failedWithWindow = applyOriginalWindow(failed, strategy, "Rewindow failed"); + } + + return SolacePublishResult.in( + solacePublishResult.getPipeline(), failedWithWindow, correctWithWindow); + } + + private static PCollection applyOriginalWindow( + PCollection pcoll, + WindowingStrategy strategy, + String label) { + Window originalWindow = captureWindowDetails(strategy); + + if (strategy.getMode() == WindowingStrategy.AccumulationMode.ACCUMULATING_FIRED_PANES) { + originalWindow = originalWindow.accumulatingFiredPanes(); + } else { + originalWindow = originalWindow.discardingFiredPanes(); + } + + return pcoll.apply(label, originalWindow); + } + + private static Window captureWindowDetails( + WindowingStrategy strategy) { + return Window.into(strategy.getWindowFn()) + .withAllowedLateness(strategy.getAllowedLateness()) + .withOnTimeBehavior(strategy.getOnTimeBehavior()) + .withTimestampCombiner(strategy.getTimestampCombiner()) + .triggering(strategy.getTrigger()); + } + + /** + * Called before running the Pipeline to verify this transform is fully and correctly + * specified. + */ + private void validate(boolean usingSolaceRecords) { + if (!usingSolaceRecords) { + Preconditions.checkArgument( + getFormatFunction() != null, + "SolaceIO.Write: If you are not using Solace.Record as the input type, you" + + " must set a format function using withFormatFunction()."); + } + + Preconditions.checkArgument( + getMaxNumOfUsedWorkers() > 0, + "SolaceIO.Write: The number of used workers must be positive."); + Preconditions.checkArgument( + getNumberOfClientsPerWorker() > 0, + "SolaceIO.Write: The number of clients per worker must be positive."); + Preconditions.checkArgument( + getDeliveryMode() == DeliveryMode.DIRECT + || getDeliveryMode() == DeliveryMode.PERSISTENT, + String.format( + "SolaceIO.Write: Delivery mode must be either DIRECT or PERSISTENT. %s" + + " not supported", + getDeliveryMode())); + if (getPublishLatencyMetrics()) { + Preconditions.checkArgument( + getDeliveryMode() == DeliveryMode.PERSISTENT, + "SolaceIO.Write: Publish latency metrics can only be enabled for PERSISTENT" + + " delivery mode."); + } + Preconditions.checkArgument( + getSessionPropertiesProvider() != null, + "SolaceIO: You need to pass a session properties provider. For basic" + + " authentication, you can use BasicAuthenticationProvider."); + } + } + + @AutoValue + public abstract static class Read extends PTransform> { + private static final Logger LOG = LoggerFactory.getLogger(Read.class); + + /** Set the queue name to read from. Use this or the `from(Topic)` method. */ + public Read from(Solace.Queue queue) { + return toBuilder().setQueue(queueFromName(queue.getName())).build(); + } + + /** Set the topic name to read from. Use this or the `from(Queue)` method. */ + public Read from(Solace.Topic topic) { + return toBuilder().setTopic(topicFromName(topic.getName())).build(); + } + + /** + * Set the timestamp function. This serializable has to output an {@link Instant}. This will + * be used to calculate watermark and define record's timestamp. + */ + public Read withTimestampFn(SerializableFunction timestampFn) { + return toBuilder().setTimestampFn(timestampFn).build(); + } + + /** + * Maximum number of read connections created to Solace cluster. This is optional, leave out + * to let the Runner decide. + */ + public Read withMaxNumConnections(Integer maxNumConnections) { + return toBuilder().setMaxNumConnections(maxNumConnections).build(); + } + + /** + * Set if the read records should be deduplicated. True by default. It will use the + * `applicationMessageId` attribute to identify duplicates. + */ + public Read withDeduplicateRecords(boolean deduplicateRecords) { + return toBuilder().setDeduplicateRecords(deduplicateRecords).build(); + } + + /** + * Set a factory that creates a {@link + * com.google.cloud.dataflow.dce.io.solace.broker.BrokerService}. + * + *

The factory `create()` method is invoked in each instance of an {@link + * com.google.cloud.dataflow.dce.io.solace.read.UnboundedSolaceReader}. Created {@link + * com.google.cloud.dataflow.dce.io.solace.broker.BrokerService} has to communicate with + * broker management API. It must support operations such as: + * + *

    + *
  • query for outstanding backlog bytes in a Queue, + *
  • query for metadata such as access-type of a Queue, + *
  • requesting creation of new Queues. + *
+ */ + public Read withSempClientFactory(SempClientFactory sempClientFactory) { + return toBuilder().setSempClientFactory(sempClientFactory).build(); + } + + /** + * Set a factory that creates a {@link SessionService}. + * + *

The factory `create()` method is invoked in each instance of an {@link + * com.google.cloud.dataflow.dce.io.solace.read.UnboundedSolaceReader}. Created {@link + * SessionService} has to be able to: + * + *

    + *
  • initialize a connection with the broker, + *
  • check liveliness of the connection, + *
  • close the connection, + *
  • create a {@link com.google.cloud.dataflow.dce.io.solace.broker.MessageReceiver}. + *
+ */ + public Read withSessionServiceFactory(SessionServiceFactory sessionServiceFactory) { + return toBuilder().setSessionServiceFactory(sessionServiceFactory).build(); + } + + abstract @Nullable Queue getQueue(); + + abstract @Nullable Topic getTopic(); + + abstract @Nullable SerializableFunction getTimestampFn(); + + abstract @Nullable Integer getMaxNumConnections(); + + abstract boolean getDeduplicateRecords(); + + abstract SerializableFunction getParseFn(); + + abstract @Nullable SempClientFactory getSempClientFactory(); + + abstract @Nullable SessionServiceFactory getSessionServiceFactory(); + + abstract TypeDescriptor getTypeDescriptor(); + + public static Builder builder() { + Builder builder = new AutoValue_SolaceIO_Read.Builder(); + builder.setDeduplicateRecords(DEFAULT_DEDUPLICATE_RECORDS); + return builder; + } + + abstract Builder toBuilder(); + + @AutoValue.Builder + public abstract static class Builder { + + abstract Builder setQueue(Queue queue); + + abstract Builder setTopic(Topic topic); + + abstract Builder setTimestampFn(SerializableFunction timestampFn); + + abstract Builder setMaxNumConnections(Integer maxNumConnections); + + abstract Builder setDeduplicateRecords(boolean deduplicateRecords); + + abstract Builder setParseFn(SerializableFunction parseFn); + + abstract Builder setSempClientFactory(SempClientFactory brokerServiceFactory); + + abstract Builder setSessionServiceFactory( + SessionServiceFactory sessionServiceFactory); + + abstract Builder setTypeDescriptor(TypeDescriptor typeDescriptor); + + abstract Read build(); + } + + @Override + public PCollection expand(PBegin input) { + validate(); + + SempClientFactory sempClientFactory = getSempClientFactory(); + String jobName = input.getPipeline().getOptions().getJobName(); + Queue queue = + getQueue() != null + ? getQueue() + : initializeQueueForTopic(jobName, sempClientFactory); + + SessionServiceFactory sessionServiceFactory = getSessionServiceFactory(); + sessionServiceFactory.setQueue(queue); + + registerDefaultCoder(input.getPipeline()); + // Infer the actual coder + Coder coder = inferCoder(input.getPipeline()); + + return input.apply( + org.apache.beam.sdk.io.Read.from( + new UnboundedSolaceSource<>( + queue, + sempClientFactory, + sessionServiceFactory, + getMaxNumConnections(), + getDeduplicateRecords(), + coder, + getTimestampFn(), + getParseFn()))); + } + + private static void registerDefaultCoder(Pipeline pipeline) { + pipeline.getCoderRegistry() + .registerCoderForType( + TypeDescriptor.of(Solace.Record.class), SolaceRecordCoder.of()); + } + + @VisibleForTesting + Coder inferCoder(Pipeline pipeline) { + Coder coderFromCoderRegistry = getFromCoderRegistry(pipeline); + if (coderFromCoderRegistry != null) { + return coderFromCoderRegistry; + } + + Coder coderFromSchemaRegistry = getFromSchemaRegistry(pipeline); + if (coderFromSchemaRegistry != null) { + return coderFromSchemaRegistry; + } + + throw new RuntimeException( + "SolaceIO.Read: Cannot infer a coder for the TypeDescriptor. Annotate you" + + " output class with @DefaultSchema annotation or create a coder manually" + + " and register it in the CoderRegistry."); + } + + private Coder getFromSchemaRegistry(Pipeline pipeline) { + try { + return pipeline.getSchemaRegistry().getSchemaCoder(getTypeDescriptor()); + } catch (NoSuchSchemaException e) { + return null; + } + } + + private Coder getFromCoderRegistry(Pipeline pipeline) { + try { + return pipeline.getCoderRegistry().getCoder(getTypeDescriptor()); + } catch (CannotProvideCoderException e) { + return null; + } + } + + // FIXME: this is public only for the sake of testing, TODO: redesign test so this is + // private + public Queue initializeQueueForTopic(String jobName, SempClientFactory sempClientFactory) { + Queue q; + if (getQueue() != null) { + q = getQueue(); + } else { + String queueName = String.format("queue-%s-%s", getTopic(), jobName); + try { + String topicName = Objects.requireNonNull(getTopic()).getName(); + q = sempClientFactory.create().createQueueForTopic(queueName, topicName); + LOG.info( + "SolaceIO.Read: A new queue {} was created. The Queue will not be" + + " deleted when this job finishes. Make sure to remove it yourself" + + " when not needed.", + q.getName()); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + return q; + } + + private void validate() { + Preconditions.checkState( + getSempClientFactory() != null, + "SolaceIO.Read: brokerServiceFactory must not be null."); + Preconditions.checkState( + getSessionServiceFactory() != null, + "SolaceIO.Read: SessionServiceFactory must not be null."); + Preconditions.checkState( + getParseFn() != null, + "SolaceIO.Read: parseFn must be set or use the `Read.readSolaceRecords()`" + + " method"); + Preconditions.checkState( + getTimestampFn() != null, + "SolaceIO.Read: timestamp function must be set or use the" + + " `Read.readSolaceRecords()` method"); + Preconditions.checkState( + (getQueue() == null ^ getTopic() == null), + "SolaceIO.Read: One of the Solace {Queue, Topic} must be set."); + } + } +} diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/BasicAuthJcsmpSessionService.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/BasicAuthJcsmpSessionService.java new file mode 100644 index 000000000000..2fb85aaf3f27 --- /dev/null +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/BasicAuthJcsmpSessionService.java @@ -0,0 +1,111 @@ +/* + * Copyright 2024 Google. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.google.cloud.dataflow.dce.io.solace.broker; + +import com.google.cloud.dataflow.dce.io.solace.RetryCallableManager; +import com.solacesystems.jcsmp.ConsumerFlowProperties; +import com.solacesystems.jcsmp.EndpointProperties; +import com.solacesystems.jcsmp.InvalidPropertiesException; +import com.solacesystems.jcsmp.JCSMPException; +import com.solacesystems.jcsmp.JCSMPFactory; +import com.solacesystems.jcsmp.JCSMPProperties; +import com.solacesystems.jcsmp.JCSMPSession; +import com.solacesystems.jcsmp.Queue; +import java.io.IOException; +import java.util.Set; + +public class BasicAuthJcsmpSessionService implements SessionService { + private final String queueName; + private final String host; + private final String username; + private final String password; + private final String vpnName; + private JCSMPSession jcsmpSession; + private final RetryCallableManager retryCallableManager = RetryCallableManager.create(); + + public BasicAuthJcsmpSessionService( + String queueName, String host, String username, String password, String vpnName) { + this.queueName = queueName; + this.host = host; + this.username = username; + this.password = password; + this.vpnName = vpnName; + } + + @Override + public void connect() { + retryCallableManager.retryCallable(this::connectSession, Set.of(JCSMPException.class)); + } + + @Override + public void close() { + if (jcsmpSession != null && !jcsmpSession.isClosed()) { + retryCallableManager.retryCallable( + () -> { + jcsmpSession.closeSession(); + return 0; + }, + Set.of(IOException.class)); + } + } + + @Override + public MessageReceiver createReceiver() { + return retryCallableManager.retryCallable( + this::createFlowReceiver, Set.of(JCSMPException.class)); + } + + @Override + public boolean isClosed() { + return jcsmpSession == null || jcsmpSession.isClosed(); + } + + private MessageReceiver createFlowReceiver() throws JCSMPException { + if (isClosed()) { + connectSession(); + } + + Queue queue = JCSMPFactory.onlyInstance().createQueue(queueName); + + ConsumerFlowProperties flowProperties = new ConsumerFlowProperties(); + flowProperties.setEndpoint(queue); + flowProperties.setAckMode(JCSMPProperties.SUPPORTED_MESSAGE_ACK_CLIENT); + + EndpointProperties endpointProperties = new EndpointProperties(); + endpointProperties.setAccessType(EndpointProperties.ACCESSTYPE_NONEXCLUSIVE); + + return new SolaceMessageReceiver( + jcsmpSession.createFlow(null, flowProperties, endpointProperties)); + } + + private int connectSession() throws JCSMPException { + if (jcsmpSession == null) { + jcsmpSession = createSessionObject(); + } + jcsmpSession.connect(); + return 0; + } + + private JCSMPSession createSessionObject() throws InvalidPropertiesException { + JCSMPProperties properties = new JCSMPProperties(); + properties.setProperty(JCSMPProperties.HOST, host); + properties.setProperty(JCSMPProperties.USERNAME, username); + properties.setProperty(JCSMPProperties.PASSWORD, password); + properties.setProperty(JCSMPProperties.VPN_NAME, vpnName); + + return JCSMPFactory.onlyInstance().createSession(properties); + } +} diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/BasicAuthJcsmpSessionServiceFactory.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/BasicAuthJcsmpSessionServiceFactory.java new file mode 100644 index 000000000000..a4f8f7f2bc07 --- /dev/null +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/BasicAuthJcsmpSessionServiceFactory.java @@ -0,0 +1,75 @@ +/* + * Copyright 2024 Google. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.google.cloud.dataflow.dce.io.solace.broker; + +import com.google.common.base.Preconditions; + +public class BasicAuthJcsmpSessionServiceFactory extends SessionServiceFactory { + private final String host; + private final String username; + private final String password; + private final String vpnName; + + private BasicAuthJcsmpSessionServiceFactory( + String host, String username, String password, String vpnName) { + this.host = host; + this.username = username; + this.password = password; + this.vpnName = vpnName; + } + + public static BasicAuthJcsmpSessionServiceFactoryBuilder builder() { + return new BasicAuthJcsmpSessionServiceFactoryBuilder(); + } + + @Override + public SessionService create() { + Preconditions.checkState(queue != null, "SolaceIO.Read: Queue is not set."); + return new BasicAuthJcsmpSessionService(queue.getName(), host, username, password, vpnName); + } + + public static class BasicAuthJcsmpSessionServiceFactoryBuilder { + + private String host; + private String username; + private String password; + private String vpnName; + + public BasicAuthJcsmpSessionServiceFactoryBuilder withHost(String host) { + this.host = host; + return this; + } + + public BasicAuthJcsmpSessionServiceFactoryBuilder withUsername(String username) { + this.username = username; + return this; + } + + public BasicAuthJcsmpSessionServiceFactoryBuilder withPassword(String password) { + this.password = password; + return this; + } + + public BasicAuthJcsmpSessionServiceFactoryBuilder withVpnName(String vpnName) { + this.vpnName = vpnName; + return this; + } + + public BasicAuthJcsmpSessionServiceFactory build() { + return new BasicAuthJcsmpSessionServiceFactory(host, username, password, vpnName); + } + } +} diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/BasicAuthSempClient.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/BasicAuthSempClient.java new file mode 100644 index 000000000000..d2cc716583f5 --- /dev/null +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/BasicAuthSempClient.java @@ -0,0 +1,251 @@ +/* + * Copyright 2024 Google. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.google.cloud.dataflow.dce.io.solace.broker; + +import com.fasterxml.jackson.databind.DeserializationFeature; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.annotation.JsonDeserialize; +import com.fasterxml.jackson.databind.annotation.JsonPOJOBuilder; +import com.fasterxml.jackson.databind.annotation.JsonSerialize; +import com.google.api.client.http.HttpRequestFactory; +import com.google.auto.value.AutoValue; +import com.google.cloud.dataflow.dce.io.solace.SerializableSupplier; +import com.solacesystems.jcsmp.JCSMPFactory; +import java.io.IOException; +import org.apache.beam.sdk.annotations.Internal; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@Internal +public class BasicAuthSempClient implements SempClient { + private static final Logger LOG = LoggerFactory.getLogger(BasicAuthSempClient.class); + private final ObjectMapper objectMapper = + new ObjectMapper().configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); + + private final SempBasicAuthClientExecutor sempBasicAuthClientExecutor; + + public BasicAuthSempClient( + String host, + String username, + String password, + String vpnName, + SerializableSupplier httpRequestFactorySupplier) { + sempBasicAuthClientExecutor = + new SempBasicAuthClientExecutor( + host, username, password, vpnName, httpRequestFactorySupplier.get()); + } + + @Override + public boolean isQueueNonExclusive(String queueName) throws IOException { + LOG.info( + "SolaceIO.Read: SempOperations: query SEMP if queue {} is nonExclusive", queueName); + BrokerResponse response = sempBasicAuthClientExecutor.getQueueResponse(queueName); + try { + Queue q = getSuccessResponseOrThrowException(response, Queue.class); + return q.data().accessType().equals("non-exclusive"); + } catch (SolaceSempException e) { + throw new IOException("SolaceIO.Read: Exception when querying for Queue metadata.", e); + } + } + + @Override + public com.solacesystems.jcsmp.Queue createQueueForTopic(String queueName, String topicName) + throws IOException { + try { + createQueue(queueName); + createSubscription(queueName, topicName); + } catch (SolaceSempException e) { + throw new IOException("SolaceIO.Read: Exception when creating a Queue for a Topic.", e); + } + return JCSMPFactory.onlyInstance().createQueue(queueName); + } + + @Override + public long getBacklogBytes(String queueName) throws IOException { + BrokerResponse response = sempBasicAuthClientExecutor.getQueueResponse(queueName); + try { + Queue q = getSuccessResponseOrThrowException(response, Queue.class); + return q.data().msgSpoolUsage(); + } catch (SolaceSempException e) { + throw new IOException("SolaceIO.Read: Exception when querying for backlog bytes.", e); + } + } + + private void createQueue(String queueName) throws SolaceSempException, IOException { + LOG.info("SolaceIO.Read: Creating new queue {}.", queueName); + BrokerResponse response = sempBasicAuthClientExecutor.createQueueResponse(queueName); + checkIfError(response); + } + + private void createSubscription(String queueName, String topicName) + throws SolaceSempException, IOException { + LOG.info("SolaceIO.Read: Creating new subscription {} for topic {}.", queueName, topicName); + BrokerResponse response = + sempBasicAuthClientExecutor.createSubscriptionResponse(queueName, topicName); + checkIfError(response); + } + + private T getSuccessResponseOrThrowException( + BrokerResponse response, Class mapSuccessToClass) + throws IOException, SolaceSempException { + checkIfError(response); + return objectMapper.readValue(response.content, mapSuccessToClass); + } + + private void checkIfError(BrokerResponse response) + throws SolaceSempException, IOException { // todo do we still need this? + if (response.code < 200 || response.code > 299) { + ErrorMessage error = objectMapper.readValue(response.content, ErrorMessage.class); + throw new SolaceSempException(error); + } + } + + static class SolaceSempException extends Exception { + public final ErrorMessage errorMessage; + + SolaceSempException(ErrorMessage errorMessage) { + super(errorMessage.meta().error().description()); + this.errorMessage = errorMessage; + } + } + + @AutoValue + @JsonSerialize(as = Queue.class) + @JsonDeserialize(builder = AutoValue_BasicAuthSempClient_Queue.Builder.class) + abstract static class Queue { + + public abstract QueueData data(); + + public static Builder builder() { + return new AutoValue_BasicAuthSempClient_Queue.Builder(); + } + + public abstract Builder toBuilder(); + + @AutoValue.Builder + @JsonPOJOBuilder(withPrefix = "set") + abstract static class Builder { + + public abstract Builder setData(QueueData queueData); + + public abstract Queue build(); + } + } + + @AutoValue + @JsonDeserialize(builder = AutoValue_BasicAuthSempClient_QueueData.Builder.class) + abstract static class QueueData { + public abstract String accessType(); + + public abstract long msgSpoolUsage(); + + public static Builder builder() { + return new AutoValue_BasicAuthSempClient_QueueData.Builder(); + } + + public abstract Builder toBuilder(); + + @AutoValue.Builder + @JsonPOJOBuilder(withPrefix = "set") + abstract static class Builder { + + public abstract Builder setAccessType(String accessType); + + public abstract Builder setMsgSpoolUsage(long msgSpoolUsage); + + public abstract QueueData build(); + } + } + + @AutoValue + @JsonSerialize(as = ErrorMessage.class) + @JsonDeserialize(builder = AutoValue_BasicAuthSempClient_ErrorMessage.Builder.class) + abstract static class ErrorMessage { + + public abstract ErrorMessageMeta meta(); + + public static Builder builder() { + return new AutoValue_BasicAuthSempClient_ErrorMessage.Builder(); + } + + public abstract Builder toBuilder(); + + @AutoValue.Builder + @JsonPOJOBuilder(withPrefix = "set") + abstract static class Builder { + + public abstract Builder setMeta(ErrorMessageMeta errorMessageMeta); + + public abstract ErrorMessage build(); + } + + @AutoValue + @JsonSerialize(as = ErrorMessageMeta.class) + @JsonDeserialize( + builder = AutoValue_BasicAuthSempClient_ErrorMessage_ErrorMessageMeta.Builder.class) + abstract static class ErrorMessageMeta { + public abstract ErrorMessageDetails error(); + + public static Builder builder() { + return new AutoValue_BasicAuthSempClient_ErrorMessage_ErrorMessageMeta.Builder(); + } + + public abstract Builder toBuilder(); + + @AutoValue.Builder + @JsonPOJOBuilder(withPrefix = "set") + abstract static class Builder { + + public abstract Builder setError(ErrorMessageDetails errorMessageDetails); + + public abstract ErrorMessageMeta build(); + } + } + + @AutoValue + @JsonSerialize(as = ErrorMessageDetails.class) + @JsonDeserialize( + builder = + AutoValue_BasicAuthSempClient_ErrorMessage_ErrorMessageDetails.Builder + .class) + abstract static class ErrorMessageDetails { + public abstract Integer code(); + + public abstract String description(); + + public abstract String status(); + + public static Builder builder() { + return new AutoValue_BasicAuthSempClient_ErrorMessage_ErrorMessageDetails.Builder(); + } + + public abstract Builder toBuilder(); + + @AutoValue.Builder + @JsonPOJOBuilder(withPrefix = "set") + abstract static class Builder { + + public abstract Builder setCode(Integer code); + + public abstract Builder setDescription(String description); + + public abstract Builder setStatus(String status); + + public abstract ErrorMessageDetails build(); + } + } + } +} diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/BasicAuthSempClientFactory.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/BasicAuthSempClientFactory.java new file mode 100644 index 000000000000..fdf5c6a04ca3 --- /dev/null +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/BasicAuthSempClientFactory.java @@ -0,0 +1,123 @@ +/* + * Copyright 2024 Google. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.google.cloud.dataflow.dce.io.solace.broker; + +import com.google.api.client.http.HttpRequestFactory; +import com.google.api.client.http.javanet.NetHttpTransport; +import com.google.cloud.dataflow.dce.io.solace.SerializableSupplier; +import com.google.common.base.Preconditions; +import org.apache.arrow.util.VisibleForTesting; + +public class BasicAuthSempClientFactory implements SempClientFactory { + + private final String host; + private final String username; + private final String password; + private final String vpnName; + private final SerializableSupplier httpRequestFactorySupplier; + + private BasicAuthSempClientFactory( + String host, + String username, + String password, + String vpnName, + SerializableSupplier httpRequestFactorySupplier) { + this.host = host; + this.username = username; + this.password = password; + this.vpnName = vpnName; + this.httpRequestFactorySupplier = httpRequestFactorySupplier; + } + + public static BasicAuthSempAuthenticationFactoryBuilder builder() { + return new BasicAuthSempAuthenticationFactoryBuilder() + .withHttpRequestFactorySupplier( + () -> new NetHttpTransport().createRequestFactory()); + } + + @Override + public SempClient create() { + return new BasicAuthSempClient( + host, username, password, vpnName, httpRequestFactorySupplier); + } + + public static class BasicAuthSempAuthenticationFactoryBuilder { + + private String host; + private String username; + private String password; + private String vpnName; + private SerializableSupplier httpRequestFactorySupplier; + + /** Set Solace host, format: Protocol://Host[:Port] */ + public BasicAuthSempAuthenticationFactoryBuilder withHost(String host) { + this.host = host; + return this; + } + + /** Set Solace username */ + public BasicAuthSempAuthenticationFactoryBuilder withUsername(String username) { + this.username = username; + return this; + } + + /** Set Solace password */ + public BasicAuthSempAuthenticationFactoryBuilder withPassword(String password) { + this.password = password; + return this; + } + + /** Set Solace vpn name */ + public BasicAuthSempAuthenticationFactoryBuilder withVpnName(String vpnName) { + this.vpnName = vpnName; + return this; + } + + @VisibleForTesting + BasicAuthSempAuthenticationFactoryBuilder withHttpRequestFactorySupplier( + SerializableSupplier httpRequestFactorySupplier) { + this.httpRequestFactorySupplier = httpRequestFactorySupplier; + return this; + } + + public BasicAuthSempClientFactory build() { + // todo update name in the error string + Preconditions.checkState( + host != null, + "SolaceIO: host in BasicAuthSempAuthenticationFactory can't be null. Set it" + + " with `withHost()` method."); + Preconditions.checkState( + username != null, + "SolaceIO: username in BasicAuthSempAuthenticationFactory can't be null. Set it" + + " with `withUsername()` method."); + Preconditions.checkState( + password != null, + "SolaceIO: password in BasicAuthSempAuthenticationFactory can't be null. Set it" + + " with `withPassword()` method."); + Preconditions.checkState( + vpnName != null, + "SolaceIO: vpnName in BasicAuthSempAuthenticationFactory can't be null. Set it" + + " with `withVpnName()` method."); + Preconditions.checkState( + httpRequestFactorySupplier != null, + "SolaceIO: httpRequestFactorySupplier in BasicAuthSempAuthenticationFactory" + + " can't be null. Set it with `withHttpRequestFactorySupplier()` method."); + + return new BasicAuthSempClientFactory( + host, username, password, vpnName, httpRequestFactorySupplier); + } + } +} diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/BrokerResponse.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/BrokerResponse.java new file mode 100644 index 000000000000..26954993650d --- /dev/null +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/BrokerResponse.java @@ -0,0 +1,63 @@ +/* + * Copyright 2024 Google. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.google.cloud.dataflow.dce.io.solace.broker; + +import com.google.api.client.http.HttpResponse; +import java.io.BufferedReader; +import java.io.IOException; +import java.io.InputStream; +import java.io.InputStreamReader; +import java.nio.charset.StandardCharsets; +import java.util.stream.Collectors; +import org.checkerframework.checker.nullness.qual.Nullable; + +public class BrokerResponse { + final int code; + final String message; + @Nullable String content; + + public BrokerResponse(int responseCode, String message, @Nullable InputStream content) + throws IOException { + this.code = responseCode; + this.message = message; + if (content != null) { + try (content) { + this.content = + new BufferedReader(new InputStreamReader(content, StandardCharsets.UTF_8)) + .lines() + .collect(Collectors.joining("\n")); + } + } + } + + public static BrokerResponse fromHttpResponse(HttpResponse response) throws IOException { + return new BrokerResponse( + response.getStatusCode(), response.getStatusMessage(), response.getContent()); + } + + @Override + public String toString() { + return "BrokerResponse{" + + "code=" + + code + + ", message='" + + message + + '\'' + + ", content=" + + content + + '}'; + } +} diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/MessageReceiver.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/MessageReceiver.java new file mode 100644 index 000000000000..82ff078d515b --- /dev/null +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/MessageReceiver.java @@ -0,0 +1,35 @@ +/* + * Copyright 2024 Google. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.google.cloud.dataflow.dce.io.solace.broker; + +import com.solacesystems.jcsmp.BytesXMLMessage; +import java.io.IOException; + +public interface MessageReceiver { + void start(); + + boolean isClosed(); + + BytesXMLMessage receive() throws IOException; + + /** + * Test clients may return {@literal true} to signal that all expected messages have been pulled + * and the test may complete. Real clients will return {@literal false}. + */ + default boolean isEOF() { + return false; + } +} diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SempBasicAuthClientExecutor.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SempBasicAuthClientExecutor.java new file mode 100644 index 000000000000..36a64dda948c --- /dev/null +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SempBasicAuthClientExecutor.java @@ -0,0 +1,170 @@ +/* + * Copyright 2024 Google. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.google.cloud.dataflow.dce.io.solace.broker; + +import com.google.api.client.http.GenericUrl; +import com.google.api.client.http.HttpContent; +import com.google.api.client.http.HttpHeaders; +import com.google.api.client.http.HttpRequest; +import com.google.api.client.http.HttpRequestFactory; +import com.google.api.client.http.HttpResponse; +import com.google.api.client.http.HttpResponseException; +import com.google.api.client.http.json.JsonHttpContent; +import com.google.api.client.json.gson.GsonFactory; +import com.google.common.collect.ImmutableMap; +import java.io.IOException; +import java.io.Serializable; +import java.net.CookieManager; +import java.net.HttpCookie; +import java.util.List; +import java.util.stream.Collectors; + +/** + * A class to execute requests to SEMP v2 with Basic Auth authentication. + * + *

This approach takes advantage of SEMP Sessions. The + * session is established when a user authenticates with HTTP Basic authentication. When the + * response is 401 Unauthorized, the client will execute an additional request with Basic Auth + * header to refresh the token. + */ +class SempBasicAuthClientExecutor implements Serializable { + private static final CookieManager COOKIE_MANAGER = new CookieManager(); + private static final String COOKIES_HEADER = "Set-Cookie"; + + private final String username; + private final String messageVpn; + private final String baseUrl; + private final String password; + private final HttpRequestFactory requestFactory; + + SempBasicAuthClientExecutor( + String host, + String username, + String password, + String vpnName, + HttpRequestFactory httpRequestFactory) { + this.baseUrl = String.format("%s/SEMP/v2", host); + this.username = username; + this.messageVpn = vpnName; + this.password = password; + this.requestFactory = httpRequestFactory; + } + + private static String getQueueEndpoint(String messageVpn, String queueName) { + return String.format("/monitor/msgVpns/%s/queues/%s", messageVpn, queueName); + } + + private static String createQueueEndpoint(String messageVpn) { + return String.format("/config/msgVpns/%s/queues", messageVpn); + } + + private static String subscriptionEndpoint(String messageVpn, String queueName) { + return String.format("/config/msgVpns/%s/queues/%s/subscriptions", messageVpn, queueName); + } + + BrokerResponse getQueueResponse(String queueName) throws IOException { + String queryUrl = getQueueEndpoint(messageVpn, queueName); + HttpResponse response = executeGet(new GenericUrl(baseUrl + queryUrl)); + return BrokerResponse.fromHttpResponse(response); + } + + BrokerResponse createQueueResponse(String queueName) throws IOException { + String queryUrl = createQueueEndpoint(messageVpn); + ImmutableMap params = + ImmutableMap.builder() + .put("accessType", "non-exclusive") + .put("queueName", queueName) + .put("owner", username) + .put("permission", "consume") + .put("ingressEnabled", true) + .put("egressEnabled", true) + .build(); + + HttpResponse response = executePost(new GenericUrl(baseUrl + queryUrl), params); + return BrokerResponse.fromHttpResponse(response); + } + + BrokerResponse createSubscriptionResponse(String queueName, String topicName) + throws IOException { + String queryUrl = subscriptionEndpoint(messageVpn, queueName); + + ImmutableMap params = + ImmutableMap.builder() + .put("subscriptionTopic", topicName) + .put("queueName", queueName) + .build(); + HttpResponse response = executePost(new GenericUrl(baseUrl + queryUrl), params); + return BrokerResponse.fromHttpResponse(response); + } + + private HttpResponse executeGet(GenericUrl url) throws IOException { + HttpRequest request = requestFactory.buildGetRequest(url); + return execute(request); + } + + private HttpResponse executePost(GenericUrl url, ImmutableMap parameters) + throws IOException { + HttpContent content = new JsonHttpContent(GsonFactory.getDefaultInstance(), parameters); + HttpRequest request = requestFactory.buildPostRequest(url, content); + return execute(request); + } + + private HttpResponse execute(HttpRequest request) throws IOException { + request.setNumberOfRetries(2); + HttpHeaders httpHeaders = new HttpHeaders(); + boolean authFromCookie = COOKIE_MANAGER.getCookieStore().getCookies().size() > 0; + if (authFromCookie) { + setCookiesFromCookieManager(httpHeaders); + request.setHeaders(httpHeaders); + } else { + httpHeaders.setBasicAuthentication(username, password); + request.setHeaders(httpHeaders); + } + + HttpResponse response; + try { + response = request.execute(); + } catch (HttpResponseException e) { + if (authFromCookie && e.getStatusCode() == 401) { + COOKIE_MANAGER.getCookieStore().removeAll(); + // execute again without cookies to refresh the token. + return execute(request); + } else { + throw e; + } + } + + storeCookiesInCookieManager(response.getHeaders()); + return response; + } + + private void setCookiesFromCookieManager(HttpHeaders httpHeaders) { + httpHeaders.setCookie( + COOKIE_MANAGER.getCookieStore().getCookies().stream() + .map(s -> s.getName() + "=" + s.getValue()) + .collect(Collectors.joining(";"))); + } + + private void storeCookiesInCookieManager(HttpHeaders headers) { + List cookiesHeader = headers.getHeaderStringValues(COOKIES_HEADER); + if (cookiesHeader != null) { + for (String cookie : cookiesHeader) { + COOKIE_MANAGER.getCookieStore().add(null, HttpCookie.parse(cookie).get(0)); + } + } + } +} diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SempClient.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SempClient.java new file mode 100644 index 000000000000..dfa65fe5bf38 --- /dev/null +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SempClient.java @@ -0,0 +1,29 @@ +/* + * Copyright 2024 Google. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.google.cloud.dataflow.dce.io.solace.broker; + +import com.solacesystems.jcsmp.Queue; +import java.io.IOException; +import java.io.Serializable; + +public interface SempClient extends Serializable { + + boolean isQueueNonExclusive(String queueName) throws IOException; + + Queue createQueueForTopic(String queueName, String topicName) throws IOException; + + long getBacklogBytes(String queueName) throws IOException; +} diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SempClientFactory.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SempClientFactory.java new file mode 100644 index 000000000000..6d3e4baec40d --- /dev/null +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SempClientFactory.java @@ -0,0 +1,22 @@ +/* + * Copyright 2024 Google. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.google.cloud.dataflow.dce.io.solace.broker; + +import java.io.Serializable; + +public interface SempClientFactory extends Serializable { + SempClient create(); +} diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SessionService.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SessionService.java new file mode 100644 index 000000000000..5766469ff8b9 --- /dev/null +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SessionService.java @@ -0,0 +1,28 @@ +/* + * Copyright 2024 Google. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.google.cloud.dataflow.dce.io.solace.broker; + +import java.io.Serializable; + +public interface SessionService extends Serializable { + void connect(); + + void close(); + + boolean isClosed(); + + MessageReceiver createReceiver(); +} diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SessionServiceFactory.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SessionServiceFactory.java new file mode 100644 index 000000000000..d5ab4f32aec7 --- /dev/null +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SessionServiceFactory.java @@ -0,0 +1,29 @@ +/* + * Copyright 2024 Google. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.google.cloud.dataflow.dce.io.solace.broker; + +import com.solacesystems.jcsmp.Queue; +import java.io.Serializable; + +public abstract class SessionServiceFactory implements Serializable { + Queue queue; + + public abstract SessionService create(); + + public void setQueue(Queue queue) { + this.queue = queue; + } +} diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SolaceMessageReceiver.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SolaceMessageReceiver.java new file mode 100644 index 000000000000..784d3223c832 --- /dev/null +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SolaceMessageReceiver.java @@ -0,0 +1,59 @@ +/* + * Copyright 2024 Google. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.google.cloud.dataflow.dce.io.solace.broker; + +import com.google.cloud.dataflow.dce.io.solace.RetryCallableManager; +import com.solacesystems.jcsmp.BytesXMLMessage; +import com.solacesystems.jcsmp.FlowReceiver; +import com.solacesystems.jcsmp.JCSMPException; +import java.io.IOException; +import java.io.Serializable; +import java.util.Set; + +public class SolaceMessageReceiver implements MessageReceiver, Serializable { + + public static final int DEFAULT_ADVANCE_TIMEOUT_IN_MILLIS = 100; + private final FlowReceiver flowReceiver; + private final RetryCallableManager retryCallableManager = RetryCallableManager.create(); + + public SolaceMessageReceiver(FlowReceiver flowReceiver) { + this.flowReceiver = flowReceiver; + } + + @Override + public void start() { + retryCallableManager.retryCallable( + () -> { + flowReceiver.start(); + return 0; + }, + Set.of(JCSMPException.class)); + } + + @Override + public boolean isClosed() { + return flowReceiver == null || flowReceiver.isClosed(); + } + + @Override + public BytesXMLMessage receive() throws IOException { + try { + return flowReceiver.receive(DEFAULT_ADVANCE_TIMEOUT_IN_MILLIS); + } catch (JCSMPException e) { + throw new IOException(e); + } + } +} diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/data/Solace.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/data/Solace.java new file mode 100644 index 000000000000..7b3dae856a24 --- /dev/null +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/data/Solace.java @@ -0,0 +1,340 @@ +/* + * Copyright 2023 Google. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.google.cloud.dataflow.dce.io.solace.data; + +import com.google.auto.value.AutoValue; +import com.solacesystems.jcsmp.BytesXMLMessage; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.Serializable; +import org.apache.beam.sdk.schemas.AutoValueSchema; +import org.apache.beam.sdk.schemas.annotations.DefaultSchema; +import org.apache.beam.sdk.schemas.annotations.SchemaFieldNumber; +import org.checkerframework.checker.nullness.qual.Nullable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * A record to be written to a Solace topic. + * + *

You need to transform to {@link Solace.Record} to be able to write to Solace. For that, you + * can use the {@link Solace.Record.Builder} provided with this class. + * + *

For instance, to create a record, use the following code: + * + *

{@code
+ * Solace.Record record = Solace.Record.builder()
+ *         .setMessageId(messageId)
+ *         .setSenderTimestamp(timestampMillis)
+ *         .setPayload(payload)
+ *         .build();
+ * }
+ * + * Setting the message id and the timestamp is mandatory. + */ +public class Solace { + + public static class Queue { + private final String name; + + private Queue(String name) { + this.name = name; + } + + public static Queue fromName(String name) { + return new Queue(name); + } + + public String getName() { + return name; + } + } + + public static class Topic { + private final String name; + + private Topic(String name) { + this.name = name; + } + + public static Topic fromName(String name) { + return new Topic(name); + } + + public String getName() { + return name; + } + } + + public enum DestinationType { + TOPIC, + QUEUE + } + + @AutoValue + @DefaultSchema(AutoValueSchema.class) + public abstract static class Destination { + @SchemaFieldNumber("0") + public abstract String getName(); + + @SchemaFieldNumber("1") + public abstract DestinationType getType(); + + public static Builder builder() { + return new AutoValue_Solace_Destination.Builder(); + } + + @AutoValue.Builder + public abstract static class Builder { + public abstract Builder setName(String name); + + public abstract Builder setType(DestinationType type); + + public abstract Destination build(); + } + } + + @AutoValue + @DefaultSchema(AutoValueSchema.class) + public abstract static class Record implements Serializable { + @SchemaFieldNumber("0") + public abstract @Nullable String getMessageId(); + + @SuppressWarnings("mutable") + @SchemaFieldNumber("1") + public abstract byte[] getPayload(); + + @SchemaFieldNumber("2") + public abstract @Nullable Destination getDestination(); + + @SchemaFieldNumber("3") + public abstract @Nullable Long getExpiration(); + + @SchemaFieldNumber("4") + public abstract @Nullable Integer getPriority(); + + @SchemaFieldNumber("5") + public abstract @Nullable Boolean getRedelivered(); + + @SchemaFieldNumber("6") + public abstract @Nullable String getReplyTo(); + + @SchemaFieldNumber("7") + public abstract @Nullable Long getReceiveTimestamp(); + + @SchemaFieldNumber("8") + public abstract @Nullable Long getSenderTimestamp(); + + @SchemaFieldNumber("9") + public abstract @Nullable Long getSequenceNumber(); + + @SchemaFieldNumber("10") + public abstract @Nullable Long getTimeToLive(); + + /** + * The ID for a particular message is only guaranteed to be the same for a particular copy + * of a message on a particular queue or topic endpoint within a replication group. The same + * message on different queues or topic endpoints within the same replication group may or + * may not have the same replication group message ID. See more at https://docs.solace.com/API/API-Developer-Guide/Detecting-Duplicate-Mess.htm + */ + @SchemaFieldNumber("11") + public abstract @Nullable String getReplicationGroupMessageId(); + + public static Builder builder() { + return new AutoValue_Solace_Record.Builder(); + } + + @AutoValue.Builder + public abstract static class Builder { + public abstract Builder setMessageId(String messageId); + + public abstract Builder setPayload(byte[] payload); + + public abstract Builder setDestination(Destination destination); + + public abstract Builder setExpiration(Long expiration); + + public abstract Builder setPriority(Integer priority); + + public abstract Builder setRedelivered(Boolean redelivered); + + public abstract Builder setReplyTo(String replyTo); + + public abstract Builder setReceiveTimestamp(Long receiveTimestamp); + + public abstract Builder setSenderTimestamp(Long senderTimestamp); + + public abstract Builder setSequenceNumber(Long sequenceNumber); + + public abstract Builder setTimeToLive(Long timeToLive); + + public abstract Builder setReplicationGroupMessageId(String replicationGroupMessageId); + + public abstract Record build(); + } + } + + /** + * The result of writing a message to Solace. This will be returned by the {@link + * com.google.cloud.dataflow.dce.io.solace.SolaceIO.Write} connector. + * + *

This class provides a builder to create instances, but you will probably not need it. The + * write connector will create and return instances of {@link Solace.PublishResult}. + * + *

If the message has been published, {@link Solace.PublishResult#getPublished()} will be + * true. If it is false, it means that the message could not be published, and {@link + * Solace.PublishResult#getError()} will contain more details about why the message could not be + * published. + */ + @AutoValue + @DefaultSchema(AutoValueSchema.class) + public abstract static class PublishResult { + /** The message id of the message that was published. */ + @SchemaFieldNumber("0") + public abstract String getMessageId(); + + /** Whether the message was published or not. */ + @SchemaFieldNumber("1") + public abstract Boolean getPublished(); + + /** + * The publishing latency in milliseconds. This is the difference between the time the + * message was created, and the time the message was published. It is only available if the + * {@link CorrelationKey} class is used as correlation key of the messages. + */ + @SchemaFieldNumber("2") + public abstract @Nullable Long getLatencyMilliseconds(); + + /** The error details if the message could not be published. */ + @SchemaFieldNumber("3") + public abstract @Nullable String getError(); + + public static Builder builder() { + return new AutoValue_Solace_PublishResult.Builder(); + } + + @AutoValue.Builder + public abstract static class Builder { + public abstract Builder setMessageId(String messageId); + + public abstract Builder setPublished(Boolean published); + + public abstract Builder setLatencyMilliseconds(Long latencyMs); + + public abstract Builder setError(String error); + + public abstract PublishResult build(); + } + } + + /** + * The correlation key is an object that is passed back to the client during the event broker + * ack or nack. + * + *

In the streaming writer is optionally used to calculate publish latencies, by calculating + * the time difference between the creation of the correlation key, and the time of the ack. + */ + @AutoValue + @DefaultSchema(AutoValueSchema.class) + public abstract static class CorrelationKey { + @SchemaFieldNumber("0") + public abstract String getMessageId(); + + @SchemaFieldNumber("1") + public abstract long getPublishMonotonicMillis(); + + public static Builder builder() { + return new AutoValue_Solace_CorrelationKey.Builder(); + } + + @AutoValue.Builder + public abstract static class Builder { + public abstract Builder setMessageId(String messageId); + + public abstract Builder setPublishMonotonicMillis(long millis); + + public abstract CorrelationKey build(); + } + } + + public static class SolaceRecordMapper { + public static final Logger LOG = LoggerFactory.getLogger(SolaceRecordMapper.class); + + public static Record map(@Nullable BytesXMLMessage msg) { + if (msg == null) { + return null; + } + + ByteArrayOutputStream outputStream = new ByteArrayOutputStream(); + if (msg.getContentLength() != 0) { + try { + outputStream.write(msg.getBytes()); + } catch (IOException e) { + LOG.error( + "Could not write Bytes from the BytesXMLMessage to the Solace.record.", + e); + } + } + if (msg.getAttachmentContentLength() != 0) { + try { + outputStream.write(msg.getAttachmentByteBuffer().array()); + } catch (IOException e) { + LOG.error( + "Could not AttachmentByteBuffer from the BytesXMLMessage to the" + + " Solace.record.", + e); + } + } + + String replyTo = (msg.getReplyTo() != null) ? msg.getReplyTo().getName() : null; + + com.solacesystems.jcsmp.Destination originalDestination = msg.getDestination(); + Destination.Builder destBuilder = + Destination.builder().setName(originalDestination.getName()); + if (originalDestination instanceof Topic) { + destBuilder.setType(DestinationType.TOPIC); + } else if (originalDestination instanceof Queue) { + destBuilder.setType(DestinationType.QUEUE); + } else { + LOG.error( + "SolaceIO: Unknown destination type for message {}, assuming that {} is a" + + " topic", + msg.getCorrelationId(), + originalDestination.getName()); + destBuilder.setType(DestinationType.TOPIC); + } + + return Record.builder() + .setDestination(destBuilder.build()) + .setExpiration(msg.getExpiration()) + .setMessageId(msg.getApplicationMessageId()) + .setPriority(msg.getPriority()) + .setRedelivered(msg.getRedelivered()) + .setReplyTo(replyTo) + .setReceiveTimestamp(msg.getReceiveTimestamp()) + .setSenderTimestamp(msg.getSenderTimestamp()) + .setSequenceNumber(msg.getSequenceNumber()) + .setTimeToLive(msg.getTimeToLive()) + .setReplicationGroupMessageId( + msg.getReplicationGroupMessageId() != null + ? msg.getReplicationGroupMessageId().toString() + : null) + .setPayload(outputStream.toByteArray()) + .build(); + } + } +} diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/data/SolaceRecordCoder.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/data/SolaceRecordCoder.java new file mode 100644 index 000000000000..bb6f851e72bd --- /dev/null +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/data/SolaceRecordCoder.java @@ -0,0 +1,93 @@ +/* + * Copyright 2024 Google. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.google.cloud.dataflow.dce.io.solace.data; + +import com.google.cloud.dataflow.dce.io.solace.data.Solace.Record; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.Objects; +import org.apache.beam.sdk.coders.BooleanCoder; +import org.apache.beam.sdk.coders.ByteArrayCoder; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.CustomCoder; +import org.apache.beam.sdk.coders.NullableCoder; +import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.coders.VarIntCoder; +import org.apache.beam.sdk.coders.VarLongCoder; + +/** + * Custom coder for the default Solace {@link Record} + * + *

A custom coder is required to update a Dataflow job. Using a coder generated with the + * `@DefaultSchema` annotation doesn't create an update-compatible coders. + */ +public class SolaceRecordCoder extends CustomCoder { + private static final Coder BYTE_CODER = ByteArrayCoder.of(); + + private static final NullableCoder LONG_CODER = NullableCoder.of(VarLongCoder.of()); + private static final NullableCoder INTEGER_CODER = NullableCoder.of(VarIntCoder.of()); + private static final NullableCoder STRING_CODER = + NullableCoder.of(StringUtf8Coder.of()); + private static final NullableCoder BOOLEAN_CODER = NullableCoder.of(BooleanCoder.of()); + + public static SolaceRecordCoder of() { + return new SolaceRecordCoder(); + } + + @Override + public void encode(Record value, OutputStream outStream) throws IOException { + // Check if the destination is a topic or a queue, and encode that info + STRING_CODER.encode(value.getMessageId(), outStream); + STRING_CODER.encode(value.getReplicationGroupMessageId(), outStream); + BYTE_CODER.encode(value.getPayload(), outStream); + STRING_CODER.encode(value.getDestination().getName(), outStream); + STRING_CODER.encode(value.getDestination().getType().toString(), outStream); + LONG_CODER.encode(value.getExpiration(), outStream); + INTEGER_CODER.encode(value.getPriority(), outStream); + BOOLEAN_CODER.encode(value.getRedelivered(), outStream); + STRING_CODER.encode(value.getReplyTo(), outStream); + LONG_CODER.encode(value.getReceiveTimestamp(), outStream); + LONG_CODER.encode(value.getSenderTimestamp(), outStream); + LONG_CODER.encode(value.getSequenceNumber(), outStream); + LONG_CODER.encode(value.getTimeToLive(), outStream); + } + + @Override + public Record decode(InputStream inStream) throws IOException { + return Record.builder() + .setMessageId(STRING_CODER.decode(inStream)) + .setReplicationGroupMessageId(STRING_CODER.decode(inStream)) + .setPayload(BYTE_CODER.decode(inStream)) + .setDestination( + Solace.Destination.builder() + .setName(STRING_CODER.decode(inStream)) + .setType( + Objects.equals(STRING_CODER.decode(inStream), "QUEUE") + ? Solace.DestinationType.QUEUE + : Solace.DestinationType.TOPIC) + .build()) + .setExpiration(LONG_CODER.decode(inStream)) + .setPriority(INTEGER_CODER.decode(inStream)) + .setRedelivered(BOOLEAN_CODER.decode(inStream)) + .setReplyTo(STRING_CODER.decode(inStream)) + .setReceiveTimestamp(LONG_CODER.decode(inStream)) + .setSenderTimestamp(LONG_CODER.decode(inStream)) + .setSequenceNumber(LONG_CODER.decode(inStream)) + .setTimeToLive(LONG_CODER.decode(inStream)) + .build(); + } +} diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/SolaceCheckpointMark.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/SolaceCheckpointMark.java new file mode 100644 index 000000000000..ba3c51f43949 --- /dev/null +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/SolaceCheckpointMark.java @@ -0,0 +1,90 @@ +/* + * Copyright 2023 Google. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.google.cloud.dataflow.dce.io.solace.read; + +import com.solacesystems.jcsmp.BytesXMLMessage; +import java.util.List; +import java.util.Objects; +import java.util.concurrent.ConcurrentLinkedDeque; +import java.util.concurrent.atomic.AtomicBoolean; +import org.apache.beam.sdk.annotations.Internal; +import org.apache.beam.sdk.coders.DefaultCoder; +import org.apache.beam.sdk.extensions.avro.coders.AvroCoder; +import org.apache.beam.sdk.io.UnboundedSource; +import org.checkerframework.checker.nullness.qual.Nullable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Checkpoint for an unbounded Solace source. Consists of the Solace messages waiting to be + * acknowledged and oldest pending message timestamp. + */ +@Internal +@DefaultCoder(AvroCoder.class) +public class SolaceCheckpointMark implements UnboundedSource.CheckpointMark { + private static final Logger LOG = LoggerFactory.getLogger(SolaceCheckpointMark.class); + + private transient AtomicBoolean activeReader; + @Nullable private transient ConcurrentLinkedDeque ackQueue; + + @SuppressWarnings("initialization") // Avro will set the fields by breaking abstraction + private SolaceCheckpointMark() {} // for Avro + + public SolaceCheckpointMark( + @Nullable AtomicBoolean activeReader, List ackQueue) { + this.activeReader = activeReader; + if (ackQueue != null) { + this.ackQueue = new ConcurrentLinkedDeque<>(ackQueue); + } + } + + @Override + public void finalizeCheckpoint() { + if (activeReader == null || !activeReader.get() || ackQueue == null) { + return; + } + + LOG.debug( + "SolaceIO.Read: SolaceCheckpointMark: Started to finalize {} with {} messages.", + this.getClass().getSimpleName(), + ackQueue.size()); + + while (ackQueue.size() > 0) { + BytesXMLMessage msg = ackQueue.poll(); + if (msg != null) { + msg.ackMessage(); + } + } + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (!(o instanceof SolaceCheckpointMark)) { + return false; + } + SolaceCheckpointMark that = (SolaceCheckpointMark) o; + return Objects.equals(activeReader, that.activeReader) + && Objects.equals(ackQueue, that.ackQueue); + } + + @Override + public int hashCode() { + return Objects.hash(activeReader, ackQueue); + } +} diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/UnboundedSolaceReader.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/UnboundedSolaceReader.java new file mode 100644 index 000000000000..fea668bb47f2 --- /dev/null +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/UnboundedSolaceReader.java @@ -0,0 +1,181 @@ +/* + * Copyright 2023 Google. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.google.cloud.dataflow.dce.io.solace.read; + +import com.google.cloud.dataflow.dce.io.solace.broker.MessageReceiver; +import com.google.cloud.dataflow.dce.io.solace.broker.SempClient; +import com.google.cloud.dataflow.dce.io.solace.broker.SessionService; +import com.google.common.annotations.VisibleForTesting; +import com.solacesystems.jcsmp.BytesXMLMessage; +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.util.ArrayDeque; +import java.util.ArrayList; +import java.util.List; +import java.util.NoSuchElementException; +import java.util.concurrent.atomic.AtomicBoolean; +import org.apache.beam.sdk.io.UnboundedSource; +import org.apache.beam.sdk.io.UnboundedSource.UnboundedReader; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.joda.time.Instant; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** Unbounded Reader to read messages from a Solace Router. */ +@VisibleForTesting +class UnboundedSolaceReader extends UnboundedSource.UnboundedReader { + + private static final Logger LOG = LoggerFactory.getLogger(UnboundedSolaceReader.class); + private final UnboundedSolaceSource currentSource; + private final WatermarkPolicy watermarkPolicy; + AtomicBoolean active = new AtomicBoolean(true); + private BytesXMLMessage solaceOriginalRecord; + private T solaceMappedRecord; + private MessageReceiver messageReceiver; + private SessionService sessionService; + private final SempClient sempClient; + + /** + * Queue to place advanced messages before {@link #getCheckpointMark()} be called non-concurrent + * queue, should only be accessed by the reader thread A given {@link UnboundedReader} object + * will only be accessed by a single thread at once. + */ + private final java.util.Queue elementsToCheckpoint = new ArrayDeque<>(); + + public UnboundedSolaceReader(UnboundedSolaceSource currentSource) { + this.currentSource = currentSource; + this.watermarkPolicy = WatermarkPolicy.create(currentSource.getTimestampFn()); + this.sessionService = currentSource.getSessionServiceFactory().create(); + this.sempClient = currentSource.getSempClientFactory().create(); + } + + @Override + public boolean start() { + populateSession(); + populateMessageConsumer(); + return advance(); + } + + public void populateSession() { + if (sessionService == null) { + sessionService = getCurrentSource().getSessionServiceFactory().create(); + } + if (sessionService.isClosed()) { + sessionService.connect(); + } + } + + private void populateMessageConsumer() { + if (messageReceiver == null) { + messageReceiver = sessionService.createReceiver(); + messageReceiver.start(); + } + if (messageReceiver.isClosed()) { + messageReceiver.start(); + } + } + + @Override + public boolean advance() { + BytesXMLMessage receivedXmlMessage; + try { + receivedXmlMessage = messageReceiver.receive(); + } catch (IOException e) { + LOG.warn("SolaceIO.Read: Exception when pulling messages from the broker.", e); + return false; + } + + if (receivedXmlMessage == null) { + return false; + } + elementsToCheckpoint.add(receivedXmlMessage); + solaceOriginalRecord = receivedXmlMessage; + solaceMappedRecord = getCurrentSource().getParseFn().apply(receivedXmlMessage); + watermarkPolicy.update(solaceMappedRecord); + return true; + } + + @Override + public void close() { + active.set(false); + sessionService.close(); + } + + @Override + public Instant getWatermark() { + // should be only used by a test receiver + if (messageReceiver.isEOF()) { + return BoundedWindow.TIMESTAMP_MAX_VALUE; + } + return watermarkPolicy.getWatermark(); + } + + @Override + public UnboundedSource.CheckpointMark getCheckpointMark() { + List ackQueue = new ArrayList<>(); + while (!elementsToCheckpoint.isEmpty()) { + BytesXMLMessage msg = elementsToCheckpoint.poll(); + ackQueue.add(msg); + } + return new SolaceCheckpointMark(active, ackQueue); + } + + @Override + public T getCurrent() throws NoSuchElementException { + if (solaceMappedRecord == null) { + throw new NoSuchElementException(); + } + return solaceMappedRecord; + } + + @Override + public byte[] getCurrentRecordId() throws NoSuchElementException { + if (solaceOriginalRecord == null) { + throw new NoSuchElementException(); + } + if (solaceOriginalRecord.getApplicationMessageId() != null) { + return solaceOriginalRecord.getApplicationMessageId().getBytes(StandardCharsets.UTF_8); + } else { + return solaceOriginalRecord + .getReplicationGroupMessageId() + .toString() + .getBytes(StandardCharsets.UTF_8); + } + } + + @Override + public UnboundedSolaceSource getCurrentSource() { + return currentSource; + } + + @Override + public Instant getCurrentTimestamp() throws NoSuchElementException { + if (getCurrent() == null) { + throw new NoSuchElementException(); + } + return currentSource.getTimestampFn().apply(getCurrent()); + } + + @Override + public long getTotalBacklogBytes() { + try { + return sempClient.getBacklogBytes(currentSource.getQueue().getName()); + } catch (IOException e) { + LOG.warn("SolaceIO.Read: Could not query backlog bytes. Returning BACKLOG_UNKNOWN", e); + return BACKLOG_UNKNOWN; + } + } +} diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/UnboundedSolaceSource.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/UnboundedSolaceSource.java new file mode 100644 index 000000000000..fde3c2d93540 --- /dev/null +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/UnboundedSolaceSource.java @@ -0,0 +1,148 @@ +/* + * Copyright 2023 Google. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.google.cloud.dataflow.dce.io.solace.read; + +import com.google.cloud.dataflow.dce.io.solace.broker.SempClientFactory; +import com.google.cloud.dataflow.dce.io.solace.broker.SessionServiceFactory; +import com.solacesystems.jcsmp.BytesXMLMessage; +import com.solacesystems.jcsmp.Queue; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import javax.annotation.Nullable; +import org.apache.beam.sdk.annotations.Internal; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.extensions.avro.coders.AvroCoder; +import org.apache.beam.sdk.io.UnboundedSource; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.transforms.SerializableFunction; +import org.joda.time.Instant; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@Internal +public class UnboundedSolaceSource extends UnboundedSource { + private static final long serialVersionUID = 42L; + private static final Logger LOG = LoggerFactory.getLogger(UnboundedSolaceSource.class); + private final Queue queue; + private final Integer maxNumConnections; + private final Coder coder; + private final boolean enableDeduplication; + private final SempClientFactory sempClientFactory; + private final SessionServiceFactory sessionServiceFactory; + private final SerializableFunction timestampFn; + private final SerializableFunction parseFn; + + public Queue getQueue() { + return queue; + } + + public SessionServiceFactory getSessionServiceFactory() { + return sessionServiceFactory; + } + + public SempClientFactory getSempClientFactory() { + return sempClientFactory; + } + + public SerializableFunction getTimestampFn() { + return timestampFn; + } + + public SerializableFunction getParseFn() { + return parseFn; + } + + public UnboundedSolaceSource( + Queue queue, + SempClientFactory sempClientFactory, + SessionServiceFactory sessionServiceFactory, + Integer maxNumConnections, + boolean enableDeduplication, + Coder coder, + SerializableFunction timestampFn, + SerializableFunction parseFn) { + this.queue = queue; + this.sempClientFactory = sempClientFactory; + this.sessionServiceFactory = sessionServiceFactory; + this.maxNumConnections = maxNumConnections; + this.enableDeduplication = enableDeduplication; + this.coder = coder; + this.timestampFn = timestampFn; + this.parseFn = parseFn; + } + + @Override + public UnboundedReader createReader( + PipelineOptions options, @Nullable SolaceCheckpointMark checkpointMark) { + // it makes no sense to resume a Solace Session with the previous checkpoint + // so don't need the pass a checkpoint to new a Solace Reader + return new UnboundedSolaceReader<>(this); + } + + @Override + public List> split(int desiredNumSplits, PipelineOptions options) + throws IOException { + boolean queueNonExclusive = sempClientFactory.create().isQueueNonExclusive(queue.getName()); + if (queueNonExclusive) { + return getSolaceSources(desiredNumSplits, maxNumConnections); + } else { + LOG.warn( + "SolaceIO.Read: The queue {} is exclusive. Provisioning only 1 read client.", + queue); + return getSolaceSources(desiredNumSplits, 1); + } + } + + private List> getSolaceSources( + int desiredNumSplits, Integer maxNumConnections) { + List> sourceList = new ArrayList<>(); + int numSplits = + maxNumConnections != null + ? Math.min(desiredNumSplits, maxNumConnections) + : desiredNumSplits; + LOG.info("SolaceIO.Read: UnboundedSolaceSource: creating {} read connections.", numSplits); + for (int i = 0; i < numSplits; i++) { + UnboundedSolaceSource source = + new UnboundedSolaceSource<>( + queue, + sempClientFactory, + sessionServiceFactory, + maxNumConnections, + enableDeduplication, + coder, + timestampFn, + parseFn); + sourceList.add(source); + } + return sourceList; + } + + @Override + public Coder getCheckpointMarkCoder() { + return AvroCoder.of(SolaceCheckpointMark.class); + } + + @Override + public Coder getOutputCoder() { + return coder; + } + + @Override + public boolean requiresDeduping() { + return enableDeduplication; + } +} diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/WatermarkParameters.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/WatermarkParameters.java new file mode 100644 index 000000000000..f1f46ca26b89 --- /dev/null +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/WatermarkParameters.java @@ -0,0 +1,88 @@ +/* + * Copyright 2024 Google. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.google.cloud.dataflow.dce.io.solace.read; + +import static com.google.common.base.Preconditions.checkArgument; + +import com.google.auto.value.AutoValue; +import java.io.Serializable; +import org.apache.beam.sdk.transforms.SerializableFunction; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.joda.time.Duration; +import org.joda.time.Instant; + +/** {@code WatermarkParameters} contains the parameters used for watermark computation. */ +@AutoValue +public abstract class WatermarkParameters implements Serializable { + + private static final Duration STANDARD_WATERMARK_IDLE_DURATION_THRESHOLD = + Duration.standardSeconds(30); + + abstract Instant getCurrentWatermark(); + + abstract Instant getLastSavedWatermark(); + + abstract Instant getLastUpdateTime(); + + abstract SerializableFunction getTimestampFn(); + + abstract Duration getWatermarkIdleDurationThreshold(); + + public abstract Builder toBuilder(); + + static Builder builder() { + return new AutoValue_WatermarkParameters.Builder() + .setCurrentWatermark(BoundedWindow.TIMESTAMP_MIN_VALUE) + .setLastSavedWatermark(BoundedWindow.TIMESTAMP_MIN_VALUE) + .setLastUpdateTime(Instant.now()) + .setWatermarkIdleDurationThreshold(STANDARD_WATERMARK_IDLE_DURATION_THRESHOLD); + } + + @AutoValue.Builder + abstract static class Builder { + abstract Builder setCurrentWatermark(Instant currentWatermark); + + abstract Builder setLastSavedWatermark(Instant eventTime); + + abstract Builder setLastUpdateTime(Instant now); + + abstract Builder setWatermarkIdleDurationThreshold( + Duration watermarkIdleDurationThreshold); + + abstract Builder setTimestampFn(SerializableFunction timestampFn); + + abstract WatermarkParameters build(); + } + + /** + * Create an instance of {@link WatermarkParameters} with a {@code SerializableFunction} to + * extract the event time. + */ + public static WatermarkParameters create(SerializableFunction timestampFn) { + checkArgument(timestampFn != null, "timestampFn function is null"); + return WatermarkParameters.builder().setTimestampFn(timestampFn).build(); + } + + /** + * Specify the watermark idle duration to consider before advancing the watermark. The default + * watermark idle duration threshold is {@link #STANDARD_WATERMARK_IDLE_DURATION_THRESHOLD}. + */ + public WatermarkParameters withWatermarkIdleDurationThreshold( + Duration idleDurationThreshold) { + checkArgument(idleDurationThreshold != null, "watermark idle duration threshold is null"); + return toBuilder().setWatermarkIdleDurationThreshold(idleDurationThreshold).build(); + } +} diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/WatermarkPolicy.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/WatermarkPolicy.java new file mode 100644 index 000000000000..3dfb0879f939 --- /dev/null +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/WatermarkPolicy.java @@ -0,0 +1,66 @@ +/* + * Copyright 2024 Google. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.google.cloud.dataflow.dce.io.solace.read; + +import com.google.common.collect.Ordering; +import java.io.Serializable; +import org.apache.beam.sdk.transforms.SerializableFunction; +import org.joda.time.Instant; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class WatermarkPolicy implements Serializable { + public static final Logger LOG = LoggerFactory.getLogger(WatermarkPolicy.class); + private WatermarkParameters watermarkParameters; + + public static WatermarkPolicy create( + SerializableFunction timestampFunction) { + return new WatermarkPolicy(WatermarkParameters.create(timestampFunction)); + } + + private WatermarkPolicy(WatermarkParameters watermarkParameters) { + this.watermarkParameters = watermarkParameters; + } + + public Instant getWatermark() { + Instant now = Instant.now(); + Instant watermarkIdleThreshold = + now.minus(watermarkParameters.getWatermarkIdleDurationThreshold()); + + Instant newWatermark = + watermarkParameters.getLastUpdateTime().isBefore(watermarkIdleThreshold) + ? watermarkIdleThreshold + : watermarkParameters.getLastSavedWatermark(); + + if (newWatermark.isAfter(watermarkParameters.getCurrentWatermark())) { + watermarkParameters = + watermarkParameters.toBuilder().setCurrentWatermark(newWatermark).build(); + } + return watermarkParameters.getCurrentWatermark(); + } + + public void update(T record) { + watermarkParameters = + watermarkParameters.toBuilder() + .setLastSavedWatermark( + Ordering.natural() + .max( + watermarkParameters.getLastSavedWatermark(), + watermarkParameters.getTimestampFn().apply(record))) + .setLastUpdateTime(Instant.now()) + .build(); + } +} diff --git a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/MockSempClient.java b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/MockSempClient.java new file mode 100644 index 000000000000..1599d5587164 --- /dev/null +++ b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/MockSempClient.java @@ -0,0 +1,86 @@ +/* + * Copyright 2024 Google. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.solace.io.solace; + +import org.apache.beam.sdk.io.solace.broker.SempClient; +import com.solacesystems.jcsmp.JCSMPFactory; +import com.solacesystems.jcsmp.Queue; +import java.io.IOException; +import org.apache.beam.sdk.transforms.SerializableFunction; + +public class MockSempClient implements SempClient { + + private final SerializableFunction isQueueNonExclusiveFn; + private final SerializableFunction getBacklogBytesFn; + private final SerializableFunction createQueueForTopicFn; + + private MockSempClient( + SerializableFunction isQueueNonExclusiveFn, + SerializableFunction getBacklogBytesFn, + SerializableFunction createQueueForTopicFn) { + this.isQueueNonExclusiveFn = isQueueNonExclusiveFn; + this.getBacklogBytesFn = getBacklogBytesFn; + this.createQueueForTopicFn = createQueueForTopicFn; + } + + public static Builder builder() { + return new Builder(); + } + + public static class Builder { + private SerializableFunction isQueueNonExclusiveFn = (queueName) -> true; + private SerializableFunction getBacklogBytesFn = (queueName) -> 0L; + private SerializableFunction createQueueForTopicFn = (queueName) -> 0; + + public Builder setIsQueueNonExclusiveFn( + SerializableFunction isQueueNonExclusiveFn) { + this.isQueueNonExclusiveFn = isQueueNonExclusiveFn; + return this; + } + + public Builder setGetBacklogBytesFn(SerializableFunction getBacklogBytesFn) { + this.getBacklogBytesFn = getBacklogBytesFn; + return this; + } + + public Builder setCreateQueueForTopicFn( + SerializableFunction createQueueForTopicFn) { + this.createQueueForTopicFn = createQueueForTopicFn; + return this; + } + + public MockSempClient build() { + return new MockSempClient( + isQueueNonExclusiveFn, getBacklogBytesFn, createQueueForTopicFn); + } + } + + @Override + public boolean isQueueNonExclusive(String queueName) throws IOException { + return isQueueNonExclusiveFn.apply(queueName); + } + + @Override + public Queue createQueueForTopic(String queueName, String topicName) throws IOException { + createQueueForTopicFn.apply(queueName); + return JCSMPFactory.onlyInstance().createQueue(queueName); + } + + @Override + public long getBacklogBytes(String queueName) throws IOException { + return getBacklogBytesFn.apply(queueName); + } +} diff --git a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/MockSempClientFactory.java b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/MockSempClientFactory.java new file mode 100644 index 000000000000..271bca073ee2 --- /dev/null +++ b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/MockSempClientFactory.java @@ -0,0 +1,33 @@ +/* + * Copyright 2024 Google. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.solace.io.solace; + +import org.apache.beam.sdk.io.solace.broker.SempClient; +import org.apache.beam.sdk.io.solace.broker.SempClientFactory; + +public class MockSempClientFactory implements SempClientFactory { + MockSempClient sempClient; + + public MockSempClientFactory(MockSempClient sempClient) { + this.sempClient = sempClient; + } + + @Override + public SempClient create() { + // todo new or existing instance? + return sempClient; + } +} diff --git a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/MockSessionService.java b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/MockSessionService.java new file mode 100644 index 000000000000..2f9fbd0908d6 --- /dev/null +++ b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/MockSessionService.java @@ -0,0 +1,87 @@ +/* + * Copyright 2024 Google. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.solace.io.solace; + +import org.apache.beam.sdk.io.solace.broker.MessageReceiver; +import org.apache.beam.sdk.io.solace.broker.SessionService; +import com.solacesystems.jcsmp.BytesXMLMessage; +import java.io.IOException; +import java.io.Serializable; +import java.util.concurrent.atomic.AtomicInteger; +import org.apache.beam.sdk.transforms.SerializableFunction; + +public class MockSessionService implements SessionService { + + private final SerializableFunction getRecordFn; + private MessageReceiver messageReceiver = null; + private final int minMessagesReceived; + + public MockSessionService( + SerializableFunction getRecordFn, int minMessagesReceived) { + this.getRecordFn = getRecordFn; + this.minMessagesReceived = minMessagesReceived; + } + + @Override + public void close() {} + + @Override + public boolean isClosed() { + return false; + } + + @Override + public MessageReceiver createReceiver() { + if (messageReceiver == null) { + messageReceiver = new MockReceiver(getRecordFn, minMessagesReceived); + } + return messageReceiver; + } + + @Override + public void connect() {} + + public static class MockReceiver implements MessageReceiver, Serializable { + private final AtomicInteger counter = new AtomicInteger(); + private final SerializableFunction getRecordFn; + private final int minMessagesReceived; + + public MockReceiver( + SerializableFunction getRecordFn, + int minMessagesReceived) { + this.getRecordFn = getRecordFn; + this.minMessagesReceived = minMessagesReceived; + } + + @Override + public void start() {} + + @Override + public boolean isClosed() { + return false; + } + + @Override + public BytesXMLMessage receive() throws IOException { + return getRecordFn.apply(counter.getAndIncrement()); + } + + @Override + public boolean isEOF() { + return counter.get() >= minMessagesReceived; + } + } +} diff --git a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/MockSessionServiceFactory.java b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/MockSessionServiceFactory.java new file mode 100644 index 000000000000..fc83ef6def49 --- /dev/null +++ b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/MockSessionServiceFactory.java @@ -0,0 +1,32 @@ +/* + * Copyright 2024 Google. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.solace.io.solace; + +import org.apache.beam.sdk.io.solace.broker.SessionService; +import org.apache.beam.sdk.io.solace.broker.SessionServiceFactory; + +public class MockSessionServiceFactory extends SessionServiceFactory { + SessionService sessionService; + + public MockSessionServiceFactory(MockSessionService clientService) { + this.sessionService = clientService; + } + + @Override + public SessionService create() { + return sessionService; + } +} diff --git a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/RetryCallableManagerTest.java b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/RetryCallableManagerTest.java new file mode 100644 index 000000000000..e37da3541837 --- /dev/null +++ b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/RetryCallableManagerTest.java @@ -0,0 +1,175 @@ +/* + * Copyright 2024 Google. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.solace.io.solace; + +import static org.junit.Assert.assertTrue; + +import com.google.api.gax.retrying.RetrySettings; +import com.google.cloud.RetryHelper.RetryHelperException; +import java.util.Set; +import java.util.concurrent.Callable; +import java.util.concurrent.atomic.AtomicInteger; +import org.apache.beam.sdk.io.solace.RetryCallableManager; +import org.junit.Before; +import org.junit.Test; + +public class RetryCallableManagerTest { + + private RetryCallableManager retryCallableManager; + + @Before + public void setUp() throws Exception { + int NUMBER_OF_RETRIES = 4; + int RETRY_INTERVAL_SECONDS = 0; + int RETRY_MULTIPLIER = 2; + int MAX_DELAY = 0; + + retryCallableManager = + RetryCallableManager.builder() + .setRetrySettings( + RetrySettings.newBuilder() + .setInitialRetryDelay( + org.threeten.bp.Duration.ofSeconds( + RETRY_INTERVAL_SECONDS)) + .setMaxAttempts(NUMBER_OF_RETRIES) + .setMaxRetryDelay( + org.threeten.bp.Duration.ofSeconds(MAX_DELAY)) + .setRetryDelayMultiplier(RETRY_MULTIPLIER) + .build()) + .build(); + } + + @Test + public void testRetryCallable_ReturnsExpected() { + AtomicInteger executeCounter = new AtomicInteger(0); + Callable incrementingFunction = + () -> { + executeCounter.incrementAndGet(); + if (executeCounter.get() < 2) { + throw new MyException(); + } + return executeCounter.get(); + }; + Integer result = + retryCallableManager.retryCallable(incrementingFunction, Set.of(MyException.class)); + assertTrue(String.format("Should return 2, instead returned %d.", result), result == 2); + } + + @Test + public void testRetryCallable_RetriesExpectedNumberOfTimes() { + AtomicInteger executeCounter = new AtomicInteger(0); + Callable incrementingFunction = + () -> { + executeCounter.incrementAndGet(); + if (executeCounter.get() < 2) { + throw new MyException(); + } + return executeCounter.get(); + }; + retryCallableManager.retryCallable(incrementingFunction, Set.of(MyException.class)); + assertTrue( + String.format("Should run 2 times, instead ran %d times.", executeCounter.get()), + executeCounter.get() == 2); + } + + @Test(expected = RetryHelperException.class) + public void testRetryCallable_ThrowsRetryHelperException() { + Callable incrementingFunction = + () -> { + { + throw new MyException(); + } + }; + retryCallableManager.retryCallable(incrementingFunction, Set.of(MyException.class)); + } + + @Test + public void testRetryCallable_ExecutionCountIsCorrectAfterMultipleExceptions() { + AtomicInteger executeCounter = new AtomicInteger(0); + Callable incrementingFunction = + () -> { + executeCounter.incrementAndGet(); + if (true) { + throw new MyException(); + } + return 0; + }; + try { + retryCallableManager.retryCallable(incrementingFunction, Set.of(MyException.class)); + } catch (RetryHelperException e) { + // ignore exception to check the executeCounter + } + assertTrue( + String.format( + "Should execute 4 times, instead executed %d times", executeCounter.get()), + executeCounter.get() == 4); + } + + @Test(expected = RetryHelperException.class) + public void testRetryCallable_ThrowsRetryHelperExceptionOnUnspecifiedException() { + Callable incrementingFunction = + () -> { + if (true) { + throw new DoNotIgnoreException(); + } + return 0; + }; + retryCallableManager.retryCallable(incrementingFunction, Set.of(MyException.class)); + } + + @Test + public void testRetryCallable_ChecksForAllDefinedExceptions() { + AtomicInteger executeCounter = new AtomicInteger(0); + Callable incrementingFunction = + () -> { + executeCounter.incrementAndGet(); + if (executeCounter.get() % 2 == 0) { + throw new MyException(); + } else if (executeCounter.get() % 2 == 1) { + throw new AnotherException(); + } + return 0; + }; + try { + retryCallableManager.retryCallable( + incrementingFunction, Set.of(MyException.class, AnotherException.class)); + } catch (RetryHelperException e) { + // ignore exception to check the executeCounter + } + assertTrue( + String.format( + "Should execute 4 times, instead executed %d times", executeCounter.get()), + executeCounter.get() == 4); + } + + private static class MyException extends Exception { + public MyException() { + super(); + } + } + + private static class AnotherException extends Exception { + public AnotherException() { + super(); + } + } + + private static class DoNotIgnoreException extends Exception { + public DoNotIgnoreException() { + super(); + } + } +} diff --git a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/SolaceDataUtils.java b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/SolaceDataUtils.java new file mode 100644 index 000000000000..d7d96b3031e3 --- /dev/null +++ b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/SolaceDataUtils.java @@ -0,0 +1,774 @@ +/* + * Copyright 2024 Google. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.solace.io.solace; + +import org.apache.beam.sdk.io.solace.SolaceIO; +import org.apache.beam.sdk.io.solace.data.Solace; +import com.solacesystems.jcsmp.BytesXMLMessage; +import com.solacesystems.jcsmp.DeliveryMode; +import com.solacesystems.jcsmp.Destination; +import com.solacesystems.jcsmp.JCSMPException; +import com.solacesystems.jcsmp.MessageType; +import com.solacesystems.jcsmp.ReplicationGroupMessageId; +import com.solacesystems.jcsmp.SDTMap; +import com.solacesystems.jcsmp.User_Cos; +import java.io.IOException; +import java.io.InputStream; +import java.nio.BufferUnderflowException; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.util.Collection; +import java.util.List; +import java.util.Objects; +import org.apache.beam.sdk.schemas.JavaBeanSchema; +import org.apache.beam.sdk.schemas.annotations.DefaultSchema; +import org.apache.beam.sdk.transforms.SerializableFunction; + +public class SolaceDataUtils { + + @DefaultSchema(JavaBeanSchema.class) + public static class SimpleRecord { + public String payload; + public String messageId; + + public SimpleRecord() {} + + public SimpleRecord(String payload, String messageId) { + this.payload = payload; + this.messageId = messageId; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (!(o instanceof SimpleRecord)) { + return false; + } + SimpleRecord that = (SimpleRecord) o; + return Objects.equals(payload, that.payload) + && Objects.equals(messageId, that.messageId); + } + + @Override + public int hashCode() { + return Objects.hash(payload, messageId); + } + + @Override + public String toString() { + return "SimpleRecord{" + + "payload='" + + payload + + '\'' + + ", messageId='" + + messageId + + '\'' + + '}'; + } + } + + public static Solace.Record getSolaceRecord(String payload, String messageId) { + return Solace.Record.builder() + .setPayload(payload.getBytes(StandardCharsets.UTF_8)) + .setMessageId(messageId) + .setDestination( + Solace.Destination.builder() + .setName("destination-topic") + .setType(Solace.DestinationType.TOPIC) + .build()) + .setExpiration(1000L) + .setPriority(0) + .setReceiveTimestamp(1708100477067L) + .setRedelivered(false) + .setReplyTo(null) + .setSequenceNumber(null) + .setTimeToLive(1000L) + .setSenderTimestamp(null) + .build(); + } + + public static BytesXMLMessage getBytesXmlMessage(String payload) { + return getBytesXmlMessage(payload, "messageId", null); + } + + public static BytesXMLMessage getBytesXmlMessage(String payload, String messageId) { + return getBytesXmlMessage(payload, messageId, null); + } + + public static BytesXMLMessage getBytesXmlMessage( + String payload, String messageId, SerializableFunction ackMessageFn) { + long receiverTimestamp = 1708100477067L; + long expiration = 1000L; + long timeToLive = 1000L; + String destination = "destination-topic"; + + return new BytesXMLMessage() { + + @Override + public byte[] getBytes() { + return payload.getBytes(StandardCharsets.UTF_8); + } + + @Override + public int getContentLength() { + return payload.getBytes(StandardCharsets.UTF_8).length; + } + + @Override + public int readBytes(byte[] arg0) { + return 0; + } + + @Override + public int readBytes(byte[] arg0, int arg1) { + return 0; + } + + @Override + public void rewindContent() { + // return null; + } + + @Override + public void writeBytes(byte[] arg0) { + // return null; + } + + @Override + public void writeBytes(byte[] arg0, int arg1, int arg2) { + // return null; + } + + @Override + public void ackMessage() { + if (ackMessageFn != null) { + ackMessageFn.apply(0); + } + } + + @Override + public void clearAttachment() { + // return null; + } + + @Override + public void clearBinaryMetadataBytes(int arg0) { + // return null; + } + + @Override + public void clearContent() { + // return null; + } + + @Override + public void clearQueueNameLocation() { + // return null; + } + + @Override + public void clearTopicNameLocation() { + // return null; + } + + @Override + public String dump() { + return null; + } + + @Override + public String dump(int arg0) { + return null; + } + + @Override + public long getAckMessageId() { + return 0; + } + + @Override + public String getAppMessageID() { + return null; + } + + @Override + public String getAppMessageType() { + return null; + } + + @Override + public String getApplicationMessageId() { + return messageId; + } + + @Override + public String getApplicationMessageType() { + return null; + } + + @Override + public ByteBuffer getAttachmentByteBuffer() { + return null; + } + + @Override + public int getAttachmentContentLength() { + return 0; + } + + @Override + public int getBinaryMetadataContentLength(int arg0) { + return 0; + } + + @Override + public Collection getBinaryMetadataTypes() { + return null; + } + + @Override + public Long getCacheRequestId() { + return null; + } + + @Override + public List getConsumerIdList() { + return null; + } + + @Override + public String getCorrelationId() { + return null; + } + + @Override + public Object getCorrelationKey() { + return null; + } + + @Override + public User_Cos getCos() { + return null; + } + + @Override + public boolean getDeliverToOne() { + return false; + } + + @Override + public int getDeliveryCount() throws UnsupportedOperationException { + return 0; + } + + @Override + public DeliveryMode getDeliveryMode() { + return null; + } + + @Override + public Destination getDestination() { + return SolaceIO.topicFromName(destination); + } + + @Override + public String getDestinationTopicSuffix() { + return null; + } + + @Override + public boolean getDiscardIndication() { + return false; + } + + @Override + public long getExpiration() { + return expiration; + } + + @Override + public String getHTTPContentEncoding() { + return null; + } + + @Override + public String getHTTPContentType() { + return null; + } + + @Override + public String getMessageId() { + return null; + } + + @Override + public long getMessageIdLong() { + return 0; + } + + @Override + public MessageType getMessageType() { + return null; + } + + @Override + public int getPriority() { + return 0; + } + + @Override + public SDTMap getProperties() { + return null; + } + + @Override + public int getQueueNameLength() { + return 0; + } + + @Override + public int getQueueNameOffset() { + return 0; + } + + @Override + public long getReceiveTimestamp() { + return receiverTimestamp; + } + + @Override + public boolean getRedelivered() { + return false; + } + + @Override + public ReplicationGroupMessageId getReplicationGroupMessageId() { + return null; + } + + @Override + public Destination getReplyTo() { + return null; + } + + @Override + public String getReplyToSuffix() { + return null; + } + + @Override + public Long getSendTimestamp() { + return null; + } + + @Override + public String getSenderID() { + return null; + } + + @Override + public String getSenderId() { + return null; + } + + @Override + public Long getSenderTimestamp() { + return null; + } + + @Override + public Long getSequenceNumber() { + return null; + } + + @Override + public byte getStructuredMsgType() { + return 0x2; + } + + @Override + public boolean getTQDiscardIndication() { + return false; + } + + @Override + public long getTimeToLive() { + return timeToLive; + } + + @Override + public int getTopicNameLength() { + return 5; + } + + @Override + public int getTopicNameOffset() { + return 0; + } + + @Override + public Long getTopicSequenceNumber() { + return null; + } + + @Override + public byte[] getUserData() { + return null; + } + + @Override + public boolean hasAttachment() { + return false; + } + + @Override + public boolean hasBinaryMetadata(int arg0) { + return false; + } + + @Override + public boolean hasContent() { + return false; + } + + @Override + public boolean hasUserData() { + return false; + } + + @Override + public boolean isAckImmediately() { + return false; + } + + @Override + public boolean isCacheMessage() { + return false; + } + + @Override + public boolean isDMQEligible() { + return false; + } + + @Override + public boolean isDeliveryCountSupported() { + return false; + } + + @Override + public boolean isElidingEligible() { + return false; + } + + @Override + public boolean isReadOnly() { + return false; + } + + @Override + public boolean isReplyMessage() { + return false; + } + + @Override + public boolean isStructuredMsg() { + return false; + } + + @Override + public boolean isSuspect() { + return false; + } + + @Override + public int readAttachmentBytes(byte[] arg0) { + return 0; + } + + @Override + public int readAttachmentBytes(byte[] arg0, int arg1) { + return 0; + } + + @Override + public int readAttachmentBytes(int arg0, byte[] arg1, int arg2, int arg3) { + return 0; + } + + @Override + public int readBinaryMetadataBytes(int arg0, byte[] arg1) { + return 0; + } + + @Override + public int readContentBytes(byte[] arg0) { + return 0; + } + + @Override + public int readContentBytes(byte[] arg0, int arg1) { + return 0; + } + + @Override + public int readContentBytes(int arg0, byte[] arg1, int arg2, int arg3) { + return 0; + } + + @Override + public void rejectMessage() { + // return null; + } + + @Override + public void reset() { + // return null; + } + + @Override + public void resetPayload() { + // return null; + } + + @Override + public void rewindAttachment() { + // return null; + } + + @Override + public void setAckImmediately(boolean arg0) { + // return null; + } + + @Override + public void setAppMessageID(String arg0) { + // return null; + } + + @Override + public void setAppMessageType(String arg0) { + // return null; + } + + @Override + public void setApplicationMessageId(String arg0) { + // return null; + } + + @Override + public void setApplicationMessageType(String arg0) { + // return null; + } + + @Override + public void setAsReplyMessage(boolean arg0) { + // return null; + } + + @Override + public void setCorrelationId(String arg0) { + // return null; + } + + @Override + public void setCorrelationKey(Object arg0) { + // return null; + } + + @Override + public void setCos(User_Cos arg0) { + // return null; + } + + @Override + public void setDMQEligible(boolean arg0) { + // return null; + } + + @Override + public void setDeliverToOne(boolean arg0) { + // return null; + } + + @Override + public void setDeliveryMode(DeliveryMode arg0) { + // return null; + } + + @Override + public void setElidingEligible(boolean arg0) { + // return null; + } + + @Override + public void setExpiration(long arg0) { + // return null; + } + + @Override + public void setHTTPContentEncoding(String arg0) { + // return null; + } + + @Override + public void setHTTPContentType(String arg0) { + // return null; + } + + @Override + public void setMessageType(MessageType arg0) { + // return null; + } + + @Override + public void setPriority(int arg0) { + // return null; + } + + @Override + public void setProperties(SDTMap arg0) { + // return null; + } + + @Override + public void setQueueNameLocation(int arg0, int arg1) { + // return null; + } + + @Override + public void setReadOnly() { + // return null; + } + + @Override + public void setReplyTo(Destination arg0) { + // return null; + } + + @Override + public void setReplyToSuffix(String arg0) { + // return null; + } + + @Override + public void setSendTimestamp(long arg0) { + // return null; + } + + @Override + public void setSenderID(String arg0) { + // return null; + } + + @Override + public void setSenderId(String arg0) { + // return null; + } + + @Override + public void setSenderTimestamp(long arg0) { + // return null; + } + + @Override + public void setSequenceNumber(long arg0) { + // return null; + } + + @Override + public void setStructuredMsg(boolean arg0) { + // return null; + } + + @Override + public void setStructuredMsgType(byte arg0) { + // return null; + } + + @Override + public void setTimeToLive(long arg0) { + // return null; + } + + @Override + public void setTopicNameLocation(int arg0, int arg1) { + // return null; + } + + @Override + public void setUserData(byte[] arg0) { + // return null; + } + + @Override + public void settle(Outcome arg0) throws JCSMPException { + // return null; + } + + @Override + public int writeAttachment(byte[] arg0) { + return 0; + } + + @Override + public int writeAttachment(InputStream arg0) throws IOException { + return 0; + } + + @Override + public int writeAttachment(byte[] arg0, int arg1, int arg2) + throws BufferUnderflowException { + return 0; + } + + @Override + public int writeBinaryMetadataBytes(int arg0, byte[] arg1) { + return 0; + } + + @Override + public int writeBinaryMetadataBytes(int arg0, byte[] arg1, int arg2, int arg3) + throws BufferUnderflowException { + // TOD0 Auto-generated method stub + return 0; + } + + @Override + public int writeNewAttachment(byte[] arg0) { + return 0; + } + + @Override + public int writeNewAttachment(InputStream arg0) throws IOException { + return 0; + } + + @Override + public int writeNewAttachment(byte[] arg0, int arg1, int arg2) + throws BufferUnderflowException { + return 0; + } + + @Override + public int writeNewAttachment(InputStream arg0, int arg1, int arg2) throws IOException { + return 0; + } + }; + } +} diff --git a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/SolaceIOTest.java b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/SolaceIOTest.java new file mode 100644 index 000000000000..594c9277c8ff --- /dev/null +++ b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/SolaceIOTest.java @@ -0,0 +1,607 @@ +/* + * Copyright 2024 Google. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.solace.io.solace; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertThrows; +import static org.junit.Assert.assertTrue; + +import org.apache.beam.sdk.io.solace.SolaceDataUtils.SimpleRecord; +import org.apache.beam.sdk.io.solace.SolaceIO; +import org.apache.beam.sdk.io.solace.SolaceIO.Read; +import org.apache.beam.sdk.io.solace.broker.SessionServiceFactory; +import org.apache.beam.sdk.io.solace.data.Solace; +import org.apache.beam.sdk.io.solace.data.Solace.Record; +import org.apache.beam.sdk.io.solace.read.SolaceCheckpointMark; +import org.apache.beam.sdk.io.solace.read.UnboundedSolaceSource; +import com.solacesystems.jcsmp.BytesXMLMessage; +import com.solacesystems.jcsmp.Destination; +import com.solacesystems.jcsmp.Queue; +import com.solacesystems.jcsmp.Topic; +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.atomic.AtomicInteger; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.io.UnboundedSource.CheckpointMark; +import org.apache.beam.sdk.io.UnboundedSource.UnboundedReader; +import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.testing.CoderProperties; +import org.apache.beam.sdk.testing.PAssert; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.transforms.MapElements; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.TypeDescriptor; +import org.apache.beam.sdk.values.TypeDescriptors; +import org.joda.time.Instant; +import org.junit.Rule; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +@RunWith(JUnit4.class) +public class SolaceIOTest { + + @Rule public final transient TestPipeline pipeline = TestPipeline.create(); + + private static BytesXMLMessage getOrNull(Integer index, List messages) { + return index != null && index < messages.size() ? messages.get(index) : null; + } + + private static MockSempClientFactory getMockSempClientFactory() { + return new MockSempClientFactory(MockSempClient.builder().build()); + } + + private static UnboundedSolaceSource getSource( + Read spec, Queue queue, TestPipeline pipeline) { + return new UnboundedSolaceSource<>( + queue, + spec.getSempClientFactory(), + spec.getSessionServiceFactory(), + spec.getMaxNumConnections(), + spec.getDeduplicateRecords(), + spec.inferCoder(pipeline), + spec.getTimestampFn(), + spec.getParseFn()); + } + + @Test + public void testReadMessages() { + // Broker that creates input data + MockSessionService mockClientService = + new MockSessionService( + index -> { + List messages = + List.of( + SolaceDataUtils.getBytesXmlMessage( + "payload_test0", "450"), + SolaceDataUtils.getBytesXmlMessage( + "payload_test1", "451"), + SolaceDataUtils.getBytesXmlMessage( + "payload_test2", "452")); + return getOrNull(index, messages); + }, + 3); + + SessionServiceFactory fakeSessionServiceFactory = + new MockSessionServiceFactory(mockClientService); + + // Expected data + List inputs = new ArrayList<>(); + inputs.add(SolaceDataUtils.getSolaceRecord("payload_test0", "450")); + inputs.add(SolaceDataUtils.getSolaceRecord("payload_test1", "451")); + inputs.add(SolaceDataUtils.getSolaceRecord("payload_test2", "452")); + + // Run the pipeline + PCollection events = + pipeline.apply( + "Read from Solace", + SolaceIO.read() + .from(Solace.Queue.fromName("queue")) + .withSempClientFactory(getMockSempClientFactory()) + .withSessionServiceFactory(fakeSessionServiceFactory) + .withMaxNumConnections(2)); + + // Assert results + PAssert.that(events).containsInAnyOrder(inputs); + pipeline.run(); + } + + @Test + public void testReadMessagesWithDeduplication() { + // Broker that creates input data + MockSessionService mockClientService = + new MockSessionService( + index -> { + List messages = + List.of( + SolaceDataUtils.getBytesXmlMessage( + "payload_test0", "450"), + SolaceDataUtils.getBytesXmlMessage( + "payload_test1", "451"), + SolaceDataUtils.getBytesXmlMessage( + "payload_test2", "451")); + return getOrNull(index, messages); + }, + 3); + + SessionServiceFactory fakeSessionServiceFactory = + new MockSessionServiceFactory(mockClientService); + + // Expected data + List inputs = new ArrayList<>(); + inputs.add(SolaceDataUtils.getSolaceRecord("payload_test0", "450")); + inputs.add(SolaceDataUtils.getSolaceRecord("payload_test1", "451")); + + // Run the pipeline + PCollection events = + pipeline.apply( + "Read from Solace", + SolaceIO.read() + .from(Solace.Queue.fromName("queue")) + .withSempClientFactory(getMockSempClientFactory()) + .withSessionServiceFactory(fakeSessionServiceFactory) + .withMaxNumConnections(2)); + // Assert results + PAssert.that(events).containsInAnyOrder(inputs); + pipeline.run(); + } + + @Test + public void testReadMessagesWithoutDeduplication() { + // Broker that creates input data + MockSessionService mockClientService = + new MockSessionService( + index -> { + List messages = + List.of( + SolaceDataUtils.getBytesXmlMessage( + "payload_test0", "450"), + SolaceDataUtils.getBytesXmlMessage( + "payload_test1", "451"), + SolaceDataUtils.getBytesXmlMessage( + "payload_test2", "451")); + return getOrNull(index, messages); + }, + 3); + SessionServiceFactory fakeSessionServiceFactory = + new MockSessionServiceFactory(mockClientService); + + // Expected data + List inputs = new ArrayList<>(); + inputs.add(SolaceDataUtils.getSolaceRecord("payload_test0", "450")); + inputs.add(SolaceDataUtils.getSolaceRecord("payload_test1", "451")); + inputs.add(SolaceDataUtils.getSolaceRecord("payload_test2", "451")); + + // Run the pipeline + PCollection events = + pipeline.apply( + "Read from Solace", + SolaceIO.read() + .from(Solace.Queue.fromName("queue")) + .withSempClientFactory(getMockSempClientFactory()) + .withSessionServiceFactory(fakeSessionServiceFactory) + .withMaxNumConnections(1) + .withDeduplicateRecords(false)); + + // Assert results + PAssert.that(events).containsInAnyOrder(inputs); + pipeline.run(); + } + + @Test + public void testReadWithCoderAndParseFnAndTimestampFn() { + // Broker that creates input data + MockSessionService mockClientService = + new MockSessionService( + index -> { + List messages = + List.of( + SolaceDataUtils.getBytesXmlMessage( + "payload_test0", "450"), + SolaceDataUtils.getBytesXmlMessage( + "payload_test1", "451"), + SolaceDataUtils.getBytesXmlMessage( + "payload_test2", "452")); + return getOrNull(index, messages); + }, + 3); + SessionServiceFactory fakeSessionServiceFactory = + new MockSessionServiceFactory(mockClientService); + + // Expected data + List inputs = new ArrayList<>(); + inputs.add(new SimpleRecord("payload_test0", "450")); + inputs.add(new SimpleRecord("payload_test1", "451")); + inputs.add(new SimpleRecord("payload_test2", "452")); + + // Run the pipeline + PCollection events = + pipeline.apply( + "Read from Solace", + SolaceIO.read( + TypeDescriptor.of(SimpleRecord.class), + input -> + new SimpleRecord( + new String( + input.getBytes(), + StandardCharsets.UTF_8), + input.getApplicationMessageId()), + input -> Instant.ofEpochMilli(1708100477061L)) + .from(Solace.Queue.fromName("queue")) + .withSempClientFactory(getMockSempClientFactory()) + .withSessionServiceFactory(fakeSessionServiceFactory) + .withMaxNumConnections(2)); + + // Assert results + PAssert.that(events).containsInAnyOrder(inputs); + pipeline.run(); + } + + @Test + public void testNoQueueAndTopicSet() { + Read spec = SolaceIO.read(); + assertThrows(IllegalStateException.class, () -> spec.expand(pipeline.begin())); + } + + @Test + public void testSplitsForExclusiveQueue() throws Exception { + MockSempClient mockSempClient = + MockSempClient.builder().setIsQueueNonExclusiveFn((q) -> false).build(); + + Read spec = + SolaceIO.read() + .from(Solace.Queue.fromName("queue")) + .withSempClientFactory(new MockSempClientFactory(mockSempClient)); + + int desiredNumSplits = 5; + + UnboundedSolaceSource initialSource = getSource(spec, spec.getQueue(), pipeline); + List> splits = + initialSource.split(desiredNumSplits, PipelineOptionsFactory.create()); + assertEquals(1, splits.size()); + } + + @Test + public void testSplitsForNonExclusiveQueueWithMaxNumConnections() throws Exception { + Read spec = + SolaceIO.read() + .from(Solace.Queue.fromName("queue")) + .withSempClientFactory(getMockSempClientFactory()) + .withMaxNumConnections(3); + + int desiredNumSplits = 5; + + UnboundedSolaceSource initialSource = getSource(spec, spec.getQueue(), pipeline); + List> splits = + initialSource.split(desiredNumSplits, PipelineOptionsFactory.create()); + assertEquals(3, splits.size()); + } + + @Test + public void testSplitsForNonExclusiveQueueWithMaxNumConnectionsRespectDesired() + throws Exception { + Read spec = + SolaceIO.read() + .from(Solace.Queue.fromName("queue")) + .withSempClientFactory(getMockSempClientFactory()) + .withMaxNumConnections(10); + + int desiredNumSplits = 5; + + UnboundedSolaceSource initialSource = getSource(spec, spec.getQueue(), pipeline); + List> splits = + initialSource.split(desiredNumSplits, PipelineOptionsFactory.create()); + assertEquals(5, splits.size()); + } + + @Test + public void testCreateQueueForTopic() throws Exception { + AtomicInteger createQueueForTopicFnCounter = new AtomicInteger(0); + MockSempClient mockSempClient = + MockSempClient.builder() + .setCreateQueueForTopicFn( + (q) -> createQueueForTopicFnCounter.incrementAndGet()) + .build(); + + Read spec = + SolaceIO.read() + .from(Solace.Topic.fromName("topic")) + .withSempClientFactory(new MockSempClientFactory(mockSempClient)); + + UnboundedSolaceSource initialSource = + getSource( + spec, + spec.initializeQueueForTopic("some-job", spec.getSempClientFactory()), + pipeline); + initialSource.split(2, PipelineOptionsFactory.create()); + + // check if createQueueForTopic was executed + assertEquals(1, createQueueForTopicFnCounter.get()); + } + + @Test + public void testCheckpointMark() throws Exception { + AtomicInteger countConsumedMessages = new AtomicInteger(0); + AtomicInteger countAckMessages = new AtomicInteger(0); + + // Broker that creates input data + MockSessionService mockClientService = + new MockSessionService( + index -> { + List messages = new ArrayList<>(); + for (int i = 0; i < 10; i++) { + messages.add( + SolaceDataUtils.getBytesXmlMessage( + "payload_test" + i, + "45" + i, + (num) -> countAckMessages.incrementAndGet())); + } + countConsumedMessages.incrementAndGet(); + return getOrNull(index, messages); + }, + 10); + + SessionServiceFactory fakeSessionServiceFactory = + new MockSessionServiceFactory(mockClientService); + Read spec = + SolaceIO.read() + .from(Solace.Queue.fromName("queue")) + .withSempClientFactory(getMockSempClientFactory()) + .withSessionServiceFactory(fakeSessionServiceFactory) + .withMaxNumConnections(4); + + UnboundedSolaceSource initialSource = getSource(spec, spec.getQueue(), pipeline); + UnboundedReader reader = + initialSource.createReader(PipelineOptionsFactory.create(), null); + + // start the reader and move to the first record + assertTrue(reader.start()); + + // consume 3 messages (NB: start already consumed the first message) + for (int i = 0; i < 3; i++) { + assertTrue(String.format("Failed at %d-th message", i), reader.advance()); + } + + // check if 4 messages were consumed + assertEquals(4, countConsumedMessages.get()); + + // check if no messages were acknowledged yet + assertEquals(0, countAckMessages.get()); + + // finalize the checkpoint + reader.getCheckpointMark().finalizeCheckpoint(); + + // check if messages were acknowledged + assertEquals(4, countAckMessages.get()); + } + + @Test + public void testCheckpointMarkAndFinalizeSeparately() throws Exception { + AtomicInteger countConsumedMessages = new AtomicInteger(0); + AtomicInteger countAckMessages = new AtomicInteger(0); + + // Broker that creates input data + MockSessionService mockClientService = + new MockSessionService( + index -> { + List messages = new ArrayList<>(); + for (int i = 0; i < 10; i++) { + messages.add( + SolaceDataUtils.getBytesXmlMessage( + "payload_test" + i, + "45" + i, + (num) -> countAckMessages.incrementAndGet())); + } + countConsumedMessages.incrementAndGet(); + return getOrNull(index, messages); + }, + 10); + SessionServiceFactory fakeSessionServiceFactory = + new MockSessionServiceFactory(mockClientService); + + Read spec = + SolaceIO.read() + .from(Solace.Queue.fromName("queue")) + .withSempClientFactory(getMockSempClientFactory()) + .withSessionServiceFactory(fakeSessionServiceFactory) + .withMaxNumConnections(4); + + UnboundedSolaceSource initialSource = getSource(spec, spec.getQueue(), pipeline); + + UnboundedReader reader = + initialSource.createReader(PipelineOptionsFactory.create(), null); + + // start the reader and move to the first record + assertTrue(reader.start()); + + // consume 3 messages (NB: start already consumed the first message) + for (int i = 0; i < 3; i++) { + assertTrue(String.format("Failed at %d-th message", i), reader.advance()); + } + + // create checkpoint but don't finalize yet + CheckpointMark checkpointMark = reader.getCheckpointMark(); + + // consume 2 more messages + reader.advance(); + reader.advance(); + + // check if messages are still not acknowledged + assertEquals(0, countAckMessages.get()); + + // acknowledge from the first checkpoint + checkpointMark.finalizeCheckpoint(); + + // only messages from the first checkpoint are acknowledged + assertEquals(4, countAckMessages.get()); + } + + @Test + public void testCheckpointMarkSafety() throws Exception { + + final int messagesToProcess = 100; + + AtomicInteger countConsumedMessages = new AtomicInteger(0); + AtomicInteger countAckMessages = new AtomicInteger(0); + + // Broker that creates input data + MockSessionService mockClientService = + new MockSessionService( + index -> { + List messages = new ArrayList<>(); + for (int i = 0; i < messagesToProcess; i++) { + messages.add( + SolaceDataUtils.getBytesXmlMessage( + "payload_test" + i, + "45" + i, + (num) -> countAckMessages.incrementAndGet())); + } + countConsumedMessages.incrementAndGet(); + return getOrNull(index, messages); + }, + 10); + + SessionServiceFactory fakeSessionServiceFactory = + new MockSessionServiceFactory(mockClientService); + Read spec = + SolaceIO.read() + .from(Solace.Queue.fromName("queue")) + .withSempClientFactory(getMockSempClientFactory()) + .withSessionServiceFactory(fakeSessionServiceFactory) + .withMaxNumConnections(4); + + UnboundedSolaceSource initialSource = getSource(spec, spec.getQueue(), pipeline); + + UnboundedReader reader = + initialSource.createReader(PipelineOptionsFactory.create(), null); + + // start the reader and move to the first record + assertTrue(reader.start()); + + // consume half the messages (NB: start already consumed the first message) + for (int i = 0; i < (messagesToProcess / 2) - 1; i++) { + assertTrue(reader.advance()); + } + + // the messages are still pending in the queue (no ACK yet) + assertEquals(0, countAckMessages.get()); + + // we finalize the checkpoint for the already-processed messages while simultaneously + // consuming the remainder of messages from the queue + Thread runner = + new Thread( + () -> { + try { + for (int i = 0; i < messagesToProcess / 2; i++) { + assertTrue(reader.advance()); + } + } catch (IOException ex) { + throw new RuntimeException(ex); + } + }); + runner.start(); + reader.getCheckpointMark().finalizeCheckpoint(); + + // Concurrency issues would cause an exception to be thrown before this method exits, + // failing the test + runner.join(); + } + + @Test + public void testCheckpointMarkDefaultCoder() throws Exception { + SolaceCheckpointMark checkpointMark = new SolaceCheckpointMark(null, null); + Coder coder = + new UnboundedSolaceSource<>(null, null, null, 0, false, null, null, null) + .getCheckpointMarkCoder(); + CoderProperties.coderSerializable(coder); + CoderProperties.coderDecodeEncodeEqual(coder, checkpointMark); + } + + @Test + public void testDestinationTopicQueueCreation() { + String topicName = "some-topic"; + String queueName = "some-queue"; + Topic topic = SolaceIO.topicFromName(topicName); + Queue queue = SolaceIO.queueFromName(queueName); + + Destination dest = topic; + assertTrue(dest instanceof Topic); + assertFalse(dest instanceof Queue); + assertEquals(topicName, dest.getName()); + + dest = queue; + assertTrue(dest instanceof Queue); + assertFalse(dest instanceof Topic); + assertEquals(queueName, dest.getName()); + + Record r = SolaceDataUtils.getSolaceRecord("payload_test0", "450"); + dest = SolaceIO.convertToJcsmpDestination(r.getDestination()); + assertTrue(dest instanceof Topic); + assertFalse(dest instanceof Queue); + } + + @Test + public void testTopicEncoding() { + MockSessionService mockClientService = + new MockSessionService( + index -> { + List messages = + List.of( + SolaceDataUtils.getBytesXmlMessage( + "payload_test0", "450"), + SolaceDataUtils.getBytesXmlMessage( + "payload_test1", "451"), + SolaceDataUtils.getBytesXmlMessage( + "payload_test2", "452")); + return getOrNull(index, messages); + }, + 3); + + SessionServiceFactory fakeSessionServiceFactory = + new MockSessionServiceFactory(mockClientService); + + // Run + PCollection events = + pipeline.apply( + "Read from Solace", + SolaceIO.read() + .from(Solace.Queue.fromName("queue")) + .withSempClientFactory(getMockSempClientFactory()) + .withSessionServiceFactory(fakeSessionServiceFactory) + .withMaxNumConnections(2)); + + // Run the pipelin + PCollection destAreTopics = + events.apply( + MapElements.into(TypeDescriptors.booleans()) + .via( + r -> { + Destination dest = + SolaceIO.convertToJcsmpDestination( + r.getDestination()); + return dest instanceof Topic; + })); + + List inputs = List.of(true, true, true); + + // Assert results + PAssert.that(destAreTopics).containsInAnyOrder(inputs); + pipeline.run(); + } +} diff --git a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/broker/SempBasicAuthClientExecutorTest.java b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/broker/SempBasicAuthClientExecutorTest.java new file mode 100644 index 000000000000..11e457aa5652 --- /dev/null +++ b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/broker/SempBasicAuthClientExecutorTest.java @@ -0,0 +1,174 @@ +/* + * Copyright 2024 Google. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.solace.broker; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertThrows; +import static org.junit.Assert.assertTrue; + +import com.google.api.client.http.HttpRequestFactory; +import com.google.api.client.http.HttpResponseException; +import com.google.api.client.http.LowLevelHttpRequest; +import com.google.api.client.http.LowLevelHttpResponse; +import com.google.api.client.json.Json; +import com.google.api.client.testing.http.MockHttpTransport; +import com.google.api.client.testing.http.MockLowLevelHttpRequest; +import com.google.api.client.testing.http.MockLowLevelHttpResponse; +import com.google.common.collect.ImmutableList; +import java.io.IOException; +import java.util.List; +import org.apache.beam.sdk.io.solace.broker.SempBasicAuthClientExecutor; +import org.junit.Test; + +public class SempBasicAuthClientExecutorTest { + + @Test + public void testExecuteStatus4xx() { + MockHttpTransport transport = + new MockHttpTransport() { + @Override + public LowLevelHttpRequest buildRequest(String method, String url) { + return new MockLowLevelHttpRequest() { + @Override + public LowLevelHttpResponse execute() { + MockLowLevelHttpResponse response = new MockLowLevelHttpResponse(); + response.setStatusCode(404); + response.setContentType(Json.MEDIA_TYPE); + response.setContent( + "{\"meta\":{\"error\":{\"code\":404,\"description\":\"some" + + " error\",\"status\":\"xx\"}}}"); + return response; + } + }; + } + }; + + HttpRequestFactory requestFactory = transport.createRequestFactory(); + SempBasicAuthClientExecutor client = + new SempBasicAuthClientExecutor( + "http://host", "username", "password", "vpnName", requestFactory); + + assertThrows(HttpResponseException.class, () -> client.getQueueResponse("queue")); + } + + /** + * In this test case, we test a situation when a session that we used to authenticate to Semp + * expires. + * + *

To test this scenario, we need to do the following: + * + *

    + *
  1. Send the first request, to initialize a session. This request has to contain the Basic + * Auth header and should not include any cookie headers. The response for this request + * contains a session cookie we can re-use in the following requests. + *
  2. Send the second request - this request should use a cookie from the previous response. + * There should be no Authorization header. To simulate an expired session scenario, we + * set the response of this request to the "401 Unauthorized". This should cause a the + * request to be retried, this time with the Authorization header. + *
  3. Validate the third request to contain the Basic Auth header and no session cookies. + *
+ */ + @Test + public void testExecuteWithUnauthorized() throws IOException { + // Making it a final array, so that we can reference it from within the MockHttpTransport + // instance + final int[] requestCounter = {0}; + MockHttpTransport transport = + new MockHttpTransport() { + @Override + public LowLevelHttpRequest buildRequest(String method, String url) { + return new MockLowLevelHttpRequest() { + @Override + public LowLevelHttpResponse execute() throws IOException { + MockLowLevelHttpResponse response = new MockLowLevelHttpResponse(); + if (requestCounter[0] == 0) { + // The first request has to include Basic Auth header + assertTrue(this.getHeaders().containsKey("authorization")); + List authorizationHeaders = + this.getHeaders().get("authorization"); + assertEquals(1, authorizationHeaders.size()); + assertTrue(authorizationHeaders.get(0).contains("Basic")); + assertFalse(this.getHeaders().containsKey("cookie")); + + // Set the response to include Session cookies + response.setHeaderNames( + ImmutableList.of("Set-Cookie", "Set-Cookie")) + .setHeaderValues( + ImmutableList.of( + "ProxySession=JddSdJaGo6FYYmQk6nt8jXxFtq6n3FCFR14ebzRGQ5w;" + + " HttpOnly; SameSite=Strict;" + + " Path=/proxy; Max-Age=2592000", + "Session=JddSdJaGo6FYYmQk6nt8jXxFtq6n3FCFR14ebzRGQ5w;" + + " HttpOnly; SameSite=Strict;" + + " Path=/SEMP; Max-Age=2592000")); + response.setStatusCode(200); + } else if (requestCounter[0] == 1) { + // The second request does not include Basic Auth header + assertFalse(this.getHeaders().containsKey("authorization")); + // It must include a cookie header + assertTrue(this.getHeaders().containsKey("cookie")); + boolean hasSessionCookie = + this.getHeaders().get("cookie").stream() + .filter( + c -> + c.contains( + "Session=JddSdJaGo6FYYmQk6nt8jXxFtq6n3FCFR14ebzRGQ5w")) + .count() + == 1; + assertTrue(hasSessionCookie); + + // Let's assume the Session expired - we return the 401 + // unauthorized + response.setStatusCode(401); + } else { + // The second request has to be retried with a Basic Auth header + // this time + assertTrue(this.getHeaders().containsKey("authorization")); + List authorizationHeaders = + this.getHeaders().get("authorization"); + assertEquals(1, authorizationHeaders.size()); + assertTrue(authorizationHeaders.get(0).contains("Basic")); + assertFalse(this.getHeaders().containsKey("cookie")); + + response.setStatusCode(200); + } + response.setContentType(Json.MEDIA_TYPE); + requestCounter[0]++; + return response; + } + }; + } + }; + + HttpRequestFactory requestFactory = transport.createRequestFactory(); + SempBasicAuthClientExecutor client = + new SempBasicAuthClientExecutor( + "http://host", "username", "password", "vpnName", requestFactory); + + // The first, initial request + client.getQueueResponse("queue"); + // The second request, which will try to authenticate with a cookie, and then with Basic + // Auth when it receives a 401 unauthorized + client.getQueueResponse("queue"); + + // There should be 3 requests executed: + // the first one is the initial one with Basic Auth, + // the second one uses the session cookie, but we simulate it being expired, + // so there should be a third request with Basic Auth to create a new session. + assertEquals(3, requestCounter[0]); + } +} diff --git a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/data/SolaceTest.java b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/data/SolaceTest.java new file mode 100644 index 000000000000..d3acab33a71f --- /dev/null +++ b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/data/SolaceTest.java @@ -0,0 +1,217 @@ +/* + * Copyright 2024 Google. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.solace.data; + +import java.nio.charset.StandardCharsets; +import java.util.Map; +import org.apache.beam.sdk.io.solace.data.Solace; +import org.apache.beam.sdk.io.solace.data.Solace.Destination; +import org.junit.Assert; +import org.junit.Test; + +public class SolaceTest { + + Map properties; + Destination destination = + Solace.Destination.builder() + .setName("some destination") + .setType(Solace.DestinationType.TOPIC) + .build(); + String messageId = "some message id"; + Long expiration = 123L; + Integer priority = 7; + Boolean redelivered = true; + String replyTo = "no-one"; + Long receiveTimestamp = 123456789L; + Long senderTimestamp = 987654321L; + long timestampMillis = 1234567890L; + Long sequenceNumber = 27L; + Long timeToLive = 34567890L; + String payloadString = "some payload"; + byte[] payload = payloadString.getBytes(StandardCharsets.UTF_8); + String publishError = "some error"; + + @Test + public void testRecordEquality() { + Solace.Record obj1 = + Solace.Record.builder() + .setDestination(destination) + .setMessageId(messageId) + .setExpiration(expiration) + .setPriority(priority) + .setRedelivered(redelivered) + .setReplyTo(replyTo) + .setReceiveTimestamp(receiveTimestamp) + .setSenderTimestamp(senderTimestamp) + .setSequenceNumber(sequenceNumber) + .setTimeToLive(timeToLive) + .setPayload(payload) + .build(); + + Solace.Record obj2 = + Solace.Record.builder() + .setDestination(destination) + .setMessageId(messageId) + .setExpiration(expiration) + .setPriority(priority) + .setRedelivered(redelivered) + .setReplyTo(replyTo) + .setReceiveTimestamp(receiveTimestamp) + .setSenderTimestamp(senderTimestamp) + .setSequenceNumber(sequenceNumber) + .setTimeToLive(timeToLive) + .setPayload(payload) + .build(); + + Solace.Record obj3 = + Solace.Record.builder() + .setDestination(destination) + .setMessageId(messageId) + .setExpiration(expiration) + .setPriority(priority) + .setRedelivered(!redelivered) + .setReplyTo(replyTo) + .setReceiveTimestamp(receiveTimestamp) + .setSenderTimestamp(senderTimestamp) + .setSequenceNumber(sequenceNumber) + .setTimeToLive(timeToLive) + .setPayload(payload) + .build(); + + Assert.assertEquals(obj1, obj2); + Assert.assertNotEquals(obj1, obj3); + Assert.assertEquals(obj1.hashCode(), obj2.hashCode()); + Assert.assertEquals(obj1.getDestination(), destination); + Assert.assertEquals(obj1.getMessageId(), messageId); + Assert.assertEquals(obj1.getExpiration(), expiration); + Assert.assertEquals(obj1.getPriority(), priority); + Assert.assertEquals(obj1.getRedelivered(), redelivered); + Assert.assertEquals(obj1.getReplyTo(), replyTo); + Assert.assertEquals(obj1.getReceiveTimestamp(), receiveTimestamp); + Assert.assertEquals(obj1.getSenderTimestamp(), senderTimestamp); + Assert.assertEquals(obj1.getSequenceNumber(), sequenceNumber); + Assert.assertEquals(obj1.getTimeToLive(), timeToLive); + Assert.assertEquals(new String(obj1.getPayload(), StandardCharsets.UTF_8), payloadString); + } + + @Test + public void testRecordNullability() { + Solace.Record obj = + Solace.Record.builder().setMessageId(messageId).setPayload(payload).build(); + Assert.assertNotNull(obj); + Assert.assertNull(obj.getDestination()); + Assert.assertEquals(obj.getMessageId(), messageId); + Assert.assertNull(obj.getExpiration()); + Assert.assertNull(obj.getPriority()); + Assert.assertNull(obj.getRedelivered()); + Assert.assertNull(obj.getReplyTo()); + Assert.assertNull(obj.getReceiveTimestamp()); + Assert.assertNull(obj.getSenderTimestamp()); + Assert.assertNull(obj.getSequenceNumber()); + Assert.assertNull(obj.getTimeToLive()); + Assert.assertEquals(new String(obj.getPayload(), StandardCharsets.UTF_8), payloadString); + } + + @Test(expected = IllegalStateException.class) + public void testRecordBuilder() { + Solace.Record.builder().build(); + } + + @Test + public void testPublishResultEquality() { + Solace.PublishResult obj1 = + Solace.PublishResult.builder() + .setPublished(redelivered) + .setLatencyMilliseconds(timestampMillis) + .setMessageId(messageId) + .setError(publishError) + .build(); + + Solace.PublishResult obj2 = + Solace.PublishResult.builder() + .setPublished(redelivered) + .setLatencyMilliseconds(timestampMillis) + .setMessageId(messageId) + .setError(publishError) + .build(); + + Solace.PublishResult obj3 = + Solace.PublishResult.builder() + .setPublished(!redelivered) + .setLatencyMilliseconds(timestampMillis) + .setMessageId(messageId) + .setError(publishError) + .build(); + + Assert.assertEquals(obj1, obj2); + Assert.assertNotEquals(obj1, obj3); + Assert.assertEquals(obj1.getPublished(), redelivered); + Assert.assertEquals(obj1.getLatencyMilliseconds().longValue(), timestampMillis); + Assert.assertEquals(obj1.getMessageId(), messageId); + Assert.assertEquals(obj1.getError(), publishError); + } + + @Test(expected = IllegalStateException.class) + public void testPublishResultBuilder() { + Solace.PublishResult.builder().build(); + } + + @Test + public void testPublishResultNullability() { + Solace.PublishResult obj = + Solace.PublishResult.builder() + .setMessageId(messageId) + .setPublished(redelivered) + .build(); + + Assert.assertNotNull(obj); + Assert.assertEquals(obj.getMessageId(), messageId); + Assert.assertEquals(obj.getPublished(), redelivered); + Assert.assertNull(obj.getLatencyMilliseconds()); + Assert.assertNull(obj.getError()); + } + + @Test + public void testCorrelationKeyEquality() { + Solace.CorrelationKey obj1 = + Solace.CorrelationKey.builder() + .setMessageId(messageId) + .setPublishMonotonicMillis(timestampMillis) + .build(); + + Solace.CorrelationKey obj2 = + Solace.CorrelationKey.builder() + .setMessageId(messageId) + .setPublishMonotonicMillis(timestampMillis) + .build(); + + Solace.CorrelationKey obj3 = + Solace.CorrelationKey.builder() + .setMessageId(messageId) + .setPublishMonotonicMillis(timestampMillis - 1L) + .build(); + + Assert.assertEquals(obj1, obj2); + Assert.assertNotEquals(obj1, obj3); + Assert.assertEquals(obj1.getMessageId(), messageId); + Assert.assertEquals(obj1.getPublishMonotonicMillis(), timestampMillis); + } + + @Test(expected = IllegalStateException.class) + public void testCorrelationKeyNullability() { + Solace.CorrelationKey.builder().build(); + } +} diff --git a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/it/SolaceContainerManager.java b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/it/SolaceContainerManager.java new file mode 100644 index 000000000000..8392c322997e --- /dev/null +++ b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/it/SolaceContainerManager.java @@ -0,0 +1,166 @@ +/* + * Copyright 2024 Google. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.solace.it; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.testcontainers.containers.output.Slf4jLogConsumer; +import org.testcontainers.solace.Service; +import org.testcontainers.solace.SolaceContainer; +import org.testcontainers.utility.DockerImageName; + +public class SolaceContainerManager { + + public static final String VPN_NAME = "default"; + public static final String PASSWORD = "password"; + public static final String USERNAME = "username"; + public static final String TOPIC_NAME = "test_topic"; + private static final Logger LOG = LoggerFactory.getLogger(SolaceContainerManager.class); + private final SolaceContainer container; + + public SolaceContainerManager() { + this.container = + new SolaceContainer(DockerImageName.parse("solace/solace-pubsub-standard:10.7")) { + { + addFixedExposedPort(55555, 55555); + addFixedExposedPort(9000, 9000); + addFixedExposedPort(8080, 8080); + addFixedExposedPort(80, 80); + } + }.withVpn(VPN_NAME) + .withCredentials(USERNAME, PASSWORD) + // .withExposedPorts(Service.SMF.getPort()); + .withTopic(TOPIC_NAME, Service.SMF) + .withLogConsumer(new Slf4jLogConsumer(LOG)); + container.addExposedPort(8080); + container.addExposedPort(55555); + } + + public void start() { + container.start(); + } + + void createQueueWithSubscriptionTopic(String queueName) { + executeCommand( + "curl", + "http://localhost:8080/SEMP/v2/config/msgVpns/" + VPN_NAME + "/topicEndpoints", + "-X", + "POST", + "-u", + "admin:admin", + "-H", + "Content-Type:application/json", + "-d", + "{\"topicEndpointName\":\"" + + TOPIC_NAME + + "\",\"accessType\":\"exclusive\",\"permission\":\"modify-topic\",\"ingressEnabled\":true,\"egressEnabled\":true}"); + executeCommand( + "curl", + "http://localhost:8080/SEMP/v2/config/msgVpns/" + VPN_NAME + "/queues", + "-X", + "POST", + "-u", + "admin:admin", + "-H", + "Content-Type:application/json", + "-d", + "{\"queueName\":\"" + + queueName + + "\",\"accessType\":\"non-exclusive\",\"maxMsgSpoolUsage\":200,\"permission\":\"consume\",\"ingressEnabled\":true,\"egressEnabled\":true}"); + executeCommand( + "curl", + "http://localhost:8080/SEMP/v2/config/msgVpns/" + + VPN_NAME + + "/queues/" + + queueName + + "/subscriptions", + "-X", + "POST", + "-u", + "admin:admin", + "-H", + "Content-Type:application/json", + "-d", + "{\"subscriptionTopic\":\"" + TOPIC_NAME + "\"}"); + } + + private void executeCommand(String... command) { + try { + org.testcontainers.containers.Container.ExecResult execResult = + container.execInContainer(command); + if (execResult.getExitCode() != 0) { + logCommandError(execResult.getStderr(), command); + } else { + LOG.info(execResult.getStdout()); + } + } catch (IOException | InterruptedException e) { + logCommandError(e.getMessage(), command); + } + } + + private void logCommandError(String error, String... command) { + LOG.error("Could not execute command {}: {}", command, error); + } + + public void stop() { + if (container != null) { + container.stop(); + } + } + + public void getQueueDetails(String queueName) { + executeCommand( + "curl", + "http://localhost:8080/SEMP/v2/monitor/msgVpns/" + + VPN_NAME + + "/queues/" + + queueName + + "/msgs", + "-X", + "GET", + "-u", + "admin:admin"); + } + + public void sendToTopic(String payload, List additionalHeaders) { + // https://docs.solace.com/API/RESTMessagingPrtl/Solace-REST-Message-Encoding.htm + + List command = + new ArrayList<>( + Arrays.asList( + "curl", + "http://localhost:9000/TOPIC/" + TOPIC_NAME, + "-X", + "POST", + "-u", + USERNAME + ":" + PASSWORD, + "--header", + "Content-Type:application/json", + "-d", + payload)); + + for (String additionalHeader : additionalHeaders) { + command.add("--header"); + command.add(additionalHeader); + } + + executeCommand(command.toArray(new String[0])); + } +} diff --git a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/it/SolaceIOIT.java b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/it/SolaceIOIT.java new file mode 100644 index 000000000000..6591317cda54 --- /dev/null +++ b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/it/SolaceIOIT.java @@ -0,0 +1,273 @@ +/* + * Copyright 2024 Google. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.solace.it; + +import static org.junit.Assert.assertNotEquals; + +import org.apache.beam.sdk.io.solace.SolaceIO; +import org.apache.beam.sdk.io.solace.broker.BasicAuthJcsmpSessionServiceFactory; +import org.apache.beam.sdk.io.solace.broker.BasicAuthSempClientFactory; +import org.apache.beam.sdk.io.solace.data.Solace.Queue; +import org.apache.beam.sdk.io.solace.data.Solace.Record; +import java.nio.charset.StandardCharsets; +import java.util.List; +import org.apache.beam.sdk.PipelineResult; +import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.options.StreamingOptions; +import org.apache.beam.sdk.testing.PAssert; +import org.apache.beam.sdk.testing.SerializableMatcher; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.testing.TestPipelineOptions; +import org.apache.beam.sdk.transforms.Combine; +import org.apache.beam.sdk.transforms.Count; +import org.apache.beam.sdk.transforms.MapElements; +import org.apache.beam.sdk.transforms.SimpleFunction; +import org.apache.beam.sdk.transforms.windowing.FixedWindows; +import org.apache.beam.sdk.transforms.windowing.Window; +import org.apache.beam.sdk.values.PCollection; +import org.hamcrest.BaseMatcher; +import org.hamcrest.Description; +import org.joda.time.Duration; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.testcontainers.shaded.org.apache.commons.lang3.builder.EqualsBuilder; + +public class SolaceIOIT { + private static final Logger LOG = LoggerFactory.getLogger(SolaceIOIT.class); + private static SolaceContainerManager solaceContainerManager; + private static final TestPipelineOptions testOptions; + + static { + testOptions = PipelineOptionsFactory.create().as(TestPipelineOptions.class); + testOptions.setBlockOnRun(false); + } + + @Rule public final transient TestPipeline pipeline = TestPipeline.fromOptions(testOptions); + + @BeforeClass + public static void setup() { + System.out.println("START"); + solaceContainerManager = new SolaceContainerManager(); + solaceContainerManager.start(); + } + + @AfterClass + public static void afterClass() { + if (solaceContainerManager != null) { + solaceContainerManager.stop(); + } + } + + @Test + public void test() { + // Similar approach to + // https://github.com/apache/beam/blob/812e98fac243bab2a88f6ea5fad6147ff8e54a97/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOIT.java#L216 + String queueName = "test_queue"; + solaceContainerManager.createQueueWithSubscriptionTopic(queueName); + + solaceContainerManager.getQueueDetails(queueName); + String payload = "{\"field_str\":\"value\",\"field_int\":123}"; + solaceContainerManager.sendToTopic(payload, List.of("Solace-Message-ID:m1")); + solaceContainerManager.sendToTopic(payload, List.of("Solace-Message-ID:m2")); + solaceContainerManager.getQueueDetails(queueName); + + pipeline.getOptions().as(StreamingOptions.class).setStreaming(true); + + PCollection events = + pipeline.apply( + "Read from Solace", + SolaceIO.read() + .from(Queue.fromName(queueName)) + .withMaxNumConnections(1) + .withSempClientFactory( + BasicAuthSempClientFactory.builder() + .withHost("http://localhost:8080") + .withUsername("admin") + .withPassword("admin") + .withVpnName(SolaceContainerManager.VPN_NAME) + .build()) + .withSessionServiceFactory( + BasicAuthJcsmpSessionServiceFactory.builder() + .withHost("localhost") + .withUsername(SolaceContainerManager.USERNAME) + .withPassword(SolaceContainerManager.PASSWORD) + .withVpnName(SolaceContainerManager.VPN_NAME) + .build())); + // PCollection count = + PCollection records = + events.apply( + "PassThrough", + MapElements.via( + new SimpleFunction() { + @Override + public Record apply(Record s) { + System.out.println("passthrough rec: " + s); + return s; + } + // })).apply("Window", + // Window.into(CalendarWindows.years(1))); + })) + .apply("Window", Window.into(FixedWindows.of(Duration.standardSeconds(5)))); + + System.out.println("xxxxx"); + + PAssert.thatSingleton( + records.apply( + "Counting element", + Combine.globally(Count.combineFn()).withoutDefaults())) + .isEqualTo(1L); + + PAssert.that(records) + .containsInAnyOrder( + partialMatch( + Record.builder() + .setMessageId("m1") + .setPayload(payload.getBytes(StandardCharsets.UTF_8)) + .build())); + + PipelineResult writeResult = pipeline.run(); + // removing this line causes the pipeline not ingest any data + PipelineResult.State writeState = writeResult.waitUntilFinish(Duration.standardSeconds(10)); + assertNotEquals(PipelineResult.State.FAILED, writeState); + + System.out.println("queue after pipeline"); + solaceContainerManager.getQueueDetails(queueName); + } + + private static SerializableMatcher partialMatch(Record expected) { + class Matcher extends BaseMatcher implements SerializableMatcher { + @Override + public boolean matches(Object item) { + LOG.info("matches!!!"); + System.out.println("matches"); + if (!(item instanceof Record)) { + return false; + } + + Record actual = (Record) item; + boolean partiallyEqual = + EqualsBuilder.reflectionEquals( + actual, expected, "replicationGroupMessageId"); + System.out.println("expected.equals(actual): " + expected.equals(actual)); + System.out.println("partiallyEqual: " + partiallyEqual); + System.out.println("expected: " + expected); + System.out.println("actual: " + actual); + + return true; + + // for (Record needle : needles) { + // if (!haystack.contains(needle)) { + // return false; + // } + // } + // return true; + } + + @Override + public void describeTo(Description description) { + description.appendText("Contains all of: "); + description.appendText(expected.toString()); + } + } + System.out.println("new matcher"); + return new Matcher(); + } + // @Test + // public void testWrite() { + // TestStream createEvents = + // TestStream.create(StringUtf8Coder.of()) + // .addElements("r1", "r2") + // .advanceWatermarkTo( + // Instant.ofEpochMilli(0L).plus(Duration.standardSeconds(10))) + // .addElements("r3", "r4") + // .advanceWatermarkToInfinity(); + // + // PCollection records = pipeline.apply(createEvents); + // + // SolacePublishResult results = + // records.apply( + // "map", + // ParDo.of( + // new DoFn() { + // @ProcessElement + // public void processElement(ProcessContext c) { + // // System.out.println("Failed: " + c.element()); + // c.output( + // buildRecord( + // c.element(), + // "payload_" + c.element())); + // } + // })) + // .apply( + // SolaceIO.writeSolaceRecords() + // .to(Topic.fromName("test_topic")) + // .withSessionPropertiesProvider( + // BasicAuthenticationProvider.builder() + // .username("xx") + // .password("xx") + // .host("localhost") + // .vpnName(solaceContainer.getVpn()) + // .build()) + // .withDeliveryMode(DeliveryMode.PERSISTENT) + // .withSubmissionMode(SubmissionMode.HIGHER_THROUGHPUT) + // .withWriterType(WriterType.BATCHED) + // .withMaxNumOfUsedWorkers(1) + // .withNumberOfClientsPerWorker(1)); + // + // results.getSuccessfulPublish() + // .apply( + // "Successful records", + // ParDo.of( + // new DoFn() { + // @ProcessElement + // public void processElement(ProcessContext c) { + // System.out.println("OK: " + c.element()); + // c.output(1); + // } + // })); + // + // results.getFailedPublish() + // .apply( + // "Failed records", + // ParDo.of( + // new DoFn() { + // @ProcessElement + // public void processElement(ProcessContext c) { + // System.out.println("Failed: " + c.element()); + // c.output(1); + // } + // })); + // + // pipeline.run().waitUntilFinish(); + // } + + // private static Record buildRecord(String id, String payload) { + // return Record.builder() + // .setMessageId(id) + // .setPayload(payload.getBytes(StandardCharsets.UTF_8)) + // .setSenderTimestamp(1712224703L) + // .setDestination( + // Destination.builder() + // .setName("test_topic") + // .setType(DestinationType.TOPIC) + // .build()) + // .build(); + // } +} From 00274a26eddfe2b98f35f7f64e79374e42fcbc66 Mon Sep 17 00:00:00 2001 From: bzablocki Date: Tue, 16 Apr 2024 12:12:46 +0200 Subject: [PATCH 02/41] wip solace connector --- .../beam/gradle/BeamModulePlugin.groovy | 5 +- sdks/java/io/solace/build.gradle | 11 + .../sdk/io/solace/RetryCallableManager.java | 148 +- .../sdk/io/solace/SerializableSupplier.java | 18 +- .../apache/beam/sdk/io/solace/SolaceIO.java | 1149 +++---------- .../broker/BasicAuthJcsmpSessionService.java | 157 +- .../BasicAuthJcsmpSessionServiceFactory.java | 104 +- .../io/solace/broker/BasicAuthSempClient.java | 350 ++-- .../broker/BasicAuthSempClientFactory.java | 188 +-- .../sdk/io/solace/broker/BrokerResponse.java | 75 +- .../sdk/io/solace/broker/MessageReceiver.java | 36 +- .../broker/SempBasicAuthClientExecutor.java | 255 +-- .../beam/sdk/io/solace/broker/SempClient.java | 22 +- .../io/solace/broker/SempClientFactory.java | 18 +- .../sdk/io/solace/broker/SessionService.java | 24 +- .../solace/broker/SessionServiceFactory.java | 26 +- .../solace/broker/SolaceMessageReceiver.java | 72 +- .../beam/sdk/io/solace/data/Solace.java | 476 +++--- .../sdk/io/solace/data/SolaceRecordCoder.java | 117 +- .../io/solace/read/SolaceCheckpointMark.java | 98 +- .../io/solace/read/UnboundedSolaceReader.java | 270 +-- .../io/solace/read/UnboundedSolaceSource.java | 210 +-- .../io/solace/read/WatermarkParameters.java | 98 +- .../sdk/io/solace/read/WatermarkPolicy.java | 82 +- .../beam/sdk/io/solace/MockSempClient.java | 115 +- .../sdk/io/solace/MockSempClientFactory.java | 33 +- .../sdk/io/solace/MockSessionService.java | 105 +- .../io/solace/MockSessionServiceFactory.java | 32 +- .../io/solace/RetryCallableManagerTest.java | 269 ++- .../beam/sdk/io/solace/SolaceDataUtils.java | 1463 ++++++++--------- .../beam/sdk/io/solace/SolaceIOTest.java | 1085 ++++++------ .../SempBasicAuthClientExecutorTest.java | 265 ++- .../beam/sdk/io/solace/data/SolaceTest.java | 391 +++-- .../io/solace/it/SolaceContainerManager.java | 256 +-- .../beam/sdk/io/solace/it/SolaceIOIT.java | 417 ++--- settings.gradle.kts | 2 + 36 files changed, 3924 insertions(+), 4518 deletions(-) diff --git a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy index 1a0164a0e53a..091d238d1008 100644 --- a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy +++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy @@ -635,10 +635,11 @@ class BeamModulePlugin implements Plugin { def sbe_tool_version = "1.25.1" def singlestore_jdbc_version = "1.1.4" def slf4j_version = "1.7.30" + def solace_version = "10.21.0" def spark2_version = "2.4.8" def spark3_version = "3.2.2" def spotbugs_version = "4.0.6" - def testcontainers_version = "1.17.3" + def testcontainers_version = "1.18.3" // [bomupgrader] determined by: org.apache.arrow:arrow-memory-core, consistent with: google_cloud_platform_libraries_bom def arrow_version = "15.0.1" def jmh_version = "1.34" @@ -873,6 +874,7 @@ class BeamModulePlugin implements Plugin { slf4j_log4j12 : "org.slf4j:slf4j-log4j12:$slf4j_version", slf4j_jcl : "org.slf4j:slf4j-jcl:$slf4j_version", snappy_java : "org.xerial.snappy:snappy-java:1.1.10.4", + solace : "com.solacesystems:sol-jcsmp:$solace_version", spark_core : "org.apache.spark:spark-core_2.11:$spark2_version", spark_streaming : "org.apache.spark:spark-streaming_2.11:$spark2_version", spark3_core : "org.apache.spark:spark-core_2.12:$spark3_version", @@ -897,6 +899,7 @@ class BeamModulePlugin implements Plugin { testcontainers_oracle : "org.testcontainers:oracle-xe:$testcontainers_version", testcontainers_postgresql : "org.testcontainers:postgresql:$testcontainers_version", testcontainers_rabbitmq : "org.testcontainers:rabbitmq:$testcontainers_version", + testcontainers_solace : "org.testcontainers:solace:$testcontainers_version", truth : "com.google.truth:truth:1.1.5", threetenbp : "org.threeten:threetenbp:1.6.8", vendored_grpc_1_60_1 : "org.apache.beam:beam-vendor-grpc-1_60_1:0.2", diff --git a/sdks/java/io/solace/build.gradle b/sdks/java/io/solace/build.gradle index 4814d82358e4..5a61315a6395 100644 --- a/sdks/java/io/solace/build.gradle +++ b/sdks/java/io/solace/build.gradle @@ -27,10 +27,20 @@ description = "Apache Beam :: SDKs :: Java :: IO :: Solace" ext.summary = """IO to read and write to Solace destinations (queues and topics).""" dependencies { + implementation enforcedPlatform(library.java.google_cloud_platform_libraries_bom) implementation library.java.vendored_guava_32_1_2_jre implementation project(path: ":sdks:java:core", configuration: "shadow") implementation library.java.slf4j_api implementation library.java.joda_time + implementation library.java.solace + implementation library.java.google_cloud_core + implementation library.java.vendored_guava_32_1_2_jre + implementation library.java.gax + implementation library.java.google_http_client + implementation 'com.google.cloud:google-cloud-secretmanager' + implementation project(":sdks:java:extensions:avro") + implementation library.java.avro + testImplementation library.java.activemq_amqp testImplementation library.java.activemq_broker testImplementation library.java.activemq_jaas @@ -42,4 +52,5 @@ dependencies { testImplementation project(path: ":sdks:java:testing:test-utils", configuration: "testRuntimeMigration") testRuntimeOnly library.java.slf4j_jdk14 testRuntimeOnly project(path: ":runners:direct-java", configuration: "shadow") + testImplementation library.java.testcontainers_solace } diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/RetryCallableManager.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/RetryCallableManager.java index ceea3c597986..212f4faa6bfc 100644 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/RetryCallableManager.java +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/RetryCallableManager.java @@ -1,11 +1,13 @@ /* - * Copyright 2024 Google. + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -13,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.google.cloud.dataflow.dce.io.solace; +package org.apache.beam.sdk.io.solace; import com.google.api.core.NanoClock; import com.google.api.gax.retrying.RetrySettings; @@ -25,89 +27,89 @@ import java.util.Set; import java.util.concurrent.Callable; import org.apache.beam.sdk.annotations.Internal; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet; @Internal @AutoValue public abstract class RetryCallableManager implements Serializable { - private static final int NUMBER_OF_RETRIES = 4; - private static final int RETRY_INTERVAL_SECONDS = 1; - private static final int RETRY_MULTIPLIER = 2; - private static final int MAX_DELAY = - NUMBER_OF_RETRIES * RETRY_MULTIPLIER * RETRY_INTERVAL_SECONDS + 1; + private static final int NUMBER_OF_RETRIES = 4; + private static final int RETRY_INTERVAL_SECONDS = 1; + private static final int RETRY_MULTIPLIER = 2; + private static final int MAX_DELAY = + NUMBER_OF_RETRIES * RETRY_MULTIPLIER * RETRY_INTERVAL_SECONDS + 1; - public static RetryCallableManager create() { - return builder().build(); - } - /** - * Method that executes and repeats the execution of the callable argument, if it throws one of - * the exceptions from the exceptionsToIntercept Set. - */ - public V retryCallable( - Callable callable, Set> exceptionsToIntercept) { - return RetryHelper.runWithRetries( - callable, - getRetrySettings(), - getExceptionHandlerForExceptions(exceptionsToIntercept), - NanoClock.getDefaultClock()); - } + public static RetryCallableManager create() { + return builder().build(); + } + /** + * Method that executes and repeats the execution of the callable argument, if it throws one of + * the exceptions from the exceptionsToIntercept Set. + */ + public V retryCallable( + Callable callable, Set> exceptionsToIntercept) { + return RetryHelper.runWithRetries( + callable, + getRetrySettings(), + getExceptionHandlerForExceptions(exceptionsToIntercept), + NanoClock.getDefaultClock()); + } - private ExceptionHandler getExceptionHandlerForExceptions( - Set> exceptionsToIntercept) { - return ExceptionHandler.newBuilder() - .abortOn(RuntimeException.class) - .addInterceptors(new ExceptionSetInterceptor(Set.copyOf(exceptionsToIntercept))) - .build(); - } + private ExceptionHandler getExceptionHandlerForExceptions( + Set> exceptionsToIntercept) { + return ExceptionHandler.newBuilder() + .abortOn(RuntimeException.class) + .addInterceptors(new ExceptionSetInterceptor(ImmutableSet.copyOf(exceptionsToIntercept))) + .build(); + } - abstract RetrySettings getRetrySettings(); + abstract RetrySettings getRetrySettings(); - abstract Builder toBuilder(); + abstract Builder toBuilder(); - static Builder builder() { - return new AutoValue_RetryCallableManager.Builder() - .setRetrySettings( - RetrySettings.newBuilder() - .setInitialRetryDelay( - org.threeten.bp.Duration.ofSeconds(RETRY_INTERVAL_SECONDS)) - .setMaxAttempts(NUMBER_OF_RETRIES) - .setMaxRetryDelay(org.threeten.bp.Duration.ofSeconds(MAX_DELAY)) - .setRetryDelayMultiplier(RETRY_MULTIPLIER) - .build()); - } + static Builder builder() { + return new AutoValue_RetryCallableManager.Builder() + .setRetrySettings( + RetrySettings.newBuilder() + .setInitialRetryDelay(org.threeten.bp.Duration.ofSeconds(RETRY_INTERVAL_SECONDS)) + .setMaxAttempts(NUMBER_OF_RETRIES) + .setMaxRetryDelay(org.threeten.bp.Duration.ofSeconds(MAX_DELAY)) + .setRetryDelayMultiplier(RETRY_MULTIPLIER) + .build()); + } - @AutoValue.Builder - abstract static class Builder { - abstract Builder setRetrySettings(RetrySettings retrySettings); + @AutoValue.Builder + abstract static class Builder { + abstract Builder setRetrySettings(RetrySettings retrySettings); - abstract RetryCallableManager build(); - } + abstract RetryCallableManager build(); + } - private static class ExceptionSetInterceptor implements Interceptor { - private static final long serialVersionUID = -8429573586820467828L; - private final Set> exceptionsToIntercept; + private static class ExceptionSetInterceptor implements Interceptor { + private static final long serialVersionUID = -8429573586820467828L; + private final Set> exceptionsToIntercept; - public ExceptionSetInterceptor(Set> exceptionsToIntercept) { - this.exceptionsToIntercept = exceptionsToIntercept; - } + public ExceptionSetInterceptor(Set> exceptionsToIntercept) { + this.exceptionsToIntercept = exceptionsToIntercept; + } - @Override - public RetryResult afterEval(Exception exception, RetryResult retryResult) { - return Interceptor.RetryResult.CONTINUE_EVALUATION; - } + @Override + public RetryResult afterEval(Exception exception, RetryResult retryResult) { + return Interceptor.RetryResult.CONTINUE_EVALUATION; + } - @Override - public RetryResult beforeEval(Exception exceptionToEvaluate) { - for (Class exceptionToIntercept : exceptionsToIntercept) { - if (isOf(exceptionToIntercept, exceptionToEvaluate)) { - return Interceptor.RetryResult.RETRY; - } - } - return Interceptor.RetryResult.CONTINUE_EVALUATION; + @Override + public RetryResult beforeEval(Exception exceptionToEvaluate) { + for (Class exceptionToIntercept : exceptionsToIntercept) { + if (isOf(exceptionToIntercept, exceptionToEvaluate)) { + return Interceptor.RetryResult.RETRY; } + } + return Interceptor.RetryResult.CONTINUE_EVALUATION; + } - private boolean isOf(Class clazz, Object obj) { - return clazz.isInstance(obj); - } + private boolean isOf(Class clazz, Object obj) { + return clazz.isInstance(obj); } + } } diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/SerializableSupplier.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/SerializableSupplier.java index 249598d8cf53..3438b39ccf94 100644 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/SerializableSupplier.java +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/SerializableSupplier.java @@ -1,11 +1,13 @@ /* - * Copyright 2024 Google. + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -13,11 +15,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.google.cloud.dataflow.dce.io.solace; +package org.apache.beam.sdk.io.solace; import java.io.Serializable; @FunctionalInterface public interface SerializableSupplier extends Serializable { - OutputT get(); + OutputT get(); } diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/SolaceIO.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/SolaceIO.java index 17e778605a25..3005dace8dc5 100644 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/SolaceIO.java +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/SolaceIO.java @@ -1,11 +1,13 @@ /* - * Copyright 2023 Google. + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -13,52 +15,34 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.google.cloud.dataflow.dce.io.solace; +package org.apache.beam.sdk.io.solace; import com.google.auto.value.AutoValue; -import com.google.cloud.dataflow.dce.io.solace.broker.SempClientFactory; -import com.google.cloud.dataflow.dce.io.solace.broker.SessionService; -import com.google.cloud.dataflow.dce.io.solace.broker.SessionServiceFactory; -import com.google.cloud.dataflow.dce.io.solace.data.Solace; -import com.google.cloud.dataflow.dce.io.solace.data.Solace.SolaceRecordMapper; -import com.google.cloud.dataflow.dce.io.solace.data.SolaceRecordCoder; -import com.google.cloud.dataflow.dce.io.solace.read.UnboundedSolaceSource; -import com.google.cloud.dataflow.dce.io.solace.write.SolacePublishResult; -import com.google.cloud.dataflow.dce.io.solace.write.UnboundedBatchedSolaceWriter; -import com.google.cloud.dataflow.dce.io.solace.write.UnboundedSolaceWriter; -import com.google.cloud.dataflow.dce.io.solace.write.UnboundedStreamingSolaceWriter; -import com.google.cloud.dataflow.dce.io.solace.write.properties.BasicAuthenticationProvider; -import com.google.cloud.dataflow.dce.io.solace.write.properties.GoogleCloudSecretProvider; -import com.google.cloud.dataflow.dce.io.solace.write.properties.SessionPropertiesProvider; -import com.google.common.base.Preconditions; import com.solacesystems.jcsmp.BytesXMLMessage; -import com.solacesystems.jcsmp.DeliveryMode; import com.solacesystems.jcsmp.Destination; import com.solacesystems.jcsmp.JCSMPFactory; import com.solacesystems.jcsmp.Queue; import com.solacesystems.jcsmp.Topic; import java.io.IOException; import java.util.Objects; -import org.apache.arrow.util.VisibleForTesting; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.CannotProvideCoderException; import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.io.solace.broker.SempClientFactory; +import org.apache.beam.sdk.io.solace.broker.SessionService; +import org.apache.beam.sdk.io.solace.broker.SessionServiceFactory; +import org.apache.beam.sdk.io.solace.data.Solace; +import org.apache.beam.sdk.io.solace.data.Solace.SolaceRecordMapper; +import org.apache.beam.sdk.io.solace.data.SolaceRecordCoder; +import org.apache.beam.sdk.io.solace.read.UnboundedSolaceSource; import org.apache.beam.sdk.schemas.NoSuchSchemaException; -import org.apache.beam.sdk.transforms.MapElements; import org.apache.beam.sdk.transforms.PTransform; -import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.SerializableFunction; -import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.transforms.windowing.GlobalWindows; -import org.apache.beam.sdk.transforms.windowing.Window; -import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.sdk.values.PCollectionTuple; -import org.apache.beam.sdk.values.TupleTag; -import org.apache.beam.sdk.values.TupleTagList; import org.apache.beam.sdk.values.TypeDescriptor; -import org.apache.beam.sdk.values.WindowingStrategy; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.Instant; import org.slf4j.Logger; @@ -71,909 +55,312 @@ * * TODO: Auth for the read connector * - *

The writer connector uses a JCSMP session properties provider, where all the authentication - * details must be set. See {@link Write#withSessionPropertiesProvider(SessionPropertiesProvider)}. - * For convenience, the connector provides a provider for basic authentication ({@link - * BasicAuthenticationProvider}) and another one to retrieve the password from Google Cloud Secret - * Manager ({@link GoogleCloudSecretProvider} - * *

Reading

* * TODO - * - *

Writing

- * - * To write to Solace, use {@link #write()}. The default VPN name is "default". - * - *

Coders and schemas support - * - *

The connector expects a {@link Solace.Record} as input, so you need to transform your data to - * this format before writing. {@link Solace.Record} uses Beam schemas by default, but you can - * override and set {@link SolaceRecordCoder} as custom coder if you prefer. - * - *

Writing to a topic of a queue - * - *

The connector uses the Solace JCSMP API. - * The clients will write to a SMF - * topic to the port 55555 of the host. If you want to use a different port, specify it in the - * host property with the format "X.X.X.X:PORT". - * - *

Once you have a {@link PCollection} of {@link Solace.Record}, you can write to Solace using - * - *

{@code
- * PCollection solaceRecs = ...;
- *
- * PCollection results =
- *         solaceRecs.apply(
- *                 "Write to Solace",
- *                 SolaceIO.write()
- *                         .to(SolaceIO.topicFromName("some-topic"))
- *                         .withSessionProvider(
- *                            BasicAuthenticationProvider.builder()
- *                              .username("username")
- *                              .password("password")
- *                              .host("host:port")
- *                              .build()));
- * }
- * - * The above code snippet will write to the VPN named "default", using 4 clients per worker (VM in - * Dataflow), and a maximum of 20 workers/VMs for writing (default values). You can change the - * default VPN name by setting the required JCSMP property in the session properties provider (in - * this case, with {@link BasicAuthenticationProvider#vpnName()}), the number of clients per worker - * with {@link Write#withNumberOfClientsPerWorker(int)} and the number of parallel write clients - * using {@link Write#withMaxNumOfUsedWorkers(int)}. - * - *

Direct and persistent messages, and latency metrics

- * - *

The connector can write either direct or persistent messages. The default mode is DIRECT. - * - *

The connector returns a {@link PCollection} of {@link Solace.PublishResult}, that you can use - * to get a confirmation of messages that have been published, or rejected, but only if it is - * publishing persistent messages. - * - *

If you are publishing persistent messages, then you can have some feedback about whether the - * messages have been published, and some publishing latency metrics. If the message has been - * published, {@link Solace.PublishResult#getPublished()} will be true. If it is false, it means - * that the message could not be published, and {@link Solace.PublishResult#getError()} will contain - * more details about why the message could not be published. To get latency metrics as well as the - * results, set the property {@link Write#publishLatencyMetrics()}. - * - *

Throughput and latency - * - *

This connector can work in two main modes: high latency or high throughput. The default mode - * favors high throughput over high latency. You can control this behavior with the methods {@link - * Write#withSubmissionMode(SubmissionMode)} and {@link Write#withWriterType(WriterType)}. - * - *

The default mode works like the following options: - * - *

{@code
- * PCollection solaceRecs = ...;
- *
- * PCollection results =
- *         solaceRecs.apply(
- *                 "Write to Solace",
- *                 SolaceIO.write()
- *                         .to(SolaceIO.topicFromName("some-topic"))
- *                         .withSessionProvider(
- *                            BasicAuthenticationProvider.builder()
- *                              .username("username")
- *                              .password("password")
- *                              .host("host:port")
- *                              .build())
- *                         .withSubmissionMode(SubmissionMode.HIGHER_THROUGHPUT)
- *                         .withWriterType(WriterType.BATCHED));
- * }
- * - *

{@link SubmissionMode#HIGHER_THROUGHPUT} and {@link WriterType#BATCHED} are the default - * values, and offer the higher possible throughput, and the lowest usage of resources in the runner - * side (due to the lower backpressure). - * - *

This connector writes bundles of 50 messages, using a bulk publish JCSMP method. This will - * increase the latency, since a message needs to "wait" until 50 messages are accumulated, before - * they are submitted to Solace. - * - *

For the lowest latency possible, use {@link SubmissionMode#LOWER_LATENCY} and {@link - * WriterType#STREAMING}. - * - *

{@code
- * PCollection results =
- *         solaceRecs.apply(
- *                 "Write to Solace",
- *                 SolaceIO.write()
- *                         .to(SolaceIO.topicFromName("some-topic"))
- *                         .withSessionProvider(
- *                            BasicAuthenticationProvider.builder()
- *                              .username("username")
- *                              .password("password")
- *                              .host("host:port")
- *                              .build())
- *                         .withSubmissionMode(SubmissionMode.LOWER_LATENCY)
- *                         .withWriterType(WriterType.STREAMING));
- * }
- * - * The streaming connector publishes each message individually, without holding up or batching - * before the message is sent to Solace. This will ensure the lowest possible latency, but it will - * offer a much lower throughput. The streaming connector does not use state & timers. - * - *

Both connectors uses state & timers to control the level of parallelism. If you are using - * Cloud Dataflow, it is recommended that you enable Streaming Engine to use this - * connector. - * - *

Connector retries - * - *

When the worker using the connector is created, the connector will attempt to connect to - * Solace. - * - *

If the client cannot connect to Solace for whatever reason, the connector will retry the - * connections using the following strategy. There will be a maximum of 4 retries. The first retry - * will be attempted 1 second after the first connection attempt. Every subsequent retry will - * multiply that time by a factor of two, with a maximum of 10 seconds. - * - *

If after those retries the client is still unable to connect to Solace, the connector will - * attempt to reconnect using the same strategy repeated for every single incoming message. If for - * some reason, there is a persistent issue that prevents the connection (e.g. client quota - * exhausted), you will need to stop your job manually, or the connector will keep retrying. - * - *

This strategy is applied to all the remote calls sent to Solace, either to connect, pull - * messages, push messages, etc. */ public class SolaceIO { - public static final Logger LOG = LoggerFactory.getLogger(SolaceIO.class); - public static final SerializableFunction SENDER_TIMESTAMP_FUNCTION = - (record) -> new Instant(record.getSenderTimestamp()); - public static final int DEFAULT_MAX_NUMBER_OF_WORKERS = 20; - public static final int DEFAULT_CLIENTS_PER_WORKER = 4; - public static final Boolean DEFAULT_PUBLISH_LATENCY_METRICS = false; - public static final SubmissionMode DEFAULT_SUBMISSION_MODE = SubmissionMode.HIGHER_THROUGHPUT; - public static final DeliveryMode DEFAULT_DELIVERY_MODE = DeliveryMode.DIRECT; - public static final WriterType DEFAULT_WRITER_TYPE = WriterType.BATCHED; - private static final boolean DEFAULT_DEDUPLICATE_RECORDS = true; - - public enum SubmissionMode { - HIGHER_THROUGHPUT, - LOWER_LATENCY - } - - public enum WriterType { - STREAMING, - BATCHED + public static final Logger LOG = LoggerFactory.getLogger(SolaceIO.class); + public static final SerializableFunction SENDER_TIMESTAMP_FUNCTION = + (record) -> new Instant(record.getSenderTimestamp()); + private static final boolean DEFAULT_DEDUPLICATE_RECORDS = true; + + /** Get a {@link Topic} object from the topic name. */ + static Topic topicFromName(String topicName) { + return JCSMPFactory.onlyInstance().createTopic(topicName); + } + + /** Get a {@link Queue} object from the queue name. */ + static Queue queueFromName(String queueName) { + return JCSMPFactory.onlyInstance().createQueue(queueName); + } + + /** + * Convert to a JCSMP destination from a schema-enabled {@link + * org.apache.beam.sdk.io.solace.data.Solace.Destination}. + * + *

This method returns a {@link Destination}, which may be either a {@link Topic} or a {@link + * Queue} + */ + public static Destination convertToJcsmpDestination(Solace.Destination destination) { + if (destination.getType().equals(Solace.DestinationType.TOPIC)) { + return topicFromName(destination.getName()); + } else if (destination.getType().equals(Solace.DestinationType.QUEUE)) { + return queueFromName(destination.getName()); + } else { + throw new IllegalArgumentException( + "SolaceIO.Write: Unknown destination type: " + destination.getType()); } - - /** Get a {@link Topic} object from the topic name. */ - static Topic topicFromName(String topicName) { - return JCSMPFactory.onlyInstance().createTopic(topicName); + } + + /** + * Create a {@link Read} transform, to read from Solace. The ingested records will be mapped to + * the {@link Solace.Record} objects. + */ + public static Read read() { + return Read.builder() + .setTypeDescriptor(TypeDescriptor.of(Solace.Record.class)) + .setParseFn(SolaceRecordMapper::map) + .setTimestampFn(SENDER_TIMESTAMP_FUNCTION) + .build(); + } + /** + * Create a {@link Read} transform, to read from Solace. Specify a {@link SerializableFunction} + * to map incoming {@link BytesXMLMessage} records, to the object of your choice. You also need + * to specify a {@link TypeDescriptor} for your class and the timestamp function which returns + * an {@link Instant} from the record. + * + *

The type descriptor will be used to infer a coder from CoderRegistry or Schema Registry. + * You can initialize a new TypeDescriptor in the following manner: + * + *

{@code
+   *   TypeDescriptor typeDescriptor = TypeDescriptor.of(YourOutputType.class);
+   * }
+   */
+  public static  Read read(
+      TypeDescriptor typeDescriptor,
+      SerializableFunction parseFn,
+      SerializableFunction timestampFn) {
+    return Read.builder()
+        .setTypeDescriptor(typeDescriptor)
+        .setParseFn(parseFn)
+        .setTimestampFn(timestampFn)
+        .build();
+  }
+
+  @AutoValue
+  public abstract static class Read extends PTransform> {
+    private static final Logger LOG = LoggerFactory.getLogger(Read.class);
+
+    /** Set the queue name to read from. Use this or the `from(Topic)` method. */
+    public Read from(Solace.Queue queue) {
+      return toBuilder().setQueue(queueFromName(queue.getName())).build();
     }
 
-    /** Get a {@link Queue} object from the queue name. */
-    static Queue queueFromName(String queueName) {
-        return JCSMPFactory.onlyInstance().createQueue(queueName);
+    /** Set the topic name to read from. Use this or the `from(Queue)` method. */
+    public Read from(Solace.Topic topic) {
+      return toBuilder().setTopic(topicFromName(topic.getName())).build();
     }
 
     /**
-     * Convert to a JCSMP destination from a schema-enabled {@link
-     * com.google.cloud.dataflow.dce.io.solace.data.Solace.Destination}.
-     *
-     * 

This method returns a {@link Destination}, which may be either a {@link Topic} or a {@link - * Queue} + * Set the timestamp function. This serializable has to output an {@link Instant}. This will be + * used to calculate watermark and define record's timestamp. */ - public static Destination convertToJcsmpDestination(Solace.Destination destination) { - if (destination.getType().equals(Solace.DestinationType.TOPIC)) { - return topicFromName(destination.getName()); - } else if (destination.getType().equals(Solace.DestinationType.QUEUE)) { - return queueFromName(destination.getName()); - } else { - throw new IllegalArgumentException( - "SolaceIO.Write: Unknown destination type: " + destination.getType()); - } + public Read withTimestampFn(SerializableFunction timestampFn) { + return toBuilder().setTimestampFn(timestampFn).build(); } - /** Create a {@link Write} transform, to write to Solace with a custom type. */ - public static Write write() { - return Write.builder().build(); + /** + * Maximum number of read connections created to Solace cluster. This is optional, leave out to + * let the Runner decide. + */ + public Read withMaxNumConnections(Integer maxNumConnections) { + return toBuilder().setMaxNumConnections(maxNumConnections).build(); } - /** Create a {@link Write} transform, to write to Solace using {@link Solace.Record} objects. */ - public static Write writeSolaceRecords() { - return Write.builder().build(); + /** + * Set if the read records should be deduplicated. True by default. It will use the + * `applicationMessageId` attribute to identify duplicates. + */ + public Read withDeduplicateRecords(boolean deduplicateRecords) { + return toBuilder().setDeduplicateRecords(deduplicateRecords).build(); } /** - * Create a {@link Read} transform, to read from Solace. The ingested records will be mapped to - * the {@link Solace.Record} objects. + * Set a factory that creates a {@link org.apache.beam.sdk.io.solace.broker.SempClientFactory}. + * + *

The factory `create()` method is invoked in each instance of an {@link + * org.apache.beam.sdk.io.solace.read.UnboundedSolaceReader}. Created {@link + * org.apache.beam.sdk.io.solace.broker.SempClient} has to communicate with broker management + * API. It must support operations such as: + * + *

    + *
  • query for outstanding backlog bytes in a Queue, + *
  • query for metadata such as access-type of a Queue, + *
  • requesting creation of new Queues. + *
*/ - public static Read read() { - return Read.builder() - .setTypeDescriptor(TypeDescriptor.of(Solace.Record.class)) - .setParseFn(SolaceRecordMapper::map) - .setTimestampFn(SENDER_TIMESTAMP_FUNCTION) - .build(); + public Read withSempClientFactory(SempClientFactory sempClientFactory) { + return toBuilder().setSempClientFactory(sempClientFactory).build(); } + /** - * Create a {@link Read} transform, to read from Solace. Specify a {@link SerializableFunction} - * to map incoming {@link BytesXMLMessage} records, to the object of your choice. You also need - * to specify a {@link TypeDescriptor} for your class and the timestamp function which returns - * an {@link Instant} from the record. + * Set a factory that creates a {@link SessionService}. * - *

The type descriptor will be used to infer a coder from CoderRegistry or Schema Registry. - * You can initialize a new TypeDescriptor in the following manner: + *

The factory `create()` method is invoked in each instance of an {@link + * org.apache.beam.sdk.io.solace.read.UnboundedSolaceReader}. Created {@link SessionService} has + * to be able to: * - *

{@code
-     *   TypeDescriptor typeDescriptor = TypeDescriptor.of(YourOutputType.class);
-     * }
+     * 
    + *
  • initialize a connection with the broker, + *
  • check liveliness of the connection, + *
  • close the connection, + *
  • create a {@link org.apache.beam.sdk.io.solace.broker.MessageReceiver}. + *
*/ - public static Read read( - TypeDescriptor typeDescriptor, - SerializableFunction parseFn, - SerializableFunction timestampFn) { - return Read.builder() - .setTypeDescriptor(typeDescriptor) - .setParseFn(parseFn) - .setTimestampFn(timestampFn) - .build(); + public Read withSessionServiceFactory(SessionServiceFactory sessionServiceFactory) { + return toBuilder().setSessionServiceFactory(sessionServiceFactory).build(); } - @AutoValue - public abstract static class Write extends PTransform, SolacePublishResult> { - - private static final Logger LOG = LoggerFactory.getLogger(Write.class); - - public static final TupleTag FAILED_PUBLISH_TAG = new TupleTag<>() {}; - public static final TupleTag SUCCESSFUL_PUBLISH_TAG = - new TupleTag<>() {}; - - /** - * Write to a Solace topic. - * - *

The topic does not need to exist before launching the pipeline. - * - *

This will write all records to the same topic, ignoring their destination field. - * - *

Optional. If not specified, the connector will use dynamic destinations based on the - * destination field of {@link Solace.Record}. - */ - public Write to(Solace.Topic topic) { - return toBuilder().setDestination(topicFromName(topic.getName())).build(); - } - - /** - * Write to a Solace queue. - * - *

The queue must exist prior to launching the pipeline. - * - *

This will write all records to the same queue, ignoring their destination field. - * - *

Optional. If not specified, the connector will use dynamic destinations based on the - * destination field of {@link Solace.Record}. - */ - public Write to(Solace.Queue queue) { - return toBuilder().setDestination(queueFromName(queue.getName())).build(); - } - - /** - * The number of workers used by the job to write to Solace. - * - *

This is optional, the default value is 20. - * - *

This is the maximum value that the job would use, but depending on the amount of data, - * the actual number of writers may be lower than this value. With the Dataflow runner, the - * connector will as maximum this number of VMs in the job (but the job itself may use more - * VMs). - * - *

Set this number taking into account the limits in the number of clients in your Solace - * cluster, and the need for performance when writing to Solace (more workers will achieve - * higher throughput). - */ - public Write withMaxNumOfUsedWorkers(int maxNumOfUsedWorkers) { - return toBuilder().setMaxNumOfUsedWorkers(maxNumOfUsedWorkers).build(); - } - - /** - * The number of clients that each worker will create. - * - *

This is optional, the default number is 4. - * - *

The number of clients is per worker. If there are more than one worker, the number of - * clients will be multiplied by the number of workers. With the Dataflow runner, this will - * be the number of clients created per VM. The clients will be re-used across different - * threads in the same worker. - * - *

Set this number in combination with {@link #withMaxNumOfUsedWorkers}, to ensure that - * the limit for number of clients in your Solace cluster is not exceeded. - * - *

Normally, using a higher number of clients with fewer workers will achieve better - * throughput at a lower cost, since the workers are better utilized. A good rule of thumb - * to use is setting as many clients per worker as vCPUs the worker has. - */ - public Write withNumberOfClientsPerWorker(int numberOfClientsPerWorker) { - return toBuilder().setNumberOfClientsPerWorker(numberOfClientsPerWorker).build(); - } - - /** - * Set the delivery mode. This is optional, the default value is DIRECT. - * - *

For more details, see https://docs.solace.com/API/API-Developer-Guide/Message-Delivery-Modes.htm - */ - public Write withDeliveryMode(DeliveryMode deliveryMode) { - return toBuilder().setDeliveryMode(deliveryMode).build(); - } - - /** - * Publish latency metrics using Beam metrics. - * - *

Latency metrics are only available if {@link #withDeliveryMode(DeliveryMode)} is set - * to PERSISTENT. In that mode, latency is measured for each single message, as the time - * difference between the message creation and the reception of the publishing confirmation. - * - *

For the batched writer, the creation time is set for every message in a batch shortly - * before the batch is submitted. So the latency is very close to the actual publishing - * latency, and it does not take into account the time spent waiting for the batch to be - * submitted. - * - *

This is optional, the default value is false (don't publish latency metrics). - */ - public Write publishLatencyMetrics() { - return toBuilder().setPublishLatencyMetrics(true).build(); - } - - /** - * This setting controls the JCSMP property MESSAGE_CALLBACK_ON_REACTOR. Optional. - * - *

For full details, please check https://docs.solace.com/API/API-Developer-Guide/Java-API-Best-Practices.htm. - * - *

The Solace JCSMP client libraries can dispatch messages using two different modes: - * - *

One of the modes dispatches messages directly from the same thread that is doing the - * rest of I/O work. This mode favors lower latency but lower throughput. Set this to - * LOWER_LATENCY to use that mode (MESSAGE_CALLBACK_ON_REACTOR set to True). - * - *

The other mode uses a parallel thread to accumulate and dispatch messages. This mode - * favors higher throughput but also has higher latency. Set this to HIGHER_THROUGHPUT to - * use that mode. This is the default mode (MESSAGE_CALLBACK_ON_REACTOR set to False). - * - *

This is optional, the default value is HIGHER_THROUGHPUT. - */ - public Write withSubmissionMode(SubmissionMode submissionMode) { - return toBuilder().setDispatchMode(submissionMode).build(); - } - - /** - * Set the type of writer used by the connector. Optional. - * - *

The Solace writer can either use the JCSMP modes in streaming or batched. - * - *

In streaming mode, the publishing latency will be lower, but the throughput will also - * be lower. - * - *

With the batched mode, messages are accumulated until a batch size of 50 is reached, - * or 5 seconds have elapsed since the first message in the batch was received. The 50 - * messages are sent to Solace in a single batch. This writer offers higher throughput but - * higher publishing latency, as messages can be held up for up to 5 seconds until they are - * published. - * - *

Notice that this is the message publishing latency, not the end-to-end latency. For - * very large scale pipelines, you will probably prefer to use the HIGHER_THROUGHPUT mode, - * as with lower throughput messages will accumulate in the pipeline, and the end-to-end - * latency may actually be higher. - * - *

This is optional, the default is the BATCHED writer. - */ - public Write withWriterType(WriterType writerType) { - return toBuilder().setWriterType(writerType).build(); - } - - /** - * Set the format function for your custom data type, and/or for dynamic destinations. - * - *

If you are using a custom data class, this function should return a {@link - * Solace.Record} corresponding to your custom data class instance. - * - *

If you are using this formatting function with dynamic destinations, you must ensure - * that you set the right value in the destination value of the {@link Solace.Record} - * messages. - * - *

In any other case, this format function is optional. - */ - public Write withFormatFunction(SerializableFunction formatFunction) { - return toBuilder().setFormatFunction(formatFunction).build(); - } - - /** - * Set the provider used to obtain the properties to initialize a new session in the broker. - * - *

This provider should define the destination host where the broker is listening, and - * all the properties related to authentication (base auth, client certificate, etc.). - */ - public Write withSessionPropertiesProvider(SessionPropertiesProvider provider) { - return toBuilder().setSessionPropertiesProvider(provider).build(); - } - - abstract int getMaxNumOfUsedWorkers(); - - abstract int getNumberOfClientsPerWorker(); - - abstract @Nullable Destination getDestination(); - - abstract DeliveryMode getDeliveryMode(); - - abstract boolean getPublishLatencyMetrics(); + abstract @Nullable Queue getQueue(); - abstract SubmissionMode getDispatchMode(); + abstract @Nullable Topic getTopic(); - abstract WriterType getWriterType(); + abstract @Nullable SerializableFunction getTimestampFn(); - abstract @Nullable SerializableFunction getFormatFunction(); + abstract @Nullable Integer getMaxNumConnections(); - abstract @Nullable SessionPropertiesProvider getSessionPropertiesProvider(); + abstract boolean getDeduplicateRecords(); - static Builder builder() { - return new AutoValue_SolaceIO_Write.Builder() - .setDeliveryMode(DEFAULT_DELIVERY_MODE) - .setMaxNumOfUsedWorkers(DEFAULT_MAX_NUMBER_OF_WORKERS) - .setNumberOfClientsPerWorker(DEFAULT_CLIENTS_PER_WORKER) - .setPublishLatencyMetrics(DEFAULT_PUBLISH_LATENCY_METRICS) - .setDispatchMode(DEFAULT_SUBMISSION_MODE) - .setWriterType(DEFAULT_WRITER_TYPE); - } - - abstract Builder toBuilder(); - - @AutoValue.Builder - abstract static class Builder { - abstract Builder setMaxNumOfUsedWorkers(int maxNumOfUsedWorkers); - - abstract Builder setNumberOfClientsPerWorker(int numberOfClientsPerWorker); - - abstract Builder setDestination(Destination topicOrQueue); - - abstract Builder setDeliveryMode(DeliveryMode deliveryMode); - - abstract Builder setPublishLatencyMetrics(Boolean publishLatencyMetrics); - - abstract Builder setDispatchMode(SubmissionMode submissionMode); - - abstract Builder setWriterType(WriterType writerType); - - abstract Builder setFormatFunction( - SerializableFunction formatFunction); - - abstract Builder setSessionPropertiesProvider( - SessionPropertiesProvider propertiesProvider); - - abstract Write build(); - } - - @Override - public SolacePublishResult expand(PCollection input) { - Class pcollClass = input.getTypeDescriptor().getRawType(); - boolean usingSolaceRecord = pcollClass.isAssignableFrom(Solace.Record.class); - // todo definitely don't merge. Seems like a bug! - // usingSolaceRecord = true; - // System.out.println(pcollClass + " " + - // pcollClass.isAssignableFrom(Solace.Record.class) ); - validate(usingSolaceRecord); - - // Register schema for subtypes needed by Record - - boolean usingDynamicDestinations = getDestination() == null; - SerializableFunction destinationFn; - if (usingDynamicDestinations) { - destinationFn = x -> SolaceIO.convertToJcsmpDestination(x.getDestination()); - } else { - // Constant destination for all messages (same topic or queue) - destinationFn = x -> getDestination(); - } - - @SuppressWarnings("unchecked") - PCollection records = - getFormatFunction() == null - ? (PCollection) input - : input.apply( - "Format records", - MapElements.into(TypeDescriptor.of(Solace.Record.class)) - .via(getFormatFunction())); - - // Store the current window used by the input - PCollection captureWindow = - records.apply( - "Capture window", - ParDo.of(new UnboundedSolaceWriter.RecordToPublishResultDoFn())); - - @SuppressWarnings("unchecked") - WindowingStrategy windowingStrategy = - (WindowingStrategy) - captureWindow.getWindowingStrategy(); - - PCollection> withShardKeys = - records.apply( - "Add shard key", - ParDo.of( - new UnboundedSolaceWriter.AddShardKeyDoFn( - getMaxNumOfUsedWorkers()))); - - PCollection> withGlobalWindow = - withShardKeys.apply("Global window", Window.into(new GlobalWindows())); - - String label = - getWriterType() == WriterType.STREAMING - ? "Publish (streaming)" - : "Publish (batched)"; - - PCollectionTuple solaceOutput = - withGlobalWindow.apply(label, getWriterTransform(destinationFn)); - - SolacePublishResult output; - if (getDeliveryMode() == DeliveryMode.PERSISTENT) { - PCollection failedPublish = - solaceOutput.get(FAILED_PUBLISH_TAG); - PCollection successfulPublish = - solaceOutput.get(SUCCESSFUL_PUBLISH_TAG); - output = - rewindow( - SolacePublishResult.in( - input.getPipeline(), failedPublish, successfulPublish), - windowingStrategy); - } else { - LOG.info( - String.format( - "Solace.Write: omitting writer output because delivery mode is %s", - getDeliveryMode())); - output = SolacePublishResult.in(input.getPipeline(), null, null); - } - - return output; - } - - private ParDo.MultiOutput, Solace.PublishResult> - getWriterTransform(SerializableFunction destinationFn) { - - ParDo.SingleOutput, Solace.PublishResult> writer = - ParDo.of( - getWriterType() == WriterType.STREAMING - ? new UnboundedStreamingSolaceWriter.WriterDoFn( - destinationFn, - getSessionPropertiesProvider(), - getDeliveryMode(), - getDispatchMode(), - getNumberOfClientsPerWorker(), - getPublishLatencyMetrics()) - : new UnboundedBatchedSolaceWriter.WriterDoFn( - destinationFn, - getSessionPropertiesProvider(), - getDeliveryMode(), - getDispatchMode(), - getNumberOfClientsPerWorker(), - getPublishLatencyMetrics())); - - return writer.withOutputTags( - FAILED_PUBLISH_TAG, TupleTagList.of(SUCCESSFUL_PUBLISH_TAG)); - } + abstract SerializableFunction getParseFn(); - private SolacePublishResult rewindow( - SolacePublishResult solacePublishResult, - WindowingStrategy strategy) { - PCollection correct = solacePublishResult.getSuccessfulPublish(); - PCollection failed = solacePublishResult.getFailedPublish(); + abstract @Nullable SempClientFactory getSempClientFactory(); - PCollection correctWithWindow = null; - PCollection failedWithWindow = null; + abstract @Nullable SessionServiceFactory getSessionServiceFactory(); - if (correct != null) { - correctWithWindow = applyOriginalWindow(correct, strategy, "Rewindow correct"); - } + abstract TypeDescriptor getTypeDescriptor(); - if (failed != null) { - failedWithWindow = applyOriginalWindow(failed, strategy, "Rewindow failed"); - } - - return SolacePublishResult.in( - solacePublishResult.getPipeline(), failedWithWindow, correctWithWindow); - } - - private static PCollection applyOriginalWindow( - PCollection pcoll, - WindowingStrategy strategy, - String label) { - Window originalWindow = captureWindowDetails(strategy); - - if (strategy.getMode() == WindowingStrategy.AccumulationMode.ACCUMULATING_FIRED_PANES) { - originalWindow = originalWindow.accumulatingFiredPanes(); - } else { - originalWindow = originalWindow.discardingFiredPanes(); - } - - return pcoll.apply(label, originalWindow); - } - - private static Window captureWindowDetails( - WindowingStrategy strategy) { - return Window.into(strategy.getWindowFn()) - .withAllowedLateness(strategy.getAllowedLateness()) - .withOnTimeBehavior(strategy.getOnTimeBehavior()) - .withTimestampCombiner(strategy.getTimestampCombiner()) - .triggering(strategy.getTrigger()); - } - - /** - * Called before running the Pipeline to verify this transform is fully and correctly - * specified. - */ - private void validate(boolean usingSolaceRecords) { - if (!usingSolaceRecords) { - Preconditions.checkArgument( - getFormatFunction() != null, - "SolaceIO.Write: If you are not using Solace.Record as the input type, you" - + " must set a format function using withFormatFunction()."); - } - - Preconditions.checkArgument( - getMaxNumOfUsedWorkers() > 0, - "SolaceIO.Write: The number of used workers must be positive."); - Preconditions.checkArgument( - getNumberOfClientsPerWorker() > 0, - "SolaceIO.Write: The number of clients per worker must be positive."); - Preconditions.checkArgument( - getDeliveryMode() == DeliveryMode.DIRECT - || getDeliveryMode() == DeliveryMode.PERSISTENT, - String.format( - "SolaceIO.Write: Delivery mode must be either DIRECT or PERSISTENT. %s" - + " not supported", - getDeliveryMode())); - if (getPublishLatencyMetrics()) { - Preconditions.checkArgument( - getDeliveryMode() == DeliveryMode.PERSISTENT, - "SolaceIO.Write: Publish latency metrics can only be enabled for PERSISTENT" - + " delivery mode."); - } - Preconditions.checkArgument( - getSessionPropertiesProvider() != null, - "SolaceIO: You need to pass a session properties provider. For basic" - + " authentication, you can use BasicAuthenticationProvider."); - } + public static Builder builder() { + Builder builder = new org.apache.beam.sdk.io.solace.AutoValue_SolaceIO_Read.Builder(); + builder.setDeduplicateRecords(DEFAULT_DEDUPLICATE_RECORDS); + return builder; } - @AutoValue - public abstract static class Read extends PTransform> { - private static final Logger LOG = LoggerFactory.getLogger(Read.class); - - /** Set the queue name to read from. Use this or the `from(Topic)` method. */ - public Read from(Solace.Queue queue) { - return toBuilder().setQueue(queueFromName(queue.getName())).build(); - } - - /** Set the topic name to read from. Use this or the `from(Queue)` method. */ - public Read from(Solace.Topic topic) { - return toBuilder().setTopic(topicFromName(topic.getName())).build(); - } - - /** - * Set the timestamp function. This serializable has to output an {@link Instant}. This will - * be used to calculate watermark and define record's timestamp. - */ - public Read withTimestampFn(SerializableFunction timestampFn) { - return toBuilder().setTimestampFn(timestampFn).build(); - } - - /** - * Maximum number of read connections created to Solace cluster. This is optional, leave out - * to let the Runner decide. - */ - public Read withMaxNumConnections(Integer maxNumConnections) { - return toBuilder().setMaxNumConnections(maxNumConnections).build(); - } - - /** - * Set if the read records should be deduplicated. True by default. It will use the - * `applicationMessageId` attribute to identify duplicates. - */ - public Read withDeduplicateRecords(boolean deduplicateRecords) { - return toBuilder().setDeduplicateRecords(deduplicateRecords).build(); - } - - /** - * Set a factory that creates a {@link - * com.google.cloud.dataflow.dce.io.solace.broker.BrokerService}. - * - *

The factory `create()` method is invoked in each instance of an {@link - * com.google.cloud.dataflow.dce.io.solace.read.UnboundedSolaceReader}. Created {@link - * com.google.cloud.dataflow.dce.io.solace.broker.BrokerService} has to communicate with - * broker management API. It must support operations such as: - * - *

    - *
  • query for outstanding backlog bytes in a Queue, - *
  • query for metadata such as access-type of a Queue, - *
  • requesting creation of new Queues. - *
- */ - public Read withSempClientFactory(SempClientFactory sempClientFactory) { - return toBuilder().setSempClientFactory(sempClientFactory).build(); - } - - /** - * Set a factory that creates a {@link SessionService}. - * - *

The factory `create()` method is invoked in each instance of an {@link - * com.google.cloud.dataflow.dce.io.solace.read.UnboundedSolaceReader}. Created {@link - * SessionService} has to be able to: - * - *

    - *
  • initialize a connection with the broker, - *
  • check liveliness of the connection, - *
  • close the connection, - *
  • create a {@link com.google.cloud.dataflow.dce.io.solace.broker.MessageReceiver}. - *
- */ - public Read withSessionServiceFactory(SessionServiceFactory sessionServiceFactory) { - return toBuilder().setSessionServiceFactory(sessionServiceFactory).build(); - } - - abstract @Nullable Queue getQueue(); - - abstract @Nullable Topic getTopic(); - - abstract @Nullable SerializableFunction getTimestampFn(); - - abstract @Nullable Integer getMaxNumConnections(); - - abstract boolean getDeduplicateRecords(); - - abstract SerializableFunction getParseFn(); - - abstract @Nullable SempClientFactory getSempClientFactory(); - - abstract @Nullable SessionServiceFactory getSessionServiceFactory(); - - abstract TypeDescriptor getTypeDescriptor(); - - public static Builder builder() { - Builder builder = new AutoValue_SolaceIO_Read.Builder(); - builder.setDeduplicateRecords(DEFAULT_DEDUPLICATE_RECORDS); - return builder; - } - - abstract Builder toBuilder(); + abstract Builder toBuilder(); - @AutoValue.Builder - public abstract static class Builder { + @AutoValue.Builder + public abstract static class Builder { - abstract Builder setQueue(Queue queue); + abstract Builder setQueue(Queue queue); - abstract Builder setTopic(Topic topic); + abstract Builder setTopic(Topic topic); - abstract Builder setTimestampFn(SerializableFunction timestampFn); + abstract Builder setTimestampFn(SerializableFunction timestampFn); - abstract Builder setMaxNumConnections(Integer maxNumConnections); + abstract Builder setMaxNumConnections(Integer maxNumConnections); - abstract Builder setDeduplicateRecords(boolean deduplicateRecords); + abstract Builder setDeduplicateRecords(boolean deduplicateRecords); - abstract Builder setParseFn(SerializableFunction parseFn); + abstract Builder setParseFn(SerializableFunction parseFn); - abstract Builder setSempClientFactory(SempClientFactory brokerServiceFactory); + abstract Builder setSempClientFactory(SempClientFactory brokerServiceFactory); - abstract Builder setSessionServiceFactory( - SessionServiceFactory sessionServiceFactory); + abstract Builder setSessionServiceFactory(SessionServiceFactory sessionServiceFactory); - abstract Builder setTypeDescriptor(TypeDescriptor typeDescriptor); + abstract Builder setTypeDescriptor(TypeDescriptor typeDescriptor); - abstract Read build(); - } + abstract Read build(); + } - @Override - public PCollection expand(PBegin input) { - validate(); - - SempClientFactory sempClientFactory = getSempClientFactory(); - String jobName = input.getPipeline().getOptions().getJobName(); - Queue queue = - getQueue() != null - ? getQueue() - : initializeQueueForTopic(jobName, sempClientFactory); - - SessionServiceFactory sessionServiceFactory = getSessionServiceFactory(); - sessionServiceFactory.setQueue(queue); - - registerDefaultCoder(input.getPipeline()); - // Infer the actual coder - Coder coder = inferCoder(input.getPipeline()); - - return input.apply( - org.apache.beam.sdk.io.Read.from( - new UnboundedSolaceSource<>( - queue, - sempClientFactory, - sessionServiceFactory, - getMaxNumConnections(), - getDeduplicateRecords(), - coder, - getTimestampFn(), - getParseFn()))); - } + @Override + public PCollection expand(PBegin input) { + validate(); + + SempClientFactory sempClientFactory = getSempClientFactory(); + String jobName = input.getPipeline().getOptions().getJobName(); + Queue queue = + getQueue() != null ? getQueue() : initializeQueueForTopic(jobName, sempClientFactory); + + SessionServiceFactory sessionServiceFactory = getSessionServiceFactory(); + sessionServiceFactory.setQueue(queue); + + registerDefaultCoder(input.getPipeline()); + // Infer the actual coder + Coder coder = inferCoder(input.getPipeline()); + + return input.apply( + org.apache.beam.sdk.io.Read.from( + new UnboundedSolaceSource<>( + queue, + sempClientFactory, + sessionServiceFactory, + getMaxNumConnections(), + getDeduplicateRecords(), + coder, + getTimestampFn(), + getParseFn()))); + } - private static void registerDefaultCoder(Pipeline pipeline) { - pipeline.getCoderRegistry() - .registerCoderForType( - TypeDescriptor.of(Solace.Record.class), SolaceRecordCoder.of()); - } + private static void registerDefaultCoder(Pipeline pipeline) { + pipeline + .getCoderRegistry() + .registerCoderForType(TypeDescriptor.of(Solace.Record.class), SolaceRecordCoder.of()); + } - @VisibleForTesting - Coder inferCoder(Pipeline pipeline) { - Coder coderFromCoderRegistry = getFromCoderRegistry(pipeline); - if (coderFromCoderRegistry != null) { - return coderFromCoderRegistry; - } - - Coder coderFromSchemaRegistry = getFromSchemaRegistry(pipeline); - if (coderFromSchemaRegistry != null) { - return coderFromSchemaRegistry; - } - - throw new RuntimeException( - "SolaceIO.Read: Cannot infer a coder for the TypeDescriptor. Annotate you" - + " output class with @DefaultSchema annotation or create a coder manually" - + " and register it in the CoderRegistry."); - } + @VisibleForTesting + Coder inferCoder(Pipeline pipeline) { + Coder coderFromCoderRegistry = getFromCoderRegistry(pipeline); + if (coderFromCoderRegistry != null) { + return coderFromCoderRegistry; + } + + Coder coderFromSchemaRegistry = getFromSchemaRegistry(pipeline); + if (coderFromSchemaRegistry != null) { + return coderFromSchemaRegistry; + } + + throw new RuntimeException( + "SolaceIO.Read: Cannot infer a coder for the TypeDescriptor. Annotate you" + + " output class with @DefaultSchema annotation or create a coder manually" + + " and register it in the CoderRegistry."); + } - private Coder getFromSchemaRegistry(Pipeline pipeline) { - try { - return pipeline.getSchemaRegistry().getSchemaCoder(getTypeDescriptor()); - } catch (NoSuchSchemaException e) { - return null; - } - } + private Coder getFromSchemaRegistry(Pipeline pipeline) { + try { + return pipeline.getSchemaRegistry().getSchemaCoder(getTypeDescriptor()); + } catch (NoSuchSchemaException e) { + return null; + } + } - private Coder getFromCoderRegistry(Pipeline pipeline) { - try { - return pipeline.getCoderRegistry().getCoder(getTypeDescriptor()); - } catch (CannotProvideCoderException e) { - return null; - } - } + private Coder getFromCoderRegistry(Pipeline pipeline) { + try { + return pipeline.getCoderRegistry().getCoder(getTypeDescriptor()); + } catch (CannotProvideCoderException e) { + return null; + } + } - // FIXME: this is public only for the sake of testing, TODO: redesign test so this is - // private - public Queue initializeQueueForTopic(String jobName, SempClientFactory sempClientFactory) { - Queue q; - if (getQueue() != null) { - q = getQueue(); - } else { - String queueName = String.format("queue-%s-%s", getTopic(), jobName); - try { - String topicName = Objects.requireNonNull(getTopic()).getName(); - q = sempClientFactory.create().createQueueForTopic(queueName, topicName); - LOG.info( - "SolaceIO.Read: A new queue {} was created. The Queue will not be" - + " deleted when this job finishes. Make sure to remove it yourself" - + " when not needed.", - q.getName()); - } catch (IOException e) { - throw new RuntimeException(e); - } - } - return q; - } + // FIXME: this is public only for the sake of testing, TODO: redesign test so this is + // private + public Queue initializeQueueForTopic(String jobName, SempClientFactory sempClientFactory) { + Queue q; + if (getQueue() != null) { + q = getQueue(); + } else { + String queueName = String.format("queue-%s-%s", getTopic(), jobName); + try { + String topicName = Objects.requireNonNull(getTopic()).getName(); + q = sempClientFactory.create().createQueueForTopic(queueName, topicName); + LOG.info( + "SolaceIO.Read: A new queue {} was created. The Queue will not be" + + " deleted when this job finishes. Make sure to remove it yourself" + + " when not needed.", + q.getName()); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + return q; + } - private void validate() { - Preconditions.checkState( - getSempClientFactory() != null, - "SolaceIO.Read: brokerServiceFactory must not be null."); - Preconditions.checkState( - getSessionServiceFactory() != null, - "SolaceIO.Read: SessionServiceFactory must not be null."); - Preconditions.checkState( - getParseFn() != null, - "SolaceIO.Read: parseFn must be set or use the `Read.readSolaceRecords()`" - + " method"); - Preconditions.checkState( - getTimestampFn() != null, - "SolaceIO.Read: timestamp function must be set or use the" - + " `Read.readSolaceRecords()` method"); - Preconditions.checkState( - (getQueue() == null ^ getTopic() == null), - "SolaceIO.Read: One of the Solace {Queue, Topic} must be set."); - } + private void validate() { + Preconditions.checkState( + getSempClientFactory() != null, "SolaceIO.Read: brokerServiceFactory must not be null."); + Preconditions.checkState( + getSessionServiceFactory() != null, + "SolaceIO.Read: SessionServiceFactory must not be null."); + Preconditions.checkState( + getParseFn() != null, + "SolaceIO.Read: parseFn must be set or use the `Read.readSolaceRecords()`" + " method"); + Preconditions.checkState( + getTimestampFn() != null, + "SolaceIO.Read: timestamp function must be set or use the" + + " `Read.readSolaceRecords()` method"); + Preconditions.checkState( + (getQueue() == null ^ getTopic() == null), + "SolaceIO.Read: One of the Solace {Queue, Topic} must be set."); } + } } diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/BasicAuthJcsmpSessionService.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/BasicAuthJcsmpSessionService.java index 2fb85aaf3f27..2843f24cf25e 100644 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/BasicAuthJcsmpSessionService.java +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/BasicAuthJcsmpSessionService.java @@ -1,11 +1,13 @@ /* - * Copyright 2024 Google. + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -13,9 +15,8 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.google.cloud.dataflow.dce.io.solace.broker; +package org.apache.beam.sdk.io.solace.broker; -import com.google.cloud.dataflow.dce.io.solace.RetryCallableManager; import com.solacesystems.jcsmp.ConsumerFlowProperties; import com.solacesystems.jcsmp.EndpointProperties; import com.solacesystems.jcsmp.InvalidPropertiesException; @@ -25,87 +26,95 @@ import com.solacesystems.jcsmp.JCSMPSession; import com.solacesystems.jcsmp.Queue; import java.io.IOException; -import java.util.Set; +import javax.annotation.Nullable; +import org.apache.beam.sdk.io.solace.RetryCallableManager; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet; public class BasicAuthJcsmpSessionService implements SessionService { - private final String queueName; - private final String host; - private final String username; - private final String password; - private final String vpnName; - private JCSMPSession jcsmpSession; - private final RetryCallableManager retryCallableManager = RetryCallableManager.create(); - - public BasicAuthJcsmpSessionService( - String queueName, String host, String username, String password, String vpnName) { - this.queueName = queueName; - this.host = host; - this.username = username; - this.password = password; - this.vpnName = vpnName; - } + private final String queueName; + private final String host; + private final String username; + private final String password; + private final String vpnName; + @Nullable private JCSMPSession jcsmpSession; + private final RetryCallableManager retryCallableManager = RetryCallableManager.create(); - @Override - public void connect() { - retryCallableManager.retryCallable(this::connectSession, Set.of(JCSMPException.class)); - } + public BasicAuthJcsmpSessionService( + String queueName, String host, String username, String password, String vpnName) { + this.queueName = queueName; + this.host = host; + this.username = username; + this.password = password; + this.vpnName = vpnName; + } - @Override - public void close() { - if (jcsmpSession != null && !jcsmpSession.isClosed()) { - retryCallableManager.retryCallable( - () -> { - jcsmpSession.closeSession(); - return 0; - }, - Set.of(IOException.class)); - } - } + @Override + public void connect() { + retryCallableManager.retryCallable(this::connectSession, ImmutableSet.of(JCSMPException.class)); + } - @Override - public MessageReceiver createReceiver() { - return retryCallableManager.retryCallable( - this::createFlowReceiver, Set.of(JCSMPException.class)); + @Override + public void close() { + if (jcsmpSession != null && !jcsmpSession.isClosed()) { + retryCallableManager.retryCallable( + () -> { + if (jcsmpSession != null) { + jcsmpSession.closeSession(); + } + return 0; + }, + ImmutableSet.of(IOException.class)); } + } - @Override - public boolean isClosed() { - return jcsmpSession == null || jcsmpSession.isClosed(); - } + @Override + public MessageReceiver createReceiver() { + return retryCallableManager.retryCallable( + this::createFlowReceiver, ImmutableSet.of(JCSMPException.class)); + } - private MessageReceiver createFlowReceiver() throws JCSMPException { - if (isClosed()) { - connectSession(); - } + @Override + public boolean isClosed() { + return jcsmpSession == null || jcsmpSession.isClosed(); + } - Queue queue = JCSMPFactory.onlyInstance().createQueue(queueName); + private MessageReceiver createFlowReceiver() throws JCSMPException, IOException { + if (isClosed()) { + connectSession(); + } - ConsumerFlowProperties flowProperties = new ConsumerFlowProperties(); - flowProperties.setEndpoint(queue); - flowProperties.setAckMode(JCSMPProperties.SUPPORTED_MESSAGE_ACK_CLIENT); + Queue queue = JCSMPFactory.onlyInstance().createQueue(queueName); - EndpointProperties endpointProperties = new EndpointProperties(); - endpointProperties.setAccessType(EndpointProperties.ACCESSTYPE_NONEXCLUSIVE); + ConsumerFlowProperties flowProperties = new ConsumerFlowProperties(); + flowProperties.setEndpoint(queue); + flowProperties.setAckMode(JCSMPProperties.SUPPORTED_MESSAGE_ACK_CLIENT); - return new SolaceMessageReceiver( - jcsmpSession.createFlow(null, flowProperties, endpointProperties)); + EndpointProperties endpointProperties = new EndpointProperties(); + endpointProperties.setAccessType(EndpointProperties.ACCESSTYPE_NONEXCLUSIVE); + if (jcsmpSession != null) { + return new SolaceMessageReceiver( + jcsmpSession.createFlow(null, flowProperties, endpointProperties)); + } else { + throw new IOException( + "SolaceIO.Read: Could not create a receiver from the Jcsmp session: session is null."); } + } - private int connectSession() throws JCSMPException { - if (jcsmpSession == null) { - jcsmpSession = createSessionObject(); - } - jcsmpSession.connect(); - return 0; + private int connectSession() throws JCSMPException { + if (jcsmpSession == null) { + jcsmpSession = createSessionObject(); } + jcsmpSession.connect(); + return 0; + } - private JCSMPSession createSessionObject() throws InvalidPropertiesException { - JCSMPProperties properties = new JCSMPProperties(); - properties.setProperty(JCSMPProperties.HOST, host); - properties.setProperty(JCSMPProperties.USERNAME, username); - properties.setProperty(JCSMPProperties.PASSWORD, password); - properties.setProperty(JCSMPProperties.VPN_NAME, vpnName); + private JCSMPSession createSessionObject() throws InvalidPropertiesException { + JCSMPProperties properties = new JCSMPProperties(); + properties.setProperty(JCSMPProperties.HOST, host); + properties.setProperty(JCSMPProperties.USERNAME, username); + properties.setProperty(JCSMPProperties.PASSWORD, password); + properties.setProperty(JCSMPProperties.VPN_NAME, vpnName); - return JCSMPFactory.onlyInstance().createSession(properties); - } + return JCSMPFactory.onlyInstance().createSession(properties); + } } diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/BasicAuthJcsmpSessionServiceFactory.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/BasicAuthJcsmpSessionServiceFactory.java index a4f8f7f2bc07..a8f727f97acd 100644 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/BasicAuthJcsmpSessionServiceFactory.java +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/BasicAuthJcsmpSessionServiceFactory.java @@ -1,11 +1,13 @@ /* - * Copyright 2024 Google. + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -13,63 +15,63 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.google.cloud.dataflow.dce.io.solace.broker; +package org.apache.beam.sdk.io.solace.broker; -import com.google.common.base.Preconditions; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; public class BasicAuthJcsmpSessionServiceFactory extends SessionServiceFactory { - private final String host; - private final String username; - private final String password; - private final String vpnName; + private final String host; + private final String username; + private final String password; + private final String vpnName; - private BasicAuthJcsmpSessionServiceFactory( - String host, String username, String password, String vpnName) { - this.host = host; - this.username = username; - this.password = password; - this.vpnName = vpnName; - } + private BasicAuthJcsmpSessionServiceFactory( + String host, String username, String password, String vpnName) { + this.host = host; + this.username = username; + this.password = password; + this.vpnName = vpnName; + } - public static BasicAuthJcsmpSessionServiceFactoryBuilder builder() { - return new BasicAuthJcsmpSessionServiceFactoryBuilder(); - } + public static BasicAuthJcsmpSessionServiceFactoryBuilder builder() { + return new BasicAuthJcsmpSessionServiceFactoryBuilder(); + } - @Override - public SessionService create() { - Preconditions.checkState(queue != null, "SolaceIO.Read: Queue is not set."); - return new BasicAuthJcsmpSessionService(queue.getName(), host, username, password, vpnName); - } + @Override + public SessionService create() { + Preconditions.checkState(queue != null, "SolaceIO.Read: Queue is not set."); + return new BasicAuthJcsmpSessionService(queue.getName(), host, username, password, vpnName); + } - public static class BasicAuthJcsmpSessionServiceFactoryBuilder { + public static class BasicAuthJcsmpSessionServiceFactoryBuilder { - private String host; - private String username; - private String password; - private String vpnName; + private String host; + private String username; + private String password; + private String vpnName; - public BasicAuthJcsmpSessionServiceFactoryBuilder withHost(String host) { - this.host = host; - return this; - } + public BasicAuthJcsmpSessionServiceFactoryBuilder withHost(String host) { + this.host = host; + return this; + } - public BasicAuthJcsmpSessionServiceFactoryBuilder withUsername(String username) { - this.username = username; - return this; - } + public BasicAuthJcsmpSessionServiceFactoryBuilder withUsername(String username) { + this.username = username; + return this; + } - public BasicAuthJcsmpSessionServiceFactoryBuilder withPassword(String password) { - this.password = password; - return this; - } + public BasicAuthJcsmpSessionServiceFactoryBuilder withPassword(String password) { + this.password = password; + return this; + } - public BasicAuthJcsmpSessionServiceFactoryBuilder withVpnName(String vpnName) { - this.vpnName = vpnName; - return this; - } + public BasicAuthJcsmpSessionServiceFactoryBuilder withVpnName(String vpnName) { + this.vpnName = vpnName; + return this; + } - public BasicAuthJcsmpSessionServiceFactory build() { - return new BasicAuthJcsmpSessionServiceFactory(host, username, password, vpnName); - } + public BasicAuthJcsmpSessionServiceFactory build() { + return new BasicAuthJcsmpSessionServiceFactory(host, username, password, vpnName); } + } } diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/BasicAuthSempClient.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/BasicAuthSempClient.java index d2cc716583f5..974e5d1884b0 100644 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/BasicAuthSempClient.java +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/BasicAuthSempClient.java @@ -1,11 +1,13 @@ /* - * Copyright 2024 Google. + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -13,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.google.cloud.dataflow.dce.io.solace.broker; +package org.apache.beam.sdk.io.solace.broker; import com.fasterxml.jackson.databind.DeserializationFeature; import com.fasterxml.jackson.databind.ObjectMapper; @@ -22,230 +24,226 @@ import com.fasterxml.jackson.databind.annotation.JsonSerialize; import com.google.api.client.http.HttpRequestFactory; import com.google.auto.value.AutoValue; -import com.google.cloud.dataflow.dce.io.solace.SerializableSupplier; import com.solacesystems.jcsmp.JCSMPFactory; import java.io.IOException; import org.apache.beam.sdk.annotations.Internal; +import org.apache.beam.sdk.io.solace.SerializableSupplier; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @Internal public class BasicAuthSempClient implements SempClient { - private static final Logger LOG = LoggerFactory.getLogger(BasicAuthSempClient.class); - private final ObjectMapper objectMapper = - new ObjectMapper().configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); - - private final SempBasicAuthClientExecutor sempBasicAuthClientExecutor; - - public BasicAuthSempClient( - String host, - String username, - String password, - String vpnName, - SerializableSupplier httpRequestFactorySupplier) { - sempBasicAuthClientExecutor = - new SempBasicAuthClientExecutor( - host, username, password, vpnName, httpRequestFactorySupplier.get()); + private static final Logger LOG = LoggerFactory.getLogger(BasicAuthSempClient.class); + private final ObjectMapper objectMapper = + new ObjectMapper().configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); + + private final SempBasicAuthClientExecutor sempBasicAuthClientExecutor; + + public BasicAuthSempClient( + String host, + String username, + String password, + String vpnName, + SerializableSupplier httpRequestFactorySupplier) { + sempBasicAuthClientExecutor = + new SempBasicAuthClientExecutor( + host, username, password, vpnName, httpRequestFactorySupplier.get()); + } + + @Override + public boolean isQueueNonExclusive(String queueName) throws IOException { + LOG.info("SolaceIO.Read: SempOperations: query SEMP if queue {} is nonExclusive", queueName); + BrokerResponse response = sempBasicAuthClientExecutor.getQueueResponse(queueName); + try { + Queue q = getSuccessResponseOrThrowException(response, Queue.class); + return q.data().accessType().equals("non-exclusive"); + } catch (SolaceSempException e) { + throw new IOException("SolaceIO.Read: Exception when querying for Queue metadata.", e); } - - @Override - public boolean isQueueNonExclusive(String queueName) throws IOException { - LOG.info( - "SolaceIO.Read: SempOperations: query SEMP if queue {} is nonExclusive", queueName); - BrokerResponse response = sempBasicAuthClientExecutor.getQueueResponse(queueName); - try { - Queue q = getSuccessResponseOrThrowException(response, Queue.class); - return q.data().accessType().equals("non-exclusive"); - } catch (SolaceSempException e) { - throw new IOException("SolaceIO.Read: Exception when querying for Queue metadata.", e); - } + } + + @Override + public com.solacesystems.jcsmp.Queue createQueueForTopic(String queueName, String topicName) + throws IOException { + try { + createQueue(queueName); + createSubscription(queueName, topicName); + } catch (SolaceSempException e) { + throw new IOException("SolaceIO.Read: Exception when creating a Queue for a Topic.", e); } - - @Override - public com.solacesystems.jcsmp.Queue createQueueForTopic(String queueName, String topicName) - throws IOException { - try { - createQueue(queueName); - createSubscription(queueName, topicName); - } catch (SolaceSempException e) { - throw new IOException("SolaceIO.Read: Exception when creating a Queue for a Topic.", e); - } - return JCSMPFactory.onlyInstance().createQueue(queueName); - } - - @Override - public long getBacklogBytes(String queueName) throws IOException { - BrokerResponse response = sempBasicAuthClientExecutor.getQueueResponse(queueName); - try { - Queue q = getSuccessResponseOrThrowException(response, Queue.class); - return q.data().msgSpoolUsage(); - } catch (SolaceSempException e) { - throw new IOException("SolaceIO.Read: Exception when querying for backlog bytes.", e); - } + return JCSMPFactory.onlyInstance().createQueue(queueName); + } + + @Override + public long getBacklogBytes(String queueName) throws IOException { + BrokerResponse response = sempBasicAuthClientExecutor.getQueueResponse(queueName); + try { + Queue q = getSuccessResponseOrThrowException(response, Queue.class); + return q.data().msgSpoolUsage(); + } catch (SolaceSempException e) { + throw new IOException("SolaceIO.Read: Exception when querying for backlog bytes.", e); } - - private void createQueue(String queueName) throws SolaceSempException, IOException { - LOG.info("SolaceIO.Read: Creating new queue {}.", queueName); - BrokerResponse response = sempBasicAuthClientExecutor.createQueueResponse(queueName); - checkIfError(response); + } + + private void createQueue(String queueName) throws SolaceSempException, IOException { + LOG.info("SolaceIO.Read: Creating new queue {}.", queueName); + BrokerResponse response = sempBasicAuthClientExecutor.createQueueResponse(queueName); + checkIfError(response); + } + + private void createSubscription(String queueName, String topicName) + throws SolaceSempException, IOException { + LOG.info("SolaceIO.Read: Creating new subscription {} for topic {}.", queueName, topicName); + BrokerResponse response = + sempBasicAuthClientExecutor.createSubscriptionResponse(queueName, topicName); + checkIfError(response); + } + + private T getSuccessResponseOrThrowException( + BrokerResponse response, Class mapSuccessToClass) throws IOException, SolaceSempException { + checkIfError(response); + return objectMapper.readValue(response.content, mapSuccessToClass); + } + + private void checkIfError(BrokerResponse response) + throws SolaceSempException, IOException { // todo do we still need this? + if (response.code < 200 || response.code > 299) { + ErrorMessage error = objectMapper.readValue(response.content, ErrorMessage.class); + throw new SolaceSempException(error); } + } - private void createSubscription(String queueName, String topicName) - throws SolaceSempException, IOException { - LOG.info("SolaceIO.Read: Creating new subscription {} for topic {}.", queueName, topicName); - BrokerResponse response = - sempBasicAuthClientExecutor.createSubscriptionResponse(queueName, topicName); - checkIfError(response); - } + static class SolaceSempException extends Exception { + public final ErrorMessage errorMessage; - private T getSuccessResponseOrThrowException( - BrokerResponse response, Class mapSuccessToClass) - throws IOException, SolaceSempException { - checkIfError(response); - return objectMapper.readValue(response.content, mapSuccessToClass); + SolaceSempException(ErrorMessage errorMessage) { + super(errorMessage.meta().error().description()); + this.errorMessage = errorMessage; } + } - private void checkIfError(BrokerResponse response) - throws SolaceSempException, IOException { // todo do we still need this? - if (response.code < 200 || response.code > 299) { - ErrorMessage error = objectMapper.readValue(response.content, ErrorMessage.class); - throw new SolaceSempException(error); - } - } + @AutoValue + @JsonSerialize(as = Queue.class) + @JsonDeserialize(builder = AutoValue_BasicAuthSempClient_Queue.Builder.class) + abstract static class Queue { - static class SolaceSempException extends Exception { - public final ErrorMessage errorMessage; + public abstract QueueData data(); - SolaceSempException(ErrorMessage errorMessage) { - super(errorMessage.meta().error().description()); - this.errorMessage = errorMessage; - } + public static Builder builder() { + return new AutoValue_BasicAuthSempClient_Queue.Builder(); } - @AutoValue - @JsonSerialize(as = Queue.class) - @JsonDeserialize(builder = AutoValue_BasicAuthSempClient_Queue.Builder.class) - abstract static class Queue { - - public abstract QueueData data(); + public abstract Builder toBuilder(); - public static Builder builder() { - return new AutoValue_BasicAuthSempClient_Queue.Builder(); - } + @AutoValue.Builder + @JsonPOJOBuilder(withPrefix = "set") + abstract static class Builder { - public abstract Builder toBuilder(); + public abstract Builder setData(QueueData queueData); - @AutoValue.Builder - @JsonPOJOBuilder(withPrefix = "set") - abstract static class Builder { - - public abstract Builder setData(QueueData queueData); - - public abstract Queue build(); - } + public abstract Queue build(); } + } - @AutoValue - @JsonDeserialize(builder = AutoValue_BasicAuthSempClient_QueueData.Builder.class) - abstract static class QueueData { - public abstract String accessType(); + @AutoValue + @JsonDeserialize(builder = AutoValue_BasicAuthSempClient_QueueData.Builder.class) + abstract static class QueueData { + public abstract String accessType(); - public abstract long msgSpoolUsage(); + public abstract long msgSpoolUsage(); - public static Builder builder() { - return new AutoValue_BasicAuthSempClient_QueueData.Builder(); - } + public static Builder builder() { + return new AutoValue_BasicAuthSempClient_QueueData.Builder(); + } - public abstract Builder toBuilder(); + public abstract Builder toBuilder(); - @AutoValue.Builder - @JsonPOJOBuilder(withPrefix = "set") - abstract static class Builder { + @AutoValue.Builder + @JsonPOJOBuilder(withPrefix = "set") + abstract static class Builder { - public abstract Builder setAccessType(String accessType); + public abstract Builder setAccessType(String accessType); - public abstract Builder setMsgSpoolUsage(long msgSpoolUsage); + public abstract Builder setMsgSpoolUsage(long msgSpoolUsage); - public abstract QueueData build(); - } + public abstract QueueData build(); } + } - @AutoValue - @JsonSerialize(as = ErrorMessage.class) - @JsonDeserialize(builder = AutoValue_BasicAuthSempClient_ErrorMessage.Builder.class) - abstract static class ErrorMessage { + @AutoValue + @JsonSerialize(as = ErrorMessage.class) + @JsonDeserialize(builder = AutoValue_BasicAuthSempClient_ErrorMessage.Builder.class) + abstract static class ErrorMessage { - public abstract ErrorMessageMeta meta(); + public abstract ErrorMessageMeta meta(); - public static Builder builder() { - return new AutoValue_BasicAuthSempClient_ErrorMessage.Builder(); - } + public static Builder builder() { + return new AutoValue_BasicAuthSempClient_ErrorMessage.Builder(); + } - public abstract Builder toBuilder(); + public abstract Builder toBuilder(); - @AutoValue.Builder - @JsonPOJOBuilder(withPrefix = "set") - abstract static class Builder { + @AutoValue.Builder + @JsonPOJOBuilder(withPrefix = "set") + abstract static class Builder { - public abstract Builder setMeta(ErrorMessageMeta errorMessageMeta); + public abstract Builder setMeta(ErrorMessageMeta errorMessageMeta); - public abstract ErrorMessage build(); - } + public abstract ErrorMessage build(); + } - @AutoValue - @JsonSerialize(as = ErrorMessageMeta.class) - @JsonDeserialize( - builder = AutoValue_BasicAuthSempClient_ErrorMessage_ErrorMessageMeta.Builder.class) - abstract static class ErrorMessageMeta { - public abstract ErrorMessageDetails error(); + @AutoValue + @JsonSerialize(as = ErrorMessageMeta.class) + @JsonDeserialize( + builder = AutoValue_BasicAuthSempClient_ErrorMessage_ErrorMessageMeta.Builder.class) + abstract static class ErrorMessageMeta { + public abstract ErrorMessageDetails error(); - public static Builder builder() { - return new AutoValue_BasicAuthSempClient_ErrorMessage_ErrorMessageMeta.Builder(); - } + public static Builder builder() { + return new AutoValue_BasicAuthSempClient_ErrorMessage_ErrorMessageMeta.Builder(); + } - public abstract Builder toBuilder(); + public abstract Builder toBuilder(); - @AutoValue.Builder - @JsonPOJOBuilder(withPrefix = "set") - abstract static class Builder { + @AutoValue.Builder + @JsonPOJOBuilder(withPrefix = "set") + abstract static class Builder { - public abstract Builder setError(ErrorMessageDetails errorMessageDetails); + public abstract Builder setError(ErrorMessageDetails errorMessageDetails); - public abstract ErrorMessageMeta build(); - } - } + public abstract ErrorMessageMeta build(); + } + } - @AutoValue - @JsonSerialize(as = ErrorMessageDetails.class) - @JsonDeserialize( - builder = - AutoValue_BasicAuthSempClient_ErrorMessage_ErrorMessageDetails.Builder - .class) - abstract static class ErrorMessageDetails { - public abstract Integer code(); + @AutoValue + @JsonSerialize(as = ErrorMessageDetails.class) + @JsonDeserialize( + builder = AutoValue_BasicAuthSempClient_ErrorMessage_ErrorMessageDetails.Builder.class) + abstract static class ErrorMessageDetails { + public abstract Integer code(); - public abstract String description(); + public abstract String description(); - public abstract String status(); + public abstract String status(); - public static Builder builder() { - return new AutoValue_BasicAuthSempClient_ErrorMessage_ErrorMessageDetails.Builder(); - } + public static Builder builder() { + return new AutoValue_BasicAuthSempClient_ErrorMessage_ErrorMessageDetails.Builder(); + } - public abstract Builder toBuilder(); + public abstract Builder toBuilder(); - @AutoValue.Builder - @JsonPOJOBuilder(withPrefix = "set") - abstract static class Builder { + @AutoValue.Builder + @JsonPOJOBuilder(withPrefix = "set") + abstract static class Builder { - public abstract Builder setCode(Integer code); + public abstract Builder setCode(Integer code); - public abstract Builder setDescription(String description); + public abstract Builder setDescription(String description); - public abstract Builder setStatus(String status); + public abstract Builder setStatus(String status); - public abstract ErrorMessageDetails build(); - } - } + public abstract ErrorMessageDetails build(); + } } + } } diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/BasicAuthSempClientFactory.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/BasicAuthSempClientFactory.java index fdf5c6a04ca3..d02200065864 100644 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/BasicAuthSempClientFactory.java +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/BasicAuthSempClientFactory.java @@ -1,11 +1,13 @@ /* - * Copyright 2024 Google. + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -13,111 +15,109 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.google.cloud.dataflow.dce.io.solace.broker; +package org.apache.beam.sdk.io.solace.broker; import com.google.api.client.http.HttpRequestFactory; import com.google.api.client.http.javanet.NetHttpTransport; -import com.google.cloud.dataflow.dce.io.solace.SerializableSupplier; -import com.google.common.base.Preconditions; -import org.apache.arrow.util.VisibleForTesting; +import org.apache.beam.sdk.io.solace.SerializableSupplier; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; public class BasicAuthSempClientFactory implements SempClientFactory { - private final String host; - private final String username; - private final String password; - private final String vpnName; - private final SerializableSupplier httpRequestFactorySupplier; + private final String host; + private final String username; + private final String password; + private final String vpnName; + private final SerializableSupplier httpRequestFactorySupplier; - private BasicAuthSempClientFactory( - String host, - String username, - String password, - String vpnName, - SerializableSupplier httpRequestFactorySupplier) { - this.host = host; - this.username = username; - this.password = password; - this.vpnName = vpnName; - this.httpRequestFactorySupplier = httpRequestFactorySupplier; - } + private BasicAuthSempClientFactory( + String host, + String username, + String password, + String vpnName, + SerializableSupplier httpRequestFactorySupplier) { + this.host = host; + this.username = username; + this.password = password; + this.vpnName = vpnName; + this.httpRequestFactorySupplier = httpRequestFactorySupplier; + } - public static BasicAuthSempAuthenticationFactoryBuilder builder() { - return new BasicAuthSempAuthenticationFactoryBuilder() - .withHttpRequestFactorySupplier( - () -> new NetHttpTransport().createRequestFactory()); - } + public static BasicAuthSempAuthenticationFactoryBuilder builder() { + return new BasicAuthSempAuthenticationFactoryBuilder() + .withHttpRequestFactorySupplier(() -> new NetHttpTransport().createRequestFactory()); + } - @Override - public SempClient create() { - return new BasicAuthSempClient( - host, username, password, vpnName, httpRequestFactorySupplier); - } + @Override + public SempClient create() { + return new BasicAuthSempClient(host, username, password, vpnName, httpRequestFactorySupplier); + } - public static class BasicAuthSempAuthenticationFactoryBuilder { + public static class BasicAuthSempAuthenticationFactoryBuilder { - private String host; - private String username; - private String password; - private String vpnName; - private SerializableSupplier httpRequestFactorySupplier; + private String host; + private String username; + private String password; + private String vpnName; + private SerializableSupplier httpRequestFactorySupplier; - /** Set Solace host, format: Protocol://Host[:Port] */ - public BasicAuthSempAuthenticationFactoryBuilder withHost(String host) { - this.host = host; - return this; - } + /** Set Solace host, format: Protocol://Host[:Port] */ + public BasicAuthSempAuthenticationFactoryBuilder withHost(String host) { + this.host = host; + return this; + } - /** Set Solace username */ - public BasicAuthSempAuthenticationFactoryBuilder withUsername(String username) { - this.username = username; - return this; - } + /** Set Solace username */ + public BasicAuthSempAuthenticationFactoryBuilder withUsername(String username) { + this.username = username; + return this; + } - /** Set Solace password */ - public BasicAuthSempAuthenticationFactoryBuilder withPassword(String password) { - this.password = password; - return this; - } + /** Set Solace password */ + public BasicAuthSempAuthenticationFactoryBuilder withPassword(String password) { + this.password = password; + return this; + } - /** Set Solace vpn name */ - public BasicAuthSempAuthenticationFactoryBuilder withVpnName(String vpnName) { - this.vpnName = vpnName; - return this; - } + /** Set Solace vpn name */ + public BasicAuthSempAuthenticationFactoryBuilder withVpnName(String vpnName) { + this.vpnName = vpnName; + return this; + } - @VisibleForTesting - BasicAuthSempAuthenticationFactoryBuilder withHttpRequestFactorySupplier( - SerializableSupplier httpRequestFactorySupplier) { - this.httpRequestFactorySupplier = httpRequestFactorySupplier; - return this; - } + @VisibleForTesting + BasicAuthSempAuthenticationFactoryBuilder withHttpRequestFactorySupplier( + SerializableSupplier httpRequestFactorySupplier) { + this.httpRequestFactorySupplier = httpRequestFactorySupplier; + return this; + } - public BasicAuthSempClientFactory build() { - // todo update name in the error string - Preconditions.checkState( - host != null, - "SolaceIO: host in BasicAuthSempAuthenticationFactory can't be null. Set it" - + " with `withHost()` method."); - Preconditions.checkState( - username != null, - "SolaceIO: username in BasicAuthSempAuthenticationFactory can't be null. Set it" - + " with `withUsername()` method."); - Preconditions.checkState( - password != null, - "SolaceIO: password in BasicAuthSempAuthenticationFactory can't be null. Set it" - + " with `withPassword()` method."); - Preconditions.checkState( - vpnName != null, - "SolaceIO: vpnName in BasicAuthSempAuthenticationFactory can't be null. Set it" - + " with `withVpnName()` method."); - Preconditions.checkState( - httpRequestFactorySupplier != null, - "SolaceIO: httpRequestFactorySupplier in BasicAuthSempAuthenticationFactory" - + " can't be null. Set it with `withHttpRequestFactorySupplier()` method."); + public BasicAuthSempClientFactory build() { + // todo update name in the error string + Preconditions.checkState( + host != null, + "SolaceIO: host in BasicAuthSempAuthenticationFactory can't be null. Set it" + + " with `withHost()` method."); + Preconditions.checkState( + username != null, + "SolaceIO: username in BasicAuthSempAuthenticationFactory can't be null. Set it" + + " with `withUsername()` method."); + Preconditions.checkState( + password != null, + "SolaceIO: password in BasicAuthSempAuthenticationFactory can't be null. Set it" + + " with `withPassword()` method."); + Preconditions.checkState( + vpnName != null, + "SolaceIO: vpnName in BasicAuthSempAuthenticationFactory can't be null. Set it" + + " with `withVpnName()` method."); + Preconditions.checkState( + httpRequestFactorySupplier != null, + "SolaceIO: httpRequestFactorySupplier in BasicAuthSempAuthenticationFactory" + + " can't be null. Set it with `withHttpRequestFactorySupplier()` method."); - return new BasicAuthSempClientFactory( - host, username, password, vpnName, httpRequestFactorySupplier); - } + return new BasicAuthSempClientFactory( + host, username, password, vpnName, httpRequestFactorySupplier); } + } } diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/BrokerResponse.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/BrokerResponse.java index 26954993650d..1a47f8012285 100644 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/BrokerResponse.java +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/BrokerResponse.java @@ -1,11 +1,13 @@ /* - * Copyright 2024 Google. + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -13,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.google.cloud.dataflow.dce.io.solace.broker; +package org.apache.beam.sdk.io.solace.broker; import com.google.api.client.http.HttpResponse; import java.io.BufferedReader; @@ -25,39 +27,36 @@ import org.checkerframework.checker.nullness.qual.Nullable; public class BrokerResponse { - final int code; - final String message; - @Nullable String content; + final int code; + final String message; + @Nullable String content; - public BrokerResponse(int responseCode, String message, @Nullable InputStream content) - throws IOException { - this.code = responseCode; - this.message = message; - if (content != null) { - try (content) { - this.content = - new BufferedReader(new InputStreamReader(content, StandardCharsets.UTF_8)) - .lines() - .collect(Collectors.joining("\n")); - } - } + public BrokerResponse(int responseCode, String message, @Nullable InputStream content) { + this.code = responseCode; + this.message = message; + if (content != null) { + this.content = + new BufferedReader(new InputStreamReader(content, StandardCharsets.UTF_8)) + .lines() + .collect(Collectors.joining("\n")); } + } - public static BrokerResponse fromHttpResponse(HttpResponse response) throws IOException { - return new BrokerResponse( - response.getStatusCode(), response.getStatusMessage(), response.getContent()); - } + public static BrokerResponse fromHttpResponse(HttpResponse response) throws IOException { + return new BrokerResponse( + response.getStatusCode(), response.getStatusMessage(), response.getContent()); + } - @Override - public String toString() { - return "BrokerResponse{" - + "code=" - + code - + ", message='" - + message - + '\'' - + ", content=" - + content - + '}'; - } + @Override + public String toString() { + return "BrokerResponse{" + + "code=" + + code + + ", message='" + + message + + '\'' + + ", content=" + + content + + '}'; + } } diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/MessageReceiver.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/MessageReceiver.java index 82ff078d515b..baf9e3beb038 100644 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/MessageReceiver.java +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/MessageReceiver.java @@ -1,11 +1,13 @@ /* - * Copyright 2024 Google. + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -13,23 +15,23 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.google.cloud.dataflow.dce.io.solace.broker; +package org.apache.beam.sdk.io.solace.broker; import com.solacesystems.jcsmp.BytesXMLMessage; import java.io.IOException; public interface MessageReceiver { - void start(); + void start(); - boolean isClosed(); + boolean isClosed(); - BytesXMLMessage receive() throws IOException; + BytesXMLMessage receive() throws IOException; - /** - * Test clients may return {@literal true} to signal that all expected messages have been pulled - * and the test may complete. Real clients will return {@literal false}. - */ - default boolean isEOF() { - return false; - } + /** + * Test clients may return {@literal true} to signal that all expected messages have been pulled + * and the test may complete. Real clients will return {@literal false}. + */ + default boolean isEOF() { + return false; + } } diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SempBasicAuthClientExecutor.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SempBasicAuthClientExecutor.java index 36a64dda948c..c1d22bf3419b 100644 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SempBasicAuthClientExecutor.java +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SempBasicAuthClientExecutor.java @@ -1,11 +1,13 @@ /* - * Copyright 2024 Google. + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -13,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.google.cloud.dataflow.dce.io.solace.broker; +package org.apache.beam.sdk.io.solace.broker; import com.google.api.client.http.GenericUrl; import com.google.api.client.http.HttpContent; @@ -24,13 +26,13 @@ import com.google.api.client.http.HttpResponseException; import com.google.api.client.http.json.JsonHttpContent; import com.google.api.client.json.gson.GsonFactory; -import com.google.common.collect.ImmutableMap; import java.io.IOException; import java.io.Serializable; import java.net.CookieManager; import java.net.HttpCookie; import java.util.List; import java.util.stream.Collectors; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; /** * A class to execute requests to SEMP v2 with Basic Auth authentication. @@ -42,129 +44,128 @@ * header to refresh the token. */ class SempBasicAuthClientExecutor implements Serializable { - private static final CookieManager COOKIE_MANAGER = new CookieManager(); - private static final String COOKIES_HEADER = "Set-Cookie"; - - private final String username; - private final String messageVpn; - private final String baseUrl; - private final String password; - private final HttpRequestFactory requestFactory; - - SempBasicAuthClientExecutor( - String host, - String username, - String password, - String vpnName, - HttpRequestFactory httpRequestFactory) { - this.baseUrl = String.format("%s/SEMP/v2", host); - this.username = username; - this.messageVpn = vpnName; - this.password = password; - this.requestFactory = httpRequestFactory; - } - - private static String getQueueEndpoint(String messageVpn, String queueName) { - return String.format("/monitor/msgVpns/%s/queues/%s", messageVpn, queueName); - } - - private static String createQueueEndpoint(String messageVpn) { - return String.format("/config/msgVpns/%s/queues", messageVpn); - } - - private static String subscriptionEndpoint(String messageVpn, String queueName) { - return String.format("/config/msgVpns/%s/queues/%s/subscriptions", messageVpn, queueName); - } - - BrokerResponse getQueueResponse(String queueName) throws IOException { - String queryUrl = getQueueEndpoint(messageVpn, queueName); - HttpResponse response = executeGet(new GenericUrl(baseUrl + queryUrl)); - return BrokerResponse.fromHttpResponse(response); - } - - BrokerResponse createQueueResponse(String queueName) throws IOException { - String queryUrl = createQueueEndpoint(messageVpn); - ImmutableMap params = - ImmutableMap.builder() - .put("accessType", "non-exclusive") - .put("queueName", queueName) - .put("owner", username) - .put("permission", "consume") - .put("ingressEnabled", true) - .put("egressEnabled", true) - .build(); - - HttpResponse response = executePost(new GenericUrl(baseUrl + queryUrl), params); - return BrokerResponse.fromHttpResponse(response); - } - - BrokerResponse createSubscriptionResponse(String queueName, String topicName) - throws IOException { - String queryUrl = subscriptionEndpoint(messageVpn, queueName); - - ImmutableMap params = - ImmutableMap.builder() - .put("subscriptionTopic", topicName) - .put("queueName", queueName) - .build(); - HttpResponse response = executePost(new GenericUrl(baseUrl + queryUrl), params); - return BrokerResponse.fromHttpResponse(response); - } - - private HttpResponse executeGet(GenericUrl url) throws IOException { - HttpRequest request = requestFactory.buildGetRequest(url); - return execute(request); + private static final CookieManager COOKIE_MANAGER = new CookieManager(); + private static final String COOKIES_HEADER = "Set-Cookie"; + + private final String username; + private final String messageVpn; + private final String baseUrl; + private final String password; + private final HttpRequestFactory requestFactory; + + SempBasicAuthClientExecutor( + String host, + String username, + String password, + String vpnName, + HttpRequestFactory httpRequestFactory) { + this.baseUrl = String.format("%s/SEMP/v2", host); + this.username = username; + this.messageVpn = vpnName; + this.password = password; + this.requestFactory = httpRequestFactory; + } + + private static String getQueueEndpoint(String messageVpn, String queueName) { + return String.format("/monitor/msgVpns/%s/queues/%s", messageVpn, queueName); + } + + private static String createQueueEndpoint(String messageVpn) { + return String.format("/config/msgVpns/%s/queues", messageVpn); + } + + private static String subscriptionEndpoint(String messageVpn, String queueName) { + return String.format("/config/msgVpns/%s/queues/%s/subscriptions", messageVpn, queueName); + } + + BrokerResponse getQueueResponse(String queueName) throws IOException { + String queryUrl = getQueueEndpoint(messageVpn, queueName); + HttpResponse response = executeGet(new GenericUrl(baseUrl + queryUrl)); + return BrokerResponse.fromHttpResponse(response); + } + + BrokerResponse createQueueResponse(String queueName) throws IOException { + String queryUrl = createQueueEndpoint(messageVpn); + ImmutableMap params = + ImmutableMap.builder() + .put("accessType", "non-exclusive") + .put("queueName", queueName) + .put("owner", username) + .put("permission", "consume") + .put("ingressEnabled", true) + .put("egressEnabled", true) + .build(); + + HttpResponse response = executePost(new GenericUrl(baseUrl + queryUrl), params); + return BrokerResponse.fromHttpResponse(response); + } + + BrokerResponse createSubscriptionResponse(String queueName, String topicName) throws IOException { + String queryUrl = subscriptionEndpoint(messageVpn, queueName); + + ImmutableMap params = + ImmutableMap.builder() + .put("subscriptionTopic", topicName) + .put("queueName", queueName) + .build(); + HttpResponse response = executePost(new GenericUrl(baseUrl + queryUrl), params); + return BrokerResponse.fromHttpResponse(response); + } + + private HttpResponse executeGet(GenericUrl url) throws IOException { + HttpRequest request = requestFactory.buildGetRequest(url); + return execute(request); + } + + private HttpResponse executePost(GenericUrl url, ImmutableMap parameters) + throws IOException { + HttpContent content = new JsonHttpContent(GsonFactory.getDefaultInstance(), parameters); + HttpRequest request = requestFactory.buildPostRequest(url, content); + return execute(request); + } + + private HttpResponse execute(HttpRequest request) throws IOException { + request.setNumberOfRetries(2); + HttpHeaders httpHeaders = new HttpHeaders(); + boolean authFromCookie = COOKIE_MANAGER.getCookieStore().getCookies().size() > 0; + if (authFromCookie) { + setCookiesFromCookieManager(httpHeaders); + request.setHeaders(httpHeaders); + } else { + httpHeaders.setBasicAuthentication(username, password); + request.setHeaders(httpHeaders); } - private HttpResponse executePost(GenericUrl url, ImmutableMap parameters) - throws IOException { - HttpContent content = new JsonHttpContent(GsonFactory.getDefaultInstance(), parameters); - HttpRequest request = requestFactory.buildPostRequest(url, content); + HttpResponse response; + try { + response = request.execute(); + } catch (HttpResponseException e) { + if (authFromCookie && e.getStatusCode() == 401) { + COOKIE_MANAGER.getCookieStore().removeAll(); + // execute again without cookies to refresh the token. return execute(request); + } else { + throw e; + } } - private HttpResponse execute(HttpRequest request) throws IOException { - request.setNumberOfRetries(2); - HttpHeaders httpHeaders = new HttpHeaders(); - boolean authFromCookie = COOKIE_MANAGER.getCookieStore().getCookies().size() > 0; - if (authFromCookie) { - setCookiesFromCookieManager(httpHeaders); - request.setHeaders(httpHeaders); - } else { - httpHeaders.setBasicAuthentication(username, password); - request.setHeaders(httpHeaders); - } - - HttpResponse response; - try { - response = request.execute(); - } catch (HttpResponseException e) { - if (authFromCookie && e.getStatusCode() == 401) { - COOKIE_MANAGER.getCookieStore().removeAll(); - // execute again without cookies to refresh the token. - return execute(request); - } else { - throw e; - } - } - - storeCookiesInCookieManager(response.getHeaders()); - return response; - } - - private void setCookiesFromCookieManager(HttpHeaders httpHeaders) { - httpHeaders.setCookie( - COOKIE_MANAGER.getCookieStore().getCookies().stream() - .map(s -> s.getName() + "=" + s.getValue()) - .collect(Collectors.joining(";"))); - } - - private void storeCookiesInCookieManager(HttpHeaders headers) { - List cookiesHeader = headers.getHeaderStringValues(COOKIES_HEADER); - if (cookiesHeader != null) { - for (String cookie : cookiesHeader) { - COOKIE_MANAGER.getCookieStore().add(null, HttpCookie.parse(cookie).get(0)); - } - } + storeCookiesInCookieManager(response.getHeaders()); + return response; + } + + private void setCookiesFromCookieManager(HttpHeaders httpHeaders) { + httpHeaders.setCookie( + COOKIE_MANAGER.getCookieStore().getCookies().stream() + .map(s -> s.getName() + "=" + s.getValue()) + .collect(Collectors.joining(";"))); + } + + private void storeCookiesInCookieManager(HttpHeaders headers) { + List cookiesHeader = headers.getHeaderStringValues(COOKIES_HEADER); + if (cookiesHeader != null) { + for (String cookie : cookiesHeader) { + COOKIE_MANAGER.getCookieStore().add(null, HttpCookie.parse(cookie).get(0)); + } } + } } diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SempClient.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SempClient.java index dfa65fe5bf38..571010316ee8 100644 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SempClient.java +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SempClient.java @@ -1,11 +1,13 @@ /* - * Copyright 2024 Google. + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -13,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.google.cloud.dataflow.dce.io.solace.broker; +package org.apache.beam.sdk.io.solace.broker; import com.solacesystems.jcsmp.Queue; import java.io.IOException; @@ -21,9 +23,9 @@ public interface SempClient extends Serializable { - boolean isQueueNonExclusive(String queueName) throws IOException; + boolean isQueueNonExclusive(String queueName) throws IOException; - Queue createQueueForTopic(String queueName, String topicName) throws IOException; + Queue createQueueForTopic(String queueName, String topicName) throws IOException; - long getBacklogBytes(String queueName) throws IOException; + long getBacklogBytes(String queueName) throws IOException; } diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SempClientFactory.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SempClientFactory.java index 6d3e4baec40d..3967ed1ab884 100644 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SempClientFactory.java +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SempClientFactory.java @@ -1,11 +1,13 @@ /* - * Copyright 2024 Google. + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -13,10 +15,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.google.cloud.dataflow.dce.io.solace.broker; +package org.apache.beam.sdk.io.solace.broker; import java.io.Serializable; public interface SempClientFactory extends Serializable { - SempClient create(); + SempClient create(); } diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SessionService.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SessionService.java index 5766469ff8b9..af276c2f3af6 100644 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SessionService.java +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SessionService.java @@ -1,11 +1,13 @@ /* - * Copyright 2024 Google. + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -13,16 +15,16 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.google.cloud.dataflow.dce.io.solace.broker; +package org.apache.beam.sdk.io.solace.broker; import java.io.Serializable; public interface SessionService extends Serializable { - void connect(); + void connect(); - void close(); + void close(); - boolean isClosed(); + boolean isClosed(); - MessageReceiver createReceiver(); + MessageReceiver createReceiver(); } diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SessionServiceFactory.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SessionServiceFactory.java index d5ab4f32aec7..5e31926fa0fb 100644 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SessionServiceFactory.java +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SessionServiceFactory.java @@ -1,11 +1,13 @@ /* - * Copyright 2024 Google. + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -13,17 +15,17 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.google.cloud.dataflow.dce.io.solace.broker; +package org.apache.beam.sdk.io.solace.broker; import com.solacesystems.jcsmp.Queue; import java.io.Serializable; public abstract class SessionServiceFactory implements Serializable { - Queue queue; + Queue queue; - public abstract SessionService create(); + public abstract SessionService create(); - public void setQueue(Queue queue) { - this.queue = queue; - } + public void setQueue(Queue queue) { + this.queue = queue; + } } diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SolaceMessageReceiver.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SolaceMessageReceiver.java index 784d3223c832..d80999c9926c 100644 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SolaceMessageReceiver.java +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SolaceMessageReceiver.java @@ -1,11 +1,13 @@ /* - * Copyright 2024 Google. + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -13,47 +15,47 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.google.cloud.dataflow.dce.io.solace.broker; +package org.apache.beam.sdk.io.solace.broker; -import com.google.cloud.dataflow.dce.io.solace.RetryCallableManager; import com.solacesystems.jcsmp.BytesXMLMessage; import com.solacesystems.jcsmp.FlowReceiver; import com.solacesystems.jcsmp.JCSMPException; import java.io.IOException; import java.io.Serializable; -import java.util.Set; +import org.apache.beam.sdk.io.solace.RetryCallableManager; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet; public class SolaceMessageReceiver implements MessageReceiver, Serializable { - public static final int DEFAULT_ADVANCE_TIMEOUT_IN_MILLIS = 100; - private final FlowReceiver flowReceiver; - private final RetryCallableManager retryCallableManager = RetryCallableManager.create(); + public static final int DEFAULT_ADVANCE_TIMEOUT_IN_MILLIS = 100; + private final FlowReceiver flowReceiver; + private final RetryCallableManager retryCallableManager = RetryCallableManager.create(); - public SolaceMessageReceiver(FlowReceiver flowReceiver) { - this.flowReceiver = flowReceiver; - } + public SolaceMessageReceiver(FlowReceiver flowReceiver) { + this.flowReceiver = flowReceiver; + } - @Override - public void start() { - retryCallableManager.retryCallable( - () -> { - flowReceiver.start(); - return 0; - }, - Set.of(JCSMPException.class)); - } + @Override + public void start() { + retryCallableManager.retryCallable( + () -> { + flowReceiver.start(); + return 0; + }, + ImmutableSet.of(JCSMPException.class)); + } - @Override - public boolean isClosed() { - return flowReceiver == null || flowReceiver.isClosed(); - } + @Override + public boolean isClosed() { + return flowReceiver == null || flowReceiver.isClosed(); + } - @Override - public BytesXMLMessage receive() throws IOException { - try { - return flowReceiver.receive(DEFAULT_ADVANCE_TIMEOUT_IN_MILLIS); - } catch (JCSMPException e) { - throw new IOException(e); - } + @Override + public BytesXMLMessage receive() throws IOException { + try { + return flowReceiver.receive(DEFAULT_ADVANCE_TIMEOUT_IN_MILLIS); + } catch (JCSMPException e) { + throw new IOException(e); } + } } diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/data/Solace.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/data/Solace.java index 7b3dae856a24..66fc363eed1a 100644 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/data/Solace.java +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/data/Solace.java @@ -1,11 +1,13 @@ /* - * Copyright 2023 Google. + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -13,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.google.cloud.dataflow.dce.io.solace.data; +package org.apache.beam.sdk.io.solace.data; import com.google.auto.value.AutoValue; import com.solacesystems.jcsmp.BytesXMLMessage; @@ -30,8 +32,8 @@ /** * A record to be written to a Solace topic. * - *

You need to transform to {@link Solace.Record} to be able to write to Solace. For that, you - * can use the {@link Solace.Record.Builder} provided with this class. + *

You need to transform to {@link Record} to be able to write to Solace. For that, you can use + * the {@link Record.Builder} provided with this class. * *

For instance, to create a record, use the following code: * @@ -47,294 +49,290 @@ */ public class Solace { - public static class Queue { - private final String name; - - private Queue(String name) { - this.name = name; - } + public static class Queue { + private final String name; - public static Queue fromName(String name) { - return new Queue(name); - } + private Queue(String name) { + this.name = name; + } - public String getName() { - return name; - } + public static Queue fromName(String name) { + return new Queue(name); } - public static class Topic { - private final String name; + public String getName() { + return name; + } + } - private Topic(String name) { - this.name = name; - } + public static class Topic { + private final String name; - public static Topic fromName(String name) { - return new Topic(name); - } + private Topic(String name) { + this.name = name; + } - public String getName() { - return name; - } + public static Topic fromName(String name) { + return new Topic(name); } - public enum DestinationType { - TOPIC, - QUEUE + public String getName() { + return name; } + } - @AutoValue - @DefaultSchema(AutoValueSchema.class) - public abstract static class Destination { - @SchemaFieldNumber("0") - public abstract String getName(); + public enum DestinationType { + TOPIC, + QUEUE + } - @SchemaFieldNumber("1") - public abstract DestinationType getType(); + @AutoValue + @DefaultSchema(AutoValueSchema.class) + public abstract static class Destination { + @SchemaFieldNumber("0") + public abstract String getName(); - public static Builder builder() { - return new AutoValue_Solace_Destination.Builder(); - } + @SchemaFieldNumber("1") + public abstract DestinationType getType(); + + public static Builder builder() { + return new AutoValue_Solace_Destination.Builder(); + } - @AutoValue.Builder - public abstract static class Builder { - public abstract Builder setName(String name); + @AutoValue.Builder + public abstract static class Builder { + public abstract Builder setName(String name); - public abstract Builder setType(DestinationType type); + public abstract Builder setType(DestinationType type); - public abstract Destination build(); - } + public abstract Destination build(); } + } - @AutoValue - @DefaultSchema(AutoValueSchema.class) - public abstract static class Record implements Serializable { - @SchemaFieldNumber("0") - public abstract @Nullable String getMessageId(); + @AutoValue + @DefaultSchema(AutoValueSchema.class) + public abstract static class Record implements Serializable { + @SchemaFieldNumber("0") + public abstract @Nullable String getMessageId(); - @SuppressWarnings("mutable") - @SchemaFieldNumber("1") - public abstract byte[] getPayload(); + @SuppressWarnings("mutable") + @SchemaFieldNumber("1") + public abstract byte[] getPayload(); - @SchemaFieldNumber("2") - public abstract @Nullable Destination getDestination(); + @SchemaFieldNumber("2") + public abstract @Nullable Destination getDestination(); - @SchemaFieldNumber("3") - public abstract @Nullable Long getExpiration(); + @SchemaFieldNumber("3") + public abstract @Nullable Long getExpiration(); - @SchemaFieldNumber("4") - public abstract @Nullable Integer getPriority(); + @SchemaFieldNumber("4") + public abstract @Nullable Integer getPriority(); - @SchemaFieldNumber("5") - public abstract @Nullable Boolean getRedelivered(); + @SchemaFieldNumber("5") + public abstract @Nullable Boolean getRedelivered(); - @SchemaFieldNumber("6") - public abstract @Nullable String getReplyTo(); + @SchemaFieldNumber("6") + public abstract @Nullable String getReplyTo(); - @SchemaFieldNumber("7") - public abstract @Nullable Long getReceiveTimestamp(); + @SchemaFieldNumber("7") + public abstract @Nullable Long getReceiveTimestamp(); - @SchemaFieldNumber("8") - public abstract @Nullable Long getSenderTimestamp(); + @SchemaFieldNumber("8") + public abstract @Nullable Long getSenderTimestamp(); - @SchemaFieldNumber("9") - public abstract @Nullable Long getSequenceNumber(); + @SchemaFieldNumber("9") + public abstract @Nullable Long getSequenceNumber(); - @SchemaFieldNumber("10") - public abstract @Nullable Long getTimeToLive(); + @SchemaFieldNumber("10") + public abstract @Nullable Long getTimeToLive(); - /** - * The ID for a particular message is only guaranteed to be the same for a particular copy - * of a message on a particular queue or topic endpoint within a replication group. The same - * message on different queues or topic endpoints within the same replication group may or - * may not have the same replication group message ID. See more at https://docs.solace.com/API/API-Developer-Guide/Detecting-Duplicate-Mess.htm - */ - @SchemaFieldNumber("11") - public abstract @Nullable String getReplicationGroupMessageId(); + /** + * The ID for a particular message is only guaranteed to be the same for a particular copy of a + * message on a particular queue or topic endpoint within a replication group. The same message + * on different queues or topic endpoints within the same replication group may or may not have + * the same replication group message ID. See more at https://docs.solace.com/API/API-Developer-Guide/Detecting-Duplicate-Mess.htm + */ + @SchemaFieldNumber("11") + public abstract @Nullable String getReplicationGroupMessageId(); - public static Builder builder() { - return new AutoValue_Solace_Record.Builder(); - } + public static Builder builder() { + return new AutoValue_Solace_Record.Builder(); + } - @AutoValue.Builder - public abstract static class Builder { - public abstract Builder setMessageId(String messageId); + @AutoValue.Builder + public abstract static class Builder { + public abstract Builder setMessageId(String messageId); - public abstract Builder setPayload(byte[] payload); + public abstract Builder setPayload(byte[] payload); - public abstract Builder setDestination(Destination destination); + public abstract Builder setDestination(Destination destination); - public abstract Builder setExpiration(Long expiration); + public abstract Builder setExpiration(Long expiration); - public abstract Builder setPriority(Integer priority); + public abstract Builder setPriority(Integer priority); - public abstract Builder setRedelivered(Boolean redelivered); + public abstract Builder setRedelivered(Boolean redelivered); - public abstract Builder setReplyTo(String replyTo); + public abstract Builder setReplyTo(String replyTo); - public abstract Builder setReceiveTimestamp(Long receiveTimestamp); + public abstract Builder setReceiveTimestamp(Long receiveTimestamp); - public abstract Builder setSenderTimestamp(Long senderTimestamp); + public abstract Builder setSenderTimestamp(Long senderTimestamp); - public abstract Builder setSequenceNumber(Long sequenceNumber); + public abstract Builder setSequenceNumber(Long sequenceNumber); - public abstract Builder setTimeToLive(Long timeToLive); + public abstract Builder setTimeToLive(Long timeToLive); - public abstract Builder setReplicationGroupMessageId(String replicationGroupMessageId); + public abstract Builder setReplicationGroupMessageId(String replicationGroupMessageId); - public abstract Record build(); - } + public abstract Record build(); } + } + + /** + * The result of writing a message to Solace. This will be returned by the {@link + * org.apache.beam.sdk.io.solace.SolaceIO.Write} connector. + * + *

This class provides a builder to create instances, but you will probably not need it. The + * write connector will create and return instances of {@link PublishResult}. + * + *

If the message has been published, {@link PublishResult#getPublished()} will be true. If it + * is false, it means that the message could not be published, and {@link + * PublishResult#getError()} will contain more details about why the message could not be + * published. + */ + @AutoValue + @DefaultSchema(AutoValueSchema.class) + public abstract static class PublishResult { + /** The message id of the message that was published. */ + @SchemaFieldNumber("0") + public abstract String getMessageId(); + + /** Whether the message was published or not. */ + @SchemaFieldNumber("1") + public abstract Boolean getPublished(); /** - * The result of writing a message to Solace. This will be returned by the {@link - * com.google.cloud.dataflow.dce.io.solace.SolaceIO.Write} connector. - * - *

This class provides a builder to create instances, but you will probably not need it. The - * write connector will create and return instances of {@link Solace.PublishResult}. - * - *

If the message has been published, {@link Solace.PublishResult#getPublished()} will be - * true. If it is false, it means that the message could not be published, and {@link - * Solace.PublishResult#getError()} will contain more details about why the message could not be - * published. + * The publishing latency in milliseconds. This is the difference between the time the message + * was created, and the time the message was published. It is only available if the {@link + * CorrelationKey} class is used as correlation key of the messages. */ - @AutoValue - @DefaultSchema(AutoValueSchema.class) - public abstract static class PublishResult { - /** The message id of the message that was published. */ - @SchemaFieldNumber("0") - public abstract String getMessageId(); - - /** Whether the message was published or not. */ - @SchemaFieldNumber("1") - public abstract Boolean getPublished(); - - /** - * The publishing latency in milliseconds. This is the difference between the time the - * message was created, and the time the message was published. It is only available if the - * {@link CorrelationKey} class is used as correlation key of the messages. - */ - @SchemaFieldNumber("2") - public abstract @Nullable Long getLatencyMilliseconds(); - - /** The error details if the message could not be published. */ - @SchemaFieldNumber("3") - public abstract @Nullable String getError(); - - public static Builder builder() { - return new AutoValue_Solace_PublishResult.Builder(); - } + @SchemaFieldNumber("2") + public abstract @Nullable Long getLatencyMilliseconds(); - @AutoValue.Builder - public abstract static class Builder { - public abstract Builder setMessageId(String messageId); + /** The error details if the message could not be published. */ + @SchemaFieldNumber("3") + public abstract @Nullable String getError(); - public abstract Builder setPublished(Boolean published); - - public abstract Builder setLatencyMilliseconds(Long latencyMs); + public static Builder builder() { + return new AutoValue_Solace_PublishResult.Builder(); + } - public abstract Builder setError(String error); + @AutoValue.Builder + public abstract static class Builder { + public abstract Builder setMessageId(String messageId); - public abstract PublishResult build(); - } - } + public abstract Builder setPublished(Boolean published); - /** - * The correlation key is an object that is passed back to the client during the event broker - * ack or nack. - * - *

In the streaming writer is optionally used to calculate publish latencies, by calculating - * the time difference between the creation of the correlation key, and the time of the ack. - */ - @AutoValue - @DefaultSchema(AutoValueSchema.class) - public abstract static class CorrelationKey { - @SchemaFieldNumber("0") - public abstract String getMessageId(); + public abstract Builder setLatencyMilliseconds(Long latencyMs); - @SchemaFieldNumber("1") - public abstract long getPublishMonotonicMillis(); + public abstract Builder setError(String error); - public static Builder builder() { - return new AutoValue_Solace_CorrelationKey.Builder(); - } + public abstract PublishResult build(); + } + } + + /** + * The correlation key is an object that is passed back to the client during the event broker ack + * or nack. + * + *

In the streaming writer is optionally used to calculate publish latencies, by calculating + * the time difference between the creation of the correlation key, and the time of the ack. + */ + @AutoValue + @DefaultSchema(AutoValueSchema.class) + public abstract static class CorrelationKey { + @SchemaFieldNumber("0") + public abstract String getMessageId(); + + @SchemaFieldNumber("1") + public abstract long getPublishMonotonicMillis(); + + public static Builder builder() { + return new AutoValue_Solace_CorrelationKey.Builder(); + } - @AutoValue.Builder - public abstract static class Builder { - public abstract Builder setMessageId(String messageId); + @AutoValue.Builder + public abstract static class Builder { + public abstract Builder setMessageId(String messageId); - public abstract Builder setPublishMonotonicMillis(long millis); + public abstract Builder setPublishMonotonicMillis(long millis); - public abstract CorrelationKey build(); - } + public abstract CorrelationKey build(); } - - public static class SolaceRecordMapper { - public static final Logger LOG = LoggerFactory.getLogger(SolaceRecordMapper.class); - - public static Record map(@Nullable BytesXMLMessage msg) { - if (msg == null) { - return null; - } - - ByteArrayOutputStream outputStream = new ByteArrayOutputStream(); - if (msg.getContentLength() != 0) { - try { - outputStream.write(msg.getBytes()); - } catch (IOException e) { - LOG.error( - "Could not write Bytes from the BytesXMLMessage to the Solace.record.", - e); - } - } - if (msg.getAttachmentContentLength() != 0) { - try { - outputStream.write(msg.getAttachmentByteBuffer().array()); - } catch (IOException e) { - LOG.error( - "Could not AttachmentByteBuffer from the BytesXMLMessage to the" - + " Solace.record.", - e); - } - } - - String replyTo = (msg.getReplyTo() != null) ? msg.getReplyTo().getName() : null; - - com.solacesystems.jcsmp.Destination originalDestination = msg.getDestination(); - Destination.Builder destBuilder = - Destination.builder().setName(originalDestination.getName()); - if (originalDestination instanceof Topic) { - destBuilder.setType(DestinationType.TOPIC); - } else if (originalDestination instanceof Queue) { - destBuilder.setType(DestinationType.QUEUE); - } else { - LOG.error( - "SolaceIO: Unknown destination type for message {}, assuming that {} is a" - + " topic", - msg.getCorrelationId(), - originalDestination.getName()); - destBuilder.setType(DestinationType.TOPIC); - } - - return Record.builder() - .setDestination(destBuilder.build()) - .setExpiration(msg.getExpiration()) - .setMessageId(msg.getApplicationMessageId()) - .setPriority(msg.getPriority()) - .setRedelivered(msg.getRedelivered()) - .setReplyTo(replyTo) - .setReceiveTimestamp(msg.getReceiveTimestamp()) - .setSenderTimestamp(msg.getSenderTimestamp()) - .setSequenceNumber(msg.getSequenceNumber()) - .setTimeToLive(msg.getTimeToLive()) - .setReplicationGroupMessageId( - msg.getReplicationGroupMessageId() != null - ? msg.getReplicationGroupMessageId().toString() - : null) - .setPayload(outputStream.toByteArray()) - .build(); + } + + public static class SolaceRecordMapper { + public static final Logger LOG = LoggerFactory.getLogger(SolaceRecordMapper.class); + + public static Record map(@Nullable BytesXMLMessage msg) { + if (msg == null) { + return null; + } + + ByteArrayOutputStream outputStream = new ByteArrayOutputStream(); + if (msg.getContentLength() != 0) { + try { + outputStream.write(msg.getBytes()); + } catch (IOException e) { + LOG.error("Could not write Bytes from the BytesXMLMessage to the Solace.record.", e); + } + } + if (msg.getAttachmentContentLength() != 0) { + try { + outputStream.write(msg.getAttachmentByteBuffer().array()); + } catch (IOException e) { + LOG.error( + "Could not AttachmentByteBuffer from the BytesXMLMessage to the" + " Solace.record.", + e); } + } + + String replyTo = (msg.getReplyTo() != null) ? msg.getReplyTo().getName() : null; + + com.solacesystems.jcsmp.Destination originalDestination = msg.getDestination(); + Destination.Builder destBuilder = + Destination.builder().setName(originalDestination.getName()); + if (originalDestination instanceof Topic) { + destBuilder.setType(DestinationType.TOPIC); + } else if (originalDestination instanceof Queue) { + destBuilder.setType(DestinationType.QUEUE); + } else { + LOG.error( + "SolaceIO: Unknown destination type for message {}, assuming that {} is a" + " topic", + msg.getCorrelationId(), + originalDestination.getName()); + destBuilder.setType(DestinationType.TOPIC); + } + + return Record.builder() + .setDestination(destBuilder.build()) + .setExpiration(msg.getExpiration()) + .setMessageId(msg.getApplicationMessageId()) + .setPriority(msg.getPriority()) + .setRedelivered(msg.getRedelivered()) + .setReplyTo(replyTo) + .setReceiveTimestamp(msg.getReceiveTimestamp()) + .setSenderTimestamp(msg.getSenderTimestamp()) + .setSequenceNumber(msg.getSequenceNumber()) + .setTimeToLive(msg.getTimeToLive()) + .setReplicationGroupMessageId( + msg.getReplicationGroupMessageId() != null + ? msg.getReplicationGroupMessageId().toString() + : null) + .setPayload(outputStream.toByteArray()) + .build(); } + } } diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/data/SolaceRecordCoder.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/data/SolaceRecordCoder.java index bb6f851e72bd..f45ef1b48447 100644 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/data/SolaceRecordCoder.java +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/data/SolaceRecordCoder.java @@ -1,11 +1,13 @@ /* - * Copyright 2024 Google. + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -13,9 +15,8 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.google.cloud.dataflow.dce.io.solace.data; +package org.apache.beam.sdk.io.solace.data; -import com.google.cloud.dataflow.dce.io.solace.data.Solace.Record; import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; @@ -28,6 +29,7 @@ import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.coders.VarIntCoder; import org.apache.beam.sdk.coders.VarLongCoder; +import org.apache.beam.sdk.io.solace.data.Solace.Record; /** * Custom coder for the default Solace {@link Record} @@ -36,58 +38,57 @@ * `@DefaultSchema` annotation doesn't create an update-compatible coders. */ public class SolaceRecordCoder extends CustomCoder { - private static final Coder BYTE_CODER = ByteArrayCoder.of(); + private static final Coder BYTE_CODER = ByteArrayCoder.of(); - private static final NullableCoder LONG_CODER = NullableCoder.of(VarLongCoder.of()); - private static final NullableCoder INTEGER_CODER = NullableCoder.of(VarIntCoder.of()); - private static final NullableCoder STRING_CODER = - NullableCoder.of(StringUtf8Coder.of()); - private static final NullableCoder BOOLEAN_CODER = NullableCoder.of(BooleanCoder.of()); + private static final NullableCoder LONG_CODER = NullableCoder.of(VarLongCoder.of()); + private static final NullableCoder INTEGER_CODER = NullableCoder.of(VarIntCoder.of()); + private static final NullableCoder STRING_CODER = NullableCoder.of(StringUtf8Coder.of()); + private static final NullableCoder BOOLEAN_CODER = NullableCoder.of(BooleanCoder.of()); - public static SolaceRecordCoder of() { - return new SolaceRecordCoder(); - } + public static SolaceRecordCoder of() { + return new SolaceRecordCoder(); + } - @Override - public void encode(Record value, OutputStream outStream) throws IOException { - // Check if the destination is a topic or a queue, and encode that info - STRING_CODER.encode(value.getMessageId(), outStream); - STRING_CODER.encode(value.getReplicationGroupMessageId(), outStream); - BYTE_CODER.encode(value.getPayload(), outStream); - STRING_CODER.encode(value.getDestination().getName(), outStream); - STRING_CODER.encode(value.getDestination().getType().toString(), outStream); - LONG_CODER.encode(value.getExpiration(), outStream); - INTEGER_CODER.encode(value.getPriority(), outStream); - BOOLEAN_CODER.encode(value.getRedelivered(), outStream); - STRING_CODER.encode(value.getReplyTo(), outStream); - LONG_CODER.encode(value.getReceiveTimestamp(), outStream); - LONG_CODER.encode(value.getSenderTimestamp(), outStream); - LONG_CODER.encode(value.getSequenceNumber(), outStream); - LONG_CODER.encode(value.getTimeToLive(), outStream); - } + @Override + public void encode(Record value, OutputStream outStream) throws IOException { + // Check if the destination is a topic or a queue, and encode that info + STRING_CODER.encode(value.getMessageId(), outStream); + STRING_CODER.encode(value.getReplicationGroupMessageId(), outStream); + BYTE_CODER.encode(value.getPayload(), outStream); + STRING_CODER.encode(value.getDestination().getName(), outStream); + STRING_CODER.encode(value.getDestination().getType().toString(), outStream); + LONG_CODER.encode(value.getExpiration(), outStream); + INTEGER_CODER.encode(value.getPriority(), outStream); + BOOLEAN_CODER.encode(value.getRedelivered(), outStream); + STRING_CODER.encode(value.getReplyTo(), outStream); + LONG_CODER.encode(value.getReceiveTimestamp(), outStream); + LONG_CODER.encode(value.getSenderTimestamp(), outStream); + LONG_CODER.encode(value.getSequenceNumber(), outStream); + LONG_CODER.encode(value.getTimeToLive(), outStream); + } - @Override - public Record decode(InputStream inStream) throws IOException { - return Record.builder() - .setMessageId(STRING_CODER.decode(inStream)) - .setReplicationGroupMessageId(STRING_CODER.decode(inStream)) - .setPayload(BYTE_CODER.decode(inStream)) - .setDestination( - Solace.Destination.builder() - .setName(STRING_CODER.decode(inStream)) - .setType( - Objects.equals(STRING_CODER.decode(inStream), "QUEUE") - ? Solace.DestinationType.QUEUE - : Solace.DestinationType.TOPIC) - .build()) - .setExpiration(LONG_CODER.decode(inStream)) - .setPriority(INTEGER_CODER.decode(inStream)) - .setRedelivered(BOOLEAN_CODER.decode(inStream)) - .setReplyTo(STRING_CODER.decode(inStream)) - .setReceiveTimestamp(LONG_CODER.decode(inStream)) - .setSenderTimestamp(LONG_CODER.decode(inStream)) - .setSequenceNumber(LONG_CODER.decode(inStream)) - .setTimeToLive(LONG_CODER.decode(inStream)) - .build(); - } + @Override + public Record decode(InputStream inStream) throws IOException { + return Record.builder() + .setMessageId(STRING_CODER.decode(inStream)) + .setReplicationGroupMessageId(STRING_CODER.decode(inStream)) + .setPayload(BYTE_CODER.decode(inStream)) + .setDestination( + Solace.Destination.builder() + .setName(STRING_CODER.decode(inStream)) + .setType( + Objects.equals(STRING_CODER.decode(inStream), "QUEUE") + ? Solace.DestinationType.QUEUE + : Solace.DestinationType.TOPIC) + .build()) + .setExpiration(LONG_CODER.decode(inStream)) + .setPriority(INTEGER_CODER.decode(inStream)) + .setRedelivered(BOOLEAN_CODER.decode(inStream)) + .setReplyTo(STRING_CODER.decode(inStream)) + .setReceiveTimestamp(LONG_CODER.decode(inStream)) + .setSenderTimestamp(LONG_CODER.decode(inStream)) + .setSequenceNumber(LONG_CODER.decode(inStream)) + .setTimeToLive(LONG_CODER.decode(inStream)) + .build(); + } } diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/SolaceCheckpointMark.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/SolaceCheckpointMark.java index ba3c51f43949..956de7ce5a84 100644 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/SolaceCheckpointMark.java +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/SolaceCheckpointMark.java @@ -1,11 +1,13 @@ /* - * Copyright 2023 Google. + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -13,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.google.cloud.dataflow.dce.io.solace.read; +package org.apache.beam.sdk.io.solace.read; import com.solacesystems.jcsmp.BytesXMLMessage; import java.util.List; @@ -35,56 +37,56 @@ @Internal @DefaultCoder(AvroCoder.class) public class SolaceCheckpointMark implements UnboundedSource.CheckpointMark { - private static final Logger LOG = LoggerFactory.getLogger(SolaceCheckpointMark.class); + private static final Logger LOG = LoggerFactory.getLogger(SolaceCheckpointMark.class); - private transient AtomicBoolean activeReader; - @Nullable private transient ConcurrentLinkedDeque ackQueue; + private transient AtomicBoolean activeReader; + @Nullable private transient ConcurrentLinkedDeque ackQueue; - @SuppressWarnings("initialization") // Avro will set the fields by breaking abstraction - private SolaceCheckpointMark() {} // for Avro + @SuppressWarnings("initialization") // Avro will set the fields by breaking abstraction + private SolaceCheckpointMark() {} // for Avro - public SolaceCheckpointMark( - @Nullable AtomicBoolean activeReader, List ackQueue) { - this.activeReader = activeReader; - if (ackQueue != null) { - this.ackQueue = new ConcurrentLinkedDeque<>(ackQueue); - } + public SolaceCheckpointMark( + @Nullable AtomicBoolean activeReader, List ackQueue) { + this.activeReader = activeReader; + if (ackQueue != null) { + this.ackQueue = new ConcurrentLinkedDeque<>(ackQueue); } + } - @Override - public void finalizeCheckpoint() { - if (activeReader == null || !activeReader.get() || ackQueue == null) { - return; - } + @Override + public void finalizeCheckpoint() { + if (activeReader == null || !activeReader.get() || ackQueue == null) { + return; + } - LOG.debug( - "SolaceIO.Read: SolaceCheckpointMark: Started to finalize {} with {} messages.", - this.getClass().getSimpleName(), - ackQueue.size()); + LOG.debug( + "SolaceIO.Read: SolaceCheckpointMark: Started to finalize {} with {} messages.", + this.getClass().getSimpleName(), + ackQueue.size()); - while (ackQueue.size() > 0) { - BytesXMLMessage msg = ackQueue.poll(); - if (msg != null) { - msg.ackMessage(); - } - } + while (ackQueue.size() > 0) { + BytesXMLMessage msg = ackQueue.poll(); + if (msg != null) { + msg.ackMessage(); + } } + } - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (!(o instanceof SolaceCheckpointMark)) { - return false; - } - SolaceCheckpointMark that = (SolaceCheckpointMark) o; - return Objects.equals(activeReader, that.activeReader) - && Objects.equals(ackQueue, that.ackQueue); + @Override + public boolean equals(Object o) { + if (this == o) { + return true; } - - @Override - public int hashCode() { - return Objects.hash(activeReader, ackQueue); + if (!(o instanceof SolaceCheckpointMark)) { + return false; } + SolaceCheckpointMark that = (SolaceCheckpointMark) o; + return Objects.equals(activeReader, that.activeReader) + && Objects.equals(ackQueue, that.ackQueue); + } + + @Override + public int hashCode() { + return Objects.hash(activeReader, ackQueue); + } } diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/UnboundedSolaceReader.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/UnboundedSolaceReader.java index fea668bb47f2..88fb2e9e6e43 100644 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/UnboundedSolaceReader.java +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/UnboundedSolaceReader.java @@ -1,11 +1,13 @@ /* - * Copyright 2023 Google. + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -13,11 +15,8 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.google.cloud.dataflow.dce.io.solace.read; +package org.apache.beam.sdk.io.solace.read; -import com.google.cloud.dataflow.dce.io.solace.broker.MessageReceiver; -import com.google.cloud.dataflow.dce.io.solace.broker.SempClient; -import com.google.cloud.dataflow.dce.io.solace.broker.SessionService; import com.google.common.annotations.VisibleForTesting; import com.solacesystems.jcsmp.BytesXMLMessage; import java.io.IOException; @@ -29,6 +28,9 @@ import java.util.concurrent.atomic.AtomicBoolean; import org.apache.beam.sdk.io.UnboundedSource; import org.apache.beam.sdk.io.UnboundedSource.UnboundedReader; +import org.apache.beam.sdk.io.solace.broker.MessageReceiver; +import org.apache.beam.sdk.io.solace.broker.SempClient; +import org.apache.beam.sdk.io.solace.broker.SessionService; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.joda.time.Instant; import org.slf4j.Logger; @@ -36,146 +38,146 @@ /** Unbounded Reader to read messages from a Solace Router. */ @VisibleForTesting -class UnboundedSolaceReader extends UnboundedSource.UnboundedReader { - - private static final Logger LOG = LoggerFactory.getLogger(UnboundedSolaceReader.class); - private final UnboundedSolaceSource currentSource; - private final WatermarkPolicy watermarkPolicy; - AtomicBoolean active = new AtomicBoolean(true); - private BytesXMLMessage solaceOriginalRecord; - private T solaceMappedRecord; - private MessageReceiver messageReceiver; - private SessionService sessionService; - private final SempClient sempClient; - - /** - * Queue to place advanced messages before {@link #getCheckpointMark()} be called non-concurrent - * queue, should only be accessed by the reader thread A given {@link UnboundedReader} object - * will only be accessed by a single thread at once. - */ - private final java.util.Queue elementsToCheckpoint = new ArrayDeque<>(); - - public UnboundedSolaceReader(UnboundedSolaceSource currentSource) { - this.currentSource = currentSource; - this.watermarkPolicy = WatermarkPolicy.create(currentSource.getTimestampFn()); - this.sessionService = currentSource.getSessionServiceFactory().create(); - this.sempClient = currentSource.getSempClientFactory().create(); +class UnboundedSolaceReader extends UnboundedReader { + + private static final Logger LOG = LoggerFactory.getLogger(UnboundedSolaceReader.class); + private final UnboundedSolaceSource currentSource; + private final WatermarkPolicy watermarkPolicy; + AtomicBoolean active = new AtomicBoolean(true); + private BytesXMLMessage solaceOriginalRecord; + private T solaceMappedRecord; + private MessageReceiver messageReceiver; + private SessionService sessionService; + private final SempClient sempClient; + + /** + * Queue to place advanced messages before {@link #getCheckpointMark()} be called non-concurrent + * queue, should only be accessed by the reader thread A given {@link UnboundedReader} object will + * only be accessed by a single thread at once. + */ + private final java.util.Queue elementsToCheckpoint = new ArrayDeque<>(); + + public UnboundedSolaceReader(UnboundedSolaceSource currentSource) { + this.currentSource = currentSource; + this.watermarkPolicy = WatermarkPolicy.create(currentSource.getTimestampFn()); + this.sessionService = currentSource.getSessionServiceFactory().create(); + this.sempClient = currentSource.getSempClientFactory().create(); + } + + @Override + public boolean start() { + populateSession(); + populateMessageConsumer(); + return advance(); + } + + public void populateSession() { + if (sessionService == null) { + sessionService = getCurrentSource().getSessionServiceFactory().create(); } - - @Override - public boolean start() { - populateSession(); - populateMessageConsumer(); - return advance(); + if (sessionService.isClosed()) { + sessionService.connect(); } + } - public void populateSession() { - if (sessionService == null) { - sessionService = getCurrentSource().getSessionServiceFactory().create(); - } - if (sessionService.isClosed()) { - sessionService.connect(); - } + private void populateMessageConsumer() { + if (messageReceiver == null) { + messageReceiver = sessionService.createReceiver(); + messageReceiver.start(); } - - private void populateMessageConsumer() { - if (messageReceiver == null) { - messageReceiver = sessionService.createReceiver(); - messageReceiver.start(); - } - if (messageReceiver.isClosed()) { - messageReceiver.start(); - } + if (messageReceiver.isClosed()) { + messageReceiver.start(); } - - @Override - public boolean advance() { - BytesXMLMessage receivedXmlMessage; - try { - receivedXmlMessage = messageReceiver.receive(); - } catch (IOException e) { - LOG.warn("SolaceIO.Read: Exception when pulling messages from the broker.", e); - return false; - } - - if (receivedXmlMessage == null) { - return false; - } - elementsToCheckpoint.add(receivedXmlMessage); - solaceOriginalRecord = receivedXmlMessage; - solaceMappedRecord = getCurrentSource().getParseFn().apply(receivedXmlMessage); - watermarkPolicy.update(solaceMappedRecord); - return true; + } + + @Override + public boolean advance() { + BytesXMLMessage receivedXmlMessage; + try { + receivedXmlMessage = messageReceiver.receive(); + } catch (IOException e) { + LOG.warn("SolaceIO.Read: Exception when pulling messages from the broker.", e); + return false; } - @Override - public void close() { - active.set(false); - sessionService.close(); + if (receivedXmlMessage == null) { + return false; } - - @Override - public Instant getWatermark() { - // should be only used by a test receiver - if (messageReceiver.isEOF()) { - return BoundedWindow.TIMESTAMP_MAX_VALUE; - } - return watermarkPolicy.getWatermark(); + elementsToCheckpoint.add(receivedXmlMessage); + solaceOriginalRecord = receivedXmlMessage; + solaceMappedRecord = getCurrentSource().getParseFn().apply(receivedXmlMessage); + watermarkPolicy.update(solaceMappedRecord); + return true; + } + + @Override + public void close() { + active.set(false); + sessionService.close(); + } + + @Override + public Instant getWatermark() { + // should be only used by a test receiver + if (messageReceiver.isEOF()) { + return BoundedWindow.TIMESTAMP_MAX_VALUE; } - - @Override - public UnboundedSource.CheckpointMark getCheckpointMark() { - List ackQueue = new ArrayList<>(); - while (!elementsToCheckpoint.isEmpty()) { - BytesXMLMessage msg = elementsToCheckpoint.poll(); - ackQueue.add(msg); - } - return new SolaceCheckpointMark(active, ackQueue); + return watermarkPolicy.getWatermark(); + } + + @Override + public UnboundedSource.CheckpointMark getCheckpointMark() { + List ackQueue = new ArrayList<>(); + while (!elementsToCheckpoint.isEmpty()) { + BytesXMLMessage msg = elementsToCheckpoint.poll(); + ackQueue.add(msg); } + return new SolaceCheckpointMark(active, ackQueue); + } - @Override - public T getCurrent() throws NoSuchElementException { - if (solaceMappedRecord == null) { - throw new NoSuchElementException(); - } - return solaceMappedRecord; + @Override + public T getCurrent() throws NoSuchElementException { + if (solaceMappedRecord == null) { + throw new NoSuchElementException(); } + return solaceMappedRecord; + } - @Override - public byte[] getCurrentRecordId() throws NoSuchElementException { - if (solaceOriginalRecord == null) { - throw new NoSuchElementException(); - } - if (solaceOriginalRecord.getApplicationMessageId() != null) { - return solaceOriginalRecord.getApplicationMessageId().getBytes(StandardCharsets.UTF_8); - } else { - return solaceOriginalRecord - .getReplicationGroupMessageId() - .toString() - .getBytes(StandardCharsets.UTF_8); - } + @Override + public byte[] getCurrentRecordId() throws NoSuchElementException { + if (solaceOriginalRecord == null) { + throw new NoSuchElementException(); } - - @Override - public UnboundedSolaceSource getCurrentSource() { - return currentSource; + if (solaceOriginalRecord.getApplicationMessageId() != null) { + return solaceOriginalRecord.getApplicationMessageId().getBytes(StandardCharsets.UTF_8); + } else { + return solaceOriginalRecord + .getReplicationGroupMessageId() + .toString() + .getBytes(StandardCharsets.UTF_8); } + } - @Override - public Instant getCurrentTimestamp() throws NoSuchElementException { - if (getCurrent() == null) { - throw new NoSuchElementException(); - } - return currentSource.getTimestampFn().apply(getCurrent()); - } + @Override + public UnboundedSolaceSource getCurrentSource() { + return currentSource; + } - @Override - public long getTotalBacklogBytes() { - try { - return sempClient.getBacklogBytes(currentSource.getQueue().getName()); - } catch (IOException e) { - LOG.warn("SolaceIO.Read: Could not query backlog bytes. Returning BACKLOG_UNKNOWN", e); - return BACKLOG_UNKNOWN; - } + @Override + public Instant getCurrentTimestamp() throws NoSuchElementException { + if (getCurrent() == null) { + throw new NoSuchElementException(); + } + return currentSource.getTimestampFn().apply(getCurrent()); + } + + @Override + public long getTotalBacklogBytes() { + try { + return sempClient.getBacklogBytes(currentSource.getQueue().getName()); + } catch (IOException e) { + LOG.warn("SolaceIO.Read: Could not query backlog bytes. Returning BACKLOG_UNKNOWN", e); + return BACKLOG_UNKNOWN; } + } } diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/UnboundedSolaceSource.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/UnboundedSolaceSource.java index fde3c2d93540..60630257e821 100644 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/UnboundedSolaceSource.java +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/UnboundedSolaceSource.java @@ -1,11 +1,13 @@ /* - * Copyright 2023 Google. + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -13,10 +15,8 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.google.cloud.dataflow.dce.io.solace.read; +package org.apache.beam.sdk.io.solace.read; -import com.google.cloud.dataflow.dce.io.solace.broker.SempClientFactory; -import com.google.cloud.dataflow.dce.io.solace.broker.SessionServiceFactory; import com.solacesystems.jcsmp.BytesXMLMessage; import com.solacesystems.jcsmp.Queue; import java.io.IOException; @@ -27,6 +27,8 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.extensions.avro.coders.AvroCoder; import org.apache.beam.sdk.io.UnboundedSource; +import org.apache.beam.sdk.io.solace.broker.SempClientFactory; +import org.apache.beam.sdk.io.solace.broker.SessionServiceFactory; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.SerializableFunction; import org.joda.time.Instant; @@ -35,114 +37,112 @@ @Internal public class UnboundedSolaceSource extends UnboundedSource { - private static final long serialVersionUID = 42L; - private static final Logger LOG = LoggerFactory.getLogger(UnboundedSolaceSource.class); - private final Queue queue; - private final Integer maxNumConnections; - private final Coder coder; - private final boolean enableDeduplication; - private final SempClientFactory sempClientFactory; - private final SessionServiceFactory sessionServiceFactory; - private final SerializableFunction timestampFn; - private final SerializableFunction parseFn; + private static final long serialVersionUID = 42L; + private static final Logger LOG = LoggerFactory.getLogger(UnboundedSolaceSource.class); + private final Queue queue; + private final Integer maxNumConnections; + private final Coder coder; + private final boolean enableDeduplication; + private final SempClientFactory sempClientFactory; + private final SessionServiceFactory sessionServiceFactory; + private final SerializableFunction timestampFn; + private final SerializableFunction parseFn; - public Queue getQueue() { - return queue; - } + public Queue getQueue() { + return queue; + } - public SessionServiceFactory getSessionServiceFactory() { - return sessionServiceFactory; - } + public SessionServiceFactory getSessionServiceFactory() { + return sessionServiceFactory; + } - public SempClientFactory getSempClientFactory() { - return sempClientFactory; - } + public SempClientFactory getSempClientFactory() { + return sempClientFactory; + } - public SerializableFunction getTimestampFn() { - return timestampFn; - } + public SerializableFunction getTimestampFn() { + return timestampFn; + } - public SerializableFunction getParseFn() { - return parseFn; - } + public SerializableFunction getParseFn() { + return parseFn; + } - public UnboundedSolaceSource( - Queue queue, - SempClientFactory sempClientFactory, - SessionServiceFactory sessionServiceFactory, - Integer maxNumConnections, - boolean enableDeduplication, - Coder coder, - SerializableFunction timestampFn, - SerializableFunction parseFn) { - this.queue = queue; - this.sempClientFactory = sempClientFactory; - this.sessionServiceFactory = sessionServiceFactory; - this.maxNumConnections = maxNumConnections; - this.enableDeduplication = enableDeduplication; - this.coder = coder; - this.timestampFn = timestampFn; - this.parseFn = parseFn; - } + public UnboundedSolaceSource( + Queue queue, + SempClientFactory sempClientFactory, + SessionServiceFactory sessionServiceFactory, + Integer maxNumConnections, + boolean enableDeduplication, + Coder coder, + SerializableFunction timestampFn, + SerializableFunction parseFn) { + this.queue = queue; + this.sempClientFactory = sempClientFactory; + this.sessionServiceFactory = sessionServiceFactory; + this.maxNumConnections = maxNumConnections; + this.enableDeduplication = enableDeduplication; + this.coder = coder; + this.timestampFn = timestampFn; + this.parseFn = parseFn; + } - @Override - public UnboundedReader createReader( - PipelineOptions options, @Nullable SolaceCheckpointMark checkpointMark) { - // it makes no sense to resume a Solace Session with the previous checkpoint - // so don't need the pass a checkpoint to new a Solace Reader - return new UnboundedSolaceReader<>(this); - } + @Override + public UnboundedReader createReader( + PipelineOptions options, @Nullable SolaceCheckpointMark checkpointMark) { + // it makes no sense to resume a Solace Session with the previous checkpoint + // so don't need the pass a checkpoint to new a Solace Reader + return new UnboundedSolaceReader<>(this); + } - @Override - public List> split(int desiredNumSplits, PipelineOptions options) - throws IOException { - boolean queueNonExclusive = sempClientFactory.create().isQueueNonExclusive(queue.getName()); - if (queueNonExclusive) { - return getSolaceSources(desiredNumSplits, maxNumConnections); - } else { - LOG.warn( - "SolaceIO.Read: The queue {} is exclusive. Provisioning only 1 read client.", - queue); - return getSolaceSources(desiredNumSplits, 1); - } + @Override + public List> split(int desiredNumSplits, PipelineOptions options) + throws IOException { + boolean queueNonExclusive = sempClientFactory.create().isQueueNonExclusive(queue.getName()); + if (queueNonExclusive) { + return getSolaceSources(desiredNumSplits, maxNumConnections); + } else { + LOG.warn("SolaceIO.Read: The queue {} is exclusive. Provisioning only 1 read client.", queue); + return getSolaceSources(desiredNumSplits, 1); } + } - private List> getSolaceSources( - int desiredNumSplits, Integer maxNumConnections) { - List> sourceList = new ArrayList<>(); - int numSplits = - maxNumConnections != null - ? Math.min(desiredNumSplits, maxNumConnections) - : desiredNumSplits; - LOG.info("SolaceIO.Read: UnboundedSolaceSource: creating {} read connections.", numSplits); - for (int i = 0; i < numSplits; i++) { - UnboundedSolaceSource source = - new UnboundedSolaceSource<>( - queue, - sempClientFactory, - sessionServiceFactory, - maxNumConnections, - enableDeduplication, - coder, - timestampFn, - parseFn); - sourceList.add(source); - } - return sourceList; + private List> getSolaceSources( + int desiredNumSplits, Integer maxNumConnections) { + List> sourceList = new ArrayList<>(); + int numSplits = + maxNumConnections != null + ? Math.min(desiredNumSplits, maxNumConnections) + : desiredNumSplits; + LOG.info("SolaceIO.Read: UnboundedSolaceSource: creating {} read connections.", numSplits); + for (int i = 0; i < numSplits; i++) { + UnboundedSolaceSource source = + new UnboundedSolaceSource<>( + queue, + sempClientFactory, + sessionServiceFactory, + maxNumConnections, + enableDeduplication, + coder, + timestampFn, + parseFn); + sourceList.add(source); } + return sourceList; + } - @Override - public Coder getCheckpointMarkCoder() { - return AvroCoder.of(SolaceCheckpointMark.class); - } + @Override + public Coder getCheckpointMarkCoder() { + return AvroCoder.of(SolaceCheckpointMark.class); + } - @Override - public Coder getOutputCoder() { - return coder; - } + @Override + public Coder getOutputCoder() { + return coder; + } - @Override - public boolean requiresDeduping() { - return enableDeduplication; - } + @Override + public boolean requiresDeduping() { + return enableDeduplication; + } } diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/WatermarkParameters.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/WatermarkParameters.java index f1f46ca26b89..901db77c8bb2 100644 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/WatermarkParameters.java +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/WatermarkParameters.java @@ -1,11 +1,13 @@ /* - * Copyright 2024 Google. + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -13,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.google.cloud.dataflow.dce.io.solace.read; +package org.apache.beam.sdk.io.solace.read; import static com.google.common.base.Preconditions.checkArgument; @@ -28,61 +30,59 @@ @AutoValue public abstract class WatermarkParameters implements Serializable { - private static final Duration STANDARD_WATERMARK_IDLE_DURATION_THRESHOLD = - Duration.standardSeconds(30); + private static final Duration STANDARD_WATERMARK_IDLE_DURATION_THRESHOLD = + Duration.standardSeconds(30); - abstract Instant getCurrentWatermark(); + abstract Instant getCurrentWatermark(); - abstract Instant getLastSavedWatermark(); + abstract Instant getLastSavedWatermark(); - abstract Instant getLastUpdateTime(); + abstract Instant getLastUpdateTime(); - abstract SerializableFunction getTimestampFn(); + abstract SerializableFunction getTimestampFn(); - abstract Duration getWatermarkIdleDurationThreshold(); + abstract Duration getWatermarkIdleDurationThreshold(); - public abstract Builder toBuilder(); + public abstract Builder toBuilder(); - static Builder builder() { - return new AutoValue_WatermarkParameters.Builder() - .setCurrentWatermark(BoundedWindow.TIMESTAMP_MIN_VALUE) - .setLastSavedWatermark(BoundedWindow.TIMESTAMP_MIN_VALUE) - .setLastUpdateTime(Instant.now()) - .setWatermarkIdleDurationThreshold(STANDARD_WATERMARK_IDLE_DURATION_THRESHOLD); - } + static Builder builder() { + return new AutoValue_WatermarkParameters.Builder() + .setCurrentWatermark(BoundedWindow.TIMESTAMP_MIN_VALUE) + .setLastSavedWatermark(BoundedWindow.TIMESTAMP_MIN_VALUE) + .setLastUpdateTime(Instant.now()) + .setWatermarkIdleDurationThreshold(STANDARD_WATERMARK_IDLE_DURATION_THRESHOLD); + } - @AutoValue.Builder - abstract static class Builder { - abstract Builder setCurrentWatermark(Instant currentWatermark); + @AutoValue.Builder + abstract static class Builder { + abstract Builder setCurrentWatermark(Instant currentWatermark); - abstract Builder setLastSavedWatermark(Instant eventTime); + abstract Builder setLastSavedWatermark(Instant eventTime); - abstract Builder setLastUpdateTime(Instant now); + abstract Builder setLastUpdateTime(Instant now); - abstract Builder setWatermarkIdleDurationThreshold( - Duration watermarkIdleDurationThreshold); + abstract Builder setWatermarkIdleDurationThreshold(Duration watermarkIdleDurationThreshold); - abstract Builder setTimestampFn(SerializableFunction timestampFn); + abstract Builder setTimestampFn(SerializableFunction timestampFn); - abstract WatermarkParameters build(); - } + abstract WatermarkParameters build(); + } - /** - * Create an instance of {@link WatermarkParameters} with a {@code SerializableFunction} to - * extract the event time. - */ - public static WatermarkParameters create(SerializableFunction timestampFn) { - checkArgument(timestampFn != null, "timestampFn function is null"); - return WatermarkParameters.builder().setTimestampFn(timestampFn).build(); - } + /** + * Create an instance of {@link WatermarkParameters} with a {@code SerializableFunction} to + * extract the event time. + */ + public static WatermarkParameters create(SerializableFunction timestampFn) { + checkArgument(timestampFn != null, "timestampFn function is null"); + return WatermarkParameters.builder().setTimestampFn(timestampFn).build(); + } - /** - * Specify the watermark idle duration to consider before advancing the watermark. The default - * watermark idle duration threshold is {@link #STANDARD_WATERMARK_IDLE_DURATION_THRESHOLD}. - */ - public WatermarkParameters withWatermarkIdleDurationThreshold( - Duration idleDurationThreshold) { - checkArgument(idleDurationThreshold != null, "watermark idle duration threshold is null"); - return toBuilder().setWatermarkIdleDurationThreshold(idleDurationThreshold).build(); - } + /** + * Specify the watermark idle duration to consider before advancing the watermark. The default + * watermark idle duration threshold is {@link #STANDARD_WATERMARK_IDLE_DURATION_THRESHOLD}. + */ + public WatermarkParameters withWatermarkIdleDurationThreshold(Duration idleDurationThreshold) { + checkArgument(idleDurationThreshold != null, "watermark idle duration threshold is null"); + return toBuilder().setWatermarkIdleDurationThreshold(idleDurationThreshold).build(); + } } diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/WatermarkPolicy.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/WatermarkPolicy.java index 3dfb0879f939..11ade16ce14d 100644 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/WatermarkPolicy.java +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/WatermarkPolicy.java @@ -1,11 +1,13 @@ /* - * Copyright 2024 Google. + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -13,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.google.cloud.dataflow.dce.io.solace.read; +package org.apache.beam.sdk.io.solace.read; import com.google.common.collect.Ordering; import java.io.Serializable; @@ -23,44 +25,44 @@ import org.slf4j.LoggerFactory; public class WatermarkPolicy implements Serializable { - public static final Logger LOG = LoggerFactory.getLogger(WatermarkPolicy.class); - private WatermarkParameters watermarkParameters; + public static final Logger LOG = LoggerFactory.getLogger(WatermarkPolicy.class); + private WatermarkParameters watermarkParameters; - public static WatermarkPolicy create( - SerializableFunction timestampFunction) { - return new WatermarkPolicy(WatermarkParameters.create(timestampFunction)); - } + public static WatermarkPolicy create(SerializableFunction timestampFunction) { + return new WatermarkPolicy(WatermarkParameters.create(timestampFunction)); + } - private WatermarkPolicy(WatermarkParameters watermarkParameters) { - this.watermarkParameters = watermarkParameters; - } + private WatermarkPolicy(WatermarkParameters watermarkParameters) { + this.watermarkParameters = watermarkParameters; + } - public Instant getWatermark() { - Instant now = Instant.now(); - Instant watermarkIdleThreshold = - now.minus(watermarkParameters.getWatermarkIdleDurationThreshold()); + public Instant getWatermark() { + Instant now = Instant.now(); + Instant watermarkIdleThreshold = + now.minus(watermarkParameters.getWatermarkIdleDurationThreshold()); - Instant newWatermark = - watermarkParameters.getLastUpdateTime().isBefore(watermarkIdleThreshold) - ? watermarkIdleThreshold - : watermarkParameters.getLastSavedWatermark(); + Instant newWatermark = + watermarkParameters.getLastUpdateTime().isBefore(watermarkIdleThreshold) + ? watermarkIdleThreshold + : watermarkParameters.getLastSavedWatermark(); - if (newWatermark.isAfter(watermarkParameters.getCurrentWatermark())) { - watermarkParameters = - watermarkParameters.toBuilder().setCurrentWatermark(newWatermark).build(); - } - return watermarkParameters.getCurrentWatermark(); + if (newWatermark.isAfter(watermarkParameters.getCurrentWatermark())) { + watermarkParameters = + watermarkParameters.toBuilder().setCurrentWatermark(newWatermark).build(); } + return watermarkParameters.getCurrentWatermark(); + } - public void update(T record) { - watermarkParameters = - watermarkParameters.toBuilder() - .setLastSavedWatermark( - Ordering.natural() - .max( - watermarkParameters.getLastSavedWatermark(), - watermarkParameters.getTimestampFn().apply(record))) - .setLastUpdateTime(Instant.now()) - .build(); - } + public void update(T record) { + watermarkParameters = + watermarkParameters + .toBuilder() + .setLastSavedWatermark( + Ordering.natural() + .max( + watermarkParameters.getLastSavedWatermark(), + watermarkParameters.getTimestampFn().apply(record))) + .setLastUpdateTime(Instant.now()) + .build(); + } } diff --git a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/MockSempClient.java b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/MockSempClient.java index 1599d5587164..d4703237371a 100644 --- a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/MockSempClient.java +++ b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/MockSempClient.java @@ -1,11 +1,13 @@ /* - * Copyright 2024 Google. + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -13,74 +15,73 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.io.solace.io.solace; +package org.apache.beam.sdk.io.solace; -import org.apache.beam.sdk.io.solace.broker.SempClient; import com.solacesystems.jcsmp.JCSMPFactory; import com.solacesystems.jcsmp.Queue; import java.io.IOException; +import org.apache.beam.sdk.io.solace.broker.SempClient; import org.apache.beam.sdk.transforms.SerializableFunction; public class MockSempClient implements SempClient { - private final SerializableFunction isQueueNonExclusiveFn; - private final SerializableFunction getBacklogBytesFn; - private final SerializableFunction createQueueForTopicFn; + private final SerializableFunction isQueueNonExclusiveFn; + private final SerializableFunction getBacklogBytesFn; + private final SerializableFunction createQueueForTopicFn; - private MockSempClient( - SerializableFunction isQueueNonExclusiveFn, - SerializableFunction getBacklogBytesFn, - SerializableFunction createQueueForTopicFn) { - this.isQueueNonExclusiveFn = isQueueNonExclusiveFn; - this.getBacklogBytesFn = getBacklogBytesFn; - this.createQueueForTopicFn = createQueueForTopicFn; - } - - public static Builder builder() { - return new Builder(); - } + private MockSempClient( + SerializableFunction isQueueNonExclusiveFn, + SerializableFunction getBacklogBytesFn, + SerializableFunction createQueueForTopicFn) { + this.isQueueNonExclusiveFn = isQueueNonExclusiveFn; + this.getBacklogBytesFn = getBacklogBytesFn; + this.createQueueForTopicFn = createQueueForTopicFn; + } - public static class Builder { - private SerializableFunction isQueueNonExclusiveFn = (queueName) -> true; - private SerializableFunction getBacklogBytesFn = (queueName) -> 0L; - private SerializableFunction createQueueForTopicFn = (queueName) -> 0; + public static Builder builder() { + return new Builder(); + } - public Builder setIsQueueNonExclusiveFn( - SerializableFunction isQueueNonExclusiveFn) { - this.isQueueNonExclusiveFn = isQueueNonExclusiveFn; - return this; - } + public static class Builder { + private SerializableFunction isQueueNonExclusiveFn = (queueName) -> true; + private SerializableFunction getBacklogBytesFn = (queueName) -> 0L; + private SerializableFunction createQueueForTopicFn = (queueName) -> 0; - public Builder setGetBacklogBytesFn(SerializableFunction getBacklogBytesFn) { - this.getBacklogBytesFn = getBacklogBytesFn; - return this; - } - - public Builder setCreateQueueForTopicFn( - SerializableFunction createQueueForTopicFn) { - this.createQueueForTopicFn = createQueueForTopicFn; - return this; - } - - public MockSempClient build() { - return new MockSempClient( - isQueueNonExclusiveFn, getBacklogBytesFn, createQueueForTopicFn); - } + public Builder setIsQueueNonExclusiveFn( + SerializableFunction isQueueNonExclusiveFn) { + this.isQueueNonExclusiveFn = isQueueNonExclusiveFn; + return this; } - @Override - public boolean isQueueNonExclusive(String queueName) throws IOException { - return isQueueNonExclusiveFn.apply(queueName); + public Builder setGetBacklogBytesFn(SerializableFunction getBacklogBytesFn) { + this.getBacklogBytesFn = getBacklogBytesFn; + return this; } - @Override - public Queue createQueueForTopic(String queueName, String topicName) throws IOException { - createQueueForTopicFn.apply(queueName); - return JCSMPFactory.onlyInstance().createQueue(queueName); + public Builder setCreateQueueForTopicFn( + SerializableFunction createQueueForTopicFn) { + this.createQueueForTopicFn = createQueueForTopicFn; + return this; } - @Override - public long getBacklogBytes(String queueName) throws IOException { - return getBacklogBytesFn.apply(queueName); + public MockSempClient build() { + return new MockSempClient(isQueueNonExclusiveFn, getBacklogBytesFn, createQueueForTopicFn); } + } + + @Override + public boolean isQueueNonExclusive(String queueName) throws IOException { + return isQueueNonExclusiveFn.apply(queueName); + } + + @Override + public Queue createQueueForTopic(String queueName, String topicName) throws IOException { + createQueueForTopicFn.apply(queueName); + return JCSMPFactory.onlyInstance().createQueue(queueName); + } + + @Override + public long getBacklogBytes(String queueName) throws IOException { + return getBacklogBytesFn.apply(queueName); + } } diff --git a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/MockSempClientFactory.java b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/MockSempClientFactory.java index 271bca073ee2..4f7b353c8181 100644 --- a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/MockSempClientFactory.java +++ b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/MockSempClientFactory.java @@ -1,11 +1,13 @@ /* - * Copyright 2024 Google. + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -13,21 +15,20 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.io.solace.io.solace; +package org.apache.beam.sdk.io.solace; import org.apache.beam.sdk.io.solace.broker.SempClient; import org.apache.beam.sdk.io.solace.broker.SempClientFactory; public class MockSempClientFactory implements SempClientFactory { - MockSempClient sempClient; + MockSempClient sempClient; - public MockSempClientFactory(MockSempClient sempClient) { - this.sempClient = sempClient; - } + public MockSempClientFactory(MockSempClient sempClient) { + this.sempClient = sempClient; + } - @Override - public SempClient create() { - // todo new or existing instance? - return sempClient; - } + @Override + public SempClient create() { + return sempClient; + } } diff --git a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/MockSessionService.java b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/MockSessionService.java index 2f9fbd0908d6..7b14da138c64 100644 --- a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/MockSessionService.java +++ b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/MockSessionService.java @@ -1,11 +1,13 @@ /* - * Copyright 2024 Google. + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -13,75 +15,74 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.io.solace.io.solace; +package org.apache.beam.sdk.io.solace; -import org.apache.beam.sdk.io.solace.broker.MessageReceiver; -import org.apache.beam.sdk.io.solace.broker.SessionService; import com.solacesystems.jcsmp.BytesXMLMessage; import java.io.IOException; import java.io.Serializable; import java.util.concurrent.atomic.AtomicInteger; +import org.apache.beam.sdk.io.solace.broker.MessageReceiver; +import org.apache.beam.sdk.io.solace.broker.SessionService; import org.apache.beam.sdk.transforms.SerializableFunction; public class MockSessionService implements SessionService { + private final SerializableFunction getRecordFn; + private MessageReceiver messageReceiver = null; + private final int minMessagesReceived; + + public MockSessionService( + SerializableFunction getRecordFn, int minMessagesReceived) { + this.getRecordFn = getRecordFn; + this.minMessagesReceived = minMessagesReceived; + } + + @Override + public void close() {} + + @Override + public boolean isClosed() { + return false; + } + + @Override + public MessageReceiver createReceiver() { + if (messageReceiver == null) { + messageReceiver = new MockReceiver(getRecordFn, minMessagesReceived); + } + return messageReceiver; + } + + @Override + public void connect() {} + + public static class MockReceiver implements MessageReceiver, Serializable { + private final AtomicInteger counter = new AtomicInteger(); private final SerializableFunction getRecordFn; - private MessageReceiver messageReceiver = null; private final int minMessagesReceived; - public MockSessionService( - SerializableFunction getRecordFn, int minMessagesReceived) { - this.getRecordFn = getRecordFn; - this.minMessagesReceived = minMessagesReceived; + public MockReceiver( + SerializableFunction getRecordFn, int minMessagesReceived) { + this.getRecordFn = getRecordFn; + this.minMessagesReceived = minMessagesReceived; } @Override - public void close() {} + public void start() {} @Override public boolean isClosed() { - return false; + return false; } @Override - public MessageReceiver createReceiver() { - if (messageReceiver == null) { - messageReceiver = new MockReceiver(getRecordFn, minMessagesReceived); - } - return messageReceiver; + public BytesXMLMessage receive() throws IOException { + return getRecordFn.apply(counter.getAndIncrement()); } @Override - public void connect() {} - - public static class MockReceiver implements MessageReceiver, Serializable { - private final AtomicInteger counter = new AtomicInteger(); - private final SerializableFunction getRecordFn; - private final int minMessagesReceived; - - public MockReceiver( - SerializableFunction getRecordFn, - int minMessagesReceived) { - this.getRecordFn = getRecordFn; - this.minMessagesReceived = minMessagesReceived; - } - - @Override - public void start() {} - - @Override - public boolean isClosed() { - return false; - } - - @Override - public BytesXMLMessage receive() throws IOException { - return getRecordFn.apply(counter.getAndIncrement()); - } - - @Override - public boolean isEOF() { - return counter.get() >= minMessagesReceived; - } + public boolean isEOF() { + return counter.get() >= minMessagesReceived; } + } } diff --git a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/MockSessionServiceFactory.java b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/MockSessionServiceFactory.java index fc83ef6def49..4585b8f186ce 100644 --- a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/MockSessionServiceFactory.java +++ b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/MockSessionServiceFactory.java @@ -1,11 +1,13 @@ /* - * Copyright 2024 Google. + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -13,20 +15,20 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.io.solace.io.solace; +package org.apache.beam.sdk.io.solace; import org.apache.beam.sdk.io.solace.broker.SessionService; import org.apache.beam.sdk.io.solace.broker.SessionServiceFactory; public class MockSessionServiceFactory extends SessionServiceFactory { - SessionService sessionService; + SessionService sessionService; - public MockSessionServiceFactory(MockSessionService clientService) { - this.sessionService = clientService; - } + public MockSessionServiceFactory(MockSessionService clientService) { + this.sessionService = clientService; + } - @Override - public SessionService create() { - return sessionService; - } + @Override + public SessionService create() { + return sessionService; + } } diff --git a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/RetryCallableManagerTest.java b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/RetryCallableManagerTest.java index e37da3541837..c743bdba27f1 100644 --- a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/RetryCallableManagerTest.java +++ b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/RetryCallableManagerTest.java @@ -1,11 +1,13 @@ /* - * Copyright 2024 Google. + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -13,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.io.solace.io.solace; +package org.apache.beam.sdk.io.solace; import static org.junit.Assert.assertTrue; @@ -22,154 +24,149 @@ import java.util.Set; import java.util.concurrent.Callable; import java.util.concurrent.atomic.AtomicInteger; -import org.apache.beam.sdk.io.solace.RetryCallableManager; import org.junit.Before; import org.junit.Test; public class RetryCallableManagerTest { - private RetryCallableManager retryCallableManager; + private RetryCallableManager retryCallableManager; - @Before - public void setUp() throws Exception { - int NUMBER_OF_RETRIES = 4; - int RETRY_INTERVAL_SECONDS = 0; - int RETRY_MULTIPLIER = 2; - int MAX_DELAY = 0; + @Before + public void setUp() throws Exception { + int NUMBER_OF_RETRIES = 4; + int RETRY_INTERVAL_SECONDS = 0; + int RETRY_MULTIPLIER = 2; + int MAX_DELAY = 0; - retryCallableManager = - RetryCallableManager.builder() - .setRetrySettings( - RetrySettings.newBuilder() - .setInitialRetryDelay( - org.threeten.bp.Duration.ofSeconds( - RETRY_INTERVAL_SECONDS)) - .setMaxAttempts(NUMBER_OF_RETRIES) - .setMaxRetryDelay( - org.threeten.bp.Duration.ofSeconds(MAX_DELAY)) - .setRetryDelayMultiplier(RETRY_MULTIPLIER) - .build()) - .build(); - } + retryCallableManager = + RetryCallableManager.builder() + .setRetrySettings( + RetrySettings.newBuilder() + .setInitialRetryDelay( + org.threeten.bp.Duration.ofSeconds(RETRY_INTERVAL_SECONDS)) + .setMaxAttempts(NUMBER_OF_RETRIES) + .setMaxRetryDelay(org.threeten.bp.Duration.ofSeconds(MAX_DELAY)) + .setRetryDelayMultiplier(RETRY_MULTIPLIER) + .build()) + .build(); + } - @Test - public void testRetryCallable_ReturnsExpected() { - AtomicInteger executeCounter = new AtomicInteger(0); - Callable incrementingFunction = - () -> { - executeCounter.incrementAndGet(); - if (executeCounter.get() < 2) { - throw new MyException(); - } - return executeCounter.get(); - }; - Integer result = - retryCallableManager.retryCallable(incrementingFunction, Set.of(MyException.class)); - assertTrue(String.format("Should return 2, instead returned %d.", result), result == 2); - } - - @Test - public void testRetryCallable_RetriesExpectedNumberOfTimes() { - AtomicInteger executeCounter = new AtomicInteger(0); - Callable incrementingFunction = - () -> { - executeCounter.incrementAndGet(); - if (executeCounter.get() < 2) { - throw new MyException(); - } - return executeCounter.get(); - }; + @Test + public void testRetryCallable_ReturnsExpected() { + AtomicInteger executeCounter = new AtomicInteger(0); + Callable incrementingFunction = + () -> { + executeCounter.incrementAndGet(); + if (executeCounter.get() < 2) { + throw new MyException(); + } + return executeCounter.get(); + }; + Integer result = retryCallableManager.retryCallable(incrementingFunction, Set.of(MyException.class)); - assertTrue( - String.format("Should run 2 times, instead ran %d times.", executeCounter.get()), - executeCounter.get() == 2); - } + assertTrue(String.format("Should return 2, instead returned %d.", result), result == 2); + } - @Test(expected = RetryHelperException.class) - public void testRetryCallable_ThrowsRetryHelperException() { - Callable incrementingFunction = - () -> { - { - throw new MyException(); - } - }; - retryCallableManager.retryCallable(incrementingFunction, Set.of(MyException.class)); - } + @Test + public void testRetryCallable_RetriesExpectedNumberOfTimes() { + AtomicInteger executeCounter = new AtomicInteger(0); + Callable incrementingFunction = + () -> { + executeCounter.incrementAndGet(); + if (executeCounter.get() < 2) { + throw new MyException(); + } + return executeCounter.get(); + }; + retryCallableManager.retryCallable(incrementingFunction, Set.of(MyException.class)); + assertTrue( + String.format("Should run 2 times, instead ran %d times.", executeCounter.get()), + executeCounter.get() == 2); + } - @Test - public void testRetryCallable_ExecutionCountIsCorrectAfterMultipleExceptions() { - AtomicInteger executeCounter = new AtomicInteger(0); - Callable incrementingFunction = - () -> { - executeCounter.incrementAndGet(); - if (true) { - throw new MyException(); - } - return 0; - }; - try { - retryCallableManager.retryCallable(incrementingFunction, Set.of(MyException.class)); - } catch (RetryHelperException e) { - // ignore exception to check the executeCounter - } - assertTrue( - String.format( - "Should execute 4 times, instead executed %d times", executeCounter.get()), - executeCounter.get() == 4); - } + @Test(expected = RetryHelperException.class) + public void testRetryCallable_ThrowsRetryHelperException() { + Callable incrementingFunction = + () -> { + { + throw new MyException(); + } + }; + retryCallableManager.retryCallable(incrementingFunction, Set.of(MyException.class)); + } - @Test(expected = RetryHelperException.class) - public void testRetryCallable_ThrowsRetryHelperExceptionOnUnspecifiedException() { - Callable incrementingFunction = - () -> { - if (true) { - throw new DoNotIgnoreException(); - } - return 0; - }; - retryCallableManager.retryCallable(incrementingFunction, Set.of(MyException.class)); + @Test + public void testRetryCallable_ExecutionCountIsCorrectAfterMultipleExceptions() { + AtomicInteger executeCounter = new AtomicInteger(0); + Callable incrementingFunction = + () -> { + executeCounter.incrementAndGet(); + if (true) { + throw new MyException(); + } + return 0; + }; + try { + retryCallableManager.retryCallable(incrementingFunction, Set.of(MyException.class)); + } catch (RetryHelperException e) { + // ignore exception to check the executeCounter } + assertTrue( + String.format("Should execute 4 times, instead executed %d times", executeCounter.get()), + executeCounter.get() == 4); + } + + @Test(expected = RetryHelperException.class) + public void testRetryCallable_ThrowsRetryHelperExceptionOnUnspecifiedException() { + Callable incrementingFunction = + () -> { + if (true) { + throw new DoNotIgnoreException(); + } + return 0; + }; + retryCallableManager.retryCallable(incrementingFunction, Set.of(MyException.class)); + } - @Test - public void testRetryCallable_ChecksForAllDefinedExceptions() { - AtomicInteger executeCounter = new AtomicInteger(0); - Callable incrementingFunction = - () -> { - executeCounter.incrementAndGet(); - if (executeCounter.get() % 2 == 0) { - throw new MyException(); - } else if (executeCounter.get() % 2 == 1) { - throw new AnotherException(); - } - return 0; - }; - try { - retryCallableManager.retryCallable( - incrementingFunction, Set.of(MyException.class, AnotherException.class)); - } catch (RetryHelperException e) { - // ignore exception to check the executeCounter - } - assertTrue( - String.format( - "Should execute 4 times, instead executed %d times", executeCounter.get()), - executeCounter.get() == 4); + @Test + public void testRetryCallable_ChecksForAllDefinedExceptions() { + AtomicInteger executeCounter = new AtomicInteger(0); + Callable incrementingFunction = + () -> { + executeCounter.incrementAndGet(); + if (executeCounter.get() % 2 == 0) { + throw new MyException(); + } else if (executeCounter.get() % 2 == 1) { + throw new AnotherException(); + } + return 0; + }; + try { + retryCallableManager.retryCallable( + incrementingFunction, Set.of(MyException.class, AnotherException.class)); + } catch (RetryHelperException e) { + // ignore exception to check the executeCounter } + assertTrue( + String.format("Should execute 4 times, instead executed %d times", executeCounter.get()), + executeCounter.get() == 4); + } - private static class MyException extends Exception { - public MyException() { - super(); - } + private static class MyException extends Exception { + public MyException() { + super(); } + } - private static class AnotherException extends Exception { - public AnotherException() { - super(); - } + private static class AnotherException extends Exception { + public AnotherException() { + super(); } + } - private static class DoNotIgnoreException extends Exception { - public DoNotIgnoreException() { - super(); - } + private static class DoNotIgnoreException extends Exception { + public DoNotIgnoreException() { + super(); } + } } diff --git a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/SolaceDataUtils.java b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/SolaceDataUtils.java index d7d96b3031e3..7f0e04b7c516 100644 --- a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/SolaceDataUtils.java +++ b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/SolaceDataUtils.java @@ -1,11 +1,13 @@ /* - * Copyright 2024 Google. + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -13,10 +15,8 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.io.solace.io.solace; +package org.apache.beam.sdk.io.solace; -import org.apache.beam.sdk.io.solace.SolaceIO; -import org.apache.beam.sdk.io.solace.data.Solace; import com.solacesystems.jcsmp.BytesXMLMessage; import com.solacesystems.jcsmp.DeliveryMode; import com.solacesystems.jcsmp.Destination; @@ -33,742 +33,741 @@ import java.util.Collection; import java.util.List; import java.util.Objects; +import org.apache.beam.sdk.io.solace.data.Solace; import org.apache.beam.sdk.schemas.JavaBeanSchema; import org.apache.beam.sdk.schemas.annotations.DefaultSchema; import org.apache.beam.sdk.transforms.SerializableFunction; public class SolaceDataUtils { - @DefaultSchema(JavaBeanSchema.class) - public static class SimpleRecord { - public String payload; - public String messageId; - - public SimpleRecord() {} + @DefaultSchema(JavaBeanSchema.class) + public static class SimpleRecord { + public String payload; + public String messageId; - public SimpleRecord(String payload, String messageId) { - this.payload = payload; - this.messageId = messageId; - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (!(o instanceof SimpleRecord)) { - return false; - } - SimpleRecord that = (SimpleRecord) o; - return Objects.equals(payload, that.payload) - && Objects.equals(messageId, that.messageId); - } + public SimpleRecord() {} - @Override - public int hashCode() { - return Objects.hash(payload, messageId); - } - - @Override - public String toString() { - return "SimpleRecord{" - + "payload='" - + payload - + '\'' - + ", messageId='" - + messageId - + '\'' - + '}'; - } + public SimpleRecord(String payload, String messageId) { + this.payload = payload; + this.messageId = messageId; } - public static Solace.Record getSolaceRecord(String payload, String messageId) { - return Solace.Record.builder() - .setPayload(payload.getBytes(StandardCharsets.UTF_8)) - .setMessageId(messageId) - .setDestination( - Solace.Destination.builder() - .setName("destination-topic") - .setType(Solace.DestinationType.TOPIC) - .build()) - .setExpiration(1000L) - .setPriority(0) - .setReceiveTimestamp(1708100477067L) - .setRedelivered(false) - .setReplyTo(null) - .setSequenceNumber(null) - .setTimeToLive(1000L) - .setSenderTimestamp(null) - .build(); + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (!(o instanceof SimpleRecord)) { + return false; + } + SimpleRecord that = (SimpleRecord) o; + return Objects.equals(payload, that.payload) && Objects.equals(messageId, that.messageId); } - public static BytesXMLMessage getBytesXmlMessage(String payload) { - return getBytesXmlMessage(payload, "messageId", null); + @Override + public int hashCode() { + return Objects.hash(payload, messageId); } - public static BytesXMLMessage getBytesXmlMessage(String payload, String messageId) { - return getBytesXmlMessage(payload, messageId, null); - } - - public static BytesXMLMessage getBytesXmlMessage( - String payload, String messageId, SerializableFunction ackMessageFn) { - long receiverTimestamp = 1708100477067L; - long expiration = 1000L; - long timeToLive = 1000L; - String destination = "destination-topic"; - - return new BytesXMLMessage() { - - @Override - public byte[] getBytes() { - return payload.getBytes(StandardCharsets.UTF_8); - } - - @Override - public int getContentLength() { - return payload.getBytes(StandardCharsets.UTF_8).length; - } - - @Override - public int readBytes(byte[] arg0) { - return 0; - } - - @Override - public int readBytes(byte[] arg0, int arg1) { - return 0; - } - - @Override - public void rewindContent() { - // return null; - } - - @Override - public void writeBytes(byte[] arg0) { - // return null; - } - - @Override - public void writeBytes(byte[] arg0, int arg1, int arg2) { - // return null; - } - - @Override - public void ackMessage() { - if (ackMessageFn != null) { - ackMessageFn.apply(0); - } - } - - @Override - public void clearAttachment() { - // return null; - } - - @Override - public void clearBinaryMetadataBytes(int arg0) { - // return null; - } - - @Override - public void clearContent() { - // return null; - } - - @Override - public void clearQueueNameLocation() { - // return null; - } - - @Override - public void clearTopicNameLocation() { - // return null; - } - - @Override - public String dump() { - return null; - } - - @Override - public String dump(int arg0) { - return null; - } - - @Override - public long getAckMessageId() { - return 0; - } - - @Override - public String getAppMessageID() { - return null; - } - - @Override - public String getAppMessageType() { - return null; - } - - @Override - public String getApplicationMessageId() { - return messageId; - } - - @Override - public String getApplicationMessageType() { - return null; - } - - @Override - public ByteBuffer getAttachmentByteBuffer() { - return null; - } - - @Override - public int getAttachmentContentLength() { - return 0; - } - - @Override - public int getBinaryMetadataContentLength(int arg0) { - return 0; - } - - @Override - public Collection getBinaryMetadataTypes() { - return null; - } - - @Override - public Long getCacheRequestId() { - return null; - } - - @Override - public List getConsumerIdList() { - return null; - } - - @Override - public String getCorrelationId() { - return null; - } - - @Override - public Object getCorrelationKey() { - return null; - } - - @Override - public User_Cos getCos() { - return null; - } - - @Override - public boolean getDeliverToOne() { - return false; - } - - @Override - public int getDeliveryCount() throws UnsupportedOperationException { - return 0; - } - - @Override - public DeliveryMode getDeliveryMode() { - return null; - } - - @Override - public Destination getDestination() { - return SolaceIO.topicFromName(destination); - } - - @Override - public String getDestinationTopicSuffix() { - return null; - } - - @Override - public boolean getDiscardIndication() { - return false; - } - - @Override - public long getExpiration() { - return expiration; - } - - @Override - public String getHTTPContentEncoding() { - return null; - } - - @Override - public String getHTTPContentType() { - return null; - } - - @Override - public String getMessageId() { - return null; - } - - @Override - public long getMessageIdLong() { - return 0; - } - - @Override - public MessageType getMessageType() { - return null; - } - - @Override - public int getPriority() { - return 0; - } - - @Override - public SDTMap getProperties() { - return null; - } - - @Override - public int getQueueNameLength() { - return 0; - } - - @Override - public int getQueueNameOffset() { - return 0; - } - - @Override - public long getReceiveTimestamp() { - return receiverTimestamp; - } - - @Override - public boolean getRedelivered() { - return false; - } - - @Override - public ReplicationGroupMessageId getReplicationGroupMessageId() { - return null; - } - - @Override - public Destination getReplyTo() { - return null; - } - - @Override - public String getReplyToSuffix() { - return null; - } - - @Override - public Long getSendTimestamp() { - return null; - } - - @Override - public String getSenderID() { - return null; - } - - @Override - public String getSenderId() { - return null; - } - - @Override - public Long getSenderTimestamp() { - return null; - } - - @Override - public Long getSequenceNumber() { - return null; - } - - @Override - public byte getStructuredMsgType() { - return 0x2; - } - - @Override - public boolean getTQDiscardIndication() { - return false; - } - - @Override - public long getTimeToLive() { - return timeToLive; - } - - @Override - public int getTopicNameLength() { - return 5; - } - - @Override - public int getTopicNameOffset() { - return 0; - } - - @Override - public Long getTopicSequenceNumber() { - return null; - } - - @Override - public byte[] getUserData() { - return null; - } - - @Override - public boolean hasAttachment() { - return false; - } - - @Override - public boolean hasBinaryMetadata(int arg0) { - return false; - } - - @Override - public boolean hasContent() { - return false; - } - - @Override - public boolean hasUserData() { - return false; - } - - @Override - public boolean isAckImmediately() { - return false; - } - - @Override - public boolean isCacheMessage() { - return false; - } - - @Override - public boolean isDMQEligible() { - return false; - } - - @Override - public boolean isDeliveryCountSupported() { - return false; - } - - @Override - public boolean isElidingEligible() { - return false; - } - - @Override - public boolean isReadOnly() { - return false; - } - - @Override - public boolean isReplyMessage() { - return false; - } - - @Override - public boolean isStructuredMsg() { - return false; - } - - @Override - public boolean isSuspect() { - return false; - } - - @Override - public int readAttachmentBytes(byte[] arg0) { - return 0; - } - - @Override - public int readAttachmentBytes(byte[] arg0, int arg1) { - return 0; - } - - @Override - public int readAttachmentBytes(int arg0, byte[] arg1, int arg2, int arg3) { - return 0; - } - - @Override - public int readBinaryMetadataBytes(int arg0, byte[] arg1) { - return 0; - } - - @Override - public int readContentBytes(byte[] arg0) { - return 0; - } - - @Override - public int readContentBytes(byte[] arg0, int arg1) { - return 0; - } - - @Override - public int readContentBytes(int arg0, byte[] arg1, int arg2, int arg3) { - return 0; - } - - @Override - public void rejectMessage() { - // return null; - } - - @Override - public void reset() { - // return null; - } - - @Override - public void resetPayload() { - // return null; - } - - @Override - public void rewindAttachment() { - // return null; - } - - @Override - public void setAckImmediately(boolean arg0) { - // return null; - } - - @Override - public void setAppMessageID(String arg0) { - // return null; - } - - @Override - public void setAppMessageType(String arg0) { - // return null; - } - - @Override - public void setApplicationMessageId(String arg0) { - // return null; - } - - @Override - public void setApplicationMessageType(String arg0) { - // return null; - } - - @Override - public void setAsReplyMessage(boolean arg0) { - // return null; - } - - @Override - public void setCorrelationId(String arg0) { - // return null; - } - - @Override - public void setCorrelationKey(Object arg0) { - // return null; - } - - @Override - public void setCos(User_Cos arg0) { - // return null; - } - - @Override - public void setDMQEligible(boolean arg0) { - // return null; - } - - @Override - public void setDeliverToOne(boolean arg0) { - // return null; - } - - @Override - public void setDeliveryMode(DeliveryMode arg0) { - // return null; - } - - @Override - public void setElidingEligible(boolean arg0) { - // return null; - } - - @Override - public void setExpiration(long arg0) { - // return null; - } - - @Override - public void setHTTPContentEncoding(String arg0) { - // return null; - } - - @Override - public void setHTTPContentType(String arg0) { - // return null; - } - - @Override - public void setMessageType(MessageType arg0) { - // return null; - } - - @Override - public void setPriority(int arg0) { - // return null; - } - - @Override - public void setProperties(SDTMap arg0) { - // return null; - } - - @Override - public void setQueueNameLocation(int arg0, int arg1) { - // return null; - } - - @Override - public void setReadOnly() { - // return null; - } - - @Override - public void setReplyTo(Destination arg0) { - // return null; - } - - @Override - public void setReplyToSuffix(String arg0) { - // return null; - } - - @Override - public void setSendTimestamp(long arg0) { - // return null; - } - - @Override - public void setSenderID(String arg0) { - // return null; - } - - @Override - public void setSenderId(String arg0) { - // return null; - } - - @Override - public void setSenderTimestamp(long arg0) { - // return null; - } - - @Override - public void setSequenceNumber(long arg0) { - // return null; - } - - @Override - public void setStructuredMsg(boolean arg0) { - // return null; - } - - @Override - public void setStructuredMsgType(byte arg0) { - // return null; - } - - @Override - public void setTimeToLive(long arg0) { - // return null; - } - - @Override - public void setTopicNameLocation(int arg0, int arg1) { - // return null; - } - - @Override - public void setUserData(byte[] arg0) { - // return null; - } - - @Override - public void settle(Outcome arg0) throws JCSMPException { - // return null; - } - - @Override - public int writeAttachment(byte[] arg0) { - return 0; - } - - @Override - public int writeAttachment(InputStream arg0) throws IOException { - return 0; - } - - @Override - public int writeAttachment(byte[] arg0, int arg1, int arg2) - throws BufferUnderflowException { - return 0; - } - - @Override - public int writeBinaryMetadataBytes(int arg0, byte[] arg1) { - return 0; - } - - @Override - public int writeBinaryMetadataBytes(int arg0, byte[] arg1, int arg2, int arg3) - throws BufferUnderflowException { - // TOD0 Auto-generated method stub - return 0; - } - - @Override - public int writeNewAttachment(byte[] arg0) { - return 0; - } - - @Override - public int writeNewAttachment(InputStream arg0) throws IOException { - return 0; - } - - @Override - public int writeNewAttachment(byte[] arg0, int arg1, int arg2) - throws BufferUnderflowException { - return 0; - } - - @Override - public int writeNewAttachment(InputStream arg0, int arg1, int arg2) throws IOException { - return 0; - } - }; + @Override + public String toString() { + return "SimpleRecord{" + + "payload='" + + payload + + '\'' + + ", messageId='" + + messageId + + '\'' + + '}'; } + } + + public static Solace.Record getSolaceRecord(String payload, String messageId) { + return Solace.Record.builder() + .setPayload(payload.getBytes(StandardCharsets.UTF_8)) + .setMessageId(messageId) + .setDestination( + Solace.Destination.builder() + .setName("destination-topic") + .setType(Solace.DestinationType.TOPIC) + .build()) + .setExpiration(1000L) + .setPriority(0) + .setReceiveTimestamp(1708100477067L) + .setRedelivered(false) + .setReplyTo(null) + .setSequenceNumber(null) + .setTimeToLive(1000L) + .setSenderTimestamp(null) + .build(); + } + + public static BytesXMLMessage getBytesXmlMessage(String payload) { + return getBytesXmlMessage(payload, "messageId", null); + } + + public static BytesXMLMessage getBytesXmlMessage(String payload, String messageId) { + return getBytesXmlMessage(payload, messageId, null); + } + + public static BytesXMLMessage getBytesXmlMessage( + String payload, String messageId, SerializableFunction ackMessageFn) { + long receiverTimestamp = 1708100477067L; + long expiration = 1000L; + long timeToLive = 1000L; + String destination = "destination-topic"; + + return new BytesXMLMessage() { + + @Override + public byte[] getBytes() { + return payload.getBytes(StandardCharsets.UTF_8); + } + + @Override + public int getContentLength() { + return payload.getBytes(StandardCharsets.UTF_8).length; + } + + @Override + public int readBytes(byte[] arg0) { + return 0; + } + + @Override + public int readBytes(byte[] arg0, int arg1) { + return 0; + } + + @Override + public void rewindContent() { + // return null; + } + + @Override + public void writeBytes(byte[] arg0) { + // return null; + } + + @Override + public void writeBytes(byte[] arg0, int arg1, int arg2) { + // return null; + } + + @Override + public void ackMessage() { + if (ackMessageFn != null) { + ackMessageFn.apply(0); + } + } + + @Override + public void clearAttachment() { + // return null; + } + + @Override + public void clearBinaryMetadataBytes(int arg0) { + // return null; + } + + @Override + public void clearContent() { + // return null; + } + + @Override + public void clearQueueNameLocation() { + // return null; + } + + @Override + public void clearTopicNameLocation() { + // return null; + } + + @Override + public String dump() { + return null; + } + + @Override + public String dump(int arg0) { + return null; + } + + @Override + public long getAckMessageId() { + return 0; + } + + @Override + public String getAppMessageID() { + return null; + } + + @Override + public String getAppMessageType() { + return null; + } + + @Override + public String getApplicationMessageId() { + return messageId; + } + + @Override + public String getApplicationMessageType() { + return null; + } + + @Override + public ByteBuffer getAttachmentByteBuffer() { + return null; + } + + @Override + public int getAttachmentContentLength() { + return 0; + } + + @Override + public int getBinaryMetadataContentLength(int arg0) { + return 0; + } + + @Override + public Collection getBinaryMetadataTypes() { + return null; + } + + @Override + public Long getCacheRequestId() { + return null; + } + + @Override + public List getConsumerIdList() { + return null; + } + + @Override + public String getCorrelationId() { + return null; + } + + @Override + public Object getCorrelationKey() { + return null; + } + + @Override + public User_Cos getCos() { + return null; + } + + @Override + public boolean getDeliverToOne() { + return false; + } + + @Override + public int getDeliveryCount() throws UnsupportedOperationException { + return 0; + } + + @Override + public DeliveryMode getDeliveryMode() { + return null; + } + + @Override + public Destination getDestination() { + return SolaceIO.topicFromName(destination); + } + + @Override + public String getDestinationTopicSuffix() { + return null; + } + + @Override + public boolean getDiscardIndication() { + return false; + } + + @Override + public long getExpiration() { + return expiration; + } + + @Override + public String getHTTPContentEncoding() { + return null; + } + + @Override + public String getHTTPContentType() { + return null; + } + + @Override + public String getMessageId() { + return null; + } + + @Override + public long getMessageIdLong() { + return 0; + } + + @Override + public MessageType getMessageType() { + return null; + } + + @Override + public int getPriority() { + return 0; + } + + @Override + public SDTMap getProperties() { + return null; + } + + @Override + public int getQueueNameLength() { + return 0; + } + + @Override + public int getQueueNameOffset() { + return 0; + } + + @Override + public long getReceiveTimestamp() { + return receiverTimestamp; + } + + @Override + public boolean getRedelivered() { + return false; + } + + @Override + public ReplicationGroupMessageId getReplicationGroupMessageId() { + return null; + } + + @Override + public Destination getReplyTo() { + return null; + } + + @Override + public String getReplyToSuffix() { + return null; + } + + @Override + public Long getSendTimestamp() { + return null; + } + + @Override + public String getSenderID() { + return null; + } + + @Override + public String getSenderId() { + return null; + } + + @Override + public Long getSenderTimestamp() { + return null; + } + + @Override + public Long getSequenceNumber() { + return null; + } + + @Override + public byte getStructuredMsgType() { + return 0x2; + } + + @Override + public boolean getTQDiscardIndication() { + return false; + } + + @Override + public long getTimeToLive() { + return timeToLive; + } + + @Override + public int getTopicNameLength() { + return 5; + } + + @Override + public int getTopicNameOffset() { + return 0; + } + + @Override + public Long getTopicSequenceNumber() { + return null; + } + + @Override + public byte[] getUserData() { + return null; + } + + @Override + public boolean hasAttachment() { + return false; + } + + @Override + public boolean hasBinaryMetadata(int arg0) { + return false; + } + + @Override + public boolean hasContent() { + return false; + } + + @Override + public boolean hasUserData() { + return false; + } + + @Override + public boolean isAckImmediately() { + return false; + } + + @Override + public boolean isCacheMessage() { + return false; + } + + @Override + public boolean isDMQEligible() { + return false; + } + + @Override + public boolean isDeliveryCountSupported() { + return false; + } + + @Override + public boolean isElidingEligible() { + return false; + } + + @Override + public boolean isReadOnly() { + return false; + } + + @Override + public boolean isReplyMessage() { + return false; + } + + @Override + public boolean isStructuredMsg() { + return false; + } + + @Override + public boolean isSuspect() { + return false; + } + + @Override + public int readAttachmentBytes(byte[] arg0) { + return 0; + } + + @Override + public int readAttachmentBytes(byte[] arg0, int arg1) { + return 0; + } + + @Override + public int readAttachmentBytes(int arg0, byte[] arg1, int arg2, int arg3) { + return 0; + } + + @Override + public int readBinaryMetadataBytes(int arg0, byte[] arg1) { + return 0; + } + + @Override + public int readContentBytes(byte[] arg0) { + return 0; + } + + @Override + public int readContentBytes(byte[] arg0, int arg1) { + return 0; + } + + @Override + public int readContentBytes(int arg0, byte[] arg1, int arg2, int arg3) { + return 0; + } + + @Override + public void rejectMessage() { + // return null; + } + + @Override + public void reset() { + // return null; + } + + @Override + public void resetPayload() { + // return null; + } + + @Override + public void rewindAttachment() { + // return null; + } + + @Override + public void setAckImmediately(boolean arg0) { + // return null; + } + + @Override + public void setAppMessageID(String arg0) { + // return null; + } + + @Override + public void setAppMessageType(String arg0) { + // return null; + } + + @Override + public void setApplicationMessageId(String arg0) { + // return null; + } + + @Override + public void setApplicationMessageType(String arg0) { + // return null; + } + + @Override + public void setAsReplyMessage(boolean arg0) { + // return null; + } + + @Override + public void setCorrelationId(String arg0) { + // return null; + } + + @Override + public void setCorrelationKey(Object arg0) { + // return null; + } + + @Override + public void setCos(User_Cos arg0) { + // return null; + } + + @Override + public void setDMQEligible(boolean arg0) { + // return null; + } + + @Override + public void setDeliverToOne(boolean arg0) { + // return null; + } + + @Override + public void setDeliveryMode(DeliveryMode arg0) { + // return null; + } + + @Override + public void setElidingEligible(boolean arg0) { + // return null; + } + + @Override + public void setExpiration(long arg0) { + // return null; + } + + @Override + public void setHTTPContentEncoding(String arg0) { + // return null; + } + + @Override + public void setHTTPContentType(String arg0) { + // return null; + } + + @Override + public void setMessageType(MessageType arg0) { + // return null; + } + + @Override + public void setPriority(int arg0) { + // return null; + } + + @Override + public void setProperties(SDTMap arg0) { + // return null; + } + + @Override + public void setQueueNameLocation(int arg0, int arg1) { + // return null; + } + + @Override + public void setReadOnly() { + // return null; + } + + @Override + public void setReplyTo(Destination arg0) { + // return null; + } + + @Override + public void setReplyToSuffix(String arg0) { + // return null; + } + + @Override + public void setSendTimestamp(long arg0) { + // return null; + } + + @Override + public void setSenderID(String arg0) { + // return null; + } + + @Override + public void setSenderId(String arg0) { + // return null; + } + + @Override + public void setSenderTimestamp(long arg0) { + // return null; + } + + @Override + public void setSequenceNumber(long arg0) { + // return null; + } + + @Override + public void setStructuredMsg(boolean arg0) { + // return null; + } + + @Override + public void setStructuredMsgType(byte arg0) { + // return null; + } + + @Override + public void setTimeToLive(long arg0) { + // return null; + } + + @Override + public void setTopicNameLocation(int arg0, int arg1) { + // return null; + } + + @Override + public void setUserData(byte[] arg0) { + // return null; + } + + @Override + public void settle(Outcome arg0) throws JCSMPException { + // return null; + } + + @Override + public int writeAttachment(byte[] arg0) { + return 0; + } + + @Override + public int writeAttachment(InputStream arg0) throws IOException { + return 0; + } + + @Override + public int writeAttachment(byte[] arg0, int arg1, int arg2) throws BufferUnderflowException { + return 0; + } + + @Override + public int writeBinaryMetadataBytes(int arg0, byte[] arg1) { + return 0; + } + + @Override + public int writeBinaryMetadataBytes(int arg0, byte[] arg1, int arg2, int arg3) + throws BufferUnderflowException { + // TOD0 Auto-generated method stub + return 0; + } + + @Override + public int writeNewAttachment(byte[] arg0) { + return 0; + } + + @Override + public int writeNewAttachment(InputStream arg0) throws IOException { + return 0; + } + + @Override + public int writeNewAttachment(byte[] arg0, int arg1, int arg2) + throws BufferUnderflowException { + return 0; + } + + @Override + public int writeNewAttachment(InputStream arg0, int arg1, int arg2) throws IOException { + return 0; + } + }; + } } diff --git a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/SolaceIOTest.java b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/SolaceIOTest.java index 594c9277c8ff..b5d37572533e 100644 --- a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/SolaceIOTest.java +++ b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/SolaceIOTest.java @@ -1,11 +1,13 @@ /* - * Copyright 2024 Google. + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -13,21 +15,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.io.solace.io.solace; +package org.apache.beam.sdk.io.solace; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertThrows; import static org.junit.Assert.assertTrue; -import org.apache.beam.sdk.io.solace.SolaceDataUtils.SimpleRecord; -import org.apache.beam.sdk.io.solace.SolaceIO; -import org.apache.beam.sdk.io.solace.SolaceIO.Read; -import org.apache.beam.sdk.io.solace.broker.SessionServiceFactory; -import org.apache.beam.sdk.io.solace.data.Solace; -import org.apache.beam.sdk.io.solace.data.Solace.Record; -import org.apache.beam.sdk.io.solace.read.SolaceCheckpointMark; -import org.apache.beam.sdk.io.solace.read.UnboundedSolaceSource; import com.solacesystems.jcsmp.BytesXMLMessage; import com.solacesystems.jcsmp.Destination; import com.solacesystems.jcsmp.Queue; @@ -40,6 +34,13 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.io.UnboundedSource.CheckpointMark; import org.apache.beam.sdk.io.UnboundedSource.UnboundedReader; +import org.apache.beam.sdk.io.solace.SolaceDataUtils.SimpleRecord; +import org.apache.beam.sdk.io.solace.SolaceIO.Read; +import org.apache.beam.sdk.io.solace.broker.SessionServiceFactory; +import org.apache.beam.sdk.io.solace.data.Solace; +import org.apache.beam.sdk.io.solace.data.Solace.Record; +import org.apache.beam.sdk.io.solace.read.SolaceCheckpointMark; +import org.apache.beam.sdk.io.solace.read.UnboundedSolaceSource; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.testing.CoderProperties; import org.apache.beam.sdk.testing.PAssert; @@ -48,6 +49,7 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TypeDescriptor; import org.apache.beam.sdk.values.TypeDescriptors; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.joda.time.Instant; import org.junit.Rule; import org.junit.Test; @@ -57,551 +59,522 @@ @RunWith(JUnit4.class) public class SolaceIOTest { - @Rule public final transient TestPipeline pipeline = TestPipeline.create(); - - private static BytesXMLMessage getOrNull(Integer index, List messages) { - return index != null && index < messages.size() ? messages.get(index) : null; - } - - private static MockSempClientFactory getMockSempClientFactory() { - return new MockSempClientFactory(MockSempClient.builder().build()); - } - - private static UnboundedSolaceSource getSource( - Read spec, Queue queue, TestPipeline pipeline) { - return new UnboundedSolaceSource<>( - queue, - spec.getSempClientFactory(), - spec.getSessionServiceFactory(), - spec.getMaxNumConnections(), - spec.getDeduplicateRecords(), - spec.inferCoder(pipeline), - spec.getTimestampFn(), - spec.getParseFn()); - } - - @Test - public void testReadMessages() { - // Broker that creates input data - MockSessionService mockClientService = - new MockSessionService( - index -> { - List messages = - List.of( - SolaceDataUtils.getBytesXmlMessage( - "payload_test0", "450"), - SolaceDataUtils.getBytesXmlMessage( - "payload_test1", "451"), - SolaceDataUtils.getBytesXmlMessage( - "payload_test2", "452")); - return getOrNull(index, messages); - }, - 3); - - SessionServiceFactory fakeSessionServiceFactory = - new MockSessionServiceFactory(mockClientService); - - // Expected data - List inputs = new ArrayList<>(); - inputs.add(SolaceDataUtils.getSolaceRecord("payload_test0", "450")); - inputs.add(SolaceDataUtils.getSolaceRecord("payload_test1", "451")); - inputs.add(SolaceDataUtils.getSolaceRecord("payload_test2", "452")); - - // Run the pipeline - PCollection events = - pipeline.apply( - "Read from Solace", - SolaceIO.read() - .from(Solace.Queue.fromName("queue")) - .withSempClientFactory(getMockSempClientFactory()) - .withSessionServiceFactory(fakeSessionServiceFactory) - .withMaxNumConnections(2)); - - // Assert results - PAssert.that(events).containsInAnyOrder(inputs); - pipeline.run(); - } - - @Test - public void testReadMessagesWithDeduplication() { - // Broker that creates input data - MockSessionService mockClientService = - new MockSessionService( - index -> { - List messages = - List.of( - SolaceDataUtils.getBytesXmlMessage( - "payload_test0", "450"), - SolaceDataUtils.getBytesXmlMessage( - "payload_test1", "451"), - SolaceDataUtils.getBytesXmlMessage( - "payload_test2", "451")); - return getOrNull(index, messages); - }, - 3); - - SessionServiceFactory fakeSessionServiceFactory = - new MockSessionServiceFactory(mockClientService); - - // Expected data - List inputs = new ArrayList<>(); - inputs.add(SolaceDataUtils.getSolaceRecord("payload_test0", "450")); - inputs.add(SolaceDataUtils.getSolaceRecord("payload_test1", "451")); - - // Run the pipeline - PCollection events = - pipeline.apply( - "Read from Solace", - SolaceIO.read() - .from(Solace.Queue.fromName("queue")) - .withSempClientFactory(getMockSempClientFactory()) - .withSessionServiceFactory(fakeSessionServiceFactory) - .withMaxNumConnections(2)); - // Assert results - PAssert.that(events).containsInAnyOrder(inputs); - pipeline.run(); - } - - @Test - public void testReadMessagesWithoutDeduplication() { - // Broker that creates input data - MockSessionService mockClientService = - new MockSessionService( - index -> { - List messages = - List.of( - SolaceDataUtils.getBytesXmlMessage( - "payload_test0", "450"), - SolaceDataUtils.getBytesXmlMessage( - "payload_test1", "451"), - SolaceDataUtils.getBytesXmlMessage( - "payload_test2", "451")); - return getOrNull(index, messages); - }, - 3); - SessionServiceFactory fakeSessionServiceFactory = - new MockSessionServiceFactory(mockClientService); - - // Expected data - List inputs = new ArrayList<>(); - inputs.add(SolaceDataUtils.getSolaceRecord("payload_test0", "450")); - inputs.add(SolaceDataUtils.getSolaceRecord("payload_test1", "451")); - inputs.add(SolaceDataUtils.getSolaceRecord("payload_test2", "451")); - - // Run the pipeline - PCollection events = - pipeline.apply( - "Read from Solace", - SolaceIO.read() - .from(Solace.Queue.fromName("queue")) - .withSempClientFactory(getMockSempClientFactory()) - .withSessionServiceFactory(fakeSessionServiceFactory) - .withMaxNumConnections(1) - .withDeduplicateRecords(false)); - - // Assert results - PAssert.that(events).containsInAnyOrder(inputs); - pipeline.run(); + @Rule public final transient TestPipeline pipeline = TestPipeline.create(); + + private static BytesXMLMessage getOrNull(Integer index, List messages) { + return index != null && index < messages.size() ? messages.get(index) : null; + } + + private static MockSempClientFactory getMockSempClientFactory() { + return new MockSempClientFactory(MockSempClient.builder().build()); + } + + private static UnboundedSolaceSource getSource( + Read spec, Queue queue, TestPipeline pipeline) { + return new UnboundedSolaceSource<>( + queue, + spec.getSempClientFactory(), + spec.getSessionServiceFactory(), + spec.getMaxNumConnections(), + spec.getDeduplicateRecords(), + spec.inferCoder(pipeline), + spec.getTimestampFn(), + spec.getParseFn()); + } + + @Test + public void testReadMessages() { + // Broker that creates input data + MockSessionService mockClientService = + new MockSessionService( + index -> { + List messages = + ImmutableList.of( + SolaceDataUtils.getBytesXmlMessage("payload_test0", "450"), + SolaceDataUtils.getBytesXmlMessage("payload_test1", "451"), + SolaceDataUtils.getBytesXmlMessage("payload_test2", "452")); + return getOrNull(index, messages); + }, + 3); + + SessionServiceFactory fakeSessionServiceFactory = + new MockSessionServiceFactory(mockClientService); + + // Expected data + List inputs = new ArrayList<>(); + inputs.add(SolaceDataUtils.getSolaceRecord("payload_test0", "450")); + inputs.add(SolaceDataUtils.getSolaceRecord("payload_test1", "451")); + inputs.add(SolaceDataUtils.getSolaceRecord("payload_test2", "452")); + + // Run the pipeline + PCollection events = + pipeline.apply( + "Read from Solace", + SolaceIO.read() + .from(Solace.Queue.fromName("queue")) + .withSempClientFactory(getMockSempClientFactory()) + .withSessionServiceFactory(fakeSessionServiceFactory) + .withMaxNumConnections(2)); + + // Assert results + PAssert.that(events).containsInAnyOrder(inputs); + pipeline.run(); + } + + @Test + public void testReadMessagesWithDeduplication() { + // Broker that creates input data + MockSessionService mockClientService = + new MockSessionService( + index -> { + List messages = + ImmutableList.of( + SolaceDataUtils.getBytesXmlMessage("payload_test0", "450"), + SolaceDataUtils.getBytesXmlMessage("payload_test1", "451"), + SolaceDataUtils.getBytesXmlMessage("payload_test2", "451")); + return getOrNull(index, messages); + }, + 3); + + SessionServiceFactory fakeSessionServiceFactory = + new MockSessionServiceFactory(mockClientService); + + // Expected data + List inputs = new ArrayList<>(); + inputs.add(SolaceDataUtils.getSolaceRecord("payload_test0", "450")); + inputs.add(SolaceDataUtils.getSolaceRecord("payload_test1", "451")); + + // Run the pipeline + PCollection events = + pipeline.apply( + "Read from Solace", + SolaceIO.read() + .from(Solace.Queue.fromName("queue")) + .withSempClientFactory(getMockSempClientFactory()) + .withSessionServiceFactory(fakeSessionServiceFactory) + .withMaxNumConnections(2)); + // Assert results + PAssert.that(events).containsInAnyOrder(inputs); + pipeline.run(); + } + + @Test + public void testReadMessagesWithoutDeduplication() { + // Broker that creates input data + MockSessionService mockClientService = + new MockSessionService( + index -> { + List messages = + ImmutableList.of( + SolaceDataUtils.getBytesXmlMessage("payload_test0", "450"), + SolaceDataUtils.getBytesXmlMessage("payload_test1", "451"), + SolaceDataUtils.getBytesXmlMessage("payload_test2", "451")); + return getOrNull(index, messages); + }, + 3); + SessionServiceFactory fakeSessionServiceFactory = + new MockSessionServiceFactory(mockClientService); + + // Expected data + List inputs = new ArrayList<>(); + inputs.add(SolaceDataUtils.getSolaceRecord("payload_test0", "450")); + inputs.add(SolaceDataUtils.getSolaceRecord("payload_test1", "451")); + inputs.add(SolaceDataUtils.getSolaceRecord("payload_test2", "451")); + + // Run the pipeline + PCollection events = + pipeline.apply( + "Read from Solace", + SolaceIO.read() + .from(Solace.Queue.fromName("queue")) + .withSempClientFactory(getMockSempClientFactory()) + .withSessionServiceFactory(fakeSessionServiceFactory) + .withMaxNumConnections(1) + .withDeduplicateRecords(false)); + + // Assert results + PAssert.that(events).containsInAnyOrder(inputs); + pipeline.run(); + } + + @Test + public void testReadWithCoderAndParseFnAndTimestampFn() { + // Broker that creates input data + MockSessionService mockClientService = + new MockSessionService( + index -> { + List messages = + ImmutableList.of( + SolaceDataUtils.getBytesXmlMessage("payload_test0", "450"), + SolaceDataUtils.getBytesXmlMessage("payload_test1", "451"), + SolaceDataUtils.getBytesXmlMessage("payload_test2", "452")); + return getOrNull(index, messages); + }, + 3); + SessionServiceFactory fakeSessionServiceFactory = + new MockSessionServiceFactory(mockClientService); + + // Expected data + List inputs = new ArrayList<>(); + inputs.add(new SimpleRecord("payload_test0", "450")); + inputs.add(new SimpleRecord("payload_test1", "451")); + inputs.add(new SimpleRecord("payload_test2", "452")); + + // Run the pipeline + PCollection events = + pipeline.apply( + "Read from Solace", + SolaceIO.read( + TypeDescriptor.of(SimpleRecord.class), + input -> + new SimpleRecord( + new String(input.getBytes(), StandardCharsets.UTF_8), + input.getApplicationMessageId()), + input -> Instant.ofEpochMilli(1708100477061L)) + .from(Solace.Queue.fromName("queue")) + .withSempClientFactory(getMockSempClientFactory()) + .withSessionServiceFactory(fakeSessionServiceFactory) + .withMaxNumConnections(2)); + + // Assert results + PAssert.that(events).containsInAnyOrder(inputs); + pipeline.run(); + } + + @Test + public void testNoQueueAndTopicSet() { + Read spec = SolaceIO.read(); + assertThrows(IllegalStateException.class, () -> spec.expand(pipeline.begin())); + } + + @Test + public void testSplitsForExclusiveQueue() throws Exception { + MockSempClient mockSempClient = + MockSempClient.builder().setIsQueueNonExclusiveFn((q) -> false).build(); + + Read spec = + SolaceIO.read() + .from(Solace.Queue.fromName("queue")) + .withSempClientFactory(new MockSempClientFactory(mockSempClient)); + + int desiredNumSplits = 5; + + UnboundedSolaceSource initialSource = getSource(spec, spec.getQueue(), pipeline); + List> splits = + initialSource.split(desiredNumSplits, PipelineOptionsFactory.create()); + assertEquals(1, splits.size()); + } + + @Test + public void testSplitsForNonExclusiveQueueWithMaxNumConnections() throws Exception { + Read spec = + SolaceIO.read() + .from(Solace.Queue.fromName("queue")) + .withSempClientFactory(getMockSempClientFactory()) + .withMaxNumConnections(3); + + int desiredNumSplits = 5; + + UnboundedSolaceSource initialSource = getSource(spec, spec.getQueue(), pipeline); + List> splits = + initialSource.split(desiredNumSplits, PipelineOptionsFactory.create()); + assertEquals(3, splits.size()); + } + + @Test + public void testSplitsForNonExclusiveQueueWithMaxNumConnectionsRespectDesired() throws Exception { + Read spec = + SolaceIO.read() + .from(Solace.Queue.fromName("queue")) + .withSempClientFactory(getMockSempClientFactory()) + .withMaxNumConnections(10); + + int desiredNumSplits = 5; + + UnboundedSolaceSource initialSource = getSource(spec, spec.getQueue(), pipeline); + List> splits = + initialSource.split(desiredNumSplits, PipelineOptionsFactory.create()); + assertEquals(5, splits.size()); + } + + @Test + public void testCreateQueueForTopic() throws Exception { + AtomicInteger createQueueForTopicFnCounter = new AtomicInteger(0); + MockSempClient mockSempClient = + MockSempClient.builder() + .setCreateQueueForTopicFn((q) -> createQueueForTopicFnCounter.incrementAndGet()) + .build(); + + Read spec = + SolaceIO.read() + .from(Solace.Topic.fromName("topic")) + .withSempClientFactory(new MockSempClientFactory(mockSempClient)); + + UnboundedSolaceSource initialSource = + getSource( + spec, spec.initializeQueueForTopic("some-job", spec.getSempClientFactory()), pipeline); + initialSource.split(2, PipelineOptionsFactory.create()); + + // check if createQueueForTopic was executed + assertEquals(1, createQueueForTopicFnCounter.get()); + } + + @Test + public void testCheckpointMark() throws Exception { + AtomicInteger countConsumedMessages = new AtomicInteger(0); + AtomicInteger countAckMessages = new AtomicInteger(0); + + // Broker that creates input data + MockSessionService mockClientService = + new MockSessionService( + index -> { + List messages = new ArrayList<>(); + for (int i = 0; i < 10; i++) { + messages.add( + SolaceDataUtils.getBytesXmlMessage( + "payload_test" + i, "45" + i, (num) -> countAckMessages.incrementAndGet())); + } + countConsumedMessages.incrementAndGet(); + return getOrNull(index, messages); + }, + 10); + + SessionServiceFactory fakeSessionServiceFactory = + new MockSessionServiceFactory(mockClientService); + Read spec = + SolaceIO.read() + .from(Solace.Queue.fromName("queue")) + .withSempClientFactory(getMockSempClientFactory()) + .withSessionServiceFactory(fakeSessionServiceFactory) + .withMaxNumConnections(4); + + UnboundedSolaceSource initialSource = getSource(spec, spec.getQueue(), pipeline); + UnboundedReader reader = + initialSource.createReader(PipelineOptionsFactory.create(), null); + + // start the reader and move to the first record + assertTrue(reader.start()); + + // consume 3 messages (NB: start already consumed the first message) + for (int i = 0; i < 3; i++) { + assertTrue(String.format("Failed at %d-th message", i), reader.advance()); } - @Test - public void testReadWithCoderAndParseFnAndTimestampFn() { - // Broker that creates input data - MockSessionService mockClientService = - new MockSessionService( - index -> { - List messages = - List.of( - SolaceDataUtils.getBytesXmlMessage( - "payload_test0", "450"), - SolaceDataUtils.getBytesXmlMessage( - "payload_test1", "451"), - SolaceDataUtils.getBytesXmlMessage( - "payload_test2", "452")); - return getOrNull(index, messages); - }, - 3); - SessionServiceFactory fakeSessionServiceFactory = - new MockSessionServiceFactory(mockClientService); - - // Expected data - List inputs = new ArrayList<>(); - inputs.add(new SimpleRecord("payload_test0", "450")); - inputs.add(new SimpleRecord("payload_test1", "451")); - inputs.add(new SimpleRecord("payload_test2", "452")); - - // Run the pipeline - PCollection events = - pipeline.apply( - "Read from Solace", - SolaceIO.read( - TypeDescriptor.of(SimpleRecord.class), - input -> - new SimpleRecord( - new String( - input.getBytes(), - StandardCharsets.UTF_8), - input.getApplicationMessageId()), - input -> Instant.ofEpochMilli(1708100477061L)) - .from(Solace.Queue.fromName("queue")) - .withSempClientFactory(getMockSempClientFactory()) - .withSessionServiceFactory(fakeSessionServiceFactory) - .withMaxNumConnections(2)); - - // Assert results - PAssert.that(events).containsInAnyOrder(inputs); - pipeline.run(); + // check if 4 messages were consumed + assertEquals(4, countConsumedMessages.get()); + + // check if no messages were acknowledged yet + assertEquals(0, countAckMessages.get()); + + // finalize the checkpoint + reader.getCheckpointMark().finalizeCheckpoint(); + + // check if messages were acknowledged + assertEquals(4, countAckMessages.get()); + } + + @Test + public void testCheckpointMarkAndFinalizeSeparately() throws Exception { + AtomicInteger countConsumedMessages = new AtomicInteger(0); + AtomicInteger countAckMessages = new AtomicInteger(0); + + // Broker that creates input data + MockSessionService mockClientService = + new MockSessionService( + index -> { + List messages = new ArrayList<>(); + for (int i = 0; i < 10; i++) { + messages.add( + SolaceDataUtils.getBytesXmlMessage( + "payload_test" + i, "45" + i, (num) -> countAckMessages.incrementAndGet())); + } + countConsumedMessages.incrementAndGet(); + return getOrNull(index, messages); + }, + 10); + SessionServiceFactory fakeSessionServiceFactory = + new MockSessionServiceFactory(mockClientService); + + Read spec = + SolaceIO.read() + .from(Solace.Queue.fromName("queue")) + .withSempClientFactory(getMockSempClientFactory()) + .withSessionServiceFactory(fakeSessionServiceFactory) + .withMaxNumConnections(4); + + UnboundedSolaceSource initialSource = getSource(spec, spec.getQueue(), pipeline); + + UnboundedReader reader = + initialSource.createReader(PipelineOptionsFactory.create(), null); + + // start the reader and move to the first record + assertTrue(reader.start()); + + // consume 3 messages (NB: start already consumed the first message) + for (int i = 0; i < 3; i++) { + assertTrue(String.format("Failed at %d-th message", i), reader.advance()); } - @Test - public void testNoQueueAndTopicSet() { - Read spec = SolaceIO.read(); - assertThrows(IllegalStateException.class, () -> spec.expand(pipeline.begin())); + // create checkpoint but don't finalize yet + CheckpointMark checkpointMark = reader.getCheckpointMark(); + + // consume 2 more messages + reader.advance(); + reader.advance(); + + // check if messages are still not acknowledged + assertEquals(0, countAckMessages.get()); + + // acknowledge from the first checkpoint + checkpointMark.finalizeCheckpoint(); + + // only messages from the first checkpoint are acknowledged + assertEquals(4, countAckMessages.get()); + } + + @Test + public void testCheckpointMarkSafety() throws Exception { + + final int messagesToProcess = 100; + + AtomicInteger countConsumedMessages = new AtomicInteger(0); + AtomicInteger countAckMessages = new AtomicInteger(0); + + // Broker that creates input data + MockSessionService mockClientService = + new MockSessionService( + index -> { + List messages = new ArrayList<>(); + for (int i = 0; i < messagesToProcess; i++) { + messages.add( + SolaceDataUtils.getBytesXmlMessage( + "payload_test" + i, "45" + i, (num) -> countAckMessages.incrementAndGet())); + } + countConsumedMessages.incrementAndGet(); + return getOrNull(index, messages); + }, + 10); + + SessionServiceFactory fakeSessionServiceFactory = + new MockSessionServiceFactory(mockClientService); + Read spec = + SolaceIO.read() + .from(Solace.Queue.fromName("queue")) + .withSempClientFactory(getMockSempClientFactory()) + .withSessionServiceFactory(fakeSessionServiceFactory) + .withMaxNumConnections(4); + + UnboundedSolaceSource initialSource = getSource(spec, spec.getQueue(), pipeline); + + UnboundedReader reader = + initialSource.createReader(PipelineOptionsFactory.create(), null); + + // start the reader and move to the first record + assertTrue(reader.start()); + + // consume half the messages (NB: start already consumed the first message) + for (int i = 0; i < (messagesToProcess / 2) - 1; i++) { + assertTrue(reader.advance()); } - @Test - public void testSplitsForExclusiveQueue() throws Exception { - MockSempClient mockSempClient = - MockSempClient.builder().setIsQueueNonExclusiveFn((q) -> false).build(); - - Read spec = - SolaceIO.read() - .from(Solace.Queue.fromName("queue")) - .withSempClientFactory(new MockSempClientFactory(mockSempClient)); - - int desiredNumSplits = 5; - - UnboundedSolaceSource initialSource = getSource(spec, spec.getQueue(), pipeline); - List> splits = - initialSource.split(desiredNumSplits, PipelineOptionsFactory.create()); - assertEquals(1, splits.size()); - } - - @Test - public void testSplitsForNonExclusiveQueueWithMaxNumConnections() throws Exception { - Read spec = - SolaceIO.read() - .from(Solace.Queue.fromName("queue")) - .withSempClientFactory(getMockSempClientFactory()) - .withMaxNumConnections(3); - - int desiredNumSplits = 5; - - UnboundedSolaceSource initialSource = getSource(spec, spec.getQueue(), pipeline); - List> splits = - initialSource.split(desiredNumSplits, PipelineOptionsFactory.create()); - assertEquals(3, splits.size()); - } - - @Test - public void testSplitsForNonExclusiveQueueWithMaxNumConnectionsRespectDesired() - throws Exception { - Read spec = - SolaceIO.read() - .from(Solace.Queue.fromName("queue")) - .withSempClientFactory(getMockSempClientFactory()) - .withMaxNumConnections(10); - - int desiredNumSplits = 5; - - UnboundedSolaceSource initialSource = getSource(spec, spec.getQueue(), pipeline); - List> splits = - initialSource.split(desiredNumSplits, PipelineOptionsFactory.create()); - assertEquals(5, splits.size()); - } - - @Test - public void testCreateQueueForTopic() throws Exception { - AtomicInteger createQueueForTopicFnCounter = new AtomicInteger(0); - MockSempClient mockSempClient = - MockSempClient.builder() - .setCreateQueueForTopicFn( - (q) -> createQueueForTopicFnCounter.incrementAndGet()) - .build(); - - Read spec = - SolaceIO.read() - .from(Solace.Topic.fromName("topic")) - .withSempClientFactory(new MockSempClientFactory(mockSempClient)); - - UnboundedSolaceSource initialSource = - getSource( - spec, - spec.initializeQueueForTopic("some-job", spec.getSempClientFactory()), - pipeline); - initialSource.split(2, PipelineOptionsFactory.create()); - - // check if createQueueForTopic was executed - assertEquals(1, createQueueForTopicFnCounter.get()); - } - - @Test - public void testCheckpointMark() throws Exception { - AtomicInteger countConsumedMessages = new AtomicInteger(0); - AtomicInteger countAckMessages = new AtomicInteger(0); - - // Broker that creates input data - MockSessionService mockClientService = - new MockSessionService( - index -> { - List messages = new ArrayList<>(); - for (int i = 0; i < 10; i++) { - messages.add( - SolaceDataUtils.getBytesXmlMessage( - "payload_test" + i, - "45" + i, - (num) -> countAckMessages.incrementAndGet())); - } - countConsumedMessages.incrementAndGet(); - return getOrNull(index, messages); - }, - 10); - - SessionServiceFactory fakeSessionServiceFactory = - new MockSessionServiceFactory(mockClientService); - Read spec = - SolaceIO.read() - .from(Solace.Queue.fromName("queue")) - .withSempClientFactory(getMockSempClientFactory()) - .withSessionServiceFactory(fakeSessionServiceFactory) - .withMaxNumConnections(4); - - UnboundedSolaceSource initialSource = getSource(spec, spec.getQueue(), pipeline); - UnboundedReader reader = - initialSource.createReader(PipelineOptionsFactory.create(), null); - - // start the reader and move to the first record - assertTrue(reader.start()); - - // consume 3 messages (NB: start already consumed the first message) - for (int i = 0; i < 3; i++) { - assertTrue(String.format("Failed at %d-th message", i), reader.advance()); - } - - // check if 4 messages were consumed - assertEquals(4, countConsumedMessages.get()); - - // check if no messages were acknowledged yet - assertEquals(0, countAckMessages.get()); - - // finalize the checkpoint - reader.getCheckpointMark().finalizeCheckpoint(); - - // check if messages were acknowledged - assertEquals(4, countAckMessages.get()); - } - - @Test - public void testCheckpointMarkAndFinalizeSeparately() throws Exception { - AtomicInteger countConsumedMessages = new AtomicInteger(0); - AtomicInteger countAckMessages = new AtomicInteger(0); - - // Broker that creates input data - MockSessionService mockClientService = - new MockSessionService( - index -> { - List messages = new ArrayList<>(); - for (int i = 0; i < 10; i++) { - messages.add( - SolaceDataUtils.getBytesXmlMessage( - "payload_test" + i, - "45" + i, - (num) -> countAckMessages.incrementAndGet())); - } - countConsumedMessages.incrementAndGet(); - return getOrNull(index, messages); - }, - 10); - SessionServiceFactory fakeSessionServiceFactory = - new MockSessionServiceFactory(mockClientService); - - Read spec = - SolaceIO.read() - .from(Solace.Queue.fromName("queue")) - .withSempClientFactory(getMockSempClientFactory()) - .withSessionServiceFactory(fakeSessionServiceFactory) - .withMaxNumConnections(4); - - UnboundedSolaceSource initialSource = getSource(spec, spec.getQueue(), pipeline); - - UnboundedReader reader = - initialSource.createReader(PipelineOptionsFactory.create(), null); - - // start the reader and move to the first record - assertTrue(reader.start()); - - // consume 3 messages (NB: start already consumed the first message) - for (int i = 0; i < 3; i++) { - assertTrue(String.format("Failed at %d-th message", i), reader.advance()); - } - - // create checkpoint but don't finalize yet - CheckpointMark checkpointMark = reader.getCheckpointMark(); - - // consume 2 more messages - reader.advance(); - reader.advance(); - - // check if messages are still not acknowledged - assertEquals(0, countAckMessages.get()); - - // acknowledge from the first checkpoint - checkpointMark.finalizeCheckpoint(); - - // only messages from the first checkpoint are acknowledged - assertEquals(4, countAckMessages.get()); - } - - @Test - public void testCheckpointMarkSafety() throws Exception { - - final int messagesToProcess = 100; - - AtomicInteger countConsumedMessages = new AtomicInteger(0); - AtomicInteger countAckMessages = new AtomicInteger(0); - - // Broker that creates input data - MockSessionService mockClientService = - new MockSessionService( - index -> { - List messages = new ArrayList<>(); - for (int i = 0; i < messagesToProcess; i++) { - messages.add( - SolaceDataUtils.getBytesXmlMessage( - "payload_test" + i, - "45" + i, - (num) -> countAckMessages.incrementAndGet())); - } - countConsumedMessages.incrementAndGet(); - return getOrNull(index, messages); - }, - 10); - - SessionServiceFactory fakeSessionServiceFactory = - new MockSessionServiceFactory(mockClientService); - Read spec = - SolaceIO.read() - .from(Solace.Queue.fromName("queue")) - .withSempClientFactory(getMockSempClientFactory()) - .withSessionServiceFactory(fakeSessionServiceFactory) - .withMaxNumConnections(4); - - UnboundedSolaceSource initialSource = getSource(spec, spec.getQueue(), pipeline); - - UnboundedReader reader = - initialSource.createReader(PipelineOptionsFactory.create(), null); - - // start the reader and move to the first record - assertTrue(reader.start()); - - // consume half the messages (NB: start already consumed the first message) - for (int i = 0; i < (messagesToProcess / 2) - 1; i++) { - assertTrue(reader.advance()); - } - - // the messages are still pending in the queue (no ACK yet) - assertEquals(0, countAckMessages.get()); - - // we finalize the checkpoint for the already-processed messages while simultaneously - // consuming the remainder of messages from the queue - Thread runner = - new Thread( - () -> { - try { - for (int i = 0; i < messagesToProcess / 2; i++) { - assertTrue(reader.advance()); - } - } catch (IOException ex) { - throw new RuntimeException(ex); - } - }); - runner.start(); - reader.getCheckpointMark().finalizeCheckpoint(); - - // Concurrency issues would cause an exception to be thrown before this method exits, - // failing the test - runner.join(); - } - - @Test - public void testCheckpointMarkDefaultCoder() throws Exception { - SolaceCheckpointMark checkpointMark = new SolaceCheckpointMark(null, null); - Coder coder = - new UnboundedSolaceSource<>(null, null, null, 0, false, null, null, null) - .getCheckpointMarkCoder(); - CoderProperties.coderSerializable(coder); - CoderProperties.coderDecodeEncodeEqual(coder, checkpointMark); - } - - @Test - public void testDestinationTopicQueueCreation() { - String topicName = "some-topic"; - String queueName = "some-queue"; - Topic topic = SolaceIO.topicFromName(topicName); - Queue queue = SolaceIO.queueFromName(queueName); - - Destination dest = topic; - assertTrue(dest instanceof Topic); - assertFalse(dest instanceof Queue); - assertEquals(topicName, dest.getName()); - - dest = queue; - assertTrue(dest instanceof Queue); - assertFalse(dest instanceof Topic); - assertEquals(queueName, dest.getName()); - - Record r = SolaceDataUtils.getSolaceRecord("payload_test0", "450"); - dest = SolaceIO.convertToJcsmpDestination(r.getDestination()); - assertTrue(dest instanceof Topic); - assertFalse(dest instanceof Queue); - } - - @Test - public void testTopicEncoding() { - MockSessionService mockClientService = - new MockSessionService( - index -> { - List messages = - List.of( - SolaceDataUtils.getBytesXmlMessage( - "payload_test0", "450"), - SolaceDataUtils.getBytesXmlMessage( - "payload_test1", "451"), - SolaceDataUtils.getBytesXmlMessage( - "payload_test2", "452")); - return getOrNull(index, messages); - }, - 3); - - SessionServiceFactory fakeSessionServiceFactory = - new MockSessionServiceFactory(mockClientService); - - // Run - PCollection events = - pipeline.apply( - "Read from Solace", - SolaceIO.read() - .from(Solace.Queue.fromName("queue")) - .withSempClientFactory(getMockSempClientFactory()) - .withSessionServiceFactory(fakeSessionServiceFactory) - .withMaxNumConnections(2)); - - // Run the pipelin - PCollection destAreTopics = - events.apply( - MapElements.into(TypeDescriptors.booleans()) - .via( - r -> { - Destination dest = - SolaceIO.convertToJcsmpDestination( - r.getDestination()); - return dest instanceof Topic; - })); - - List inputs = List.of(true, true, true); - - // Assert results - PAssert.that(destAreTopics).containsInAnyOrder(inputs); - pipeline.run(); - } + // the messages are still pending in the queue (no ACK yet) + assertEquals(0, countAckMessages.get()); + + // we finalize the checkpoint for the already-processed messages while simultaneously + // consuming the remainder of messages from the queue + Thread runner = + new Thread( + () -> { + try { + for (int i = 0; i < messagesToProcess / 2; i++) { + assertTrue(reader.advance()); + } + } catch (IOException ex) { + throw new RuntimeException(ex); + } + }); + runner.start(); + reader.getCheckpointMark().finalizeCheckpoint(); + + // Concurrency issues would cause an exception to be thrown before this method exits, + // failing the test + runner.join(); + } + + @Test + public void testCheckpointMarkDefaultCoder() throws Exception { + SolaceCheckpointMark checkpointMark = new SolaceCheckpointMark(null, null); + Coder coder = + new UnboundedSolaceSource<>(null, null, null, 0, false, null, null, null) + .getCheckpointMarkCoder(); + CoderProperties.coderSerializable(coder); + CoderProperties.coderDecodeEncodeEqual(coder, checkpointMark); + } + + @Test + public void testDestinationTopicQueueCreation() { + String topicName = "some-topic"; + String queueName = "some-queue"; + Topic topic = SolaceIO.topicFromName(topicName); + Queue queue = SolaceIO.queueFromName(queueName); + + Destination dest = topic; + assertTrue(dest instanceof Topic); + assertFalse(dest instanceof Queue); + assertEquals(topicName, dest.getName()); + + dest = queue; + assertTrue(dest instanceof Queue); + assertFalse(dest instanceof Topic); + assertEquals(queueName, dest.getName()); + + Record r = SolaceDataUtils.getSolaceRecord("payload_test0", "450"); + dest = SolaceIO.convertToJcsmpDestination(r.getDestination()); + assertTrue(dest instanceof Topic); + assertFalse(dest instanceof Queue); + } + + @Test + public void testTopicEncoding() { + MockSessionService mockClientService = + new MockSessionService( + index -> { + List messages = + ImmutableList.of( + SolaceDataUtils.getBytesXmlMessage("payload_test0", "450"), + SolaceDataUtils.getBytesXmlMessage("payload_test1", "451"), + SolaceDataUtils.getBytesXmlMessage("payload_test2", "452")); + return getOrNull(index, messages); + }, + 3); + + SessionServiceFactory fakeSessionServiceFactory = + new MockSessionServiceFactory(mockClientService); + + // Run + PCollection events = + pipeline.apply( + "Read from Solace", + SolaceIO.read() + .from(Solace.Queue.fromName("queue")) + .withSempClientFactory(getMockSempClientFactory()) + .withSessionServiceFactory(fakeSessionServiceFactory) + .withMaxNumConnections(2)); + + // Run the pipelin + PCollection destAreTopics = + events.apply( + MapElements.into(TypeDescriptors.booleans()) + .via( + r -> { + Destination dest = SolaceIO.convertToJcsmpDestination(r.getDestination()); + return dest instanceof Topic; + })); + + List inputs = ImmutableList.of(true, true, true); + + // Assert results + PAssert.that(destAreTopics).containsInAnyOrder(inputs); + pipeline.run(); + } } diff --git a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/broker/SempBasicAuthClientExecutorTest.java b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/broker/SempBasicAuthClientExecutorTest.java index 11e457aa5652..b0038e428654 100644 --- a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/broker/SempBasicAuthClientExecutorTest.java +++ b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/broker/SempBasicAuthClientExecutorTest.java @@ -1,11 +1,13 @@ /* - * Copyright 2024 Google. + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -31,144 +33,141 @@ import com.google.common.collect.ImmutableList; import java.io.IOException; import java.util.List; -import org.apache.beam.sdk.io.solace.broker.SempBasicAuthClientExecutor; import org.junit.Test; public class SempBasicAuthClientExecutorTest { - @Test - public void testExecuteStatus4xx() { - MockHttpTransport transport = - new MockHttpTransport() { - @Override - public LowLevelHttpRequest buildRequest(String method, String url) { - return new MockLowLevelHttpRequest() { - @Override - public LowLevelHttpResponse execute() { - MockLowLevelHttpResponse response = new MockLowLevelHttpResponse(); - response.setStatusCode(404); - response.setContentType(Json.MEDIA_TYPE); - response.setContent( - "{\"meta\":{\"error\":{\"code\":404,\"description\":\"some" - + " error\",\"status\":\"xx\"}}}"); - return response; - } - }; - } - }; + @Test + public void testExecuteStatus4xx() { + MockHttpTransport transport = + new MockHttpTransport() { + @Override + public LowLevelHttpRequest buildRequest(String method, String url) { + return new MockLowLevelHttpRequest() { + @Override + public LowLevelHttpResponse execute() { + MockLowLevelHttpResponse response = new MockLowLevelHttpResponse(); + response.setStatusCode(404); + response.setContentType(Json.MEDIA_TYPE); + response.setContent( + "{\"meta\":{\"error\":{\"code\":404,\"description\":\"some" + + " error\",\"status\":\"xx\"}}}"); + return response; + } + }; + } + }; - HttpRequestFactory requestFactory = transport.createRequestFactory(); - SempBasicAuthClientExecutor client = - new SempBasicAuthClientExecutor( - "http://host", "username", "password", "vpnName", requestFactory); + HttpRequestFactory requestFactory = transport.createRequestFactory(); + SempBasicAuthClientExecutor client = + new SempBasicAuthClientExecutor( + "http://host", "username", "password", "vpnName", requestFactory); - assertThrows(HttpResponseException.class, () -> client.getQueueResponse("queue")); - } + assertThrows(HttpResponseException.class, () -> client.getQueueResponse("queue")); + } - /** - * In this test case, we test a situation when a session that we used to authenticate to Semp - * expires. - * - *

To test this scenario, we need to do the following: - * - *

    - *
  1. Send the first request, to initialize a session. This request has to contain the Basic - * Auth header and should not include any cookie headers. The response for this request - * contains a session cookie we can re-use in the following requests. - *
  2. Send the second request - this request should use a cookie from the previous response. - * There should be no Authorization header. To simulate an expired session scenario, we - * set the response of this request to the "401 Unauthorized". This should cause a the - * request to be retried, this time with the Authorization header. - *
  3. Validate the third request to contain the Basic Auth header and no session cookies. - *
- */ - @Test - public void testExecuteWithUnauthorized() throws IOException { - // Making it a final array, so that we can reference it from within the MockHttpTransport - // instance - final int[] requestCounter = {0}; - MockHttpTransport transport = - new MockHttpTransport() { - @Override - public LowLevelHttpRequest buildRequest(String method, String url) { - return new MockLowLevelHttpRequest() { - @Override - public LowLevelHttpResponse execute() throws IOException { - MockLowLevelHttpResponse response = new MockLowLevelHttpResponse(); - if (requestCounter[0] == 0) { - // The first request has to include Basic Auth header - assertTrue(this.getHeaders().containsKey("authorization")); - List authorizationHeaders = - this.getHeaders().get("authorization"); - assertEquals(1, authorizationHeaders.size()); - assertTrue(authorizationHeaders.get(0).contains("Basic")); - assertFalse(this.getHeaders().containsKey("cookie")); + /** + * In this test case, we test a situation when a session that we used to authenticate to Semp + * expires. + * + *

To test this scenario, we need to do the following: + * + *

    + *
  1. Send the first request, to initialize a session. This request has to contain the Basic + * Auth header and should not include any cookie headers. The response for this request + * contains a session cookie we can re-use in the following requests. + *
  2. Send the second request - this request should use a cookie from the previous response. + * There should be no Authorization header. To simulate an expired session scenario, we set + * the response of this request to the "401 Unauthorized". This should cause a the request + * to be retried, this time with the Authorization header. + *
  3. Validate the third request to contain the Basic Auth header and no session cookies. + *
+ */ + @Test + public void testExecuteWithUnauthorized() throws IOException { + // Making it a final array, so that we can reference it from within the MockHttpTransport + // instance + final int[] requestCounter = {0}; + MockHttpTransport transport = + new MockHttpTransport() { + @Override + public LowLevelHttpRequest buildRequest(String method, String url) { + return new MockLowLevelHttpRequest() { + @Override + public LowLevelHttpResponse execute() throws IOException { + MockLowLevelHttpResponse response = new MockLowLevelHttpResponse(); + if (requestCounter[0] == 0) { + // The first request has to include Basic Auth header + assertTrue(this.getHeaders().containsKey("authorization")); + List authorizationHeaders = this.getHeaders().get("authorization"); + assertEquals(1, authorizationHeaders.size()); + assertTrue(authorizationHeaders.get(0).contains("Basic")); + assertFalse(this.getHeaders().containsKey("cookie")); - // Set the response to include Session cookies - response.setHeaderNames( - ImmutableList.of("Set-Cookie", "Set-Cookie")) - .setHeaderValues( - ImmutableList.of( - "ProxySession=JddSdJaGo6FYYmQk6nt8jXxFtq6n3FCFR14ebzRGQ5w;" - + " HttpOnly; SameSite=Strict;" - + " Path=/proxy; Max-Age=2592000", - "Session=JddSdJaGo6FYYmQk6nt8jXxFtq6n3FCFR14ebzRGQ5w;" - + " HttpOnly; SameSite=Strict;" - + " Path=/SEMP; Max-Age=2592000")); - response.setStatusCode(200); - } else if (requestCounter[0] == 1) { - // The second request does not include Basic Auth header - assertFalse(this.getHeaders().containsKey("authorization")); - // It must include a cookie header - assertTrue(this.getHeaders().containsKey("cookie")); - boolean hasSessionCookie = - this.getHeaders().get("cookie").stream() - .filter( - c -> - c.contains( - "Session=JddSdJaGo6FYYmQk6nt8jXxFtq6n3FCFR14ebzRGQ5w")) - .count() - == 1; - assertTrue(hasSessionCookie); + // Set the response to include Session cookies + response + .setHeaderNames(ImmutableList.of("Set-Cookie", "Set-Cookie")) + .setHeaderValues( + ImmutableList.of( + "ProxySession=JddSdJaGo6FYYmQk6nt8jXxFtq6n3FCFR14ebzRGQ5w;" + + " HttpOnly; SameSite=Strict;" + + " Path=/proxy; Max-Age=2592000", + "Session=JddSdJaGo6FYYmQk6nt8jXxFtq6n3FCFR14ebzRGQ5w;" + + " HttpOnly; SameSite=Strict;" + + " Path=/SEMP; Max-Age=2592000")); + response.setStatusCode(200); + } else if (requestCounter[0] == 1) { + // The second request does not include Basic Auth header + assertFalse(this.getHeaders().containsKey("authorization")); + // It must include a cookie header + assertTrue(this.getHeaders().containsKey("cookie")); + boolean hasSessionCookie = + this.getHeaders().get("cookie").stream() + .filter( + c -> + c.contains( + "Session=JddSdJaGo6FYYmQk6nt8jXxFtq6n3FCFR14ebzRGQ5w")) + .count() + == 1; + assertTrue(hasSessionCookie); - // Let's assume the Session expired - we return the 401 - // unauthorized - response.setStatusCode(401); - } else { - // The second request has to be retried with a Basic Auth header - // this time - assertTrue(this.getHeaders().containsKey("authorization")); - List authorizationHeaders = - this.getHeaders().get("authorization"); - assertEquals(1, authorizationHeaders.size()); - assertTrue(authorizationHeaders.get(0).contains("Basic")); - assertFalse(this.getHeaders().containsKey("cookie")); + // Let's assume the Session expired - we return the 401 + // unauthorized + response.setStatusCode(401); + } else { + // The second request has to be retried with a Basic Auth header + // this time + assertTrue(this.getHeaders().containsKey("authorization")); + List authorizationHeaders = this.getHeaders().get("authorization"); + assertEquals(1, authorizationHeaders.size()); + assertTrue(authorizationHeaders.get(0).contains("Basic")); + assertFalse(this.getHeaders().containsKey("cookie")); - response.setStatusCode(200); - } - response.setContentType(Json.MEDIA_TYPE); - requestCounter[0]++; - return response; - } - }; - } - }; + response.setStatusCode(200); + } + response.setContentType(Json.MEDIA_TYPE); + requestCounter[0]++; + return response; + } + }; + } + }; - HttpRequestFactory requestFactory = transport.createRequestFactory(); - SempBasicAuthClientExecutor client = - new SempBasicAuthClientExecutor( - "http://host", "username", "password", "vpnName", requestFactory); + HttpRequestFactory requestFactory = transport.createRequestFactory(); + SempBasicAuthClientExecutor client = + new SempBasicAuthClientExecutor( + "http://host", "username", "password", "vpnName", requestFactory); - // The first, initial request - client.getQueueResponse("queue"); - // The second request, which will try to authenticate with a cookie, and then with Basic - // Auth when it receives a 401 unauthorized - client.getQueueResponse("queue"); + // The first, initial request + client.getQueueResponse("queue"); + // The second request, which will try to authenticate with a cookie, and then with Basic + // Auth when it receives a 401 unauthorized + client.getQueueResponse("queue"); - // There should be 3 requests executed: - // the first one is the initial one with Basic Auth, - // the second one uses the session cookie, but we simulate it being expired, - // so there should be a third request with Basic Auth to create a new session. - assertEquals(3, requestCounter[0]); - } + // There should be 3 requests executed: + // the first one is the initial one with Basic Auth, + // the second one uses the session cookie, but we simulate it being expired, + // so there should be a third request with Basic Auth to create a new session. + assertEquals(3, requestCounter[0]); + } } diff --git a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/data/SolaceTest.java b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/data/SolaceTest.java index d3acab33a71f..7dcf12559e3b 100644 --- a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/data/SolaceTest.java +++ b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/data/SolaceTest.java @@ -1,11 +1,13 @@ /* - * Copyright 2024 Google. + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -17,201 +19,196 @@ import java.nio.charset.StandardCharsets; import java.util.Map; -import org.apache.beam.sdk.io.solace.data.Solace; import org.apache.beam.sdk.io.solace.data.Solace.Destination; import org.junit.Assert; import org.junit.Test; public class SolaceTest { - Map properties; - Destination destination = - Solace.Destination.builder() - .setName("some destination") - .setType(Solace.DestinationType.TOPIC) - .build(); - String messageId = "some message id"; - Long expiration = 123L; - Integer priority = 7; - Boolean redelivered = true; - String replyTo = "no-one"; - Long receiveTimestamp = 123456789L; - Long senderTimestamp = 987654321L; - long timestampMillis = 1234567890L; - Long sequenceNumber = 27L; - Long timeToLive = 34567890L; - String payloadString = "some payload"; - byte[] payload = payloadString.getBytes(StandardCharsets.UTF_8); - String publishError = "some error"; - - @Test - public void testRecordEquality() { - Solace.Record obj1 = - Solace.Record.builder() - .setDestination(destination) - .setMessageId(messageId) - .setExpiration(expiration) - .setPriority(priority) - .setRedelivered(redelivered) - .setReplyTo(replyTo) - .setReceiveTimestamp(receiveTimestamp) - .setSenderTimestamp(senderTimestamp) - .setSequenceNumber(sequenceNumber) - .setTimeToLive(timeToLive) - .setPayload(payload) - .build(); - - Solace.Record obj2 = - Solace.Record.builder() - .setDestination(destination) - .setMessageId(messageId) - .setExpiration(expiration) - .setPriority(priority) - .setRedelivered(redelivered) - .setReplyTo(replyTo) - .setReceiveTimestamp(receiveTimestamp) - .setSenderTimestamp(senderTimestamp) - .setSequenceNumber(sequenceNumber) - .setTimeToLive(timeToLive) - .setPayload(payload) - .build(); - - Solace.Record obj3 = - Solace.Record.builder() - .setDestination(destination) - .setMessageId(messageId) - .setExpiration(expiration) - .setPriority(priority) - .setRedelivered(!redelivered) - .setReplyTo(replyTo) - .setReceiveTimestamp(receiveTimestamp) - .setSenderTimestamp(senderTimestamp) - .setSequenceNumber(sequenceNumber) - .setTimeToLive(timeToLive) - .setPayload(payload) - .build(); - - Assert.assertEquals(obj1, obj2); - Assert.assertNotEquals(obj1, obj3); - Assert.assertEquals(obj1.hashCode(), obj2.hashCode()); - Assert.assertEquals(obj1.getDestination(), destination); - Assert.assertEquals(obj1.getMessageId(), messageId); - Assert.assertEquals(obj1.getExpiration(), expiration); - Assert.assertEquals(obj1.getPriority(), priority); - Assert.assertEquals(obj1.getRedelivered(), redelivered); - Assert.assertEquals(obj1.getReplyTo(), replyTo); - Assert.assertEquals(obj1.getReceiveTimestamp(), receiveTimestamp); - Assert.assertEquals(obj1.getSenderTimestamp(), senderTimestamp); - Assert.assertEquals(obj1.getSequenceNumber(), sequenceNumber); - Assert.assertEquals(obj1.getTimeToLive(), timeToLive); - Assert.assertEquals(new String(obj1.getPayload(), StandardCharsets.UTF_8), payloadString); - } - - @Test - public void testRecordNullability() { - Solace.Record obj = - Solace.Record.builder().setMessageId(messageId).setPayload(payload).build(); - Assert.assertNotNull(obj); - Assert.assertNull(obj.getDestination()); - Assert.assertEquals(obj.getMessageId(), messageId); - Assert.assertNull(obj.getExpiration()); - Assert.assertNull(obj.getPriority()); - Assert.assertNull(obj.getRedelivered()); - Assert.assertNull(obj.getReplyTo()); - Assert.assertNull(obj.getReceiveTimestamp()); - Assert.assertNull(obj.getSenderTimestamp()); - Assert.assertNull(obj.getSequenceNumber()); - Assert.assertNull(obj.getTimeToLive()); - Assert.assertEquals(new String(obj.getPayload(), StandardCharsets.UTF_8), payloadString); - } - - @Test(expected = IllegalStateException.class) - public void testRecordBuilder() { - Solace.Record.builder().build(); - } - - @Test - public void testPublishResultEquality() { - Solace.PublishResult obj1 = - Solace.PublishResult.builder() - .setPublished(redelivered) - .setLatencyMilliseconds(timestampMillis) - .setMessageId(messageId) - .setError(publishError) - .build(); - - Solace.PublishResult obj2 = - Solace.PublishResult.builder() - .setPublished(redelivered) - .setLatencyMilliseconds(timestampMillis) - .setMessageId(messageId) - .setError(publishError) - .build(); - - Solace.PublishResult obj3 = - Solace.PublishResult.builder() - .setPublished(!redelivered) - .setLatencyMilliseconds(timestampMillis) - .setMessageId(messageId) - .setError(publishError) - .build(); - - Assert.assertEquals(obj1, obj2); - Assert.assertNotEquals(obj1, obj3); - Assert.assertEquals(obj1.getPublished(), redelivered); - Assert.assertEquals(obj1.getLatencyMilliseconds().longValue(), timestampMillis); - Assert.assertEquals(obj1.getMessageId(), messageId); - Assert.assertEquals(obj1.getError(), publishError); - } - - @Test(expected = IllegalStateException.class) - public void testPublishResultBuilder() { - Solace.PublishResult.builder().build(); - } - - @Test - public void testPublishResultNullability() { - Solace.PublishResult obj = - Solace.PublishResult.builder() - .setMessageId(messageId) - .setPublished(redelivered) - .build(); - - Assert.assertNotNull(obj); - Assert.assertEquals(obj.getMessageId(), messageId); - Assert.assertEquals(obj.getPublished(), redelivered); - Assert.assertNull(obj.getLatencyMilliseconds()); - Assert.assertNull(obj.getError()); - } - - @Test - public void testCorrelationKeyEquality() { - Solace.CorrelationKey obj1 = - Solace.CorrelationKey.builder() - .setMessageId(messageId) - .setPublishMonotonicMillis(timestampMillis) - .build(); - - Solace.CorrelationKey obj2 = - Solace.CorrelationKey.builder() - .setMessageId(messageId) - .setPublishMonotonicMillis(timestampMillis) - .build(); - - Solace.CorrelationKey obj3 = - Solace.CorrelationKey.builder() - .setMessageId(messageId) - .setPublishMonotonicMillis(timestampMillis - 1L) - .build(); - - Assert.assertEquals(obj1, obj2); - Assert.assertNotEquals(obj1, obj3); - Assert.assertEquals(obj1.getMessageId(), messageId); - Assert.assertEquals(obj1.getPublishMonotonicMillis(), timestampMillis); - } - - @Test(expected = IllegalStateException.class) - public void testCorrelationKeyNullability() { - Solace.CorrelationKey.builder().build(); - } + Map properties; + Destination destination = + Solace.Destination.builder() + .setName("some destination") + .setType(Solace.DestinationType.TOPIC) + .build(); + String messageId = "some message id"; + Long expiration = 123L; + Integer priority = 7; + Boolean redelivered = true; + String replyTo = "no-one"; + Long receiveTimestamp = 123456789L; + Long senderTimestamp = 987654321L; + long timestampMillis = 1234567890L; + Long sequenceNumber = 27L; + Long timeToLive = 34567890L; + String payloadString = "some payload"; + byte[] payload = payloadString.getBytes(StandardCharsets.UTF_8); + String publishError = "some error"; + + @Test + public void testRecordEquality() { + Solace.Record obj1 = + Solace.Record.builder() + .setDestination(destination) + .setMessageId(messageId) + .setExpiration(expiration) + .setPriority(priority) + .setRedelivered(redelivered) + .setReplyTo(replyTo) + .setReceiveTimestamp(receiveTimestamp) + .setSenderTimestamp(senderTimestamp) + .setSequenceNumber(sequenceNumber) + .setTimeToLive(timeToLive) + .setPayload(payload) + .build(); + + Solace.Record obj2 = + Solace.Record.builder() + .setDestination(destination) + .setMessageId(messageId) + .setExpiration(expiration) + .setPriority(priority) + .setRedelivered(redelivered) + .setReplyTo(replyTo) + .setReceiveTimestamp(receiveTimestamp) + .setSenderTimestamp(senderTimestamp) + .setSequenceNumber(sequenceNumber) + .setTimeToLive(timeToLive) + .setPayload(payload) + .build(); + + Solace.Record obj3 = + Solace.Record.builder() + .setDestination(destination) + .setMessageId(messageId) + .setExpiration(expiration) + .setPriority(priority) + .setRedelivered(!redelivered) + .setReplyTo(replyTo) + .setReceiveTimestamp(receiveTimestamp) + .setSenderTimestamp(senderTimestamp) + .setSequenceNumber(sequenceNumber) + .setTimeToLive(timeToLive) + .setPayload(payload) + .build(); + + Assert.assertEquals(obj1, obj2); + Assert.assertNotEquals(obj1, obj3); + Assert.assertEquals(obj1.hashCode(), obj2.hashCode()); + Assert.assertEquals(obj1.getDestination(), destination); + Assert.assertEquals(obj1.getMessageId(), messageId); + Assert.assertEquals(obj1.getExpiration(), expiration); + Assert.assertEquals(obj1.getPriority(), priority); + Assert.assertEquals(obj1.getRedelivered(), redelivered); + Assert.assertEquals(obj1.getReplyTo(), replyTo); + Assert.assertEquals(obj1.getReceiveTimestamp(), receiveTimestamp); + Assert.assertEquals(obj1.getSenderTimestamp(), senderTimestamp); + Assert.assertEquals(obj1.getSequenceNumber(), sequenceNumber); + Assert.assertEquals(obj1.getTimeToLive(), timeToLive); + Assert.assertEquals(new String(obj1.getPayload(), StandardCharsets.UTF_8), payloadString); + } + + @Test + public void testRecordNullability() { + Solace.Record obj = Solace.Record.builder().setMessageId(messageId).setPayload(payload).build(); + Assert.assertNotNull(obj); + Assert.assertNull(obj.getDestination()); + Assert.assertEquals(obj.getMessageId(), messageId); + Assert.assertNull(obj.getExpiration()); + Assert.assertNull(obj.getPriority()); + Assert.assertNull(obj.getRedelivered()); + Assert.assertNull(obj.getReplyTo()); + Assert.assertNull(obj.getReceiveTimestamp()); + Assert.assertNull(obj.getSenderTimestamp()); + Assert.assertNull(obj.getSequenceNumber()); + Assert.assertNull(obj.getTimeToLive()); + Assert.assertEquals(new String(obj.getPayload(), StandardCharsets.UTF_8), payloadString); + } + + @Test(expected = IllegalStateException.class) + public void testRecordBuilder() { + Solace.Record.builder().build(); + } + + @Test + public void testPublishResultEquality() { + Solace.PublishResult obj1 = + Solace.PublishResult.builder() + .setPublished(redelivered) + .setLatencyMilliseconds(timestampMillis) + .setMessageId(messageId) + .setError(publishError) + .build(); + + Solace.PublishResult obj2 = + Solace.PublishResult.builder() + .setPublished(redelivered) + .setLatencyMilliseconds(timestampMillis) + .setMessageId(messageId) + .setError(publishError) + .build(); + + Solace.PublishResult obj3 = + Solace.PublishResult.builder() + .setPublished(!redelivered) + .setLatencyMilliseconds(timestampMillis) + .setMessageId(messageId) + .setError(publishError) + .build(); + + Assert.assertEquals(obj1, obj2); + Assert.assertNotEquals(obj1, obj3); + Assert.assertEquals(obj1.getPublished(), redelivered); + Assert.assertEquals(obj1.getLatencyMilliseconds().longValue(), timestampMillis); + Assert.assertEquals(obj1.getMessageId(), messageId); + Assert.assertEquals(obj1.getError(), publishError); + } + + @Test(expected = IllegalStateException.class) + public void testPublishResultBuilder() { + Solace.PublishResult.builder().build(); + } + + @Test + public void testPublishResultNullability() { + Solace.PublishResult obj = + Solace.PublishResult.builder().setMessageId(messageId).setPublished(redelivered).build(); + + Assert.assertNotNull(obj); + Assert.assertEquals(obj.getMessageId(), messageId); + Assert.assertEquals(obj.getPublished(), redelivered); + Assert.assertNull(obj.getLatencyMilliseconds()); + Assert.assertNull(obj.getError()); + } + + @Test + public void testCorrelationKeyEquality() { + Solace.CorrelationKey obj1 = + Solace.CorrelationKey.builder() + .setMessageId(messageId) + .setPublishMonotonicMillis(timestampMillis) + .build(); + + Solace.CorrelationKey obj2 = + Solace.CorrelationKey.builder() + .setMessageId(messageId) + .setPublishMonotonicMillis(timestampMillis) + .build(); + + Solace.CorrelationKey obj3 = + Solace.CorrelationKey.builder() + .setMessageId(messageId) + .setPublishMonotonicMillis(timestampMillis - 1L) + .build(); + + Assert.assertEquals(obj1, obj2); + Assert.assertNotEquals(obj1, obj3); + Assert.assertEquals(obj1.getMessageId(), messageId); + Assert.assertEquals(obj1.getPublishMonotonicMillis(), timestampMillis); + } + + @Test(expected = IllegalStateException.class) + public void testCorrelationKeyNullability() { + Solace.CorrelationKey.builder().build(); + } } diff --git a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/it/SolaceContainerManager.java b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/it/SolaceContainerManager.java index 8392c322997e..e9c3fe7dfcb6 100644 --- a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/it/SolaceContainerManager.java +++ b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/it/SolaceContainerManager.java @@ -1,11 +1,13 @@ /* - * Copyright 2024 Google. + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -28,139 +30,139 @@ public class SolaceContainerManager { - public static final String VPN_NAME = "default"; - public static final String PASSWORD = "password"; - public static final String USERNAME = "username"; - public static final String TOPIC_NAME = "test_topic"; - private static final Logger LOG = LoggerFactory.getLogger(SolaceContainerManager.class); - private final SolaceContainer container; + public static final String VPN_NAME = "default"; + public static final String PASSWORD = "password"; + public static final String USERNAME = "username"; + public static final String TOPIC_NAME = "test_topic"; + private static final Logger LOG = LoggerFactory.getLogger(SolaceContainerManager.class); + private final SolaceContainer container; - public SolaceContainerManager() { - this.container = - new SolaceContainer(DockerImageName.parse("solace/solace-pubsub-standard:10.7")) { - { - addFixedExposedPort(55555, 55555); - addFixedExposedPort(9000, 9000); - addFixedExposedPort(8080, 8080); - addFixedExposedPort(80, 80); - } - }.withVpn(VPN_NAME) - .withCredentials(USERNAME, PASSWORD) - // .withExposedPorts(Service.SMF.getPort()); - .withTopic(TOPIC_NAME, Service.SMF) - .withLogConsumer(new Slf4jLogConsumer(LOG)); - container.addExposedPort(8080); - container.addExposedPort(55555); - } + public SolaceContainerManager() { + this.container = + new SolaceContainer(DockerImageName.parse("solace/solace-pubsub-standard:10.7")) { + { + addFixedExposedPort(55555, 55555); + addFixedExposedPort(9000, 9000); + addFixedExposedPort(8080, 8080); + addFixedExposedPort(80, 80); + } + }.withVpn(VPN_NAME) + .withCredentials(USERNAME, PASSWORD) + // .withExposedPorts(Service.SMF.getPort()); + .withTopic(TOPIC_NAME, Service.SMF) + .withLogConsumer(new Slf4jLogConsumer(LOG)); + container.addExposedPort(8080); + container.addExposedPort(55555); + } - public void start() { - container.start(); - } + public void start() { + container.start(); + } - void createQueueWithSubscriptionTopic(String queueName) { - executeCommand( - "curl", - "http://localhost:8080/SEMP/v2/config/msgVpns/" + VPN_NAME + "/topicEndpoints", - "-X", - "POST", - "-u", - "admin:admin", - "-H", - "Content-Type:application/json", - "-d", - "{\"topicEndpointName\":\"" - + TOPIC_NAME - + "\",\"accessType\":\"exclusive\",\"permission\":\"modify-topic\",\"ingressEnabled\":true,\"egressEnabled\":true}"); - executeCommand( - "curl", - "http://localhost:8080/SEMP/v2/config/msgVpns/" + VPN_NAME + "/queues", - "-X", - "POST", - "-u", - "admin:admin", - "-H", - "Content-Type:application/json", - "-d", - "{\"queueName\":\"" - + queueName - + "\",\"accessType\":\"non-exclusive\",\"maxMsgSpoolUsage\":200,\"permission\":\"consume\",\"ingressEnabled\":true,\"egressEnabled\":true}"); - executeCommand( - "curl", - "http://localhost:8080/SEMP/v2/config/msgVpns/" - + VPN_NAME - + "/queues/" - + queueName - + "/subscriptions", - "-X", - "POST", - "-u", - "admin:admin", - "-H", - "Content-Type:application/json", - "-d", - "{\"subscriptionTopic\":\"" + TOPIC_NAME + "\"}"); - } + void createQueueWithSubscriptionTopic(String queueName) { + executeCommand( + "curl", + "http://localhost:8080/SEMP/v2/config/msgVpns/" + VPN_NAME + "/topicEndpoints", + "-X", + "POST", + "-u", + "admin:admin", + "-H", + "Content-Type:application/json", + "-d", + "{\"topicEndpointName\":\"" + + TOPIC_NAME + + "\",\"accessType\":\"exclusive\",\"permission\":\"modify-topic\",\"ingressEnabled\":true,\"egressEnabled\":true}"); + executeCommand( + "curl", + "http://localhost:8080/SEMP/v2/config/msgVpns/" + VPN_NAME + "/queues", + "-X", + "POST", + "-u", + "admin:admin", + "-H", + "Content-Type:application/json", + "-d", + "{\"queueName\":\"" + + queueName + + "\",\"accessType\":\"non-exclusive\",\"maxMsgSpoolUsage\":200,\"permission\":\"consume\",\"ingressEnabled\":true,\"egressEnabled\":true}"); + executeCommand( + "curl", + "http://localhost:8080/SEMP/v2/config/msgVpns/" + + VPN_NAME + + "/queues/" + + queueName + + "/subscriptions", + "-X", + "POST", + "-u", + "admin:admin", + "-H", + "Content-Type:application/json", + "-d", + "{\"subscriptionTopic\":\"" + TOPIC_NAME + "\"}"); + } - private void executeCommand(String... command) { - try { - org.testcontainers.containers.Container.ExecResult execResult = - container.execInContainer(command); - if (execResult.getExitCode() != 0) { - logCommandError(execResult.getStderr(), command); - } else { - LOG.info(execResult.getStdout()); - } - } catch (IOException | InterruptedException e) { - logCommandError(e.getMessage(), command); - } + private void executeCommand(String... command) { + try { + org.testcontainers.containers.Container.ExecResult execResult = + container.execInContainer(command); + if (execResult.getExitCode() != 0) { + logCommandError(execResult.getStderr(), command); + } else { + LOG.info(execResult.getStdout()); + } + } catch (IOException | InterruptedException e) { + logCommandError(e.getMessage(), command); } + } - private void logCommandError(String error, String... command) { - LOG.error("Could not execute command {}: {}", command, error); - } + private void logCommandError(String error, String... command) { + LOG.error("Could not execute command {}: {}", command, error); + } - public void stop() { - if (container != null) { - container.stop(); - } + public void stop() { + if (container != null) { + container.stop(); } + } + + public void getQueueDetails(String queueName) { + executeCommand( + "curl", + "http://localhost:8080/SEMP/v2/monitor/msgVpns/" + + VPN_NAME + + "/queues/" + + queueName + + "/msgs", + "-X", + "GET", + "-u", + "admin:admin"); + } + + public void sendToTopic(String payload, List additionalHeaders) { + // https://docs.solace.com/API/RESTMessagingPrtl/Solace-REST-Message-Encoding.htm - public void getQueueDetails(String queueName) { - executeCommand( + List command = + new ArrayList<>( + Arrays.asList( "curl", - "http://localhost:8080/SEMP/v2/monitor/msgVpns/" - + VPN_NAME - + "/queues/" - + queueName - + "/msgs", + "http://localhost:9000/TOPIC/" + TOPIC_NAME, "-X", - "GET", + "POST", "-u", - "admin:admin"); - } - - public void sendToTopic(String payload, List additionalHeaders) { - // https://docs.solace.com/API/RESTMessagingPrtl/Solace-REST-Message-Encoding.htm - - List command = - new ArrayList<>( - Arrays.asList( - "curl", - "http://localhost:9000/TOPIC/" + TOPIC_NAME, - "-X", - "POST", - "-u", - USERNAME + ":" + PASSWORD, - "--header", - "Content-Type:application/json", - "-d", - payload)); - - for (String additionalHeader : additionalHeaders) { - command.add("--header"); - command.add(additionalHeader); - } + USERNAME + ":" + PASSWORD, + "--header", + "Content-Type:application/json", + "-d", + payload)); - executeCommand(command.toArray(new String[0])); + for (String additionalHeader : additionalHeaders) { + command.add("--header"); + command.add(additionalHeader); } + + executeCommand(command.toArray(new String[0])); + } } diff --git a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/it/SolaceIOIT.java b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/it/SolaceIOIT.java index 6591317cda54..fff61b328a63 100644 --- a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/it/SolaceIOIT.java +++ b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/it/SolaceIOIT.java @@ -1,11 +1,13 @@ /* - * Copyright 2024 Google. + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -17,14 +19,13 @@ import static org.junit.Assert.assertNotEquals; +import java.nio.charset.StandardCharsets; +import org.apache.beam.sdk.PipelineResult; import org.apache.beam.sdk.io.solace.SolaceIO; import org.apache.beam.sdk.io.solace.broker.BasicAuthJcsmpSessionServiceFactory; import org.apache.beam.sdk.io.solace.broker.BasicAuthSempClientFactory; import org.apache.beam.sdk.io.solace.data.Solace.Queue; import org.apache.beam.sdk.io.solace.data.Solace.Record; -import java.nio.charset.StandardCharsets; -import java.util.List; -import org.apache.beam.sdk.PipelineResult; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.options.StreamingOptions; import org.apache.beam.sdk.testing.PAssert; @@ -38,6 +39,7 @@ import org.apache.beam.sdk.transforms.windowing.FixedWindows; import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.hamcrest.BaseMatcher; import org.hamcrest.Description; import org.joda.time.Duration; @@ -50,224 +52,223 @@ import org.testcontainers.shaded.org.apache.commons.lang3.builder.EqualsBuilder; public class SolaceIOIT { - private static final Logger LOG = LoggerFactory.getLogger(SolaceIOIT.class); - private static SolaceContainerManager solaceContainerManager; - private static final TestPipelineOptions testOptions; + private static final Logger LOG = LoggerFactory.getLogger(SolaceIOIT.class); + private static SolaceContainerManager solaceContainerManager; + private static final TestPipelineOptions testOptions; - static { - testOptions = PipelineOptionsFactory.create().as(TestPipelineOptions.class); - testOptions.setBlockOnRun(false); - } + static { + testOptions = PipelineOptionsFactory.create().as(TestPipelineOptions.class); + testOptions.setBlockOnRun(false); + } - @Rule public final transient TestPipeline pipeline = TestPipeline.fromOptions(testOptions); + @Rule public final transient TestPipeline pipeline = TestPipeline.fromOptions(testOptions); - @BeforeClass - public static void setup() { - System.out.println("START"); - solaceContainerManager = new SolaceContainerManager(); - solaceContainerManager.start(); - } + @BeforeClass + public static void setup() { + System.out.println("START"); + solaceContainerManager = new SolaceContainerManager(); + solaceContainerManager.start(); + } - @AfterClass - public static void afterClass() { - if (solaceContainerManager != null) { - solaceContainerManager.stop(); - } + @AfterClass + public static void afterClass() { + if (solaceContainerManager != null) { + solaceContainerManager.stop(); } + } - @Test - public void test() { - // Similar approach to - // https://github.com/apache/beam/blob/812e98fac243bab2a88f6ea5fad6147ff8e54a97/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOIT.java#L216 - String queueName = "test_queue"; - solaceContainerManager.createQueueWithSubscriptionTopic(queueName); + @Test + public void test() { + // Similar approach to + // https://github.com/apache/beam/blob/812e98fac243bab2a88f6ea5fad6147ff8e54a97/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOIT.java#L216 + String queueName = "test_queue"; + solaceContainerManager.createQueueWithSubscriptionTopic(queueName); - solaceContainerManager.getQueueDetails(queueName); - String payload = "{\"field_str\":\"value\",\"field_int\":123}"; - solaceContainerManager.sendToTopic(payload, List.of("Solace-Message-ID:m1")); - solaceContainerManager.sendToTopic(payload, List.of("Solace-Message-ID:m2")); - solaceContainerManager.getQueueDetails(queueName); + solaceContainerManager.getQueueDetails(queueName); + String payload = "{\"field_str\":\"value\",\"field_int\":123}"; + solaceContainerManager.sendToTopic(payload, ImmutableList.of("Solace-Message-ID:m1")); + solaceContainerManager.sendToTopic(payload, ImmutableList.of("Solace-Message-ID:m2")); + solaceContainerManager.getQueueDetails(queueName); - pipeline.getOptions().as(StreamingOptions.class).setStreaming(true); + pipeline.getOptions().as(StreamingOptions.class).setStreaming(true); - PCollection events = - pipeline.apply( - "Read from Solace", - SolaceIO.read() - .from(Queue.fromName(queueName)) - .withMaxNumConnections(1) - .withSempClientFactory( - BasicAuthSempClientFactory.builder() - .withHost("http://localhost:8080") - .withUsername("admin") - .withPassword("admin") - .withVpnName(SolaceContainerManager.VPN_NAME) - .build()) - .withSessionServiceFactory( - BasicAuthJcsmpSessionServiceFactory.builder() - .withHost("localhost") - .withUsername(SolaceContainerManager.USERNAME) - .withPassword(SolaceContainerManager.PASSWORD) - .withVpnName(SolaceContainerManager.VPN_NAME) - .build())); - // PCollection count = - PCollection records = - events.apply( - "PassThrough", - MapElements.via( - new SimpleFunction() { - @Override - public Record apply(Record s) { - System.out.println("passthrough rec: " + s); - return s; - } - // })).apply("Window", - // Window.into(CalendarWindows.years(1))); - })) - .apply("Window", Window.into(FixedWindows.of(Duration.standardSeconds(5)))); + PCollection events = + pipeline.apply( + "Read from Solace", + SolaceIO.read() + .from(Queue.fromName(queueName)) + .withMaxNumConnections(1) + .withSempClientFactory( + BasicAuthSempClientFactory.builder() + .withHost("http://localhost:8080") + .withUsername("admin") + .withPassword("admin") + .withVpnName(SolaceContainerManager.VPN_NAME) + .build()) + .withSessionServiceFactory( + BasicAuthJcsmpSessionServiceFactory.builder() + .withHost("localhost") + .withUsername(SolaceContainerManager.USERNAME) + .withPassword(SolaceContainerManager.PASSWORD) + .withVpnName(SolaceContainerManager.VPN_NAME) + .build())); + // PCollection count = + PCollection records = + events + .apply( + "PassThrough", + MapElements.via( + new SimpleFunction() { + @Override + public Record apply(Record s) { + System.out.println("passthrough rec: " + s); + return s; + } + // })).apply("Window", + // Window.into(CalendarWindows.years(1))); + })) + .apply("Window", Window.into(FixedWindows.of(Duration.standardSeconds(5)))); - System.out.println("xxxxx"); + System.out.println("xxxxx"); - PAssert.thatSingleton( - records.apply( - "Counting element", - Combine.globally(Count.combineFn()).withoutDefaults())) - .isEqualTo(1L); + PAssert.thatSingleton( + records.apply( + "Counting element", Combine.globally(Count.combineFn()).withoutDefaults())) + .isEqualTo(1L); - PAssert.that(records) - .containsInAnyOrder( - partialMatch( - Record.builder() - .setMessageId("m1") - .setPayload(payload.getBytes(StandardCharsets.UTF_8)) - .build())); + PAssert.that(records) + .containsInAnyOrder( + partialMatch( + Record.builder() + .setMessageId("m1") + .setPayload(payload.getBytes(StandardCharsets.UTF_8)) + .build())); - PipelineResult writeResult = pipeline.run(); - // removing this line causes the pipeline not ingest any data - PipelineResult.State writeState = writeResult.waitUntilFinish(Duration.standardSeconds(10)); - assertNotEquals(PipelineResult.State.FAILED, writeState); + PipelineResult writeResult = pipeline.run(); + // removing this line causes the pipeline not ingest any data + PipelineResult.State writeState = writeResult.waitUntilFinish(Duration.standardSeconds(10)); + assertNotEquals(PipelineResult.State.FAILED, writeState); - System.out.println("queue after pipeline"); - solaceContainerManager.getQueueDetails(queueName); - } + System.out.println("queue after pipeline"); + solaceContainerManager.getQueueDetails(queueName); + } - private static SerializableMatcher partialMatch(Record expected) { - class Matcher extends BaseMatcher implements SerializableMatcher { - @Override - public boolean matches(Object item) { - LOG.info("matches!!!"); - System.out.println("matches"); - if (!(item instanceof Record)) { - return false; - } + private static SerializableMatcher partialMatch(Record expected) { + class Matcher extends BaseMatcher implements SerializableMatcher { + @Override + public boolean matches(Object item) { + LOG.info("matches!!!"); + System.out.println("matches"); + if (!(item instanceof Record)) { + return false; + } - Record actual = (Record) item; - boolean partiallyEqual = - EqualsBuilder.reflectionEquals( - actual, expected, "replicationGroupMessageId"); - System.out.println("expected.equals(actual): " + expected.equals(actual)); - System.out.println("partiallyEqual: " + partiallyEqual); - System.out.println("expected: " + expected); - System.out.println("actual: " + actual); + Record actual = (Record) item; + boolean partiallyEqual = + EqualsBuilder.reflectionEquals(actual, expected, "replicationGroupMessageId"); + System.out.println("expected.equals(actual): " + expected.equals(actual)); + System.out.println("partiallyEqual: " + partiallyEqual); + System.out.println("expected: " + expected); + System.out.println("actual: " + actual); - return true; + return true; - // for (Record needle : needles) { - // if (!haystack.contains(needle)) { - // return false; - // } - // } - // return true; - } + // for (Record needle : needles) { + // if (!haystack.contains(needle)) { + // return false; + // } + // } + // return true; + } - @Override - public void describeTo(Description description) { - description.appendText("Contains all of: "); - description.appendText(expected.toString()); - } - } - System.out.println("new matcher"); - return new Matcher(); + @Override + public void describeTo(Description description) { + description.appendText("Contains all of: "); + description.appendText(expected.toString()); + } } - // @Test - // public void testWrite() { - // TestStream createEvents = - // TestStream.create(StringUtf8Coder.of()) - // .addElements("r1", "r2") - // .advanceWatermarkTo( - // Instant.ofEpochMilli(0L).plus(Duration.standardSeconds(10))) - // .addElements("r3", "r4") - // .advanceWatermarkToInfinity(); - // - // PCollection records = pipeline.apply(createEvents); - // - // SolacePublishResult results = - // records.apply( - // "map", - // ParDo.of( - // new DoFn() { - // @ProcessElement - // public void processElement(ProcessContext c) { - // // System.out.println("Failed: " + c.element()); - // c.output( - // buildRecord( - // c.element(), - // "payload_" + c.element())); - // } - // })) - // .apply( - // SolaceIO.writeSolaceRecords() - // .to(Topic.fromName("test_topic")) - // .withSessionPropertiesProvider( - // BasicAuthenticationProvider.builder() - // .username("xx") - // .password("xx") - // .host("localhost") - // .vpnName(solaceContainer.getVpn()) - // .build()) - // .withDeliveryMode(DeliveryMode.PERSISTENT) - // .withSubmissionMode(SubmissionMode.HIGHER_THROUGHPUT) - // .withWriterType(WriterType.BATCHED) - // .withMaxNumOfUsedWorkers(1) - // .withNumberOfClientsPerWorker(1)); - // - // results.getSuccessfulPublish() - // .apply( - // "Successful records", - // ParDo.of( - // new DoFn() { - // @ProcessElement - // public void processElement(ProcessContext c) { - // System.out.println("OK: " + c.element()); - // c.output(1); - // } - // })); - // - // results.getFailedPublish() - // .apply( - // "Failed records", - // ParDo.of( - // new DoFn() { - // @ProcessElement - // public void processElement(ProcessContext c) { - // System.out.println("Failed: " + c.element()); - // c.output(1); - // } - // })); - // - // pipeline.run().waitUntilFinish(); - // } + System.out.println("new matcher"); + return new Matcher(); + } + // @Test + // public void testWrite() { + // TestStream createEvents = + // TestStream.create(StringUtf8Coder.of()) + // .addElements("r1", "r2") + // .advanceWatermarkTo( + // Instant.ofEpochMilli(0L).plus(Duration.standardSeconds(10))) + // .addElements("r3", "r4") + // .advanceWatermarkToInfinity(); + // + // PCollection records = pipeline.apply(createEvents); + // + // SolacePublishResult results = + // records.apply( + // "map", + // ParDo.of( + // new DoFn() { + // @ProcessElement + // public void processElement(ProcessContext c) { + // // System.out.println("Failed: " + c.element()); + // c.output( + // buildRecord( + // c.element(), + // "payload_" + c.element())); + // } + // })) + // .apply( + // SolaceIO.writeSolaceRecords() + // .to(Topic.fromName("test_topic")) + // .withSessionPropertiesProvider( + // BasicAuthenticationProvider.builder() + // .username("xx") + // .password("xx") + // .host("localhost") + // .vpnName(solaceContainer.getVpn()) + // .build()) + // .withDeliveryMode(DeliveryMode.PERSISTENT) + // .withSubmissionMode(SubmissionMode.HIGHER_THROUGHPUT) + // .withWriterType(WriterType.BATCHED) + // .withMaxNumOfUsedWorkers(1) + // .withNumberOfClientsPerWorker(1)); + // + // results.getSuccessfulPublish() + // .apply( + // "Successful records", + // ParDo.of( + // new DoFn() { + // @ProcessElement + // public void processElement(ProcessContext c) { + // System.out.println("OK: " + c.element()); + // c.output(1); + // } + // })); + // + // results.getFailedPublish() + // .apply( + // "Failed records", + // ParDo.of( + // new DoFn() { + // @ProcessElement + // public void processElement(ProcessContext c) { + // System.out.println("Failed: " + c.element()); + // c.output(1); + // } + // })); + // + // pipeline.run().waitUntilFinish(); + // } - // private static Record buildRecord(String id, String payload) { - // return Record.builder() - // .setMessageId(id) - // .setPayload(payload.getBytes(StandardCharsets.UTF_8)) - // .setSenderTimestamp(1712224703L) - // .setDestination( - // Destination.builder() - // .setName("test_topic") - // .setType(DestinationType.TOPIC) - // .build()) - // .build(); - // } + // private static Record buildRecord(String id, String payload) { + // return Record.builder() + // .setMessageId(id) + // .setPayload(payload.getBytes(StandardCharsets.UTF_8)) + // .setSenderTimestamp(1712224703L) + // .setDestination( + // Destination.builder() + // .setName("test_topic") + // .setType(DestinationType.TOPIC) + // .build()) + // .build(); + // } } diff --git a/settings.gradle.kts b/settings.gradle.kts index cca547c9e04e..0a080e489afe 100644 --- a/settings.gradle.kts +++ b/settings.gradle.kts @@ -357,3 +357,5 @@ include("sdks:java:managed") findProject(":sdks:java:managed")?.name = "managed" include("sdks:java:io:iceberg") findProject(":sdks:java:io:iceberg")?.name = "iceberg" +include("sdks:java:io:solace") +findProject(":sdks:java:io:solace")?.name = "solace" From 399ffccd981305da21985eb877bb13f2d087bc43 Mon Sep 17 00:00:00 2001 From: bzablocki Date: Wed, 17 Apr 2024 14:23:34 +0200 Subject: [PATCH 03/41] some checker errors resolved --- .../broker/BasicAuthJcsmpSessionService.java | 16 ++- .../BasicAuthJcsmpSessionServiceFactory.java | 78 ++++++------ .../io/solace/broker/BasicAuthSempClient.java | 63 +++------- .../broker/BasicAuthSempClientFactory.java | 114 ++++++------------ .../solace/broker/SessionServiceFactory.java | 3 +- .../sdk/io/solace/MockSempClientFactory.java | 4 +- .../beam/sdk/io/solace/SolaceIOTest.java | 2 +- .../SempBasicAuthClientExecutorTest.java | 29 +++++ .../beam/sdk/io/solace/it/SolaceIOIT.java | 16 +-- 9 files changed, 142 insertions(+), 183 deletions(-) diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/BasicAuthJcsmpSessionService.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/BasicAuthJcsmpSessionService.java index 2843f24cf25e..79a6de23092d 100644 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/BasicAuthJcsmpSessionService.java +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/BasicAuthJcsmpSessionService.java @@ -19,6 +19,7 @@ import com.solacesystems.jcsmp.ConsumerFlowProperties; import com.solacesystems.jcsmp.EndpointProperties; +import com.solacesystems.jcsmp.FlowReceiver; import com.solacesystems.jcsmp.InvalidPropertiesException; import com.solacesystems.jcsmp.JCSMPException; import com.solacesystems.jcsmp.JCSMPFactory; @@ -93,12 +94,23 @@ private MessageReceiver createFlowReceiver() throws JCSMPException, IOException endpointProperties.setAccessType(EndpointProperties.ACCESSTYPE_NONEXCLUSIVE); if (jcsmpSession != null) { return new SolaceMessageReceiver( - jcsmpSession.createFlow(null, flowProperties, endpointProperties)); + createFlowReceiver(jcsmpSession, flowProperties, endpointProperties)); } else { throw new IOException( - "SolaceIO.Read: Could not create a receiver from the Jcsmp session: session is null."); + "SolaceIO.Read: Could not create a receiver from the Jcsmp session: session object is null."); } } + // The `@SuppressWarning` is needed here, because the checkerframework reports an error for the + // first argument of the `createFlow` being null, even though the documentation allows it: + // https://docs.solace.com/API-Developer-Online-Ref-Documentation/java/com/solacesystems/jcsmp/JCSMPSession.html#createFlow-com.solacesystems.jcsmp.XMLMessageListener-com.solacesystems.jcsmp.ConsumerFlowProperties-com.solacesystems.jcsmp.EndpointProperties- + @SuppressWarnings("nullness") + private static FlowReceiver createFlowReceiver( + JCSMPSession jcsmpSession, + ConsumerFlowProperties flowProperties, + EndpointProperties endpointProperties) + throws JCSMPException { + return jcsmpSession.createFlow(null, flowProperties, endpointProperties); + } private int connectSession() throws JCSMPException { if (jcsmpSession == null) { diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/BasicAuthJcsmpSessionServiceFactory.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/BasicAuthJcsmpSessionServiceFactory.java index a8f727f97acd..c9a754c41bba 100644 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/BasicAuthJcsmpSessionServiceFactory.java +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/BasicAuthJcsmpSessionServiceFactory.java @@ -17,61 +17,51 @@ */ package org.apache.beam.sdk.io.solace.broker; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; +import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkNotNull; -public class BasicAuthJcsmpSessionServiceFactory extends SessionServiceFactory { - private final String host; - private final String username; - private final String password; - private final String vpnName; +import com.google.auto.value.AutoValue; - private BasicAuthJcsmpSessionServiceFactory( - String host, String username, String password, String vpnName) { - this.host = host; - this.username = username; - this.password = password; - this.vpnName = vpnName; - } +@AutoValue +public abstract class BasicAuthJcsmpSessionServiceFactory extends SessionServiceFactory { + public abstract String host(); - public static BasicAuthJcsmpSessionServiceFactoryBuilder builder() { - return new BasicAuthJcsmpSessionServiceFactoryBuilder(); - } + public abstract String username(); - @Override - public SessionService create() { - Preconditions.checkState(queue != null, "SolaceIO.Read: Queue is not set."); - return new BasicAuthJcsmpSessionService(queue.getName(), host, username, password, vpnName); + public abstract String password(); + + public abstract String vpnName(); + + public static Builder builder() { + return new AutoValue_BasicAuthJcsmpSessionServiceFactory.Builder(); } - public static class BasicAuthJcsmpSessionServiceFactoryBuilder { + @AutoValue.Builder + public abstract static class Builder { - private String host; - private String username; - private String password; - private String vpnName; + /** Set Solace host, format: Protocol://Host[:Port] */ + public abstract Builder host(String host); - public BasicAuthJcsmpSessionServiceFactoryBuilder withHost(String host) { - this.host = host; - return this; - } + /** Set Solace username */ + public abstract Builder username(String username); + /** Set Solace password */ + public abstract Builder password(String password); - public BasicAuthJcsmpSessionServiceFactoryBuilder withUsername(String username) { - this.username = username; - return this; - } + /** Set Solace vpn name */ + public abstract Builder vpnName(String vpnName); - public BasicAuthJcsmpSessionServiceFactoryBuilder withPassword(String password) { - this.password = password; - return this; - } + public abstract BasicAuthJcsmpSessionServiceFactory build(); + } - public BasicAuthJcsmpSessionServiceFactoryBuilder withVpnName(String vpnName) { - this.vpnName = vpnName; - return this; - } + @Override + public SessionService create() { + checkNotNull(queue, "SolaceIO.Read: Queue is not set."); + return createBasicAuthJcsmpSessionService( + queue.getName(), host(), username(), password(), vpnName()); + } - public BasicAuthJcsmpSessionServiceFactory build() { - return new BasicAuthJcsmpSessionServiceFactory(host, username, password, vpnName); - } + @SuppressWarnings("dereference") + private BasicAuthJcsmpSessionService createBasicAuthJcsmpSessionService( + String queueName, String host, String username, String password, String vpnName) { + return new BasicAuthJcsmpSessionService(queueName, host, username, password, vpnName); } } diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/BasicAuthSempClient.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/BasicAuthSempClient.java index 974e5d1884b0..fa84389e8281 100644 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/BasicAuthSempClient.java +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/BasicAuthSempClient.java @@ -17,6 +17,7 @@ */ package org.apache.beam.sdk.io.solace.broker; +import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.DeserializationFeature; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.annotation.JsonDeserialize; @@ -54,72 +55,44 @@ public BasicAuthSempClient( public boolean isQueueNonExclusive(String queueName) throws IOException { LOG.info("SolaceIO.Read: SempOperations: query SEMP if queue {} is nonExclusive", queueName); BrokerResponse response = sempBasicAuthClientExecutor.getQueueResponse(queueName); - try { - Queue q = getSuccessResponseOrThrowException(response, Queue.class); - return q.data().accessType().equals("non-exclusive"); - } catch (SolaceSempException e) { - throw new IOException("SolaceIO.Read: Exception when querying for Queue metadata.", e); + if (response.content == null) { + throw new IOException("SolaceIO: response from SEMP is empty!"); } + Queue q = mapJsonToClass(response.content, Queue.class); + return q.data().accessType().equals("non-exclusive"); } @Override public com.solacesystems.jcsmp.Queue createQueueForTopic(String queueName, String topicName) throws IOException { - try { - createQueue(queueName); - createSubscription(queueName, topicName); - } catch (SolaceSempException e) { - throw new IOException("SolaceIO.Read: Exception when creating a Queue for a Topic.", e); - } + createQueue(queueName); + createSubscription(queueName, topicName); return JCSMPFactory.onlyInstance().createQueue(queueName); } @Override public long getBacklogBytes(String queueName) throws IOException { BrokerResponse response = sempBasicAuthClientExecutor.getQueueResponse(queueName); - try { - Queue q = getSuccessResponseOrThrowException(response, Queue.class); - return q.data().msgSpoolUsage(); - } catch (SolaceSempException e) { - throw new IOException("SolaceIO.Read: Exception when querying for backlog bytes.", e); + if (response.content == null) { + throw new IOException("SolaceIO: response from SEMP is empty!"); } + Queue q = mapJsonToClass(response.content, Queue.class); + return q.data().msgSpoolUsage(); } - private void createQueue(String queueName) throws SolaceSempException, IOException { + private void createQueue(String queueName) throws IOException { LOG.info("SolaceIO.Read: Creating new queue {}.", queueName); - BrokerResponse response = sempBasicAuthClientExecutor.createQueueResponse(queueName); - checkIfError(response); + sempBasicAuthClientExecutor.createQueueResponse(queueName); } - private void createSubscription(String queueName, String topicName) - throws SolaceSempException, IOException { + private void createSubscription(String queueName, String topicName) throws IOException { LOG.info("SolaceIO.Read: Creating new subscription {} for topic {}.", queueName, topicName); - BrokerResponse response = - sempBasicAuthClientExecutor.createSubscriptionResponse(queueName, topicName); - checkIfError(response); - } - - private T getSuccessResponseOrThrowException( - BrokerResponse response, Class mapSuccessToClass) throws IOException, SolaceSempException { - checkIfError(response); - return objectMapper.readValue(response.content, mapSuccessToClass); - } - - private void checkIfError(BrokerResponse response) - throws SolaceSempException, IOException { // todo do we still need this? - if (response.code < 200 || response.code > 299) { - ErrorMessage error = objectMapper.readValue(response.content, ErrorMessage.class); - throw new SolaceSempException(error); - } + sempBasicAuthClientExecutor.createSubscriptionResponse(queueName, topicName); } - static class SolaceSempException extends Exception { - public final ErrorMessage errorMessage; - - SolaceSempException(ErrorMessage errorMessage) { - super(errorMessage.meta().error().description()); - this.errorMessage = errorMessage; - } + private T mapJsonToClass(String content, Class mapSuccessToClass) + throws JsonProcessingException { + return objectMapper.readValue(content, mapSuccessToClass); } @AutoValue diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/BasicAuthSempClientFactory.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/BasicAuthSempClientFactory.java index d02200065864..ea05a54fc51d 100644 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/BasicAuthSempClientFactory.java +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/BasicAuthSempClientFactory.java @@ -19,105 +19,59 @@ import com.google.api.client.http.HttpRequestFactory; import com.google.api.client.http.javanet.NetHttpTransport; +import com.google.auto.value.AutoValue; import org.apache.beam.sdk.io.solace.SerializableSupplier; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; +import org.checkerframework.checker.nullness.qual.NonNull; +import org.checkerframework.checker.nullness.qual.Nullable; -public class BasicAuthSempClientFactory implements SempClientFactory { +@AutoValue +public abstract class BasicAuthSempClientFactory implements SempClientFactory { - private final String host; - private final String username; - private final String password; - private final String vpnName; - private final SerializableSupplier httpRequestFactorySupplier; + public abstract String host(); - private BasicAuthSempClientFactory( - String host, - String username, - String password, - String vpnName, - SerializableSupplier httpRequestFactorySupplier) { - this.host = host; - this.username = username; - this.password = password; - this.vpnName = vpnName; - this.httpRequestFactorySupplier = httpRequestFactorySupplier; - } + public abstract String username(); - public static BasicAuthSempAuthenticationFactoryBuilder builder() { - return new BasicAuthSempAuthenticationFactoryBuilder() - .withHttpRequestFactorySupplier(() -> new NetHttpTransport().createRequestFactory()); - } + public abstract String password(); - @Override - public SempClient create() { - return new BasicAuthSempClient(host, username, password, vpnName, httpRequestFactorySupplier); - } + public abstract String vpnName(); - public static class BasicAuthSempAuthenticationFactoryBuilder { + public abstract @Nullable SerializableSupplier httpRequestFactorySupplier(); - private String host; - private String username; - private String password; - private String vpnName; - private SerializableSupplier httpRequestFactorySupplier; + public static Builder builder() { + return new AutoValue_BasicAuthSempClientFactory.Builder(); + } + @AutoValue.Builder + public abstract static class Builder { /** Set Solace host, format: Protocol://Host[:Port] */ - public BasicAuthSempAuthenticationFactoryBuilder withHost(String host) { - this.host = host; - return this; - } + public abstract Builder host(String host); /** Set Solace username */ - public BasicAuthSempAuthenticationFactoryBuilder withUsername(String username) { - this.username = username; - return this; - } - + public abstract Builder username(String username); /** Set Solace password */ - public BasicAuthSempAuthenticationFactoryBuilder withPassword(String password) { - this.password = password; - return this; - } + public abstract Builder password(String password); /** Set Solace vpn name */ - public BasicAuthSempAuthenticationFactoryBuilder withVpnName(String vpnName) { - this.vpnName = vpnName; - return this; - } + public abstract Builder vpnName(String vpnName); @VisibleForTesting - BasicAuthSempAuthenticationFactoryBuilder withHttpRequestFactorySupplier( - SerializableSupplier httpRequestFactorySupplier) { - this.httpRequestFactorySupplier = httpRequestFactorySupplier; - return this; - } + abstract Builder httpRequestFactorySupplier( + SerializableSupplier httpRequestFactorySupplier); + + public abstract BasicAuthSempClientFactory build(); + } - public BasicAuthSempClientFactory build() { - // todo update name in the error string - Preconditions.checkState( - host != null, - "SolaceIO: host in BasicAuthSempAuthenticationFactory can't be null. Set it" - + " with `withHost()` method."); - Preconditions.checkState( - username != null, - "SolaceIO: username in BasicAuthSempAuthenticationFactory can't be null. Set it" - + " with `withUsername()` method."); - Preconditions.checkState( - password != null, - "SolaceIO: password in BasicAuthSempAuthenticationFactory can't be null. Set it" - + " with `withPassword()` method."); - Preconditions.checkState( - vpnName != null, - "SolaceIO: vpnName in BasicAuthSempAuthenticationFactory can't be null. Set it" - + " with `withVpnName()` method."); - Preconditions.checkState( - httpRequestFactorySupplier != null, - "SolaceIO: httpRequestFactorySupplier in BasicAuthSempAuthenticationFactory" - + " can't be null. Set it with `withHttpRequestFactorySupplier()` method."); + @Override + public SempClient create() { + return new BasicAuthSempClient( + host(), username(), password(), vpnName(), getHttpRequestFactorySupplier()); + } - return new BasicAuthSempClientFactory( - host, username, password, vpnName, httpRequestFactorySupplier); - } + @SuppressWarnings("return") + private @NonNull SerializableSupplier getHttpRequestFactorySupplier() { + return httpRequestFactorySupplier() != null + ? httpRequestFactorySupplier() + : () -> new NetHttpTransport().createRequestFactory(); } } diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SessionServiceFactory.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SessionServiceFactory.java index 5e31926fa0fb..d4f698d8299c 100644 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SessionServiceFactory.java +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SessionServiceFactory.java @@ -19,9 +19,10 @@ import com.solacesystems.jcsmp.Queue; import java.io.Serializable; +import org.checkerframework.checker.nullness.qual.Nullable; public abstract class SessionServiceFactory implements Serializable { - Queue queue; + @Nullable Queue queue; public abstract SessionService create(); diff --git a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/MockSempClientFactory.java b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/MockSempClientFactory.java index 4f7b353c8181..5d8849d5cb3c 100644 --- a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/MockSempClientFactory.java +++ b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/MockSempClientFactory.java @@ -21,9 +21,9 @@ import org.apache.beam.sdk.io.solace.broker.SempClientFactory; public class MockSempClientFactory implements SempClientFactory { - MockSempClient sempClient; + SempClient sempClient; - public MockSempClientFactory(MockSempClient sempClient) { + public MockSempClientFactory(SempClient sempClient) { this.sempClient = sempClient; } diff --git a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/SolaceIOTest.java b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/SolaceIOTest.java index b5d37572533e..c2ff329e71f7 100644 --- a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/SolaceIOTest.java +++ b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/SolaceIOTest.java @@ -561,7 +561,7 @@ public void testTopicEncoding() { .withSessionServiceFactory(fakeSessionServiceFactory) .withMaxNumConnections(2)); - // Run the pipelin + // Run the pipeline PCollection destAreTopics = events.apply( MapElements.into(TypeDescriptors.booleans()) diff --git a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/broker/SempBasicAuthClientExecutorTest.java b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/broker/SempBasicAuthClientExecutorTest.java index b0038e428654..efd1ac80c0bf 100644 --- a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/broker/SempBasicAuthClientExecutorTest.java +++ b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/broker/SempBasicAuthClientExecutorTest.java @@ -66,6 +66,35 @@ public LowLevelHttpResponse execute() { assertThrows(HttpResponseException.class, () -> client.getQueueResponse("queue")); } + @Test + public void testExecuteStatus3xx() { + MockHttpTransport transport = + new MockHttpTransport() { + @Override + public LowLevelHttpRequest buildRequest(String method, String url) { + return new MockLowLevelHttpRequest() { + @Override + public LowLevelHttpResponse execute() { + MockLowLevelHttpResponse response = new MockLowLevelHttpResponse(); + response.setStatusCode(301); + response.setContentType(Json.MEDIA_TYPE); + response.setContent( + "{\"meta\":{\"error\":{\"code\":301,\"description\":\"some" + + " error\",\"status\":\"xx\"}}}"); + return response; + } + }; + } + }; + + HttpRequestFactory requestFactory = transport.createRequestFactory(); + SempBasicAuthClientExecutor client = + new SempBasicAuthClientExecutor( + "http://host", "username", "password", "vpnName", requestFactory); + + assertThrows(HttpResponseException.class, () -> client.getQueueResponse("queue")); + } + /** * In this test case, we test a situation when a session that we used to authenticate to Semp * expires. diff --git a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/it/SolaceIOIT.java b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/it/SolaceIOIT.java index fff61b328a63..bb9e0f20e8bc 100644 --- a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/it/SolaceIOIT.java +++ b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/it/SolaceIOIT.java @@ -100,17 +100,17 @@ public void test() { .withMaxNumConnections(1) .withSempClientFactory( BasicAuthSempClientFactory.builder() - .withHost("http://localhost:8080") - .withUsername("admin") - .withPassword("admin") - .withVpnName(SolaceContainerManager.VPN_NAME) + .host("http://localhost:8080") + .username("admin") + .password("admin") + .vpnName(SolaceContainerManager.VPN_NAME) .build()) .withSessionServiceFactory( BasicAuthJcsmpSessionServiceFactory.builder() - .withHost("localhost") - .withUsername(SolaceContainerManager.USERNAME) - .withPassword(SolaceContainerManager.PASSWORD) - .withVpnName(SolaceContainerManager.VPN_NAME) + .host("localhost") + .username(SolaceContainerManager.USERNAME) + .password(SolaceContainerManager.PASSWORD) + .vpnName(SolaceContainerManager.VPN_NAME) .build())); // PCollection count = PCollection records = From 62dfa4a29c53e39e419ce893d9409ed60bc65989 Mon Sep 17 00:00:00 2001 From: bzablocki Date: Wed, 17 Apr 2024 18:13:01 +0200 Subject: [PATCH 04/41] all checker errors resolved --- sdks/java/io/solace/build.gradle | 9 +- .../apache/beam/sdk/io/solace/SolaceIO.java | 112 +++++---- .../BasicAuthJcsmpSessionServiceFactory.java | 23 +- .../broker/BasicAuthSempClientFactory.java | 8 +- .../broker/SempBasicAuthClientExecutor.java | 2 +- .../solace/broker/SolaceMessageReceiver.java | 2 +- .../sdk/io/solace/broker/package-info.java | 2 + .../beam/sdk/io/solace/data/Solace.java | 116 ++------- .../sdk/io/solace/data/SolaceRecordCoder.java | 12 +- .../beam/sdk/io/solace/data/package-info.java | 2 + .../beam/sdk/io/solace/package-info.java | 2 + .../io/solace/read/SolaceCheckpointMark.java | 32 +-- .../io/solace/read/UnboundedSolaceReader.java | 40 +-- .../io/solace/read/UnboundedSolaceSource.java | 14 +- .../io/solace/read/WatermarkParameters.java | 2 +- .../sdk/io/solace/read/WatermarkPolicy.java | 11 +- .../beam/sdk/io/solace/read/package-info.java | 2 + .../io/solace/RetryCallableManagerTest.java | 23 +- .../beam/sdk/io/solace/SolaceIOTest.java | 2 +- .../SempBasicAuthClientExecutorTest.java | 2 +- .../beam/sdk/io/solace/data/SolaceTest.java | 82 ------- .../beam/sdk/io/solace/it/SolaceIOIT.java | 228 ++++-------------- 22 files changed, 224 insertions(+), 504 deletions(-) create mode 100644 sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/package-info.java create mode 100644 sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/data/package-info.java create mode 100644 sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/package-info.java create mode 100644 sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/package-info.java diff --git a/sdks/java/io/solace/build.gradle b/sdks/java/io/solace/build.gradle index 5a61315a6395..4a82b98c5ba4 100644 --- a/sdks/java/io/solace/build.gradle +++ b/sdks/java/io/solace/build.gradle @@ -37,9 +37,16 @@ dependencies { implementation library.java.vendored_guava_32_1_2_jre implementation library.java.gax implementation library.java.google_http_client - implementation 'com.google.cloud:google-cloud-secretmanager' + implementation library.java.google_http_client_gson implementation project(":sdks:java:extensions:avro") implementation library.java.avro + permitUnusedDeclared library.java.avro + implementation library.java.jackson_core + implementation library.java.jackson_databind + implementation library.java.google_api_common + implementation library.java.guava + permitUnusedDeclared library.java.guava + implementation library.java.threetenbp testImplementation library.java.activemq_amqp testImplementation library.java.activemq_broker diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/SolaceIO.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/SolaceIO.java index 3005dace8dc5..43cd4738696c 100644 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/SolaceIO.java +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/SolaceIO.java @@ -17,6 +17,9 @@ */ package org.apache.beam.sdk.io.solace; +import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkNotNull; +import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkState; + import com.google.auto.value.AutoValue; import com.solacesystems.jcsmp.BytesXMLMessage; import com.solacesystems.jcsmp.Destination; @@ -24,7 +27,6 @@ import com.solacesystems.jcsmp.Queue; import com.solacesystems.jcsmp.Topic; import java.io.IOException; -import java.util.Objects; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.CannotProvideCoderException; import org.apache.beam.sdk.coders.Coder; @@ -42,7 +44,7 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TypeDescriptor; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; +import org.checkerframework.checker.nullness.qual.NonNull; import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.Instant; import org.slf4j.Logger; @@ -61,9 +63,15 @@ */ public class SolaceIO { - public static final Logger LOG = LoggerFactory.getLogger(SolaceIO.class); public static final SerializableFunction SENDER_TIMESTAMP_FUNCTION = - (record) -> new Instant(record.getSenderTimestamp()); + (record) -> { + Long senderTimestamp = record != null ? record.getSenderTimestamp() : null; + if (senderTimestamp != null) { + return Instant.ofEpochMilli(senderTimestamp); + } else { + return Instant.now(); + } + }; private static final boolean DEFAULT_DEDUPLICATE_RECORDS = true; /** Get a {@link Topic} object from the topic name. */ @@ -85,9 +93,9 @@ static Queue queueFromName(String queueName) { */ public static Destination convertToJcsmpDestination(Solace.Destination destination) { if (destination.getType().equals(Solace.DestinationType.TOPIC)) { - return topicFromName(destination.getName()); + return topicFromName(checkNotNull(destination.getName())); } else if (destination.getType().equals(Solace.DestinationType.QUEUE)) { - return queueFromName(destination.getName()); + return queueFromName(checkNotNull(destination.getName())); } else { throw new IllegalArgumentException( "SolaceIO.Write: Unknown destination type: " + destination.getType()); @@ -106,22 +114,25 @@ public static Read read() { .build(); } /** - * Create a {@link Read} transform, to read from Solace. Specify a {@link SerializableFunction} - * to map incoming {@link BytesXMLMessage} records, to the object of your choice. You also need - * to specify a {@link TypeDescriptor} for your class and the timestamp function which returns - * an {@link Instant} from the record. + * Create a {@link Read} transform, to read from Solace. Specify a {@link SerializableFunction} to + * map incoming {@link BytesXMLMessage} records, to the object of your choice. You also need to + * specify a {@link TypeDescriptor} for your class and the timestamp function which returns an + * {@link Instant} from the record. * - *

The type descriptor will be used to infer a coder from CoderRegistry or Schema Registry. - * You can initialize a new TypeDescriptor in the following manner: + *

The type descriptor will be used to infer a coder from CoderRegistry or Schema Registry. You + * can initialize a new TypeDescriptor in the following manner: * *

{@code
-   *   TypeDescriptor typeDescriptor = TypeDescriptor.of(YourOutputType.class);
-   * }
+   * TypeDescriptor typeDescriptor = TypeDescriptor.of(YourOutputType.class);
+   * }
*/ public static Read read( TypeDescriptor typeDescriptor, - SerializableFunction parseFn, + SerializableFunction parseFn, SerializableFunction timestampFn) { + checkState(typeDescriptor != null, "SolaceIO.Read: typeDescriptor must not be null"); + checkState(parseFn != null, "SolaceIO.Read: parseFn must not be null"); + checkState(timestampFn != null, "SolaceIO.Read: timestampFn must not be null"); return Read.builder() .setTypeDescriptor(typeDescriptor) .setParseFn(parseFn) @@ -148,6 +159,10 @@ public Read from(Solace.Topic topic) { * used to calculate watermark and define record's timestamp. */ public Read withTimestampFn(SerializableFunction timestampFn) { + checkState( + timestampFn != null, + "SolaceIO.Read: timestamp function must be set or use the" + + " `Read.readSolaceRecords()` method"); return toBuilder().setTimestampFn(timestampFn).build(); } @@ -182,6 +197,7 @@ public Read withDeduplicateRecords(boolean deduplicateRecords) { * */ public Read withSempClientFactory(SempClientFactory sempClientFactory) { + checkState(sempClientFactory != null, "SolaceIO.Read: sempClientFactory must not be null."); return toBuilder().setSempClientFactory(sempClientFactory).build(); } @@ -200,6 +216,8 @@ public Read withSempClientFactory(SempClientFactory sempClientFactory) { * */ public Read withSessionServiceFactory(SessionServiceFactory sessionServiceFactory) { + checkState( + sessionServiceFactory != null, "SolaceIO.Read: sessionServiceFactory must not be null."); return toBuilder().setSessionServiceFactory(sessionServiceFactory).build(); } @@ -213,7 +231,7 @@ public Read withSessionServiceFactory(SessionServiceFactory sessionServiceFac abstract boolean getDeduplicateRecords(); - abstract SerializableFunction getParseFn(); + abstract SerializableFunction getParseFn(); abstract @Nullable SempClientFactory getSempClientFactory(); @@ -242,7 +260,7 @@ public abstract static class Builder { abstract Builder setDeduplicateRecords(boolean deduplicateRecords); - abstract Builder setParseFn(SerializableFunction parseFn); + abstract Builder setParseFn(SerializableFunction parseFn); abstract Builder setSempClientFactory(SempClientFactory brokerServiceFactory); @@ -253,16 +271,23 @@ public abstract static class Builder { abstract Read build(); } + private static T castIfNull(@Nullable T arg0, @NonNull T arg1) { + return arg0 != null ? arg0 : arg1; + } + @Override public PCollection expand(PBegin input) { - validate(); + checkState( + (getQueue() == null ^ getTopic() == null), + "SolaceIO.Read: One of the Solace {Queue, Topic} must be set."); - SempClientFactory sempClientFactory = getSempClientFactory(); + SempClientFactory sempClientFactory = + checkNotNull(getSempClientFactory(), "SolaceIO: sempClientFactory is null."); String jobName = input.getPipeline().getOptions().getJobName(); - Queue queue = - getQueue() != null ? getQueue() : initializeQueueForTopic(jobName, sempClientFactory); + Queue queue = castIfNull(getQueue(), initializeQueueForTopic(jobName, sempClientFactory)); - SessionServiceFactory sessionServiceFactory = getSessionServiceFactory(); + SessionServiceFactory sessionServiceFactory = + checkNotNull(getSessionServiceFactory(), "SolaceIO: sessionServiceFactory is null."); sessionServiceFactory.setQueue(queue); registerDefaultCoder(input.getPipeline()); @@ -278,8 +303,8 @@ public PCollection expand(PBegin input) { getMaxNumConnections(), getDeduplicateRecords(), coder, - getTimestampFn(), - getParseFn()))); + checkNotNull(getTimestampFn()), + checkNotNull(getParseFn())))); } private static void registerDefaultCoder(Pipeline pipeline) { @@ -301,12 +326,12 @@ Coder inferCoder(Pipeline pipeline) { } throw new RuntimeException( - "SolaceIO.Read: Cannot infer a coder for the TypeDescriptor. Annotate you" + "SolaceIO.Read: Cannot infer a coder for the TypeDescriptor. Annotate your" + " output class with @DefaultSchema annotation or create a coder manually" + " and register it in the CoderRegistry."); } - private Coder getFromSchemaRegistry(Pipeline pipeline) { + private @Nullable Coder getFromSchemaRegistry(Pipeline pipeline) { try { return pipeline.getSchemaRegistry().getSchemaCoder(getTypeDescriptor()); } catch (NoSuchSchemaException e) { @@ -314,7 +339,7 @@ private Coder getFromSchemaRegistry(Pipeline pipeline) { } } - private Coder getFromCoderRegistry(Pipeline pipeline) { + private @Nullable Coder getFromCoderRegistry(Pipeline pipeline) { try { return pipeline.getCoderRegistry().getCoder(getTypeDescriptor()); } catch (CannotProvideCoderException e) { @@ -325,42 +350,25 @@ private Coder getFromCoderRegistry(Pipeline pipeline) { // FIXME: this is public only for the sake of testing, TODO: redesign test so this is // private public Queue initializeQueueForTopic(String jobName, SempClientFactory sempClientFactory) { - Queue q; - if (getQueue() != null) { - q = getQueue(); + Queue initializedQueue; + Queue solaceQueue = getQueue(); + if (solaceQueue != null) { + return solaceQueue; } else { String queueName = String.format("queue-%s-%s", getTopic(), jobName); try { - String topicName = Objects.requireNonNull(getTopic()).getName(); - q = sempClientFactory.create().createQueueForTopic(queueName, topicName); + String topicName = checkNotNull(getTopic()).getName(); + initializedQueue = sempClientFactory.create().createQueueForTopic(queueName, topicName); LOG.info( "SolaceIO.Read: A new queue {} was created. The Queue will not be" + " deleted when this job finishes. Make sure to remove it yourself" + " when not needed.", - q.getName()); + initializedQueue.getName()); + return initializedQueue; } catch (IOException e) { throw new RuntimeException(e); } } - return q; - } - - private void validate() { - Preconditions.checkState( - getSempClientFactory() != null, "SolaceIO.Read: brokerServiceFactory must not be null."); - Preconditions.checkState( - getSessionServiceFactory() != null, - "SolaceIO.Read: SessionServiceFactory must not be null."); - Preconditions.checkState( - getParseFn() != null, - "SolaceIO.Read: parseFn must be set or use the `Read.readSolaceRecords()`" + " method"); - Preconditions.checkState( - getTimestampFn() != null, - "SolaceIO.Read: timestamp function must be set or use the" - + " `Read.readSolaceRecords()` method"); - Preconditions.checkState( - (getQueue() == null ^ getTopic() == null), - "SolaceIO.Read: One of the Solace {Queue, Topic} must be set."); } } } diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/BasicAuthJcsmpSessionServiceFactory.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/BasicAuthJcsmpSessionServiceFactory.java index c9a754c41bba..a1d994470eb6 100644 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/BasicAuthJcsmpSessionServiceFactory.java +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/BasicAuthJcsmpSessionServiceFactory.java @@ -38,15 +38,15 @@ public static Builder builder() { @AutoValue.Builder public abstract static class Builder { - /** Set Solace host, format: Protocol://Host[:Port] */ + /** Set Solace host, format: Protocol://Host[:Port]. */ public abstract Builder host(String host); - /** Set Solace username */ + /** Set Solace username. */ public abstract Builder username(String username); - /** Set Solace password */ + /** Set Solace password. */ public abstract Builder password(String password); - /** Set Solace vpn name */ + /** Set Solace vpn name. */ public abstract Builder vpnName(String vpnName); public abstract BasicAuthJcsmpSessionServiceFactory build(); @@ -54,14 +54,11 @@ public abstract static class Builder { @Override public SessionService create() { - checkNotNull(queue, "SolaceIO.Read: Queue is not set."); - return createBasicAuthJcsmpSessionService( - queue.getName(), host(), username(), password(), vpnName()); - } - - @SuppressWarnings("dereference") - private BasicAuthJcsmpSessionService createBasicAuthJcsmpSessionService( - String queueName, String host, String username, String password, String vpnName) { - return new BasicAuthJcsmpSessionService(queueName, host, username, password, vpnName); + return new BasicAuthJcsmpSessionService( + checkNotNull(queue, "SolaceIO.Read: Queue is not set.").getName(), + host(), + username(), + password(), + vpnName()); } } diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/BasicAuthSempClientFactory.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/BasicAuthSempClientFactory.java index ea05a54fc51d..8275bbd667be 100644 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/BasicAuthSempClientFactory.java +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/BasicAuthSempClientFactory.java @@ -44,15 +44,15 @@ public static Builder builder() { @AutoValue.Builder public abstract static class Builder { - /** Set Solace host, format: Protocol://Host[:Port] */ + /** Set Solace host, format: Protocol://Host[:Port]. */ public abstract Builder host(String host); - /** Set Solace username */ + /** Set Solace username. */ public abstract Builder username(String username); - /** Set Solace password */ + /** Set Solace password. */ public abstract Builder password(String password); - /** Set Solace vpn name */ + /** Set Solace vpn name. */ public abstract Builder vpnName(String vpnName); @VisibleForTesting diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SempBasicAuthClientExecutor.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SempBasicAuthClientExecutor.java index c1d22bf3419b..23f4d6401526 100644 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SempBasicAuthClientExecutor.java +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SempBasicAuthClientExecutor.java @@ -51,7 +51,7 @@ class SempBasicAuthClientExecutor implements Serializable { private final String messageVpn; private final String baseUrl; private final String password; - private final HttpRequestFactory requestFactory; + private final transient HttpRequestFactory requestFactory; SempBasicAuthClientExecutor( String host, diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SolaceMessageReceiver.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SolaceMessageReceiver.java index d80999c9926c..a993f6f26881 100644 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SolaceMessageReceiver.java +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SolaceMessageReceiver.java @@ -28,7 +28,7 @@ public class SolaceMessageReceiver implements MessageReceiver, Serializable { public static final int DEFAULT_ADVANCE_TIMEOUT_IN_MILLIS = 100; - private final FlowReceiver flowReceiver; + private final transient FlowReceiver flowReceiver; private final RetryCallableManager retryCallableManager = RetryCallableManager.create(); public SolaceMessageReceiver(FlowReceiver flowReceiver) { diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/package-info.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/package-info.java new file mode 100644 index 000000000000..ded1c2b002e0 --- /dev/null +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/package-info.java @@ -0,0 +1,2 @@ +/** Solace IO broker-related classes. */ +package org.apache.beam.sdk.io.solace.broker; diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/data/Solace.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/data/Solace.java index 66fc363eed1a..4e8d7d1118ca 100644 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/data/Solace.java +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/data/Solace.java @@ -90,7 +90,7 @@ public enum DestinationType { @DefaultSchema(AutoValueSchema.class) public abstract static class Destination { @SchemaFieldNumber("0") - public abstract String getName(); + public abstract @Nullable String getName(); @SchemaFieldNumber("1") public abstract DestinationType getType(); @@ -101,7 +101,7 @@ public static Builder builder() { @AutoValue.Builder public abstract static class Builder { - public abstract Builder setName(String name); + public abstract Builder setName(@Nullable String name); public abstract Builder setType(DestinationType type); @@ -162,121 +162,39 @@ public static Builder builder() { @AutoValue.Builder public abstract static class Builder { - public abstract Builder setMessageId(String messageId); + public abstract Builder setMessageId(@Nullable String messageId); public abstract Builder setPayload(byte[] payload); public abstract Builder setDestination(Destination destination); - public abstract Builder setExpiration(Long expiration); + public abstract Builder setExpiration(@Nullable Long expiration); - public abstract Builder setPriority(Integer priority); + public abstract Builder setPriority(@Nullable Integer priority); - public abstract Builder setRedelivered(Boolean redelivered); + public abstract Builder setRedelivered(@Nullable Boolean redelivered); - public abstract Builder setReplyTo(String replyTo); + public abstract Builder setReplyTo(@Nullable String replyTo); - public abstract Builder setReceiveTimestamp(Long receiveTimestamp); + public abstract Builder setReceiveTimestamp(@Nullable Long receiveTimestamp); - public abstract Builder setSenderTimestamp(Long senderTimestamp); + public abstract Builder setSenderTimestamp(@Nullable Long senderTimestamp); - public abstract Builder setSequenceNumber(Long sequenceNumber); + public abstract Builder setSequenceNumber(@Nullable Long sequenceNumber); - public abstract Builder setTimeToLive(Long timeToLive); + public abstract Builder setTimeToLive(@Nullable Long timeToLive); - public abstract Builder setReplicationGroupMessageId(String replicationGroupMessageId); + public abstract Builder setReplicationGroupMessageId( + @Nullable String replicationGroupMessageId); public abstract Record build(); } } - /** - * The result of writing a message to Solace. This will be returned by the {@link - * org.apache.beam.sdk.io.solace.SolaceIO.Write} connector. - * - *

This class provides a builder to create instances, but you will probably not need it. The - * write connector will create and return instances of {@link PublishResult}. - * - *

If the message has been published, {@link PublishResult#getPublished()} will be true. If it - * is false, it means that the message could not be published, and {@link - * PublishResult#getError()} will contain more details about why the message could not be - * published. - */ - @AutoValue - @DefaultSchema(AutoValueSchema.class) - public abstract static class PublishResult { - /** The message id of the message that was published. */ - @SchemaFieldNumber("0") - public abstract String getMessageId(); - - /** Whether the message was published or not. */ - @SchemaFieldNumber("1") - public abstract Boolean getPublished(); - - /** - * The publishing latency in milliseconds. This is the difference between the time the message - * was created, and the time the message was published. It is only available if the {@link - * CorrelationKey} class is used as correlation key of the messages. - */ - @SchemaFieldNumber("2") - public abstract @Nullable Long getLatencyMilliseconds(); - - /** The error details if the message could not be published. */ - @SchemaFieldNumber("3") - public abstract @Nullable String getError(); - - public static Builder builder() { - return new AutoValue_Solace_PublishResult.Builder(); - } - - @AutoValue.Builder - public abstract static class Builder { - public abstract Builder setMessageId(String messageId); - - public abstract Builder setPublished(Boolean published); - - public abstract Builder setLatencyMilliseconds(Long latencyMs); - - public abstract Builder setError(String error); - - public abstract PublishResult build(); - } - } - - /** - * The correlation key is an object that is passed back to the client during the event broker ack - * or nack. - * - *

In the streaming writer is optionally used to calculate publish latencies, by calculating - * the time difference between the creation of the correlation key, and the time of the ack. - */ - @AutoValue - @DefaultSchema(AutoValueSchema.class) - public abstract static class CorrelationKey { - @SchemaFieldNumber("0") - public abstract String getMessageId(); - - @SchemaFieldNumber("1") - public abstract long getPublishMonotonicMillis(); - - public static Builder builder() { - return new AutoValue_Solace_CorrelationKey.Builder(); - } - - @AutoValue.Builder - public abstract static class Builder { - public abstract Builder setMessageId(String messageId); - - public abstract Builder setPublishMonotonicMillis(long millis); - - public abstract CorrelationKey build(); - } - } - public static class SolaceRecordMapper { - public static final Logger LOG = LoggerFactory.getLogger(SolaceRecordMapper.class); + private static final Logger LOG = LoggerFactory.getLogger(SolaceRecordMapper.class); - public static Record map(@Nullable BytesXMLMessage msg) { + public static @Nullable Record map(@Nullable BytesXMLMessage msg) { if (msg == null) { return null; } @@ -304,9 +222,9 @@ public static Record map(@Nullable BytesXMLMessage msg) { com.solacesystems.jcsmp.Destination originalDestination = msg.getDestination(); Destination.Builder destBuilder = Destination.builder().setName(originalDestination.getName()); - if (originalDestination instanceof Topic) { + if (originalDestination instanceof com.solacesystems.jcsmp.Topic) { destBuilder.setType(DestinationType.TOPIC); - } else if (originalDestination instanceof Queue) { + } else if (originalDestination instanceof com.solacesystems.jcsmp.Queue) { destBuilder.setType(DestinationType.QUEUE); } else { LOG.error( diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/data/SolaceRecordCoder.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/data/SolaceRecordCoder.java index f45ef1b48447..589fda6a3895 100644 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/data/SolaceRecordCoder.java +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/data/SolaceRecordCoder.java @@ -29,7 +29,9 @@ import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.coders.VarIntCoder; import org.apache.beam.sdk.coders.VarLongCoder; +import org.apache.beam.sdk.io.solace.data.Solace.Destination; import org.apache.beam.sdk.io.solace.data.Solace.Record; +import org.checkerframework.checker.nullness.qual.NonNull; /** * Custom coder for the default Solace {@link Record} @@ -50,13 +52,15 @@ public static SolaceRecordCoder of() { } @Override - public void encode(Record value, OutputStream outStream) throws IOException { - // Check if the destination is a topic or a queue, and encode that info + public void encode(Record value, @NonNull OutputStream outStream) throws IOException { STRING_CODER.encode(value.getMessageId(), outStream); STRING_CODER.encode(value.getReplicationGroupMessageId(), outStream); BYTE_CODER.encode(value.getPayload(), outStream); - STRING_CODER.encode(value.getDestination().getName(), outStream); - STRING_CODER.encode(value.getDestination().getType().toString(), outStream); + Destination destination = value.getDestination(); + String destinationName = destination == null ? null : destination.getName(); + String typeName = destination == null ? null : destination.getType().toString(); + STRING_CODER.encode(destinationName, outStream); + STRING_CODER.encode(typeName, outStream); LONG_CODER.encode(value.getExpiration(), outStream); INTEGER_CODER.encode(value.getPriority(), outStream); BOOLEAN_CODER.encode(value.getRedelivered(), outStream); diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/data/package-info.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/data/package-info.java new file mode 100644 index 000000000000..edde1a8665c3 --- /dev/null +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/data/package-info.java @@ -0,0 +1,2 @@ +/** Solace IO connector - data-related classes. */ +package org.apache.beam.sdk.io.solace.data; diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/package-info.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/package-info.java new file mode 100644 index 000000000000..7f2662a463df --- /dev/null +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/package-info.java @@ -0,0 +1,2 @@ +/** Solace IO connector. */ +package org.apache.beam.sdk.io.solace; diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/SolaceCheckpointMark.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/SolaceCheckpointMark.java index 956de7ce5a84..8ac774a35f1e 100644 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/SolaceCheckpointMark.java +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/SolaceCheckpointMark.java @@ -18,17 +18,17 @@ package org.apache.beam.sdk.io.solace.read; import com.solacesystems.jcsmp.BytesXMLMessage; +import java.util.ArrayDeque; +import java.util.ArrayList; import java.util.List; import java.util.Objects; -import java.util.concurrent.ConcurrentLinkedDeque; import java.util.concurrent.atomic.AtomicBoolean; import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.sdk.coders.DefaultCoder; import org.apache.beam.sdk.extensions.avro.coders.AvroCoder; import org.apache.beam.sdk.io.UnboundedSource; +import org.checkerframework.checker.nullness.qual.NonNull; import org.checkerframework.checker.nullness.qual.Nullable; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** * Checkpoint for an unbounded Solace source. Consists of the Solace messages waiting to be @@ -37,20 +37,16 @@ @Internal @DefaultCoder(AvroCoder.class) public class SolaceCheckpointMark implements UnboundedSource.CheckpointMark { - private static final Logger LOG = LoggerFactory.getLogger(SolaceCheckpointMark.class); - - private transient AtomicBoolean activeReader; - @Nullable private transient ConcurrentLinkedDeque ackQueue; + private transient @Nullable AtomicBoolean activeReader; + private ArrayDeque ackQueue; @SuppressWarnings("initialization") // Avro will set the fields by breaking abstraction private SolaceCheckpointMark() {} // for Avro public SolaceCheckpointMark( - @Nullable AtomicBoolean activeReader, List ackQueue) { + @Nullable AtomicBoolean activeReader, @NonNull List ackQueue) { this.activeReader = activeReader; - if (ackQueue != null) { - this.ackQueue = new ConcurrentLinkedDeque<>(ackQueue); - } + this.ackQueue = new ArrayDeque<>(ackQueue); } @Override @@ -59,11 +55,6 @@ public void finalizeCheckpoint() { return; } - LOG.debug( - "SolaceIO.Read: SolaceCheckpointMark: Started to finalize {} with {} messages.", - this.getClass().getSimpleName(), - ackQueue.size()); - while (ackQueue.size() > 0) { BytesXMLMessage msg = ackQueue.poll(); if (msg != null) { @@ -73,7 +64,7 @@ public void finalizeCheckpoint() { } @Override - public boolean equals(Object o) { + public boolean equals(@Nullable Object o) { if (this == o) { return true; } @@ -81,8 +72,11 @@ public boolean equals(Object o) { return false; } SolaceCheckpointMark that = (SolaceCheckpointMark) o; - return Objects.equals(activeReader, that.activeReader) - && Objects.equals(ackQueue, that.ackQueue); + // Needed to convert to ArrayList because ArrayDeque.equals checks only for reference, not + // content. + ArrayList ackList = new ArrayList<>(ackQueue); + ArrayList thatAckList = new ArrayList<>(that.ackQueue); + return Objects.equals(activeReader, that.activeReader) && Objects.equals(ackList, thatAckList); } @Override diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/UnboundedSolaceReader.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/UnboundedSolaceReader.java index 88fb2e9e6e43..0155345a2323 100644 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/UnboundedSolaceReader.java +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/UnboundedSolaceReader.java @@ -17,7 +17,8 @@ */ package org.apache.beam.sdk.io.solace.read; -import com.google.common.annotations.VisibleForTesting; +import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkNotNull; + import com.solacesystems.jcsmp.BytesXMLMessage; import java.io.IOException; import java.nio.charset.StandardCharsets; @@ -32,6 +33,8 @@ import org.apache.beam.sdk.io.solace.broker.SempClient; import org.apache.beam.sdk.io.solace.broker.SessionService; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; +import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.Instant; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -43,12 +46,12 @@ class UnboundedSolaceReader extends UnboundedReader { private static final Logger LOG = LoggerFactory.getLogger(UnboundedSolaceReader.class); private final UnboundedSolaceSource currentSource; private final WatermarkPolicy watermarkPolicy; - AtomicBoolean active = new AtomicBoolean(true); - private BytesXMLMessage solaceOriginalRecord; - private T solaceMappedRecord; - private MessageReceiver messageReceiver; - private SessionService sessionService; private final SempClient sempClient; + private @Nullable BytesXMLMessage solaceOriginalRecord; + private @Nullable T solaceMappedRecord; + private @Nullable MessageReceiver messageReceiver; + private @Nullable SessionService sessionService; + AtomicBoolean active = new AtomicBoolean(true); /** * Queue to place advanced messages before {@link #getCheckpointMark()} be called non-concurrent @@ -76,17 +79,18 @@ public void populateSession() { sessionService = getCurrentSource().getSessionServiceFactory().create(); } if (sessionService.isClosed()) { - sessionService.connect(); + checkNotNull(sessionService).connect(); } } private void populateMessageConsumer() { if (messageReceiver == null) { - messageReceiver = sessionService.createReceiver(); + messageReceiver = checkNotNull(sessionService).createReceiver(); messageReceiver.start(); } - if (messageReceiver.isClosed()) { - messageReceiver.start(); + MessageReceiver receiver = checkNotNull(messageReceiver); + if (receiver.isClosed()) { + receiver.start(); } } @@ -94,7 +98,7 @@ private void populateMessageConsumer() { public boolean advance() { BytesXMLMessage receivedXmlMessage; try { - receivedXmlMessage = messageReceiver.receive(); + receivedXmlMessage = checkNotNull(messageReceiver).receive(); } catch (IOException e) { LOG.warn("SolaceIO.Read: Exception when pulling messages from the broker.", e); return false; @@ -113,13 +117,13 @@ public boolean advance() { @Override public void close() { active.set(false); - sessionService.close(); + checkNotNull(sessionService).close(); } @Override public Instant getWatermark() { // should be only used by a test receiver - if (messageReceiver.isEOF()) { + if (checkNotNull(messageReceiver).isEOF()) { return BoundedWindow.TIMESTAMP_MAX_VALUE; } return watermarkPolicy.getWatermark(); @@ -130,7 +134,9 @@ public UnboundedSource.CheckpointMark getCheckpointMark() { List ackQueue = new ArrayList<>(); while (!elementsToCheckpoint.isEmpty()) { BytesXMLMessage msg = elementsToCheckpoint.poll(); - ackQueue.add(msg); + if (msg != null) { + ackQueue.add(msg); + } } return new SolaceCheckpointMark(active, ackQueue); } @@ -149,9 +155,11 @@ public byte[] getCurrentRecordId() throws NoSuchElementException { throw new NoSuchElementException(); } if (solaceOriginalRecord.getApplicationMessageId() != null) { - return solaceOriginalRecord.getApplicationMessageId().getBytes(StandardCharsets.UTF_8); + return checkNotNull(solaceOriginalRecord) + .getApplicationMessageId() + .getBytes(StandardCharsets.UTF_8); } else { - return solaceOriginalRecord + return checkNotNull(solaceOriginalRecord) .getReplicationGroupMessageId() .toString() .getBytes(StandardCharsets.UTF_8); diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/UnboundedSolaceSource.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/UnboundedSolaceSource.java index 60630257e821..918b2fc5328b 100644 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/UnboundedSolaceSource.java +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/UnboundedSolaceSource.java @@ -22,7 +22,6 @@ import java.io.IOException; import java.util.ArrayList; import java.util.List; -import javax.annotation.Nullable; import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.extensions.avro.coders.AvroCoder; @@ -31,6 +30,7 @@ import org.apache.beam.sdk.io.solace.broker.SessionServiceFactory; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.SerializableFunction; +import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.Instant; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -40,13 +40,13 @@ public class UnboundedSolaceSource extends UnboundedSource coder; private final boolean enableDeduplication; private final SempClientFactory sempClientFactory; private final SessionServiceFactory sessionServiceFactory; private final SerializableFunction timestampFn; - private final SerializableFunction parseFn; + private final SerializableFunction parseFn; public Queue getQueue() { return queue; @@ -64,7 +64,7 @@ public SerializableFunction getTimestampFn() { return timestampFn; } - public SerializableFunction getParseFn() { + public SerializableFunction getParseFn() { return parseFn; } @@ -72,11 +72,11 @@ public UnboundedSolaceSource( Queue queue, SempClientFactory sempClientFactory, SessionServiceFactory sessionServiceFactory, - Integer maxNumConnections, + @Nullable Integer maxNumConnections, boolean enableDeduplication, Coder coder, SerializableFunction timestampFn, - SerializableFunction parseFn) { + SerializableFunction parseFn) { this.queue = queue; this.sempClientFactory = sempClientFactory; this.sessionServiceFactory = sessionServiceFactory; @@ -108,7 +108,7 @@ public List> split(int desiredNumSplits, PipelineOption } private List> getSolaceSources( - int desiredNumSplits, Integer maxNumConnections) { + int desiredNumSplits, @Nullable Integer maxNumConnections) { List> sourceList = new ArrayList<>(); int numSplits = maxNumConnections != null diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/WatermarkParameters.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/WatermarkParameters.java index 901db77c8bb2..48de8c5f31f2 100644 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/WatermarkParameters.java +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/WatermarkParameters.java @@ -17,7 +17,7 @@ */ package org.apache.beam.sdk.io.solace.read; -import static com.google.common.base.Preconditions.checkArgument; +import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; import com.google.auto.value.AutoValue; import java.io.Serializable; diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/WatermarkPolicy.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/WatermarkPolicy.java index 11ade16ce14d..582c20165da6 100644 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/WatermarkPolicy.java +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/WatermarkPolicy.java @@ -17,15 +17,13 @@ */ package org.apache.beam.sdk.io.solace.read; -import com.google.common.collect.Ordering; import java.io.Serializable; import org.apache.beam.sdk.transforms.SerializableFunction; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Ordering; +import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.Instant; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; public class WatermarkPolicy implements Serializable { - public static final Logger LOG = LoggerFactory.getLogger(WatermarkPolicy.class); private WatermarkParameters watermarkParameters; public static WatermarkPolicy create(SerializableFunction timestampFunction) { @@ -53,7 +51,10 @@ public Instant getWatermark() { return watermarkParameters.getCurrentWatermark(); } - public void update(T record) { + public void update(@Nullable T record) { + if (record == null) { + return; + } watermarkParameters = watermarkParameters .toBuilder() diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/package-info.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/package-info.java new file mode 100644 index 000000000000..4bcb8cabaac0 --- /dev/null +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/package-info.java @@ -0,0 +1,2 @@ +/** Solace IO connector - read connector classes. */ +package org.apache.beam.sdk.io.solace.read; diff --git a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/RetryCallableManagerTest.java b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/RetryCallableManagerTest.java index c743bdba27f1..be090d839ca1 100644 --- a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/RetryCallableManagerTest.java +++ b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/RetryCallableManagerTest.java @@ -21,22 +21,22 @@ import com.google.api.gax.retrying.RetrySettings; import com.google.cloud.RetryHelper.RetryHelperException; -import java.util.Set; import java.util.concurrent.Callable; import java.util.concurrent.atomic.AtomicInteger; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet; import org.junit.Before; import org.junit.Test; public class RetryCallableManagerTest { + private static final int NUMBER_OF_RETRIES = 4; + private static final int RETRY_INTERVAL_SECONDS = 0; + private static final int RETRY_MULTIPLIER = 2; + private static final int MAX_DELAY = 0; private RetryCallableManager retryCallableManager; @Before public void setUp() throws Exception { - int NUMBER_OF_RETRIES = 4; - int RETRY_INTERVAL_SECONDS = 0; - int RETRY_MULTIPLIER = 2; - int MAX_DELAY = 0; retryCallableManager = RetryCallableManager.builder() @@ -63,7 +63,8 @@ public void testRetryCallable_ReturnsExpected() { return executeCounter.get(); }; Integer result = - retryCallableManager.retryCallable(incrementingFunction, Set.of(MyException.class)); + retryCallableManager.retryCallable( + incrementingFunction, ImmutableSet.of(MyException.class)); assertTrue(String.format("Should return 2, instead returned %d.", result), result == 2); } @@ -78,7 +79,7 @@ public void testRetryCallable_RetriesExpectedNumberOfTimes() { } return executeCounter.get(); }; - retryCallableManager.retryCallable(incrementingFunction, Set.of(MyException.class)); + retryCallableManager.retryCallable(incrementingFunction, ImmutableSet.of(MyException.class)); assertTrue( String.format("Should run 2 times, instead ran %d times.", executeCounter.get()), executeCounter.get() == 2); @@ -92,7 +93,7 @@ public void testRetryCallable_ThrowsRetryHelperException() { throw new MyException(); } }; - retryCallableManager.retryCallable(incrementingFunction, Set.of(MyException.class)); + retryCallableManager.retryCallable(incrementingFunction, ImmutableSet.of(MyException.class)); } @Test @@ -107,7 +108,7 @@ public void testRetryCallable_ExecutionCountIsCorrectAfterMultipleExceptions() { return 0; }; try { - retryCallableManager.retryCallable(incrementingFunction, Set.of(MyException.class)); + retryCallableManager.retryCallable(incrementingFunction, ImmutableSet.of(MyException.class)); } catch (RetryHelperException e) { // ignore exception to check the executeCounter } @@ -125,7 +126,7 @@ public void testRetryCallable_ThrowsRetryHelperExceptionOnUnspecifiedException() } return 0; }; - retryCallableManager.retryCallable(incrementingFunction, Set.of(MyException.class)); + retryCallableManager.retryCallable(incrementingFunction, ImmutableSet.of(MyException.class)); } @Test @@ -143,7 +144,7 @@ public void testRetryCallable_ChecksForAllDefinedExceptions() { }; try { retryCallableManager.retryCallable( - incrementingFunction, Set.of(MyException.class, AnotherException.class)); + incrementingFunction, ImmutableSet.of(MyException.class, AnotherException.class)); } catch (RetryHelperException e) { // ignore exception to check the executeCounter } diff --git a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/SolaceIOTest.java b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/SolaceIOTest.java index c2ff329e71f7..8f02a850d01e 100644 --- a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/SolaceIOTest.java +++ b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/SolaceIOTest.java @@ -503,7 +503,7 @@ public void testCheckpointMarkSafety() throws Exception { @Test public void testCheckpointMarkDefaultCoder() throws Exception { - SolaceCheckpointMark checkpointMark = new SolaceCheckpointMark(null, null); + SolaceCheckpointMark checkpointMark = new SolaceCheckpointMark(null, new ArrayList<>()); Coder coder = new UnboundedSolaceSource<>(null, null, null, 0, false, null, null, null) .getCheckpointMarkCoder(); diff --git a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/broker/SempBasicAuthClientExecutorTest.java b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/broker/SempBasicAuthClientExecutorTest.java index efd1ac80c0bf..8cc48ed17ef6 100644 --- a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/broker/SempBasicAuthClientExecutorTest.java +++ b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/broker/SempBasicAuthClientExecutorTest.java @@ -30,9 +30,9 @@ import com.google.api.client.testing.http.MockHttpTransport; import com.google.api.client.testing.http.MockLowLevelHttpRequest; import com.google.api.client.testing.http.MockLowLevelHttpResponse; -import com.google.common.collect.ImmutableList; import java.io.IOException; import java.util.List; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.junit.Test; public class SempBasicAuthClientExecutorTest { diff --git a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/data/SolaceTest.java b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/data/SolaceTest.java index 7dcf12559e3b..b53e113086ac 100644 --- a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/data/SolaceTest.java +++ b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/data/SolaceTest.java @@ -129,86 +129,4 @@ public void testRecordNullability() { public void testRecordBuilder() { Solace.Record.builder().build(); } - - @Test - public void testPublishResultEquality() { - Solace.PublishResult obj1 = - Solace.PublishResult.builder() - .setPublished(redelivered) - .setLatencyMilliseconds(timestampMillis) - .setMessageId(messageId) - .setError(publishError) - .build(); - - Solace.PublishResult obj2 = - Solace.PublishResult.builder() - .setPublished(redelivered) - .setLatencyMilliseconds(timestampMillis) - .setMessageId(messageId) - .setError(publishError) - .build(); - - Solace.PublishResult obj3 = - Solace.PublishResult.builder() - .setPublished(!redelivered) - .setLatencyMilliseconds(timestampMillis) - .setMessageId(messageId) - .setError(publishError) - .build(); - - Assert.assertEquals(obj1, obj2); - Assert.assertNotEquals(obj1, obj3); - Assert.assertEquals(obj1.getPublished(), redelivered); - Assert.assertEquals(obj1.getLatencyMilliseconds().longValue(), timestampMillis); - Assert.assertEquals(obj1.getMessageId(), messageId); - Assert.assertEquals(obj1.getError(), publishError); - } - - @Test(expected = IllegalStateException.class) - public void testPublishResultBuilder() { - Solace.PublishResult.builder().build(); - } - - @Test - public void testPublishResultNullability() { - Solace.PublishResult obj = - Solace.PublishResult.builder().setMessageId(messageId).setPublished(redelivered).build(); - - Assert.assertNotNull(obj); - Assert.assertEquals(obj.getMessageId(), messageId); - Assert.assertEquals(obj.getPublished(), redelivered); - Assert.assertNull(obj.getLatencyMilliseconds()); - Assert.assertNull(obj.getError()); - } - - @Test - public void testCorrelationKeyEquality() { - Solace.CorrelationKey obj1 = - Solace.CorrelationKey.builder() - .setMessageId(messageId) - .setPublishMonotonicMillis(timestampMillis) - .build(); - - Solace.CorrelationKey obj2 = - Solace.CorrelationKey.builder() - .setMessageId(messageId) - .setPublishMonotonicMillis(timestampMillis) - .build(); - - Solace.CorrelationKey obj3 = - Solace.CorrelationKey.builder() - .setMessageId(messageId) - .setPublishMonotonicMillis(timestampMillis - 1L) - .build(); - - Assert.assertEquals(obj1, obj2); - Assert.assertNotEquals(obj1, obj3); - Assert.assertEquals(obj1.getMessageId(), messageId); - Assert.assertEquals(obj1.getPublishMonotonicMillis(), timestampMillis); - } - - @Test(expected = IllegalStateException.class) - public void testCorrelationKeyNullability() { - Solace.CorrelationKey.builder().build(); - } } diff --git a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/it/SolaceIOIT.java b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/it/SolaceIOIT.java index bb9e0f20e8bc..d8a1612eb951 100644 --- a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/it/SolaceIOIT.java +++ b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/it/SolaceIOIT.java @@ -17,55 +17,46 @@ */ package org.apache.beam.sdk.io.solace.it; -import static org.junit.Assert.assertNotEquals; +import static org.junit.Assert.assertEquals; -import java.nio.charset.StandardCharsets; import org.apache.beam.sdk.PipelineResult; import org.apache.beam.sdk.io.solace.SolaceIO; import org.apache.beam.sdk.io.solace.broker.BasicAuthJcsmpSessionServiceFactory; import org.apache.beam.sdk.io.solace.broker.BasicAuthSempClientFactory; import org.apache.beam.sdk.io.solace.data.Solace.Queue; -import org.apache.beam.sdk.io.solace.data.Solace.Record; +import org.apache.beam.sdk.metrics.Counter; +import org.apache.beam.sdk.metrics.Metrics; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.options.StreamingOptions; -import org.apache.beam.sdk.testing.PAssert; -import org.apache.beam.sdk.testing.SerializableMatcher; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.testing.TestPipelineOptions; -import org.apache.beam.sdk.transforms.Combine; -import org.apache.beam.sdk.transforms.Count; -import org.apache.beam.sdk.transforms.MapElements; -import org.apache.beam.sdk.transforms.SimpleFunction; -import org.apache.beam.sdk.transforms.windowing.FixedWindows; -import org.apache.beam.sdk.transforms.windowing.Window; -import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.testutils.metrics.MetricsReader; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; -import org.hamcrest.BaseMatcher; -import org.hamcrest.Description; import org.joda.time.Duration; import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.Rule; import org.junit.Test; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.testcontainers.shaded.org.apache.commons.lang3.builder.EqualsBuilder; public class SolaceIOIT { - private static final Logger LOG = LoggerFactory.getLogger(SolaceIOIT.class); + private static final String NAMESPACE = SolaceIOIT.class.getName(); + private static final String READ_COUNT = "read_count"; private static SolaceContainerManager solaceContainerManager; - private static final TestPipelineOptions testOptions; + private static final TestPipelineOptions readPipelineOptions; static { - testOptions = PipelineOptionsFactory.create().as(TestPipelineOptions.class); - testOptions.setBlockOnRun(false); + readPipelineOptions = PipelineOptionsFactory.create().as(TestPipelineOptions.class); + readPipelineOptions.setBlockOnRun(false); + readPipelineOptions.as(TestPipelineOptions.class).setBlockOnRun(false); + readPipelineOptions.as(StreamingOptions.class).setStreaming(false); } - @Rule public final transient TestPipeline pipeline = TestPipeline.fromOptions(testOptions); + @Rule public final TestPipeline readPipeline = TestPipeline.fromOptions(readPipelineOptions); @BeforeClass public static void setup() { - System.out.println("START"); solaceContainerManager = new SolaceContainerManager(); solaceContainerManager.start(); } @@ -78,22 +69,20 @@ public static void afterClass() { } @Test - public void test() { - // Similar approach to - // https://github.com/apache/beam/blob/812e98fac243bab2a88f6ea5fad6147ff8e54a97/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOIT.java#L216 + public void testRead() { String queueName = "test_queue"; solaceContainerManager.createQueueWithSubscriptionTopic(queueName); - solaceContainerManager.getQueueDetails(queueName); - String payload = "{\"field_str\":\"value\",\"field_int\":123}"; - solaceContainerManager.sendToTopic(payload, ImmutableList.of("Solace-Message-ID:m1")); - solaceContainerManager.sendToTopic(payload, ImmutableList.of("Solace-Message-ID:m2")); - solaceContainerManager.getQueueDetails(queueName); - - pipeline.getOptions().as(StreamingOptions.class).setStreaming(true); + // todo this is very slow, needs to be replaced with the SolaceIO.write connector. + int publishMessagesCount = 20; + for (int i = 0; i < publishMessagesCount; i++) { + solaceContainerManager.sendToTopic( + "{\"field_str\":\"value\",\"field_int\":123}", + ImmutableList.of("Solace-Message-ID:m" + i)); + } - PCollection events = - pipeline.apply( + readPipeline + .apply( "Read from Solace", SolaceIO.read() .from(Queue.fromName(queueName)) @@ -111,164 +100,31 @@ public void test() { .username(SolaceContainerManager.USERNAME) .password(SolaceContainerManager.PASSWORD) .vpnName(SolaceContainerManager.VPN_NAME) - .build())); - // PCollection count = - PCollection records = - events - .apply( - "PassThrough", - MapElements.via( - new SimpleFunction() { - @Override - public Record apply(Record s) { - System.out.println("passthrough rec: " + s); - return s; - } - // })).apply("Window", - // Window.into(CalendarWindows.years(1))); - })) - .apply("Window", Window.into(FixedWindows.of(Duration.standardSeconds(5)))); - - System.out.println("xxxxx"); - - PAssert.thatSingleton( - records.apply( - "Counting element", Combine.globally(Count.combineFn()).withoutDefaults())) - .isEqualTo(1L); + .build())) + .apply("Count", ParDo.of(new CountingFn<>(NAMESPACE, READ_COUNT))); - PAssert.that(records) - .containsInAnyOrder( - partialMatch( - Record.builder() - .setMessageId("m1") - .setPayload(payload.getBytes(StandardCharsets.UTF_8)) - .build())); + PipelineResult pipelineResult = readPipeline.run(); + // todo this needs to be parametrized when moved to the official repo and used with the + // :integrationTest plugin + pipelineResult.waitUntilFinish(Duration.standardSeconds(15)); - PipelineResult writeResult = pipeline.run(); - // removing this line causes the pipeline not ingest any data - PipelineResult.State writeState = writeResult.waitUntilFinish(Duration.standardSeconds(10)); - assertNotEquals(PipelineResult.State.FAILED, writeState); - - System.out.println("queue after pipeline"); - solaceContainerManager.getQueueDetails(queueName); + MetricsReader metricsReader = new MetricsReader(pipelineResult, NAMESPACE); + long actualRecordsCount = metricsReader.getCounterMetric(READ_COUNT); + assertEquals(publishMessagesCount, actualRecordsCount); } - private static SerializableMatcher partialMatch(Record expected) { - class Matcher extends BaseMatcher implements SerializableMatcher { - @Override - public boolean matches(Object item) { - LOG.info("matches!!!"); - System.out.println("matches"); - if (!(item instanceof Record)) { - return false; - } - - Record actual = (Record) item; - boolean partiallyEqual = - EqualsBuilder.reflectionEquals(actual, expected, "replicationGroupMessageId"); - System.out.println("expected.equals(actual): " + expected.equals(actual)); - System.out.println("partiallyEqual: " + partiallyEqual); - System.out.println("expected: " + expected); - System.out.println("actual: " + actual); + private static class CountingFn extends DoFn { - return true; + private final Counter elementCounter; - // for (Record needle : needles) { - // if (!haystack.contains(needle)) { - // return false; - // } - // } - // return true; - } + CountingFn(String namespace, String name) { + elementCounter = Metrics.counter(namespace, name); + } - @Override - public void describeTo(Description description) { - description.appendText("Contains all of: "); - description.appendText(expected.toString()); - } + @ProcessElement + public void processElement(@Element T record, OutputReceiver c) { + elementCounter.inc(1L); + c.output(record); } - System.out.println("new matcher"); - return new Matcher(); } - // @Test - // public void testWrite() { - // TestStream createEvents = - // TestStream.create(StringUtf8Coder.of()) - // .addElements("r1", "r2") - // .advanceWatermarkTo( - // Instant.ofEpochMilli(0L).plus(Duration.standardSeconds(10))) - // .addElements("r3", "r4") - // .advanceWatermarkToInfinity(); - // - // PCollection records = pipeline.apply(createEvents); - // - // SolacePublishResult results = - // records.apply( - // "map", - // ParDo.of( - // new DoFn() { - // @ProcessElement - // public void processElement(ProcessContext c) { - // // System.out.println("Failed: " + c.element()); - // c.output( - // buildRecord( - // c.element(), - // "payload_" + c.element())); - // } - // })) - // .apply( - // SolaceIO.writeSolaceRecords() - // .to(Topic.fromName("test_topic")) - // .withSessionPropertiesProvider( - // BasicAuthenticationProvider.builder() - // .username("xx") - // .password("xx") - // .host("localhost") - // .vpnName(solaceContainer.getVpn()) - // .build()) - // .withDeliveryMode(DeliveryMode.PERSISTENT) - // .withSubmissionMode(SubmissionMode.HIGHER_THROUGHPUT) - // .withWriterType(WriterType.BATCHED) - // .withMaxNumOfUsedWorkers(1) - // .withNumberOfClientsPerWorker(1)); - // - // results.getSuccessfulPublish() - // .apply( - // "Successful records", - // ParDo.of( - // new DoFn() { - // @ProcessElement - // public void processElement(ProcessContext c) { - // System.out.println("OK: " + c.element()); - // c.output(1); - // } - // })); - // - // results.getFailedPublish() - // .apply( - // "Failed records", - // ParDo.of( - // new DoFn() { - // @ProcessElement - // public void processElement(ProcessContext c) { - // System.out.println("Failed: " + c.element()); - // c.output(1); - // } - // })); - // - // pipeline.run().waitUntilFinish(); - // } - - // private static Record buildRecord(String id, String payload) { - // return Record.builder() - // .setMessageId(id) - // .setPayload(payload.getBytes(StandardCharsets.UTF_8)) - // .setSenderTimestamp(1712224703L) - // .setDestination( - // Destination.builder() - // .setName("test_topic") - // .setType(DestinationType.TOPIC) - // .build()) - // .build(); - // } } From ea202fb7564b520a7089c540308ecfbb48529a1e Mon Sep 17 00:00:00 2001 From: bzablocki Date: Thu, 18 Apr 2024 15:59:28 +0200 Subject: [PATCH 05/41] improving unit tests --- .../broker/BasicAuthSempClientFactory.java | 5 +- .../io/solace/RetryCallableManagerTest.java | 12 +-- .../beam/sdk/io/solace/SolaceDataUtils.java | 34 +++++++- .../beam/sdk/io/solace/SolaceIOTest.java | 87 ++++++++++++++----- 4 files changed, 104 insertions(+), 34 deletions(-) diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/BasicAuthSempClientFactory.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/BasicAuthSempClientFactory.java index 8275bbd667be..b2ae28c00314 100644 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/BasicAuthSempClientFactory.java +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/BasicAuthSempClientFactory.java @@ -70,8 +70,9 @@ public SempClient create() { @SuppressWarnings("return") private @NonNull SerializableSupplier getHttpRequestFactorySupplier() { - return httpRequestFactorySupplier() != null - ? httpRequestFactorySupplier() + SerializableSupplier httpRequestSupplier = httpRequestFactorySupplier(); + return httpRequestSupplier != null + ? httpRequestSupplier : () -> new NetHttpTransport().createRequestFactory(); } } diff --git a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/RetryCallableManagerTest.java b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/RetryCallableManagerTest.java index be090d839ca1..a747dbe3c4cb 100644 --- a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/RetryCallableManagerTest.java +++ b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/RetryCallableManagerTest.java @@ -36,7 +36,7 @@ public class RetryCallableManagerTest { private RetryCallableManager retryCallableManager; @Before - public void setUp() throws Exception { + public void setUp() { retryCallableManager = RetryCallableManager.builder() @@ -102,10 +102,7 @@ public void testRetryCallable_ExecutionCountIsCorrectAfterMultipleExceptions() { Callable incrementingFunction = () -> { executeCounter.incrementAndGet(); - if (true) { - throw new MyException(); - } - return 0; + throw new MyException(); }; try { retryCallableManager.retryCallable(incrementingFunction, ImmutableSet.of(MyException.class)); @@ -121,10 +118,7 @@ public void testRetryCallable_ExecutionCountIsCorrectAfterMultipleExceptions() { public void testRetryCallable_ThrowsRetryHelperExceptionOnUnspecifiedException() { Callable incrementingFunction = () -> { - if (true) { - throw new DoNotIgnoreException(); - } - return 0; + throw new DoNotIgnoreException(); }; retryCallableManager.retryCallable(incrementingFunction, ImmutableSet.of(MyException.class)); } diff --git a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/SolaceDataUtils.java b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/SolaceDataUtils.java index 7f0e04b7c516..ab7a64e0f702 100644 --- a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/SolaceDataUtils.java +++ b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/SolaceDataUtils.java @@ -25,6 +25,7 @@ import com.solacesystems.jcsmp.ReplicationGroupMessageId; import com.solacesystems.jcsmp.SDTMap; import com.solacesystems.jcsmp.User_Cos; +import com.solacesystems.jcsmp.impl.ReplicationGroupMessageIdImpl; import java.io.IOException; import java.io.InputStream; import java.nio.BufferUnderflowException; @@ -37,8 +38,11 @@ import org.apache.beam.sdk.schemas.JavaBeanSchema; import org.apache.beam.sdk.schemas.annotations.DefaultSchema; import org.apache.beam.sdk.transforms.SerializableFunction; +import org.checkerframework.checker.nullness.qual.Nullable; public class SolaceDataUtils { + public static final ReplicationGroupMessageId DEFAULT_REPLICATION_GROUP_ID = + new ReplicationGroupMessageIdImpl(1L, 136L); @DefaultSchema(JavaBeanSchema.class) public static class SimpleRecord { @@ -83,6 +87,17 @@ public String toString() { } public static Solace.Record getSolaceRecord(String payload, String messageId) { + return getSolaceRecord(payload, messageId, null); + } + + public static Solace.Record getSolaceRecord( + String payload, + String messageId, + @Nullable ReplicationGroupMessageId replicationGroupMessageId) { + String replicationGroupMessageIdString = + replicationGroupMessageId != null + ? replicationGroupMessageId.toString() + : DEFAULT_REPLICATION_GROUP_ID.toString(); return Solace.Record.builder() .setPayload(payload.getBytes(StandardCharsets.UTF_8)) .setMessageId(messageId) @@ -99,6 +114,7 @@ public static Solace.Record getSolaceRecord(String payload, String messageId) { .setSequenceNumber(null) .setTimeToLive(1000L) .setSenderTimestamp(null) + .setReplicationGroupMessageId(replicationGroupMessageIdString) .build(); } @@ -107,16 +123,29 @@ public static BytesXMLMessage getBytesXmlMessage(String payload) { } public static BytesXMLMessage getBytesXmlMessage(String payload, String messageId) { - return getBytesXmlMessage(payload, messageId, null); + return getBytesXmlMessage(payload, messageId, null, null); } public static BytesXMLMessage getBytesXmlMessage( String payload, String messageId, SerializableFunction ackMessageFn) { + + return getBytesXmlMessage(payload, messageId, ackMessageFn, null); + } + + public static BytesXMLMessage getBytesXmlMessage( + String payload, + String messageId, + SerializableFunction ackMessageFn, + ReplicationGroupMessageId replicationGroupMessageId) { long receiverTimestamp = 1708100477067L; long expiration = 1000L; long timeToLive = 1000L; String destination = "destination-topic"; + ReplicationGroupMessageId useReplicationGroupId = + replicationGroupMessageId != null + ? replicationGroupMessageId + : DEFAULT_REPLICATION_GROUP_ID; return new BytesXMLMessage() { @Override @@ -358,7 +387,8 @@ public boolean getRedelivered() { @Override public ReplicationGroupMessageId getReplicationGroupMessageId() { - return null; + // this is always set by Solace + return useReplicationGroupId; } @Override diff --git a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/SolaceIOTest.java b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/SolaceIOTest.java index 8f02a850d01e..29fa5e13f48c 100644 --- a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/SolaceIOTest.java +++ b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/SolaceIOTest.java @@ -26,6 +26,7 @@ import com.solacesystems.jcsmp.Destination; import com.solacesystems.jcsmp.Queue; import com.solacesystems.jcsmp.Topic; +import com.solacesystems.jcsmp.impl.ReplicationGroupMessageIdImpl; import java.io.IOException; import java.nio.charset.StandardCharsets; import java.util.ArrayList; @@ -101,10 +102,10 @@ public void testReadMessages() { new MockSessionServiceFactory(mockClientService); // Expected data - List inputs = new ArrayList<>(); - inputs.add(SolaceDataUtils.getSolaceRecord("payload_test0", "450")); - inputs.add(SolaceDataUtils.getSolaceRecord("payload_test1", "451")); - inputs.add(SolaceDataUtils.getSolaceRecord("payload_test2", "452")); + List expected = new ArrayList<>(); + expected.add(SolaceDataUtils.getSolaceRecord("payload_test0", "450")); + expected.add(SolaceDataUtils.getSolaceRecord("payload_test1", "451")); + expected.add(SolaceDataUtils.getSolaceRecord("payload_test2", "452")); // Run the pipeline PCollection events = @@ -117,7 +118,7 @@ public void testReadMessages() { .withMaxNumConnections(2)); // Assert results - PAssert.that(events).containsInAnyOrder(inputs); + PAssert.that(events).containsInAnyOrder(expected); pipeline.run(); } @@ -140,9 +141,9 @@ public void testReadMessagesWithDeduplication() { new MockSessionServiceFactory(mockClientService); // Expected data - List inputs = new ArrayList<>(); - inputs.add(SolaceDataUtils.getSolaceRecord("payload_test0", "450")); - inputs.add(SolaceDataUtils.getSolaceRecord("payload_test1", "451")); + List expected = new ArrayList<>(); + expected.add(SolaceDataUtils.getSolaceRecord("payload_test0", "450")); + expected.add(SolaceDataUtils.getSolaceRecord("payload_test1", "451")); // Run the pipeline PCollection events = @@ -154,7 +155,7 @@ public void testReadMessagesWithDeduplication() { .withSessionServiceFactory(fakeSessionServiceFactory) .withMaxNumConnections(2)); // Assert results - PAssert.that(events).containsInAnyOrder(inputs); + PAssert.that(events).containsInAnyOrder(expected); pipeline.run(); } @@ -176,10 +177,10 @@ public void testReadMessagesWithoutDeduplication() { new MockSessionServiceFactory(mockClientService); // Expected data - List inputs = new ArrayList<>(); - inputs.add(SolaceDataUtils.getSolaceRecord("payload_test0", "450")); - inputs.add(SolaceDataUtils.getSolaceRecord("payload_test1", "451")); - inputs.add(SolaceDataUtils.getSolaceRecord("payload_test2", "451")); + List expected = new ArrayList<>(); + expected.add(SolaceDataUtils.getSolaceRecord("payload_test0", "450")); + expected.add(SolaceDataUtils.getSolaceRecord("payload_test1", "451")); + expected.add(SolaceDataUtils.getSolaceRecord("payload_test2", "451")); // Run the pipeline PCollection events = @@ -193,7 +194,51 @@ public void testReadMessagesWithoutDeduplication() { .withDeduplicateRecords(false)); // Assert results - PAssert.that(events).containsInAnyOrder(inputs); + PAssert.that(events).containsInAnyOrder(expected); + pipeline.run(); + } + + @Test + public void testReadMessagesWithDeduplicationOnReplicationGroupMessageId() { + // Broker that creates input data + MockSessionService mockClientService = + new MockSessionService( + index -> { + List messages = + ImmutableList.of( + SolaceDataUtils.getBytesXmlMessage( + "payload_test0", null, null, new ReplicationGroupMessageIdImpl(2L, 1L)), + SolaceDataUtils.getBytesXmlMessage( + "payload_test1", null, null, new ReplicationGroupMessageIdImpl(2L, 2L)), + SolaceDataUtils.getBytesXmlMessage( + "payload_test2", null, null, new ReplicationGroupMessageIdImpl(2L, 2L))); + return getOrNull(index, messages); + }, + 3); + + SessionServiceFactory fakeSessionServiceFactory = + new MockSessionServiceFactory(mockClientService); + + // Expected data + List expected = new ArrayList<>(); + expected.add( + SolaceDataUtils.getSolaceRecord( + "payload_test0", null, new ReplicationGroupMessageIdImpl(2L, 1L))); + expected.add( + SolaceDataUtils.getSolaceRecord( + "payload_test1", null, new ReplicationGroupMessageIdImpl(2L, 2L))); + + // Run the pipeline + PCollection events = + pipeline.apply( + "Read from Solace", + SolaceIO.read() + .from(Solace.Queue.fromName("queue")) + .withSempClientFactory(getMockSempClientFactory()) + .withSessionServiceFactory(fakeSessionServiceFactory) + .withMaxNumConnections(2)); + // Assert results + PAssert.that(events).containsInAnyOrder(expected); pipeline.run(); } @@ -215,10 +260,10 @@ public void testReadWithCoderAndParseFnAndTimestampFn() { new MockSessionServiceFactory(mockClientService); // Expected data - List inputs = new ArrayList<>(); - inputs.add(new SimpleRecord("payload_test0", "450")); - inputs.add(new SimpleRecord("payload_test1", "451")); - inputs.add(new SimpleRecord("payload_test2", "452")); + List expected = new ArrayList<>(); + expected.add(new SimpleRecord("payload_test0", "450")); + expected.add(new SimpleRecord("payload_test1", "451")); + expected.add(new SimpleRecord("payload_test2", "452")); // Run the pipeline PCollection events = @@ -237,7 +282,7 @@ public void testReadWithCoderAndParseFnAndTimestampFn() { .withMaxNumConnections(2)); // Assert results - PAssert.that(events).containsInAnyOrder(inputs); + PAssert.that(events).containsInAnyOrder(expected); pipeline.run(); } @@ -571,10 +616,10 @@ public void testTopicEncoding() { return dest instanceof Topic; })); - List inputs = ImmutableList.of(true, true, true); + List expected = ImmutableList.of(true, true, true); // Assert results - PAssert.that(destAreTopics).containsInAnyOrder(inputs); + PAssert.that(destAreTopics).containsInAnyOrder(expected); pipeline.run(); } } From 260f6451cec836e16605d9139743901832a4b745 Mon Sep 17 00:00:00 2001 From: bzablocki Date: Fri, 19 Apr 2024 16:48:42 +0200 Subject: [PATCH 06/41] respond to pr commments --- .../apache/beam/sdk/io/solace/SolaceIO.java | 22 ++++++------- .../solace/broker/SolaceMessageReceiver.java | 5 ++- .../beam/sdk/io/solace/data/Solace.java | 4 +-- .../sdk/io/solace/data/SolaceRecordCoder.java | 32 ++++++++++++------- .../io/solace/read/SolaceCheckpointMark.java | 11 ++++--- .../io/solace/read/UnboundedSolaceSource.java | 6 ++-- 6 files changed, 44 insertions(+), 36 deletions(-) diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/SolaceIO.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/SolaceIO.java index 43cd4738696c..e57a3b44950f 100644 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/SolaceIO.java +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/SolaceIO.java @@ -44,7 +44,6 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TypeDescriptor; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; -import org.checkerframework.checker.nullness.qual.NonNull; import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.Instant; import org.slf4j.Logger; @@ -128,7 +127,7 @@ public static Read read() { */ public static Read read( TypeDescriptor typeDescriptor, - SerializableFunction parseFn, + SerializableFunction<@Nullable BytesXMLMessage, @Nullable T> parseFn, SerializableFunction timestampFn) { checkState(typeDescriptor != null, "SolaceIO.Read: typeDescriptor must not be null"); checkState(parseFn != null, "SolaceIO.Read: parseFn must not be null"); @@ -231,7 +230,7 @@ public Read withSessionServiceFactory(SessionServiceFactory sessionServiceFac abstract boolean getDeduplicateRecords(); - abstract SerializableFunction getParseFn(); + abstract SerializableFunction<@Nullable BytesXMLMessage, @Nullable T> getParseFn(); abstract @Nullable SempClientFactory getSempClientFactory(); @@ -260,7 +259,8 @@ public abstract static class Builder { abstract Builder setDeduplicateRecords(boolean deduplicateRecords); - abstract Builder setParseFn(SerializableFunction parseFn); + abstract Builder setParseFn( + SerializableFunction<@Nullable BytesXMLMessage, @Nullable T> parseFn); abstract Builder setSempClientFactory(SempClientFactory brokerServiceFactory); @@ -271,10 +271,6 @@ public abstract static class Builder { abstract Read build(); } - private static T castIfNull(@Nullable T arg0, @NonNull T arg1) { - return arg0 != null ? arg0 : arg1; - } - @Override public PCollection expand(PBegin input) { checkState( @@ -284,11 +280,15 @@ public PCollection expand(PBegin input) { SempClientFactory sempClientFactory = checkNotNull(getSempClientFactory(), "SolaceIO: sempClientFactory is null."); String jobName = input.getPipeline().getOptions().getJobName(); - Queue queue = castIfNull(getQueue(), initializeQueueForTopic(jobName, sempClientFactory)); + Queue queueFromOptions = getQueue(); + Queue initializedQueue = + queueFromOptions != null + ? queueFromOptions + : initializeQueueForTopic(jobName, sempClientFactory); SessionServiceFactory sessionServiceFactory = checkNotNull(getSessionServiceFactory(), "SolaceIO: sessionServiceFactory is null."); - sessionServiceFactory.setQueue(queue); + sessionServiceFactory.setQueue(initializedQueue); registerDefaultCoder(input.getPipeline()); // Infer the actual coder @@ -297,7 +297,7 @@ public PCollection expand(PBegin input) { return input.apply( org.apache.beam.sdk.io.Read.from( new UnboundedSolaceSource<>( - queue, + initializedQueue, sempClientFactory, sessionServiceFactory, getMaxNumConnections(), diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SolaceMessageReceiver.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SolaceMessageReceiver.java index a993f6f26881..f8c4bb5ab503 100644 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SolaceMessageReceiver.java +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SolaceMessageReceiver.java @@ -21,14 +21,13 @@ import com.solacesystems.jcsmp.FlowReceiver; import com.solacesystems.jcsmp.JCSMPException; import java.io.IOException; -import java.io.Serializable; import org.apache.beam.sdk.io.solace.RetryCallableManager; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet; -public class SolaceMessageReceiver implements MessageReceiver, Serializable { +public class SolaceMessageReceiver implements MessageReceiver { public static final int DEFAULT_ADVANCE_TIMEOUT_IN_MILLIS = 100; - private final transient FlowReceiver flowReceiver; + private final FlowReceiver flowReceiver; private final RetryCallableManager retryCallableManager = RetryCallableManager.create(); public SolaceMessageReceiver(FlowReceiver flowReceiver) { diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/data/Solace.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/data/Solace.java index 4e8d7d1118ca..e75ba0c2b1fa 100644 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/data/Solace.java +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/data/Solace.java @@ -90,7 +90,7 @@ public enum DestinationType { @DefaultSchema(AutoValueSchema.class) public abstract static class Destination { @SchemaFieldNumber("0") - public abstract @Nullable String getName(); + public abstract String getName(); @SchemaFieldNumber("1") public abstract DestinationType getType(); @@ -101,7 +101,7 @@ public static Builder builder() { @AutoValue.Builder public abstract static class Builder { - public abstract Builder setName(@Nullable String name); + public abstract Builder setName(String name); public abstract Builder setType(DestinationType type); diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/data/SolaceRecordCoder.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/data/SolaceRecordCoder.java index 589fda6a3895..d79c20c48e89 100644 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/data/SolaceRecordCoder.java +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/data/SolaceRecordCoder.java @@ -30,6 +30,7 @@ import org.apache.beam.sdk.coders.VarIntCoder; import org.apache.beam.sdk.coders.VarLongCoder; import org.apache.beam.sdk.io.solace.data.Solace.Destination; +import org.apache.beam.sdk.io.solace.data.Solace.DestinationType; import org.apache.beam.sdk.io.solace.data.Solace.Record; import org.checkerframework.checker.nullness.qual.NonNull; @@ -73,18 +74,25 @@ public void encode(Record value, @NonNull OutputStream outStream) throws IOExcep @Override public Record decode(InputStream inStream) throws IOException { - return Record.builder() - .setMessageId(STRING_CODER.decode(inStream)) - .setReplicationGroupMessageId(STRING_CODER.decode(inStream)) - .setPayload(BYTE_CODER.decode(inStream)) - .setDestination( - Solace.Destination.builder() - .setName(STRING_CODER.decode(inStream)) - .setType( - Objects.equals(STRING_CODER.decode(inStream), "QUEUE") - ? Solace.DestinationType.QUEUE - : Solace.DestinationType.TOPIC) - .build()) + Record.Builder builder = + Record.builder() + .setMessageId(STRING_CODER.decode(inStream)) + .setReplicationGroupMessageId(STRING_CODER.decode(inStream)) + .setPayload(BYTE_CODER.decode(inStream)); + + String destinationName = STRING_CODER.decode(inStream); + String destinationType = STRING_CODER.decode(inStream); + if (destinationName != null) { + builder.setDestination( + Destination.builder() + .setName(destinationName) + .setType( + Objects.equals(destinationType, "QUEUE") + ? DestinationType.QUEUE + : DestinationType.TOPIC) + .build()); + } + return builder .setExpiration(LONG_CODER.decode(inStream)) .setPriority(INTEGER_CODER.decode(inStream)) .setRedelivered(BOOLEAN_CODER.decode(inStream)) diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/SolaceCheckpointMark.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/SolaceCheckpointMark.java index 8ac774a35f1e..0aad5164ee74 100644 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/SolaceCheckpointMark.java +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/SolaceCheckpointMark.java @@ -27,7 +27,6 @@ import org.apache.beam.sdk.coders.DefaultCoder; import org.apache.beam.sdk.extensions.avro.coders.AvroCoder; import org.apache.beam.sdk.io.UnboundedSource; -import org.checkerframework.checker.nullness.qual.NonNull; import org.checkerframework.checker.nullness.qual.Nullable; /** @@ -37,14 +36,13 @@ @Internal @DefaultCoder(AvroCoder.class) public class SolaceCheckpointMark implements UnboundedSource.CheckpointMark { - private transient @Nullable AtomicBoolean activeReader; + private transient AtomicBoolean activeReader; private ArrayDeque ackQueue; @SuppressWarnings("initialization") // Avro will set the fields by breaking abstraction - private SolaceCheckpointMark() {} // for Avro + private SolaceCheckpointMark() {} - public SolaceCheckpointMark( - @Nullable AtomicBoolean activeReader, @NonNull List ackQueue) { + public SolaceCheckpointMark(AtomicBoolean activeReader, List ackQueue) { this.activeReader = activeReader; this.ackQueue = new ArrayDeque<>(ackQueue); } @@ -65,6 +63,9 @@ public void finalizeCheckpoint() { @Override public boolean equals(@Nullable Object o) { + if (o == null) { + return false; + } if (this == o) { return true; } diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/UnboundedSolaceSource.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/UnboundedSolaceSource.java index 918b2fc5328b..370159994941 100644 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/UnboundedSolaceSource.java +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/UnboundedSolaceSource.java @@ -46,7 +46,7 @@ public class UnboundedSolaceSource extends UnboundedSource timestampFn; - private final SerializableFunction parseFn; + private final SerializableFunction<@Nullable BytesXMLMessage, @Nullable T> parseFn; public Queue getQueue() { return queue; @@ -64,7 +64,7 @@ public SerializableFunction getTimestampFn() { return timestampFn; } - public SerializableFunction getParseFn() { + public SerializableFunction<@Nullable BytesXMLMessage, @Nullable T> getParseFn() { return parseFn; } @@ -76,7 +76,7 @@ public UnboundedSolaceSource( boolean enableDeduplication, Coder coder, SerializableFunction timestampFn, - SerializableFunction parseFn) { + SerializableFunction<@Nullable BytesXMLMessage, @Nullable T> parseFn) { this.queue = queue; this.sempClientFactory = sempClientFactory; this.sessionServiceFactory = sessionServiceFactory; From 905786099a572f60c982ea8acd8e035bb2ed248e Mon Sep 17 00:00:00 2001 From: bzablocki Date: Fri, 19 Apr 2024 10:38:45 +0200 Subject: [PATCH 07/41] Documentation --- .../apache/beam/sdk/io/solace/SolaceIO.java | 207 +++++++++++++++++- .../BasicAuthJcsmpSessionServiceFactory.java | 2 +- .../broker/BasicAuthSempClientFactory.java | 2 +- 3 files changed, 205 insertions(+), 6 deletions(-) diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/SolaceIO.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/SolaceIO.java index e57a3b44950f..ffbc681efce7 100644 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/SolaceIO.java +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/SolaceIO.java @@ -52,13 +52,212 @@ /** * A {@link PTransform} to read and write from/to Solace. * - *

Authentication

+ *

Authentication

* - * TODO: Auth for the read connector + *

When reading from Solace, the user must use {@link + * Read#withSessionServiceFactory(SessionServiceFactory)} to create a JCSMP session and {@link + * Read#withSempClientFactory(SempClientFactory)} to authenticate to the SEMP API. * - *

Reading

+ *

See {@link Read#withSessionServiceFactory(SessionServiceFactory)} for session authentication. + * The connector provides implementation of the {@link SessionServiceFactory} using the Basic + * Authentication: {@link org.apache.beam.sdk.io.solace.broker.BasicAuthJcsmpSessionService}. * - * TODO + *

For the authentication to the SEMP API ({@link Read#withSempClientFactory(SempClientFactory)}) + * the connector provides {@link org.apache.beam.sdk.io.solace.broker.BasicAuthSempClientFactory} to + * authenticate using the Basic Authentication. + * + *

Reading

+ * + * To read from Solace, use the {@link SolaceIO#read()} or {@link SolaceIO#read(TypeDescriptor, + * SerializableFunction, SerializableFunction)}. + * + *

No-arg {@link SolaceIO#read()} top-level method

+ * + *

This method returns a PCollection of {@link Solace.Record} objects. It uses a default mapper + * ({@link SolaceRecordMapper#map(BytesXMLMessage)}) to map from the received {@link + * BytesXMLMessage} from Solace, to the {@link Solace.Record} objects. + * + *

By default, it also uses a {@link BytesXMLMessage#getSenderTimestamp()} for watermark + * estimation. This {@link SerializableFunction} can be overridden with {@link + * Read#withTimestampFn(SerializableFunction)} method. + * + *

When using this method, the Coders are inferred automatically. + * + *

Advanced {@link SolaceIO#read(TypeDescriptor, SerializableFunction, SerializableFunction)} + * top-level method

+ * + *

With this method, the user can: + * + *

    + *
  • specify custom output type of the PTransform (for example their own class consisting only + * of the relevant fields, optimized for their use-case), + *
  • create a custom mapping between {@link BytesXMLMessage} and their output type and + *
  • specify what field to use for watermark estimation from their mapped field (for example, in + * this method the user can use a field which is encoded in the payload as a timestamp, which + * cannot be done with the {@link SolaceIO#read()} method. + *
+ * + *

Reading from a queue ({@link Read#from(Solace.Queue)}} or a topic ({@link + * Read#from(Solace.Topic)})

+ * + *

Regardless of the top-level read method choice, the user can specify whether to read from a + * Queue - {@link Read#from(Solace.Queue)}, or a Topic {@link Read#from(Solace.Topic)}. + * + *

Note: when a user specifies to read from a Topic, the connector will create a matching Queue + * and a Subscription. The user must ensure that the SEMP API is reachable from the driver program + * and must provide credentials that have `write` permission to the SEMP Config API. The created Queue + * will be non-exclusive. The Queue will not be deleted when the pipeline is terminated. + * + *

Note: If the user specifies to read from a Queue, the driver program + * will execute a call to the SEMP API to check if the Queue is `exclusive` or `non-exclusive`. The + * user must ensure that the SEMP API is reachable from the driver program and provide credentials + * with `read` permission to the {@link Read#withSempClientFactory(SempClientFactory)}. + * + *

{@link Read#withSempClientFactory(SempClientFactory)}

+ * + *

The user can specify a factory class that creates a {@link + * org.apache.beam.sdk.io.solace.broker.SempClient}. + * + *

An existing implementation of the SempClientFactory includes {@link + * org.apache.beam.sdk.io.solace.broker.BasicAuthSempClientFactory} which implements the Basic + * Authentication to Solace. + * + *

To use it, specify the credentials with the builder methods. + * + *

The format of the host is `[Protocol://]Host[:Port]` + * + *

{@code
+ * .withSempClientFactory(
+ *         BasicAuthSempClientFactory.builder()
+ *                 .host("http://" + options.getSolaceHost() + ":8080")
+ *                 .username(options.getSolaceUsername())
+ *                 .password(options.getSolacePassword())
+ *                 .vpnName(options.getSolaceVpnName())
+ *                 .build())
+ * }
+ * + *

{@link Read#withSessionServiceFactory(SessionServiceFactory)}

+ * + *

The user can specify a factory class that creates a {@link SessionService}. + * + *

An existing implementation of the SempClientFactory includes {@link + * org.apache.beam.sdk.io.solace.broker.BasicAuthJcsmpSessionService} which implements the Basic + * Authentication to Solace. + * + *

To use it, specify the credentials with the builder methods. + * + *

The format of the host is `[Protocol://]Host[:Port]` + * + *

{@code
+ * BasicAuthJcsmpSessionServiceFactory.builder()
+ *         .host(options.getSolaceHost())
+ *         .username(options.getSolaceUsername())
+ *         .password(options.getSolacePassword())
+ *         .vpnName(options.getSolaceVpnName())
+ *         .build()));
+ * }
+ * + *

{@link Read#withMaxNumConnections(Integer)}

+ * + *

Optional. Sets the maximum number of connections to the broker. The actual number of sessions + * is determined by this and the number set by the runner. If not set, the number of sessions is + * determined by the runner. The number of connections created follows this logic: + * `numberOfConnections = min(maxNumConnections, desiredNumberOfSplits)`, where the + * `desiredNumberOfSplits` is set by the runner. + * + *

{@link Read#withDeduplicateRecords(boolean)}

+ * + *

Optional, default: true. Set to deduplicate messages based on the {@link + * BytesXMLMessage#getApplicationMessageId()} of the incoming {@link BytesXMLMessage}. If the field + * is null, then the {@link BytesXMLMessage#getReplicationGroupMessageId()} will be used, which is + * always set by Solace. + * + *

{@link Read#withTimestampFn(SerializableFunction)}

+ * + *

The timestamp function, used for estimating the watermark, mapping the record T to an {@link + * Instant} + * + *

Optional when using the no-arg {@link SolaceIO#read()} method. Defaults to {@link + * SolaceIO#SENDER_TIMESTAMP_FUNCTION}. When using the {@link SolaceIO#read(TypeDescriptor, + * SerializableFunction, SerializableFunction)} method, the function mapping from T to {@link + * Instant} has to be passed as an argument. + * + *

Usage example

+ * + *

The no-arg {@link SolaceIO#read()} method

+ * + *

The minimal example - reading from an existing Queue, using the no-arg {@link SolaceIO#read()} + * method, with all the default configuration options. + * + *

{@code
+ * PCollection events =
+ *   pipeline.apply(
+ *     SolaceIO.read()
+ *         .from(Queue.fromName(options.getSolaceReadQueue()))
+ *         .withSempClientFactory(
+ *                 BasicAuthSempClientFactory.builder()
+ *                         .host("http://" + options.getSolaceHost() + ":8080")
+ *                         .username(options.getSolaceUsername())
+ *                         .password(options.getSolacePassword())
+ *                         .vpnName(options.getSolaceVpnName())
+ *                         .build())
+ *         .withSessionServiceFactory(
+ *                 BasicAuthJcsmpSessionServiceFactory.builder()
+ *                         .host(options.getSolaceHost())
+ *                         .username(options.getSolaceUsername())
+ *                         .password(options.getSolacePassword())
+ *                         .vpnName(options.getSolaceVpnName())
+ *                         .build()));
+ * }
+ * + *

The advanced {@link SolaceIO#read(TypeDescriptor, SerializableFunction, + * SerializableFunction)} method

+ * + *

When using this method we can specify a custom output PCollection type and a custom timestamp + * function. + * + *

{@code
+ * @DefaultSchema(JavaBeanSchema.class)
+ * public static class SimpleRecord {
+ *    public String payload;
+ *    public String messageId;
+ *    public Instant timestamp;
+ *
+ *    public SimpleRecord() {}
+ *
+ *    public SimpleRecord(String payload, String messageId, Instant timestamp) {
+ *        this.payload = payload;
+ *        this.messageId = messageId;
+ *        this.timestamp = timestamp;
+ *    }
+ * }
+ *
+ * private static SimpleRecord toSimpleRecord(BytesXMLMessage record) {
+ *    if (record == null) {
+ *        return null;
+ *    }
+ *    return new SimpleRecord(
+ *            new String(record.getBytes(), StandardCharsets.UTF_8),
+ *            record.getApplicationMessageId(),
+ *            record.getSenderTimestamp() != null
+ *                    ? Instant.ofEpochMilli(record.getSenderTimestamp())
+ *                    : Instant.now());
+ * }
+ *
+ * PCollection events =
+ *  pipeline.apply(
+ *      SolaceIO.read(
+ *                      TypeDescriptor.of(SimpleRecord.class),
+ *                      record -> toSimpleRecord(record),
+ *                      record -> record.timestamp)
+ *              .from(Topic.fromName(options.getSolaceReadTopic()))
+ *              .withSempClientFactory(...)
+ *              .withSessionServiceFactory(...);
+ *
+ *
+ * }
*/ public class SolaceIO { diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/BasicAuthJcsmpSessionServiceFactory.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/BasicAuthJcsmpSessionServiceFactory.java index a1d994470eb6..e8e7484df312 100644 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/BasicAuthJcsmpSessionServiceFactory.java +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/BasicAuthJcsmpSessionServiceFactory.java @@ -38,7 +38,7 @@ public static Builder builder() { @AutoValue.Builder public abstract static class Builder { - /** Set Solace host, format: Protocol://Host[:Port]. */ + /** Set Solace host, format: [Protocol://]Host[:Port]. */ public abstract Builder host(String host); /** Set Solace username. */ diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/BasicAuthSempClientFactory.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/BasicAuthSempClientFactory.java index b2ae28c00314..b3a8db418a77 100644 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/BasicAuthSempClientFactory.java +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/BasicAuthSempClientFactory.java @@ -44,7 +44,7 @@ public static Builder builder() { @AutoValue.Builder public abstract static class Builder { - /** Set Solace host, format: Protocol://Host[:Port]. */ + /** Set Solace host, format: [Protocol://]Host[:Port]. */ public abstract Builder host(String host); /** Set Solace username. */ From a74b6a657164360dc81fe15d3bd69a859ce69630 Mon Sep 17 00:00:00 2001 From: bzablocki Date: Mon, 22 Apr 2024 17:26:24 +0200 Subject: [PATCH 08/41] Small refactor - move data classes out of the client --- .../io/solace/broker/BasicAuthSempClient.java | 130 +----------------- .../apache/beam/sdk/io/solace/data/Semp.java | 74 ++++++++++ 2 files changed, 75 insertions(+), 129 deletions(-) create mode 100644 sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/data/Semp.java diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/BasicAuthSempClient.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/BasicAuthSempClient.java index fa84389e8281..4a24d4f748b5 100644 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/BasicAuthSempClient.java +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/BasicAuthSempClient.java @@ -20,15 +20,12 @@ import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.DeserializationFeature; import com.fasterxml.jackson.databind.ObjectMapper; -import com.fasterxml.jackson.databind.annotation.JsonDeserialize; -import com.fasterxml.jackson.databind.annotation.JsonPOJOBuilder; -import com.fasterxml.jackson.databind.annotation.JsonSerialize; import com.google.api.client.http.HttpRequestFactory; -import com.google.auto.value.AutoValue; import com.solacesystems.jcsmp.JCSMPFactory; import java.io.IOException; import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.sdk.io.solace.SerializableSupplier; +import org.apache.beam.sdk.io.solace.data.Semp.Queue; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -94,129 +91,4 @@ private T mapJsonToClass(String content, Class mapSuccessToClass) throws JsonProcessingException { return objectMapper.readValue(content, mapSuccessToClass); } - - @AutoValue - @JsonSerialize(as = Queue.class) - @JsonDeserialize(builder = AutoValue_BasicAuthSempClient_Queue.Builder.class) - abstract static class Queue { - - public abstract QueueData data(); - - public static Builder builder() { - return new AutoValue_BasicAuthSempClient_Queue.Builder(); - } - - public abstract Builder toBuilder(); - - @AutoValue.Builder - @JsonPOJOBuilder(withPrefix = "set") - abstract static class Builder { - - public abstract Builder setData(QueueData queueData); - - public abstract Queue build(); - } - } - - @AutoValue - @JsonDeserialize(builder = AutoValue_BasicAuthSempClient_QueueData.Builder.class) - abstract static class QueueData { - public abstract String accessType(); - - public abstract long msgSpoolUsage(); - - public static Builder builder() { - return new AutoValue_BasicAuthSempClient_QueueData.Builder(); - } - - public abstract Builder toBuilder(); - - @AutoValue.Builder - @JsonPOJOBuilder(withPrefix = "set") - abstract static class Builder { - - public abstract Builder setAccessType(String accessType); - - public abstract Builder setMsgSpoolUsage(long msgSpoolUsage); - - public abstract QueueData build(); - } - } - - @AutoValue - @JsonSerialize(as = ErrorMessage.class) - @JsonDeserialize(builder = AutoValue_BasicAuthSempClient_ErrorMessage.Builder.class) - abstract static class ErrorMessage { - - public abstract ErrorMessageMeta meta(); - - public static Builder builder() { - return new AutoValue_BasicAuthSempClient_ErrorMessage.Builder(); - } - - public abstract Builder toBuilder(); - - @AutoValue.Builder - @JsonPOJOBuilder(withPrefix = "set") - abstract static class Builder { - - public abstract Builder setMeta(ErrorMessageMeta errorMessageMeta); - - public abstract ErrorMessage build(); - } - - @AutoValue - @JsonSerialize(as = ErrorMessageMeta.class) - @JsonDeserialize( - builder = AutoValue_BasicAuthSempClient_ErrorMessage_ErrorMessageMeta.Builder.class) - abstract static class ErrorMessageMeta { - public abstract ErrorMessageDetails error(); - - public static Builder builder() { - return new AutoValue_BasicAuthSempClient_ErrorMessage_ErrorMessageMeta.Builder(); - } - - public abstract Builder toBuilder(); - - @AutoValue.Builder - @JsonPOJOBuilder(withPrefix = "set") - abstract static class Builder { - - public abstract Builder setError(ErrorMessageDetails errorMessageDetails); - - public abstract ErrorMessageMeta build(); - } - } - - @AutoValue - @JsonSerialize(as = ErrorMessageDetails.class) - @JsonDeserialize( - builder = AutoValue_BasicAuthSempClient_ErrorMessage_ErrorMessageDetails.Builder.class) - abstract static class ErrorMessageDetails { - public abstract Integer code(); - - public abstract String description(); - - public abstract String status(); - - public static Builder builder() { - return new AutoValue_BasicAuthSempClient_ErrorMessage_ErrorMessageDetails.Builder(); - } - - public abstract Builder toBuilder(); - - @AutoValue.Builder - @JsonPOJOBuilder(withPrefix = "set") - abstract static class Builder { - - public abstract Builder setCode(Integer code); - - public abstract Builder setDescription(String description); - - public abstract Builder setStatus(String status); - - public abstract ErrorMessageDetails build(); - } - } - } } diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/data/Semp.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/data/Semp.java new file mode 100644 index 000000000000..f6f0fb51d22e --- /dev/null +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/data/Semp.java @@ -0,0 +1,74 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.solace.data; + +import com.fasterxml.jackson.databind.annotation.JsonDeserialize; +import com.fasterxml.jackson.databind.annotation.JsonPOJOBuilder; +import com.fasterxml.jackson.databind.annotation.JsonSerialize; +import com.google.auto.value.AutoValue; + +public class Semp { + + @AutoValue + @JsonSerialize(as = Queue.class) + @JsonDeserialize(builder = AutoValue_Semp_Queue.Builder.class) + public abstract static class Queue { + + public abstract QueueData data(); + + public static Builder builder() { + return new AutoValue_Semp_Queue.Builder(); + } + + public abstract Builder toBuilder(); + + @AutoValue.Builder + @JsonPOJOBuilder(withPrefix = "set") + abstract static class Builder { + + public abstract Builder setData(QueueData queueData); + + public abstract Queue build(); + } + } + + @AutoValue + @JsonDeserialize(builder = AutoValue_Semp_QueueData.Builder.class) + public abstract static class QueueData { + public abstract String accessType(); + + public abstract long msgSpoolUsage(); + + public static Builder builder() { + return new AutoValue_Semp_QueueData.Builder(); + } + + public abstract Builder toBuilder(); + + @AutoValue.Builder + @JsonPOJOBuilder(withPrefix = "set") + abstract static class Builder { + + public abstract Builder setAccessType(String accessType); + + public abstract Builder setMsgSpoolUsage(long msgSpoolUsage); + + public abstract QueueData build(); + } + } +} From 552cab468ce779ccc4a8274c6eae30c3de480ea7 Mon Sep 17 00:00:00 2001 From: bzablocki Date: Tue, 23 Apr 2024 16:46:14 +0200 Subject: [PATCH 09/41] refactor --- .../main/java/org/apache/beam/sdk/io/solace/data/Solace.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/data/Solace.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/data/Solace.java index e75ba0c2b1fa..6bd7c5d1f429 100644 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/data/Solace.java +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/data/Solace.java @@ -111,7 +111,7 @@ public abstract static class Builder { @AutoValue @DefaultSchema(AutoValueSchema.class) - public abstract static class Record implements Serializable { + public abstract static class Record { @SchemaFieldNumber("0") public abstract @Nullable String getMessageId(); From 5d847b37e9f26d9b22cf7701ce24d570013866dd Mon Sep 17 00:00:00 2001 From: bzablocki Date: Tue, 23 Apr 2024 16:56:34 +0200 Subject: [PATCH 10/41] Add github action for integration test of Solace --- .../beam_PreCommit_Java_Solace_IO_Direct.yml | 126 ++++++++++++++++++ 1 file changed, 126 insertions(+) create mode 100644 .github/workflows/beam_PreCommit_Java_Solace_IO_Direct.yml diff --git a/.github/workflows/beam_PreCommit_Java_Solace_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Solace_IO_Direct.yml new file mode 100644 index 000000000000..814fddf59208 --- /dev/null +++ b/.github/workflows/beam_PreCommit_Java_Solace_IO_Direct.yml @@ -0,0 +1,126 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +name: PreCommit Java Solace IO Direct + +on: + push: + tags: ['v*'] + branches: ['master', 'release-*'] + paths: + - "sdks/java/io/solace/**" + - ".github/workflows/beam_PreCommit_Java_Solace_IO_Direct.yml" + pull_request_target: + branches: ['master', 'release-*'] + paths: + - "sdks/java/io/solace/**" + - 'release/trigger_all_tests.json' + - '.github/trigger_files/beam_PreCommit_Java_Solace_IO_Direct.json' + issue_comment: + types: [created] + schedule: + - cron: '45 1/6 * * *' + workflow_dispatch: + +#Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + actions: write + pull-requests: write + checks: write + contents: read + deployments: read + id-token: none + issues: write + discussions: read + packages: read + pages: read + repository-projects: read + security-events: read + statuses: read + +# This allows a subsequently queued workflow run to interrupt previous runs +concurrency: + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + +jobs: + beam_PreCommit_Java_Solace_IO_Direct: + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + strategy: + matrix: + job_name: ["beam_PreCommit_Java_Solace_IO_Direct"] + job_phrase: ["Run Java_Solace_IO_Direct PreCommit"] + timeout-minutes: 60 + if: | + github.event_name == 'push' || + github.event_name == 'pull_request_target' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || + github.event_name == 'workflow_dispatch' || + github.event.comment.body == 'Run Java_Solace_IO_Direct PreCommit' + runs-on: [self-hosted, ubuntu-20.04, main] + steps: + - uses: actions/checkout@v4 + - name: Setup repository + uses: ./.github/actions/setup-action + with: + comment_phrase: ${{ matrix.job_phrase }} + github_token: ${{ secrets.GITHUB_TOKEN }} + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action + - name: run Solace IO build script + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:io:solace:build + arguments: | + -PdisableSpotlessCheck=true \ + -PdisableCheckStyle=true \ + - name: run Solace IO IT script + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:io:solace:integrationTest + arguments: | + -PdisableSpotlessCheck=true \ + -PdisableCheckStyle=true \ + - name: Archive JUnit Test Results + uses: actions/upload-artifact@v4 + if: ${{ !success() }} + with: + name: JUnit Test Results + path: "**/build/reports/tests/" + - name: Publish JUnit Test Results + uses: EnricoMi/publish-unit-test-result-action@v2 + if: always() + with: + commit: '${{ env.prsha || env.GITHUB_SHA }}' + comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} + files: '**/build/test-results/**/*.xml' + - name: Archive SpotBugs Results + uses: actions/upload-artifact@v4 + if: always() + with: + name: SpotBugs Results + path: '**/build/reports/spotbugs/*.html' + - name: Publish SpotBugs Results + uses: jwgmeligmeyling/spotbugs-github-action@v1.2 + if: always() + with: + name: Publish SpotBugs + path: '**/build/reports/spotbugs/*.html' \ No newline at end of file From 23292a1927d2fa25e0025b5da71b63e0cf7153d4 Mon Sep 17 00:00:00 2001 From: bzablocki Date: Wed, 24 Apr 2024 11:04:53 +0200 Subject: [PATCH 11/41] testing github workflow --- .github/workflows/beam_PreCommit_Java_Solace_IO_Direct.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/beam_PreCommit_Java_Solace_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Solace_IO_Direct.yml index 814fddf59208..d049f44cfb13 100644 --- a/.github/workflows/beam_PreCommit_Java_Solace_IO_Direct.yml +++ b/.github/workflows/beam_PreCommit_Java_Solace_IO_Direct.yml @@ -74,7 +74,7 @@ jobs: (github.event_name == 'schedule' && github.repository == 'apache/beam') || github.event_name == 'workflow_dispatch' || github.event.comment.body == 'Run Java_Solace_IO_Direct PreCommit' - runs-on: [self-hosted, ubuntu-20.04, main] + runs-on: ubuntu-20.04 steps: - uses: actions/checkout@v4 - name: Setup repository From a69e10a9111ed54d33a03e0658f9ba45541bd1a6 Mon Sep 17 00:00:00 2001 From: bzablocki Date: Wed, 24 Apr 2024 20:15:22 +0200 Subject: [PATCH 12/41] bump testcontainers to 1.19.7 - soalce semp was updated with an admin user access --- .../main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy index 091d238d1008..8a063ba06f04 100644 --- a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy +++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy @@ -639,7 +639,7 @@ class BeamModulePlugin implements Plugin { def spark2_version = "2.4.8" def spark3_version = "3.2.2" def spotbugs_version = "4.0.6" - def testcontainers_version = "1.18.3" + def testcontainers_version = "1.19.7" // [bomupgrader] determined by: org.apache.arrow:arrow-memory-core, consistent with: google_cloud_platform_libraries_bom def arrow_version = "15.0.1" def jmh_version = "1.34" From f4b0d6c54cb95c8aba1331141ecec8e95be49bc2 Mon Sep 17 00:00:00 2001 From: bzablocki Date: Mon, 29 Apr 2024 14:17:09 +0200 Subject: [PATCH 13/41] Use FlowHandle to acknowledge messages to make SolaceCheckpointMark's fields serializable. --- .../sdk/io/solace/broker/MessageReceiver.java | 2 ++ .../solace/broker/SolaceMessageReceiver.java | 9 +++++ .../beam/sdk/io/solace/data/Solace.java | 1 - .../io/solace/read/SolaceCheckpointMark.java | 35 +++++++++++++------ .../io/solace/read/UnboundedSolaceReader.java | 17 ++++----- .../sdk/io/solace/MockSessionService.java | 30 +++++++++++++--- .../beam/sdk/io/solace/SolaceDataUtils.java | 24 ++----------- .../beam/sdk/io/solace/SolaceIOTest.java | 16 +++++---- 8 files changed, 82 insertions(+), 52 deletions(-) diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/MessageReceiver.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/MessageReceiver.java index baf9e3beb038..4b63e42c34ea 100644 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/MessageReceiver.java +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/MessageReceiver.java @@ -25,6 +25,8 @@ public interface MessageReceiver { boolean isClosed(); + void ack(long ackId) throws IOException; + BytesXMLMessage receive() throws IOException; /** diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SolaceMessageReceiver.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SolaceMessageReceiver.java index f8c4bb5ab503..98b19b951799 100644 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SolaceMessageReceiver.java +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SolaceMessageReceiver.java @@ -20,6 +20,7 @@ import com.solacesystems.jcsmp.BytesXMLMessage; import com.solacesystems.jcsmp.FlowReceiver; import com.solacesystems.jcsmp.JCSMPException; +import com.solacesystems.jcsmp.impl.flow.FlowHandle; import java.io.IOException; import org.apache.beam.sdk.io.solace.RetryCallableManager; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet; @@ -57,4 +58,12 @@ public BytesXMLMessage receive() throws IOException { throw new IOException(e); } } + + @Override + public void ack(long ackId) throws IOException { + if (isClosed()) { + throw new IOException("SolaceIO: FlowReceiver is closed, can't acknowledge messages."); + } + ((FlowHandle) flowReceiver).sendSingleAck(ackId, true); + } } diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/data/Solace.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/data/Solace.java index 6bd7c5d1f429..0816f4507ac8 100644 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/data/Solace.java +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/data/Solace.java @@ -21,7 +21,6 @@ import com.solacesystems.jcsmp.BytesXMLMessage; import java.io.ByteArrayOutputStream; import java.io.IOException; -import java.io.Serializable; import org.apache.beam.sdk.schemas.AutoValueSchema; import org.apache.beam.sdk.schemas.annotations.DefaultSchema; import org.apache.beam.sdk.schemas.annotations.SchemaFieldNumber; diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/SolaceCheckpointMark.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/SolaceCheckpointMark.java index 0aad5164ee74..70612bfe3768 100644 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/SolaceCheckpointMark.java +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/SolaceCheckpointMark.java @@ -17,17 +17,21 @@ */ package org.apache.beam.sdk.io.solace.read; -import com.solacesystems.jcsmp.BytesXMLMessage; +import java.io.IOException; import java.util.ArrayDeque; import java.util.ArrayList; import java.util.List; import java.util.Objects; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicReference; import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.sdk.coders.DefaultCoder; import org.apache.beam.sdk.extensions.avro.coders.AvroCoder; import org.apache.beam.sdk.io.UnboundedSource; +import org.apache.beam.sdk.io.solace.broker.MessageReceiver; import org.checkerframework.checker.nullness.qual.Nullable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Checkpoint for an unbounded Solace source. Consists of the Solace messages waiting to be @@ -36,27 +40,38 @@ @Internal @DefaultCoder(AvroCoder.class) public class SolaceCheckpointMark implements UnboundedSource.CheckpointMark { + private static final Logger LOG = LoggerFactory.getLogger(SolaceCheckpointMark.class); private transient AtomicBoolean activeReader; - private ArrayDeque ackQueue; + private ArrayDeque ackQueue; + private AtomicReference sessionServiceRef; @SuppressWarnings("initialization") // Avro will set the fields by breaking abstraction private SolaceCheckpointMark() {} - public SolaceCheckpointMark(AtomicBoolean activeReader, List ackQueue) { + public SolaceCheckpointMark( + AtomicBoolean activeReader, + List ackQueue, + AtomicReference messageReceiverRef) { this.activeReader = activeReader; this.ackQueue = new ArrayDeque<>(ackQueue); + this.sessionServiceRef = messageReceiverRef; } @Override - public void finalizeCheckpoint() { + public void finalizeCheckpoint() throws IOException { if (activeReader == null || !activeReader.get() || ackQueue == null) { return; } - + MessageReceiver receiver = sessionServiceRef.get(); + if (receiver == null) { + LOG.warn( + "SolaceIO: MessageReceiver is null, can't ack messages. They will" + " be redelivered."); + return; + } while (ackQueue.size() > 0) { - BytesXMLMessage msg = ackQueue.poll(); - if (msg != null) { - msg.ackMessage(); + Long ackId = ackQueue.poll(); + if (ackId != null) { + receiver.ack(ackId); } } } @@ -75,8 +90,8 @@ public boolean equals(@Nullable Object o) { SolaceCheckpointMark that = (SolaceCheckpointMark) o; // Needed to convert to ArrayList because ArrayDeque.equals checks only for reference, not // content. - ArrayList ackList = new ArrayList<>(ackQueue); - ArrayList thatAckList = new ArrayList<>(that.ackQueue); + ArrayList ackList = new ArrayList<>(ackQueue); + ArrayList thatAckList = new ArrayList<>(that.ackQueue); return Objects.equals(activeReader, that.activeReader) && Objects.equals(ackList, thatAckList); } diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/UnboundedSolaceReader.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/UnboundedSolaceReader.java index 0155345a2323..640dfe063d02 100644 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/UnboundedSolaceReader.java +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/UnboundedSolaceReader.java @@ -27,6 +27,7 @@ import java.util.List; import java.util.NoSuchElementException; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicReference; import org.apache.beam.sdk.io.UnboundedSource; import org.apache.beam.sdk.io.UnboundedSource.UnboundedReader; import org.apache.beam.sdk.io.solace.broker.MessageReceiver; @@ -70,7 +71,7 @@ public UnboundedSolaceReader(UnboundedSolaceSource currentSource) { @Override public boolean start() { populateSession(); - populateMessageConsumer(); + populateMessageReceiver(); return advance(); } @@ -83,14 +84,13 @@ public void populateSession() { } } - private void populateMessageConsumer() { + private void populateMessageReceiver() { if (messageReceiver == null) { messageReceiver = checkNotNull(sessionService).createReceiver(); messageReceiver.start(); } - MessageReceiver receiver = checkNotNull(messageReceiver); - if (receiver.isClosed()) { - receiver.start(); + if (messageReceiver != null && messageReceiver.isClosed()) { + checkNotNull(messageReceiver).start(); } } @@ -131,14 +131,15 @@ public Instant getWatermark() { @Override public UnboundedSource.CheckpointMark getCheckpointMark() { - List ackQueue = new ArrayList<>(); + List ackQueue = new ArrayList<>(); while (!elementsToCheckpoint.isEmpty()) { BytesXMLMessage msg = elementsToCheckpoint.poll(); if (msg != null) { - ackQueue.add(msg); + ackQueue.add(msg.getMessageIdLong()); } } - return new SolaceCheckpointMark(active, ackQueue); + return new SolaceCheckpointMark( + active, ackQueue, new AtomicReference<>(checkNotNull(messageReceiver))); } @Override diff --git a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/MockSessionService.java b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/MockSessionService.java index 7b14da138c64..54cc20d42096 100644 --- a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/MockSessionService.java +++ b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/MockSessionService.java @@ -28,6 +28,7 @@ public class MockSessionService implements SessionService { private final SerializableFunction getRecordFn; + private final AtomicInteger ackCounter; private MessageReceiver messageReceiver = null; private final int minMessagesReceived; @@ -35,6 +36,16 @@ public MockSessionService( SerializableFunction getRecordFn, int minMessagesReceived) { this.getRecordFn = getRecordFn; this.minMessagesReceived = minMessagesReceived; + this.ackCounter = new AtomicInteger(); + } + + public MockSessionService( + SerializableFunction getRecordFn, + AtomicInteger ackCounter, + int minMessagesReceived) { + this.getRecordFn = getRecordFn; + this.minMessagesReceived = minMessagesReceived; + this.ackCounter = ackCounter; } @Override @@ -48,7 +59,7 @@ public boolean isClosed() { @Override public MessageReceiver createReceiver() { if (messageReceiver == null) { - messageReceiver = new MockReceiver(getRecordFn, minMessagesReceived); + messageReceiver = new MockReceiver(getRecordFn, ackCounter, minMessagesReceived); } return messageReceiver; } @@ -57,13 +68,17 @@ public MessageReceiver createReceiver() { public void connect() {} public static class MockReceiver implements MessageReceiver, Serializable { - private final AtomicInteger counter = new AtomicInteger(); + private final AtomicInteger receiveCounter = new AtomicInteger(); + private final AtomicInteger ackCounter; private final SerializableFunction getRecordFn; private final int minMessagesReceived; public MockReceiver( - SerializableFunction getRecordFn, int minMessagesReceived) { + SerializableFunction getRecordFn, + AtomicInteger ackCounter, + int minMessagesReceived) { this.getRecordFn = getRecordFn; + this.ackCounter = ackCounter; this.minMessagesReceived = minMessagesReceived; } @@ -77,12 +92,17 @@ public boolean isClosed() { @Override public BytesXMLMessage receive() throws IOException { - return getRecordFn.apply(counter.getAndIncrement()); + return getRecordFn.apply(receiveCounter.getAndIncrement()); + } + + @Override + public void ack(long ackId) throws IOException { + ackCounter.getAndIncrement(); } @Override public boolean isEOF() { - return counter.get() >= minMessagesReceived; + return receiveCounter.get() >= minMessagesReceived; } } } diff --git a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/SolaceDataUtils.java b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/SolaceDataUtils.java index ab7a64e0f702..e727c3c804a5 100644 --- a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/SolaceDataUtils.java +++ b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/SolaceDataUtils.java @@ -37,7 +37,6 @@ import org.apache.beam.sdk.io.solace.data.Solace; import org.apache.beam.sdk.schemas.JavaBeanSchema; import org.apache.beam.sdk.schemas.annotations.DefaultSchema; -import org.apache.beam.sdk.transforms.SerializableFunction; import org.checkerframework.checker.nullness.qual.Nullable; public class SolaceDataUtils { @@ -118,25 +117,12 @@ public static Solace.Record getSolaceRecord( .build(); } - public static BytesXMLMessage getBytesXmlMessage(String payload) { - return getBytesXmlMessage(payload, "messageId", null); - } - public static BytesXMLMessage getBytesXmlMessage(String payload, String messageId) { - return getBytesXmlMessage(payload, messageId, null, null); + return getBytesXmlMessage(payload, messageId, null); } public static BytesXMLMessage getBytesXmlMessage( - String payload, String messageId, SerializableFunction ackMessageFn) { - - return getBytesXmlMessage(payload, messageId, ackMessageFn, null); - } - - public static BytesXMLMessage getBytesXmlMessage( - String payload, - String messageId, - SerializableFunction ackMessageFn, - ReplicationGroupMessageId replicationGroupMessageId) { + String payload, String messageId, ReplicationGroupMessageId replicationGroupMessageId) { long receiverTimestamp = 1708100477067L; long expiration = 1000L; long timeToLive = 1000L; @@ -184,11 +170,7 @@ public void writeBytes(byte[] arg0, int arg1, int arg2) { } @Override - public void ackMessage() { - if (ackMessageFn != null) { - ackMessageFn.apply(0); - } - } + public void ackMessage() {} @Override public void clearAttachment() { diff --git a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/SolaceIOTest.java b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/SolaceIOTest.java index 29fa5e13f48c..ffe8fcdd0ed2 100644 --- a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/SolaceIOTest.java +++ b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/SolaceIOTest.java @@ -32,6 +32,7 @@ import java.util.ArrayList; import java.util.List; import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.io.UnboundedSource.CheckpointMark; import org.apache.beam.sdk.io.UnboundedSource.UnboundedReader; @@ -207,11 +208,11 @@ public void testReadMessagesWithDeduplicationOnReplicationGroupMessageId() { List messages = ImmutableList.of( SolaceDataUtils.getBytesXmlMessage( - "payload_test0", null, null, new ReplicationGroupMessageIdImpl(2L, 1L)), + "payload_test0", null, new ReplicationGroupMessageIdImpl(2L, 1L)), SolaceDataUtils.getBytesXmlMessage( - "payload_test1", null, null, new ReplicationGroupMessageIdImpl(2L, 2L)), + "payload_test1", null, new ReplicationGroupMessageIdImpl(2L, 2L)), SolaceDataUtils.getBytesXmlMessage( - "payload_test2", null, null, new ReplicationGroupMessageIdImpl(2L, 2L))); + "payload_test2", null, new ReplicationGroupMessageIdImpl(2L, 2L))); return getOrNull(index, messages); }, 3); @@ -375,13 +376,12 @@ public void testCheckpointMark() throws Exception { index -> { List messages = new ArrayList<>(); for (int i = 0; i < 10; i++) { - messages.add( - SolaceDataUtils.getBytesXmlMessage( - "payload_test" + i, "45" + i, (num) -> countAckMessages.incrementAndGet())); + messages.add(SolaceDataUtils.getBytesXmlMessage("payload_test" + i, "45" + i)); } countConsumedMessages.incrementAndGet(); return getOrNull(index, messages); }, + countAckMessages, 10); SessionServiceFactory fakeSessionServiceFactory = @@ -436,6 +436,7 @@ public void testCheckpointMarkAndFinalizeSeparately() throws Exception { countConsumedMessages.incrementAndGet(); return getOrNull(index, messages); }, + countAckMessages, 10); SessionServiceFactory fakeSessionServiceFactory = new MockSessionServiceFactory(mockClientService); @@ -548,7 +549,8 @@ public void testCheckpointMarkSafety() throws Exception { @Test public void testCheckpointMarkDefaultCoder() throws Exception { - SolaceCheckpointMark checkpointMark = new SolaceCheckpointMark(null, new ArrayList<>()); + SolaceCheckpointMark checkpointMark = + new SolaceCheckpointMark(null, new ArrayList<>(), new AtomicReference<>()); Coder coder = new UnboundedSolaceSource<>(null, null, null, 0, false, null, null, null) .getCheckpointMarkCoder(); From a1ca8b3d7c9b80c741dc71184c28c6a63c221369 Mon Sep 17 00:00:00 2001 From: bzablocki Date: Wed, 8 May 2024 17:27:10 +0200 Subject: [PATCH 14/41] Handle StaleSessionException error --- .../io/solace/broker/SolaceMessageReceiver.java | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SolaceMessageReceiver.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SolaceMessageReceiver.java index 98b19b951799..181b6c1a890c 100644 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SolaceMessageReceiver.java +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SolaceMessageReceiver.java @@ -20,10 +20,12 @@ import com.solacesystems.jcsmp.BytesXMLMessage; import com.solacesystems.jcsmp.FlowReceiver; import com.solacesystems.jcsmp.JCSMPException; +import com.solacesystems.jcsmp.StaleSessionException; import com.solacesystems.jcsmp.impl.flow.FlowHandle; import java.io.IOException; import org.apache.beam.sdk.io.solace.RetryCallableManager; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet; +import org.checkerframework.checker.nullness.qual.Nullable; public class SolaceMessageReceiver implements MessageReceiver { @@ -37,6 +39,10 @@ public SolaceMessageReceiver(FlowReceiver flowReceiver) { @Override public void start() { + startFlowReceiver(); + } + + private void startFlowReceiver() { retryCallableManager.retryCallable( () -> { flowReceiver.start(); @@ -52,8 +58,19 @@ public boolean isClosed() { @Override public BytesXMLMessage receive() throws IOException { + return receive(0, null); + } + + private BytesXMLMessage receive(int count, @Nullable Exception lastException) throws IOException { + if (count >= 2) { + throw new IOException( + "SolaceIO: tried to pull messages " + (count + 1) + " times, aborting.", lastException); + } try { return flowReceiver.receive(DEFAULT_ADVANCE_TIMEOUT_IN_MILLIS); + } catch (StaleSessionException e) { + startFlowReceiver(); + return receive(count + 1, e); } catch (JCSMPException e) { throw new IOException(e); } From 8007a33e998f94a0c4a567f111aad82a38732dba Mon Sep 17 00:00:00 2001 From: bzablocki Date: Wed, 8 May 2024 17:42:53 +0200 Subject: [PATCH 15/41] Add @Internal annotation to mark the SolaceIO API beta and subject to change. --- .../main/java/org/apache/beam/sdk/io/solace/SolaceIO.java | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/SolaceIO.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/SolaceIO.java index ffbc681efce7..7dfffd55413b 100644 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/SolaceIO.java +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/SolaceIO.java @@ -28,6 +28,7 @@ import com.solacesystems.jcsmp.Topic; import java.io.IOException; import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.sdk.coders.CannotProvideCoderException; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.io.solace.broker.SempClientFactory; @@ -50,7 +51,9 @@ import org.slf4j.LoggerFactory; /** - * A {@link PTransform} to read and write from/to Solace. + * A {@link PTransform} to read and write from/to Solace event broker. + * + *

Note: this API is beta and subject to change. * *

Authentication

* @@ -259,6 +262,7 @@ * * }
*/ +@Internal public class SolaceIO { public static final SerializableFunction SENDER_TIMESTAMP_FUNCTION = From cff31053e760284d5fa9f9453097183075d61b1f Mon Sep 17 00:00:00 2001 From: bzablocki Date: Mon, 13 May 2024 13:45:51 +0200 Subject: [PATCH 16/41] Improve documentation --- .../apache/beam/sdk/io/solace/SolaceIO.java | 126 +++++++----------- 1 file changed, 51 insertions(+), 75 deletions(-) diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/SolaceIO.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/SolaceIO.java index 7dfffd55413b..955709c62a4d 100644 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/SolaceIO.java +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/SolaceIO.java @@ -118,75 +118,6 @@ * user must ensure that the SEMP API is reachable from the driver program and provide credentials * with `read` permission to the {@link Read#withSempClientFactory(SempClientFactory)}. * - *

{@link Read#withSempClientFactory(SempClientFactory)}

- * - *

The user can specify a factory class that creates a {@link - * org.apache.beam.sdk.io.solace.broker.SempClient}. - * - *

An existing implementation of the SempClientFactory includes {@link - * org.apache.beam.sdk.io.solace.broker.BasicAuthSempClientFactory} which implements the Basic - * Authentication to Solace. - * - *

To use it, specify the credentials with the builder methods. - * - *

The format of the host is `[Protocol://]Host[:Port]` - * - *

{@code
- * .withSempClientFactory(
- *         BasicAuthSempClientFactory.builder()
- *                 .host("http://" + options.getSolaceHost() + ":8080")
- *                 .username(options.getSolaceUsername())
- *                 .password(options.getSolacePassword())
- *                 .vpnName(options.getSolaceVpnName())
- *                 .build())
- * }
- * - *

{@link Read#withSessionServiceFactory(SessionServiceFactory)}

- * - *

The user can specify a factory class that creates a {@link SessionService}. - * - *

An existing implementation of the SempClientFactory includes {@link - * org.apache.beam.sdk.io.solace.broker.BasicAuthJcsmpSessionService} which implements the Basic - * Authentication to Solace. - * - *

To use it, specify the credentials with the builder methods. - * - *

The format of the host is `[Protocol://]Host[:Port]` - * - *

{@code
- * BasicAuthJcsmpSessionServiceFactory.builder()
- *         .host(options.getSolaceHost())
- *         .username(options.getSolaceUsername())
- *         .password(options.getSolacePassword())
- *         .vpnName(options.getSolaceVpnName())
- *         .build()));
- * }
- * - *

{@link Read#withMaxNumConnections(Integer)}

- * - *

Optional. Sets the maximum number of connections to the broker. The actual number of sessions - * is determined by this and the number set by the runner. If not set, the number of sessions is - * determined by the runner. The number of connections created follows this logic: - * `numberOfConnections = min(maxNumConnections, desiredNumberOfSplits)`, where the - * `desiredNumberOfSplits` is set by the runner. - * - *

{@link Read#withDeduplicateRecords(boolean)}

- * - *

Optional, default: true. Set to deduplicate messages based on the {@link - * BytesXMLMessage#getApplicationMessageId()} of the incoming {@link BytesXMLMessage}. If the field - * is null, then the {@link BytesXMLMessage#getReplicationGroupMessageId()} will be used, which is - * always set by Solace. - * - *

{@link Read#withTimestampFn(SerializableFunction)}

- * - *

The timestamp function, used for estimating the watermark, mapping the record T to an {@link - * Instant} - * - *

Optional when using the no-arg {@link SolaceIO#read()} method. Defaults to {@link - * SolaceIO#SENDER_TIMESTAMP_FUNCTION}. When using the {@link SolaceIO#read(TypeDescriptor, - * SerializableFunction, SerializableFunction)} method, the function mapping from T to {@link - * Instant} has to be passed as an argument. - * *

Usage example

* *

The no-arg {@link SolaceIO#read()} method

@@ -357,8 +288,13 @@ public Read from(Solace.Topic topic) { } /** - * Set the timestamp function. This serializable has to output an {@link Instant}. This will be - * used to calculate watermark and define record's timestamp. + * The timestamp function, used for estimating the watermark, mapping the record T to an {@link + * Instant} + * + *

Optional when using the no-arg {@link SolaceIO#read()} method. Defaults to {@link + * SolaceIO#SENDER_TIMESTAMP_FUNCTION}. When using the {@link SolaceIO#read(TypeDescriptor, + * SerializableFunction, SerializableFunction)} method, the function mapping from T to {@link + * Instant} has to be passed as an argument. */ public Read withTimestampFn(SerializableFunction timestampFn) { checkState( @@ -369,16 +305,21 @@ public Read withTimestampFn(SerializableFunction timestampFn) { } /** - * Maximum number of read connections created to Solace cluster. This is optional, leave out to - * let the Runner decide. + * Optional. Sets the maximum number of connections to the broker. The actual number of sessions + * is determined by this and the number set by the runner. If not set, the number of sessions is + * determined by the runner. The number of connections created follows this logic: + * `numberOfConnections = min(maxNumConnections, desiredNumberOfSplits)`, where the + * `desiredNumberOfSplits` is set by the runner. */ public Read withMaxNumConnections(Integer maxNumConnections) { return toBuilder().setMaxNumConnections(maxNumConnections).build(); } /** - * Set if the read records should be deduplicated. True by default. It will use the - * `applicationMessageId` attribute to identify duplicates. + * Optional, default: true. Set to deduplicate messages based on the {@link + * BytesXMLMessage#getApplicationMessageId()} of the incoming {@link BytesXMLMessage}. If the + * field is null, then the {@link BytesXMLMessage#getReplicationGroupMessageId()} will be used, + * which is always set by Solace. */ public Read withDeduplicateRecords(boolean deduplicateRecords) { return toBuilder().setDeduplicateRecords(deduplicateRecords).build(); @@ -397,6 +338,24 @@ public Read withDeduplicateRecords(boolean deduplicateRecords) { *

  • query for metadata such as access-type of a Queue, *
  • requesting creation of new Queues. * + * + *

    An existing implementation of the SempClientFactory includes {@link + * org.apache.beam.sdk.io.solace.broker.BasicAuthSempClientFactory} which implements the Basic + * Authentication to Solace. + * + *

    To use it, specify the credentials with the builder methods. + * + *

    The format of the host is `[Protocol://]Host[:Port]` + * + *

    {@code
    +     * .withSempClientFactory(
    +     *         BasicAuthSempClientFactory.builder()
    +     *                 .host("http://" + options.getSolaceHost() + ":8080")
    +     *                 .username(options.getSolaceUsername())
    +     *                 .password(options.getSolacePassword())
    +     *                 .vpnName(options.getSolaceVpnName())
    +     *                 .build())
    +     * }
    */ public Read withSempClientFactory(SempClientFactory sempClientFactory) { checkState(sempClientFactory != null, "SolaceIO.Read: sempClientFactory must not be null."); @@ -416,6 +375,23 @@ public Read withSempClientFactory(SempClientFactory sempClientFactory) { *
  • close the connection, *
  • create a {@link org.apache.beam.sdk.io.solace.broker.MessageReceiver}. * + * + *

    An existing implementation of the SempClientFactory includes {@link + * org.apache.beam.sdk.io.solace.broker.BasicAuthJcsmpSessionService} which implements the Basic + * Authentication to Solace. * + * + *

    To use it, specify the credentials with the builder methods. * + * + *

    The format of the host is `[Protocol://]Host[:Port]` * + * + *

    {@code
    +     * BasicAuthJcsmpSessionServiceFactory.builder()
    +     *         .host(options.getSolaceHost())
    +     *         .username(options.getSolaceUsername())
    +     *         .password(options.getSolacePassword())
    +     *         .vpnName(options.getSolaceVpnName())
    +     *         .build()));
    +     * }
    */ public Read withSessionServiceFactory(SessionServiceFactory sessionServiceFactory) { checkState( From f3eaabe8da9540dcffc683864b2be539c01da25a Mon Sep 17 00:00:00 2001 From: bzablocki Date: Wed, 29 May 2024 14:15:58 +0200 Subject: [PATCH 17/41] Back to ack based on bytesxmlmessages. Deduplicate default to false. --- .../apache/beam/sdk/io/solace/SolaceIO.java | 4 +-- .../beam/sdk/io/solace/data/Solace.java | 4 +-- .../sdk/io/solace/data/SolaceRecordCoder.java | 5 ++- .../io/solace/read/SolaceCheckpointMark.java | 35 ++++++------------- .../io/solace/read/UnboundedSolaceReader.java | 17 +++++---- .../io/solace/read/WatermarkParameters.java | 8 ++--- .../sdk/io/solace/MockSessionService.java | 20 +++++------ .../beam/sdk/io/solace/SolaceDataUtils.java | 19 ++++++++-- .../beam/sdk/io/solace/SolaceIOTest.java | 30 ++++++++-------- 9 files changed, 68 insertions(+), 74 deletions(-) diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/SolaceIO.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/SolaceIO.java index 955709c62a4d..e13b85e423e2 100644 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/SolaceIO.java +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/SolaceIO.java @@ -205,7 +205,7 @@ public class SolaceIO { return Instant.now(); } }; - private static final boolean DEFAULT_DEDUPLICATE_RECORDS = true; + private static final boolean DEFAULT_DEDUPLICATE_RECORDS = false; /** Get a {@link Topic} object from the topic name. */ static Topic topicFromName(String topicName) { @@ -316,7 +316,7 @@ public Read withMaxNumConnections(Integer maxNumConnections) { } /** - * Optional, default: true. Set to deduplicate messages based on the {@link + * Optional, default: false. Set to deduplicate messages based on the {@link * BytesXMLMessage#getApplicationMessageId()} of the incoming {@link BytesXMLMessage}. If the * field is null, then the {@link BytesXMLMessage#getReplicationGroupMessageId()} will be used, * which is always set by Solace. diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/data/Solace.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/data/Solace.java index 0816f4507ac8..72571f311b29 100644 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/data/Solace.java +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/data/Solace.java @@ -31,8 +31,8 @@ /** * A record to be written to a Solace topic. * - *

    You need to transform to {@link Record} to be able to write to Solace. For that, you can use - * the {@link Record.Builder} provided with this class. + *

    You need to transform to {@link Solace.Record} to be able to write to Solace. For that, you + * can use the {@link Solace.Record.Builder} provided with this class. * *

    For instance, to create a record, use the following code: * diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/data/SolaceRecordCoder.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/data/SolaceRecordCoder.java index d79c20c48e89..fcad3b77a0bc 100644 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/data/SolaceRecordCoder.java +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/data/SolaceRecordCoder.java @@ -30,7 +30,6 @@ import org.apache.beam.sdk.coders.VarIntCoder; import org.apache.beam.sdk.coders.VarLongCoder; import org.apache.beam.sdk.io.solace.data.Solace.Destination; -import org.apache.beam.sdk.io.solace.data.Solace.DestinationType; import org.apache.beam.sdk.io.solace.data.Solace.Record; import org.checkerframework.checker.nullness.qual.NonNull; @@ -88,8 +87,8 @@ public Record decode(InputStream inStream) throws IOException { .setName(destinationName) .setType( Objects.equals(destinationType, "QUEUE") - ? DestinationType.QUEUE - : DestinationType.TOPIC) + ? Solace.DestinationType.QUEUE + : Solace.DestinationType.TOPIC) .build()); } return builder diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/SolaceCheckpointMark.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/SolaceCheckpointMark.java index 70612bfe3768..07b6666389f1 100644 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/SolaceCheckpointMark.java +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/SolaceCheckpointMark.java @@ -17,21 +17,17 @@ */ package org.apache.beam.sdk.io.solace.read; -import java.io.IOException; +import com.solacesystems.jcsmp.BytesXMLMessage; import java.util.ArrayDeque; import java.util.ArrayList; import java.util.List; import java.util.Objects; import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicReference; import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.sdk.coders.DefaultCoder; import org.apache.beam.sdk.extensions.avro.coders.AvroCoder; import org.apache.beam.sdk.io.UnboundedSource; -import org.apache.beam.sdk.io.solace.broker.MessageReceiver; import org.checkerframework.checker.nullness.qual.Nullable; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** * Checkpoint for an unbounded Solace source. Consists of the Solace messages waiting to be @@ -40,38 +36,27 @@ @Internal @DefaultCoder(AvroCoder.class) public class SolaceCheckpointMark implements UnboundedSource.CheckpointMark { - private static final Logger LOG = LoggerFactory.getLogger(SolaceCheckpointMark.class); private transient AtomicBoolean activeReader; - private ArrayDeque ackQueue; - private AtomicReference sessionServiceRef; + private transient ArrayDeque ackQueue; @SuppressWarnings("initialization") // Avro will set the fields by breaking abstraction private SolaceCheckpointMark() {} - public SolaceCheckpointMark( - AtomicBoolean activeReader, - List ackQueue, - AtomicReference messageReceiverRef) { + public SolaceCheckpointMark(AtomicBoolean activeReader, List ackQueue) { this.activeReader = activeReader; this.ackQueue = new ArrayDeque<>(ackQueue); - this.sessionServiceRef = messageReceiverRef; } @Override - public void finalizeCheckpoint() throws IOException { + public void finalizeCheckpoint() { if (activeReader == null || !activeReader.get() || ackQueue == null) { return; } - MessageReceiver receiver = sessionServiceRef.get(); - if (receiver == null) { - LOG.warn( - "SolaceIO: MessageReceiver is null, can't ack messages. They will" + " be redelivered."); - return; - } + while (ackQueue.size() > 0) { - Long ackId = ackQueue.poll(); - if (ackId != null) { - receiver.ack(ackId); + BytesXMLMessage msg = ackQueue.poll(); + if (msg != null) { + msg.ackMessage(); } } } @@ -90,8 +75,8 @@ public boolean equals(@Nullable Object o) { SolaceCheckpointMark that = (SolaceCheckpointMark) o; // Needed to convert to ArrayList because ArrayDeque.equals checks only for reference, not // content. - ArrayList ackList = new ArrayList<>(ackQueue); - ArrayList thatAckList = new ArrayList<>(that.ackQueue); + ArrayList ackList = new ArrayList<>(ackQueue); + ArrayList thatAckList = new ArrayList<>(that.ackQueue); return Objects.equals(activeReader, that.activeReader) && Objects.equals(ackList, thatAckList); } diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/UnboundedSolaceReader.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/UnboundedSolaceReader.java index 640dfe063d02..0155345a2323 100644 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/UnboundedSolaceReader.java +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/UnboundedSolaceReader.java @@ -27,7 +27,6 @@ import java.util.List; import java.util.NoSuchElementException; import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicReference; import org.apache.beam.sdk.io.UnboundedSource; import org.apache.beam.sdk.io.UnboundedSource.UnboundedReader; import org.apache.beam.sdk.io.solace.broker.MessageReceiver; @@ -71,7 +70,7 @@ public UnboundedSolaceReader(UnboundedSolaceSource currentSource) { @Override public boolean start() { populateSession(); - populateMessageReceiver(); + populateMessageConsumer(); return advance(); } @@ -84,13 +83,14 @@ public void populateSession() { } } - private void populateMessageReceiver() { + private void populateMessageConsumer() { if (messageReceiver == null) { messageReceiver = checkNotNull(sessionService).createReceiver(); messageReceiver.start(); } - if (messageReceiver != null && messageReceiver.isClosed()) { - checkNotNull(messageReceiver).start(); + MessageReceiver receiver = checkNotNull(messageReceiver); + if (receiver.isClosed()) { + receiver.start(); } } @@ -131,15 +131,14 @@ public Instant getWatermark() { @Override public UnboundedSource.CheckpointMark getCheckpointMark() { - List ackQueue = new ArrayList<>(); + List ackQueue = new ArrayList<>(); while (!elementsToCheckpoint.isEmpty()) { BytesXMLMessage msg = elementsToCheckpoint.poll(); if (msg != null) { - ackQueue.add(msg.getMessageIdLong()); + ackQueue.add(msg); } } - return new SolaceCheckpointMark( - active, ackQueue, new AtomicReference<>(checkNotNull(messageReceiver))); + return new SolaceCheckpointMark(active, ackQueue); } @Override diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/WatermarkParameters.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/WatermarkParameters.java index 48de8c5f31f2..dd9dc6739a45 100644 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/WatermarkParameters.java +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/WatermarkParameters.java @@ -17,12 +17,11 @@ */ package org.apache.beam.sdk.io.solace.read; -import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; - import com.google.auto.value.AutoValue; import java.io.Serializable; import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; import org.joda.time.Duration; import org.joda.time.Instant; @@ -73,7 +72,7 @@ abstract static class Builder { * extract the event time. */ public static WatermarkParameters create(SerializableFunction timestampFn) { - checkArgument(timestampFn != null, "timestampFn function is null"); + Preconditions.checkArgument(timestampFn != null, "timestampFn function is null"); return WatermarkParameters.builder().setTimestampFn(timestampFn).build(); } @@ -82,7 +81,8 @@ public static WatermarkParameters create(SerializableFunction * watermark idle duration threshold is {@link #STANDARD_WATERMARK_IDLE_DURATION_THRESHOLD}. */ public WatermarkParameters withWatermarkIdleDurationThreshold(Duration idleDurationThreshold) { - checkArgument(idleDurationThreshold != null, "watermark idle duration threshold is null"); + Preconditions.checkArgument( + idleDurationThreshold != null, "watermark idle duration threshold is null"); return toBuilder().setWatermarkIdleDurationThreshold(idleDurationThreshold).build(); } } diff --git a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/MockSessionService.java b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/MockSessionService.java index 54cc20d42096..e25564630800 100644 --- a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/MockSessionService.java +++ b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/MockSessionService.java @@ -30,21 +30,21 @@ public class MockSessionService implements SessionService { private final SerializableFunction getRecordFn; private final AtomicInteger ackCounter; private MessageReceiver messageReceiver = null; - private final int minMessagesReceived; + private final int isEndOfStreamAfterCount; public MockSessionService( - SerializableFunction getRecordFn, int minMessagesReceived) { + SerializableFunction getRecordFn, int isEndOfStreamAfterCount) { this.getRecordFn = getRecordFn; - this.minMessagesReceived = minMessagesReceived; + this.isEndOfStreamAfterCount = isEndOfStreamAfterCount; this.ackCounter = new AtomicInteger(); } public MockSessionService( SerializableFunction getRecordFn, AtomicInteger ackCounter, - int minMessagesReceived) { + int isEndOfStreamAfterCount) { this.getRecordFn = getRecordFn; - this.minMessagesReceived = minMessagesReceived; + this.isEndOfStreamAfterCount = isEndOfStreamAfterCount; this.ackCounter = ackCounter; } @@ -59,7 +59,7 @@ public boolean isClosed() { @Override public MessageReceiver createReceiver() { if (messageReceiver == null) { - messageReceiver = new MockReceiver(getRecordFn, ackCounter, minMessagesReceived); + messageReceiver = new MockReceiver(getRecordFn, ackCounter, isEndOfStreamAfterCount); } return messageReceiver; } @@ -71,15 +71,15 @@ public static class MockReceiver implements MessageReceiver, Serializable { private final AtomicInteger receiveCounter = new AtomicInteger(); private final AtomicInteger ackCounter; private final SerializableFunction getRecordFn; - private final int minMessagesReceived; + private final int isEndOfStreamAfterCount; public MockReceiver( SerializableFunction getRecordFn, AtomicInteger ackCounter, - int minMessagesReceived) { + int isEndOfStreamAfterCount) { this.getRecordFn = getRecordFn; this.ackCounter = ackCounter; - this.minMessagesReceived = minMessagesReceived; + this.isEndOfStreamAfterCount = isEndOfStreamAfterCount; } @Override @@ -102,7 +102,7 @@ public void ack(long ackId) throws IOException { @Override public boolean isEOF() { - return receiveCounter.get() >= minMessagesReceived; + return receiveCounter.get() >= isEndOfStreamAfterCount; } } } diff --git a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/SolaceDataUtils.java b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/SolaceDataUtils.java index e727c3c804a5..105cce57ff95 100644 --- a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/SolaceDataUtils.java +++ b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/SolaceDataUtils.java @@ -37,6 +37,7 @@ import org.apache.beam.sdk.io.solace.data.Solace; import org.apache.beam.sdk.schemas.JavaBeanSchema; import org.apache.beam.sdk.schemas.annotations.DefaultSchema; +import org.apache.beam.sdk.transforms.SerializableFunction; import org.checkerframework.checker.nullness.qual.Nullable; public class SolaceDataUtils { @@ -118,11 +119,19 @@ public static Solace.Record getSolaceRecord( } public static BytesXMLMessage getBytesXmlMessage(String payload, String messageId) { - return getBytesXmlMessage(payload, messageId, null); + return getBytesXmlMessage(payload, messageId, null, null); } public static BytesXMLMessage getBytesXmlMessage( - String payload, String messageId, ReplicationGroupMessageId replicationGroupMessageId) { + String payload, String messageId, SerializableFunction ackMessageFn) { + return getBytesXmlMessage(payload, messageId, ackMessageFn, null); + } + + public static BytesXMLMessage getBytesXmlMessage( + String payload, + String messageId, + SerializableFunction ackMessageFn, + ReplicationGroupMessageId replicationGroupMessageId) { long receiverTimestamp = 1708100477067L; long expiration = 1000L; long timeToLive = 1000L; @@ -170,7 +179,11 @@ public void writeBytes(byte[] arg0, int arg1, int arg2) { } @Override - public void ackMessage() {} + public void ackMessage() { + if (ackMessageFn != null) { + ackMessageFn.apply(0); + } + } @Override public void clearAttachment() { diff --git a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/SolaceIOTest.java b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/SolaceIOTest.java index ffe8fcdd0ed2..29aa24fc7cb8 100644 --- a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/SolaceIOTest.java +++ b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/SolaceIOTest.java @@ -32,7 +32,6 @@ import java.util.ArrayList; import java.util.List; import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.atomic.AtomicReference; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.io.UnboundedSource.CheckpointMark; import org.apache.beam.sdk.io.UnboundedSource.UnboundedReader; @@ -116,7 +115,7 @@ public void testReadMessages() { .from(Solace.Queue.fromName("queue")) .withSempClientFactory(getMockSempClientFactory()) .withSessionServiceFactory(fakeSessionServiceFactory) - .withMaxNumConnections(2)); + .withMaxNumConnections(1)); // Assert results PAssert.that(events).containsInAnyOrder(expected); @@ -154,7 +153,8 @@ public void testReadMessagesWithDeduplication() { .from(Solace.Queue.fromName("queue")) .withSempClientFactory(getMockSempClientFactory()) .withSessionServiceFactory(fakeSessionServiceFactory) - .withMaxNumConnections(2)); + .withDeduplicateRecords(true) + .withMaxNumConnections(1)); // Assert results PAssert.that(events).containsInAnyOrder(expected); pipeline.run(); @@ -208,11 +208,11 @@ public void testReadMessagesWithDeduplicationOnReplicationGroupMessageId() { List messages = ImmutableList.of( SolaceDataUtils.getBytesXmlMessage( - "payload_test0", null, new ReplicationGroupMessageIdImpl(2L, 1L)), + "payload_test0", null, null, new ReplicationGroupMessageIdImpl(2L, 1L)), SolaceDataUtils.getBytesXmlMessage( - "payload_test1", null, new ReplicationGroupMessageIdImpl(2L, 2L)), + "payload_test1", null, null, new ReplicationGroupMessageIdImpl(2L, 2L)), SolaceDataUtils.getBytesXmlMessage( - "payload_test2", null, new ReplicationGroupMessageIdImpl(2L, 2L))); + "payload_test2", null, null, new ReplicationGroupMessageIdImpl(2L, 2L))); return getOrNull(index, messages); }, 3); @@ -237,7 +237,8 @@ public void testReadMessagesWithDeduplicationOnReplicationGroupMessageId() { .from(Solace.Queue.fromName("queue")) .withSempClientFactory(getMockSempClientFactory()) .withSessionServiceFactory(fakeSessionServiceFactory) - .withMaxNumConnections(2)); + .withDeduplicateRecords(true) + .withMaxNumConnections(1)); // Assert results PAssert.that(events).containsInAnyOrder(expected); pipeline.run(); @@ -280,7 +281,7 @@ public void testReadWithCoderAndParseFnAndTimestampFn() { .from(Solace.Queue.fromName("queue")) .withSempClientFactory(getMockSempClientFactory()) .withSessionServiceFactory(fakeSessionServiceFactory) - .withMaxNumConnections(2)); + .withMaxNumConnections(1)); // Assert results PAssert.that(events).containsInAnyOrder(expected); @@ -376,12 +377,13 @@ public void testCheckpointMark() throws Exception { index -> { List messages = new ArrayList<>(); for (int i = 0; i < 10; i++) { - messages.add(SolaceDataUtils.getBytesXmlMessage("payload_test" + i, "45" + i)); + messages.add( + SolaceDataUtils.getBytesXmlMessage( + "payload_test" + i, "45" + i, (num) -> countAckMessages.incrementAndGet())); } countConsumedMessages.incrementAndGet(); return getOrNull(index, messages); }, - countAckMessages, 10); SessionServiceFactory fakeSessionServiceFactory = @@ -391,7 +393,7 @@ public void testCheckpointMark() throws Exception { .from(Solace.Queue.fromName("queue")) .withSempClientFactory(getMockSempClientFactory()) .withSessionServiceFactory(fakeSessionServiceFactory) - .withMaxNumConnections(4); + .withMaxNumConnections(1); UnboundedSolaceSource initialSource = getSource(spec, spec.getQueue(), pipeline); UnboundedReader reader = @@ -436,7 +438,6 @@ public void testCheckpointMarkAndFinalizeSeparately() throws Exception { countConsumedMessages.incrementAndGet(); return getOrNull(index, messages); }, - countAckMessages, 10); SessionServiceFactory fakeSessionServiceFactory = new MockSessionServiceFactory(mockClientService); @@ -548,14 +549,11 @@ public void testCheckpointMarkSafety() throws Exception { } @Test - public void testCheckpointMarkDefaultCoder() throws Exception { - SolaceCheckpointMark checkpointMark = - new SolaceCheckpointMark(null, new ArrayList<>(), new AtomicReference<>()); + public void testDefaultCoder() { Coder coder = new UnboundedSolaceSource<>(null, null, null, 0, false, null, null, null) .getCheckpointMarkCoder(); CoderProperties.coderSerializable(coder); - CoderProperties.coderDecodeEncodeEqual(coder, checkpointMark); } @Test From 7b24fdba0df93937dfa6eee5bf42ef0f1dfcbdf8 Mon Sep 17 00:00:00 2001 From: bzablocki Date: Wed, 29 May 2024 16:30:37 +0200 Subject: [PATCH 18/41] update changes.md with Solace read connector --- CHANGES.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CHANGES.md b/CHANGES.md index a81ff3c28121..b2b738c32ff4 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -62,7 +62,7 @@ ## I/Os -* Support for X source added (Java/Python) ([#X](https://github.com/apache/beam/issues/X)). +* Support for reading from Solace message broker added (Java) ([#31440](https://github.com/apache/beam/issues/31440)). ## New Features / Improvements From be74c86951b67c650f223e1cab406f75613dada6 Mon Sep 17 00:00:00 2001 From: bzablocki Date: Mon, 3 Jun 2024 14:15:29 +0200 Subject: [PATCH 19/41] remove ack by id code --- .../sdk/io/solace/broker/MessageReceiver.java | 2 - .../solace/broker/SolaceMessageReceiver.java | 24 +-- .../sdk/io/solace/MockSessionService.java | 40 +--- .../beam/sdk/io/solace/SolaceDataUtils.java | 185 +++++------------- 4 files changed, 61 insertions(+), 190 deletions(-) diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/MessageReceiver.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/MessageReceiver.java index 4b63e42c34ea..baf9e3beb038 100644 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/MessageReceiver.java +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/MessageReceiver.java @@ -25,8 +25,6 @@ public interface MessageReceiver { boolean isClosed(); - void ack(long ackId) throws IOException; - BytesXMLMessage receive() throws IOException; /** diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SolaceMessageReceiver.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SolaceMessageReceiver.java index 181b6c1a890c..e5f129d3ddfc 100644 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SolaceMessageReceiver.java +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SolaceMessageReceiver.java @@ -21,13 +21,14 @@ import com.solacesystems.jcsmp.FlowReceiver; import com.solacesystems.jcsmp.JCSMPException; import com.solacesystems.jcsmp.StaleSessionException; -import com.solacesystems.jcsmp.impl.flow.FlowHandle; import java.io.IOException; import org.apache.beam.sdk.io.solace.RetryCallableManager; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet; -import org.checkerframework.checker.nullness.qual.Nullable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class SolaceMessageReceiver implements MessageReceiver { + private static final Logger LOG = LoggerFactory.getLogger(SolaceMessageReceiver.class); public static final int DEFAULT_ADVANCE_TIMEOUT_IN_MILLIS = 100; private final FlowReceiver flowReceiver; @@ -58,29 +59,14 @@ public boolean isClosed() { @Override public BytesXMLMessage receive() throws IOException { - return receive(0, null); - } - - private BytesXMLMessage receive(int count, @Nullable Exception lastException) throws IOException { - if (count >= 2) { - throw new IOException( - "SolaceIO: tried to pull messages " + (count + 1) + " times, aborting.", lastException); - } try { return flowReceiver.receive(DEFAULT_ADVANCE_TIMEOUT_IN_MILLIS); } catch (StaleSessionException e) { + LOG.warn("SolaceIO: Caught StaleSessionException, restarting the FlowReceiver."); startFlowReceiver(); - return receive(count + 1, e); + throw new IOException(e); } catch (JCSMPException e) { throw new IOException(e); } } - - @Override - public void ack(long ackId) throws IOException { - if (isClosed()) { - throw new IOException("SolaceIO: FlowReceiver is closed, can't acknowledge messages."); - } - ((FlowHandle) flowReceiver).sendSingleAck(ackId, true); - } } diff --git a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/MockSessionService.java b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/MockSessionService.java index e25564630800..7b14da138c64 100644 --- a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/MockSessionService.java +++ b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/MockSessionService.java @@ -28,24 +28,13 @@ public class MockSessionService implements SessionService { private final SerializableFunction getRecordFn; - private final AtomicInteger ackCounter; private MessageReceiver messageReceiver = null; - private final int isEndOfStreamAfterCount; + private final int minMessagesReceived; public MockSessionService( - SerializableFunction getRecordFn, int isEndOfStreamAfterCount) { + SerializableFunction getRecordFn, int minMessagesReceived) { this.getRecordFn = getRecordFn; - this.isEndOfStreamAfterCount = isEndOfStreamAfterCount; - this.ackCounter = new AtomicInteger(); - } - - public MockSessionService( - SerializableFunction getRecordFn, - AtomicInteger ackCounter, - int isEndOfStreamAfterCount) { - this.getRecordFn = getRecordFn; - this.isEndOfStreamAfterCount = isEndOfStreamAfterCount; - this.ackCounter = ackCounter; + this.minMessagesReceived = minMessagesReceived; } @Override @@ -59,7 +48,7 @@ public boolean isClosed() { @Override public MessageReceiver createReceiver() { if (messageReceiver == null) { - messageReceiver = new MockReceiver(getRecordFn, ackCounter, isEndOfStreamAfterCount); + messageReceiver = new MockReceiver(getRecordFn, minMessagesReceived); } return messageReceiver; } @@ -68,18 +57,14 @@ public MessageReceiver createReceiver() { public void connect() {} public static class MockReceiver implements MessageReceiver, Serializable { - private final AtomicInteger receiveCounter = new AtomicInteger(); - private final AtomicInteger ackCounter; + private final AtomicInteger counter = new AtomicInteger(); private final SerializableFunction getRecordFn; - private final int isEndOfStreamAfterCount; + private final int minMessagesReceived; public MockReceiver( - SerializableFunction getRecordFn, - AtomicInteger ackCounter, - int isEndOfStreamAfterCount) { + SerializableFunction getRecordFn, int minMessagesReceived) { this.getRecordFn = getRecordFn; - this.ackCounter = ackCounter; - this.isEndOfStreamAfterCount = isEndOfStreamAfterCount; + this.minMessagesReceived = minMessagesReceived; } @Override @@ -92,17 +77,12 @@ public boolean isClosed() { @Override public BytesXMLMessage receive() throws IOException { - return getRecordFn.apply(receiveCounter.getAndIncrement()); - } - - @Override - public void ack(long ackId) throws IOException { - ackCounter.getAndIncrement(); + return getRecordFn.apply(counter.getAndIncrement()); } @Override public boolean isEOF() { - return receiveCounter.get() >= isEndOfStreamAfterCount; + return counter.get() >= minMessagesReceived; } } } diff --git a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/SolaceDataUtils.java b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/SolaceDataUtils.java index 105cce57ff95..2f61d0846f14 100644 --- a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/SolaceDataUtils.java +++ b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/SolaceDataUtils.java @@ -164,19 +164,13 @@ public int readBytes(byte[] arg0, int arg1) { } @Override - public void rewindContent() { - // return null; - } + public void rewindContent() {} @Override - public void writeBytes(byte[] arg0) { - // return null; - } + public void writeBytes(byte[] arg0) {} @Override - public void writeBytes(byte[] arg0, int arg1, int arg2) { - // return null; - } + public void writeBytes(byte[] arg0, int arg1, int arg2) {} @Override public void ackMessage() { @@ -186,29 +180,19 @@ public void ackMessage() { } @Override - public void clearAttachment() { - // return null; - } + public void clearAttachment() {} @Override - public void clearBinaryMetadataBytes(int arg0) { - // return null; - } + public void clearBinaryMetadataBytes(int arg0) {} @Override - public void clearContent() { - // return null; - } + public void clearContent() {} @Override - public void clearQueueNameLocation() { - // return null; - } + public void clearQueueNameLocation() {} @Override - public void clearTopicNameLocation() { - // return null; - } + public void clearTopicNameLocation() {} @Override public String dump() { @@ -557,194 +541,118 @@ public int readContentBytes(int arg0, byte[] arg1, int arg2, int arg3) { } @Override - public void rejectMessage() { - // return null; - } + public void rejectMessage() {} @Override - public void reset() { - // return null; - } + public void reset() {} @Override - public void resetPayload() { - // return null; - } + public void resetPayload() {} @Override - public void rewindAttachment() { - // return null; - } + public void rewindAttachment() {} @Override - public void setAckImmediately(boolean arg0) { - // return null; - } + public void setAckImmediately(boolean arg0) {} @Override - public void setAppMessageID(String arg0) { - // return null; - } + public void setAppMessageID(String arg0) {} @Override - public void setAppMessageType(String arg0) { - // return null; - } + public void setAppMessageType(String arg0) {} @Override - public void setApplicationMessageId(String arg0) { - // return null; - } + public void setApplicationMessageId(String arg0) {} @Override - public void setApplicationMessageType(String arg0) { - // return null; - } + public void setApplicationMessageType(String arg0) {} @Override - public void setAsReplyMessage(boolean arg0) { - // return null; - } + public void setAsReplyMessage(boolean arg0) {} @Override - public void setCorrelationId(String arg0) { - // return null; - } + public void setCorrelationId(String arg0) {} @Override - public void setCorrelationKey(Object arg0) { - // return null; - } + public void setCorrelationKey(Object arg0) {} @Override - public void setCos(User_Cos arg0) { - // return null; - } + public void setCos(User_Cos arg0) {} @Override - public void setDMQEligible(boolean arg0) { - // return null; - } + public void setDMQEligible(boolean arg0) {} @Override - public void setDeliverToOne(boolean arg0) { - // return null; - } + public void setDeliverToOne(boolean arg0) {} @Override - public void setDeliveryMode(DeliveryMode arg0) { - // return null; - } + public void setDeliveryMode(DeliveryMode arg0) {} @Override - public void setElidingEligible(boolean arg0) { - // return null; - } + public void setElidingEligible(boolean arg0) {} @Override - public void setExpiration(long arg0) { - // return null; - } + public void setExpiration(long arg0) {} @Override - public void setHTTPContentEncoding(String arg0) { - // return null; - } + public void setHTTPContentEncoding(String arg0) {} @Override - public void setHTTPContentType(String arg0) { - // return null; - } + public void setHTTPContentType(String arg0) {} @Override - public void setMessageType(MessageType arg0) { - // return null; - } + public void setMessageType(MessageType arg0) {} @Override - public void setPriority(int arg0) { - // return null; - } + public void setPriority(int arg0) {} @Override - public void setProperties(SDTMap arg0) { - // return null; - } + public void setProperties(SDTMap arg0) {} @Override - public void setQueueNameLocation(int arg0, int arg1) { - // return null; - } + public void setQueueNameLocation(int arg0, int arg1) {} @Override - public void setReadOnly() { - // return null; - } + public void setReadOnly() {} @Override - public void setReplyTo(Destination arg0) { - // return null; - } + public void setReplyTo(Destination arg0) {} @Override - public void setReplyToSuffix(String arg0) { - // return null; - } + public void setReplyToSuffix(String arg0) {} @Override - public void setSendTimestamp(long arg0) { - // return null; - } + public void setSendTimestamp(long arg0) {} @Override - public void setSenderID(String arg0) { - // return null; - } + public void setSenderID(String arg0) {} @Override - public void setSenderId(String arg0) { - // return null; - } + public void setSenderId(String arg0) {} @Override - public void setSenderTimestamp(long arg0) { - // return null; - } + public void setSenderTimestamp(long arg0) {} @Override - public void setSequenceNumber(long arg0) { - // return null; - } + public void setSequenceNumber(long arg0) {} @Override - public void setStructuredMsg(boolean arg0) { - // return null; - } + public void setStructuredMsg(boolean arg0) {} @Override - public void setStructuredMsgType(byte arg0) { - // return null; - } + public void setStructuredMsgType(byte arg0) {} @Override - public void setTimeToLive(long arg0) { - // return null; - } + public void setTimeToLive(long arg0) {} @Override - public void setTopicNameLocation(int arg0, int arg1) { - // return null; - } + public void setTopicNameLocation(int arg0, int arg1) {} @Override - public void setUserData(byte[] arg0) { - // return null; - } + public void setUserData(byte[] arg0) {} @Override - public void settle(Outcome arg0) throws JCSMPException { - // return null; - } + public void settle(Outcome arg0) throws JCSMPException {} @Override public int writeAttachment(byte[] arg0) { @@ -769,7 +677,6 @@ public int writeBinaryMetadataBytes(int arg0, byte[] arg1) { @Override public int writeBinaryMetadataBytes(int arg0, byte[] arg1, int arg2, int arg3) throws BufferUnderflowException { - // TOD0 Auto-generated method stub return 0; } From a85c68c751291c8e27c4164490db8b110d73bb5c Mon Sep 17 00:00:00 2001 From: bzablocki Date: Mon, 3 Jun 2024 14:20:50 +0200 Subject: [PATCH 20/41] remove todo comment --- .../test/java/org/apache/beam/sdk/io/solace/it/SolaceIOIT.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/it/SolaceIOIT.java b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/it/SolaceIOIT.java index d8a1612eb951..7dcd1829609a 100644 --- a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/it/SolaceIOIT.java +++ b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/it/SolaceIOIT.java @@ -104,8 +104,6 @@ public void testRead() { .apply("Count", ParDo.of(new CountingFn<>(NAMESPACE, READ_COUNT))); PipelineResult pipelineResult = readPipeline.run(); - // todo this needs to be parametrized when moved to the official repo and used with the - // :integrationTest plugin pipelineResult.waitUntilFinish(Duration.standardSeconds(15)); MetricsReader metricsReader = new MetricsReader(pipelineResult, NAMESPACE); From 533f122af1d2f294f0256f6e917dab834ec766d5 Mon Sep 17 00:00:00 2001 From: bzablocki Date: Mon, 3 Jun 2024 14:44:11 +0200 Subject: [PATCH 21/41] Add licenses to package-info.java files --- .../sdk/io/solace/broker/package-info.java | 18 ++++++++++++++++++ .../beam/sdk/io/solace/data/package-info.java | 18 ++++++++++++++++++ .../beam/sdk/io/solace/package-info.java | 18 ++++++++++++++++++ .../beam/sdk/io/solace/read/package-info.java | 18 ++++++++++++++++++ 4 files changed, 72 insertions(+) diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/package-info.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/package-info.java index ded1c2b002e0..960e24e2a1b3 100644 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/package-info.java +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/package-info.java @@ -1,2 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + /** Solace IO broker-related classes. */ package org.apache.beam.sdk.io.solace.broker; diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/data/package-info.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/data/package-info.java index edde1a8665c3..edf584310cab 100644 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/data/package-info.java +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/data/package-info.java @@ -1,2 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + /** Solace IO connector - data-related classes. */ package org.apache.beam.sdk.io.solace.data; diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/package-info.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/package-info.java index 7f2662a463df..3996b9ad3e04 100644 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/package-info.java +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/package-info.java @@ -1,2 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + /** Solace IO connector. */ package org.apache.beam.sdk.io.solace; diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/package-info.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/package-info.java index 4bcb8cabaac0..ce24cc1a3088 100644 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/package-info.java +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/package-info.java @@ -1,2 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + /** Solace IO connector - read connector classes. */ package org.apache.beam.sdk.io.solace.read; From 2f413803f4a261742e2272939b0fe0304b9e9c9d Mon Sep 17 00:00:00 2001 From: bzablocki Date: Mon, 3 Jun 2024 17:00:04 +0200 Subject: [PATCH 22/41] Restructure documentation --- .../apache/beam/sdk/io/solace/SolaceIO.java | 28 +++++++++---------- 1 file changed, 14 insertions(+), 14 deletions(-) diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/SolaceIO.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/SolaceIO.java index e13b85e423e2..be37fa93eba7 100644 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/SolaceIO.java +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/SolaceIO.java @@ -55,20 +55,6 @@ * *

    Note: this API is beta and subject to change. * - *

    Authentication

    - * - *

    When reading from Solace, the user must use {@link - * Read#withSessionServiceFactory(SessionServiceFactory)} to create a JCSMP session and {@link - * Read#withSempClientFactory(SempClientFactory)} to authenticate to the SEMP API. - * - *

    See {@link Read#withSessionServiceFactory(SessionServiceFactory)} for session authentication. - * The connector provides implementation of the {@link SessionServiceFactory} using the Basic - * Authentication: {@link org.apache.beam.sdk.io.solace.broker.BasicAuthJcsmpSessionService}. - * - *

    For the authentication to the SEMP API ({@link Read#withSempClientFactory(SempClientFactory)}) - * the connector provides {@link org.apache.beam.sdk.io.solace.broker.BasicAuthSempClientFactory} to - * authenticate using the Basic Authentication. - * *

    Reading

    * * To read from Solace, use the {@link SolaceIO#read()} or {@link SolaceIO#read(TypeDescriptor, @@ -192,6 +178,20 @@ * * * }
  • + * + *

    Authentication

    + * + *

    When reading from Solace, the user must use {@link + * Read#withSessionServiceFactory(SessionServiceFactory)} to create a JCSMP session and {@link + * Read#withSempClientFactory(SempClientFactory)} to authenticate to the SEMP API. + * + *

    See {@link Read#withSessionServiceFactory(SessionServiceFactory)} for session authentication. + * The connector provides implementation of the {@link SessionServiceFactory} using the Basic + * Authentication: {@link org.apache.beam.sdk.io.solace.broker.BasicAuthJcsmpSessionService}. + * + *

    For the authentication to the SEMP API ({@link Read#withSempClientFactory(SempClientFactory)}) + * the connector provides {@link org.apache.beam.sdk.io.solace.broker.BasicAuthSempClientFactory} to + * authenticate using the Basic Authentication. */ @Internal public class SolaceIO { From 9a317a85018a2e905d331e0d2fceec5c4697fc6b Mon Sep 17 00:00:00 2001 From: Bartosz Zablocki Date: Wed, 5 Jun 2024 12:45:25 +0000 Subject: [PATCH 23/41] update aws test after upgrading testcontainers version. --- .../test/java/org/apache/beam/sdk/io/aws/ITEnvironment.java | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/sdks/java/io/amazon-web-services/src/test/java/org/apache/beam/sdk/io/aws/ITEnvironment.java b/sdks/java/io/amazon-web-services/src/test/java/org/apache/beam/sdk/io/aws/ITEnvironment.java index ee882a71a5cd..3415a11bf9f0 100644 --- a/sdks/java/io/amazon-web-services/src/test/java/org/apache/beam/sdk/io/aws/ITEnvironment.java +++ b/sdks/java/io/amazon-web-services/src/test/java/org/apache/beam/sdk/io/aws/ITEnvironment.java @@ -20,6 +20,8 @@ import static org.apache.beam.sdk.testing.TestPipeline.testingPipelineOptions; import static org.testcontainers.containers.localstack.LocalStackContainer.Service.S3; +import com.amazonaws.auth.AWSStaticCredentialsProvider; +import com.amazonaws.auth.BasicAWSCredentials; import com.amazonaws.client.builder.AwsClientBuilder; import com.amazonaws.client.builder.AwsClientBuilder.EndpointConfiguration; import org.apache.beam.sdk.io.aws.options.AwsOptions; @@ -139,6 +141,8 @@ private void startLocalstack() { options.setAwsServiceEndpoint( localstack.getEndpointOverride(S3).toString()); // service irrelevant options.setAwsRegion(localstack.getRegion()); - options.setAwsCredentialsProvider(localstack.getDefaultCredentialsProvider()); + options.setAwsCredentialsProvider( + new AWSStaticCredentialsProvider( + new BasicAWSCredentials(localstack.getAccessKey(), localstack.getSecretKey()))); } } From aa29100edfc086ea3ee0b80cab19a9335885ffc1 Mon Sep 17 00:00:00 2001 From: bzablocki Date: Thu, 6 Jun 2024 11:58:39 +0200 Subject: [PATCH 24/41] Disable publishing docs until the first pass on the master branch --- sdks/java/io/solace/build.gradle | 2 ++ 1 file changed, 2 insertions(+) diff --git a/sdks/java/io/solace/build.gradle b/sdks/java/io/solace/build.gradle index 4a82b98c5ba4..e3fa0fc72f91 100644 --- a/sdks/java/io/solace/build.gradle +++ b/sdks/java/io/solace/build.gradle @@ -19,6 +19,8 @@ plugins { id 'org.apache.beam.module' } applyJavaNature( automaticModuleName: 'org.apache.beam.sdk.io.solace', + exportJavadoc: false, + publish: false, ) provideIntegrationTestingDependencies() enableJavaPerformanceTesting() From c625069f488f51d79d03a193427cfef1c965c18f Mon Sep 17 00:00:00 2001 From: Bartosz Zablocki Date: Thu, 6 Jun 2024 16:45:07 +0200 Subject: [PATCH 25/41] Remove files from this branch to split PR into smaller chunks --- sdks/java/io/solace/build.gradle | 17 -- .../sdk/io/solace/RetryCallableManager.java | 115 ---------- .../sdk/io/solace/SerializableSupplier.java | 25 --- .../apache/beam/sdk/io/solace/SolaceIO.java | 4 +- .../broker/BasicAuthJcsmpSessionService.java | 132 ------------ .../BasicAuthJcsmpSessionServiceFactory.java | 64 ------ .../io/solace/broker/BasicAuthSempClient.java | 94 -------- .../broker/BasicAuthSempClientFactory.java | 78 ------- .../sdk/io/solace/broker/BrokerResponse.java | 62 ------ .../broker/SempBasicAuthClientExecutor.java | 171 --------------- .../solace/broker/SolaceMessageReceiver.java | 72 ------- .../apache/beam/sdk/io/solace/data/Semp.java | 74 ------- .../io/solace/read/SolaceCheckpointMark.java | 2 + .../io/solace/RetryCallableManagerTest.java | 167 --------------- .../SempBasicAuthClientExecutorTest.java | 202 ------------------ .../beam/sdk/io/solace/data/SolaceTest.java | 2 - 16 files changed, 3 insertions(+), 1278 deletions(-) delete mode 100644 sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/RetryCallableManager.java delete mode 100644 sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/SerializableSupplier.java delete mode 100644 sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/BasicAuthJcsmpSessionService.java delete mode 100644 sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/BasicAuthJcsmpSessionServiceFactory.java delete mode 100644 sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/BasicAuthSempClient.java delete mode 100644 sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/BasicAuthSempClientFactory.java delete mode 100644 sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/BrokerResponse.java delete mode 100644 sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SempBasicAuthClientExecutor.java delete mode 100644 sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SolaceMessageReceiver.java delete mode 100644 sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/data/Semp.java delete mode 100644 sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/RetryCallableManagerTest.java delete mode 100644 sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/broker/SempBasicAuthClientExecutorTest.java diff --git a/sdks/java/io/solace/build.gradle b/sdks/java/io/solace/build.gradle index e3fa0fc72f91..674f4c0fe6c2 100644 --- a/sdks/java/io/solace/build.gradle +++ b/sdks/java/io/solace/build.gradle @@ -35,31 +35,14 @@ dependencies { implementation library.java.slf4j_api implementation library.java.joda_time implementation library.java.solace - implementation library.java.google_cloud_core implementation library.java.vendored_guava_32_1_2_jre - implementation library.java.gax - implementation library.java.google_http_client - implementation library.java.google_http_client_gson implementation project(":sdks:java:extensions:avro") implementation library.java.avro permitUnusedDeclared library.java.avro - implementation library.java.jackson_core - implementation library.java.jackson_databind - implementation library.java.google_api_common - implementation library.java.guava - permitUnusedDeclared library.java.guava - implementation library.java.threetenbp - testImplementation library.java.activemq_amqp - testImplementation library.java.activemq_broker - testImplementation library.java.activemq_jaas - testImplementation library.java.activemq_kahadb_store - testImplementation library.java.activemq_client - testImplementation library.java.hamcrest testImplementation library.java.junit testImplementation project(path: ":sdks:java:io:common", configuration: "testRuntimeMigration") testImplementation project(path: ":sdks:java:testing:test-utils", configuration: "testRuntimeMigration") testRuntimeOnly library.java.slf4j_jdk14 testRuntimeOnly project(path: ":runners:direct-java", configuration: "shadow") - testImplementation library.java.testcontainers_solace } diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/RetryCallableManager.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/RetryCallableManager.java deleted file mode 100644 index 212f4faa6bfc..000000000000 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/RetryCallableManager.java +++ /dev/null @@ -1,115 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.sdk.io.solace; - -import com.google.api.core.NanoClock; -import com.google.api.gax.retrying.RetrySettings; -import com.google.auto.value.AutoValue; -import com.google.cloud.ExceptionHandler; -import com.google.cloud.ExceptionHandler.Interceptor; -import com.google.cloud.RetryHelper; -import java.io.Serializable; -import java.util.Set; -import java.util.concurrent.Callable; -import org.apache.beam.sdk.annotations.Internal; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet; - -@Internal -@AutoValue -public abstract class RetryCallableManager implements Serializable { - - private static final int NUMBER_OF_RETRIES = 4; - private static final int RETRY_INTERVAL_SECONDS = 1; - private static final int RETRY_MULTIPLIER = 2; - private static final int MAX_DELAY = - NUMBER_OF_RETRIES * RETRY_MULTIPLIER * RETRY_INTERVAL_SECONDS + 1; - - public static RetryCallableManager create() { - return builder().build(); - } - /** - * Method that executes and repeats the execution of the callable argument, if it throws one of - * the exceptions from the exceptionsToIntercept Set. - */ - public V retryCallable( - Callable callable, Set> exceptionsToIntercept) { - return RetryHelper.runWithRetries( - callable, - getRetrySettings(), - getExceptionHandlerForExceptions(exceptionsToIntercept), - NanoClock.getDefaultClock()); - } - - private ExceptionHandler getExceptionHandlerForExceptions( - Set> exceptionsToIntercept) { - return ExceptionHandler.newBuilder() - .abortOn(RuntimeException.class) - .addInterceptors(new ExceptionSetInterceptor(ImmutableSet.copyOf(exceptionsToIntercept))) - .build(); - } - - abstract RetrySettings getRetrySettings(); - - abstract Builder toBuilder(); - - static Builder builder() { - return new AutoValue_RetryCallableManager.Builder() - .setRetrySettings( - RetrySettings.newBuilder() - .setInitialRetryDelay(org.threeten.bp.Duration.ofSeconds(RETRY_INTERVAL_SECONDS)) - .setMaxAttempts(NUMBER_OF_RETRIES) - .setMaxRetryDelay(org.threeten.bp.Duration.ofSeconds(MAX_DELAY)) - .setRetryDelayMultiplier(RETRY_MULTIPLIER) - .build()); - } - - @AutoValue.Builder - abstract static class Builder { - abstract Builder setRetrySettings(RetrySettings retrySettings); - - abstract RetryCallableManager build(); - } - - private static class ExceptionSetInterceptor implements Interceptor { - private static final long serialVersionUID = -8429573586820467828L; - private final Set> exceptionsToIntercept; - - public ExceptionSetInterceptor(Set> exceptionsToIntercept) { - this.exceptionsToIntercept = exceptionsToIntercept; - } - - @Override - public RetryResult afterEval(Exception exception, RetryResult retryResult) { - return Interceptor.RetryResult.CONTINUE_EVALUATION; - } - - @Override - public RetryResult beforeEval(Exception exceptionToEvaluate) { - for (Class exceptionToIntercept : exceptionsToIntercept) { - if (isOf(exceptionToIntercept, exceptionToEvaluate)) { - return Interceptor.RetryResult.RETRY; - } - } - return Interceptor.RetryResult.CONTINUE_EVALUATION; - } - - private boolean isOf(Class clazz, Object obj) { - return clazz.isInstance(obj); - } - } -} diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/SerializableSupplier.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/SerializableSupplier.java deleted file mode 100644 index 3438b39ccf94..000000000000 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/SerializableSupplier.java +++ /dev/null @@ -1,25 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.sdk.io.solace; - -import java.io.Serializable; - -@FunctionalInterface -public interface SerializableSupplier extends Serializable { - OutputT get(); -} diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/SolaceIO.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/SolaceIO.java index be37fa93eba7..948c4ed3c676 100644 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/SolaceIO.java +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/SolaceIO.java @@ -526,9 +526,7 @@ Coder inferCoder(Pipeline pipeline) { } } - // FIXME: this is public only for the sake of testing, TODO: redesign test so this is - // private - public Queue initializeQueueForTopic(String jobName, SempClientFactory sempClientFactory) { + Queue initializeQueueForTopic(String jobName, SempClientFactory sempClientFactory) { Queue initializedQueue; Queue solaceQueue = getQueue(); if (solaceQueue != null) { diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/BasicAuthJcsmpSessionService.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/BasicAuthJcsmpSessionService.java deleted file mode 100644 index 79a6de23092d..000000000000 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/BasicAuthJcsmpSessionService.java +++ /dev/null @@ -1,132 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.sdk.io.solace.broker; - -import com.solacesystems.jcsmp.ConsumerFlowProperties; -import com.solacesystems.jcsmp.EndpointProperties; -import com.solacesystems.jcsmp.FlowReceiver; -import com.solacesystems.jcsmp.InvalidPropertiesException; -import com.solacesystems.jcsmp.JCSMPException; -import com.solacesystems.jcsmp.JCSMPFactory; -import com.solacesystems.jcsmp.JCSMPProperties; -import com.solacesystems.jcsmp.JCSMPSession; -import com.solacesystems.jcsmp.Queue; -import java.io.IOException; -import javax.annotation.Nullable; -import org.apache.beam.sdk.io.solace.RetryCallableManager; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet; - -public class BasicAuthJcsmpSessionService implements SessionService { - private final String queueName; - private final String host; - private final String username; - private final String password; - private final String vpnName; - @Nullable private JCSMPSession jcsmpSession; - private final RetryCallableManager retryCallableManager = RetryCallableManager.create(); - - public BasicAuthJcsmpSessionService( - String queueName, String host, String username, String password, String vpnName) { - this.queueName = queueName; - this.host = host; - this.username = username; - this.password = password; - this.vpnName = vpnName; - } - - @Override - public void connect() { - retryCallableManager.retryCallable(this::connectSession, ImmutableSet.of(JCSMPException.class)); - } - - @Override - public void close() { - if (jcsmpSession != null && !jcsmpSession.isClosed()) { - retryCallableManager.retryCallable( - () -> { - if (jcsmpSession != null) { - jcsmpSession.closeSession(); - } - return 0; - }, - ImmutableSet.of(IOException.class)); - } - } - - @Override - public MessageReceiver createReceiver() { - return retryCallableManager.retryCallable( - this::createFlowReceiver, ImmutableSet.of(JCSMPException.class)); - } - - @Override - public boolean isClosed() { - return jcsmpSession == null || jcsmpSession.isClosed(); - } - - private MessageReceiver createFlowReceiver() throws JCSMPException, IOException { - if (isClosed()) { - connectSession(); - } - - Queue queue = JCSMPFactory.onlyInstance().createQueue(queueName); - - ConsumerFlowProperties flowProperties = new ConsumerFlowProperties(); - flowProperties.setEndpoint(queue); - flowProperties.setAckMode(JCSMPProperties.SUPPORTED_MESSAGE_ACK_CLIENT); - - EndpointProperties endpointProperties = new EndpointProperties(); - endpointProperties.setAccessType(EndpointProperties.ACCESSTYPE_NONEXCLUSIVE); - if (jcsmpSession != null) { - return new SolaceMessageReceiver( - createFlowReceiver(jcsmpSession, flowProperties, endpointProperties)); - } else { - throw new IOException( - "SolaceIO.Read: Could not create a receiver from the Jcsmp session: session object is null."); - } - } - // The `@SuppressWarning` is needed here, because the checkerframework reports an error for the - // first argument of the `createFlow` being null, even though the documentation allows it: - // https://docs.solace.com/API-Developer-Online-Ref-Documentation/java/com/solacesystems/jcsmp/JCSMPSession.html#createFlow-com.solacesystems.jcsmp.XMLMessageListener-com.solacesystems.jcsmp.ConsumerFlowProperties-com.solacesystems.jcsmp.EndpointProperties- - @SuppressWarnings("nullness") - private static FlowReceiver createFlowReceiver( - JCSMPSession jcsmpSession, - ConsumerFlowProperties flowProperties, - EndpointProperties endpointProperties) - throws JCSMPException { - return jcsmpSession.createFlow(null, flowProperties, endpointProperties); - } - - private int connectSession() throws JCSMPException { - if (jcsmpSession == null) { - jcsmpSession = createSessionObject(); - } - jcsmpSession.connect(); - return 0; - } - - private JCSMPSession createSessionObject() throws InvalidPropertiesException { - JCSMPProperties properties = new JCSMPProperties(); - properties.setProperty(JCSMPProperties.HOST, host); - properties.setProperty(JCSMPProperties.USERNAME, username); - properties.setProperty(JCSMPProperties.PASSWORD, password); - properties.setProperty(JCSMPProperties.VPN_NAME, vpnName); - - return JCSMPFactory.onlyInstance().createSession(properties); - } -} diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/BasicAuthJcsmpSessionServiceFactory.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/BasicAuthJcsmpSessionServiceFactory.java deleted file mode 100644 index e8e7484df312..000000000000 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/BasicAuthJcsmpSessionServiceFactory.java +++ /dev/null @@ -1,64 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.sdk.io.solace.broker; - -import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkNotNull; - -import com.google.auto.value.AutoValue; - -@AutoValue -public abstract class BasicAuthJcsmpSessionServiceFactory extends SessionServiceFactory { - public abstract String host(); - - public abstract String username(); - - public abstract String password(); - - public abstract String vpnName(); - - public static Builder builder() { - return new AutoValue_BasicAuthJcsmpSessionServiceFactory.Builder(); - } - - @AutoValue.Builder - public abstract static class Builder { - - /** Set Solace host, format: [Protocol://]Host[:Port]. */ - public abstract Builder host(String host); - - /** Set Solace username. */ - public abstract Builder username(String username); - /** Set Solace password. */ - public abstract Builder password(String password); - - /** Set Solace vpn name. */ - public abstract Builder vpnName(String vpnName); - - public abstract BasicAuthJcsmpSessionServiceFactory build(); - } - - @Override - public SessionService create() { - return new BasicAuthJcsmpSessionService( - checkNotNull(queue, "SolaceIO.Read: Queue is not set.").getName(), - host(), - username(), - password(), - vpnName()); - } -} diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/BasicAuthSempClient.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/BasicAuthSempClient.java deleted file mode 100644 index 4a24d4f748b5..000000000000 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/BasicAuthSempClient.java +++ /dev/null @@ -1,94 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.sdk.io.solace.broker; - -import com.fasterxml.jackson.core.JsonProcessingException; -import com.fasterxml.jackson.databind.DeserializationFeature; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.api.client.http.HttpRequestFactory; -import com.solacesystems.jcsmp.JCSMPFactory; -import java.io.IOException; -import org.apache.beam.sdk.annotations.Internal; -import org.apache.beam.sdk.io.solace.SerializableSupplier; -import org.apache.beam.sdk.io.solace.data.Semp.Queue; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -@Internal -public class BasicAuthSempClient implements SempClient { - private static final Logger LOG = LoggerFactory.getLogger(BasicAuthSempClient.class); - private final ObjectMapper objectMapper = - new ObjectMapper().configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); - - private final SempBasicAuthClientExecutor sempBasicAuthClientExecutor; - - public BasicAuthSempClient( - String host, - String username, - String password, - String vpnName, - SerializableSupplier httpRequestFactorySupplier) { - sempBasicAuthClientExecutor = - new SempBasicAuthClientExecutor( - host, username, password, vpnName, httpRequestFactorySupplier.get()); - } - - @Override - public boolean isQueueNonExclusive(String queueName) throws IOException { - LOG.info("SolaceIO.Read: SempOperations: query SEMP if queue {} is nonExclusive", queueName); - BrokerResponse response = sempBasicAuthClientExecutor.getQueueResponse(queueName); - if (response.content == null) { - throw new IOException("SolaceIO: response from SEMP is empty!"); - } - Queue q = mapJsonToClass(response.content, Queue.class); - return q.data().accessType().equals("non-exclusive"); - } - - @Override - public com.solacesystems.jcsmp.Queue createQueueForTopic(String queueName, String topicName) - throws IOException { - createQueue(queueName); - createSubscription(queueName, topicName); - return JCSMPFactory.onlyInstance().createQueue(queueName); - } - - @Override - public long getBacklogBytes(String queueName) throws IOException { - BrokerResponse response = sempBasicAuthClientExecutor.getQueueResponse(queueName); - if (response.content == null) { - throw new IOException("SolaceIO: response from SEMP is empty!"); - } - Queue q = mapJsonToClass(response.content, Queue.class); - return q.data().msgSpoolUsage(); - } - - private void createQueue(String queueName) throws IOException { - LOG.info("SolaceIO.Read: Creating new queue {}.", queueName); - sempBasicAuthClientExecutor.createQueueResponse(queueName); - } - - private void createSubscription(String queueName, String topicName) throws IOException { - LOG.info("SolaceIO.Read: Creating new subscription {} for topic {}.", queueName, topicName); - sempBasicAuthClientExecutor.createSubscriptionResponse(queueName, topicName); - } - - private T mapJsonToClass(String content, Class mapSuccessToClass) - throws JsonProcessingException { - return objectMapper.readValue(content, mapSuccessToClass); - } -} diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/BasicAuthSempClientFactory.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/BasicAuthSempClientFactory.java deleted file mode 100644 index b3a8db418a77..000000000000 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/BasicAuthSempClientFactory.java +++ /dev/null @@ -1,78 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.sdk.io.solace.broker; - -import com.google.api.client.http.HttpRequestFactory; -import com.google.api.client.http.javanet.NetHttpTransport; -import com.google.auto.value.AutoValue; -import org.apache.beam.sdk.io.solace.SerializableSupplier; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; -import org.checkerframework.checker.nullness.qual.NonNull; -import org.checkerframework.checker.nullness.qual.Nullable; - -@AutoValue -public abstract class BasicAuthSempClientFactory implements SempClientFactory { - - public abstract String host(); - - public abstract String username(); - - public abstract String password(); - - public abstract String vpnName(); - - public abstract @Nullable SerializableSupplier httpRequestFactorySupplier(); - - public static Builder builder() { - return new AutoValue_BasicAuthSempClientFactory.Builder(); - } - - @AutoValue.Builder - public abstract static class Builder { - /** Set Solace host, format: [Protocol://]Host[:Port]. */ - public abstract Builder host(String host); - - /** Set Solace username. */ - public abstract Builder username(String username); - /** Set Solace password. */ - public abstract Builder password(String password); - - /** Set Solace vpn name. */ - public abstract Builder vpnName(String vpnName); - - @VisibleForTesting - abstract Builder httpRequestFactorySupplier( - SerializableSupplier httpRequestFactorySupplier); - - public abstract BasicAuthSempClientFactory build(); - } - - @Override - public SempClient create() { - return new BasicAuthSempClient( - host(), username(), password(), vpnName(), getHttpRequestFactorySupplier()); - } - - @SuppressWarnings("return") - private @NonNull SerializableSupplier getHttpRequestFactorySupplier() { - SerializableSupplier httpRequestSupplier = httpRequestFactorySupplier(); - return httpRequestSupplier != null - ? httpRequestSupplier - : () -> new NetHttpTransport().createRequestFactory(); - } -} diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/BrokerResponse.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/BrokerResponse.java deleted file mode 100644 index 1a47f8012285..000000000000 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/BrokerResponse.java +++ /dev/null @@ -1,62 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.sdk.io.solace.broker; - -import com.google.api.client.http.HttpResponse; -import java.io.BufferedReader; -import java.io.IOException; -import java.io.InputStream; -import java.io.InputStreamReader; -import java.nio.charset.StandardCharsets; -import java.util.stream.Collectors; -import org.checkerframework.checker.nullness.qual.Nullable; - -public class BrokerResponse { - final int code; - final String message; - @Nullable String content; - - public BrokerResponse(int responseCode, String message, @Nullable InputStream content) { - this.code = responseCode; - this.message = message; - if (content != null) { - this.content = - new BufferedReader(new InputStreamReader(content, StandardCharsets.UTF_8)) - .lines() - .collect(Collectors.joining("\n")); - } - } - - public static BrokerResponse fromHttpResponse(HttpResponse response) throws IOException { - return new BrokerResponse( - response.getStatusCode(), response.getStatusMessage(), response.getContent()); - } - - @Override - public String toString() { - return "BrokerResponse{" - + "code=" - + code - + ", message='" - + message - + '\'' - + ", content=" - + content - + '}'; - } -} diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SempBasicAuthClientExecutor.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SempBasicAuthClientExecutor.java deleted file mode 100644 index 23f4d6401526..000000000000 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SempBasicAuthClientExecutor.java +++ /dev/null @@ -1,171 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.sdk.io.solace.broker; - -import com.google.api.client.http.GenericUrl; -import com.google.api.client.http.HttpContent; -import com.google.api.client.http.HttpHeaders; -import com.google.api.client.http.HttpRequest; -import com.google.api.client.http.HttpRequestFactory; -import com.google.api.client.http.HttpResponse; -import com.google.api.client.http.HttpResponseException; -import com.google.api.client.http.json.JsonHttpContent; -import com.google.api.client.json.gson.GsonFactory; -import java.io.IOException; -import java.io.Serializable; -import java.net.CookieManager; -import java.net.HttpCookie; -import java.util.List; -import java.util.stream.Collectors; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; - -/** - * A class to execute requests to SEMP v2 with Basic Auth authentication. - * - *

    This approach takes advantage of SEMP Sessions. The - * session is established when a user authenticates with HTTP Basic authentication. When the - * response is 401 Unauthorized, the client will execute an additional request with Basic Auth - * header to refresh the token. - */ -class SempBasicAuthClientExecutor implements Serializable { - private static final CookieManager COOKIE_MANAGER = new CookieManager(); - private static final String COOKIES_HEADER = "Set-Cookie"; - - private final String username; - private final String messageVpn; - private final String baseUrl; - private final String password; - private final transient HttpRequestFactory requestFactory; - - SempBasicAuthClientExecutor( - String host, - String username, - String password, - String vpnName, - HttpRequestFactory httpRequestFactory) { - this.baseUrl = String.format("%s/SEMP/v2", host); - this.username = username; - this.messageVpn = vpnName; - this.password = password; - this.requestFactory = httpRequestFactory; - } - - private static String getQueueEndpoint(String messageVpn, String queueName) { - return String.format("/monitor/msgVpns/%s/queues/%s", messageVpn, queueName); - } - - private static String createQueueEndpoint(String messageVpn) { - return String.format("/config/msgVpns/%s/queues", messageVpn); - } - - private static String subscriptionEndpoint(String messageVpn, String queueName) { - return String.format("/config/msgVpns/%s/queues/%s/subscriptions", messageVpn, queueName); - } - - BrokerResponse getQueueResponse(String queueName) throws IOException { - String queryUrl = getQueueEndpoint(messageVpn, queueName); - HttpResponse response = executeGet(new GenericUrl(baseUrl + queryUrl)); - return BrokerResponse.fromHttpResponse(response); - } - - BrokerResponse createQueueResponse(String queueName) throws IOException { - String queryUrl = createQueueEndpoint(messageVpn); - ImmutableMap params = - ImmutableMap.builder() - .put("accessType", "non-exclusive") - .put("queueName", queueName) - .put("owner", username) - .put("permission", "consume") - .put("ingressEnabled", true) - .put("egressEnabled", true) - .build(); - - HttpResponse response = executePost(new GenericUrl(baseUrl + queryUrl), params); - return BrokerResponse.fromHttpResponse(response); - } - - BrokerResponse createSubscriptionResponse(String queueName, String topicName) throws IOException { - String queryUrl = subscriptionEndpoint(messageVpn, queueName); - - ImmutableMap params = - ImmutableMap.builder() - .put("subscriptionTopic", topicName) - .put("queueName", queueName) - .build(); - HttpResponse response = executePost(new GenericUrl(baseUrl + queryUrl), params); - return BrokerResponse.fromHttpResponse(response); - } - - private HttpResponse executeGet(GenericUrl url) throws IOException { - HttpRequest request = requestFactory.buildGetRequest(url); - return execute(request); - } - - private HttpResponse executePost(GenericUrl url, ImmutableMap parameters) - throws IOException { - HttpContent content = new JsonHttpContent(GsonFactory.getDefaultInstance(), parameters); - HttpRequest request = requestFactory.buildPostRequest(url, content); - return execute(request); - } - - private HttpResponse execute(HttpRequest request) throws IOException { - request.setNumberOfRetries(2); - HttpHeaders httpHeaders = new HttpHeaders(); - boolean authFromCookie = COOKIE_MANAGER.getCookieStore().getCookies().size() > 0; - if (authFromCookie) { - setCookiesFromCookieManager(httpHeaders); - request.setHeaders(httpHeaders); - } else { - httpHeaders.setBasicAuthentication(username, password); - request.setHeaders(httpHeaders); - } - - HttpResponse response; - try { - response = request.execute(); - } catch (HttpResponseException e) { - if (authFromCookie && e.getStatusCode() == 401) { - COOKIE_MANAGER.getCookieStore().removeAll(); - // execute again without cookies to refresh the token. - return execute(request); - } else { - throw e; - } - } - - storeCookiesInCookieManager(response.getHeaders()); - return response; - } - - private void setCookiesFromCookieManager(HttpHeaders httpHeaders) { - httpHeaders.setCookie( - COOKIE_MANAGER.getCookieStore().getCookies().stream() - .map(s -> s.getName() + "=" + s.getValue()) - .collect(Collectors.joining(";"))); - } - - private void storeCookiesInCookieManager(HttpHeaders headers) { - List cookiesHeader = headers.getHeaderStringValues(COOKIES_HEADER); - if (cookiesHeader != null) { - for (String cookie : cookiesHeader) { - COOKIE_MANAGER.getCookieStore().add(null, HttpCookie.parse(cookie).get(0)); - } - } - } -} diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SolaceMessageReceiver.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SolaceMessageReceiver.java deleted file mode 100644 index e5f129d3ddfc..000000000000 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SolaceMessageReceiver.java +++ /dev/null @@ -1,72 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.sdk.io.solace.broker; - -import com.solacesystems.jcsmp.BytesXMLMessage; -import com.solacesystems.jcsmp.FlowReceiver; -import com.solacesystems.jcsmp.JCSMPException; -import com.solacesystems.jcsmp.StaleSessionException; -import java.io.IOException; -import org.apache.beam.sdk.io.solace.RetryCallableManager; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class SolaceMessageReceiver implements MessageReceiver { - private static final Logger LOG = LoggerFactory.getLogger(SolaceMessageReceiver.class); - - public static final int DEFAULT_ADVANCE_TIMEOUT_IN_MILLIS = 100; - private final FlowReceiver flowReceiver; - private final RetryCallableManager retryCallableManager = RetryCallableManager.create(); - - public SolaceMessageReceiver(FlowReceiver flowReceiver) { - this.flowReceiver = flowReceiver; - } - - @Override - public void start() { - startFlowReceiver(); - } - - private void startFlowReceiver() { - retryCallableManager.retryCallable( - () -> { - flowReceiver.start(); - return 0; - }, - ImmutableSet.of(JCSMPException.class)); - } - - @Override - public boolean isClosed() { - return flowReceiver == null || flowReceiver.isClosed(); - } - - @Override - public BytesXMLMessage receive() throws IOException { - try { - return flowReceiver.receive(DEFAULT_ADVANCE_TIMEOUT_IN_MILLIS); - } catch (StaleSessionException e) { - LOG.warn("SolaceIO: Caught StaleSessionException, restarting the FlowReceiver."); - startFlowReceiver(); - throw new IOException(e); - } catch (JCSMPException e) { - throw new IOException(e); - } - } -} diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/data/Semp.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/data/Semp.java deleted file mode 100644 index f6f0fb51d22e..000000000000 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/data/Semp.java +++ /dev/null @@ -1,74 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.sdk.io.solace.data; - -import com.fasterxml.jackson.databind.annotation.JsonDeserialize; -import com.fasterxml.jackson.databind.annotation.JsonPOJOBuilder; -import com.fasterxml.jackson.databind.annotation.JsonSerialize; -import com.google.auto.value.AutoValue; - -public class Semp { - - @AutoValue - @JsonSerialize(as = Queue.class) - @JsonDeserialize(builder = AutoValue_Semp_Queue.Builder.class) - public abstract static class Queue { - - public abstract QueueData data(); - - public static Builder builder() { - return new AutoValue_Semp_Queue.Builder(); - } - - public abstract Builder toBuilder(); - - @AutoValue.Builder - @JsonPOJOBuilder(withPrefix = "set") - abstract static class Builder { - - public abstract Builder setData(QueueData queueData); - - public abstract Queue build(); - } - } - - @AutoValue - @JsonDeserialize(builder = AutoValue_Semp_QueueData.Builder.class) - public abstract static class QueueData { - public abstract String accessType(); - - public abstract long msgSpoolUsage(); - - public static Builder builder() { - return new AutoValue_Semp_QueueData.Builder(); - } - - public abstract Builder toBuilder(); - - @AutoValue.Builder - @JsonPOJOBuilder(withPrefix = "set") - abstract static class Builder { - - public abstract Builder setAccessType(String accessType); - - public abstract Builder setMsgSpoolUsage(long msgSpoolUsage); - - public abstract QueueData build(); - } - } -} diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/SolaceCheckpointMark.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/SolaceCheckpointMark.java index 07b6666389f1..10336d00d8b5 100644 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/SolaceCheckpointMark.java +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/SolaceCheckpointMark.java @@ -37,6 +37,8 @@ @DefaultCoder(AvroCoder.class) public class SolaceCheckpointMark implements UnboundedSource.CheckpointMark { private transient AtomicBoolean activeReader; + // BytesXMLMessage is not serializable so if a job restarts from the checkpoint, we cannot retry + // these messages here. We relay on Solace's retry mechanism. private transient ArrayDeque ackQueue; @SuppressWarnings("initialization") // Avro will set the fields by breaking abstraction diff --git a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/RetryCallableManagerTest.java b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/RetryCallableManagerTest.java deleted file mode 100644 index a747dbe3c4cb..000000000000 --- a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/RetryCallableManagerTest.java +++ /dev/null @@ -1,167 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.sdk.io.solace; - -import static org.junit.Assert.assertTrue; - -import com.google.api.gax.retrying.RetrySettings; -import com.google.cloud.RetryHelper.RetryHelperException; -import java.util.concurrent.Callable; -import java.util.concurrent.atomic.AtomicInteger; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet; -import org.junit.Before; -import org.junit.Test; - -public class RetryCallableManagerTest { - private static final int NUMBER_OF_RETRIES = 4; - private static final int RETRY_INTERVAL_SECONDS = 0; - private static final int RETRY_MULTIPLIER = 2; - private static final int MAX_DELAY = 0; - - private RetryCallableManager retryCallableManager; - - @Before - public void setUp() { - - retryCallableManager = - RetryCallableManager.builder() - .setRetrySettings( - RetrySettings.newBuilder() - .setInitialRetryDelay( - org.threeten.bp.Duration.ofSeconds(RETRY_INTERVAL_SECONDS)) - .setMaxAttempts(NUMBER_OF_RETRIES) - .setMaxRetryDelay(org.threeten.bp.Duration.ofSeconds(MAX_DELAY)) - .setRetryDelayMultiplier(RETRY_MULTIPLIER) - .build()) - .build(); - } - - @Test - public void testRetryCallable_ReturnsExpected() { - AtomicInteger executeCounter = new AtomicInteger(0); - Callable incrementingFunction = - () -> { - executeCounter.incrementAndGet(); - if (executeCounter.get() < 2) { - throw new MyException(); - } - return executeCounter.get(); - }; - Integer result = - retryCallableManager.retryCallable( - incrementingFunction, ImmutableSet.of(MyException.class)); - assertTrue(String.format("Should return 2, instead returned %d.", result), result == 2); - } - - @Test - public void testRetryCallable_RetriesExpectedNumberOfTimes() { - AtomicInteger executeCounter = new AtomicInteger(0); - Callable incrementingFunction = - () -> { - executeCounter.incrementAndGet(); - if (executeCounter.get() < 2) { - throw new MyException(); - } - return executeCounter.get(); - }; - retryCallableManager.retryCallable(incrementingFunction, ImmutableSet.of(MyException.class)); - assertTrue( - String.format("Should run 2 times, instead ran %d times.", executeCounter.get()), - executeCounter.get() == 2); - } - - @Test(expected = RetryHelperException.class) - public void testRetryCallable_ThrowsRetryHelperException() { - Callable incrementingFunction = - () -> { - { - throw new MyException(); - } - }; - retryCallableManager.retryCallable(incrementingFunction, ImmutableSet.of(MyException.class)); - } - - @Test - public void testRetryCallable_ExecutionCountIsCorrectAfterMultipleExceptions() { - AtomicInteger executeCounter = new AtomicInteger(0); - Callable incrementingFunction = - () -> { - executeCounter.incrementAndGet(); - throw new MyException(); - }; - try { - retryCallableManager.retryCallable(incrementingFunction, ImmutableSet.of(MyException.class)); - } catch (RetryHelperException e) { - // ignore exception to check the executeCounter - } - assertTrue( - String.format("Should execute 4 times, instead executed %d times", executeCounter.get()), - executeCounter.get() == 4); - } - - @Test(expected = RetryHelperException.class) - public void testRetryCallable_ThrowsRetryHelperExceptionOnUnspecifiedException() { - Callable incrementingFunction = - () -> { - throw new DoNotIgnoreException(); - }; - retryCallableManager.retryCallable(incrementingFunction, ImmutableSet.of(MyException.class)); - } - - @Test - public void testRetryCallable_ChecksForAllDefinedExceptions() { - AtomicInteger executeCounter = new AtomicInteger(0); - Callable incrementingFunction = - () -> { - executeCounter.incrementAndGet(); - if (executeCounter.get() % 2 == 0) { - throw new MyException(); - } else if (executeCounter.get() % 2 == 1) { - throw new AnotherException(); - } - return 0; - }; - try { - retryCallableManager.retryCallable( - incrementingFunction, ImmutableSet.of(MyException.class, AnotherException.class)); - } catch (RetryHelperException e) { - // ignore exception to check the executeCounter - } - assertTrue( - String.format("Should execute 4 times, instead executed %d times", executeCounter.get()), - executeCounter.get() == 4); - } - - private static class MyException extends Exception { - public MyException() { - super(); - } - } - - private static class AnotherException extends Exception { - public AnotherException() { - super(); - } - } - - private static class DoNotIgnoreException extends Exception { - public DoNotIgnoreException() { - super(); - } - } -} diff --git a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/broker/SempBasicAuthClientExecutorTest.java b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/broker/SempBasicAuthClientExecutorTest.java deleted file mode 100644 index 8cc48ed17ef6..000000000000 --- a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/broker/SempBasicAuthClientExecutorTest.java +++ /dev/null @@ -1,202 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.sdk.io.solace.broker; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertThrows; -import static org.junit.Assert.assertTrue; - -import com.google.api.client.http.HttpRequestFactory; -import com.google.api.client.http.HttpResponseException; -import com.google.api.client.http.LowLevelHttpRequest; -import com.google.api.client.http.LowLevelHttpResponse; -import com.google.api.client.json.Json; -import com.google.api.client.testing.http.MockHttpTransport; -import com.google.api.client.testing.http.MockLowLevelHttpRequest; -import com.google.api.client.testing.http.MockLowLevelHttpResponse; -import java.io.IOException; -import java.util.List; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; -import org.junit.Test; - -public class SempBasicAuthClientExecutorTest { - - @Test - public void testExecuteStatus4xx() { - MockHttpTransport transport = - new MockHttpTransport() { - @Override - public LowLevelHttpRequest buildRequest(String method, String url) { - return new MockLowLevelHttpRequest() { - @Override - public LowLevelHttpResponse execute() { - MockLowLevelHttpResponse response = new MockLowLevelHttpResponse(); - response.setStatusCode(404); - response.setContentType(Json.MEDIA_TYPE); - response.setContent( - "{\"meta\":{\"error\":{\"code\":404,\"description\":\"some" - + " error\",\"status\":\"xx\"}}}"); - return response; - } - }; - } - }; - - HttpRequestFactory requestFactory = transport.createRequestFactory(); - SempBasicAuthClientExecutor client = - new SempBasicAuthClientExecutor( - "http://host", "username", "password", "vpnName", requestFactory); - - assertThrows(HttpResponseException.class, () -> client.getQueueResponse("queue")); - } - - @Test - public void testExecuteStatus3xx() { - MockHttpTransport transport = - new MockHttpTransport() { - @Override - public LowLevelHttpRequest buildRequest(String method, String url) { - return new MockLowLevelHttpRequest() { - @Override - public LowLevelHttpResponse execute() { - MockLowLevelHttpResponse response = new MockLowLevelHttpResponse(); - response.setStatusCode(301); - response.setContentType(Json.MEDIA_TYPE); - response.setContent( - "{\"meta\":{\"error\":{\"code\":301,\"description\":\"some" - + " error\",\"status\":\"xx\"}}}"); - return response; - } - }; - } - }; - - HttpRequestFactory requestFactory = transport.createRequestFactory(); - SempBasicAuthClientExecutor client = - new SempBasicAuthClientExecutor( - "http://host", "username", "password", "vpnName", requestFactory); - - assertThrows(HttpResponseException.class, () -> client.getQueueResponse("queue")); - } - - /** - * In this test case, we test a situation when a session that we used to authenticate to Semp - * expires. - * - *

    To test this scenario, we need to do the following: - * - *

      - *
    1. Send the first request, to initialize a session. This request has to contain the Basic - * Auth header and should not include any cookie headers. The response for this request - * contains a session cookie we can re-use in the following requests. - *
    2. Send the second request - this request should use a cookie from the previous response. - * There should be no Authorization header. To simulate an expired session scenario, we set - * the response of this request to the "401 Unauthorized". This should cause a the request - * to be retried, this time with the Authorization header. - *
    3. Validate the third request to contain the Basic Auth header and no session cookies. - *
    - */ - @Test - public void testExecuteWithUnauthorized() throws IOException { - // Making it a final array, so that we can reference it from within the MockHttpTransport - // instance - final int[] requestCounter = {0}; - MockHttpTransport transport = - new MockHttpTransport() { - @Override - public LowLevelHttpRequest buildRequest(String method, String url) { - return new MockLowLevelHttpRequest() { - @Override - public LowLevelHttpResponse execute() throws IOException { - MockLowLevelHttpResponse response = new MockLowLevelHttpResponse(); - if (requestCounter[0] == 0) { - // The first request has to include Basic Auth header - assertTrue(this.getHeaders().containsKey("authorization")); - List authorizationHeaders = this.getHeaders().get("authorization"); - assertEquals(1, authorizationHeaders.size()); - assertTrue(authorizationHeaders.get(0).contains("Basic")); - assertFalse(this.getHeaders().containsKey("cookie")); - - // Set the response to include Session cookies - response - .setHeaderNames(ImmutableList.of("Set-Cookie", "Set-Cookie")) - .setHeaderValues( - ImmutableList.of( - "ProxySession=JddSdJaGo6FYYmQk6nt8jXxFtq6n3FCFR14ebzRGQ5w;" - + " HttpOnly; SameSite=Strict;" - + " Path=/proxy; Max-Age=2592000", - "Session=JddSdJaGo6FYYmQk6nt8jXxFtq6n3FCFR14ebzRGQ5w;" - + " HttpOnly; SameSite=Strict;" - + " Path=/SEMP; Max-Age=2592000")); - response.setStatusCode(200); - } else if (requestCounter[0] == 1) { - // The second request does not include Basic Auth header - assertFalse(this.getHeaders().containsKey("authorization")); - // It must include a cookie header - assertTrue(this.getHeaders().containsKey("cookie")); - boolean hasSessionCookie = - this.getHeaders().get("cookie").stream() - .filter( - c -> - c.contains( - "Session=JddSdJaGo6FYYmQk6nt8jXxFtq6n3FCFR14ebzRGQ5w")) - .count() - == 1; - assertTrue(hasSessionCookie); - - // Let's assume the Session expired - we return the 401 - // unauthorized - response.setStatusCode(401); - } else { - // The second request has to be retried with a Basic Auth header - // this time - assertTrue(this.getHeaders().containsKey("authorization")); - List authorizationHeaders = this.getHeaders().get("authorization"); - assertEquals(1, authorizationHeaders.size()); - assertTrue(authorizationHeaders.get(0).contains("Basic")); - assertFalse(this.getHeaders().containsKey("cookie")); - - response.setStatusCode(200); - } - response.setContentType(Json.MEDIA_TYPE); - requestCounter[0]++; - return response; - } - }; - } - }; - - HttpRequestFactory requestFactory = transport.createRequestFactory(); - SempBasicAuthClientExecutor client = - new SempBasicAuthClientExecutor( - "http://host", "username", "password", "vpnName", requestFactory); - - // The first, initial request - client.getQueueResponse("queue"); - // The second request, which will try to authenticate with a cookie, and then with Basic - // Auth when it receives a 401 unauthorized - client.getQueueResponse("queue"); - - // There should be 3 requests executed: - // the first one is the initial one with Basic Auth, - // the second one uses the session cookie, but we simulate it being expired, - // so there should be a third request with Basic Auth to create a new session. - assertEquals(3, requestCounter[0]); - } -} diff --git a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/data/SolaceTest.java b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/data/SolaceTest.java index b53e113086ac..47b065be98a0 100644 --- a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/data/SolaceTest.java +++ b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/data/SolaceTest.java @@ -38,12 +38,10 @@ public class SolaceTest { String replyTo = "no-one"; Long receiveTimestamp = 123456789L; Long senderTimestamp = 987654321L; - long timestampMillis = 1234567890L; Long sequenceNumber = 27L; Long timeToLive = 34567890L; String payloadString = "some payload"; byte[] payload = payloadString.getBytes(StandardCharsets.UTF_8); - String publishError = "some error"; @Test public void testRecordEquality() { From 1521f403f59b9ea8c4651853ef001c7094ad5547 Mon Sep 17 00:00:00 2001 From: Bartosz Zablocki Date: Thu, 6 Jun 2024 16:59:54 +0200 Subject: [PATCH 26/41] refactor tests for readability --- .../beam/sdk/io/solace/SolaceIOTest.java | 69 +++++-------------- 1 file changed, 19 insertions(+), 50 deletions(-) diff --git a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/SolaceIOTest.java b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/SolaceIOTest.java index 29aa24fc7cb8..750588bc561c 100644 --- a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/SolaceIOTest.java +++ b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/SolaceIOTest.java @@ -62,6 +62,17 @@ public class SolaceIOTest { @Rule public final transient TestPipeline pipeline = TestPipeline.create(); + private static Read getDefaultQueueRead(SessionServiceFactory fakeSessionServiceFactory) { + return getDefaultRead(fakeSessionServiceFactory).from(Solace.Queue.fromName("queue")); + } + + private static Read getDefaultRead(SessionServiceFactory fakeSessionServiceFactory) { + return SolaceIO.read() + .withSempClientFactory(getMockSempClientFactory()) + .withSessionServiceFactory(fakeSessionServiceFactory) + .withMaxNumConnections(1); + } + private static BytesXMLMessage getOrNull(Integer index, List messages) { return index != null && index < messages.size() ? messages.get(index) : null; } @@ -109,13 +120,7 @@ public void testReadMessages() { // Run the pipeline PCollection events = - pipeline.apply( - "Read from Solace", - SolaceIO.read() - .from(Solace.Queue.fromName("queue")) - .withSempClientFactory(getMockSempClientFactory()) - .withSessionServiceFactory(fakeSessionServiceFactory) - .withMaxNumConnections(1)); + pipeline.apply("Read from Solace", getDefaultQueueRead(fakeSessionServiceFactory)); // Assert results PAssert.that(events).containsInAnyOrder(expected); @@ -149,12 +154,7 @@ public void testReadMessagesWithDeduplication() { PCollection events = pipeline.apply( "Read from Solace", - SolaceIO.read() - .from(Solace.Queue.fromName("queue")) - .withSempClientFactory(getMockSempClientFactory()) - .withSessionServiceFactory(fakeSessionServiceFactory) - .withDeduplicateRecords(true) - .withMaxNumConnections(1)); + getDefaultQueueRead(fakeSessionServiceFactory).withDeduplicateRecords(true)); // Assert results PAssert.that(events).containsInAnyOrder(expected); pipeline.run(); @@ -185,14 +185,7 @@ public void testReadMessagesWithoutDeduplication() { // Run the pipeline PCollection events = - pipeline.apply( - "Read from Solace", - SolaceIO.read() - .from(Solace.Queue.fromName("queue")) - .withSempClientFactory(getMockSempClientFactory()) - .withSessionServiceFactory(fakeSessionServiceFactory) - .withMaxNumConnections(1) - .withDeduplicateRecords(false)); + pipeline.apply("Read from Solace", getDefaultQueueRead(fakeSessionServiceFactory)); // Assert results PAssert.that(events).containsInAnyOrder(expected); @@ -233,12 +226,7 @@ public void testReadMessagesWithDeduplicationOnReplicationGroupMessageId() { PCollection events = pipeline.apply( "Read from Solace", - SolaceIO.read() - .from(Solace.Queue.fromName("queue")) - .withSempClientFactory(getMockSempClientFactory()) - .withSessionServiceFactory(fakeSessionServiceFactory) - .withDeduplicateRecords(true) - .withMaxNumConnections(1)); + getDefaultQueueRead(fakeSessionServiceFactory).withDeduplicateRecords(true)); // Assert results PAssert.that(events).containsInAnyOrder(expected); pipeline.run(); @@ -388,12 +376,7 @@ public void testCheckpointMark() throws Exception { SessionServiceFactory fakeSessionServiceFactory = new MockSessionServiceFactory(mockClientService); - Read spec = - SolaceIO.read() - .from(Solace.Queue.fromName("queue")) - .withSempClientFactory(getMockSempClientFactory()) - .withSessionServiceFactory(fakeSessionServiceFactory) - .withMaxNumConnections(1); + Read spec = getDefaultQueueRead(fakeSessionServiceFactory); UnboundedSolaceSource initialSource = getSource(spec, spec.getQueue(), pipeline); UnboundedReader reader = @@ -442,12 +425,7 @@ public void testCheckpointMarkAndFinalizeSeparately() throws Exception { SessionServiceFactory fakeSessionServiceFactory = new MockSessionServiceFactory(mockClientService); - Read spec = - SolaceIO.read() - .from(Solace.Queue.fromName("queue")) - .withSempClientFactory(getMockSempClientFactory()) - .withSessionServiceFactory(fakeSessionServiceFactory) - .withMaxNumConnections(4); + Read spec = getDefaultQueueRead(fakeSessionServiceFactory).withMaxNumConnections(4); UnboundedSolaceSource initialSource = getSource(spec, spec.getQueue(), pipeline); @@ -504,12 +482,7 @@ public void testCheckpointMarkSafety() throws Exception { SessionServiceFactory fakeSessionServiceFactory = new MockSessionServiceFactory(mockClientService); - Read spec = - SolaceIO.read() - .from(Solace.Queue.fromName("queue")) - .withSempClientFactory(getMockSempClientFactory()) - .withSessionServiceFactory(fakeSessionServiceFactory) - .withMaxNumConnections(4); + Read spec = getDefaultQueueRead(fakeSessionServiceFactory).withMaxNumConnections(4); UnboundedSolaceSource initialSource = getSource(spec, spec.getQueue(), pipeline); @@ -600,11 +573,7 @@ public void testTopicEncoding() { PCollection events = pipeline.apply( "Read from Solace", - SolaceIO.read() - .from(Solace.Queue.fromName("queue")) - .withSempClientFactory(getMockSempClientFactory()) - .withSessionServiceFactory(fakeSessionServiceFactory) - .withMaxNumConnections(2)); + getDefaultQueueRead(fakeSessionServiceFactory)); // Run the pipeline PCollection destAreTopics = From b8152864d3ec459f384a0417d9d7f9e89f40a8c1 Mon Sep 17 00:00:00 2001 From: Bartosz Zablocki Date: Thu, 6 Jun 2024 17:26:05 +0200 Subject: [PATCH 27/41] revert upgrade of testcontainers - not needed in this PR chunk --- .../groovy/org/apache/beam/gradle/BeamModulePlugin.groovy | 3 +-- .../java/org/apache/beam/sdk/io/aws2/ITEnvironment.java | 6 +----- 2 files changed, 2 insertions(+), 7 deletions(-) diff --git a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy index 46085ba85ca6..a9a9d5d10088 100644 --- a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy +++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy @@ -640,7 +640,7 @@ class BeamModulePlugin implements Plugin { def spark2_version = "2.4.8" def spark3_version = "3.2.2" def spotbugs_version = "4.0.6" - def testcontainers_version = "1.19.7" + def testcontainers_version = "1.17.3" // [bomupgrader] determined by: org.apache.arrow:arrow-memory-core, consistent with: google_cloud_platform_libraries_bom def arrow_version = "15.0.2" def jmh_version = "1.34" @@ -903,7 +903,6 @@ class BeamModulePlugin implements Plugin { testcontainers_oracle : "org.testcontainers:oracle-xe:$testcontainers_version", testcontainers_postgresql : "org.testcontainers:postgresql:$testcontainers_version", testcontainers_rabbitmq : "org.testcontainers:rabbitmq:$testcontainers_version", - testcontainers_solace : "org.testcontainers:solace:$testcontainers_version", truth : "com.google.truth:truth:1.1.5", threetenbp : "org.threeten:threetenbp:1.6.8", vendored_grpc_1_60_1 : "org.apache.beam:beam-vendor-grpc-1_60_1:0.2", diff --git a/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/ITEnvironment.java b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/ITEnvironment.java index 4497ededb3c9..649abdf724f9 100644 --- a/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/ITEnvironment.java +++ b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/ITEnvironment.java @@ -34,8 +34,6 @@ import org.testcontainers.containers.localstack.LocalStackContainer.Service; import org.testcontainers.containers.output.Slf4jLogConsumer; import org.testcontainers.utility.DockerImageName; -import software.amazon.awssdk.auth.credentials.AwsBasicCredentials; -import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider; import software.amazon.awssdk.awscore.client.builder.AwsClientBuilder; import software.amazon.awssdk.regions.Region; @@ -141,8 +139,6 @@ private void startLocalstack() { localstack.start(); options.setEndpoint(localstack.getEndpointOverride(S3)); // service irrelevant options.setAwsRegion(Region.of(localstack.getRegion())); - options.setAwsCredentialsProvider( - StaticCredentialsProvider.create( - AwsBasicCredentials.create(localstack.getAccessKey(), localstack.getSecretKey()))); + options.setAwsCredentialsProvider(localstack.getDefaultCredentialsProvider()); } } From 08b777cd1436af710d2a9cd9d0873533f01122c6 Mon Sep 17 00:00:00 2001 From: Bartosz Zablocki Date: Thu, 6 Jun 2024 17:32:14 +0200 Subject: [PATCH 28/41] revert upgrade of testcontainers - not needed in this PR chunk --- .../test/java/org/apache/beam/sdk/io/aws/ITEnvironment.java | 6 +----- .../java/org/apache/beam/sdk/io/aws2/ITEnvironment.java | 6 +++++- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/sdks/java/io/amazon-web-services/src/test/java/org/apache/beam/sdk/io/aws/ITEnvironment.java b/sdks/java/io/amazon-web-services/src/test/java/org/apache/beam/sdk/io/aws/ITEnvironment.java index 3415a11bf9f0..ee882a71a5cd 100644 --- a/sdks/java/io/amazon-web-services/src/test/java/org/apache/beam/sdk/io/aws/ITEnvironment.java +++ b/sdks/java/io/amazon-web-services/src/test/java/org/apache/beam/sdk/io/aws/ITEnvironment.java @@ -20,8 +20,6 @@ import static org.apache.beam.sdk.testing.TestPipeline.testingPipelineOptions; import static org.testcontainers.containers.localstack.LocalStackContainer.Service.S3; -import com.amazonaws.auth.AWSStaticCredentialsProvider; -import com.amazonaws.auth.BasicAWSCredentials; import com.amazonaws.client.builder.AwsClientBuilder; import com.amazonaws.client.builder.AwsClientBuilder.EndpointConfiguration; import org.apache.beam.sdk.io.aws.options.AwsOptions; @@ -141,8 +139,6 @@ private void startLocalstack() { options.setAwsServiceEndpoint( localstack.getEndpointOverride(S3).toString()); // service irrelevant options.setAwsRegion(localstack.getRegion()); - options.setAwsCredentialsProvider( - new AWSStaticCredentialsProvider( - new BasicAWSCredentials(localstack.getAccessKey(), localstack.getSecretKey()))); + options.setAwsCredentialsProvider(localstack.getDefaultCredentialsProvider()); } } diff --git a/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/ITEnvironment.java b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/ITEnvironment.java index 649abdf724f9..4497ededb3c9 100644 --- a/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/ITEnvironment.java +++ b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/ITEnvironment.java @@ -34,6 +34,8 @@ import org.testcontainers.containers.localstack.LocalStackContainer.Service; import org.testcontainers.containers.output.Slf4jLogConsumer; import org.testcontainers.utility.DockerImageName; +import software.amazon.awssdk.auth.credentials.AwsBasicCredentials; +import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider; import software.amazon.awssdk.awscore.client.builder.AwsClientBuilder; import software.amazon.awssdk.regions.Region; @@ -139,6 +141,8 @@ private void startLocalstack() { localstack.start(); options.setEndpoint(localstack.getEndpointOverride(S3)); // service irrelevant options.setAwsRegion(Region.of(localstack.getRegion())); - options.setAwsCredentialsProvider(localstack.getDefaultCredentialsProvider()); + options.setAwsCredentialsProvider( + StaticCredentialsProvider.create( + AwsBasicCredentials.create(localstack.getAccessKey(), localstack.getSecretKey()))); } } From 0f63749389e4460a964fdf3d1d2ffd72a3a38b85 Mon Sep 17 00:00:00 2001 From: Bartosz Zablocki Date: Thu, 6 Jun 2024 17:48:09 +0200 Subject: [PATCH 29/41] spotless --- .../test/java/org/apache/beam/sdk/io/solace/SolaceIOTest.java | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/SolaceIOTest.java b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/SolaceIOTest.java index 750588bc561c..010bffe31b26 100644 --- a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/SolaceIOTest.java +++ b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/SolaceIOTest.java @@ -571,9 +571,7 @@ public void testTopicEncoding() { // Run PCollection events = - pipeline.apply( - "Read from Solace", - getDefaultQueueRead(fakeSessionServiceFactory)); + pipeline.apply("Read from Solace", getDefaultQueueRead(fakeSessionServiceFactory)); // Run the pipeline PCollection destAreTopics = From e90e69a51c230df8191c3b3d3218d104df7ac79e Mon Sep 17 00:00:00 2001 From: Bartosz Zablocki Date: Thu, 6 Jun 2024 17:55:55 +0200 Subject: [PATCH 30/41] remove IT tests from this pr --- .../io/solace/it/SolaceContainerManager.java | 168 ------------------ .../beam/sdk/io/solace/it/SolaceIOIT.java | 128 ------------- 2 files changed, 296 deletions(-) delete mode 100644 sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/it/SolaceContainerManager.java delete mode 100644 sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/it/SolaceIOIT.java diff --git a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/it/SolaceContainerManager.java b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/it/SolaceContainerManager.java deleted file mode 100644 index e9c3fe7dfcb6..000000000000 --- a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/it/SolaceContainerManager.java +++ /dev/null @@ -1,168 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.sdk.io.solace.it; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.testcontainers.containers.output.Slf4jLogConsumer; -import org.testcontainers.solace.Service; -import org.testcontainers.solace.SolaceContainer; -import org.testcontainers.utility.DockerImageName; - -public class SolaceContainerManager { - - public static final String VPN_NAME = "default"; - public static final String PASSWORD = "password"; - public static final String USERNAME = "username"; - public static final String TOPIC_NAME = "test_topic"; - private static final Logger LOG = LoggerFactory.getLogger(SolaceContainerManager.class); - private final SolaceContainer container; - - public SolaceContainerManager() { - this.container = - new SolaceContainer(DockerImageName.parse("solace/solace-pubsub-standard:10.7")) { - { - addFixedExposedPort(55555, 55555); - addFixedExposedPort(9000, 9000); - addFixedExposedPort(8080, 8080); - addFixedExposedPort(80, 80); - } - }.withVpn(VPN_NAME) - .withCredentials(USERNAME, PASSWORD) - // .withExposedPorts(Service.SMF.getPort()); - .withTopic(TOPIC_NAME, Service.SMF) - .withLogConsumer(new Slf4jLogConsumer(LOG)); - container.addExposedPort(8080); - container.addExposedPort(55555); - } - - public void start() { - container.start(); - } - - void createQueueWithSubscriptionTopic(String queueName) { - executeCommand( - "curl", - "http://localhost:8080/SEMP/v2/config/msgVpns/" + VPN_NAME + "/topicEndpoints", - "-X", - "POST", - "-u", - "admin:admin", - "-H", - "Content-Type:application/json", - "-d", - "{\"topicEndpointName\":\"" - + TOPIC_NAME - + "\",\"accessType\":\"exclusive\",\"permission\":\"modify-topic\",\"ingressEnabled\":true,\"egressEnabled\":true}"); - executeCommand( - "curl", - "http://localhost:8080/SEMP/v2/config/msgVpns/" + VPN_NAME + "/queues", - "-X", - "POST", - "-u", - "admin:admin", - "-H", - "Content-Type:application/json", - "-d", - "{\"queueName\":\"" - + queueName - + "\",\"accessType\":\"non-exclusive\",\"maxMsgSpoolUsage\":200,\"permission\":\"consume\",\"ingressEnabled\":true,\"egressEnabled\":true}"); - executeCommand( - "curl", - "http://localhost:8080/SEMP/v2/config/msgVpns/" - + VPN_NAME - + "/queues/" - + queueName - + "/subscriptions", - "-X", - "POST", - "-u", - "admin:admin", - "-H", - "Content-Type:application/json", - "-d", - "{\"subscriptionTopic\":\"" + TOPIC_NAME + "\"}"); - } - - private void executeCommand(String... command) { - try { - org.testcontainers.containers.Container.ExecResult execResult = - container.execInContainer(command); - if (execResult.getExitCode() != 0) { - logCommandError(execResult.getStderr(), command); - } else { - LOG.info(execResult.getStdout()); - } - } catch (IOException | InterruptedException e) { - logCommandError(e.getMessage(), command); - } - } - - private void logCommandError(String error, String... command) { - LOG.error("Could not execute command {}: {}", command, error); - } - - public void stop() { - if (container != null) { - container.stop(); - } - } - - public void getQueueDetails(String queueName) { - executeCommand( - "curl", - "http://localhost:8080/SEMP/v2/monitor/msgVpns/" - + VPN_NAME - + "/queues/" - + queueName - + "/msgs", - "-X", - "GET", - "-u", - "admin:admin"); - } - - public void sendToTopic(String payload, List additionalHeaders) { - // https://docs.solace.com/API/RESTMessagingPrtl/Solace-REST-Message-Encoding.htm - - List command = - new ArrayList<>( - Arrays.asList( - "curl", - "http://localhost:9000/TOPIC/" + TOPIC_NAME, - "-X", - "POST", - "-u", - USERNAME + ":" + PASSWORD, - "--header", - "Content-Type:application/json", - "-d", - payload)); - - for (String additionalHeader : additionalHeaders) { - command.add("--header"); - command.add(additionalHeader); - } - - executeCommand(command.toArray(new String[0])); - } -} diff --git a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/it/SolaceIOIT.java b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/it/SolaceIOIT.java deleted file mode 100644 index 7dcd1829609a..000000000000 --- a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/it/SolaceIOIT.java +++ /dev/null @@ -1,128 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.sdk.io.solace.it; - -import static org.junit.Assert.assertEquals; - -import org.apache.beam.sdk.PipelineResult; -import org.apache.beam.sdk.io.solace.SolaceIO; -import org.apache.beam.sdk.io.solace.broker.BasicAuthJcsmpSessionServiceFactory; -import org.apache.beam.sdk.io.solace.broker.BasicAuthSempClientFactory; -import org.apache.beam.sdk.io.solace.data.Solace.Queue; -import org.apache.beam.sdk.metrics.Counter; -import org.apache.beam.sdk.metrics.Metrics; -import org.apache.beam.sdk.options.PipelineOptionsFactory; -import org.apache.beam.sdk.options.StreamingOptions; -import org.apache.beam.sdk.testing.TestPipeline; -import org.apache.beam.sdk.testing.TestPipelineOptions; -import org.apache.beam.sdk.testutils.metrics.MetricsReader; -import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.transforms.ParDo; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; -import org.joda.time.Duration; -import org.junit.AfterClass; -import org.junit.BeforeClass; -import org.junit.Rule; -import org.junit.Test; - -public class SolaceIOIT { - private static final String NAMESPACE = SolaceIOIT.class.getName(); - private static final String READ_COUNT = "read_count"; - private static SolaceContainerManager solaceContainerManager; - private static final TestPipelineOptions readPipelineOptions; - - static { - readPipelineOptions = PipelineOptionsFactory.create().as(TestPipelineOptions.class); - readPipelineOptions.setBlockOnRun(false); - readPipelineOptions.as(TestPipelineOptions.class).setBlockOnRun(false); - readPipelineOptions.as(StreamingOptions.class).setStreaming(false); - } - - @Rule public final TestPipeline readPipeline = TestPipeline.fromOptions(readPipelineOptions); - - @BeforeClass - public static void setup() { - solaceContainerManager = new SolaceContainerManager(); - solaceContainerManager.start(); - } - - @AfterClass - public static void afterClass() { - if (solaceContainerManager != null) { - solaceContainerManager.stop(); - } - } - - @Test - public void testRead() { - String queueName = "test_queue"; - solaceContainerManager.createQueueWithSubscriptionTopic(queueName); - - // todo this is very slow, needs to be replaced with the SolaceIO.write connector. - int publishMessagesCount = 20; - for (int i = 0; i < publishMessagesCount; i++) { - solaceContainerManager.sendToTopic( - "{\"field_str\":\"value\",\"field_int\":123}", - ImmutableList.of("Solace-Message-ID:m" + i)); - } - - readPipeline - .apply( - "Read from Solace", - SolaceIO.read() - .from(Queue.fromName(queueName)) - .withMaxNumConnections(1) - .withSempClientFactory( - BasicAuthSempClientFactory.builder() - .host("http://localhost:8080") - .username("admin") - .password("admin") - .vpnName(SolaceContainerManager.VPN_NAME) - .build()) - .withSessionServiceFactory( - BasicAuthJcsmpSessionServiceFactory.builder() - .host("localhost") - .username(SolaceContainerManager.USERNAME) - .password(SolaceContainerManager.PASSWORD) - .vpnName(SolaceContainerManager.VPN_NAME) - .build())) - .apply("Count", ParDo.of(new CountingFn<>(NAMESPACE, READ_COUNT))); - - PipelineResult pipelineResult = readPipeline.run(); - pipelineResult.waitUntilFinish(Duration.standardSeconds(15)); - - MetricsReader metricsReader = new MetricsReader(pipelineResult, NAMESPACE); - long actualRecordsCount = metricsReader.getCounterMetric(READ_COUNT); - assertEquals(publishMessagesCount, actualRecordsCount); - } - - private static class CountingFn extends DoFn { - - private final Counter elementCounter; - - CountingFn(String namespace, String name) { - elementCounter = Metrics.counter(namespace, name); - } - - @ProcessElement - public void processElement(@Element T record, OutputReceiver c) { - elementCounter.inc(1L); - c.output(record); - } - } -} From f6833d2e2b4cd47846d738618fa5ebb1844b96c9 Mon Sep 17 00:00:00 2001 From: Bartosz Zablocki Date: Fri, 7 Jun 2024 10:11:19 +0200 Subject: [PATCH 31/41] Tech Writer review --- .../main/java/org/apache/beam/sdk/io/solace/SolaceIO.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/SolaceIO.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/SolaceIO.java index 948c4ed3c676..8c4d1d7d44bd 100644 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/SolaceIO.java +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/SolaceIO.java @@ -60,7 +60,7 @@ * To read from Solace, use the {@link SolaceIO#read()} or {@link SolaceIO#read(TypeDescriptor, * SerializableFunction, SerializableFunction)}. * - *

    No-arg {@link SolaceIO#read()} top-level method

    + *

    No-argument {@link SolaceIO#read()} top-level method

    * *

    This method returns a PCollection of {@link Solace.Record} objects. It uses a default mapper * ({@link SolaceRecordMapper#map(BytesXMLMessage)}) to map from the received {@link @@ -78,8 +78,8 @@ *

    With this method, the user can: * *

      - *
    • specify custom output type of the PTransform (for example their own class consisting only - * of the relevant fields, optimized for their use-case), + *
    • specify a custom output type for the PTransform (for example their own class consisting + * only of the relevant fields, optimized for their use-case), and *
    • create a custom mapping between {@link BytesXMLMessage} and their output type and *
    • specify what field to use for watermark estimation from their mapped field (for example, in * this method the user can use a field which is encoded in the payload as a timestamp, which @@ -135,7 +135,7 @@ *

      The advanced {@link SolaceIO#read(TypeDescriptor, SerializableFunction, * SerializableFunction)} method

      * - *

      When using this method we can specify a custom output PCollection type and a custom timestamp + *

      When using this method you can specify a custom output PCollection type and a custom timestamp * function. * *

      {@code
      
      From df1eb6cb5a7cdf62c49515f9642d8cb47b7c18c0 Mon Sep 17 00:00:00 2001
      From: Bartosz Zablocki 
      Date: Fri, 7 Jun 2024 15:59:26 +0200
      Subject: [PATCH 32/41] Add a field to Solace.Record mapped from
       BytesXMLMessage.getAttachmentByteBuffer()
      
      ---
       .../beam/sdk/io/solace/data/Solace.java       | 26 ++++++++++++-------
       .../sdk/io/solace/data/SolaceRecordCoder.java |  2 ++
       .../beam/sdk/io/solace/data/SolaceTest.java   | 10 +++++--
       3 files changed, 27 insertions(+), 11 deletions(-)
      
      diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/data/Solace.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/data/Solace.java
      index 72571f311b29..ad3c01b72ccf 100644
      --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/data/Solace.java
      +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/data/Solace.java
      @@ -155,8 +155,12 @@ public abstract static class Record {
           @SchemaFieldNumber("11")
           public abstract @Nullable String getReplicationGroupMessageId();
       
      +    @SuppressWarnings("mutable")
      +    @SchemaFieldNumber("12")
      +    public abstract byte[] getAttachmentBytes();
      +
           public static Builder builder() {
      -      return new AutoValue_Solace_Record.Builder();
      +      return new AutoValue_Solace_Record.Builder().setAttachmentBytes(new byte[0]);
           }
       
           @AutoValue.Builder
      @@ -186,6 +190,8 @@ public abstract static class Builder {
             public abstract Builder setReplicationGroupMessageId(
                 @Nullable String replicationGroupMessageId);
       
      +      public abstract Builder setAttachmentBytes(byte[] attachmentBytes);
      +
             public abstract Record build();
           }
         }
      @@ -198,21 +204,22 @@ public static class SolaceRecordMapper {
               return null;
             }
       
      -      ByteArrayOutputStream outputStream = new ByteArrayOutputStream();
      +      ByteArrayOutputStream payloadBytesStream = new ByteArrayOutputStream();
             if (msg.getContentLength() != 0) {
               try {
      -          outputStream.write(msg.getBytes());
      +          payloadBytesStream.write(msg.getBytes());
               } catch (IOException e) {
      -          LOG.error("Could not write Bytes from the BytesXMLMessage to the Solace.record.", e);
      +          LOG.error("Could not write bytes from the BytesXMLMessage to the Solace.record.", e);
               }
             }
      +
      +      ByteArrayOutputStream attachmentBytesStream = new ByteArrayOutputStream();
             if (msg.getAttachmentContentLength() != 0) {
               try {
      -          outputStream.write(msg.getAttachmentByteBuffer().array());
      +          attachmentBytesStream.write(msg.getAttachmentByteBuffer().array());
               } catch (IOException e) {
                 LOG.error(
      -              "Could not AttachmentByteBuffer from the BytesXMLMessage to the" + " Solace.record.",
      -              e);
      +              "Could not AttachmentByteBuffer from the BytesXMLMessage to the Solace.record.", e);
               }
             }
       
      @@ -227,7 +234,7 @@ public static class SolaceRecordMapper {
               destBuilder.setType(DestinationType.QUEUE);
             } else {
               LOG.error(
      -            "SolaceIO: Unknown destination type for message {}, assuming that {} is a" + " topic",
      +            "SolaceIO: Unknown destination type for message {}, assuming that {} is a topic",
                   msg.getCorrelationId(),
                   originalDestination.getName());
               destBuilder.setType(DestinationType.TOPIC);
      @@ -248,7 +255,8 @@ public static class SolaceRecordMapper {
                     msg.getReplicationGroupMessageId() != null
                         ? msg.getReplicationGroupMessageId().toString()
                         : null)
      -          .setPayload(outputStream.toByteArray())
      +          .setPayload(payloadBytesStream.toByteArray())
      +          .setAttachmentBytes(attachmentBytesStream.toByteArray())
                 .build();
           }
         }
      diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/data/SolaceRecordCoder.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/data/SolaceRecordCoder.java
      index fcad3b77a0bc..319ec7829534 100644
      --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/data/SolaceRecordCoder.java
      +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/data/SolaceRecordCoder.java
      @@ -69,6 +69,7 @@ public void encode(Record value, @NonNull OutputStream outStream) throws IOExcep
           LONG_CODER.encode(value.getSenderTimestamp(), outStream);
           LONG_CODER.encode(value.getSequenceNumber(), outStream);
           LONG_CODER.encode(value.getTimeToLive(), outStream);
      +    BYTE_CODER.encode(value.getAttachmentBytes(), outStream);
         }
       
         @Override
      @@ -100,6 +101,7 @@ public Record decode(InputStream inStream) throws IOException {
               .setSenderTimestamp(LONG_CODER.decode(inStream))
               .setSequenceNumber(LONG_CODER.decode(inStream))
               .setTimeToLive(LONG_CODER.decode(inStream))
      +        .setAttachmentBytes(BYTE_CODER.decode(inStream))
               .build();
         }
       }
      diff --git a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/data/SolaceTest.java b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/data/SolaceTest.java
      index 47b065be98a0..2972ddc15cbd 100644
      --- a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/data/SolaceTest.java
      +++ b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/data/SolaceTest.java
      @@ -18,14 +18,12 @@
       package org.apache.beam.sdk.io.solace.data;
       
       import java.nio.charset.StandardCharsets;
      -import java.util.Map;
       import org.apache.beam.sdk.io.solace.data.Solace.Destination;
       import org.junit.Assert;
       import org.junit.Test;
       
       public class SolaceTest {
       
      -  Map properties;
         Destination destination =
             Solace.Destination.builder()
                 .setName("some destination")
      @@ -42,6 +40,8 @@ public class SolaceTest {
         Long timeToLive = 34567890L;
         String payloadString = "some payload";
         byte[] payload = payloadString.getBytes(StandardCharsets.UTF_8);
      +  String attachmentString = "some attachment";
      +  byte[] attachment = attachmentString.getBytes(StandardCharsets.UTF_8);
       
         @Test
         public void testRecordEquality() {
      @@ -58,6 +58,7 @@ public void testRecordEquality() {
                   .setSequenceNumber(sequenceNumber)
                   .setTimeToLive(timeToLive)
                   .setPayload(payload)
      +            .setAttachmentBytes(attachment)
                   .build();
       
           Solace.Record obj2 =
      @@ -73,6 +74,7 @@ public void testRecordEquality() {
                   .setSequenceNumber(sequenceNumber)
                   .setTimeToLive(timeToLive)
                   .setPayload(payload)
      +            .setAttachmentBytes(attachment)
                   .build();
       
           Solace.Record obj3 =
      @@ -88,6 +90,7 @@ public void testRecordEquality() {
                   .setSequenceNumber(sequenceNumber)
                   .setTimeToLive(timeToLive)
                   .setPayload(payload)
      +            .setAttachmentBytes(attachment)
                   .build();
       
           Assert.assertEquals(obj1, obj2);
      @@ -104,6 +107,8 @@ public void testRecordEquality() {
           Assert.assertEquals(obj1.getSequenceNumber(), sequenceNumber);
           Assert.assertEquals(obj1.getTimeToLive(), timeToLive);
           Assert.assertEquals(new String(obj1.getPayload(), StandardCharsets.UTF_8), payloadString);
      +    Assert.assertEquals(
      +        new String(obj1.getAttachmentBytes(), StandardCharsets.UTF_8), attachmentString);
         }
       
         @Test
      @@ -120,6 +125,7 @@ public void testRecordNullability() {
           Assert.assertNull(obj.getSenderTimestamp());
           Assert.assertNull(obj.getSequenceNumber());
           Assert.assertNull(obj.getTimeToLive());
      +    Assert.assertArrayEquals(obj.getAttachmentBytes(), new byte[0]);
           Assert.assertEquals(new String(obj.getPayload(), StandardCharsets.UTF_8), payloadString);
         }
       
      
      From 5015b32f14cfb546ce3267f3e0fc597ec51a13f7 Mon Sep 17 00:00:00 2001
      From: Bartosz Zablocki 
      Date: Mon, 10 Jun 2024 16:02:37 +0200
      Subject: [PATCH 33/41] Add and fix some documentation
      
      ---
       .../apache/beam/sdk/io/solace/SolaceIO.java   | 56 ++++++++++---------
       .../sdk/io/solace/broker/MessageReceiver.java | 24 +++++++-
       .../beam/sdk/io/solace/broker/SempClient.java | 18 ++++++
       .../io/solace/broker/SempClientFactory.java   | 11 ++++
       .../sdk/io/solace/broker/SessionService.java  | 20 +++++++
       .../solace/broker/SessionServiceFactory.java  | 20 +++++++
       6 files changed, 123 insertions(+), 26 deletions(-)
      
      diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/SolaceIO.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/SolaceIO.java
      index 8c4d1d7d44bd..5e5eb93ff1b3 100644
      --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/SolaceIO.java
      +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/SolaceIO.java
      @@ -51,11 +51,12 @@
       import org.slf4j.LoggerFactory;
       
       /**
      - * A {@link PTransform} to read and write from/to Solace event broker.
      + * A {@link PTransform} to read and write from/to Solace event
      + * broker.
        *
        * 

      Note: this API is beta and subject to change. * - *

      Reading

      + *

      Reading from Solace

      * * To read from Solace, use the {@link SolaceIO#read()} or {@link SolaceIO#read(TypeDescriptor, * SerializableFunction, SerializableFunction)}. @@ -115,20 +116,21 @@ * PCollection events = * pipeline.apply( * SolaceIO.read() - * .from(Queue.fromName(options.getSolaceReadQueue())) + * .from(Queue.fromName("your-queue-name")) * .withSempClientFactory( * BasicAuthSempClientFactory.builder() - * .host("http://" + options.getSolaceHost() + ":8080") - * .username(options.getSolaceUsername()) - * .password(options.getSolacePassword()) - * .vpnName(options.getSolaceVpnName()) + * .host("your-host-name-with-protocol") // e.g. "http://12.34.56.78:8080" + * .username("semp-username") + * .password("semp-password") + * .vpnName("vpn-name") * .build()) * .withSessionServiceFactory( * BasicAuthJcsmpSessionServiceFactory.builder() - * .host(options.getSolaceHost()) - * .username(options.getSolaceUsername()) - * .password(options.getSolacePassword()) - * .vpnName(options.getSolaceVpnName()) + * .host("your-host-name") + * // e.g. "12.34.56.78", or "[fe80::1]", or "12.34.56.78:4444" + * .username("username") + * .password("password") + * .vpnName("vpn-name") * .build())); * }
      * @@ -172,7 +174,7 @@ * TypeDescriptor.of(SimpleRecord.class), * record -> toSimpleRecord(record), * record -> record.timestamp) - * .from(Topic.fromName(options.getSolaceReadTopic())) + * .from(Topic.fromName("your-topic-name")) * .withSempClientFactory(...) * .withSessionServiceFactory(...); * @@ -340,8 +342,8 @@ public Read withDeduplicateRecords(boolean deduplicateRecords) { *
    * *

    An existing implementation of the SempClientFactory includes {@link - * org.apache.beam.sdk.io.solace.broker.BasicAuthSempClientFactory} which implements the Basic - * Authentication to Solace. + * org.apache.beam.sdk.io.solace.broker.BasicAuthSempClientFactory} which implements connection + * to the SEMP with the Basic Authentication method. * *

    To use it, specify the credentials with the builder methods. * @@ -350,11 +352,11 @@ public Read withDeduplicateRecords(boolean deduplicateRecords) { *

    {@code
          * .withSempClientFactory(
          *         BasicAuthSempClientFactory.builder()
    -     *                 .host("http://" + options.getSolaceHost() + ":8080")
    -     *                 .username(options.getSolaceUsername())
    -     *                 .password(options.getSolacePassword())
    -     *                 .vpnName(options.getSolaceVpnName())
    -     *                 .build())
    +     *               .host("your-host-name-with-protocol") // e.g. "http://12.34.56.78:8080"
    +     *               .username("username")
    +     *               .password("password")
    +     *               .vpnName("vpn-name")
    +     *               .build())
          * }
    */ public Read withSempClientFactory(SempClientFactory sempClientFactory) { @@ -382,15 +384,19 @@ public Read withSempClientFactory(SempClientFactory sempClientFactory) { * *

    To use it, specify the credentials with the builder methods. * * - *

    The format of the host is `[Protocol://]Host[:Port]` * + *

    The host is the IPv4 or IPv6 or host name of the appliance. IPv5 addresses must be encoded + * in brackets ([]). For example, "12.34.56.78", or "[fe80::1]". If connecting to a non-default + * port, it can be specified here using the "Host:Port" format. For example, "12.34.56.78:4444", + * or "[fe80::1]:4444". * *

    {@code
          * BasicAuthJcsmpSessionServiceFactory.builder()
    -     *         .host(options.getSolaceHost())
    -     *         .username(options.getSolaceUsername())
    -     *         .password(options.getSolacePassword())
    -     *         .vpnName(options.getSolaceVpnName())
    -     *         .build()));
    +     *     .host("your-host-name")
    +     *           // e.g. "12.34.56.78", or "[fe80::1]", or "12.34.56.78:4444"
    +     *     .username("semp-username")
    +     *     .password("semp-password")
    +     *     .vpnName("vpn-name")
    +     *     .build()));
          * }
    */ public Read withSessionServiceFactory(SessionServiceFactory sessionServiceFactory) { diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/MessageReceiver.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/MessageReceiver.java index baf9e3beb038..199a83e322bd 100644 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/MessageReceiver.java +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/MessageReceiver.java @@ -20,16 +20,38 @@ import com.solacesystems.jcsmp.BytesXMLMessage; import java.io.IOException; +/** + * Interface for receiving messages from a Solace broker. + * + *

    Implementations of this interface are responsible for managing the connection to the broker + * and for receiving messages from the broker. + */ public interface MessageReceiver { + /** + * Starts the message receiver. + * + *

    This method is called in the {@link + * org.apache.beam.sdk.io.solace.read.UnboundedSolaceReader#start()} method. + */ void start(); + /** + * Returns {@literal true} if the message receiver is closed, {@literal false} otherwise. + * + *

    A message receiver is closed when it is no longer able to receive messages. + */ boolean isClosed(); + /** + * Receives a message from the broker. + * + *

    This method will block until a message is received. + */ BytesXMLMessage receive() throws IOException; /** * Test clients may return {@literal true} to signal that all expected messages have been pulled - * and the test may complete. Real clients will return {@literal false}. + * and the test may complete. Real clients should always return {@literal false}. */ default boolean isEOF() { return false; diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SempClient.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SempClient.java index 571010316ee8..465f37c14036 100644 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SempClient.java +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SempClient.java @@ -21,11 +21,29 @@ import java.io.IOException; import java.io.Serializable; +/** + * This interface defines methods for interacting with a Solace message broker using the Solace + * Element Management Protocol (SEMP). SEMP provides a way to manage and monitor various aspects of + * the broker, including queues and topics. + */ public interface SempClient extends Serializable { + /** + * Determines if the specified queue is non-exclusive. In Solace, non-exclusive queues allow + * multiple consumers to receive messages from the queue. + */ boolean isQueueNonExclusive(String queueName) throws IOException; + /** + * This is only called when a user requests to read data from a topic. This method creates a new + * queue on the Solace broker and associates it with the specified topic. This ensures that + * messages published to the topic are delivered to the queue, allowing consumers to receive them. + */ Queue createQueueForTopic(String queueName, String topicName) throws IOException; + /** + * Retrieves the size of the backlog (in bytes) for the specified queue. The backlog represents + * the amount of data in messages that are waiting to be delivered to consumers. + */ long getBacklogBytes(String queueName) throws IOException; } diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SempClientFactory.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SempClientFactory.java index 3967ed1ab884..79f690fde175 100644 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SempClientFactory.java +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SempClientFactory.java @@ -19,6 +19,17 @@ import java.io.Serializable; +/** + * This interface serves as a blueprint for creating SempClient objects, which are used to interact + * with a Solace message broker using the Solace Element Management Protocol (SEMP). + */ public interface SempClientFactory extends Serializable { + + /** + * This method is the core of the factory interface. It defines how to construct and return a + * SempClient object. Implementations of this interface will provide the specific logic for + * creating a client instance, which might involve connecting to the broker, handling + * authentication, and configuring other settings. + */ SempClient create(); } diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SessionService.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SessionService.java index af276c2f3af6..cd368865f0c3 100644 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SessionService.java +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SessionService.java @@ -19,12 +19,32 @@ import java.io.Serializable; +/** + * The SessionService interface provides a set of methods for managing a session with the Solace + * messaging system. It allows for establishing a connection, creating a message-receiver object, + * checking if the connection is closed or not, and gracefully closing the session. + */ public interface SessionService extends Serializable { + + /** + * Establishes a connection to the service. This could involve providing connection details like + * host, port, VPN name, username, and password. + */ void connect(); + /** Gracefully closes the connection to the service. */ void close(); + /** + * Checks whether the connection to the service is currently closed. This method is called when an + * `UnboundedSolaceReader` is starting to read messages - a session will be created if this + * returns true. + */ boolean isClosed(); + /** + * Creates a MessageReceiver object for receiving messages from Solace. Typically, this object is + * created from the session instance. + */ MessageReceiver createReceiver(); } diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SessionServiceFactory.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SessionServiceFactory.java index d4f698d8299c..7d1dee7a1187 100644 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SessionServiceFactory.java +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SessionServiceFactory.java @@ -21,11 +21,31 @@ import java.io.Serializable; import org.checkerframework.checker.nullness.qual.Nullable; +/** + * This abstract class serves as a blueprint for creating `SessionService` objects. It introduces a + * queue property and mandates the implementation of a create() method in concrete subclasses. + */ public abstract class SessionServiceFactory implements Serializable { + + /** + * A reference to a Queue object. This is set when the pipline is constructed (in the {@link + * org.apache.beam.sdk.io.solace.SolaceIO.Read#expand(org.apache.beam.sdk.values.PBegin)} method). + * This could be used to associate the created SessionService with a specific queue for message + * handling. + */ @Nullable Queue queue; + /** + * This is the core method that subclasses must implement. It defines how to construct and return + * a SessionService object. + */ public abstract SessionService create(); + /** + * This method is called in the {@link + * org.apache.beam.sdk.io.solace.SolaceIO.Read#expand(org.apache.beam.sdk.values.PBegin)} method + * to set the Queue reference. + */ public void setQueue(Queue queue) { this.queue = queue; } From 2e1c10e0b4c0f124af779ee4f284fcc79ccc8fc9 Mon Sep 17 00:00:00 2001 From: Bartosz Zablocki Date: Mon, 10 Jun 2024 18:08:03 +0200 Subject: [PATCH 34/41] Remove CheckpointMark's reference to the UnboundedSolaceReader - unnecessary. --- .../sdk/io/solace/read/SolaceCheckpointMark.java | 13 +++++-------- .../sdk/io/solace/read/UnboundedSolaceReader.java | 5 +---- 2 files changed, 6 insertions(+), 12 deletions(-) diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/SolaceCheckpointMark.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/SolaceCheckpointMark.java index 10336d00d8b5..133c1c094d13 100644 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/SolaceCheckpointMark.java +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/SolaceCheckpointMark.java @@ -22,7 +22,6 @@ import java.util.ArrayList; import java.util.List; import java.util.Objects; -import java.util.concurrent.atomic.AtomicBoolean; import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.sdk.coders.DefaultCoder; import org.apache.beam.sdk.extensions.avro.coders.AvroCoder; @@ -36,7 +35,6 @@ @Internal @DefaultCoder(AvroCoder.class) public class SolaceCheckpointMark implements UnboundedSource.CheckpointMark { - private transient AtomicBoolean activeReader; // BytesXMLMessage is not serializable so if a job restarts from the checkpoint, we cannot retry // these messages here. We relay on Solace's retry mechanism. private transient ArrayDeque ackQueue; @@ -44,18 +42,17 @@ public class SolaceCheckpointMark implements UnboundedSource.CheckpointMark { @SuppressWarnings("initialization") // Avro will set the fields by breaking abstraction private SolaceCheckpointMark() {} - public SolaceCheckpointMark(AtomicBoolean activeReader, List ackQueue) { - this.activeReader = activeReader; + public SolaceCheckpointMark(List ackQueue) { this.ackQueue = new ArrayDeque<>(ackQueue); } @Override public void finalizeCheckpoint() { - if (activeReader == null || !activeReader.get() || ackQueue == null) { + if (ackQueue == null) { return; } - while (ackQueue.size() > 0) { + while (!ackQueue.isEmpty()) { BytesXMLMessage msg = ackQueue.poll(); if (msg != null) { msg.ackMessage(); @@ -79,11 +76,11 @@ public boolean equals(@Nullable Object o) { // content. ArrayList ackList = new ArrayList<>(ackQueue); ArrayList thatAckList = new ArrayList<>(that.ackQueue); - return Objects.equals(activeReader, that.activeReader) && Objects.equals(ackList, thatAckList); + return Objects.equals(ackList, thatAckList); } @Override public int hashCode() { - return Objects.hash(activeReader, ackQueue); + return Objects.hash(ackQueue); } } diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/UnboundedSolaceReader.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/UnboundedSolaceReader.java index 0155345a2323..c57b10222fd5 100644 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/UnboundedSolaceReader.java +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/UnboundedSolaceReader.java @@ -26,7 +26,6 @@ import java.util.ArrayList; import java.util.List; import java.util.NoSuchElementException; -import java.util.concurrent.atomic.AtomicBoolean; import org.apache.beam.sdk.io.UnboundedSource; import org.apache.beam.sdk.io.UnboundedSource.UnboundedReader; import org.apache.beam.sdk.io.solace.broker.MessageReceiver; @@ -51,7 +50,6 @@ class UnboundedSolaceReader extends UnboundedReader { private @Nullable T solaceMappedRecord; private @Nullable MessageReceiver messageReceiver; private @Nullable SessionService sessionService; - AtomicBoolean active = new AtomicBoolean(true); /** * Queue to place advanced messages before {@link #getCheckpointMark()} be called non-concurrent @@ -116,7 +114,6 @@ public boolean advance() { @Override public void close() { - active.set(false); checkNotNull(sessionService).close(); } @@ -138,7 +135,7 @@ public UnboundedSource.CheckpointMark getCheckpointMark() { ackQueue.add(msg); } } - return new SolaceCheckpointMark(active, ackQueue); + return new SolaceCheckpointMark(ackQueue); } @Override From 4d02b997f7c9176f8a0c71c09f9aacbc428477f7 Mon Sep 17 00:00:00 2001 From: Bartosz Zablocki Date: Tue, 11 Jun 2024 15:40:14 +0200 Subject: [PATCH 35/41] Revert "Remove CheckpointMark's reference to the UnboundedSolaceReader - unnecessary." This reverts commit 2e1c10e0b4c0f124af779ee4f284fcc79ccc8fc9. --- .../sdk/io/solace/read/SolaceCheckpointMark.java | 13 ++++++++----- .../sdk/io/solace/read/UnboundedSolaceReader.java | 5 ++++- 2 files changed, 12 insertions(+), 6 deletions(-) diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/SolaceCheckpointMark.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/SolaceCheckpointMark.java index 133c1c094d13..10336d00d8b5 100644 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/SolaceCheckpointMark.java +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/SolaceCheckpointMark.java @@ -22,6 +22,7 @@ import java.util.ArrayList; import java.util.List; import java.util.Objects; +import java.util.concurrent.atomic.AtomicBoolean; import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.sdk.coders.DefaultCoder; import org.apache.beam.sdk.extensions.avro.coders.AvroCoder; @@ -35,6 +36,7 @@ @Internal @DefaultCoder(AvroCoder.class) public class SolaceCheckpointMark implements UnboundedSource.CheckpointMark { + private transient AtomicBoolean activeReader; // BytesXMLMessage is not serializable so if a job restarts from the checkpoint, we cannot retry // these messages here. We relay on Solace's retry mechanism. private transient ArrayDeque ackQueue; @@ -42,17 +44,18 @@ public class SolaceCheckpointMark implements UnboundedSource.CheckpointMark { @SuppressWarnings("initialization") // Avro will set the fields by breaking abstraction private SolaceCheckpointMark() {} - public SolaceCheckpointMark(List ackQueue) { + public SolaceCheckpointMark(AtomicBoolean activeReader, List ackQueue) { + this.activeReader = activeReader; this.ackQueue = new ArrayDeque<>(ackQueue); } @Override public void finalizeCheckpoint() { - if (ackQueue == null) { + if (activeReader == null || !activeReader.get() || ackQueue == null) { return; } - while (!ackQueue.isEmpty()) { + while (ackQueue.size() > 0) { BytesXMLMessage msg = ackQueue.poll(); if (msg != null) { msg.ackMessage(); @@ -76,11 +79,11 @@ public boolean equals(@Nullable Object o) { // content. ArrayList ackList = new ArrayList<>(ackQueue); ArrayList thatAckList = new ArrayList<>(that.ackQueue); - return Objects.equals(ackList, thatAckList); + return Objects.equals(activeReader, that.activeReader) && Objects.equals(ackList, thatAckList); } @Override public int hashCode() { - return Objects.hash(ackQueue); + return Objects.hash(activeReader, ackQueue); } } diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/UnboundedSolaceReader.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/UnboundedSolaceReader.java index c57b10222fd5..0155345a2323 100644 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/UnboundedSolaceReader.java +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/UnboundedSolaceReader.java @@ -26,6 +26,7 @@ import java.util.ArrayList; import java.util.List; import java.util.NoSuchElementException; +import java.util.concurrent.atomic.AtomicBoolean; import org.apache.beam.sdk.io.UnboundedSource; import org.apache.beam.sdk.io.UnboundedSource.UnboundedReader; import org.apache.beam.sdk.io.solace.broker.MessageReceiver; @@ -50,6 +51,7 @@ class UnboundedSolaceReader extends UnboundedReader { private @Nullable T solaceMappedRecord; private @Nullable MessageReceiver messageReceiver; private @Nullable SessionService sessionService; + AtomicBoolean active = new AtomicBoolean(true); /** * Queue to place advanced messages before {@link #getCheckpointMark()} be called non-concurrent @@ -114,6 +116,7 @@ public boolean advance() { @Override public void close() { + active.set(false); checkNotNull(sessionService).close(); } @@ -135,7 +138,7 @@ public UnboundedSource.CheckpointMark getCheckpointMark() { ackQueue.add(msg); } } - return new SolaceCheckpointMark(ackQueue); + return new SolaceCheckpointMark(active, ackQueue); } @Override From bee8fafa2397d6cd557d6f4e755cc5f6788e67e9 Mon Sep 17 00:00:00 2001 From: Bartosz Zablocki Date: Tue, 11 Jun 2024 21:06:15 +0200 Subject: [PATCH 36/41] Solace project init - github workflow file, gradle module --- .../beam_PreCommit_Java_Solace_IO_Direct.yml | 126 ++++++++++++++++++ sdks/java/io/solace/build.gradle | 33 +++++ settings.gradle.kts | 2 + 3 files changed, 161 insertions(+) create mode 100644 .github/workflows/beam_PreCommit_Java_Solace_IO_Direct.yml create mode 100644 sdks/java/io/solace/build.gradle diff --git a/.github/workflows/beam_PreCommit_Java_Solace_IO_Direct.yml b/.github/workflows/beam_PreCommit_Java_Solace_IO_Direct.yml new file mode 100644 index 000000000000..71ca48afd7c8 --- /dev/null +++ b/.github/workflows/beam_PreCommit_Java_Solace_IO_Direct.yml @@ -0,0 +1,126 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +name: PreCommit Java Solace IO Direct + +on: + push: + tags: ['v*'] + branches: ['master', 'release-*'] + paths: + - "sdks/java/io/solace/**" + - ".github/workflows/beam_PreCommit_Java_Solace_IO_Direct.yml" + pull_request_target: + branches: ['master', 'release-*'] + paths: + - "sdks/java/io/solace/**" + - 'release/trigger_all_tests.json' + - '.github/trigger_files/beam_PreCommit_Java_Solace_IO_Direct.json' + issue_comment: + types: [created] + schedule: + - cron: '45 1/6 * * *' + workflow_dispatch: + +#Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + actions: write + pull-requests: write + checks: write + contents: read + deployments: read + id-token: none + issues: write + discussions: read + packages: read + pages: read + repository-projects: read + security-events: read + statuses: read + +# This allows a subsequently queued workflow run to interrupt previous runs +concurrency: + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + +jobs: + beam_PreCommit_Java_Solace_IO_Direct: + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + strategy: + matrix: + job_name: ["beam_PreCommit_Java_Solace_IO_Direct"] + job_phrase: ["Run Java_Solace_IO_Direct PreCommit"] + timeout-minutes: 60 + if: | + github.event_name == 'push' || + github.event_name == 'pull_request_target' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') || + github.event_name == 'workflow_dispatch' || + github.event.comment.body == 'Run Java_Solace_IO_Direct PreCommit' + runs-on: ubuntu-20.04 + steps: + - uses: actions/checkout@v4 + - name: Setup repository + uses: ./.github/actions/setup-action + with: + comment_phrase: ${{ matrix.job_phrase }} + github_token: ${{ secrets.GITHUB_TOKEN }} + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action + - name: run Solace IO build script + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:io:solace:build + arguments: | + -PdisableSpotlessCheck=true \ + -PdisableCheckStyle=true \ + - name: run Solace IO IT script + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:io:solace:integrationTest + arguments: | + -PdisableSpotlessCheck=true \ + -PdisableCheckStyle=true \ + - name: Archive JUnit Test Results + uses: actions/upload-artifact@v4 + if: ${{ !success() }} + with: + name: JUnit Test Results + path: "**/build/reports/tests/" + - name: Publish JUnit Test Results + uses: EnricoMi/publish-unit-test-result-action@v2 + if: always() + with: + commit: '${{ env.prsha || env.GITHUB_SHA }}' + comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} + files: '**/build/test-results/**/*.xml' + - name: Archive SpotBugs Results + uses: actions/upload-artifact@v4 + if: always() + with: + name: SpotBugs Results + path: '**/build/reports/spotbugs/*.html' + - name: Publish SpotBugs Results + uses: jwgmeligmeyling/spotbugs-github-action@v1.2 + if: always() + with: + name: Publish SpotBugs + path: '**/build/reports/spotbugs/*.html' \ No newline at end of file diff --git a/sdks/java/io/solace/build.gradle b/sdks/java/io/solace/build.gradle new file mode 100644 index 000000000000..c09df4245015 --- /dev/null +++ b/sdks/java/io/solace/build.gradle @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * License); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an AS IS BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +plugins { id 'org.apache.beam.module' } +applyJavaNature( + automaticModuleName: 'org.apache.beam.sdk.io.solace', + exportJavadoc: false, + publish: false, +) +provideIntegrationTestingDependencies() +enableJavaPerformanceTesting() + +description = "Apache Beam :: SDKs :: Java :: IO :: Solace" +ext.summary = """IO to read and write to Solace destinations (queues and topics).""" + +dependencies { + implementation enforcedPlatform(library.java.google_cloud_platform_libraries_bom) +} \ No newline at end of file diff --git a/settings.gradle.kts b/settings.gradle.kts index f8f3bc5c5c34..fd2f29ea001e 100644 --- a/settings.gradle.kts +++ b/settings.gradle.kts @@ -367,3 +367,5 @@ include("sdks:java:managed") findProject(":sdks:java:managed")?.name = "managed" include("sdks:java:io:iceberg") findProject(":sdks:java:io:iceberg")?.name = "iceberg" +include("sdks:java:io:solace") +findProject(":sdks:java:io:solace")?.name = "solace" From f2d284a8402e1e45e4e963e0950976760a4f0d96 Mon Sep 17 00:00:00 2001 From: Bartosz Zablocki Date: Thu, 13 Jun 2024 21:15:27 +0200 Subject: [PATCH 37/41] Splitting the #31476 - Leaving only PTransform AutoValue configurations --- sdks/java/io/solace/build.gradle | 4 - .../apache/beam/sdk/io/solace/SolaceIO.java | 322 +------- .../sdk/io/solace/broker/MessageReceiver.java | 59 -- .../beam/sdk/io/solace/broker/SempClient.java | 49 -- .../io/solace/broker/SempClientFactory.java | 11 +- .../sdk/io/solace/broker/SessionService.java | 50 -- .../solace/broker/SessionServiceFactory.java | 28 +- .../beam/sdk/io/solace/data/Solace.java | 192 ----- .../sdk/io/solace/data/SolaceRecordCoder.java | 107 --- .../io/solace/read/SolaceCheckpointMark.java | 89 --- .../io/solace/read/UnboundedSolaceReader.java | 191 ----- .../io/solace/read/UnboundedSolaceSource.java | 148 ---- .../io/solace/read/WatermarkParameters.java | 88 --- .../sdk/io/solace/read/WatermarkPolicy.java | 69 -- .../beam/sdk/io/solace/read/package-info.java | 20 - .../beam/sdk/io/solace/MockSempClient.java | 87 --- .../sdk/io/solace/MockSempClientFactory.java | 34 - .../sdk/io/solace/MockSessionService.java | 88 --- .../io/solace/MockSessionServiceFactory.java | 34 - .../beam/sdk/io/solace/SolaceDataUtils.java | 705 ------------------ .../beam/sdk/io/solace/SolaceIOTest.java | 592 --------------- .../beam/sdk/io/solace/data/SolaceTest.java | 136 ---- 22 files changed, 3 insertions(+), 3100 deletions(-) delete mode 100644 sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/MessageReceiver.java delete mode 100644 sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SempClient.java delete mode 100644 sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SessionService.java delete mode 100644 sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/data/SolaceRecordCoder.java delete mode 100644 sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/SolaceCheckpointMark.java delete mode 100644 sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/UnboundedSolaceReader.java delete mode 100644 sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/UnboundedSolaceSource.java delete mode 100644 sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/WatermarkParameters.java delete mode 100644 sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/WatermarkPolicy.java delete mode 100644 sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/package-info.java delete mode 100644 sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/MockSempClient.java delete mode 100644 sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/MockSempClientFactory.java delete mode 100644 sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/MockSessionService.java delete mode 100644 sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/MockSessionServiceFactory.java delete mode 100644 sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/SolaceDataUtils.java delete mode 100644 sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/SolaceIOTest.java delete mode 100644 sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/data/SolaceTest.java diff --git a/sdks/java/io/solace/build.gradle b/sdks/java/io/solace/build.gradle index 674f4c0fe6c2..d06b4f9317db 100644 --- a/sdks/java/io/solace/build.gradle +++ b/sdks/java/io/solace/build.gradle @@ -32,13 +32,9 @@ dependencies { implementation enforcedPlatform(library.java.google_cloud_platform_libraries_bom) implementation library.java.vendored_guava_32_1_2_jre implementation project(path: ":sdks:java:core", configuration: "shadow") - implementation library.java.slf4j_api implementation library.java.joda_time implementation library.java.solace implementation library.java.vendored_guava_32_1_2_jre - implementation project(":sdks:java:extensions:avro") - implementation library.java.avro - permitUnusedDeclared library.java.avro testImplementation library.java.junit testImplementation project(path: ":sdks:java:io:common", configuration: "testRuntimeMigration") diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/SolaceIO.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/SolaceIO.java index 5e5eb93ff1b3..2b02783eba3b 100644 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/SolaceIO.java +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/SolaceIO.java @@ -17,196 +17,26 @@ */ package org.apache.beam.sdk.io.solace; -import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkNotNull; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkState; import com.google.auto.value.AutoValue; import com.solacesystems.jcsmp.BytesXMLMessage; -import com.solacesystems.jcsmp.Destination; import com.solacesystems.jcsmp.JCSMPFactory; import com.solacesystems.jcsmp.Queue; import com.solacesystems.jcsmp.Topic; -import java.io.IOException; -import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.annotations.Internal; -import org.apache.beam.sdk.coders.CannotProvideCoderException; -import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.io.solace.broker.SempClientFactory; -import org.apache.beam.sdk.io.solace.broker.SessionService; import org.apache.beam.sdk.io.solace.broker.SessionServiceFactory; import org.apache.beam.sdk.io.solace.data.Solace; -import org.apache.beam.sdk.io.solace.data.Solace.SolaceRecordMapper; -import org.apache.beam.sdk.io.solace.data.SolaceRecordCoder; -import org.apache.beam.sdk.io.solace.read.UnboundedSolaceSource; -import org.apache.beam.sdk.schemas.NoSuchSchemaException; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TypeDescriptor; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.Instant; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -/** - * A {@link PTransform} to read and write from/to Solace event - * broker. - * - *

    Note: this API is beta and subject to change. - * - *

    Reading from Solace

    - * - * To read from Solace, use the {@link SolaceIO#read()} or {@link SolaceIO#read(TypeDescriptor, - * SerializableFunction, SerializableFunction)}. - * - *

    No-argument {@link SolaceIO#read()} top-level method

    - * - *

    This method returns a PCollection of {@link Solace.Record} objects. It uses a default mapper - * ({@link SolaceRecordMapper#map(BytesXMLMessage)}) to map from the received {@link - * BytesXMLMessage} from Solace, to the {@link Solace.Record} objects. - * - *

    By default, it also uses a {@link BytesXMLMessage#getSenderTimestamp()} for watermark - * estimation. This {@link SerializableFunction} can be overridden with {@link - * Read#withTimestampFn(SerializableFunction)} method. - * - *

    When using this method, the Coders are inferred automatically. - * - *

    Advanced {@link SolaceIO#read(TypeDescriptor, SerializableFunction, SerializableFunction)} - * top-level method

    - * - *

    With this method, the user can: - * - *

      - *
    • specify a custom output type for the PTransform (for example their own class consisting - * only of the relevant fields, optimized for their use-case), and - *
    • create a custom mapping between {@link BytesXMLMessage} and their output type and - *
    • specify what field to use for watermark estimation from their mapped field (for example, in - * this method the user can use a field which is encoded in the payload as a timestamp, which - * cannot be done with the {@link SolaceIO#read()} method. - *
    - * - *

    Reading from a queue ({@link Read#from(Solace.Queue)}} or a topic ({@link - * Read#from(Solace.Topic)})

    - * - *

    Regardless of the top-level read method choice, the user can specify whether to read from a - * Queue - {@link Read#from(Solace.Queue)}, or a Topic {@link Read#from(Solace.Topic)}. - * - *

    Note: when a user specifies to read from a Topic, the connector will create a matching Queue - * and a Subscription. The user must ensure that the SEMP API is reachable from the driver program - * and must provide credentials that have `write` permission to the SEMP Config API. The created Queue - * will be non-exclusive. The Queue will not be deleted when the pipeline is terminated. - * - *

    Note: If the user specifies to read from a Queue, the driver program - * will execute a call to the SEMP API to check if the Queue is `exclusive` or `non-exclusive`. The - * user must ensure that the SEMP API is reachable from the driver program and provide credentials - * with `read` permission to the {@link Read#withSempClientFactory(SempClientFactory)}. - * - *

    Usage example

    - * - *

    The no-arg {@link SolaceIO#read()} method

    - * - *

    The minimal example - reading from an existing Queue, using the no-arg {@link SolaceIO#read()} - * method, with all the default configuration options. - * - *

    {@code
    - * PCollection events =
    - *   pipeline.apply(
    - *     SolaceIO.read()
    - *         .from(Queue.fromName("your-queue-name"))
    - *         .withSempClientFactory(
    - *                 BasicAuthSempClientFactory.builder()
    - *                         .host("your-host-name-with-protocol") // e.g. "http://12.34.56.78:8080"
    - *                         .username("semp-username")
    - *                         .password("semp-password")
    - *                         .vpnName("vpn-name")
    - *                         .build())
    - *         .withSessionServiceFactory(
    - *                 BasicAuthJcsmpSessionServiceFactory.builder()
    - *                         .host("your-host-name")
    - *                               // e.g. "12.34.56.78", or "[fe80::1]", or "12.34.56.78:4444"
    - *                         .username("username")
    - *                         .password("password")
    - *                         .vpnName("vpn-name")
    - *                         .build()));
    - * }
    - * - *

    The advanced {@link SolaceIO#read(TypeDescriptor, SerializableFunction, - * SerializableFunction)} method

    - * - *

    When using this method you can specify a custom output PCollection type and a custom timestamp - * function. - * - *

    {@code
    - * @DefaultSchema(JavaBeanSchema.class)
    - * public static class SimpleRecord {
    - *    public String payload;
    - *    public String messageId;
    - *    public Instant timestamp;
    - *
    - *    public SimpleRecord() {}
    - *
    - *    public SimpleRecord(String payload, String messageId, Instant timestamp) {
    - *        this.payload = payload;
    - *        this.messageId = messageId;
    - *        this.timestamp = timestamp;
    - *    }
    - * }
    - *
    - * private static SimpleRecord toSimpleRecord(BytesXMLMessage record) {
    - *    if (record == null) {
    - *        return null;
    - *    }
    - *    return new SimpleRecord(
    - *            new String(record.getBytes(), StandardCharsets.UTF_8),
    - *            record.getApplicationMessageId(),
    - *            record.getSenderTimestamp() != null
    - *                    ? Instant.ofEpochMilli(record.getSenderTimestamp())
    - *                    : Instant.now());
    - * }
    - *
    - * PCollection events =
    - *  pipeline.apply(
    - *      SolaceIO.read(
    - *                      TypeDescriptor.of(SimpleRecord.class),
    - *                      record -> toSimpleRecord(record),
    - *                      record -> record.timestamp)
    - *              .from(Topic.fromName("your-topic-name"))
    - *              .withSempClientFactory(...)
    - *              .withSessionServiceFactory(...);
    - *
    - *
    - * }
    - * - *

    Authentication

    - * - *

    When reading from Solace, the user must use {@link - * Read#withSessionServiceFactory(SessionServiceFactory)} to create a JCSMP session and {@link - * Read#withSempClientFactory(SempClientFactory)} to authenticate to the SEMP API. - * - *

    See {@link Read#withSessionServiceFactory(SessionServiceFactory)} for session authentication. - * The connector provides implementation of the {@link SessionServiceFactory} using the Basic - * Authentication: {@link org.apache.beam.sdk.io.solace.broker.BasicAuthJcsmpSessionService}. - * - *

    For the authentication to the SEMP API ({@link Read#withSempClientFactory(SempClientFactory)}) - * the connector provides {@link org.apache.beam.sdk.io.solace.broker.BasicAuthSempClientFactory} to - * authenticate using the Basic Authentication. - */ -@Internal public class SolaceIO { - public static final SerializableFunction SENDER_TIMESTAMP_FUNCTION = - (record) -> { - Long senderTimestamp = record != null ? record.getSenderTimestamp() : null; - if (senderTimestamp != null) { - return Instant.ofEpochMilli(senderTimestamp); - } else { - return Instant.now(); - } - }; private static final boolean DEFAULT_DEDUPLICATE_RECORDS = false; /** Get a {@link Topic} object from the topic name. */ @@ -219,65 +49,8 @@ static Queue queueFromName(String queueName) { return JCSMPFactory.onlyInstance().createQueue(queueName); } - /** - * Convert to a JCSMP destination from a schema-enabled {@link - * org.apache.beam.sdk.io.solace.data.Solace.Destination}. - * - *

    This method returns a {@link Destination}, which may be either a {@link Topic} or a {@link - * Queue} - */ - public static Destination convertToJcsmpDestination(Solace.Destination destination) { - if (destination.getType().equals(Solace.DestinationType.TOPIC)) { - return topicFromName(checkNotNull(destination.getName())); - } else if (destination.getType().equals(Solace.DestinationType.QUEUE)) { - return queueFromName(checkNotNull(destination.getName())); - } else { - throw new IllegalArgumentException( - "SolaceIO.Write: Unknown destination type: " + destination.getType()); - } - } - - /** - * Create a {@link Read} transform, to read from Solace. The ingested records will be mapped to - * the {@link Solace.Record} objects. - */ - public static Read read() { - return Read.builder() - .setTypeDescriptor(TypeDescriptor.of(Solace.Record.class)) - .setParseFn(SolaceRecordMapper::map) - .setTimestampFn(SENDER_TIMESTAMP_FUNCTION) - .build(); - } - /** - * Create a {@link Read} transform, to read from Solace. Specify a {@link SerializableFunction} to - * map incoming {@link BytesXMLMessage} records, to the object of your choice. You also need to - * specify a {@link TypeDescriptor} for your class and the timestamp function which returns an - * {@link Instant} from the record. - * - *

    The type descriptor will be used to infer a coder from CoderRegistry or Schema Registry. You - * can initialize a new TypeDescriptor in the following manner: - * - *

    {@code
    -   * TypeDescriptor typeDescriptor = TypeDescriptor.of(YourOutputType.class);
    -   * }
    - */ - public static Read read( - TypeDescriptor typeDescriptor, - SerializableFunction<@Nullable BytesXMLMessage, @Nullable T> parseFn, - SerializableFunction timestampFn) { - checkState(typeDescriptor != null, "SolaceIO.Read: typeDescriptor must not be null"); - checkState(parseFn != null, "SolaceIO.Read: parseFn must not be null"); - checkState(timestampFn != null, "SolaceIO.Read: timestampFn must not be null"); - return Read.builder() - .setTypeDescriptor(typeDescriptor) - .setParseFn(parseFn) - .setTimestampFn(timestampFn) - .build(); - } - @AutoValue public abstract static class Read extends PTransform> { - private static final Logger LOG = LoggerFactory.getLogger(Read.class); /** Set the queue name to read from. Use this or the `from(Topic)` method. */ public Read from(Solace.Queue queue) { @@ -458,100 +231,7 @@ abstract Builder setParseFn( @Override public PCollection expand(PBegin input) { - checkState( - (getQueue() == null ^ getTopic() == null), - "SolaceIO.Read: One of the Solace {Queue, Topic} must be set."); - - SempClientFactory sempClientFactory = - checkNotNull(getSempClientFactory(), "SolaceIO: sempClientFactory is null."); - String jobName = input.getPipeline().getOptions().getJobName(); - Queue queueFromOptions = getQueue(); - Queue initializedQueue = - queueFromOptions != null - ? queueFromOptions - : initializeQueueForTopic(jobName, sempClientFactory); - - SessionServiceFactory sessionServiceFactory = - checkNotNull(getSessionServiceFactory(), "SolaceIO: sessionServiceFactory is null."); - sessionServiceFactory.setQueue(initializedQueue); - - registerDefaultCoder(input.getPipeline()); - // Infer the actual coder - Coder coder = inferCoder(input.getPipeline()); - - return input.apply( - org.apache.beam.sdk.io.Read.from( - new UnboundedSolaceSource<>( - initializedQueue, - sempClientFactory, - sessionServiceFactory, - getMaxNumConnections(), - getDeduplicateRecords(), - coder, - checkNotNull(getTimestampFn()), - checkNotNull(getParseFn())))); - } - - private static void registerDefaultCoder(Pipeline pipeline) { - pipeline - .getCoderRegistry() - .registerCoderForType(TypeDescriptor.of(Solace.Record.class), SolaceRecordCoder.of()); - } - - @VisibleForTesting - Coder inferCoder(Pipeline pipeline) { - Coder coderFromCoderRegistry = getFromCoderRegistry(pipeline); - if (coderFromCoderRegistry != null) { - return coderFromCoderRegistry; - } - - Coder coderFromSchemaRegistry = getFromSchemaRegistry(pipeline); - if (coderFromSchemaRegistry != null) { - return coderFromSchemaRegistry; - } - - throw new RuntimeException( - "SolaceIO.Read: Cannot infer a coder for the TypeDescriptor. Annotate your" - + " output class with @DefaultSchema annotation or create a coder manually" - + " and register it in the CoderRegistry."); - } - - private @Nullable Coder getFromSchemaRegistry(Pipeline pipeline) { - try { - return pipeline.getSchemaRegistry().getSchemaCoder(getTypeDescriptor()); - } catch (NoSuchSchemaException e) { - return null; - } - } - - private @Nullable Coder getFromCoderRegistry(Pipeline pipeline) { - try { - return pipeline.getCoderRegistry().getCoder(getTypeDescriptor()); - } catch (CannotProvideCoderException e) { - return null; - } - } - - Queue initializeQueueForTopic(String jobName, SempClientFactory sempClientFactory) { - Queue initializedQueue; - Queue solaceQueue = getQueue(); - if (solaceQueue != null) { - return solaceQueue; - } else { - String queueName = String.format("queue-%s-%s", getTopic(), jobName); - try { - String topicName = checkNotNull(getTopic()).getName(); - initializedQueue = sempClientFactory.create().createQueueForTopic(queueName, topicName); - LOG.info( - "SolaceIO.Read: A new queue {} was created. The Queue will not be" - + " deleted when this job finishes. Make sure to remove it yourself" - + " when not needed.", - initializedQueue.getName()); - return initializedQueue; - } catch (IOException e) { - throw new RuntimeException(e); - } - } + throw new UnsupportedOperationException(""); } } } diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/MessageReceiver.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/MessageReceiver.java deleted file mode 100644 index 199a83e322bd..000000000000 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/MessageReceiver.java +++ /dev/null @@ -1,59 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.sdk.io.solace.broker; - -import com.solacesystems.jcsmp.BytesXMLMessage; -import java.io.IOException; - -/** - * Interface for receiving messages from a Solace broker. - * - *

    Implementations of this interface are responsible for managing the connection to the broker - * and for receiving messages from the broker. - */ -public interface MessageReceiver { - /** - * Starts the message receiver. - * - *

    This method is called in the {@link - * org.apache.beam.sdk.io.solace.read.UnboundedSolaceReader#start()} method. - */ - void start(); - - /** - * Returns {@literal true} if the message receiver is closed, {@literal false} otherwise. - * - *

    A message receiver is closed when it is no longer able to receive messages. - */ - boolean isClosed(); - - /** - * Receives a message from the broker. - * - *

    This method will block until a message is received. - */ - BytesXMLMessage receive() throws IOException; - - /** - * Test clients may return {@literal true} to signal that all expected messages have been pulled - * and the test may complete. Real clients should always return {@literal false}. - */ - default boolean isEOF() { - return false; - } -} diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SempClient.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SempClient.java deleted file mode 100644 index 465f37c14036..000000000000 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SempClient.java +++ /dev/null @@ -1,49 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.sdk.io.solace.broker; - -import com.solacesystems.jcsmp.Queue; -import java.io.IOException; -import java.io.Serializable; - -/** - * This interface defines methods for interacting with a Solace message broker using the Solace - * Element Management Protocol (SEMP). SEMP provides a way to manage and monitor various aspects of - * the broker, including queues and topics. - */ -public interface SempClient extends Serializable { - - /** - * Determines if the specified queue is non-exclusive. In Solace, non-exclusive queues allow - * multiple consumers to receive messages from the queue. - */ - boolean isQueueNonExclusive(String queueName) throws IOException; - - /** - * This is only called when a user requests to read data from a topic. This method creates a new - * queue on the Solace broker and associates it with the specified topic. This ensures that - * messages published to the topic are delivered to the queue, allowing consumers to receive them. - */ - Queue createQueueForTopic(String queueName, String topicName) throws IOException; - - /** - * Retrieves the size of the backlog (in bytes) for the specified queue. The backlog represents - * the amount of data in messages that are waiting to be delivered to consumers. - */ - long getBacklogBytes(String queueName) throws IOException; -} diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SempClientFactory.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SempClientFactory.java index 79f690fde175..b5cb53e14b39 100644 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SempClientFactory.java +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SempClientFactory.java @@ -23,13 +23,4 @@ * This interface serves as a blueprint for creating SempClient objects, which are used to interact * with a Solace message broker using the Solace Element Management Protocol (SEMP). */ -public interface SempClientFactory extends Serializable { - - /** - * This method is the core of the factory interface. It defines how to construct and return a - * SempClient object. Implementations of this interface will provide the specific logic for - * creating a client instance, which might involve connecting to the broker, handling - * authentication, and configuring other settings. - */ - SempClient create(); -} +public interface SempClientFactory extends Serializable {} diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SessionService.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SessionService.java deleted file mode 100644 index cd368865f0c3..000000000000 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SessionService.java +++ /dev/null @@ -1,50 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.sdk.io.solace.broker; - -import java.io.Serializable; - -/** - * The SessionService interface provides a set of methods for managing a session with the Solace - * messaging system. It allows for establishing a connection, creating a message-receiver object, - * checking if the connection is closed or not, and gracefully closing the session. - */ -public interface SessionService extends Serializable { - - /** - * Establishes a connection to the service. This could involve providing connection details like - * host, port, VPN name, username, and password. - */ - void connect(); - - /** Gracefully closes the connection to the service. */ - void close(); - - /** - * Checks whether the connection to the service is currently closed. This method is called when an - * `UnboundedSolaceReader` is starting to read messages - a session will be created if this - * returns true. - */ - boolean isClosed(); - - /** - * Creates a MessageReceiver object for receiving messages from Solace. Typically, this object is - * created from the session instance. - */ - MessageReceiver createReceiver(); -} diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SessionServiceFactory.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SessionServiceFactory.java index 7d1dee7a1187..ab1f55ae7a9c 100644 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SessionServiceFactory.java +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/broker/SessionServiceFactory.java @@ -17,36 +17,10 @@ */ package org.apache.beam.sdk.io.solace.broker; -import com.solacesystems.jcsmp.Queue; import java.io.Serializable; -import org.checkerframework.checker.nullness.qual.Nullable; /** * This abstract class serves as a blueprint for creating `SessionService` objects. It introduces a * queue property and mandates the implementation of a create() method in concrete subclasses. */ -public abstract class SessionServiceFactory implements Serializable { - - /** - * A reference to a Queue object. This is set when the pipline is constructed (in the {@link - * org.apache.beam.sdk.io.solace.SolaceIO.Read#expand(org.apache.beam.sdk.values.PBegin)} method). - * This could be used to associate the created SessionService with a specific queue for message - * handling. - */ - @Nullable Queue queue; - - /** - * This is the core method that subclasses must implement. It defines how to construct and return - * a SessionService object. - */ - public abstract SessionService create(); - - /** - * This method is called in the {@link - * org.apache.beam.sdk.io.solace.SolaceIO.Read#expand(org.apache.beam.sdk.values.PBegin)} method - * to set the Queue reference. - */ - public void setQueue(Queue queue) { - this.queue = queue; - } -} +public abstract class SessionServiceFactory implements Serializable {} diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/data/Solace.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/data/Solace.java index ad3c01b72ccf..076a16b96ceb 100644 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/data/Solace.java +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/data/Solace.java @@ -17,17 +17,6 @@ */ package org.apache.beam.sdk.io.solace.data; -import com.google.auto.value.AutoValue; -import com.solacesystems.jcsmp.BytesXMLMessage; -import java.io.ByteArrayOutputStream; -import java.io.IOException; -import org.apache.beam.sdk.schemas.AutoValueSchema; -import org.apache.beam.sdk.schemas.annotations.DefaultSchema; -import org.apache.beam.sdk.schemas.annotations.SchemaFieldNumber; -import org.checkerframework.checker.nullness.qual.Nullable; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - /** * A record to be written to a Solace topic. * @@ -79,185 +68,4 @@ public String getName() { return name; } } - - public enum DestinationType { - TOPIC, - QUEUE - } - - @AutoValue - @DefaultSchema(AutoValueSchema.class) - public abstract static class Destination { - @SchemaFieldNumber("0") - public abstract String getName(); - - @SchemaFieldNumber("1") - public abstract DestinationType getType(); - - public static Builder builder() { - return new AutoValue_Solace_Destination.Builder(); - } - - @AutoValue.Builder - public abstract static class Builder { - public abstract Builder setName(String name); - - public abstract Builder setType(DestinationType type); - - public abstract Destination build(); - } - } - - @AutoValue - @DefaultSchema(AutoValueSchema.class) - public abstract static class Record { - @SchemaFieldNumber("0") - public abstract @Nullable String getMessageId(); - - @SuppressWarnings("mutable") - @SchemaFieldNumber("1") - public abstract byte[] getPayload(); - - @SchemaFieldNumber("2") - public abstract @Nullable Destination getDestination(); - - @SchemaFieldNumber("3") - public abstract @Nullable Long getExpiration(); - - @SchemaFieldNumber("4") - public abstract @Nullable Integer getPriority(); - - @SchemaFieldNumber("5") - public abstract @Nullable Boolean getRedelivered(); - - @SchemaFieldNumber("6") - public abstract @Nullable String getReplyTo(); - - @SchemaFieldNumber("7") - public abstract @Nullable Long getReceiveTimestamp(); - - @SchemaFieldNumber("8") - public abstract @Nullable Long getSenderTimestamp(); - - @SchemaFieldNumber("9") - public abstract @Nullable Long getSequenceNumber(); - - @SchemaFieldNumber("10") - public abstract @Nullable Long getTimeToLive(); - - /** - * The ID for a particular message is only guaranteed to be the same for a particular copy of a - * message on a particular queue or topic endpoint within a replication group. The same message - * on different queues or topic endpoints within the same replication group may or may not have - * the same replication group message ID. See more at https://docs.solace.com/API/API-Developer-Guide/Detecting-Duplicate-Mess.htm - */ - @SchemaFieldNumber("11") - public abstract @Nullable String getReplicationGroupMessageId(); - - @SuppressWarnings("mutable") - @SchemaFieldNumber("12") - public abstract byte[] getAttachmentBytes(); - - public static Builder builder() { - return new AutoValue_Solace_Record.Builder().setAttachmentBytes(new byte[0]); - } - - @AutoValue.Builder - public abstract static class Builder { - public abstract Builder setMessageId(@Nullable String messageId); - - public abstract Builder setPayload(byte[] payload); - - public abstract Builder setDestination(Destination destination); - - public abstract Builder setExpiration(@Nullable Long expiration); - - public abstract Builder setPriority(@Nullable Integer priority); - - public abstract Builder setRedelivered(@Nullable Boolean redelivered); - - public abstract Builder setReplyTo(@Nullable String replyTo); - - public abstract Builder setReceiveTimestamp(@Nullable Long receiveTimestamp); - - public abstract Builder setSenderTimestamp(@Nullable Long senderTimestamp); - - public abstract Builder setSequenceNumber(@Nullable Long sequenceNumber); - - public abstract Builder setTimeToLive(@Nullable Long timeToLive); - - public abstract Builder setReplicationGroupMessageId( - @Nullable String replicationGroupMessageId); - - public abstract Builder setAttachmentBytes(byte[] attachmentBytes); - - public abstract Record build(); - } - } - - public static class SolaceRecordMapper { - private static final Logger LOG = LoggerFactory.getLogger(SolaceRecordMapper.class); - - public static @Nullable Record map(@Nullable BytesXMLMessage msg) { - if (msg == null) { - return null; - } - - ByteArrayOutputStream payloadBytesStream = new ByteArrayOutputStream(); - if (msg.getContentLength() != 0) { - try { - payloadBytesStream.write(msg.getBytes()); - } catch (IOException e) { - LOG.error("Could not write bytes from the BytesXMLMessage to the Solace.record.", e); - } - } - - ByteArrayOutputStream attachmentBytesStream = new ByteArrayOutputStream(); - if (msg.getAttachmentContentLength() != 0) { - try { - attachmentBytesStream.write(msg.getAttachmentByteBuffer().array()); - } catch (IOException e) { - LOG.error( - "Could not AttachmentByteBuffer from the BytesXMLMessage to the Solace.record.", e); - } - } - - String replyTo = (msg.getReplyTo() != null) ? msg.getReplyTo().getName() : null; - - com.solacesystems.jcsmp.Destination originalDestination = msg.getDestination(); - Destination.Builder destBuilder = - Destination.builder().setName(originalDestination.getName()); - if (originalDestination instanceof com.solacesystems.jcsmp.Topic) { - destBuilder.setType(DestinationType.TOPIC); - } else if (originalDestination instanceof com.solacesystems.jcsmp.Queue) { - destBuilder.setType(DestinationType.QUEUE); - } else { - LOG.error( - "SolaceIO: Unknown destination type for message {}, assuming that {} is a topic", - msg.getCorrelationId(), - originalDestination.getName()); - destBuilder.setType(DestinationType.TOPIC); - } - - return Record.builder() - .setDestination(destBuilder.build()) - .setExpiration(msg.getExpiration()) - .setMessageId(msg.getApplicationMessageId()) - .setPriority(msg.getPriority()) - .setRedelivered(msg.getRedelivered()) - .setReplyTo(replyTo) - .setReceiveTimestamp(msg.getReceiveTimestamp()) - .setSenderTimestamp(msg.getSenderTimestamp()) - .setSequenceNumber(msg.getSequenceNumber()) - .setTimeToLive(msg.getTimeToLive()) - .setReplicationGroupMessageId( - msg.getReplicationGroupMessageId() != null - ? msg.getReplicationGroupMessageId().toString() - : null) - .setPayload(payloadBytesStream.toByteArray()) - .setAttachmentBytes(attachmentBytesStream.toByteArray()) - .build(); - } - } } diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/data/SolaceRecordCoder.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/data/SolaceRecordCoder.java deleted file mode 100644 index 319ec7829534..000000000000 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/data/SolaceRecordCoder.java +++ /dev/null @@ -1,107 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.sdk.io.solace.data; - -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.util.Objects; -import org.apache.beam.sdk.coders.BooleanCoder; -import org.apache.beam.sdk.coders.ByteArrayCoder; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.CustomCoder; -import org.apache.beam.sdk.coders.NullableCoder; -import org.apache.beam.sdk.coders.StringUtf8Coder; -import org.apache.beam.sdk.coders.VarIntCoder; -import org.apache.beam.sdk.coders.VarLongCoder; -import org.apache.beam.sdk.io.solace.data.Solace.Destination; -import org.apache.beam.sdk.io.solace.data.Solace.Record; -import org.checkerframework.checker.nullness.qual.NonNull; - -/** - * Custom coder for the default Solace {@link Record} - * - *

    A custom coder is required to update a Dataflow job. Using a coder generated with the - * `@DefaultSchema` annotation doesn't create an update-compatible coders. - */ -public class SolaceRecordCoder extends CustomCoder { - private static final Coder BYTE_CODER = ByteArrayCoder.of(); - - private static final NullableCoder LONG_CODER = NullableCoder.of(VarLongCoder.of()); - private static final NullableCoder INTEGER_CODER = NullableCoder.of(VarIntCoder.of()); - private static final NullableCoder STRING_CODER = NullableCoder.of(StringUtf8Coder.of()); - private static final NullableCoder BOOLEAN_CODER = NullableCoder.of(BooleanCoder.of()); - - public static SolaceRecordCoder of() { - return new SolaceRecordCoder(); - } - - @Override - public void encode(Record value, @NonNull OutputStream outStream) throws IOException { - STRING_CODER.encode(value.getMessageId(), outStream); - STRING_CODER.encode(value.getReplicationGroupMessageId(), outStream); - BYTE_CODER.encode(value.getPayload(), outStream); - Destination destination = value.getDestination(); - String destinationName = destination == null ? null : destination.getName(); - String typeName = destination == null ? null : destination.getType().toString(); - STRING_CODER.encode(destinationName, outStream); - STRING_CODER.encode(typeName, outStream); - LONG_CODER.encode(value.getExpiration(), outStream); - INTEGER_CODER.encode(value.getPriority(), outStream); - BOOLEAN_CODER.encode(value.getRedelivered(), outStream); - STRING_CODER.encode(value.getReplyTo(), outStream); - LONG_CODER.encode(value.getReceiveTimestamp(), outStream); - LONG_CODER.encode(value.getSenderTimestamp(), outStream); - LONG_CODER.encode(value.getSequenceNumber(), outStream); - LONG_CODER.encode(value.getTimeToLive(), outStream); - BYTE_CODER.encode(value.getAttachmentBytes(), outStream); - } - - @Override - public Record decode(InputStream inStream) throws IOException { - Record.Builder builder = - Record.builder() - .setMessageId(STRING_CODER.decode(inStream)) - .setReplicationGroupMessageId(STRING_CODER.decode(inStream)) - .setPayload(BYTE_CODER.decode(inStream)); - - String destinationName = STRING_CODER.decode(inStream); - String destinationType = STRING_CODER.decode(inStream); - if (destinationName != null) { - builder.setDestination( - Destination.builder() - .setName(destinationName) - .setType( - Objects.equals(destinationType, "QUEUE") - ? Solace.DestinationType.QUEUE - : Solace.DestinationType.TOPIC) - .build()); - } - return builder - .setExpiration(LONG_CODER.decode(inStream)) - .setPriority(INTEGER_CODER.decode(inStream)) - .setRedelivered(BOOLEAN_CODER.decode(inStream)) - .setReplyTo(STRING_CODER.decode(inStream)) - .setReceiveTimestamp(LONG_CODER.decode(inStream)) - .setSenderTimestamp(LONG_CODER.decode(inStream)) - .setSequenceNumber(LONG_CODER.decode(inStream)) - .setTimeToLive(LONG_CODER.decode(inStream)) - .setAttachmentBytes(BYTE_CODER.decode(inStream)) - .build(); - } -} diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/SolaceCheckpointMark.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/SolaceCheckpointMark.java deleted file mode 100644 index 10336d00d8b5..000000000000 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/SolaceCheckpointMark.java +++ /dev/null @@ -1,89 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.sdk.io.solace.read; - -import com.solacesystems.jcsmp.BytesXMLMessage; -import java.util.ArrayDeque; -import java.util.ArrayList; -import java.util.List; -import java.util.Objects; -import java.util.concurrent.atomic.AtomicBoolean; -import org.apache.beam.sdk.annotations.Internal; -import org.apache.beam.sdk.coders.DefaultCoder; -import org.apache.beam.sdk.extensions.avro.coders.AvroCoder; -import org.apache.beam.sdk.io.UnboundedSource; -import org.checkerframework.checker.nullness.qual.Nullable; - -/** - * Checkpoint for an unbounded Solace source. Consists of the Solace messages waiting to be - * acknowledged and oldest pending message timestamp. - */ -@Internal -@DefaultCoder(AvroCoder.class) -public class SolaceCheckpointMark implements UnboundedSource.CheckpointMark { - private transient AtomicBoolean activeReader; - // BytesXMLMessage is not serializable so if a job restarts from the checkpoint, we cannot retry - // these messages here. We relay on Solace's retry mechanism. - private transient ArrayDeque ackQueue; - - @SuppressWarnings("initialization") // Avro will set the fields by breaking abstraction - private SolaceCheckpointMark() {} - - public SolaceCheckpointMark(AtomicBoolean activeReader, List ackQueue) { - this.activeReader = activeReader; - this.ackQueue = new ArrayDeque<>(ackQueue); - } - - @Override - public void finalizeCheckpoint() { - if (activeReader == null || !activeReader.get() || ackQueue == null) { - return; - } - - while (ackQueue.size() > 0) { - BytesXMLMessage msg = ackQueue.poll(); - if (msg != null) { - msg.ackMessage(); - } - } - } - - @Override - public boolean equals(@Nullable Object o) { - if (o == null) { - return false; - } - if (this == o) { - return true; - } - if (!(o instanceof SolaceCheckpointMark)) { - return false; - } - SolaceCheckpointMark that = (SolaceCheckpointMark) o; - // Needed to convert to ArrayList because ArrayDeque.equals checks only for reference, not - // content. - ArrayList ackList = new ArrayList<>(ackQueue); - ArrayList thatAckList = new ArrayList<>(that.ackQueue); - return Objects.equals(activeReader, that.activeReader) && Objects.equals(ackList, thatAckList); - } - - @Override - public int hashCode() { - return Objects.hash(activeReader, ackQueue); - } -} diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/UnboundedSolaceReader.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/UnboundedSolaceReader.java deleted file mode 100644 index 0155345a2323..000000000000 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/UnboundedSolaceReader.java +++ /dev/null @@ -1,191 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.sdk.io.solace.read; - -import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkNotNull; - -import com.solacesystems.jcsmp.BytesXMLMessage; -import java.io.IOException; -import java.nio.charset.StandardCharsets; -import java.util.ArrayDeque; -import java.util.ArrayList; -import java.util.List; -import java.util.NoSuchElementException; -import java.util.concurrent.atomic.AtomicBoolean; -import org.apache.beam.sdk.io.UnboundedSource; -import org.apache.beam.sdk.io.UnboundedSource.UnboundedReader; -import org.apache.beam.sdk.io.solace.broker.MessageReceiver; -import org.apache.beam.sdk.io.solace.broker.SempClient; -import org.apache.beam.sdk.io.solace.broker.SessionService; -import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; -import org.checkerframework.checker.nullness.qual.Nullable; -import org.joda.time.Instant; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** Unbounded Reader to read messages from a Solace Router. */ -@VisibleForTesting -class UnboundedSolaceReader extends UnboundedReader { - - private static final Logger LOG = LoggerFactory.getLogger(UnboundedSolaceReader.class); - private final UnboundedSolaceSource currentSource; - private final WatermarkPolicy watermarkPolicy; - private final SempClient sempClient; - private @Nullable BytesXMLMessage solaceOriginalRecord; - private @Nullable T solaceMappedRecord; - private @Nullable MessageReceiver messageReceiver; - private @Nullable SessionService sessionService; - AtomicBoolean active = new AtomicBoolean(true); - - /** - * Queue to place advanced messages before {@link #getCheckpointMark()} be called non-concurrent - * queue, should only be accessed by the reader thread A given {@link UnboundedReader} object will - * only be accessed by a single thread at once. - */ - private final java.util.Queue elementsToCheckpoint = new ArrayDeque<>(); - - public UnboundedSolaceReader(UnboundedSolaceSource currentSource) { - this.currentSource = currentSource; - this.watermarkPolicy = WatermarkPolicy.create(currentSource.getTimestampFn()); - this.sessionService = currentSource.getSessionServiceFactory().create(); - this.sempClient = currentSource.getSempClientFactory().create(); - } - - @Override - public boolean start() { - populateSession(); - populateMessageConsumer(); - return advance(); - } - - public void populateSession() { - if (sessionService == null) { - sessionService = getCurrentSource().getSessionServiceFactory().create(); - } - if (sessionService.isClosed()) { - checkNotNull(sessionService).connect(); - } - } - - private void populateMessageConsumer() { - if (messageReceiver == null) { - messageReceiver = checkNotNull(sessionService).createReceiver(); - messageReceiver.start(); - } - MessageReceiver receiver = checkNotNull(messageReceiver); - if (receiver.isClosed()) { - receiver.start(); - } - } - - @Override - public boolean advance() { - BytesXMLMessage receivedXmlMessage; - try { - receivedXmlMessage = checkNotNull(messageReceiver).receive(); - } catch (IOException e) { - LOG.warn("SolaceIO.Read: Exception when pulling messages from the broker.", e); - return false; - } - - if (receivedXmlMessage == null) { - return false; - } - elementsToCheckpoint.add(receivedXmlMessage); - solaceOriginalRecord = receivedXmlMessage; - solaceMappedRecord = getCurrentSource().getParseFn().apply(receivedXmlMessage); - watermarkPolicy.update(solaceMappedRecord); - return true; - } - - @Override - public void close() { - active.set(false); - checkNotNull(sessionService).close(); - } - - @Override - public Instant getWatermark() { - // should be only used by a test receiver - if (checkNotNull(messageReceiver).isEOF()) { - return BoundedWindow.TIMESTAMP_MAX_VALUE; - } - return watermarkPolicy.getWatermark(); - } - - @Override - public UnboundedSource.CheckpointMark getCheckpointMark() { - List ackQueue = new ArrayList<>(); - while (!elementsToCheckpoint.isEmpty()) { - BytesXMLMessage msg = elementsToCheckpoint.poll(); - if (msg != null) { - ackQueue.add(msg); - } - } - return new SolaceCheckpointMark(active, ackQueue); - } - - @Override - public T getCurrent() throws NoSuchElementException { - if (solaceMappedRecord == null) { - throw new NoSuchElementException(); - } - return solaceMappedRecord; - } - - @Override - public byte[] getCurrentRecordId() throws NoSuchElementException { - if (solaceOriginalRecord == null) { - throw new NoSuchElementException(); - } - if (solaceOriginalRecord.getApplicationMessageId() != null) { - return checkNotNull(solaceOriginalRecord) - .getApplicationMessageId() - .getBytes(StandardCharsets.UTF_8); - } else { - return checkNotNull(solaceOriginalRecord) - .getReplicationGroupMessageId() - .toString() - .getBytes(StandardCharsets.UTF_8); - } - } - - @Override - public UnboundedSolaceSource getCurrentSource() { - return currentSource; - } - - @Override - public Instant getCurrentTimestamp() throws NoSuchElementException { - if (getCurrent() == null) { - throw new NoSuchElementException(); - } - return currentSource.getTimestampFn().apply(getCurrent()); - } - - @Override - public long getTotalBacklogBytes() { - try { - return sempClient.getBacklogBytes(currentSource.getQueue().getName()); - } catch (IOException e) { - LOG.warn("SolaceIO.Read: Could not query backlog bytes. Returning BACKLOG_UNKNOWN", e); - return BACKLOG_UNKNOWN; - } - } -} diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/UnboundedSolaceSource.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/UnboundedSolaceSource.java deleted file mode 100644 index 370159994941..000000000000 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/UnboundedSolaceSource.java +++ /dev/null @@ -1,148 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.sdk.io.solace.read; - -import com.solacesystems.jcsmp.BytesXMLMessage; -import com.solacesystems.jcsmp.Queue; -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; -import org.apache.beam.sdk.annotations.Internal; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.extensions.avro.coders.AvroCoder; -import org.apache.beam.sdk.io.UnboundedSource; -import org.apache.beam.sdk.io.solace.broker.SempClientFactory; -import org.apache.beam.sdk.io.solace.broker.SessionServiceFactory; -import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.transforms.SerializableFunction; -import org.checkerframework.checker.nullness.qual.Nullable; -import org.joda.time.Instant; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -@Internal -public class UnboundedSolaceSource extends UnboundedSource { - private static final long serialVersionUID = 42L; - private static final Logger LOG = LoggerFactory.getLogger(UnboundedSolaceSource.class); - private final Queue queue; - private final @Nullable Integer maxNumConnections; - private final Coder coder; - private final boolean enableDeduplication; - private final SempClientFactory sempClientFactory; - private final SessionServiceFactory sessionServiceFactory; - private final SerializableFunction timestampFn; - private final SerializableFunction<@Nullable BytesXMLMessage, @Nullable T> parseFn; - - public Queue getQueue() { - return queue; - } - - public SessionServiceFactory getSessionServiceFactory() { - return sessionServiceFactory; - } - - public SempClientFactory getSempClientFactory() { - return sempClientFactory; - } - - public SerializableFunction getTimestampFn() { - return timestampFn; - } - - public SerializableFunction<@Nullable BytesXMLMessage, @Nullable T> getParseFn() { - return parseFn; - } - - public UnboundedSolaceSource( - Queue queue, - SempClientFactory sempClientFactory, - SessionServiceFactory sessionServiceFactory, - @Nullable Integer maxNumConnections, - boolean enableDeduplication, - Coder coder, - SerializableFunction timestampFn, - SerializableFunction<@Nullable BytesXMLMessage, @Nullable T> parseFn) { - this.queue = queue; - this.sempClientFactory = sempClientFactory; - this.sessionServiceFactory = sessionServiceFactory; - this.maxNumConnections = maxNumConnections; - this.enableDeduplication = enableDeduplication; - this.coder = coder; - this.timestampFn = timestampFn; - this.parseFn = parseFn; - } - - @Override - public UnboundedReader createReader( - PipelineOptions options, @Nullable SolaceCheckpointMark checkpointMark) { - // it makes no sense to resume a Solace Session with the previous checkpoint - // so don't need the pass a checkpoint to new a Solace Reader - return new UnboundedSolaceReader<>(this); - } - - @Override - public List> split(int desiredNumSplits, PipelineOptions options) - throws IOException { - boolean queueNonExclusive = sempClientFactory.create().isQueueNonExclusive(queue.getName()); - if (queueNonExclusive) { - return getSolaceSources(desiredNumSplits, maxNumConnections); - } else { - LOG.warn("SolaceIO.Read: The queue {} is exclusive. Provisioning only 1 read client.", queue); - return getSolaceSources(desiredNumSplits, 1); - } - } - - private List> getSolaceSources( - int desiredNumSplits, @Nullable Integer maxNumConnections) { - List> sourceList = new ArrayList<>(); - int numSplits = - maxNumConnections != null - ? Math.min(desiredNumSplits, maxNumConnections) - : desiredNumSplits; - LOG.info("SolaceIO.Read: UnboundedSolaceSource: creating {} read connections.", numSplits); - for (int i = 0; i < numSplits; i++) { - UnboundedSolaceSource source = - new UnboundedSolaceSource<>( - queue, - sempClientFactory, - sessionServiceFactory, - maxNumConnections, - enableDeduplication, - coder, - timestampFn, - parseFn); - sourceList.add(source); - } - return sourceList; - } - - @Override - public Coder getCheckpointMarkCoder() { - return AvroCoder.of(SolaceCheckpointMark.class); - } - - @Override - public Coder getOutputCoder() { - return coder; - } - - @Override - public boolean requiresDeduping() { - return enableDeduplication; - } -} diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/WatermarkParameters.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/WatermarkParameters.java deleted file mode 100644 index dd9dc6739a45..000000000000 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/WatermarkParameters.java +++ /dev/null @@ -1,88 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.sdk.io.solace.read; - -import com.google.auto.value.AutoValue; -import java.io.Serializable; -import org.apache.beam.sdk.transforms.SerializableFunction; -import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; -import org.joda.time.Duration; -import org.joda.time.Instant; - -/** {@code WatermarkParameters} contains the parameters used for watermark computation. */ -@AutoValue -public abstract class WatermarkParameters implements Serializable { - - private static final Duration STANDARD_WATERMARK_IDLE_DURATION_THRESHOLD = - Duration.standardSeconds(30); - - abstract Instant getCurrentWatermark(); - - abstract Instant getLastSavedWatermark(); - - abstract Instant getLastUpdateTime(); - - abstract SerializableFunction getTimestampFn(); - - abstract Duration getWatermarkIdleDurationThreshold(); - - public abstract Builder toBuilder(); - - static Builder builder() { - return new AutoValue_WatermarkParameters.Builder() - .setCurrentWatermark(BoundedWindow.TIMESTAMP_MIN_VALUE) - .setLastSavedWatermark(BoundedWindow.TIMESTAMP_MIN_VALUE) - .setLastUpdateTime(Instant.now()) - .setWatermarkIdleDurationThreshold(STANDARD_WATERMARK_IDLE_DURATION_THRESHOLD); - } - - @AutoValue.Builder - abstract static class Builder { - abstract Builder setCurrentWatermark(Instant currentWatermark); - - abstract Builder setLastSavedWatermark(Instant eventTime); - - abstract Builder setLastUpdateTime(Instant now); - - abstract Builder setWatermarkIdleDurationThreshold(Duration watermarkIdleDurationThreshold); - - abstract Builder setTimestampFn(SerializableFunction timestampFn); - - abstract WatermarkParameters build(); - } - - /** - * Create an instance of {@link WatermarkParameters} with a {@code SerializableFunction} to - * extract the event time. - */ - public static WatermarkParameters create(SerializableFunction timestampFn) { - Preconditions.checkArgument(timestampFn != null, "timestampFn function is null"); - return WatermarkParameters.builder().setTimestampFn(timestampFn).build(); - } - - /** - * Specify the watermark idle duration to consider before advancing the watermark. The default - * watermark idle duration threshold is {@link #STANDARD_WATERMARK_IDLE_DURATION_THRESHOLD}. - */ - public WatermarkParameters withWatermarkIdleDurationThreshold(Duration idleDurationThreshold) { - Preconditions.checkArgument( - idleDurationThreshold != null, "watermark idle duration threshold is null"); - return toBuilder().setWatermarkIdleDurationThreshold(idleDurationThreshold).build(); - } -} diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/WatermarkPolicy.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/WatermarkPolicy.java deleted file mode 100644 index 582c20165da6..000000000000 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/WatermarkPolicy.java +++ /dev/null @@ -1,69 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.sdk.io.solace.read; - -import java.io.Serializable; -import org.apache.beam.sdk.transforms.SerializableFunction; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Ordering; -import org.checkerframework.checker.nullness.qual.Nullable; -import org.joda.time.Instant; - -public class WatermarkPolicy implements Serializable { - private WatermarkParameters watermarkParameters; - - public static WatermarkPolicy create(SerializableFunction timestampFunction) { - return new WatermarkPolicy(WatermarkParameters.create(timestampFunction)); - } - - private WatermarkPolicy(WatermarkParameters watermarkParameters) { - this.watermarkParameters = watermarkParameters; - } - - public Instant getWatermark() { - Instant now = Instant.now(); - Instant watermarkIdleThreshold = - now.minus(watermarkParameters.getWatermarkIdleDurationThreshold()); - - Instant newWatermark = - watermarkParameters.getLastUpdateTime().isBefore(watermarkIdleThreshold) - ? watermarkIdleThreshold - : watermarkParameters.getLastSavedWatermark(); - - if (newWatermark.isAfter(watermarkParameters.getCurrentWatermark())) { - watermarkParameters = - watermarkParameters.toBuilder().setCurrentWatermark(newWatermark).build(); - } - return watermarkParameters.getCurrentWatermark(); - } - - public void update(@Nullable T record) { - if (record == null) { - return; - } - watermarkParameters = - watermarkParameters - .toBuilder() - .setLastSavedWatermark( - Ordering.natural() - .max( - watermarkParameters.getLastSavedWatermark(), - watermarkParameters.getTimestampFn().apply(record))) - .setLastUpdateTime(Instant.now()) - .build(); - } -} diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/package-info.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/package-info.java deleted file mode 100644 index ce24cc1a3088..000000000000 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/read/package-info.java +++ /dev/null @@ -1,20 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -/** Solace IO connector - read connector classes. */ -package org.apache.beam.sdk.io.solace.read; diff --git a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/MockSempClient.java b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/MockSempClient.java deleted file mode 100644 index d4703237371a..000000000000 --- a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/MockSempClient.java +++ /dev/null @@ -1,87 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.sdk.io.solace; - -import com.solacesystems.jcsmp.JCSMPFactory; -import com.solacesystems.jcsmp.Queue; -import java.io.IOException; -import org.apache.beam.sdk.io.solace.broker.SempClient; -import org.apache.beam.sdk.transforms.SerializableFunction; - -public class MockSempClient implements SempClient { - - private final SerializableFunction isQueueNonExclusiveFn; - private final SerializableFunction getBacklogBytesFn; - private final SerializableFunction createQueueForTopicFn; - - private MockSempClient( - SerializableFunction isQueueNonExclusiveFn, - SerializableFunction getBacklogBytesFn, - SerializableFunction createQueueForTopicFn) { - this.isQueueNonExclusiveFn = isQueueNonExclusiveFn; - this.getBacklogBytesFn = getBacklogBytesFn; - this.createQueueForTopicFn = createQueueForTopicFn; - } - - public static Builder builder() { - return new Builder(); - } - - public static class Builder { - private SerializableFunction isQueueNonExclusiveFn = (queueName) -> true; - private SerializableFunction getBacklogBytesFn = (queueName) -> 0L; - private SerializableFunction createQueueForTopicFn = (queueName) -> 0; - - public Builder setIsQueueNonExclusiveFn( - SerializableFunction isQueueNonExclusiveFn) { - this.isQueueNonExclusiveFn = isQueueNonExclusiveFn; - return this; - } - - public Builder setGetBacklogBytesFn(SerializableFunction getBacklogBytesFn) { - this.getBacklogBytesFn = getBacklogBytesFn; - return this; - } - - public Builder setCreateQueueForTopicFn( - SerializableFunction createQueueForTopicFn) { - this.createQueueForTopicFn = createQueueForTopicFn; - return this; - } - - public MockSempClient build() { - return new MockSempClient(isQueueNonExclusiveFn, getBacklogBytesFn, createQueueForTopicFn); - } - } - - @Override - public boolean isQueueNonExclusive(String queueName) throws IOException { - return isQueueNonExclusiveFn.apply(queueName); - } - - @Override - public Queue createQueueForTopic(String queueName, String topicName) throws IOException { - createQueueForTopicFn.apply(queueName); - return JCSMPFactory.onlyInstance().createQueue(queueName); - } - - @Override - public long getBacklogBytes(String queueName) throws IOException { - return getBacklogBytesFn.apply(queueName); - } -} diff --git a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/MockSempClientFactory.java b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/MockSempClientFactory.java deleted file mode 100644 index 5d8849d5cb3c..000000000000 --- a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/MockSempClientFactory.java +++ /dev/null @@ -1,34 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.sdk.io.solace; - -import org.apache.beam.sdk.io.solace.broker.SempClient; -import org.apache.beam.sdk.io.solace.broker.SempClientFactory; - -public class MockSempClientFactory implements SempClientFactory { - SempClient sempClient; - - public MockSempClientFactory(SempClient sempClient) { - this.sempClient = sempClient; - } - - @Override - public SempClient create() { - return sempClient; - } -} diff --git a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/MockSessionService.java b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/MockSessionService.java deleted file mode 100644 index 7b14da138c64..000000000000 --- a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/MockSessionService.java +++ /dev/null @@ -1,88 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.sdk.io.solace; - -import com.solacesystems.jcsmp.BytesXMLMessage; -import java.io.IOException; -import java.io.Serializable; -import java.util.concurrent.atomic.AtomicInteger; -import org.apache.beam.sdk.io.solace.broker.MessageReceiver; -import org.apache.beam.sdk.io.solace.broker.SessionService; -import org.apache.beam.sdk.transforms.SerializableFunction; - -public class MockSessionService implements SessionService { - - private final SerializableFunction getRecordFn; - private MessageReceiver messageReceiver = null; - private final int minMessagesReceived; - - public MockSessionService( - SerializableFunction getRecordFn, int minMessagesReceived) { - this.getRecordFn = getRecordFn; - this.minMessagesReceived = minMessagesReceived; - } - - @Override - public void close() {} - - @Override - public boolean isClosed() { - return false; - } - - @Override - public MessageReceiver createReceiver() { - if (messageReceiver == null) { - messageReceiver = new MockReceiver(getRecordFn, minMessagesReceived); - } - return messageReceiver; - } - - @Override - public void connect() {} - - public static class MockReceiver implements MessageReceiver, Serializable { - private final AtomicInteger counter = new AtomicInteger(); - private final SerializableFunction getRecordFn; - private final int minMessagesReceived; - - public MockReceiver( - SerializableFunction getRecordFn, int minMessagesReceived) { - this.getRecordFn = getRecordFn; - this.minMessagesReceived = minMessagesReceived; - } - - @Override - public void start() {} - - @Override - public boolean isClosed() { - return false; - } - - @Override - public BytesXMLMessage receive() throws IOException { - return getRecordFn.apply(counter.getAndIncrement()); - } - - @Override - public boolean isEOF() { - return counter.get() >= minMessagesReceived; - } - } -} diff --git a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/MockSessionServiceFactory.java b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/MockSessionServiceFactory.java deleted file mode 100644 index 4585b8f186ce..000000000000 --- a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/MockSessionServiceFactory.java +++ /dev/null @@ -1,34 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.sdk.io.solace; - -import org.apache.beam.sdk.io.solace.broker.SessionService; -import org.apache.beam.sdk.io.solace.broker.SessionServiceFactory; - -public class MockSessionServiceFactory extends SessionServiceFactory { - SessionService sessionService; - - public MockSessionServiceFactory(MockSessionService clientService) { - this.sessionService = clientService; - } - - @Override - public SessionService create() { - return sessionService; - } -} diff --git a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/SolaceDataUtils.java b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/SolaceDataUtils.java deleted file mode 100644 index 2f61d0846f14..000000000000 --- a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/SolaceDataUtils.java +++ /dev/null @@ -1,705 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.sdk.io.solace; - -import com.solacesystems.jcsmp.BytesXMLMessage; -import com.solacesystems.jcsmp.DeliveryMode; -import com.solacesystems.jcsmp.Destination; -import com.solacesystems.jcsmp.JCSMPException; -import com.solacesystems.jcsmp.MessageType; -import com.solacesystems.jcsmp.ReplicationGroupMessageId; -import com.solacesystems.jcsmp.SDTMap; -import com.solacesystems.jcsmp.User_Cos; -import com.solacesystems.jcsmp.impl.ReplicationGroupMessageIdImpl; -import java.io.IOException; -import java.io.InputStream; -import java.nio.BufferUnderflowException; -import java.nio.ByteBuffer; -import java.nio.charset.StandardCharsets; -import java.util.Collection; -import java.util.List; -import java.util.Objects; -import org.apache.beam.sdk.io.solace.data.Solace; -import org.apache.beam.sdk.schemas.JavaBeanSchema; -import org.apache.beam.sdk.schemas.annotations.DefaultSchema; -import org.apache.beam.sdk.transforms.SerializableFunction; -import org.checkerframework.checker.nullness.qual.Nullable; - -public class SolaceDataUtils { - public static final ReplicationGroupMessageId DEFAULT_REPLICATION_GROUP_ID = - new ReplicationGroupMessageIdImpl(1L, 136L); - - @DefaultSchema(JavaBeanSchema.class) - public static class SimpleRecord { - public String payload; - public String messageId; - - public SimpleRecord() {} - - public SimpleRecord(String payload, String messageId) { - this.payload = payload; - this.messageId = messageId; - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (!(o instanceof SimpleRecord)) { - return false; - } - SimpleRecord that = (SimpleRecord) o; - return Objects.equals(payload, that.payload) && Objects.equals(messageId, that.messageId); - } - - @Override - public int hashCode() { - return Objects.hash(payload, messageId); - } - - @Override - public String toString() { - return "SimpleRecord{" - + "payload='" - + payload - + '\'' - + ", messageId='" - + messageId - + '\'' - + '}'; - } - } - - public static Solace.Record getSolaceRecord(String payload, String messageId) { - return getSolaceRecord(payload, messageId, null); - } - - public static Solace.Record getSolaceRecord( - String payload, - String messageId, - @Nullable ReplicationGroupMessageId replicationGroupMessageId) { - String replicationGroupMessageIdString = - replicationGroupMessageId != null - ? replicationGroupMessageId.toString() - : DEFAULT_REPLICATION_GROUP_ID.toString(); - return Solace.Record.builder() - .setPayload(payload.getBytes(StandardCharsets.UTF_8)) - .setMessageId(messageId) - .setDestination( - Solace.Destination.builder() - .setName("destination-topic") - .setType(Solace.DestinationType.TOPIC) - .build()) - .setExpiration(1000L) - .setPriority(0) - .setReceiveTimestamp(1708100477067L) - .setRedelivered(false) - .setReplyTo(null) - .setSequenceNumber(null) - .setTimeToLive(1000L) - .setSenderTimestamp(null) - .setReplicationGroupMessageId(replicationGroupMessageIdString) - .build(); - } - - public static BytesXMLMessage getBytesXmlMessage(String payload, String messageId) { - return getBytesXmlMessage(payload, messageId, null, null); - } - - public static BytesXMLMessage getBytesXmlMessage( - String payload, String messageId, SerializableFunction ackMessageFn) { - return getBytesXmlMessage(payload, messageId, ackMessageFn, null); - } - - public static BytesXMLMessage getBytesXmlMessage( - String payload, - String messageId, - SerializableFunction ackMessageFn, - ReplicationGroupMessageId replicationGroupMessageId) { - long receiverTimestamp = 1708100477067L; - long expiration = 1000L; - long timeToLive = 1000L; - String destination = "destination-topic"; - - ReplicationGroupMessageId useReplicationGroupId = - replicationGroupMessageId != null - ? replicationGroupMessageId - : DEFAULT_REPLICATION_GROUP_ID; - return new BytesXMLMessage() { - - @Override - public byte[] getBytes() { - return payload.getBytes(StandardCharsets.UTF_8); - } - - @Override - public int getContentLength() { - return payload.getBytes(StandardCharsets.UTF_8).length; - } - - @Override - public int readBytes(byte[] arg0) { - return 0; - } - - @Override - public int readBytes(byte[] arg0, int arg1) { - return 0; - } - - @Override - public void rewindContent() {} - - @Override - public void writeBytes(byte[] arg0) {} - - @Override - public void writeBytes(byte[] arg0, int arg1, int arg2) {} - - @Override - public void ackMessage() { - if (ackMessageFn != null) { - ackMessageFn.apply(0); - } - } - - @Override - public void clearAttachment() {} - - @Override - public void clearBinaryMetadataBytes(int arg0) {} - - @Override - public void clearContent() {} - - @Override - public void clearQueueNameLocation() {} - - @Override - public void clearTopicNameLocation() {} - - @Override - public String dump() { - return null; - } - - @Override - public String dump(int arg0) { - return null; - } - - @Override - public long getAckMessageId() { - return 0; - } - - @Override - public String getAppMessageID() { - return null; - } - - @Override - public String getAppMessageType() { - return null; - } - - @Override - public String getApplicationMessageId() { - return messageId; - } - - @Override - public String getApplicationMessageType() { - return null; - } - - @Override - public ByteBuffer getAttachmentByteBuffer() { - return null; - } - - @Override - public int getAttachmentContentLength() { - return 0; - } - - @Override - public int getBinaryMetadataContentLength(int arg0) { - return 0; - } - - @Override - public Collection getBinaryMetadataTypes() { - return null; - } - - @Override - public Long getCacheRequestId() { - return null; - } - - @Override - public List getConsumerIdList() { - return null; - } - - @Override - public String getCorrelationId() { - return null; - } - - @Override - public Object getCorrelationKey() { - return null; - } - - @Override - public User_Cos getCos() { - return null; - } - - @Override - public boolean getDeliverToOne() { - return false; - } - - @Override - public int getDeliveryCount() throws UnsupportedOperationException { - return 0; - } - - @Override - public DeliveryMode getDeliveryMode() { - return null; - } - - @Override - public Destination getDestination() { - return SolaceIO.topicFromName(destination); - } - - @Override - public String getDestinationTopicSuffix() { - return null; - } - - @Override - public boolean getDiscardIndication() { - return false; - } - - @Override - public long getExpiration() { - return expiration; - } - - @Override - public String getHTTPContentEncoding() { - return null; - } - - @Override - public String getHTTPContentType() { - return null; - } - - @Override - public String getMessageId() { - return null; - } - - @Override - public long getMessageIdLong() { - return 0; - } - - @Override - public MessageType getMessageType() { - return null; - } - - @Override - public int getPriority() { - return 0; - } - - @Override - public SDTMap getProperties() { - return null; - } - - @Override - public int getQueueNameLength() { - return 0; - } - - @Override - public int getQueueNameOffset() { - return 0; - } - - @Override - public long getReceiveTimestamp() { - return receiverTimestamp; - } - - @Override - public boolean getRedelivered() { - return false; - } - - @Override - public ReplicationGroupMessageId getReplicationGroupMessageId() { - // this is always set by Solace - return useReplicationGroupId; - } - - @Override - public Destination getReplyTo() { - return null; - } - - @Override - public String getReplyToSuffix() { - return null; - } - - @Override - public Long getSendTimestamp() { - return null; - } - - @Override - public String getSenderID() { - return null; - } - - @Override - public String getSenderId() { - return null; - } - - @Override - public Long getSenderTimestamp() { - return null; - } - - @Override - public Long getSequenceNumber() { - return null; - } - - @Override - public byte getStructuredMsgType() { - return 0x2; - } - - @Override - public boolean getTQDiscardIndication() { - return false; - } - - @Override - public long getTimeToLive() { - return timeToLive; - } - - @Override - public int getTopicNameLength() { - return 5; - } - - @Override - public int getTopicNameOffset() { - return 0; - } - - @Override - public Long getTopicSequenceNumber() { - return null; - } - - @Override - public byte[] getUserData() { - return null; - } - - @Override - public boolean hasAttachment() { - return false; - } - - @Override - public boolean hasBinaryMetadata(int arg0) { - return false; - } - - @Override - public boolean hasContent() { - return false; - } - - @Override - public boolean hasUserData() { - return false; - } - - @Override - public boolean isAckImmediately() { - return false; - } - - @Override - public boolean isCacheMessage() { - return false; - } - - @Override - public boolean isDMQEligible() { - return false; - } - - @Override - public boolean isDeliveryCountSupported() { - return false; - } - - @Override - public boolean isElidingEligible() { - return false; - } - - @Override - public boolean isReadOnly() { - return false; - } - - @Override - public boolean isReplyMessage() { - return false; - } - - @Override - public boolean isStructuredMsg() { - return false; - } - - @Override - public boolean isSuspect() { - return false; - } - - @Override - public int readAttachmentBytes(byte[] arg0) { - return 0; - } - - @Override - public int readAttachmentBytes(byte[] arg0, int arg1) { - return 0; - } - - @Override - public int readAttachmentBytes(int arg0, byte[] arg1, int arg2, int arg3) { - return 0; - } - - @Override - public int readBinaryMetadataBytes(int arg0, byte[] arg1) { - return 0; - } - - @Override - public int readContentBytes(byte[] arg0) { - return 0; - } - - @Override - public int readContentBytes(byte[] arg0, int arg1) { - return 0; - } - - @Override - public int readContentBytes(int arg0, byte[] arg1, int arg2, int arg3) { - return 0; - } - - @Override - public void rejectMessage() {} - - @Override - public void reset() {} - - @Override - public void resetPayload() {} - - @Override - public void rewindAttachment() {} - - @Override - public void setAckImmediately(boolean arg0) {} - - @Override - public void setAppMessageID(String arg0) {} - - @Override - public void setAppMessageType(String arg0) {} - - @Override - public void setApplicationMessageId(String arg0) {} - - @Override - public void setApplicationMessageType(String arg0) {} - - @Override - public void setAsReplyMessage(boolean arg0) {} - - @Override - public void setCorrelationId(String arg0) {} - - @Override - public void setCorrelationKey(Object arg0) {} - - @Override - public void setCos(User_Cos arg0) {} - - @Override - public void setDMQEligible(boolean arg0) {} - - @Override - public void setDeliverToOne(boolean arg0) {} - - @Override - public void setDeliveryMode(DeliveryMode arg0) {} - - @Override - public void setElidingEligible(boolean arg0) {} - - @Override - public void setExpiration(long arg0) {} - - @Override - public void setHTTPContentEncoding(String arg0) {} - - @Override - public void setHTTPContentType(String arg0) {} - - @Override - public void setMessageType(MessageType arg0) {} - - @Override - public void setPriority(int arg0) {} - - @Override - public void setProperties(SDTMap arg0) {} - - @Override - public void setQueueNameLocation(int arg0, int arg1) {} - - @Override - public void setReadOnly() {} - - @Override - public void setReplyTo(Destination arg0) {} - - @Override - public void setReplyToSuffix(String arg0) {} - - @Override - public void setSendTimestamp(long arg0) {} - - @Override - public void setSenderID(String arg0) {} - - @Override - public void setSenderId(String arg0) {} - - @Override - public void setSenderTimestamp(long arg0) {} - - @Override - public void setSequenceNumber(long arg0) {} - - @Override - public void setStructuredMsg(boolean arg0) {} - - @Override - public void setStructuredMsgType(byte arg0) {} - - @Override - public void setTimeToLive(long arg0) {} - - @Override - public void setTopicNameLocation(int arg0, int arg1) {} - - @Override - public void setUserData(byte[] arg0) {} - - @Override - public void settle(Outcome arg0) throws JCSMPException {} - - @Override - public int writeAttachment(byte[] arg0) { - return 0; - } - - @Override - public int writeAttachment(InputStream arg0) throws IOException { - return 0; - } - - @Override - public int writeAttachment(byte[] arg0, int arg1, int arg2) throws BufferUnderflowException { - return 0; - } - - @Override - public int writeBinaryMetadataBytes(int arg0, byte[] arg1) { - return 0; - } - - @Override - public int writeBinaryMetadataBytes(int arg0, byte[] arg1, int arg2, int arg3) - throws BufferUnderflowException { - return 0; - } - - @Override - public int writeNewAttachment(byte[] arg0) { - return 0; - } - - @Override - public int writeNewAttachment(InputStream arg0) throws IOException { - return 0; - } - - @Override - public int writeNewAttachment(byte[] arg0, int arg1, int arg2) - throws BufferUnderflowException { - return 0; - } - - @Override - public int writeNewAttachment(InputStream arg0, int arg1, int arg2) throws IOException { - return 0; - } - }; - } -} diff --git a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/SolaceIOTest.java b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/SolaceIOTest.java deleted file mode 100644 index 010bffe31b26..000000000000 --- a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/SolaceIOTest.java +++ /dev/null @@ -1,592 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.sdk.io.solace; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertThrows; -import static org.junit.Assert.assertTrue; - -import com.solacesystems.jcsmp.BytesXMLMessage; -import com.solacesystems.jcsmp.Destination; -import com.solacesystems.jcsmp.Queue; -import com.solacesystems.jcsmp.Topic; -import com.solacesystems.jcsmp.impl.ReplicationGroupMessageIdImpl; -import java.io.IOException; -import java.nio.charset.StandardCharsets; -import java.util.ArrayList; -import java.util.List; -import java.util.concurrent.atomic.AtomicInteger; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.io.UnboundedSource.CheckpointMark; -import org.apache.beam.sdk.io.UnboundedSource.UnboundedReader; -import org.apache.beam.sdk.io.solace.SolaceDataUtils.SimpleRecord; -import org.apache.beam.sdk.io.solace.SolaceIO.Read; -import org.apache.beam.sdk.io.solace.broker.SessionServiceFactory; -import org.apache.beam.sdk.io.solace.data.Solace; -import org.apache.beam.sdk.io.solace.data.Solace.Record; -import org.apache.beam.sdk.io.solace.read.SolaceCheckpointMark; -import org.apache.beam.sdk.io.solace.read.UnboundedSolaceSource; -import org.apache.beam.sdk.options.PipelineOptionsFactory; -import org.apache.beam.sdk.testing.CoderProperties; -import org.apache.beam.sdk.testing.PAssert; -import org.apache.beam.sdk.testing.TestPipeline; -import org.apache.beam.sdk.transforms.MapElements; -import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.sdk.values.TypeDescriptor; -import org.apache.beam.sdk.values.TypeDescriptors; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; -import org.joda.time.Instant; -import org.junit.Rule; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; - -@RunWith(JUnit4.class) -public class SolaceIOTest { - - @Rule public final transient TestPipeline pipeline = TestPipeline.create(); - - private static Read getDefaultQueueRead(SessionServiceFactory fakeSessionServiceFactory) { - return getDefaultRead(fakeSessionServiceFactory).from(Solace.Queue.fromName("queue")); - } - - private static Read getDefaultRead(SessionServiceFactory fakeSessionServiceFactory) { - return SolaceIO.read() - .withSempClientFactory(getMockSempClientFactory()) - .withSessionServiceFactory(fakeSessionServiceFactory) - .withMaxNumConnections(1); - } - - private static BytesXMLMessage getOrNull(Integer index, List messages) { - return index != null && index < messages.size() ? messages.get(index) : null; - } - - private static MockSempClientFactory getMockSempClientFactory() { - return new MockSempClientFactory(MockSempClient.builder().build()); - } - - private static UnboundedSolaceSource getSource( - Read spec, Queue queue, TestPipeline pipeline) { - return new UnboundedSolaceSource<>( - queue, - spec.getSempClientFactory(), - spec.getSessionServiceFactory(), - spec.getMaxNumConnections(), - spec.getDeduplicateRecords(), - spec.inferCoder(pipeline), - spec.getTimestampFn(), - spec.getParseFn()); - } - - @Test - public void testReadMessages() { - // Broker that creates input data - MockSessionService mockClientService = - new MockSessionService( - index -> { - List messages = - ImmutableList.of( - SolaceDataUtils.getBytesXmlMessage("payload_test0", "450"), - SolaceDataUtils.getBytesXmlMessage("payload_test1", "451"), - SolaceDataUtils.getBytesXmlMessage("payload_test2", "452")); - return getOrNull(index, messages); - }, - 3); - - SessionServiceFactory fakeSessionServiceFactory = - new MockSessionServiceFactory(mockClientService); - - // Expected data - List expected = new ArrayList<>(); - expected.add(SolaceDataUtils.getSolaceRecord("payload_test0", "450")); - expected.add(SolaceDataUtils.getSolaceRecord("payload_test1", "451")); - expected.add(SolaceDataUtils.getSolaceRecord("payload_test2", "452")); - - // Run the pipeline - PCollection events = - pipeline.apply("Read from Solace", getDefaultQueueRead(fakeSessionServiceFactory)); - - // Assert results - PAssert.that(events).containsInAnyOrder(expected); - pipeline.run(); - } - - @Test - public void testReadMessagesWithDeduplication() { - // Broker that creates input data - MockSessionService mockClientService = - new MockSessionService( - index -> { - List messages = - ImmutableList.of( - SolaceDataUtils.getBytesXmlMessage("payload_test0", "450"), - SolaceDataUtils.getBytesXmlMessage("payload_test1", "451"), - SolaceDataUtils.getBytesXmlMessage("payload_test2", "451")); - return getOrNull(index, messages); - }, - 3); - - SessionServiceFactory fakeSessionServiceFactory = - new MockSessionServiceFactory(mockClientService); - - // Expected data - List expected = new ArrayList<>(); - expected.add(SolaceDataUtils.getSolaceRecord("payload_test0", "450")); - expected.add(SolaceDataUtils.getSolaceRecord("payload_test1", "451")); - - // Run the pipeline - PCollection events = - pipeline.apply( - "Read from Solace", - getDefaultQueueRead(fakeSessionServiceFactory).withDeduplicateRecords(true)); - // Assert results - PAssert.that(events).containsInAnyOrder(expected); - pipeline.run(); - } - - @Test - public void testReadMessagesWithoutDeduplication() { - // Broker that creates input data - MockSessionService mockClientService = - new MockSessionService( - index -> { - List messages = - ImmutableList.of( - SolaceDataUtils.getBytesXmlMessage("payload_test0", "450"), - SolaceDataUtils.getBytesXmlMessage("payload_test1", "451"), - SolaceDataUtils.getBytesXmlMessage("payload_test2", "451")); - return getOrNull(index, messages); - }, - 3); - SessionServiceFactory fakeSessionServiceFactory = - new MockSessionServiceFactory(mockClientService); - - // Expected data - List expected = new ArrayList<>(); - expected.add(SolaceDataUtils.getSolaceRecord("payload_test0", "450")); - expected.add(SolaceDataUtils.getSolaceRecord("payload_test1", "451")); - expected.add(SolaceDataUtils.getSolaceRecord("payload_test2", "451")); - - // Run the pipeline - PCollection events = - pipeline.apply("Read from Solace", getDefaultQueueRead(fakeSessionServiceFactory)); - - // Assert results - PAssert.that(events).containsInAnyOrder(expected); - pipeline.run(); - } - - @Test - public void testReadMessagesWithDeduplicationOnReplicationGroupMessageId() { - // Broker that creates input data - MockSessionService mockClientService = - new MockSessionService( - index -> { - List messages = - ImmutableList.of( - SolaceDataUtils.getBytesXmlMessage( - "payload_test0", null, null, new ReplicationGroupMessageIdImpl(2L, 1L)), - SolaceDataUtils.getBytesXmlMessage( - "payload_test1", null, null, new ReplicationGroupMessageIdImpl(2L, 2L)), - SolaceDataUtils.getBytesXmlMessage( - "payload_test2", null, null, new ReplicationGroupMessageIdImpl(2L, 2L))); - return getOrNull(index, messages); - }, - 3); - - SessionServiceFactory fakeSessionServiceFactory = - new MockSessionServiceFactory(mockClientService); - - // Expected data - List expected = new ArrayList<>(); - expected.add( - SolaceDataUtils.getSolaceRecord( - "payload_test0", null, new ReplicationGroupMessageIdImpl(2L, 1L))); - expected.add( - SolaceDataUtils.getSolaceRecord( - "payload_test1", null, new ReplicationGroupMessageIdImpl(2L, 2L))); - - // Run the pipeline - PCollection events = - pipeline.apply( - "Read from Solace", - getDefaultQueueRead(fakeSessionServiceFactory).withDeduplicateRecords(true)); - // Assert results - PAssert.that(events).containsInAnyOrder(expected); - pipeline.run(); - } - - @Test - public void testReadWithCoderAndParseFnAndTimestampFn() { - // Broker that creates input data - MockSessionService mockClientService = - new MockSessionService( - index -> { - List messages = - ImmutableList.of( - SolaceDataUtils.getBytesXmlMessage("payload_test0", "450"), - SolaceDataUtils.getBytesXmlMessage("payload_test1", "451"), - SolaceDataUtils.getBytesXmlMessage("payload_test2", "452")); - return getOrNull(index, messages); - }, - 3); - SessionServiceFactory fakeSessionServiceFactory = - new MockSessionServiceFactory(mockClientService); - - // Expected data - List expected = new ArrayList<>(); - expected.add(new SimpleRecord("payload_test0", "450")); - expected.add(new SimpleRecord("payload_test1", "451")); - expected.add(new SimpleRecord("payload_test2", "452")); - - // Run the pipeline - PCollection events = - pipeline.apply( - "Read from Solace", - SolaceIO.read( - TypeDescriptor.of(SimpleRecord.class), - input -> - new SimpleRecord( - new String(input.getBytes(), StandardCharsets.UTF_8), - input.getApplicationMessageId()), - input -> Instant.ofEpochMilli(1708100477061L)) - .from(Solace.Queue.fromName("queue")) - .withSempClientFactory(getMockSempClientFactory()) - .withSessionServiceFactory(fakeSessionServiceFactory) - .withMaxNumConnections(1)); - - // Assert results - PAssert.that(events).containsInAnyOrder(expected); - pipeline.run(); - } - - @Test - public void testNoQueueAndTopicSet() { - Read spec = SolaceIO.read(); - assertThrows(IllegalStateException.class, () -> spec.expand(pipeline.begin())); - } - - @Test - public void testSplitsForExclusiveQueue() throws Exception { - MockSempClient mockSempClient = - MockSempClient.builder().setIsQueueNonExclusiveFn((q) -> false).build(); - - Read spec = - SolaceIO.read() - .from(Solace.Queue.fromName("queue")) - .withSempClientFactory(new MockSempClientFactory(mockSempClient)); - - int desiredNumSplits = 5; - - UnboundedSolaceSource initialSource = getSource(spec, spec.getQueue(), pipeline); - List> splits = - initialSource.split(desiredNumSplits, PipelineOptionsFactory.create()); - assertEquals(1, splits.size()); - } - - @Test - public void testSplitsForNonExclusiveQueueWithMaxNumConnections() throws Exception { - Read spec = - SolaceIO.read() - .from(Solace.Queue.fromName("queue")) - .withSempClientFactory(getMockSempClientFactory()) - .withMaxNumConnections(3); - - int desiredNumSplits = 5; - - UnboundedSolaceSource initialSource = getSource(spec, spec.getQueue(), pipeline); - List> splits = - initialSource.split(desiredNumSplits, PipelineOptionsFactory.create()); - assertEquals(3, splits.size()); - } - - @Test - public void testSplitsForNonExclusiveQueueWithMaxNumConnectionsRespectDesired() throws Exception { - Read spec = - SolaceIO.read() - .from(Solace.Queue.fromName("queue")) - .withSempClientFactory(getMockSempClientFactory()) - .withMaxNumConnections(10); - - int desiredNumSplits = 5; - - UnboundedSolaceSource initialSource = getSource(spec, spec.getQueue(), pipeline); - List> splits = - initialSource.split(desiredNumSplits, PipelineOptionsFactory.create()); - assertEquals(5, splits.size()); - } - - @Test - public void testCreateQueueForTopic() throws Exception { - AtomicInteger createQueueForTopicFnCounter = new AtomicInteger(0); - MockSempClient mockSempClient = - MockSempClient.builder() - .setCreateQueueForTopicFn((q) -> createQueueForTopicFnCounter.incrementAndGet()) - .build(); - - Read spec = - SolaceIO.read() - .from(Solace.Topic.fromName("topic")) - .withSempClientFactory(new MockSempClientFactory(mockSempClient)); - - UnboundedSolaceSource initialSource = - getSource( - spec, spec.initializeQueueForTopic("some-job", spec.getSempClientFactory()), pipeline); - initialSource.split(2, PipelineOptionsFactory.create()); - - // check if createQueueForTopic was executed - assertEquals(1, createQueueForTopicFnCounter.get()); - } - - @Test - public void testCheckpointMark() throws Exception { - AtomicInteger countConsumedMessages = new AtomicInteger(0); - AtomicInteger countAckMessages = new AtomicInteger(0); - - // Broker that creates input data - MockSessionService mockClientService = - new MockSessionService( - index -> { - List messages = new ArrayList<>(); - for (int i = 0; i < 10; i++) { - messages.add( - SolaceDataUtils.getBytesXmlMessage( - "payload_test" + i, "45" + i, (num) -> countAckMessages.incrementAndGet())); - } - countConsumedMessages.incrementAndGet(); - return getOrNull(index, messages); - }, - 10); - - SessionServiceFactory fakeSessionServiceFactory = - new MockSessionServiceFactory(mockClientService); - Read spec = getDefaultQueueRead(fakeSessionServiceFactory); - - UnboundedSolaceSource initialSource = getSource(spec, spec.getQueue(), pipeline); - UnboundedReader reader = - initialSource.createReader(PipelineOptionsFactory.create(), null); - - // start the reader and move to the first record - assertTrue(reader.start()); - - // consume 3 messages (NB: start already consumed the first message) - for (int i = 0; i < 3; i++) { - assertTrue(String.format("Failed at %d-th message", i), reader.advance()); - } - - // check if 4 messages were consumed - assertEquals(4, countConsumedMessages.get()); - - // check if no messages were acknowledged yet - assertEquals(0, countAckMessages.get()); - - // finalize the checkpoint - reader.getCheckpointMark().finalizeCheckpoint(); - - // check if messages were acknowledged - assertEquals(4, countAckMessages.get()); - } - - @Test - public void testCheckpointMarkAndFinalizeSeparately() throws Exception { - AtomicInteger countConsumedMessages = new AtomicInteger(0); - AtomicInteger countAckMessages = new AtomicInteger(0); - - // Broker that creates input data - MockSessionService mockClientService = - new MockSessionService( - index -> { - List messages = new ArrayList<>(); - for (int i = 0; i < 10; i++) { - messages.add( - SolaceDataUtils.getBytesXmlMessage( - "payload_test" + i, "45" + i, (num) -> countAckMessages.incrementAndGet())); - } - countConsumedMessages.incrementAndGet(); - return getOrNull(index, messages); - }, - 10); - SessionServiceFactory fakeSessionServiceFactory = - new MockSessionServiceFactory(mockClientService); - - Read spec = getDefaultQueueRead(fakeSessionServiceFactory).withMaxNumConnections(4); - - UnboundedSolaceSource initialSource = getSource(spec, spec.getQueue(), pipeline); - - UnboundedReader reader = - initialSource.createReader(PipelineOptionsFactory.create(), null); - - // start the reader and move to the first record - assertTrue(reader.start()); - - // consume 3 messages (NB: start already consumed the first message) - for (int i = 0; i < 3; i++) { - assertTrue(String.format("Failed at %d-th message", i), reader.advance()); - } - - // create checkpoint but don't finalize yet - CheckpointMark checkpointMark = reader.getCheckpointMark(); - - // consume 2 more messages - reader.advance(); - reader.advance(); - - // check if messages are still not acknowledged - assertEquals(0, countAckMessages.get()); - - // acknowledge from the first checkpoint - checkpointMark.finalizeCheckpoint(); - - // only messages from the first checkpoint are acknowledged - assertEquals(4, countAckMessages.get()); - } - - @Test - public void testCheckpointMarkSafety() throws Exception { - - final int messagesToProcess = 100; - - AtomicInteger countConsumedMessages = new AtomicInteger(0); - AtomicInteger countAckMessages = new AtomicInteger(0); - - // Broker that creates input data - MockSessionService mockClientService = - new MockSessionService( - index -> { - List messages = new ArrayList<>(); - for (int i = 0; i < messagesToProcess; i++) { - messages.add( - SolaceDataUtils.getBytesXmlMessage( - "payload_test" + i, "45" + i, (num) -> countAckMessages.incrementAndGet())); - } - countConsumedMessages.incrementAndGet(); - return getOrNull(index, messages); - }, - 10); - - SessionServiceFactory fakeSessionServiceFactory = - new MockSessionServiceFactory(mockClientService); - Read spec = getDefaultQueueRead(fakeSessionServiceFactory).withMaxNumConnections(4); - - UnboundedSolaceSource initialSource = getSource(spec, spec.getQueue(), pipeline); - - UnboundedReader reader = - initialSource.createReader(PipelineOptionsFactory.create(), null); - - // start the reader and move to the first record - assertTrue(reader.start()); - - // consume half the messages (NB: start already consumed the first message) - for (int i = 0; i < (messagesToProcess / 2) - 1; i++) { - assertTrue(reader.advance()); - } - - // the messages are still pending in the queue (no ACK yet) - assertEquals(0, countAckMessages.get()); - - // we finalize the checkpoint for the already-processed messages while simultaneously - // consuming the remainder of messages from the queue - Thread runner = - new Thread( - () -> { - try { - for (int i = 0; i < messagesToProcess / 2; i++) { - assertTrue(reader.advance()); - } - } catch (IOException ex) { - throw new RuntimeException(ex); - } - }); - runner.start(); - reader.getCheckpointMark().finalizeCheckpoint(); - - // Concurrency issues would cause an exception to be thrown before this method exits, - // failing the test - runner.join(); - } - - @Test - public void testDefaultCoder() { - Coder coder = - new UnboundedSolaceSource<>(null, null, null, 0, false, null, null, null) - .getCheckpointMarkCoder(); - CoderProperties.coderSerializable(coder); - } - - @Test - public void testDestinationTopicQueueCreation() { - String topicName = "some-topic"; - String queueName = "some-queue"; - Topic topic = SolaceIO.topicFromName(topicName); - Queue queue = SolaceIO.queueFromName(queueName); - - Destination dest = topic; - assertTrue(dest instanceof Topic); - assertFalse(dest instanceof Queue); - assertEquals(topicName, dest.getName()); - - dest = queue; - assertTrue(dest instanceof Queue); - assertFalse(dest instanceof Topic); - assertEquals(queueName, dest.getName()); - - Record r = SolaceDataUtils.getSolaceRecord("payload_test0", "450"); - dest = SolaceIO.convertToJcsmpDestination(r.getDestination()); - assertTrue(dest instanceof Topic); - assertFalse(dest instanceof Queue); - } - - @Test - public void testTopicEncoding() { - MockSessionService mockClientService = - new MockSessionService( - index -> { - List messages = - ImmutableList.of( - SolaceDataUtils.getBytesXmlMessage("payload_test0", "450"), - SolaceDataUtils.getBytesXmlMessage("payload_test1", "451"), - SolaceDataUtils.getBytesXmlMessage("payload_test2", "452")); - return getOrNull(index, messages); - }, - 3); - - SessionServiceFactory fakeSessionServiceFactory = - new MockSessionServiceFactory(mockClientService); - - // Run - PCollection events = - pipeline.apply("Read from Solace", getDefaultQueueRead(fakeSessionServiceFactory)); - - // Run the pipeline - PCollection destAreTopics = - events.apply( - MapElements.into(TypeDescriptors.booleans()) - .via( - r -> { - Destination dest = SolaceIO.convertToJcsmpDestination(r.getDestination()); - return dest instanceof Topic; - })); - - List expected = ImmutableList.of(true, true, true); - - // Assert results - PAssert.that(destAreTopics).containsInAnyOrder(expected); - pipeline.run(); - } -} diff --git a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/data/SolaceTest.java b/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/data/SolaceTest.java deleted file mode 100644 index 2972ddc15cbd..000000000000 --- a/sdks/java/io/solace/src/test/java/org/apache/beam/sdk/io/solace/data/SolaceTest.java +++ /dev/null @@ -1,136 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.sdk.io.solace.data; - -import java.nio.charset.StandardCharsets; -import org.apache.beam.sdk.io.solace.data.Solace.Destination; -import org.junit.Assert; -import org.junit.Test; - -public class SolaceTest { - - Destination destination = - Solace.Destination.builder() - .setName("some destination") - .setType(Solace.DestinationType.TOPIC) - .build(); - String messageId = "some message id"; - Long expiration = 123L; - Integer priority = 7; - Boolean redelivered = true; - String replyTo = "no-one"; - Long receiveTimestamp = 123456789L; - Long senderTimestamp = 987654321L; - Long sequenceNumber = 27L; - Long timeToLive = 34567890L; - String payloadString = "some payload"; - byte[] payload = payloadString.getBytes(StandardCharsets.UTF_8); - String attachmentString = "some attachment"; - byte[] attachment = attachmentString.getBytes(StandardCharsets.UTF_8); - - @Test - public void testRecordEquality() { - Solace.Record obj1 = - Solace.Record.builder() - .setDestination(destination) - .setMessageId(messageId) - .setExpiration(expiration) - .setPriority(priority) - .setRedelivered(redelivered) - .setReplyTo(replyTo) - .setReceiveTimestamp(receiveTimestamp) - .setSenderTimestamp(senderTimestamp) - .setSequenceNumber(sequenceNumber) - .setTimeToLive(timeToLive) - .setPayload(payload) - .setAttachmentBytes(attachment) - .build(); - - Solace.Record obj2 = - Solace.Record.builder() - .setDestination(destination) - .setMessageId(messageId) - .setExpiration(expiration) - .setPriority(priority) - .setRedelivered(redelivered) - .setReplyTo(replyTo) - .setReceiveTimestamp(receiveTimestamp) - .setSenderTimestamp(senderTimestamp) - .setSequenceNumber(sequenceNumber) - .setTimeToLive(timeToLive) - .setPayload(payload) - .setAttachmentBytes(attachment) - .build(); - - Solace.Record obj3 = - Solace.Record.builder() - .setDestination(destination) - .setMessageId(messageId) - .setExpiration(expiration) - .setPriority(priority) - .setRedelivered(!redelivered) - .setReplyTo(replyTo) - .setReceiveTimestamp(receiveTimestamp) - .setSenderTimestamp(senderTimestamp) - .setSequenceNumber(sequenceNumber) - .setTimeToLive(timeToLive) - .setPayload(payload) - .setAttachmentBytes(attachment) - .build(); - - Assert.assertEquals(obj1, obj2); - Assert.assertNotEquals(obj1, obj3); - Assert.assertEquals(obj1.hashCode(), obj2.hashCode()); - Assert.assertEquals(obj1.getDestination(), destination); - Assert.assertEquals(obj1.getMessageId(), messageId); - Assert.assertEquals(obj1.getExpiration(), expiration); - Assert.assertEquals(obj1.getPriority(), priority); - Assert.assertEquals(obj1.getRedelivered(), redelivered); - Assert.assertEquals(obj1.getReplyTo(), replyTo); - Assert.assertEquals(obj1.getReceiveTimestamp(), receiveTimestamp); - Assert.assertEquals(obj1.getSenderTimestamp(), senderTimestamp); - Assert.assertEquals(obj1.getSequenceNumber(), sequenceNumber); - Assert.assertEquals(obj1.getTimeToLive(), timeToLive); - Assert.assertEquals(new String(obj1.getPayload(), StandardCharsets.UTF_8), payloadString); - Assert.assertEquals( - new String(obj1.getAttachmentBytes(), StandardCharsets.UTF_8), attachmentString); - } - - @Test - public void testRecordNullability() { - Solace.Record obj = Solace.Record.builder().setMessageId(messageId).setPayload(payload).build(); - Assert.assertNotNull(obj); - Assert.assertNull(obj.getDestination()); - Assert.assertEquals(obj.getMessageId(), messageId); - Assert.assertNull(obj.getExpiration()); - Assert.assertNull(obj.getPriority()); - Assert.assertNull(obj.getRedelivered()); - Assert.assertNull(obj.getReplyTo()); - Assert.assertNull(obj.getReceiveTimestamp()); - Assert.assertNull(obj.getSenderTimestamp()); - Assert.assertNull(obj.getSequenceNumber()); - Assert.assertNull(obj.getTimeToLive()); - Assert.assertArrayEquals(obj.getAttachmentBytes(), new byte[0]); - Assert.assertEquals(new String(obj.getPayload(), StandardCharsets.UTF_8), payloadString); - } - - @Test(expected = IllegalStateException.class) - public void testRecordBuilder() { - Solace.Record.builder().build(); - } -} From 9af318585b23a3c6db2e383cafc05e27394a5461 Mon Sep 17 00:00:00 2001 From: Bartosz Zablocki Date: Thu, 13 Jun 2024 21:21:19 +0200 Subject: [PATCH 38/41] remove unnecessary dependencies --- sdks/java/io/solace/build.gradle | 6 ------ 1 file changed, 6 deletions(-) diff --git a/sdks/java/io/solace/build.gradle b/sdks/java/io/solace/build.gradle index d06b4f9317db..21e1d3ff2cda 100644 --- a/sdks/java/io/solace/build.gradle +++ b/sdks/java/io/solace/build.gradle @@ -35,10 +35,4 @@ dependencies { implementation library.java.joda_time implementation library.java.solace implementation library.java.vendored_guava_32_1_2_jre - - testImplementation library.java.junit - testImplementation project(path: ":sdks:java:io:common", configuration: "testRuntimeMigration") - testImplementation project(path: ":sdks:java:testing:test-utils", configuration: "testRuntimeMigration") - testRuntimeOnly library.java.slf4j_jdk14 - testRuntimeOnly project(path: ":runners:direct-java", configuration: "shadow") } From 60d854d5f7d70df9533d11744108396731f699f0 Mon Sep 17 00:00:00 2001 From: Bartosz Zablocki Date: Thu, 13 Jun 2024 21:24:09 +0200 Subject: [PATCH 39/41] remove info from CHANGES.md --- CHANGES.md | 2 -- 1 file changed, 2 deletions(-) diff --git a/CHANGES.md b/CHANGES.md index 50d56692b0f5..a93e7089e7b7 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -96,8 +96,6 @@ ## I/Os -* Support for X source added (Java/Python) ([#X](https://github.com/apache/beam/issues/X)). -* Support for reading from Solace message broker added (Java) ([#31440](https://github.com/apache/beam/issues/31440)). * Ensure that BigtableIO closes the reader streams ([#31477](https://github.com/apache/beam/issues/31477)). ## New Features / Improvements From 4f654279cfd65ad9fabdc7c88a33b85ae20adc65 Mon Sep 17 00:00:00 2001 From: Yi Hu Date: Thu, 13 Jun 2024 16:45:32 -0400 Subject: [PATCH 40/41] Update sdks/java/io/solace/build.gradle --- sdks/java/io/solace/build.gradle | 1 - 1 file changed, 1 deletion(-) diff --git a/sdks/java/io/solace/build.gradle b/sdks/java/io/solace/build.gradle index 21e1d3ff2cda..c49b79f96a3d 100644 --- a/sdks/java/io/solace/build.gradle +++ b/sdks/java/io/solace/build.gradle @@ -34,5 +34,4 @@ dependencies { implementation project(path: ":sdks:java:core", configuration: "shadow") implementation library.java.joda_time implementation library.java.solace - implementation library.java.vendored_guava_32_1_2_jre } From c662360922efd078806ed5ca160ffbaf64428e18 Mon Sep 17 00:00:00 2001 From: Yi Hu Date: Thu, 13 Jun 2024 16:45:47 -0400 Subject: [PATCH 41/41] Update sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/SolaceIO.java --- .../src/main/java/org/apache/beam/sdk/io/solace/SolaceIO.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/SolaceIO.java b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/SolaceIO.java index 2b02783eba3b..ca8cd615ac68 100644 --- a/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/SolaceIO.java +++ b/sdks/java/io/solace/src/main/java/org/apache/beam/sdk/io/solace/SolaceIO.java @@ -157,7 +157,7 @@ public Read withSempClientFactory(SempClientFactory sempClientFactory) { * *

    To use it, specify the credentials with the builder methods. * * - *

    The host is the IPv4 or IPv6 or host name of the appliance. IPv5 addresses must be encoded + *

    The host is the IPv4 or IPv6 or host name of the appliance. IPv6 addresses must be encoded * in brackets ([]). For example, "12.34.56.78", or "[fe80::1]". If connecting to a non-default * port, it can be specified here using the "Host:Port" format. For example, "12.34.56.78:4444", * or "[fe80::1]:4444".