You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by "Alan Liang (JIRA)" <ji...@apache.org> on 2011/06/09 21:02:59 UTC

[jira] [Created] (CASSANDRA-2753) Capture the max client timestamp for an SSTable

Capture the max client timestamp for an SSTable
-----------------------------------------------

                 Key: CASSANDRA-2753
                 URL: https://issues.apache.org/jira/browse/CASSANDRA-2753
             Project: Cassandra
          Issue Type: New Feature
          Components: Core
            Reporter: Alan Liang
            Assignee: Alan Liang
            Priority: Minor




--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

[jira] [Commented] (CASSANDRA-2753) Capture the max client timestamp for an SSTable

Posted by "Stu Hood (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-2753?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13049530#comment-13049530 ] 

Stu Hood commented on CASSANDRA-2753:
-------------------------------------

bq. So it looks like I have to capture the max timestamp inside #write.
Sounds right. You should be able to modify the AbstractCompactedRow interface to return a max timestamp, like it already does for column count and size.

> Capture the max client timestamp for an SSTable
> -----------------------------------------------
>
>                 Key: CASSANDRA-2753
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-2753
>             Project: Cassandra
>          Issue Type: New Feature
>          Components: Core
>            Reporter: Alan Liang
>            Assignee: Alan Liang
>            Priority: Minor
>         Attachments: 0003-capture-max-timestamp-for-sstable-and-introduced-SST.patch
>
>


--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (CASSANDRA-2753) Capture the max client timestamp for an SSTable

Posted by "Jonathan Ellis (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-2753?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13054725#comment-13054725 ] 

Jonathan Ellis commented on CASSANDRA-2753:
-------------------------------------------

IMO SSTM deserialize versioning logic would be clearer if it were all in SSTMSerializer instead of split between that and openFromDescriptor.

> Capture the max client timestamp for an SSTable
> -----------------------------------------------
>
>                 Key: CASSANDRA-2753
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-2753
>             Project: Cassandra
>          Issue Type: New Feature
>          Components: Core
>            Reporter: Alan Liang
>            Assignee: Alan Liang
>            Priority: Minor
>         Attachments: 0001-capture-max-timestamp-and-created-SSTableMetadata-to.patch, 0003-capture-max-timestamp-for-sstable-and-introduced-SST.patch
>
>


--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Updated] (CASSANDRA-2753) Capture the max client timestamp for an SSTable

Posted by "Alan Liang (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/CASSANDRA-2753?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Alan Liang updated CASSANDRA-2753:
----------------------------------

    Attachment: 0001-capture-max-timestamp-and-created-SSTableMetadata-to.patch

2nd attempt based on Jonathan Ellis' comments. 

Highlights of the patch are:

- captures max column timestamp at the following places: memtable flush, compaction and rebuilding after streamed
- store max timestamp in stats file and created SSTableMetadata class to encapsulate the stats file
- moved estimated histograms for column/row counts and replay position into stats file
- bumped version number
- tests


> Capture the max client timestamp for an SSTable
> -----------------------------------------------
>
>                 Key: CASSANDRA-2753
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-2753
>             Project: Cassandra
>          Issue Type: New Feature
>          Components: Core
>            Reporter: Alan Liang
>            Assignee: Alan Liang
>            Priority: Minor
>         Attachments: 0001-capture-max-timestamp-and-created-SSTableMetadata-to.patch, 0003-capture-max-timestamp-for-sstable-and-introduced-SST.patch
>
>


--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (CASSANDRA-2753) Capture the max client timestamp for an SSTable

Posted by "Jonathan Ellis (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-2753?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13049417#comment-13049417 ] 

Jonathan Ellis commented on CASSANDRA-2753:
-------------------------------------------

Can you elaborate?

> Capture the max client timestamp for an SSTable
> -----------------------------------------------
>
>                 Key: CASSANDRA-2753
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-2753
>             Project: Cassandra
>          Issue Type: New Feature
>          Components: Core
>            Reporter: Alan Liang
>            Assignee: Alan Liang
>            Priority: Minor
>         Attachments: 0003-capture-max-timestamp-for-sstable-and-introduced-SST.patch
>
>


--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (CASSANDRA-2753) Capture the max client timestamp for an SSTable

Posted by "Jonathan Ellis (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-2753?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13054731#comment-13054731 ] 

Jonathan Ellis commented on CASSANDRA-2753:
-------------------------------------------

Suggest adding a comment that SSTableWriter.append(AbstractCompactedRow row) deliberately avoids calling updateMaxTimestamp b/c otherwise we'd have to deserialize EchoedRow.

where is the max-timestamp-of-compacted-sstables logic?  I didn't notice it.

