Hello, My excuses for not having commented on this thread before. Thanks for bringing this new IO connector!
After reading the proposal I think we need to create this as a new independent CassandraIO (v4) IO connector different from the existing one based on v3 for the following reasons: 1. The new API follows a different programming model that is binary incompatible with existing clients, so if we don't want to break existing users we need to put it in a different module e.g. `sdks/java/cassandra4/` (we can keep the same name CassandraIO). The alternative would be to keep it in the same module and expose it via a different API for example `CassandraIO.readV4()` but sharing the same module brings the inconvenience of having to deal with the legacy v3 dependencies and implementation constraints. (I don't understand yet why the PR still keeps the v3 dependencies, is for this reason?) 2. Being independent will allow us to design a more appropriate API for Cassandra 4 without the previous constraints, if users want to migrate to v4 they will have to do the adaptation themselves and we (Beam) will be free of the responsability of providing API guarantees or a migration path because both Beam APIs would be independent. Of course the tax we have to pay on the Beam side is to keep maintaining both versions for a while, but given how slowly the existing CassandraIO connector has changed in the last years [1] I would not expect many changes coming on the v3 version and most new ones to target v4. 3. Be able to run tests with a Cassandra v4 cluster. The current tests in the PR are still targeting an older version of Cassandra. Of course the Cassandra v4 client is able to connect to earlier versions of Cassandra [2] but one big selling point of moving to use the Cassandra v4 dependencies is to support the most recent version of the Cassandra cluster so it is in our interest to target it in the tests. Finally I would suggest to adapt this new version to be based on a DoFn translation and follow the ReadAll pattern (ideally if possible using SplittableDoFn but not mandatory). We have been in the process of refactoring the existing CassandraIO (v3) with Vincent Marquez for some months and this should be hopefully finished soon so you can take this as a reference [3], and of course you can contact me (us) in case of questions on details. (of course this is optional at the moment but good to have). Regards, Ismaël [1] https://github.com/apache/beam/commits/master/sdks/java/io/cassandra/src/main/java/org/apache/beam/sdk/io/cassandra/CassandraIO.java [2] https://docs.datastax.com/en/driver-matrix/doc/driver_matrix/common/versionCompatibility.html [3] https://github.com/vmarquez/beam/tree/feature/BEAM-9008%2Fcassandraio_readall On Tue, Apr 20, 2021 at 7:48 AM D, Anup (Nokia - IN/Bangalore) < [email protected]> wrote: > Hi All, > > > > Satwik and myself have been working together on this. > > 4.x has been a major revamp and we have highlighted below major > differences that were seen during this activity. > > Please review and provide feedback. > > > > 1. Package names : > > 3.x : com.datastax.cassandra > > 4.x : com.datastax.oss > > Comment : 4.x is different from 3.x. We think both can co-exist. Please > see JanusGraph who have included both the packages for reference [1] > > > > 1. Mapping : > > 3.x : Default Object Mapper took care of mapping all Entity types at > runtime - org.apache.beam.sdk.io.cassandra.DefaultObjectMapper > > 4.x : Mapper auto-generates helper classes during compile time by > processing annotations on Mapper,Dao and Entity. Then, use either a > specific Dao or Generic Dao to access/map classes.[2][3] > > Comment : With objective to avoid/limit breaking changes, we could find > providing a Generic/Base Dao via inheritance has limited breakage.[4] > > Impacts : > > 1. Requires mapperFactoryFunction to be mandatorily supplied that can > return SpecificDao reference. > 2. @GetEntity is the annotation that maps ResultSet to Entity which > performs strict column checking among the two. This was not the case in > 3.x. We had posted query to Cassandra community [5] > > > > 1. HadoopFormatIO > > Unit test in HadoopFormatIO that interacts with Cassandra failed when > driver was upgraded to 4.x. Latest Apache Cassandra server still uses 3.x > Cassandra connector. > > There is an open JIRA [6][7] > > > > 1. Load Balancing policy > > 3.x : Providing data center name is optional. > > 4.x : Load balancing policies have been revamped. Providing data center > name is mandatory.[8] > > > > 1. Configuration > > 3.x : This was done by configuring classes. > > 4.x : Along with configuring classes, file-based configuration is > supported. [9][10] > > Comment : We did test loading some part of configuration via file and some > programmatically. There is no impact as such but this is a new > complimenting feature . > > > > 1. Driver compatibility > > Cassandra 4.5+ drivers are fully compatible with Apache Cassandra 2.1+ > versions.[11] > > The open source driver implementatation “com.datastax.oss” will be > supported for interacting with Open source, commercial Cassandra > > There is no impact but highlighting > > > > [1] Update Cassandra driver to 4.x version · Issue #1510 · > JanusGraph/janusgraph (github.com) > <https://github.com/JanusGraph/janusgraph/issues/1510> > > [2] > https://stackoverflow.com/questions/34701817/what-is-the-most-efficient-way-to-map-transform-cast-a-cassandra-boundstatement > > [3] > https://docs.datastax.com/en/developer/java-driver/4.5/upgrade_guide/#object-mapper > > [4] > https://stackoverflow.com/questions/61298743/genericdao-on-datastax-java-driver-4 > > [5] cassandra - Strict column checking in Datastax java driver 4 causing > problems - Stack Overflow > <https://stackoverflow.com/questions/66985742/strict-column-checking-in-datastax-java-driver-4-causing-problems> > > [6] https://issues.apache.org/jira/browse/CASSANDRA-15750 > > [7] > https://javadoc.io/doc/org.apache.cassandra/cassandra-all/latest/org/apache/cassandra/hadoop/cql3/CqlInputFormat.html > > [8] > https://docs.datastax.com/en/developer/java-driver/4.10/manual/core/load_balancing/ > > [9] > https://github.com/datastax/java-driver/tree/4.0.0/upgrade_guide#configuration > > [10] > https://docs.datastax.com/en/developer/java-driver/4.0/manual/core/configuration/ > > [11] > https://docs.datastax.com/en/driver-matrix/doc/driver_matrix/javaDrivers.html > > > > > > Thanks > > Anup > > > > *From:* Alexey Romanenko <[email protected]> > *Sent:* Friday, April 16, 2021 11:02 PM > *To:* [email protected] > *Subject:* Re: [PROPOSAL] Upgrade Cassandra driver from 3.x to 4.x in > CassandraIO > > > > Thank you for design doc and starting a discussion on mailing list! > > > > I’m the next after Kenn to ask about the potential breaking changes with > this upgrade. Could you elaborate a bit on this and can we support both > versions in the same time? > > > > Alexey > > > > On 15 Apr 2021, at 12:32, S Bhandiwad, Satwik (Nokia - IN/Bangalore) < > [email protected]> wrote: > > > > Hi All, > > > > We would like to upgrade Cassandra driver version from 3.x to 4.x in > CassandraIO Connector. > > Design Document - link > <https://docs.google.com/document/d/1qsNksUJ_a6PL623iBZ-3QQDkFae81IKbqVQl1chsOfU/edit?usp=sharing> > > Pull Request - https://github.com/apache/beam/pull/14457/ > > > > Please go through the design doc & PR and let us know your thoughts. > > > > Regards, > > Satwik > > >
