> On 26 Apr 2021, at 20:15, S Bhandiwad, Satwik (Nokia - IN/Bangalore) > <[email protected]> wrote: > > Point 1: I have updated the impact section of the design doc with all the > breaking changes for users. link > <https://docs.google.com/document/d/1qsNksUJ_a6PL623iBZ-3QQDkFae81IKbqVQl1chsOfU/edit?usp=sharing> Thank you!
I think we should be fine with these required changes since CassandraIO is still labeled with “@Experimental(Kind.SOURCE_SINK)” but we need to mention these in CHANGES.md CC: Kenn, Etienne, Ismaël Wdyt? > Point 2: We have ran only Integration Test, let us know if you have some > suggestions we'll try to do it. Actually, Beam tends to have "2-in-1" - integration tests can be used as simple performance tests as well by specifying a different number of records by CLI option "—numberOfRecords”. It would be great if you could run the same configuration of this test [1] with significant number of records against two different Cassandra driver versions. --- Alexey [1] https://github.com/apache/beam/blob/master/sdks/java/io/cassandra/src/test/java/org/apache/beam/sdk/io/cassandra/CassandraIOIT.java > > Regards, > Satwik > > > > From: Alexey Romanenko <[email protected] > <mailto:[email protected]>> > Sent: Thursday, April 22, 2021 7:58 PM > To: [email protected] <mailto:[email protected]> > Subject: Re: [PROPOSAL] Upgrade Cassandra driver from 3.x to 4.x in > CassandraIO > > Thanks, it looks promising! > > I just have a couple things to ask. > > 1) Could you briefly summarise and add here or/and to design doc all breaking > changes for users that you expect (if any)? Can we avoid them, at least, > maybe temporary? For example, we used to deprecate an old public API and keep > it for the next three Beam releases before removing it completely. > > 2) Also, did you run any load tests to compare the performance between two > driver versions for the same pipeline and datasets? If yes, could you share > the results, please? > > > -- > Alexey > > > On 20 Apr 2021, at 07:47, D, Anup (Nokia - IN/Bangalore) <[email protected] > <mailto:[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. > > 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] > > 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 : > a. Requires mapperFactoryFunction to be mandatorily supplied that can > return SpecificDao reference. > b. @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] > > 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] > > 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] > > 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 . > > 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 > > <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 > > <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 > > <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 > <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 > > <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/ > > <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 > > <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/ > > <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 > <https://docs.datastax.com/en/driver-matrix/doc/driver_matrix/javaDrivers.html> > > > Thanks > Anup > > From: Alexey Romanenko <[email protected] > <mailto:[email protected]>> > Sent: Friday, April 16, 2021 11:02 PM > To: [email protected] <mailto:[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] <mailto:[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/ > <https://github.com/apache/beam/pull/14457/> > > Please go through the design doc & PR and let us know your thoughts. > > Regards, > Satwik >