nit: renaming SSTableWriter.writeMetadata feels gratuitous

nit: prefer initializing fields that don't need constructor parameters, at declaration time (looking at RowIndexer.sstMC)

> Capture the max client timestamp for an SSTable
> -----------------------------------------------
>
>                 Key: CASSANDRA-2753
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-2753
>             Project: Cassandra
>          Issue Type: New Feature
>          Components: Core
>            Reporter: Alan Liang
>            Assignee: Alan Liang
>            Priority: Minor
>         Attachments: 0001-capture-max-timestamp-and-created-SSTableMetadata-to.patch, 0003-capture-max-timestamp-for-sstable-and-introduced-SST.patch
>
>


--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (CASSANDRA-2753) Capture the max client timestamp for an SSTable

Posted by "Alan Liang (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-2753?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13049413#comment-13049413 ] 

Alan Liang commented on CASSANDRA-2753:
---------------------------------------

I already have a solution to capture max timestamp for non counter data as seen in the current patch. So this really is only a problem for streamed counter data. 

> Capture the max client timestamp for an SSTable
> -----------------------------------------------
>
>                 Key: CASSANDRA-2753
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-2753
>             Project: Cassandra
>          Issue Type: New Feature
>          Components: Core
>            Reporter: Alan Liang
>            Assignee: Alan Liang
>            Priority: Minor
>         Attachments: 0003-capture-max-timestamp-for-sstable-and-introduced-SST.patch
>
>


--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (CASSANDRA-2753) Capture the max client timestamp for an SSTable

Posted by "Jonathan Ellis (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-2753?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13065292#comment-13065292 ] 

Jonathan Ellis commented on CASSANDRA-2753:
-------------------------------------------

lgtm, thanks!

> Capture the max client timestamp for an SSTable
> -----------------------------------------------
>
>                 Key: CASSANDRA-2753
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-2753
>             Project: Cassandra
>          Issue Type: New Feature
>          Components: Core
>            Reporter: Alan Liang
>            Assignee: Alan Liang
>            Priority: Minor
>             Fix For: 1.0
>
>         Attachments: 0001-capture-max-timestamp-and-created-SSTableMetadata-to-V2.patch, 0001-capture-max-timestamp-and-created-SSTableMetadata-to-V3.patch, 0001-capture-max-timestamp-and-created-SSTableMetadata-to.patch, 0003-capture-max-timestamp-for-sstable-and-introduced-SST.patch, SSTableWriterTest.patch, supercolumn.patch
>
>


--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (CASSANDRA-2753) Capture the max client timestamp for an SSTable

Posted by "Jonathan Ellis (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-2753?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13049388#comment-13049388 ] 

Jonathan Ellis commented on CASSANDRA-2753:
-------------------------------------------

bq. also capture max timestamp of counter data being streamed over from the other nodes

is this really counter-specific or does it affect all streamed data?

> Capture the max client timestamp for an SSTable
> -----------------------------------------------
>
>                 Key: CASSANDRA-2753
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-2753
>             Project: Cassandra
>          Issue Type: New Feature
>          Components: Core
>            Reporter: Alan Liang
>            Assignee: Alan Liang
>            Priority: Minor
>         Attachments: 0003-capture-max-timestamp-for-sstable-and-introduced-SST.patch
>
>


--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (CASSANDRA-2753) Capture the max client timestamp for an SSTable

Posted by "Jonathan Ellis (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-2753?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13058767#comment-13058767 ] 

Jonathan Ellis commented on CASSANDRA-2753:
-------------------------------------------

Is there a reason not to have the max timestamp code in an IColumn method?

> Capture the max client timestamp for an SSTable
> -----------------------------------------------
>
>                 Key: CASSANDRA-2753
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-2753
>             Project: Cassandra
>          Issue Type: New Feature
>          Components: Core
>            Reporter: Alan Liang
>            Assignee: Alan Liang
>            Priority: Minor
>         Attachments: 0001-capture-max-timestamp-and-created-SSTableMetadata-to-V2.patch, 0001-capture-max-timestamp-and-created-SSTableMetadata-to.patch, 0003-capture-max-timestamp-for-sstable-and-introduced-SST.patch
>
>


--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (CASSANDRA-2753) Capture the max client timestamp for an SSTable

Posted by "Jonathan Ellis (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-2753?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13054716#comment-13054716 ] 

Jonathan Ellis commented on CASSANDRA-2753:
-------------------------------------------

{code}
    public long maxTimestamp()
    {
        long maxTimestamp = Long.MIN_VALUE;
        for (IColumn column : columns.values())
        {
            if (column instanceof Column)
                maxTimestamp = Math.max(maxTimestamp, column.timestamp());
        }
        return maxTimestamp;
    }
{code}

