mosche commented on a change in pull request #15858:
URL: https://github.com/apache/beam/pull/15858#discussion_r768686614
##########
File path:
sdks/java/io/redis/src/main/java/org/apache/beam/sdk/io/redis/RedisIO.java
##########
@@ -709,4 +720,146 @@ public void teardown() {
}
}
}
+
+ /**
+ * A {@link PTransform} to write stream key pairs
(https://redis.io/topics/streams-intro) to a
+ * Redis server.
+ */
+ @AutoValue
+ public abstract static class WriteStreams
+ extends PTransform<PCollection<KV<String, Map<String, String>>>, PDone> {
+
+ abstract @Nullable RedisConnectionConfiguration connectionConfiguration();
+
+ abstract @Nullable Long maxLen();
+
+ abstract boolean approximateTrim();
+
+ abstract Builder toBuilder();
+
+ @AutoValue.Builder
+ abstract static class Builder {
+
+ abstract Builder setConnectionConfiguration(
+ RedisConnectionConfiguration connectionConfiguration);
+
+ abstract Builder setMaxLen(Long maxLen);
+
+ abstract Builder setApproximateTrim(boolean approximateTrim);
+
+ abstract WriteStreams build();
+ }
+
+ public WriteStreams withEndpoint(String host, int port) {
+ checkArgument(host != null, "host can not be null");
+ checkArgument(port > 0, "port can not be negative or 0");
+ return toBuilder()
+
.setConnectionConfiguration(connectionConfiguration().withHost(host).withPort(port))
+ .build();
+ }
+
+ public WriteStreams withAuth(String auth) {
+ checkArgument(auth != null, "auth can not be null");
+ return toBuilder()
+ .setConnectionConfiguration(connectionConfiguration().withAuth(auth))
+ .build();
+ }
+
+ public WriteStreams withTimeout(int timeout) {
+ checkArgument(timeout >= 0, "timeout can not be negative");
+ return toBuilder()
+
.setConnectionConfiguration(connectionConfiguration().withTimeout(timeout))
+ .build();
+ }
+
+ public WriteStreams
withConnectionConfiguration(RedisConnectionConfiguration connection) {
+ checkArgument(connection != null, "connection can not be null");
+ return toBuilder().setConnectionConfiguration(connection).build();
+ }
+
+ public WriteStreams withMaxLen(Long maxLen) {
+ checkArgument(maxLen >= 0L, "maxLen must be positive if set");
+ return toBuilder().setMaxLen(maxLen).build();
+ }
+
+ public WriteStreams withApproximateTrim(boolean approximateTrim) {
+ return toBuilder().setApproximateTrim(approximateTrim).build();
+ }
+
+ @Override
+ public PDone expand(PCollection<KV<String, Map<String, String>>> input) {
+ checkArgument(connectionConfiguration() != null,
"withConnectionConfiguration() is required");
+
+ input.apply(ParDo.of(new WriteStreamFn(this)));
+ return PDone.in(input.getPipeline());
+ }
+
+ private static class WriteStreamFn extends DoFn<KV<String, Map<String,
String>>, Void> {
+
+ private static final int DEFAULT_BATCH_SIZE = 1000;
+
+ private final WriteStreams spec;
+
+ private transient Jedis jedis;
+ private transient Pipeline pipeline;
+
+ private int batchCount;
+
+ public WriteStreamFn(WriteStreams spec) {
+ this.spec = spec;
+ }
+
+ @Setup
+ public void setup() {
+ jedis = spec.connectionConfiguration().connect();
+ }
+
+ @StartBundle
+ public void startBundle() {
+ pipeline = jedis.pipelined();
+ pipeline.multi();
+ batchCount = 0;
+ }
+
+ @ProcessElement
+ public void processElement(ProcessContext c) {
+ KV<String, Map<String, String>> record = c.element();
+
+ writeRecord(record);
+
+ batchCount++;
+
+ if (batchCount >= DEFAULT_BATCH_SIZE) {
+ pipeline.exec();
+ pipeline.sync();
+ pipeline.multi();
+ batchCount = 0;
+ }
+ }
+
+ private void writeRecord(KV<String, Map<String, String>> record) {
+ String key = record.getKey();
+ Map<String, String> value = record.getValue();
+ if (spec.maxLen() > 0L) {
+ pipeline.xadd(key, StreamEntryID.NEW_ENTRY, value, spec.maxLen(),
spec.approximateTrim());
+ } else {
+ pipeline.xadd(key, StreamEntryID.NEW_ENTRY, value);
+ }
+ }
+
+ @FinishBundle
+ public void finishBundle() {
+ if (pipeline.isInMulti()) {
+ pipeline.exec();
Review comment:
I created a ticket to follow up on above. @n-oden if you are interested
in contributing more, you are more than welcome :)
https://issues.apache.org/jira/browse/BEAM-13458
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]