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

2020-06-09 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on BEAM-9008:


Author: ASF GitHub Bot
Created on: 09/Jun/20 16:06
Start Date: 09/Jun/20 16:06
Worklog Time Spent: 10m 
  Work Description: vmarquez commented on a change in pull request #10546:
URL: https://github.com/apache/beam/pull/10546#discussion_r436855523



##
File path: 
sdks/java/io/cassandra/src/main/java/org/apache/beam/sdk/io/cassandra/CassandraIO.java
##
@@ -326,7 +371,78 @@ private CassandraIO() {}
   checkArgument(entity() != null, "withEntity() is required");
   checkArgument(coder() != null, "withCoder() is required");
 
-  return input.apply(org.apache.beam.sdk.io.Read.from(new 
CassandraSource<>(this, null)));
+  ReadAll readAll = CassandraIO.readAll().withCoder(this.coder());
+
+  return input
+  .apply(Create.of(this))
+  .apply(ParDo.of(new SplitFn()))

Review comment:
   > Move the Split as the first step of the ReadAll expansion so non 
advanced users (those who do not specify RingRange manually could get their 
code 'partitioned' correctly.
   
   I am not sure I understand what you mean here, could you explain more? 





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: 442866)
Time Spent: 13h 40m  (was: 13.5h)

> 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: P3
>  Time Spent: 13h 40m
>  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 accounts = getRelevantAccounts();
> Set dateRange = generateDateRange("2018-01-01", "2019-01-01");
> PCollection tokens = generateTokens(accounts, dateRange); 
> {code}
>  
>  I propose an additional _readAll()_ PTransform that can take a PCollection 
> of token ranges and can return a PCollection 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)


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

2020-06-04 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on BEAM-9008:


Author: ASF GitHub Bot
Created on: 04/Jun/20 20:11
Start Date: 04/Jun/20 20:11
Worklog Time Spent: 10m 
  Work Description: vmarquez commented on a change in pull request #10546:
URL: https://github.com/apache/beam/pull/10546#discussion_r434870508