No support for supercolumns?

> Capture the max client timestamp for an SSTable
> -----------------------------------------------
>
>                 Key: CASSANDRA-2753
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-2753
>             Project: Cassandra
>          Issue Type: New Feature
>          Components: Core
>            Reporter: Alan Liang
>            Assignee: Alan Liang
>            Priority: Minor
>         Attachments: 0001-capture-max-timestamp-and-created-SSTableMetadata-to.patch, 0003-capture-max-timestamp-for-sstable-and-introduced-SST.patch
>
>


--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Issue Comment Edited] (CASSANDRA-2753) Capture the max client timestamp for an SSTable

Posted by "Alan Liang (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-2753?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13049444#comment-13049444 ] 

Alan Liang edited comment on CASSANDRA-2753 at 6/14/11 9:40 PM:
----------------------------------------------------------------

There are basically 3 places where we need to track max timestamps:

1. Memtable flush
2. During compaction (we simply take the max timestamp already recorded for the sstables)
3. Streamed data (normal columns and counter columns)

The challenge here is to capture the max timestamp for newly streamed data. 

For non-counter streamed data, RowIndexer#doIndexing goes through the streamed data files and simply updates the cache for the new rows. It iterates over the column families without deserializing the columns. To capture max timestamp here, I actually deserialize the columns from disk. This incurs more CPU but since it is already doing disk seeks when calling  deserializeFromSSTableNoColumns(), the seek is less costly.

For counter streamed data, CommutativeRowIndexer#doIndexing actually creates new data files from the streamed data files. It does this by building an AbstractCompactedRow which can be either PreCompactedRow or LazilyCompactedRow. Collecting the max timestamp for PreCompactedRow is easy since all the columns are in memory. For LazilyCompactedRow, the only place where I can observe the max timestamp is during the #write method. Capturing the max timestamp inside #write is obviously not ideal since it would introduce a side effect. Alternatively, I could capture the max timestamp by deserializing the entire LazilyCompactedRow again but this obviously would mean more IO/CPU.

So it looks like I have to capture the max timestamp inside #write.

      was (Author: alanliang):
    There are basically 3 places where we need to track max timestamps:

1. Memtable flush
2. During compaction (we simply take the max timestamp already recorded for the sstables)
3. Streamed data (normal columns and counter columns)

The challenge here is to capture the max timestamp for newly streamed data. 

For non-counter streamed data, RowIndexer#doIndexing goes through the streamed data files and simply updates the cache for the new rows. It iterates over the column families without deserializing the columns. To capture max timestamp here, I actually deserialize the columns from disk. This incurs more CPU but since it is already doing disk seeks when calling  deserializeFromSSTableNoColumns(), the seek is less costly.

For counter streamed data, CommutativeRowIndexer#doIndexing actually creates new data files from the streamed data files. It does this by building an AbstractCompactedRow which can be either PreCompactedRow or LazilyCompactedRow. Collecting the max timestamp for PreCompactedRow is easy since all the columns are in memory. For LazilyCompactedRow, the only place where I can observe the max timestamp is during the #write method. Capturing the max timestamp is obviously not ideal since it would introduce a side effect. Alternatively, I could capture the max timestamp by deserializing the entire LazilyCompactedRow again but this obviously would mean more IO/CPU.

So it looks like I have to capture the max timestamp inside #write.
  
> Capture the max client timestamp for an SSTable
> -----------------------------------------------
>
>                 Key: CASSANDRA-2753
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-2753
>             Project: Cassandra
>          Issue Type: New Feature
>          Components: Core
>            Reporter: Alan Liang
>            Assignee: Alan Liang
>            Priority: Minor
>         Attachments: 0003-capture-max-timestamp-for-sstable-and-introduced-SST.patch
>
>


--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Reopened] (CASSANDRA-2753) Capture the max client timestamp for an SSTable

Posted by "Daniel Doubleday (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/CASSANDRA-2753?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Daniel Doubleday reopened CASSANDRA-2753:
-----------------------------------------


While looking into CASSANDRA-2498 I think I found a problem in SuperColumn.maxTimestamp.

One of the tests is writing a single deleted SC in a SST and the maxTimestamp is negativ which broke my superseding code.

See attached patch

