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

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

                Author: ASF GitHub Bot
            Created on: 23/Jan/20 22:57
            Start Date: 23/Jan/20 22:57
    Worklog Time Spent: 10m 
      Work Description: iemejia commented on issue #10546: [BEAM-9008] Add 
CassandraIO readAll method
URL: https://github.com/apache/beam/pull/10546#issuecomment-577916801
 
 
   Sorry again for taking a bit to respond, my review queue ended up being 
longer than usual this week.
   
   I agree on your intuition on letting the component idiomatic, I just took a 
look at SpannerIO and it is indeed aligned with my first idea `ReadOperation` 
== `Query`, (I call it query following the naming of `HbaseQuery`). Naming is 
less important than the fact that this object ‘should have’ only parameters 
related to the Read/Query and not to the connections. So I agreed 100% on that.
   
   However my intuition changed a bit when I started to think that 
`ReadOperation/Query` in every case is a subset of Read. Read = Connection + 
ReadOperation/Query, so why we need ReadOperation at all? We can have just the 
normal Read but if you set up explicitly the RingRage as an extra parameter 
well we don’t split and just read from it. SolrIO creates an artificial KV to 
put that ‘query’ in the side of the normal Read, but for example DynamoDBIO 
does the read based on an attribute and you could have an ultra simple ReadAll.
   
   ```java
     public static class ReadAll<T> extends PTransform<PCollection<Read<T>>, 
PCollection<T>> {
       @Override
       public PCollection<T> expand(PCollection<Read<T>> input) {
         return input.apply(ParDo.of(new ReadFn<>()));
       }
     }
   ```
   
   In the end `ReadAll` is more a kind of `Read Split/Partition` (without 
trying to split), so the only extra thing we need to have for that is the info 
that allows us to read the specific partition: `ReplicaInfo` in `Solr`,  
`segmentId` in `DynamoDB` and `RingRange` in `Cassandra`. Also having the full 
Read would allow users to read from different systems once the user creates the 
PCollection<Read> so this is a huge reusability advantage.
   
   Do you see any other tradeoffs, for me it looks quite straight forward and 
simpler, we won't need to maintain additional objects. Maybe worth trying it.
 
----------------------------------------------------------------
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: 376638)
    Time Spent: 3h  (was: 2h 50m)

> Add readAll() method to CassandraIO
> -----------------------------------
>
>                 Key: BEAM-9008
>                 URL: https://issues.apache.org/jira/browse/BEAM-9008
>             Project: Beam
>          Issue Type: New Feature
>          Components: io-java-cassandra
>    Affects Versions: 2.16.0
>            Reporter: vincent marquez
>            Assignee: vincent marquez
>            Priority: Minor
>          Time Spent: 3h
>  Remaining Estimate: 0h
>
> When querying a large cassandra database, it's often *much* more useful to 
> programatically generate the queries needed to to be run rather than reading 
> all partitions and attempting some filtering.  
> As an example:
> {code:java}
> public class Event { 
>    @PartitionKey(0) public UUID accountId;
>    @PartitionKey(1)public String yearMonthDay; 
>    @ClusteringKey public UUID eventId;  
>    //other data...
> }{code}
> If there is ten years worth of data, you may want to only query one year's 
> worth.  Here each token range would represent one 'token' but all events for 
> the day. 
> {code:java}
> Set<UUID> accounts = getRelevantAccounts();
> Set<String> dateRange = generateDateRange("2018-01-01", "2019-01-01");
> PCollection<TokenRange> tokens = generateTokens(accounts, dateRange); 
> {code}
>  
>  I propose an additional _readAll()_ PTransform that can take a PCollection 
> of token ranges and can return a PCollection<T> of what the query would 
> return. 
> *Question: How much code should be in common between both methods?* 
> Currently the read connector already groups all partitions into a List of 
> Token Ranges, so it would be simple to refactor the current read() based 
> method to a 'ParDo' based one and have them both share the same function.  
> Reasons against sharing code between read and readAll
>  * Not having the read based method return a BoundedSource connector would 
> mean losing the ability to know the size of the data returned
>  * Currently the CassandraReader executes all the grouped TokenRange queries 
> *asynchronously* which is (maybe?) fine when all that's happening is 
> splitting up all the partition ranges but terrible for executing potentially 
> millions of queries. 
>  Reasons _for_ sharing code would be simplified code base and that both of 
> the above issues would most likely have a negligable performance impact. 
>  
>  
>  



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

Reply via email to