Skip to content

Use Datastax Cassandra java driver v4#7828

Merged
ebyhr merged 2 commits intotrinodb:masterfrom
findinpath:task/7729
May 2, 2022
Merged

Use Datastax Cassandra java driver v4#7828
ebyhr merged 2 commits intotrinodb:masterfrom
findinpath:task/7729

Conversation

@findinpath
Copy link
Copy Markdown
Contributor

@findinpath findinpath commented May 3, 2021

Description

This PR addresses #7729

Upgrade to latest java-driver-core version 4.14.0 in order to make use
of the latest improvements in the Cassandra java driver.

The version 4 of the Cassandra java driver is not backwards compatible
with the version 3 although most of the concepts still overlap.

Most of the changes revolve to a very customizable way of building the CqlSession
(which is the replacement for Cluster & Session from the version 3 of the driver).
This is why there are a great deal of changes in the CassandraClientModule class.
Drop the support for WhiteListPolicy because this load balancing policy
is not anymore present in v4 of the Cassandra driver.

HostAddressFactory deals currently only with nodes with endpoints which
resolve to InetSocketAddress.

Keep working with internal String names for tables and schema instead of
the newly introduced CqlIdentifier in order to keep the compatibility
to trino-spi layer.

NOTE that these changes introduce breaking changes in the configuration of the
trino-cassandra connector for the following sections:

  • load balancing policy
  • retry policy type

Is this change a fix, improvement, new feature, refactoring, or other?

Refactoring

Is this a change to the core query engine, a connector, client library, or the SPI interfaces? (be specific)

trino-cassandra connector

How would you describe this change to a non-technical end user or system administrator?

The Datastax Cassandra Java driver is described like:

A modern, feature-rich and highly tunable Java client library for Apache Cassandra® (2.1+) and DataStax Enterprise (4.7+), and DataStax Astra, using exclusively Cassandra’s binary protocol and Cassandra Query Language (CQL) v3.

Pragmatically speaking, the driver for Cassandra 3.x won't get new functionality, only critical bug fixes. So it would be wise to switch to the actively maintained library.

Related issues, pull requests, and links

Fixes #7729

Once this PR is being merged and actively used in trino, the repository https://github.com/trinodb/trino-cassandra-driver can be archived because it will no longer be in use.

Documentation

(x) No documentation is needed.
( ) Sufficient documentation is included in this PR.
( ) Documentation PR is available with #prnumber.
( ) Documentation issue #issuenumber is filed, and can be handled later.

Release notes

( ) No release notes entries required.
(x) Release notes entries required with the following suggested text:

# Cassandra
* Use Datastax java driver v4 to communicate with Cassandra. 
  This change adds supports for v5 and v6 protocol and removes support for v2 protocol. Also, it makes 
  `cassandra.load-policy.use-dc-aware` and `cassandra.load-policy.dc-aware.local-dc` 
  config properties required.  ({issue}`7729`)

@cla-bot cla-bot bot added the cla-signed label May 3, 2021
@findinpath
Copy link
Copy Markdown
Contributor Author

This PR is an initial draft (although the trino-cassandra project builds and should have green tests) and has the purpose of getting feedback on how to shape the connector with Cassandra java driver v4 in order to be production ready.

@findinpath findinpath requested a review from ebyhr May 3, 2021 23:18
@kokosing
Copy link
Copy Markdown
Member

kokosing commented May 4, 2021

Should we have split cassandra plugin into two like we did with phoenix, so we could have two plugins cassandra3 and cassadra5. Higher maintenance cost but no backward compatible changes. WDYT?

@hashhar
Copy link
Copy Markdown
Member

hashhar commented May 4, 2021

The Cassandra version being dropped (2.0.x) has had no new releases since 2015.

The version 4 of the Cassandra java driver is not backwards compatible
with the version 3 although most of the concepts still overlap.

This refers to the driver version. The 4.x driver is compatible with Cassandra 2.1.x which is the oldest active release (released in 2014). (@findinpath It would help to clarify this in the commit message and PR description. Instead of talking about driver versions we can talk about Cassandra versions which is more clear.)

@findinpath
Copy link
Copy Markdown
Contributor Author

findinpath commented May 4, 2021

@findinpath It would help to clarify this in the commit message and PR description. Instead of talking about driver versions we can talk about Cassandra versions which is more clear.

@hashhar I am pretty sure that this commit should probably be split in several other commits to provide more insights for the maintainer in why certain changes have been made. My initial intention was to have a decent draft on which we can start the discussion.

@findinpath
Copy link
Copy Markdown
Contributor Author

findinpath commented May 4, 2021

Should we have split cassandra plugin into two like we did with phoenix, so we could have two plugins cassandra3 and cassadra5.

@kokosing
https://docs.datastax.com/en/developer/java-driver/4.11/

A modern, feature-rich and highly tunable Java client library for Apache Cassandra® (2.1+) and DataStax Enterprise (4.7+), using exclusively Cassandra’s binary protocol and Cassandra Query Language v3.

I am actually more in favor of doing the update of the cassandra client library and take advantage of the richness of configuration options for CqlSession builder.

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I understand that V2 no longer works, am I right? If so we should not allow users to set this.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Indeed @kokosing . Dropping the support for V2 of the protocol was a design decision from Datastax

Please see also: #7729 (comment)

Copy link
Copy Markdown
Contributor Author

@findinpath findinpath May 4, 2021

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The approach for setting the load balancing policy in the v4 version of the driver implies to rethink the appropriate settings for the cassandra trino plugin.