> Capture the max client timestamp for an SSTable
> -----------------------------------------------
>
>                 Key: CASSANDRA-2753
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-2753
>             Project: Cassandra
>          Issue Type: New Feature
>          Components: Core
>            Reporter: Alan Liang
>            Assignee: Alan Liang
>            Priority: Minor
>             Fix For: 1.0
>
>         Attachments: 0001-capture-max-timestamp-and-created-SSTableMetadata-to-V2.patch, 0001-capture-max-timestamp-and-created-SSTableMetadata-to-V3.patch, 0001-capture-max-timestamp-and-created-SSTableMetadata-to.patch, 0003-capture-max-timestamp-for-sstable-and-introduced-SST.patch, supercolumn.patch
>
>


--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (CASSANDRA-2753) Capture the max client timestamp for an SSTable

Posted by "Alan Liang (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-2753?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13056362#comment-13056362 ] 

Alan Liang commented on CASSANDRA-2753:
---------------------------------------

bq. No support for supercolumns?

Wow. Good catch. I've added test tests for this as well.

bq. it would be more clear if observeColumnsInSSTable took a CFMetaData object instead of a CF, to get a serializer from.

I've added a helper method CFMetaData.getColumnSerializer() to do this.

bq. nit: SSTMC.setMaxTimestamp would be more accurately named updateMaxTimestamp

Makes sense.

bq. IMO SSTM deserialize versioning logic would be clearer if it were all in SSTMSerializer instead of split between that and openFromDescriptor.

Makes sense.

bq. Suggest adding a comment that SSTableWriter.append(AbstractCompactedRow row) deliberately avoids calling updateMaxTimestamp b/c otherwise we'd have to deserialize EchoedRow.

Sounds good.

bq. where is the max-timestamp-of-compacted-sstables logic? I didn't notice it.

I put this in ColumnFamilyStore.createCompactionWriter():

{code}
public SSTableWriter createCompactionWriter(long estimatedRows, String location, Collection<SSTableReader> sstables) throws IOException
{    
    ReplayPosition rp = ReplayPosition.getReplayPosition(sstables);
    SSTableMetadata.Collector sstableMetadataCollector = SSTableMetadata.createCollector().replayPosition(rp);

    // get the max timestamp of the precompacted sstables
    for (SSTableReader sstable : sstables)
        sstableMetadataCollector.updateMaxTimestamp(sstable.getMaxTimestamp());

    return new SSTableWriter(getTempSSTablePath(location), estimatedRows, metadata, partitioner, sstableMetadataCollector);
}
{code}

bq. nit: renaming SSTableWriter.writeMetadata feels gratuitous

I renamed it back to writeMetadata.

bq. nit: prefer initializing fields that don't need constructor parameters, at declaration time (looking at RowIndexer.sstMC)

Makes sense.


> Capture the max client timestamp for an SSTable
> -----------------------------------------------
>
>                 Key: CASSANDRA-2753
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-2753
>             Project: Cassandra
>          Issue Type: New Feature
>          Components: Core
>            Reporter: Alan Liang
>            Assignee: Alan Liang
>            Priority: Minor
>         Attachments: 0001-capture-max-timestamp-and-created-SSTableMetadata-to.patch, 0003-capture-max-timestamp-for-sstable-and-introduced-SST.patch
>
>


--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Updated] (CASSANDRA-2753) Capture the max client timestamp for an SSTable

Posted by "Alan Liang (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/CASSANDRA-2753?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Alan Liang updated CASSANDRA-2753:
----------------------------------

    Attachment: 0003-capture-max-timestamp-for-sstable-and-introduced-SST.patch

> Capture the max client timestamp for an SSTable
> -----------------------------------------------
>
>                 Key: CASSANDRA-2753
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-2753
>             Project: Cassandra
>          Issue Type: New Feature
>          Components: Core
>            Reporter: Alan Liang
>            Assignee: Alan Liang
>            Priority: Minor
>         Attachments: 0003-capture-max-timestamp-for-sstable-and-introduced-SST.patch
>
>


--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

[jira] [Commented] (CASSANDRA-2753) Capture the max client timestamp for an SSTable

Posted by "Alan Liang (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-2753?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13049323#comment-13049323 ] 

Alan Liang commented on CASSANDRA-2753:
---------------------------------------

Makes sense, I'll move the tracking outside of the serializer. However, one thing I realized that I missed is to also capture max timestamp of counter data being streamed over from the other nodes. The challenge is where to capture the max timestamp without doing it within the AbstractedCompactedRow#write method. But it seems like I have no choice without sacrificing performance by iterating over the file again to collect the max timestamp. This is because a LazilyCompactedRow keeps only a single column in memory and this only happens within the write method. What do you think?

