[
https://issues.apache.org/jira/browse/CASSANDRA-7304?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14253005#comment-14253005
]
Davide commented on CASSANDRA-7304:
-----------------------------------
Hi guys,
I tried hard to understand this issue but it is not entirely clear to me.
I'm one that is affected by an huge amount of tombstones (and there isn't a
single DELETE in our code)
I use prepared statements everywhere.
If I have a table with columns: _A_, _B_ and _C_.
What happens if using a *prepared statements* I do:
{code:sql}
INSERT INTO test (A, B) VALUES (1, 2)
{code}
Will this generate a tombstone because I didn't set the column _C_?
Second question, this problem comes from the java driver, but since is a
protocol issue I assume is affecting all right?
Last question, the query above *without* *prepared* statements is not going to
generate any tombstone right? _(assuming that's the case with prepared ones)_
Can some of you provide examples of cases where we can generate tombstones
using prepared statements?
> Ability to distinguish between NULL and UNSET values in Prepared Statements
> ---------------------------------------------------------------------------
>
> Key: CASSANDRA-7304
> URL: https://issues.apache.org/jira/browse/CASSANDRA-7304
> Project: Cassandra
> Issue Type: Sub-task
> Reporter: Drew Kutcharian
> Assignee: Oded Peer
> Labels: cql, protocolv4
> Fix For: 3.0
>
> Attachments: 7304-03.patch, 7304-04.patch, 7304-2.patch, 7304.patch
>
>
> Currently Cassandra inserts tombstones when a value of a column is bound to
> NULL in a prepared statement. At higher insert rates managing all these
> tombstones becomes an unnecessary overhead. This limits the usefulness of the
> prepared statements since developers have to either create multiple prepared
> statements (each with a different combination of column names, which at times
> is just unfeasible because of the sheer number of possible combinations) or
> fall back to using regular (non-prepared) statements.
> This JIRA is here to explore the possibility of either:
> A. Have a flag on prepared statements that once set, tells Cassandra to
> ignore null columns
> or
> B. Have an "UNSET" value which makes Cassandra skip the null columns and not
> tombstone them
> Basically, in the context of a prepared statement, a null value means delete,
> but we don’t have anything that means "ignore" (besides creating a new
> prepared statement without the ignored column).
> Please refer to the original conversation on DataStax Java Driver mailing
> list for more background:
> https://groups.google.com/a/lists.datastax.com/d/topic/java-driver-user/cHE3OOSIXBU/discussion
> *EDIT 18/12/14 - [~odpeer] Implementation Notes:*
> The motivation hasn't changed.
> Protocol version 4 specifies that bind variables do not require having a
> value when executing a statement. Bind variables without a value are called
> 'unset'. The 'unset' bind variable is serialized as the int value '-2'
> without following bytes.
> \\
> \\
> * An unset bind variable in an EXECUTE or BATCH request
> ** On a {{value}} does not modify the value and does not create a tombstone
> ** On the {{ttl}} clause is treated as 'unlimited'
> ** On the {{timestamp}} clause is treated as 'now'
> ** On a map key or a list index throws {{InvalidRequestException}}
> ** On a {{counter}} increment or decrement operation does not change the
> counter value, e.g. {{UPDATE my_tab SET c = c - ? WHERE k = 1}} does change
> the value of counter {{c}}
> ** On a tuple field or UDT field throws {{InvalidRequestException}}
> * An unset bind variable in a QUERY request
> ** On a partition column, clustering column or index column in the {{WHERE}}
> clause throws {{InvalidRequestException}}
> ** On the {{limit}} clause is treated as 'unlimited'
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)