[ 
https://issues.apache.org/jira/browse/BEAM-11997?focusedWorklogId=667874&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-667874
 ]

ASF GitHub Bot logged work on BEAM-11997:
-----------------------------------------

                Author: ASF GitHub Bot
            Created on: 20/Oct/21 16:52
            Start Date: 20/Oct/21 16:52
    Worklog Time Spent: 10m 
      Work Description: lukecwik commented on a change in pull request #15549:
URL: https://github.com/apache/beam/pull/15549#discussion_r732970567



##########
File path: 
sdks/java/io/redis/src/main/java/org/apache/beam/sdk/io/redis/RedisIO.java
##########
@@ -334,94 +336,85 @@ public void setup() {
     public void teardown() {
       jedis.close();
     }
-  }
-
-  private static class ReadKeysWithPattern extends BaseReadFn<String> {
 
-    ReadKeysWithPattern(RedisConnectionConfiguration connectionConfiguration) {
-      super(connectionConfiguration);
+    @GetInitialRestriction
+    public OffsetRange getInitialRestriction(@Element String pattern) {
+      return new OffsetRange(0, getKeyPatters(pattern).size());
     }
 
     @ProcessElement
-    public void processElement(ProcessContext c) {
-      ScanParams scanParams = new ScanParams();
-      scanParams.match(c.element());
-
-      String cursor = ScanParams.SCAN_POINTER_START;
-      boolean finished = false;
-      while (!finished) {
-        ScanResult<String> scanResult = jedis.scan(cursor, scanParams);
-        List<String> keys = scanResult.getResult();
-        for (String k : keys) {
-          c.output(k);
+    public void processElement(ProcessContext c, 
RestrictionTracker<OffsetRange, Long> tracker) {

Review comment:
       Note that if you can't get the cursor comparison logic working from 
https://stackoverflow.com/a/59569053, you can always implement a self 
checkpointing loop:
   ```
   processElement(...) {
     cursor = restriction.getFrom()
     values, nextCursor = redis.scan(cursor)
     if (!tryClaim(nextCursor)) {
       return STOP
     }
     output values
     if (nextCursor == END) {
        return STOP
     }
     return RESUME
   }
   ```
   
   This will help address the output limits for runners but isn't as good as 
the previous solution since the SDF is choosing when to stop instead of when 
the runner is asking the SDF to checkpoint.




-- 
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]


Issue Time Tracking
-------------------

    Worklog Id:     (was: 667874)
    Time Spent: 2.5h  (was: 2h 20m)

> Implement RedisIO on top of Splittable DoFn
> -------------------------------------------
>
>                 Key: BEAM-11997
>                 URL: https://issues.apache.org/jira/browse/BEAM-11997
>             Project: Beam
>          Issue Type: Improvement
>          Components: io-java-redis
>            Reporter: Boyuan Zhang
>            Assignee: Miguel Anzo
>            Priority: P2
>          Time Spent: 2.5h
>  Remaining Estimate: 0h
>




--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to