> Capture the max client timestamp for an SSTable
> -----------------------------------------------
>
>                 Key: CASSANDRA-2753
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-2753
>             Project: Cassandra
>          Issue Type: New Feature
>          Components: Core
>            Reporter: Alan Liang
>            Assignee: Alan Liang
>            Priority: Minor
>         Attachments: 0003-capture-max-timestamp-for-sstable-and-introduced-SST.patch
>
>


--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Updated] (CASSANDRA-2753) Capture the max client timestamp for an SSTable

Posted by "Alan Liang (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/CASSANDRA-2753?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Alan Liang updated CASSANDRA-2753:
----------------------------------

    Attachment: 0001-capture-max-timestamp-and-created-SSTableMetadata-to-V3.patch

added maxTimestamp() to IColumn

> Capture the max client timestamp for an SSTable
> -----------------------------------------------
>
>                 Key: CASSANDRA-2753
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-2753
>             Project: Cassandra
>          Issue Type: New Feature
>          Components: Core
>            Reporter: Alan Liang
>            Assignee: Alan Liang
>            Priority: Minor
>         Attachments: 0001-capture-max-timestamp-and-created-SSTableMetadata-to-V2.patch, 0001-capture-max-timestamp-and-created-SSTableMetadata-to-V3.patch, 0001-capture-max-timestamp-and-created-SSTableMetadata-to.patch, 0003-capture-max-timestamp-for-sstable-and-introduced-SST.patch
>
>


--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (CASSANDRA-2753) Capture the max client timestamp for an SSTable

Posted by "Jonathan Ellis (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-2753?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13060988#comment-13060988 ] 

Jonathan Ellis commented on CASSANDRA-2753:
-------------------------------------------

committed.  thanks Alan!

> Capture the max client timestamp for an SSTable
> -----------------------------------------------
>
>                 Key: CASSANDRA-2753
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-2753
>             Project: Cassandra
>          Issue Type: New Feature
>          Components: Core
>            Reporter: Alan Liang
>            Assignee: Alan Liang
>            Priority: Minor
>         Attachments: 0001-capture-max-timestamp-and-created-SSTableMetadata-to-V2.patch, 0001-capture-max-timestamp-and-created-SSTableMetadata-to-V3.patch, 0001-capture-max-timestamp-and-created-SSTableMetadata-to.patch, 0003-capture-max-timestamp-for-sstable-and-introduced-SST.patch
>
>


--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (CASSANDRA-2753) Capture the max client timestamp for an SSTable

Posted by "Alan Liang (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-2753?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13064230#comment-13064230 ] 

Alan Liang commented on CASSANDRA-2753:
---------------------------------------

Daniel, 

Which test does this break? Can you elaborate?

> Capture the max client timestamp for an SSTable
> -----------------------------------------------
>
>                 Key: CASSANDRA-2753
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-2753
>             Project: Cassandra
>          Issue Type: New Feature
>          Components: Core
>            Reporter: Alan Liang
>            Assignee: Alan Liang
>            Priority: Minor
>             Fix For: 1.0
>
>         Attachments: 0001-capture-max-timestamp-and-created-SSTableMetadata-to-V2.patch, 0001-capture-max-timestamp-and-created-SSTableMetadata-to-V3.patch, 0001-capture-max-timestamp-and-created-SSTableMetadata-to.patch, 0003-capture-max-timestamp-for-sstable-and-introduced-SST.patch, supercolumn.patch
>
>


--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (CASSANDRA-2753) Capture the max client timestamp for an SSTable

Posted by "Jonathan Ellis (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-2753?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13047448#comment-13047448 ] 

Jonathan Ellis commented on CASSANDRA-2753:
-------------------------------------------

CFSerializer is the wrong place for this; serializers should serialize with (ideally) no other side effects.

We can do the tracking inside SSTableWriter and only one method needs to update it:
{code}
    public void append(DecoratedKey decoratedKey, ColumnFamily cf) throws IOException
{code}

(The other append method used by compaction doesn't need to do this because for we can treat a compacted sstabel's max timestamp as max(source sstable timestamps) without loss of correctness.)

> Capture the max client timestamp for an SSTable
> -----------------------------------------------
>
>                 Key: CASSANDRA-2753
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-2753
>             Project: Cassandra
>          Issue Type: New Feature
>          Components: Core
>            Reporter: Alan Liang
>            Assignee: Alan Liang
>            Priority: Minor
>         Attachments: 0003-capture-max-timestamp-for-sstable-and-introduced-SST.patch
>
>


--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

[jira] [Updated] (CASSANDRA-2753) Capture the max client timestamp for an SSTable

Posted by "Daniel Doubleday (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/CASSANDRA-2753?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Daniel Doubleday updated CASSANDRA-2753:
----------------------------------------

    Attachment: SSTableWriterTest.patch

