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

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

                Author: ASF GitHub Bot
            Created on: 14/Jun/21 17:26
            Start Date: 14/Jun/21 17:26
    Worklog Time Spent: 10m 
      Work Description: MiguelAnzoWizeline commented on a change in pull 
request #14811:
URL: https://github.com/apache/beam/pull/14811#discussion_r651139190



##########
File path: 
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/BatchSpannerRead.java
##########
@@ -73,18 +74,16 @@ public static BatchSpannerRead create(
         .apply(
             "Generate Partitions",
             ParDo.of(new GeneratePartitionsFn(getSpannerConfig(), 
txView)).withSideInputs(txView))
-        .apply("Shuffle partitions", Reshuffle.<Partition>viaRandomKey())
         .apply(
             "Read from Partitions",
             ParDo.of(new ReadFromPartitionFn(getSpannerConfig(), 
txView)).withSideInputs(txView));
   }
 
   @VisibleForTesting
-  static class GeneratePartitionsFn extends DoFn<ReadOperation, Partition> {
+  static class GeneratePartitionsFn extends DoFn<ReadOperation, 
List<Partition>> {

Review comment:
       Hello @boyuanzz I have been working in trying to get this part done but 
I have encountered a problem with the approach. I have changed the code to a 
single DoFn to remove `GeneratePartitionsFn` as you mentioned, now my DoFn 
looks like this `private static class ReadFromPartitionFn extends 
DoFn<ReadOperation, Struct>`. But I'm having problems getting 
`@GetInitialRestriction `and `@SplitRestriction` right, in order to know the 
size of the Partition List we are going to process I need to use the side input 
`c.sideInput(txView);` to process the ReadOperation, is there a way to get the 
SideInput in `@GetInitialRestriction` or `@SplitRestriction` to get the List 
Size and the amount of partitions we are going to process? or the approach you 
were suggesting was something different? Thanks in advance.




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

For queries about this service, please contact Infrastructure at:
[email protected]


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

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

> Implement SpannerIO on top of Splittable DoFn
> ---------------------------------------------
>
>                 Key: BEAM-11996
>                 URL: https://issues.apache.org/jira/browse/BEAM-11996
>             Project: Beam
>          Issue Type: Improvement
>          Components: io-java-gcp
>            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