Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

CASSANDRA-19796 – cassandra-gocql-driver CI #1812

Closed
wants to merge 9 commits into from
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
183 changes: 26 additions & 157 deletions .github/workflows/main.yml
Original file line number Diff line number Diff line change
Expand Up @@ -7,16 +7,13 @@ on:
pull_request:
types: [ opened, synchronize, reopened ]

env:
CCM_VERSION: "6e71061146f7ae67b84ccd2b1d90d7319b640e4c"

jobs:
build:
name: Unit tests
runs-on: ubuntu-latest
strategy:
matrix:
go: [ '1.19', '1.20' ]
go: [ '1.22', '1.23' ]
steps:
- uses: actions/checkout@v3
- uses: actions/setup-go@v4
Expand All @@ -25,177 +22,49 @@ jobs:
- run: go vet
- name: Run unit tests
run: go test -v -tags unit -race
integration-cassandra:
timeout-minutes: 15
needs:
- build
name: Integration Tests

integration-tc:
needs: build
runs-on: ubuntu-latest
name: integration-tc
strategy:
fail-fast: false
matrix:
go: [ '1.19', '1.20' ]
cassandra_version: [ '4.0.8', '4.1.1' ]
go: [ '1.22', '1.23' ]
cassandra_version: [ '4.0.13', '4.1.6', '5.0.0']
auth: [ "false" ]
compressor: [ "snappy" ]
tags: [ "cassandra", "integration", "ccm" ]
tags: [ "cassandra", "integration", "tc"]
steps:
- uses: actions/checkout@v2
- uses: actions/setup-go@v2
- uses: actions/checkout@v3
- uses: actions/setup-go@v4
with:
go-version: ${{ matrix.go }}
- uses: actions/cache@v2
id: gomod-cache
with:
path: ~/go/pkg/mod
key: ${{ runner.os }}-go-${{ hashFiles('go.mod') }}
restore-keys: |
${{ runner.os }}-go-
- name: Install CCM
run: pip install "git+https://github.com/riptano/ccm.git@${CCM_VERSION}"
- name: Start cassandra nodes
- name: setup
run: |
VERSION=${{ matrix.cassandra_version }}
keypath="$(pwd)/testdata/pki"
conf=(
"client_encryption_options.enabled: true"
"client_encryption_options.keystore: $keypath/.keystore"
"client_encryption_options.keystore_password: cassandra"
"client_encryption_options.require_client_auth: true"
"client_encryption_options.truststore: $keypath/.truststore"
"client_encryption_options.truststore_password: cassandra"
"concurrent_reads: 2"
"concurrent_writes: 2"
"write_request_timeout_in_ms: 5000"
"read_request_timeout_in_ms: 5000"
)

if [[ $VERSION == 3.*.* ]]; then
conf+=(
"rpc_server_type: sync"
"rpc_min_threads: 2"
"rpc_max_threads: 2"
"enable_user_defined_functions: true"
"enable_materialized_views: true"
)
elif [[ $VERSION == 4.0.* ]]; then
conf+=(
"enable_user_defined_functions: true"
"enable_materialized_views: true"
)
else
conf+=(
"user_defined_functions_enabled: true"
"materialized_views_enabled: true"
)
fi

ccm remove test || true

ccm create test -v $VERSION -n 3 -d --vnodes --jvm_arg="-Xmx256m -XX:NewSize=100m"
ccm updateconf "${conf[@]}"

export JVM_EXTRA_OPTS=" -Dcassandra.test.fail_writes_ks=test -Dcassandra.custom_query_handler_class=org.apache.cassandra.cql3.CustomPayloadMirroringQueryHandler"

ccm start --wait-for-binary-proto --verbose
ccm status
ccm node1 nodetool status

args="-gocql.timeout=60s -runssl -proto=4 -rf=3 -clusterSize=3 -autowait=2000ms -compressor=${{ matrix.compressor }} -gocql.cversion=$VERSION -cluster=$(ccm liveset) ./..."

args="-gocql.timeout=60s -runssl -proto=4 -rf=3 -clusterSize=3 -autowait=2000ms -compressor=${{ matrix.compressor }} -gocql.cversion=${{ matrix.cassandra_version }} ./..."
echo "args=$args" >> $GITHUB_ENV
echo "JVM_EXTRA_OPTS=$JVM_EXTRA_OPTS" >> $GITHUB_ENV
- name: Integration tests
run: |
export JVM_EXTRA_OPTS="${{env.JVM_EXTRA_OPTS}}"
go test -v -tags "${{ matrix.tags }} gocql_debug" -timeout=5m -race ${{ env.args }}
- name: 'Save ccm logs'
if: 'failure()'
uses: actions/upload-artifact@v3
with:
name: ccm-cluster
path: /home/runner/.ccm/test
retention-days: 5
integration-auth-cassandra:
timeout-minutes: 15
needs:
- build
name: Integration Tests with auth
- name: run
run: |
go test -v -tags "${{ matrix.tags }} gocql_debug" -timeout=10m -race ${{ env.args }}

integration-auth-tc:
needs: build
runs-on: ubuntu-latest
name: integration-auth-tc
strategy:
fail-fast: false
matrix:
go: [ '1.19', '1.20' ]
cassandra_version: [ '4.0.8' ]
go: [ '1.22', '1.23' ]
cassandra_version: ['4.1.6', '5.0.0' ]
compressor: [ "snappy" ]
tags: [ "integration" ]

steps:
- uses: actions/checkout@v3
- uses: actions/setup-go@v4
with:
go-version: ${{ matrix.go }}
- name: Install CCM
run: pip install "git+https://github.com/riptano/ccm.git@${CCM_VERSION}"
- name: Start cassandra nodes
- name: setup
run: |
VERSION=${{ matrix.cassandra_version }}
keypath="$(pwd)/testdata/pki"
conf=(
"client_encryption_options.enabled: true"
"client_encryption_options.keystore: $keypath/.keystore"
"client_encryption_options.keystore_password: cassandra"
"client_encryption_options.require_client_auth: true"
"client_encryption_options.truststore: $keypath/.truststore"
"client_encryption_options.truststore_password: cassandra"
"concurrent_reads: 2"
"concurrent_writes: 2"
"write_request_timeout_in_ms: 5000"
"read_request_timeout_in_ms: 5000"
"authenticator: PasswordAuthenticator"
"authorizer: CassandraAuthorizer"
"enable_user_defined_functions: true"
)

if [[ $VERSION == 3.*.* ]]; then
conf+=(
"rpc_server_type: sync"
"rpc_min_threads: 2"
"rpc_max_threads: 2"
"enable_user_defined_functions: true"
"enable_materialized_views: true"
)
elif [[ $VERSION == 4.0.* ]]; then
conf+=(
"enable_user_defined_functions: true"
"enable_materialized_views: true"
)
else
conf+=(
"user_defined_functions_enabled: true"
"materialized_views_enabled: true"
)
fi

ccm remove test || true

ccm create test -v $VERSION -n 1 -d --vnodes --jvm_arg="-Xmx256m -XX:NewSize=100m"
ccm updateconf "${conf[@]}"

rm -rf $HOME/.ccm/test/node1/data/system_auth

export JVM_EXTRA_OPTS=" -Dcassandra.test.fail_writes_ks=test -Dcassandra.custom_query_handler_class=org.apache.cassandra.cql3.CustomPayloadMirroringQueryHandler"

ccm start --wait-for-binary-proto --verbose
ccm status
ccm node1 nodetool status

args="-gocql.timeout=60s -runssl -proto=4 -rf=3 -clusterSize=1 -autowait=2000ms -compressor=${{ matrix.compressor }} -gocql.cversion=$VERSION -cluster=$(ccm liveset) ./..."

args="-gocql.timeout=60s -runssl -proto=4 -rf=3 -clusterSize=1 -autowait=2000ms -runauth -compressor=${{ matrix.compressor }} -gocql.cversion=${{ matrix.cassandra_version }}"
echo "args=$args" >> $GITHUB_ENV
echo "JVM_EXTRA_OPTS=$JVM_EXTRA_OPTS" >> $GITHUB_ENV
sleep 30s
- name: Integration tests
run: |
export JVM_EXTRA_OPTS="${{env.JVM_EXTRA_OPTS}}"
go test -v -run=TestAuthentication -tags "${{ matrix.tags }} gocql_debug" -timeout=15s -runauth ${{ env.args }}
- name: run
run: |
go test -v -run=TestAuthentication -tags ""integration" gocql_debug" -timeout=15s -runauth ${{ env.args }}
5 changes: 3 additions & 2 deletions CHANGELOG.md
Original file line number Diff line number Diff line change
Expand Up @@ -7,10 +7,11 @@ and this project adheres to [Semantic Versioning](https://semver.org/spec/v2.0.0
## [Unreleased]

### Added

- Integrated testcontainers-go as the new test runner for integration tests.
### Changed

- Updated GitHub workflows to use `testcontainers-go` for CCM integration tests, replacing the previous CCM integration test runner.
### Fixed
- Corrected `rpc_address` to `connect_address` in the `TestDiscoverViaProxy` test case.

## [1.6.0] - 2023-08-28

Expand Down
17 changes: 17 additions & 0 deletions CONTRIBUTING.md
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,23 @@ The following is a check list of requirements that need to be satisfied in order
* The merge commit passes the regression test suite on GitHub Actions
* `go fmt` has been applied to the submitted code
* Notable changes (i.e. new features or changed behavior, bugfixes) are appropriately documented in CHANGELOG.md, functional changes also in godoc
* Before a Pull Request or a new commit, ensure that all tests pass successfully. You can run all tests locally using the following command (this process takes approximately 15 minutes):
```
go test -v -tags "integration cassandra tc gocql_debug" -timeout=20m -race -gocql.timeout=60s -runssl -proto=4 -rf=3 -clusterSize=3 -autowait=2000ms -compressor=snappy -gocql.cversion=4.0.8 ./...
```
and
```
go test -v -run=TestAuthentication -tags ""integration" gocql_debug" -timeout=2m -runauth -gocql.timeout=60s -runssl -proto=4 -rf=3 -clusterSize=1 -autowait=2000ms -runauth -compressor=snappy -gocql.cversion=4.0.8
```
### ℹ️ **Note:**

> The main_test.go file serves as the entry point for integration tests. It sets up an actual Cassandra cluster, sharing resources before tests are run and cleaning them up afterward, ensuring a consistent and efficient testing environment. The cluster is created locally, so you will need Docker installed.
>
>
> The setup supports both single-node (quick setup in seconds) and multi-node Cassandra clusters (approx. 5 minutes, configurable via the `-clusterSize` flag). If you're curious to try integration tests, feel free to run them yourself. Just ensure that your tests are tagged correctly (e.g., `integration`), since `main_test.go` skips unit tests for performance reasons (no need to start a cluster for unit tests)
>
> You can find examples of running integration tests in the "Before a Pull Request" section.
>
* A correctly formatted commit message, see below

If there are any requirements that can't be reasonably satisfied, please state this either on the pull request or as part of discussion on the mailing list. Where appropriate, the core team may apply discretion and make an exception to these requirements.
Expand Down
1 change: 1 addition & 0 deletions NOTICE
Original file line number Diff line number Diff line change
Expand Up @@ -182,3 +182,4 @@ Dmitry Kropachev <[email protected]>
Oliver Boyle <[email protected]> *
Jackson Fleming <[email protected]> *
Sylwia Szunejko <[email protected]> *
Stanislav Bychkov <[email protected]>
15 changes: 6 additions & 9 deletions cassandra_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -479,15 +479,15 @@ func TestCAS(t *testing.T) {
}

insertBatch := session.NewBatch(LoggedBatch)
insertBatch.Query("INSERT INTO cas_table (title, revid, last_modified) VALUES ('_foo', 2c3af400-73a4-11e5-9381-29463d90c3f0, DATEOF(NOW()))")
insertBatch.Query("INSERT INTO cas_table (title, revid, last_modified) VALUES ('_foo', 3e4ad2f1-73a4-11e5-9381-29463d90c3f0, DATEOF(NOW()))")
insertBatch.Query("INSERT INTO cas_table (title, revid, last_modified) VALUES ('_foo', 2c3af400-73a4-11e5-9381-29463d90c3f0, toTimestamp(NOW()))")
insertBatch.Query("INSERT INTO cas_table (title, revid, last_modified) VALUES ('_foo', 3e4ad2f1-73a4-11e5-9381-29463d90c3f0, toTimestamp(NOW()))")
if err := session.ExecuteBatch(insertBatch); err != nil {
t.Fatal("insert:", err)
}

failBatch = session.NewBatch(LoggedBatch)
failBatch.Query("UPDATE cas_table SET last_modified = DATEOF(NOW()) WHERE title='_foo' AND revid=2c3af400-73a4-11e5-9381-29463d90c3f0 IF last_modified=DATEOF(NOW());")
failBatch.Query("UPDATE cas_table SET last_modified = DATEOF(NOW()) WHERE title='_foo' AND revid=3e4ad2f1-73a4-11e5-9381-29463d90c3f0 IF last_modified=DATEOF(NOW());")
failBatch.Query("UPDATE cas_table SET last_modified = toTimestamp(NOW()) WHERE title='_foo' AND revid=2c3af400-73a4-11e5-9381-29463d90c3f0 IF last_modified=toTimestamp(NOW());")
failBatch.Query("UPDATE cas_table SET last_modified = toTimestamp(NOW()) WHERE title='_foo' AND revid=3e4ad2f1-73a4-11e5-9381-29463d90c3f0 IF last_modified=toTimestamp(NOW());")
if applied, iter, err := session.ExecuteBatchCAS(failBatch, &titleCAS, &revidCAS, &modifiedCAS); err != nil {
t.Fatal("insert:", err)
} else if applied {
Expand Down Expand Up @@ -628,7 +628,6 @@ func TestBatch(t *testing.T) {
}

func TestUnpreparedBatch(t *testing.T) {
t.Skip("FLAKE skipping")
session := createSession(t)
defer session.Close()

Expand Down Expand Up @@ -3029,14 +3028,12 @@ func TestDiscoverViaProxy(t *testing.T) {
session := createSessionFromCluster(cluster, t)
defer session.Close()

// we shouldnt need this but to be safe
time.Sleep(1 * time.Second)

session.pool.mu.RLock()

for _, host := range clusterHosts {
found := false
for _, hi := range session.pool.hostConnPools {
if hi.host.ConnectAddress().String() == host {
if hi.host.RPCAddress().String() == host {
found = true
break
}
Expand Down
Loading
Loading