Cassandra
  1. Cassandra
  2. CASSANDRA-3761 CQL 3.0
  3. CASSANDRA-3919

Dropping a column should do more than just remove the definition

    Details

      Description

      Dropping a column should:

      • immediately make it unavailable for SELECT, including SELECT *
      • eventually (i.e., post-compaction) reclaim the space formerly used by that column

        Activity

        Hide
        Aleksey Yeschenko added a comment -

        Added the required dtests.

        Show
        Aleksey Yeschenko added a comment - Added the required dtests.
        Hide
        Aleksey Yeschenko added a comment -

        Thanks, committed (883c34b7c513eb36f2a45d591db706fa7e36e145). Will resolve once the tests are in as well.

        Show
        Aleksey Yeschenko added a comment - Thanks, committed (883c34b7c513eb36f2a45d591db706fa7e36e145). Will resolve once the tests are in as well.
        Hide
        Jonathan Ellis added a comment -

        +1

        Show
        Jonathan Ellis added a comment - +1
        Hide
        Aleksey Yeschenko added a comment -

        https://github.com/iamaleksey/cassandra/compare/3919-2

        trunk-based branch with all the nits fixed.

        Show
        Aleksey Yeschenko added a comment - https://github.com/iamaleksey/cassandra/compare/3919-2 trunk-based branch with all the nits fixed.
        Hide
        Jonathan Ellis added a comment -

        I guess "last component" is basically trying to say "the user-visible cql column name?"

        Kind of surprised that we haven't needed that code before, but I guess it works.

        Show
        Jonathan Ellis added a comment - I guess "last component" is basically trying to say "the user-visible cql column name?" Kind of surprised that we haven't needed that code before, but I guess it works.
        Hide
        Aleksey Yeschenko added a comment -

        K, I think I see how "Why do we only call discardDropped if it has re-added columns?" is valid. The other thing is most likely wrong, though.

        Will push a new, trunk-based, branch tomorrow.

        Show
        Aleksey Yeschenko added a comment - K, I think I see how "Why do we only call discardDropped if it has re-added columns?" is valid. The other thing is most likely wrong, though. Will push a new, trunk-based, branch tomorrow.
        Hide
        Aleksey Yeschenko added a comment -

        Now you lost me.

        What's up with getLastComponent then?

        Can you give me a link (with a line/no) to what you mean?

        Don't we do slice-based queries for multiple cql rows w/in a partition though?

        How is it important?

        Show
        Aleksey Yeschenko added a comment - Now you lost me. What's up with getLastComponent then? Can you give me a link (with a line/no) to what you mean? Don't we do slice-based queries for multiple cql rows w/in a partition though? How is it important?
        Hide
        Jonathan Ellis added a comment -

        SelectStatement will reject unknown columns, so there is no need to do extra-filtering unless some have been readded.

        Don't we do slice-based queries for multiple cql rows w/in a partition though?

        It's only in the value

        What's up with getLastComponent then?

        Show
        Jonathan Ellis added a comment - SelectStatement will reject unknown columns, so there is no need to do extra-filtering unless some have been readded. Don't we do slice-based queries for multiple cql rows w/in a partition though? It's only in the value What's up with getLastComponent then?
        Hide
        Aleksey Yeschenko added a comment -

        Why do we only call discardDropped if it has re-added columns?

        SelectStatement will reject unknown columns, so there is no need to do extra-filtering unless some have been readded.

        Why is the dropped time the last part of the cell name? Isn't it the value as well? That seems odd.

        It's only in the value:
        https://github.com/iamaleksey/cassandra/blob/64f8b93ef8a61ea4beb93485458bc76a1af803c6/src/java/org/apache/cassandra/config/CFMetaData.java#L1388

        Would prefer to store the values natively as micros rather than fix it up on load into CFMD.

        wfm, I just liked having it nicely-formatted in cqlsh.

        !cf.metadata().getDroppedColumns().isEmpty() could move into isDropped

        Huh. Yes, it should. Or I could move it outside the while-loop.

        Leaning towards "we should probably not put this into 1.2 this late in the release cycle," is it going to kill people to wait for 2.0?

        +1

        Show
        Aleksey Yeschenko added a comment - Why do we only call discardDropped if it has re-added columns? SelectStatement will reject unknown columns, so there is no need to do extra-filtering unless some have been readded. Why is the dropped time the last part of the cell name? Isn't it the value as well? That seems odd. It's only in the value: https://github.com/iamaleksey/cassandra/blob/64f8b93ef8a61ea4beb93485458bc76a1af803c6/src/java/org/apache/cassandra/config/CFMetaData.java#L1388 Would prefer to store the values natively as micros rather than fix it up on load into CFMD. wfm, I just liked having it nicely-formatted in cqlsh. !cf.metadata().getDroppedColumns().isEmpty() could move into isDropped Huh. Yes, it should. Or I could move it outside the while-loop. Leaning towards "we should probably not put this into 1.2 this late in the release cycle," is it going to kill people to wait for 2.0? +1
        Hide
        Jonathan Ellis added a comment -

        Why do we only call discardDropped if it has re-added columns?

        Why is the dropped time the last part of the cell name? Isn't it the value as well? That seems odd. Nit: would prefer to store the values natively as micros rather than fix it up on load into CFMD.

        Nit 2: !cf.metadata().getDroppedColumns().isEmpty() could move into isDropped.

        Leaning towards "we should probably not put this into 1.2 this late in the release cycle," is it going to kill people to wait for 2.0?

        Show
        Jonathan Ellis added a comment - Why do we only call discardDropped if it has re-added columns? Why is the dropped time the last part of the cell name? Isn't it the value as well? That seems odd. Nit: would prefer to store the values natively as micros rather than fix it up on load into CFMD. Nit 2: !cf.metadata().getDroppedColumns().isEmpty() could move into isDropped. Leaning towards "we should probably not put this into 1.2 this late in the release cycle," is it going to kill people to wait for 2.0?
        Hide
        Aleksey Yeschenko added a comment -

        https://github.com/iamaleksey/cassandra/compare/cassandra-1.2...3919

        Optimization for when we know we've purged it all is left for another ticket.
        Also, I guess I'm open to Sylvain's option 1 now (don't allow to add back a dropped column ever)

        At this point I either need some feedback or a +1.

        Show
        Aleksey Yeschenko added a comment - https://github.com/iamaleksey/cassandra/compare/cassandra-1.2...3919 Optimization for when we know we've purged it all is left for another ticket. Also, I guess I'm open to Sylvain's option 1 now (don't allow to add back a dropped column ever) At this point I either need some feedback or a +1.
        Hide
        Aleksey Yeschenko added a comment -

        Updated the branch to use proper maps instead of json pseudomaps.

        Show
        Aleksey Yeschenko added a comment - Updated the branch to use proper maps instead of json pseudomaps.
        Hide
        Sylvain Lebresne added a comment -

        the map is a json pseudomap, not a collection. let's convert everything in one go in CASSANDRA-4603

        If this ticket is going to go in 1.2 as the current fix version suggests, I'd advise not using a json pseudomap because it is quite unclear to me that CASSANDRA-4603 won't require special casings in the code for each property that needs to be converted. So the pseudomaps we have, the more work/special casings we might have to do.

        Show
        Sylvain Lebresne added a comment - the map is a json pseudomap, not a collection. let's convert everything in one go in CASSANDRA-4603 If this ticket is going to go in 1.2 as the current fix version suggests, I'd advise not using a json pseudomap because it is quite unclear to me that CASSANDRA-4603 won't require special casings in the code for each property that needs to be converted. So the pseudomaps we have, the more work/special casings we might have to do.
        Hide
        Aleksey Yeschenko added a comment -

        Sorry, I'm an idiot. Meant ALTER TABLE DROP. Will edit the comment.

        Show
        Aleksey Yeschenko added a comment - Sorry, I'm an idiot. Meant ALTER TABLE DROP. Will edit the comment.
        Hide
        Jonathan Ellis added a comment -

        Does limiting RENAME mean we can't change the "columnN" defaults that upgraded < 1.2 tables would have?

        Show
        Jonathan Ellis added a comment - Does limiting RENAME mean we can't change the "columnN" defaults that upgraded < 1.2 tables would have?
        Hide
        Aleksey Yeschenko added a comment - - edited

        https://github.com/iamaleksey/cassandra/compare/3919

        • ALTER TABLE DROP is limited to CQL3 tables (composite && !compact)
        • dropped columns are stored as a map<column, dropped_time>
        • the map is a json pseudomap, not a collection. let's convert everything in one go in CASSANDRA-4603
        • for compaction, it seems like modifying CFS.removeDeletedStandard() alone is enough

        Tested modified compaction manually before implementing extra-filtering in reads, tested reads manually (+ collection updates that require read-before-write). Everything seems to work.

        What's left:

        • write dtests for reads
        • write unit tests for compaction leaving out dropped columns (can't be a dtest)
        • update cqlsh help
        • update CQL3 doc

        Will do these after the patch is committed, but before resolving the issue in jira.

        Show
        Aleksey Yeschenko added a comment - - edited https://github.com/iamaleksey/cassandra/compare/3919 ALTER TABLE DROP is limited to CQL3 tables (composite && !compact) dropped columns are stored as a map<column, dropped_time> the map is a json pseudomap, not a collection. let's convert everything in one go in CASSANDRA-4603 for compaction, it seems like modifying CFS.removeDeletedStandard() alone is enough Tested modified compaction manually before implementing extra-filtering in reads, tested reads manually (+ collection updates that require read-before-write). Everything seems to work. What's left: write dtests for reads write unit tests for compaction leaving out dropped columns (can't be a dtest) update cqlsh help update CQL3 doc Will do these after the patch is committed, but before resolving the issue in jira.
        Hide
        Sylvain Lebresne added a comment -

        Can we assume it safely, though?

        I think that for CQL3, it is fair game, yes. And by that I mean that CQL3 uses microseconds timestamp by default, so it is, in a way, part of the API. So I think for this issue, which only make sense for CQL3 really, it's ok. Of course, I'm absolutely ok for documenting more precisely that this is an assumption ALTER TABLE DROP does, but what I mean here is that we should probably be clear that if you start using CQL3, you're expected to use microseconds timestamps.

        Show
        Sylvain Lebresne added a comment - Can we assume it safely, though? I think that for CQL3, it is fair game, yes. And by that I mean that CQL3 uses microseconds timestamp by default , so it is, in a way, part of the API. So I think for this issue, which only make sense for CQL3 really, it's ok. Of course, I'm absolutely ok for documenting more precisely that this is an assumption ALTER TABLE DROP does, but what I mean here is that we should probably be clear that if you start using CQL3, you're expected to use microseconds timestamps.
        Hide
        Aleksey Yeschenko added a comment -

        assume that the column timestamp is a real timestamp

        Can we assume it safely, though? It is the most common scenario, but not everyone is setting 'timestamp' to an actual timestamp. I guess we could document somewhere that you shouldn't perform ALTER TABLE DROP if you use timestamps in a non-standard way in that CF.. will that be good enough?

        Show
        Aleksey Yeschenko added a comment - assume that the column timestamp is a real timestamp Can we assume it safely, though? It is the most common scenario, but not everyone is setting 'timestamp' to an actual timestamp. I guess we could document somewhere that you shouldn't perform ALTER TABLE DROP if you use timestamps in a non-standard way in that CF.. will that be good enough?
        Hide
        Jonathan Ellis added a comment -

        3 feels reasonable to me.

        Show
        Jonathan Ellis added a comment - 3 feels reasonable to me.
        Hide
        Sylvain Lebresne added a comment -

        Actually I think there is a small problem, which is 'what if someone drop a column, then add it back later'? I guess there is a few options there:

        1. don't allow to add back a dropped column ever
        2. have a way to know when the column have been fully purged from the data, and only allow to re-add the column then (but we'd have to check that all node have purged it first).
        3. assume that the column timestamp is a real timestamp. Then in the dropped_columns set we could keep the timestamp of when the column was drop, and if it is added back, during read we would return the column only if the timestamp > dropped_timestamp.

        My preference goes to 3 as this feels actually easier to get right than 2 and is much less limiting than 1. Note that even with 3, we could still add a way to know when the column has been fully purged so that we remove it to the dropped_colums set (but that can be left to a following ticket as it would really just be an optimisation).

        Show
        Sylvain Lebresne added a comment - Actually I think there is a small problem, which is 'what if someone drop a column, then add it back later'? I guess there is a few options there: don't allow to add back a dropped column ever have a way to know when the column have been fully purged from the data, and only allow to re-add the column then (but we'd have to check that all node have purged it first). assume that the column timestamp is a real timestamp. Then in the dropped_columns set we could keep the timestamp of when the column was drop, and if it is added back, during read we would return the column only if the timestamp > dropped_timestamp. My preference goes to 3 as this feels actually easier to get right than 2 and is much less limiting than 1. Note that even with 3, we could still add a way to know when the column has been fully purged so that we remove it to the dropped_colums set (but that can be left to a following ticket as it would really just be an optimisation).
        Hide
        Jonathan Ellis added a comment -

        This is conceptually simple. At a high level this just involves

        • adding a dropped_columns Set to CFMetadata
        • wiring that in to AlterTableStatement
        • updating AbstractCompactedRow implementations to leave them out of merged rows

        We don't even have to update the IColumnIterator implementations to ignore dropped columns, since CQL3 SelectStatement already rejects non-existing column names in prepare().

        The one wrinkle I can think of is that dropping columns should not be allowed with COMPACT STORAGE. Should be easy to check for that in the CFMetadata, though (i.e., empty column_aliases).

        Show
        Jonathan Ellis added a comment - This is conceptually simple. At a high level this just involves adding a dropped_columns Set to CFMetadata wiring that in to AlterTableStatement updating AbstractCompactedRow implementations to leave them out of merged rows We don't even have to update the IColumnIterator implementations to ignore dropped columns, since CQL3 SelectStatement already rejects non-existing column names in prepare(). The one wrinkle I can think of is that dropping columns should not be allowed with COMPACT STORAGE. Should be easy to check for that in the CFMetadata, though (i.e., empty column_aliases).

          People

          • Assignee:
            Aleksey Yeschenko
            Reporter:
            Jonathan Ellis
            Reviewer:
            Jonathan Ellis
          • Votes:
            0 Vote for this issue
            Watchers:
            8 Start watching this issue

            Dates

            • Created:
              Updated:
              Resolved:

              Development