Have a look over https://github.com/datastax/java-driver/blob/4.x/core/src/main/resources/reference.conf for the available options for load balancing policy:

  • DefaultLoadBalancingPolicy
  • BasicLoadBalancingPolicy
  • DcInferringLoadBalancingPolicy
  • ...

NOTE that the current implementation hardcodes the usage of DefaultLoadBalancingPolicy load balancing policy.

@ebyhr
Copy link
Copy Markdown
Member

ebyhr commented Dec 16, 2021

@findinpath Sorry for my late review. Could you rebase on upstream to resolve conflicts?

@robd003
Copy link
Copy Markdown

robd003 commented Dec 16, 2021

@ebyhr Just did a rebase here: https://github.com/robd003/trino

@findinpath
Copy link
Copy Markdown
Contributor Author

findinpath commented Dec 16, 2021

@robd003 thanks for the help.

@ebyhr concerning this PR, the tests were passing at the time when I wrote the PR, but I have identified at the time the fact that cassandra v4 driver offers much much more configuration capabilities compared to cassandra v3 driver.

See #7828 (comment)

The branch has now been rebased on top of master

@findinpath findinpath force-pushed the task/7729 branch 2 times, most recently from afdb0ab to f7efc82 Compare December 16, 2021 10:33
@findinpath findinpath force-pushed the task/7729 branch 2 times, most recently from b7fe0e5 to 5db3b3f Compare January 3, 2022 10:53
@findinpath
Copy link
Copy Markdown
Contributor Author

findinpath commented Jan 3, 2022

@ebyhr regarding the comments related to having more commits, I definitely agree with you.

Initially I considered this PR just a draft and wanted to first gather feedback regarding the implementation. Moreover, I wasn't sure whether this PR has any chance on landing on master.

I would split the commit in more commits as soon as we have a basic agreement on whether this PR makes sense to land on master.

After analysing the content of the commit, it seems that all the files changed are linked to upgrading the cassandra driver version and for this reason I think they should share the same commit. Please advice whether you see a better approach on how to split the commit in more readable / independent commits (which still compile and have the tests running successfully).

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Is here IAE appropriate or rather TrinoException ?

@findinpath findinpath force-pushed the task/7729 branch 2 times, most recently from de7f747 to 416642d Compare January 13, 2022 09:06
@findinpath findinpath requested a review from ebyhr January 13, 2022 09:06
@findinpath findinpath force-pushed the task/7729 branch 2 times, most recently from dc9bff7 to ea1f1e3 Compare January 24, 2022 06:06
Copy link
Copy Markdown
Member

@ebyhr ebyhr left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Could you update cassandra.rst where mentions protocol versions?

@findinpath
Copy link
Copy Markdown
Contributor Author

Rebasing on master to solve compilation issue in TestCassandraTypeMapping.java

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@mosabua FYI

@findinpath findinpath force-pushed the task/7729 branch 3 times, most recently from 244754b to 0891a63 Compare April 30, 2022 17:33
@findinpath findinpath requested a review from ebyhr April 30, 2022 17:34
After extracting the logic related to SSLContext creation to plugin-toolkit
these methods became leftover in `CassandraClientModule`.
Upgrade to latest java-driver-core version 4.14.0 in order to make use
of the latest improvements in the Cassandra java driver.

The version 4 of the Cassandra java driver is not backwards compatible
with the version 3 although most of the concepts still overlap.

Most of the changes revolve to a very customizable way of building the CqlSession
(which is the replacement for Cluster & Session from the version 3 of the driver).
This is why there are a great deal of changes in the CassandraClientModule class.
Drop the support for WhiteListPolicy because this load balancing policy
is not anymore present in v4 of the Cassandra driver.

HostAddressFactory deals currently only with nodes with endpoints which
resolve to InetSocketAddress.

Keep working with internal String names for tables and schema instead of
the newly introduced CqlIdentifier in order to keep the compatibility
to trino-spi layer.

When specifying explicit contact points for Cassandra, it is mandatory
to specify also the local data center by setting correspondingly
the properties:

- cassandra.load-policy.use-dc-aware
- cassandra.load-policy.dc-aware.local-dc
@ebyhr ebyhr merged commit 9f5357c into trinodb:master May 2, 2022
@github-actions github-actions bot added this to the 380 milestone May 2, 2022
@ebyhr ebyhr mentioned this pull request May 2, 2022
tangjiangling added a commit to tangjiangling/trino that referenced this pull request Jul 8, 2022
With the introduction of Cassandra driver v4 via PR trinodb#7828 there are a few
properties that are not anymore needed:

- cassandra.load-policy.use-token-aware
- cassandra.load-policy.token-aware.shuffle-replicas
- cassandra.load-policy.allowed-addresses
ebyhr pushed a commit that referenced this pull request Jul 9, 2022
With the introduction of Cassandra driver v4 via PR #7828 there are a few
properties that are not anymore needed:

- cassandra.load-policy.use-token-aware
- cassandra.load-policy.token-aware.shuffle-replicas
- cassandra.load-policy.allowed-addresses
avaidyanatha pushed a commit to avaidyanatha/trino that referenced this pull request Mar 8, 2023
With the introduction of Cassandra driver v4 via PR trinodb#7828 there are a few
properties that are not anymore needed:

- cassandra.load-policy.use-token-aware
- cassandra.load-policy.token-aware.shuffle-replicas
- cassandra.load-policy.allowed-addresses
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Development

Successfully merging this pull request may close these issues.

Use latest version of DataStax Java Driver For Apache Cassandra instead of trino-cassandra-driver

6 participants