You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@beam.apache.org by "ASF GitHub Bot (Jira)" <ji...@apache.org> on 2020/01/21 20:59:00 UTC

[jira] [Work logged] (BEAM-9008) Add readAll() method to CassandraIO

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

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

                Author: ASF GitHub Bot
            Created on: 21/Jan/20 20:58
            Start Date: 21/Jan/20 20:58
    Worklog Time Spent: 10m 
      Work Description: vmarquez commented on pull request #10546: [BEAM-9008] Add CassandraIO readAll method
URL: https://github.com/apache/beam/pull/10546#discussion_r369239579
 
 

 ##########
 File path: sdks/java/io/cassandra/src/main/java/org/apache/beam/sdk/io/cassandra/QueryFn.java
 ##########
 @@ -0,0 +1,89 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.io.cassandra;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ColumnMetadata;
+import com.datastax.driver.core.PreparedStatement;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.Token;
+import java.util.Iterator;
+import java.util.stream.Collectors;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class QueryFn<T> extends DoFn<Iterable<RingRange>, T> {
 
 Review comment:
   I don't think we want this flattened here, the way I'm using readAll currently is I am hashing into groups of queries based on actual shard, which is owned by a core (in ScyllaDB), so grouping queries gives finer grained control over how much load we put on a core.  Someone using cassandra could use a similar split idea for making sure to limit the number of concurrent TokenRanges owned by the same node in Cassandra. 
 
----------------------------------------------------------------
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:
users@infra.apache.org


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

    Worklog Id:     (was: 375209)
    Time Spent: 2h 20m  (was: 2h 10m)

> 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: 2h 20m
>  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)