Dunno if SSTableWriterTest is the right place but the added test would break.

> Capture the max client timestamp for an SSTable
> -----------------------------------------------
>
>                 Key: CASSANDRA-2753
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-2753
>             Project: Cassandra
>          Issue Type: New Feature
>          Components: Core
>            Reporter: Alan Liang
>            Assignee: Alan Liang
>            Priority: Minor
>             Fix For: 1.0
>
>         Attachments: 0001-capture-max-timestamp-and-created-SSTableMetadata-to-V2.patch, 0001-capture-max-timestamp-and-created-SSTableMetadata-to-V3.patch, 0001-capture-max-timestamp-and-created-SSTableMetadata-to.patch, 0003-capture-max-timestamp-for-sstable-and-introduced-SST.patch, SSTableWriterTest.patch, supercolumn.patch
>
>


--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (CASSANDRA-2753) Capture the max client timestamp for an SSTable

Posted by "Hudson (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-2753?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13064252#comment-13064252 ] 

Hudson commented on CASSANDRA-2753:
-----------------------------------

Integrated in Cassandra #955 (See [https://builds.apache.org/job/Cassandra/955/])
    fix supercolumn maxtimestamp bug
patch by Daniel Doubleday; reviewed by jbellis for CASSANDRA-2753

jbellis : http://svn.apache.org/viewcvs.cgi/?root=Apache-SVN&view=rev&rev=1145819
Files : 
* /cassandra/trunk/src/java/org/apache/cassandra/db/SuperColumn.java


> Capture the max client timestamp for an SSTable
> -----------------------------------------------
>
>                 Key: CASSANDRA-2753
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-2753
>             Project: Cassandra
>          Issue Type: New Feature
>          Components: Core
>            Reporter: Alan Liang
>            Assignee: Alan Liang
>            Priority: Minor
>             Fix For: 1.0
>
>         Attachments: 0001-capture-max-timestamp-and-created-SSTableMetadata-to-V2.patch, 0001-capture-max-timestamp-and-created-SSTableMetadata-to-V3.patch, 0001-capture-max-timestamp-and-created-SSTableMetadata-to.patch, 0003-capture-max-timestamp-for-sstable-and-introduced-SST.patch, supercolumn.patch
>
>


--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (CASSANDRA-2753) Capture the max client timestamp for an SSTable

Posted by "Hudson (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-2753?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13060998#comment-13060998 ] 

Hudson commented on CASSANDRA-2753:
-----------------------------------

Integrated in Cassandra #952 (See [https://builds.apache.org/job/Cassandra/952/])
    track max client timestamp per-sstable
patch by Alan Liang; reviewed by jbellis for CASSANDRA-2753

jbellis : http://svn.apache.org/viewcvs.cgi/?root=Apache-SVN&view=rev&rev=1143627
Files : 
* /cassandra/trunk/src/java/org/apache/cassandra/io/sstable/SSTable.java
* /cassandra/trunk/src/java/org/apache/cassandra/db/compaction/LazilyCompactedRow.java
* /cassandra/trunk/src/java/org/apache/cassandra/io/sstable/Descriptor.java
* /cassandra/trunk/src/java/org/apache/cassandra/db/SuperColumn.java
* /cassandra/trunk/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
* /cassandra/trunk/test/unit/org/apache/cassandra/db/compaction/CompactionsTest.java
* /cassandra/trunk/test/unit/org/apache/cassandra/io/sstable/SSTableWriterTest.java
* /cassandra/trunk/test/unit/org/apache/cassandra/db/CleanupTest.java
* /cassandra/trunk/src/java/org/apache/cassandra/io/sstable/SSTableWriter.java
* /cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
* /cassandra/trunk/test/unit/org/apache/cassandra/Util.java
* /cassandra/trunk/src/java/org/apache/cassandra/io/sstable/SSTableMetadata.java
* /cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamily.java
* /cassandra/trunk/src/java/org/apache/cassandra/db/compaction/AbstractCompactedRow.java
* /cassandra/trunk/src/java/org/apache/cassandra/db/Column.java
* /cassandra/trunk/src/java/org/apache/cassandra/db/IColumn.java
* /cassandra/trunk/src/java/org/apache/cassandra/db/commitlog/ReplayPosition.java
* /cassandra/trunk/test/unit/org/apache/cassandra/io/sstable/SSTableWriterCommutativeTest.java
* /cassandra/trunk/src/java/org/apache/cassandra/db/compaction/PrecompactedRow.java
* /cassandra/trunk/test/unit/org/apache/cassandra/io/sstable/SSTableMetadataSerializerTest.java
* /cassandra/trunk/src/java/org/apache/cassandra/io/sstable/SSTableReader.java
* /cassandra/trunk/src/java/org/apache/cassandra/config/CFMetaData.java
* /cassandra/trunk/src/java/org/apache/cassandra/utils/EstimatedHistogram.java
* /cassandra/trunk/src/java/org/apache/cassandra/db/EchoedRow.java
* /cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamilySerializer.java


