[ 
https://issues.apache.org/jira/browse/BEAM-6324?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16804605#comment-16804605
 ] 

Shahar Frank commented on BEAM-6324:
------------------------------------

The PR introduces a problem IMHO:

The code:
  
{code:java}
p.apply(CassandraIO.<Table1>read().withHosts(Arrays.asList(config.cassandraHosts()))
.withPort(config.cassandraPort()).withEntity(Table1.class).withCoder(SerializableCoder.of(Table1.class))
                .withKeyspace(config.cassandraKeyspace())
                .withTable(config.cassandraTable1())
                .withQuery(String.format("select * from %s.%s where 
data='Whether'",
                                config.cassandraKeyspace(), 
config.cassandraTable1())))
{code}
 
Will not work if the 'withKeyspace' line is commented:
{code:java}
Exception in thread "main" java.lang.IllegalArgumentException: withKeyspace() 
is required
        at 
org.apache.beam.vendor.guava.v20_0.com.google.common.base.Preconditions.checkArgument(Preconditions.java:122)
        at 
org.apache.beam.sdk.io.cassandra.CassandraIO$Read.expand(CassandraIO.java:323)
        at 
org.apache.beam.sdk.io.cassandra.CassandraIO$Read.expand(CassandraIO.java:132)
        at org.apache.beam.sdk.Pipeline.applyInternal(Pipeline.java:537)
        at org.apache.beam.sdk.Pipeline.applyTransform(Pipeline.java:471)
        at org.apache.beam.sdk.values.PBegin.apply(PBegin.java:44)
        at org.apache.beam.sdk.Pipeline.apply(Pipeline.java:167)
        at org.apache.beam.examples.ReadCassandra.main(ReadCassandra.java:16)
{code}

Nor with 'withTable ' commented:
{code:java}
Exception in thread "main" java.lang.IllegalArgumentException: withTable() is 
required
        at 
org.apache.beam.vendor.guava.v20_0.com.google.common.base.Preconditions.checkArgument(Preconditions.java:122)
        at 
org.apache.beam.sdk.io.cassandra.CassandraIO$Read.expand(CassandraIO.java:324)
        at 
org.apache.beam.sdk.io.cassandra.CassandraIO$Read.expand(CassandraIO.java:132)
        at org.apache.beam.sdk.Pipeline.applyInternal(Pipeline.java:537)
        at org.apache.beam.sdk.Pipeline.applyTransform(Pipeline.java:471)
        at org.apache.beam.sdk.values.PBegin.apply(PBegin.java:44)
        at org.apache.beam.sdk.Pipeline.apply(Pipeline.java:167)
        at org.apache.beam.examples.ReadCassandra.main(ReadCassandra.java:16)
{code}

I think this redundancy may lead to many user errors. E.g. what happens if the 
input to `withTable` is not the same as the table used within the query?

WDYT?

> CassandraIO.Read - Add the ability to provide a filter to the query
> -------------------------------------------------------------------
>
>                 Key: BEAM-6324
>                 URL: https://issues.apache.org/jira/browse/BEAM-6324
>             Project: Beam
>          Issue Type: Improvement
>          Components: io-java-cassandra
>            Reporter: Shahar Frank
>            Assignee: Shahar Frank
>            Priority: Major
>              Labels: performance, pull-request-available, triaged
>             Fix For: Not applicable
>
>          Time Spent: 12h 50m
>  Remaining Estimate: 0h
>
> CassandraIO.Read doesn't support using WHERE to filter the input at the 
> source (In Cassandra) which might provide great performance boost.
> Already implemented by:
> https://github.com/apache/beam/pull/7340



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to