Skip to content
/ beam Public
forked from apache/beam

Commit

Permalink
This is a follow-up PR to apache#31953, and part of the issue apache#…
Browse files Browse the repository at this point in the history
…31905.

This PR adds the actual writer functionality, and some additional
testing, including integration testing.

This should be final PR for the SolaceIO write connector to be
complete.
  • Loading branch information
iht committed Aug 27, 2024
1 parent 24255ac commit e5d67fd
Show file tree
Hide file tree
Showing 27 changed files with 2,155 additions and 293 deletions.
1 change: 1 addition & 0 deletions CHANGES.md
Original file line number Diff line number Diff line change
Expand Up @@ -63,6 +63,7 @@
## I/Os

* Support for X source added (Java/Python) ([#X](https://github.com/apache/beam/issues/X)).
* Support for writing to [Solace messages queues](https://solace.com/) (`SolaceIO.Write`) added (Java) ([#31905](https://github.com/apache/beam/issues/31905)).

## New Features / Improvements

Expand Down
1 change: 1 addition & 0 deletions sdks/java/io/solace/build.gradle
Original file line number Diff line number Diff line change
Expand Up @@ -38,6 +38,7 @@ dependencies {
implementation library.java.proto_google_cloud_secret_manager_v1
implementation library.java.protobuf_java
implementation library.java.vendored_guava_32_1_2_jre
implementation library.java.vendored_grpc_1_60_1
implementation project(":sdks:java:extensions:avro")
implementation library.java.avro
permitUnusedDeclared library.java.avro
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -41,15 +41,28 @@
import org.apache.beam.sdk.io.solace.data.Solace.SolaceRecordMapper;
import org.apache.beam.sdk.io.solace.read.UnboundedSolaceSource;
import org.apache.beam.sdk.io.solace.write.SolaceOutput;
import org.apache.beam.sdk.io.solace.write.UnboundedBatchedSolaceWriter;
import org.apache.beam.sdk.io.solace.write.UnboundedSolaceWriter;
import org.apache.beam.sdk.io.solace.write.UnboundedStreamingSolaceWriter;
import org.apache.beam.sdk.options.PipelineOptions;
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.Duration;
import org.joda.time.Instant;
Expand Down Expand Up @@ -805,7 +818,8 @@ private Queue initializeQueueForTopicIfNeeded(

public enum SubmissionMode {
HIGHER_THROUGHPUT,
LOWER_LATENCY
LOWER_LATENCY,
TESTING // Send acks 1 by 1, this will be very slow, never use this in an actual pipeline!
}

public enum WriterType {
Expand All @@ -816,6 +830,8 @@ public enum WriterType {
@AutoValue
public abstract static class Write<T> extends PTransform<PCollection<T>, SolaceOutput> {

private static final Logger LOG = LoggerFactory.getLogger(Write.class);

public static final TupleTag<Solace.PublishResult> FAILED_PUBLISH_TAG =
new TupleTag<Solace.PublishResult>() {};
public static final TupleTag<Solace.PublishResult> SUCCESSFUL_PUBLISH_TAG =
Expand Down Expand Up @@ -961,6 +977,21 @@ public Write<T> withWriterType(WriterType writerType) {
return toBuilder().setWriterType(writerType).build();
}

/**
* Set the format function for your custom data type, and/or for dynamic destinations.
*
* <p>If you are using a custom data class, this function should return a {@link Solace.Record}
* corresponding to your custom data class instance.
*
* <p>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.
*
* <p>In any other case, this format function is optional.
*/
public Write<T> withFormatFunction(SerializableFunction<T, Solace.Record> formatFunction) {
return toBuilder().setFormatFunction(formatFunction).build();
}

/**
* Set the provider used to obtain the properties to initialize a new session in the broker.
*
Expand Down Expand Up @@ -1026,8 +1057,180 @@ abstract static class Builder<T> {

@Override
public SolaceOutput expand(PCollection<T> input) {
// TODO: will be sent in upcoming PR
return SolaceOutput.in(input.getPipeline(), null, null);
Class<? super T> pcollClass = checkNotNull(input.getTypeDescriptor()).getRawType();
boolean usingSolaceRecord =
pcollClass
.getTypeName()
.equals("org.apache.beam.sdk.io.solace.data.AutoValue_Solace_Record")
|| pcollClass.isAssignableFrom(Solace.Record.class);

validateWriteTransform(usingSolaceRecord);

boolean usingDynamicDestinations = getDestination() == null;
SerializableFunction<Solace.Record, Destination> destinationFn;
if (usingDynamicDestinations) {
destinationFn = x -> SolaceIO.convertToJcsmpDestination(checkNotNull(x.getDestination()));
} else {
// Constant destination for all messages (same topic or queue)
// This should not be non-null, as nulls would have been flagged by the
// validateWriteTransform method
destinationFn = x -> checkNotNull(getDestination());
}

@SuppressWarnings("unchecked")
PCollection<Solace.Record> records =
getFormatFunction() == null
? (PCollection<Solace.Record>) input
: input.apply(
"Format records",
MapElements.into(TypeDescriptor.of(Solace.Record.class))
.via(checkNotNull(getFormatFunction())));

// Store the current window used by the input
PCollection<Solace.PublishResult> captureWindow =
records.apply(
"Capture window", ParDo.of(new UnboundedSolaceWriter.RecordToPublishResultDoFn()));

@SuppressWarnings("unchecked")
WindowingStrategy<Solace.PublishResult, BoundedWindow> windowingStrategy =
(WindowingStrategy<Solace.PublishResult, BoundedWindow>)
captureWindow.getWindowingStrategy();

PCollection<Solace.Record> withGlobalWindow =
records.apply("Global window", Window.into(new GlobalWindows()));

PCollection<KV<Integer, Solace.Record>> withShardKeys =
withGlobalWindow.apply(
"Add shard key",
ParDo.of(new UnboundedSolaceWriter.AddShardKeyDoFn(getMaxNumOfUsedWorkers())));

String label =
getWriterType() == WriterType.STREAMING ? "Publish (streaming)" : "Publish (batched)";

PCollectionTuple solaceOutput = withShardKeys.apply(label, getWriterTransform(destinationFn));

SolaceOutput output;
if (getDeliveryMode() == DeliveryMode.PERSISTENT) {
PCollection<Solace.PublishResult> failedPublish = solaceOutput.get(FAILED_PUBLISH_TAG);
PCollection<Solace.PublishResult> successfulPublish =
solaceOutput.get(SUCCESSFUL_PUBLISH_TAG);
output =
rewindow(
SolaceOutput.in(input.getPipeline(), failedPublish, successfulPublish),
windowingStrategy);
} else {
LOG.info(
String.format(
"Solace.Write: omitting writer output because delivery mode is %s",
getDeliveryMode()));
output = SolaceOutput.in(input.getPipeline(), null, null);
}

return output;
}

private ParDo.MultiOutput<KV<Integer, Solace.Record>, Solace.PublishResult> getWriterTransform(
SerializableFunction<Solace.Record, Destination> destinationFn) {

ParDo.SingleOutput<KV<Integer, Solace.Record>, Solace.PublishResult> writer =
ParDo.of(
getWriterType() == WriterType.STREAMING
? new UnboundedStreamingSolaceWriter.WriterDoFn(
destinationFn,
checkNotNull(getSessionServiceFactory()),
getDeliveryMode(),
getDispatchMode(),
getNumberOfClientsPerWorker(),
getPublishLatencyMetrics())
: new UnboundedBatchedSolaceWriter.WriterDoFn(
destinationFn,
checkNotNull(getSessionServiceFactory()),
getDeliveryMode(),
getDispatchMode(),
getNumberOfClientsPerWorker(),
getPublishLatencyMetrics()));

return writer.withOutputTags(FAILED_PUBLISH_TAG, TupleTagList.of(SUCCESSFUL_PUBLISH_TAG));
}

private SolaceOutput rewindow(
SolaceOutput solacePublishResult,
WindowingStrategy<Solace.PublishResult, BoundedWindow> strategy) {
PCollection<Solace.PublishResult> correct = solacePublishResult.getSuccessfulPublish();
PCollection<Solace.PublishResult> failed = solacePublishResult.getFailedPublish();

PCollection<Solace.PublishResult> correctWithWindow = null;
PCollection<Solace.PublishResult> failedWithWindow = null;

if (correct != null) {
correctWithWindow = applyOriginalWindow(correct, strategy, "Rewindow correct");
}

if (failed != null) {
failedWithWindow = applyOriginalWindow(failed, strategy, "Rewindow failed");
}

return SolaceOutput.in(
solacePublishResult.getPipeline(), failedWithWindow, correctWithWindow);
}

private static PCollection<Solace.PublishResult> applyOriginalWindow(
PCollection<Solace.PublishResult> pcoll,
WindowingStrategy<Solace.PublishResult, BoundedWindow> strategy,
String label) {
Window<Solace.PublishResult> 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<Solace.PublishResult> captureWindowDetails(
WindowingStrategy<Solace.PublishResult, BoundedWindow> strategy) {
return Window.<Solace.PublishResult>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 validateWriteTransform(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(
getSessionServiceFactory() != null,
"SolaceIO: You need to pass a session service factory. For basic"
+ " authentication, you can use BasicAuthJcsmpSessionServiceFactory.");
}
}
}
Loading

0 comments on commit e5d67fd

Please sign in to comment.