> Capture the max client timestamp for an SSTable
> -----------------------------------------------
>
>                 Key: CASSANDRA-2753
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-2753
>             Project: Cassandra
>          Issue Type: New Feature
>          Components: Core
>            Reporter: Alan Liang
>            Assignee: Alan Liang
>            Priority: Minor
>             Fix For: 1.0
>
>         Attachments: 0001-capture-max-timestamp-and-created-SSTableMetadata-to-V2.patch, 0001-capture-max-timestamp-and-created-SSTableMetadata-to-V3.patch, 0001-capture-max-timestamp-and-created-SSTableMetadata-to.patch, 0003-capture-max-timestamp-for-sstable-and-introduced-SST.patch
>
>


--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Updated] (CASSANDRA-2753) Capture the max client timestamp for an SSTable

Posted by "Daniel Doubleday (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/CASSANDRA-2753?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Daniel Doubleday updated CASSANDRA-2753:
----------------------------------------

    Attachment: supercolumn.patch

> Capture the max client timestamp for an SSTable
> -----------------------------------------------
>
>                 Key: CASSANDRA-2753
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-2753
>             Project: Cassandra
>          Issue Type: New Feature
>          Components: Core
>            Reporter: Alan Liang
>            Assignee: Alan Liang
>            Priority: Minor
>             Fix For: 1.0
>
>         Attachments: 0001-capture-max-timestamp-and-created-SSTableMetadata-to-V2.patch, 0001-capture-max-timestamp-and-created-SSTableMetadata-to-V3.patch, 0001-capture-max-timestamp-and-created-SSTableMetadata-to.patch, 0003-capture-max-timestamp-for-sstable-and-introduced-SST.patch, supercolumn.patch
>
>


--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (CASSANDRA-2753) Capture the max client timestamp for an SSTable

Posted by "Hudson (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-2753?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13065316#comment-13065316 ] 

Hudson commented on CASSANDRA-2753:
-----------------------------------

Integrated in Cassandra #958 (See [https://builds.apache.org/job/Cassandra/958/])
    add test for including supercolumn tombstone time in max timestamp computation
patch by Daniel Doubleday; reviewed by jbellis for CASSANDRA-2753

jbellis : http://svn.apache.org/viewcvs.cgi/?root=Apache-SVN&view=rev&rev=1146732
Files : 
* /cassandra/trunk/test/unit/org/apache/cassandra/io/sstable/SSTableWriterTest.java


> Capture the max client timestamp for an SSTable
> -----------------------------------------------
>
>                 Key: CASSANDRA-2753
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-2753
>             Project: Cassandra
>          Issue Type: New Feature
>          Components: Core
>            Reporter: Alan Liang
>            Assignee: Alan Liang
>            Priority: Minor
>             Fix For: 1.0
>
>         Attachments: 0001-capture-max-timestamp-and-created-SSTableMetadata-to-V2.patch, 0001-capture-max-timestamp-and-created-SSTableMetadata-to-V3.patch, 0001-capture-max-timestamp-and-created-SSTableMetadata-to.patch, 0003-capture-max-timestamp-for-sstable-and-introduced-SST.patch, SSTableWriterTest.patch, supercolumn.patch
>
>


--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Resolved] (CASSANDRA-2753) Capture the max client timestamp for an SSTable

Posted by "Jonathan Ellis (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/CASSANDRA-2753?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Jonathan Ellis resolved CASSANDRA-2753.
---------------------------------------

    Resolution: Fixed

good find, Daniel.  committed.

as Alan implies it would be nice to have a test to catch this too.

> Capture the max client timestamp for an SSTable
> -----------------------------------------------
>
>                 Key: CASSANDRA-2753
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-2753
>             Project: Cassandra
>          Issue Type: New Feature
>          Components: Core
>            Reporter: Alan Liang
>            Assignee: Alan Liang
>            Priority: Minor
>             Fix For: 1.0
>
>         Attachments: 0001-capture-max-timestamp-and-created-SSTableMetadata-to-V2.patch, 0001-capture-max-timestamp-and-created-SSTableMetadata-to-V3.patch, 0001-capture-max-timestamp-and-created-SSTableMetadata-to.patch, 0003-capture-max-timestamp-for-sstable-and-introduced-SST.patch, supercolumn.patch
>
>


