[ https://issues.apache.org/jira/browse/BEAM-9008?focusedWorklogId=378338&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-378338 ]
ASF GitHub Bot logged work on BEAM-9008: ---------------------------------------- Author: ASF GitHub Bot Created on: 28/Jan/20 15:55 Start Date: 28/Jan/20 15:55 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-579317145 > @iemejia I think that could work, thanks for your patience as I try to understand what you're thinking. Some questions: No, thanks to you who has been the patient one during this discussion. > 1. If our `ReadFn<A> extends DoFn<Read<A>, A>` and the only way we have connection information is from the `Read<A>` passed in to the processElement, that means we need to re-establish a DB connection for each batch of queries we run? As in, the connection would be established in the `processElement` method and could not be in `setup` method? Yes exactly this will make the method simpler and the cost of starting a connection gets amortized by the processElement producing multiple outputs from a single connection. > 2. How would that work for the end user of a `PTransform<PCollection<Read<A>, PCollection<A>>`? Here is what I did in the test and would wand to document how end users could generate 'queries', > https://github.com/apache/beam/pull/10546/files#diff-8ba4ea3b09d563a67e29ff8584269d35R499 > Would we instead want to return a `PCollection<Read<A>>` by using something like `return CassandraIO.read().withRingRange(new RingRange(start, finish))`? If we do that however, we'd need to do the `withHosts` and all the other connection information, no? The other option is establishing one `ReadAll` PTransform that maps over the `Read<A>` input and enriches the db connection information? You have a point here!. We need `class ReadAll extends PTransform<PCollection<Read<A>, PCollection<A>>` and there we read as intended with `ReadFn`. You would have to modify however the `expand` of `Read` to do `input.apply(Create.of(this)).apply(CassandraIO.readAll())` where `ReadAll` should expand into `input.apply(ParDo.of(splitFn)).apply(Reshuffle).apply(Read)` users should deal with building the PCollection of `Reads` before passing that collection to `ReadAll`. > 3. Originally I had wanted to have the ReadFn operate on a _collection_ of 'query' objects to ensure a way to enforce linearizability with our queries (mainly so we don't oversaturate a single node/shard). Currently the groupBy function a user passes in operates on the `RingRange` object, would we keep it that way and just, under the hood, allow for a single `Read<A>` to hold a collection of RingRanges? If I understand this correctly this is covered by following the Create -> Split -> Reshuffle -> Read pattern mentioned above (in the mentioned IOs). So Split is the one who will generate a collection of `Read`s for each given `RingRange` then we use Reshuffle to guarantee that reads are redistributed and finally each read request is read by one worker. Hope this helps, don't hesitate to ask me more questions if still. I will try to answer quickly this time. Hope this helps ---------------------------------------------------------------- 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: us...@infra.apache.org Issue Time Tracking ------------------- Worklog Id: (was: 378338) Time Spent: 4h (was: 3h 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: 4h > 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)