##
File path: 
sdks/java/io/cassandra/src/main/java/org/apache/beam/sdk/io/cassandra/CassandraIO.java
##
@@ -1170,4 +898,44 @@ 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 extends 
PTransform>, PCollection> {
+
+@Nullable
+abstract Coder coder();
+
+abstract ReadAll.Builder builder();
+
+/** Specify the {@link Coder} used to serialize the entity in the {@link 
PCollection}. */
+public ReadAll withCoder(Coder coder) {

Review comment:
   I think we do, but am not sure.  You have to call `setCoder` on the 
PCollection itself, so we don't have access to the `PCollection>` at a 
point when we also have access to a single `Read` (they are only supplied in 
the `ReadFn` which can't call `setCoder` on the PTransform).  Is my thinking 
correct?  I could be unaware of a different way to do this.  





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: 441503)
Time Spent: 13.5h  (was: 13h 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: P3
>  Time Spent: 13.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 accounts = getRelevantAccounts();
> Set dateRange = generateDateRange("2018-01-01", "2019-01-01");
> PCollection tokens = generateTokens(accounts, dateRange); 
> {code}
>  
>  I propose an additional _readAll()_ PTransform that can take a PCollection 
> of token ranges and can return a PCollection 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)


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

2020-06-03 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on BEAM-9008:


Author: ASF GitHub Bot
Created on: 03/Jun/20 21:33
Start Date: 03/Jun/20 21:33
Worklog Time Spent: 10m 
  Work Description: vmarquez commented on a change in pull request #10546:
URL: https://github.com/apache/beam/pull/10546#discussion_r434870508



##
File path: 
sdks/java/io/cassandra/src/main/java/org/apache/beam/sdk/io/cassandra/CassandraIO.java
##
@@ -1170,4 +898,44 @@ 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 extends 
PTransform>, PCollection> {
+
+@Nullable
+abstract Coder coder();
+
+abstract ReadAll.Builder builder();
+
+/** Specify the {@link Coder} used to serialize the entity in the {@link 
PCollection}. */
+public ReadAll withCoder(Coder coder) {

Review comment:
   I think we do, but am not sure.  You have to call `setCoder` on the 
PCollection itself, so we don't have access to the `Read` at a point when we 
also have access to a single `Read` (they are only supplied in the `ReadFn` 
which can't call `setCoder` on the returned PTransform).  Is my thinking 
correct?  I could be unaware of a different way to do this.  





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: 440999)
Time Spent: 13h 20m  (was: 13h 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: P3
>  Time Spent: 13h 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 accounts = getRelevantAccounts();
> Set dateRange = generateDateRange("2018-01-01", "2019-01-01");
> PCollection tokens = generateTokens(accounts, dateRange); 
> {code}
>  
>  I propose an additional _readAll()_ PTransform that can take a PCollection 
> of token ranges and can return a PCollection 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)


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

2020-06-02 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on BEAM-9008:


Author: ASF GitHub Bot
Created on: 03/Jun/20 05:38
Start Date: 03/Jun/20 05:38
Worklog Time Spent: 10m 
  Work Description: vmarquez commented on a change in pull request #10546:
URL: https://github.com/apache/beam/pull/10546#discussion_r434318846



##
File path: 
sdks/java/io/cassandra/src/main/java/org/apache/beam/sdk/io/cassandra/CassandraIO.java
##
@@ -370,384 +488,16 @@ private CassandraIO() {}
 return autoBuild();
   }
 }
-  }
-
-  @VisibleForTesting
-  static class CassandraSource extends BoundedSource {
-final Read spec;
-final List splitQueries;
-// split source ached size - can't be calculated when already split
-Long estimatedSize;
-private static final String MURMUR3PARTITIONER = 
"org.apache.cassandra.dht.Murmur3Partitioner";
-
-CassandraSource(Read spec, List splitQueries) {
-  this(spec, splitQueries, null);
-}
-
-private CassandraSource(Read spec, List splitQueries, Long 
estimatedSize) {
-  this.estimatedSize = estimatedSize;
-  this.spec = spec;
-  this.splitQueries = splitQueries;
-}
-
-@Override
-public Coder getOutputCoder() {
-  return spec.coder();
-}
-
-@Override
-public BoundedReader createReader(PipelineOptions pipelineOptions) {
-  return new CassandraReader(this);
-}
-
-@Override
-public List> split(
-long desiredBundleSizeBytes, PipelineOptions pipelineOptions) {
-  try (Cluster cluster =
-  getCluster(
-  spec.hosts(),
-  spec.port(),
-  spec.username(),
-  spec.password(),
-  spec.localDc(),
-  spec.consistencyLevel())) {
-if (isMurmur3Partitioner(cluster)) {
-  LOG.info("Murmur3Partitioner detected, splitting");
-  return splitWithTokenRanges(
-  spec, desiredBundleSizeBytes, 
getEstimatedSizeBytes(pipelineOptions), cluster);
-} else {
-  LOG.warn(
-  "Only Murmur3Partitioner is supported for splitting, using a 
unique source for "
-  + "the read");
-  return Collections.singletonList(
-  new CassandraIO.CassandraSource<>(spec, 
Collections.singletonList(buildQuery(spec;
-}
-  }
-}
-
-private static String buildQuery(Read spec) {
-  return (spec.query() == null)
-  ? String.format("SELECT * FROM %s.%s", spec.keyspace().get(), 
spec.table().get())
-  : spec.query().get().toString();
-}
-
-/**
- * Compute the number of splits based on the estimated size and the 
desired bundle size, and
- * create several sources.
- */
-private List> splitWithTokenRanges(
-CassandraIO.Read spec,
-long desiredBundleSizeBytes,
-long estimatedSizeBytes,
-Cluster cluster) {
-  long numSplits =
-  getNumSplits(desiredBundleSizeBytes, estimatedSizeBytes, 
spec.minNumberOfSplits());
-  LOG.info("Number of desired splits is {}", numSplits);
-
-  SplitGenerator splitGenerator = new 
SplitGenerator(cluster.getMetadata().getPartitioner());
-  List tokens =
-  cluster.getMetadata().getTokenRanges().stream()
-  .map(tokenRange -> new 
BigInteger(tokenRange.getEnd().getValue().toString()))
-  .collect(Collectors.toList());
-  List> splits = splitGenerator.generateSplits(numSplits, 
tokens);
-  LOG.info("{} splits were actually generated", splits.size());
-
-  final String partitionKey =
-  
cluster.getMetadata().getKeyspace(spec.keyspace().get()).getTable(spec.table().get())
-  .getPartitionKey().stream()
-  .map(ColumnMetadata::getName)
-  .collect(Collectors.joining(","));
-
-  List tokenRanges =
-  getTokenRanges(cluster, spec.keyspace().get(), spec.table().get());
-  final long estimatedSize = 
getEstimatedSizeBytesFromTokenRanges(tokenRanges) / splits.size();
-
-  List> sources = new ArrayList<>();
-  for (List split : splits) {
-List queries = new ArrayList<>();
-for (RingRange range : split) {
-  if (range.isWrapping()) {

Review comment:
   Hm, let me think about this. 





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: 440603)
Time Spent: 13h 10m  (was: 13h)

> Add readAll() method to CassandraIO
> 

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

2020-06-02 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on BEAM-9008:


Author: ASF GitHub Bot
Created on: 03/Jun/20 05:37
Start Date: 03/Jun/20 05:37
Worklog Time Spent: 10m 
  Work Description: vmarquez commented on a change in pull request #10546:
URL: https://github.com/apache/beam/pull/10546#discussion_r434318532



##
File path: 
sdks/java/io/cassandra/src/main/java/org/apache/beam/sdk/io/cassandra/CassandraIO.java
##
@@ -326,7 +371,78 @@ private CassandraIO() {}
   checkArgument(entity() != null, "withEntity() is required");
   checkArgument(coder() != null, "withCoder() is required");
 
-  return input.apply(org.apache.beam.sdk.io.Read.from(new 
CassandraSource<>(this, null)));
+  ReadAll readAll = CassandraIO.readAll().withCoder(this.coder());
+
+  return input
+  .apply(Create.of(this))
+  .apply(ParDo.of(new SplitFn()))
+  .setCoder(SerializableCoder.of(new TypeDescriptor>() {}))
+  // .apply(Reshuffle.viaRandomKey())
+  .apply(readAll);
+}
+
+private class SplitFn extends DoFn, Read> {
+
+  @ProcessElement
+  public void process(
+  @Element CassandraIO.Read read, OutputReceiver> 
outputReceiver) {
+
+try (Cluster cluster =
+getCluster(
+read.hosts(),
+read.port(),
+read.username(),
+read.password(),
+read.localDc(),
+read.consistencyLevel())) {
+  if (isMurmur3Partitioner(cluster)) {
+LOG.info("Murmur3Partitioner detected, splitting");
+
+List tokens =
+cluster.getMetadata().getTokenRanges().stream()
+.map(tokenRange -> new 
BigInteger(tokenRange.getEnd().getValue().toString()))
+.collect(Collectors.toList());
+Integer splitCount = cluster.getMetadata().getAllHosts().size();
+if (read.minNumberOfSplits() != null && 
read.minNumberOfSplits().get() != null) {
+  splitCount = read.minNumberOfSplits().get();
+}
+
+SplitGenerator splitGenerator =
+new SplitGenerator(cluster.getMetadata().getPartitioner());
+splitGenerator
+.generateSplits(splitCount, tokens)
+.forEach(
+rr ->
+outputReceiver.output(
+CassandraIO.read()

Review comment:
   I think we still need to create a new `Read` because the 
`SplitGenerator` returns a `List>`, so each of the outer list 
will be a different Read, the inner List will be set to the RingRange. 





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: 440602)
Time Spent: 13h  (was: 12h 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: P3
>  Time Spent: 13h
>  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 accounts = getRelevantAccounts();
> Set dateRange = generateDateRange("2018-01-01", "2019-01-01");
> PCollection tokens = generateTokens(accounts, dateRange); 
> {code}
>  
>  I propose an additional _readAll()_ PTransform that can take a PCollection 
> of token ranges and can return a PCollection 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 

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

2020-06-02 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on BEAM-9008:


Author: ASF GitHub Bot
Created on: 03/Jun/20 04:45
Start Date: 03/Jun/20 04:45
Worklog Time Spent: 10m 
  Work Description: vmarquez commented on a change in pull request #10546:
URL: https://github.com/apache/beam/pull/10546#discussion_r434304763



##
File path: 
sdks/java/io/cassandra/src/test/java/org/apache/beam/sdk/io/cassandra/CassandraIOTest.java
##
@@ -480,66 +527,22 @@ public void testCustomMapperImplDelete() {
 assertEquals(1, counter.intValue());
   }
 
-  @Test
-  public void testSplit() throws Exception {
-PipelineOptions options = PipelineOptionsFactory.create();
-CassandraIO.Read read =
-CassandraIO.read()
-.withHosts(Collections.singletonList(CASSANDRA_HOST))
-.withPort(cassandraPort)
-.withKeyspace(CASSANDRA_KEYSPACE)
-.withTable(CASSANDRA_TABLE)
-.withEntity(Scientist.class)
-.withCoder(SerializableCoder.of(Scientist.class));
-
-// initialSource will be read without splitting (which does not happen in 
production)
-// so we need to provide splitQueries to avoid NPE in source.reader.start()
-String splitQuery = QueryBuilder.select().from(CASSANDRA_KEYSPACE, 
CASSANDRA_TABLE).toString();
-CassandraIO.CassandraSource initialSource =
-new CassandraIO.CassandraSource<>(read, 
Collections.singletonList(splitQuery));
-int desiredBundleSizeBytes = 2048;
-long estimatedSize = initialSource.getEstimatedSizeBytes(options);
-List> splits = 
initialSource.split(desiredBundleSizeBytes, options);
-SourceTestUtils.assertSourcesEqualReferenceSource(initialSource, splits, 
options);
-float expectedNumSplitsloat =
-(float) initialSource.getEstimatedSizeBytes(options) / 
desiredBundleSizeBytes;
-long sum = 0;
-
-for (BoundedSource subSource : splits) {
-  sum += subSource.getEstimatedSizeBytes(options);
-}
-
-// due to division and cast estimateSize != sum but will be close. Exact 
equals checked below
-assertEquals((long) (estimatedSize / splits.size()) * splits.size(), sum);
-
-int expectedNumSplits = (int) Math.ceil(expectedNumSplitsloat);
-assertEquals("Wrong number of splits", expectedNumSplits, splits.size());
-int emptySplits = 0;
-for (BoundedSource subSource : splits) {
-  if (readFromSource(subSource, options).isEmpty()) {
-emptySplits += 1;
-  }
-}
-assertThat(
-"There are too many empty splits, parallelism is sub-optimal",
-emptySplits,
-lessThan((int) (ACCEPTABLE_EMPTY_SPLITS_PERCENTAGE * splits.size(;
-  }
-
   private List getRows(String table) {
 ResultSet result =
 session.execute(
 String.format("select person_id,person_name from %s.%s", 
CASSANDRA_KEYSPACE, table));
 return result.all();
   }
 
+  // TEMP TEST

Review comment:
   Ooops I think this was left in from when I was experimenting with 
something, I will remove.





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: 440596)
Time Spent: 12h 50m  (was: 12h 40m)

> 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: P3
>  Time Spent: 12h 50m
>  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 accounts = getRelevantAccounts();
> Set dateRange = generateDateRange("2018-01-01", "2019-01-01");
> PCollection tokens = generateTokens(accounts, dateRange); 
> {code}
>  
>  I propose an 

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

2020-06-02 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on BEAM-9008:


Author: ASF GitHub Bot
Created on: 03/Jun/20 04:41
Start Date: 03/Jun/20 04:41
Worklog Time Spent: 10m 
  Work Description: vmarquez commented on a change in pull request #10546:
URL: https://github.com/apache/beam/pull/10546#discussion_r434303784



##
File path: 
sdks/java/io/cassandra/src/main/java/org/apache/beam/sdk/io/cassandra/CassandraIO.java
##
@@ -326,7 +371,78 @@ private CassandraIO() {}
   checkArgument(entity() != null, "withEntity() is required");
   checkArgument(coder() != null, "withCoder() is required");
 
-  return input.apply(org.apache.beam.sdk.io.Read.from(new 
CassandraSource<>(this, null)));
+  ReadAll readAll = CassandraIO.readAll().withCoder(this.coder());
+
+  return input
+  .apply(Create.of(this))
+  .apply(ParDo.of(new SplitFn()))
+  .setCoder(SerializableCoder.of(new TypeDescriptor>() {}))
+  // .apply(Reshuffle.viaRandomKey())
+  .apply(readAll);
+}
+
+private class SplitFn extends DoFn, Read> {
+
+  @ProcessElement
+  public void process(
+  @Element CassandraIO.Read read, OutputReceiver> 
outputReceiver) {
+
+try (Cluster cluster =
+getCluster(
+read.hosts(),
+read.port(),
+read.username(),
+read.password(),
+read.localDc(),
+read.consistencyLevel())) {
+  if (isMurmur3Partitioner(cluster)) {

Review comment:
   Oops I did forget that, I will add. Good catch. 





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: 440594)
Time Spent: 12h 40m  (was: 12.5h)

> 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: P3
>  Time Spent: 12h 40m
>  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 accounts = getRelevantAccounts();
> Set dateRange = generateDateRange("2018-01-01", "2019-01-01");
> PCollection tokens = generateTokens(accounts, dateRange); 
> {code}
>  
>  I propose an additional _readAll()_ PTransform that can take a PCollection 
> of token ranges and can return a PCollection 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)


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

2020-06-02 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on BEAM-9008:


Author: ASF GitHub Bot
Created on: 02/Jun/20 21:28
Start Date: 02/Jun/20 21:28
Worklog Time Spent: 10m 
  Work Description: iemejia commented on a change in pull request #10546:
URL: https://github.com/apache/beam/pull/10546#discussion_r433896372



##
File path: 
sdks/java/io/cassandra/src/main/java/org/apache/beam/sdk/io/cassandra/RingRange.java
##
@@ -55,4 +58,9 @@ public boolean isWrapping() {
   public String toString() {
 return String.format("(%s,%s]", start.toString(), end.toString());
   }
+
+  public static RingRange fromEncodedKey(Metadata metadata, ByteBuffer... bb) {

Review comment:
   Can we move this method to the test class where it is used. I don't want 
to add Cassandra specific `Metadata` to the public API of RingRange with the 
hope this will help us evolve RingRange into a proper `Restriction` (future 
work out of the scope of this PR)
   
   Can you also please add a `public static RingRange of(BigInteger start, 
BigInteger send)` method and make the normal constructor private and refactor 
in every use.

##
File path: 
sdks/java/io/cassandra/src/main/java/org/apache/beam/sdk/io/cassandra/CassandraIO.java
##
@@ -326,7 +371,78 @@ private CassandraIO() {}
   checkArgument(entity() != null, "withEntity() is required");
   checkArgument(coder() != null, "withCoder() is required");
 
-  return input.apply(org.apache.beam.sdk.io.Read.from(new 
CassandraSource<>(this, null)));
+  ReadAll readAll = CassandraIO.readAll().withCoder(this.coder());
+
+  return input
+  .apply(Create.of(this))
+  .apply(ParDo.of(new SplitFn()))

Review comment:
   Move the Split as the first step of the `ReadAll` expansion so non 
advanced users (those who do not specify `RingRange` manually could get their 
code 'partitioned' correctly.

##
File path: 
sdks/java/io/cassandra/src/test/java/org/apache/beam/sdk/io/cassandra/CassandraIOTest.java
##
@@ -317,9 +302,70 @@ public void testRead() throws Exception {
 PAssert.that(mapped.apply("Count occurrences per scientist", 
Count.perKey()))
 .satisfies(
 input -> {
+  int count = 0;
   for (KV element : input) {
+count++;
 assertEquals(element.getKey(), NUM_ROWS / 10, 
element.getValue().longValue());
   }
+  assertEquals(11, count);
+  return null;
+});
+
+pipeline.run();
+  }
+
+  CassandraIO.Read getReadWithRingRange(RingRange... rr) {

Review comment:
   private

##
File path: 
sdks/java/io/cassandra/src/test/java/org/apache/beam/sdk/io/cassandra/CassandraIOTest.java
##
@@ -191,39 +186,44 @@ private static void insertData() throws Exception {
 LOG.info("Create Cassandra tables");
 session.execute(
 String.format(
-"CREATE TABLE IF NOT EXISTS %s.%s(person_id int, person_name text, 
PRIMARY KEY"
-+ "(person_id));",
+"CREATE TABLE IF NOT EXISTS %s.%s(person_department text, 
person_id int, person_name text, PRIMARY KEY"

Review comment:
   nit: For the tests can we assure that every reference to the tables and 
Scientist object usage follows this order: id, name, department.
   

##
File path: 
sdks/java/io/cassandra/src/test/java/org/apache/beam/sdk/io/cassandra/CassandraIOTest.java
##
@@ -480,66 +527,22 @@ public void testCustomMapperImplDelete() {
 assertEquals(1, counter.intValue());
   }
 
-  @Test
-  public void testSplit() throws Exception {
-PipelineOptions options = PipelineOptionsFactory.create();
-CassandraIO.Read read =
-CassandraIO.read()
-.withHosts(Collections.singletonList(CASSANDRA_HOST))
-.withPort(cassandraPort)
-.withKeyspace(CASSANDRA_KEYSPACE)
-.withTable(CASSANDRA_TABLE)
-.withEntity(Scientist.class)
-.withCoder(SerializableCoder.of(Scientist.class));
-
-// initialSource will be read without splitting (which does not happen in 
production)
-// so we need to provide splitQueries to avoid NPE in source.reader.start()
-String splitQuery = QueryBuilder.select().from(CASSANDRA_KEYSPACE, 
CASSANDRA_TABLE).toString();
-CassandraIO.CassandraSource initialSource =
-new CassandraIO.CassandraSource<>(read, 
Collections.singletonList(splitQuery));
-int desiredBundleSizeBytes = 2048;
-long estimatedSize = initialSource.getEstimatedSizeBytes(options);
-List> splits = 
initialSource.split(desiredBundleSizeBytes, options);
-SourceTestUtils.assertSourcesEqualReferenceSource(initialSource, splits, 
options);
-float 

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

2020-06-02 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on BEAM-9008:


Author: ASF GitHub Bot
Created on: 02/Jun/20 14:15
Start Date: 02/Jun/20 14:15
Worklog Time Spent: 10m 
  Work Description: iemejia removed a comment on pull request #10546:
URL: https://github.com/apache/beam/pull/10546#issuecomment-637571013


   retest this please



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: 440151)
Time Spent: 12h 20m  (was: 12h 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: P3
>  Labels: stale-assigned
>  Time Spent: 12h 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 accounts = getRelevantAccounts();
> Set dateRange = generateDateRange("2018-01-01", "2019-01-01");
> PCollection tokens = generateTokens(accounts, dateRange); 
> {code}
>  
>  I propose an additional _readAll()_ PTransform that can take a PCollection 
> of token ranges and can return a PCollection 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)


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

2020-06-02 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on BEAM-9008:


Author: ASF GitHub Bot
Created on: 02/Jun/20 14:14
Start Date: 02/Jun/20 14:14
Worklog Time Spent: 10m 
  Work Description: iemejia commented on pull request #10546:
URL: https://github.com/apache/beam/pull/10546#issuecomment-637571013


   retest this please



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: 440150)
Time Spent: 12h 10m  (was: 12h)

> 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: P3
>  Labels: stale-assigned
>  Time Spent: 12h 10m
>  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 accounts = getRelevantAccounts();
> Set dateRange = generateDateRange("2018-01-01", "2019-01-01");
> PCollection tokens = generateTokens(accounts, dateRange); 
> {code}
>  
>  I propose an additional _readAll()_ PTransform that can take a PCollection 
> of token ranges and can return a PCollection 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)


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

2020-04-28 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on BEAM-9008:


Author: ASF GitHub Bot
Created on: 28/Apr/20 23:05
Start Date: 28/Apr/20 23:05
Worklog Time Spent: 10m 
  Work Description: iemejia commented on a change in pull request #10546:
URL: https://github.com/apache/beam/pull/10546#discussion_r416976043



##
File path: 
sdks/java/io/cassandra/src/main/java/org/apache/beam/sdk/io/cassandra/CassandraIO.java
##
@@ -391,36 +393,57 @@ static Cluster getCluster(
 .withSplitCount(splitCount)
 .withMapperFactoryFn(this.mapperFactoryFn());
 
-if (isMurmur3Partitioner(cluster)) {
-  LOG.info("Murmur3Partitioner detected, splitting");
-
-  List tokens =
-  cluster.getMetadata().getTokenRanges().stream()
-  .map(tokenRange -> new 
BigInteger(tokenRange.getEnd().getValue().toString()))
-  .collect(Collectors.toList());
-
-  SplitGenerator splitGenerator =
-  new SplitGenerator(cluster.getMetadata().getPartitioner());
-
-  List> splits =
-  splitGenerator.generateSplits(splitCount, tokens).stream()
-  .map(rr -> CassandraIO.read().withRingRange(rr))
-  .collect(Collectors.toList());
+return input
+.apply(Create.of(this))
+.apply(ParDo.of(new SplitFn()))
+.setCoder(SerializableCoder.of(new TypeDescriptor>() {}))
+.apply(Reshuffle.viaRandomKey())
+.apply(readAll);

Review comment:
   Now we need to tackle this in two parts maybe, one is to implement the 
read with a `ReadFn` like method  and as a next step to get rid of all the 
methods on `ReadAll` to simplify it to its core.





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: 428428)
Time Spent: 12h  (was: 11h 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: 12h
>  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 accounts = getRelevantAccounts();
> Set dateRange = generateDateRange("2018-01-01", "2019-01-01");
> PCollection tokens = generateTokens(accounts, dateRange); 
> {code}
>  
>  I propose an additional _readAll()_ PTransform that can take a PCollection 
> of token ranges and can return a PCollection 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)


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

2020-04-28 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on BEAM-9008:


Author: ASF GitHub Bot
Created on: 28/Apr/20 23:04
Start Date: 28/Apr/20 23:04
Worklog Time Spent: 10m 
  Work Description: iemejia commented on a change in pull request #10546:
URL: https://github.com/apache/beam/pull/10546#discussion_r416975426



##
File path: 
sdks/java/io/cassandra/src/main/java/org/apache/beam/sdk/io/cassandra/CassandraIO.java
##
@@ -391,36 +393,57 @@ static Cluster getCluster(
 .withSplitCount(splitCount)
 .withMapperFactoryFn(this.mapperFactoryFn());
 
-if (isMurmur3Partitioner(cluster)) {
-  LOG.info("Murmur3Partitioner detected, splitting");
-
-  List tokens =
-  cluster.getMetadata().getTokenRanges().stream()
-  .map(tokenRange -> new 
BigInteger(tokenRange.getEnd().getValue().toString()))
-  .collect(Collectors.toList());
-
-  SplitGenerator splitGenerator =
-  new SplitGenerator(cluster.getMetadata().getPartitioner());
-
-  List> splits =
-  splitGenerator.generateSplits(splitCount, tokens).stream()
-  .map(rr -> CassandraIO.read().withRingRange(rr))
-  .collect(Collectors.toList());
+return input
+.apply(Create.of(this))
+.apply(ParDo.of(new SplitFn()))
+.setCoder(SerializableCoder.of(new TypeDescriptor>() {}))
+.apply(Reshuffle.viaRandomKey())
+.apply(readAll);
+  }
+}
 
-  return input.apply("Creating splits", 
Create.of(splits)).apply("readAll", readAll);
+private static class SplitFn extends DoFn, Read> {

Review comment:
   This looks perfect!





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: 428425)
Time Spent: 11h 50m  (was: 11h 40m)

> 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: 11h 50m
>  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 accounts = getRelevantAccounts();
> Set dateRange = generateDateRange("2018-01-01", "2019-01-01");
> PCollection tokens = generateTokens(accounts, dateRange); 
> {code}
>  
>  I propose an additional _readAll()_ PTransform that can take a PCollection 
> of token ranges and can return a PCollection 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)


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

2020-04-08 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on BEAM-9008:


Author: ASF GitHub Bot
Created on: 09/Apr/20 01:23
Start Date: 09/Apr/20 01:23
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_r405875425
 
 

 ##
 File path: 
sdks/java/io/cassandra/src/main/java/org/apache/beam/sdk/io/cassandra/CassandraIO.java
 ##
 @@ -318,15 +354,84 @@ private CassandraIO() {}
   return builder().setMapperFactoryFn(mapperFactory).build();
 }
 
+public Read withRingRange(RingRange ringRange) {
+  return withRingRange(ValueProvider.StaticValueProvider.of(ringRange));
+}
+
+public Read withRingRange(ValueProvider ringRange) {
+  return builder().setRingRange(ringRange).build();
+}
+
 @Override
 public PCollection expand(PBegin 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");
+  try (Cluster cluster =
+  getCluster(hosts(), port(), username(), password(), localDc(), 
consistencyLevel())) {
+Integer splitCount = cluster.getMetadata().getAllHosts().size();
+if (minNumberOfSplits() != null && minNumberOfSplits().get() != null) {
+  splitCount = minNumberOfSplits().get();
+}
+ReadAll readAll =
+CassandraIO.readAll()
+.withCoder(this.coder())
+.withConsistencyLevel(this.consistencyLevel())
+.withEntity(this.entity())
+.withHosts(this.hosts())
+.withKeyspace(this.keyspace())
+.withLocalDc(this.localDc())
+.withPort(this.port())
+.withPassword(this.password())
+.withQuery(this.query())
+.withTable(this.table())
+.withUsername(this.username())
+.withSplitCount(splitCount)
+.withMapperFactoryFn(this.mapperFactoryFn());
 
-  return input.apply(org.apache.beam.sdk.io.Read.from(new 
CassandraSource<>(this, null)));
+if (isMurmur3Partitioner(cluster)) {
+  LOG.info("Murmur3Partitioner detected, splitting");
+
+  List tokens =
+  cluster.getMetadata().getTokenRanges().stream()
+  .map(tokenRange -> new 
BigInteger(tokenRange.getEnd().getValue().toString()))
+  .collect(Collectors.toList());
+
+  SplitGenerator splitGenerator =
+  new SplitGenerator(cluster.getMetadata().getPartitioner());
+
+  List> splits =
 
 Review comment:
   Hmm, not sure what you mean, I do Create.of(splits)on line 410, do you want 
the Create.of() moved to line 406? 
 

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: 419058)
Time Spent: 11h 40m  (was: 11.5h)

> 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: 11h 40m
>  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 accounts = getRelevantAccounts();
> Set dateRange = generateDateRange("2018-01-01", "2019-01-01");
> PCollection tokens = generateTokens(accounts, dateRange); 
> {code}
>  
>  

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

2020-04-08 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on BEAM-9008:


Author: ASF GitHub Bot
Created on: 09/Apr/20 01:22
Start Date: 09/Apr/20 01:22
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_r405875425
 
 

 ##
 File path: 
sdks/java/io/cassandra/src/main/java/org/apache/beam/sdk/io/cassandra/CassandraIO.java
 ##
 @@ -318,15 +354,84 @@ private CassandraIO() {}
   return builder().setMapperFactoryFn(mapperFactory).build();
 }
 
+public Read withRingRange(RingRange ringRange) {
+  return withRingRange(ValueProvider.StaticValueProvider.of(ringRange));
+}
+
+public Read withRingRange(ValueProvider ringRange) {
+  return builder().setRingRange(ringRange).build();
+}
+
 @Override
 public PCollection expand(PBegin 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");
+  try (Cluster cluster =
+  getCluster(hosts(), port(), username(), password(), localDc(), 
consistencyLevel())) {
+Integer splitCount = cluster.getMetadata().getAllHosts().size();
+if (minNumberOfSplits() != null && minNumberOfSplits().get() != null) {
+  splitCount = minNumberOfSplits().get();
+}
+ReadAll readAll =
+CassandraIO.readAll()
+.withCoder(this.coder())
+.withConsistencyLevel(this.consistencyLevel())
+.withEntity(this.entity())
+.withHosts(this.hosts())
+.withKeyspace(this.keyspace())
+.withLocalDc(this.localDc())
+.withPort(this.port())
+.withPassword(this.password())
+.withQuery(this.query())
+.withTable(this.table())
+.withUsername(this.username())
+.withSplitCount(splitCount)
+.withMapperFactoryFn(this.mapperFactoryFn());
 
-  return input.apply(org.apache.beam.sdk.io.Read.from(new 
CassandraSource<>(this, null)));
+if (isMurmur3Partitioner(cluster)) {
+  LOG.info("Murmur3Partitioner detected, splitting");
+
+  List tokens =
+  cluster.getMetadata().getTokenRanges().stream()
+  .map(tokenRange -> new 
BigInteger(tokenRange.getEnd().getValue().toString()))
+  .collect(Collectors.toList());
+
+  SplitGenerator splitGenerator =
+  new SplitGenerator(cluster.getMetadata().getPartitioner());
+
+  List> splits =
 
 Review comment:
   Hmm, not sure what you mean, I do Create.of(splits)on line 410, do you want 
the Create.of() moved to line 406? 
 

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: 419056)
Time Spent: 11.5h  (was: 11h 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: 11.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 accounts = getRelevantAccounts();
> Set dateRange = generateDateRange("2018-01-01", "2019-01-01");
> PCollection tokens = generateTokens(accounts, dateRange); 
> {code}
>  
>  I 

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

2020-04-08 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on BEAM-9008:


Author: ASF GitHub Bot
Created on: 09/Apr/20 00:59
Start Date: 09/Apr/20 00:59
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_r405898356
 
 

 ##
 File path: 
sdks/java/io/cassandra/src/main/java/org/apache/beam/sdk/io/cassandra/CassandraConfig.java
 ##
 @@ -0,0 +1,88 @@
+/*
+ * 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.Session;
+import com.google.auto.value.AutoValue;
+import java.io.Serializable;
+import java.util.List;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.options.ValueProvider;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+
+@AutoValue
+abstract class CassandraConfig implements Serializable {
+  @Nullable
+  abstract ValueProvider> hosts();
+
+  @Nullable
+  abstract ValueProvider query();
 
 Review comment:
   This class isn't used anymore, I will delete this.  All of the relevant 
information for a connection is now on the Read being passed in to ReadAll. 
 

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: 419051)
Time Spent: 11h 20m  (was: 11h 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: 11h 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 accounts = getRelevantAccounts();
> Set dateRange = generateDateRange("2018-01-01", "2019-01-01");
> PCollection tokens = generateTokens(accounts, dateRange); 
> {code}
>  
>  I propose an additional _readAll()_ PTransform that can take a PCollection 
> of token ranges and can return a PCollection 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 

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

2020-04-08 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on BEAM-9008:


Author: ASF GitHub Bot
Created on: 09/Apr/20 00:49
Start Date: 09/Apr/20 00:49
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_r405895178
 
 

 ##
 File path: 
sdks/java/io/cassandra/src/main/java/org/apache/beam/sdk/io/cassandra/ReadFn.java
 ##
 @@ -0,0 +1,109 @@
+/*
+ * 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.io.cassandra.CassandraIO.Read;
+import org.apache.beam.sdk.io.cassandra.CassandraIO.ReadAll;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Joiner;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+class ReadFn extends DoFn, T> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ReadFn.class);
+
+  private transient Cluster cluster;
+
+  private transient Session session;
+
+  private final ReadAll readAll;
+
+  public ReadFn(ReadAll readAll) {
+this.readAll = readAll;
 
 Review comment:
   @iemejia the tests don't even finish running *locally* after ten minutes 
when running `CassandraIOTest` if we don't cache the connection, I don't think 
it will ever finish running on the Jenkins server :)  I have already tried 
doing it without establishing a connection in `setup`, it's not workable.  The 
CassandraIOTest returns over *200* partitions to return *10* rows.  That means 
it has to establish over 20 connections and tear them down per row... 
 

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: 419050)
Time Spent: 11h 10m  (was: 11h)

> 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: 11h 10m
>  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 accounts = getRelevantAccounts();
> Set dateRange = generateDateRange("2018-01-01", "2019-01-01");
> PCollection tokens = generateTokens(accounts, dateRange); 
> {code}
>  
>  I propose an additional _readAll()_ PTransform that can take a PCollection 
> of token ranges and can return a PCollection of what the query would 
> return. 
> *Question: How much code should be in common between both methods?* 
> Currently the read connector already groups all 

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

2020-04-08 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on BEAM-9008:


Author: ASF GitHub Bot
Created on: 09/Apr/20 00:47
Start Date: 09/Apr/20 00:47
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_r405895178
 
 

 ##
 File path: 
sdks/java/io/cassandra/src/main/java/org/apache/beam/sdk/io/cassandra/ReadFn.java
 ##
 @@ -0,0 +1,109 @@
+/*
+ * 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.io.cassandra.CassandraIO.Read;
+import org.apache.beam.sdk.io.cassandra.CassandraIO.ReadAll;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Joiner;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+class ReadFn extends DoFn, T> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ReadFn.class);
+
+  private transient Cluster cluster;
+
+  private transient Session session;
+
+  private final ReadAll readAll;
+
+  public ReadFn(ReadAll readAll) {
+this.readAll = readAll;
 
 Review comment:
   @iemejia the tests don't even finish running *locally* after ten minutes 
when running `CassandraIOTest` if we don't cache the connection.  I have 
already tried doing it without establishing a connection in `setup`, it's not 
workable.  The CassandraIOTest returns over *200* partitions to return *10* 
rows.  That means it has to establish over 20 connections and tear them down 
per row... 
 

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: 419049)
Time Spent: 11h  (was: 10h 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: 11h
>  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 accounts = getRelevantAccounts();
> Set dateRange = generateDateRange("2018-01-01", "2019-01-01");
> PCollection tokens = generateTokens(accounts, dateRange); 
> {code}
>  
>  I propose an additional _readAll()_ PTransform that can take a PCollection 
> of token ranges and can return a PCollection 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 

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

2020-04-08 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on BEAM-9008:


Author: ASF GitHub Bot
Created on: 09/Apr/20 00:44
Start Date: 09/Apr/20 00:44
Worklog Time Spent: 10m 
  Work Description: iemejia commented on pull request #10546: [BEAM-9008] 
Add CassandraIO readAll method
URL: https://github.com/apache/beam/pull/10546#discussion_r405894039
 
 

 ##
 File path: 
sdks/java/io/cassandra/src/main/java/org/apache/beam/sdk/io/cassandra/ReadFn.java
 ##
 @@ -0,0 +1,109 @@
+/*
+ * 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.io.cassandra.CassandraIO.Read;
+import org.apache.beam.sdk.io.cassandra.CassandraIO.ReadAll;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Joiner;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+class ReadFn extends DoFn, T> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ReadFn.class);
+
+  private transient Cluster cluster;
+
+  private transient Session session;
+
+  private final ReadAll readAll;
+
+  public ReadFn(ReadAll readAll) {
+this.readAll = readAll;
 
 Review comment:
   Caching connections is something we can do in a subsequent PR. Let's get 
first right the API and then optimize that part. We can cache connections at 
the class level on the `ReadFn` by creating some size limited pool of 
connections that we identify by some combination of parameters and we reuse if 
possible.
 

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: 419047)
Time Spent: 10h 40m  (was: 10.5h)

> 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: 10h 40m
>  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 accounts = getRelevantAccounts();
> Set dateRange = generateDateRange("2018-01-01", "2019-01-01");
> PCollection tokens = generateTokens(accounts, dateRange); 
> {code}
>  
>  I propose an additional _readAll()_ PTransform that can take a PCollection 
> of token ranges and can return a PCollection 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 

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

2020-04-08 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on BEAM-9008:


Author: ASF GitHub Bot
Created on: 09/Apr/20 00:44
Start Date: 09/Apr/20 00:44
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-611265311
 
 
   Just for ref I opened the PR for SolrIO #11357 that introduces ReadAll 
there. Maybe that helps make what I suggested above more clear.
 

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: 419048)
Time Spent: 10h 50m  (was: 10h 40m)

> 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: 10h 50m
>  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 accounts = getRelevantAccounts();
> Set dateRange = generateDateRange("2018-01-01", "2019-01-01");
> PCollection tokens = generateTokens(accounts, dateRange); 
> {code}
>  
>  I propose an additional _readAll()_ PTransform that can take a PCollection 
> of token ranges and can return a PCollection 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)


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

2020-04-08 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on BEAM-9008:


Author: ASF GitHub Bot
Created on: 09/Apr/20 00:43
Start Date: 09/Apr/20 00:43
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_r405874887
 
 

 ##
 File path: 
sdks/java/io/cassandra/src/main/java/org/apache/beam/sdk/io/cassandra/ReadFn.java
 ##
 @@ -0,0 +1,109 @@
+/*
+ * 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.io.cassandra.CassandraIO.Read;
+import org.apache.beam.sdk.io.cassandra.CassandraIO.ReadAll;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Joiner;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+class ReadFn extends DoFn, T> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ReadFn.class);
+
+  private transient Cluster cluster;
+
+  private transient Session session;
+
+  private final ReadAll readAll;
+
+  public ReadFn(ReadAll readAll) {
+this.readAll = readAll;
 
 Review comment:
   I'm fairly certain we need the ReadAll passed in here so we have the 
information on how to construct a connection on `setup`. 
   
   If you want I can spend more time getting performance data, but without 
caching the connection on setup I saw more than two orders of magnitude 
slowdown on just the CassandraIOTest.  
   
   
 

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: 419045)
Time Spent: 10h 20m  (was: 10h 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: 10h 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 accounts = getRelevantAccounts();
> Set dateRange = generateDateRange("2018-01-01", "2019-01-01");
> PCollection tokens = generateTokens(accounts, dateRange); 
> {code}
>  
>  I propose an additional _readAll()_ PTransform that can take a PCollection 
> of token ranges and can return a PCollection 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' 

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

2020-04-08 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on BEAM-9008:


Author: ASF GitHub Bot
Created on: 09/Apr/20 00:43
Start Date: 09/Apr/20 00:43
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_r405888316
 
 

 ##
 File path: 
sdks/java/io/cassandra/src/main/java/org/apache/beam/sdk/io/cassandra/ReadFn.java
 ##
 @@ -0,0 +1,109 @@
+/*
+ * 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.io.cassandra.CassandraIO.Read;
+import org.apache.beam.sdk.io.cassandra.CassandraIO.ReadAll;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Joiner;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+class ReadFn extends DoFn, T> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ReadFn.class);
+
+  private transient Cluster cluster;
+
+  private transient Session session;
+
+  private final ReadAll readAll;
+
+  public ReadFn(ReadAll readAll) {
+this.readAll = readAll;
 
 Review comment:
   If you want I can spend more time getting performance data, but without 
caching the connection on setup I saw more than two orders of magnitude 
slowdown on just the `CassandraIOTest`
 

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: 419046)
Time Spent: 10.5h  (was: 10h 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: 10.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 accounts = getRelevantAccounts();
> Set dateRange = generateDateRange("2018-01-01", "2019-01-01");
> PCollection tokens = generateTokens(accounts, dateRange); 
> {code}
>  
>  I propose an additional _readAll()_ PTransform that can take a PCollection 
> of token ranges and can return a PCollection 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 

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

2020-04-08 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on BEAM-9008:


Author: ASF GitHub Bot
Created on: 09/Apr/20 00:41
Start Date: 09/Apr/20 00:41
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-611265311
 
 
   Just for ref I opened the PR for SolrIO #11357 that introduces ReadAll there.
 

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: 419044)
Time Spent: 10h 10m  (was: 10h)

> 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: 10h 10m
>  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 accounts = getRelevantAccounts();
> Set dateRange = generateDateRange("2018-01-01", "2019-01-01");
> PCollection tokens = generateTokens(accounts, dateRange); 
> {code}
>  
>  I propose an additional _readAll()_ PTransform that can take a PCollection 
> of token ranges and can return a PCollection 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)


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

2020-04-08 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on BEAM-9008:


Author: ASF GitHub Bot
Created on: 09/Apr/20 00:21
Start Date: 09/Apr/20 00:21
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_r405888316
 
 

 ##
 File path: 
sdks/java/io/cassandra/src/main/java/org/apache/beam/sdk/io/cassandra/ReadFn.java
 ##
 @@ -0,0 +1,109 @@
+/*
+ * 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.io.cassandra.CassandraIO.Read;
+import org.apache.beam.sdk.io.cassandra.CassandraIO.ReadAll;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Joiner;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+class ReadFn extends DoFn, T> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ReadFn.class);
+
+  private transient Cluster cluster;
+
+  private transient Session session;
+
+  private final ReadAll readAll;
+
+  public ReadFn(ReadAll readAll) {
+this.readAll = readAll;
 
 Review comment:
   If you want I can spend more time getting performance data, but without 
caching the connection on setup I saw more than two orders of magnitude 
slowdown on just the `CassandraIOTest`
 

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: 419023)
Time Spent: 10h  (was: 9h 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: 10h
>  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 accounts = getRelevantAccounts();
> Set dateRange = generateDateRange("2018-01-01", "2019-01-01");
> PCollection tokens = generateTokens(accounts, dateRange); 
> {code}
>  
>  I propose an additional _readAll()_ PTransform that can take a PCollection 
> of token ranges and can return a PCollection 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 

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

2020-04-08 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on BEAM-9008:


Author: ASF GitHub Bot
Created on: 09/Apr/20 00:02
Start Date: 09/Apr/20 00:02
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 extends 
PTransform>, PCollection> {
+
+@Nullable
+abstract ValueProvider> hosts();
+
+@Nullable
+abstract ValueProvider query();
+
+@Nullable
+abstract ValueProvider port();
+
+@Nullable
+abstract ValueProvider keyspace();
+
+@Nullable
+abstract ValueProvider table();
+
+@Nullable
+abstract Class entity();
+
+@Nullable
+abstract Coder coder();
+
+@Nullable
+abstract ValueProvider username();
+
+@Nullable
+abstract ValueProvider password();
+
+@Nullable
+abstract ValueProvider localDc();
+
+@Nullable
+abstract ValueProvider consistencyLevel();
+
+@Nullable
+abstract ValueProvider splitCount();
+
+@Nullable
+abstract SerializableFunction mapperFactoryFn();
+
+@Nullable
+abstract SerializableFunction groupingFn();
+
+abstract Builder builder();
+
+/** Specify the hosts of the Apache Cassandra instances. */
+public ReadAll withHosts(List 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 withHosts(ValueProvider> hosts) {
+  return builder().setHosts(hosts).build();
+}
+
+/** Specify the port number of the Apache Cassandra instances. */
+public ReadAll 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 withPort(ValueProvider port) {
+  return builder().setPort(port).build();
+}
+
+/** Specify the Cassandra keyspace where to read data. */
+public ReadAll 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 withKeyspace(ValueProvider keyspace) {
+  return builder().setKeyspace(keyspace).build();
+}
+
+/** Specify the Cassandra table where to read data. */
+public ReadAll 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 withTable(ValueProvider table) {
+  return builder().setTable(table).build();
+}
+
+/** Specify the query to read data. */
+public ReadAll 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 withQuery(ValueProvider 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 withEntity(Class 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 withCoder(Coder coder) {
+  checkArgument(coder != null, "coder can not be null");
+  return builder().setCoder(coder).build();
+}
+
+/** Specify the username for authentication. */
+public ReadAll withUsername(String username) {
+  checkArgument(username != null, "username can not be null");
+  return 

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

2020-04-08 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on BEAM-9008:


Author: ASF GitHub Bot
Created on: 08/Apr/20 23:57
Start Date: 08/Apr/20 23:57
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 extends 
PTransform>, PCollection> {
+
+@Nullable
+abstract ValueProvider> hosts();
+
+@Nullable
+abstract ValueProvider query();
+
+@Nullable
+abstract ValueProvider port();
+
+@Nullable
+abstract ValueProvider keyspace();
+
+@Nullable
+abstract ValueProvider table();
+
+@Nullable
+abstract Class entity();
+
+@Nullable
+abstract Coder coder();
+
+@Nullable
+abstract ValueProvider username();
+
+@Nullable
+abstract ValueProvider password();
+
+@Nullable
+abstract ValueProvider localDc();
+
+@Nullable
+abstract ValueProvider consistencyLevel();
+
+@Nullable
+abstract ValueProvider splitCount();
+
+@Nullable
+abstract SerializableFunction mapperFactoryFn();
+
+@Nullable
+abstract SerializableFunction groupingFn();
+
+abstract Builder builder();
+
+/** Specify the hosts of the Apache Cassandra instances. */
+public ReadAll withHosts(List 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 withHosts(ValueProvider> hosts) {
+  return builder().setHosts(hosts).build();
+}
+
+/** Specify the port number of the Apache Cassandra instances. */
+public ReadAll 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 withPort(ValueProvider port) {
+  return builder().setPort(port).build();
+}
+
+/** Specify the Cassandra keyspace where to read data. */
+public ReadAll 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 withKeyspace(ValueProvider keyspace) {
+  return builder().setKeyspace(keyspace).build();
+}
+
+/** Specify the Cassandra table where to read data. */
+public ReadAll 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 withTable(ValueProvider table) {
+  return builder().setTable(table).build();
+}
+
+/** Specify the query to read data. */
+public ReadAll 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 withQuery(ValueProvider 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 withEntity(Class 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 withCoder(Coder coder) {
+  checkArgument(coder != null, "coder can not be null");
+  return builder().setCoder(coder).build();
+}
+
+/** Specify the username for authentication. */
+public ReadAll withUsername(String username) {
+  checkArgument(username != null, "username can not be null");
+  return 

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

2020-04-08 Thread ASF GitHub Bot (Jira)


 [ 
https://issues.apache.org/jira/browse/BEAM-9008?focusedWorklogId=419003=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 extends 
PTransform>, PCollection> {
+
+@Nullable
+abstract ValueProvider> hosts();
+
+@Nullable
+abstract ValueProvider query();
+
+@Nullable
+abstract ValueProvider port();
+
+@Nullable
+abstract ValueProvider keyspace();
+
+@Nullable
+abstract ValueProvider table();
+
+@Nullable
+abstract Class entity();
+
+@Nullable
+abstract Coder coder();
+
+@Nullable
+abstract ValueProvider username();
+
+@Nullable
+abstract ValueProvider password();
+
+@Nullable
+abstract ValueProvider localDc();
+
+@Nullable
+abstract ValueProvider consistencyLevel();
+
+@Nullable
+abstract ValueProvider splitCount();
+
+@Nullable
+abstract SerializableFunction mapperFactoryFn();
+
+@Nullable
+abstract SerializableFunction groupingFn();
+
+abstract Builder builder();
+
+/** Specify the hosts of the Apache Cassandra instances. */
+public ReadAll withHosts(List 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 withHosts(ValueProvider> hosts) {
+  return builder().setHosts(hosts).build();
+}
+
+/** Specify the port number of the Apache Cassandra instances. */
+public ReadAll 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 withPort(ValueProvider port) {
+  return builder().setPort(port).build();
+}
+
+/** Specify the Cassandra keyspace where to read data. */
+public ReadAll 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 withKeyspace(ValueProvider keyspace) {
+  return builder().setKeyspace(keyspace).build();
+}
+
+/** Specify the Cassandra table where to read data. */
+public ReadAll 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 withTable(ValueProvider table) {
+  return builder().setTable(table).build();
+}
+
+/** Specify the query to read data. */
+public ReadAll 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 withQuery(ValueProvider 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 withEntity(Class 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 withCoder(Coder coder) {
+  checkArgument(coder != null, "coder can not be null");
+  return builder().setCoder(coder).build();
+}
+
+/** Specify the username for authentication. */
+public ReadAll withUsername(String username) {
+  checkArgument(username != null, "username can not be null");
+  return 

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

2020-04-08 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on BEAM-9008:


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

 ##
 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 extends 
PTransform>, PCollection> {
+
+@Nullable
+abstract ValueProvider> hosts();
 
 Review comment:
   For ReadAll to be useable we need to establish connection information in the 
`setup` phase of the QueryFn, so we'll need all this before a `Read` comes 
in.   I'd also like to keep the API between ReadAll and Read similar so it's 
easy for the end user to switch a `Read` with a `ReadAll`, and just 
operating on a different type. 
 

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: 418997)
Time Spent: 9h 20m  (was: 9h 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: 9h 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 accounts = getRelevantAccounts();
> Set dateRange = generateDateRange("2018-01-01", "2019-01-01");
> PCollection tokens = generateTokens(accounts, dateRange); 
> {code}
>  
>  I propose an additional _readAll()_ PTransform that can take a PCollection 
> of token ranges and can return a PCollection 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)


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

2020-04-08 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on BEAM-9008:


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

 ##
 File path: 
sdks/java/io/cassandra/src/main/java/org/apache/beam/sdk/io/cassandra/ReadFn.java
 ##
 @@ -0,0 +1,109 @@
+/*
+ * 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.io.cassandra.CassandraIO.Read;
+import org.apache.beam.sdk.io.cassandra.CassandraIO.ReadAll;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Joiner;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+class ReadFn extends DoFn, T> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ReadFn.class);
+
+  private transient Cluster cluster;
+
+  private transient Session session;
+
+  private final ReadAll readAll;
+
+  public ReadFn(ReadAll readAll) {
+this.readAll = readAll;
 
 Review comment:
   You need this here so we can set up a connection with Setup, this won't work 
without 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:
us...@infra.apache.org


Issue Time Tracking
---

Worklog Id: (was: 418994)
Time Spent: 9h 10m  (was: 9h)

> 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: 9h 10m
>  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 accounts = getRelevantAccounts();
> Set dateRange = generateDateRange("2018-01-01", "2019-01-01");
> PCollection tokens = generateTokens(accounts, dateRange); 
> {code}
>  
>  I propose an additional _readAll()_ PTransform that can take a PCollection 
> of token ranges and can return a PCollection 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 

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

2020-04-08 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on BEAM-9008:


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

 ##
 File path: 
sdks/java/io/cassandra/src/main/java/org/apache/beam/sdk/io/cassandra/CassandraIO.java
 ##
 @@ -318,15 +354,84 @@ private CassandraIO() {}
   return builder().setMapperFactoryFn(mapperFactory).build();
 }
 
+public Read withRingRange(RingRange ringRange) {
+  return withRingRange(ValueProvider.StaticValueProvider.of(ringRange));
+}
+
+public Read withRingRange(ValueProvider ringRange) {
+  return builder().setRingRange(ringRange).build();
+}
+
 @Override
 public PCollection expand(PBegin 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");
+  try (Cluster cluster =
+  getCluster(hosts(), port(), username(), password(), localDc(), 
consistencyLevel())) {
+Integer splitCount = cluster.getMetadata().getAllHosts().size();
+if (minNumberOfSplits() != null && minNumberOfSplits().get() != null) {
+  splitCount = minNumberOfSplits().get();
+}
+ReadAll readAll =
+CassandraIO.readAll()
+.withCoder(this.coder())
+.withConsistencyLevel(this.consistencyLevel())
+.withEntity(this.entity())
+.withHosts(this.hosts())
+.withKeyspace(this.keyspace())
+.withLocalDc(this.localDc())
+.withPort(this.port())
+.withPassword(this.password())
+.withQuery(this.query())
+.withTable(this.table())
+.withUsername(this.username())
+.withSplitCount(splitCount)
+.withMapperFactoryFn(this.mapperFactoryFn());
 
-  return input.apply(org.apache.beam.sdk.io.Read.from(new 
CassandraSource<>(this, null)));
+if (isMurmur3Partitioner(cluster)) {
+  LOG.info("Murmur3Partitioner detected, splitting");
+
+  List tokens =
+  cluster.getMetadata().getTokenRanges().stream()
+  .map(tokenRange -> new 
BigInteger(tokenRange.getEnd().getValue().toString()))
+  .collect(Collectors.toList());
+
+  SplitGenerator splitGenerator =
+  new SplitGenerator(cluster.getMetadata().getPartitioner());
+
+  List> splits =
 
 Review comment:
   This is more or less leaving the splitting logic the same as it was before, 
is there a reason we need to change it now? 
 

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: 418990)
Time Spent: 9h  (was: 8h 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: 9h
>  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 accounts = getRelevantAccounts();
> Set dateRange = generateDateRange("2018-01-01", "2019-01-01");
> PCollection tokens = generateTokens(accounts, dateRange); 
> {code}
>  
>  

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

2020-04-08 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on BEAM-9008:


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

 ##
 File path: 
sdks/java/io/cassandra/src/main/java/org/apache/beam/sdk/io/cassandra/ReadFn.java
 ##
 @@ -0,0 +1,109 @@
+/*
+ * 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.io.cassandra.CassandraIO.Read;
+import org.apache.beam.sdk.io.cassandra.CassandraIO.ReadAll;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Joiner;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+class ReadFn extends DoFn, T> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ReadFn.class);
+
+  private transient Cluster cluster;
+
+  private transient Session session;
+
+  private final ReadAll readAll;
+
+  public ReadFn(ReadAll readAll) {
+this.readAll = readAll;
 
 Review comment:
   You need this here so we can set up a connection with Setup, this won't work 
without 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:
us...@infra.apache.org


Issue Time Tracking
---

Worklog Id: (was: 418989)
Time Spent: 8h 50m  (was: 8h 40m)

> 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: 8h 50m
>  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 accounts = getRelevantAccounts();
> Set dateRange = generateDateRange("2018-01-01", "2019-01-01");
> PCollection tokens = generateTokens(accounts, dateRange); 
> {code}
>  
>  I propose an additional _readAll()_ PTransform that can take a PCollection 
> of token ranges and can return a PCollection 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 

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

2020-04-08 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on BEAM-9008:


Author: ASF GitHub Bot
Created on: 08/Apr/20 23:29
Start Date: 08/Apr/20 23:29
Worklog Time Spent: 10m 
  Work Description: iemejia commented on pull request #10546: [BEAM-9008] 
Add CassandraIO readAll method
URL: https://github.com/apache/beam/pull/10546#discussion_r405869911
 
 

 ##
 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 extends 
PTransform>, PCollection> {
+
+@Nullable
+abstract ValueProvider> hosts();
+
+@Nullable
+abstract ValueProvider query();
+
+@Nullable
+abstract ValueProvider port();
+
+@Nullable
+abstract ValueProvider keyspace();
+
+@Nullable
+abstract ValueProvider table();
+
+@Nullable
+abstract Class entity();
+
+@Nullable
+abstract Coder coder();
+
+@Nullable
+abstract ValueProvider username();
+
+@Nullable
+abstract ValueProvider password();
+
+@Nullable
+abstract ValueProvider localDc();
+
+@Nullable
+abstract ValueProvider consistencyLevel();
+
+@Nullable
+abstract ValueProvider splitCount();
+
+@Nullable
+abstract SerializableFunction mapperFactoryFn();
+
+@Nullable
+abstract SerializableFunction groupingFn();
+
+abstract Builder builder();
+
+/** Specify the hosts of the Apache Cassandra instances. */
+public ReadAll withHosts(List 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 withHosts(ValueProvider> hosts) {
+  return builder().setHosts(hosts).build();
+}
+
+/** Specify the port number of the Apache Cassandra instances. */
+public ReadAll 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 withPort(ValueProvider port) {
+  return builder().setPort(port).build();
+}
+
+/** Specify the Cassandra keyspace where to read data. */
+public ReadAll 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 withKeyspace(ValueProvider keyspace) {
+  return builder().setKeyspace(keyspace).build();
+}
+
+/** Specify the Cassandra table where to read data. */
+public ReadAll 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 withTable(ValueProvider table) {
+  return builder().setTable(table).build();
+}
+
+/** Specify the query to read data. */
+public ReadAll 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 withQuery(ValueProvider 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 withEntity(Class 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 withCoder(Coder coder) {
+  checkArgument(coder != null, "coder can not be null");
+  return builder().setCoder(coder).build();
+}
+
+/** Specify the username for authentication. */
+public ReadAll withUsername(String username) {
+  checkArgument(username != null, "username can not be null");
+  return 

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

2020-04-08 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on BEAM-9008:


Author: ASF GitHub Bot
Created on: 08/Apr/20 23:29
Start Date: 08/Apr/20 23:29
Worklog Time Spent: 10m 
  Work Description: iemejia commented on pull request #10546: [BEAM-9008] 
Add CassandraIO readAll method
URL: https://github.com/apache/beam/pull/10546#discussion_r405864840
 
 

 ##
 File path: 
sdks/java/io/cassandra/src/main/java/org/apache/beam/sdk/io/cassandra/CassandraIO.java
 ##
 @@ -318,15 +354,84 @@ private CassandraIO() {}
   return builder().setMapperFactoryFn(mapperFactory).build();
 }
 
+public Read withRingRange(RingRange ringRange) {
+  return withRingRange(ValueProvider.StaticValueProvider.of(ringRange));
+}
+
+public Read withRingRange(ValueProvider ringRange) {
+  return builder().setRingRange(ringRange).build();
+}
+
 @Override
 public PCollection expand(PBegin 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");
+  try (Cluster cluster =
+  getCluster(hosts(), port(), username(), password(), localDc(), 
consistencyLevel())) {
+Integer splitCount = cluster.getMetadata().getAllHosts().size();
+if (minNumberOfSplits() != null && minNumberOfSplits().get() != null) {
+  splitCount = minNumberOfSplits().get();
+}
+ReadAll readAll =
+CassandraIO.readAll()
+.withCoder(this.coder())
+.withConsistencyLevel(this.consistencyLevel())
+.withEntity(this.entity())
+.withHosts(this.hosts())
+.withKeyspace(this.keyspace())
+.withLocalDc(this.localDc())
+.withPort(this.port())
+.withPassword(this.password())
+.withQuery(this.query())
+.withTable(this.table())
+.withUsername(this.username())
+.withSplitCount(splitCount)
+.withMapperFactoryFn(this.mapperFactoryFn());
 
-  return input.apply(org.apache.beam.sdk.io.Read.from(new 
CassandraSource<>(this, null)));
+if (isMurmur3Partitioner(cluster)) {
+  LOG.info("Murmur3Partitioner detected, splitting");
+
+  List tokens =
+  cluster.getMetadata().getTokenRanges().stream()
+  .map(tokenRange -> new 
BigInteger(tokenRange.getEnd().getValue().toString()))
+  .collect(Collectors.toList());
+
+  SplitGenerator splitGenerator =
+  new SplitGenerator(cluster.getMetadata().getPartitioner());
+
+  List> splits =
 
 Review comment:
   We should produce a PCollection> here instead and this splitting 
logic should be moved into the SplitFn function.
 

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: 418979)
Time Spent: 8h 10m  (was: 8h)

> 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: 8h 10m
>  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 accounts = getRelevantAccounts();
> Set dateRange = generateDateRange("2018-01-01", "2019-01-01");
> PCollection tokens = generateTokens(accounts, dateRange); 
> {code}
>  
>  I 

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

2020-04-08 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on BEAM-9008:


Author: ASF GitHub Bot
Created on: 08/Apr/20 23:29
Start Date: 08/Apr/20 23:29
Worklog Time Spent: 10m 
  Work Description: iemejia commented on pull request #10546: [BEAM-9008] 
Add CassandraIO readAll method
URL: https://github.com/apache/beam/pull/10546#discussion_r405870149
 
 

 ##
 File path: 
sdks/java/io/cassandra/src/main/java/org/apache/beam/sdk/io/cassandra/ReadFn.java
 ##
 @@ -0,0 +1,109 @@
+/*
+ * 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.io.cassandra.CassandraIO.Read;
+import org.apache.beam.sdk.io.cassandra.CassandraIO.ReadAll;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Joiner;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+class ReadFn extends DoFn, T> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ReadFn.class);
+
+  private transient Cluster cluster;
+
+  private transient Session session;
+
+  private final ReadAll readAll;
+
+  public ReadFn(ReadAll readAll) {
+this.readAll = readAll;
 
 Review comment:
   You don't need this ReadAll ref here, Read is enough, ReadAll is the one 
that delegates to a normal Read.
 

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: 418984)

> 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: 8h 40m
>  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 accounts = getRelevantAccounts();
> Set dateRange = generateDateRange("2018-01-01", "2019-01-01");
> PCollection tokens = generateTokens(accounts, dateRange); 
> {code}
>  
>  I propose an additional _readAll()_ PTransform that can take a PCollection 
> of token ranges and can return a PCollection 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
>  * 

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

2020-04-08 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on BEAM-9008:


Author: ASF GitHub Bot
Created on: 08/Apr/20 23:29
Start Date: 08/Apr/20 23:29
Worklog Time Spent: 10m 
  Work Description: iemejia commented on pull request #10546: [BEAM-9008] 
Add CassandraIO readAll method
URL: https://github.com/apache/beam/pull/10546#discussion_r405866030
 
 

 ##
 File path: 
sdks/java/io/cassandra/src/main/java/org/apache/beam/sdk/io/cassandra/CassandraIO.java
 ##
 @@ -318,15 +354,84 @@ private CassandraIO() {}
   return builder().setMapperFactoryFn(mapperFactory).build();
 }
 
+public Read withRingRange(RingRange ringRange) {
+  return withRingRange(ValueProvider.StaticValueProvider.of(ringRange));
+}
+
+public Read withRingRange(ValueProvider ringRange) {
+  return builder().setRingRange(ringRange).build();
+}
+
 @Override
 public PCollection expand(PBegin 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");
+  try (Cluster cluster =
+  getCluster(hosts(), port(), username(), password(), localDc(), 
consistencyLevel())) {
+Integer splitCount = cluster.getMetadata().getAllHosts().size();
+if (minNumberOfSplits() != null && minNumberOfSplits().get() != null) {
+  splitCount = minNumberOfSplits().get();
+}
+ReadAll readAll =
+CassandraIO.readAll()
+.withCoder(this.coder())
+.withConsistencyLevel(this.consistencyLevel())
+.withEntity(this.entity())
+.withHosts(this.hosts())
+.withKeyspace(this.keyspace())
+.withLocalDc(this.localDc())
+.withPort(this.port())
+.withPassword(this.password())
+.withQuery(this.query())
+.withTable(this.table())
+.withUsername(this.username())
+.withSplitCount(splitCount)
+.withMapperFactoryFn(this.mapperFactoryFn());
 
-  return input.apply(org.apache.beam.sdk.io.Read.from(new 
CassandraSource<>(this, null)));
+if (isMurmur3Partitioner(cluster)) {
+  LOG.info("Murmur3Partitioner detected, splitting");
+
+  List tokens =
+  cluster.getMetadata().getTokenRanges().stream()
+  .map(tokenRange -> new 
BigInteger(tokenRange.getEnd().getValue().toString()))
+  .collect(Collectors.toList());
+
+  SplitGenerator splitGenerator =
+  new SplitGenerator(cluster.getMetadata().getPartitioner());
+
+  List> splits =
+  splitGenerator.generateSplits(splitCount, tokens).stream()
+  .map(rr -> CassandraIO.read().withRingRange(rr))
+  .collect(Collectors.toList());
+
+  return input.apply("Creating splits", 
Create.of(splits)).apply("readAll", readAll);
+
+} else {
+  LOG.warn(
+  "Only Murmur3Partitioner is supported for splitting, using an 
unique source for "
+  + "the read");
+  String partitioner = cluster.getMetadata().getPartitioner();
+  RingRange totalRingRange =
+  new RingRange(
+  SplitGenerator.getRangeMin(partitioner), 
SplitGenerator.getRangeMax(partitioner));
+  return input
+  
.apply(Create.of(CassandraIO.read().withRingRange(totalRingRange)))
+  .apply(readAll)
+  .setCoder(coder());
+}
+  }
+}
+
+private static class SplitFn extends DoFn, Read> {
+  @ProcessElement
+  public void processElement(@Element List input, 
OutputReceiver> output) {
+for (RingRange rr : input) {
 
 Review comment:
   The split logic that is above should be here.
 

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: 418983)
Time Spent: 8h 40m  (was: 8.5h)

> Add readAll() method to CassandraIO
> ---
>
> Key: BEAM-9008
> URL: https://issues.apache.org/jira/browse/BEAM-9008
> Project: Beam
>  Issue Type: New 

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

2020-04-08 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on BEAM-9008:


Author: ASF GitHub Bot
Created on: 08/Apr/20 23:29
Start Date: 08/Apr/20 23:29
Worklog Time Spent: 10m 
  Work Description: iemejia commented on pull request #10546: [BEAM-9008] 
Add CassandraIO readAll method
URL: https://github.com/apache/beam/pull/10546#discussion_r405867511
 
 

 ##
 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 extends 
PTransform>, PCollection> {
+
+@Nullable
+abstract ValueProvider> hosts();
 
 Review comment:
   This transform should not have any attribute / with method, those come from 
the Read in the input PCollection.
 

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: 418980)
Time Spent: 8h 20m  (was: 8h 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: 8h 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 accounts = getRelevantAccounts();
> Set dateRange = generateDateRange("2018-01-01", "2019-01-01");
> PCollection tokens = generateTokens(accounts, dateRange); 
> {code}
>  
>  I propose an additional _readAll()_ PTransform that can take a PCollection 
> of token ranges and can return a PCollection 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)


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

2020-04-08 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on BEAM-9008:


Author: ASF GitHub Bot
Created on: 08/Apr/20 23:29
Start Date: 08/Apr/20 23:29
Worklog Time Spent: 10m 
  Work Description: iemejia commented on pull request #10546: [BEAM-9008] 
Add CassandraIO readAll method
URL: https://github.com/apache/beam/pull/10546#discussion_r405865873
 
 

 ##
 File path: 
sdks/java/io/cassandra/src/main/java/org/apache/beam/sdk/io/cassandra/CassandraIO.java
 ##
 @@ -318,15 +354,84 @@ private CassandraIO() {}
   return builder().setMapperFactoryFn(mapperFactory).build();
 }
 
+public Read withRingRange(RingRange ringRange) {
+  return withRingRange(ValueProvider.StaticValueProvider.of(ringRange));
+}
+
+public Read withRingRange(ValueProvider ringRange) {
+  return builder().setRingRange(ringRange).build();
+}
+
 @Override
 public PCollection expand(PBegin 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");
+  try (Cluster cluster =
+  getCluster(hosts(), port(), username(), password(), localDc(), 
consistencyLevel())) {
+Integer splitCount = cluster.getMetadata().getAllHosts().size();
+if (minNumberOfSplits() != null && minNumberOfSplits().get() != null) {
+  splitCount = minNumberOfSplits().get();
+}
+ReadAll readAll =
+CassandraIO.readAll()
+.withCoder(this.coder())
+.withConsistencyLevel(this.consistencyLevel())
+.withEntity(this.entity())
+.withHosts(this.hosts())
+.withKeyspace(this.keyspace())
+.withLocalDc(this.localDc())
+.withPort(this.port())
+.withPassword(this.password())
+.withQuery(this.query())
+.withTable(this.table())
+.withUsername(this.username())
+.withSplitCount(splitCount)
+.withMapperFactoryFn(this.mapperFactoryFn());
 
-  return input.apply(org.apache.beam.sdk.io.Read.from(new 
CassandraSource<>(this, null)));
+if (isMurmur3Partitioner(cluster)) {
+  LOG.info("Murmur3Partitioner detected, splitting");
+
+  List tokens =
+  cluster.getMetadata().getTokenRanges().stream()
+  .map(tokenRange -> new 
BigInteger(tokenRange.getEnd().getValue().toString()))
+  .collect(Collectors.toList());
+
+  SplitGenerator splitGenerator =
+  new SplitGenerator(cluster.getMetadata().getPartitioner());
+
+  List> splits =
+  splitGenerator.generateSplits(splitCount, tokens).stream()
+  .map(rr -> CassandraIO.read().withRingRange(rr))
+  .collect(Collectors.toList());
+
+  return input.apply("Creating splits", 
Create.of(splits)).apply("readAll", readAll);
+
+} else {
+  LOG.warn(
+  "Only Murmur3Partitioner is supported for splitting, using an 
unique source for "
+  + "the read");
+  String partitioner = cluster.getMetadata().getPartitioner();
+  RingRange totalRingRange =
+  new RingRange(
+  SplitGenerator.getRangeMin(partitioner), 
SplitGenerator.getRangeMax(partitioner));
+  return input
+  
.apply(Create.of(CassandraIO.read().withRingRange(totalRingRange)))
+  .apply(readAll)
+  .setCoder(coder());
+}
+  }
+}
+
+private static class SplitFn extends DoFn, Read> {
 
 Review comment:
   `private static class SplitFn extends DoFn, Read>`
   Notice that in the second Read you should setup the `RingRange` so each 
individual Read can then be Read by a DoFn that knows how to read `Read` 
specifications with a RingRange.
 

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: 418982)
Time Spent: 8.5h  (was: 8h 20m)

> Add readAll() method to CassandraIO
> ---
>
> Key: BEAM-9008
> URL: https://issues.apache.org/jira/browse/BEAM-9008
> 

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

2020-04-07 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on BEAM-9008:


Author: ASF GitHub Bot
Created on: 07/Apr/20 07:33
Start Date: 07/Apr/20 07:33
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-610226254
 
 
   Yes looks like a flaky test there is already a ticket tracking it 
https://issues.apache.org/jira/browse/BEAM-9534
 

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: 417506)
Time Spent: 8h  (was: 7h 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: 8h
>  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 accounts = getRelevantAccounts();
> Set dateRange = generateDateRange("2018-01-01", "2019-01-01");
> PCollection tokens = generateTokens(accounts, dateRange); 
> {code}
>  
>  I propose an additional _readAll()_ PTransform that can take a PCollection 
> of token ranges and can return a PCollection 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)


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

2020-04-06 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on BEAM-9008:


Author: ASF GitHub Bot
Created on: 07/Apr/20 00:43
Start Date: 07/Apr/20 00:43
Worklog Time Spent: 10m 
  Work Description: vmarquez commented on issue #10546: [BEAM-9008] Add 
CassandraIO readAll method
URL: https://github.com/apache/beam/pull/10546#issuecomment-610109654
 
 
   @iemejia is that a spurious failure or did something I do break the Flink 
test?  I tested locally and all seems to work... LMK if you need anything from 
me. 
 

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: 417341)
Time Spent: 7h 50m  (was: 7h 40m)

> 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: 7h 50m
>  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 accounts = getRelevantAccounts();
> Set dateRange = generateDateRange("2018-01-01", "2019-01-01");
> PCollection tokens = generateTokens(accounts, dateRange); 
> {code}
>  
>  I propose an additional _readAll()_ PTransform that can take a PCollection 
> of token ranges and can return a PCollection 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)


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

2020-04-06 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on BEAM-9008:


Author: ASF GitHub Bot
Created on: 06/Apr/20 22:18
Start Date: 06/Apr/20 22:18
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-609998644
 
 
   retest this please
 

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: 417228)
Time Spent: 7.5h  (was: 7h 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: 7.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 accounts = getRelevantAccounts();
> Set dateRange = generateDateRange("2018-01-01", "2019-01-01");
> PCollection tokens = generateTokens(accounts, dateRange); 
> {code}
>  
>  I propose an additional _readAll()_ PTransform that can take a PCollection 
> of token ranges and can return a PCollection 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)


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

2020-04-06 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on BEAM-9008:


Author: ASF GitHub Bot
Created on: 06/Apr/20 22:18
Start Date: 06/Apr/20 22:18
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-610064027
 
 
   retest this please
 

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: 417229)
Time Spent: 7h 40m  (was: 7.5h)

> 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: 7h 40m
>  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 accounts = getRelevantAccounts();
> Set dateRange = generateDateRange("2018-01-01", "2019-01-01");
> PCollection tokens = generateTokens(accounts, dateRange); 
> {code}
>  
>  I propose an additional _readAll()_ PTransform that can take a PCollection 
> of token ranges and can return a PCollection 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)


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

2020-04-06 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on BEAM-9008:


Author: ASF GitHub Bot
Created on: 06/Apr/20 22:18
Start Date: 06/Apr/20 22:18
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-609995788
 
 
   bang
 

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: 417226)
Time Spent: 7h 20m  (was: 7h 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: 7h 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 accounts = getRelevantAccounts();
> Set dateRange = generateDateRange("2018-01-01", "2019-01-01");
> PCollection tokens = generateTokens(accounts, dateRange); 
> {code}
>  
>  I propose an additional _readAll()_ PTransform that can take a PCollection 
> of token ranges and can return a PCollection 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)


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

2020-04-06 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on BEAM-9008:


Author: ASF GitHub Bot
Created on: 06/Apr/20 22:13
Start Date: 06/Apr/20 22:13
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-609995652
 
 
   retest this please
 

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: 417220)
Time Spent: 7h 10m  (was: 7h)

> 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: 7h 10m
>  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 accounts = getRelevantAccounts();
> Set dateRange = generateDateRange("2018-01-01", "2019-01-01");
> PCollection tokens = generateTokens(accounts, dateRange); 
> {code}
>  
>  I propose an additional _readAll()_ PTransform that can take a PCollection 
> of token ranges and can return a PCollection 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)


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

2020-04-06 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on BEAM-9008:


Author: ASF GitHub Bot
Created on: 06/Apr/20 22:12
Start Date: 06/Apr/20 22:12
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-609995502
 
 
   retest this please
 

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: 417219)
Time Spent: 7h  (was: 6h 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: 7h
>  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 accounts = getRelevantAccounts();
> Set dateRange = generateDateRange("2018-01-01", "2019-01-01");
> PCollection tokens = generateTokens(accounts, dateRange); 
> {code}
>  
>  I propose an additional _readAll()_ PTransform that can take a PCollection 
> of token ranges and can return a PCollection 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)


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

2020-04-06 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on BEAM-9008:


Author: ASF GitHub Bot
Created on: 06/Apr/20 22:11
Start Date: 06/Apr/20 22:11
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-610064027
 
 
   retest this please
 

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: 417218)
Time Spent: 6h 50m  (was: 6h 40m)

> 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: 6h 50m
>  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 accounts = getRelevantAccounts();
> Set dateRange = generateDateRange("2018-01-01", "2019-01-01");
> PCollection tokens = generateTokens(accounts, dateRange); 
> {code}
>  
>  I propose an additional _readAll()_ PTransform that can take a PCollection 
> of token ranges and can return a PCollection 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)


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

2020-04-06 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on BEAM-9008:


Author: ASF GitHub Bot
Created on: 06/Apr/20 19:43
Start Date: 06/Apr/20 19:43
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-609998644
 
 
   retest this please
 

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: 417061)
Time Spent: 6h 40m  (was: 6.5h)

> 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: 6h 40m
>  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 accounts = getRelevantAccounts();
> Set dateRange = generateDateRange("2018-01-01", "2019-01-01");
> PCollection tokens = generateTokens(accounts, dateRange); 
> {code}
>  
>  I propose an additional _readAll()_ PTransform that can take a PCollection 
> of token ranges and can return a PCollection 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)


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

2020-04-06 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on BEAM-9008:


Author: ASF GitHub Bot
Created on: 06/Apr/20 19:37
Start Date: 06/Apr/20 19:37
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-609995788
 
 
   bang
 

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: 417060)
Time Spent: 6.5h  (was: 6h 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: 6.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 accounts = getRelevantAccounts();
> Set dateRange = generateDateRange("2018-01-01", "2019-01-01");
> PCollection tokens = generateTokens(accounts, dateRange); 
> {code}
>  
>  I propose an additional _readAll()_ PTransform that can take a PCollection 
> of token ranges and can return a PCollection 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)


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

2020-04-06 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on BEAM-9008:


Author: ASF GitHub Bot
Created on: 06/Apr/20 19:37
Start Date: 06/Apr/20 19:37
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-609995652
 
 
   retest this please
 

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: 417059)
Time Spent: 6h 20m  (was: 6h 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: 6h 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 accounts = getRelevantAccounts();
> Set dateRange = generateDateRange("2018-01-01", "2019-01-01");
> PCollection tokens = generateTokens(accounts, dateRange); 
> {code}
>  
>  I propose an additional _readAll()_ PTransform that can take a PCollection 
> of token ranges and can return a PCollection 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)


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

2020-04-06 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on BEAM-9008:


Author: ASF GitHub Bot
Created on: 06/Apr/20 19:36
Start Date: 06/Apr/20 19:36
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-609995502
 
 
   retest this please
 

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: 417058)
Time Spent: 6h 10m  (was: 6h)

> 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: 6h 10m
>  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 accounts = getRelevantAccounts();
> Set dateRange = generateDateRange("2018-01-01", "2019-01-01");
> PCollection tokens = generateTokens(accounts, dateRange); 
> {code}
>  
>  I propose an additional _readAll()_ PTransform that can take a PCollection 
> of token ranges and can return a PCollection 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)


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

2020-04-06 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on BEAM-9008:


Author: ASF GitHub Bot
Created on: 06/Apr/20 09:18
Start Date: 06/Apr/20 09:18
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-609676142
 
 
   retest this please
 

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: 416444)
Time Spent: 6h  (was: 5h 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
> Fix For: 2.21.0
>
>  Time Spent: 6h
>  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 accounts = getRelevantAccounts();
> Set dateRange = generateDateRange("2018-01-01", "2019-01-01");
> PCollection tokens = generateTokens(accounts, dateRange); 
> {code}
>  
>  I propose an additional _readAll()_ PTransform that can take a PCollection 
> of token ranges and can return a PCollection 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)


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

2020-04-03 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on BEAM-9008:


Author: ASF GitHub Bot
Created on: 04/Apr/20 02:27
Start Date: 04/Apr/20 02:27
Worklog Time Spent: 10m 
  Work Description: vmarquez commented on issue #10546: [BEAM-9008] Add 
CassandraIO readAll method
URL: https://github.com/apache/beam/pull/10546#issuecomment-608958438
 
 
   @iemejia I've fixed the merge conflicts now with a rebase and I'm ready for 
further review. 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:
us...@infra.apache.org


Issue Time Tracking
---

Worklog Id: (was: 415904)
Time Spent: 5h 50m  (was: 5h 40m)

> 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
> Fix For: 2.21.0
>
>  Time Spent: 5h 50m
>  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 accounts = getRelevantAccounts();
> Set dateRange = generateDateRange("2018-01-01", "2019-01-01");
> PCollection tokens = generateTokens(accounts, dateRange); 
> {code}
>  
>  I propose an additional _readAll()_ PTransform that can take a PCollection 
> of token ranges and can return a PCollection 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)


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

2020-04-03 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on BEAM-9008:


Author: ASF GitHub Bot
Created on: 04/Apr/20 02:26
Start Date: 04/Apr/20 02:26
Worklog Time Spent: 10m 
  Work Description: vmarquez commented on issue #10546: [BEAM-9008] Add 
CassandraIO readAll method
URL: https://github.com/apache/beam/pull/10546#issuecomment-607043571
 
 
   Hi @iemejia I've been busy with work and life but was finally able to this 
just about finished up!  To get around the connection issue (not caching the 
connection was causing the Cassandra tests to run for ~ten minutes!), by 
passing in a ReadAll to the ReadFn, we're able to initiate a connection in 
the setup method, but we can still 'dynamically' use the passed in Read to 
generate specific queries or query ranges.  
   
   As for a more advanced connection pooling, I'd prefer to get this merged in 
and then perhaps work on an additional PR.  
   
   
   
   
 

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: 415903)
Time Spent: 5h 40m  (was: 5.5h)

> 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
> Fix For: 2.21.0
>
>  Time Spent: 5h 40m
>  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 accounts = getRelevantAccounts();
> Set dateRange = generateDateRange("2018-01-01", "2019-01-01");
> PCollection tokens = generateTokens(accounts, dateRange); 
> {code}
>  
>  I propose an additional _readAll()_ PTransform that can take a PCollection 
> of token ranges and can return a PCollection 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)


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

2020-03-31 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on BEAM-9008:


Author: ASF GitHub Bot
Created on: 01/Apr/20 05:37
Start Date: 01/Apr/20 05:37
Worklog Time Spent: 10m 
  Work Description: vmarquez commented on issue #10546: [BEAM-9008] Add 
CassandraIO readAll method
URL: https://github.com/apache/beam/pull/10546#issuecomment-607043571
 
 
   Hi @iemejia I've been busy with work and life but was finally able to this 
just about finished up!  To get around the connection issue (not caching the 
connection was causing the Cassandra tests to run for ~ten minutes!), by 
passing in a ReadAll to the ReadFn, we're able to initiate a connection in 
the setup method, but we can still 'dynamically' use the passed in Read to 
generate specific queries or query ranges.  
   
   As for a more advanced connection pooling, I'd prefer to get this merged in 
and then perhaps work on an additional PR.  
   
   There are some minor merge conflicts but I wasn't sure how you wanted me to 
handle that, do you prefer I rebase against master (? that might then cause 
issues with Github's UI seeing other folks commits in my changelist), or should 
I merge master into this then another commit(which makes later squashing a bit 
harder).  Advice on that would be appreciated!  Thanks again for your guidance 
and help on this. I like the new design and the flexibility to modify the 
queries as well as ring ranges is pretty neat. 
   
   
   
   
 

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: 413792)
Time Spent: 5.5h  (was: 5h 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
> Fix For: 2.21.0
>
>  Time Spent: 5.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 accounts = getRelevantAccounts();
> Set dateRange = generateDateRange("2018-01-01", "2019-01-01");
> PCollection tokens = generateTokens(accounts, dateRange); 
> {code}
>  
>  I propose an additional _readAll()_ PTransform that can take a PCollection 
> of token ranges and can return a PCollection 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)


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

2020-03-31 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on BEAM-9008:


Author: ASF GitHub Bot
Created on: 01/Apr/20 05:36
Start Date: 01/Apr/20 05:36
Worklog Time Spent: 10m 
  Work Description: vmarquez commented on issue #10546: [BEAM-9008] Add 
CassandraIO readAll method
URL: https://github.com/apache/beam/pull/10546#issuecomment-607043571
 
 
   Hi @iemejia I've been busy with work and life but was finally able to this 
just about finished up!  To get around the connection issue (not caching the 
connection was causing the Cassandra tests to run for ~ten minutes!), by 
passing in a ReadAll to the ReadFn, we're able to initiate a connection in 
the setup method, but we can still 'dynamically' use the passed in Read to 
generate specific queries or query ranges.  
   
   As for a more advanced connection pooling, I'd prefer to get this merged in 
and then perhaps work on an additional PR.  
   
   There are some minor conflicts but I wasn't sure how you wanted to handle 
that, do you prefer I rebase against master (? that might then cause issues 
with Github's UI seeing other folks commits in my changelist), or should I 
merge master into this then another commit(which makes later squashing a bit 
harder).  Advice on that would be appreciated!  Thanks again for your guidance 
and help on this. I like the new design and the flexibility to modify the 
queries as well as ring ranges is pretty neat. 
   
   
   
   
 

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: 413791)
Time Spent: 5h 20m  (was: 5h 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
> Fix For: 2.21.0
>
>  Time Spent: 5h 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 accounts = getRelevantAccounts();
> Set dateRange = generateDateRange("2018-01-01", "2019-01-01");
> PCollection tokens = generateTokens(accounts, dateRange); 
> {code}
>  
>  I propose an additional _readAll()_ PTransform that can take a PCollection 
> of token ranges and can return a PCollection 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)


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

2020-03-31 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on BEAM-9008:


Author: ASF GitHub Bot
Created on: 31/Mar/20 07:17
Start Date: 31/Mar/20 07:17
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-606446222
 
 
   I was also thinking about the potential connection issue and I think we can 
try to tackle it by creating some sort of static connection pool and we id the 
connections by their server/config and reuse it if possible. We can work on 
this in this PR or on a follow up, as you prefer.
 

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: 412902)
Time Spent: 5h 10m  (was: 5h)

> 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
> Fix For: 2.21.0
>
>  Time Spent: 5h 10m
>  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 accounts = getRelevantAccounts();
> Set dateRange = generateDateRange("2018-01-01", "2019-01-01");
> PCollection tokens = generateTokens(accounts, dateRange); 
> {code}
>  
>  I propose an additional _readAll()_ PTransform that can take a PCollection 
> of token ranges and can return a PCollection 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)


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

2020-03-31 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on BEAM-9008:


Author: ASF GitHub Bot
Created on: 31/Mar/20 07:16
Start Date: 31/Mar/20 07:16
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-606445740
 
 
   I think this is needing a rebase. Please ping me when this is ready for 
review. I am eager to do so hoping this can make it into Beam 2.21.0
   
 

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: 412901)
Time Spent: 5h  (was: 4h 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
> Fix For: 2.21.0
>
>  Time Spent: 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 accounts = getRelevantAccounts();
> Set dateRange = generateDateRange("2018-01-01", "2019-01-01");
> PCollection tokens = generateTokens(accounts, dateRange); 
> {code}
>  
>  I propose an additional _readAll()_ PTransform that can take a PCollection 
> of token ranges and can return a PCollection 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)


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

2020-01-31 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on BEAM-9008:


Author: ASF GitHub Bot
Created on: 01/Feb/20 00:05
Start Date: 01/Feb/20 00:05
Worklog Time Spent: 10m 
  Work Description: vmarquez commented on issue #10546: [BEAM-9008] Add 
CassandraIO readAll method
URL: https://github.com/apache/beam/pull/10546#issuecomment-580962843
 
 
   > Couldn't the user in the specific case you mention of many Reads generate 
queries for RingRanges that match more 'regions' and achieve the same result?
   
I don't think so, a user might be querying specific partition keys (only 
one key that returns multiple results) that are not contiguous some of which 
have data and many of which may have no data. 
 

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: 380267)
Time Spent: 4h 50m  (was: 4h 40m)

> 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 50m
>  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 accounts = getRelevantAccounts();
> Set dateRange = generateDateRange("2018-01-01", "2019-01-01");
> PCollection tokens = generateTokens(accounts, dateRange); 
> {code}
>  
>  I propose an additional _readAll()_ PTransform that can take a PCollection 
> of token ranges and can return a PCollection 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)


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

2020-01-31 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on BEAM-9008:


Author: ASF GitHub Bot
Created on: 31/Jan/20 23:52
Start Date: 31/Jan/20 23:52
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-580960037
 
 
   Cassandra is extremely slow opening new connections however the average case 
would be that for each `Read` there would be many `T` outputs which in 
principle would amortize the slow connection time. Couldn't the user in the 
specific case you mention of many Reads generate queries for RingRanges that 
match more 'regions' and achieve the same result?
   
   If this is really ends up being a problem one eventual workaround would be 
to batch requests and create some sort of pool of connections but hopefully 
this won't be needed.
 

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: 380259)
Time Spent: 4h 40m  (was: 4.5h)

> 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 40m
>  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 accounts = getRelevantAccounts();
> Set dateRange = generateDateRange("2018-01-01", "2019-01-01");
> PCollection tokens = generateTokens(accounts, dateRange); 
> {code}
>  
>  I propose an additional _readAll()_ PTransform that can take a PCollection 
> of token ranges and can return a PCollection 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)


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

2020-01-30 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on BEAM-9008:


Author: ASF GitHub Bot
Created on: 31/Jan/20 07:33
Start Date: 31/Jan/20 07:33
Worklog Time Spent: 10m 
  Work Description: vmarquez commented on issue #10546: [BEAM-9008] Add 
CassandraIO readAll method
URL: https://github.com/apache/beam/pull/10546#issuecomment-580619044
 
 
   @iemejia I have started working on a different implementation of ReadAll 
that extends `PTransform>, PCollection>`, however, I 
have thought about it more and I am thinking that it is bad more undesireable 
than we thought to *not* establish a connection in `setup` of the `ReadFn` 
class.  Just this week at work I had to do some work involving generating 
hundreds of thousands of queries (`RingRanges`), and the vast majority of them 
returned no or almost no data!  The data was sparse and never the less I still 
needed to perform the queries, so having to establish a new connection per 
`RingRange` in the `processElement` method would have slown things down a lot.  
   
   I understand the desire to have simpler code, but I think the performance 
impact would be noticable if we only keep connection information in the 
`Read` that comes into the `processElement` method.   Do you still want to 
proceed with that? 
 

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: 379781)
Time Spent: 4.5h  (was: 4h 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: 4.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 accounts = getRelevantAccounts();
> Set dateRange = generateDateRange("2018-01-01", "2019-01-01");
> PCollection tokens = generateTokens(accounts, dateRange); 
> {code}
>  
>  I propose an additional _readAll()_ PTransform that can take a PCollection 
> of token ranges and can return a PCollection 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)


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

2020-01-30 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on BEAM-9008:


Author: ASF GitHub Bot
Created on: 31/Jan/20 07:33
Start Date: 31/Jan/20 07:33
Worklog Time Spent: 10m 
  Work Description: vmarquez commented on issue #10546: [BEAM-9008] Add 
CassandraIO readAll method
URL: https://github.com/apache/beam/pull/10546#issuecomment-580619044
 
 
   @iemejia I have started working on a different implementation of ReadAll 
that extends `PTransform>, PCollection>`, however, I 
have thought about it more and I am thinking that it is bad to not be able to 
establish a connection in `setup` of the `ReadFn` class.  Just this week at 
work I had to do some work involving generating hundreds of thousands of 
queries (`RingRanges`), and the vast majority of them returned no or almost no 
data!  The data was sparse and never the less I still needed to perform the 
queries, so having to establish a new connection per `RingRange` in the 
`processElement` method would have slown things down a lot.  
   
   I understand the desire to have simpler code, but I think the performance 
impact would be noticable if we only keep connection information in the 
`Read` that comes into the `processElement` method.   Do you still want to 
proceed with that? 
 

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: 379780)
Time Spent: 4h 20m  (was: 4h 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: 4h 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 accounts = getRelevantAccounts();
> Set dateRange = generateDateRange("2018-01-01", "2019-01-01");
> PCollection tokens = generateTokens(accounts, dateRange); 
> {code}
>  
>  I propose an additional _readAll()_ PTransform that can take a PCollection 
> of token ranges and can return a PCollection 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)


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

2020-01-28 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on BEAM-9008:


Author: ASF GitHub Bot
Created on: 28/Jan/20 15:58
Start Date: 28/Jan/20 15:58
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-579318866
 
 
   Forgot to mention that in the above comment that in the Split function you 
have to split in every case save if the user provided a specific RingRange to 
read from.
 

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: 378339)
Time Spent: 4h 10m  (was: 4h)

> 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 10m
>  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 accounts = getRelevantAccounts();
> Set dateRange = generateDateRange("2018-01-01", "2019-01-01");
> PCollection tokens = generateTokens(accounts, dateRange); 
> {code}
>  
>  I propose an additional _readAll()_ PTransform that can take a PCollection 
> of token ranges and can return a PCollection 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)


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

2020-01-28 Thread ASF GitHub Bot (Jira)


 [ 
https://issues.apache.org/jira/browse/BEAM-9008?focusedWorklogId=378338=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 extends DoFn, A>` and the only way we 
have connection information is from the `Read` 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>`?  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>` 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` input and enriches the db 
connection information?
   
   You have a point here!. We need `class ReadAll extends 
PTransform, PCollection>` 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` 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 accounts = getRelevantAccounts();
> Set dateRange = 

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

2020-01-24 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on BEAM-9008:


Author: ASF GitHub Bot
Created on: 24/Jan/20 17:35
Start Date: 24/Jan/20 17:35
Worklog Time Spent: 10m 
  Work Description: vmarquez commented on issue #10546: [BEAM-9008] Add 
CassandraIO readAll method
URL: https://github.com/apache/beam/pull/10546#issuecomment-577996918
 
 
   @iemejia I think that could work, thanks for your patience as I try to 
understand what you're thinking.  Some questions:
   
   1.  If our `ReadFn extends DoFn, A>` and the only way we have 
connection information is from the `Read` 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? 
   
   2. How would that work for the end user of a 
`PTransform, PCollection>`?  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>` 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` input and enriches the db connection 
information? 
   
   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` to hold a 
collection of RingRanges? 
   
   
 

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: 376934)
Time Spent: 3h 50m  (was: 3h 40m)

> 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 50m
>  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 accounts = getRelevantAccounts();
> Set dateRange = generateDateRange("2018-01-01", "2019-01-01");
> PCollection tokens = generateTokens(accounts, dateRange); 
> {code}
>  
>  I propose an additional _readAll()_ PTransform that can take a PCollection 
> of token ranges and can return a PCollection 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)


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

2020-01-23 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on BEAM-9008:


Author: ASF GitHub Bot
Created on: 24/Jan/20 05:34
Start Date: 24/Jan/20 05:34
Worklog Time Spent: 10m 
  Work Description: vmarquez commented on issue #10546: [BEAM-9008] Add 
CassandraIO readAll method
URL: https://github.com/apache/beam/pull/10546#issuecomment-577996918
 
 
   @iemejia I think that could work, thanks for your patience as I try to 
understand what you're thinking.  Some questions:
   
   1.  If our `ReadFn extends DoFn, A>` and the only way we have 
connection information is from the actual read, 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? 
   
   2. How would that work for the end user of a 
`PTransform, PCollection>`?  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>` 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` input and enriches the db connection 
information? 
   
   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` to hold a 
collection of RingRanges? 
   
   
 

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: 376711)
Time Spent: 3h 40m  (was: 3.5h)

> 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 40m
>  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 accounts = getRelevantAccounts();
> Set dateRange = generateDateRange("2018-01-01", "2019-01-01");
> PCollection tokens = generateTokens(accounts, dateRange); 
> {code}
>  
>  I propose an additional _readAll()_ PTransform that can take a PCollection 
> of token ranges and can return a PCollection 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)


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

2020-01-23 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on BEAM-9008:


Author: ASF GitHub Bot
Created on: 24/Jan/20 05:32
Start Date: 24/Jan/20 05:32
Worklog Time Spent: 10m 
  Work Description: vmarquez commented on issue #10546: [BEAM-9008] Add 
CassandraIO readAll method
URL: https://github.com/apache/beam/pull/10546#issuecomment-577996918
 
 
   @iemejia I think that could work, thanks for your patience as I try to 
understand what you're thinking.  Some questions:
   
   1.  If our `ReadFn extends DoFn, A>` and the only way we have 
connection information is from the actual read, that means we need to 
re-establish a DB connection for each  batch of queries we run?  
   
   2. How would that work for the end user of a 
`PTransform, PCollection>`?  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>` 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` input and enriches the db connection 
information? 
   
   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` to hold a 
collection of RingRanges? 
   
   
 

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: 376710)
Time Spent: 3.5h  (was: 3h 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: 3.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 accounts = getRelevantAccounts();
> Set dateRange = generateDateRange("2018-01-01", "2019-01-01");
> PCollection tokens = generateTokens(accounts, dateRange); 
> {code}
>  
>  I propose an additional _readAll()_ PTransform that can take a PCollection 
> of token ranges and can return a PCollection 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)


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

2020-01-23 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on BEAM-9008:


Author: ASF GitHub Bot
Created on: 24/Jan/20 05:31
Start Date: 24/Jan/20 05:31
Worklog Time Spent: 10m 
  Work Description: vmarquez commented on issue #10546: [BEAM-9008] Add 
CassandraIO readAll method
URL: https://github.com/apache/beam/pull/10546#issuecomment-577996918
 
 
   @iemejia I think that could work, thanks for your patience as I try to 
understand what you're thinking.  Some questions:
   
   1.  If our `ReadFn extends DoFn, A>` and the only way we have 
connection information is from the actual read, that means we need to 
re-establish a DB connection for each  batch of queries we run?  
   
   2. How would that work for the end user of a 
`PTransform, PCollection>`?  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>` 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` input and enriches the db connection 
information? 
   
   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` to hold a 
collection of RingRanges? 
   
   
 

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: 376709)
Time Spent: 3h 20m  (was: 3h 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: 3h 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 accounts = getRelevantAccounts();
> Set dateRange = generateDateRange("2018-01-01", "2019-01-01");
> PCollection tokens = generateTokens(accounts, dateRange); 
> {code}
>  
>  I propose an additional _readAll()_ PTransform that can take a PCollection 
> of token ranges and can return a PCollection 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)


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

2020-01-23 Thread ASF GitHub Bot (Jira)


 [ 
https://issues.apache.org/jira/browse/BEAM-9008?focusedWorklogId=376638=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 extends PTransform>, 
PCollection> {
   @Override
   public PCollection expand(PCollection> 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 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:
us...@infra.apache.org


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 accounts = getRelevantAccounts();
> Set dateRange = generateDateRange("2018-01-01", "2019-01-01");
> PCollection tokens = generateTokens(accounts, dateRange); 
> {code}
>  
>  I propose an additional _readAll()_ PTransform that can take a PCollection 
> of token ranges and can return a PCollection 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 

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

2020-01-23 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on BEAM-9008:


Author: ASF GitHub Bot
Created on: 23/Jan/20 21:26
Start Date: 23/Jan/20 21:26
Worklog Time Spent: 10m 
  Work Description: vmarquez commented on issue #10546: [BEAM-9008] Add 
CassandraIO readAll method
URL: https://github.com/apache/beam/pull/10546#issuecomment-576998681
 
 
   Thanks for taking a look, I've pushed a new commit (I won't be squashing 
anymore until we finalize everything) that hopefully addresses all the minor 
style issues.  LMK if I missed anything. 
   
   I do want to make sure we keep the cassandraIO 'idiomatic' to the rest of 
the IO connectors, but I don't think modeling this after the SOLR one will 
work.  For one thing, if we want to share the `ReadFn` class between both Read 
and ReadAll, it means we have to have some way of having both use it and pass 
in 'connection' information, which we can't do if the signature of ReadFn is 
`ReadFn extends DoFn, A>`. (unless we copy everything over into a 
new ReadAll whenever we create a Read which seems a bit clumsy?)  I think 
another class to look at for something that has both Read and ReadAll 
PTransforms is the SpannerIO, which is modeled similarly to how I did it here 
(though not exactly). 
   
   
https://github.com/apache/beam/blob/master/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java#L315
   
   They have a configuration class there is public (which we can't do since we 
want to keep backwards compatibility with the current way `Read` works), it has 
two different PTransforms, `Read` uses `ReadAll` internally, etc.  
   
   I do think instead of taking a collection of RingRanges, taking some sort of 
'Query' object makes sense, and the idea that it doesn't have to tie in to the 
actual connection means we can split up the CassandraConfig class.  Thoughts on 
that? 
 

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: 376582)
Time Spent: 2h 50m  (was: 2h 40m)

> 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 50m
>  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 accounts = getRelevantAccounts();
> Set dateRange = generateDateRange("2018-01-01", "2019-01-01");
> PCollection tokens = generateTokens(accounts, dateRange); 
> {code}
>  
>  I propose an additional _readAll()_ PTransform that can take a PCollection 
> of token ranges and can return a PCollection 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)


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

2020-01-21 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on BEAM-9008:


Author: ASF GitHub Bot
Created on: 22/Jan/20 04:09
Start Date: 22/Jan/20 04:09
Worklog Time Spent: 10m 
  Work Description: vmarquez commented on issue #10546: [BEAM-9008] Add 
CassandraIO readAll method
URL: https://github.com/apache/beam/pull/10546#issuecomment-576998681
 
 
   Thanks for taking a look, I've pushed a new commit (I won't be squashing 
anymore until we finalize everything) that hopefully addresses all the minor 
style issues.  LMK if I missed anything. 
   
   I do want to make sure we keep the cassandraIO 'idiomatic' to the rest of 
the IO connectors, but I don't think modeling this after the SOLR one will 
work.  For one thing, if we want to share the `ReadFn` class between both Read 
and ReadAll, it means we have to have some way of having both use it and pass 
in 'connection' information, which we can't do if the signature of ReadFn is 
`ReadFn extends DoFn, A>`.  I think another class to look at for 
something that has both Read and ReadAll PTransforms is the SpannerIO, which is 
modeled similarly to how I did it here (though not exactly). 
   
   
https://github.com/apache/beam/blob/master/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java#L315
   
   They have a configuration class there is public (which we can't do since we 
want to keep backwards compatibility with the current way `Read` works), it has 
two different PTransforms, `Read` uses `ReadAll` internally, etc.  
   
   I do think instead of taking a collection of RingRanges, taking some sort of 
'Query' object makes sense, and the idea that it doesn't have to tie in to the 
actual connection means we can split up the CassandraConfig class.  Thoughts on 
that? 
 

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: 375391)
Time Spent: 2h 40m  (was: 2.5h)

> 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 40m
>  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 accounts = getRelevantAccounts();
> Set dateRange = generateDateRange("2018-01-01", "2019-01-01");
> PCollection tokens = generateTokens(accounts, dateRange); 
> {code}
>  
>  I propose an additional _readAll()_ PTransform that can take a PCollection 
> of token ranges and can return a PCollection 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)


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

2020-01-21 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on BEAM-9008:


Author: ASF GitHub Bot
Created on: 21/Jan/20 21:11
Start Date: 21/Jan/20 21:11
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_r369244935
 
 

 ##
 File path: 
sdks/java/io/cassandra/src/test/java/org/apache/beam/sdk/io/cassandra/CassandraIOTest.java
 ##
 @@ -466,40 +500,9 @@ public void testCustomMapperImplDelete() {
 assertEquals(1, counter.intValue());
   }
 
-  @Test
-  public void testSplit() throws Exception {
-PipelineOptions options = PipelineOptionsFactory.create();
 
 Review comment:
   Hmm, if you know of relevant issues that we previously had, lmk, I will also 
look through the issue tracker to see if there were previous bugs. 
 

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: 375214)
Time Spent: 2.5h  (was: 2h 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: 2.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 accounts = getRelevantAccounts();
> Set dateRange = generateDateRange("2018-01-01", "2019-01-01");
> PCollection tokens = generateTokens(accounts, dateRange); 
> {code}
>  
>  I propose an additional _readAll()_ PTransform that can take a PCollection 
> of token ranges and can return a PCollection 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)


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

2020-01-21 Thread ASF GitHub Bot (Jira)


 [ 
https://issues.apache.org/jira/browse/BEAM-9008?focusedWorklogId=375209=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 extends DoFn, 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:
us...@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 accounts = getRelevantAccounts();
> Set dateRange = generateDateRange("2018-01-01", "2019-01-01");
> PCollection tokens = generateTokens(accounts, dateRange); 
> {code}
>  
>  I propose an additional _readAll()_ PTransform that can take a PCollection 
> of token ranges and can return a PCollection 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 

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

2020-01-20 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on BEAM-9008:


Author: ASF GitHub Bot
Created on: 20/Jan/20 16:40
Start Date: 20/Jan/20 16:40
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-576352218
 
 
   Btw, don't hesitate to ask questions or discuss here if you have any doubts 
before starting.
 

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: 374664)
Time Spent: 2h 10m  (was: 2h)

> 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 10m
>  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 accounts = getRelevantAccounts();
> Set dateRange = generateDateRange("2018-01-01", "2019-01-01");
> PCollection tokens = generateTokens(accounts, dateRange); 
> {code}
>  
>  I propose an additional _readAll()_ PTransform that can take a PCollection 
> of token ranges and can return a PCollection 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)


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

2020-01-20 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on BEAM-9008:


Author: ASF GitHub Bot
Created on: 20/Jan/20 16:36
Start Date: 20/Jan/20 16:36
Worklog Time Spent: 10m 
  Work Description: iemejia commented on pull request #10546: [BEAM-9008] 
add CassandraIO readAll method
URL: https://github.com/apache/beam/pull/10546#discussion_r368623996
 
 

 ##
 File path: 
sdks/java/io/cassandra/src/main/java/org/apache/beam/sdk/io/cassandra/CassandraIO.java
 ##
 @@ -608,121 +470,32 @@ static double getRingFraction(List 
tokenRanges) {
   return ringFraction;
 }
 
-/**
- * Check if the current partitioner is the Murmur3 (default in Cassandra 
version newer than 2).
- */
-@VisibleForTesting
-static boolean isMurmur3Partitioner(Cluster cluster) {
-  return MURMUR3PARTITIONER.equals(cluster.getMetadata().getPartitioner());
-}
-
 /** Measure distance between two tokens. */
 @VisibleForTesting
 static BigInteger distance(BigInteger left, BigInteger right) {
   return (right.compareTo(left) > 0)
   ? right.subtract(left)
   : 
right.subtract(left).add(SplitGenerator.getRangeSize(MURMUR3PARTITIONER));
 }
+  }
 
-/**
- * Represent a token range in Cassandra instance, wrapping the partition 
count, size and token
- * range.
- */
-@VisibleForTesting
-static class TokenRange {
-  private final long partitionCount;
-  private final long meanPartitionSize;
-  private final BigInteger rangeStart;
-  private final BigInteger rangeEnd;
-
-  TokenRange(
-  long partitionCount, long meanPartitionSize, BigInteger rangeStart, 
BigInteger rangeEnd) {
-this.partitionCount = partitionCount;
-this.meanPartitionSize = meanPartitionSize;
-this.rangeStart = rangeStart;
-this.rangeEnd = rangeEnd;
-  }
-}
-
-private class CassandraReader extends BoundedSource.BoundedReader {
-  private final CassandraIO.CassandraSource source;
-  private Cluster cluster;
-  private Session session;
-  private Iterator iterator;
-  private T current;
-
-  CassandraReader(CassandraSource source) {
-this.source = source;
-  }
-
-  @Override
-  public boolean start() {
-LOG.debug("Starting Cassandra reader");
-cluster =
-getCluster(
-source.spec.hosts(),
-source.spec.port(),
-source.spec.username(),
-source.spec.password(),
-source.spec.localDc(),
-source.spec.consistencyLevel());
-session = cluster.connect(source.spec.keyspace().get());
-LOG.debug("Queries: " + source.splitQueries);
-List futures = new ArrayList<>();
-for (String query : source.splitQueries) {
-  futures.add(session.executeAsync(query));
-}
-
-final Mapper mapper = getMapper(session, source.spec.entity());
-
-for (ResultSetFuture result : futures) {
-  if (iterator == null) {
-iterator = mapper.map(result.getUninterruptibly());
-  } else {
-iterator = Iterators.concat(iterator, 
mapper.map(result.getUninterruptibly()));
-  }
-}
-
-return advance();
-  }
-
-  @Override
-  public boolean advance() {
-if (iterator.hasNext()) {
-  current = iterator.next();
-  return true;
-}
-current = null;
-return false;
-  }
-
-  @Override
-  public void close() {
-LOG.debug("Closing Cassandra reader");
-if (session != null) {
-  session.close();
-}
-if (cluster != null) {
-  cluster.close();
-}
-  }
-
-  @Override
-  public T getCurrent() throws NoSuchElementException {
-if (current == null) {
-  throw new NoSuchElementException();
-}
-return current;
-  }
-
-  @Override
-  public CassandraIO.CassandraSource getCurrentSource() {
-return source;
-  }
-
-  private Mapper getMapper(Session session, Class enitity) {
-return source.spec.mapperFactoryFn().apply(session);
-  }
+  /**
+   * Represent a token range in Cassandra instance, wrapping the partition 
count, size and token
+   * range.
+   */
+  @VisibleForTesting
+  static class TokenRange {
 
 Review comment:
   We don't need this one anymore I think we can remove this class now as well 
as: `CassandraIO.getRingFraction` and `CassandraIOTest.testRingFraction`. Can 
you remove these too please.
 

This is an automated message from the Apache Git Service.
To respond to the message, please log 

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

2020-01-20 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on BEAM-9008:


Author: ASF GitHub Bot
Created on: 20/Jan/20 16:36
Start Date: 20/Jan/20 16:36
Worklog Time Spent: 10m 
  Work Description: iemejia commented on pull request #10546: [BEAM-9008] 
add CassandraIO readAll method
URL: https://github.com/apache/beam/pull/10546#discussion_r368626298
 
 

 ##
 File path: 
sdks/java/io/cassandra/src/main/java/org/apache/beam/sdk/io/cassandra/RingRange.java
 ##
 @@ -17,10 +17,11 @@
  */
 package org.apache.beam.sdk.io.cassandra;
 
+import java.io.Serializable;
 import java.math.BigInteger;
 
 /** Models a Cassandra token range. */
-final class RingRange {
+final class RingRange implements Serializable {
 
 Review comment:
   Actually this should probably be done not for this class but for the 
`CassandraQuery` object
 

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: 374662)

> 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
>  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 accounts = getRelevantAccounts();
> Set dateRange = generateDateRange("2018-01-01", "2019-01-01");
> PCollection tokens = generateTokens(accounts, dateRange); 
> {code}
>  
>  I propose an additional _readAll()_ PTransform that can take a PCollection 
> of token ranges and can return a PCollection 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)


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

2020-01-20 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on BEAM-9008:


Author: ASF GitHub Bot
Created on: 20/Jan/20 16:36
Start Date: 20/Jan/20 16:36
Worklog Time Spent: 10m 
  Work Description: iemejia commented on pull request #10546: [BEAM-9008] 
add CassandraIO readAll method
URL: https://github.com/apache/beam/pull/10546#discussion_r367997934
 
 

 ##
 File path: 
sdks/java/io/cassandra/src/main/java/org/apache/beam/sdk/io/cassandra/CassandraIO.java
 ##
 @@ -1146,4 +918,297 @@ 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
+  extends PTransform, PCollection> {
+@Nullable
+abstract ValueProvider> hosts();
+
+@Nullable
+abstract ValueProvider query();
+
+@Nullable
+abstract ValueProvider port();
+
+@Nullable
+abstract ValueProvider keyspace();
+
+@Nullable
+abstract ValueProvider table();
+
+@Nullable
+abstract Class entity();
+
+@Nullable
+abstract Coder coder();
+
+@Nullable
+abstract ValueProvider username();
+
+@Nullable
+abstract ValueProvider password();
+
+@Nullable
+abstract ValueProvider localDc();
+
+@Nullable
+abstract ValueProvider consistencyLevel();
+
+@Nullable
+abstract ValueProvider splitCount();
+
+@Nullable
+abstract SerializableFunction mapperFactoryFn();
+
+@Nullable
+abstract SerializableFunction groupingFn();
+
+abstract Builder builder();
+
+/** Specify the hosts of the Apache Cassandra instances. */
+public ReadAll withHosts(List 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 withHosts(ValueProvider> hosts) {
+  return builder().setHosts(hosts).build();
+}
+
+/** Specify the port number of the Apache Cassandra instances. */
+public ReadAll 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 withPort(ValueProvider port) {
+  return builder().setPort(port).build();
+}
+
+/** Specify the Cassandra keyspace where to read data. */
+public ReadAll 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 withKeyspace(ValueProvider keyspace) {
+  return builder().setKeyspace(keyspace).build();
+}
+
+/** Specify the Cassandra table where to read data. */
+public ReadAll 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 withTable(ValueProvider table) {
+  return builder().setTable(table).build();
+}
+
+/** Specify the query to read data. */
+public ReadAll 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 withQuery(ValueProvider 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 withEntity(Class 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 withCoder(Coder coder) {
+  checkArgument(coder != null, "coder can not be null");
+  return builder().setCoder(coder).build();
+}
+
+/** Specify the username for authentication. */
+public ReadAll withUsername(String username) {
+  checkArgument(username != null, "username can not be null");
+  

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

2020-01-20 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on BEAM-9008:


Author: ASF GitHub Bot
Created on: 20/Jan/20 16:36
Start Date: 20/Jan/20 16:36
Worklog Time Spent: 10m 
  Work Description: iemejia commented on pull request #10546: [BEAM-9008] 
add CassandraIO readAll method
URL: https://github.com/apache/beam/pull/10546#discussion_r367999012
 
 

 ##
 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 extends DoFn, T> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(CassandraIO.class);
 
 Review comment:
   Replace `CassandraIO.class` with the new `ReadFn.class`
 

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: 374648)
Time Spent: 1h 20m  (was: 1h 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: 1h 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 accounts = getRelevantAccounts();
> Set dateRange = generateDateRange("2018-01-01", "2019-01-01");
> PCollection tokens = generateTokens(accounts, dateRange); 
> {code}
>  
>  I propose an additional _readAll()_ PTransform that can take a PCollection 
> of token ranges and can return a PCollection 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 

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

2020-01-20 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on BEAM-9008:


Author: ASF GitHub Bot
Created on: 20/Jan/20 16:36
Start Date: 20/Jan/20 16:36
Worklog Time Spent: 10m 
  Work Description: iemejia commented on pull request #10546: [BEAM-9008] 
add CassandraIO readAll method
URL: https://github.com/apache/beam/pull/10546#discussion_r368626645
 
 

 ##
 File path: 
sdks/java/io/cassandra/src/test/java/org/apache/beam/sdk/io/cassandra/CassandraIOTest.java
 ##
 @@ -387,6 +420,7 @@ public Mapper apply(Session input) {
 
 @Override
 public Iterator map(ResultSet resultSet) {
+
 
 Review comment:
   nit: remove space
 

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: 374653)
Time Spent: 1h 40m  (was: 1.5h)

> 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: 1h 40m
>  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 accounts = getRelevantAccounts();
> Set dateRange = generateDateRange("2018-01-01", "2019-01-01");
> PCollection tokens = generateTokens(accounts, dateRange); 
> {code}
>  
>  I propose an additional _readAll()_ PTransform that can take a PCollection 
> of token ranges and can return a PCollection 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)


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

2020-01-20 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on BEAM-9008:


Author: ASF GitHub Bot
Created on: 20/Jan/20 16:36
Start Date: 20/Jan/20 16:36
Worklog Time Spent: 10m 
  Work Description: iemejia commented on pull request #10546: [BEAM-9008] 
add CassandraIO readAll method
URL: https://github.com/apache/beam/pull/10546#discussion_r368627136
 
 

 ##
 File path: 
sdks/java/io/cassandra/src/test/java/org/apache/beam/sdk/io/cassandra/CassandraIOTest.java
 ##
 @@ -466,40 +500,9 @@ public void testCustomMapperImplDelete() {
 assertEquals(1, counter.intValue());
   }
 
-  @Test
-  public void testSplit() throws Exception {
-PipelineOptions options = PipelineOptionsFactory.create();
 
 Review comment:
   Once we have the SplitFn we should probably test this, actually we had a lot 
of historic issues with Splitting on this connector so please pay attention to 
ensure we don't introduce regressions.
 

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: 374659)
Time Spent: 1h 50m  (was: 1h 40m)

> 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: 1h 50m
>  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 accounts = getRelevantAccounts();
> Set dateRange = generateDateRange("2018-01-01", "2019-01-01");
> PCollection tokens = generateTokens(accounts, dateRange); 
> {code}
>  
>  I propose an additional _readAll()_ PTransform that can take a PCollection 
> of token ranges and can return a PCollection 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)


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

2020-01-20 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on BEAM-9008:


Author: ASF GitHub Bot
Created on: 20/Jan/20 16:36
Start Date: 20/Jan/20 16:36
Worklog Time Spent: 10m 
  Work Description: iemejia commented on pull request #10546: [BEAM-9008] 
add CassandraIO readAll method
URL: https://github.com/apache/beam/pull/10546#discussion_r368013640
 
 

 ##
 File path: 
sdks/java/io/cassandra/src/main/java/org/apache/beam/sdk/io/cassandra/CassandraIO.java
 ##
 @@ -732,6 +505,35 @@ public T getCurrent() throws NoSuchElementException {
 DELETE
   }
 
+  /**
+   * Check if the current partitioner is the Murmur3 (default in Cassandra 
version newer than 2).
+   */
+  @VisibleForTesting
+  static boolean isMurmur3Partitioner(Cluster cluster) {
+return MURMUR3PARTITIONER.equals(cluster.getMetadata().getPartitioner());
+  }
+
+  static String generateRangeQuery(CassandraConfig spec, String partitionKey) {
 
 Review comment:
   Can you move this one to `QueryFn` since that's the only place where it is 
used.
 

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: 374652)
Time Spent: 1h 40m  (was: 1.5h)

> 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: 1h 40m
>  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 accounts = getRelevantAccounts();
> Set dateRange = generateDateRange("2018-01-01", "2019-01-01");
> PCollection tokens = generateTokens(accounts, dateRange); 
> {code}
>  
>  I propose an additional _readAll()_ PTransform that can take a PCollection 
> of token ranges and can return a PCollection 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)


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

2020-01-20 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on BEAM-9008:


Author: ASF GitHub Bot
Created on: 20/Jan/20 16:36
Start Date: 20/Jan/20 16:36
Worklog Time Spent: 10m 
  Work Description: iemejia commented on pull request #10546: [BEAM-9008] 
add CassandraIO readAll method
URL: https://github.com/apache/beam/pull/10546#discussion_r368622445
 
 

 ##
 File path: 
sdks/java/io/cassandra/src/main/java/org/apache/beam/sdk/io/cassandra/CassandraIO.java
 ##
 @@ -325,7 +359,58 @@ private CassandraIO() {}
   checkArgument(entity() != null, "withEntity() is required");
   checkArgument(coder() != null, "withCoder() is required");
 
-  return input.apply(org.apache.beam.sdk.io.Read.from(new 
CassandraSource<>(this, null)));
+  try (Cluster cluster =
+  getCluster(hosts(), port(), username(), password(), localDc(), 
consistencyLevel())) {
+if (isMurmur3Partitioner(cluster)) {
+  LOG.info("Murmur3Partitioner detected, splitting");
+
+  List tokens =
 
 Review comment:
   For maintenance coherence in DoFn based IOs we follow a pattern where the 
first part Creates the required object to query (the one I mentioned above), 
the second splits the query into multiples queries for each partition via a 
`SplitFn extends DoFn` and then with the result of such splits we do a read 
equivalent to current `QueryFn`. Can you refactor this segment to behave like 
that (with the query object I mentioned above).
 

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: 374661)

> 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
>  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 accounts = getRelevantAccounts();
> Set dateRange = generateDateRange("2018-01-01", "2019-01-01");
> PCollection tokens = generateTokens(accounts, dateRange); 
> {code}
>  
>  I propose an additional _readAll()_ PTransform that can take a PCollection 
> of token ranges and can return a PCollection 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)


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

2020-01-20 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on BEAM-9008:


Author: ASF GitHub Bot
Created on: 20/Jan/20 16:36
Start Date: 20/Jan/20 16:36
Worklog Time Spent: 10m 
  Work Description: iemejia commented on pull request #10546: [BEAM-9008] 
add CassandraIO readAll method
URL: https://github.com/apache/beam/pull/10546#discussion_r368626515
 
 

 ##
 File path: 
sdks/java/io/cassandra/src/test/java/org/apache/beam/sdk/io/cassandra/CassandraIOTest.java
 ##
 @@ -312,6 +298,52 @@ public void testRead() throws Exception {
 pipeline.run();
   }
 
+  @Test
+  public void testReadAll() {
+
 
 Review comment:
   nit: remove space
 

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: 374656)
Time Spent: 1h 50m  (was: 1h 40m)

> 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: 1h 50m
>  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 accounts = getRelevantAccounts();
> Set dateRange = generateDateRange("2018-01-01", "2019-01-01");
> PCollection tokens = generateTokens(accounts, dateRange); 
> {code}
>  
>  I propose an additional _readAll()_ PTransform that can take a PCollection 
> of token ranges and can return a PCollection 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)


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

2020-01-20 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on BEAM-9008:


Author: ASF GitHub Bot
Created on: 20/Jan/20 16:36
Start Date: 20/Jan/20 16:36
Worklog Time Spent: 10m 
  Work Description: iemejia commented on pull request #10546: [BEAM-9008] 
add CassandraIO readAll method
URL: https://github.com/apache/beam/pull/10546#discussion_r367999517
 
 

 ##
 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 extends DoFn, T> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(CassandraIO.class);
+
+  private final CassandraConfig read;
+
+  private transient Cluster cluster;
+
+  private transient Session session;
+
+  private String partitionKey;
+
+  public QueryFn(CassandraConfig read) {
+this.read = read;
+  }
+
+  @Setup
+  public void setup() {
+this.cluster =
+CassandraIO.getCluster(
+read.hosts(),
+read.port(),
+read.username(),
+read.password(),
+read.localDc(),
+read.consistencyLevel());
+this.session = this.cluster.connect(read.keyspace().get());
+this.partitionKey =
+
cluster.getMetadata().getKeyspace(read.keyspace().get()).getTable(read.table().get())
+.getPartitionKey().stream()
+.map(ColumnMetadata::getName)
+.collect(Collectors.joining(","));
+  }
+
+  public void teardown() {
+this.session.close();
+this.cluster.close();
+  }
+
+  @ProcessElement
+  public void processElement(@Element Iterable tokens, 
OutputReceiver receiver) {
+
 
 Review comment:
   nit: remove space
 

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: 374657)
Time Spent: 1h 50m  (was: 1h 40m)

> 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: 1h 50m
>  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 accounts = getRelevantAccounts();
> Set dateRange = generateDateRange("2018-01-01", "2019-01-01");
> PCollection tokens = generateTokens(accounts, dateRange); 
> {code}
>  
>  I propose an additional _readAll()_ PTransform that can take a PCollection 

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

2020-01-20 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on BEAM-9008:


Author: ASF GitHub Bot
Created on: 20/Jan/20 16:36
Start Date: 20/Jan/20 16:36
Worklog Time Spent: 10m 
  Work Description: iemejia commented on pull request #10546: [BEAM-9008] 
add CassandraIO readAll method
URL: https://github.com/apache/beam/pull/10546#discussion_r368619172
 
 

 ##
 File path: 
sdks/java/io/cassandra/src/main/java/org/apache/beam/sdk/io/cassandra/CassandraConfig.java
 ##
 @@ -0,0 +1,88 @@
+/*
+ * 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.Session;
+import com.google.auto.value.AutoValue;
+import java.io.Serializable;
+import java.util.List;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.options.ValueProvider;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+
+@AutoValue
+abstract class CassandraConfig implements Serializable {
+  @Nullable
+  abstract ValueProvider> hosts();
+
+  @Nullable
+  abstract ValueProvider query();
 
 Review comment:
   We should divide this object into two: One that has the parameters we need 
to query elements and other that has just the connection part. The goal is that 
the query part will be sufficient to do a read and to be splitted if necessary, 
with this we might have a richer set of options than just with `RingRange` for 
example to have ReadAll query multiple tables. This object probably should 
include: query, keyspace, table, entity and RingRange.
 

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: 374658)
Time Spent: 1h 50m  (was: 1h 40m)

> 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: 1h 50m
>  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 accounts = getRelevantAccounts();
> Set dateRange = generateDateRange("2018-01-01", "2019-01-01");
> PCollection tokens = generateTokens(accounts, dateRange); 
> {code}
>  
>  I propose an additional _readAll()_ PTransform that can take a PCollection 
> of token ranges and can return a PCollection 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* 

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

2020-01-20 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on BEAM-9008:


Author: ASF GitHub Bot
Created on: 20/Jan/20 16:36
Start Date: 20/Jan/20 16:36
Worklog Time Spent: 10m 
  Work Description: iemejia commented on pull request #10546: [BEAM-9008] 
add CassandraIO readAll method
URL: https://github.com/apache/beam/pull/10546#discussion_r367999232
 
 

 ##
 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 extends DoFn, T> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(CassandraIO.class);
+
+  private final CassandraConfig read;
+
+  private transient Cluster cluster;
+
+  private transient Session session;
+
+  private String partitionKey;
+
+  public QueryFn(CassandraConfig read) {
 
 Review comment:
   Remove public
 

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: 374651)
Time Spent: 1.5h  (was: 1h 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: 1.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 accounts = getRelevantAccounts();
> Set dateRange = generateDateRange("2018-01-01", "2019-01-01");
> PCollection tokens = generateTokens(accounts, dateRange); 
> {code}
>  
>  I propose an additional _readAll()_ PTransform that can take a PCollection 
> of token ranges and can return a PCollection 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 

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

2020-01-20 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on BEAM-9008:


Author: ASF GitHub Bot
Created on: 20/Jan/20 16:36
Start Date: 20/Jan/20 16:36
Worklog Time Spent: 10m 
  Work Description: iemejia commented on pull request #10546: [BEAM-9008] 
add CassandraIO readAll method
URL: https://github.com/apache/beam/pull/10546#discussion_r368000993
 
 

 ##
 File path: 
sdks/java/io/cassandra/src/main/java/org/apache/beam/sdk/io/cassandra/RingRange.java
 ##
 @@ -17,10 +17,11 @@
  */
 package org.apache.beam.sdk.io.cassandra;
 
+import java.io.Serializable;
 import java.math.BigInteger;
 
 /** Models a Cassandra token range. */
-final class RingRange {
+final class RingRange implements Serializable {
 
 Review comment:
   Since RingRange may end up been compared for equality too can you please add 
equals/hashcode too. And add a test that tests that Serializability and 
equality works in Beam we have the 
`SerializableUtils.ensureSerializableRoundTrip` that should make 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:
us...@infra.apache.org


Issue Time Tracking
---

Worklog Id: (was: 374655)
Time Spent: 1h 50m  (was: 1h 40m)

> 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: 1h 50m
>  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 accounts = getRelevantAccounts();
> Set dateRange = generateDateRange("2018-01-01", "2019-01-01");
> PCollection tokens = generateTokens(accounts, dateRange); 
> {code}
>  
>  I propose an additional _readAll()_ PTransform that can take a PCollection 
> of token ranges and can return a PCollection 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)


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

2020-01-20 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on BEAM-9008:


Author: ASF GitHub Bot
Created on: 20/Jan/20 16:36
Start Date: 20/Jan/20 16:36
Worklog Time Spent: 10m 
  Work Description: iemejia commented on pull request #10546: [BEAM-9008] 
add CassandraIO readAll method
URL: https://github.com/apache/beam/pull/10546#discussion_r368011332
 
 

 ##
 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 extends DoFn, T> {
 
 Review comment:
   this can be the flatted version no? `DoFn` or actually the new 
Query objectr I mention above `DoFn`.
 

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: 374649)
Time Spent: 1.5h  (was: 1h 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: 1.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 accounts = getRelevantAccounts();
> Set dateRange = generateDateRange("2018-01-01", "2019-01-01");
> PCollection tokens = generateTokens(accounts, dateRange); 
> {code}
>  
>  I propose an additional _readAll()_ PTransform that can take a PCollection 
> of token ranges and can return a PCollection 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 

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

2020-01-20 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on BEAM-9008:


Author: ASF GitHub Bot
Created on: 20/Jan/20 16:36
Start Date: 20/Jan/20 16:36
Worklog Time Spent: 10m 
  Work Description: iemejia commented on pull request #10546: [BEAM-9008] 
add CassandraIO readAll method
URL: https://github.com/apache/beam/pull/10546#discussion_r367998534
 
 

 ##
 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 extends DoFn, T> {
 
 Review comment:
   remove public and rename to ReadFn since the goal here would be just to Read
 

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: 374646)
Time Spent: 1h 10m  (was: 1h)

> 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: 1h 10m
>  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 accounts = getRelevantAccounts();
> Set dateRange = generateDateRange("2018-01-01", "2019-01-01");
> PCollection tokens = generateTokens(accounts, dateRange); 
> {code}
>  
>  I propose an additional _readAll()_ PTransform that can take a PCollection 
> of token ranges and can return a PCollection 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 

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

2020-01-20 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on BEAM-9008:


Author: ASF GitHub Bot
Created on: 20/Jan/20 16:36
Start Date: 20/Jan/20 16:36
Worklog Time Spent: 10m 
  Work Description: iemejia commented on pull request #10546: [BEAM-9008] 
add CassandraIO readAll method
URL: https://github.com/apache/beam/pull/10546#discussion_r368013780
 
 

 ##
 File path: 
sdks/java/io/cassandra/src/main/java/org/apache/beam/sdk/io/cassandra/CassandraIO.java
 ##
 @@ -732,6 +505,35 @@ public T getCurrent() throws NoSuchElementException {
 DELETE
   }
 
+  /**
+   * Check if the current partitioner is the Murmur3 (default in Cassandra 
version newer than 2).
+   */
+  @VisibleForTesting
+  static boolean isMurmur3Partitioner(Cluster cluster) {
+return MURMUR3PARTITIONER.equals(cluster.getMetadata().getPartitioner());
+  }
+
+  static String generateRangeQuery(CassandraConfig spec, String partitionKey) {
+final String rangeFilter =
+Joiner.on(" AND ")
+.skipNulls()
+.join(
+String.format("(token(%s) >= ?)", partitionKey),
+String.format("(token(%s) < ?)", partitionKey));
+final String query =
+(spec.query() == null)
+? buildQuery(spec) + " WHERE " + rangeFilter
+: buildQuery(spec) + " AND " + rangeFilter;
+LOG.debug("CassandraIO generated query : {}", query);
+return query;
+  }
+
+  private static String buildQuery(CassandraConfig spec) {
 
 Review comment:
   Same for this one, move to `QueryFn`
 

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: 374660)
Time Spent: 2h  (was: 1h 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: 2h
>  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 accounts = getRelevantAccounts();
> Set dateRange = generateDateRange("2018-01-01", "2019-01-01");
> PCollection tokens = generateTokens(accounts, dateRange); 
> {code}
>  
>  I propose an additional _readAll()_ PTransform that can take a PCollection 
> of token ranges and can return a PCollection 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)


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

2020-01-20 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on BEAM-9008:


Author: ASF GitHub Bot
Created on: 20/Jan/20 16:36
Start Date: 20/Jan/20 16:36
Worklog Time Spent: 10m 
  Work Description: iemejia commented on pull request #10546: [BEAM-9008] 
add CassandraIO readAll method
URL: https://github.com/apache/beam/pull/10546#discussion_r365717276
 
 

 ##
 File path: 
sdks/java/io/cassandra/src/main/java/org/apache/beam/sdk/io/cassandra/CassandraConfig.java
 ##
 @@ -0,0 +1,60 @@
+package org.apache.beam.sdk.io.cassandra;
+
+import com.datastax.driver.core.Session;
+import com.google.auto.value.AutoValue;
+import java.io.Serializable;
+import java.util.List;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.options.ValueProvider;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+
+@AutoValue
+abstract class CassandraConfig implements Serializable {
+  @Nullable
+  abstract ValueProvider> hosts();
+
+  @Nullable
+  abstract ValueProvider query();
+
+  @Nullable
+  abstract ValueProvider port();
+
+  @Nullable
+  abstract ValueProvider keyspace();
+
+  @Nullable
+  abstract ValueProvider table();
+
+  @Nullable
+  abstract ValueProvider username();
+
+  @Nullable
+  abstract ValueProvider password();
+
+  @Nullable
+  abstract ValueProvider localDc();
+
+  @Nullable
+  abstract ValueProvider consistencyLevel();
+
+  @Nullable
+  abstract SerializableFunction mapperFactoryFn();
+
+  @Nullable
+  abstract Class entity();
+
+  public static  CassandraConfig Create(ValueProvider> 
hosts,
 
 Review comment:
   lowercase c
 

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: 374650)
Time Spent: 1.5h  (was: 1h 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: 1.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 accounts = getRelevantAccounts();
> Set dateRange = generateDateRange("2018-01-01", "2019-01-01");
> PCollection tokens = generateTokens(accounts, dateRange); 
> {code}
>  
>  I propose an additional _readAll()_ PTransform that can take a PCollection 
> of token ranges and can return a PCollection 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)


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

2020-01-17 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on BEAM-9008:


Author: ASF GitHub Bot
Created on: 17/Jan/20 15:58
Start Date: 17/Jan/20 15:58
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-57568
 
 
   Working still in the review sorry for the delay @vmarquez 
 

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: 373686)
Time Spent: 1h  (was: 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: 1h
>  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 accounts = getRelevantAccounts();
> Set dateRange = generateDateRange("2018-01-01", "2019-01-01");
> PCollection tokens = generateTokens(accounts, dateRange); 
> {code}
>  
>  I propose an additional _readAll()_ PTransform that can take a PCollection 
> of token ranges and can return a PCollection 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)


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

2020-01-17 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on BEAM-9008:


Author: ASF GitHub Bot
Created on: 17/Jan/20 13:54
Start Date: 17/Jan/20 13:54
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-575634734
 
 
   Arrghh unit tests did not run because of the issues with jenkins this week, 
running again.
 

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: 373623)
Time Spent: 50m  (was: 40m)

> 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: 50m
>  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 accounts = getRelevantAccounts();
> Set dateRange = generateDateRange("2018-01-01", "2019-01-01");
> PCollection tokens = generateTokens(accounts, dateRange); 
> {code}
>  
>  I propose an additional _readAll()_ PTransform that can take a PCollection 
> of token ranges and can return a PCollection 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)


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

2020-01-14 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on BEAM-9008:


Author: ASF GitHub Bot
Created on: 15/Jan/20 03:04
Start Date: 15/Jan/20 03:04
Worklog Time Spent: 10m 
  Work Description: vmarquez commented on issue #10546: [BEAM-9008] add 
CassandraIO readAll method
URL: https://github.com/apache/beam/pull/10546#issuecomment-574474228
 
 
   @suztomo @iemejia Thank you for pointing that out, I was not aware of the 
formatting issues.  Maybe after this MR I can add some of that to the 
contributing document. :)  I fixed the style and formatting issues, and will 
await other change requests. 
 

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: 372099)
Time Spent: 40m  (was: 0.5h)

> 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: 40m
>  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 accounts = getRelevantAccounts();
> Set dateRange = generateDateRange("2018-01-01", "2019-01-01");
> PCollection tokens = generateTokens(accounts, dateRange); 
> {code}
>  
>  I propose an additional _readAll()_ PTransform that can take a PCollection 
> of token ranges and can return a PCollection 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)


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

2020-01-13 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on BEAM-9008:


Author: ASF GitHub Bot
Created on: 13/Jan/20 09:54
Start Date: 13/Jan/20 09:54
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-573583696
 
 
   Arrghh tests for this one did not start, for that reason you had not seen 
the failures that @suztomo mentioned.
   You can auto fix the code format issues by running `./gradlew 
:sdks:java:io:cassandra:check spotlessApply`.
   I will start letting some review comments in the meantime
 

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: 370686)
Time Spent: 0.5h  (was: 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: 0.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 accounts = getRelevantAccounts();
> Set dateRange = generateDateRange("2018-01-01", "2019-01-01");
> PCollection tokens = generateTokens(accounts, dateRange); 
> {code}
>  
>  I propose an additional _readAll()_ PTransform that can take a PCollection 
> of token ranges and can return a PCollection 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)


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

2020-01-12 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on BEAM-9008:


Author: ASF GitHub Bot
Created on: 13/Jan/20 02:24
Start Date: 13/Jan/20 02:24
Worklog Time Spent: 10m 
  Work Description: suztomo commented on issue #10546: [BEAM-9008] add 
CassandraIO readAll method
URL: https://github.com/apache/beam/pull/10546#issuecomment-573487705
 
 
   Would you ensure the change passes ` ./gradlew -p sdks/java check`? I see it 
fails with format violations:
   
   ```
   suztomo-macbookpro44:vmarquez-beam suztomo$ git log -1
   commit d85e6b95d64664af37148c792b35ff02abfd90f5 (HEAD -> 
feature/BEAM-9008/cassandraio_readall, 
origin/feature/BEAM-9008/cassandraio_readall)
   Author: Vincent Marquez 
   Date:   Wed Dec 18 16:48:52 2019 -0800
   
   implementing CassandraIO.readAll
   suztomo-macbookpro44:vmarquez-beam suztomo$ ./gradlew -p sdks/java check
   Configuration on demand is an incubating feature.
   
   ...
   
   * What went wrong:
   Execution failed for task ':sdks:java:io:cassandra:spotlessJava'.
   > The following files had format violations:
 
sdks/java/io/cassandra/src/test/java/org/apache/beam/sdk/io/cassandra/CassandraIOTest.java
 @@ -184,31 +184,33 @@
  
  LOGGER.info("Insert·records");
  String[][]·scientists·=·{
 -··new·String·[]·{"phys",·"Einstein"},
 -··new·String·[]·{"bio",·"Darwin"},
 -··new·String·[]·{"phys",·"Copernicus"},
 -··new·String·[]·{"bio",·"Pasteur"},
 -··new·String·[]·{"bio",·"Curie"},
 -··new·String·[]·{"phys",·"Faraday"},
 -··new·String·[]·{"math",·"Newton"},
 -··new·String·[]·{"phys",·"Bohr"},
 -··new·String·[]·{"phys",·"Galilei"},
 -··new·String·[]·{·"math",·"Maxwell"},
 +··new·String[]·{"phys",·"Einstein"},
 +··new·String[]·{"bio",·"Darwin"},
 +··new·String[]·{"phys",·"Copernicus"},
 +··new·String[]·{"bio",·"Pasteur"},
 +··new·String[]·{"bio",·"Curie"},
 +··new·String[]·{"phys",·"Faraday"},
 +··new·String[]·{"math",·"Newton"},
 +··new·String[]·{"phys",·"Bohr"},
 +··new·String[]·{"phys",·"Galilei"},
 +··new·String[]·{"math",·"Maxwell"},
  };
  for·(int·i·=·0;·i·<·NUM_ROWS;·i++)·{
  ··int·index·=·i·%·scientists.length;
 -··String·insertStr·=·String.format(
 
-··"INSERT·INTO·%s.%s(person_department,·person_id,·person_name)·values("
 -··+·"'"·+·scientists[index][0]·+·"',·"
 -··+·i
 -··+·",·'"
 -··+·scientists[index][1]
 -··+·"');",
 -··CASSANDRA_KEYSPACE,
 -··CASSANDRA_TABLE);
 -LOGGER.error("Error·with·str·=·"·+·insertStr);
 -··session.execute(insertStr
 -··);
 +··String·insertStr·=
 +··String.format(
 
+··"INSERT·INTO·%s.%s(person_department,·person_id,·person_name)·values("
 +··+·"'"
 +··+·scientists[index][0]
 +··+·"',·"
 +··+·i
 +··+·",·'"
 +··+·scientists[index][1]
 +··+·"');",
 ... (102 more lines that didn't fit)
 Violations also present in:
 
sdks/java/io/cassandra/src/main/java/org/apache/beam/sdk/io/cassandra/QueryFn.java
 
sdks/java/io/cassandra/src/main/java/org/apache/beam/sdk/io/cassandra/CassandraConfig.java
 
sdks/java/io/cassandra/src/main/java/org/apache/beam/sdk/io/cassandra/CassandraIO.java
 Run 'gradlew spotlessApply' to fix these violations.
   ```
 

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: 370558)
Time Spent: 20m  (was: 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 

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

2020-01-09 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot logged work on BEAM-9008:


Author: ASF GitHub Bot
Created on: 09/Jan/20 22:30
Start Date: 09/Jan/20 22:30
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
 
 
   This PR adds a readAll method that returns a `PTransform, 
A>` to allow for parallel querying of a small subset of a Cassandra databse.  
In addition, it 
refactors the current readAll PTransform to be a ParDo based one, so we can 
share code between read and readAll.  
   
   Note: One thing I wasn't sure about was how to abstract over AutoBuilder 
classes.  I'm much more familiar with Scala than Java, so I wasn't sure if 
there was a better way than what I did with having a `getCassandraConfig` on 
both the static Read class as well as the ReadAll class.  Don't think I can use 
an interface since I don't want the methods public.  Open to suggestions here. 
   
   Thanks in advance for being patient with a large change and please let me 
know if I can change/add anything!  
   
   R:@iemejia 
   

   
   Lang | SDK | Apex | Dataflow | Flink | Gearpump | Samza | Spark
   --- | --- | --- | --- | --- | --- | --- | ---
   Go | [![Build 
Status](https://builds.apache.org/job/beam_PostCommit_Go/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Go/lastCompletedBuild/)
 | --- | --- | [![Build 
Status](https://builds.apache.org/job/beam_PostCommit_Go_VR_Flink/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Go_VR_Flink/lastCompletedBuild/)
 | --- | --- | [![Build 
Status](https://builds.apache.org/job/beam_PostCommit_Go_VR_Spark/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Go_VR_Spark/lastCompletedBuild/)
   Java | [![Build 
Status](https://builds.apache.org/job/beam_PostCommit_Java/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Java/lastCompletedBuild/)
 | [![Build 
Status](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Apex/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Apex/lastCompletedBuild/)
 | [![Build 
Status](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Dataflow/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Dataflow/lastCompletedBuild/)
 | [![Build 
Status](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Flink/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Flink/lastCompletedBuild/)[![Build
 
Status](https://builds.apache.org/job/beam_PostCommit_Java_PVR_Flink_Batch/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Java_PVR_Flink_Batch/lastCompletedBuild/)[![Build
 
Status](https://builds.apache.org/job/beam_PostCommit_Java_PVR_Flink_Streaming/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Java_PVR_Flink_Streaming/lastCompletedBuild/)
 | [![Build 
Status](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Gearpump/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Gearpump/lastCompletedBuild/)
 | [![Build 
Status](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Samza/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Samza/lastCompletedBuild/)
 | [![Build 
Status](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Spark/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Spark/lastCompletedBuild/)[![Build
 
Status](https://builds.apache.org/job/beam_PostCommit_Java_PVR_Spark_Batch/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Java_PVR_Spark_Batch/lastCompletedBuild/)[![Build
 
Status](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming/lastCompletedBuild/)
   Python | [![Build 
Status](https://builds.apache.org/job/beam_PostCommit_Python2/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Python2/lastCompletedBuild/)[![Build
 
Status](https://builds.apache.org/job/beam_PostCommit_Python35/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Python35/lastCompletedBuild/)[![Build