--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (CASSANDRA-2753) Capture the max client timestamp for an SSTable

Posted by "Jonathan Ellis (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-2753?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13054717#comment-13054717 ] 

Jonathan Ellis commented on CASSANDRA-2753:
-------------------------------------------

it would be more clear if observeColumnsInSSTable took a CFMetaData object instead of a CF, to get a serializer from.

nit: SSTMC.setMaxTimestamp would be more accurately named updateMaxTimestamp

> Capture the max client timestamp for an SSTable
> -----------------------------------------------
>
>                 Key: CASSANDRA-2753
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-2753
>             Project: Cassandra
>          Issue Type: New Feature
>          Components: Core
>            Reporter: Alan Liang
>            Assignee: Alan Liang
>            Priority: Minor
>         Attachments: 0001-capture-max-timestamp-and-created-SSTableMetadata-to.patch, 0003-capture-max-timestamp-for-sstable-and-introduced-SST.patch
>
>


--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Updated] (CASSANDRA-2753) Capture the max client timestamp for an SSTable

Posted by "Alan Liang (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/CASSANDRA-2753?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Alan Liang updated CASSANDRA-2753:
----------------------------------

    Attachment: 0001-capture-max-timestamp-and-created-SSTableMetadata-to-V2.patch

V2 patch based on jbellis' comments

> Capture the max client timestamp for an SSTable
> -----------------------------------------------
>
>                 Key: CASSANDRA-2753
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-2753
>             Project: Cassandra
>          Issue Type: New Feature
>          Components: Core
>            Reporter: Alan Liang
>            Assignee: Alan Liang
>            Priority: Minor
>         Attachments: 0001-capture-max-timestamp-and-created-SSTableMetadata-to-V2.patch, 0001-capture-max-timestamp-and-created-SSTableMetadata-to.patch, 0003-capture-max-timestamp-for-sstable-and-introduced-SST.patch
>
>


--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (CASSANDRA-2753) Capture the max client timestamp for an SSTable

Posted by "Alan Liang (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-2753?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13049444#comment-13049444 ] 

Alan Liang commented on CASSANDRA-2753:
---------------------------------------

There are basically 3 places where we need to track max timestamps:

1. Memtable flush
2. During compaction (we simply take the max timestamp already recorded for the sstables)
3. Streamed data (normal columns and counter columns)

The challenge here is to capture the max timestamp for newly streamed data. 

For non-counter streamed data, RowIndexer#doIndexing goes through the streamed data files and simply updates the cache for the new rows. It iterates over the column families without deserializing the columns. To capture max timestamp here, I actually deserialize the columns from disk. This incurs more CPU but since it is already doing disk seeks when calling  deserializeFromSSTableNoColumns(), the seek is less costly.

For counter streamed data, CommutativeRowIndexer#doIndexing actually creates new data files from the streamed data files. It does this by building an AbstractCompactedRow which can be either PreCompactedRow or LazilyCompactedRow. Collecting the max timestamp for PreCompactedRow is easy since all the columns are in memory. For LazilyCompactedRow, the only place where I can observe the max timestamp is during the #write method. Capturing the max timestamp is obviously not ideal since it would introduce a side effect. Alternatively, I could capture the max timestamp by deserializing the entire LazilyCompactedRow again but this obviously would mean more IO/CPU.

So it looks like I have to capture the max timestamp inside #write.

> Capture the max client timestamp for an SSTable
> -----------------------------------------------
>
>                 Key: CASSANDRA-2753
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-2753
>             Project: Cassandra
>          Issue Type: New Feature
>          Components: Core
>            Reporter: Alan Liang
>            Assignee: Alan Liang
>            Priority: Minor
>         Attachments: 0003-capture-max-timestamp-for-sstable-and-introduced-SST.patch
>
>


--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (CASSANDRA-2753) Capture the max client timestamp for an SSTable

Posted by "Alan Liang (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-2753?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13046939#comment-13046939 ] 

Alan Liang commented on CASSANDRA-2753:
---------------------------------------

In this patch, I've captured the max timestamp and stored it as part of the stats file. I've encapsulated this file through a class called SSTableMetadata. Estimated histograms for row size and column counts and replay positions will also be available via this class.

> Capture the max client timestamp for an SSTable
> -----------------------------------------------
>
>                 Key: CASSANDRA-2753
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-2753
>             Project: Cassandra
>          Issue Type: New Feature
>          Components: Core
>            Reporter: Alan Liang
>            Assignee: Alan Liang
>            Priority: Minor
>         Attachments: 0003-capture-max-timestamp-for-sstable-and-introduced-SST.patch
>
>


--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira