apache / cassandra-gocql-driver

GoCQL Driver for Apache Cassandra®
https://cassandra.apache.org/
Apache License 2.0
2.57k stars 616 forks source link

gocql RetryPolicies still don't use query idempotence #1803

Open Rikkuru opened 3 weeks ago

Rikkuru commented 3 weeks ago

Issue https://github.com/apache/cassandra-gocql-driver/issues/1154 was meant to add idempotence check in gocql RetryPolicies

Issue was closed but Retry Policies still don't check if query IsIdempotent BUT documentations states that IsIdempotent affects retries

Need to implement this logic or change documentation

OleksiienkoMykyta commented 2 weeks ago

I will check it, but could you please provide more details. How you tested it, versions of C*, golang, and gocql, etc.

Rikkuru commented 2 weeks ago

What version of Cassandra are you using?

scylladb 5.1.19

What version of Gocql are you using?

fork scylladb/gocql v1.12.0 ( includes v1.6.0 gocql/gocql)

What version of Go are you using?

1.21

Rikkuru commented 2 weeks ago

my project does not set Idempotent (not intentionally ) but we expect retries I was trying to replacy some of our retry policies with standard ones from gocql and noticed it, but even with gocql retry policies queries without IsIdempotent get retries (tested with DowngradingConsistencyRetryPolicy)

none of the implemented retry policies use this query method - https://github.com/apache/cassandra-gocql-driver/blob/trunk/policies.go

lukasz-antoniak commented 2 weeks ago

Do you expect the retry of non-idempotent queries or not? I could not clearly understand expected behaviour.

The ticket you mentioned was implementing speculative retry policy, which is something different. Retry policy in Java driver is:

  1. Used for Query statements, because those are implicitly idempotent.
  2. For write operations it depends on idempotency flag set CqlRequestHandler.java#L775.

Retry policy implementations do not need to check idempotency flag. IMHO same logic shall apply in GoCQL. Retry policy shall not be even invoked for non0idempotent write operations.

Do I understand the issue correctly? If so, I will submit a fix for it.

Rikkuru commented 2 weeks ago

Do you expect the retry of non-idempotent queries or not? I

I expect that https://pkg.go.dev/github.com/gocql/gocql#Query with IsIdempotent returning false will not be retried. You are right that it is probably better to check it outside of retry policy. But I am sure it is not checked in gocql v1.6.0

The ticket you mentioned was implementing speculative retry policy, which is something different.

From this issue https://github.com/apache/cassandra-gocql-driver/issues/1153 I understood that https://github.com/apache/cassandra-gocql-driver/issues/1083 contains work on speculative execution and https://github.com/apache/cassandra-gocql-driver/issues/1154 was supposed to support non-idempotent queries in retries ? Otherwise issue https://github.com/apache/cassandra-gocql-driver/issues/1153 should not have been closed.

For write operations it depends on idempotency

I am not sure we should think too much about read/write ops here , gocql Query is the same for them and does not provide convenient way to check if query writes or reads data. I think gocql should just check method IsIdempotent and trust user to fill this field correctly ?

OleksiienkoMykyta commented 2 weeks ago

I have tested it and found that Queries is not idempotent by default IsIdempotent = false unless you set it explicitly. Also, I didn't found any retries, just getting a single error when the consistency level doesn't match Query failed: Operation timed out for system_schema.keyspaces - received only 1 responses from 3 CL=THREE., tested with cluster.RetryPolicy = &gocql.DowngradingConsistencyRetryPolicy{ConsistencyLevelsToTry: []gocql.Consistency{gocql.Two, gocql.One}} and cluster.Consistency = gocql.Three. It looks like everything works according to spec, but maybe I missed something. Please, provide more details how you are getting retries, code example, or smth, also would be nice if you would share the results of the logs.

Rikkuru commented 2 weeks ago

Query failed: Operation timed out for system_schema.keyspaces - received only 1 responses from 3 CL=THREE., tested with cluster.RetryPolicy = &gocql.DowngradingConsistencyRetryPolicy{ConsistencyLevelsToTry: []gocql.Consistency{gocql.Two, gocql.One}} and cluster.Consistency = gocql.Three.

Was this a write or read query ? Just checking because write timeout is not retried in DowngradingConsistencyRetryPolicy if some nodes responded.

Rikkuru commented 2 weeks ago

I tested retries with gocql.DowngradingConsistencyRetryPolicy{ConsistencyLevelsToTry: []gocql.Consistency{ gocql.One}} and CL=Quorum with 2 nodes one of which is down. First attempt gets an error (read/write get the same error) This is logged in observer on first attempt

gogateway[12476]: 12:14:38.938400 error: scylla [4055867439]: pool [3]: host [runner1***]: attempt [0]: read query failed: Cannot achieve consistency level for cl QUORUM. Requires 2, alive 1

and on retry observer logs attempt with lower CL (this one is successful)

gogateway[12476]: 12:14:38.944768 warn: scylla [4055867439]: pool [3]: host [runner1***]: attempt [1]: read CL downgrade QUORUM -> ONE

Our Query does not set Idempotent so it shouldn't have been retried

func (r *Reader) initQuery(req ReadRequestInterface) *gocql.Query {
    q := r.session.Query(getQueryString(req))
    q.Observer(r)
    q.Prefetch(*r.cfg.ReadPrefetchFactor)
    q.PageSize(*r.cfg.PageSize)
    q.RetryPolicy(r.retryPolicy)

    q.SetConsistency(r.consistencyLevel)
    return q
}

We also don't change DefaultIdempotence in cluster config

OleksiienkoMykyta commented 2 weeks ago

Please share an exec query you are using to reproduce the issue, if you tried multiple queries provide them as well.

Rikkuru commented 2 weeks ago
SELECT event_id, time, args
        FROM my.events
        WHERE user = ? AND project_id = ?
            AND year = ? AND week = ?
        ORDER BY time DESC
        LIMIT ?

Something like that on read with

CREATE TABLE my.events (
    user text,
    year smallint,
    week tinyint,
    project_id smallint,
    time timeuuid,
    args map<text, text>,
    event_id smallint,
    ip inet,
    PRIMARY KEY ((user, year, week, project_id), time)

I will try to find time on weekends to create an easier example in some sandbox

Rikkuru commented 1 week ago

Please share an exec query you are using to reproduce the issue, if you tried multiple queries provide them as well.

Hi I have an axample

main.go (go 1.21.11 , github.com/gocql/gocql v1.6.0)

there is a loop to wait for scylladb start and second loop to have time to down one of the nodes

package main

import (
    "context"
    "fmt"
    "time"

    "github.com/gocql/gocql"
)

type MyRetryPolicy struct {
}

func (*MyRetryPolicy) Attempt(q gocql.RetryableQuery) bool {
    if q.Attempts() > 2 {
        return false
    }
    return true
}

func (*MyRetryPolicy) GetRetryType(error) gocql.RetryType {
    return gocql.Retry
}

type LoggingObserver struct{}

func (*LoggingObserver) ObserveQuery(ctx context.Context, q gocql.ObservedQuery) {
    fmt.Printf("observer attempt: %d err: %s\n", q.Attempt, q.Err)
}

func main() {
    cluster := gocql.NewCluster("some-scylla", "some-scylla3", "some-scylla2")
    var err error
    var s *gocql.Session
    for i := 1; i <= 100; i++ {
        fmt.Printf("%d attempt\n", i)
        s, err = gocql.NewSession(*cluster)
        if err == nil {
            break
        }
        time.Sleep(1 * time.Second)
    }
    if err != nil {
        fmt.Printf("session err: %s\n", err)
        return
    }
    defer s.Close()

    fmt.Printf("CONNECTED!")

    time.Sleep(60 * time.Second)
    for i := 1; i <= 100; i++ {
        q := s.Query("INSERT INTO  mykeyspace.events(event_id, time, args) VALUES (?,?,?)", 1, gocql.UUIDFromTime(time.Now()), "test")

        q.RetryPolicy(&MyRetryPolicy{})
        q.Observer(&LoggingObserver{})
        q.Consistency(gocql.All)

        fmt.Printf("%d QUERY IsIdempotent: %v\n", i, q.IsIdempotent())

        err = q.Exec()
        if err != nil {
            fmt.Printf("exec err: %s\n", err)
        }
        time.Sleep(1 * time.Second)
    }
}

cql init for scylla

CREATE KEYSPACE mykeyspace WITH replication = {'class': 'NetworkTopologyStrategy', 'datacenter1': '3'}  AND durable_writes = true;
CREATE TABLE mykeyspace.events (event_id int, time timeuuid, args text, PRIMARY KEY ((event_id), time));

docker-compose for nodes and main.go

version: '3'

services:
  some-scylla:
    image: scylladb/scylla:5.1
    container_name: some-scylla
    command: --overprovisioned 1 --smp 1
    ports:
      - 9042:9042

  some-scylla2:
    image: scylladb/scylla:5.1
    container_name: some-scylla2
    command: --seeds=some-scylla --overprovisioned 1 --smp 1

  some-scylla3:
    image: scylladb/scylla:5.1
    container_name: some-scylla3
    command: --seeds=some-scylla --overprovisioned 1 --smp 1

  myapp2:
    image: golang:1.21.11
    container_name: myapp2
    command: go run /app/main.go
    working_dir: /app
    volumes:
    - ./:/app

I can see in log that query is retried

2024-09-02 18:40:30 90 QUERY IsIdempotent: false
2024-09-02 18:40:30 observer attempt: 0 err: Cannot achieve consistency level for cl ALL. Requires 3, alive 2
2024-09-02 18:40:30 observer attempt: 1 err: Cannot achieve consistency level for cl ALL. Requires 3, alive 2
2024-09-02 18:40:30 observer attempt: 2 err: Cannot achieve consistency level for cl ALL. Requires 3, alive 2
2024-09-02 18:40:30 exec err: Cannot achieve consistency level for cl ALL. Requires 3, alive 2
OleksiienkoMykyta commented 1 week ago

@Rikkuru Thank you for providing the details. I reproduced the issue you faced, and as you said, according to documentation it's misbehavior. I added a small fix, retested it and it works properly now.