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

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

                Author: ASF GitHub Bot
            Created on: 08/Apr/20 23:54
            Start Date: 08/Apr/20 23:54
    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_r405880091
 
 

 ##########
 File path: 
sdks/java/io/cassandra/src/main/java/org/apache/beam/sdk/io/cassandra/CassandraIO.java
 ##########
 @@ -1170,4 +887,344 @@ private void waitForFuturesToFinish() throws 
ExecutionException, InterruptedExce
       }
     }
   }
+
+  /**
+   * A {@link PTransform} to read data from Apache Cassandra. See {@link 
CassandraIO} for more
+   * information on usage and configuration.
+   */
+  @AutoValue
+  public abstract static class ReadAll<T> extends 
PTransform<PCollection<Read<T>>, PCollection<T>> {
+
+    @Nullable
+    abstract ValueProvider<List<String>> hosts();
+
+    @Nullable
+    abstract ValueProvider<String> query();
+
+    @Nullable
+    abstract ValueProvider<Integer> port();
+
+    @Nullable
+    abstract ValueProvider<String> keyspace();
+
+    @Nullable
+    abstract ValueProvider<String> table();
+
+    @Nullable
+    abstract Class<T> entity();
+
+    @Nullable
+    abstract Coder<T> coder();
+
+    @Nullable
+    abstract ValueProvider<String> username();
+
+    @Nullable
+    abstract ValueProvider<String> password();
+
+    @Nullable
+    abstract ValueProvider<String> localDc();
+
+    @Nullable
+    abstract ValueProvider<String> consistencyLevel();
+
+    @Nullable
+    abstract ValueProvider<Integer> splitCount();
+
+    @Nullable
+    abstract SerializableFunction<Session, Mapper> mapperFactoryFn();
+
+    @Nullable
+    abstract SerializableFunction<RingRange, Integer> groupingFn();
+
+    abstract Builder<T> builder();
+
+    /** Specify the hosts of the Apache Cassandra instances. */
+    public ReadAll<T> withHosts(List<String> hosts) {
+      checkArgument(hosts != null, "hosts can not be null");
+      checkArgument(!hosts.isEmpty(), "hosts can not be empty");
+      return withHosts(ValueProvider.StaticValueProvider.of(hosts));
+    }
+
+    /** Specify the hosts of the Apache Cassandra instances. */
+    public ReadAll<T> withHosts(ValueProvider<List<String>> hosts) {
+      return builder().setHosts(hosts).build();
+    }
+
+    /** Specify the port number of the Apache Cassandra instances. */
+    public ReadAll<T> withPort(int port) {
+      checkArgument(port > 0, "port must be > 0, but was: %s", port);
+      return withPort(ValueProvider.StaticValueProvider.of(port));
+    }
+
+    /** Specify the port number of the Apache Cassandra instances. */
+    public ReadAll<T> withPort(ValueProvider<Integer> port) {
+      return builder().setPort(port).build();
+    }
+
+    /** Specify the Cassandra keyspace where to read data. */
+    public ReadAll<T> withKeyspace(String keyspace) {
+      checkArgument(keyspace != null, "keyspace can not be null");
+      return withKeyspace(ValueProvider.StaticValueProvider.of(keyspace));
+    }
+
+    /** Specify the Cassandra keyspace where to read data. */
+    public ReadAll<T> withKeyspace(ValueProvider<String> keyspace) {
+      return builder().setKeyspace(keyspace).build();
+    }
+
+    /** Specify the Cassandra table where to read data. */
+    public ReadAll<T> withTable(String table) {
+      checkArgument(table != null, "table can not be null");
+      return withTable(ValueProvider.StaticValueProvider.of(table));
+    }
+
+    /** Specify the Cassandra table where to read data. */
+    public ReadAll<T> withTable(ValueProvider<String> table) {
+      return builder().setTable(table).build();
+    }
+
+    /** Specify the query to read data. */
+    public ReadAll<T> withQuery(String query) {
+      checkArgument(query != null && query.length() > 0, "query cannot be 
null");
+      return withQuery(ValueProvider.StaticValueProvider.of(query));
+    }
+
+    /** Specify the query to read data. */
+    public ReadAll<T> withQuery(ValueProvider<String> query) {
+      return builder().setQuery(query).build();
+    }
+
+    /**
+     * Specify the entity class (annotated POJO). The {@link CassandraIO} will 
read the data and
+     * convert the data as entity instances. The {@link PCollection} resulting 
from the read will
+     * contains entity elements.
+     */
+    public ReadAll<T> withEntity(Class<T> entity) {
+      checkArgument(entity != null, "entity can not be null");
+      return builder().setEntity(entity).build();
+    }
+
+    /** Specify the {@link Coder} used to serialize the entity in the {@link 
PCollection}. */
+    public ReadAll<T> withCoder(Coder<T> coder) {
+      checkArgument(coder != null, "coder can not be null");
+      return builder().setCoder(coder).build();
+    }
+
+    /** Specify the username for authentication. */
+    public ReadAll<T> withUsername(String username) {
+      checkArgument(username != null, "username can not be null");
+      return withUsername(ValueProvider.StaticValueProvider.of(username));
+    }
+
+    /** Specify the username for authentication. */
+    public ReadAll<T> withUsername(ValueProvider<String> username) {
+      return builder().setUsername(username).build();
+    }
+
+    /** Specify the password used for authentication. */
+    public ReadAll<T> withPassword(String password) {
+      checkArgument(password != null, "password can not be null");
+      return withPassword(ValueProvider.StaticValueProvider.of(password));
+    }
+
+    /** Specify the password used for authentication. */
+    public ReadAll<T> withPassword(ValueProvider<String> password) {
+      return builder().setPassword(password).build();
+    }
+
+    /** Specify the local DC used for the load balancing. */
+    public ReadAll<T> withLocalDc(String localDc) {
+      checkArgument(localDc != null, "localDc can not be null");
+      return withLocalDc(ValueProvider.StaticValueProvider.of(localDc));
+    }
+
+    /** Specify the local DC used for the load balancing. */
+    public ReadAll<T> withLocalDc(ValueProvider<String> localDc) {
+      return builder().setLocalDc(localDc).build();
+    }
+
+    public ReadAll<T> withConsistencyLevel(String consistencyLevel) {
+      checkArgument(consistencyLevel != null, "consistencyLevel can not be 
null");
+      return 
withConsistencyLevel(ValueProvider.StaticValueProvider.of(consistencyLevel));
+    }
+
+    public ReadAll<T> withConsistencyLevel(ValueProvider<String> 
consistencyLevel) {
+      return builder().setConsistencyLevel(consistencyLevel).build();
+    }
+
+    public ReadAll<T> withGroupingFn(SerializableFunction<RingRange, Integer> 
groupingFunction) {
+      return builder().setGroupingFn(groupingFunction).build();
+    }
+
+    public ReadAll<T> withSplitCount(ValueProvider<Integer> splitCount) {
+      return builder().setSplitCount(splitCount).build();
+    }
+
+    public ReadAll<T> withSplitCount(Integer splitCount) {
+      checkArgument(splitCount != null, "splitCount can not be null");
+      return 
withSplitCount(ValueProvider.StaticValueProvider.<Integer>of(splitCount));
+    }
+
+    /**
+     * A factory to create a specific {@link Mapper} for a given Cassandra 
Session. This is useful
+     * to provide mappers that don't rely in Cassandra annotated objects.
+     */
+    public ReadAll<T> withMapperFactoryFn(SerializableFunction<Session, 
Mapper> mapperFactory) {
+      checkArgument(
+          mapperFactory != null,
+          "CassandraIO.withMapperFactory" + "(withMapperFactory) called with 
null value");
+      return builder().setMapperFactoryFn(mapperFactory).build();
+    }
+
+    @Override
+    public PCollection<T> expand(PCollection<Read<T>> input) {
+      checkArgument((hosts() != null && port() != null), "WithHosts() and 
withPort() are required");
+      checkArgument(keyspace() != null, "withKeyspace() is required");
+      checkArgument(table() != null, "withTable() is required");
+      checkArgument(entity() != null, "withEntity() is required");
+      checkArgument(coder() != null, "withCoder() is required");
+      checkArgument(groupingFn() != null, "GroupingFn OR splitCount must be 
set");
+      try (Cluster cluster =
 
 Review comment:
   Not sure I understand what you mean here.  I'm doing a shuffle by key here, 
`FlattenGrouped` here is what I called it, following the other connectors 
general pattern. 
 
----------------------------------------------------------------
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: 419003)
    Time Spent: 9.5h  (was: 9h 20m)

> 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: 9.5h
>  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